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