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