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