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