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