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