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