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