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