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.eq;
40 import static org.mockito.Mockito.never;
41 import static org.mockito.Mockito.spy;
42 import static org.mockito.Mockito.times;
43 import static org.mockito.Mockito.verify;
44
45 import java.io.IOException;
46 import java.io.InterruptedIOException;
47 import java.security.PrivilegedExceptionAction;
48 import java.util.ArrayList;
49 import java.util.Arrays;
50 import java.util.Collection;
51 import java.util.List;
52 import java.util.Map;
53 import java.util.NavigableMap;
54 import java.util.TreeMap;
55 import java.util.UUID;
56 import java.util.concurrent.atomic.AtomicBoolean;
57 import java.util.concurrent.atomic.AtomicInteger;
58 import java.util.concurrent.atomic.AtomicReference;
59
60 import org.apache.commons.logging.Log;
61 import org.apache.commons.logging.LogFactory;
62 import org.apache.hadoop.conf.Configuration;
63 import org.apache.hadoop.fs.FSDataOutputStream;
64 import org.apache.hadoop.fs.FileStatus;
65 import org.apache.hadoop.fs.FileSystem;
66 import org.apache.hadoop.fs.Path;
67 import org.apache.hadoop.hbase.Cell;
68 import org.apache.hadoop.hbase.HBaseConfiguration;
69 import org.apache.hadoop.hbase.CellComparator;
70 import org.apache.hadoop.hbase.CellUtil;
71 import org.apache.hadoop.hbase.CompatibilitySingletonFactory;
72 import org.apache.hadoop.hbase.HBaseTestCase;
73 import org.apache.hadoop.hbase.HBaseTestingUtility;
74 import org.apache.hadoop.hbase.HColumnDescriptor;
75 import org.apache.hadoop.hbase.HConstants;
76 import org.apache.hadoop.hbase.HConstants.OperationStatusCode;
77 import org.apache.hadoop.hbase.HDFSBlocksDistribution;
78 import org.apache.hadoop.hbase.HRegionInfo;
79 import org.apache.hadoop.hbase.HTableDescriptor;
80 import org.apache.hadoop.hbase.KeyValue;
81 import org.apache.hadoop.hbase.MediumTests;
82 import org.apache.hadoop.hbase.MiniHBaseCluster;
83 import org.apache.hadoop.hbase.MultithreadedTestUtil;
84 import org.apache.hadoop.hbase.MultithreadedTestUtil.RepeatingTestThread;
85 import org.apache.hadoop.hbase.MultithreadedTestUtil.TestThread;
86 import org.apache.hadoop.hbase.NotServingRegionException;
87 import org.apache.hadoop.hbase.TableName;
88 import org.apache.hadoop.hbase.Waiter;
89 import org.apache.hadoop.hbase.client.Append;
90 import org.apache.hadoop.hbase.client.Delete;
91 import org.apache.hadoop.hbase.client.Durability;
92 import org.apache.hadoop.hbase.client.Get;
93 import org.apache.hadoop.hbase.client.HTable;
94 import org.apache.hadoop.hbase.client.Increment;
95 import org.apache.hadoop.hbase.client.Put;
96 import org.apache.hadoop.hbase.client.Result;
97 import org.apache.hadoop.hbase.client.Scan;
98 import org.apache.hadoop.hbase.exceptions.FailedSanityCheckException;
99 import org.apache.hadoop.hbase.filter.BinaryComparator;
100 import org.apache.hadoop.hbase.filter.ColumnCountGetFilter;
101 import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
102 import org.apache.hadoop.hbase.filter.Filter;
103 import org.apache.hadoop.hbase.filter.FilterBase;
104 import org.apache.hadoop.hbase.filter.FilterList;
105 import org.apache.hadoop.hbase.filter.NullComparator;
106 import org.apache.hadoop.hbase.filter.PrefixFilter;
107 import org.apache.hadoop.hbase.filter.SingleColumnValueExcludeFilter;
108 import org.apache.hadoop.hbase.filter.SingleColumnValueFilter;
109 import org.apache.hadoop.hbase.monitoring.MonitoredRPCHandler;
110 import org.apache.hadoop.hbase.monitoring.MonitoredTask;
111 import org.apache.hadoop.hbase.monitoring.TaskMonitor;
112 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
113 import org.apache.hadoop.hbase.protobuf.generated.WALProtos.CompactionDescriptor;
114 import org.apache.hadoop.hbase.regionserver.HRegion.RegionScannerImpl;
115 import org.apache.hadoop.hbase.regionserver.HRegion.RowLock;
116 import org.apache.hadoop.hbase.regionserver.TestStore.FaultyFileSystem;
117 import org.apache.hadoop.hbase.regionserver.wal.HLog;
118 import org.apache.hadoop.hbase.regionserver.wal.HLogFactory;
119 import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
120 import org.apache.hadoop.hbase.regionserver.wal.HLogUtil;
121 import org.apache.hadoop.hbase.regionserver.wal.MetricsWALSource;
122 import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
123 import org.apache.hadoop.hbase.security.User;
124 import org.apache.hadoop.hbase.test.MetricsAssertHelper;
125 import org.apache.hadoop.hbase.util.Bytes;
126 import org.apache.hadoop.hbase.util.EnvironmentEdgeManagerTestHelper;
127 import org.apache.hadoop.hbase.util.FSUtils;
128 import org.apache.hadoop.hbase.util.IncrementingEnvironmentEdge;
129 import org.apache.hadoop.hbase.util.PairOfSameType;
130 import org.apache.hadoop.hbase.util.Threads;
131 import org.junit.After;
132 import org.junit.Assert;
133 import org.junit.Before;
134 import org.junit.Rule;
135 import org.junit.Test;
136 import org.junit.experimental.categories.Category;
137 import org.junit.rules.TestName;
138 import org.mockito.Mockito;
139 import org.apache.hadoop.hbase.DroppedSnapshotException;
140
141 import com.google.common.collect.Lists;
142
143
144
145
146
147
148
149 @Category(MediumTests.class)
150 @SuppressWarnings("deprecation")
151 public class TestHRegion {
152
153
154 static final Log LOG = LogFactory.getLog(TestHRegion.class);
155 @Rule public TestName name = new TestName();
156
157 private static final String COLUMN_FAMILY = "MyCF";
158 private static final byte [] COLUMN_FAMILY_BYTES = Bytes.toBytes(COLUMN_FAMILY);
159
160 HRegion region = null;
161
162 private static HBaseTestingUtility TEST_UTIL;
163 public static Configuration CONF ;
164 private String dir;
165 private static FileSystem FILESYSTEM;
166 private final int MAX_VERSIONS = 2;
167
168
169 protected byte[] tableName;
170 protected String method;
171 protected final byte[] qual1 = Bytes.toBytes("qual1");
172 protected final byte[] qual2 = Bytes.toBytes("qual2");
173 protected final byte[] qual3 = Bytes.toBytes("qual3");
174 protected final byte[] value1 = Bytes.toBytes("value1");
175 protected final byte[] value2 = Bytes.toBytes("value2");
176 protected final byte[] row = Bytes.toBytes("rowA");
177 protected final byte[] row2 = Bytes.toBytes("rowB");
178
179 protected final MetricsAssertHelper metricsAssertHelper = CompatibilitySingletonFactory
180 .getInstance(MetricsAssertHelper.class);
181
182 @Before
183 public void setup() throws IOException {
184 TEST_UTIL = HBaseTestingUtility.createLocalHTU();
185 FILESYSTEM = TEST_UTIL.getTestFileSystem();
186 CONF = TEST_UTIL.getConfiguration();
187 dir = TEST_UTIL.getDataTestDir("TestHRegion").toString();
188 method = name.getMethodName();
189 tableName = Bytes.toBytes(name.getMethodName());
190 }
191
192 @After
193 public void tearDown() throws Exception {
194 EnvironmentEdgeManagerTestHelper.reset();
195 }
196
197 String getName() {
198 return name.getMethodName();
199 }
200
201
202
203
204
205
206
207
208
209
210
211 @Test (timeout=60000)
212 public void testCloseCarryingSnapshot() throws IOException {
213 HRegion region = initHRegion(tableName, name.getMethodName(), CONF, COLUMN_FAMILY_BYTES);
214 Store store = region.getStore(COLUMN_FAMILY_BYTES);
215
216 byte [] value = Bytes.toBytes(name.getMethodName());
217
218 Put put = new Put(value);
219 put.add(COLUMN_FAMILY_BYTES, null, value);
220
221 region.put(put);
222 StoreFlushContext storeFlushCtx = store.createFlushContext(12345);
223 storeFlushCtx.prepare();
224
225 put.add(COLUMN_FAMILY_BYTES, Bytes.toBytes("abc"), value);
226 region.put(put);
227
228 region.close();
229 assertEquals(0, region.getMemstoreSize().get());
230 HRegion.closeHRegion(region);
231 }
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248 @Test (timeout=60000)
249 public void testFlushSizeAccounting() throws Exception {
250 final Configuration conf = HBaseConfiguration.create(CONF);
251
252 conf.setInt("hbase.hstore.flush.retries.number", 1);
253 final User user =
254 User.createUserForTesting(conf, this.name.getMethodName(), new String[]{"foo"});
255
256 conf.setClass("fs.file.impl", FaultyFileSystem.class, FileSystem.class);
257 user.runAs(new PrivilegedExceptionAction<Object>() {
258 public Object run() throws Exception {
259
260 FileSystem fs = FileSystem.get(conf);
261 Assert.assertEquals(FaultyFileSystem.class, fs.getClass());
262 FaultyFileSystem ffs = (FaultyFileSystem)fs;
263 HRegion region = null;
264 try {
265
266 region = initHRegion(tableName, name.getMethodName(), conf, COLUMN_FAMILY_BYTES);
267 long size = region.getMemstoreSize().get();
268 Assert.assertEquals(0, size);
269
270 Put p1 = new Put(row);
271 p1.add(new KeyValue(row, COLUMN_FAMILY_BYTES, qual1, 1, (byte[])null));
272 region.put(p1);
273 final long sizeOfOnePut = region.getMemstoreSize().get();
274
275 try {
276 LOG.info("Flushing");
277 region.flushcache();
278 Assert.fail("Didn't bubble up IOE!");
279 } catch (DroppedSnapshotException dse) {
280
281 }
282
283 ffs.fault.set(false);
284
285 Assert.assertEquals(sizeOfOnePut, region.getMemstoreSize().get());
286
287
288 Put p2 = new Put(row);
289 p2.add(new KeyValue(row, COLUMN_FAMILY_BYTES, qual2, 2, (byte[])null));
290 p2.add(new KeyValue(row, COLUMN_FAMILY_BYTES, qual3, 3, (byte[])null));
291 region.put(p2);
292 Assert.assertEquals(sizeOfOnePut * 3, region.getMemstoreSize().get());
293
294
295
296 region.flushcache();
297
298 Assert.assertEquals(sizeOfOnePut * 2, region.getMemstoreSize().get());
299 } finally {
300 HRegion.closeHRegion(region);
301 }
302 return null;
303 }
304 });
305 FileSystem.closeAllForUGI(user.getUGI());
306 }
307
308 @Test
309 public void testCompactionAffectedByScanners() throws Exception {
310 byte[] family = Bytes.toBytes("family");
311 this.region = initHRegion(tableName, method, CONF, family);
312
313 Put put = new Put(Bytes.toBytes("r1"));
314 put.add(family, Bytes.toBytes("q1"), Bytes.toBytes("v1"));
315 region.put(put);
316 region.flushcache();
317
318 Scan scan = new Scan();
319 scan.setMaxVersions(3);
320
321 RegionScanner scanner1 = region.getScanner(scan);
322
323 Delete delete = new Delete(Bytes.toBytes("r1"));
324 region.delete(delete);
325 region.flushcache();
326
327
328 RegionScanner scanner2 = region.getScanner(scan);
329
330 List<Cell> results = new ArrayList<Cell>();
331
332 System.out.println("Smallest read point:" + region.getSmallestReadPoint());
333
334
335 region.compactStores(true);
336
337
338 RegionScanner scanner3 = region.getScanner(scan);
339
340
341 scanner1.next(results);
342 System.out.println(results);
343 assertEquals(1, results.size());
344
345 results.clear();
346 scanner2.next(results);
347 System.out.println(results);
348 assertEquals(0, results.size());
349
350 results.clear();
351 scanner3.next(results);
352 System.out.println(results);
353 assertEquals(0, results.size());
354 }
355
356 @Test
357 public void testToShowNPEOnRegionScannerReseek() throws Exception {
358 byte[] family = Bytes.toBytes("family");
359 this.region = initHRegion(tableName, method, CONF, family);
360
361 Put put = new Put(Bytes.toBytes("r1"));
362 put.add(family, Bytes.toBytes("q1"), Bytes.toBytes("v1"));
363 region.put(put);
364 put = new Put(Bytes.toBytes("r2"));
365 put.add(family, Bytes.toBytes("q1"), Bytes.toBytes("v1"));
366 region.put(put);
367 region.flushcache();
368
369 Scan scan = new Scan();
370 scan.setMaxVersions(3);
371
372 RegionScanner scanner1 = region.getScanner(scan);
373
374 System.out.println("Smallest read point:" + region.getSmallestReadPoint());
375
376 region.compactStores(true);
377
378 scanner1.reseek(Bytes.toBytes("r2"));
379 List<Cell> results = new ArrayList<Cell>();
380 scanner1.next(results);
381 Cell keyValue = results.get(0);
382 Assert.assertTrue(Bytes.compareTo(CellUtil.cloneRow(keyValue), Bytes.toBytes("r2")) == 0);
383 scanner1.close();
384 }
385
386 @Test
387 public void testSkipRecoveredEditsReplay() throws Exception {
388 String method = "testSkipRecoveredEditsReplay";
389 TableName tableName = TableName.valueOf(method);
390 byte[] family = Bytes.toBytes("family");
391 this.region = initHRegion(tableName, method, CONF, family);
392 try {
393 Path regiondir = region.getRegionFileSystem().getRegionDir();
394 FileSystem fs = region.getRegionFileSystem().getFileSystem();
395 byte[] regionName = region.getRegionInfo().getEncodedNameAsBytes();
396
397 Path recoveredEditsDir = HLogUtil.getRegionDirRecoveredEditsDir(regiondir);
398
399 long maxSeqId = 1050;
400 long minSeqId = 1000;
401
402 for (long i = minSeqId; i <= maxSeqId; i += 10) {
403 Path recoveredEdits = new Path(recoveredEditsDir, String.format("%019d", i));
404 fs.create(recoveredEdits);
405 HLog.Writer writer = HLogFactory.createRecoveredEditsWriter(fs, recoveredEdits, CONF);
406
407 long time = System.nanoTime();
408 WALEdit edit = new WALEdit();
409 edit.add(new KeyValue(row, family, Bytes.toBytes(i), time, KeyValue.Type.Put, Bytes
410 .toBytes(i)));
411 writer.append(new HLog.Entry(new HLogKey(regionName, tableName, i, time,
412 HConstants.DEFAULT_CLUSTER_ID), edit));
413
414 writer.close();
415 }
416 MonitoredTask status = TaskMonitor.get().createStatus(method);
417 Map<byte[], Long> maxSeqIdInStores = new TreeMap<byte[], Long>(Bytes.BYTES_COMPARATOR);
418 for (Store store : region.getStores().values()) {
419 maxSeqIdInStores.put(store.getColumnFamilyName().getBytes(), minSeqId - 1);
420 }
421 long seqId = region.replayRecoveredEditsIfAny(regiondir, maxSeqIdInStores, null, status);
422 assertEquals(maxSeqId, seqId);
423 Get get = new Get(row);
424 Result result = region.get(get);
425 for (long i = minSeqId; i <= maxSeqId; i += 10) {
426 List<Cell> kvs = result.getColumnCells(family, Bytes.toBytes(i));
427 assertEquals(1, kvs.size());
428 assertArrayEquals(Bytes.toBytes(i), CellUtil.cloneValue(kvs.get(0)));
429 }
430 } finally {
431 HRegion.closeHRegion(this.region);
432 this.region = null;
433 }
434 }
435
436 @Test
437 public void testSkipRecoveredEditsReplaySomeIgnored() throws Exception {
438 String method = "testSkipRecoveredEditsReplaySomeIgnored";
439 TableName tableName = TableName.valueOf(method);
440 byte[] family = Bytes.toBytes("family");
441 this.region = initHRegion(tableName, method, CONF, family);
442 try {
443 Path regiondir = region.getRegionFileSystem().getRegionDir();
444 FileSystem fs = region.getRegionFileSystem().getFileSystem();
445 byte[] regionName = region.getRegionInfo().getEncodedNameAsBytes();
446
447 Path recoveredEditsDir = HLogUtil.getRegionDirRecoveredEditsDir(regiondir);
448
449 long maxSeqId = 1050;
450 long minSeqId = 1000;
451
452 for (long i = minSeqId; i <= maxSeqId; i += 10) {
453 Path recoveredEdits = new Path(recoveredEditsDir, String.format("%019d", i));
454 fs.create(recoveredEdits);
455 HLog.Writer writer = HLogFactory.createRecoveredEditsWriter(fs, recoveredEdits, CONF);
456
457 long time = System.nanoTime();
458 WALEdit edit = new WALEdit();
459 edit.add(new KeyValue(row, family, Bytes.toBytes(i), time, KeyValue.Type.Put, Bytes
460 .toBytes(i)));
461 writer.append(new HLog.Entry(new HLogKey(regionName, tableName, i, time,
462 HConstants.DEFAULT_CLUSTER_ID), edit));
463
464 writer.close();
465 }
466 long recoverSeqId = 1030;
467 MonitoredTask status = TaskMonitor.get().createStatus(method);
468 Map<byte[], Long> maxSeqIdInStores = new TreeMap<byte[], Long>(Bytes.BYTES_COMPARATOR);
469 for (Store store : region.getStores().values()) {
470 maxSeqIdInStores.put(store.getColumnFamilyName().getBytes(), recoverSeqId - 1);
471 }
472 long seqId = region.replayRecoveredEditsIfAny(regiondir, maxSeqIdInStores, null, status);
473 assertEquals(maxSeqId, seqId);
474 Get get = new Get(row);
475 Result result = region.get(get);
476 for (long i = minSeqId; i <= maxSeqId; i += 10) {
477 List<Cell> kvs = result.getColumnCells(family, Bytes.toBytes(i));
478 if (i < recoverSeqId) {
479 assertEquals(0, kvs.size());
480 } else {
481 assertEquals(1, kvs.size());
482 assertArrayEquals(Bytes.toBytes(i), CellUtil.cloneValue(kvs.get(0)));
483 }
484 }
485 } finally {
486 HRegion.closeHRegion(this.region);
487 this.region = null;
488 }
489 }
490
491 @Test
492 public void testSkipRecoveredEditsReplayAllIgnored() throws Exception {
493 byte[] family = Bytes.toBytes("family");
494 this.region = initHRegion(tableName, method, CONF, family);
495 try {
496 Path regiondir = region.getRegionFileSystem().getRegionDir();
497 FileSystem fs = region.getRegionFileSystem().getFileSystem();
498
499 Path recoveredEditsDir = HLogUtil.getRegionDirRecoveredEditsDir(regiondir);
500 for (int i = 1000; i < 1050; i += 10) {
501 Path recoveredEdits = new Path(recoveredEditsDir, String.format("%019d", i));
502 FSDataOutputStream dos = fs.create(recoveredEdits);
503 dos.writeInt(i);
504 dos.close();
505 }
506 long minSeqId = 2000;
507 Path recoveredEdits = new Path(recoveredEditsDir, String.format("%019d", minSeqId - 1));
508 FSDataOutputStream dos = fs.create(recoveredEdits);
509 dos.close();
510
511 Map<byte[], Long> maxSeqIdInStores = new TreeMap<byte[], Long>(Bytes.BYTES_COMPARATOR);
512 for (Store store : region.getStores().values()) {
513 maxSeqIdInStores.put(store.getColumnFamilyName().getBytes(), minSeqId);
514 }
515 long seqId = region.replayRecoveredEditsIfAny(regiondir, maxSeqIdInStores, null, null);
516 assertEquals(minSeqId, seqId);
517 } finally {
518 HRegion.closeHRegion(this.region);
519 this.region = null;
520 }
521 }
522
523 @Test
524 public void testRecoveredEditsReplayCompaction() throws Exception {
525 String method = name.getMethodName();
526 TableName tableName = TableName.valueOf(method);
527 byte[] family = Bytes.toBytes("family");
528 this.region = initHRegion(tableName, method, CONF, family);
529 try {
530 Path regiondir = region.getRegionFileSystem().getRegionDir();
531 FileSystem fs = region.getRegionFileSystem().getFileSystem();
532 byte[] regionName = region.getRegionInfo().getEncodedNameAsBytes();
533
534 long maxSeqId = 3;
535 long minSeqId = 0;
536
537 for (long i = minSeqId; i < maxSeqId; i++) {
538 Put put = new Put(Bytes.toBytes(i));
539 put.add(family, Bytes.toBytes(i), Bytes.toBytes(i));
540 region.put(put);
541 region.flushcache();
542 }
543
544
545 assertEquals(3, region.getStore(family).getStorefilesCount());
546 List<Path> storeFiles = new ArrayList<Path>(3);
547 for (StoreFile sf : region.getStore(family).getStorefiles()) {
548 storeFiles.add(sf.getPath());
549 }
550
551
552 CONF.setBoolean("hbase.hstore.compaction.complete", false);
553 region.compactStores();
554
555
556 assertEquals(3, region.getStore(family).getStorefilesCount());
557
558
559 Path tmpDir = region.getRegionFileSystem().getTempDir();
560 FileStatus[] files = FSUtils.listStatus(fs, tmpDir);
561 String errorMsg = "Expected to find 1 file in the region temp directory "
562 + "from the compaction, could not find any";
563 assertNotNull(errorMsg, files);
564 assertEquals(errorMsg, 1, files.length);
565
566 Path newFile = region.getRegionFileSystem().commitStoreFile(Bytes.toString(family),
567 files[0].getPath());
568
569 CompactionDescriptor compactionDescriptor = ProtobufUtil.toCompactionDescriptor(this.region
570 .getRegionInfo(), family, storeFiles, Lists.newArrayList(newFile), region
571 .getRegionFileSystem().getStoreDir(Bytes.toString(family)));
572
573 HLogUtil.writeCompactionMarker(region.getLog(), this.region.getTableDesc(),
574 this.region.getRegionInfo(), compactionDescriptor);
575
576 Path recoveredEditsDir = HLogUtil.getRegionDirRecoveredEditsDir(regiondir);
577
578 Path recoveredEdits = new Path(recoveredEditsDir, String.format("%019d", 1000));
579 fs.create(recoveredEdits);
580 HLog.Writer writer = HLogFactory.createRecoveredEditsWriter(fs, recoveredEdits, CONF);
581
582 long time = System.nanoTime();
583
584 writer.append(new HLog.Entry(new HLogKey(regionName, tableName, 10, time,
585 HConstants.DEFAULT_CLUSTER_ID), WALEdit.createCompaction(compactionDescriptor)));
586 writer.close();
587
588
589 region.getTableDesc();
590 region.getRegionInfo();
591 region.close();
592 region = HRegion.openHRegion(region, null);
593
594
595 Collection<StoreFile> sfs = region.getStore(family).getStorefiles();
596 for (StoreFile sf : sfs) {
597 LOG.info(sf.getPath());
598 }
599 assertEquals(1, region.getStore(family).getStorefilesCount());
600 files = FSUtils.listStatus(fs, tmpDir);
601 assertTrue("Expected to find 0 files inside " + tmpDir, files == null || files.length == 0);
602
603 for (long i = minSeqId; i < maxSeqId; i++) {
604 Get get = new Get(Bytes.toBytes(i));
605 Result result = region.get(get);
606 byte[] value = result.getValue(family, Bytes.toBytes(i));
607 assertArrayEquals(Bytes.toBytes(i), value);
608 }
609 } finally {
610 HRegion.closeHRegion(this.region);
611 this.region = null;
612 }
613 }
614
615 @Test
616 public void testGetWhileRegionClose() throws IOException {
617 TableName tableName = TableName.valueOf(name.getMethodName());
618 Configuration hc = initSplit();
619 int numRows = 100;
620 byte[][] families = { fam1, fam2, fam3 };
621
622
623 String method = name.getMethodName();
624 this.region = initHRegion(tableName, method, hc, families);
625 try {
626
627 final int startRow = 100;
628 putData(startRow, numRows, qual1, families);
629 putData(startRow, numRows, qual2, families);
630 putData(startRow, numRows, qual3, families);
631 final AtomicBoolean done = new AtomicBoolean(false);
632 final AtomicInteger gets = new AtomicInteger(0);
633 GetTillDoneOrException[] threads = new GetTillDoneOrException[10];
634 try {
635
636 for (int i = 0; i < threads.length / 2; i++) {
637 threads[i] = new GetTillDoneOrException(i, Bytes.toBytes("" + startRow), done, gets);
638 threads[i].setDaemon(true);
639 threads[i].start();
640 }
641
642
643 this.region.closing.set(true);
644 for (int i = threads.length / 2; i < threads.length; i++) {
645 threads[i] = new GetTillDoneOrException(i, Bytes.toBytes("" + startRow), done, gets);
646 threads[i].setDaemon(true);
647 threads[i].start();
648 }
649 } finally {
650 if (this.region != null) {
651 HRegion.closeHRegion(this.region);
652 }
653 }
654 done.set(true);
655 for (GetTillDoneOrException t : threads) {
656 try {
657 t.join();
658 } catch (InterruptedException e) {
659 e.printStackTrace();
660 }
661 if (t.e != null) {
662 LOG.info("Exception=" + t.e);
663 assertFalse("Found a NPE in " + t.getName(), t.e instanceof NullPointerException);
664 }
665 }
666 } finally {
667 HRegion.closeHRegion(this.region);
668 this.region = null;
669 }
670 }
671
672
673
674
675
676 class GetTillDoneOrException extends Thread {
677 private final Get g;
678 private final AtomicBoolean done;
679 private final AtomicInteger count;
680 private Exception e;
681
682 GetTillDoneOrException(final int i, final byte[] r, final AtomicBoolean d, final AtomicInteger c) {
683 super("getter." + i);
684 this.g = new Get(r);
685 this.done = d;
686 this.count = c;
687 }
688
689 @Override
690 public void run() {
691 while (!this.done.get()) {
692 try {
693 assertTrue(region.get(g).size() > 0);
694 this.count.incrementAndGet();
695 } catch (Exception e) {
696 this.e = e;
697 break;
698 }
699 }
700 }
701 }
702
703
704
705
706 @Test
707 public void testWeirdCacheBehaviour() throws Exception {
708 byte[] TABLE = Bytes.toBytes("testWeirdCacheBehaviour");
709 byte[][] FAMILIES = new byte[][] { Bytes.toBytes("trans-blob"), Bytes.toBytes("trans-type"),
710 Bytes.toBytes("trans-date"), Bytes.toBytes("trans-tags"), Bytes.toBytes("trans-group") };
711 this.region = initHRegion(TABLE, getName(), CONF, FAMILIES);
712 try {
713 String value = "this is the value";
714 String value2 = "this is some other value";
715 String keyPrefix1 = "prefix1";
716 String keyPrefix2 = "prefix2";
717 String keyPrefix3 = "prefix3";
718 putRows(this.region, 3, value, keyPrefix1);
719 putRows(this.region, 3, value, keyPrefix2);
720 putRows(this.region, 3, value, keyPrefix3);
721 putRows(this.region, 3, value2, keyPrefix1);
722 putRows(this.region, 3, value2, keyPrefix2);
723 putRows(this.region, 3, value2, keyPrefix3);
724 System.out.println("Checking values for key: " + keyPrefix1);
725 assertEquals("Got back incorrect number of rows from scan", 3,
726 getNumberOfRows(keyPrefix1, value2, this.region));
727 System.out.println("Checking values for key: " + keyPrefix2);
728 assertEquals("Got back incorrect number of rows from scan", 3,
729 getNumberOfRows(keyPrefix2, value2, this.region));
730 System.out.println("Checking values for key: " + keyPrefix3);
731 assertEquals("Got back incorrect number of rows from scan", 3,
732 getNumberOfRows(keyPrefix3, value2, this.region));
733 deleteColumns(this.region, value2, keyPrefix1);
734 deleteColumns(this.region, value2, keyPrefix2);
735 deleteColumns(this.region, value2, keyPrefix3);
736 System.out.println("Starting important checks.....");
737 assertEquals("Got back incorrect number of rows from scan: " + keyPrefix1, 0,
738 getNumberOfRows(keyPrefix1, value2, this.region));
739 assertEquals("Got back incorrect number of rows from scan: " + keyPrefix2, 0,
740 getNumberOfRows(keyPrefix2, value2, this.region));
741 assertEquals("Got back incorrect number of rows from scan: " + keyPrefix3, 0,
742 getNumberOfRows(keyPrefix3, value2, this.region));
743 } finally {
744 HRegion.closeHRegion(this.region);
745 this.region = null;
746 }
747 }
748
749 @Test
750 public void testAppendWithReadOnlyTable() throws Exception {
751 byte[] TABLE = Bytes.toBytes("readOnlyTable");
752 this.region = initHRegion(TABLE, getName(), CONF, true, Bytes.toBytes("somefamily"));
753 boolean exceptionCaught = false;
754 Append append = new Append(Bytes.toBytes("somerow"));
755 append.setDurability(Durability.SKIP_WAL);
756 append.add(Bytes.toBytes("somefamily"), Bytes.toBytes("somequalifier"),
757 Bytes.toBytes("somevalue"));
758 try {
759 region.append(append);
760 } catch (IOException e) {
761 exceptionCaught = true;
762 } finally {
763 HRegion.closeHRegion(this.region);
764 this.region = null;
765 }
766 assertTrue(exceptionCaught == true);
767 }
768
769 @Test
770 public void testIncrWithReadOnlyTable() throws Exception {
771 byte[] TABLE = Bytes.toBytes("readOnlyTable");
772 this.region = initHRegion(TABLE, getName(), CONF, true, Bytes.toBytes("somefamily"));
773 boolean exceptionCaught = false;
774 Increment inc = new Increment(Bytes.toBytes("somerow"));
775 inc.setDurability(Durability.SKIP_WAL);
776 inc.addColumn(Bytes.toBytes("somefamily"), Bytes.toBytes("somequalifier"), 1L);
777 try {
778 region.increment(inc);
779 } catch (IOException e) {
780 exceptionCaught = true;
781 } finally {
782 HRegion.closeHRegion(this.region);
783 this.region = null;
784 }
785 assertTrue(exceptionCaught == true);
786 }
787
788 private void deleteColumns(HRegion r, String value, String keyPrefix) throws IOException {
789 InternalScanner scanner = buildScanner(keyPrefix, value, r);
790 int count = 0;
791 boolean more = false;
792 List<Cell> results = new ArrayList<Cell>();
793 do {
794 more = scanner.next(results);
795 if (results != null && !results.isEmpty())
796 count++;
797 else
798 break;
799 Delete delete = new Delete(CellUtil.cloneRow(results.get(0)));
800 delete.deleteColumn(Bytes.toBytes("trans-tags"), Bytes.toBytes("qual2"));
801 r.delete(delete);
802 results.clear();
803 } while (more);
804 assertEquals("Did not perform correct number of deletes", 3, count);
805 }
806
807 private int getNumberOfRows(String keyPrefix, String value, HRegion r) throws Exception {
808 InternalScanner resultScanner = buildScanner(keyPrefix, value, r);
809 int numberOfResults = 0;
810 List<Cell> results = new ArrayList<Cell>();
811 boolean more = false;
812 do {
813 more = resultScanner.next(results);
814 if (results != null && !results.isEmpty())
815 numberOfResults++;
816 else
817 break;
818 for (Cell kv : results) {
819 System.out.println("kv=" + kv.toString() + ", " + Bytes.toString(CellUtil.cloneValue(kv)));
820 }
821 results.clear();
822 } while (more);
823 return numberOfResults;
824 }
825
826 private InternalScanner buildScanner(String keyPrefix, String value, HRegion r)
827 throws IOException {
828
829 FilterList allFilters = new FilterList();
830 allFilters.addFilter(new PrefixFilter(Bytes.toBytes(keyPrefix)));
831
832 SingleColumnValueFilter filter = new SingleColumnValueFilter(Bytes.toBytes("trans-tags"),
833 Bytes.toBytes("qual2"), CompareOp.EQUAL, Bytes.toBytes(value));
834 filter.setFilterIfMissing(true);
835 allFilters.addFilter(filter);
836 Scan scan = new Scan();
837 scan.addFamily(Bytes.toBytes("trans-blob"));
838 scan.addFamily(Bytes.toBytes("trans-type"));
839 scan.addFamily(Bytes.toBytes("trans-date"));
840 scan.addFamily(Bytes.toBytes("trans-tags"));
841 scan.addFamily(Bytes.toBytes("trans-group"));
842 scan.setFilter(allFilters);
843 return r.getScanner(scan);
844 }
845
846 private void putRows(HRegion r, int numRows, String value, String key) throws IOException {
847 for (int i = 0; i < numRows; i++) {
848 String row = key + "_" + i
849 System.out.println(String.format("Saving row: %s, with value %s", row, value));
850 Put put = new Put(Bytes.toBytes(row));
851 put.setDurability(Durability.SKIP_WAL);
852 put.add(Bytes.toBytes("trans-blob"), null, Bytes.toBytes("value for blob"));
853 put.add(Bytes.toBytes("trans-type"), null, Bytes.toBytes("statement"));
854 put.add(Bytes.toBytes("trans-date"), null, Bytes.toBytes("20090921010101999"));
855 put.add(Bytes.toBytes("trans-tags"), Bytes.toBytes("qual2"), Bytes.toBytes(value));
856 put.add(Bytes.toBytes("trans-group"), null, Bytes.toBytes("adhocTransactionGroupId"));
857 r.put(put);
858 }
859 }
860
861 @Test
862 public void testFamilyWithAndWithoutColon() throws Exception {
863 byte[] b = Bytes.toBytes(getName());
864 byte[] cf = Bytes.toBytes(COLUMN_FAMILY);
865 this.region = initHRegion(b, getName(), CONF, cf);
866 try {
867 Put p = new Put(b);
868 byte[] cfwithcolon = Bytes.toBytes(COLUMN_FAMILY + ":");
869 p.add(cfwithcolon, cfwithcolon, cfwithcolon);
870 boolean exception = false;
871 try {
872 this.region.put(p);
873 } catch (NoSuchColumnFamilyException e) {
874 exception = true;
875 }
876 assertTrue(exception);
877 } finally {
878 HRegion.closeHRegion(this.region);
879 this.region = null;
880 }
881 }
882
883 @Test
884 public void testBatchPut() throws Exception {
885 byte[] b = Bytes.toBytes(getName());
886 byte[] cf = Bytes.toBytes(COLUMN_FAMILY);
887 byte[] qual = Bytes.toBytes("qual");
888 byte[] val = Bytes.toBytes("val");
889 this.region = initHRegion(b, getName(), CONF, cf);
890 MetricsWALSource source = CompatibilitySingletonFactory.getInstance(MetricsWALSource.class);
891 try {
892 long syncs = metricsAssertHelper.getCounter("syncTimeNumOps", source);
893 metricsAssertHelper.assertCounter("syncTimeNumOps", syncs, source);
894
895 LOG.info("First a batch put with all valid puts");
896 final Put[] puts = new Put[10];
897 for (int i = 0; i < 10; i++) {
898 puts[i] = new Put(Bytes.toBytes("row_" + i));
899 puts[i].add(cf, qual, val);
900 }
901
902 OperationStatus[] codes = this.region.batchMutate(puts);
903 assertEquals(10, codes.length);
904 for (int i = 0; i < 10; i++) {
905 assertEquals(OperationStatusCode.SUCCESS, codes[i].getOperationStatusCode());
906 }
907 metricsAssertHelper.assertCounter("syncTimeNumOps", syncs + 1, source);
908
909 LOG.info("Next a batch put with one invalid family");
910 puts[5].add(Bytes.toBytes("BAD_CF"), qual, val);
911 codes = this.region.batchMutate(puts);
912 assertEquals(10, codes.length);
913 for (int i = 0; i < 10; i++) {
914 assertEquals((i == 5) ? OperationStatusCode.BAD_FAMILY : OperationStatusCode.SUCCESS,
915 codes[i].getOperationStatusCode());
916 }
917
918 metricsAssertHelper.assertCounter("syncTimeNumOps", syncs + 2, source);
919
920 LOG.info("Next a batch put that has to break into two batches to avoid a lock");
921 RowLock rowLock = region.getRowLock(Bytes.toBytes("row_2"));
922
923 MultithreadedTestUtil.TestContext ctx = new MultithreadedTestUtil.TestContext(CONF);
924 final AtomicReference<OperationStatus[]> retFromThread = new AtomicReference<OperationStatus[]>();
925 TestThread putter = new TestThread(ctx) {
926 @Override
927 public void doWork() throws IOException {
928 retFromThread.set(region.batchMutate(puts));
929 }
930 };
931 LOG.info("...starting put thread while holding lock");
932 ctx.addThread(putter);
933 ctx.startThreads();
934
935 LOG.info("...waiting for put thread to sync first time");
936 long startWait = System.currentTimeMillis();
937 while (metricsAssertHelper.getCounter("syncTimeNumOps", source) == syncs + 2) {
938 Thread.sleep(100);
939 if (System.currentTimeMillis() - startWait > 10000) {
940 fail("Timed out waiting for thread to sync first minibatch");
941 }
942 }
943 LOG.info("...releasing row lock, which should let put thread continue");
944 rowLock.release();
945 LOG.info("...joining on thread");
946 ctx.stop();
947 LOG.info("...checking that next batch was synced");
948 metricsAssertHelper.assertCounter("syncTimeNumOps", syncs + 4, source);
949 codes = retFromThread.get();
950 for (int i = 0; i < 10; i++) {
951 assertEquals((i == 5) ? OperationStatusCode.BAD_FAMILY : OperationStatusCode.SUCCESS,
952 codes[i].getOperationStatusCode());
953 }
954
955 } finally {
956 HRegion.closeHRegion(this.region);
957 this.region = null;
958 }
959 }
960
961 @Test
962 public void testBatchPutWithTsSlop() throws Exception {
963 byte[] b = Bytes.toBytes(getName());
964 byte[] cf = Bytes.toBytes(COLUMN_FAMILY);
965 byte[] qual = Bytes.toBytes("qual");
966 byte[] val = Bytes.toBytes("val");
967
968
969 CONF.setInt("hbase.hregion.keyvalue.timestamp.slop.millisecs", 1000);
970 this.region = initHRegion(b, getName(), CONF, cf);
971
972 try {
973 MetricsWALSource source = CompatibilitySingletonFactory.getInstance(MetricsWALSource.class);
974 long syncs = metricsAssertHelper.getCounter("syncTimeNumOps", source);
975 metricsAssertHelper.assertCounter("syncTimeNumOps", syncs, source);
976
977 final Put[] puts = new Put[10];
978 for (int i = 0; i < 10; i++) {
979 puts[i] = new Put(Bytes.toBytes("row_" + i), Long.MAX_VALUE - 100);
980 puts[i].add(cf, qual, val);
981 }
982
983 OperationStatus[] codes = this.region.batchMutate(puts);
984 assertEquals(10, codes.length);
985 for (int i = 0; i < 10; i++) {
986 assertEquals(OperationStatusCode.SANITY_CHECK_FAILURE, codes[i].getOperationStatusCode());
987 }
988 metricsAssertHelper.assertCounter("syncTimeNumOps", syncs, source);
989
990 } finally {
991 HRegion.closeHRegion(this.region);
992 this.region = null;
993 }
994
995 }
996
997
998
999
1000 @Test
1001 public void testCheckAndMutate_WithEmptyRowValue() throws IOException {
1002 byte[] row1 = Bytes.toBytes("row1");
1003 byte[] fam1 = Bytes.toBytes("fam1");
1004 byte[] qf1 = Bytes.toBytes("qualifier");
1005 byte[] emptyVal = new byte[] {};
1006 byte[] val1 = Bytes.toBytes("value1");
1007 byte[] val2 = Bytes.toBytes("value2");
1008
1009
1010 String method = this.getName();
1011 this.region = initHRegion(tableName, method, CONF, fam1);
1012 try {
1013
1014 Put put = new Put(row1);
1015 put.add(fam1, qf1, emptyVal);
1016
1017
1018 boolean res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL, new BinaryComparator(
1019 emptyVal), put, true);
1020 assertTrue(res);
1021
1022
1023 put = new Put(row1);
1024 put.add(fam1, qf1, val1);
1025
1026
1027 res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL, new BinaryComparator(emptyVal),
1028 put, true);
1029 assertTrue(res);
1030
1031
1032 res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL, new BinaryComparator(emptyVal),
1033 put, true);
1034 assertFalse(res);
1035
1036 Delete delete = new Delete(row1);
1037 delete.deleteColumn(fam1, qf1);
1038 res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL, new BinaryComparator(emptyVal),
1039 delete, true);
1040 assertFalse(res);
1041
1042 put = new Put(row1);
1043 put.add(fam1, qf1, val2);
1044
1045 res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL, new BinaryComparator(val1),
1046 put, true);
1047 assertTrue(res);
1048
1049
1050 delete = new Delete(row1);
1051 delete.deleteColumn(fam1, qf1);
1052 delete.deleteColumn(fam1, qf1);
1053 res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL, new BinaryComparator(val2),
1054 delete, true);
1055 assertTrue(res);
1056
1057 delete = new Delete(row1);
1058 res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL, new BinaryComparator(emptyVal),
1059 delete, true);
1060 assertTrue(res);
1061
1062
1063 put = new Put(row1);
1064 put.add(fam1, qf1, val1);
1065
1066 res = region
1067 .checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL, new NullComparator(), put, true);
1068 assertTrue(res);
1069 } finally {
1070 HRegion.closeHRegion(this.region);
1071 this.region = null;
1072 }
1073 }
1074
1075 @Test
1076 public void testCheckAndMutate_WithWrongValue() throws IOException {
1077 byte[] row1 = Bytes.toBytes("row1");
1078 byte[] fam1 = Bytes.toBytes("fam1");
1079 byte[] qf1 = Bytes.toBytes("qualifier");
1080 byte[] val1 = Bytes.toBytes("value1");
1081 byte[] val2 = Bytes.toBytes("value2");
1082
1083
1084 String method = this.getName();
1085 this.region = initHRegion(tableName, method, CONF, fam1);
1086 try {
1087
1088 Put put = new Put(row1);
1089 put.add(fam1, qf1, val1);
1090 region.put(put);
1091
1092
1093 boolean res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL, new BinaryComparator(
1094 val2), put, true);
1095 assertEquals(false, res);
1096
1097
1098 Delete delete = new Delete(row1);
1099 delete.deleteFamily(fam1);
1100 res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL, new BinaryComparator(val2),
1101 delete, true);
1102 assertEquals(false, res);
1103 } finally {
1104 HRegion.closeHRegion(this.region);
1105 this.region = null;
1106 }
1107 }
1108
1109 @Test
1110 public void testCheckAndMutate_WithCorrectValue() throws IOException {
1111 byte[] row1 = Bytes.toBytes("row1");
1112 byte[] fam1 = Bytes.toBytes("fam1");
1113 byte[] qf1 = Bytes.toBytes("qualifier");
1114 byte[] val1 = Bytes.toBytes("value1");
1115
1116
1117 String method = this.getName();
1118 this.region = initHRegion(tableName, method, CONF, fam1);
1119 try {
1120
1121 Put put = new Put(row1);
1122 put.add(fam1, qf1, val1);
1123 region.put(put);
1124
1125
1126 boolean res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL, new BinaryComparator(
1127 val1), put, true);
1128 assertEquals(true, res);
1129
1130
1131 Delete delete = new Delete(row1);
1132 delete.deleteColumn(fam1, qf1);
1133 res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL, new BinaryComparator(val1),
1134 put, true);
1135 assertEquals(true, res);
1136 } finally {
1137 HRegion.closeHRegion(this.region);
1138 this.region = null;
1139 }
1140 }
1141
1142 @Test
1143 public void testCheckAndMutate_WithNonEqualCompareOp() throws IOException {
1144 byte[] row1 = Bytes.toBytes("row1");
1145 byte[] fam1 = Bytes.toBytes("fam1");
1146 byte[] qf1 = Bytes.toBytes("qualifier");
1147 byte[] val1 = Bytes.toBytes("value1");
1148 byte[] val2 = Bytes.toBytes("value2");
1149 byte[] val3 = Bytes.toBytes("value3");
1150 byte[] val4 = Bytes.toBytes("value4");
1151
1152
1153 String method = this.getName();
1154 this.region = initHRegion(tableName, method, CONF, fam1);
1155 try {
1156
1157 Put put = new Put(row1);
1158 put.add(fam1, qf1, val3);
1159 region.put(put);
1160
1161
1162 boolean res = region.checkAndMutate(row1, fam1, qf1, CompareOp.LESS,
1163 new BinaryComparator(val3), put, true);
1164 assertEquals(false, res);
1165
1166
1167 res = region.checkAndMutate(row1, fam1, qf1, CompareOp.LESS,
1168 new BinaryComparator(val4), put, true);
1169 assertEquals(false, res);
1170
1171
1172
1173 put = new Put(row1);
1174 put.add(fam1, qf1, val2);
1175 res = region.checkAndMutate(row1, fam1, qf1, CompareOp.LESS,
1176 new BinaryComparator(val2), put, true);
1177 assertEquals(true, res);
1178
1179
1180 res = region.checkAndMutate(row1, fam1, qf1, CompareOp.LESS_OR_EQUAL,
1181 new BinaryComparator(val3), put, true);
1182 assertEquals(false, res);
1183
1184
1185
1186 res = region.checkAndMutate(row1, fam1, qf1, CompareOp.LESS_OR_EQUAL,
1187 new BinaryComparator(val2), put, true);
1188 assertEquals(true, res);
1189
1190
1191
1192 put = new Put(row1);
1193 put.add(fam1, qf1, val3);
1194 res = region.checkAndMutate(row1, fam1, qf1, CompareOp.LESS_OR_EQUAL,
1195 new BinaryComparator(val1), put, true);
1196 assertEquals(true, res);
1197
1198
1199 res = region.checkAndMutate(row1, fam1, qf1, CompareOp.GREATER,
1200 new BinaryComparator(val3), put, true);
1201 assertEquals(false, res);
1202
1203
1204 res = region.checkAndMutate(row1, fam1, qf1, CompareOp.GREATER,
1205 new BinaryComparator(val2), put, true);
1206 assertEquals(false, res);
1207
1208
1209
1210 put = new Put(row1);
1211 put.add(fam1, qf1, val2);
1212 res = region.checkAndMutate(row1, fam1, qf1, CompareOp.GREATER,
1213 new BinaryComparator(val4), put, true);
1214 assertEquals(true, res);
1215
1216
1217 res = region.checkAndMutate(row1, fam1, qf1, CompareOp.GREATER_OR_EQUAL,
1218 new BinaryComparator(val1), put, true);
1219 assertEquals(false, res);
1220
1221
1222
1223 res = region.checkAndMutate(row1, fam1, qf1, CompareOp.GREATER_OR_EQUAL,
1224 new BinaryComparator(val2), put, true);
1225 assertEquals(true, res);
1226
1227
1228 res = region.checkAndMutate(row1, fam1, qf1, CompareOp.GREATER_OR_EQUAL,
1229 new BinaryComparator(val3), put, true);
1230 assertEquals(true, res);
1231 } finally {
1232 HRegion.closeHRegion(this.region);
1233 this.region = null;
1234 }
1235 }
1236
1237 @Test
1238 public void testCheckAndPut_ThatPutWasWritten() throws IOException {
1239 byte[] row1 = Bytes.toBytes("row1");
1240 byte[] fam1 = Bytes.toBytes("fam1");
1241 byte[] fam2 = Bytes.toBytes("fam2");
1242 byte[] qf1 = Bytes.toBytes("qualifier");
1243 byte[] val1 = Bytes.toBytes("value1");
1244 byte[] val2 = Bytes.toBytes("value2");
1245
1246 byte[][] families = { fam1, fam2 };
1247
1248
1249 String method = this.getName();
1250 this.region = initHRegion(tableName, method, CONF, families);
1251 try {
1252
1253 Put put = new Put(row1);
1254 put.add(fam1, qf1, val1);
1255 region.put(put);
1256
1257
1258 long ts = System.currentTimeMillis();
1259 KeyValue kv = new KeyValue(row1, fam2, qf1, ts, KeyValue.Type.Put, val2);
1260 put = new Put(row1);
1261 put.add(kv);
1262
1263
1264 HStore store = (HStore) region.getStore(fam1);
1265 store.memstore.kvset.size();
1266
1267 boolean res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL, new BinaryComparator(
1268 val1), put, true);
1269 assertEquals(true, res);
1270 store.memstore.kvset.size();
1271
1272 Get get = new Get(row1);
1273 get.addColumn(fam2, qf1);
1274 Cell[] actual = region.get(get).rawCells();
1275
1276 Cell[] expected = { kv };
1277
1278 assertEquals(expected.length, actual.length);
1279 for (int i = 0; i < actual.length; i++) {
1280 assertEquals(expected[i], actual[i]);
1281 }
1282 } finally {
1283 HRegion.closeHRegion(this.region);
1284 this.region = null;
1285 }
1286 }
1287
1288 @Test
1289 public void testCheckAndPut_wrongRowInPut() throws IOException {
1290 TableName tableName = TableName.valueOf(name.getMethodName());
1291 this.region = initHRegion(tableName, this.getName(), CONF, COLUMNS);
1292 try {
1293 Put put = new Put(row2);
1294 put.add(fam1, qual1, value1);
1295 try {
1296 region.checkAndMutate(row, fam1, qual1, CompareOp.EQUAL,
1297 new BinaryComparator(value2), put, false);
1298 fail();
1299 } catch (org.apache.hadoop.hbase.DoNotRetryIOException expected) {
1300
1301 }
1302 } finally {
1303 HRegion.closeHRegion(this.region);
1304 this.region = null;
1305 }
1306 }
1307
1308 @Test
1309 public void testCheckAndDelete_ThatDeleteWasWritten() throws IOException {
1310 byte[] row1 = Bytes.toBytes("row1");
1311 byte[] fam1 = Bytes.toBytes("fam1");
1312 byte[] fam2 = Bytes.toBytes("fam2");
1313 byte[] qf1 = Bytes.toBytes("qualifier1");
1314 byte[] qf2 = Bytes.toBytes("qualifier2");
1315 byte[] qf3 = Bytes.toBytes("qualifier3");
1316 byte[] val1 = Bytes.toBytes("value1");
1317 byte[] val2 = Bytes.toBytes("value2");
1318 byte[] val3 = Bytes.toBytes("value3");
1319 byte[] emptyVal = new byte[] {};
1320
1321 byte[][] families = { fam1, fam2 };
1322
1323
1324 String method = this.getName();
1325 this.region = initHRegion(tableName, method, CONF, families);
1326 try {
1327
1328 Put put = new Put(row1);
1329 put.add(fam1, qf1, val1);
1330 region.put(put);
1331 Threads.sleep(2);
1332
1333 put = new Put(row1);
1334 put.add(fam1, qf1, val2);
1335 put.add(fam2, qf1, val3);
1336 put.add(fam2, qf2, val2);
1337 put.add(fam2, qf3, val1);
1338 put.add(fam1, qf3, val1);
1339 region.put(put);
1340
1341
1342 Delete delete = new Delete(row1);
1343 delete.deleteColumn(fam1, qf1);
1344 delete.deleteColumn(fam2, qf1);
1345 delete.deleteColumn(fam1, qf3);
1346 boolean res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL, new BinaryComparator(
1347 val2), delete, true);
1348 assertEquals(true, res);
1349
1350 Get get = new Get(row1);
1351 get.addColumn(fam1, qf1);
1352 get.addColumn(fam1, qf3);
1353 get.addColumn(fam2, qf2);
1354 Result r = region.get(get);
1355 assertEquals(2, r.size());
1356 assertArrayEquals(val1, r.getValue(fam1, qf1));
1357 assertArrayEquals(val2, r.getValue(fam2, qf2));
1358
1359
1360 delete = new Delete(row1);
1361 delete.deleteFamily(fam2);
1362 res = region.checkAndMutate(row1, fam2, qf1, CompareOp.EQUAL, new BinaryComparator(emptyVal),
1363 delete, true);
1364 assertEquals(true, res);
1365
1366 get = new Get(row1);
1367 r = region.get(get);
1368 assertEquals(1, r.size());
1369 assertArrayEquals(val1, r.getValue(fam1, qf1));
1370
1371
1372 delete = new Delete(row1);
1373 res = region.checkAndMutate(row1, fam1, qf1, CompareOp.EQUAL, new BinaryComparator(val1),
1374 delete, true);
1375 assertEquals(true, res);
1376 get = new Get(row1);
1377 r = region.get(get);
1378 assertEquals(0, r.size());
1379 } finally {
1380 HRegion.closeHRegion(this.region);
1381 this.region = null;
1382 }
1383 }
1384
1385
1386
1387
1388 @Test
1389 public void testDelete_multiDeleteColumn() throws IOException {
1390 byte[] row1 = Bytes.toBytes("row1");
1391 byte[] fam1 = Bytes.toBytes("fam1");
1392 byte[] qual = Bytes.toBytes("qualifier");
1393 byte[] value = Bytes.toBytes("value");
1394
1395 Put put = new Put(row1);
1396 put.add(fam1, qual, 1, value);
1397 put.add(fam1, qual, 2, value);
1398
1399 String method = this.getName();
1400 this.region = initHRegion(tableName, method, CONF, fam1);
1401 try {
1402 region.put(put);
1403
1404
1405 Delete delete = new Delete(row1);
1406 delete.deleteColumn(fam1, qual);
1407 delete.deleteColumn(fam1, qual);
1408 region.delete(delete);
1409
1410 Get get = new Get(row1);
1411 get.addFamily(fam1);
1412 Result r = region.get(get);
1413 assertEquals(0, r.size());
1414 } finally {
1415 HRegion.closeHRegion(this.region);
1416 this.region = null;
1417 }
1418 }
1419
1420 @Test
1421 public void testDelete_CheckFamily() throws IOException {
1422 byte[] row1 = Bytes.toBytes("row1");
1423 byte[] fam1 = Bytes.toBytes("fam1");
1424 byte[] fam2 = Bytes.toBytes("fam2");
1425 byte[] fam3 = Bytes.toBytes("fam3");
1426 byte[] fam4 = Bytes.toBytes("fam4");
1427
1428
1429 String method = this.getName();
1430 this.region = initHRegion(tableName, method, CONF, fam1, fam2, fam3);
1431 try {
1432 List<Cell> kvs = new ArrayList<Cell>();
1433 kvs.add(new KeyValue(row1, fam4, null, null));
1434
1435
1436 byte[] family = fam2;
1437 try {
1438 NavigableMap<byte[], List<Cell>> deleteMap = new TreeMap<byte[], List<Cell>>(
1439 Bytes.BYTES_COMPARATOR);
1440 deleteMap.put(family, kvs);
1441 region.delete(deleteMap, Durability.SYNC_WAL);
1442 } catch (Exception e) {
1443 assertTrue("Family " + new String(family) + " does not exist", false);
1444 }
1445
1446
1447 boolean ok = false;
1448 family = fam4;
1449 try {
1450 NavigableMap<byte[], List<Cell>> deleteMap = new TreeMap<byte[], List<Cell>>(
1451 Bytes.BYTES_COMPARATOR);
1452 deleteMap.put(family, kvs);
1453 region.delete(deleteMap, Durability.SYNC_WAL);
1454 } catch (Exception e) {
1455 ok = true;
1456 }
1457 assertEquals("Family " + new String(family) + " does exist", true, ok);
1458 } finally {
1459 HRegion.closeHRegion(this.region);
1460 this.region = null;
1461 }
1462 }
1463
1464 @Test
1465 public void testDelete_mixed() throws IOException, InterruptedException {
1466 byte[] fam = Bytes.toBytes("info");
1467 byte[][] families = { fam };
1468 String method = this.getName();
1469 this.region = initHRegion(tableName, method, CONF, families);
1470 try {
1471 EnvironmentEdgeManagerTestHelper.injectEdge(new IncrementingEnvironmentEdge());
1472
1473 byte[] row = Bytes.toBytes("table_name");
1474
1475 byte[] serverinfo = Bytes.toBytes("serverinfo");
1476 byte[] splitA = Bytes.toBytes("splitA");
1477 byte[] splitB = Bytes.toBytes("splitB");
1478
1479
1480 Put put = new Put(row);
1481 put.add(fam, splitA, Bytes.toBytes("reference_A"));
1482 region.put(put);
1483
1484 put = new Put(row);
1485 put.add(fam, splitB, Bytes.toBytes("reference_B"));
1486 region.put(put);
1487
1488 put = new Put(row);
1489 put.add(fam, serverinfo, Bytes.toBytes("ip_address"));
1490 region.put(put);
1491
1492
1493 Delete delete = new Delete(row);
1494 delete.deleteColumns(fam, splitA);
1495 region.delete(delete);
1496
1497
1498 Get get = new Get(row).addColumn(fam, serverinfo);
1499 Result result = region.get(get);
1500 assertEquals(1, result.size());
1501
1502 get = new Get(row).addColumn(fam, splitA);
1503 result = region.get(get);
1504 assertEquals(0, result.size());
1505
1506 get = new Get(row).addColumn(fam, splitB);
1507 result = region.get(get);
1508 assertEquals(1, result.size());
1509
1510
1511 put = new Put(row);
1512 put.add(fam, splitA, Bytes.toBytes("reference_A"));
1513 region.put(put);
1514 get = new Get(row);
1515 result = region.get(get);
1516 assertEquals(3, result.size());
1517
1518
1519 delete = new Delete(row);
1520 region.delete(delete);
1521 assertEquals(0, region.get(get).size());
1522
1523 region.put(new Put(row).add(fam, splitA, Bytes.toBytes("reference_A")));
1524 result = region.get(get);
1525 assertEquals(1, result.size());
1526 } finally {
1527 HRegion.closeHRegion(this.region);
1528 this.region = null;
1529 }
1530 }
1531
1532 @Test
1533 public void testDeleteRowWithFutureTs() throws IOException {
1534 byte[] fam = Bytes.toBytes("info");
1535 byte[][] families = { fam };
1536 String method = this.getName();
1537 this.region = initHRegion(tableName, method, CONF, families);
1538 try {
1539 byte[] row = Bytes.toBytes("table_name");
1540
1541 byte[] serverinfo = Bytes.toBytes("serverinfo");
1542
1543
1544 Put put = new Put(row);
1545 put.add(fam, serverinfo, HConstants.LATEST_TIMESTAMP - 5, Bytes.toBytes("value"));
1546 region.put(put);
1547
1548
1549 Delete delete = new Delete(row);
1550 region.delete(delete);
1551
1552
1553 Get get = new Get(row).addColumn(fam, serverinfo);
1554 Result result = region.get(get);
1555 assertEquals(1, result.size());
1556
1557
1558 delete = new Delete(row, HConstants.LATEST_TIMESTAMP - 3);
1559 region.delete(delete);
1560
1561
1562 get = new Get(row).addColumn(fam, serverinfo);
1563 result = region.get(get);
1564 assertEquals(0, result.size());
1565 } finally {
1566 HRegion.closeHRegion(this.region);
1567 this.region = null;
1568 }
1569 }
1570
1571
1572
1573
1574
1575 @Test
1576 public void testPutWithLatestTS() throws IOException {
1577 byte[] fam = Bytes.toBytes("info");
1578 byte[][] families = { fam };
1579 String method = this.getName();
1580 this.region = initHRegion(tableName, method, CONF, families);
1581 try {
1582 byte[] row = Bytes.toBytes("row1");
1583
1584 byte[] qual = Bytes.toBytes("qual");
1585
1586
1587 Put put = new Put(row);
1588 put.add(fam, qual, HConstants.LATEST_TIMESTAMP, Bytes.toBytes("value"));
1589 region.put(put);
1590
1591
1592 Get get = new Get(row).addColumn(fam, qual);
1593 Result result = region.get(get);
1594 assertEquals(1, result.size());
1595 Cell kv = result.rawCells()[0];
1596 LOG.info("Got: " + kv);
1597 assertTrue("LATEST_TIMESTAMP was not replaced with real timestamp",
1598 kv.getTimestamp() != HConstants.LATEST_TIMESTAMP);
1599
1600
1601
1602 row = Bytes.toBytes("row2");
1603 put = new Put(row);
1604 put.add(fam, qual, HConstants.LATEST_TIMESTAMP, Bytes.toBytes("value"));
1605 region.put(put);
1606
1607
1608 get = new Get(row).addColumn(fam, qual);
1609 result = region.get(get);
1610 assertEquals(1, result.size());
1611 kv = result.rawCells()[0];
1612 LOG.info("Got: " + kv);
1613 assertTrue("LATEST_TIMESTAMP was not replaced with real timestamp",
1614 kv.getTimestamp() != HConstants.LATEST_TIMESTAMP);
1615 } finally {
1616 HRegion.closeHRegion(this.region);
1617 this.region = null;
1618 }
1619
1620 }
1621
1622
1623
1624
1625
1626
1627 @Test
1628 public void testPutWithTsSlop() throws IOException {
1629 byte[] fam = Bytes.toBytes("info");
1630 byte[][] families = { fam };
1631 String method = this.getName();
1632
1633
1634 CONF.setInt("hbase.hregion.keyvalue.timestamp.slop.millisecs", 1000);
1635 this.region = initHRegion(tableName, method, CONF, families);
1636 boolean caughtExcep = false;
1637 try {
1638 try {
1639
1640 region.put(new Put(row).add(fam, Bytes.toBytes("qual"), Bytes.toBytes("value")));
1641
1642 region.put(new Put(row).add(fam, Bytes.toBytes("qual"), System.currentTimeMillis() + 2000,
1643 Bytes.toBytes("value")));
1644 fail("Expected IOE for TS out of configured timerange");
1645 } catch (FailedSanityCheckException ioe) {
1646 LOG.debug("Received expected exception", ioe);
1647 caughtExcep = true;
1648 }
1649 assertTrue("Should catch FailedSanityCheckException", caughtExcep);
1650 } finally {
1651 HRegion.closeHRegion(this.region);
1652 this.region = null;
1653 }
1654 }
1655
1656 @Test
1657 public void testScanner_DeleteOneFamilyNotAnother() throws IOException {
1658 byte[] fam1 = Bytes.toBytes("columnA");
1659 byte[] fam2 = Bytes.toBytes("columnB");
1660 this.region = initHRegion(tableName, getName(), CONF, fam1, fam2);
1661 try {
1662 byte[] rowA = Bytes.toBytes("rowA");
1663 byte[] rowB = Bytes.toBytes("rowB");
1664
1665 byte[] value = Bytes.toBytes("value");
1666
1667 Delete delete = new Delete(rowA);
1668 delete.deleteFamily(fam1);
1669
1670 region.delete(delete);
1671
1672
1673 Put put = new Put(rowA);
1674 put.add(fam2, null, value);
1675 region.put(put);
1676
1677 put = new Put(rowB);
1678 put.add(fam1, null, value);
1679 put.add(fam2, null, value);
1680 region.put(put);
1681
1682 Scan scan = new Scan();
1683 scan.addFamily(fam1).addFamily(fam2);
1684 InternalScanner s = region.getScanner(scan);
1685 List<Cell> results = new ArrayList<Cell>();
1686 s.next(results);
1687 assertTrue(CellUtil.matchingRow(results.get(0), rowA));
1688
1689 results.clear();
1690 s.next(results);
1691 assertTrue(CellUtil.matchingRow(results.get(0), rowB));
1692 } finally {
1693 HRegion.closeHRegion(this.region);
1694 this.region = null;
1695 }
1696 }
1697
1698 @Test
1699 public void testDeleteColumns_PostInsert() throws IOException, InterruptedException {
1700 Delete delete = new Delete(row);
1701 delete.deleteColumns(fam1, qual1);
1702 doTestDelete_AndPostInsert(delete);
1703 }
1704
1705 @Test
1706 public void testDeleteFamily_PostInsert() throws IOException, InterruptedException {
1707 Delete delete = new Delete(row);
1708 delete.deleteFamily(fam1);
1709 doTestDelete_AndPostInsert(delete);
1710 }
1711
1712 public void doTestDelete_AndPostInsert(Delete delete) throws IOException, InterruptedException {
1713 TableName tableName = TableName.valueOf(name.getMethodName());
1714 this.region = initHRegion(tableName, getName(), CONF, fam1);
1715 try {
1716 EnvironmentEdgeManagerTestHelper.injectEdge(new IncrementingEnvironmentEdge());
1717 Put put = new Put(row);
1718 put.add(fam1, qual1, value1);
1719 region.put(put);
1720
1721
1722 region.delete(delete);
1723
1724
1725 put = new Put(row);
1726 put.add(fam1, qual1, value2);
1727 region.put(put);
1728
1729
1730 Get get = new Get(row);
1731 get.addColumn(fam1, qual1);
1732
1733 Result r = region.get(get);
1734 assertEquals(1, r.size());
1735 assertArrayEquals(value2, r.getValue(fam1, qual1));
1736
1737
1738 Scan scan = new Scan(row);
1739 scan.addColumn(fam1, qual1);
1740 InternalScanner s = region.getScanner(scan);
1741
1742 List<Cell> results = new ArrayList<Cell>();
1743 assertEquals(false, s.next(results));
1744 assertEquals(1, results.size());
1745 Cell kv = results.get(0);
1746
1747 assertArrayEquals(value2, CellUtil.cloneValue(kv));
1748 assertArrayEquals(fam1, CellUtil.cloneFamily(kv));
1749 assertArrayEquals(qual1, CellUtil.cloneQualifier(kv));
1750 assertArrayEquals(row, CellUtil.cloneRow(kv));
1751 } finally {
1752 HRegion.closeHRegion(this.region);
1753 this.region = null;
1754 }
1755 }
1756
1757 @Test
1758 public void testDelete_CheckTimestampUpdated() throws IOException {
1759 TableName tableName = TableName.valueOf(name.getMethodName());
1760 byte[] row1 = Bytes.toBytes("row1");
1761 byte[] col1 = Bytes.toBytes("col1");
1762 byte[] col2 = Bytes.toBytes("col2");
1763 byte[] col3 = Bytes.toBytes("col3");
1764
1765
1766 String method = this.getName();
1767 this.region = initHRegion(tableName, method, CONF, fam1);
1768 try {
1769
1770 List<Cell> kvs = new ArrayList<Cell>();
1771 kvs.add(new KeyValue(row1, fam1, col1, null));
1772 kvs.add(new KeyValue(row1, fam1, col2, null));
1773 kvs.add(new KeyValue(row1, fam1, col3, null));
1774
1775 NavigableMap<byte[], List<Cell>> deleteMap = new TreeMap<byte[], List<Cell>>(
1776 Bytes.BYTES_COMPARATOR);
1777 deleteMap.put(fam1, kvs);
1778 region.delete(deleteMap, Durability.SYNC_WAL);
1779
1780
1781
1782 long now = System.currentTimeMillis();
1783 KeyValue firstKv = ((HStore) region.getStore(fam1)).memstore.kvset.first();
1784 assertTrue(firstKv.getTimestamp() <= now);
1785 now = firstKv.getTimestamp();
1786 for (KeyValue kv : ((HStore) region.getStore(fam1)).memstore.kvset) {
1787 assertTrue(kv.getTimestamp() <= now);
1788 now = kv.getTimestamp();
1789 }
1790 } finally {
1791 HRegion.closeHRegion(this.region);
1792 this.region = null;
1793 }
1794 }
1795
1796
1797
1798
1799 @Test
1800 public void testGet_FamilyChecker() throws IOException {
1801 byte[] row1 = Bytes.toBytes("row1");
1802 byte[] fam1 = Bytes.toBytes("fam1");
1803 byte[] fam2 = Bytes.toBytes("False");
1804 byte[] col1 = Bytes.toBytes("col1");
1805
1806
1807 String method = this.getName();
1808 this.region = initHRegion(tableName, method, CONF, fam1);
1809 try {
1810 Get get = new Get(row1);
1811 get.addColumn(fam2, col1);
1812
1813
1814 try {
1815 region.get(get);
1816 } catch (org.apache.hadoop.hbase.DoNotRetryIOException e) {
1817 assertFalse(false);
1818 return;
1819 }
1820 assertFalse(true);
1821 } finally {
1822 HRegion.closeHRegion(this.region);
1823 this.region = null;
1824 }
1825 }
1826
1827 @Test
1828 public void testGet_Basic() throws IOException {
1829 byte[] row1 = Bytes.toBytes("row1");
1830 byte[] fam1 = Bytes.toBytes("fam1");
1831 byte[] col1 = Bytes.toBytes("col1");
1832 byte[] col2 = Bytes.toBytes("col2");
1833 byte[] col3 = Bytes.toBytes("col3");
1834 byte[] col4 = Bytes.toBytes("col4");
1835 byte[] col5 = Bytes.toBytes("col5");
1836
1837
1838 String method = this.getName();
1839 this.region = initHRegion(tableName, method, CONF, fam1);
1840 try {
1841
1842 Put put = new Put(row1);
1843 put.add(fam1, col1, null);
1844 put.add(fam1, col2, null);
1845 put.add(fam1, col3, null);
1846 put.add(fam1, col4, null);
1847 put.add(fam1, col5, null);
1848 region.put(put);
1849
1850 Get get = new Get(row1);
1851 get.addColumn(fam1, col2);
1852 get.addColumn(fam1, col4);
1853
1854 KeyValue kv1 = new KeyValue(row1, fam1, col2);
1855 KeyValue kv2 = new KeyValue(row1, fam1, col4);
1856 KeyValue[] expected = { kv1, kv2 };
1857
1858
1859 Result res = region.get(get);
1860 assertEquals(expected.length, res.size());
1861 for (int i = 0; i < res.size(); i++) {
1862 assertTrue(CellUtil.matchingRow(expected[i], res.rawCells()[i]));
1863 assertTrue(CellUtil.matchingFamily(expected[i], res.rawCells()[i]));
1864 assertTrue(CellUtil.matchingQualifier(expected[i], res.rawCells()[i]));
1865 }
1866
1867
1868 Get g = new Get(row1);
1869 final int count = 2;
1870 g.setFilter(new ColumnCountGetFilter(count));
1871 res = region.get(g);
1872 assertEquals(count, res.size());
1873 } finally {
1874 HRegion.closeHRegion(this.region);
1875 this.region = null;
1876 }
1877 }
1878
1879 @Test
1880 public void testGet_Empty() throws IOException {
1881 byte[] row = Bytes.toBytes("row");
1882 byte[] fam = Bytes.toBytes("fam");
1883
1884 String method = this.getName();
1885 this.region = initHRegion(tableName, method, CONF, fam);
1886 try {
1887 Get get = new Get(row);
1888 get.addFamily(fam);
1889 Result r = region.get(get);
1890
1891 assertTrue(r.isEmpty());
1892 } finally {
1893 HRegion.closeHRegion(this.region);
1894 this.region = null;
1895 }
1896 }
1897
1898
1899
1900
1901 @Test
1902 public void testMerge() throws IOException {
1903 byte[][] families = { fam1, fam2, fam3 };
1904 Configuration hc = initSplit();
1905
1906 String method = this.getName();
1907 this.region = initHRegion(tableName, method, hc, families);
1908 try {
1909 LOG.info("" + HBaseTestCase.addContent(region, fam3));
1910 region.flushcache();
1911 region.compactStores();
1912 byte[] splitRow = region.checkSplit();
1913 assertNotNull(splitRow);
1914 LOG.info("SplitRow: " + Bytes.toString(splitRow));
1915 HRegion[] subregions = splitRegion(region, splitRow);
1916 try {
1917
1918 for (int i = 0; i < subregions.length; i++) {
1919 HRegion.openHRegion(subregions[i], null);
1920 subregions[i].compactStores();
1921 }
1922 Path oldRegionPath = region.getRegionFileSystem().getRegionDir();
1923 Path oldRegion1 = subregions[0].getRegionFileSystem().getRegionDir();
1924 Path oldRegion2 = subregions[1].getRegionFileSystem().getRegionDir();
1925 long startTime = System.currentTimeMillis();
1926 region = HRegion.mergeAdjacent(subregions[0], subregions[1]);
1927 LOG.info("Merge regions elapsed time: "
1928 + ((System.currentTimeMillis() - startTime) / 1000.0));
1929 FILESYSTEM.delete(oldRegion1, true);
1930 FILESYSTEM.delete(oldRegion2, true);
1931 FILESYSTEM.delete(oldRegionPath, true);
1932 LOG.info("splitAndMerge completed.");
1933 } finally {
1934 for (int i = 0; i < subregions.length; i++) {
1935 try {
1936 HRegion.closeHRegion(subregions[i]);
1937 } catch (IOException e) {
1938
1939 }
1940 }
1941 }
1942 } finally {
1943 HRegion.closeHRegion(this.region);
1944 this.region = null;
1945 }
1946 }
1947
1948
1949
1950
1951
1952
1953
1954
1955
1956 HRegion[] splitRegion(final HRegion parent, final byte[] midkey) throws IOException {
1957 PairOfSameType<HRegion> result = null;
1958 SplitTransaction st = new SplitTransaction(parent, midkey);
1959
1960
1961 if (!st.prepare())
1962 return null;
1963 try {
1964 result = st.execute(null, null);
1965 } catch (IOException ioe) {
1966 try {
1967 LOG.info("Running rollback of failed split of " + parent.getRegionNameAsString() + "; "
1968 + ioe.getMessage());
1969 st.rollback(null, null);
1970 LOG.info("Successful rollback of failed split of " + parent.getRegionNameAsString());
1971 return null;
1972 } catch (RuntimeException e) {
1973
1974 LOG.info("Failed rollback of failed split of " + parent.getRegionNameAsString()
1975 + " -- aborting server", e);
1976 }
1977 }
1978 return new HRegion[] { result.getFirst(), result.getSecond() };
1979 }
1980
1981
1982
1983
1984 @Test
1985 public void testGetScanner_WithOkFamilies() throws IOException {
1986 byte[] fam1 = Bytes.toBytes("fam1");
1987 byte[] fam2 = Bytes.toBytes("fam2");
1988
1989 byte[][] families = { fam1, fam2 };
1990
1991
1992 String method = this.getName();
1993 this.region = initHRegion(tableName, method, CONF, families);
1994 try {
1995 Scan scan = new Scan();
1996 scan.addFamily(fam1);
1997 scan.addFamily(fam2);
1998 try {
1999 region.getScanner(scan);
2000 } catch (Exception e) {
2001 assertTrue("Families could not be found in Region", false);
2002 }
2003 } finally {
2004 HRegion.closeHRegion(this.region);
2005 this.region = null;
2006 }
2007 }
2008
2009 @Test
2010 public void testGetScanner_WithNotOkFamilies() throws IOException {
2011 byte[] fam1 = Bytes.toBytes("fam1");
2012 byte[] fam2 = Bytes.toBytes("fam2");
2013
2014 byte[][] families = { fam1 };
2015
2016
2017 String method = this.getName();
2018 this.region = initHRegion(tableName, method, CONF, families);
2019 try {
2020 Scan scan = new Scan();
2021 scan.addFamily(fam2);
2022 boolean ok = false;
2023 try {
2024 region.getScanner(scan);
2025 } catch (Exception e) {
2026 ok = true;
2027 }
2028 assertTrue("Families could not be found in Region", ok);
2029 } finally {
2030 HRegion.closeHRegion(this.region);
2031 this.region = null;
2032 }
2033 }
2034
2035 @Test
2036 public void testGetScanner_WithNoFamilies() throws IOException {
2037 byte[] row1 = Bytes.toBytes("row1");
2038 byte[] fam1 = Bytes.toBytes("fam1");
2039 byte[] fam2 = Bytes.toBytes("fam2");
2040 byte[] fam3 = Bytes.toBytes("fam3");
2041 byte[] fam4 = Bytes.toBytes("fam4");
2042
2043 byte[][] families = { fam1, fam2, fam3, fam4 };
2044
2045
2046 String method = this.getName();
2047 this.region = initHRegion(tableName, method, CONF, families);
2048 try {
2049
2050
2051 Put put = new Put(row1);
2052 put.add(fam1, null, null);
2053 put.add(fam2, null, null);
2054 put.add(fam3, null, null);
2055 put.add(fam4, null, null);
2056 region.put(put);
2057
2058 Scan scan = null;
2059 HRegion.RegionScannerImpl is = null;
2060
2061
2062
2063
2064 scan = new Scan();
2065 scan.addFamily(fam2);
2066 scan.addFamily(fam4);
2067 is = (RegionScannerImpl) region.getScanner(scan);
2068 MultiVersionConsistencyControl.resetThreadReadPoint(region.getMVCC());
2069 assertEquals(1, ((RegionScannerImpl) is).storeHeap.getHeap().size());
2070
2071 scan = new Scan();
2072 is = (RegionScannerImpl) region.getScanner(scan);
2073 MultiVersionConsistencyControl.resetThreadReadPoint(region.getMVCC());
2074 assertEquals(families.length - 1, ((RegionScannerImpl) is).storeHeap.getHeap().size());
2075 } finally {
2076 HRegion.closeHRegion(this.region);
2077 this.region = null;
2078 }
2079 }
2080
2081
2082
2083
2084
2085
2086 @Test
2087 public void testGetScanner_WithRegionClosed() throws IOException {
2088 byte[] fam1 = Bytes.toBytes("fam1");
2089 byte[] fam2 = Bytes.toBytes("fam2");
2090
2091 byte[][] families = { fam1, fam2 };
2092
2093
2094 String method = this.getName();
2095 try {
2096 this.region = initHRegion(tableName, method, CONF, families);
2097 } catch (IOException e) {
2098 e.printStackTrace();
2099 fail("Got IOException during initHRegion, " + e.getMessage());
2100 }
2101 try {
2102 region.closed.set(true);
2103 try {
2104 region.getScanner(null);
2105 fail("Expected to get an exception during getScanner on a region that is closed");
2106 } catch (NotServingRegionException e) {
2107
2108 } catch (IOException e) {
2109 fail("Got wrong type of exception - should be a NotServingRegionException, but was an IOException: "
2110 + e.getMessage());
2111 }
2112 } finally {
2113 HRegion.closeHRegion(this.region);
2114 this.region = null;
2115 }
2116 }
2117
2118 @Test
2119 public void testRegionScanner_Next() throws IOException {
2120 byte[] row1 = Bytes.toBytes("row1");
2121 byte[] row2 = Bytes.toBytes("row2");
2122 byte[] fam1 = Bytes.toBytes("fam1");
2123 byte[] fam2 = Bytes.toBytes("fam2");
2124 byte[] fam3 = Bytes.toBytes("fam3");
2125 byte[] fam4 = Bytes.toBytes("fam4");
2126
2127 byte[][] families = { fam1, fam2, fam3, fam4 };
2128 long ts = System.currentTimeMillis();
2129
2130
2131 String method = this.getName();
2132 this.region = initHRegion(tableName, method, CONF, families);
2133 try {
2134
2135 Put put = null;
2136 put = new Put(row1);
2137 put.add(fam1, (byte[]) null, ts, null);
2138 put.add(fam2, (byte[]) null, ts, null);
2139 put.add(fam3, (byte[]) null, ts, null);
2140 put.add(fam4, (byte[]) null, ts, null);
2141 region.put(put);
2142
2143 put = new Put(row2);
2144 put.add(fam1, (byte[]) null, ts, null);
2145 put.add(fam2, (byte[]) null, ts, null);
2146 put.add(fam3, (byte[]) null, ts, null);
2147 put.add(fam4, (byte[]) null, ts, null);
2148 region.put(put);
2149
2150 Scan scan = new Scan();
2151 scan.addFamily(fam2);
2152 scan.addFamily(fam4);
2153 InternalScanner is = region.getScanner(scan);
2154
2155 List<Cell> res = null;
2156
2157
2158 List<Cell> expected1 = new ArrayList<Cell>();
2159 expected1.add(new KeyValue(row1, fam2, null, ts, KeyValue.Type.Put, null));
2160 expected1.add(new KeyValue(row1, fam4, null, ts, KeyValue.Type.Put, null));
2161
2162 res = new ArrayList<Cell>();
2163 is.next(res);
2164 for (int i = 0; i < res.size(); i++) {
2165 assertTrue(CellComparator.equalsIgnoreMvccVersion(expected1.get(i), res.get(i)));
2166 }
2167
2168
2169 List<Cell> expected2 = new ArrayList<Cell>();
2170 expected2.add(new KeyValue(row2, fam2, null, ts, KeyValue.Type.Put, null));
2171 expected2.add(new KeyValue(row2, fam4, null, ts, KeyValue.Type.Put, null));
2172
2173 res = new ArrayList<Cell>();
2174 is.next(res);
2175 for (int i = 0; i < res.size(); i++) {
2176 assertTrue(CellComparator.equalsIgnoreMvccVersion(expected2.get(i), res.get(i)));
2177 }
2178 } finally {
2179 HRegion.closeHRegion(this.region);
2180 this.region = null;
2181 }
2182 }
2183
2184 @Test
2185 public void testScanner_ExplicitColumns_FromMemStore_EnforceVersions() throws IOException {
2186 byte[] row1 = Bytes.toBytes("row1");
2187 byte[] qf1 = Bytes.toBytes("qualifier1");
2188 byte[] qf2 = Bytes.toBytes("qualifier2");
2189 byte[] fam1 = Bytes.toBytes("fam1");
2190 byte[][] families = { fam1 };
2191
2192 long ts1 = System.currentTimeMillis();
2193 long ts2 = ts1 + 1;
2194 long ts3 = ts1 + 2;
2195
2196
2197 String method = this.getName();
2198 this.region = initHRegion(tableName, method, CONF, families);
2199 try {
2200
2201 Put put = null;
2202 KeyValue kv13 = new KeyValue(row1, fam1, qf1, ts3, KeyValue.Type.Put, null);
2203 KeyValue kv12 = new KeyValue(row1, fam1, qf1, ts2, KeyValue.Type.Put, null);
2204 KeyValue kv11 = new KeyValue(row1, fam1, qf1, ts1, KeyValue.Type.Put, null);
2205
2206 KeyValue kv23 = new KeyValue(row1, fam1, qf2, ts3, KeyValue.Type.Put, null);
2207 KeyValue kv22 = new KeyValue(row1, fam1, qf2, ts2, KeyValue.Type.Put, null);
2208 KeyValue kv21 = new KeyValue(row1, fam1, qf2, ts1, KeyValue.Type.Put, null);
2209
2210 put = new Put(row1);
2211 put.add(kv13);
2212 put.add(kv12);
2213 put.add(kv11);
2214 put.add(kv23);
2215 put.add(kv22);
2216 put.add(kv21);
2217 region.put(put);
2218
2219
2220 List<Cell> expected = new ArrayList<Cell>();
2221 expected.add(kv13);
2222 expected.add(kv12);
2223
2224 Scan scan = new Scan(row1);
2225 scan.addColumn(fam1, qf1);
2226 scan.setMaxVersions(MAX_VERSIONS);
2227 List<Cell> actual = new ArrayList<Cell>();
2228 InternalScanner scanner = region.getScanner(scan);
2229
2230 boolean hasNext = scanner.next(actual);
2231 assertEquals(false, hasNext);
2232
2233
2234 for (int i = 0; i < expected.size(); i++) {
2235 assertEquals(expected.get(i), actual.get(i));
2236 }
2237 } finally {
2238 HRegion.closeHRegion(this.region);
2239 this.region = null;
2240 }
2241 }
2242
2243 @Test
2244 public void testScanner_ExplicitColumns_FromFilesOnly_EnforceVersions() throws IOException {
2245 byte[] row1 = Bytes.toBytes("row1");
2246 byte[] qf1 = Bytes.toBytes("qualifier1");
2247 byte[] qf2 = Bytes.toBytes("qualifier2");
2248 byte[] fam1 = Bytes.toBytes("fam1");
2249 byte[][] families = { fam1 };
2250
2251 long ts1 = 1;
2252 long ts2 = ts1 + 1;
2253 long ts3 = ts1 + 2;
2254
2255
2256 String method = this.getName();
2257 this.region = initHRegion(tableName, method, CONF, families);
2258 try {
2259
2260 Put put = null;
2261 KeyValue kv13 = new KeyValue(row1, fam1, qf1, ts3, KeyValue.Type.Put, null);
2262 KeyValue kv12 = new KeyValue(row1, fam1, qf1, ts2, KeyValue.Type.Put, null);
2263 KeyValue kv11 = new KeyValue(row1, fam1, qf1, ts1, KeyValue.Type.Put, null);
2264
2265 KeyValue kv23 = new KeyValue(row1, fam1, qf2, ts3, KeyValue.Type.Put, null);
2266 KeyValue kv22 = new KeyValue(row1, fam1, qf2, ts2, KeyValue.Type.Put, null);
2267 KeyValue kv21 = new KeyValue(row1, fam1, qf2, ts1, KeyValue.Type.Put, null);
2268
2269 put = new Put(row1);
2270 put.add(kv13);
2271 put.add(kv12);
2272 put.add(kv11);
2273 put.add(kv23);
2274 put.add(kv22);
2275 put.add(kv21);
2276 region.put(put);
2277 region.flushcache();
2278
2279
2280 List<Cell> expected = new ArrayList<Cell>();
2281 expected.add(kv13);
2282 expected.add(kv12);
2283 expected.add(kv23);
2284 expected.add(kv22);
2285
2286 Scan scan = new Scan(row1);
2287 scan.addColumn(fam1, qf1);
2288 scan.addColumn(fam1, qf2);
2289 scan.setMaxVersions(MAX_VERSIONS);
2290 List<Cell> actual = new ArrayList<Cell>();
2291 InternalScanner scanner = region.getScanner(scan);
2292
2293 boolean hasNext = scanner.next(actual);
2294 assertEquals(false, hasNext);
2295
2296
2297 for (int i = 0; i < expected.size(); i++) {
2298 assertTrue(CellComparator.equalsIgnoreMvccVersion(expected.get(i), actual.get(i)));
2299 }
2300 } finally {
2301 HRegion.closeHRegion(this.region);
2302 this.region = null;
2303 }
2304 }
2305
2306 @Test
2307 public void testScanner_ExplicitColumns_FromMemStoreAndFiles_EnforceVersions() throws IOException {
2308 byte[] row1 = Bytes.toBytes("row1");
2309 byte[] fam1 = Bytes.toBytes("fam1");
2310 byte[][] families = { fam1 };
2311 byte[] qf1 = Bytes.toBytes("qualifier1");
2312 byte[] qf2 = Bytes.toBytes("qualifier2");
2313
2314 long ts1 = 1;
2315 long ts2 = ts1 + 1;
2316 long ts3 = ts1 + 2;
2317 long ts4 = ts1 + 3;
2318
2319
2320 String method = this.getName();
2321 this.region = initHRegion(tableName, method, CONF, families);
2322 try {
2323
2324 KeyValue kv14 = new KeyValue(row1, fam1, qf1, ts4, KeyValue.Type.Put, null);
2325 KeyValue kv13 = new KeyValue(row1, fam1, qf1, ts3, KeyValue.Type.Put, null);
2326 KeyValue kv12 = new KeyValue(row1, fam1, qf1, ts2, KeyValue.Type.Put, null);
2327 KeyValue kv11 = new KeyValue(row1, fam1, qf1, ts1, KeyValue.Type.Put, null);
2328
2329 KeyValue kv24 = new KeyValue(row1, fam1, qf2, ts4, KeyValue.Type.Put, null);
2330 KeyValue kv23 = new KeyValue(row1, fam1, qf2, ts3, KeyValue.Type.Put, null);
2331 KeyValue kv22 = new KeyValue(row1, fam1, qf2, ts2, KeyValue.Type.Put, null);
2332 KeyValue kv21 = new KeyValue(row1, fam1, qf2, ts1, KeyValue.Type.Put, null);
2333
2334 Put put = null;
2335 put = new Put(row1);
2336 put.add(kv14);
2337 put.add(kv24);
2338 region.put(put);
2339 region.flushcache();
2340
2341 put = new Put(row1);
2342 put.add(kv23);
2343 put.add(kv13);
2344 region.put(put);
2345 region.flushcache();
2346
2347 put = new Put(row1);
2348 put.add(kv22);
2349 put.add(kv12);
2350 region.put(put);
2351 region.flushcache();
2352
2353 put = new Put(row1);
2354 put.add(kv21);
2355 put.add(kv11);
2356 region.put(put);
2357
2358
2359 List<Cell> expected = new ArrayList<Cell>();
2360 expected.add(kv14);
2361 expected.add(kv13);
2362 expected.add(kv12);
2363 expected.add(kv24);
2364 expected.add(kv23);
2365 expected.add(kv22);
2366
2367 Scan scan = new Scan(row1);
2368 scan.addColumn(fam1, qf1);
2369 scan.addColumn(fam1, qf2);
2370 int versions = 3;
2371 scan.setMaxVersions(versions);
2372 List<Cell> actual = new ArrayList<Cell>();
2373 InternalScanner scanner = region.getScanner(scan);
2374
2375 boolean hasNext = scanner.next(actual);
2376 assertEquals(false, hasNext);
2377
2378
2379 for (int i = 0; i < expected.size(); i++) {
2380 assertTrue(CellComparator.equalsIgnoreMvccVersion(expected.get(i), actual.get(i)));
2381 }
2382 } finally {
2383 HRegion.closeHRegion(this.region);
2384 this.region = null;
2385 }
2386 }
2387
2388 @Test
2389 public void testScanner_Wildcard_FromMemStore_EnforceVersions() throws IOException {
2390 byte[] row1 = Bytes.toBytes("row1");
2391 byte[] qf1 = Bytes.toBytes("qualifier1");
2392 byte[] qf2 = Bytes.toBytes("qualifier2");
2393 byte[] fam1 = Bytes.toBytes("fam1");
2394 byte[][] families = { fam1 };
2395
2396 long ts1 = System.currentTimeMillis();
2397 long ts2 = ts1 + 1;
2398 long ts3 = ts1 + 2;
2399
2400
2401 String method = this.getName();
2402 this.region = initHRegion(tableName, method, CONF, families);
2403 try {
2404
2405 Put put = null;
2406 KeyValue kv13 = new KeyValue(row1, fam1, qf1, ts3, KeyValue.Type.Put, null);
2407 KeyValue kv12 = new KeyValue(row1, fam1, qf1, ts2, KeyValue.Type.Put, null);
2408 KeyValue kv11 = new KeyValue(row1, fam1, qf1, ts1, KeyValue.Type.Put, null);
2409
2410 KeyValue kv23 = new KeyValue(row1, fam1, qf2, ts3, KeyValue.Type.Put, null);
2411 KeyValue kv22 = new KeyValue(row1, fam1, qf2, ts2, KeyValue.Type.Put, null);
2412 KeyValue kv21 = new KeyValue(row1, fam1, qf2, ts1, KeyValue.Type.Put, null);
2413
2414 put = new Put(row1);
2415 put.add(kv13);
2416 put.add(kv12);
2417 put.add(kv11);
2418 put.add(kv23);
2419 put.add(kv22);
2420 put.add(kv21);
2421 region.put(put);
2422
2423
2424 List<Cell> expected = new ArrayList<Cell>();
2425 expected.add(kv13);
2426 expected.add(kv12);
2427 expected.add(kv23);
2428 expected.add(kv22);
2429
2430 Scan scan = new Scan(row1);
2431 scan.addFamily(fam1);
2432 scan.setMaxVersions(MAX_VERSIONS);
2433 List<Cell> actual = new ArrayList<Cell>();
2434 InternalScanner scanner = region.getScanner(scan);
2435
2436 boolean hasNext = scanner.next(actual);
2437 assertEquals(false, hasNext);
2438
2439
2440 for (int i = 0; i < expected.size(); i++) {
2441 assertEquals(expected.get(i), actual.get(i));
2442 }
2443 } finally {
2444 HRegion.closeHRegion(this.region);
2445 this.region = null;
2446 }
2447 }
2448
2449 @Test
2450 public void testScanner_Wildcard_FromFilesOnly_EnforceVersions() throws IOException {
2451 byte[] row1 = Bytes.toBytes("row1");
2452 byte[] qf1 = Bytes.toBytes("qualifier1");
2453 byte[] qf2 = Bytes.toBytes("qualifier2");
2454 byte[] fam1 = Bytes.toBytes("fam1");
2455
2456 long ts1 = 1;
2457 long ts2 = ts1 + 1;
2458 long ts3 = ts1 + 2;
2459
2460
2461 String method = this.getName();
2462 this.region = initHRegion(tableName, method, CONF, fam1);
2463 try {
2464
2465 Put put = null;
2466 KeyValue kv13 = new KeyValue(row1, fam1, qf1, ts3, KeyValue.Type.Put, null);
2467 KeyValue kv12 = new KeyValue(row1, fam1, qf1, ts2, KeyValue.Type.Put, null);
2468 KeyValue kv11 = new KeyValue(row1, fam1, qf1, ts1, KeyValue.Type.Put, null);
2469
2470 KeyValue kv23 = new KeyValue(row1, fam1, qf2, ts3, KeyValue.Type.Put, null);
2471 KeyValue kv22 = new KeyValue(row1, fam1, qf2, ts2, KeyValue.Type.Put, null);
2472 KeyValue kv21 = new KeyValue(row1, fam1, qf2, ts1, KeyValue.Type.Put, null);
2473
2474 put = new Put(row1);
2475 put.add(kv13);
2476 put.add(kv12);
2477 put.add(kv11);
2478 put.add(kv23);
2479 put.add(kv22);
2480 put.add(kv21);
2481 region.put(put);
2482 region.flushcache();
2483
2484
2485 List<Cell> expected = new ArrayList<Cell>();
2486 expected.add(kv13);
2487 expected.add(kv12);
2488 expected.add(kv23);
2489 expected.add(kv22);
2490
2491 Scan scan = new Scan(row1);
2492 scan.addFamily(fam1);
2493 scan.setMaxVersions(MAX_VERSIONS);
2494 List<Cell> actual = new ArrayList<Cell>();
2495 InternalScanner scanner = region.getScanner(scan);
2496
2497 boolean hasNext = scanner.next(actual);
2498 assertEquals(false, hasNext);
2499
2500
2501 for (int i = 0; i < expected.size(); i++) {
2502 assertTrue(CellComparator.equalsIgnoreMvccVersion(expected.get(i), actual.get(i)));
2503 }
2504 } finally {
2505 HRegion.closeHRegion(this.region);
2506 this.region = null;
2507 }
2508 }
2509
2510 @Test
2511 public void testScanner_StopRow1542() throws IOException {
2512 byte[] family = Bytes.toBytes("testFamily");
2513 this.region = initHRegion(tableName, getName(), CONF, family);
2514 try {
2515 byte[] row1 = Bytes.toBytes("row111");
2516 byte[] row2 = Bytes.toBytes("row222");
2517 byte[] row3 = Bytes.toBytes("row333");
2518 byte[] row4 = Bytes.toBytes("row444");
2519 byte[] row5 = Bytes.toBytes("row555");
2520
2521 byte[] col1 = Bytes.toBytes("Pub111");
2522 byte[] col2 = Bytes.toBytes("Pub222");
2523
2524 Put put = new Put(row1);
2525 put.add(family, col1, Bytes.toBytes(10L));
2526 region.put(put);
2527
2528 put = new Put(row2);
2529 put.add(family, col1, Bytes.toBytes(15L));
2530 region.put(put);
2531
2532 put = new Put(row3);
2533 put.add(family, col2, Bytes.toBytes(20L));
2534 region.put(put);
2535
2536 put = new Put(row4);
2537 put.add(family, col2, Bytes.toBytes(30L));
2538 region.put(put);
2539
2540 put = new Put(row5);
2541 put.add(family, col1, Bytes.toBytes(40L));
2542 region.put(put);
2543
2544 Scan scan = new Scan(row3, row4);
2545 scan.setMaxVersions();
2546 scan.addColumn(family, col1);
2547 InternalScanner s = region.getScanner(scan);
2548
2549 List<Cell> results = new ArrayList<Cell>();
2550 assertEquals(false, s.next(results));
2551 assertEquals(0, results.size());
2552 } finally {
2553 HRegion.closeHRegion(this.region);
2554 this.region = null;
2555 }
2556 }
2557
2558 private void assertICV(byte [] row,
2559 byte [] familiy,
2560 byte[] qualifier,
2561 long amount) throws IOException {
2562
2563 Get get = new Get(row);
2564 get.addColumn(familiy, qualifier);
2565 Result result = region.get(get);
2566 assertEquals(1, result.size());
2567
2568 Cell kv = result.rawCells()[0];
2569 long r = Bytes.toLong(CellUtil.cloneValue(kv));
2570 assertEquals(amount, r);
2571 }
2572
2573 @Test
2574 public void testScanner_Wildcard_FromMemStoreAndFiles_EnforceVersions() throws IOException {
2575 byte[] row1 = Bytes.toBytes("row1");
2576 byte[] fam1 = Bytes.toBytes("fam1");
2577 byte[] qf1 = Bytes.toBytes("qualifier1");
2578 byte[] qf2 = Bytes.toBytes("quateslifier2");
2579
2580 long ts1 = 1;
2581 long ts2 = ts1 + 1;
2582 long ts3 = ts1 + 2;
2583 long ts4 = ts1 + 3;
2584
2585
2586 String method = this.getName();
2587 this.region = initHRegion(tableName, method, CONF, fam1);
2588 try {
2589
2590 KeyValue kv14 = new KeyValue(row1, fam1, qf1, ts4, KeyValue.Type.Put, null);
2591 KeyValue kv13 = new KeyValue(row1, fam1, qf1, ts3, KeyValue.Type.Put, null);
2592 KeyValue kv12 = new KeyValue(row1, fam1, qf1, ts2, KeyValue.Type.Put, null);
2593 KeyValue kv11 = new KeyValue(row1, fam1, qf1, ts1, KeyValue.Type.Put, null);
2594
2595 KeyValue kv24 = new KeyValue(row1, fam1, qf2, ts4, KeyValue.Type.Put, null);
2596 KeyValue kv23 = new KeyValue(row1, fam1, qf2, ts3, KeyValue.Type.Put, null);
2597 KeyValue kv22 = new KeyValue(row1, fam1, qf2, ts2, KeyValue.Type.Put, null);
2598 KeyValue kv21 = new KeyValue(row1, fam1, qf2, ts1, KeyValue.Type.Put, null);
2599
2600 Put put = null;
2601 put = new Put(row1);
2602 put.add(kv14);
2603 put.add(kv24);
2604 region.put(put);
2605 region.flushcache();
2606
2607 put = new Put(row1);
2608 put.add(kv23);
2609 put.add(kv13);
2610 region.put(put);
2611 region.flushcache();
2612
2613 put = new Put(row1);
2614 put.add(kv22);
2615 put.add(kv12);
2616 region.put(put);
2617 region.flushcache();
2618
2619 put = new Put(row1);
2620 put.add(kv21);
2621 put.add(kv11);
2622 region.put(put);
2623
2624
2625 List<KeyValue> expected = new ArrayList<KeyValue>();
2626 expected.add(kv14);
2627 expected.add(kv13);
2628 expected.add(kv12);
2629 expected.add(kv24);
2630 expected.add(kv23);
2631 expected.add(kv22);
2632
2633 Scan scan = new Scan(row1);
2634 int versions = 3;
2635 scan.setMaxVersions(versions);
2636 List<Cell> actual = new ArrayList<Cell>();
2637 InternalScanner scanner = region.getScanner(scan);
2638
2639 boolean hasNext = scanner.next(actual);
2640 assertEquals(false, hasNext);
2641
2642
2643 for (int i = 0; i < expected.size(); i++) {
2644 assertTrue(CellComparator.equalsIgnoreMvccVersion(expected.get(i), actual.get(i)));
2645 }
2646 } finally {
2647 HRegion.closeHRegion(this.region);
2648 this.region = null;
2649 }
2650 }
2651
2652
2653
2654
2655
2656
2657
2658 @Test
2659 public void testScanner_JoinedScanners() throws IOException {
2660 byte[] cf_essential = Bytes.toBytes("essential");
2661 byte[] cf_joined = Bytes.toBytes("joined");
2662 byte[] cf_alpha = Bytes.toBytes("alpha");
2663 this.region = initHRegion(tableName, getName(), CONF, cf_essential, cf_joined, cf_alpha);
2664 try {
2665 byte[] row1 = Bytes.toBytes("row1");
2666 byte[] row2 = Bytes.toBytes("row2");
2667 byte[] row3 = Bytes.toBytes("row3");
2668
2669 byte[] col_normal = Bytes.toBytes("d");
2670 byte[] col_alpha = Bytes.toBytes("a");
2671
2672 byte[] filtered_val = Bytes.toBytes(3);
2673
2674 Put put = new Put(row1);
2675 put.add(cf_essential, col_normal, Bytes.toBytes(1));
2676 put.add(cf_joined, col_alpha, Bytes.toBytes(1));
2677 region.put(put);
2678
2679 put = new Put(row2);
2680 put.add(cf_essential, col_alpha, Bytes.toBytes(2));
2681 put.add(cf_joined, col_normal, Bytes.toBytes(2));
2682 put.add(cf_alpha, col_alpha, Bytes.toBytes(2));
2683 region.put(put);
2684
2685 put = new Put(row3);
2686 put.add(cf_essential, col_normal, filtered_val);
2687 put.add(cf_joined, col_normal, filtered_val);
2688 region.put(put);
2689
2690
2691
2692
2693
2694 Scan scan = new Scan();
2695 Filter filter = new SingleColumnValueExcludeFilter(cf_essential, col_normal,
2696 CompareOp.NOT_EQUAL, filtered_val);
2697 scan.setFilter(filter);
2698 scan.setLoadColumnFamiliesOnDemand(true);
2699 InternalScanner s = region.getScanner(scan);
2700
2701 List<Cell> results = new ArrayList<Cell>();
2702 assertTrue(s.next(results));
2703 assertEquals(results.size(), 1);
2704 results.clear();
2705
2706 assertTrue(s.next(results));
2707 assertEquals(results.size(), 3);
2708 assertTrue("orderCheck", CellUtil.matchingFamily(results.get(0), cf_alpha));
2709 assertTrue("orderCheck", CellUtil.matchingFamily(results.get(1), cf_essential));
2710 assertTrue("orderCheck", CellUtil.matchingFamily(results.get(2), cf_joined));
2711 results.clear();
2712
2713 assertFalse(s.next(results));
2714 assertEquals(results.size(), 0);
2715 } finally {
2716 HRegion.closeHRegion(this.region);
2717 this.region = null;
2718 }
2719 }
2720
2721
2722
2723
2724
2725
2726 @Test
2727 public void testScanner_JoinedScannersWithLimits() throws IOException {
2728 final byte[] cf_first = Bytes.toBytes("first");
2729 final byte[] cf_second = Bytes.toBytes("second");
2730
2731 this.region = initHRegion(tableName, getName(), CONF, cf_first, cf_second);
2732 try {
2733 final byte[] col_a = Bytes.toBytes("a");
2734 final byte[] col_b = Bytes.toBytes("b");
2735
2736 Put put;
2737
2738 for (int i = 0; i < 10; i++) {
2739 put = new Put(Bytes.toBytes("r" + Integer.toString(i)));
2740 put.add(cf_first, col_a, Bytes.toBytes(i));
2741 if (i < 5) {
2742 put.add(cf_first, col_b, Bytes.toBytes(i));
2743 put.add(cf_second, col_a, Bytes.toBytes(i));
2744 put.add(cf_second, col_b, Bytes.toBytes(i));
2745 }
2746 region.put(put);
2747 }
2748
2749 Scan scan = new Scan();
2750 scan.setLoadColumnFamiliesOnDemand(true);
2751 Filter bogusFilter = new FilterBase() {
2752 @Override
2753 public boolean isFamilyEssential(byte[] name) {
2754 return Bytes.equals(name, cf_first);
2755 }
2756 };
2757
2758 scan.setFilter(bogusFilter);
2759 InternalScanner s = region.getScanner(scan);
2760
2761
2762
2763
2764
2765
2766
2767
2768
2769
2770
2771
2772
2773
2774
2775
2776
2777
2778
2779
2780
2781
2782
2783
2784
2785
2786
2787
2788
2789
2790 List<Cell> results = new ArrayList<Cell>();
2791 int index = 0;
2792 while (true) {
2793 boolean more = s.next(results, 3);
2794 if ((index >> 1) < 5) {
2795 if (index % 2 == 0)
2796 assertEquals(results.size(), 3);
2797 else
2798 assertEquals(results.size(), 1);
2799 } else
2800 assertEquals(results.size(), 1);
2801 results.clear();
2802 index++;
2803 if (!more)
2804 break;
2805 }
2806 } finally {
2807 HRegion.closeHRegion(this.region);
2808 this.region = null;
2809 }
2810 }
2811
2812
2813
2814
2815
2816
2817
2818
2819
2820 @Test
2821 public void testBasicSplit() throws Exception {
2822 byte[][] families = { fam1, fam2, fam3 };
2823
2824 Configuration hc = initSplit();
2825
2826 String method = this.getName();
2827 this.region = initHRegion(tableName, method, hc, families);
2828
2829 try {
2830 LOG.info("" + HBaseTestCase.addContent(region, fam3));
2831 region.flushcache();
2832 region.compactStores();
2833 byte[] splitRow = region.checkSplit();
2834 assertNotNull(splitRow);
2835 LOG.info("SplitRow: " + Bytes.toString(splitRow));
2836 HRegion[] regions = splitRegion(region, splitRow);
2837 try {
2838
2839
2840
2841 for (int i = 0; i < regions.length; i++) {
2842 regions[i] = HRegion.openHRegion(regions[i], null);
2843 }
2844
2845
2846 assertGet(regions[0], fam3, Bytes.toBytes(START_KEY));
2847 assertGet(regions[1], fam3, splitRow);
2848
2849 assertScan(regions[0], fam3, Bytes.toBytes(START_KEY));
2850 assertScan(regions[1], fam3, splitRow);
2851
2852 for (int i = 0; i < regions.length; i++) {
2853
2854
2855 for (int j = 0; j < 2; j++) {
2856 HBaseTestCase.addContent(regions[i], fam3);
2857 }
2858 HBaseTestCase.addContent(regions[i], fam2);
2859 HBaseTestCase.addContent(regions[i], fam1);
2860 regions[i].flushcache();
2861 }
2862
2863 byte[][] midkeys = new byte[regions.length][];
2864
2865 for (int i = 0; i < regions.length; i++) {
2866 regions[i].compactStores();
2867 midkeys[i] = regions[i].checkSplit();
2868 }
2869
2870 TreeMap<String, HRegion> sortedMap = new TreeMap<String, HRegion>();
2871
2872
2873 for (int i = 0; i < regions.length; i++) {
2874 HRegion[] rs = null;
2875 if (midkeys[i] != null) {
2876 rs = splitRegion(regions[i], midkeys[i]);
2877 for (int j = 0; j < rs.length; j++) {
2878 sortedMap.put(Bytes.toString(rs[j].getRegionName()), HRegion.openHRegion(rs[j], null));
2879 }
2880 }
2881 }
2882 LOG.info("Made 4 regions");
2883
2884
2885 int interval = (LAST_CHAR - FIRST_CHAR) / 3;
2886 byte[] b = Bytes.toBytes(START_KEY);
2887 for (HRegion r : sortedMap.values()) {
2888 assertGet(r, fam3, b);
2889 b[0] += interval;
2890 }
2891 } finally {
2892 for (int i = 0; i < regions.length; i++) {
2893 try {
2894 regions[i].close();
2895 } catch (IOException e) {
2896
2897 }
2898 }
2899 }
2900 } finally {
2901 HRegion.closeHRegion(this.region);
2902 this.region = null;
2903 }
2904 }
2905
2906 @Test
2907 public void testSplitRegion() throws IOException {
2908 byte[] qualifier = Bytes.toBytes("qualifier");
2909 Configuration hc = initSplit();
2910 int numRows = 10;
2911 byte[][] families = { fam1, fam3 };
2912
2913
2914 String method = this.getName();
2915 this.region = initHRegion(tableName, method, hc, families);
2916
2917
2918 int startRow = 100;
2919 putData(startRow, numRows, qualifier, families);
2920 int splitRow = startRow + numRows;
2921 putData(splitRow, numRows, qualifier, families);
2922 region.flushcache();
2923
2924 HRegion[] regions = null;
2925 try {
2926 regions = splitRegion(region, Bytes.toBytes("" + splitRow));
2927
2928 for (int i = 0; i < regions.length; i++) {
2929 regions[i] = HRegion.openHRegion(regions[i], null);
2930 }
2931
2932 assertEquals(2, regions.length);
2933
2934
2935
2936 verifyData(regions[0], startRow, numRows, qualifier, families);
2937 verifyData(regions[1], splitRow, numRows, qualifier, families);
2938
2939 } finally {
2940 HRegion.closeHRegion(this.region);
2941 this.region = null;
2942 }
2943 }
2944
2945
2946
2947
2948
2949
2950
2951
2952
2953
2954
2955 @Test
2956 public void testFlushCacheWhileScanning() throws IOException, InterruptedException {
2957 byte[] family = Bytes.toBytes("family");
2958 int numRows = 1000;
2959 int flushAndScanInterval = 10;
2960 int compactInterval = 10 * flushAndScanInterval;
2961
2962 String method = "testFlushCacheWhileScanning";
2963 this.region = initHRegion(tableName, method, CONF, family);
2964 try {
2965 FlushThread flushThread = new FlushThread();
2966 flushThread.start();
2967
2968 Scan scan = new Scan();
2969 scan.addFamily(family);
2970 scan.setFilter(new SingleColumnValueFilter(family, qual1, CompareOp.EQUAL,
2971 new BinaryComparator(Bytes.toBytes(5L))));
2972
2973 int expectedCount = 0;
2974 List<Cell> res = new ArrayList<Cell>();
2975
2976 boolean toggle = true;
2977 for (long i = 0; i < numRows; i++) {
2978 Put put = new Put(Bytes.toBytes(i));
2979 put.setDurability(Durability.SKIP_WAL);
2980 put.add(family, qual1, Bytes.toBytes(i % 10));
2981 region.put(put);
2982
2983 if (i != 0 && i % compactInterval == 0) {
2984
2985 region.compactStores(true);
2986 }
2987
2988 if (i % 10 == 5L) {
2989 expectedCount++;
2990 }
2991
2992 if (i != 0 && i % flushAndScanInterval == 0) {
2993 res.clear();
2994 InternalScanner scanner = region.getScanner(scan);
2995 if (toggle) {
2996 flushThread.flush();
2997 }
2998 while (scanner.next(res))
2999 ;
3000 if (!toggle) {
3001 flushThread.flush();
3002 }
3003 assertEquals("i=" + i, expectedCount, res.size());
3004 toggle = !toggle;
3005 }
3006 }
3007
3008 flushThread.done();
3009 flushThread.join();
3010 flushThread.checkNoError();
3011 } finally {
3012 HRegion.closeHRegion(this.region);
3013 this.region = null;
3014 }
3015 }
3016
3017 protected class FlushThread extends Thread {
3018 private volatile boolean done;
3019 private Throwable error = null;
3020
3021 public void done() {
3022 done = true;
3023 synchronized (this) {
3024 interrupt();
3025 }
3026 }
3027
3028 public void checkNoError() {
3029 if (error != null) {
3030 assertNull(error);
3031 }
3032 }
3033
3034 @Override
3035 public void run() {
3036 done = false;
3037 while (!done) {
3038 synchronized (this) {
3039 try {
3040 wait();
3041 } catch (InterruptedException ignored) {
3042 if (done) {
3043 break;
3044 }
3045 }
3046 }
3047 try {
3048 region.flushcache();
3049 } catch (IOException e) {
3050 if (!done) {
3051 LOG.error("Error while flusing cache", e);
3052 error = e;
3053 }
3054 break;
3055 }
3056 }
3057
3058 }
3059
3060 public void flush() {
3061 synchronized (this) {
3062 notify();
3063 }
3064
3065 }
3066 }
3067
3068
3069
3070
3071
3072
3073
3074
3075
3076
3077 @Test
3078 public void testWritesWhileScanning() throws IOException, InterruptedException {
3079 int testCount = 100;
3080 int numRows = 1;
3081 int numFamilies = 10;
3082 int numQualifiers = 100;
3083 int flushInterval = 7;
3084 int compactInterval = 5 * flushInterval;
3085 byte[][] families = new byte[numFamilies][];
3086 for (int i = 0; i < numFamilies; i++) {
3087 families[i] = Bytes.toBytes("family" + i);
3088 }
3089 byte[][] qualifiers = new byte[numQualifiers][];
3090 for (int i = 0; i < numQualifiers; i++) {
3091 qualifiers[i] = Bytes.toBytes("qual" + i);
3092 }
3093
3094 String method = "testWritesWhileScanning";
3095 this.region = initHRegion(tableName, method, CONF, families);
3096 try {
3097 PutThread putThread = new PutThread(numRows, families, qualifiers);
3098 putThread.start();
3099 putThread.waitForFirstPut();
3100
3101 FlushThread flushThread = new FlushThread();
3102 flushThread.start();
3103
3104 Scan scan = new Scan(Bytes.toBytes("row0"), Bytes.toBytes("row1"));
3105
3106 int expectedCount = numFamilies * numQualifiers;
3107 List<Cell> res = new ArrayList<Cell>();
3108
3109 long prevTimestamp = 0L;
3110 for (int i = 0; i < testCount; i++) {
3111
3112 if (i != 0 && i % compactInterval == 0) {
3113 region.compactStores(true);
3114 }
3115
3116 if (i != 0 && i % flushInterval == 0) {
3117 flushThread.flush();
3118 }
3119
3120 boolean previousEmpty = res.isEmpty();
3121 res.clear();
3122 InternalScanner scanner = region.getScanner(scan);
3123 while (scanner.next(res))
3124 ;
3125 if (!res.isEmpty() || !previousEmpty || i > compactInterval) {
3126 assertEquals("i=" + i, expectedCount, res.size());
3127 long timestamp = res.get(0).getTimestamp();
3128 assertTrue("Timestamps were broke: " + timestamp + " prev: " + prevTimestamp,
3129 timestamp >= prevTimestamp);
3130 prevTimestamp = timestamp;
3131 }
3132 }
3133
3134 putThread.done();
3135
3136 region.flushcache();
3137
3138 putThread.join();
3139 putThread.checkNoError();
3140
3141 flushThread.done();
3142 flushThread.join();
3143 flushThread.checkNoError();
3144 } finally {
3145 try {
3146 HRegion.closeHRegion(this.region);
3147 } catch (DroppedSnapshotException dse) {
3148
3149
3150
3151
3152 }
3153 this.region = null;
3154 }
3155 }
3156
3157 protected class PutThread extends Thread {
3158 private volatile boolean done;
3159 private volatile int numPutsFinished = 0;
3160
3161 private Throwable error = null;
3162 private int numRows;
3163 private byte[][] families;
3164 private byte[][] qualifiers;
3165
3166 private PutThread(int numRows, byte[][] families, byte[][] qualifiers) {
3167 this.numRows = numRows;
3168 this.families = families;
3169 this.qualifiers = qualifiers;
3170 }
3171
3172
3173
3174
3175 public void waitForFirstPut() throws InterruptedException {
3176
3177 while (numPutsFinished == 0) {
3178 checkNoError();
3179 Thread.sleep(50);
3180 }
3181 }
3182
3183 public void done() {
3184 done = true;
3185 synchronized (this) {
3186 interrupt();
3187 }
3188 }
3189
3190 public void checkNoError() {
3191 if (error != null) {
3192 assertNull(error);
3193 }
3194 }
3195
3196 @Override
3197 public void run() {
3198 done = false;
3199 while (!done) {
3200 try {
3201 for (int r = 0; r < numRows; r++) {
3202 byte[] row = Bytes.toBytes("row" + r);
3203 Put put = new Put(row);
3204 put.setDurability(Durability.SKIP_WAL);
3205 byte[] value = Bytes.toBytes(String.valueOf(numPutsFinished));
3206 for (byte[] family : families) {
3207 for (byte[] qualifier : qualifiers) {
3208 put.add(family, qualifier, (long) numPutsFinished, value);
3209 }
3210 }
3211 region.put(put);
3212 numPutsFinished++;
3213 if (numPutsFinished > 0 && numPutsFinished % 47 == 0) {
3214 System.out.println("put iteration = " + numPutsFinished);
3215 Delete delete = new Delete(row, (long) numPutsFinished - 30);
3216 region.delete(delete);
3217 }
3218 numPutsFinished++;
3219 }
3220 } catch (InterruptedIOException e) {
3221
3222 } catch (IOException e) {
3223 LOG.error("error while putting records", e);
3224 error = e;
3225 break;
3226 }
3227 }
3228
3229 }
3230
3231 }
3232
3233
3234
3235
3236
3237
3238
3239
3240
3241
3242 @Test
3243 public void testWritesWhileGetting() throws Exception {
3244 int testCount = 100;
3245 int numRows = 1;
3246 int numFamilies = 10;
3247 int numQualifiers = 100;
3248 int compactInterval = 100;
3249 byte[][] families = new byte[numFamilies][];
3250 for (int i = 0; i < numFamilies; i++) {
3251 families[i] = Bytes.toBytes("family" + i);
3252 }
3253 byte[][] qualifiers = new byte[numQualifiers][];
3254 for (int i = 0; i < numQualifiers; i++) {
3255 qualifiers[i] = Bytes.toBytes("qual" + i);
3256 }
3257
3258
3259 String method = "testWritesWhileGetting";
3260
3261
3262
3263
3264
3265 Configuration conf = HBaseConfiguration.create(CONF);
3266 conf.setInt("hbase.hstore.compaction.min", 1);
3267 conf.setInt("hbase.hstore.compaction.max", 1000);
3268 this.region = initHRegion(tableName, method, conf, families);
3269 PutThread putThread = null;
3270 MultithreadedTestUtil.TestContext ctx = new MultithreadedTestUtil.TestContext(conf);
3271 try {
3272 putThread = new PutThread(numRows, families, qualifiers);
3273 putThread.start();
3274 putThread.waitForFirstPut();
3275
3276
3277 ctx.addThread(new RepeatingTestThread(ctx) {
3278 private int flushesSinceCompact = 0;
3279 private final int maxFlushesSinceCompact = 20;
3280
3281 @Override
3282 public void doAnAction() throws Exception {
3283 if (region.flushcache()) {
3284 ++flushesSinceCompact;
3285 }
3286
3287
3288 if (flushesSinceCompact == maxFlushesSinceCompact) {
3289 region.compactStores(false);
3290 flushesSinceCompact = 0;
3291 }
3292 }
3293 });
3294 ctx.startThreads();
3295
3296 Get get = new Get(Bytes.toBytes("row0"));
3297 Result result = null;
3298
3299 int expectedCount = numFamilies * numQualifiers;
3300
3301 long prevTimestamp = 0L;
3302 for (int i = 0; i < testCount; i++) {
3303
3304 boolean previousEmpty = result == null || result.isEmpty();
3305 result = region.get(get);
3306 if (!result.isEmpty() || !previousEmpty || i > compactInterval) {
3307 assertEquals("i=" + i, expectedCount, result.size());
3308
3309
3310 long timestamp = 0;
3311 for (Cell kv : result.rawCells()) {
3312 if (CellUtil.matchingFamily(kv, families[0])
3313 && CellUtil.matchingQualifier(kv, qualifiers[0])) {
3314 timestamp = kv.getTimestamp();
3315 }
3316 }
3317 assertTrue(timestamp >= prevTimestamp);
3318 prevTimestamp = timestamp;
3319 Cell previousKV = null;
3320
3321 for (Cell kv : result.rawCells()) {
3322 byte[] thisValue = CellUtil.cloneValue(kv);
3323 if (previousKV != null) {
3324 if (Bytes.compareTo(CellUtil.cloneValue(previousKV), thisValue) != 0) {
3325 LOG.warn("These two KV should have the same value." + " Previous KV:" + previousKV
3326 + "(memStoreTS:" + previousKV.getMvccVersion() + ")" + ", New KV: " + kv
3327 + "(memStoreTS:" + kv.getMvccVersion() + ")");
3328 assertEquals(0, Bytes.compareTo(CellUtil.cloneValue(previousKV), thisValue));
3329 }
3330 }
3331 previousKV = kv;
3332 }
3333 }
3334 }
3335 } finally {
3336 if (putThread != null)
3337 putThread.done();
3338
3339 region.flushcache();
3340
3341 if (putThread != null) {
3342 putThread.join();
3343 putThread.checkNoError();
3344 }
3345
3346 ctx.stop();
3347 HRegion.closeHRegion(this.region);
3348 this.region = null;
3349 }
3350 }
3351
3352 @Test
3353 public void testHolesInMeta() throws Exception {
3354 byte[] family = Bytes.toBytes("family");
3355 this.region = initHRegion(tableName, Bytes.toBytes("x"), Bytes.toBytes("z"), method, CONF,
3356 false, family);
3357 try {
3358 byte[] rowNotServed = Bytes.toBytes("a");
3359 Get g = new Get(rowNotServed);
3360 try {
3361 region.get(g);
3362 fail();
3363 } catch (WrongRegionException x) {
3364
3365 }
3366 byte[] row = Bytes.toBytes("y");
3367 g = new Get(row);
3368 region.get(g);
3369 } finally {
3370 HRegion.closeHRegion(this.region);
3371 this.region = null;
3372 }
3373 }
3374
3375 @Test
3376 public void testIndexesScanWithOneDeletedRow() throws IOException {
3377 byte[] family = Bytes.toBytes("family");
3378
3379
3380 String method = "testIndexesScanWithOneDeletedRow";
3381 this.region = initHRegion(tableName, method, CONF, family);
3382 try {
3383 Put put = new Put(Bytes.toBytes(1L));
3384 put.add(family, qual1, 1L, Bytes.toBytes(1L));
3385 region.put(put);
3386
3387 region.flushcache();
3388
3389 Delete delete = new Delete(Bytes.toBytes(1L), 1L);
3390 region.delete(delete);
3391
3392 put = new Put(Bytes.toBytes(2L));
3393 put.add(family, qual1, 2L, Bytes.toBytes(2L));
3394 region.put(put);
3395
3396 Scan idxScan = new Scan();
3397 idxScan.addFamily(family);
3398 idxScan.setFilter(new FilterList(FilterList.Operator.MUST_PASS_ALL, Arrays.<Filter> asList(
3399 new SingleColumnValueFilter(family, qual1, CompareOp.GREATER_OR_EQUAL,
3400 new BinaryComparator(Bytes.toBytes(0L))), new SingleColumnValueFilter(family, qual1,
3401 CompareOp.LESS_OR_EQUAL, new BinaryComparator(Bytes.toBytes(3L))))));
3402 InternalScanner scanner = region.getScanner(idxScan);
3403 List<Cell> res = new ArrayList<Cell>();
3404
3405 while (scanner.next(res))
3406 ;
3407 assertEquals(1L, res.size());
3408 } finally {
3409 HRegion.closeHRegion(this.region);
3410 this.region = null;
3411 }
3412 }
3413
3414
3415
3416
3417 @Test
3418 public void testBloomFilterSize() throws IOException {
3419 byte[] fam1 = Bytes.toBytes("fam1");
3420 byte[] qf1 = Bytes.toBytes("col");
3421 byte[] val1 = Bytes.toBytes("value1");
3422
3423 HColumnDescriptor hcd = new HColumnDescriptor(fam1).setMaxVersions(Integer.MAX_VALUE)
3424 .setBloomFilterType(BloomType.ROWCOL);
3425
3426 HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(tableName));
3427 htd.addFamily(hcd);
3428 HRegionInfo info = new HRegionInfo(htd.getTableName(), null, null, false);
3429 this.region = TEST_UTIL.createLocalHRegion(info, htd);
3430 try {
3431 int num_unique_rows = 10;
3432 int duplicate_multiplier = 2;
3433 int num_storefiles = 4;
3434
3435 int version = 0;
3436 for (int f = 0; f < num_storefiles; f++) {
3437 for (int i = 0; i < duplicate_multiplier; i++) {
3438 for (int j = 0; j < num_unique_rows; j++) {
3439 Put put = new Put(Bytes.toBytes("row" + j));
3440 put.setDurability(Durability.SKIP_WAL);
3441 put.add(fam1, qf1, version++, val1);
3442 region.put(put);
3443 }
3444 }
3445 region.flushcache();
3446 }
3447
3448 HStore store = (HStore) region.getStore(fam1);
3449 Collection<StoreFile> storeFiles = store.getStorefiles();
3450 for (StoreFile storefile : storeFiles) {
3451 StoreFile.Reader reader = storefile.getReader();
3452 reader.loadFileInfo();
3453 reader.loadBloomfilter();
3454 assertEquals(num_unique_rows * duplicate_multiplier, reader.getEntries());
3455 assertEquals(num_unique_rows, reader.getFilterEntries());
3456 }
3457
3458 region.compactStores(true);
3459
3460
3461 storeFiles = store.getStorefiles();
3462 for (StoreFile storefile : storeFiles) {
3463 StoreFile.Reader reader = storefile.getReader();
3464 reader.loadFileInfo();
3465 reader.loadBloomfilter();
3466 assertEquals(num_unique_rows * duplicate_multiplier * num_storefiles, reader.getEntries());
3467 assertEquals(num_unique_rows, reader.getFilterEntries());
3468 }
3469 } finally {
3470 HRegion.closeHRegion(this.region);
3471 this.region = null;
3472 }
3473 }
3474
3475 @Test
3476 public void testAllColumnsWithBloomFilter() throws IOException {
3477 byte[] TABLE = Bytes.toBytes("testAllColumnsWithBloomFilter");
3478 byte[] FAMILY = Bytes.toBytes("family");
3479
3480
3481 HColumnDescriptor hcd = new HColumnDescriptor(FAMILY).setMaxVersions(Integer.MAX_VALUE)
3482 .setBloomFilterType(BloomType.ROWCOL);
3483 HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(TABLE));
3484 htd.addFamily(hcd);
3485 HRegionInfo info = new HRegionInfo(htd.getTableName(), null, null, false);
3486 this.region = TEST_UTIL.createLocalHRegion(info, htd);
3487 try {
3488
3489 byte row[] = Bytes.toBytes("row:" + 0);
3490 byte column[] = Bytes.toBytes("column:" + 0);
3491 Put put = new Put(row);
3492 put.setDurability(Durability.SKIP_WAL);
3493 for (long idx = 1; idx <= 4; idx++) {
3494 put.add(FAMILY, column, idx, Bytes.toBytes("value-version-" + idx));
3495 }
3496 region.put(put);
3497
3498
3499 region.flushcache();
3500
3501
3502 Get get = new Get(row);
3503 get.setMaxVersions();
3504 Cell[] kvs = region.get(get).rawCells();
3505
3506
3507 assertEquals(4, kvs.length);
3508 checkOneCell(kvs[0], FAMILY, 0, 0, 4);
3509 checkOneCell(kvs[1], FAMILY, 0, 0, 3);
3510 checkOneCell(kvs[2], FAMILY, 0, 0, 2);
3511 checkOneCell(kvs[3], FAMILY, 0, 0, 1);
3512 } finally {
3513 HRegion.closeHRegion(this.region);
3514 this.region = null;
3515 }
3516 }
3517
3518
3519
3520
3521
3522
3523 @Test
3524 public void testDeleteRowWithBloomFilter() throws IOException {
3525 byte[] familyName = Bytes.toBytes("familyName");
3526
3527
3528 HColumnDescriptor hcd = new HColumnDescriptor(familyName).setMaxVersions(Integer.MAX_VALUE)
3529 .setBloomFilterType(BloomType.ROWCOL);
3530
3531 HTableDescriptor htd = new HTableDescriptor(TableName.valueOf(tableName));
3532 htd.addFamily(hcd);
3533 HRegionInfo info = new HRegionInfo(htd.getTableName(), null, null, false);
3534 this.region = TEST_UTIL.createLocalHRegion(info, htd);
3535 try {
3536
3537 byte row[] = Bytes.toBytes("row1");
3538 byte col[] = Bytes.toBytes("col1");
3539
3540 Put put = new Put(row);
3541 put.add(familyName, col, 1, Bytes.toBytes("SomeRandomValue"));
3542 region.put(put);
3543 region.flushcache();
3544
3545 Delete del = new Delete(row);
3546 region.delete(del);
3547 region.flushcache();
3548
3549
3550 Get get = new Get(row);
3551 get.addColumn(familyName, col);
3552
3553 Cell[] keyValues = region.get(get).rawCells();
3554 assertTrue(keyValues.length == 0);
3555 } finally {
3556 HRegion.closeHRegion(this.region);
3557 this.region = null;
3558 }
3559 }
3560
3561 @Test
3562 public void testgetHDFSBlocksDistribution() throws Exception {
3563 HBaseTestingUtility htu = new HBaseTestingUtility();
3564 final int DEFAULT_BLOCK_SIZE = 1024;
3565 htu.getConfiguration().setLong("dfs.block.size", DEFAULT_BLOCK_SIZE);
3566 htu.getConfiguration().setInt("dfs.replication", 2);
3567
3568
3569 MiniHBaseCluster cluster = null;
3570 String dataNodeHosts[] = new String[] { "host1", "host2", "host3" };
3571 int regionServersCount = 3;
3572
3573 try {
3574 cluster = htu.startMiniCluster(1, regionServersCount, dataNodeHosts);
3575 byte[][] families = { fam1, fam2 };
3576 HTable ht = htu.createTable(Bytes.toBytes(this.getName()), families);
3577
3578
3579 byte row[] = Bytes.toBytes("row1");
3580 byte col[] = Bytes.toBytes("col1");
3581
3582 Put put = new Put(row);
3583 put.add(fam1, col, 1, Bytes.toBytes("test1"));
3584 put.add(fam2, col, 1, Bytes.toBytes("test2"));
3585 ht.put(put);
3586
3587 HRegion firstRegion = htu.getHBaseCluster().getRegions(TableName.valueOf(this.getName()))
3588 .get(0);
3589 firstRegion.flushcache();
3590 HDFSBlocksDistribution blocksDistribution1 = firstRegion.getHDFSBlocksDistribution();
3591
3592
3593
3594
3595
3596 long uniqueBlocksWeight1 = blocksDistribution1.getUniqueBlocksTotalWeight();
3597
3598 String topHost = blocksDistribution1.getTopHosts().get(0);
3599 long topHostWeight = blocksDistribution1.getWeight(topHost);
3600 assertTrue(uniqueBlocksWeight1 == topHostWeight);
3601
3602
3603
3604 HDFSBlocksDistribution blocksDistribution2 = HRegion.computeHDFSBlocksDistribution(
3605 htu.getConfiguration(), firstRegion.getTableDesc(), firstRegion.getRegionInfo());
3606 long uniqueBlocksWeight2 = blocksDistribution2.getUniqueBlocksTotalWeight();
3607
3608 assertTrue(uniqueBlocksWeight1 == uniqueBlocksWeight2);
3609
3610 ht.close();
3611 } finally {
3612 if (cluster != null) {
3613 htu.shutdownMiniCluster();
3614 }
3615 }
3616 }
3617
3618
3619
3620
3621
3622
3623
3624 @Test
3625 public void testStatusSettingToAbortIfAnyExceptionDuringRegionInitilization() throws Exception {
3626 TableName tableName = TableName.valueOf(name.getMethodName());
3627 HRegionInfo info = null;
3628 try {
3629 FileSystem fs = Mockito.mock(FileSystem.class);
3630 Mockito.when(fs.exists((Path) Mockito.anyObject())).thenThrow(new IOException());
3631 HTableDescriptor htd = new HTableDescriptor(tableName);
3632 htd.addFamily(new HColumnDescriptor("cf"));
3633 info = new HRegionInfo(htd.getTableName(), HConstants.EMPTY_BYTE_ARRAY,
3634 HConstants.EMPTY_BYTE_ARRAY, false);
3635 Path path = new Path(dir + "testStatusSettingToAbortIfAnyExceptionDuringRegionInitilization");
3636 region = HRegion.newHRegion(path, null, fs, CONF, info, htd, null);
3637
3638 region.initialize();
3639 fail("Region initialization should fail due to IOException");
3640 } catch (IOException io) {
3641 List<MonitoredTask> tasks = TaskMonitor.get().getTasks();
3642 for (MonitoredTask monitoredTask : tasks) {
3643 if (!(monitoredTask instanceof MonitoredRPCHandler)
3644 && monitoredTask.getDescription().contains(region.toString())) {
3645 assertTrue("Region state should be ABORTED.",
3646 monitoredTask.getState().equals(MonitoredTask.State.ABORTED));
3647 break;
3648 }
3649 }
3650 } finally {
3651 HRegion.closeHRegion(region);
3652 }
3653 }
3654
3655
3656
3657
3658
3659 @Test
3660 public void testRegionInfoFileCreation() throws IOException {
3661 Path rootDir = new Path(dir + "testRegionInfoFileCreation");
3662
3663 HTableDescriptor htd = new HTableDescriptor(TableName.valueOf("testtb"));
3664 htd.addFamily(new HColumnDescriptor("cf"));
3665
3666 HRegionInfo hri = new HRegionInfo(htd.getTableName());
3667
3668
3669 HRegion region = HRegion.createHRegion(hri, rootDir, CONF, htd, null, false, true);
3670
3671 Path regionDir = region.getRegionFileSystem().getRegionDir();
3672 FileSystem fs = region.getRegionFileSystem().getFileSystem();
3673 HRegion.closeHRegion(region);
3674
3675 Path regionInfoFile = new Path(regionDir, HRegionFileSystem.REGION_INFO_FILE);
3676
3677
3678 assertTrue(HRegionFileSystem.REGION_INFO_FILE + " should be present in the region dir",
3679 fs.exists(regionInfoFile));
3680
3681
3682 region = HRegion.openHRegion(rootDir, hri, htd, null, CONF);
3683 assertEquals(regionDir, region.getRegionFileSystem().getRegionDir());
3684 HRegion.closeHRegion(region);
3685
3686
3687 assertTrue(HRegionFileSystem.REGION_INFO_FILE + " should be present in the region dir",
3688 fs.exists(regionInfoFile));
3689
3690
3691 fs.delete(regionInfoFile);
3692 assertFalse(HRegionFileSystem.REGION_INFO_FILE + " should be removed from the region dir",
3693 fs.exists(regionInfoFile));
3694
3695 region = HRegion.openHRegion(rootDir, hri, htd, null, CONF);
3696
3697 assertEquals(regionDir, region.getRegionFileSystem().getRegionDir());
3698 HRegion.closeHRegion(region);
3699
3700
3701 assertTrue(HRegionFileSystem.REGION_INFO_FILE + " should be present in the region dir",
3702 fs.exists(new Path(regionDir, HRegionFileSystem.REGION_INFO_FILE)));
3703 }
3704
3705
3706
3707
3708 private static class Incrementer implements Runnable {
3709 private HRegion region;
3710 private final static byte[] incRow = Bytes.toBytes("incRow");
3711 private final static byte[] family = Bytes.toBytes("family");
3712 private final static byte[] qualifier = Bytes.toBytes("qualifier");
3713 private final static long ONE = 1l;
3714 private int incCounter;
3715
3716 public Incrementer(HRegion region, int incCounter) {
3717 this.region = region;
3718 this.incCounter = incCounter;
3719 }
3720
3721 @Override
3722 public void run() {
3723 int count = 0;
3724 while (count < incCounter) {
3725 Increment inc = new Increment(incRow);
3726 inc.addColumn(family, qualifier, ONE);
3727 count++;
3728 try {
3729 region.increment(inc);
3730 } catch (IOException e) {
3731 e.printStackTrace();
3732 break;
3733 }
3734 }
3735 }
3736 }
3737
3738
3739
3740
3741
3742 @Test
3743 public void testParallelIncrementWithMemStoreFlush() throws Exception {
3744 byte[] family = Incrementer.family;
3745 this.region = initHRegion(tableName, method, CONF, family);
3746 final HRegion region = this.region;
3747 final AtomicBoolean incrementDone = new AtomicBoolean(false);
3748 Runnable flusher = new Runnable() {
3749 @Override
3750 public void run() {
3751 while (!incrementDone.get()) {
3752 try {
3753 region.flushcache();
3754 } catch (Exception e) {
3755 e.printStackTrace();
3756 }
3757 }
3758 }
3759 };
3760
3761
3762 int threadNum = 20;
3763 int incCounter = 100;
3764 long expected = threadNum * incCounter;
3765 Thread[] incrementers = new Thread[threadNum];
3766 Thread flushThread = new Thread(flusher);
3767 for (int i = 0; i < threadNum; i++) {
3768 incrementers[i] = new Thread(new Incrementer(this.region, incCounter));
3769 incrementers[i].start();
3770 }
3771 flushThread.start();
3772 for (int i = 0; i < threadNum; i++) {
3773 incrementers[i].join();
3774 }
3775
3776 incrementDone.set(true);
3777 flushThread.join();
3778
3779 Get get = new Get(Incrementer.incRow);
3780 get.addColumn(Incrementer.family, Incrementer.qualifier);
3781 get.setMaxVersions(1);
3782 Result res = this.region.get(get);
3783 List<Cell> kvs = res.getColumnCells(Incrementer.family, Incrementer.qualifier);
3784
3785
3786 assertEquals(kvs.size(), 1);
3787 Cell kv = kvs.get(0);
3788 assertEquals(expected, Bytes.toLong(kv.getValueArray(), kv.getValueOffset()));
3789 this.region = null;
3790 }
3791
3792
3793
3794
3795 private static class Appender implements Runnable {
3796 private HRegion region;
3797 private final static byte[] appendRow = Bytes.toBytes("appendRow");
3798 private final static byte[] family = Bytes.toBytes("family");
3799 private final static byte[] qualifier = Bytes.toBytes("qualifier");
3800 private final static byte[] CHAR = Bytes.toBytes("a");
3801 private int appendCounter;
3802
3803 public Appender(HRegion region, int appendCounter) {
3804 this.region = region;
3805 this.appendCounter = appendCounter;
3806 }
3807
3808 @Override
3809 public void run() {
3810 int count = 0;
3811 while (count < appendCounter) {
3812 Append app = new Append(appendRow);
3813 app.add(family, qualifier, CHAR);
3814 count++;
3815 try {
3816 region.append(app);
3817 } catch (IOException e) {
3818 e.printStackTrace();
3819 break;
3820 }
3821 }
3822 }
3823 }
3824
3825
3826
3827
3828
3829 @Test
3830 public void testParallelAppendWithMemStoreFlush() throws Exception {
3831 byte[] family = Appender.family;
3832 this.region = initHRegion(tableName, method, CONF, family);
3833 final HRegion region = this.region;
3834 final AtomicBoolean appendDone = new AtomicBoolean(false);
3835 Runnable flusher = new Runnable() {
3836 @Override
3837 public void run() {
3838 while (!appendDone.get()) {
3839 try {
3840 region.flushcache();
3841 } catch (Exception e) {
3842 e.printStackTrace();
3843 }
3844 }
3845 }
3846 };
3847
3848
3849
3850 int threadNum = 20;
3851 int appendCounter = 100;
3852 byte[] expected = new byte[threadNum * appendCounter];
3853 for (int i = 0; i < threadNum * appendCounter; i++) {
3854 System.arraycopy(Appender.CHAR, 0, expected, i, 1);
3855 }
3856 Thread[] appenders = new Thread[threadNum];
3857 Thread flushThread = new Thread(flusher);
3858 for (int i = 0; i < threadNum; i++) {
3859 appenders[i] = new Thread(new Appender(this.region, appendCounter));
3860 appenders[i].start();
3861 }
3862 flushThread.start();
3863 for (int i = 0; i < threadNum; i++) {
3864 appenders[i].join();
3865 }
3866
3867 appendDone.set(true);
3868 flushThread.join();
3869
3870 Get get = new Get(Appender.appendRow);
3871 get.addColumn(Appender.family, Appender.qualifier);
3872 get.setMaxVersions(1);
3873 Result res = this.region.get(get);
3874 List<Cell> kvs = res.getColumnCells(Appender.family, Appender.qualifier);
3875
3876
3877 assertEquals(kvs.size(), 1);
3878 Cell kv = kvs.get(0);
3879 byte[] appendResult = new byte[kv.getValueLength()];
3880 System.arraycopy(kv.getValueArray(), kv.getValueOffset(), appendResult, 0, kv.getValueLength());
3881 assertArrayEquals(expected, appendResult);
3882 this.region = null;
3883 }
3884
3885
3886
3887
3888
3889 @Test
3890 public void testPutWithMemStoreFlush() throws Exception {
3891 byte[] family = Bytes.toBytes("family");
3892 ;
3893 byte[] qualifier = Bytes.toBytes("qualifier");
3894 byte[] row = Bytes.toBytes("putRow");
3895 byte[] value = null;
3896 this.region = initHRegion(tableName, method, CONF, family);
3897 Put put = null;
3898 Get get = null;
3899 List<Cell> kvs = null;
3900 Result res = null;
3901
3902 put = new Put(row);
3903 value = Bytes.toBytes("value0");
3904 put.add(family, qualifier, 1234567l, value);
3905 region.put(put);
3906 get = new Get(row);
3907 get.addColumn(family, qualifier);
3908 get.setMaxVersions();
3909 res = this.region.get(get);
3910 kvs = res.getColumnCells(family, qualifier);
3911 assertEquals(1, kvs.size());
3912 assertArrayEquals(Bytes.toBytes("value0"), CellUtil.cloneValue(kvs.get(0)));
3913
3914 region.flushcache();
3915 get = new Get(row);
3916 get.addColumn(family, qualifier);
3917 get.setMaxVersions();
3918 res = this.region.get(get);
3919 kvs = res.getColumnCells(family, qualifier);
3920 assertEquals(1, kvs.size());
3921 assertArrayEquals(Bytes.toBytes("value0"), CellUtil.cloneValue(kvs.get(0)));
3922
3923 put = new Put(row);
3924 value = Bytes.toBytes("value1");
3925 put.add(family, qualifier, 1234567l, value);
3926 region.put(put);
3927 get = new Get(row);
3928 get.addColumn(family, qualifier);
3929 get.setMaxVersions();
3930 res = this.region.get(get);
3931 kvs = res.getColumnCells(family, qualifier);
3932 assertEquals(1, kvs.size());
3933 assertArrayEquals(Bytes.toBytes("value1"), CellUtil.cloneValue(kvs.get(0)));
3934
3935 region.flushcache();
3936 get = new Get(row);
3937 get.addColumn(family, qualifier);
3938 get.setMaxVersions();
3939 res = this.region.get(get);
3940 kvs = res.getColumnCells(family, qualifier);
3941 assertEquals(1, kvs.size());
3942 assertArrayEquals(Bytes.toBytes("value1"), CellUtil.cloneValue(kvs.get(0)));
3943 }
3944
3945 @Test
3946 public void testDurability() throws Exception {
3947 String method = "testDurability";
3948
3949
3950
3951
3952
3953 durabilityTest(method, Durability.SYNC_WAL, Durability.SYNC_WAL, 0, true, true, false);
3954 durabilityTest(method, Durability.SYNC_WAL, Durability.FSYNC_WAL, 0, true, true, false);
3955 durabilityTest(method, Durability.SYNC_WAL, Durability.USE_DEFAULT, 0, true, true, false);
3956
3957 durabilityTest(method, Durability.FSYNC_WAL, Durability.SYNC_WAL, 0, true, true, false);
3958 durabilityTest(method, Durability.FSYNC_WAL, Durability.FSYNC_WAL, 0, true, true, false);
3959 durabilityTest(method, Durability.FSYNC_WAL, Durability.USE_DEFAULT, 0, true, true, false);
3960
3961 durabilityTest(method, Durability.ASYNC_WAL, Durability.SYNC_WAL, 0, true, true, false);
3962 durabilityTest(method, Durability.ASYNC_WAL, Durability.FSYNC_WAL, 0, true, true, false);
3963
3964 durabilityTest(method, Durability.SKIP_WAL, Durability.SYNC_WAL, 0, true, true, false);
3965 durabilityTest(method, Durability.SKIP_WAL, Durability.FSYNC_WAL, 0, true, true, false);
3966
3967 durabilityTest(method, Durability.USE_DEFAULT, Durability.SYNC_WAL, 0, true, true, false);
3968 durabilityTest(method, Durability.USE_DEFAULT, Durability.FSYNC_WAL, 0, true, true, false);
3969 durabilityTest(method, Durability.USE_DEFAULT, Durability.USE_DEFAULT, 0, true, true, false);
3970
3971
3972
3973 CONF.setLong("hbase.regionserver.optionallogflushinterval", Integer.MAX_VALUE);
3974 durabilityTest(method, Durability.SYNC_WAL, Durability.ASYNC_WAL, 0, true, false, false);
3975 durabilityTest(method, Durability.FSYNC_WAL, Durability.ASYNC_WAL, 0, true, false, false);
3976 durabilityTest(method, Durability.ASYNC_WAL, Durability.ASYNC_WAL, 0, true, false, false);
3977 durabilityTest(method, Durability.SKIP_WAL, Durability.ASYNC_WAL, 0, true, false, false);
3978 durabilityTest(method, Durability.USE_DEFAULT, Durability.ASYNC_WAL, 0, true, false, false);
3979 durabilityTest(method, Durability.ASYNC_WAL, Durability.USE_DEFAULT, 0, true, false, false);
3980
3981
3982 CONF.setLong("hbase.regionserver.optionallogflushinterval", 5);
3983 durabilityTest(method, Durability.SYNC_WAL, Durability.ASYNC_WAL, 5000, true, false, true);
3984 durabilityTest(method, Durability.FSYNC_WAL, Durability.ASYNC_WAL, 5000, true, false, true);
3985 durabilityTest(method, Durability.ASYNC_WAL, Durability.ASYNC_WAL, 5000, true, false, true);
3986 durabilityTest(method, Durability.SKIP_WAL, Durability.ASYNC_WAL, 5000, true, false, true);
3987 durabilityTest(method, Durability.USE_DEFAULT, Durability.ASYNC_WAL, 5000, true, false, true);
3988 durabilityTest(method, Durability.ASYNC_WAL, Durability.USE_DEFAULT, 5000, true, false, true);
3989
3990
3991 durabilityTest(method, Durability.SYNC_WAL, Durability.SKIP_WAL, 0, false, false, false);
3992 durabilityTest(method, Durability.FSYNC_WAL, Durability.SKIP_WAL, 0, false, false, false);
3993 durabilityTest(method, Durability.ASYNC_WAL, Durability.SKIP_WAL, 0, false, false, false);
3994 durabilityTest(method, Durability.SKIP_WAL, Durability.SKIP_WAL, 0, false, false, false);
3995 durabilityTest(method, Durability.USE_DEFAULT, Durability.SKIP_WAL, 0, false, false, false);
3996 durabilityTest(method, Durability.SKIP_WAL, Durability.USE_DEFAULT, 0, false, false, false);
3997
3998 }
3999
4000 private void durabilityTest(String method, Durability tableDurability,
4001 Durability mutationDurability, long timeout, boolean expectAppend, final boolean expectSync,
4002 final boolean expectSyncFromLogSyncer) throws Exception {
4003 Configuration conf = HBaseConfiguration.create(CONF);
4004 method = method + "_" + tableDurability.name() + "_" + mutationDurability.name();
4005 TableName tableName = TableName.valueOf(method);
4006 byte[] family = Bytes.toBytes("family");
4007 Path logDir = new Path(new Path(dir + method), "log");
4008 HLog hlog = HLogFactory.createHLog(FILESYSTEM, logDir, UUID.randomUUID().toString(), conf);
4009 final HLog log = spy(hlog);
4010 this.region = initHRegion(tableName.getName(), HConstants.EMPTY_START_ROW,
4011 HConstants.EMPTY_END_ROW, method, conf, false, tableDurability, log,
4012 new byte[][] { family });
4013
4014 Put put = new Put(Bytes.toBytes("r1"));
4015 put.add(family, Bytes.toBytes("q1"), Bytes.toBytes("v1"));
4016 put.setDurability(mutationDurability);
4017 region.put(put);
4018
4019
4020 verify(log, expectAppend ? times(1) : never()).appendNoSync((HRegionInfo) any(), eq(tableName),
4021 (WALEdit) any(), (List<UUID>) any(), anyLong(), (HTableDescriptor) any());
4022
4023
4024 if (expectSync || expectSyncFromLogSyncer) {
4025 TEST_UTIL.waitFor(timeout, new Waiter.Predicate<Exception>() {
4026 @Override
4027 public boolean evaluate() throws Exception {
4028 try {
4029 if (expectSync) {
4030 verify(log, times(1)).sync(anyLong());
4031 } else if (expectSyncFromLogSyncer) {
4032 verify(log, times(1)).sync();
4033 }
4034 } catch (Throwable ignore) {
4035 }
4036 return true;
4037 }
4038 });
4039 } else {
4040 verify(log, never()).sync(anyLong());
4041 verify(log, never()).sync();
4042 }
4043
4044 HRegion.closeHRegion(this.region);
4045 this.region = null;
4046 }
4047
4048 private void putData(int startRow, int numRows, byte[] qf, byte[]... families) throws IOException {
4049 for (int i = startRow; i < startRow + numRows; i++) {
4050 Put put = new Put(Bytes.toBytes("" + i));
4051 put.setDurability(Durability.SKIP_WAL);
4052 for (byte[] family : families) {
4053 put.add(family, qf, null);
4054 }
4055 region.put(put);
4056 }
4057 }
4058
4059 private void verifyData(HRegion newReg, int startRow, int numRows, byte[] qf, byte[]... families)
4060 throws IOException {
4061 for (int i = startRow; i < startRow + numRows; i++) {
4062 byte[] row = Bytes.toBytes("" + i);
4063 Get get = new Get(row);
4064 for (byte[] family : families) {
4065 get.addColumn(family, qf);
4066 }
4067 Result result = newReg.get(get);
4068 Cell[] raw = result.rawCells();
4069 assertEquals(families.length, result.size());
4070 for (int j = 0; j < families.length; j++) {
4071 assertTrue(CellUtil.matchingRow(raw[j], row));
4072 assertTrue(CellUtil.matchingFamily(raw[j], families[j]));
4073 assertTrue(CellUtil.matchingQualifier(raw[j], qf));
4074 }
4075 }
4076 }
4077
4078 private void assertGet(final HRegion r, final byte[] family, final byte[] k) throws IOException {
4079
4080 Get get = new Get(k).addFamily(family).setMaxVersions();
4081 Cell[] results = r.get(get).rawCells();
4082 for (int j = 0; j < results.length; j++) {
4083 byte[] tmp = CellUtil.cloneValue(results[j]);
4084
4085 assertTrue(Bytes.equals(k, tmp));
4086 }
4087 }
4088
4089
4090
4091
4092
4093
4094
4095
4096
4097
4098
4099
4100 private void assertScan(final HRegion r, final byte[] fs, final byte[] firstValue)
4101 throws IOException {
4102 byte[][] families = { fs };
4103 Scan scan = new Scan();
4104 for (int i = 0; i < families.length; i++)
4105 scan.addFamily(families[i]);
4106 InternalScanner s = r.getScanner(scan);
4107 try {
4108 List<Cell> curVals = new ArrayList<Cell>();
4109 boolean first = true;
4110 OUTER_LOOP: while (s.next(curVals)) {
4111 for (Cell kv : curVals) {
4112 byte[] val = CellUtil.cloneValue(kv);
4113 byte[] curval = val;
4114 if (first) {
4115 first = false;
4116 assertTrue(Bytes.compareTo(curval, firstValue) == 0);
4117 } else {
4118
4119 break OUTER_LOOP;
4120 }
4121 }
4122 }
4123 } finally {
4124 s.close();
4125 }
4126 }
4127
4128 private Configuration initSplit() {
4129
4130 CONF.setInt("hbase.hstore.compactionThreshold", 2);
4131
4132
4133 CONF.setInt("hbase.master.lease.thread.wakefrequency", 5 * 1000);
4134
4135 CONF.setInt(HConstants.HBASE_CLIENT_SCANNER_TIMEOUT_PERIOD, 10 * 1000);
4136
4137
4138 CONF.setLong("hbase.client.pause", 15 * 1000);
4139
4140
4141
4142 CONF.setLong(HConstants.HREGION_MAX_FILESIZE, 1024 * 128);
4143 return CONF;
4144 }
4145
4146
4147
4148
4149
4150
4151
4152
4153
4154
4155 public static HRegion initHRegion(TableName tableName, String callingMethod, Configuration conf,
4156 byte[]... families) throws IOException {
4157 return initHRegion(tableName.getName(), null, null, callingMethod, conf, false, families);
4158 }
4159
4160
4161
4162
4163
4164
4165
4166
4167
4168
4169 public static HRegion initHRegion(byte[] tableName, String callingMethod, Configuration conf,
4170 byte[]... families) throws IOException {
4171 return initHRegion(tableName, null, null, callingMethod, conf, false, families);
4172 }
4173
4174
4175
4176
4177
4178
4179
4180
4181
4182
4183
4184 public static HRegion initHRegion(byte[] tableName, String callingMethod, Configuration conf,
4185 boolean isReadOnly, byte[]... families) throws IOException {
4186 return initHRegion(tableName, null, null, callingMethod, conf, isReadOnly, families);
4187 }
4188
4189 private static HRegion initHRegion(byte[] tableName, byte[] startKey, byte[] stopKey,
4190 String callingMethod, Configuration conf, boolean isReadOnly, byte[]... families)
4191 throws IOException {
4192 return initHRegion(tableName, startKey, stopKey, callingMethod, conf, isReadOnly,
4193 Durability.SYNC_WAL, null, families);
4194 }
4195
4196
4197
4198
4199
4200
4201
4202
4203
4204
4205
4206
4207
4208 private static HRegion initHRegion(byte[] tableName, byte[] startKey, byte[] stopKey,
4209 String callingMethod, Configuration conf, boolean isReadOnly, Durability durability,
4210 HLog hlog, byte[]... families) throws IOException {
4211 return TEST_UTIL.createLocalHRegion(tableName, startKey, stopKey, callingMethod, conf, isReadOnly, durability, hlog, families);
4212 }
4213
4214
4215
4216
4217
4218 private void checkOneCell(Cell kv, byte[] cf, int rowIdx, int colIdx, long ts) {
4219 String ctx = "rowIdx=" + rowIdx + "; colIdx=" + colIdx + "; ts=" + ts;
4220 assertEquals("Row mismatch which checking: " + ctx, "row:" + rowIdx,
4221 Bytes.toString(CellUtil.cloneRow(kv)));
4222 assertEquals("ColumnFamily mismatch while checking: " + ctx, Bytes.toString(cf),
4223 Bytes.toString(CellUtil.cloneFamily(kv)));
4224 assertEquals("Column qualifier mismatch while checking: " + ctx, "column:" + colIdx,
4225 Bytes.toString(CellUtil.cloneQualifier(kv)));
4226 assertEquals("Timestamp mismatch while checking: " + ctx, ts, kv.getTimestamp());
4227 assertEquals("Value mismatch while checking: " + ctx, "value-version-" + ts,
4228 Bytes.toString(CellUtil.cloneValue(kv)));
4229 }
4230 }