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