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