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