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