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