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