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.IOException;
24 import java.io.InterruptedIOException;
25 import java.io.UnsupportedEncodingException;
26 import java.lang.reflect.Constructor;
27 import java.lang.reflect.InvocationTargetException;
28 import java.lang.reflect.Method;
29 import java.text.ParseException;
30 import java.util.AbstractList;
31 import java.util.ArrayList;
32 import java.util.Arrays;
33 import java.util.Collection;
34 import java.util.Collections;
35 import java.util.HashMap;
36 import java.util.List;
37 import java.util.Map;
38 import java.util.NavigableMap;
39 import java.util.NavigableSet;
40 import java.util.Random;
41 import java.util.TreeMap;
42 import java.util.UUID;
43 import java.util.concurrent.ConcurrentHashMap;
44 import java.util.concurrent.ConcurrentSkipListMap;
45 import java.util.concurrent.CountDownLatch;
46 import java.util.concurrent.TimeUnit;
47 import java.util.concurrent.atomic.AtomicBoolean;
48 import java.util.concurrent.atomic.AtomicInteger;
49 import java.util.concurrent.atomic.AtomicLong;
50 import java.util.concurrent.locks.ReentrantReadWriteLock;
51
52 import org.apache.commons.logging.Log;
53 import org.apache.commons.logging.LogFactory;
54 import org.apache.hadoop.conf.Configuration;
55 import org.apache.hadoop.fs.FSDataOutputStream;
56 import org.apache.hadoop.fs.FileStatus;
57 import org.apache.hadoop.fs.FileSystem;
58 import org.apache.hadoop.fs.Path;
59 import org.apache.hadoop.hbase.DoNotRetryIOException;
60 import org.apache.hadoop.hbase.DroppedSnapshotException;
61 import org.apache.hadoop.hbase.HBaseConfiguration;
62 import org.apache.hadoop.hbase.HColumnDescriptor;
63 import org.apache.hadoop.hbase.HConstants;
64 import org.apache.hadoop.hbase.HDFSBlocksDistribution;
65 import org.apache.hadoop.hbase.HRegionInfo;
66 import org.apache.hadoop.hbase.HTableDescriptor;
67 import org.apache.hadoop.hbase.KeyValue;
68 import org.apache.hadoop.hbase.NotServingRegionException;
69 import org.apache.hadoop.hbase.UnknownScannerException;
70 import org.apache.hadoop.hbase.HConstants.OperationStatusCode;
71 import org.apache.hadoop.hbase.client.Delete;
72 import org.apache.hadoop.hbase.client.Get;
73 import org.apache.hadoop.hbase.client.Increment;
74 import org.apache.hadoop.hbase.client.Put;
75 import org.apache.hadoop.hbase.client.Result;
76 import org.apache.hadoop.hbase.client.Row;
77 import org.apache.hadoop.hbase.client.RowLock;
78 import org.apache.hadoop.hbase.client.Scan;
79 import org.apache.hadoop.hbase.client.coprocessor.Exec;
80 import org.apache.hadoop.hbase.client.coprocessor.ExecResult;
81 import org.apache.hadoop.hbase.filter.Filter;
82 import org.apache.hadoop.hbase.filter.IncompatibleFilterException;
83 import org.apache.hadoop.hbase.filter.WritableByteArrayComparable;
84 import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
85 import org.apache.hadoop.hbase.io.HeapSize;
86 import org.apache.hadoop.hbase.io.TimeRange;
87 import org.apache.hadoop.hbase.io.hfile.BlockCache;
88 import org.apache.hadoop.hbase.io.hfile.CacheConfig;
89 import org.apache.hadoop.hbase.ipc.CoprocessorProtocol;
90 import org.apache.hadoop.hbase.ipc.HBaseRPC;
91 import org.apache.hadoop.hbase.monitoring.MonitoredTask;
92 import org.apache.hadoop.hbase.monitoring.TaskMonitor;
93 import org.apache.hadoop.hbase.regionserver.compactions.CompactionRequest;
94 import org.apache.hadoop.hbase.regionserver.wal.HLog;
95 import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
96 import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
97 import org.apache.hadoop.hbase.util.Bytes;
98 import org.apache.hadoop.hbase.util.CancelableProgressable;
99 import org.apache.hadoop.hbase.util.ClassSize;
100 import org.apache.hadoop.hbase.util.CompressionTest;
101 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
102 import org.apache.hadoop.hbase.util.FSUtils;
103 import org.apache.hadoop.hbase.util.HashedBytes;
104 import org.apache.hadoop.hbase.util.Pair;
105 import org.apache.hadoop.hbase.util.Writables;
106 import org.apache.hadoop.io.MultipleIOException;
107 import org.apache.hadoop.io.Writable;
108 import org.apache.hadoop.util.StringUtils;
109 import org.cliffc.high_scale_lib.Counter;
110
111 import com.google.common.base.Preconditions;
112 import com.google.common.collect.ClassToInstanceMap;
113 import com.google.common.collect.Lists;
114 import com.google.common.collect.MutableClassToInstanceMap;
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152 public class HRegion implements HeapSize {
153 public static final Log LOG = LogFactory.getLog(HRegion.class);
154 static final String MERGEDIR = "merges";
155
156 final AtomicBoolean closed = new AtomicBoolean(false);
157
158
159
160
161
162 final AtomicBoolean closing = new AtomicBoolean(false);
163
164
165
166
167
168 private final ConcurrentHashMap<HashedBytes, CountDownLatch> lockedRows =
169 new ConcurrentHashMap<HashedBytes, CountDownLatch>();
170 private final ConcurrentHashMap<Integer, HashedBytes> lockIds =
171 new ConcurrentHashMap<Integer, HashedBytes>();
172 private final AtomicInteger lockIdGenerator = new AtomicInteger(1);
173 static private Random rand = new Random();
174
175 protected final Map<byte [], Store> stores =
176 new ConcurrentSkipListMap<byte [], Store>(Bytes.BYTES_RAWCOMPARATOR);
177
178
179 private ClassToInstanceMap<CoprocessorProtocol>
180 protocolHandlers = MutableClassToInstanceMap.create();
181
182
183
184
185
186
187
188 final AtomicLong memstoreSize = new AtomicLong(0);
189
190 final Counter readRequestsCount = new Counter();
191 final Counter writeRequestsCount = new Counter();
192
193
194
195
196
197 final Path tableDir;
198
199 final HLog log;
200 final FileSystem fs;
201 final Configuration conf;
202 final int rowLockWaitDuration;
203 static final int DEFAULT_ROWLOCK_WAIT_DURATION = 30000;
204 final HRegionInfo regionInfo;
205 final Path regiondir;
206 KeyValue.KVComparator comparator;
207
208 private ConcurrentHashMap<RegionScanner, Long> scannerReadPoints;
209
210
211
212
213
214
215 public long getSmallestReadPoint() {
216 long minimumReadPoint;
217
218
219
220 synchronized(scannerReadPoints) {
221 minimumReadPoint = mvcc.memstoreReadPoint();
222
223 for (Long readPoint: this.scannerReadPoints.values()) {
224 if (readPoint < minimumReadPoint) {
225 minimumReadPoint = readPoint;
226 }
227 }
228 }
229 return minimumReadPoint;
230 }
231
232
233
234
235 static class WriteState {
236
237 volatile boolean flushing = false;
238
239 volatile boolean flushRequested = false;
240
241 volatile int compacting = 0;
242
243 volatile boolean writesEnabled = true;
244
245 volatile boolean readOnly = false;
246
247
248
249
250
251
252 synchronized void setReadOnly(final boolean onOff) {
253 this.writesEnabled = !onOff;
254 this.readOnly = onOff;
255 }
256
257 boolean isReadOnly() {
258 return this.readOnly;
259 }
260
261 boolean isFlushRequested() {
262 return this.flushRequested;
263 }
264
265 static final long HEAP_SIZE = ClassSize.align(
266 ClassSize.OBJECT + 5 * Bytes.SIZEOF_BOOLEAN);
267 }
268
269 final WriteState writestate = new WriteState();
270
271 long memstoreFlushSize;
272 private volatile long lastFlushTime;
273 final RegionServerServices rsServices;
274 private List<Pair<Long, Long>> recentFlushes = new ArrayList<Pair<Long,Long>>();
275 private long blockingMemStoreSize;
276 final long threadWakeFrequency;
277
278 final ReentrantReadWriteLock lock =
279 new ReentrantReadWriteLock();
280
281
282 private final ReentrantReadWriteLock updatesLock =
283 new ReentrantReadWriteLock();
284 private boolean splitRequest;
285 private byte[] explicitSplitPoint = null;
286
287 private final MultiVersionConsistencyControl mvcc =
288 new MultiVersionConsistencyControl();
289
290
291 private RegionCoprocessorHost coprocessorHost;
292
293
294
295
296 public final static String REGIONINFO_FILE = ".regioninfo";
297 private HTableDescriptor htableDescriptor = null;
298 private RegionSplitPolicy splitPolicy;
299
300
301
302
303 public HRegion(){
304 this.tableDir = null;
305 this.blockingMemStoreSize = 0L;
306 this.conf = null;
307 this.rowLockWaitDuration = DEFAULT_ROWLOCK_WAIT_DURATION;
308 this.rsServices = null;
309 this.fs = null;
310 this.memstoreFlushSize = 0L;
311 this.log = null;
312 this.regiondir = null;
313 this.regionInfo = null;
314 this.htableDescriptor = null;
315 this.threadWakeFrequency = 0L;
316 this.coprocessorHost = null;
317 this.scannerReadPoints = new ConcurrentHashMap<RegionScanner, Long>();
318 }
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343 public HRegion(Path tableDir, HLog log, FileSystem fs, Configuration conf,
344 HRegionInfo regionInfo, final HTableDescriptor htd,
345 RegionServerServices rsServices) {
346 this.tableDir = tableDir;
347 this.comparator = regionInfo.getComparator();
348 this.log = log;
349 this.fs = fs;
350 this.conf = conf;
351 this.rowLockWaitDuration = conf.getInt("hbase.rowlock.wait.duration",
352 DEFAULT_ROWLOCK_WAIT_DURATION);
353 this.regionInfo = regionInfo;
354 this.htableDescriptor = htd;
355 this.rsServices = rsServices;
356 this.threadWakeFrequency = conf.getLong(HConstants.THREAD_WAKE_FREQUENCY,
357 10 * 1000);
358 String encodedNameStr = this.regionInfo.getEncodedName();
359 setHTableSpecificConf();
360 this.regiondir = getRegionDir(this.tableDir, encodedNameStr);
361 this.scannerReadPoints = new ConcurrentHashMap<RegionScanner, Long>();
362
363
364
365 if (rsServices != null) {
366 this.coprocessorHost = new RegionCoprocessorHost(this, rsServices, conf);
367 }
368 if (LOG.isDebugEnabled()) {
369
370 LOG.debug("Instantiated " + this);
371 }
372 }
373
374 void setHTableSpecificConf() {
375 if (this.htableDescriptor == null) return;
376 LOG.info("Setting up tabledescriptor config now ...");
377 long flushSize = this.htableDescriptor.getMemStoreFlushSize();
378 if (flushSize == HTableDescriptor.DEFAULT_MEMSTORE_FLUSH_SIZE) {
379 flushSize = conf.getLong("hbase.hregion.memstore.flush.size",
380 HTableDescriptor.DEFAULT_MEMSTORE_FLUSH_SIZE);
381 }
382 this.memstoreFlushSize = flushSize;
383 this.blockingMemStoreSize = this.memstoreFlushSize *
384 conf.getLong("hbase.hregion.memstore.block.multiplier", 2);
385 }
386
387
388
389
390
391
392 public long initialize() throws IOException {
393 return initialize(null);
394 }
395
396
397
398
399
400
401
402
403 public long initialize(final CancelableProgressable reporter)
404 throws IOException {
405
406 MonitoredTask status = TaskMonitor.get().createStatus(
407 "Initializing region " + this);
408
409 if (coprocessorHost != null) {
410 status.setStatus("Running coprocessor pre-open hook");
411 coprocessorHost.preOpen();
412 }
413
414 this.closing.set(false);
415 this.closed.set(false);
416
417
418 status.setStatus("Writing region info on filesystem");
419 checkRegioninfoOnFilesystem();
420
421
422 status.setStatus("Cleaning up temporary data from old regions");
423 cleanupTmpDir();
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441 long minSeqId = -1;
442 long maxSeqId = -1;
443
444 long maxMemstoreTS = -1;
445 for (HColumnDescriptor c : this.htableDescriptor.getFamilies()) {
446 status.setStatus("Instantiating store for column family " + c);
447 Store store = instantiateHStore(this.tableDir, c);
448 this.stores.put(c.getName(), store);
449 long storeSeqId = store.getMaxSequenceId();
450 if (minSeqId == -1 || storeSeqId < minSeqId) {
451 minSeqId = storeSeqId;
452 }
453 if (maxSeqId == -1 || storeSeqId > maxSeqId) {
454 maxSeqId = storeSeqId;
455 }
456 long maxStoreMemstoreTS = store.getMaxMemstoreTS();
457 if (maxStoreMemstoreTS > maxMemstoreTS) {
458 maxMemstoreTS = maxStoreMemstoreTS;
459 }
460 }
461 mvcc.initialize(maxMemstoreTS + 1);
462
463 maxSeqId = Math.max(maxSeqId, replayRecoveredEditsIfAny(
464 this.regiondir, minSeqId, reporter, status));
465
466 status.setStatus("Cleaning up detritus from prior splits");
467
468
469
470 SplitTransaction.cleanupAnySplitDetritus(this);
471 FSUtils.deleteDirectory(this.fs, new Path(regiondir, MERGEDIR));
472
473 this.writestate.setReadOnly(this.htableDescriptor.isReadOnly());
474
475 this.writestate.compacting = 0;
476
477
478 this.splitPolicy = RegionSplitPolicy.create(this, conf);
479
480 this.lastFlushTime = EnvironmentEdgeManager.currentTimeMillis();
481
482
483 long nextSeqid = maxSeqId + 1;
484 LOG.info("Onlined " + this.toString() + "; next sequenceid=" + nextSeqid);
485
486
487 if (coprocessorHost != null) {
488 status.setStatus("Running coprocessor post-open hooks");
489 coprocessorHost.postOpen();
490 }
491 status.markComplete("Region opened successfully");
492 return nextSeqid;
493 }
494
495
496
497
498
499
500
501 static void moveInitialFilesIntoPlace(final FileSystem fs,
502 final Path initialFiles, final Path regiondir)
503 throws IOException {
504 if (initialFiles != null && fs.exists(initialFiles)) {
505 if (!fs.rename(initialFiles, regiondir)) {
506 LOG.warn("Unable to rename " + initialFiles + " to " + regiondir);
507 }
508 }
509 }
510
511
512
513
514 public boolean hasReferences() {
515 for (Store store : this.stores.values()) {
516 for (StoreFile sf : store.getStorefiles()) {
517
518 if (sf.isReference()) return true;
519 }
520 }
521 return false;
522 }
523
524
525
526
527
528
529 public HDFSBlocksDistribution getHDFSBlocksDistribution() {
530 HDFSBlocksDistribution hdfsBlocksDistribution =
531 new HDFSBlocksDistribution();
532 synchronized (this.stores) {
533 for (Store store : this.stores.values()) {
534 for (StoreFile sf : store.getStorefiles()) {
535 HDFSBlocksDistribution storeFileBlocksDistribution =
536 sf.getHDFSBlockDistribution();
537 hdfsBlocksDistribution.add(storeFileBlocksDistribution);
538 }
539 }
540 }
541 return hdfsBlocksDistribution;
542 }
543
544
545
546
547
548
549
550
551
552 static public HDFSBlocksDistribution computeHDFSBlocksDistribution(
553 Configuration conf, HTableDescriptor tableDescriptor,
554 String regionEncodedName) throws IOException {
555 HDFSBlocksDistribution hdfsBlocksDistribution =
556 new HDFSBlocksDistribution();
557 Path tablePath = FSUtils.getTablePath(FSUtils.getRootDir(conf),
558 tableDescriptor.getName());
559 FileSystem fs = tablePath.getFileSystem(conf);
560
561 for (HColumnDescriptor family: tableDescriptor.getFamilies()) {
562 Path storeHomeDir = Store.getStoreHomedir(tablePath, regionEncodedName,
563 family.getName());
564 if (!fs.exists(storeHomeDir))continue;
565
566 FileStatus[] hfilesStatus = null;
567 hfilesStatus = fs.listStatus(storeHomeDir);
568
569 for (FileStatus hfileStatus : hfilesStatus) {
570 HDFSBlocksDistribution storeFileBlocksDistribution =
571 FSUtils.computeHDFSBlocksDistribution(fs, hfileStatus, 0,
572 hfileStatus.getLen());
573 hdfsBlocksDistribution.add(storeFileBlocksDistribution);
574 }
575 }
576 return hdfsBlocksDistribution;
577 }
578
579 public AtomicLong getMemstoreSize() {
580 return memstoreSize;
581 }
582
583
584
585
586
587
588
589 public long addAndGetGlobalMemstoreSize(long memStoreSize) {
590 if (this.rsServices != null) {
591 RegionServerAccounting rsAccounting =
592 this.rsServices.getRegionServerAccounting();
593
594 if (rsAccounting != null) {
595 rsAccounting.addAndGetGlobalMemstoreSize(memStoreSize);
596 }
597 }
598 return this.memstoreSize.getAndAdd(memStoreSize);
599 }
600
601
602
603
604
605
606 private void checkRegioninfoOnFilesystem() throws IOException {
607 Path regioninfoPath = new Path(this.regiondir, REGIONINFO_FILE);
608 if (this.fs.exists(regioninfoPath) &&
609 this.fs.getFileStatus(regioninfoPath).getLen() > 0) {
610 return;
611 }
612
613
614
615
616 Path tmpPath = new Path(getTmpDir(), REGIONINFO_FILE);
617 FSDataOutputStream out = this.fs.create(tmpPath, true);
618 try {
619 this.regionInfo.write(out);
620 out.write('\n');
621 out.write('\n');
622 out.write(Bytes.toBytes(this.regionInfo.toString()));
623 } finally {
624 out.close();
625 }
626 if (!fs.rename(tmpPath, regioninfoPath)) {
627 throw new IOException("Unable to rename " + tmpPath + " to " +
628 regioninfoPath);
629 }
630 }
631
632
633 public HRegionInfo getRegionInfo() {
634 return this.regionInfo;
635 }
636
637
638 public long getRequestsCount() {
639 return this.readRequestsCount.get() + this.writeRequestsCount.get();
640 }
641
642
643 public long getReadRequestsCount() {
644 return this.readRequestsCount.get();
645 }
646
647
648 public long getWriteRequestsCount() {
649 return this.writeRequestsCount.get();
650 }
651
652
653 public boolean isClosed() {
654 return this.closed.get();
655 }
656
657
658
659
660 public boolean isClosing() {
661 return this.closing.get();
662 }
663
664 boolean areWritesEnabled() {
665 synchronized(this.writestate) {
666 return this.writestate.writesEnabled;
667 }
668 }
669
670 public MultiVersionConsistencyControl getMVCC() {
671 return mvcc;
672 }
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687 public List<StoreFile> close() throws IOException {
688 return close(false);
689 }
690
691 private final Object closeLock = new Object();
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707 public List<StoreFile> close(final boolean abort) throws IOException {
708
709
710 MonitoredTask status = TaskMonitor.get().createStatus(
711 "Closing region " + this +
712 (abort ? " due to abort" : ""));
713
714 status.setStatus("Waiting for close lock");
715 try {
716 synchronized (closeLock) {
717 return doClose(abort, status);
718 }
719 } finally {
720 status.cleanup();
721 }
722 }
723
724 private List<StoreFile> doClose(
725 final boolean abort, MonitoredTask status)
726 throws IOException {
727 if (isClosed()) {
728 LOG.warn("Region " + this + " already closed");
729 return null;
730 }
731
732 if (coprocessorHost != null) {
733 status.setStatus("Running coprocessor pre-close hooks");
734 this.coprocessorHost.preClose(abort);
735 }
736
737 status.setStatus("Disabling compacts and flushes for region");
738 boolean wasFlushing = false;
739 synchronized (writestate) {
740
741
742 writestate.writesEnabled = false;
743 wasFlushing = writestate.flushing;
744 LOG.debug("Closing " + this + ": disabling compactions & flushes");
745 while (writestate.compacting > 0 || writestate.flushing) {
746 LOG.debug("waiting for " + writestate.compacting + " compactions" +
747 (writestate.flushing ? " & cache flush" : "") +
748 " to complete for region " + this);
749 try {
750 writestate.wait();
751 } catch (InterruptedException iex) {
752
753 }
754 }
755 }
756
757
758
759 if (!abort && !wasFlushing && worthPreFlushing()) {
760 status.setStatus("Pre-flushing region before close");
761 LOG.info("Running close preflush of " + this.getRegionNameAsString());
762 internalFlushcache(status);
763 }
764
765 this.closing.set(true);
766 status.setStatus("Disabling writes for close");
767 lock.writeLock().lock();
768 try {
769 if (this.isClosed()) {
770 status.abort("Already got closed by another process");
771
772 return null;
773 }
774 LOG.debug("Updates disabled for region " + this);
775
776 if (!abort) {
777 internalFlushcache(status);
778 }
779
780 List<StoreFile> result = new ArrayList<StoreFile>();
781 for (Store store : stores.values()) {
782 result.addAll(store.close());
783 }
784 this.closed.set(true);
785
786 if (coprocessorHost != null) {
787 status.setStatus("Running coprocessor post-close hooks");
788 this.coprocessorHost.postClose(abort);
789 }
790 status.markComplete("Closed");
791 LOG.info("Closed " + this);
792 return result;
793 } finally {
794 lock.writeLock().unlock();
795 }
796 }
797
798
799
800
801 private boolean worthPreFlushing() {
802 return this.memstoreSize.get() >
803 this.conf.getLong("hbase.hregion.preclose.flush.size", 1024 * 1024 * 5);
804 }
805
806
807
808
809
810
811 public byte [] getStartKey() {
812 return this.regionInfo.getStartKey();
813 }
814
815
816 public byte [] getEndKey() {
817 return this.regionInfo.getEndKey();
818 }
819
820
821 public long getRegionId() {
822 return this.regionInfo.getRegionId();
823 }
824
825
826 public byte [] getRegionName() {
827 return this.regionInfo.getRegionName();
828 }
829
830
831 public String getRegionNameAsString() {
832 return this.regionInfo.getRegionNameAsString();
833 }
834
835
836 public HTableDescriptor getTableDesc() {
837 return this.htableDescriptor;
838 }
839
840
841 public HLog getLog() {
842 return this.log;
843 }
844
845
846 public Configuration getConf() {
847 return this.conf;
848 }
849
850
851 public Path getRegionDir() {
852 return this.regiondir;
853 }
854
855
856
857
858
859
860
861
862 public static Path getRegionDir(final Path tabledir, final String name) {
863 return new Path(tabledir, name);
864 }
865
866
867 public FileSystem getFilesystem() {
868 return this.fs;
869 }
870
871
872 public long getLastFlushTime() {
873 return this.lastFlushTime;
874 }
875
876
877 public List<Pair<Long,Long>> getRecentFlushInfo() {
878 this.lock.readLock().lock();
879 List<Pair<Long,Long>> ret = this.recentFlushes;
880 this.recentFlushes = new ArrayList<Pair<Long,Long>>();
881 this.lock.readLock().unlock();
882 return ret;
883 }
884
885
886
887
888
889
890
891
892
893 public long getLargestHStoreSize() {
894 long size = 0;
895 for (Store h: stores.values()) {
896 long storeSize = h.getSize();
897 if (storeSize > size) {
898 size = storeSize;
899 }
900 }
901 return size;
902 }
903
904
905
906
907
908 void doRegionCompactionPrep() throws IOException {
909 }
910
911
912
913
914 private void cleanupTmpDir() throws IOException {
915 FSUtils.deleteDirectory(this.fs, getTmpDir());
916 }
917
918
919
920
921
922 Path getTmpDir() {
923 return new Path(getRegionDir(), ".tmp");
924 }
925
926 void triggerMajorCompaction() {
927 for (Store h: stores.values()) {
928 h.triggerMajorCompaction();
929 }
930 }
931
932
933
934
935
936
937
938
939 void compactStores(final boolean majorCompaction)
940 throws IOException {
941 if (majorCompaction) {
942 this.triggerMajorCompaction();
943 }
944 compactStores();
945 }
946
947
948
949
950
951
952
953 public void compactStores() throws IOException {
954 for(Store s : getStores().values()) {
955 CompactionRequest cr = s.requestCompaction();
956 if(cr != null) {
957 try {
958 compact(cr);
959 } finally {
960 s.finishRequest(cr);
961 }
962 }
963 }
964 }
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981 public boolean compact(CompactionRequest cr)
982 throws IOException {
983 if (cr == null) {
984 return false;
985 }
986 if (this.closing.get() || this.closed.get()) {
987 LOG.debug("Skipping compaction on " + this + " because closing/closed");
988 return false;
989 }
990 Preconditions.checkArgument(cr.getHRegion().equals(this));
991 lock.readLock().lock();
992 MonitoredTask status = TaskMonitor.get().createStatus(
993 "Compacting " + cr.getStore() + " in " + this);
994 try {
995 if (this.closed.get()) {
996 LOG.debug("Skipping compaction on " + this + " because closed");
997 return false;
998 }
999 boolean decr = true;
1000 try {
1001 synchronized (writestate) {
1002 if (writestate.writesEnabled) {
1003 ++writestate.compacting;
1004 } else {
1005 String msg = "NOT compacting region " + this + ". Writes disabled.";
1006 LOG.info(msg);
1007 status.abort(msg);
1008 decr = false;
1009 return false;
1010 }
1011 }
1012 LOG.info("Starting compaction on " + cr.getStore() + " in region "
1013 + this);
1014 doRegionCompactionPrep();
1015 try {
1016 status.setStatus("Compacting store " + cr.getStore());
1017 cr.getStore().compact(cr);
1018 } catch (InterruptedIOException iioe) {
1019 String msg = "compaction interrupted by user";
1020 LOG.info(msg, iioe);
1021 status.abort(msg);
1022 return false;
1023 }
1024 } finally {
1025 if (decr) {
1026 synchronized (writestate) {
1027 --writestate.compacting;
1028 if (writestate.compacting <= 0) {
1029 writestate.notifyAll();
1030 }
1031 }
1032 }
1033 }
1034 status.markComplete("Compaction complete");
1035 return true;
1036 } finally {
1037 status.cleanup();
1038 lock.readLock().unlock();
1039 }
1040 }
1041
1042
1043
1044
1045
1046
1047
1048
1049
1050
1051
1052
1053
1054
1055
1056
1057
1058
1059
1060
1061
1062 public boolean flushcache() throws IOException {
1063
1064 if (this.closing.get()) {
1065 LOG.debug("Skipping flush on " + this + " because closing");
1066 return false;
1067 }
1068 MonitoredTask status = TaskMonitor.get().createStatus("Flushing " + this);
1069 status.setStatus("Acquiring readlock on region");
1070 lock.readLock().lock();
1071 try {
1072 if (this.closed.get()) {
1073 LOG.debug("Skipping flush on " + this + " because closed");
1074 status.abort("Skipped: closed");
1075 return false;
1076 }
1077 if (coprocessorHost != null) {
1078 status.setStatus("Running coprocessor pre-flush hooks");
1079 coprocessorHost.preFlush();
1080 }
1081 try {
1082 synchronized (writestate) {
1083 if (!writestate.flushing && writestate.writesEnabled) {
1084 this.writestate.flushing = true;
1085 } else {
1086 if (LOG.isDebugEnabled()) {
1087 LOG.debug("NOT flushing memstore for region " + this +
1088 ", flushing=" +
1089 writestate.flushing + ", writesEnabled=" +
1090 writestate.writesEnabled);
1091 }
1092 status.abort("Not flushing since " +
1093 (writestate.flushing ? "already flushing" : "writes not enabled"));
1094 return false;
1095 }
1096 }
1097 boolean result = internalFlushcache(status);
1098
1099 if (coprocessorHost != null) {
1100 status.setStatus("Running post-flush coprocessor hooks");
1101 coprocessorHost.postFlush();
1102 }
1103
1104 status.markComplete("Flush successful");
1105 return result;
1106 } finally {
1107 synchronized (writestate) {
1108 writestate.flushing = false;
1109 this.writestate.flushRequested = false;
1110 writestate.notifyAll();
1111 }
1112 }
1113 } finally {
1114 lock.readLock().unlock();
1115 status.cleanup();
1116 }
1117 }
1118
1119
1120
1121
1122
1123
1124
1125
1126
1127
1128
1129
1130
1131
1132
1133
1134
1135
1136
1137
1138
1139
1140
1141
1142
1143
1144
1145
1146
1147
1148
1149
1150
1151
1152
1153
1154 protected boolean internalFlushcache(MonitoredTask status) throws IOException {
1155 return internalFlushcache(this.log, -1, status);
1156 }
1157
1158
1159
1160
1161
1162
1163
1164
1165
1166
1167 protected boolean internalFlushcache(
1168 final HLog wal, final long myseqid, MonitoredTask status)
1169 throws IOException {
1170 final long startTime = EnvironmentEdgeManager.currentTimeMillis();
1171
1172
1173 this.lastFlushTime = startTime;
1174
1175 if (this.memstoreSize.get() <= 0) {
1176 return false;
1177 }
1178 if (LOG.isDebugEnabled()) {
1179 LOG.debug("Started memstore flush for " + this +
1180 ", current region memstore size " +
1181 StringUtils.humanReadableInt(this.memstoreSize.get()) +
1182 ((wal != null)? "": "; wal is null, using passed sequenceid=" + myseqid));
1183 }
1184
1185
1186
1187
1188
1189
1190
1191
1192 long sequenceId = -1L;
1193 long completeSequenceId = -1L;
1194 MultiVersionConsistencyControl.WriteEntry w = null;
1195
1196
1197
1198
1199 status.setStatus("Obtaining lock to block concurrent updates");
1200 this.updatesLock.writeLock().lock();
1201 long flushsize = this.memstoreSize.get();
1202 status.setStatus("Preparing to flush by snapshotting stores");
1203 List<StoreFlusher> storeFlushers = new ArrayList<StoreFlusher>(stores.size());
1204 try {
1205
1206 w = mvcc.beginMemstoreInsert();
1207 mvcc.advanceMemstore(w);
1208
1209 sequenceId = (wal == null)? myseqid:
1210 wal.startCacheFlush(this.regionInfo.getEncodedNameAsBytes());
1211 completeSequenceId = this.getCompleteCacheFlushSequenceId(sequenceId);
1212
1213 for (Store s : stores.values()) {
1214 storeFlushers.add(s.getStoreFlusher(completeSequenceId));
1215 }
1216
1217
1218 for (StoreFlusher flusher : storeFlushers) {
1219 flusher.prepare();
1220 }
1221 } finally {
1222 this.updatesLock.writeLock().unlock();
1223 }
1224 String s = "Finished snapshotting " + this +
1225 ", commencing wait for mvcc, flushsize=" + flushsize;
1226 status.setStatus(s);
1227 LOG.debug(s);
1228
1229
1230
1231
1232
1233
1234 mvcc.waitForRead(w);
1235
1236
1237
1238
1239
1240 boolean compactionRequested = false;
1241 try {
1242
1243
1244
1245
1246 for (StoreFlusher flusher : storeFlushers) {
1247 flusher.flushCache(status);
1248 }
1249
1250
1251 for (StoreFlusher flusher : storeFlushers) {
1252 boolean needsCompaction = flusher.commit();
1253 if (needsCompaction) {
1254 compactionRequested = true;
1255 }
1256 }
1257 storeFlushers.clear();
1258
1259
1260 this.addAndGetGlobalMemstoreSize(-flushsize);
1261 } catch (Throwable t) {
1262
1263
1264
1265
1266
1267
1268 if (wal != null) {
1269 wal.abortCacheFlush(this.regionInfo.getEncodedNameAsBytes());
1270 }
1271 DroppedSnapshotException dse = new DroppedSnapshotException("region: " +
1272 Bytes.toStringBinary(getRegionName()));
1273 dse.initCause(t);
1274 status.abort("Flush failed: " + StringUtils.stringifyException(t));
1275 throw dse;
1276 }
1277
1278
1279
1280
1281
1282
1283
1284
1285 if (wal != null) {
1286 wal.completeCacheFlush(this.regionInfo.getEncodedNameAsBytes(),
1287 regionInfo.getTableName(), completeSequenceId,
1288 this.getRegionInfo().isMetaRegion());
1289 }
1290
1291
1292
1293 synchronized (this) {
1294 notifyAll();
1295 }
1296
1297 long time = EnvironmentEdgeManager.currentTimeMillis() - startTime;
1298 long memstoresize = this.memstoreSize.get();
1299 String msg = "Finished memstore flush of ~" +
1300 StringUtils.humanReadableInt(flushsize) + "/" + flushsize +
1301 ", currentsize=" +
1302 StringUtils.humanReadableInt(memstoresize) + "/" + memstoresize +
1303 " for region " + this + " in " + time + "ms, sequenceid=" + sequenceId +
1304 ", compaction requested=" + compactionRequested +
1305 ((wal == null)? "; wal=null": "");
1306 LOG.info(msg);
1307 status.setStatus(msg);
1308 this.recentFlushes.add(new Pair<Long,Long>(time/1000, flushsize));
1309
1310 return compactionRequested;
1311 }
1312
1313
1314
1315
1316
1317
1318
1319
1320
1321 protected long getCompleteCacheFlushSequenceId(long currentSequenceId) {
1322 return currentSequenceId;
1323 }
1324
1325
1326
1327
1328
1329
1330
1331
1332
1333
1334
1335
1336
1337 Result getClosestRowBefore(final byte [] row)
1338 throws IOException{
1339 return getClosestRowBefore(row, HConstants.CATALOG_FAMILY);
1340 }
1341
1342
1343
1344
1345
1346
1347
1348
1349
1350
1351
1352 public Result getClosestRowBefore(final byte [] row, final byte [] family)
1353 throws IOException {
1354 if (coprocessorHost != null) {
1355 Result result = new Result();
1356 if (coprocessorHost.preGetClosestRowBefore(row, family, result)) {
1357 return result;
1358 }
1359 }
1360
1361
1362 checkRow(row, "getClosestRowBefore");
1363 startRegionOperation();
1364 this.readRequestsCount.increment();
1365 try {
1366 Store store = getStore(family);
1367
1368 KeyValue key = store.getRowKeyAtOrBefore(row);
1369 Result result = null;
1370 if (key != null) {
1371 Get get = new Get(key.getRow());
1372 get.addFamily(family);
1373 result = get(get, null);
1374 }
1375 if (coprocessorHost != null) {
1376 coprocessorHost.postGetClosestRowBefore(row, family, result);
1377 }
1378 return result;
1379 } finally {
1380 closeRegionOperation();
1381 }
1382 }
1383
1384
1385
1386
1387
1388
1389
1390
1391
1392
1393
1394 public RegionScanner getScanner(Scan scan) throws IOException {
1395 return getScanner(scan, null);
1396 }
1397
1398 void prepareScanner(Scan scan) throws IOException {
1399 if(!scan.hasFamilies()) {
1400
1401 for(byte[] family: this.htableDescriptor.getFamiliesKeys()){
1402 scan.addFamily(family);
1403 }
1404 }
1405 }
1406
1407 protected RegionScanner getScanner(Scan scan,
1408 List<KeyValueScanner> additionalScanners) throws IOException {
1409 startRegionOperation();
1410 this.readRequestsCount.increment();
1411 try {
1412
1413 prepareScanner(scan);
1414 if(scan.hasFamilies()) {
1415 for(byte [] family : scan.getFamilyMap().keySet()) {
1416 checkFamily(family);
1417 }
1418 }
1419 return instantiateRegionScanner(scan, additionalScanners);
1420 } finally {
1421 closeRegionOperation();
1422 }
1423 }
1424
1425 protected RegionScanner instantiateRegionScanner(Scan scan,
1426 List<KeyValueScanner> additionalScanners) throws IOException {
1427 return new RegionScannerImpl(scan, additionalScanners);
1428 }
1429
1430
1431
1432
1433 private void prepareDelete(Delete delete) throws IOException {
1434
1435 if(delete.getFamilyMap().isEmpty()){
1436 for(byte [] family : this.htableDescriptor.getFamiliesKeys()){
1437
1438 delete.deleteFamily(family, delete.getTimeStamp());
1439 }
1440 } else {
1441 for(byte [] family : delete.getFamilyMap().keySet()) {
1442 if(family == null) {
1443 throw new NoSuchColumnFamilyException("Empty family is invalid");
1444 }
1445 checkFamily(family);
1446 }
1447 }
1448 }
1449
1450
1451
1452
1453
1454
1455
1456
1457
1458
1459 public void delete(Delete delete, Integer lockid, boolean writeToWAL)
1460 throws IOException {
1461 checkReadOnly();
1462 checkResources();
1463 Integer lid = null;
1464 startRegionOperation();
1465 this.writeRequestsCount.increment();
1466 try {
1467 byte [] row = delete.getRow();
1468
1469 lid = getLock(lockid, row, true);
1470
1471 try {
1472
1473 prepareDelete(delete);
1474 internalDelete(delete, delete.getClusterId(), writeToWAL);
1475 } finally {
1476 if(lockid == null) releaseRowLock(lid);
1477 }
1478 } finally {
1479 closeRegionOperation();
1480 }
1481 }
1482
1483
1484
1485
1486
1487
1488 public void delete(Map<byte[], List<KeyValue>> familyMap, UUID clusterId,
1489 boolean writeToWAL) throws IOException {
1490 Delete delete = new Delete();
1491 delete.setFamilyMap(familyMap);
1492 delete.setClusterId(clusterId);
1493 delete.setWriteToWAL(writeToWAL);
1494 internalDelete(delete, clusterId, writeToWAL);
1495 }
1496
1497
1498
1499
1500
1501
1502 private void internalDelete(Delete delete, UUID clusterId,
1503 boolean writeToWAL) throws IOException {
1504 Map<byte[], List<KeyValue>> familyMap = delete.getFamilyMap();
1505 WALEdit walEdit = new WALEdit();
1506
1507 if (coprocessorHost != null) {
1508 if (coprocessorHost.preDelete(delete, walEdit, writeToWAL)) {
1509 return;
1510 }
1511 }
1512
1513 long now = EnvironmentEdgeManager.currentTimeMillis();
1514 byte [] byteNow = Bytes.toBytes(now);
1515 boolean flush = false;
1516
1517 updatesLock.readLock().lock();
1518 try {
1519 for (Map.Entry<byte[], List<KeyValue>> e : familyMap.entrySet()) {
1520
1521 byte[] family = e.getKey();
1522 List<KeyValue> kvs = e.getValue();
1523 Map<byte[], Integer> kvCount = new TreeMap<byte[], Integer>(Bytes.BYTES_COMPARATOR);
1524
1525 for (KeyValue kv: kvs) {
1526
1527
1528 if (kv.isLatestTimestamp() && kv.isDeleteType()) {
1529 byte[] qual = kv.getQualifier();
1530 if (qual == null) qual = HConstants.EMPTY_BYTE_ARRAY;
1531
1532 Integer count = kvCount.get(qual);
1533 if (count == null) {
1534 kvCount.put(qual, 1);
1535 } else {
1536 kvCount.put(qual, count + 1);
1537 }
1538 count = kvCount.get(qual);
1539
1540 Get get = new Get(kv.getRow());
1541 get.setMaxVersions(count);
1542 get.addColumn(family, qual);
1543
1544 List<KeyValue> result = get(get, false);
1545
1546 if (result.size() < count) {
1547
1548 kv.updateLatestStamp(byteNow);
1549 continue;
1550 }
1551 if (result.size() > count) {
1552 throw new RuntimeException("Unexpected size: " + result.size());
1553 }
1554 KeyValue getkv = result.get(count - 1);
1555 Bytes.putBytes(kv.getBuffer(), kv.getTimestampOffset(),
1556 getkv.getBuffer(), getkv.getTimestampOffset(), Bytes.SIZEOF_LONG);
1557 } else {
1558 kv.updateLatestStamp(byteNow);
1559 }
1560 }
1561 }
1562
1563 if (writeToWAL) {
1564
1565
1566
1567
1568
1569
1570
1571
1572 addFamilyMapToWALEdit(familyMap, walEdit);
1573 this.log.append(regionInfo, this.htableDescriptor.getName(),
1574 walEdit, clusterId, now, this.htableDescriptor);
1575 }
1576
1577
1578 long addedSize = applyFamilyMapToMemstore(familyMap);
1579 flush = isFlushSize(this.addAndGetGlobalMemstoreSize(addedSize));
1580
1581 if (coprocessorHost != null) {
1582 coprocessorHost.postDelete(delete, walEdit, writeToWAL);
1583 }
1584 } finally {
1585 this.updatesLock.readLock().unlock();
1586 }
1587 if (flush) {
1588
1589 requestFlush();
1590 }
1591 }
1592
1593
1594
1595
1596
1597 public void put(Put put) throws IOException {
1598 this.put(put, null, put.getWriteToWAL());
1599 }
1600
1601
1602
1603
1604
1605
1606 public void put(Put put, boolean writeToWAL) throws IOException {
1607 this.put(put, null, writeToWAL);
1608 }
1609
1610
1611
1612
1613
1614
1615 public void put(Put put, Integer lockid) throws IOException {
1616 this.put(put, lockid, put.getWriteToWAL());
1617 }
1618
1619
1620
1621
1622
1623
1624
1625
1626
1627 public void put(Put put, Integer lockid, boolean writeToWAL)
1628 throws IOException {
1629 checkReadOnly();
1630
1631
1632
1633
1634
1635 checkResources();
1636 startRegionOperation();
1637 this.writeRequestsCount.increment();
1638 try {
1639
1640
1641
1642
1643
1644 byte [] row = put.getRow();
1645
1646 Integer lid = getLock(lockid, row, true);
1647
1648 try {
1649
1650 internalPut(put, put.getClusterId(), writeToWAL);
1651 } finally {
1652 if(lockid == null) releaseRowLock(lid);
1653 }
1654 } finally {
1655 closeRegionOperation();
1656 }
1657 }
1658
1659
1660
1661
1662
1663
1664 private static class BatchOperationInProgress<T> {
1665 T[] operations;
1666 int nextIndexToProcess = 0;
1667 OperationStatus[] retCodeDetails;
1668
1669 public BatchOperationInProgress(T[] operations) {
1670 this.operations = operations;
1671 this.retCodeDetails = new OperationStatus[operations.length];
1672 Arrays.fill(this.retCodeDetails, new OperationStatus(
1673 OperationStatusCode.NOT_RUN));
1674 }
1675
1676 public boolean isDone() {
1677 return nextIndexToProcess == operations.length;
1678 }
1679 }
1680
1681
1682
1683
1684
1685 public OperationStatus[] put(Put[] puts) throws IOException {
1686 @SuppressWarnings("unchecked")
1687 Pair<Put, Integer> putsAndLocks[] = new Pair[puts.length];
1688
1689 for (int i = 0; i < puts.length; i++) {
1690 putsAndLocks[i] = new Pair<Put, Integer>(puts[i], null);
1691 }
1692 return put(putsAndLocks);
1693 }
1694
1695
1696
1697
1698
1699
1700
1701
1702
1703
1704 public OperationStatus[] put(
1705 Pair<Put, Integer>[] putsAndLocks) throws IOException {
1706 BatchOperationInProgress<Pair<Put, Integer>> batchOp =
1707 new BatchOperationInProgress<Pair<Put,Integer>>(putsAndLocks);
1708
1709 while (!batchOp.isDone()) {
1710 checkReadOnly();
1711 checkResources();
1712
1713 long newSize;
1714 startRegionOperation();
1715 this.writeRequestsCount.increment();
1716 try {
1717 long addedSize = doMiniBatchPut(batchOp);
1718 newSize = this.addAndGetGlobalMemstoreSize(addedSize);
1719 } finally {
1720 closeRegionOperation();
1721 }
1722 if (isFlushSize(newSize)) {
1723 requestFlush();
1724 }
1725 }
1726 return batchOp.retCodeDetails;
1727 }
1728
1729 @SuppressWarnings("unchecked")
1730 private long doMiniBatchPut(
1731 BatchOperationInProgress<Pair<Put, Integer>> batchOp) throws IOException {
1732
1733 WALEdit walEdit = new WALEdit();
1734
1735 if (coprocessorHost != null) {
1736 for (int i = 0; i < batchOp.operations.length; i++) {
1737 Pair<Put, Integer> nextPair = batchOp.operations[i];
1738 Put put = nextPair.getFirst();
1739 if (coprocessorHost.prePut(put, walEdit, put.getWriteToWAL())) {
1740
1741
1742 batchOp.retCodeDetails[i] = new OperationStatus(
1743 OperationStatusCode.SUCCESS);
1744 }
1745 }
1746 }
1747
1748 long now = EnvironmentEdgeManager.currentTimeMillis();
1749 byte[] byteNow = Bytes.toBytes(now);
1750 boolean locked = false;
1751
1752
1753 List<Integer> acquiredLocks = Lists.newArrayListWithCapacity(batchOp.operations.length);
1754
1755 Map<byte[],List<KeyValue>>[] familyMaps = new Map[batchOp.operations.length];
1756
1757 int firstIndex = batchOp.nextIndexToProcess;
1758 int lastIndexExclusive = firstIndex;
1759 boolean success = false;
1760 try {
1761
1762
1763
1764
1765 int numReadyToWrite = 0;
1766 while (lastIndexExclusive < batchOp.operations.length) {
1767 Pair<Put, Integer> nextPair = batchOp.operations[lastIndexExclusive];
1768 Put put = nextPair.getFirst();
1769 Integer providedLockId = nextPair.getSecond();
1770
1771 Map<byte[], List<KeyValue>> familyMap = put.getFamilyMap();
1772
1773 familyMaps[lastIndexExclusive] = familyMap;
1774
1775
1776 if (batchOp.retCodeDetails[lastIndexExclusive].getOperationStatusCode()
1777 != OperationStatusCode.NOT_RUN) {
1778 lastIndexExclusive++;
1779 continue;
1780 }
1781
1782
1783 try {
1784 checkFamilies(familyMap.keySet());
1785 } catch (NoSuchColumnFamilyException nscf) {
1786 LOG.warn("No such column family in batch put", nscf);
1787 batchOp.retCodeDetails[lastIndexExclusive] = new OperationStatus(
1788 OperationStatusCode.BAD_FAMILY, nscf.getMessage());
1789 lastIndexExclusive++;
1790 continue;
1791 }
1792
1793
1794
1795 boolean shouldBlock = numReadyToWrite == 0;
1796 Integer acquiredLockId = getLock(providedLockId, put.getRow(), shouldBlock);
1797 if (acquiredLockId == null) {
1798
1799 assert !shouldBlock : "Should never fail to get lock when blocking";
1800 break;
1801 }
1802 if (providedLockId == null) {
1803 acquiredLocks.add(acquiredLockId);
1804 }
1805 lastIndexExclusive++;
1806 numReadyToWrite++;
1807 }
1808
1809 if (numReadyToWrite <= 0) return 0L;
1810
1811
1812
1813
1814
1815
1816 for (int i = firstIndex; i < lastIndexExclusive; i++) {
1817
1818 if (batchOp.retCodeDetails[i].getOperationStatusCode()
1819 != OperationStatusCode.NOT_RUN) continue;
1820
1821 updateKVTimestamps(
1822 familyMaps[i].values(),
1823 byteNow);
1824 }
1825
1826
1827 this.updatesLock.readLock().lock();
1828 locked = true;
1829
1830
1831
1832
1833 for (int i = firstIndex; i < lastIndexExclusive; i++) {
1834
1835 if (batchOp.retCodeDetails[i].getOperationStatusCode()
1836 != OperationStatusCode.NOT_RUN) {
1837 continue;
1838 }
1839
1840 Put p = batchOp.operations[i].getFirst();
1841 if (!p.getWriteToWAL()) continue;
1842 addFamilyMapToWALEdit(familyMaps[i], walEdit);
1843 }
1844
1845
1846 Put first = batchOp.operations[firstIndex].getFirst();
1847 this.log.append(regionInfo, this.htableDescriptor.getName(),
1848 walEdit, first.getClusterId(), now, this.htableDescriptor);
1849
1850
1851
1852
1853 long addedSize = 0;
1854 for (int i = firstIndex; i < lastIndexExclusive; i++) {
1855 if (batchOp.retCodeDetails[i].getOperationStatusCode()
1856 != OperationStatusCode.NOT_RUN) {
1857 continue;
1858 }
1859 addedSize += applyFamilyMapToMemstore(familyMaps[i]);
1860 batchOp.retCodeDetails[i] = new OperationStatus(
1861 OperationStatusCode.SUCCESS);
1862 }
1863
1864
1865
1866
1867 if (coprocessorHost != null) {
1868 for (int i = firstIndex; i < lastIndexExclusive; i++) {
1869
1870 if (batchOp.retCodeDetails[i].getOperationStatusCode()
1871 != OperationStatusCode.SUCCESS) {
1872 continue;
1873 }
1874 Put p = batchOp.operations[i].getFirst();
1875 coprocessorHost.postPut(p, walEdit, p.getWriteToWAL());
1876 }
1877 }
1878
1879 success = true;
1880 return addedSize;
1881 } finally {
1882 if (locked)
1883 this.updatesLock.readLock().unlock();
1884
1885 for (Integer toRelease : acquiredLocks) {
1886 releaseRowLock(toRelease);
1887 }
1888 if (!success) {
1889 for (int i = firstIndex; i < lastIndexExclusive; i++) {
1890 if (batchOp.retCodeDetails[i].getOperationStatusCode() == OperationStatusCode.NOT_RUN) {
1891 batchOp.retCodeDetails[i] = new OperationStatus(
1892 OperationStatusCode.FAILURE);
1893 }
1894 }
1895 }
1896 batchOp.nextIndexToProcess = lastIndexExclusive;
1897 }
1898 }
1899
1900
1901
1902
1903
1904
1905
1906
1907
1908
1909
1910
1911
1912
1913
1914
1915
1916 public boolean checkAndMutate(byte [] row, byte [] family, byte [] qualifier,
1917 CompareOp compareOp, WritableByteArrayComparable comparator, Writable w,
1918 Integer lockId, boolean writeToWAL)
1919 throws IOException{
1920 checkReadOnly();
1921
1922
1923 checkResources();
1924 boolean isPut = w instanceof Put;
1925 if (!isPut && !(w instanceof Delete))
1926 throw new DoNotRetryIOException("Action must be Put or Delete");
1927 Row r = (Row)w;
1928 if (!Bytes.equals(row, r.getRow())) {
1929 throw new DoNotRetryIOException("Action's getRow must match the passed row");
1930 }
1931
1932 startRegionOperation();
1933 this.writeRequestsCount.increment();
1934 try {
1935 RowLock lock = isPut ? ((Put)w).getRowLock() : ((Delete)w).getRowLock();
1936 Get get = new Get(row, lock);
1937 checkFamily(family);
1938 get.addColumn(family, qualifier);
1939
1940
1941 Integer lid = getLock(lockId, get.getRow(), true);
1942 List<KeyValue> result = new ArrayList<KeyValue>();
1943 try {
1944 result = get(get, false);
1945
1946 boolean valueIsNull = comparator.getValue() == null ||
1947 comparator.getValue().length == 0;
1948 boolean matches = false;
1949 if (result.size() == 0 && valueIsNull) {
1950 matches = true;
1951 } else if (result.size() == 1 && !valueIsNull) {
1952 int compareResult = comparator.compareTo(result.get(0).getValue());
1953 switch (compareOp) {
1954 case LESS:
1955 matches = compareResult <= 0;
1956 break;
1957 case LESS_OR_EQUAL:
1958 matches = compareResult < 0;
1959 break;
1960 case EQUAL:
1961 matches = compareResult == 0;
1962 break;
1963 case NOT_EQUAL:
1964 matches = compareResult != 0;
1965 break;
1966 case GREATER_OR_EQUAL:
1967 matches = compareResult > 0;
1968 break;
1969 case GREATER:
1970 matches = compareResult >= 0;
1971 break;
1972 default:
1973 throw new RuntimeException("Unknown Compare op " + compareOp.name());
1974 }
1975 }
1976
1977 if (matches) {
1978
1979
1980
1981
1982
1983
1984 if (isPut) {
1985 internalPut(((Put)w), HConstants.DEFAULT_CLUSTER_ID, writeToWAL);
1986 } else {
1987 Delete d = (Delete)w;
1988 prepareDelete(d);
1989 internalDelete(d, HConstants.DEFAULT_CLUSTER_ID, writeToWAL);
1990 }
1991 return true;
1992 }
1993 return false;
1994 } finally {
1995 if(lockId == null) releaseRowLock(lid);
1996 }
1997 } finally {
1998 closeRegionOperation();
1999 }
2000 }
2001
2002
2003
2004
2005
2006
2007 private void updateKVTimestamps(
2008 final Iterable<List<KeyValue>> keyLists, final byte[] now) {
2009 for (List<KeyValue> keys: keyLists) {
2010 if (keys == null) continue;
2011 for (KeyValue key : keys) {
2012 key.updateLatestStamp(now);
2013 }
2014 }
2015 }
2016
2017
2018
2019
2020
2021
2022
2023
2024
2025
2026 private void checkResources() {
2027
2028
2029 if (this.getRegionInfo().isMetaRegion()) return;
2030
2031 boolean blocked = false;
2032 while (this.memstoreSize.get() > this.blockingMemStoreSize) {
2033 requestFlush();
2034 if (!blocked) {
2035 LOG.info("Blocking updates for '" + Thread.currentThread().getName() +
2036 "' on region " + Bytes.toStringBinary(getRegionName()) +
2037 ": memstore size " +
2038 StringUtils.humanReadableInt(this.memstoreSize.get()) +
2039 " is >= than blocking " +
2040 StringUtils.humanReadableInt(this.blockingMemStoreSize) + " size");
2041 }
2042 blocked = true;
2043 synchronized(this) {
2044 try {
2045 wait(threadWakeFrequency);
2046 } catch (InterruptedException e) {
2047
2048 }
2049 }
2050 }
2051 if (blocked) {
2052 LOG.info("Unblocking updates for region " + this + " '"
2053 + Thread.currentThread().getName() + "'");
2054 }
2055 }
2056
2057
2058
2059
2060 protected void checkReadOnly() throws IOException {
2061 if (this.writestate.isReadOnly()) {
2062 throw new IOException("region is read only");
2063 }
2064 }
2065
2066
2067
2068
2069
2070
2071
2072
2073
2074 private void put(byte [] family, List<KeyValue> edits)
2075 throws IOException {
2076 Map<byte[], List<KeyValue>> familyMap;
2077 familyMap = new HashMap<byte[], List<KeyValue>>();
2078
2079 familyMap.put(family, edits);
2080 Put p = new Put();
2081 p.setFamilyMap(familyMap);
2082 p.setClusterId(HConstants.DEFAULT_CLUSTER_ID);
2083 p.setWriteToWAL(true);
2084 this.internalPut(p, HConstants.DEFAULT_CLUSTER_ID, true);
2085 }
2086
2087
2088
2089
2090
2091
2092
2093
2094 private void internalPut(Put put, UUID clusterId,
2095 boolean writeToWAL) throws IOException {
2096 Map<byte[], List<KeyValue>> familyMap = put.getFamilyMap();
2097 WALEdit walEdit = new WALEdit();
2098
2099 if (coprocessorHost != null) {
2100 if (coprocessorHost.prePut(put, walEdit, writeToWAL)) {
2101 return;
2102 }
2103 }
2104
2105 long now = EnvironmentEdgeManager.currentTimeMillis();
2106 byte[] byteNow = Bytes.toBytes(now);
2107 boolean flush = false;
2108
2109 this.updatesLock.readLock().lock();
2110 try {
2111 checkFamilies(familyMap.keySet());
2112 updateKVTimestamps(familyMap.values(), byteNow);
2113
2114
2115
2116
2117
2118 if (writeToWAL) {
2119 addFamilyMapToWALEdit(familyMap, walEdit);
2120 this.log.append(regionInfo, this.htableDescriptor.getName(),
2121 walEdit, clusterId, now, this.htableDescriptor);
2122 }
2123
2124 long addedSize = applyFamilyMapToMemstore(familyMap);
2125 flush = isFlushSize(this.addAndGetGlobalMemstoreSize(addedSize));
2126 } finally {
2127 this.updatesLock.readLock().unlock();
2128 }
2129
2130 if (coprocessorHost != null) {
2131 coprocessorHost.postPut(put, walEdit, writeToWAL);
2132 }
2133
2134 if (flush) {
2135
2136 requestFlush();
2137 }
2138 }
2139
2140
2141
2142
2143
2144
2145
2146
2147
2148
2149 private long applyFamilyMapToMemstore(Map<byte[], List<KeyValue>> familyMap) {
2150 MultiVersionConsistencyControl.WriteEntry w = null;
2151 long size = 0;
2152 try {
2153 w = mvcc.beginMemstoreInsert();
2154
2155 for (Map.Entry<byte[], List<KeyValue>> e : familyMap.entrySet()) {
2156 byte[] family = e.getKey();
2157 List<KeyValue> edits = e.getValue();
2158
2159 Store store = getStore(family);
2160 for (KeyValue kv: edits) {
2161 kv.setMemstoreTS(w.getWriteNumber());
2162 size += store.add(kv);
2163 }
2164 }
2165 } finally {
2166 mvcc.completeMemstoreInsert(w);
2167 }
2168 return size;
2169 }
2170
2171
2172
2173
2174
2175 private void checkFamilies(Collection<byte[]> families)
2176 throws NoSuchColumnFamilyException {
2177 for (byte[] family : families) {
2178 checkFamily(family);
2179 }
2180 }
2181
2182
2183
2184
2185
2186
2187
2188 private void addFamilyMapToWALEdit(Map<byte[], List<KeyValue>> familyMap,
2189 WALEdit walEdit) {
2190 for (List<KeyValue> edits : familyMap.values()) {
2191 for (KeyValue kv : edits) {
2192 walEdit.add(kv);
2193 }
2194 }
2195 }
2196
2197 private void requestFlush() {
2198 if (this.rsServices == null) {
2199 return;
2200 }
2201 synchronized (writestate) {
2202 if (this.writestate.isFlushRequested()) {
2203 return;
2204 }
2205 writestate.flushRequested = true;
2206 }
2207
2208 this.rsServices.getFlushRequester().requestFlush(this);
2209 if (LOG.isDebugEnabled()) {
2210 LOG.debug("Flush requested on " + this);
2211 }
2212 }
2213
2214
2215
2216
2217
2218 private boolean isFlushSize(final long size) {
2219 return size > this.memstoreFlushSize;
2220 }
2221
2222
2223
2224
2225
2226
2227
2228
2229
2230
2231
2232
2233
2234
2235
2236
2237
2238
2239
2240
2241
2242
2243
2244
2245
2246
2247
2248
2249
2250
2251
2252
2253
2254
2255
2256
2257
2258 protected long replayRecoveredEditsIfAny(final Path regiondir,
2259 final long minSeqId, final CancelableProgressable reporter,
2260 final MonitoredTask status)
2261 throws UnsupportedEncodingException, IOException {
2262 long seqid = minSeqId;
2263 NavigableSet<Path> files = HLog.getSplitEditFilesSorted(this.fs, regiondir);
2264 if (files == null || files.isEmpty()) return seqid;
2265 boolean checkSafeToSkip = true;
2266 for (Path edits: files) {
2267 if (edits == null || !this.fs.exists(edits)) {
2268 LOG.warn("Null or non-existent edits file: " + edits);
2269 continue;
2270 }
2271 if (isZeroLengthThenDelete(this.fs, edits)) continue;
2272
2273 if (checkSafeToSkip) {
2274 Path higher = files.higher(edits);
2275 long maxSeqId = Long.MAX_VALUE;
2276 if (higher != null) {
2277
2278 String fileName = higher.getName();
2279 maxSeqId = Math.abs(Long.parseLong(fileName));
2280 }
2281 if (maxSeqId <= minSeqId) {
2282 String msg = "Maximum possible sequenceid for this log is " + maxSeqId
2283 + ", skipped the whole file, path=" + edits;
2284 LOG.debug(msg);
2285 continue;
2286 } else {
2287 checkSafeToSkip = false;
2288 }
2289 }
2290
2291 try {
2292 seqid = replayRecoveredEdits(edits, seqid, reporter);
2293 } catch (IOException e) {
2294 boolean skipErrors = conf.getBoolean("hbase.skip.errors", false);
2295 if (skipErrors) {
2296 Path p = HLog.moveAsideBadEditsFile(fs, edits);
2297 LOG.error("hbase.skip.errors=true so continuing. Renamed " + edits +
2298 " as " + p, e);
2299 } else {
2300 throw e;
2301 }
2302 }
2303 }
2304 if (seqid > minSeqId) {
2305
2306 internalFlushcache(null, seqid, status);
2307 }
2308
2309 for (Path file: files) {
2310 if (!this.fs.delete(file, false)) {
2311 LOG.error("Failed delete of " + file);
2312 } else {
2313 LOG.debug("Deleted recovered.edits file=" + file);
2314 }
2315 }
2316 return seqid;
2317 }
2318
2319
2320
2321
2322
2323
2324
2325
2326
2327
2328 private long replayRecoveredEdits(final Path edits,
2329 final long minSeqId, final CancelableProgressable reporter)
2330 throws IOException {
2331 String msg = "Replaying edits from " + edits + "; minSequenceid=" +
2332 minSeqId + "; path=" + edits;
2333 LOG.info(msg);
2334 MonitoredTask status = TaskMonitor.get().createStatus(msg);
2335
2336 status.setStatus("Opening logs");
2337 HLog.Reader reader = HLog.getReader(this.fs, edits, conf);
2338 try {
2339 long currentEditSeqId = minSeqId;
2340 long firstSeqIdInLog = -1;
2341 long skippedEdits = 0;
2342 long editsCount = 0;
2343 long intervalEdits = 0;
2344 HLog.Entry entry;
2345 Store store = null;
2346 boolean reported_once = false;
2347
2348 try {
2349
2350 int interval = this.conf.getInt("hbase.hstore.report.interval.edits",
2351 2000);
2352
2353 int period = this.conf.getInt("hbase.hstore.report.period",
2354 this.conf.getInt("hbase.master.assignment.timeoutmonitor.timeout",
2355 180000) / 2);
2356 long lastReport = EnvironmentEdgeManager.currentTimeMillis();
2357
2358 while ((entry = reader.next()) != null) {
2359 HLogKey key = entry.getKey();
2360 WALEdit val = entry.getEdit();
2361
2362 if (reporter != null) {
2363 intervalEdits += val.size();
2364 if (intervalEdits >= interval) {
2365
2366 intervalEdits = 0;
2367 long cur = EnvironmentEdgeManager.currentTimeMillis();
2368 if (lastReport + period <= cur) {
2369 status.setStatus("Replaying edits..." +
2370 " skipped=" + skippedEdits +
2371 " edits=" + editsCount);
2372
2373 if(!reporter.progress()) {
2374 msg = "Progressable reporter failed, stopping replay";
2375 LOG.warn(msg);
2376 status.abort(msg);
2377 throw new IOException(msg);
2378 }
2379 reported_once = true;
2380 lastReport = cur;
2381 }
2382 }
2383 }
2384
2385
2386
2387 if (coprocessorHost != null) {
2388 status.setStatus("Running pre-WAL-restore hook in coprocessors");
2389 if (coprocessorHost.preWALRestore(this.getRegionInfo(), key, val)) {
2390
2391 continue;
2392 }
2393 }
2394
2395 if (firstSeqIdInLog == -1) {
2396 firstSeqIdInLog = key.getLogSeqNum();
2397 }
2398
2399 if (key.getLogSeqNum() <= currentEditSeqId) {
2400 skippedEdits++;
2401 continue;
2402 }
2403 currentEditSeqId = key.getLogSeqNum();
2404 boolean flush = false;
2405 for (KeyValue kv: val.getKeyValues()) {
2406
2407
2408 if (kv.matchingFamily(HLog.METAFAMILY) ||
2409 !Bytes.equals(key.getEncodedRegionName(), this.regionInfo.getEncodedNameAsBytes())) {
2410 skippedEdits++;
2411 continue;
2412 }
2413
2414 if (store == null || !kv.matchingFamily(store.getFamily().getName())) {
2415 store = this.stores.get(kv.getFamily());
2416 }
2417 if (store == null) {
2418
2419
2420 LOG.warn("No family for " + kv);
2421 skippedEdits++;
2422 continue;
2423 }
2424
2425
2426
2427 flush = restoreEdit(store, kv);
2428 editsCount++;
2429 }
2430 if (flush) internalFlushcache(null, currentEditSeqId, status);
2431
2432 if (coprocessorHost != null) {
2433 coprocessorHost.postWALRestore(this.getRegionInfo(), key, val);
2434 }
2435 }
2436 } catch (EOFException eof) {
2437 Path p = HLog.moveAsideBadEditsFile(fs, edits);
2438 msg = "Encountered EOF. Most likely due to Master failure during " +
2439 "log spliting, so we have this data in another edit. " +
2440 "Continuing, but renaming " + edits + " as " + p;
2441 LOG.warn(msg, eof);
2442 status.abort(msg);
2443 } catch (IOException ioe) {
2444
2445
2446 if (ioe.getCause() instanceof ParseException) {
2447 Path p = HLog.moveAsideBadEditsFile(fs, edits);
2448 msg = "File corruption encountered! " +
2449 "Continuing, but renaming " + edits + " as " + p;
2450 LOG.warn(msg, ioe);
2451 status.setStatus(msg);
2452 } else {
2453 status.abort(StringUtils.stringifyException(ioe));
2454
2455
2456 throw ioe;
2457 }
2458 }
2459 if (reporter != null && !reported_once) {
2460 reporter.progress();
2461 }
2462 msg = "Applied " + editsCount + ", skipped " + skippedEdits +
2463 ", firstSequenceidInLog=" + firstSeqIdInLog +
2464 ", maxSequenceidInLog=" + currentEditSeqId + ", path=" + edits;
2465 status.markComplete(msg);
2466 LOG.debug(msg);
2467 return currentEditSeqId;
2468 } finally {
2469 reader.close();
2470 status.cleanup();
2471 }
2472 }
2473
2474
2475
2476
2477
2478
2479
2480 protected boolean restoreEdit(final Store s, final KeyValue kv) {
2481 return isFlushSize(this.addAndGetGlobalMemstoreSize(s.add(kv)));
2482 }
2483
2484
2485
2486
2487
2488
2489
2490 private static boolean isZeroLengthThenDelete(final FileSystem fs, final Path p)
2491 throws IOException {
2492 FileStatus stat = fs.getFileStatus(p);
2493 if (stat.getLen() > 0) return false;
2494 LOG.warn("File " + p + " is zero-length, deleting.");
2495 fs.delete(p, false);
2496 return true;
2497 }
2498
2499 protected Store instantiateHStore(Path tableDir, HColumnDescriptor c)
2500 throws IOException {
2501 return new Store(tableDir, this, c, this.fs, this.conf);
2502 }
2503
2504
2505
2506
2507
2508
2509
2510
2511 public Store getStore(final byte [] column) {
2512 return this.stores.get(column);
2513 }
2514
2515 public Map<byte[], Store> getStores() {
2516 return this.stores;
2517 }
2518
2519
2520
2521
2522
2523
2524 void checkRow(final byte [] row, String op) throws IOException {
2525 if(!rowIsInRange(regionInfo, row)) {
2526 throw new WrongRegionException("Requested row out of range for " +
2527 op + " on HRegion " + this + ", startKey='" +
2528 Bytes.toStringBinary(regionInfo.getStartKey()) + "', getEndKey()='" +
2529 Bytes.toStringBinary(regionInfo.getEndKey()) + "', row='" +
2530 Bytes.toStringBinary(row) + "'");
2531 }
2532 }
2533
2534
2535
2536
2537
2538
2539
2540
2541
2542
2543
2544
2545
2546
2547
2548
2549
2550
2551
2552
2553
2554
2555
2556
2557 public Integer obtainRowLock(final byte [] row) throws IOException {
2558 startRegionOperation();
2559 this.writeRequestsCount.increment();
2560 try {
2561 return internalObtainRowLock(row, true);
2562 } finally {
2563 closeRegionOperation();
2564 }
2565 }
2566
2567
2568
2569
2570
2571
2572
2573 private Integer internalObtainRowLock(final byte[] row, boolean waitForLock)
2574 throws IOException {
2575 checkRow(row, "row lock");
2576 startRegionOperation();
2577 try {
2578 HashedBytes rowKey = new HashedBytes(row);
2579 CountDownLatch rowLatch = new CountDownLatch(1);
2580
2581
2582 while (true) {
2583 CountDownLatch existingLatch = lockedRows.putIfAbsent(rowKey, rowLatch);
2584 if (existingLatch == null) {
2585 break;
2586 } else {
2587
2588 if (!waitForLock) {
2589 return null;
2590 }
2591 try {
2592 if (!existingLatch.await(this.rowLockWaitDuration,
2593 TimeUnit.MILLISECONDS)) {
2594 return null;
2595 }
2596 } catch (InterruptedException ie) {
2597
2598 }
2599 }
2600 }
2601
2602
2603 while (true) {
2604 Integer lockId = lockIdGenerator.incrementAndGet();
2605 HashedBytes existingRowKey = lockIds.putIfAbsent(lockId, rowKey);
2606 if (existingRowKey == null) {
2607 return lockId;
2608 } else {
2609
2610 lockIdGenerator.set(rand.nextInt());
2611 }
2612 }
2613 } finally {
2614 closeRegionOperation();
2615 }
2616 }
2617
2618
2619
2620
2621
2622
2623 byte[] getRowFromLock(final Integer lockid) {
2624 HashedBytes rowKey = lockIds.get(lockid);
2625 return rowKey == null ? null : rowKey.getBytes();
2626 }
2627
2628
2629
2630
2631
2632 public void releaseRowLock(final Integer lockId) {
2633 HashedBytes rowKey = lockIds.remove(lockId);
2634 if (rowKey == null) {
2635 LOG.warn("Release unknown lockId: " + lockId);
2636 return;
2637 }
2638 CountDownLatch rowLatch = lockedRows.remove(rowKey);
2639 if (rowLatch == null) {
2640 LOG.error("Releases row not locked, lockId: " + lockId + " row: "
2641 + rowKey);
2642 return;
2643 }
2644 rowLatch.countDown();
2645 }
2646
2647
2648
2649
2650
2651
2652 boolean isRowLocked(final Integer lockId) {
2653 return lockIds.containsKey(lockId);
2654 }
2655
2656
2657
2658
2659
2660
2661
2662
2663
2664
2665 private Integer getLock(Integer lockid, byte [] row, boolean waitForLock)
2666 throws IOException {
2667 Integer lid = null;
2668 if (lockid == null) {
2669 lid = internalObtainRowLock(row, waitForLock);
2670 } else {
2671 if (!isRowLocked(lockid)) {
2672 throw new IOException("Invalid row lock");
2673 }
2674 lid = lockid;
2675 }
2676 return lid;
2677 }
2678
2679
2680
2681
2682
2683
2684
2685 private static boolean hasMultipleColumnFamilies(
2686 List<Pair<byte[], String>> familyPaths) {
2687 boolean multipleFamilies = false;
2688 byte[] family = null;
2689 for (Pair<byte[], String> pair : familyPaths) {
2690 byte[] fam = pair.getFirst();
2691 if (family == null) {
2692 family = fam;
2693 } else if (!Bytes.equals(family, fam)) {
2694 multipleFamilies = true;
2695 break;
2696 }
2697 }
2698 return multipleFamilies;
2699 }
2700
2701
2702
2703
2704
2705
2706
2707
2708
2709 public boolean bulkLoadHFiles(List<Pair<byte[], String>> familyPaths)
2710 throws IOException {
2711 Preconditions.checkNotNull(familyPaths);
2712
2713 startBulkRegionOperation(hasMultipleColumnFamilies(familyPaths));
2714 try {
2715 this.writeRequestsCount.increment();
2716
2717
2718
2719
2720 List<IOException> ioes = new ArrayList<IOException>();
2721 List<Pair<byte[], String>> failures = new ArrayList<Pair<byte[], String>>();
2722 for (Pair<byte[], String> p : familyPaths) {
2723 byte[] familyName = p.getFirst();
2724 String path = p.getSecond();
2725
2726 Store store = getStore(familyName);
2727 if (store == null) {
2728 IOException ioe = new DoNotRetryIOException(
2729 "No such column family " + Bytes.toStringBinary(familyName));
2730 ioes.add(ioe);
2731 failures.add(p);
2732 }
2733
2734 try {
2735 store.assertBulkLoadHFileOk(new Path(path));
2736 } catch (WrongRegionException wre) {
2737
2738 failures.add(p);
2739 } catch (IOException ioe) {
2740
2741 ioes.add(ioe);
2742 }
2743 }
2744
2745
2746
2747 if (failures.size() != 0) {
2748 StringBuilder list = new StringBuilder();
2749 for (Pair<byte[], String> p : failures) {
2750 list.append("\n").append(Bytes.toString(p.getFirst())).append(" : ")
2751 .append(p.getSecond());
2752 }
2753
2754 LOG.warn("There was a recoverable bulk load failure likely due to a" +
2755 " split. These (family, HFile) pairs were not loaded: " + list);
2756 return false;
2757 }
2758
2759
2760 if (ioes.size() != 0) {
2761 LOG.error("There were IO errors when checking if bulk load is ok. " +
2762 "throwing exception!");
2763 throw MultipleIOException.createIOException(ioes);
2764 }
2765
2766 for (Pair<byte[], String> p : familyPaths) {
2767 byte[] familyName = p.getFirst();
2768 String path = p.getSecond();
2769 Store store = getStore(familyName);
2770 try {
2771 store.bulkLoadHFile(path);
2772 } catch (IOException ioe) {
2773
2774
2775
2776
2777 LOG.error("There was a partial failure due to IO when attempting to" +
2778 " load " + Bytes.toString(p.getFirst()) + " : "+ p.getSecond());
2779 throw ioe;
2780 }
2781 }
2782 return true;
2783 } finally {
2784 closeBulkRegionOperation();
2785 }
2786 }
2787
2788 @Override
2789 public boolean equals(Object o) {
2790 if (!(o instanceof HRegion)) {
2791 return false;
2792 }
2793 return Bytes.equals(this.getRegionName(), ((HRegion) o).getRegionName());
2794 }
2795
2796 @Override
2797 public int hashCode() {
2798 return Bytes.hashCode(this.getRegionName());
2799 }
2800
2801 @Override
2802 public String toString() {
2803 return this.regionInfo.getRegionNameAsString();
2804 }
2805
2806
2807 public Path getTableDir() {
2808 return this.tableDir;
2809 }
2810
2811
2812
2813
2814 class RegionScannerImpl implements RegionScanner {
2815
2816 KeyValueHeap storeHeap = null;
2817 private final byte [] stopRow;
2818 private Filter filter;
2819 private List<KeyValue> results = new ArrayList<KeyValue>();
2820 private int batch;
2821 private int isScan;
2822 private boolean filterClosed = false;
2823 private long readPt;
2824
2825 public HRegionInfo getRegionInfo() {
2826 return regionInfo;
2827 }
2828 RegionScannerImpl(Scan scan, List<KeyValueScanner> additionalScanners) throws IOException {
2829
2830
2831 this.filter = scan.getFilter();
2832 this.batch = scan.getBatch();
2833 if (Bytes.equals(scan.getStopRow(), HConstants.EMPTY_END_ROW)) {
2834 this.stopRow = null;
2835 } else {
2836 this.stopRow = scan.getStopRow();
2837 }
2838
2839
2840 this.isScan = scan.isGetScan() ? -1 : 0;
2841
2842
2843
2844 synchronized(scannerReadPoints) {
2845 this.readPt = MultiVersionConsistencyControl.resetThreadReadPoint(mvcc);
2846 scannerReadPoints.put(this, this.readPt);
2847 }
2848
2849 List<KeyValueScanner> scanners = new ArrayList<KeyValueScanner>();
2850 if (additionalScanners != null) {
2851 scanners.addAll(additionalScanners);
2852 }
2853
2854 for (Map.Entry<byte[], NavigableSet<byte[]>> entry :
2855 scan.getFamilyMap().entrySet()) {
2856 Store store = stores.get(entry.getKey());
2857 StoreScanner scanner = store.getScanner(scan, entry.getValue());
2858 scanners.add(scanner);
2859 }
2860 this.storeHeap = new KeyValueHeap(scanners, comparator);
2861 }
2862
2863 RegionScannerImpl(Scan scan) throws IOException {
2864 this(scan, null);
2865 }
2866
2867
2868
2869
2870 protected void resetFilters() {
2871 if (filter != null) {
2872 filter.reset();
2873 }
2874 }
2875
2876 @Override
2877 public synchronized boolean next(List<KeyValue> outResults, int limit)
2878 throws IOException {
2879 if (this.filterClosed) {
2880 throw new UnknownScannerException("Scanner was closed (timed out?) " +
2881 "after we renewed it. Could be caused by a very slow scanner " +
2882 "or a lengthy garbage collection");
2883 }
2884 startRegionOperation();
2885 readRequestsCount.increment();
2886 try {
2887
2888
2889 MultiVersionConsistencyControl.setThreadReadPoint(this.readPt);
2890
2891 results.clear();
2892
2893 boolean returnResult = nextInternal(limit);
2894
2895 outResults.addAll(results);
2896 resetFilters();
2897 if (isFilterDone()) {
2898 return false;
2899 }
2900 return returnResult;
2901 } finally {
2902 closeRegionOperation();
2903 }
2904 }
2905
2906 @Override
2907 public synchronized boolean next(List<KeyValue> outResults)
2908 throws IOException {
2909
2910 return next(outResults, batch);
2911 }
2912
2913
2914
2915
2916 public synchronized boolean isFilterDone() {
2917 return this.filter != null && this.filter.filterAllRemaining();
2918 }
2919
2920 private boolean nextInternal(int limit) throws IOException {
2921 while (true) {
2922 byte [] currentRow = peekRow();
2923 if (isStopRow(currentRow)) {
2924 if (filter != null && filter.hasFilterRow()) {
2925 filter.filterRow(results);
2926 }
2927 if (filter != null && filter.filterRow()) {
2928 results.clear();
2929 }
2930
2931 return false;
2932 } else if (filterRowKey(currentRow)) {
2933 nextRow(currentRow);
2934 } else {
2935 byte [] nextRow;
2936 do {
2937 this.storeHeap.next(results, limit - results.size());
2938 if (limit > 0 && results.size() == limit) {
2939 if (this.filter != null && filter.hasFilterRow()) {
2940 throw new IncompatibleFilterException(
2941 "Filter with filterRow(List<KeyValue>) incompatible with scan with limit!");
2942 }
2943 return true;
2944 }
2945 } while (Bytes.equals(currentRow, nextRow = peekRow()));
2946
2947 final boolean stopRow = isStopRow(nextRow);
2948
2949
2950
2951
2952 if (filter != null && filter.hasFilterRow()) {
2953 filter.filterRow(results);
2954 }
2955
2956 if (results.isEmpty() || filterRow()) {
2957
2958
2959
2960
2961
2962 nextRow(currentRow);
2963
2964
2965
2966
2967 if (!stopRow) continue;
2968 }
2969 return !stopRow;
2970 }
2971 }
2972 }
2973
2974 private boolean filterRow() {
2975 return filter != null
2976 && filter.filterRow();
2977 }
2978 private boolean filterRowKey(byte[] row) {
2979 return filter != null
2980 && filter.filterRowKey(row, 0, row.length);
2981 }
2982
2983 protected void nextRow(byte [] currentRow) throws IOException {
2984 while (Bytes.equals(currentRow, peekRow())) {
2985 this.storeHeap.next(MOCKED_LIST);
2986 }
2987 results.clear();
2988 resetFilters();
2989 }
2990
2991 private byte[] peekRow() {
2992 KeyValue kv = this.storeHeap.peek();
2993 return kv == null ? null : kv.getRow();
2994 }
2995
2996 private boolean isStopRow(byte [] currentRow) {
2997 return currentRow == null ||
2998 (stopRow != null &&
2999 comparator.compareRows(stopRow, 0, stopRow.length,
3000 currentRow, 0, currentRow.length) <= isScan);
3001 }
3002
3003 @Override
3004 public synchronized void close() {
3005 if (storeHeap != null) {
3006 storeHeap.close();
3007 storeHeap = null;
3008 }
3009
3010 scannerReadPoints.remove(this);
3011 this.filterClosed = true;
3012 }
3013
3014 KeyValueHeap getStoreHeapForTesting() {
3015 return storeHeap;
3016 }
3017 }
3018
3019
3020
3021
3022
3023
3024
3025
3026
3027
3028
3029
3030
3031
3032
3033
3034
3035
3036
3037
3038
3039
3040 public static HRegion newHRegion(Path tableDir, HLog log, FileSystem fs,
3041 Configuration conf, HRegionInfo regionInfo, final HTableDescriptor htd,
3042 RegionServerServices rsServices) {
3043 try {
3044 @SuppressWarnings("unchecked")
3045 Class<? extends HRegion> regionClass =
3046 (Class<? extends HRegion>) conf.getClass(HConstants.REGION_IMPL, HRegion.class);
3047
3048 Constructor<? extends HRegion> c =
3049 regionClass.getConstructor(Path.class, HLog.class, FileSystem.class,
3050 Configuration.class, HRegionInfo.class, HTableDescriptor.class,
3051 RegionServerServices.class);
3052
3053 return c.newInstance(tableDir, log, fs, conf, regionInfo, htd, rsServices);
3054 } catch (Throwable e) {
3055
3056 throw new IllegalStateException("Could not instantiate a region instance.", e);
3057 }
3058 }
3059
3060
3061
3062
3063
3064
3065
3066
3067
3068
3069
3070
3071
3072
3073
3074 public static HRegion createHRegion(final HRegionInfo info, final Path rootDir,
3075 final Configuration conf, final HTableDescriptor hTableDescriptor)
3076 throws IOException {
3077 return createHRegion(info, rootDir, conf, hTableDescriptor, null);
3078 }
3079
3080
3081
3082
3083
3084
3085
3086
3087
3088
3089
3090
3091
3092
3093
3094 public static HRegion createHRegion(final HRegionInfo info, final Path rootDir,
3095 final Configuration conf,
3096 final HTableDescriptor hTableDescriptor,
3097 final HLog hlog)
3098 throws IOException {
3099 LOG.info("creating HRegion " + info.getTableNameAsString()
3100 + " HTD == " + hTableDescriptor + " RootDir = " + rootDir +
3101 " Table name == " + info.getTableNameAsString());
3102
3103 Path tableDir =
3104 HTableDescriptor.getTableDir(rootDir, info.getTableName());
3105 Path regionDir = HRegion.getRegionDir(tableDir, info.getEncodedName());
3106 FileSystem fs = FileSystem.get(conf);
3107 fs.mkdirs(regionDir);
3108 HLog effectiveHLog = hlog;
3109 if (hlog == null) {
3110 effectiveHLog = new HLog(fs, new Path(regionDir, HConstants.HREGION_LOGDIR_NAME),
3111 new Path(regionDir, HConstants.HREGION_OLDLOGDIR_NAME), conf);
3112 }
3113 HRegion region = HRegion.newHRegion(tableDir,
3114 effectiveHLog, fs, conf, info, hTableDescriptor, null);
3115 region.initialize();
3116 return region;
3117 }
3118
3119
3120
3121
3122
3123
3124
3125
3126
3127
3128
3129
3130
3131 public static HRegion openHRegion(final HRegionInfo info,
3132 final HTableDescriptor htd, final HLog wal,
3133 final Configuration conf)
3134 throws IOException {
3135 return openHRegion(info, htd, wal, conf, null, null);
3136 }
3137
3138
3139
3140
3141
3142
3143
3144
3145
3146
3147
3148
3149
3150
3151
3152
3153 public static HRegion openHRegion(final HRegionInfo info,
3154 final HTableDescriptor htd, final HLog wal, final Configuration conf,
3155 final RegionServerServices rsServices,
3156 final CancelableProgressable reporter)
3157 throws IOException {
3158 if (LOG.isDebugEnabled()) {
3159 LOG.debug("Opening region: " + info);
3160 }
3161 if (info == null) {
3162 throw new NullPointerException("Passed region info is null");
3163 }
3164 Path dir = HTableDescriptor.getTableDir(FSUtils.getRootDir(conf),
3165 info.getTableName());
3166 HRegion r = HRegion.newHRegion(dir, wal, FileSystem.get(conf), conf, info,
3167 htd, rsServices);
3168 return r.openHRegion(reporter);
3169 }
3170
3171 public static HRegion openHRegion(Path tableDir, final HRegionInfo info,
3172 final HTableDescriptor htd, final HLog wal, final Configuration conf)
3173 throws IOException {
3174 return openHRegion(tableDir, info, htd, wal, conf, null, null);
3175 }
3176
3177
3178
3179
3180
3181
3182
3183
3184
3185
3186
3187
3188
3189
3190
3191 public static HRegion openHRegion(final Path tableDir, final HRegionInfo info,
3192 final HTableDescriptor htd, final HLog wal, final Configuration conf,
3193 final RegionServerServices rsServices,
3194 final CancelableProgressable reporter)
3195 throws IOException {
3196 if (info == null) throw new NullPointerException("Passed region info is null");
3197 LOG.info("HRegion.openHRegion Region name ==" + info.getRegionNameAsString());
3198 if (LOG.isDebugEnabled()) {
3199 LOG.debug("Opening region: " + info);
3200 }
3201 Path dir = HTableDescriptor.getTableDir(tableDir,
3202 info.getTableName());
3203 HRegion r = HRegion.newHRegion(dir, wal, FileSystem.get(conf), conf, info,
3204 htd, rsServices);
3205 return r.openHRegion(reporter);
3206 }
3207
3208
3209
3210
3211
3212
3213
3214
3215
3216 protected HRegion openHRegion(final CancelableProgressable reporter)
3217 throws IOException {
3218 checkCompressionCodecs();
3219
3220 long seqid = initialize(reporter);
3221 if (this.log != null) {
3222 this.log.setSequenceNumber(seqid);
3223 }
3224 return this;
3225 }
3226
3227 private void checkCompressionCodecs() throws IOException {
3228 for (HColumnDescriptor fam: this.htableDescriptor.getColumnFamilies()) {
3229 CompressionTest.testCompression(fam.getCompression());
3230 CompressionTest.testCompression(fam.getCompactionCompression());
3231 }
3232 }
3233
3234
3235
3236
3237
3238
3239
3240
3241
3242
3243
3244 public static void addRegionToMETA(HRegion meta, HRegion r)
3245 throws IOException {
3246 meta.checkResources();
3247
3248 byte[] row = r.getRegionName();
3249 Integer lid = meta.obtainRowLock(row);
3250 try {
3251 final long now = EnvironmentEdgeManager.currentTimeMillis();
3252 final List<KeyValue> edits = new ArrayList<KeyValue>(2);
3253 edits.add(new KeyValue(row, HConstants.CATALOG_FAMILY,
3254 HConstants.REGIONINFO_QUALIFIER, now,
3255 Writables.getBytes(r.getRegionInfo())));
3256
3257 edits.add(new KeyValue(row, HConstants.CATALOG_FAMILY,
3258 HConstants.META_VERSION_QUALIFIER, now,
3259 Bytes.toBytes(HConstants.META_VERSION)));
3260 meta.put(HConstants.CATALOG_FAMILY, edits);
3261 } finally {
3262 meta.releaseRowLock(lid);
3263 }
3264 }
3265
3266
3267
3268
3269
3270
3271
3272
3273
3274 public static void deleteRegion(FileSystem fs, Path rootdir, HRegionInfo info)
3275 throws IOException {
3276 deleteRegion(fs, HRegion.getRegionDir(rootdir, info));
3277 }
3278
3279 private static void deleteRegion(FileSystem fs, Path regiondir)
3280 throws IOException {
3281 if (LOG.isDebugEnabled()) {
3282 LOG.debug("DELETING region " + regiondir.toString());
3283 }
3284 if (!fs.delete(regiondir, true)) {
3285 LOG.warn("Failed delete of " + regiondir);
3286 }
3287 }
3288
3289
3290
3291
3292
3293
3294
3295
3296 public static Path getRegionDir(final Path rootdir, final HRegionInfo info) {
3297 return new Path(
3298 HTableDescriptor.getTableDir(rootdir, info.getTableName()),
3299 info.getEncodedName());
3300 }
3301
3302
3303
3304
3305
3306
3307
3308
3309
3310 public static boolean rowIsInRange(HRegionInfo info, final byte [] row) {
3311 return ((info.getStartKey().length == 0) ||
3312 (Bytes.compareTo(info.getStartKey(), row) <= 0)) &&
3313 ((info.getEndKey().length == 0) ||
3314 (Bytes.compareTo(info.getEndKey(), row) > 0));
3315 }
3316
3317
3318
3319
3320
3321
3322
3323
3324
3325
3326 public static void makeColumnFamilyDirs(FileSystem fs, Path tabledir,
3327 final HRegionInfo hri, byte [] colFamily)
3328 throws IOException {
3329 Path dir = Store.getStoreHomedir(tabledir, hri.getEncodedName(), colFamily);
3330 if (!fs.mkdirs(dir)) {
3331 LOG.warn("Failed to create " + dir);
3332 }
3333 }
3334
3335
3336
3337
3338
3339
3340
3341
3342
3343 public static HRegion mergeAdjacent(final HRegion srcA, final HRegion srcB)
3344 throws IOException {
3345 HRegion a = srcA;
3346 HRegion b = srcB;
3347
3348
3349
3350 if (srcA.getStartKey() == null) {
3351 if (srcB.getStartKey() == null) {
3352 throw new IOException("Cannot merge two regions with null start key");
3353 }
3354
3355 } else if ((srcB.getStartKey() == null) ||
3356 (Bytes.compareTo(srcA.getStartKey(), srcB.getStartKey()) > 0)) {
3357 a = srcB;
3358 b = srcA;
3359 }
3360
3361 if (!(Bytes.compareTo(a.getEndKey(), b.getStartKey()) == 0)) {
3362 throw new IOException("Cannot merge non-adjacent regions");
3363 }
3364 return merge(a, b);
3365 }
3366
3367
3368
3369
3370
3371
3372
3373
3374
3375 public static HRegion merge(HRegion a, HRegion b)
3376 throws IOException {
3377 if (!a.getRegionInfo().getTableNameAsString().equals(
3378 b.getRegionInfo().getTableNameAsString())) {
3379 throw new IOException("Regions do not belong to the same table");
3380 }
3381
3382 FileSystem fs = a.getFilesystem();
3383
3384
3385
3386 a.flushcache();
3387 b.flushcache();
3388
3389
3390
3391 a.compactStores(true);
3392 if (LOG.isDebugEnabled()) {
3393 LOG.debug("Files for region: " + a);
3394 listPaths(fs, a.getRegionDir());
3395 }
3396 b.compactStores(true);
3397 if (LOG.isDebugEnabled()) {
3398 LOG.debug("Files for region: " + b);
3399 listPaths(fs, b.getRegionDir());
3400 }
3401
3402 Configuration conf = a.getConf();
3403 HTableDescriptor tabledesc = a.getTableDesc();
3404 HLog log = a.getLog();
3405 Path tableDir = a.getTableDir();
3406
3407
3408 final byte[] startKey =
3409 (a.comparator.matchingRows(a.getStartKey(), 0, a.getStartKey().length,
3410 HConstants.EMPTY_BYTE_ARRAY, 0, HConstants.EMPTY_BYTE_ARRAY.length)
3411 || b.comparator.matchingRows(b.getStartKey(), 0,
3412 b.getStartKey().length, HConstants.EMPTY_BYTE_ARRAY, 0,
3413 HConstants.EMPTY_BYTE_ARRAY.length))
3414 ? HConstants.EMPTY_BYTE_ARRAY
3415 : (a.comparator.compareRows(a.getStartKey(), 0, a.getStartKey().length,
3416 b.getStartKey(), 0, b.getStartKey().length) <= 0
3417 ? a.getStartKey()
3418 : b.getStartKey());
3419 final byte[] endKey =
3420 (a.comparator.matchingRows(a.getEndKey(), 0, a.getEndKey().length,
3421 HConstants.EMPTY_BYTE_ARRAY, 0, HConstants.EMPTY_BYTE_ARRAY.length)
3422 || a.comparator.matchingRows(b.getEndKey(), 0, b.getEndKey().length,
3423 HConstants.EMPTY_BYTE_ARRAY, 0,
3424 HConstants.EMPTY_BYTE_ARRAY.length))
3425 ? HConstants.EMPTY_BYTE_ARRAY
3426 : (a.comparator.compareRows(a.getEndKey(), 0, a.getEndKey().length,
3427 b.getEndKey(), 0, b.getEndKey().length) <= 0
3428 ? b.getEndKey()
3429 : a.getEndKey());
3430
3431 HRegionInfo newRegionInfo =
3432 new HRegionInfo(tabledesc.getName(), startKey, endKey);
3433 LOG.info("Creating new region " + newRegionInfo.toString());
3434 String encodedName = newRegionInfo.getEncodedName();
3435 Path newRegionDir = HRegion.getRegionDir(a.getTableDir(), encodedName);
3436 if(fs.exists(newRegionDir)) {
3437 throw new IOException("Cannot merge; target file collision at " +
3438 newRegionDir);
3439 }
3440 fs.mkdirs(newRegionDir);
3441
3442 LOG.info("starting merge of regions: " + a + " and " + b +
3443 " into new region " + newRegionInfo.toString() +
3444 " with start key <" + Bytes.toStringBinary(startKey) + "> and end key <" +
3445 Bytes.toStringBinary(endKey) + ">");
3446
3447
3448 Map<byte [], List<StoreFile>> byFamily =
3449 new TreeMap<byte [], List<StoreFile>>(Bytes.BYTES_COMPARATOR);
3450 byFamily = filesByFamily(byFamily, a.close());
3451 byFamily = filesByFamily(byFamily, b.close());
3452 for (Map.Entry<byte [], List<StoreFile>> es : byFamily.entrySet()) {
3453 byte [] colFamily = es.getKey();
3454 makeColumnFamilyDirs(fs, tableDir, newRegionInfo, colFamily);
3455
3456
3457 List<StoreFile> srcFiles = es.getValue();
3458 if (srcFiles.size() == 2) {
3459 long seqA = srcFiles.get(0).getMaxSequenceId();
3460 long seqB = srcFiles.get(1).getMaxSequenceId();
3461 if (seqA == seqB) {
3462
3463
3464
3465 throw new IOException("Files have same sequenceid: " + seqA);
3466 }
3467 }
3468 for (StoreFile hsf: srcFiles) {
3469 StoreFile.rename(fs, hsf.getPath(),
3470 StoreFile.getUniqueFile(fs, Store.getStoreHomedir(tableDir,
3471 newRegionInfo.getEncodedName(), colFamily)));
3472 }
3473 }
3474 if (LOG.isDebugEnabled()) {
3475 LOG.debug("Files for new region");
3476 listPaths(fs, newRegionDir);
3477 }
3478 HRegion dstRegion = HRegion.newHRegion(tableDir, log, fs, conf,
3479 newRegionInfo, a.getTableDesc(), null);
3480 dstRegion.readRequestsCount.set(a.readRequestsCount.get() + b.readRequestsCount.get());
3481 dstRegion.writeRequestsCount.set(a.writeRequestsCount.get() + b.writeRequestsCount.get());
3482 dstRegion.initialize();
3483 dstRegion.compactStores();
3484 if (LOG.isDebugEnabled()) {
3485 LOG.debug("Files for new region");
3486 listPaths(fs, dstRegion.getRegionDir());
3487 }
3488 deleteRegion(fs, a.getRegionDir());
3489 deleteRegion(fs, b.getRegionDir());
3490
3491 LOG.info("merge completed. New region is " + dstRegion);
3492
3493 return dstRegion;
3494 }
3495
3496
3497
3498
3499
3500
3501
3502
3503 private static Map<byte [], List<StoreFile>> filesByFamily(
3504 Map<byte [], List<StoreFile>> byFamily, List<StoreFile> storeFiles) {
3505 for (StoreFile src: storeFiles) {
3506 byte [] family = src.getFamily();
3507 List<StoreFile> v = byFamily.get(family);
3508 if (v == null) {
3509 v = new ArrayList<StoreFile>();
3510 byFamily.put(family, v);
3511 }
3512 v.add(src);
3513 }
3514 return byFamily;
3515 }
3516
3517
3518
3519
3520
3521 boolean isMajorCompaction() throws IOException {
3522 for (Store store: this.stores.values()) {
3523 if (store.isMajorCompaction()) {
3524 return true;
3525 }
3526 }
3527 return false;
3528 }
3529
3530
3531
3532
3533
3534
3535
3536
3537 private static void listPaths(FileSystem fs, Path dir) throws IOException {
3538 if (LOG.isDebugEnabled()) {
3539 FileStatus[] stats = FSUtils.listStatus(fs, dir, null);
3540 if (stats == null || stats.length == 0) {
3541 return;
3542 }
3543 for (int i = 0; i < stats.length; i++) {
3544 String path = stats[i].getPath().toString();
3545 if (stats[i].isDir()) {
3546 LOG.debug("d " + path);
3547 listPaths(fs, stats[i].getPath());
3548 } else {
3549 LOG.debug("f " + path + " size=" + stats[i].getLen());
3550 }
3551 }
3552 }
3553 }
3554
3555
3556
3557
3558
3559
3560
3561
3562
3563
3564
3565 public Result get(final Get get, final Integer lockid) throws IOException {
3566 checkRow(get.getRow(), "Get");
3567
3568 if (get.hasFamilies()) {
3569 for (byte [] family: get.familySet()) {
3570 checkFamily(family);
3571 }
3572 } else {
3573 for (byte[] family: this.htableDescriptor.getFamiliesKeys()) {
3574 get.addFamily(family);
3575 }
3576 }
3577 List<KeyValue> results = get(get, true);
3578 return new Result(results);
3579 }
3580
3581
3582
3583
3584
3585
3586
3587
3588
3589
3590
3591
3592
3593
3594
3595 private List<KeyValue> getLastIncrement(final Get get) throws IOException {
3596 InternalScan iscan = new InternalScan(get);
3597
3598 List<KeyValue> results = new ArrayList<KeyValue>();
3599
3600
3601 iscan.checkOnlyMemStore();
3602 RegionScanner scanner = null;
3603 try {
3604 scanner = getScanner(iscan);
3605 scanner.next(results);
3606 } finally {
3607 if (scanner != null)
3608 scanner.close();
3609 }
3610
3611
3612 int expected = 0;
3613 Map<byte[], NavigableSet<byte[]>> familyMap = get.getFamilyMap();
3614 for (NavigableSet<byte[]> qfs : familyMap.values()) {
3615 expected += qfs.size();
3616 }
3617
3618
3619 if (results.size() == expected) {
3620 return results;
3621 }
3622
3623
3624 if (results != null && !results.isEmpty()) {
3625
3626 for (KeyValue kv : results) {
3627 byte [] family = kv.getFamily();
3628 NavigableSet<byte[]> qfs = familyMap.get(family);
3629 qfs.remove(kv.getQualifier());
3630 if (qfs.isEmpty()) familyMap.remove(family);
3631 expected--;
3632 }
3633
3634 Get newGet = new Get(get.getRow());
3635 for (Map.Entry<byte[], NavigableSet<byte[]>> f : familyMap.entrySet()) {
3636 byte [] family = f.getKey();
3637 for (byte [] qualifier : f.getValue()) {
3638 newGet.addColumn(family, qualifier);
3639 }
3640 }
3641 newGet.setTimeRange(get.getTimeRange().getMin(),
3642 get.getTimeRange().getMax());
3643 iscan = new InternalScan(newGet);
3644 }
3645
3646
3647 List<KeyValue> fileResults = new ArrayList<KeyValue>();
3648 iscan.checkOnlyStoreFiles();
3649 scanner = null;
3650 try {
3651 scanner = getScanner(iscan);
3652 scanner.next(fileResults);
3653 } finally {
3654 if (scanner != null)
3655 scanner.close();
3656 }
3657
3658
3659 results.addAll(fileResults);
3660 Collections.sort(results, KeyValue.COMPARATOR);
3661 return results;
3662 }
3663
3664
3665
3666
3667
3668
3669 private List<KeyValue> get(Get get, boolean withCoprocessor)
3670 throws IOException {
3671 Scan scan = new Scan(get);
3672
3673 List<KeyValue> results = new ArrayList<KeyValue>();
3674
3675
3676 if (withCoprocessor && (coprocessorHost != null)) {
3677 if (coprocessorHost.preGet(get, results)) {
3678 return results;
3679 }
3680 }
3681
3682 RegionScanner scanner = null;
3683 try {
3684 scanner = getScanner(scan);
3685 scanner.next(results);
3686 } finally {
3687 if (scanner != null)
3688 scanner.close();
3689 }
3690
3691
3692 if (withCoprocessor && (coprocessorHost != null)) {
3693 coprocessorHost.postGet(get, results);
3694 }
3695
3696 return results;
3697 }
3698
3699
3700
3701
3702
3703
3704
3705
3706
3707
3708
3709
3710
3711 public Result increment(Increment increment, Integer lockid,
3712 boolean writeToWAL)
3713 throws IOException {
3714
3715 byte [] row = increment.getRow();
3716 checkRow(row, "increment");
3717 TimeRange tr = increment.getTimeRange();
3718 boolean flush = false;
3719 WALEdit walEdits = null;
3720 List<KeyValue> allKVs = new ArrayList<KeyValue>(increment.numColumns());
3721 List<KeyValue> kvs = new ArrayList<KeyValue>(increment.numColumns());
3722 long now = EnvironmentEdgeManager.currentTimeMillis();
3723 long size = 0;
3724
3725
3726 startRegionOperation();
3727 this.writeRequestsCount.increment();
3728 try {
3729 Integer lid = getLock(lockid, row, true);
3730 this.updatesLock.readLock().lock();
3731 try {
3732
3733 for (Map.Entry<byte [], NavigableMap<byte [], Long>> family :
3734 increment.getFamilyMap().entrySet()) {
3735
3736 Store store = stores.get(family.getKey());
3737
3738
3739 Get get = new Get(row);
3740 for (Map.Entry<byte [], Long> column : family.getValue().entrySet()) {
3741 get.addColumn(family.getKey(), column.getKey());
3742 }
3743 get.setTimeRange(tr.getMin(), tr.getMax());
3744 List<KeyValue> results = getLastIncrement(get);
3745
3746
3747
3748 int idx = 0;
3749 for (Map.Entry<byte [], Long> column : family.getValue().entrySet()) {
3750 long amount = column.getValue();
3751 if (idx < results.size() &&
3752 results.get(idx).matchingQualifier(column.getKey())) {
3753 amount += Bytes.toLong(results.get(idx).getValue());
3754 idx++;
3755 }
3756
3757
3758 KeyValue newKV = new KeyValue(row, family.getKey(), column.getKey(),
3759 now, Bytes.toBytes(amount));
3760 kvs.add(newKV);
3761
3762
3763 if (writeToWAL) {
3764 if (walEdits == null) {
3765 walEdits = new WALEdit();
3766 }
3767 walEdits.add(newKV);
3768 }
3769 }
3770
3771
3772 size += store.upsert(kvs);
3773 allKVs.addAll(kvs);
3774 kvs.clear();
3775 }
3776
3777
3778 if (writeToWAL) {
3779
3780
3781
3782 this.log.append(regionInfo, this.htableDescriptor.getName(),
3783 walEdits, HConstants.DEFAULT_CLUSTER_ID, now,
3784 this.htableDescriptor);
3785 }
3786
3787 size = this.addAndGetGlobalMemstoreSize(size);
3788 flush = isFlushSize(size);
3789 } finally {
3790 this.updatesLock.readLock().unlock();
3791 releaseRowLock(lid);
3792 }
3793 } finally {
3794 closeRegionOperation();
3795 }
3796
3797 if (flush) {
3798
3799 requestFlush();
3800 }
3801
3802 return new Result(allKVs);
3803 }
3804
3805
3806
3807
3808
3809
3810
3811
3812
3813
3814 public long incrementColumnValue(byte [] row, byte [] family,
3815 byte [] qualifier, long amount, boolean writeToWAL)
3816 throws IOException {
3817 checkRow(row, "increment");
3818 boolean flush = false;
3819 boolean wrongLength = false;
3820
3821 long result = amount;
3822 startRegionOperation();
3823 this.writeRequestsCount.increment();
3824 try {
3825 Integer lid = obtainRowLock(row);
3826 this.updatesLock.readLock().lock();
3827 try {
3828 Store store = stores.get(family);
3829
3830
3831 Get get = new Get(row);
3832 get.addColumn(family, qualifier);
3833
3834
3835
3836 List<KeyValue> results = getLastIncrement(get);
3837
3838 if (!results.isEmpty()) {
3839 KeyValue kv = results.get(0);
3840 if(kv.getValueLength() == 8){
3841 byte [] buffer = kv.getBuffer();
3842 int valueOffset = kv.getValueOffset();
3843 result += Bytes.toLong(buffer, valueOffset, Bytes.SIZEOF_LONG);
3844 }
3845 else{
3846 wrongLength = true;
3847 }
3848 }
3849 if(!wrongLength){
3850
3851 KeyValue newKv = new KeyValue(row, family,
3852 qualifier, EnvironmentEdgeManager.currentTimeMillis(),
3853 Bytes.toBytes(result));
3854
3855
3856 if (writeToWAL) {
3857 long now = EnvironmentEdgeManager.currentTimeMillis();
3858 WALEdit walEdit = new WALEdit();
3859 walEdit.add(newKv);
3860
3861
3862
3863 this.log.append(regionInfo, this.htableDescriptor.getName(),
3864 walEdit, HConstants.DEFAULT_CLUSTER_ID, now,
3865 this.htableDescriptor);
3866 }
3867
3868
3869
3870
3871 long size = store.updateColumnValue(row, family, qualifier, result);
3872
3873 size = this.addAndGetGlobalMemstoreSize(size);
3874 flush = isFlushSize(size);
3875 }
3876 } finally {
3877 this.updatesLock.readLock().unlock();
3878 releaseRowLock(lid);
3879 }
3880 } finally {
3881 closeRegionOperation();
3882 }
3883
3884 if (flush) {
3885
3886 requestFlush();
3887 }
3888 if(wrongLength){
3889 throw new IOException("Attempted to increment field that isn't 64 bits wide");
3890 }
3891 return result;
3892 }
3893
3894
3895
3896
3897
3898
3899 private void checkFamily(final byte [] family)
3900 throws NoSuchColumnFamilyException {
3901 if (!this.htableDescriptor.hasFamily(family)) {
3902 throw new NoSuchColumnFamilyException("Column family " +
3903 Bytes.toString(family) + " does not exist in region " + this
3904 + " in table " + this.htableDescriptor);
3905 }
3906 }
3907
3908 public static final long FIXED_OVERHEAD = ClassSize.align(
3909 ClassSize.OBJECT +
3910 ClassSize.ARRAY +
3911 29 * ClassSize.REFERENCE + Bytes.SIZEOF_INT +
3912 (4 * Bytes.SIZEOF_LONG) +
3913 Bytes.SIZEOF_BOOLEAN);
3914
3915 public static final long DEEP_OVERHEAD = FIXED_OVERHEAD +
3916 ClassSize.OBJECT +
3917 (2 * ClassSize.ATOMIC_BOOLEAN) +
3918 ClassSize.ATOMIC_LONG +
3919 ClassSize.ATOMIC_INTEGER +
3920 (3 * ClassSize.CONCURRENT_HASHMAP) +
3921 WriteState.HEAP_SIZE +
3922 ClassSize.CONCURRENT_SKIPLISTMAP + ClassSize.CONCURRENT_SKIPLISTMAP_ENTRY +
3923 (2 * ClassSize.REENTRANT_LOCK) +
3924 ClassSize.ARRAYLIST +
3925 MultiVersionConsistencyControl.FIXED_SIZE
3926 ;
3927
3928 @Override
3929 public long heapSize() {
3930 long heapSize = DEEP_OVERHEAD;
3931 for(Store store : this.stores.values()) {
3932 heapSize += store.heapSize();
3933 }
3934
3935 return heapSize;
3936 }
3937
3938
3939
3940
3941
3942 private static void printUsageAndExit(final String message) {
3943 if (message != null && message.length() > 0) System.out.println(message);
3944 System.out.println("Usage: HRegion CATLALOG_TABLE_DIR [major_compact]");
3945 System.out.println("Options:");
3946 System.out.println(" major_compact Pass this option to major compact " +
3947 "passed region.");
3948 System.out.println("Default outputs scan of passed region.");
3949 System.exit(1);
3950 }
3951
3952
3953
3954
3955
3956
3957
3958
3959
3960
3961
3962
3963
3964
3965
3966
3967
3968
3969 public <T extends CoprocessorProtocol> boolean registerProtocol(
3970 Class<T> protocol, T handler) {
3971
3972
3973
3974
3975 if (protocolHandlers.containsKey(protocol)) {
3976 LOG.error("Protocol "+protocol.getName()+
3977 " already registered, rejecting request from "+
3978 handler
3979 );
3980 return false;
3981 }
3982
3983 protocolHandlers.putInstance(protocol, handler);
3984 if (LOG.isDebugEnabled()) {
3985 LOG.debug("Registered protocol handler: region="+
3986 Bytes.toStringBinary(getRegionName())+" protocol="+protocol.getName());
3987 }
3988 return true;
3989 }
3990
3991
3992
3993
3994
3995
3996
3997
3998
3999
4000
4001
4002
4003
4004
4005
4006 public ExecResult exec(Exec call)
4007 throws IOException {
4008 Class<? extends CoprocessorProtocol> protocol = call.getProtocol();
4009 if (!protocolHandlers.containsKey(protocol)) {
4010 throw new HBaseRPC.UnknownProtocolException(protocol,
4011 "No matching handler for protocol "+protocol.getName()+
4012 " in region "+Bytes.toStringBinary(getRegionName()));
4013 }
4014
4015 CoprocessorProtocol handler = protocolHandlers.getInstance(protocol);
4016 Object value;
4017
4018 try {
4019 Method method = protocol.getMethod(
4020 call.getMethodName(), call.getParameterClasses());
4021 method.setAccessible(true);
4022
4023 value = method.invoke(handler, call.getParameters());
4024 } catch (InvocationTargetException e) {
4025 Throwable target = e.getTargetException();
4026 if (target instanceof IOException) {
4027 throw (IOException)target;
4028 }
4029 IOException ioe = new IOException(target.toString());
4030 ioe.setStackTrace(target.getStackTrace());
4031 throw ioe;
4032 } catch (Throwable e) {
4033 if (!(e instanceof IOException)) {
4034 LOG.error("Unexpected throwable object ", e);
4035 }
4036 IOException ioe = new IOException(e.toString());
4037 ioe.setStackTrace(e.getStackTrace());
4038 throw ioe;
4039 }
4040
4041 return new ExecResult(getRegionName(), value);
4042 }
4043
4044
4045
4046
4047
4048
4049
4050
4051
4052
4053
4054 private static void processTable(final FileSystem fs, final Path p,
4055 final HLog log, final Configuration c,
4056 final boolean majorCompact)
4057 throws IOException {
4058 HRegion region = null;
4059 String rootStr = Bytes.toString(HConstants.ROOT_TABLE_NAME);
4060 String metaStr = Bytes.toString(HConstants.META_TABLE_NAME);
4061
4062 if (p.getName().startsWith(rootStr)) {
4063 region = HRegion.newHRegion(p, log, fs, c, HRegionInfo.ROOT_REGIONINFO,
4064 HTableDescriptor.ROOT_TABLEDESC, null);
4065 } else if (p.getName().startsWith(metaStr)) {
4066 region = HRegion.newHRegion(p, log, fs, c,
4067 HRegionInfo.FIRST_META_REGIONINFO, HTableDescriptor.META_TABLEDESC, null);
4068 } else {
4069 throw new IOException("Not a known catalog table: " + p.toString());
4070 }
4071 try {
4072 region.initialize();
4073 if (majorCompact) {
4074 region.compactStores(true);
4075 } else {
4076
4077 Scan scan = new Scan();
4078
4079 RegionScanner scanner = region.getScanner(scan);
4080 try {
4081 List<KeyValue> kvs = new ArrayList<KeyValue>();
4082 boolean done = false;
4083 do {
4084 kvs.clear();
4085 done = scanner.next(kvs);
4086 if (kvs.size() > 0) LOG.info(kvs);
4087 } while (done);
4088 } finally {
4089 scanner.close();
4090 }
4091 }
4092 } finally {
4093 region.close();
4094 }
4095 }
4096
4097 boolean shouldForceSplit() {
4098 return this.splitRequest;
4099 }
4100
4101 byte[] getExplicitSplitPoint() {
4102 return this.explicitSplitPoint;
4103 }
4104
4105 void forceSplit(byte[] sp) {
4106
4107
4108 this.splitRequest = true;
4109 if (sp != null) {
4110 this.explicitSplitPoint = sp;
4111 }
4112 }
4113
4114 void clearSplit_TESTS_ONLY() {
4115 this.splitRequest = false;
4116 }
4117
4118
4119
4120
4121 protected void prepareToSplit() {
4122
4123 }
4124
4125
4126
4127
4128
4129
4130
4131 public byte[] checkSplit() {
4132
4133 if (getRegionInfo().isMetaRegion()) {
4134 if (shouldForceSplit()) {
4135 LOG.warn("Cannot split meta regions in HBase 0.20 and above");
4136 }
4137 return null;
4138 }
4139
4140 if (this.explicitSplitPoint != null) {
4141 return this.explicitSplitPoint;
4142 }
4143
4144 if (!splitPolicy.shouldSplit()) {
4145 return null;
4146 }
4147
4148 byte[] ret = splitPolicy.getSplitPoint();
4149
4150 if (ret != null) {
4151 try {
4152 checkRow(ret, "calculated split");
4153 } catch (IOException e) {
4154 LOG.error("Ignoring invalid split", e);
4155 return null;
4156 }
4157 }
4158 return ret;
4159 }
4160
4161
4162
4163
4164 public int getCompactPriority() {
4165 int count = Integer.MAX_VALUE;
4166 for(Store store : stores.values()) {
4167 count = Math.min(count, store.getCompactPriority());
4168 }
4169 return count;
4170 }
4171
4172
4173
4174
4175
4176
4177 public boolean needsCompaction() {
4178 for(Store store : stores.values()) {
4179 if(store.needsCompaction()) {
4180 return true;
4181 }
4182 }
4183 return false;
4184 }
4185
4186
4187 public RegionCoprocessorHost getCoprocessorHost() {
4188 return coprocessorHost;
4189 }
4190
4191
4192 public void setCoprocessorHost(final RegionCoprocessorHost coprocessorHost) {
4193 this.coprocessorHost = coprocessorHost;
4194 }
4195
4196
4197
4198
4199
4200
4201
4202
4203 private void startRegionOperation() throws NotServingRegionException {
4204 if (this.closing.get()) {
4205 throw new NotServingRegionException(regionInfo.getRegionNameAsString() +
4206 " is closing");
4207 }
4208 lock.readLock().lock();
4209 if (this.closed.get()) {
4210 lock.readLock().unlock();
4211 throw new NotServingRegionException(regionInfo.getRegionNameAsString() +
4212 " is closed");
4213 }
4214 }
4215
4216
4217
4218
4219
4220 private void closeRegionOperation(){
4221 lock.readLock().unlock();
4222 }
4223
4224
4225
4226
4227
4228
4229
4230
4231 private void startBulkRegionOperation(boolean writeLockNeeded)
4232 throws NotServingRegionException {
4233 if (this.closing.get()) {
4234 throw new NotServingRegionException(regionInfo.getRegionNameAsString() +
4235 " is closing");
4236 }
4237 if (writeLockNeeded) lock.writeLock().lock();
4238 else lock.readLock().lock();
4239 if (this.closed.get()) {
4240 if (writeLockNeeded) lock.writeLock().unlock();
4241 else lock.readLock().unlock();
4242 throw new NotServingRegionException(regionInfo.getRegionNameAsString() +
4243 " is closed");
4244 }
4245 }
4246
4247
4248
4249
4250
4251 private void closeBulkRegionOperation(){
4252 if (lock.writeLock().isHeldByCurrentThread()) lock.writeLock().unlock();
4253 else lock.readLock().unlock();
4254 }
4255
4256
4257
4258
4259 private static final List<KeyValue> MOCKED_LIST = new AbstractList<KeyValue>() {
4260
4261 @Override
4262 public void add(int index, KeyValue element) {
4263
4264 }
4265
4266 @Override
4267 public boolean addAll(int index, Collection<? extends KeyValue> c) {
4268 return false;
4269 }
4270
4271 @Override
4272 public KeyValue get(int index) {
4273 throw new UnsupportedOperationException();
4274 }
4275
4276 @Override
4277 public int size() {
4278 return 0;
4279 }
4280 };
4281
4282
4283
4284
4285
4286
4287
4288
4289
4290
4291
4292
4293 public static void main(String[] args) throws IOException {
4294 if (args.length < 1) {
4295 printUsageAndExit(null);
4296 }
4297 boolean majorCompact = false;
4298 if (args.length > 1) {
4299 if (!args[1].toLowerCase().startsWith("major")) {
4300 printUsageAndExit("ERROR: Unrecognized option <" + args[1] + ">");
4301 }
4302 majorCompact = true;
4303 }
4304 final Path tableDir = new Path(args[0]);
4305 final Configuration c = HBaseConfiguration.create();
4306 final FileSystem fs = FileSystem.get(c);
4307 final Path logdir = new Path(c.get("hbase.tmp.dir"),
4308 "hlog" + tableDir.getName()
4309 + EnvironmentEdgeManager.currentTimeMillis());
4310 final Path oldLogDir = new Path(c.get("hbase.tmp.dir"),
4311 HConstants.HREGION_OLDLOGDIR_NAME);
4312 final HLog log = new HLog(fs, logdir, oldLogDir, c);
4313 try {
4314 processTable(fs, tableDir, log, c, majorCompact);
4315 } finally {
4316 log.close();
4317
4318 BlockCache bc = new CacheConfig(c).getBlockCache();
4319 if (bc != null) bc.shutdown();
4320 }
4321 }
4322 }