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