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