1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20 package org.apache.hadoop.hbase.regionserver;
21
22 import java.io.EOFException;
23 import java.io.FileNotFoundException;
24 import java.io.IOException;
25 import java.io.InterruptedIOException;
26 import java.io.UnsupportedEncodingException;
27 import java.lang.reflect.Constructor;
28 import java.lang.reflect.InvocationTargetException;
29 import java.lang.reflect.Method;
30 import java.text.ParseException;
31 import java.util.AbstractList;
32 import java.util.ArrayList;
33 import java.util.Arrays;
34 import java.util.Collection;
35 import java.util.Collections;
36 import java.util.HashMap;
37 import java.util.List;
38 import java.util.Map;
39 import java.util.NavigableMap;
40 import java.util.NavigableSet;
41 import java.util.Random;
42 import java.util.Set;
43 import java.util.TreeMap;
44 import java.util.UUID;
45 import java.util.concurrent.Callable;
46 import java.util.concurrent.CompletionService;
47 import java.util.concurrent.ConcurrentHashMap;
48 import java.util.concurrent.ConcurrentSkipListMap;
49 import java.util.concurrent.CountDownLatch;
50 import java.util.concurrent.ExecutionException;
51 import java.util.concurrent.ExecutorCompletionService;
52 import java.util.concurrent.Future;
53 import java.util.concurrent.ThreadFactory;
54 import java.util.concurrent.ThreadPoolExecutor;
55 import java.util.concurrent.TimeUnit;
56 import java.util.concurrent.atomic.AtomicBoolean;
57 import java.util.concurrent.atomic.AtomicInteger;
58 import java.util.concurrent.atomic.AtomicLong;
59 import java.util.concurrent.locks.Lock;
60 import java.util.concurrent.locks.ReentrantReadWriteLock;
61
62 import org.apache.commons.logging.Log;
63 import org.apache.commons.logging.LogFactory;
64 import org.apache.hadoop.conf.Configuration;
65 import org.apache.hadoop.fs.FSDataOutputStream;
66 import org.apache.hadoop.fs.FSDataInputStream;
67 import org.apache.hadoop.fs.FileStatus;
68 import org.apache.hadoop.fs.FileSystem;
69 import org.apache.hadoop.fs.Path;
70 import org.apache.hadoop.fs.permission.FsPermission;
71 import org.apache.hadoop.hbase.DoNotRetryIOException;
72 import org.apache.hadoop.hbase.DroppedSnapshotException;
73 import org.apache.hadoop.hbase.HBaseConfiguration;
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.NotServingRegionException;
82 import org.apache.hadoop.hbase.RegionTooBusyException;
83 import org.apache.hadoop.hbase.UnknownScannerException;
84 import org.apache.hadoop.hbase.backup.HFileArchiver;
85 import org.apache.hadoop.hbase.client.Append;
86 import org.apache.hadoop.hbase.client.RowMutations;
87 import org.apache.hadoop.hbase.client.Delete;
88 import org.apache.hadoop.hbase.client.Get;
89 import org.apache.hadoop.hbase.client.Increment;
90 import org.apache.hadoop.hbase.client.IsolationLevel;
91 import org.apache.hadoop.hbase.client.Mutation;
92 import org.apache.hadoop.hbase.client.Put;
93 import org.apache.hadoop.hbase.client.Result;
94 import org.apache.hadoop.hbase.client.Row;
95 import org.apache.hadoop.hbase.client.RowLock;
96 import org.apache.hadoop.hbase.client.Scan;
97 import org.apache.hadoop.hbase.client.coprocessor.Exec;
98 import org.apache.hadoop.hbase.client.coprocessor.ExecResult;
99 import org.apache.hadoop.hbase.errorhandling.ForeignExceptionSnare;
100 import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
101 import org.apache.hadoop.hbase.filter.Filter;
102 import org.apache.hadoop.hbase.filter.FilterBase;
103 import org.apache.hadoop.hbase.filter.IncompatibleFilterException;
104 import org.apache.hadoop.hbase.filter.WritableByteArrayComparable;
105 import org.apache.hadoop.hbase.io.HeapSize;
106 import org.apache.hadoop.hbase.io.TimeRange;
107 import org.apache.hadoop.hbase.io.hfile.BlockCache;
108 import org.apache.hadoop.hbase.io.hfile.CacheConfig;
109 import org.apache.hadoop.hbase.ipc.CoprocessorProtocol;
110 import org.apache.hadoop.hbase.ipc.HBaseRPC;
111 import org.apache.hadoop.hbase.ipc.HBaseServer;
112 import org.apache.hadoop.hbase.ipc.RpcCallContext;
113 import org.apache.hadoop.hbase.monitoring.MonitoredTask;
114 import org.apache.hadoop.hbase.monitoring.TaskMonitor;
115 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
116 import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
117 import org.apache.hadoop.hbase.regionserver.metrics.OperationMetrics;
118 import org.apache.hadoop.hbase.regionserver.metrics.SchemaMetrics;
119 import org.apache.hadoop.hbase.regionserver.wal.HLog;
120 import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
121 import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
122 import org.apache.hadoop.hbase.snapshot.TakeSnapshotUtils;
123 import org.apache.hadoop.hbase.util.Bytes;
124 import org.apache.hadoop.hbase.util.CancelableProgressable;
125 import org.apache.hadoop.hbase.util.ClassSize;
126 import org.apache.hadoop.hbase.util.CompressionTest;
127 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
128 import org.apache.hadoop.hbase.util.FSUtils;
129 import org.apache.hadoop.hbase.util.HashedBytes;
130 import org.apache.hadoop.hbase.util.Pair;
131 import org.apache.hadoop.hbase.util.Threads;
132 import org.apache.hadoop.hbase.util.Writables;
133 import org.apache.hadoop.io.MultipleIOException;
134 import org.apache.hadoop.io.Writable;
135 import org.apache.hadoop.util.StringUtils;
136 import org.cliffc.high_scale_lib.Counter;
137
138 import com.google.common.base.Preconditions;
139 import com.google.common.collect.ClassToInstanceMap;
140 import com.google.common.collect.ImmutableList;
141 import com.google.common.collect.Lists;
142 import com.google.common.collect.Maps;
143 import com.google.common.collect.MutableClassToInstanceMap;
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181 public class HRegion implements HeapSize {
182 public static final Log LOG = LogFactory.getLog(HRegion.class);
183 private static final String MERGEDIR = ".merges";
184
185 public static final String LOAD_CFS_ON_DEMAND_CONFIG_KEY = "hbase.hregion.scan.loadColumnFamiliesOnDemand";
186
187 final AtomicBoolean closed = new AtomicBoolean(false);
188
189
190
191
192
193 final AtomicBoolean closing = new AtomicBoolean(false);
194
195
196
197
198
199 private final ConcurrentHashMap<HashedBytes, CountDownLatch> lockedRows =
200 new ConcurrentHashMap<HashedBytes, CountDownLatch>();
201 private final ConcurrentHashMap<Integer, HashedBytes> lockIds =
202 new ConcurrentHashMap<Integer, HashedBytes>();
203 private final AtomicInteger lockIdGenerator = new AtomicInteger(1);
204 static private Random rand = new Random();
205
206 protected final Map<byte [], Store> stores =
207 new ConcurrentSkipListMap<byte [], Store>(Bytes.BYTES_RAWCOMPARATOR);
208
209
210 private ClassToInstanceMap<CoprocessorProtocol>
211 protocolHandlers = MutableClassToInstanceMap.create();
212
213 private Map<String, Class<? extends CoprocessorProtocol>>
214 protocolHandlerNames = Maps.newHashMap();
215
216
217
218
219 public static final String REGION_TEMP_SUBDIR = ".tmp";
220
221
222
223
224
225
226
227 final AtomicLong memstoreSize = new AtomicLong(0);
228
229
230 final AtomicLong numPutsWithoutWAL = new AtomicLong(0);
231 final AtomicLong dataInMemoryWithoutWAL = new AtomicLong(0);
232
233 final Counter readRequestsCount = new Counter();
234 final Counter writeRequestsCount = new Counter();
235 final Counter updatesBlockedMs = new Counter();
236
237
238
239
240
241 private final Path tableDir;
242
243 private final HLog log;
244 private final FileSystem fs;
245 private final Configuration conf;
246 private final int rowLockWaitDuration;
247 static final int DEFAULT_ROWLOCK_WAIT_DURATION = 30000;
248
249
250
251
252
253
254
255 final long busyWaitDuration;
256 static final long DEFAULT_BUSY_WAIT_DURATION = HConstants.DEFAULT_HBASE_RPC_TIMEOUT;
257
258
259
260
261 final int maxBusyWaitMultiplier;
262
263
264
265 final long maxBusyWaitDuration;
266
267 private final HRegionInfo regionInfo;
268 private final Path regiondir;
269 KeyValue.KVComparator comparator;
270
271 private ConcurrentHashMap<RegionScanner, Long> scannerReadPoints;
272
273
274
275
276 private boolean isLoadingCfsOnDemandDefault = false;
277
278
279
280
281
282
283 public long getSmallestReadPoint() {
284 long minimumReadPoint;
285
286
287
288 synchronized(scannerReadPoints) {
289 minimumReadPoint = mvcc.memstoreReadPoint();
290
291 for (Long readPoint: this.scannerReadPoints.values()) {
292 if (readPoint < minimumReadPoint) {
293 minimumReadPoint = readPoint;
294 }
295 }
296 }
297 return minimumReadPoint;
298 }
299
300
301
302
303 static class WriteState {
304
305 volatile boolean flushing = false;
306
307 volatile boolean flushRequested = false;
308
309 volatile int compacting = 0;
310
311 volatile boolean writesEnabled = true;
312
313 volatile boolean readOnly = false;
314
315
316
317
318
319
320 synchronized void setReadOnly(final boolean onOff) {
321 this.writesEnabled = !onOff;
322 this.readOnly = onOff;
323 }
324
325 boolean isReadOnly() {
326 return this.readOnly;
327 }
328
329 boolean isFlushRequested() {
330 return this.flushRequested;
331 }
332
333 static final long HEAP_SIZE = ClassSize.align(
334 ClassSize.OBJECT + 5 * Bytes.SIZEOF_BOOLEAN);
335 }
336
337 final WriteState writestate = new WriteState();
338
339 long memstoreFlushSize;
340 final long timestampSlop;
341 private volatile long lastFlushTime;
342 final RegionServerServices rsServices;
343 private RegionServerAccounting rsAccounting;
344 private List<Pair<Long, Long>> recentFlushes = new ArrayList<Pair<Long,Long>>();
345 private long blockingMemStoreSize;
346 final long threadWakeFrequency;
347
348 final ReentrantReadWriteLock lock =
349 new ReentrantReadWriteLock();
350
351
352 private final ReentrantReadWriteLock updatesLock =
353 new ReentrantReadWriteLock();
354 private boolean splitRequest;
355 private byte[] explicitSplitPoint = null;
356
357 private final MultiVersionConsistencyControl mvcc =
358 new MultiVersionConsistencyControl();
359
360
361 private RegionCoprocessorHost coprocessorHost;
362
363
364
365
366 public final static String REGIONINFO_FILE = ".regioninfo";
367 private HTableDescriptor htableDescriptor = null;
368 private RegionSplitPolicy splitPolicy;
369 private final OperationMetrics opMetrics;
370 private final boolean deferredLogSyncDisabled;
371
372
373
374
375 public HRegion(){
376 this.tableDir = null;
377 this.blockingMemStoreSize = 0L;
378 this.conf = null;
379 this.rowLockWaitDuration = DEFAULT_ROWLOCK_WAIT_DURATION;
380 this.rsServices = null;
381 this.fs = null;
382 this.timestampSlop = HConstants.LATEST_TIMESTAMP;
383 this.memstoreFlushSize = 0L;
384 this.log = null;
385 this.regiondir = null;
386 this.regionInfo = null;
387 this.htableDescriptor = null;
388 this.threadWakeFrequency = 0L;
389 this.coprocessorHost = null;
390 this.scannerReadPoints = new ConcurrentHashMap<RegionScanner, Long>();
391 this.opMetrics = new OperationMetrics();
392
393 this.maxBusyWaitDuration = 2 * HConstants.DEFAULT_HBASE_RPC_TIMEOUT;
394 this.busyWaitDuration = DEFAULT_BUSY_WAIT_DURATION;
395 this.maxBusyWaitMultiplier = 2;
396 this.deferredLogSyncDisabled = false;
397 }
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422 public HRegion(Path tableDir, HLog log, FileSystem fs, Configuration conf,
423 final HRegionInfo regionInfo, final HTableDescriptor htd,
424 RegionServerServices rsServices) {
425 this.tableDir = tableDir;
426 this.comparator = regionInfo.getComparator();
427 this.log = log;
428 this.fs = fs;
429 this.conf = conf;
430 this.rowLockWaitDuration = conf.getInt("hbase.rowlock.wait.duration",
431 DEFAULT_ROWLOCK_WAIT_DURATION);
432
433 this.isLoadingCfsOnDemandDefault = conf.getBoolean(LOAD_CFS_ON_DEMAND_CONFIG_KEY, false);
434 this.regionInfo = regionInfo;
435 this.htableDescriptor = htd;
436 this.rsServices = rsServices;
437 this.threadWakeFrequency = conf.getLong(HConstants.THREAD_WAKE_FREQUENCY,
438 10 * 1000);
439 String encodedNameStr = this.regionInfo.getEncodedName();
440 setHTableSpecificConf();
441 this.regiondir = getRegionDir(this.tableDir, encodedNameStr);
442 this.scannerReadPoints = new ConcurrentHashMap<RegionScanner, Long>();
443 this.opMetrics = new OperationMetrics(conf, this.regionInfo);
444
445 this.busyWaitDuration = conf.getLong(
446 "hbase.busy.wait.duration", DEFAULT_BUSY_WAIT_DURATION);
447 this.maxBusyWaitMultiplier = conf.getInt("hbase.busy.wait.multiplier.max", 2);
448 if (busyWaitDuration * maxBusyWaitMultiplier <= 0L) {
449 throw new IllegalArgumentException("Invalid hbase.busy.wait.duration ("
450 + busyWaitDuration + ") or hbase.busy.wait.multiplier.max ("
451 + maxBusyWaitMultiplier + "). Their product should be positive");
452 }
453 this.maxBusyWaitDuration = conf.getLong("ipc.client.call.purge.timeout",
454 2 * HConstants.DEFAULT_HBASE_RPC_TIMEOUT);
455
456
457
458
459
460
461
462 this.timestampSlop = conf.getLong(
463 "hbase.hregion.keyvalue.timestamp.slop.millisecs",
464 HConstants.LATEST_TIMESTAMP);
465
466 this.deferredLogSyncDisabled = conf.getLong("hbase.regionserver.optionallogflushinterval",
467 1 * 1000) <= 0;
468
469 if (rsServices != null) {
470 this.rsAccounting = this.rsServices.getRegionServerAccounting();
471
472
473 this.coprocessorHost = new RegionCoprocessorHost(this, rsServices, conf);
474 }
475 if (LOG.isDebugEnabled()) {
476
477 LOG.debug("Instantiated " + this);
478 }
479 }
480
481 void setHTableSpecificConf() {
482 if (this.htableDescriptor == null) return;
483 LOG.info("Setting up tabledescriptor config now ...");
484 long flushSize = this.htableDescriptor.getMemStoreFlushSize();
485
486 if (flushSize <= 0) {
487 flushSize = conf.getLong(HConstants.HREGION_MEMSTORE_FLUSH_SIZE,
488 HTableDescriptor.DEFAULT_MEMSTORE_FLUSH_SIZE);
489 }
490 this.memstoreFlushSize = flushSize;
491 this.blockingMemStoreSize = this.memstoreFlushSize *
492 conf.getLong("hbase.hregion.memstore.block.multiplier", 2);
493 }
494
495
496
497
498
499
500 public long initialize() throws IOException {
501 return initialize(null);
502 }
503
504
505
506
507
508
509
510
511 public long initialize(final CancelableProgressable reporter)
512 throws IOException {
513
514 MonitoredTask status = TaskMonitor.get().createStatus(
515 "Initializing region " + this);
516
517 long nextSeqId = -1;
518 try {
519 nextSeqId = initializeRegionInternals(reporter, status);
520 return nextSeqId;
521 } finally {
522
523
524 if (nextSeqId == -1) {
525 status.abort("Exception during region " + this.getRegionNameAsString()
526 + " initialization.");
527 }
528 }
529 }
530
531 private long initializeRegionInternals(final CancelableProgressable reporter,
532 MonitoredTask status) throws IOException, UnsupportedEncodingException {
533 if (coprocessorHost != null) {
534 status.setStatus("Running coprocessor pre-open hook");
535 coprocessorHost.preOpen();
536 }
537
538
539 status.setStatus("Writing region info on filesystem");
540 checkRegioninfoOnFilesystem();
541
542
543 status.setStatus("Cleaning up temporary data from old regions");
544 cleanupTmpDir();
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562 long minSeqId = -1;
563 long maxSeqId = -1;
564
565 long maxMemstoreTS = -1;
566
567 if (this.htableDescriptor != null &&
568 !htableDescriptor.getFamilies().isEmpty()) {
569
570 ThreadPoolExecutor storeOpenerThreadPool =
571 getStoreOpenAndCloseThreadPool(
572 "StoreOpenerThread-" + this.regionInfo.getRegionNameAsString());
573 CompletionService<Store> completionService =
574 new ExecutorCompletionService<Store>(storeOpenerThreadPool);
575
576
577 for (final HColumnDescriptor family : htableDescriptor.getFamilies()) {
578 status.setStatus("Instantiating store for column family " + family);
579 completionService.submit(new Callable<Store>() {
580 public Store call() throws IOException {
581 return instantiateHStore(tableDir, family);
582 }
583 });
584 }
585 try {
586 for (int i = 0; i < htableDescriptor.getFamilies().size(); i++) {
587 Future<Store> future = completionService.take();
588 Store store = future.get();
589
590 this.stores.put(store.getColumnFamilyName().getBytes(), store);
591 long storeSeqId = store.getMaxSequenceId();
592 if (minSeqId == -1 || storeSeqId < minSeqId) {
593 minSeqId = storeSeqId;
594 }
595 if (maxSeqId == -1 || storeSeqId > maxSeqId) {
596 maxSeqId = storeSeqId;
597 }
598 long maxStoreMemstoreTS = store.getMaxMemstoreTS();
599 if (maxStoreMemstoreTS > maxMemstoreTS) {
600 maxMemstoreTS = maxStoreMemstoreTS;
601 }
602 }
603 } catch (InterruptedException e) {
604 throw new IOException(e);
605 } catch (ExecutionException e) {
606 throw new IOException(e.getCause());
607 } finally {
608 storeOpenerThreadPool.shutdownNow();
609 }
610 }
611 mvcc.initialize(maxMemstoreTS + 1);
612
613 maxSeqId = Math.max(maxSeqId, replayRecoveredEditsIfAny(
614 this.regiondir, minSeqId, reporter, status));
615
616 status.setStatus("Cleaning up detritus from prior splits");
617
618
619
620 SplitTransaction.cleanupAnySplitDetritus(this);
621 FSUtils.deleteDirectory(this.fs, new Path(regiondir, MERGEDIR));
622
623 this.writestate.setReadOnly(this.htableDescriptor.isReadOnly());
624
625 this.writestate.flushRequested = false;
626 this.writestate.compacting = 0;
627
628
629 this.splitPolicy = RegionSplitPolicy.create(this, conf);
630
631 this.lastFlushTime = EnvironmentEdgeManager.currentTimeMillis();
632
633
634 long nextSeqid = maxSeqId + 1;
635 LOG.info("Onlined " + this.toString() + "; next sequenceid=" + nextSeqid);
636
637
638 this.closing.set(false);
639 this.closed.set(false);
640
641 if (coprocessorHost != null) {
642 status.setStatus("Running coprocessor post-open hooks");
643 coprocessorHost.postOpen();
644 }
645
646 status.markComplete("Region opened successfully");
647 return nextSeqid;
648 }
649
650
651
652
653
654
655
656 static void moveInitialFilesIntoPlace(final FileSystem fs,
657 final Path initialFiles, final Path regiondir)
658 throws IOException {
659 if (initialFiles != null && fs.exists(initialFiles)) {
660 if (!fs.rename(initialFiles, regiondir)) {
661 LOG.warn("Unable to rename " + initialFiles + " to " + regiondir);
662 }
663 }
664 }
665
666
667
668
669 public boolean hasReferences() {
670 for (Store store : this.stores.values()) {
671 for (StoreFile sf : store.getStorefiles()) {
672
673 if (sf.isReference()) return true;
674 }
675 }
676 return false;
677 }
678
679
680
681
682
683
684 public HDFSBlocksDistribution getHDFSBlocksDistribution() {
685 HDFSBlocksDistribution hdfsBlocksDistribution =
686 new HDFSBlocksDistribution();
687 synchronized (this.stores) {
688 for (Store store : this.stores.values()) {
689 for (StoreFile sf : store.getStorefiles()) {
690 HDFSBlocksDistribution storeFileBlocksDistribution =
691 sf.getHDFSBlockDistribution();
692 hdfsBlocksDistribution.add(storeFileBlocksDistribution);
693 }
694 }
695 }
696 return hdfsBlocksDistribution;
697 }
698
699
700
701
702
703
704
705
706
707 static public HDFSBlocksDistribution computeHDFSBlocksDistribution(
708 Configuration conf, HTableDescriptor tableDescriptor,
709 String regionEncodedName) throws IOException {
710 HDFSBlocksDistribution hdfsBlocksDistribution =
711 new HDFSBlocksDistribution();
712 Path tablePath = FSUtils.getTablePath(FSUtils.getRootDir(conf),
713 tableDescriptor.getName());
714 FileSystem fs = tablePath.getFileSystem(conf);
715
716 for (HColumnDescriptor family: tableDescriptor.getFamilies()) {
717 Path storeHomeDir = Store.getStoreHomedir(tablePath, regionEncodedName,
718 family.getName());
719 if (!fs.exists(storeHomeDir))continue;
720
721 FileStatus[] hfilesStatus = null;
722 hfilesStatus = fs.listStatus(storeHomeDir);
723
724 for (FileStatus hfileStatus : hfilesStatus) {
725 HDFSBlocksDistribution storeFileBlocksDistribution =
726 FSUtils.computeHDFSBlocksDistribution(fs, hfileStatus, 0,
727 hfileStatus.getLen());
728 hdfsBlocksDistribution.add(storeFileBlocksDistribution);
729 }
730 }
731 return hdfsBlocksDistribution;
732 }
733
734 public AtomicLong getMemstoreSize() {
735 return memstoreSize;
736 }
737
738
739
740
741
742
743
744 public long addAndGetGlobalMemstoreSize(long memStoreSize) {
745 if (this.rsAccounting != null) {
746 rsAccounting.addAndGetGlobalMemstoreSize(memStoreSize);
747 }
748 return this.memstoreSize.getAndAdd(memStoreSize);
749 }
750
751
752
753
754
755
756 private void checkRegioninfoOnFilesystem() throws IOException {
757 checkRegioninfoOnFilesystem(this.regiondir);
758 }
759
760
761
762
763
764
765 private void checkRegioninfoOnFilesystem(Path regiondir) throws IOException {
766 writeRegioninfoOnFilesystem(regionInfo, regiondir, getFilesystem(), conf);
767 }
768
769
770
771
772
773
774
775
776
777
778 public static void writeRegioninfoOnFilesystem(HRegionInfo regionInfo, Path regiondir,
779 FileSystem fs, Configuration conf) throws IOException {
780 Path regioninfoPath = new Path(regiondir, REGIONINFO_FILE);
781 if (fs.exists(regioninfoPath) &&
782 fs.getFileStatus(regioninfoPath).getLen() > 0) {
783 return;
784 }
785
786
787
788
789
790
791 FsPermission perms = FSUtils.getFilePermissions(fs, conf,
792 HConstants.DATA_FILE_UMASK_KEY);
793
794
795 Path tmpPath = new Path(getTmpDir(regiondir), REGIONINFO_FILE);
796
797
798
799
800
801 if (FSUtils.isExists(fs, tmpPath)) {
802 FSUtils.delete(fs, tmpPath, true);
803 }
804
805 FSDataOutputStream out = FSUtils.create(fs, tmpPath, perms);
806
807 try {
808 regionInfo.write(out);
809 out.write('\n');
810 out.write('\n');
811 out.write(Bytes.toBytes(regionInfo.toString()));
812 } finally {
813 out.close();
814 }
815 if (!fs.rename(tmpPath, regioninfoPath)) {
816 throw new IOException("Unable to rename " + tmpPath + " to " +
817 regioninfoPath);
818 }
819 }
820
821
822
823
824
825
826
827 public static HRegionInfo loadDotRegionInfoFileContent(final FileSystem fs, final Path dir)
828 throws IOException {
829 Path regioninfo = new Path(dir, HRegion.REGIONINFO_FILE);
830 if (!fs.exists(regioninfo)) throw new FileNotFoundException(regioninfo.toString());
831 FSDataInputStream in = fs.open(regioninfo);
832 try {
833 HRegionInfo hri = new HRegionInfo();
834 hri.readFields(in);
835 return hri;
836 } finally {
837 in.close();
838 }
839 }
840
841
842 public HRegionInfo getRegionInfo() {
843 return this.regionInfo;
844 }
845
846
847
848
849
850 RegionServerServices getRegionServerServices() {
851 return this.rsServices;
852 }
853
854
855 public long getRequestsCount() {
856 return this.readRequestsCount.get() + this.writeRequestsCount.get();
857 }
858
859
860 public long getReadRequestsCount() {
861 return this.readRequestsCount.get();
862 }
863
864
865 public long getWriteRequestsCount() {
866 return this.writeRequestsCount.get();
867 }
868
869
870 public boolean isClosed() {
871 return this.closed.get();
872 }
873
874
875
876
877 public boolean isClosing() {
878 return this.closing.get();
879 }
880
881
882 public boolean isAvailable() {
883 return !isClosed() && !isClosing();
884 }
885
886
887 public boolean isSplittable() {
888 return isAvailable() && !hasReferences();
889 }
890
891 boolean areWritesEnabled() {
892 synchronized(this.writestate) {
893 return this.writestate.writesEnabled;
894 }
895 }
896
897 public MultiVersionConsistencyControl getMVCC() {
898 return mvcc;
899 }
900
901 public boolean isLoadingCfsOnDemandDefault() {
902 return this.isLoadingCfsOnDemandDefault;
903 }
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918 public List<StoreFile> close() throws IOException {
919 return close(false);
920 }
921
922 private final Object closeLock = new Object();
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938 public List<StoreFile> close(final boolean abort) throws IOException {
939
940
941 MonitoredTask status = TaskMonitor.get().createStatus(
942 "Closing region " + this +
943 (abort ? " due to abort" : ""));
944
945 status.setStatus("Waiting for close lock");
946 try {
947 synchronized (closeLock) {
948 return doClose(abort, status);
949 }
950 } finally {
951 status.cleanup();
952 }
953 }
954
955 private List<StoreFile> doClose(
956 final boolean abort, MonitoredTask status)
957 throws IOException {
958 if (isClosed()) {
959 LOG.warn("Region " + this + " already closed");
960 return null;
961 }
962
963 if (coprocessorHost != null) {
964 status.setStatus("Running coprocessor pre-close hooks");
965 this.coprocessorHost.preClose(abort);
966 }
967
968 status.setStatus("Disabling compacts and flushes for region");
969 boolean wasFlushing = false;
970 synchronized (writestate) {
971
972
973 writestate.writesEnabled = false;
974 wasFlushing = writestate.flushing;
975 LOG.debug("Closing " + this + ": disabling compactions & flushes");
976 waitForFlushesAndCompactions();
977 }
978
979
980
981 if (!abort && !wasFlushing && worthPreFlushing()) {
982 status.setStatus("Pre-flushing region before close");
983 LOG.info("Running close preflush of " + this.getRegionNameAsString());
984 internalFlushcache(status);
985 }
986
987 this.closing.set(true);
988 status.setStatus("Disabling writes for close");
989
990 lock.writeLock().lock();
991 try {
992 if (this.isClosed()) {
993 status.abort("Already got closed by another process");
994
995 return null;
996 }
997 LOG.debug("Updates disabled for region " + this);
998
999 if (!abort) {
1000 internalFlushcache(status);
1001 }
1002
1003 List<StoreFile> result = new ArrayList<StoreFile>();
1004 if (!stores.isEmpty()) {
1005
1006 ThreadPoolExecutor storeCloserThreadPool =
1007 getStoreOpenAndCloseThreadPool("StoreCloserThread-"
1008 + this.regionInfo.getRegionNameAsString());
1009 CompletionService<ImmutableList<StoreFile>> completionService =
1010 new ExecutorCompletionService<ImmutableList<StoreFile>>(
1011 storeCloserThreadPool);
1012
1013
1014 for (final Store store : stores.values()) {
1015 completionService
1016 .submit(new Callable<ImmutableList<StoreFile>>() {
1017 public ImmutableList<StoreFile> call() throws IOException {
1018 return store.close();
1019 }
1020 });
1021 }
1022 try {
1023 for (int i = 0; i < stores.size(); i++) {
1024 Future<ImmutableList<StoreFile>> future = completionService
1025 .take();
1026 ImmutableList<StoreFile> storeFileList = future.get();
1027 result.addAll(storeFileList);
1028 }
1029 } catch (InterruptedException e) {
1030 throw new IOException(e);
1031 } catch (ExecutionException e) {
1032 throw new IOException(e.getCause());
1033 } finally {
1034 storeCloserThreadPool.shutdownNow();
1035 }
1036 }
1037 this.closed.set(true);
1038
1039 if (coprocessorHost != null) {
1040 status.setStatus("Running coprocessor post-close hooks");
1041 this.coprocessorHost.postClose(abort);
1042 }
1043 this.opMetrics.closeMetrics(this.getRegionInfo().getEncodedName());
1044 status.markComplete("Closed");
1045 LOG.info("Closed " + this);
1046 return result;
1047 } finally {
1048 lock.writeLock().unlock();
1049 }
1050 }
1051
1052
1053
1054
1055
1056
1057 public void waitForFlushesAndCompactions() {
1058 synchronized (writestate) {
1059 while (writestate.compacting > 0 || writestate.flushing) {
1060 LOG.debug("waiting for " + writestate.compacting + " compactions"
1061 + (writestate.flushing ? " & cache flush" : "") + " to complete for region " + this);
1062 try {
1063 writestate.wait();
1064 } catch (InterruptedException iex) {
1065
1066 Thread.currentThread().interrupt();
1067 }
1068 }
1069 }
1070 }
1071
1072 protected ThreadPoolExecutor getStoreOpenAndCloseThreadPool(
1073 final String threadNamePrefix) {
1074 int numStores = Math.max(1, this.htableDescriptor.getFamilies().size());
1075 int maxThreads = Math.min(numStores,
1076 conf.getInt(HConstants.HSTORE_OPEN_AND_CLOSE_THREADS_MAX,
1077 HConstants.DEFAULT_HSTORE_OPEN_AND_CLOSE_THREADS_MAX));
1078 return getOpenAndCloseThreadPool(maxThreads, threadNamePrefix);
1079 }
1080
1081 protected ThreadPoolExecutor getStoreFileOpenAndCloseThreadPool(
1082 final String threadNamePrefix) {
1083 int numStores = Math.max(1, this.htableDescriptor.getFamilies().size());
1084 int maxThreads = Math.max(1,
1085 conf.getInt(HConstants.HSTORE_OPEN_AND_CLOSE_THREADS_MAX,
1086 HConstants.DEFAULT_HSTORE_OPEN_AND_CLOSE_THREADS_MAX)
1087 / numStores);
1088 return getOpenAndCloseThreadPool(maxThreads, threadNamePrefix);
1089 }
1090
1091 static ThreadPoolExecutor getOpenAndCloseThreadPool(int maxThreads,
1092 final String threadNamePrefix) {
1093 return Threads.getBoundedCachedThreadPool(maxThreads, 30L, TimeUnit.SECONDS,
1094 new ThreadFactory() {
1095 private int count = 1;
1096
1097 public Thread newThread(Runnable r) {
1098 return new Thread(r, threadNamePrefix + "-" + count++);
1099 }
1100 });
1101 }
1102
1103
1104
1105
1106 private boolean worthPreFlushing() {
1107 return this.memstoreSize.get() >
1108 this.conf.getLong("hbase.hregion.preclose.flush.size", 1024 * 1024 * 5);
1109 }
1110
1111
1112
1113
1114
1115
1116 public byte [] getStartKey() {
1117 return this.regionInfo.getStartKey();
1118 }
1119
1120
1121 public byte [] getEndKey() {
1122 return this.regionInfo.getEndKey();
1123 }
1124
1125
1126 public long getRegionId() {
1127 return this.regionInfo.getRegionId();
1128 }
1129
1130
1131 public byte [] getRegionName() {
1132 return this.regionInfo.getRegionName();
1133 }
1134
1135
1136 public String getRegionNameAsString() {
1137 return this.regionInfo.getRegionNameAsString();
1138 }
1139
1140
1141 public HTableDescriptor getTableDesc() {
1142 return this.htableDescriptor;
1143 }
1144
1145
1146 public HLog getLog() {
1147 return this.log;
1148 }
1149
1150
1151 public Configuration getConf() {
1152 return this.conf;
1153 }
1154
1155
1156 public Path getRegionDir() {
1157 return this.regiondir;
1158 }
1159
1160
1161
1162
1163
1164
1165
1166
1167 public static Path getRegionDir(final Path tabledir, final String name) {
1168 return new Path(tabledir, name);
1169 }
1170
1171
1172 public FileSystem getFilesystem() {
1173 return this.fs;
1174 }
1175
1176
1177 public long getLastFlushTime() {
1178 return this.lastFlushTime;
1179 }
1180
1181
1182 public List<Pair<Long,Long>> getRecentFlushInfo() {
1183 this.lock.readLock().lock();
1184 List<Pair<Long,Long>> ret = this.recentFlushes;
1185 this.recentFlushes = new ArrayList<Pair<Long,Long>>();
1186 this.lock.readLock().unlock();
1187 return ret;
1188 }
1189
1190
1191
1192
1193
1194
1195
1196
1197
1198 public long getLargestHStoreSize() {
1199 long size = 0;
1200 for (Store h: stores.values()) {
1201 long storeSize = h.getSize();
1202 if (storeSize > size) {
1203 size = storeSize;
1204 }
1205 }
1206 return size;
1207 }
1208
1209
1210
1211
1212
1213 void doRegionCompactionPrep() throws IOException {
1214 }
1215
1216
1217
1218
1219 private void cleanupTmpDir() throws IOException {
1220 FSUtils.deleteDirectory(this.fs, getTmpDir());
1221 }
1222
1223
1224
1225
1226
1227 Path getTmpDir() {
1228 return getTmpDir(getRegionDir());
1229 }
1230
1231 static Path getTmpDir(Path regionDir) {
1232 return new Path(regionDir, REGION_TEMP_SUBDIR);
1233 }
1234
1235 void triggerMajorCompaction() {
1236 for (Store h: stores.values()) {
1237 h.triggerMajorCompaction();
1238 }
1239 }
1240
1241
1242
1243
1244
1245
1246
1247
1248 public void compactStores(final boolean majorCompaction)
1249 throws IOException {
1250 if (majorCompaction) {
1251 this.triggerMajorCompaction();
1252 }
1253 compactStores();
1254 }
1255
1256
1257
1258
1259
1260
1261
1262 public void compactStores() throws IOException {
1263 for(Store s : getStores().values()) {
1264 CompactionRequest cr = s.requestCompaction();
1265 if(cr != null) {
1266 try {
1267 compact(cr);
1268 } finally {
1269 s.finishRequest(cr);
1270 }
1271 }
1272 }
1273 }
1274
1275
1276
1277
1278
1279
1280
1281
1282
1283
1284
1285
1286
1287
1288
1289
1290 public boolean compact(CompactionRequest cr)
1291 throws IOException {
1292 if (cr == null) {
1293 return false;
1294 }
1295 if (this.closing.get() || this.closed.get()) {
1296 LOG.debug("Skipping compaction on " + this + " because closing/closed");
1297 return false;
1298 }
1299 Preconditions.checkArgument(cr.getHRegion().equals(this));
1300
1301 lock.readLock().lock();
1302 MonitoredTask status = TaskMonitor.get().createStatus(
1303 "Compacting " + cr.getStore() + " in " + this);
1304 try {
1305 if (this.closed.get()) {
1306 LOG.debug("Skipping compaction on " + this + " because closed");
1307 return false;
1308 }
1309 boolean decr = true;
1310 try {
1311 synchronized (writestate) {
1312 if (writestate.writesEnabled) {
1313 ++writestate.compacting;
1314 } else {
1315 String msg = "NOT compacting region " + this + ". Writes disabled.";
1316 LOG.info(msg);
1317 status.abort(msg);
1318 decr = false;
1319 return false;
1320 }
1321 }
1322 LOG.info("Starting compaction on " + cr.getStore() + " in region "
1323 + this + (cr.getCompactSelection().isOffPeakCompaction()?" as an off-peak compaction":""));
1324 doRegionCompactionPrep();
1325 try {
1326 status.setStatus("Compacting store " + cr.getStore());
1327 cr.getStore().compact(cr);
1328 } catch (InterruptedIOException iioe) {
1329 String msg = "compaction interrupted by user";
1330 LOG.info(msg, iioe);
1331 status.abort(msg);
1332 return false;
1333 }
1334 } finally {
1335 if (decr) {
1336 synchronized (writestate) {
1337 --writestate.compacting;
1338 if (writestate.compacting <= 0) {
1339 writestate.notifyAll();
1340 }
1341 }
1342 }
1343 }
1344 status.markComplete("Compaction complete");
1345 return true;
1346 } finally {
1347 status.cleanup();
1348 lock.readLock().unlock();
1349 }
1350 }
1351
1352
1353
1354
1355
1356
1357
1358
1359
1360
1361
1362
1363
1364
1365
1366
1367
1368
1369
1370
1371
1372 public boolean flushcache() throws IOException {
1373
1374 if (this.closing.get()) {
1375 LOG.debug("Skipping flush on " + this + " because closing");
1376 return false;
1377 }
1378 MonitoredTask status = TaskMonitor.get().createStatus("Flushing " + this);
1379 status.setStatus("Acquiring readlock on region");
1380
1381 lock.readLock().lock();
1382 try {
1383 if (this.closed.get()) {
1384 LOG.debug("Skipping flush on " + this + " because closed");
1385 status.abort("Skipped: closed");
1386 return false;
1387 }
1388 if (coprocessorHost != null) {
1389 status.setStatus("Running coprocessor pre-flush hooks");
1390 coprocessorHost.preFlush();
1391 }
1392 if (numPutsWithoutWAL.get() > 0) {
1393 numPutsWithoutWAL.set(0);
1394 dataInMemoryWithoutWAL.set(0);
1395 }
1396 synchronized (writestate) {
1397 if (!writestate.flushing && writestate.writesEnabled) {
1398 this.writestate.flushing = true;
1399 } else {
1400 if (LOG.isDebugEnabled()) {
1401 LOG.debug("NOT flushing memstore for region " + this
1402 + ", flushing=" + writestate.flushing + ", writesEnabled="
1403 + writestate.writesEnabled);
1404 }
1405 status.abort("Not flushing since "
1406 + (writestate.flushing ? "already flushing"
1407 : "writes not enabled"));
1408 return false;
1409 }
1410 }
1411 try {
1412 boolean result = internalFlushcache(status);
1413
1414 if (coprocessorHost != null) {
1415 status.setStatus("Running post-flush coprocessor hooks");
1416 coprocessorHost.postFlush();
1417 }
1418
1419 status.markComplete("Flush successful");
1420 return result;
1421 } finally {
1422 synchronized (writestate) {
1423 writestate.flushing = false;
1424 this.writestate.flushRequested = false;
1425 writestate.notifyAll();
1426 }
1427 }
1428 } finally {
1429 lock.readLock().unlock();
1430 status.cleanup();
1431 }
1432 }
1433
1434
1435
1436
1437
1438
1439
1440
1441
1442
1443
1444
1445
1446
1447
1448
1449
1450
1451
1452
1453
1454
1455
1456
1457
1458
1459
1460
1461
1462
1463
1464
1465
1466
1467
1468
1469 protected boolean internalFlushcache(MonitoredTask status)
1470 throws IOException {
1471 return internalFlushcache(this.log, -1, status);
1472 }
1473
1474
1475
1476
1477
1478
1479
1480
1481
1482
1483 protected boolean internalFlushcache(
1484 final HLog wal, final long myseqid, MonitoredTask status)
1485 throws IOException {
1486 if (this.rsServices != null && this.rsServices.isAborted()) {
1487
1488 throw new IOException("Aborting flush because server is abortted...");
1489 }
1490 final long startTime = EnvironmentEdgeManager.currentTimeMillis();
1491
1492
1493 this.lastFlushTime = startTime;
1494
1495 if (this.memstoreSize.get() <= 0) {
1496 return false;
1497 }
1498 if (LOG.isDebugEnabled()) {
1499 LOG.debug("Started memstore flush for " + this +
1500 ", current region memstore size " +
1501 StringUtils.humanReadableInt(this.memstoreSize.get()) +
1502 ((wal != null)? "": "; wal is null, using passed sequenceid=" + myseqid));
1503 }
1504
1505
1506
1507
1508
1509
1510
1511
1512 long sequenceId = -1L;
1513 long completeSequenceId = -1L;
1514 MultiVersionConsistencyControl.WriteEntry w = null;
1515
1516
1517
1518
1519 status.setStatus("Obtaining lock to block concurrent updates");
1520
1521 this.updatesLock.writeLock().lock();
1522 long flushsize = this.memstoreSize.get();
1523 status.setStatus("Preparing to flush by snapshotting stores");
1524 List<StoreFlusher> storeFlushers = new ArrayList<StoreFlusher>(stores.size());
1525 try {
1526
1527 w = mvcc.beginMemstoreInsert();
1528 mvcc.advanceMemstore(w);
1529
1530 sequenceId = (wal == null)? myseqid:
1531 wal.startCacheFlush(this.regionInfo.getEncodedNameAsBytes());
1532 completeSequenceId = this.getCompleteCacheFlushSequenceId(sequenceId);
1533
1534 for (Store s : stores.values()) {
1535 storeFlushers.add(s.getStoreFlusher(completeSequenceId));
1536 }
1537
1538
1539 for (StoreFlusher flusher : storeFlushers) {
1540 flusher.prepare();
1541 }
1542 } finally {
1543 this.updatesLock.writeLock().unlock();
1544 }
1545 String s = "Finished snapshotting " + this +
1546 ", commencing wait for mvcc, flushsize=" + flushsize;
1547 status.setStatus(s);
1548 LOG.debug(s);
1549
1550
1551
1552
1553
1554
1555 mvcc.waitForRead(w);
1556
1557 status.setStatus("Flushing stores");
1558 LOG.debug("Finished snapshotting, commencing flushing stores");
1559
1560
1561
1562
1563
1564 boolean compactionRequested = false;
1565 try {
1566
1567
1568
1569
1570
1571 for (StoreFlusher flusher : storeFlushers) {
1572 flusher.flushCache(status);
1573 }
1574
1575
1576
1577 for (StoreFlusher flusher : storeFlushers) {
1578 boolean needsCompaction = flusher.commit(status);
1579 if (needsCompaction) {
1580 compactionRequested = true;
1581 }
1582 }
1583 storeFlushers.clear();
1584
1585
1586 this.addAndGetGlobalMemstoreSize(-flushsize);
1587 } catch (Throwable t) {
1588
1589
1590
1591
1592
1593
1594 if (wal != null) {
1595 wal.abortCacheFlush(this.regionInfo.getEncodedNameAsBytes());
1596 }
1597 DroppedSnapshotException dse = new DroppedSnapshotException("region: " +
1598 Bytes.toStringBinary(getRegionName()));
1599 dse.initCause(t);
1600 status.abort("Flush failed: " + StringUtils.stringifyException(t));
1601 throw dse;
1602 }
1603
1604
1605
1606
1607
1608
1609
1610
1611 if (wal != null) {
1612 wal.completeCacheFlush(this.regionInfo.getEncodedNameAsBytes(),
1613 regionInfo.getTableName(), completeSequenceId,
1614 this.getRegionInfo().isMetaRegion());
1615 }
1616
1617
1618
1619 synchronized (this) {
1620 notifyAll();
1621 }
1622
1623 long time = EnvironmentEdgeManager.currentTimeMillis() - startTime;
1624 long memstoresize = this.memstoreSize.get();
1625 String msg = "Finished memstore flush of ~" +
1626 StringUtils.humanReadableInt(flushsize) + "/" + flushsize +
1627 ", currentsize=" +
1628 StringUtils.humanReadableInt(memstoresize) + "/" + memstoresize +
1629 " for region " + this + " in " + time + "ms, sequenceid=" + sequenceId +
1630 ", compaction requested=" + compactionRequested +
1631 ((wal == null)? "; wal=null": "");
1632 LOG.info(msg);
1633 status.setStatus(msg);
1634 this.recentFlushes.add(new Pair<Long,Long>(time/1000, flushsize));
1635
1636 return compactionRequested;
1637 }
1638
1639
1640
1641
1642
1643
1644
1645
1646
1647 protected long getCompleteCacheFlushSequenceId(long currentSequenceId) {
1648 return currentSequenceId;
1649 }
1650
1651
1652
1653
1654
1655
1656
1657
1658
1659
1660
1661
1662
1663 Result getClosestRowBefore(final byte [] row)
1664 throws IOException{
1665 return getClosestRowBefore(row, HConstants.CATALOG_FAMILY);
1666 }
1667
1668
1669
1670
1671
1672
1673
1674
1675
1676
1677
1678 public Result getClosestRowBefore(final byte [] row, final byte [] family)
1679 throws IOException {
1680 if (coprocessorHost != null) {
1681 Result result = new Result();
1682 if (coprocessorHost.preGetClosestRowBefore(row, family, result)) {
1683 return result;
1684 }
1685 }
1686
1687
1688 checkRow(row, "getClosestRowBefore");
1689 startRegionOperation();
1690 this.readRequestsCount.increment();
1691 this.opMetrics.setReadRequestCountMetrics(this.readRequestsCount.get());
1692 try {
1693 Store store = getStore(family);
1694
1695 KeyValue key = store.getRowKeyAtOrBefore(row);
1696 Result result = null;
1697 if (key != null) {
1698 Get get = new Get(key.getRow());
1699 get.addFamily(family);
1700 result = get(get, null);
1701 }
1702 if (coprocessorHost != null) {
1703 coprocessorHost.postGetClosestRowBefore(row, family, result);
1704 }
1705 return result;
1706 } finally {
1707 closeRegionOperation();
1708 }
1709 }
1710
1711
1712
1713
1714
1715
1716
1717
1718
1719
1720
1721 public RegionScanner getScanner(Scan scan) throws IOException {
1722 return getScanner(scan, null);
1723 }
1724
1725 void prepareScanner(Scan scan) throws IOException {
1726 if(!scan.hasFamilies()) {
1727
1728 for(byte[] family: this.htableDescriptor.getFamiliesKeys()){
1729 scan.addFamily(family);
1730 }
1731 }
1732 }
1733
1734 protected RegionScanner getScanner(Scan scan,
1735 List<KeyValueScanner> additionalScanners) throws IOException {
1736 startRegionOperation();
1737 this.readRequestsCount.increment();
1738 this.opMetrics.setReadRequestCountMetrics(this.readRequestsCount.get());
1739 try {
1740
1741 prepareScanner(scan);
1742 if(scan.hasFamilies()) {
1743 for(byte [] family : scan.getFamilyMap().keySet()) {
1744 checkFamily(family);
1745 }
1746 }
1747 return instantiateRegionScanner(scan, additionalScanners);
1748 } finally {
1749 closeRegionOperation();
1750 }
1751 }
1752
1753 protected RegionScanner instantiateRegionScanner(Scan scan,
1754 List<KeyValueScanner> additionalScanners) throws IOException {
1755 return new RegionScannerImpl(scan, additionalScanners, this);
1756 }
1757
1758
1759
1760
1761 private void prepareDelete(Delete delete) throws IOException {
1762
1763 if(delete.getFamilyMap().isEmpty()){
1764 for(byte [] family : this.htableDescriptor.getFamiliesKeys()){
1765
1766 delete.deleteFamily(family, delete.getTimeStamp());
1767 }
1768 } else {
1769 for(byte [] family : delete.getFamilyMap().keySet()) {
1770 if(family == null) {
1771 throw new NoSuchColumnFamilyException("Empty family is invalid");
1772 }
1773 checkFamily(family);
1774 }
1775 }
1776 }
1777
1778
1779
1780
1781
1782
1783
1784
1785
1786
1787 public void delete(Delete delete, boolean writeToWAL)
1788 throws IOException {
1789 delete(delete, null, writeToWAL);
1790 }
1791
1792
1793
1794
1795
1796
1797
1798
1799 public void delete(Delete delete, Integer lockid, boolean writeToWAL)
1800 throws IOException {
1801 checkReadOnly();
1802 checkResources();
1803 Integer lid = null;
1804 startRegionOperation();
1805 this.writeRequestsCount.increment();
1806 this.opMetrics.setWriteRequestCountMetrics(this.writeRequestsCount.get());
1807 try {
1808 byte [] row = delete.getRow();
1809
1810 lid = getLock(lockid, row, true);
1811
1812 try {
1813
1814 prepareDelete(delete);
1815 internalDelete(delete, delete.getClusterId(), writeToWAL);
1816 } finally {
1817 if(lockid == null) releaseRowLock(lid);
1818 }
1819 } finally {
1820 closeRegionOperation();
1821 }
1822 }
1823
1824
1825
1826
1827
1828
1829
1830 void delete(Map<byte[], List<KeyValue>> familyMap, UUID clusterId,
1831 boolean writeToWAL) throws IOException {
1832 Delete delete = new Delete();
1833 delete.setFamilyMap(familyMap);
1834 delete.setClusterId(clusterId);
1835 delete.setWriteToWAL(writeToWAL);
1836 internalDelete(delete, clusterId, writeToWAL);
1837 }
1838
1839
1840
1841
1842
1843
1844
1845
1846 private void prepareDeleteTimestamps(Map<byte[], List<KeyValue>> familyMap, byte[] byteNow)
1847 throws IOException {
1848 for (Map.Entry<byte[], List<KeyValue>> e : familyMap.entrySet()) {
1849
1850 byte[] family = e.getKey();
1851 List<KeyValue> kvs = e.getValue();
1852 Map<byte[], Integer> kvCount = new TreeMap<byte[], Integer>(Bytes.BYTES_COMPARATOR);
1853
1854 for (KeyValue kv: kvs) {
1855
1856
1857 if (kv.isLatestTimestamp() && kv.isDeleteType()) {
1858 byte[] qual = kv.getQualifier();
1859 if (qual == null) qual = HConstants.EMPTY_BYTE_ARRAY;
1860
1861 Integer count = kvCount.get(qual);
1862 if (count == null) {
1863 kvCount.put(qual, 1);
1864 } else {
1865 kvCount.put(qual, count + 1);
1866 }
1867 count = kvCount.get(qual);
1868
1869 Get get = new Get(kv.getRow());
1870 get.setMaxVersions(count);
1871 get.addColumn(family, qual);
1872
1873 List<KeyValue> result = get(get, false);
1874
1875 if (result.size() < count) {
1876
1877 kv.updateLatestStamp(byteNow);
1878 continue;
1879 }
1880 if (result.size() > count) {
1881 throw new RuntimeException("Unexpected size: " + result.size());
1882 }
1883 KeyValue getkv = result.get(count - 1);
1884 Bytes.putBytes(kv.getBuffer(), kv.getTimestampOffset(),
1885 getkv.getBuffer(), getkv.getTimestampOffset(), Bytes.SIZEOF_LONG);
1886 } else {
1887 kv.updateLatestStamp(byteNow);
1888 }
1889 }
1890 }
1891 }
1892
1893
1894
1895
1896
1897
1898
1899 private void internalDelete(Delete delete, UUID clusterId,
1900 boolean writeToWAL) throws IOException {
1901 Map<byte[], List<KeyValue>> familyMap = delete.getFamilyMap();
1902 WALEdit walEdit = new WALEdit();
1903
1904 if (coprocessorHost != null) {
1905 if (coprocessorHost.preDelete(delete, walEdit, writeToWAL)) {
1906 return;
1907 }
1908 }
1909
1910 long now = EnvironmentEdgeManager.currentTimeMillis();
1911 byte [] byteNow = Bytes.toBytes(now);
1912 boolean flush = false;
1913
1914 lock(updatesLock.readLock());
1915 try {
1916 prepareDeleteTimestamps(delete.getFamilyMap(), byteNow);
1917
1918 if (writeToWAL) {
1919
1920
1921
1922
1923
1924
1925
1926
1927 addFamilyMapToWALEdit(familyMap, walEdit);
1928 this.log.append(regionInfo, this.htableDescriptor.getName(),
1929 walEdit, clusterId, now, this.htableDescriptor);
1930 }
1931
1932
1933 long addedSize = applyFamilyMapToMemstore(familyMap, null);
1934 flush = isFlushSize(this.addAndGetGlobalMemstoreSize(addedSize));
1935
1936 } finally {
1937 this.updatesLock.readLock().unlock();
1938 }
1939
1940 if (coprocessorHost != null) {
1941 coprocessorHost.postDelete(delete, walEdit, writeToWAL);
1942 }
1943 final long after = EnvironmentEdgeManager.currentTimeMillis();
1944 this.opMetrics.updateDeleteMetrics(familyMap.keySet(), after-now);
1945
1946 if (flush) {
1947
1948 requestFlush();
1949 }
1950 }
1951
1952
1953
1954
1955
1956 public void put(Put put) throws IOException {
1957 this.put(put, null, put.getWriteToWAL());
1958 }
1959
1960
1961
1962
1963
1964
1965 public void put(Put put, boolean writeToWAL) throws IOException {
1966 this.put(put, null, writeToWAL);
1967 }
1968
1969
1970
1971
1972
1973
1974
1975 public void put(Put put, Integer lockid) throws IOException {
1976 this.put(put, lockid, put.getWriteToWAL());
1977 }
1978
1979
1980
1981
1982
1983
1984
1985
1986
1987
1988 public void put(Put put, Integer lockid, boolean writeToWAL)
1989 throws IOException {
1990 checkReadOnly();
1991
1992
1993
1994
1995
1996 checkResources();
1997 startRegionOperation();
1998 this.writeRequestsCount.increment();
1999 this.opMetrics.setWriteRequestCountMetrics(this.writeRequestsCount.get());
2000 try {
2001
2002
2003
2004
2005
2006 byte [] row = put.getRow();
2007
2008 Integer lid = getLock(lockid, row, true);
2009
2010 try {
2011
2012 internalPut(put, put.getClusterId(), writeToWAL);
2013 } finally {
2014 if(lockid == null) releaseRowLock(lid);
2015 }
2016 } finally {
2017 closeRegionOperation();
2018 }
2019 }
2020
2021
2022
2023
2024
2025
2026 private static class BatchOperationInProgress<T> {
2027 T[] operations;
2028 int nextIndexToProcess = 0;
2029 OperationStatus[] retCodeDetails;
2030 WALEdit[] walEditsFromCoprocessors;
2031
2032 public BatchOperationInProgress(T[] operations) {
2033 this.operations = operations;
2034 this.retCodeDetails = new OperationStatus[operations.length];
2035 this.walEditsFromCoprocessors = new WALEdit[operations.length];
2036 Arrays.fill(this.retCodeDetails, OperationStatus.NOT_RUN);
2037 }
2038
2039 public boolean isDone() {
2040 return nextIndexToProcess == operations.length;
2041 }
2042 }
2043
2044
2045
2046
2047
2048 public OperationStatus[] put(Put[] puts) throws IOException {
2049 @SuppressWarnings("unchecked")
2050 Pair<Mutation, Integer> putsAndLocks[] = new Pair[puts.length];
2051
2052 for (int i = 0; i < puts.length; i++) {
2053 putsAndLocks[i] = new Pair<Mutation, Integer>(puts[i], null);
2054 }
2055 return batchMutate(putsAndLocks);
2056 }
2057
2058
2059
2060
2061
2062
2063
2064
2065
2066
2067 @Deprecated
2068 public OperationStatus[] put(Pair<Put, Integer>[] putsAndLocks) throws IOException {
2069 Pair<Mutation, Integer>[] mutationsAndLocks = new Pair[putsAndLocks.length];
2070 System.arraycopy(putsAndLocks, 0, mutationsAndLocks, 0, putsAndLocks.length);
2071 return batchMutate(mutationsAndLocks);
2072 }
2073
2074
2075
2076
2077
2078
2079
2080
2081
2082
2083 public OperationStatus[] batchMutate(
2084 Pair<Mutation, Integer>[] mutationsAndLocks) throws IOException {
2085 BatchOperationInProgress<Pair<Mutation, Integer>> batchOp =
2086 new BatchOperationInProgress<Pair<Mutation,Integer>>(mutationsAndLocks);
2087
2088 boolean initialized = false;
2089
2090 while (!batchOp.isDone()) {
2091 checkReadOnly();
2092 checkResources();
2093
2094 long newSize;
2095 startRegionOperation();
2096
2097 try {
2098 if (!initialized) {
2099 this.writeRequestsCount.increment();
2100 this.opMetrics.setWriteRequestCountMetrics(this.writeRequestsCount.get());
2101 doPreMutationHook(batchOp);
2102 initialized = true;
2103 }
2104 long addedSize = doMiniBatchMutation(batchOp);
2105 newSize = this.addAndGetGlobalMemstoreSize(addedSize);
2106 } finally {
2107 closeRegionOperation();
2108 }
2109 if (isFlushSize(newSize)) {
2110 requestFlush();
2111 }
2112 }
2113 return batchOp.retCodeDetails;
2114 }
2115
2116 private void doPreMutationHook(BatchOperationInProgress<Pair<Mutation, Integer>> batchOp)
2117 throws IOException {
2118
2119 WALEdit walEdit = new WALEdit();
2120 if (coprocessorHost != null) {
2121 for (int i = 0; i < batchOp.operations.length; i++) {
2122 Pair<Mutation, Integer> nextPair = batchOp.operations[i];
2123 Mutation m = nextPair.getFirst();
2124 if (m instanceof Put) {
2125 if (coprocessorHost.prePut((Put) m, walEdit, m.getWriteToWAL())) {
2126
2127
2128 batchOp.retCodeDetails[i] = OperationStatus.SUCCESS;
2129 }
2130 } else if (m instanceof Delete) {
2131 if (coprocessorHost.preDelete((Delete) m, walEdit, m.getWriteToWAL())) {
2132
2133
2134 batchOp.retCodeDetails[i] = OperationStatus.SUCCESS;
2135 }
2136 } else {
2137
2138
2139
2140 batchOp.retCodeDetails[i] = new OperationStatus(OperationStatusCode.FAILURE,
2141 "Put/Delete mutations only supported in batchMutate() now");
2142 }
2143 if (!walEdit.isEmpty()) {
2144 batchOp.walEditsFromCoprocessors[i] = walEdit;
2145 walEdit = new WALEdit();
2146 }
2147 }
2148 }
2149 }
2150
2151
2152 @SuppressWarnings("unchecked")
2153 private long doMiniBatchMutation(
2154 BatchOperationInProgress<Pair<Mutation, Integer>> batchOp) throws IOException {
2155
2156
2157 Set<byte[]> putsCfSet = null;
2158
2159 boolean putsCfSetConsistent = true;
2160
2161 Set<byte[]> deletesCfSet = null;
2162
2163 boolean deletesCfSetConsistent = true;
2164 long startTimeMs = EnvironmentEdgeManager.currentTimeMillis();
2165
2166 WALEdit walEdit = new WALEdit();
2167
2168 MultiVersionConsistencyControl.WriteEntry w = null;
2169 long txid = 0;
2170 boolean walSyncSuccessful = false;
2171 boolean locked = false;
2172
2173
2174 List<Integer> acquiredLocks = Lists.newArrayListWithCapacity(batchOp.operations.length);
2175
2176 Map<byte[],List<KeyValue>>[] familyMaps = new Map[batchOp.operations.length];
2177
2178 int firstIndex = batchOp.nextIndexToProcess;
2179 int lastIndexExclusive = firstIndex;
2180 boolean success = false;
2181 int noOfPuts = 0, noOfDeletes = 0;
2182 try {
2183
2184
2185
2186
2187 int numReadyToWrite = 0;
2188 long now = EnvironmentEdgeManager.currentTimeMillis();
2189 while (lastIndexExclusive < batchOp.operations.length) {
2190 Pair<Mutation, Integer> nextPair = batchOp.operations[lastIndexExclusive];
2191 Mutation mutation = nextPair.getFirst();
2192 Integer providedLockId = nextPair.getSecond();
2193
2194 Map<byte[], List<KeyValue>> familyMap = mutation.getFamilyMap();
2195
2196 familyMaps[lastIndexExclusive] = familyMap;
2197
2198
2199 if (batchOp.retCodeDetails[lastIndexExclusive].getOperationStatusCode()
2200 != OperationStatusCode.NOT_RUN) {
2201 lastIndexExclusive++;
2202 continue;
2203 }
2204
2205 try {
2206 if (mutation instanceof Put) {
2207 checkFamilies(familyMap.keySet());
2208 checkTimestamps(mutation.getFamilyMap(), now);
2209 } else {
2210 prepareDelete((Delete) mutation);
2211 }
2212 } catch (NoSuchColumnFamilyException nscf) {
2213 LOG.warn("No such column family in batch mutation", nscf);
2214 batchOp.retCodeDetails[lastIndexExclusive] = new OperationStatus(
2215 OperationStatusCode.BAD_FAMILY, nscf.getMessage());
2216 lastIndexExclusive++;
2217 continue;
2218 } catch (DoNotRetryIOException fsce) {
2219
2220
2221
2222 LOG.warn("Batch Mutation did not pass sanity check", fsce);
2223 batchOp.retCodeDetails[lastIndexExclusive] = new OperationStatus(
2224 OperationStatusCode.SANITY_CHECK_FAILURE, fsce.getMessage());
2225 lastIndexExclusive++;
2226 continue;
2227 }
2228
2229
2230 boolean shouldBlock = numReadyToWrite == 0;
2231 Integer acquiredLockId = null;
2232 try {
2233 acquiredLockId = getLock(providedLockId, mutation.getRow(),
2234 shouldBlock);
2235 } catch (IOException ioe) {
2236 LOG.warn("Failed getting lock in batch put, row="
2237 + Bytes.toStringBinary(mutation.getRow()), ioe);
2238 }
2239 if (acquiredLockId == null) {
2240
2241 assert !shouldBlock : "Should never fail to get lock when blocking";
2242 break;
2243 }
2244 if (providedLockId == null) {
2245 acquiredLocks.add(acquiredLockId);
2246 }
2247 lastIndexExclusive++;
2248 numReadyToWrite++;
2249
2250 if (mutation instanceof Put) {
2251
2252
2253
2254 if (putsCfSet == null) {
2255 putsCfSet = mutation.getFamilyMap().keySet();
2256 } else {
2257 putsCfSetConsistent = putsCfSetConsistent
2258 && mutation.getFamilyMap().keySet().equals(putsCfSet);
2259 }
2260 } else {
2261 if (deletesCfSet == null) {
2262 deletesCfSet = mutation.getFamilyMap().keySet();
2263 } else {
2264 deletesCfSetConsistent = deletesCfSetConsistent
2265 && mutation.getFamilyMap().keySet().equals(deletesCfSet);
2266 }
2267 }
2268 }
2269
2270
2271
2272 now = EnvironmentEdgeManager.currentTimeMillis();
2273 byte[] byteNow = Bytes.toBytes(now);
2274
2275
2276 if (numReadyToWrite <= 0) return 0L;
2277
2278
2279
2280
2281
2282
2283 for (int i = firstIndex; i < lastIndexExclusive; i++) {
2284
2285 if (batchOp.retCodeDetails[i].getOperationStatusCode()
2286 != OperationStatusCode.NOT_RUN) continue;
2287 Mutation mutation = batchOp.operations[i].getFirst();
2288 if (mutation instanceof Put) {
2289 updateKVTimestamps(familyMaps[i].values(), byteNow);
2290 noOfPuts++;
2291 } else {
2292 prepareDeleteTimestamps(familyMaps[i], byteNow);
2293 noOfDeletes++;
2294 }
2295 }
2296
2297 lock(this.updatesLock.readLock(), numReadyToWrite);
2298 locked = true;
2299
2300
2301
2302
2303
2304 w = mvcc.beginMemstoreInsert();
2305
2306
2307 if (coprocessorHost != null) {
2308 MiniBatchOperationInProgress<Pair<Mutation, Integer>> miniBatchOp =
2309 new MiniBatchOperationInProgress<Pair<Mutation, Integer>>(batchOp.operations,
2310 batchOp.retCodeDetails, batchOp.walEditsFromCoprocessors, firstIndex, lastIndexExclusive);
2311 if (coprocessorHost.preBatchMutate(miniBatchOp)) return 0L;
2312 }
2313
2314
2315
2316
2317
2318
2319
2320
2321
2322
2323 long addedSize = 0;
2324 for (int i = firstIndex; i < lastIndexExclusive; i++) {
2325 if (batchOp.retCodeDetails[i].getOperationStatusCode()
2326 != OperationStatusCode.NOT_RUN) {
2327 continue;
2328 }
2329 addedSize += applyFamilyMapToMemstore(familyMaps[i], w);
2330 }
2331
2332
2333
2334
2335 for (int i = firstIndex; i < lastIndexExclusive; i++) {
2336
2337 if (batchOp.retCodeDetails[i].getOperationStatusCode()
2338 != OperationStatusCode.NOT_RUN) {
2339 continue;
2340 }
2341 batchOp.retCodeDetails[i] = OperationStatus.SUCCESS;
2342
2343 Mutation m = batchOp.operations[i].getFirst();
2344 if (!m.getWriteToWAL()) {
2345 if (m instanceof Put) {
2346 recordPutWithoutWal(m.getFamilyMap());
2347 }
2348 continue;
2349 }
2350
2351 WALEdit fromCP = batchOp.walEditsFromCoprocessors[i];
2352 if (fromCP != null) {
2353 for (KeyValue kv : fromCP.getKeyValues()) {
2354 walEdit.add(kv);
2355 }
2356 }
2357 addFamilyMapToWALEdit(familyMaps[i], walEdit);
2358 }
2359
2360
2361
2362
2363 Mutation first = batchOp.operations[firstIndex].getFirst();
2364 txid = this.log.appendNoSync(regionInfo, this.htableDescriptor.getName(),
2365 walEdit, first.getClusterId(), now, this.htableDescriptor);
2366
2367
2368
2369
2370 if (locked) {
2371 this.updatesLock.readLock().unlock();
2372 locked = false;
2373 }
2374 if (acquiredLocks != null) {
2375 for (Integer toRelease : acquiredLocks) {
2376 releaseRowLock(toRelease);
2377 }
2378 acquiredLocks = null;
2379 }
2380
2381
2382
2383 if (walEdit.size() > 0) {
2384 syncOrDefer(txid);
2385 }
2386 walSyncSuccessful = true;
2387
2388 if (coprocessorHost != null) {
2389 MiniBatchOperationInProgress<Pair<Mutation, Integer>> miniBatchOp =
2390 new MiniBatchOperationInProgress<Pair<Mutation, Integer>>(batchOp.operations,
2391 batchOp.retCodeDetails, batchOp.walEditsFromCoprocessors, firstIndex, lastIndexExclusive);
2392 coprocessorHost.postBatchMutate(miniBatchOp);
2393 }
2394
2395
2396
2397
2398 if (w != null) {
2399 mvcc.completeMemstoreInsert(w);
2400 w = null;
2401 }
2402
2403
2404
2405
2406
2407 if (coprocessorHost != null) {
2408 for (int i = firstIndex; i < lastIndexExclusive; i++) {
2409
2410 if (batchOp.retCodeDetails[i].getOperationStatusCode()
2411 != OperationStatusCode.SUCCESS) {
2412 continue;
2413 }
2414 Mutation m = batchOp.operations[i].getFirst();
2415 if (m instanceof Put) {
2416 coprocessorHost.postPut((Put) m, walEdit, m.getWriteToWAL());
2417 } else {
2418 coprocessorHost.postDelete((Delete) m, walEdit, m.getWriteToWAL());
2419 }
2420 }
2421 }
2422 success = true;
2423 return addedSize;
2424 } finally {
2425
2426
2427 if (!walSyncSuccessful) {
2428 rollbackMemstore(batchOp, familyMaps, firstIndex, lastIndexExclusive);
2429 }
2430 if (w != null) mvcc.completeMemstoreInsert(w);
2431
2432 if (locked) {
2433 this.updatesLock.readLock().unlock();
2434 }
2435
2436 if (acquiredLocks != null) {
2437 for (Integer toRelease : acquiredLocks) {
2438 releaseRowLock(toRelease);
2439 }
2440 }
2441
2442
2443 final long netTimeMs = EnvironmentEdgeManager.currentTimeMillis()- startTimeMs;
2444
2445
2446
2447
2448
2449
2450 long timeTakenForPuts = 0;
2451 if (noOfPuts > 0) {
2452
2453 double noOfMutations = noOfPuts + noOfDeletes;
2454 timeTakenForPuts = (long) (netTimeMs * (noOfPuts / noOfMutations));
2455 final Set<byte[]> keptCfs = putsCfSetConsistent ? putsCfSet : null;
2456 this.opMetrics.updateMultiPutMetrics(keptCfs, timeTakenForPuts);
2457 }
2458 if (noOfDeletes > 0) {
2459
2460 final Set<byte[]> keptCfs = deletesCfSetConsistent ? deletesCfSet : null;
2461 this.opMetrics.updateMultiDeleteMetrics(keptCfs, netTimeMs - timeTakenForPuts);
2462 }
2463 if (!success) {
2464 for (int i = firstIndex; i < lastIndexExclusive; i++) {
2465 if (batchOp.retCodeDetails[i].getOperationStatusCode() == OperationStatusCode.NOT_RUN) {
2466 batchOp.retCodeDetails[i] = OperationStatus.FAILURE;
2467 }
2468 }
2469 }
2470 batchOp.nextIndexToProcess = lastIndexExclusive;
2471 }
2472 }
2473
2474
2475
2476
2477
2478
2479
2480
2481
2482
2483
2484
2485
2486
2487
2488
2489 public boolean checkAndMutate(byte [] row, byte [] family, byte [] qualifier,
2490 CompareOp compareOp, WritableByteArrayComparable comparator, Writable w,
2491 boolean writeToWAL)
2492 throws IOException {
2493 return checkAndMutate(row, family, qualifier, compareOp, comparator, w, null, writeToWAL);
2494 }
2495
2496
2497
2498
2499
2500
2501
2502
2503
2504
2505
2506
2507
2508
2509 public boolean checkAndMutate(byte [] row, byte [] family, byte [] qualifier,
2510 CompareOp compareOp, WritableByteArrayComparable comparator, Writable w,
2511 Integer lockId, boolean writeToWAL)
2512 throws IOException{
2513 checkReadOnly();
2514
2515
2516 checkResources();
2517 boolean isPut = w instanceof Put;
2518 if (!isPut && !(w instanceof Delete))
2519 throw new DoNotRetryIOException("Action must be Put or Delete");
2520 Row r = (Row)w;
2521 if (!Bytes.equals(row, r.getRow())) {
2522 throw new DoNotRetryIOException("Action's getRow must match the passed row");
2523 }
2524
2525 startRegionOperation();
2526 this.writeRequestsCount.increment();
2527 this.opMetrics.setWriteRequestCountMetrics(this.writeRequestsCount.get());
2528 try {
2529 RowLock lock = isPut ? ((Put)w).getRowLock() : ((Delete)w).getRowLock();
2530 Get get = new Get(row, lock);
2531 checkFamily(family);
2532 get.addColumn(family, qualifier);
2533
2534
2535 Integer lid = getLock(lockId, get.getRow(), true);
2536
2537 mvcc.completeMemstoreInsert(mvcc.beginMemstoreInsert());
2538 List<KeyValue> result = new ArrayList<KeyValue>();
2539 try {
2540 result = get(get, false);
2541
2542 boolean valueIsNull = comparator.getValue() == null ||
2543 comparator.getValue().length == 0;
2544 boolean matches = false;
2545 if (result.size() == 0 && valueIsNull) {
2546 matches = true;
2547 } else if (result.size() > 0 && result.get(0).getValue().length == 0 &&
2548 valueIsNull) {
2549 matches = true;
2550 } else if (result.size() == 1 && !valueIsNull) {
2551 KeyValue kv = result.get(0);
2552 int compareResult = comparator.compareTo(kv.getBuffer(),
2553 kv.getValueOffset(), kv.getValueLength());
2554 switch (compareOp) {
2555 case LESS:
2556 matches = compareResult <= 0;
2557 break;
2558 case LESS_OR_EQUAL:
2559 matches = compareResult < 0;
2560 break;
2561 case EQUAL:
2562 matches = compareResult == 0;
2563 break;
2564 case NOT_EQUAL:
2565 matches = compareResult != 0;
2566 break;
2567 case GREATER_OR_EQUAL:
2568 matches = compareResult > 0;
2569 break;
2570 case GREATER:
2571 matches = compareResult >= 0;
2572 break;
2573 default:
2574 throw new RuntimeException("Unknown Compare op " + compareOp.name());
2575 }
2576 }
2577
2578 if (matches) {
2579
2580
2581
2582
2583
2584
2585 if (isPut) {
2586 internalPut(((Put) w), HConstants.DEFAULT_CLUSTER_ID, writeToWAL);
2587 } else {
2588 Delete d = (Delete)w;
2589 prepareDelete(d);
2590 internalDelete(d, HConstants.DEFAULT_CLUSTER_ID, writeToWAL);
2591 }
2592 return true;
2593 }
2594 return false;
2595 } finally {
2596 if(lockId == null) releaseRowLock(lid);
2597 }
2598 } finally {
2599 closeRegionOperation();
2600 }
2601 }
2602
2603
2604
2605
2606
2607
2608
2609
2610
2611
2612
2613
2614
2615
2616
2617 public void addRegionToSnapshot(SnapshotDescription desc,
2618 ForeignExceptionSnare exnSnare) throws IOException {
2619
2620
2621 Path rootDir = FSUtils.getRootDir(this.rsServices.getConfiguration());
2622 Path snapshotRegionDir = TakeSnapshotUtils.getRegionSnapshotDirectory(desc, rootDir,
2623 regionInfo.getEncodedName());
2624
2625
2626 LOG.debug("Storing region-info for snapshot.");
2627 checkRegioninfoOnFilesystem(snapshotRegionDir);
2628
2629
2630 LOG.debug("Creating references for hfiles");
2631
2632
2633
2634
2635
2636
2637 for (Store store : stores.values()) {
2638
2639 Path dstStoreDir = TakeSnapshotUtils.getStoreSnapshotDirectory(snapshotRegionDir,
2640 Bytes.toString(store.getFamily().getName()));
2641 List<StoreFile> storeFiles = store.getStorefiles();
2642 if (LOG.isDebugEnabled()) {
2643 LOG.debug("Adding snapshot references for " + storeFiles + " hfiles");
2644 }
2645
2646
2647 int sz = storeFiles.size();
2648 for (int i = 0; i < sz; i++) {
2649 if (exnSnare != null) {
2650 exnSnare.rethrowException();
2651 }
2652 Path file = storeFiles.get(i).getPath();
2653
2654
2655
2656
2657 LOG.debug("Creating reference for file (" + (i+1) + "/" + sz + ") : " + file);
2658 Path referenceFile = new Path(dstStoreDir, file.getName());
2659 boolean success = fs.createNewFile(referenceFile);
2660 if (!success) {
2661 throw new IOException("Failed to create reference file:" + referenceFile);
2662 }
2663 }
2664 }
2665 }
2666
2667
2668
2669
2670
2671 private void updateKVTimestamps(
2672 final Iterable<List<KeyValue>> keyLists, final byte[] now) {
2673 for (List<KeyValue> keys: keyLists) {
2674 if (keys == null) continue;
2675 for (KeyValue key : keys) {
2676 key.updateLatestStamp(now);
2677 }
2678 }
2679 }
2680
2681
2682
2683
2684
2685
2686
2687
2688
2689
2690 private void checkResources()
2691 throws RegionTooBusyException, InterruptedIOException {
2692
2693
2694 if (this.getRegionInfo().isMetaRegion()) return;
2695
2696 boolean blocked = false;
2697 long startTime = 0;
2698 while (this.memstoreSize.get() > this.blockingMemStoreSize) {
2699 requestFlush();
2700 if (!blocked) {
2701 startTime = EnvironmentEdgeManager.currentTimeMillis();
2702 LOG.info("Blocking updates for '" + Thread.currentThread().getName() +
2703 "' on region " + Bytes.toStringBinary(getRegionName()) +
2704 ": memstore size " +
2705 StringUtils.humanReadableInt(this.memstoreSize.get()) +
2706 " is >= than blocking " +
2707 StringUtils.humanReadableInt(this.blockingMemStoreSize) + " size");
2708 }
2709 long now = EnvironmentEdgeManager.currentTimeMillis();
2710 long timeToWait = startTime + busyWaitDuration - now;
2711 if (timeToWait <= 0L) {
2712 final long totalTime = now - startTime;
2713 this.updatesBlockedMs.add(totalTime);
2714 LOG.info("Failed to unblock updates for region " + this + " '"
2715 + Thread.currentThread().getName() + "' in " + totalTime
2716 + "ms. The region is still busy.");
2717 throw new RegionTooBusyException("region is flushing");
2718 }
2719 blocked = true;
2720 synchronized(this) {
2721 try {
2722 wait(Math.min(timeToWait, threadWakeFrequency));
2723 } catch (InterruptedException ie) {
2724 final long totalTime = EnvironmentEdgeManager.currentTimeMillis() - startTime;
2725 if (totalTime > 0) {
2726 this.updatesBlockedMs.add(totalTime);
2727 }
2728 LOG.info("Interrupted while waiting to unblock updates for region "
2729 + this + " '" + Thread.currentThread().getName() + "'");
2730 InterruptedIOException iie = new InterruptedIOException();
2731 iie.initCause(ie);
2732 throw iie;
2733 }
2734 }
2735 }
2736 if (blocked) {
2737
2738 final long totalTime = EnvironmentEdgeManager.currentTimeMillis() - startTime;
2739 if(totalTime > 0 ){
2740 this.updatesBlockedMs.add(totalTime);
2741 }
2742 LOG.info("Unblocking updates for region " + this + " '"
2743 + Thread.currentThread().getName() + "'");
2744 }
2745 }
2746
2747
2748
2749
2750 protected void checkReadOnly() throws IOException {
2751 if (this.writestate.isReadOnly()) {
2752 throw new IOException("region is read only");
2753 }
2754 }
2755
2756
2757
2758
2759
2760
2761
2762
2763
2764 private void put(byte [] family, List<KeyValue> edits)
2765 throws IOException {
2766 Map<byte[], List<KeyValue>> familyMap;
2767 familyMap = new HashMap<byte[], List<KeyValue>>();
2768
2769 familyMap.put(family, edits);
2770 Put p = new Put();
2771 p.setFamilyMap(familyMap);
2772 p.setClusterId(HConstants.DEFAULT_CLUSTER_ID);
2773 p.setWriteToWAL(true);
2774 this.internalPut(p, HConstants.DEFAULT_CLUSTER_ID, true);
2775 }
2776
2777
2778
2779
2780
2781
2782
2783
2784
2785 private void internalPut(Put put, UUID clusterId, boolean writeToWAL) throws IOException {
2786 Map<byte[], List<KeyValue>> familyMap = put.getFamilyMap();
2787 WALEdit walEdit = new WALEdit();
2788
2789 if (coprocessorHost != null) {
2790 if (coprocessorHost.prePut(put, walEdit, writeToWAL)) {
2791 return;
2792 }
2793 }
2794
2795 long now = EnvironmentEdgeManager.currentTimeMillis();
2796 byte[] byteNow = Bytes.toBytes(now);
2797 boolean flush = false;
2798
2799 lock(this.updatesLock.readLock());
2800 try {
2801 checkFamilies(familyMap.keySet());
2802 checkTimestamps(familyMap, now);
2803 updateKVTimestamps(familyMap.values(), byteNow);
2804
2805
2806
2807
2808
2809 if (writeToWAL) {
2810 addFamilyMapToWALEdit(familyMap, walEdit);
2811 this.log.append(regionInfo, this.htableDescriptor.getName(),
2812 walEdit, clusterId, now, this.htableDescriptor);
2813 } else {
2814 recordPutWithoutWal(familyMap);
2815 }
2816
2817 long addedSize = applyFamilyMapToMemstore(familyMap, null);
2818 flush = isFlushSize(this.addAndGetGlobalMemstoreSize(addedSize));
2819 } finally {
2820 this.updatesLock.readLock().unlock();
2821 }
2822
2823 if (coprocessorHost != null) {
2824 coprocessorHost.postPut(put, walEdit, writeToWAL);
2825 }
2826
2827
2828 final long after = EnvironmentEdgeManager.currentTimeMillis();
2829 this.opMetrics.updatePutMetrics(familyMap.keySet(), after - now);
2830
2831 if (flush) {
2832
2833 requestFlush();
2834 }
2835 }
2836
2837
2838
2839
2840
2841
2842
2843
2844
2845
2846
2847
2848
2849 private long applyFamilyMapToMemstore(Map<byte[], List<KeyValue>> familyMap,
2850 MultiVersionConsistencyControl.WriteEntry localizedWriteEntry) {
2851 long size = 0;
2852 boolean freemvcc = false;
2853
2854 try {
2855 if (localizedWriteEntry == null) {
2856 localizedWriteEntry = mvcc.beginMemstoreInsert();
2857 freemvcc = true;
2858 }
2859
2860 for (Map.Entry<byte[], List<KeyValue>> e : familyMap.entrySet()) {
2861 byte[] family = e.getKey();
2862 List<KeyValue> edits = e.getValue();
2863
2864 Store store = getStore(family);
2865 for (KeyValue kv: edits) {
2866 kv.setMemstoreTS(localizedWriteEntry.getWriteNumber());
2867 size += store.add(kv);
2868 }
2869 }
2870 } finally {
2871 if (freemvcc) {
2872 mvcc.completeMemstoreInsert(localizedWriteEntry);
2873 }
2874 }
2875
2876 return size;
2877 }
2878
2879
2880
2881
2882
2883
2884 private void rollbackMemstore(BatchOperationInProgress<Pair<Mutation, Integer>> batchOp,
2885 Map<byte[], List<KeyValue>>[] familyMaps,
2886 int start, int end) {
2887 int kvsRolledback = 0;
2888 for (int i = start; i < end; i++) {
2889
2890 if (batchOp.retCodeDetails[i].getOperationStatusCode()
2891 != OperationStatusCode.SUCCESS) {
2892 continue;
2893 }
2894
2895
2896 Map<byte[], List<KeyValue>> familyMap = familyMaps[i];
2897 for (Map.Entry<byte[], List<KeyValue>> e : familyMap.entrySet()) {
2898 byte[] family = e.getKey();
2899 List<KeyValue> edits = e.getValue();
2900
2901
2902
2903
2904 Store store = getStore(family);
2905 for (KeyValue kv: edits) {
2906 store.rollback(kv);
2907 kvsRolledback++;
2908 }
2909 }
2910 }
2911 LOG.debug("rollbackMemstore rolled back " + kvsRolledback +
2912 " keyvalues from start:" + start + " to end:" + end);
2913 }
2914
2915
2916
2917
2918
2919 private void checkFamilies(Collection<byte[]> families)
2920 throws NoSuchColumnFamilyException {
2921 for (byte[] family : families) {
2922 checkFamily(family);
2923 }
2924 }
2925
2926 private void checkTimestamps(final Map<byte[], List<KeyValue>> familyMap,
2927 long now) throws DoNotRetryIOException {
2928 if (timestampSlop == HConstants.LATEST_TIMESTAMP) {
2929 return;
2930 }
2931 long maxTs = now + timestampSlop;
2932 for (List<KeyValue> kvs : familyMap.values()) {
2933 for (KeyValue kv : kvs) {
2934
2935 if (!kv.isLatestTimestamp() && kv.getTimestamp() > maxTs) {
2936 throw new DoNotRetryIOException("Timestamp for KV out of range "
2937 + kv + " (too.new=" + timestampSlop + ")");
2938 }
2939 }
2940 }
2941 }
2942
2943
2944
2945
2946
2947
2948
2949 private void addFamilyMapToWALEdit(Map<byte[], List<KeyValue>> familyMap,
2950 WALEdit walEdit) {
2951 for (List<KeyValue> edits : familyMap.values()) {
2952 for (KeyValue kv : edits) {
2953 walEdit.add(kv);
2954 }
2955 }
2956 }
2957
2958 private void requestFlush() {
2959 if (this.rsServices == null) {
2960 return;
2961 }
2962 synchronized (writestate) {
2963 if (this.writestate.isFlushRequested()) {
2964 return;
2965 }
2966 writestate.flushRequested = true;
2967 }
2968
2969 this.rsServices.getFlushRequester().requestFlush(this);
2970 if (LOG.isDebugEnabled()) {
2971 LOG.debug("Flush requested on " + this);
2972 }
2973 }
2974
2975
2976
2977
2978
2979 private boolean isFlushSize(final long size) {
2980 return size > this.memstoreFlushSize;
2981 }
2982
2983
2984
2985
2986
2987
2988
2989
2990
2991
2992
2993
2994
2995
2996
2997
2998
2999
3000
3001
3002
3003
3004
3005
3006
3007
3008
3009
3010
3011
3012
3013
3014
3015
3016
3017
3018
3019 protected long replayRecoveredEditsIfAny(final Path regiondir,
3020 final long minSeqId, final CancelableProgressable reporter,
3021 final MonitoredTask status)
3022 throws UnsupportedEncodingException, IOException {
3023 long seqid = minSeqId;
3024 NavigableSet<Path> files = HLog.getSplitEditFilesSorted(this.fs, regiondir);
3025 if (files == null || files.isEmpty()) return seqid;
3026 for (Path edits: files) {
3027 if (edits == null || !this.fs.exists(edits)) {
3028 LOG.warn("Null or non-existent edits file: " + edits);
3029 continue;
3030 }
3031 if (isZeroLengthThenDelete(this.fs, edits)) continue;
3032
3033 long maxSeqId = Long.MAX_VALUE;
3034 String fileName = edits.getName();
3035 maxSeqId = Math.abs(Long.parseLong(fileName));
3036 if (maxSeqId <= minSeqId) {
3037 String msg = "Maximum sequenceid for this log is " + maxSeqId
3038 + " and minimum sequenceid for the region is " + minSeqId
3039 + ", skipped the whole file, path=" + edits;
3040 LOG.debug(msg);
3041 continue;
3042 }
3043
3044 try {
3045 seqid = replayRecoveredEdits(edits, seqid, reporter);
3046 } catch (IOException e) {
3047 boolean skipErrors = conf.getBoolean("hbase.skip.errors", false);
3048 if (skipErrors) {
3049 Path p = HLog.moveAsideBadEditsFile(fs, edits);
3050 LOG.error("hbase.skip.errors=true so continuing. Renamed " + edits +
3051 " as " + p, e);
3052 } else {
3053 throw e;
3054 }
3055 }
3056
3057
3058 if (this.rsAccounting != null) {
3059 this.rsAccounting.clearRegionReplayEditsSize(this.regionInfo.getRegionName());
3060 }
3061 }
3062 if (seqid > minSeqId) {
3063
3064 internalFlushcache(null, seqid, status);
3065 }
3066
3067 for (Path file: files) {
3068 if (!this.fs.delete(file, false)) {
3069 LOG.error("Failed delete of " + file);
3070 } else {
3071 LOG.debug("Deleted recovered.edits file=" + file);
3072 }
3073 }
3074 return seqid;
3075 }
3076
3077
3078
3079
3080
3081
3082
3083
3084
3085
3086 private long replayRecoveredEdits(final Path edits,
3087 final long minSeqId, final CancelableProgressable reporter)
3088 throws IOException {
3089 String msg = "Replaying edits from " + edits + "; minSequenceid=" +
3090 minSeqId + "; path=" + edits;
3091 LOG.info(msg);
3092 MonitoredTask status = TaskMonitor.get().createStatus(msg);
3093
3094 status.setStatus("Opening logs");
3095 HLog.Reader reader = null;
3096 try {
3097 reader = HLog.getReader(this.fs, edits, conf);
3098 long currentEditSeqId = minSeqId;
3099 long firstSeqIdInLog = -1;
3100 long skippedEdits = 0;
3101 long editsCount = 0;
3102 long intervalEdits = 0;
3103 HLog.Entry entry;
3104 Store store = null;
3105 boolean reported_once = false;
3106
3107 try {
3108
3109 int interval = this.conf.getInt("hbase.hstore.report.interval.edits",
3110 2000);
3111
3112 int period = this.conf.getInt("hbase.hstore.report.period",
3113 this.conf.getInt("hbase.master.assignment.timeoutmonitor.timeout",
3114 180000) / 2);
3115 long lastReport = EnvironmentEdgeManager.currentTimeMillis();
3116
3117 while ((entry = reader.next()) != null) {
3118 HLogKey key = entry.getKey();
3119 WALEdit val = entry.getEdit();
3120
3121 if (reporter != null) {
3122 intervalEdits += val.size();
3123 if (intervalEdits >= interval) {
3124
3125 intervalEdits = 0;
3126 long cur = EnvironmentEdgeManager.currentTimeMillis();
3127 if (lastReport + period <= cur) {
3128 status.setStatus("Replaying edits..." +
3129 " skipped=" + skippedEdits +
3130 " edits=" + editsCount);
3131
3132 if(!reporter.progress()) {
3133 msg = "Progressable reporter failed, stopping replay";
3134 LOG.warn(msg);
3135 status.abort(msg);
3136 throw new IOException(msg);
3137 }
3138 reported_once = true;
3139 lastReport = cur;
3140 }
3141 }
3142 }
3143
3144
3145
3146 if (coprocessorHost != null) {
3147 status.setStatus("Running pre-WAL-restore hook in coprocessors");
3148 if (coprocessorHost.preWALRestore(this.getRegionInfo(), key, val)) {
3149
3150 continue;
3151 }
3152 }
3153
3154 if (firstSeqIdInLog == -1) {
3155 firstSeqIdInLog = key.getLogSeqNum();
3156 }
3157
3158 if (key.getLogSeqNum() <= currentEditSeqId) {
3159 skippedEdits++;
3160 continue;
3161 }
3162 currentEditSeqId = key.getLogSeqNum();
3163 boolean flush = false;
3164 for (KeyValue kv: val.getKeyValues()) {
3165
3166
3167 if (kv.matchingFamily(HLog.METAFAMILY) ||
3168 !Bytes.equals(key.getEncodedRegionName(), this.regionInfo.getEncodedNameAsBytes())) {
3169 skippedEdits++;
3170 continue;
3171 }
3172
3173 if (store == null || !kv.matchingFamily(store.getFamily().getName())) {
3174 store = this.stores.get(kv.getFamily());
3175 }
3176 if (store == null) {
3177
3178
3179 LOG.warn("No family for " + kv);
3180 skippedEdits++;
3181 continue;
3182 }
3183
3184
3185
3186 flush = restoreEdit(store, kv);
3187 editsCount++;
3188 }
3189 if (flush) internalFlushcache(null, currentEditSeqId, status);
3190
3191 if (coprocessorHost != null) {
3192 coprocessorHost.postWALRestore(this.getRegionInfo(), key, val);
3193 }
3194 }
3195 } catch (EOFException eof) {
3196 Path p = HLog.moveAsideBadEditsFile(fs, edits);
3197 msg = "Encountered EOF. Most likely due to Master failure during " +
3198 "log spliting, so we have this data in another edit. " +
3199 "Continuing, but renaming " + edits + " as " + p;
3200 LOG.warn(msg, eof);
3201 status.abort(msg);
3202 } catch (IOException ioe) {
3203
3204
3205 if (ioe.getCause() instanceof ParseException) {
3206 Path p = HLog.moveAsideBadEditsFile(fs, edits);
3207 msg = "File corruption encountered! " +
3208 "Continuing, but renaming " + edits + " as " + p;
3209 LOG.warn(msg, ioe);
3210 status.setStatus(msg);
3211 } else {
3212 status.abort(StringUtils.stringifyException(ioe));
3213
3214
3215 throw ioe;
3216 }
3217 }
3218 if (reporter != null && !reported_once) {
3219 reporter.progress();
3220 }
3221 msg = "Applied " + editsCount + ", skipped " + skippedEdits +
3222 ", firstSequenceidInLog=" + firstSeqIdInLog +
3223 ", maxSequenceidInLog=" + currentEditSeqId + ", path=" + edits;
3224 status.markComplete(msg);
3225 LOG.debug(msg);
3226 return currentEditSeqId;
3227 } finally {
3228 status.cleanup();
3229 if (reader != null) {
3230 reader.close();
3231 }
3232 }
3233 }
3234
3235
3236
3237
3238
3239
3240
3241 protected boolean restoreEdit(final Store s, final KeyValue kv) {
3242 long kvSize = s.add(kv);
3243 if (this.rsAccounting != null) {
3244 rsAccounting.addAndGetRegionReplayEditsSize(this.regionInfo.getRegionName(), kvSize);
3245 }
3246 return isFlushSize(this.addAndGetGlobalMemstoreSize(kvSize));
3247 }
3248
3249
3250
3251
3252
3253
3254
3255 private static boolean isZeroLengthThenDelete(final FileSystem fs, final Path p)
3256 throws IOException {
3257 FileStatus stat = fs.getFileStatus(p);
3258 if (stat.getLen() > 0) return false;
3259 LOG.warn("File " + p + " is zero-length, deleting.");
3260 fs.delete(p, false);
3261 return true;
3262 }
3263
3264 protected Store instantiateHStore(Path tableDir, HColumnDescriptor c)
3265 throws IOException {
3266 return new Store(tableDir, this, c, this.fs, this.conf);
3267 }
3268
3269
3270
3271
3272
3273
3274
3275
3276 public Store getStore(final byte [] column) {
3277 return this.stores.get(column);
3278 }
3279
3280 public Map<byte[], Store> getStores() {
3281 return this.stores;
3282 }
3283
3284
3285
3286
3287
3288
3289
3290
3291 public List<String> getStoreFileList(final byte [][] columns)
3292 throws IllegalArgumentException {
3293 List<String> storeFileNames = new ArrayList<String>();
3294 synchronized(closeLock) {
3295 for(byte[] column : columns) {
3296 Store store = this.stores.get(column);
3297 if (store == null) {
3298 throw new IllegalArgumentException("No column family : " +
3299 new String(column) + " available");
3300 }
3301 List<StoreFile> storeFiles = store.getStorefiles();
3302 for (StoreFile storeFile: storeFiles) {
3303 storeFileNames.add(storeFile.getPath().toString());
3304 }
3305 }
3306 }
3307 return storeFileNames;
3308 }
3309
3310
3311
3312
3313
3314 void checkRow(final byte [] row, String op) throws IOException {
3315 if(!rowIsInRange(regionInfo, row)) {
3316 throw new WrongRegionException("Requested row out of range for " +
3317 op + " on HRegion " + this + ", startKey='" +
3318 Bytes.toStringBinary(regionInfo.getStartKey()) + "', getEndKey()='" +
3319 Bytes.toStringBinary(regionInfo.getEndKey()) + "', row='" +
3320 Bytes.toStringBinary(row) + "'");
3321 }
3322 }
3323
3324
3325
3326
3327
3328
3329
3330
3331
3332
3333
3334
3335
3336
3337
3338
3339
3340
3341
3342
3343
3344
3345
3346
3347 public Integer obtainRowLock(final byte [] row) throws IOException {
3348 startRegionOperation();
3349 this.writeRequestsCount.increment();
3350 this.opMetrics.setWriteRequestCountMetrics( this.writeRequestsCount.get());
3351 try {
3352 return internalObtainRowLock(row, true);
3353 } finally {
3354 closeRegionOperation();
3355 }
3356 }
3357
3358
3359
3360
3361
3362
3363
3364 private Integer internalObtainRowLock(final byte[] row, boolean waitForLock)
3365 throws IOException {
3366 checkRow(row, "row lock");
3367 startRegionOperation();
3368 try {
3369 HashedBytes rowKey = new HashedBytes(row);
3370 CountDownLatch rowLatch = new CountDownLatch(1);
3371
3372
3373 while (true) {
3374 CountDownLatch existingLatch = lockedRows.putIfAbsent(rowKey, rowLatch);
3375 if (existingLatch == null) {
3376 break;
3377 } else {
3378
3379 if (!waitForLock) {
3380 return null;
3381 }
3382 try {
3383 if (!existingLatch.await(this.rowLockWaitDuration,
3384 TimeUnit.MILLISECONDS)) {
3385 throw new IOException("Timed out on getting lock for row="
3386 + Bytes.toStringBinary(row));
3387 }
3388 } catch (InterruptedException ie) {
3389
3390 }
3391 }
3392 }
3393
3394
3395 while (true) {
3396 Integer lockId = lockIdGenerator.incrementAndGet();
3397 HashedBytes existingRowKey = lockIds.putIfAbsent(lockId, rowKey);
3398 if (existingRowKey == null) {
3399 return lockId;
3400 } else {
3401
3402 lockIdGenerator.set(rand.nextInt());
3403 }
3404 }
3405 } finally {
3406 closeRegionOperation();
3407 }
3408 }
3409
3410
3411
3412
3413
3414
3415 byte[] getRowFromLock(final Integer lockid) {
3416 HashedBytes rowKey = lockIds.get(lockid);
3417 return rowKey == null ? null : rowKey.getBytes();
3418 }
3419
3420
3421
3422
3423
3424 public void releaseRowLock(final Integer lockId) {
3425 if (lockId == null) return;
3426 HashedBytes rowKey = lockIds.remove(lockId);
3427 if (rowKey == null) {
3428 LOG.warn("Release unknown lockId: " + lockId);
3429 return;
3430 }
3431 CountDownLatch rowLatch = lockedRows.remove(rowKey);
3432 if (rowLatch == null) {
3433 LOG.error("Releases row not locked, lockId: " + lockId + " row: "
3434 + rowKey);
3435 return;
3436 }
3437 rowLatch.countDown();
3438 }
3439
3440
3441
3442
3443
3444
3445 boolean isRowLocked(final Integer lockId) {
3446 return lockIds.containsKey(lockId);
3447 }
3448
3449
3450
3451
3452
3453
3454
3455
3456
3457
3458 public Integer getLock(Integer lockid, byte [] row, boolean waitForLock)
3459 throws IOException {
3460 Integer lid = null;
3461 if (lockid == null) {
3462 lid = internalObtainRowLock(row, waitForLock);
3463 } else {
3464 if (!isRowLocked(lockid)) {
3465 throw new IOException("Invalid row lock");
3466 }
3467 lid = lockid;
3468 }
3469 return lid;
3470 }
3471
3472
3473
3474
3475
3476
3477
3478 private static boolean hasMultipleColumnFamilies(
3479 List<Pair<byte[], String>> familyPaths) {
3480 boolean multipleFamilies = false;
3481 byte[] family = null;
3482 for (Pair<byte[], String> pair : familyPaths) {
3483 byte[] fam = pair.getFirst();
3484 if (family == null) {
3485 family = fam;
3486 } else if (!Bytes.equals(family, fam)) {
3487 multipleFamilies = true;
3488 break;
3489 }
3490 }
3491 return multipleFamilies;
3492 }
3493
3494
3495
3496
3497
3498
3499
3500
3501
3502 public boolean bulkLoadHFiles(List<Pair<byte[], String>> familyPaths) throws IOException {
3503 return bulkLoadHFiles(familyPaths, null);
3504 }
3505
3506
3507
3508
3509
3510
3511
3512
3513
3514
3515
3516 public boolean bulkLoadHFiles(List<Pair<byte[], String>> familyPaths,
3517 BulkLoadListener bulkLoadListener) throws IOException {
3518 Preconditions.checkNotNull(familyPaths);
3519
3520 startBulkRegionOperation(hasMultipleColumnFamilies(familyPaths));
3521 try {
3522 this.writeRequestsCount.increment();
3523 this.opMetrics.setWriteRequestCountMetrics( this.writeRequestsCount.get());
3524
3525
3526
3527
3528 List<IOException> ioes = new ArrayList<IOException>();
3529 List<Pair<byte[], String>> failures = new ArrayList<Pair<byte[], String>>();
3530 for (Pair<byte[], String> p : familyPaths) {
3531 byte[] familyName = p.getFirst();
3532 String path = p.getSecond();
3533
3534 Store store = getStore(familyName);
3535 if (store == null) {
3536 IOException ioe = new DoNotRetryIOException(
3537 "No such column family " + Bytes.toStringBinary(familyName));
3538 ioes.add(ioe);
3539 failures.add(p);
3540 } else {
3541 try {
3542 store.assertBulkLoadHFileOk(new Path(path));
3543 } catch (WrongRegionException wre) {
3544
3545 failures.add(p);
3546 } catch (IOException ioe) {
3547
3548 ioes.add(ioe);
3549 }
3550 }
3551 }
3552
3553
3554 if (failures.size() != 0) {
3555 StringBuilder list = new StringBuilder();
3556 for (Pair<byte[], String> p : failures) {
3557 list.append("\n").append(Bytes.toString(p.getFirst())).append(" : ")
3558 .append(p.getSecond());
3559 }
3560
3561 LOG.warn("There was a recoverable bulk load failure likely due to a" +
3562 " split. These (family, HFile) pairs were not loaded: " + list);
3563 return false;
3564 }
3565
3566
3567 if (ioes.size() != 0) {
3568 LOG.error("There were IO errors when checking if bulk load is ok. " +
3569 "throwing exception!");
3570 throw MultipleIOException.createIOException(ioes);
3571 }
3572
3573 for (Pair<byte[], String> p : familyPaths) {
3574 byte[] familyName = p.getFirst();
3575 String path = p.getSecond();
3576 Store store = getStore(familyName);
3577 try {
3578 String finalPath = path;
3579 if(bulkLoadListener != null) {
3580 finalPath = bulkLoadListener.prepareBulkLoad(familyName, path);
3581 }
3582 store.bulkLoadHFile(finalPath);
3583 if(bulkLoadListener != null) {
3584 bulkLoadListener.doneBulkLoad(familyName, path);
3585 }
3586 } catch (IOException ioe) {
3587
3588
3589
3590
3591 LOG.error("There was a partial failure due to IO when attempting to" +
3592 " load " + Bytes.toString(p.getFirst()) + " : "+ p.getSecond());
3593 if(bulkLoadListener != null) {
3594 try {
3595 bulkLoadListener.failedBulkLoad(familyName, path);
3596 } catch (Exception ex) {
3597 LOG.error("Error while calling failedBulkLoad for family "+
3598 Bytes.toString(familyName)+" with path "+path, ex);
3599 }
3600 }
3601 throw ioe;
3602 }
3603 }
3604 return true;
3605 } finally {
3606 closeBulkRegionOperation();
3607 }
3608 }
3609
3610 @Override
3611 public boolean equals(Object o) {
3612 if (!(o instanceof HRegion)) {
3613 return false;
3614 }
3615 return Bytes.equals(this.getRegionName(), ((HRegion) o).getRegionName());
3616 }
3617
3618 @Override
3619 public int hashCode() {
3620 return Bytes.hashCode(this.getRegionName());
3621 }
3622
3623 @Override
3624 public String toString() {
3625 return this.regionInfo.getRegionNameAsString();
3626 }
3627
3628
3629 public Path getTableDir() {
3630 return this.tableDir;
3631 }
3632
3633
3634
3635
3636 class RegionScannerImpl implements RegionScanner {
3637
3638 KeyValueHeap storeHeap = null;
3639
3640
3641 KeyValueHeap joinedHeap = null;
3642
3643
3644
3645 private KeyValue joinedContinuationRow = null;
3646
3647 private final KeyValue KV_LIMIT = new KeyValue();
3648 private final byte [] stopRow;
3649 private Filter filter;
3650 private List<KeyValue> results = new ArrayList<KeyValue>();
3651 private int batch;
3652 private int isScan;
3653 private boolean filterClosed = false;
3654 private long readPt;
3655 private HRegion region;
3656
3657 public HRegionInfo getRegionInfo() {
3658 return regionInfo;
3659 }
3660
3661 RegionScannerImpl(Scan scan, List<KeyValueScanner> additionalScanners, HRegion region)
3662 throws IOException {
3663
3664 this.region = region;
3665 this.filter = scan.getFilter();
3666 this.batch = scan.getBatch();
3667 if (Bytes.equals(scan.getStopRow(), HConstants.EMPTY_END_ROW)) {
3668 this.stopRow = null;
3669 } else {
3670 this.stopRow = scan.getStopRow();
3671 }
3672
3673
3674 this.isScan = scan.isGetScan() ? -1 : 0;
3675
3676
3677
3678 IsolationLevel isolationLevel = scan.getIsolationLevel();
3679 synchronized(scannerReadPoints) {
3680 if (isolationLevel == IsolationLevel.READ_UNCOMMITTED) {
3681
3682 this.readPt = Long.MAX_VALUE;
3683 MultiVersionConsistencyControl.setThreadReadPoint(this.readPt);
3684 } else {
3685 this.readPt = MultiVersionConsistencyControl.resetThreadReadPoint(mvcc);
3686 }
3687 scannerReadPoints.put(this, this.readPt);
3688 }
3689
3690
3691
3692 List<KeyValueScanner> scanners = new ArrayList<KeyValueScanner>();
3693 List<KeyValueScanner> joinedScanners = new ArrayList<KeyValueScanner>();
3694 if (additionalScanners != null) {
3695 scanners.addAll(additionalScanners);
3696 }
3697
3698 for (Map.Entry<byte[], NavigableSet<byte[]>> entry :
3699 scan.getFamilyMap().entrySet()) {
3700 Store store = stores.get(entry.getKey());
3701 KeyValueScanner scanner = store.getScanner(scan, entry.getValue());
3702 if (this.filter == null || !scan.doLoadColumnFamiliesOnDemand()
3703 || FilterBase.isFamilyEssential(this.filter, entry.getKey())) {
3704 scanners.add(scanner);
3705 } else {
3706 joinedScanners.add(scanner);
3707 }
3708 }
3709 this.storeHeap = new KeyValueHeap(scanners, comparator);
3710 if (!joinedScanners.isEmpty()) {
3711 this.joinedHeap = new KeyValueHeap(joinedScanners, comparator);
3712 }
3713 }
3714
3715 RegionScannerImpl(Scan scan, HRegion region) throws IOException {
3716 this(scan, null, region);
3717 }
3718
3719 @Override
3720 public long getMvccReadPoint() {
3721 return this.readPt;
3722 }
3723
3724
3725
3726 protected void resetFilters() {
3727 if (filter != null) {
3728 filter.reset();
3729 }
3730 }
3731
3732 @Override
3733 public boolean next(List<KeyValue> outResults, int limit)
3734 throws IOException {
3735 return next(outResults, limit, null);
3736 }
3737
3738 @Override
3739 public synchronized boolean next(List<KeyValue> outResults, int limit,
3740 String metric) throws IOException {
3741 if (this.filterClosed) {
3742 throw new UnknownScannerException("Scanner was closed (timed out?) " +
3743 "after we renewed it. Could be caused by a very slow scanner " +
3744 "or a lengthy garbage collection");
3745 }
3746 startRegionOperation();
3747 readRequestsCount.increment();
3748 opMetrics.setReadRequestCountMetrics(readRequestsCount.get());
3749 try {
3750
3751
3752 MultiVersionConsistencyControl.setThreadReadPoint(this.readPt);
3753
3754 return nextRaw(outResults, limit, metric);
3755 } finally {
3756 closeRegionOperation();
3757 }
3758 }
3759
3760 @Override
3761 public boolean nextRaw(List<KeyValue> outResults, String metric)
3762 throws IOException {
3763 return nextRaw(outResults, batch, metric);
3764 }
3765
3766 @Override
3767 public boolean nextRaw(List<KeyValue> outResults, int limit,
3768 String metric) throws IOException {
3769 results.clear();
3770
3771 boolean returnResult = nextInternal(limit, metric);
3772
3773 outResults.addAll(results);
3774 resetFilters();
3775 if (isFilterDone()) {
3776 return false;
3777 }
3778 return returnResult;
3779 }
3780
3781 @Override
3782 public boolean next(List<KeyValue> outResults)
3783 throws IOException {
3784
3785 return next(outResults, batch, null);
3786 }
3787
3788 @Override
3789 public boolean next(List<KeyValue> outResults, String metric)
3790 throws IOException {
3791
3792 return next(outResults, batch, metric);
3793 }
3794
3795 private void populateFromJoinedHeap(int limit, String metric) throws IOException {
3796 assert joinedContinuationRow != null;
3797 KeyValue kv = populateResult(this.joinedHeap, limit, joinedContinuationRow.getBuffer(),
3798 joinedContinuationRow.getRowOffset(), joinedContinuationRow.getRowLength(), metric);
3799 if (kv != KV_LIMIT) {
3800
3801 joinedContinuationRow = null;
3802 }
3803
3804
3805 Collections.sort(results, comparator);
3806 }
3807
3808
3809
3810
3811
3812
3813
3814
3815
3816
3817
3818 private KeyValue populateResult(KeyValueHeap heap, int limit, byte[] currentRow, int offset,
3819 short length, String metric) throws IOException {
3820 KeyValue nextKv;
3821 do {
3822 heap.next(results, limit - results.size(), metric);
3823 if (limit > 0 && results.size() == limit) {
3824 return KV_LIMIT;
3825 }
3826 nextKv = heap.peek();
3827 } while (nextKv != null && nextKv.matchingRow(currentRow, offset, length));
3828 return nextKv;
3829 }
3830
3831
3832
3833
3834 public synchronized boolean isFilterDone() {
3835 return this.filter != null && this.filter.filterAllRemaining();
3836 }
3837
3838 private boolean nextInternal(int limit, String metric) throws IOException {
3839 RpcCallContext rpcCall = HBaseServer.getCurrentCall();
3840
3841
3842
3843
3844
3845 while (true) {
3846 if (rpcCall != null) {
3847
3848
3849
3850
3851 rpcCall.throwExceptionIfCallerDisconnected();
3852 }
3853
3854
3855 KeyValue current = this.storeHeap.peek();
3856
3857 byte[] currentRow = null;
3858 int offset = 0;
3859 short length = 0;
3860 if (current != null) {
3861 currentRow = current.getBuffer();
3862 offset = current.getRowOffset();
3863 length = current.getRowLength();
3864 }
3865 boolean stopRow = isStopRow(currentRow, offset, length);
3866
3867
3868 if (joinedContinuationRow == null) {
3869
3870 if (stopRow) {
3871 if (filter != null && filter.hasFilterRow()) {
3872 filter.filterRow(results);
3873 }
3874 if (filter != null && filter.filterRow()) {
3875 results.clear();
3876 }
3877 return false;
3878 }
3879
3880
3881
3882 if (filterRowKey(currentRow, offset, length)) {
3883 boolean moreRows = nextRow(currentRow, offset, length);
3884 if (!moreRows) return false;
3885 continue;
3886 }
3887
3888
3889 KeyValue nextKv = populateResult(this.storeHeap, limit, currentRow, offset, length, metric);
3890 if (nextKv == KV_LIMIT) {
3891 if (this.filter != null && filter.hasFilterRow()) {
3892 throw new IncompatibleFilterException(
3893 "Filter whose hasFilterRow() returns true is incompatible with scan with limit!");
3894 }
3895 return true;
3896 }
3897 stopRow = nextKv == null || isStopRow(nextKv.getBuffer(), nextKv.getRowOffset(), nextKv.getRowLength());
3898
3899 final boolean isEmptyRow = results.isEmpty();
3900
3901
3902
3903 if (filter != null && filter.hasFilterRow()) {
3904 filter.filterRow(results);
3905 }
3906
3907 if (isEmptyRow || filterRow()) {
3908
3909
3910
3911
3912
3913 boolean moreRows = nextRow(currentRow, offset, length);
3914 if (!moreRows) return false;
3915
3916
3917
3918 if (!stopRow) continue;
3919 return false;
3920 }
3921
3922
3923
3924
3925
3926 if (this.joinedHeap != null) {
3927 KeyValue nextJoinedKv = joinedHeap.peek();
3928
3929 boolean mayHaveData =
3930 (nextJoinedKv != null && nextJoinedKv.matchingRow(currentRow, offset, length))
3931 || (this.joinedHeap.seek(KeyValue.createFirstOnRow(currentRow, offset, length))
3932 && joinedHeap.peek() != null
3933 && joinedHeap.peek().matchingRow(currentRow, offset, length));
3934 if (mayHaveData) {
3935 joinedContinuationRow = current;
3936 populateFromJoinedHeap(limit, metric);
3937 }
3938 }
3939 } else {
3940
3941 populateFromJoinedHeap(limit, metric);
3942 }
3943
3944
3945
3946 if (joinedContinuationRow != null) {
3947 return true;
3948 }
3949
3950
3951
3952
3953 if (results.isEmpty()) {
3954 boolean moreRows = nextRow(currentRow, offset, length);
3955 if (!moreRows) return false;
3956 if (!stopRow) continue;
3957 }
3958
3959
3960 return !stopRow;
3961 }
3962 }
3963
3964 private boolean filterRow() {
3965 return filter != null
3966 && filter.filterRow();
3967 }
3968 private boolean filterRowKey(byte[] row, int offset, short length) {
3969 return filter != null
3970 && filter.filterRowKey(row, offset, length);
3971 }
3972
3973 protected boolean nextRow(byte [] currentRow, int offset, short length) throws IOException {
3974 KeyValue next;
3975 while((next = this.storeHeap.peek()) != null && next.matchingRow(currentRow, offset, length)) {
3976 this.storeHeap.next(MOCKED_LIST);
3977 }
3978 results.clear();
3979 resetFilters();
3980
3981 if (this.region.getCoprocessorHost() != null) {
3982 return this.region.getCoprocessorHost().postScannerFilterRow(this, currentRow);
3983 }
3984 return true;
3985 }
3986
3987 private boolean isStopRow(byte [] currentRow, int offset, short length) {
3988 return currentRow == null ||
3989 (stopRow != null &&
3990 comparator.compareRows(stopRow, 0, stopRow.length,
3991 currentRow, offset, length) <= isScan);
3992 }
3993
3994 @Override
3995 public synchronized void close() {
3996 if (storeHeap != null) {
3997 storeHeap.close();
3998 storeHeap = null;
3999 }
4000 if (joinedHeap != null) {
4001 joinedHeap.close();
4002 joinedHeap = null;
4003 }
4004
4005 scannerReadPoints.remove(this);
4006 this.filterClosed = true;
4007 }
4008
4009 KeyValueHeap getStoreHeapForTesting() {
4010 return storeHeap;
4011 }
4012
4013 @Override
4014 public synchronized boolean reseek(byte[] row) throws IOException {
4015 if (row == null) {
4016 throw new IllegalArgumentException("Row cannot be null.");
4017 }
4018 boolean result = false;
4019 startRegionOperation();
4020 try {
4021
4022 MultiVersionConsistencyControl.setThreadReadPoint(this.readPt);
4023 KeyValue kv = KeyValue.createFirstOnRow(row);
4024
4025 result = this.storeHeap.requestSeek(kv, true, true);
4026 if (this.joinedHeap != null) {
4027 result = this.joinedHeap.requestSeek(kv, true, true) || result;
4028 }
4029 } finally {
4030 closeRegionOperation();
4031 }
4032 return result;
4033 }
4034 }
4035
4036
4037
4038
4039
4040
4041
4042
4043
4044
4045
4046
4047
4048
4049
4050
4051
4052
4053
4054
4055
4056
4057 public static HRegion newHRegion(Path tableDir, HLog log, FileSystem fs,
4058 Configuration conf, HRegionInfo regionInfo, final HTableDescriptor htd,
4059 RegionServerServices rsServices) {
4060 try {
4061 @SuppressWarnings("unchecked")
4062 Class<? extends HRegion> regionClass =
4063 (Class<? extends HRegion>) conf.getClass(HConstants.REGION_IMPL, HRegion.class);
4064
4065 Constructor<? extends HRegion> c =
4066 regionClass.getConstructor(Path.class, HLog.class, FileSystem.class,
4067 Configuration.class, HRegionInfo.class, HTableDescriptor.class,
4068 RegionServerServices.class);
4069
4070 return c.newInstance(tableDir, log, fs, conf, regionInfo, htd, rsServices);
4071 } catch (Throwable e) {
4072
4073 throw new IllegalStateException("Could not instantiate a region instance.", e);
4074 }
4075 }
4076
4077
4078
4079
4080
4081
4082
4083
4084
4085
4086
4087
4088
4089
4090
4091
4092
4093
4094
4095 public static HRegion createHRegion(final HRegionInfo info, final Path rootDir,
4096 final Configuration conf, final HTableDescriptor hTableDescriptor)
4097 throws IOException {
4098 return createHRegion(info, rootDir, conf, hTableDescriptor, null);
4099 }
4100
4101
4102
4103
4104
4105
4106
4107
4108
4109
4110
4111 public static void closeHRegion(final HRegion r) throws IOException {
4112 if (r == null) return;
4113 r.close();
4114 if (r.getLog() == null) return;
4115 r.getLog().closeAndDelete();
4116 }
4117
4118
4119
4120
4121
4122
4123
4124
4125
4126
4127
4128
4129
4130
4131
4132
4133 public static HRegion createHRegion(final HRegionInfo info, final Path rootDir,
4134 final Configuration conf,
4135 final HTableDescriptor hTableDescriptor,
4136 final HLog hlog,
4137 final boolean initialize)
4138 throws IOException {
4139 return createHRegion(info, rootDir, conf, hTableDescriptor,
4140 hlog, initialize, false);
4141 }
4142
4143
4144
4145
4146
4147
4148
4149
4150
4151
4152
4153
4154
4155
4156
4157
4158
4159
4160
4161 public static HRegion createHRegion(final HRegionInfo info, final Path rootDir,
4162 final Configuration conf,
4163 final HTableDescriptor hTableDescriptor,
4164 final HLog hlog,
4165 final boolean initialize, final boolean ignoreHLog)
4166 throws IOException {
4167 LOG.info("creating HRegion " + info.getTableNameAsString()
4168 + " HTD == " + hTableDescriptor + " RootDir = " + rootDir +
4169 " Table name == " + info.getTableNameAsString());
4170
4171 Path tableDir =
4172 HTableDescriptor.getTableDir(rootDir, info.getTableName());
4173 Path regionDir = HRegion.getRegionDir(tableDir, info.getEncodedName());
4174 FileSystem fs = FileSystem.get(conf);
4175 fs.mkdirs(regionDir);
4176
4177 writeRegioninfoOnFilesystem(info, regionDir, fs, conf);
4178 HLog effectiveHLog = hlog;
4179 if (hlog == null && !ignoreHLog) {
4180 effectiveHLog = new HLog(fs, new Path(regionDir, HConstants.HREGION_LOGDIR_NAME),
4181 new Path(regionDir, HConstants.HREGION_OLDLOGDIR_NAME), conf);
4182 }
4183 HRegion region = HRegion.newHRegion(tableDir,
4184 effectiveHLog, fs, conf, info, hTableDescriptor, null);
4185 if (initialize) {
4186 region.initialize();
4187 }
4188 return region;
4189 }
4190
4191 public static HRegion createHRegion(final HRegionInfo info, final Path rootDir,
4192 final Configuration conf,
4193 final HTableDescriptor hTableDescriptor,
4194 final HLog hlog)
4195 throws IOException {
4196 return createHRegion(info, rootDir, conf, hTableDescriptor, hlog, true);
4197 }
4198
4199
4200
4201
4202
4203
4204
4205
4206
4207
4208
4209
4210
4211 public static HRegion openHRegion(final HRegionInfo info,
4212 final HTableDescriptor htd, final HLog wal,
4213 final Configuration conf)
4214 throws IOException {
4215 return openHRegion(info, htd, wal, conf, null, null);
4216 }
4217
4218
4219
4220
4221
4222
4223
4224
4225
4226
4227
4228
4229
4230
4231
4232
4233 public static HRegion openHRegion(final HRegionInfo info,
4234 final HTableDescriptor htd, final HLog wal, final Configuration conf,
4235 final RegionServerServices rsServices,
4236 final CancelableProgressable reporter)
4237 throws IOException {
4238 if (LOG.isDebugEnabled()) {
4239 LOG.debug("Opening region: " + info);
4240 }
4241 if (info == null) {
4242 throw new NullPointerException("Passed region info is null");
4243 }
4244 Path dir = HTableDescriptor.getTableDir(FSUtils.getRootDir(conf),
4245 info.getTableName());
4246 FileSystem fs = null;
4247 if (rsServices != null) {
4248 fs = rsServices.getFileSystem();
4249 }
4250 if (fs == null) {
4251 fs = FileSystem.get(conf);
4252 }
4253 HRegion r = HRegion.newHRegion(dir, wal, fs, conf, info,
4254 htd, rsServices);
4255 return r.openHRegion(reporter);
4256 }
4257
4258 public static HRegion openHRegion(Path tableDir, final HRegionInfo info,
4259 final HTableDescriptor htd, final HLog wal, final Configuration conf)
4260 throws IOException {
4261 return openHRegion(tableDir, info, htd, wal, conf, null, null);
4262 }
4263
4264
4265
4266
4267
4268
4269
4270
4271
4272
4273
4274
4275
4276
4277
4278 public static HRegion openHRegion(final Path tableDir, final HRegionInfo info,
4279 final HTableDescriptor htd, final HLog wal, final Configuration conf,
4280 final RegionServerServices rsServices,
4281 final CancelableProgressable reporter)
4282 throws IOException {
4283 if (info == null) throw new NullPointerException("Passed region info is null");
4284 LOG.info("HRegion.openHRegion Region name ==" + info.getRegionNameAsString());
4285 if (LOG.isDebugEnabled()) {
4286 LOG.debug("Opening region: " + info);
4287 }
4288 Path dir = HTableDescriptor.getTableDir(tableDir,
4289 info.getTableName());
4290 HRegion r = HRegion.newHRegion(dir, wal, FileSystem.get(conf), conf, info,
4291 htd, rsServices);
4292 return r.openHRegion(reporter);
4293 }
4294
4295
4296
4297
4298
4299
4300
4301
4302
4303 protected HRegion openHRegion(final CancelableProgressable reporter)
4304 throws IOException {
4305 checkCompressionCodecs();
4306
4307 long seqid = initialize(reporter);
4308 if (this.log != null) {
4309 this.log.setSequenceNumber(seqid);
4310 }
4311 return this;
4312 }
4313
4314 private void checkCompressionCodecs() throws IOException {
4315 for (HColumnDescriptor fam: this.htableDescriptor.getColumnFamilies()) {
4316 CompressionTest.testCompression(fam.getCompression());
4317 CompressionTest.testCompression(fam.getCompactionCompression());
4318 }
4319 }
4320
4321
4322
4323
4324
4325
4326
4327
4328
4329
4330
4331 public static void addRegionToMETA(HRegion meta, HRegion r)
4332 throws IOException {
4333 meta.checkResources();
4334
4335 byte[] row = r.getRegionName();
4336 Integer lid = meta.obtainRowLock(row);
4337 try {
4338 final long now = EnvironmentEdgeManager.currentTimeMillis();
4339 final List<KeyValue> edits = new ArrayList<KeyValue>(2);
4340 edits.add(new KeyValue(row, HConstants.CATALOG_FAMILY,
4341 HConstants.REGIONINFO_QUALIFIER, now,
4342 Writables.getBytes(r.getRegionInfo())));
4343
4344 edits.add(new KeyValue(row, HConstants.CATALOG_FAMILY,
4345 HConstants.META_VERSION_QUALIFIER, now,
4346 Bytes.toBytes(HConstants.META_VERSION)));
4347 meta.put(HConstants.CATALOG_FAMILY, edits);
4348 } finally {
4349 meta.releaseRowLock(lid);
4350 }
4351 }
4352
4353
4354
4355
4356
4357
4358
4359
4360
4361 public static void deleteRegion(FileSystem fs, Path rootdir, HRegionInfo info)
4362 throws IOException {
4363 deleteRegion(fs, HRegion.getRegionDir(rootdir, info));
4364 }
4365
4366 private static void deleteRegion(FileSystem fs, Path regiondir)
4367 throws IOException {
4368 if (LOG.isDebugEnabled()) {
4369 LOG.debug("DELETING region " + regiondir.toString());
4370 }
4371 if (!fs.delete(regiondir, true)) {
4372 LOG.warn("Failed delete of " + regiondir);
4373 }
4374 }
4375
4376
4377
4378
4379
4380
4381
4382
4383 public static Path getRegionDir(final Path rootdir, final HRegionInfo info) {
4384 return new Path(
4385 HTableDescriptor.getTableDir(rootdir, info.getTableName()),
4386 info.getEncodedName());
4387 }
4388
4389
4390
4391
4392
4393
4394
4395
4396
4397 public static boolean rowIsInRange(HRegionInfo info, final byte [] row) {
4398 return ((info.getStartKey().length == 0) ||
4399 (Bytes.compareTo(info.getStartKey(), row) <= 0)) &&
4400 ((info.getEndKey().length == 0) ||
4401 (Bytes.compareTo(info.getEndKey(), row) > 0));
4402 }
4403
4404
4405
4406
4407
4408
4409
4410
4411
4412
4413 public static void makeColumnFamilyDirs(FileSystem fs, Path tabledir,
4414 final HRegionInfo hri, byte [] colFamily)
4415 throws IOException {
4416 Path dir = Store.getStoreHomedir(tabledir, hri.getEncodedName(), colFamily);
4417 if (!fs.mkdirs(dir)) {
4418 LOG.warn("Failed to create " + dir);
4419 }
4420 }
4421
4422
4423
4424
4425
4426
4427
4428
4429
4430 public static HRegion mergeAdjacent(final HRegion srcA, final HRegion srcB)
4431 throws IOException {
4432 HRegion a = srcA;
4433 HRegion b = srcB;
4434
4435
4436
4437 if (srcA.getStartKey() == null) {
4438 if (srcB.getStartKey() == null) {
4439 throw new IOException("Cannot merge two regions with null start key");
4440 }
4441
4442 } else if ((srcB.getStartKey() == null) ||
4443 (Bytes.compareTo(srcA.getStartKey(), srcB.getStartKey()) > 0)) {
4444 a = srcB;
4445 b = srcA;
4446 }
4447
4448 if (!(Bytes.compareTo(a.getEndKey(), b.getStartKey()) == 0)) {
4449 throw new IOException("Cannot merge non-adjacent regions");
4450 }
4451 return merge(a, b);
4452 }
4453
4454
4455
4456
4457
4458
4459
4460
4461
4462 public static HRegion merge(HRegion a, HRegion b)
4463 throws IOException {
4464 if (!a.getRegionInfo().getTableNameAsString().equals(
4465 b.getRegionInfo().getTableNameAsString())) {
4466 throw new IOException("Regions do not belong to the same table");
4467 }
4468
4469 FileSystem fs = a.getFilesystem();
4470
4471
4472
4473 a.flushcache();
4474 b.flushcache();
4475
4476
4477
4478 a.compactStores(true);
4479 if (LOG.isDebugEnabled()) {
4480 LOG.debug("Files for region: " + a);
4481 listPaths(fs, a.getRegionDir());
4482 }
4483 b.compactStores(true);
4484 if (LOG.isDebugEnabled()) {
4485 LOG.debug("Files for region: " + b);
4486 listPaths(fs, b.getRegionDir());
4487 }
4488
4489 Configuration conf = a.getConf();
4490 HTableDescriptor tabledesc = a.getTableDesc();
4491 HLog log = a.getLog();
4492 Path tableDir = a.getTableDir();
4493
4494
4495 final byte[] startKey =
4496 (a.comparator.matchingRows(a.getStartKey(), 0, a.getStartKey().length,
4497 HConstants.EMPTY_BYTE_ARRAY, 0, HConstants.EMPTY_BYTE_ARRAY.length)
4498 || b.comparator.matchingRows(b.getStartKey(), 0,
4499 b.getStartKey().length, HConstants.EMPTY_BYTE_ARRAY, 0,
4500 HConstants.EMPTY_BYTE_ARRAY.length))
4501 ? HConstants.EMPTY_BYTE_ARRAY
4502 : (a.comparator.compareRows(a.getStartKey(), 0, a.getStartKey().length,
4503 b.getStartKey(), 0, b.getStartKey().length) <= 0
4504 ? a.getStartKey()
4505 : b.getStartKey());
4506 final byte[] endKey =
4507 (a.comparator.matchingRows(a.getEndKey(), 0, a.getEndKey().length,
4508 HConstants.EMPTY_BYTE_ARRAY, 0, HConstants.EMPTY_BYTE_ARRAY.length)
4509 || a.comparator.matchingRows(b.getEndKey(), 0, b.getEndKey().length,
4510 HConstants.EMPTY_BYTE_ARRAY, 0,
4511 HConstants.EMPTY_BYTE_ARRAY.length))
4512 ? HConstants.EMPTY_BYTE_ARRAY
4513 : (a.comparator.compareRows(a.getEndKey(), 0, a.getEndKey().length,
4514 b.getEndKey(), 0, b.getEndKey().length) <= 0
4515 ? b.getEndKey()
4516 : a.getEndKey());
4517
4518 HRegionInfo newRegionInfo =
4519 new HRegionInfo(tabledesc.getName(), startKey, endKey);
4520 LOG.info("Creating new region " + newRegionInfo.toString());
4521 String encodedName = newRegionInfo.getEncodedName();
4522 Path newRegionDir = HRegion.getRegionDir(a.getTableDir(), encodedName);
4523 if(fs.exists(newRegionDir)) {
4524 throw new IOException("Cannot merge; target file collision at " +
4525 newRegionDir);
4526 }
4527 fs.mkdirs(newRegionDir);
4528
4529 LOG.info("starting merge of regions: " + a + " and " + b +
4530 " into new region " + newRegionInfo.toString() +
4531 " with start key <" + Bytes.toStringBinary(startKey) + "> and end key <" +
4532 Bytes.toStringBinary(endKey) + ">");
4533
4534
4535 Map<byte [], List<StoreFile>> byFamily =
4536 new TreeMap<byte [], List<StoreFile>>(Bytes.BYTES_COMPARATOR);
4537 byFamily = filesByFamily(byFamily, a.close());
4538 byFamily = filesByFamily(byFamily, b.close());
4539 for (Map.Entry<byte [], List<StoreFile>> es : byFamily.entrySet()) {
4540 byte [] colFamily = es.getKey();
4541 makeColumnFamilyDirs(fs, tableDir, newRegionInfo, colFamily);
4542
4543
4544 List<StoreFile> srcFiles = es.getValue();
4545 if (srcFiles.size() == 2) {
4546 long seqA = srcFiles.get(0).getMaxSequenceId();
4547 long seqB = srcFiles.get(1).getMaxSequenceId();
4548 if (seqA == seqB) {
4549
4550
4551
4552 throw new IOException("Files have same sequenceid: " + seqA);
4553 }
4554 }
4555 for (StoreFile hsf: srcFiles) {
4556 StoreFile.rename(fs, hsf.getPath(),
4557 StoreFile.getUniqueFile(fs, Store.getStoreHomedir(tableDir,
4558 newRegionInfo.getEncodedName(), colFamily)));
4559 }
4560 }
4561 if (LOG.isDebugEnabled()) {
4562 LOG.debug("Files for new region");
4563 listPaths(fs, newRegionDir);
4564 }
4565 HRegion dstRegion = HRegion.newHRegion(tableDir, log, fs, conf,
4566 newRegionInfo, a.getTableDesc(), null);
4567 long totalReadRequestCount = a.readRequestsCount.get() + b.readRequestsCount.get();
4568 dstRegion.readRequestsCount.set(totalReadRequestCount);
4569 dstRegion.opMetrics.setReadRequestCountMetrics(totalReadRequestCount);
4570
4571 long totalWriteRequestCount = a.writeRequestsCount.get() + b.writeRequestsCount.get();
4572 dstRegion.writeRequestsCount.set(totalWriteRequestCount);
4573 dstRegion.opMetrics.setWriteRequestCountMetrics(totalWriteRequestCount);
4574
4575 dstRegion.initialize();
4576 dstRegion.compactStores();
4577 if (LOG.isDebugEnabled()) {
4578 LOG.debug("Files for new region");
4579 listPaths(fs, dstRegion.getRegionDir());
4580 }
4581
4582
4583 HFileArchiver.archiveRegion(fs, FSUtils.getRootDir(a.getConf()),
4584 a.getTableDir(), a.getRegionDir());
4585
4586 HFileArchiver.archiveRegion(fs, FSUtils.getRootDir(b.getConf()),
4587 b.getTableDir(), b.getRegionDir());
4588
4589 LOG.info("merge completed. New region is " + dstRegion);
4590
4591 return dstRegion;
4592 }
4593
4594
4595
4596
4597
4598
4599
4600
4601 private static Map<byte [], List<StoreFile>> filesByFamily(
4602 Map<byte [], List<StoreFile>> byFamily, List<StoreFile> storeFiles) {
4603 for (StoreFile src: storeFiles) {
4604 byte [] family = src.getFamily();
4605 List<StoreFile> v = byFamily.get(family);
4606 if (v == null) {
4607 v = new ArrayList<StoreFile>();
4608 byFamily.put(family, v);
4609 }
4610 v.add(src);
4611 }
4612 return byFamily;
4613 }
4614
4615
4616
4617
4618
4619 boolean isMajorCompaction() throws IOException {
4620 for (Store store: this.stores.values()) {
4621 if (store.isMajorCompaction()) {
4622 return true;
4623 }
4624 }
4625 return false;
4626 }
4627
4628
4629
4630
4631
4632
4633
4634
4635 private static void listPaths(FileSystem fs, Path dir) throws IOException {
4636 if (LOG.isDebugEnabled()) {
4637 FileStatus[] stats = FSUtils.listStatus(fs, dir, null);
4638 if (stats == null || stats.length == 0) {
4639 return;
4640 }
4641 for (int i = 0; i < stats.length; i++) {
4642 String path = stats[i].getPath().toString();
4643 if (stats[i].isDir()) {
4644 LOG.debug("d " + path);
4645 listPaths(fs, stats[i].getPath());
4646 } else {
4647 LOG.debug("f " + path + " size=" + stats[i].getLen());
4648 }
4649 }
4650 }
4651 }
4652
4653
4654
4655
4656
4657
4658
4659
4660
4661
4662 public Result get(final Get get) throws IOException {
4663 return get(get, null);
4664 }
4665
4666
4667
4668
4669
4670
4671
4672
4673 public Result get(final Get get, final Integer lockid) throws IOException {
4674 checkRow(get.getRow(), "Get");
4675
4676 if (get.hasFamilies()) {
4677 for (byte [] family: get.familySet()) {
4678 checkFamily(family);
4679 }
4680 } else {
4681 for (byte[] family: this.htableDescriptor.getFamiliesKeys()) {
4682 get.addFamily(family);
4683 }
4684 }
4685 List<KeyValue> results = get(get, true);
4686 return new Result(results);
4687 }
4688
4689
4690
4691
4692
4693
4694 private List<KeyValue> get(Get get, boolean withCoprocessor)
4695 throws IOException {
4696 long now = EnvironmentEdgeManager.currentTimeMillis();
4697
4698 List<KeyValue> results = new ArrayList<KeyValue>();
4699
4700
4701 if (withCoprocessor && (coprocessorHost != null)) {
4702 if (coprocessorHost.preGet(get, results)) {
4703 return results;
4704 }
4705 }
4706
4707 Scan scan = new Scan(get);
4708
4709 RegionScanner scanner = null;
4710 try {
4711 scanner = getScanner(scan);
4712 scanner.next(results, SchemaMetrics.METRIC_GETSIZE);
4713 } finally {
4714 if (scanner != null)
4715 scanner.close();
4716 }
4717
4718
4719 if (withCoprocessor && (coprocessorHost != null)) {
4720 coprocessorHost.postGet(get, results);
4721 }
4722
4723
4724 final long after = EnvironmentEdgeManager.currentTimeMillis();
4725 this.opMetrics.updateGetMetrics(get.familySet(), after - now);
4726
4727 return results;
4728 }
4729
4730 public void mutateRow(RowMutations rm) throws IOException {
4731 mutateRowsWithLocks(rm.getMutations(), Collections.singleton(rm.getRow()));
4732 }
4733
4734
4735
4736
4737
4738
4739
4740
4741
4742
4743
4744 public void mutateRowsWithLocks(Collection<Mutation> mutations,
4745 Collection<byte[]> rowsToLock) throws IOException {
4746 boolean flush = false;
4747
4748 checkReadOnly();
4749 checkResources();
4750
4751 startRegionOperation();
4752 List<Integer> acquiredLocks = null;
4753 try {
4754
4755
4756
4757
4758 WALEdit walEdit = new WALEdit();
4759 if (coprocessorHost != null) {
4760 for (Mutation m : mutations) {
4761 if (m instanceof Put) {
4762 if (coprocessorHost.prePut((Put) m, walEdit, m.getWriteToWAL())) {
4763
4764 return;
4765 }
4766 } else if (m instanceof Delete) {
4767 Delete d = (Delete) m;
4768 prepareDelete(d);
4769 if (coprocessorHost.preDelete(d, walEdit, d.getWriteToWAL())) {
4770
4771 return;
4772 }
4773 }
4774 }
4775 }
4776
4777 long txid = 0;
4778 boolean walSyncSuccessful = false;
4779 boolean locked = false;
4780
4781
4782 acquiredLocks = new ArrayList<Integer>(rowsToLock.size());
4783 for (byte[] row : rowsToLock) {
4784
4785 Integer lid = getLock(null, row, true);
4786 if (lid == null) {
4787 throw new IOException("Failed to acquire lock on "
4788 + Bytes.toStringBinary(row));
4789 }
4790 acquiredLocks.add(lid);
4791 }
4792
4793
4794 lock(this.updatesLock.readLock(), acquiredLocks.size());
4795 locked = true;
4796
4797
4798 MultiVersionConsistencyControl.WriteEntry w = mvcc.beginMemstoreInsert();
4799
4800 long now = EnvironmentEdgeManager.currentTimeMillis();
4801 byte[] byteNow = Bytes.toBytes(now);
4802 try {
4803
4804 for (Mutation m : mutations) {
4805 if (m instanceof Put) {
4806 Map<byte[], List<KeyValue>> familyMap = m.getFamilyMap();
4807 checkFamilies(familyMap.keySet());
4808 checkTimestamps(familyMap, now);
4809 updateKVTimestamps(familyMap.values(), byteNow);
4810 } else if (m instanceof Delete) {
4811 Delete d = (Delete) m;
4812 prepareDelete(d);
4813 prepareDeleteTimestamps(d.getFamilyMap(), byteNow);
4814 } else {
4815 throw new DoNotRetryIOException(
4816 "Action must be Put or Delete. But was: "
4817 + m.getClass().getName());
4818 }
4819 if (m.getWriteToWAL()) {
4820 addFamilyMapToWALEdit(m.getFamilyMap(), walEdit);
4821 }
4822 }
4823
4824
4825 if (walEdit.size() > 0) {
4826 txid = this.log.appendNoSync(regionInfo,
4827 this.htableDescriptor.getName(), walEdit,
4828 HConstants.DEFAULT_CLUSTER_ID, now, this.htableDescriptor);
4829 }
4830
4831
4832 long addedSize = 0;
4833 for (Mutation m : mutations) {
4834 addedSize += applyFamilyMapToMemstore(m.getFamilyMap(), w);
4835 }
4836 flush = isFlushSize(this.addAndGetGlobalMemstoreSize(addedSize));
4837
4838
4839 this.updatesLock.readLock().unlock();
4840 locked = false;
4841 if (acquiredLocks != null) {
4842 for (Integer lid : acquiredLocks) {
4843 releaseRowLock(lid);
4844 }
4845 acquiredLocks = null;
4846 }
4847
4848
4849 if (walEdit.size() > 0) {
4850 syncOrDefer(txid);
4851 }
4852 walSyncSuccessful = true;
4853
4854
4855 mvcc.completeMemstoreInsert(w);
4856 w = null;
4857
4858
4859
4860
4861 if (coprocessorHost != null) {
4862 for (Mutation m : mutations) {
4863 if (m instanceof Put) {
4864 coprocessorHost.postPut((Put) m, walEdit, m.getWriteToWAL());
4865 } else if (m instanceof Delete) {
4866 coprocessorHost.postDelete((Delete) m, walEdit, m.getWriteToWAL());
4867 }
4868 }
4869 }
4870 } finally {
4871
4872 if (!walSyncSuccessful) {
4873 int kvsRolledback = 0;
4874 for (Mutation m : mutations) {
4875 for (Map.Entry<byte[], List<KeyValue>> e : m.getFamilyMap()
4876 .entrySet()) {
4877 List<KeyValue> kvs = e.getValue();
4878 byte[] family = e.getKey();
4879 Store store = getStore(family);
4880
4881 for (KeyValue kv : kvs) {
4882 store.rollback(kv);
4883 kvsRolledback++;
4884 }
4885 }
4886 }
4887 LOG.info("mutateRowWithLocks: rolled back " + kvsRolledback
4888 + " KeyValues");
4889 }
4890
4891 if (w != null) {
4892 mvcc.completeMemstoreInsert(w);
4893 }
4894
4895 if (locked) {
4896 this.updatesLock.readLock().unlock();
4897 }
4898
4899 if (acquiredLocks != null) {
4900 for (Integer lid : acquiredLocks) {
4901 releaseRowLock(lid);
4902 }
4903 }
4904 }
4905 } finally {
4906 if (flush) {
4907
4908 requestFlush();
4909 }
4910 closeRegionOperation();
4911 }
4912 }
4913
4914
4915
4916
4917
4918
4919
4920
4921
4922
4923
4924
4925
4926
4927
4928
4929 public Result append(Append append, boolean writeToWAL)
4930 throws IOException {
4931 return append(append, null, writeToWAL);
4932 }
4933
4934
4935
4936
4937
4938
4939
4940
4941
4942
4943
4944
4945
4946
4947 public Result append(Append append, Integer lockid, boolean writeToWAL)
4948 throws IOException {
4949
4950 byte[] row = append.getRow();
4951 checkRow(row, "append");
4952 boolean flush = false;
4953 WALEdit walEdits = null;
4954 List<KeyValue> allKVs = new ArrayList<KeyValue>(append.size());
4955 Map<Store, List<KeyValue>> tempMemstore = new HashMap<Store, List<KeyValue>>();
4956 long before = EnvironmentEdgeManager.currentTimeMillis();
4957 long size = 0;
4958 long txid = 0;
4959
4960 checkReadOnly();
4961
4962 startRegionOperation();
4963 this.writeRequestsCount.increment();
4964 this.opMetrics.setWriteRequestCountMetrics(this.writeRequestsCount.get());
4965 try {
4966 Integer lid = getLock(lockid, row, true);
4967 lock(this.updatesLock.readLock());
4968 try {
4969 long now = EnvironmentEdgeManager.currentTimeMillis();
4970
4971 for (Map.Entry<byte[], List<KeyValue>> family : append.getFamilyMap()
4972 .entrySet()) {
4973
4974 Store store = stores.get(family.getKey());
4975 List<KeyValue> kvs = new ArrayList<KeyValue>(family.getValue().size());
4976
4977
4978 Get get = new Get(row);
4979 for (KeyValue kv : family.getValue()) {
4980 get.addColumn(family.getKey(), kv.getQualifier());
4981 }
4982 List<KeyValue> results = get(get, false);
4983
4984
4985
4986
4987
4988
4989
4990 int idx = 0;
4991 for (KeyValue kv : family.getValue()) {
4992 KeyValue newKV;
4993 if (idx < results.size()
4994 && results.get(idx).matchingQualifier(kv.getBuffer(),
4995 kv.getQualifierOffset(), kv.getQualifierLength())) {
4996 KeyValue oldKv = results.get(idx);
4997
4998 newKV = new KeyValue(row.length, kv.getFamilyLength(),
4999 kv.getQualifierLength(), now, KeyValue.Type.Put,
5000 oldKv.getValueLength() + kv.getValueLength());
5001
5002 System.arraycopy(oldKv.getBuffer(), oldKv.getValueOffset(),
5003 newKV.getBuffer(), newKV.getValueOffset(),
5004 oldKv.getValueLength());
5005 System.arraycopy(kv.getBuffer(), kv.getValueOffset(),
5006 newKV.getBuffer(),
5007 newKV.getValueOffset() + oldKv.getValueLength(),
5008 kv.getValueLength());
5009 idx++;
5010 } else {
5011
5012 newKV = new KeyValue(row.length, kv.getFamilyLength(),
5013 kv.getQualifierLength(), now, KeyValue.Type.Put,
5014 kv.getValueLength());
5015
5016 System.arraycopy(kv.getBuffer(), kv.getValueOffset(),
5017 newKV.getBuffer(), newKV.getValueOffset(),
5018 kv.getValueLength());
5019 }
5020
5021 System.arraycopy(kv.getBuffer(), kv.getRowOffset(),
5022 newKV.getBuffer(), newKV.getRowOffset(), kv.getRowLength());
5023 System.arraycopy(kv.getBuffer(), kv.getFamilyOffset(),
5024 newKV.getBuffer(), newKV.getFamilyOffset(),
5025 kv.getFamilyLength());
5026 System.arraycopy(kv.getBuffer(), kv.getQualifierOffset(),
5027 newKV.getBuffer(), newKV.getQualifierOffset(),
5028 kv.getQualifierLength());
5029
5030 kvs.add(newKV);
5031
5032
5033 if (writeToWAL) {
5034 if (walEdits == null) {
5035 walEdits = new WALEdit();
5036 }
5037 walEdits.add(newKV);
5038 }
5039 }
5040
5041
5042 tempMemstore.put(store, kvs);
5043 }
5044
5045
5046 if (writeToWAL) {
5047
5048
5049
5050 txid = this.log.appendNoSync(regionInfo,
5051 this.htableDescriptor.getName(), walEdits,
5052 HConstants.DEFAULT_CLUSTER_ID, EnvironmentEdgeManager.currentTimeMillis(),
5053 this.htableDescriptor);
5054 }
5055
5056 for (Map.Entry<Store, List<KeyValue>> entry : tempMemstore.entrySet()) {
5057 Store store = entry.getKey();
5058 size += store.upsert(entry.getValue());
5059 allKVs.addAll(entry.getValue());
5060 }
5061 size = this.addAndGetGlobalMemstoreSize(size);
5062 flush = isFlushSize(size);
5063 } finally {
5064 this.updatesLock.readLock().unlock();
5065 releaseRowLock(lid);
5066 }
5067 if (writeToWAL) {
5068 syncOrDefer(txid);
5069 }
5070 } finally {
5071 closeRegionOperation();
5072 }
5073
5074
5075 long after = EnvironmentEdgeManager.currentTimeMillis();
5076 this.opMetrics.updateAppendMetrics(append.getFamilyMap().keySet(), after - before);
5077
5078 if (flush) {
5079
5080 requestFlush();
5081 }
5082
5083 return append.isReturnResults() ? new Result(allKVs) : null;
5084 }
5085
5086
5087
5088
5089
5090
5091
5092
5093
5094
5095
5096
5097 public Result increment(Increment increment, boolean writeToWAL)
5098 throws IOException {
5099 return increment(increment, null, writeToWAL);
5100 }
5101
5102
5103
5104
5105
5106
5107
5108
5109
5110
5111
5112
5113
5114
5115
5116 public Result increment(Increment increment, Integer lockid,
5117 boolean writeToWAL)
5118 throws IOException {
5119
5120 byte [] row = increment.getRow();
5121 checkRow(row, "increment");
5122 TimeRange tr = increment.getTimeRange();
5123 boolean flush = false;
5124 WALEdit walEdits = null;
5125 List<KeyValue> allKVs = new ArrayList<KeyValue>(increment.numColumns());
5126 Map<Store, List<KeyValue>> tempMemstore = new HashMap<Store, List<KeyValue>>();
5127 long before = EnvironmentEdgeManager.currentTimeMillis();
5128 long size = 0;
5129 long txid = 0;
5130
5131 checkReadOnly();
5132
5133 startRegionOperation();
5134 this.writeRequestsCount.increment();
5135 this.opMetrics.setWriteRequestCountMetrics(this.writeRequestsCount.get());
5136 try {
5137 Integer lid = getLock(lockid, row, true);
5138 lock(this.updatesLock.readLock());
5139 try {
5140 long now = EnvironmentEdgeManager.currentTimeMillis();
5141
5142 for (Map.Entry<byte [], NavigableMap<byte [], Long>> family :
5143 increment.getFamilyMap().entrySet()) {
5144
5145 Store store = stores.get(family.getKey());
5146 List<KeyValue> kvs = new ArrayList<KeyValue>(family.getValue().size());
5147
5148
5149 Get get = new Get(row);
5150 for (Map.Entry<byte [], Long> column : family.getValue().entrySet()) {
5151 get.addColumn(family.getKey(), column.getKey());
5152 }
5153 get.setTimeRange(tr.getMin(), tr.getMax());
5154 List<KeyValue> results = get(get, false);
5155
5156
5157
5158 int idx = 0;
5159 for (Map.Entry<byte [], Long> column : family.getValue().entrySet()) {
5160 long amount = column.getValue();
5161 if (idx < results.size() &&
5162 results.get(idx).matchingQualifier(column.getKey())) {
5163 KeyValue kv = results.get(idx);
5164 if(kv.getValueLength() == Bytes.SIZEOF_LONG) {
5165 amount += Bytes.toLong(kv.getBuffer(), kv.getValueOffset(), Bytes.SIZEOF_LONG);
5166 } else {
5167
5168 throw new DoNotRetryIOException(
5169 "Attempted to increment field that isn't 64 bits wide");
5170 }
5171 idx++;
5172 }
5173
5174
5175 KeyValue newKV = new KeyValue(row, family.getKey(), column.getKey(),
5176 now, Bytes.toBytes(amount));
5177 kvs.add(newKV);
5178
5179
5180 if (writeToWAL) {
5181 if (walEdits == null) {
5182 walEdits = new WALEdit();
5183 }
5184 walEdits.add(newKV);
5185 }
5186 }
5187
5188
5189 tempMemstore.put(store, kvs);
5190 }
5191
5192
5193 if (writeToWAL) {
5194
5195
5196
5197 txid = this.log.appendNoSync(regionInfo, this.htableDescriptor.getName(),
5198 walEdits, HConstants.DEFAULT_CLUSTER_ID, EnvironmentEdgeManager.currentTimeMillis(),
5199 this.htableDescriptor);
5200 }
5201
5202
5203 for (Map.Entry<Store, List<KeyValue>> entry : tempMemstore.entrySet()) {
5204 Store store = entry.getKey();
5205 size += store.upsert(entry.getValue());
5206 allKVs.addAll(entry.getValue());
5207 }
5208 size = this.addAndGetGlobalMemstoreSize(size);
5209 flush = isFlushSize(size);
5210 } finally {
5211 this.updatesLock.readLock().unlock();
5212 releaseRowLock(lid);
5213 }
5214 if (writeToWAL) {
5215 syncOrDefer(txid);
5216 }
5217 } finally {
5218 closeRegionOperation();
5219 long after = EnvironmentEdgeManager.currentTimeMillis();
5220 this.opMetrics.updateIncrementMetrics(increment.getFamilyMap().keySet(), after - before);
5221 }
5222
5223 if (flush) {
5224
5225 requestFlush();
5226 }
5227
5228 return new Result(allKVs);
5229 }
5230
5231
5232
5233
5234
5235
5236
5237
5238
5239
5240 public long incrementColumnValue(byte [] row, byte [] family,
5241 byte [] qualifier, long amount, boolean writeToWAL)
5242 throws IOException {
5243
5244 long before = EnvironmentEdgeManager.currentTimeMillis();
5245
5246 checkRow(row, "increment");
5247 boolean flush = false;
5248 boolean wrongLength = false;
5249 long txid = 0;
5250
5251 long result = amount;
5252 startRegionOperation();
5253 this.writeRequestsCount.increment();
5254 this.opMetrics.setWriteRequestCountMetrics(this.writeRequestsCount.get());
5255 try {
5256 Integer lid = obtainRowLock(row);
5257 lock(this.updatesLock.readLock());
5258 try {
5259 Store store = stores.get(family);
5260
5261
5262 Get get = new Get(row);
5263 get.addColumn(family, qualifier);
5264
5265
5266
5267 List<KeyValue> results = get(get, false);
5268
5269 if (!results.isEmpty()) {
5270 KeyValue kv = results.get(0);
5271 if(kv.getValueLength() == Bytes.SIZEOF_LONG){
5272 byte [] buffer = kv.getBuffer();
5273 int valueOffset = kv.getValueOffset();
5274 result += Bytes.toLong(buffer, valueOffset, Bytes.SIZEOF_LONG);
5275 }
5276 else{
5277 wrongLength = true;
5278 }
5279 }
5280 if(!wrongLength){
5281
5282 KeyValue newKv = new KeyValue(row, family,
5283 qualifier, EnvironmentEdgeManager.currentTimeMillis(),
5284 Bytes.toBytes(result));
5285
5286
5287 if (writeToWAL) {
5288 long now = EnvironmentEdgeManager.currentTimeMillis();
5289 WALEdit walEdit = new WALEdit();
5290 walEdit.add(newKv);
5291
5292
5293
5294 txid = this.log.appendNoSync(regionInfo, this.htableDescriptor.getName(),
5295 walEdit, HConstants.DEFAULT_CLUSTER_ID, now,
5296 this.htableDescriptor);
5297 }
5298
5299
5300
5301
5302 long size = store.updateColumnValue(row, family, qualifier, result);
5303
5304 size = this.addAndGetGlobalMemstoreSize(size);
5305 flush = isFlushSize(size);
5306 }
5307 } finally {
5308 this.updatesLock.readLock().unlock();
5309 releaseRowLock(lid);
5310 }
5311 if (writeToWAL) {
5312 syncOrDefer(txid);
5313 }
5314 } finally {
5315 closeRegionOperation();
5316 }
5317
5318
5319 long after = EnvironmentEdgeManager.currentTimeMillis();
5320 this.opMetrics.updateIncrementColumnValueMetrics(family, after - before);
5321
5322 if (flush) {
5323
5324 requestFlush();
5325 }
5326 if(wrongLength){
5327 throw new DoNotRetryIOException(
5328 "Attempted to increment field that isn't 64 bits wide");
5329 }
5330 return result;
5331 }
5332
5333
5334
5335
5336
5337
5338 private void checkFamily(final byte [] family)
5339 throws NoSuchColumnFamilyException {
5340 if (!this.htableDescriptor.hasFamily(family)) {
5341 throw new NoSuchColumnFamilyException("Column family " +
5342 Bytes.toString(family) + " does not exist in region " + this
5343 + " in table " + this.htableDescriptor);
5344 }
5345 }
5346
5347 public static final long FIXED_OVERHEAD = ClassSize.align(
5348 ClassSize.OBJECT +
5349 ClassSize.ARRAY +
5350 35 * ClassSize.REFERENCE + 2 * Bytes.SIZEOF_INT +
5351 (7 * Bytes.SIZEOF_LONG) +
5352 Bytes.SIZEOF_BOOLEAN);
5353
5354 public static final long DEEP_OVERHEAD = FIXED_OVERHEAD +
5355 ClassSize.OBJECT +
5356 (2 * ClassSize.ATOMIC_BOOLEAN) +
5357 (3 * ClassSize.ATOMIC_LONG) +
5358 ClassSize.ATOMIC_INTEGER +
5359 (3 * ClassSize.CONCURRENT_HASHMAP) +
5360 WriteState.HEAP_SIZE +
5361 ClassSize.CONCURRENT_SKIPLISTMAP + ClassSize.CONCURRENT_SKIPLISTMAP_ENTRY +
5362 (2 * ClassSize.REENTRANT_LOCK) +
5363 ClassSize.ARRAYLIST +
5364 MultiVersionConsistencyControl.FIXED_SIZE
5365 ;
5366
5367 @Override
5368 public long heapSize() {
5369 long heapSize = DEEP_OVERHEAD;
5370 for(Store store : this.stores.values()) {
5371 heapSize += store.heapSize();
5372 }
5373
5374 return heapSize;
5375 }
5376
5377
5378
5379
5380
5381 private static void printUsageAndExit(final String message) {
5382 if (message != null && message.length() > 0) System.out.println(message);
5383 System.out.println("Usage: HRegion CATLALOG_TABLE_DIR [major_compact]");
5384 System.out.println("Options:");
5385 System.out.println(" major_compact Pass this option to major compact " +
5386 "passed region.");
5387 System.out.println("Default outputs scan of passed region.");
5388 System.exit(1);
5389 }
5390
5391
5392
5393
5394
5395
5396
5397
5398
5399
5400
5401
5402
5403
5404
5405
5406
5407
5408 public <T extends CoprocessorProtocol> boolean registerProtocol(
5409 Class<T> protocol, T handler) {
5410
5411
5412
5413
5414 if (protocolHandlers.containsKey(protocol)) {
5415 LOG.error("Protocol "+protocol.getName()+
5416 " already registered, rejecting request from "+
5417 handler
5418 );
5419 return false;
5420 }
5421
5422 protocolHandlers.putInstance(protocol, handler);
5423 protocolHandlerNames.put(protocol.getName(), protocol);
5424 if (LOG.isDebugEnabled()) {
5425 LOG.debug("Registered protocol handler: region="+
5426 Bytes.toStringBinary(getRegionName())+" protocol="+protocol.getName());
5427 }
5428 return true;
5429 }
5430
5431
5432
5433
5434
5435
5436
5437
5438
5439
5440
5441
5442
5443
5444
5445
5446 public ExecResult exec(Exec call)
5447 throws IOException {
5448 Class<? extends CoprocessorProtocol> protocol = call.getProtocol();
5449 if (protocol == null) {
5450 String protocolName = call.getProtocolName();
5451 if (LOG.isDebugEnabled()) {
5452 LOG.debug("Received dynamic protocol exec call with protocolName " + protocolName);
5453 }
5454
5455 protocol = protocolHandlerNames.get(protocolName);
5456 if (protocol == null) {
5457 throw new HBaseRPC.UnknownProtocolException(protocol,
5458 "No matching handler for protocol "+protocolName+
5459 " in region "+Bytes.toStringBinary(getRegionName()));
5460 }
5461 }
5462 if (!protocolHandlers.containsKey(protocol)) {
5463 throw new HBaseRPC.UnknownProtocolException(protocol,
5464 "No matching handler for protocol "+protocol.getName()+
5465 " in region "+Bytes.toStringBinary(getRegionName()));
5466 }
5467
5468 CoprocessorProtocol handler = protocolHandlers.getInstance(protocol);
5469 Object value;
5470
5471 try {
5472 Method method = protocol.getMethod(
5473 call.getMethodName(), call.getParameterClasses());
5474 method.setAccessible(true);
5475
5476 value = method.invoke(handler, call.getParameters());
5477 } catch (InvocationTargetException e) {
5478 Throwable target = e.getTargetException();
5479 if (target instanceof IOException) {
5480 throw (IOException)target;
5481 }
5482 IOException ioe = new IOException(target.toString());
5483 ioe.setStackTrace(target.getStackTrace());
5484 throw ioe;
5485 } catch (Throwable e) {
5486 if (!(e instanceof IOException)) {
5487 LOG.error("Unexpected throwable object ", e);
5488 }
5489 IOException ioe = new IOException(e.toString());
5490 ioe.setStackTrace(e.getStackTrace());
5491 throw ioe;
5492 }
5493
5494 return new ExecResult(getRegionName(), value);
5495 }
5496
5497
5498
5499
5500
5501
5502
5503
5504
5505
5506
5507 private static void processTable(final FileSystem fs, final Path p,
5508 final HLog log, final Configuration c,
5509 final boolean majorCompact)
5510 throws IOException {
5511 HRegion region = null;
5512 String rootStr = Bytes.toString(HConstants.ROOT_TABLE_NAME);
5513 String metaStr = Bytes.toString(HConstants.META_TABLE_NAME);
5514
5515 if (p.getName().startsWith(rootStr)) {
5516 region = HRegion.newHRegion(p, log, fs, c, HRegionInfo.ROOT_REGIONINFO,
5517 HTableDescriptor.ROOT_TABLEDESC, null);
5518 } else if (p.getName().startsWith(metaStr)) {
5519 region = HRegion.newHRegion(p, log, fs, c,
5520 HRegionInfo.FIRST_META_REGIONINFO, HTableDescriptor.META_TABLEDESC, null);
5521 } else {
5522 throw new IOException("Not a known catalog table: " + p.toString());
5523 }
5524 try {
5525 region.initialize();
5526 if (majorCompact) {
5527 region.compactStores(true);
5528 } else {
5529
5530 Scan scan = new Scan();
5531
5532 RegionScanner scanner = region.getScanner(scan);
5533 try {
5534 List<KeyValue> kvs = new ArrayList<KeyValue>();
5535 boolean done = false;
5536 do {
5537 kvs.clear();
5538 done = scanner.next(kvs);
5539 if (kvs.size() > 0) LOG.info(kvs);
5540 } while (done);
5541 } finally {
5542 scanner.close();
5543 }
5544 }
5545 } finally {
5546 region.close();
5547 }
5548 }
5549
5550 boolean shouldForceSplit() {
5551 return this.splitRequest;
5552 }
5553
5554 byte[] getExplicitSplitPoint() {
5555 return this.explicitSplitPoint;
5556 }
5557
5558 void forceSplit(byte[] sp) {
5559
5560
5561 this.splitRequest = true;
5562 if (sp != null) {
5563 this.explicitSplitPoint = sp;
5564 }
5565 }
5566
5567 void clearSplit_TESTS_ONLY() {
5568 this.splitRequest = false;
5569 }
5570
5571
5572
5573
5574 protected void prepareToSplit() {
5575
5576 }
5577
5578
5579
5580
5581
5582
5583
5584 public byte[] checkSplit() {
5585
5586 if (this.regionInfo.isMetaTable()) {
5587 if (shouldForceSplit()) {
5588 LOG.warn("Cannot split root/meta regions in HBase 0.20 and above");
5589 }
5590 return null;
5591 }
5592
5593 if (!splitPolicy.shouldSplit()) {
5594 return null;
5595 }
5596
5597 byte[] ret = splitPolicy.getSplitPoint();
5598
5599 if (ret != null) {
5600 try {
5601 checkRow(ret, "calculated split");
5602 } catch (IOException e) {
5603 LOG.error("Ignoring invalid split", e);
5604 return null;
5605 }
5606 }
5607 return ret;
5608 }
5609
5610
5611
5612
5613 public int getCompactPriority() {
5614 int count = Integer.MAX_VALUE;
5615 for(Store store : stores.values()) {
5616 count = Math.min(count, store.getCompactPriority());
5617 }
5618 return count;
5619 }
5620
5621
5622
5623
5624
5625
5626 public boolean needsCompaction() {
5627 for(Store store : stores.values()) {
5628 if(store.needsCompaction()) {
5629 return true;
5630 }
5631 }
5632 return false;
5633 }
5634
5635
5636 public RegionCoprocessorHost getCoprocessorHost() {
5637 return coprocessorHost;
5638 }
5639
5640
5641
5642
5643
5644 public void setOpMetricsReadRequestCount(long value)
5645 {
5646 this.opMetrics.setReadRequestCountMetrics(value);
5647 }
5648
5649
5650
5651
5652
5653 public void setOpMetricsWriteRequestCount(long value)
5654 {
5655 this.opMetrics.setWriteRequestCountMetrics(value);
5656 }
5657
5658
5659 public void setCoprocessorHost(final RegionCoprocessorHost coprocessorHost) {
5660 this.coprocessorHost = coprocessorHost;
5661 }
5662
5663
5664
5665
5666
5667
5668
5669
5670
5671
5672 public void startRegionOperation()
5673 throws NotServingRegionException, RegionTooBusyException, InterruptedIOException {
5674 if (this.closing.get()) {
5675 throw new NotServingRegionException(regionInfo.getRegionNameAsString() +
5676 " is closing");
5677 }
5678 lock(lock.readLock());
5679 if (this.closed.get()) {
5680 lock.readLock().unlock();
5681 throw new NotServingRegionException(regionInfo.getRegionNameAsString() +
5682 " is closed");
5683 }
5684 }
5685
5686
5687
5688
5689
5690 public void closeRegionOperation(){
5691 lock.readLock().unlock();
5692 }
5693
5694
5695
5696
5697
5698
5699
5700
5701
5702
5703 private void startBulkRegionOperation(boolean writeLockNeeded)
5704 throws NotServingRegionException, RegionTooBusyException, InterruptedIOException {
5705 if (this.closing.get()) {
5706 throw new NotServingRegionException(regionInfo.getRegionNameAsString() +
5707 " is closing");
5708 }
5709 if (writeLockNeeded) lock(lock.writeLock());
5710 else lock(lock.readLock());
5711 if (this.closed.get()) {
5712 if (writeLockNeeded) lock.writeLock().unlock();
5713 else lock.readLock().unlock();
5714 throw new NotServingRegionException(regionInfo.getRegionNameAsString() +
5715 " is closed");
5716 }
5717 }
5718
5719
5720
5721
5722
5723 private void closeBulkRegionOperation() {
5724 if (lock.writeLock().isHeldByCurrentThread()) lock.writeLock().unlock();
5725 else lock.readLock().unlock();
5726 }
5727
5728
5729
5730
5731
5732 private void recordPutWithoutWal(final Map<byte [], List<KeyValue>> familyMap) {
5733 if (numPutsWithoutWAL.getAndIncrement() == 0) {
5734 LOG.info("writing data to region " + this +
5735 " with WAL disabled. Data may be lost in the event of a crash.");
5736 }
5737
5738 long putSize = 0;
5739 for (List<KeyValue> edits : familyMap.values()) {
5740 for (KeyValue kv : edits) {
5741 putSize += kv.getKeyLength() + kv.getValueLength();
5742 }
5743 }
5744
5745 dataInMemoryWithoutWAL.addAndGet(putSize);
5746 }
5747
5748 private void lock(final Lock lock)
5749 throws RegionTooBusyException, InterruptedIOException {
5750 lock(lock, 1);
5751 }
5752
5753
5754
5755
5756
5757
5758 private void lock(final Lock lock, final int multiplier)
5759 throws RegionTooBusyException, InterruptedIOException {
5760 try {
5761 final long waitTime = Math.min(maxBusyWaitDuration,
5762 busyWaitDuration * Math.min(multiplier, maxBusyWaitMultiplier));
5763 if (!lock.tryLock(waitTime, TimeUnit.MILLISECONDS)) {
5764 throw new RegionTooBusyException(
5765 "failed to get a lock in " + waitTime + "ms");
5766 }
5767 } catch (InterruptedException ie) {
5768 LOG.info("Interrupted while waiting for a lock");
5769 InterruptedIOException iie = new InterruptedIOException();
5770 iie.initCause(ie);
5771 throw iie;
5772 }
5773 }
5774
5775
5776
5777
5778
5779
5780
5781 private void syncOrDefer(long txid) throws IOException {
5782 if (this.regionInfo.isMetaRegion() ||
5783 !this.htableDescriptor.isDeferredLogFlush() || this.deferredLogSyncDisabled) {
5784 this.log.sync(txid);
5785 }
5786 }
5787
5788
5789
5790
5791 private static final List<KeyValue> MOCKED_LIST = new AbstractList<KeyValue>() {
5792
5793 @Override
5794 public void add(int index, KeyValue element) {
5795
5796 }
5797
5798 @Override
5799 public boolean addAll(int index, Collection<? extends KeyValue> c) {
5800 return false;
5801 }
5802
5803 @Override
5804 public KeyValue get(int index) {
5805 throw new UnsupportedOperationException();
5806 }
5807
5808 @Override
5809 public int size() {
5810 return 0;
5811 }
5812 };
5813
5814
5815
5816
5817
5818
5819
5820
5821
5822
5823
5824 public static void main(String[] args) throws IOException {
5825 if (args.length < 1) {
5826 printUsageAndExit(null);
5827 }
5828 boolean majorCompact = false;
5829 if (args.length > 1) {
5830 if (!args[1].toLowerCase().startsWith("major")) {
5831 printUsageAndExit("ERROR: Unrecognized option <" + args[1] + ">");
5832 }
5833 majorCompact = true;
5834 }
5835 final Path tableDir = new Path(args[0]);
5836 final Configuration c = HBaseConfiguration.create();
5837 final FileSystem fs = FileSystem.get(c);
5838 final Path logdir = new Path(c.get("hbase.tmp.dir"),
5839 "hlog" + tableDir.getName()
5840 + EnvironmentEdgeManager.currentTimeMillis());
5841 final Path oldLogDir = new Path(c.get("hbase.tmp.dir"),
5842 HConstants.HREGION_OLDLOGDIR_NAME);
5843 final HLog log = new HLog(fs, logdir, oldLogDir, c);
5844 try {
5845 processTable(fs, tableDir, log, c, majorCompact);
5846 } finally {
5847 log.close();
5848
5849 BlockCache bc = new CacheConfig(c).getBlockCache();
5850 if (bc != null) bc.shutdown();
5851 }
5852 }
5853
5854
5855
5856
5857
5858
5859 public static interface BulkLoadListener {
5860
5861
5862
5863
5864
5865
5866
5867
5868 String prepareBulkLoad(byte[] family, String srcPath) throws IOException;
5869
5870
5871
5872
5873
5874
5875
5876 void doneBulkLoad(byte[] family, String srcPath) throws IOException;
5877
5878
5879
5880
5881
5882
5883
5884 void failedBulkLoad(byte[] family, String srcPath) throws IOException;
5885
5886 }
5887 }