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