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