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 com.google.common.base.Optional;
22 import com.google.common.base.Preconditions;
23 import com.google.common.collect.Iterables;
24 import com.google.common.collect.Lists;
25 import com.google.common.collect.Maps;
26 import com.google.common.collect.Sets;
27 import com.google.common.io.Closeables;
28 import com.google.protobuf.ByteString;
29 import com.google.protobuf.Descriptors;
30 import com.google.protobuf.Message;
31 import com.google.protobuf.RpcCallback;
32 import com.google.protobuf.RpcController;
33 import com.google.protobuf.Service;
34 import com.google.protobuf.TextFormat;
35
36 import java.io.EOFException;
37 import java.io.FileNotFoundException;
38 import java.io.IOException;
39 import java.io.InterruptedIOException;
40 import java.lang.reflect.Constructor;
41 import java.nio.charset.StandardCharsets;
42 import java.text.ParseException;
43 import java.util.AbstractList;
44 import java.util.ArrayList;
45 import java.util.Arrays;
46 import java.util.Collection;
47 import java.util.Collections;
48 import java.util.Comparator;
49 import java.util.HashMap;
50 import java.util.HashSet;
51 import java.util.Iterator;
52 import java.util.LinkedHashMap;
53 import java.util.List;
54 import java.util.Locale;
55 import java.util.Map;
56 import java.util.Map.Entry;
57 import java.util.NavigableMap;
58 import java.util.NavigableSet;
59 import java.util.RandomAccess;
60 import java.util.Set;
61 import java.util.TreeMap;
62 import java.util.concurrent.Callable;
63 import java.util.concurrent.CompletionService;
64 import java.util.concurrent.ConcurrentHashMap;
65 import java.util.concurrent.ConcurrentMap;
66 import java.util.concurrent.ConcurrentSkipListMap;
67 import java.util.concurrent.ExecutionException;
68 import java.util.concurrent.ExecutorCompletionService;
69 import java.util.concurrent.ExecutorService;
70 import java.util.concurrent.Executors;
71 import java.util.concurrent.Future;
72 import java.util.concurrent.FutureTask;
73 import java.util.concurrent.ThreadFactory;
74 import java.util.concurrent.ThreadPoolExecutor;
75 import java.util.concurrent.TimeUnit;
76 import java.util.concurrent.TimeoutException;
77 import java.util.concurrent.atomic.AtomicBoolean;
78 import java.util.concurrent.atomic.AtomicInteger;
79 import java.util.concurrent.atomic.AtomicLong;
80 import java.util.concurrent.locks.Lock;
81 import java.util.concurrent.locks.ReadWriteLock;
82 import java.util.concurrent.locks.ReentrantReadWriteLock;
83
84 import org.apache.commons.lang.RandomStringUtils;
85 import org.apache.commons.logging.Log;
86 import org.apache.commons.logging.LogFactory;
87 import org.apache.hadoop.conf.Configuration;
88 import org.apache.hadoop.fs.FileStatus;
89 import org.apache.hadoop.fs.FileSystem;
90 import org.apache.hadoop.fs.LocatedFileStatus;
91 import org.apache.hadoop.fs.Path;
92 import org.apache.hadoop.hbase.Cell;
93 import org.apache.hadoop.hbase.CellScanner;
94 import org.apache.hadoop.hbase.CellUtil;
95 import org.apache.hadoop.hbase.CompoundConfiguration;
96 import org.apache.hadoop.hbase.DoNotRetryIOException;
97 import org.apache.hadoop.hbase.DroppedSnapshotException;
98 import org.apache.hadoop.hbase.HBaseConfiguration;
99 import org.apache.hadoop.hbase.HColumnDescriptor;
100 import org.apache.hadoop.hbase.HConstants;
101 import org.apache.hadoop.hbase.HConstants.OperationStatusCode;
102 import org.apache.hadoop.hbase.HDFSBlocksDistribution;
103 import org.apache.hadoop.hbase.HRegionInfo;
104 import org.apache.hadoop.hbase.HTableDescriptor;
105 import org.apache.hadoop.hbase.KeyValue;
106 import org.apache.hadoop.hbase.KeyValueUtil;
107 import org.apache.hadoop.hbase.NamespaceDescriptor;
108 import org.apache.hadoop.hbase.NotServingRegionException;
109 import org.apache.hadoop.hbase.RegionTooBusyException;
110 import org.apache.hadoop.hbase.TableName;
111 import org.apache.hadoop.hbase.Tag;
112 import org.apache.hadoop.hbase.TagType;
113 import org.apache.hadoop.hbase.UnknownScannerException;
114 import org.apache.hadoop.hbase.backup.HFileArchiver;
115 import org.apache.hadoop.hbase.classification.InterfaceAudience;
116 import org.apache.hadoop.hbase.client.Append;
117 import org.apache.hadoop.hbase.client.Delete;
118 import org.apache.hadoop.hbase.client.Durability;
119 import org.apache.hadoop.hbase.client.Get;
120 import org.apache.hadoop.hbase.client.Increment;
121 import org.apache.hadoop.hbase.client.IsolationLevel;
122 import org.apache.hadoop.hbase.client.Mutation;
123 import org.apache.hadoop.hbase.client.PackagePrivateFieldAccessor;
124 import org.apache.hadoop.hbase.client.Put;
125 import org.apache.hadoop.hbase.client.RegionReplicaUtil;
126 import org.apache.hadoop.hbase.client.Result;
127 import org.apache.hadoop.hbase.client.RowMutations;
128 import org.apache.hadoop.hbase.client.Scan;
129 import org.apache.hadoop.hbase.conf.ConfigurationManager;
130 import org.apache.hadoop.hbase.conf.PropagatingConfigurationObserver;
131 import org.apache.hadoop.hbase.coprocessor.RegionObserver;
132 import org.apache.hadoop.hbase.errorhandling.ForeignExceptionSnare;
133 import org.apache.hadoop.hbase.exceptions.FailedSanityCheckException;
134 import org.apache.hadoop.hbase.exceptions.RegionInRecoveryException;
135 import org.apache.hadoop.hbase.exceptions.TimeoutIOException;
136 import org.apache.hadoop.hbase.exceptions.UnknownProtocolException;
137 import org.apache.hadoop.hbase.filter.ByteArrayComparable;
138 import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp;
139 import org.apache.hadoop.hbase.filter.FilterWrapper;
140 import org.apache.hadoop.hbase.filter.IncompatibleFilterException;
141 import org.apache.hadoop.hbase.filter.PrefixFilter;
142 import org.apache.hadoop.hbase.io.HFileLink;
143 import org.apache.hadoop.hbase.io.HeapSize;
144 import org.apache.hadoop.hbase.io.TimeRange;
145 import org.apache.hadoop.hbase.io.hfile.BlockCache;
146 import org.apache.hadoop.hbase.io.hfile.CacheConfig;
147 import org.apache.hadoop.hbase.io.hfile.HFile;
148 import org.apache.hadoop.hbase.ipc.CallerDisconnectedException;
149 import org.apache.hadoop.hbase.ipc.CoprocessorRpcUtils;
150 import org.apache.hadoop.hbase.ipc.RpcCallContext;
151 import org.apache.hadoop.hbase.ipc.RpcServer;
152 import org.apache.hadoop.hbase.monitoring.MonitoredTask;
153 import org.apache.hadoop.hbase.monitoring.TaskMonitor;
154 import org.apache.hadoop.hbase.protobuf.ProtobufUtil;
155 import org.apache.hadoop.hbase.protobuf.ResponseConverter;
156 import org.apache.hadoop.hbase.protobuf.generated.AdminProtos.GetRegionInfoResponse.CompactionState;
157 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos;
158 import org.apache.hadoop.hbase.protobuf.generated.ClientProtos.CoprocessorServiceCall;
159 import org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.RegionLoad;
160 import org.apache.hadoop.hbase.protobuf.generated.ClusterStatusProtos.StoreSequenceId;
161 import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.SnapshotDescription;
162 import org.apache.hadoop.hbase.protobuf.generated.WALProtos;
163 import org.apache.hadoop.hbase.protobuf.generated.WALProtos.CompactionDescriptor;
164 import org.apache.hadoop.hbase.protobuf.generated.WALProtos.FlushDescriptor;
165 import org.apache.hadoop.hbase.protobuf.generated.WALProtos.FlushDescriptor.FlushAction;
166 import org.apache.hadoop.hbase.protobuf.generated.WALProtos.FlushDescriptor.StoreFlushDescriptor;
167 import org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor;
168 import org.apache.hadoop.hbase.protobuf.generated.WALProtos.RegionEventDescriptor.EventType;
169 import org.apache.hadoop.hbase.protobuf.generated.WALProtos.StoreDescriptor;
170 import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl.WriteEntry;
171 import org.apache.hadoop.hbase.regionserver.ScannerContext.LimitScope;
172 import org.apache.hadoop.hbase.regionserver.ScannerContext.NextState;
173 import org.apache.hadoop.hbase.regionserver.compactions.CompactionContext;
174 import org.apache.hadoop.hbase.regionserver.throttle.CompactionThroughputControllerFactory;
175 import org.apache.hadoop.hbase.regionserver.throttle.NoLimitThroughputController;
176 import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController;
177 import org.apache.hadoop.hbase.regionserver.wal.HLogKey;
178 import org.apache.hadoop.hbase.regionserver.wal.MetricsWAL;
179 import org.apache.hadoop.hbase.regionserver.wal.ReplayHLogKey;
180 import org.apache.hadoop.hbase.regionserver.wal.WALActionsListener;
181 import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
182 import org.apache.hadoop.hbase.regionserver.wal.WALUtil;
183 import org.apache.hadoop.hbase.security.User;
184 import org.apache.hadoop.hbase.snapshot.SnapshotDescriptionUtils;
185 import org.apache.hadoop.hbase.snapshot.SnapshotManifest;
186 import org.apache.hadoop.hbase.util.ByteStringer;
187 import org.apache.hadoop.hbase.util.Bytes;
188 import org.apache.hadoop.hbase.util.CancelableProgressable;
189 import org.apache.hadoop.hbase.util.ClassSize;
190 import org.apache.hadoop.hbase.util.CompressionTest;
191 import org.apache.hadoop.hbase.util.Counter;
192 import org.apache.hadoop.hbase.util.EncryptionTest;
193 import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
194 import org.apache.hadoop.hbase.util.FSTableDescriptors;
195 import org.apache.hadoop.hbase.util.FSUtils;
196 import org.apache.hadoop.hbase.util.HashedBytes;
197 import org.apache.hadoop.hbase.util.Pair;
198 import org.apache.hadoop.hbase.util.ServerRegionReplicaUtil;
199 import org.apache.hadoop.hbase.util.Threads;
200 import org.apache.hadoop.hbase.wal.WAL;
201 import org.apache.hadoop.hbase.wal.WALFactory;
202 import org.apache.hadoop.hbase.wal.WALKey;
203 import org.apache.hadoop.hbase.wal.WALSplitter;
204 import org.apache.hadoop.hbase.wal.WALSplitter.MutationReplay;
205 import org.apache.hadoop.io.MultipleIOException;
206 import org.apache.hadoop.util.StringUtils;
207 import org.apache.htrace.Trace;
208 import org.apache.htrace.TraceScope;
209
210 @InterfaceAudience.Private
211 @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="JLM_JSR166_UTILCONCURRENT_MONITORENTER",
212 justification="Synchronization on concurrent map is intended")
213 public class HRegion implements HeapSize, PropagatingConfigurationObserver, Region {
214 private static final Log LOG = LogFactory.getLog(HRegion.class);
215
216 public static final String LOAD_CFS_ON_DEMAND_CONFIG_KEY =
217 "hbase.hregion.scan.loadColumnFamiliesOnDemand";
218
219
220 public static final String HREGION_MVCC_PRE_ASSIGN = "hbase.hregion.mvcc.preassign";
221 public static final boolean DEFAULT_HREGION_MVCC_PRE_ASSIGN = true;
222
223 public static final String HBASE_MAX_CELL_SIZE_KEY = "hbase.server.keyvalue.maxsize";
224 public static final int DEFAULT_MAX_CELL_SIZE = 10485760;
225
226 public static final String WAL_HSYNC_CONF_KEY = "hbase.wal.hsync";
227 public static final boolean DEFAULT_WAL_HSYNC = false;
228
229
230
231
232
233
234
235
236
237 private final int maxWaitForSeqId;
238 private static final String MAX_WAIT_FOR_SEQ_ID_KEY = "hbase.hregion.max.wait.for.sequenceid.ms";
239 private static final int DEFAULT_MAX_WAIT_FOR_SEQ_ID = 30000;
240
241
242
243
244
245 private static final Durability DEFAULT_DURABILITY = Durability.SYNC_WAL;
246
247 final AtomicBoolean closed = new AtomicBoolean(false);
248
249
250
251
252
253
254 final AtomicBoolean closing = new AtomicBoolean(false);
255
256
257
258
259
260 private volatile long maxFlushedSeqId = HConstants.NO_SEQNUM;
261
262
263
264
265
266
267 private volatile long lastFlushOpSeqId = HConstants.NO_SEQNUM;
268
269
270
271
272
273
274 protected volatile long lastReplayedOpenRegionSeqId = -1L;
275 protected volatile long lastReplayedCompactionSeqId = -1L;
276
277
278
279
280
281
282
283
284
285
286 private final ConcurrentHashMap<HashedBytes, RowLockContext> lockedRows =
287 new ConcurrentHashMap<HashedBytes, RowLockContext>();
288
289 protected final Map<byte[], Store> stores = new ConcurrentSkipListMap<byte[], Store>(
290 Bytes.BYTES_RAWCOMPARATOR);
291
292
293 private Map<String, Service> coprocessorServiceHandlers = Maps.newHashMap();
294
295 private final AtomicLong memstoreSize = new AtomicLong(0);
296
297
298 final Counter numMutationsWithoutWAL = new Counter();
299 final Counter dataInMemoryWithoutWAL = new Counter();
300
301
302 final Counter checkAndMutateChecksPassed = new Counter();
303 final Counter checkAndMutateChecksFailed = new Counter();
304
305
306
307 final Counter readRequestsCount = new Counter();
308
309 final Counter writeRequestsCount = new Counter();
310
311
312 private final Counter blockedRequestsCount = new Counter();
313
314
315 final AtomicLong compactionsFinished = new AtomicLong(0L);
316 final AtomicLong compactionsFailed = new AtomicLong(0L);
317 final AtomicLong compactionNumFilesCompacted = new AtomicLong(0L);
318 final AtomicLong compactionNumBytesCompacted = new AtomicLong(0L);
319 final AtomicLong compactionsQueued = new AtomicLong(0L);
320 final AtomicLong flushesQueued = new AtomicLong(0L);
321
322 private final WAL wal;
323 private final HRegionFileSystem fs;
324 protected final Configuration conf;
325 private final Configuration baseConf;
326 private final KeyValue.KVComparator comparator;
327 private final int rowLockWaitDuration;
328 static final int DEFAULT_ROWLOCK_WAIT_DURATION = 30000;
329
330 private Path regionDir;
331 private FileSystem walFS;
332
333
334
335
336
337
338
339 final long busyWaitDuration;
340 static final long DEFAULT_BUSY_WAIT_DURATION = HConstants.DEFAULT_HBASE_RPC_TIMEOUT;
341
342
343
344
345 final int maxBusyWaitMultiplier;
346
347
348
349 final long maxBusyWaitDuration;
350
351
352
353 final long maxCellSize;
354
355
356 static final long DEFAULT_ROW_PROCESSOR_TIMEOUT = 60 * 1000L;
357 final ExecutorService rowProcessorExecutor = Executors.newCachedThreadPool();
358
359 private final ConcurrentHashMap<RegionScanner, Long> scannerReadPoints;
360
361
362
363
364 private long openSeqNum = HConstants.NO_SEQNUM;
365
366
367
368
369
370 private boolean isLoadingCfsOnDemandDefault = false;
371
372 private final AtomicInteger majorInProgress = new AtomicInteger(0);
373 private final AtomicInteger minorInProgress = new AtomicInteger(0);
374
375
376
377
378
379
380
381 Map<byte[], Long> maxSeqIdInStores = new TreeMap<byte[], Long>(Bytes.BYTES_COMPARATOR);
382
383
384 private PrepareFlushResult prepareFlushResult = null;
385
386
387
388
389 private boolean disallowWritesInRecovering = false;
390
391
392 private volatile boolean recovering = false;
393
394 private volatile Optional<ConfigurationManager> configurationManager;
395
396
397
398
399
400
401 public long getSmallestReadPoint() {
402 long minimumReadPoint;
403
404
405
406 synchronized(scannerReadPoints) {
407 minimumReadPoint = mvcc.getReadPoint();
408
409 for (Long readPoint: this.scannerReadPoints.values()) {
410 if (readPoint < minimumReadPoint) {
411 minimumReadPoint = readPoint;
412 }
413 }
414 }
415 return minimumReadPoint;
416 }
417
418
419
420
421
422 static class WriteState {
423
424 volatile boolean flushing = false;
425
426 volatile boolean flushRequested = false;
427
428 AtomicInteger compacting = new AtomicInteger(0);
429
430 volatile boolean writesEnabled = true;
431
432 volatile boolean readOnly = false;
433
434
435 volatile boolean readsEnabled = true;
436
437
438
439
440
441
442 synchronized void setReadOnly(final boolean onOff) {
443 this.writesEnabled = !onOff;
444 this.readOnly = onOff;
445 }
446
447 boolean isReadOnly() {
448 return this.readOnly;
449 }
450
451 boolean isFlushRequested() {
452 return this.flushRequested;
453 }
454
455 void setReadsEnabled(boolean readsEnabled) {
456 this.readsEnabled = readsEnabled;
457 }
458
459 static final long HEAP_SIZE = ClassSize.align(
460 ClassSize.OBJECT + 5 * Bytes.SIZEOF_BOOLEAN);
461 }
462
463
464
465
466
467
468
469 public static class FlushResultImpl implements FlushResult {
470 final Result result;
471 final String failureReason;
472 final long flushSequenceId;
473 final boolean wroteFlushWalMarker;
474
475
476
477
478
479
480
481
482 FlushResultImpl(Result result, long flushSequenceId) {
483 this(result, flushSequenceId, null, false);
484 assert result == Result.FLUSHED_NO_COMPACTION_NEEDED || result == Result
485 .FLUSHED_COMPACTION_NEEDED;
486 }
487
488
489
490
491
492
493 FlushResultImpl(Result result, String failureReason, boolean wroteFlushMarker) {
494 this(result, -1, failureReason, wroteFlushMarker);
495 assert result == Result.CANNOT_FLUSH_MEMSTORE_EMPTY || result == Result.CANNOT_FLUSH;
496 }
497
498
499
500
501
502
503
504 FlushResultImpl(Result result, long flushSequenceId, String failureReason,
505 boolean wroteFlushMarker) {
506 this.result = result;
507 this.flushSequenceId = flushSequenceId;
508 this.failureReason = failureReason;
509 this.wroteFlushWalMarker = wroteFlushMarker;
510 }
511
512
513
514
515
516
517 @Override
518 public boolean isFlushSucceeded() {
519 return result == Result.FLUSHED_NO_COMPACTION_NEEDED || result == Result
520 .FLUSHED_COMPACTION_NEEDED;
521 }
522
523
524
525
526
527 @Override
528 public boolean isCompactionNeeded() {
529 return result == Result.FLUSHED_COMPACTION_NEEDED;
530 }
531
532 @Override
533 public String toString() {
534 return new StringBuilder()
535 .append("flush result:").append(result).append(", ")
536 .append("failureReason:").append(failureReason).append(",")
537 .append("flush seq id").append(flushSequenceId).toString();
538 }
539
540 @Override
541 public Result getResult() {
542 return result;
543 }
544 }
545
546
547 static class PrepareFlushResult {
548 final FlushResult result;
549 final TreeMap<byte[], StoreFlushContext> storeFlushCtxs;
550 final TreeMap<byte[], List<Path>> committedFiles;
551 final TreeMap<byte[], Long> storeFlushableSize;
552 final long startTime;
553 final long flushOpSeqId;
554 final long flushedSeqId;
555 final long totalFlushableSize;
556
557
558 PrepareFlushResult(FlushResult result, long flushSeqId) {
559 this(result, null, null, null, Math.max(0, flushSeqId), 0, 0, 0);
560 }
561
562
563 PrepareFlushResult(
564 TreeMap<byte[], StoreFlushContext> storeFlushCtxs,
565 TreeMap<byte[], List<Path>> committedFiles,
566 TreeMap<byte[], Long> storeFlushableSize, long startTime, long flushSeqId,
567 long flushedSeqId, long totalFlushableSize) {
568 this(null, storeFlushCtxs, committedFiles, storeFlushableSize, startTime,
569 flushSeqId, flushedSeqId, totalFlushableSize);
570 }
571
572 private PrepareFlushResult(
573 FlushResult result,
574 TreeMap<byte[], StoreFlushContext> storeFlushCtxs,
575 TreeMap<byte[], List<Path>> committedFiles,
576 TreeMap<byte[], Long> storeFlushableSize, long startTime, long flushSeqId,
577 long flushedSeqId, long totalFlushableSize) {
578 this.result = result;
579 this.storeFlushCtxs = storeFlushCtxs;
580 this.committedFiles = committedFiles;
581 this.storeFlushableSize = storeFlushableSize;
582 this.startTime = startTime;
583 this.flushOpSeqId = flushSeqId;
584 this.flushedSeqId = flushedSeqId;
585 this.totalFlushableSize = totalFlushableSize;
586 }
587
588 public FlushResult getResult() {
589 return this.result;
590 }
591 }
592
593
594
595
596 static class ObservedExceptionsInBatch {
597 private boolean wrongRegion = false;
598 private boolean failedSanityCheck = false;
599 private boolean wrongFamily = false;
600
601
602
603
604 boolean hasSeenWrongRegion() {
605 return wrongRegion;
606 }
607
608
609
610
611 void sawWrongRegion() {
612 wrongRegion = true;
613 }
614
615
616
617
618 boolean hasSeenFailedSanityCheck() {
619 return failedSanityCheck;
620 }
621
622
623
624
625 void sawFailedSanityCheck() {
626 failedSanityCheck = true;
627 }
628
629
630
631
632 boolean hasSeenNoSuchFamily() {
633 return wrongFamily;
634 }
635
636
637
638
639 void sawNoSuchFamily() {
640 wrongFamily = true;
641 }
642 }
643
644 final WriteState writestate = new WriteState();
645
646 long memstoreFlushSize;
647 final long timestampSlop;
648 final long rowProcessorTimeout;
649
650
651 private final ConcurrentMap<Store, Long> lastStoreFlushTimeMap =
652 new ConcurrentHashMap<Store, Long>();
653
654 protected RegionServerServices rsServices;
655 private RegionServerAccounting rsAccounting;
656 private long flushCheckInterval;
657
658 private long flushPerChanges;
659 private long blockingMemStoreSize;
660 final long threadWakeFrequency;
661
662 final ReentrantReadWriteLock lock;
663
664
665
666 final ConcurrentHashMap<Thread, Boolean> regionLockHolders;
667
668
669 private final ReentrantReadWriteLock updatesLock =
670 new ReentrantReadWriteLock();
671 private boolean splitRequest;
672 private byte[] explicitSplitPoint = null;
673
674 private final MultiVersionConcurrencyControl mvcc;
675
676
677 private RegionCoprocessorHost coprocessorHost;
678
679 private HTableDescriptor htableDescriptor = null;
680 private RegionSplitPolicy splitPolicy;
681 private FlushPolicy flushPolicy;
682
683 private final MetricsRegion metricsRegion;
684 private final MetricsRegionWrapperImpl metricsRegionWrapper;
685 private final Durability durability;
686 private final boolean regionStatsEnabled;
687
688 private static final List<String> EMPTY_CLUSTERID_LIST = new ArrayList<String>();
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711 @Deprecated
712 public HRegion(final Path tableDir, final WAL wal, final FileSystem fs,
713 final Configuration confParam, final HRegionInfo regionInfo,
714 final HTableDescriptor htd, final RegionServerServices rsServices) {
715 this(new HRegionFileSystem(confParam, fs, tableDir, regionInfo),
716 wal, confParam, htd, rsServices);
717 }
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735 public HRegion(final HRegionFileSystem fs, final WAL wal, final Configuration confParam,
736 final HTableDescriptor htd, final RegionServerServices rsServices) {
737 if (htd == null) {
738 throw new IllegalArgumentException("Need table descriptor");
739 }
740
741 if (confParam instanceof CompoundConfiguration) {
742 throw new IllegalArgumentException("Need original base configuration");
743 }
744
745 this.comparator = fs.getRegionInfo().getComparator();
746 this.wal = wal;
747 this.fs = fs;
748 this.mvcc = new MultiVersionConcurrencyControl(getRegionInfo().getShortNameToLog());
749
750
751 this.baseConf = confParam;
752 this.conf = new CompoundConfiguration()
753 .add(confParam)
754 .addStringMap(htd.getConfiguration())
755 .addWritableMap(htd.getValues());
756 this.lock = new ReentrantReadWriteLock(conf.getBoolean(FAIR_REENTRANT_CLOSE_LOCK,
757 DEFAULT_FAIR_REENTRANT_CLOSE_LOCK));
758 this.regionLockHolders = new ConcurrentHashMap<Thread,Boolean>();
759 this.flushCheckInterval = conf.getInt(MEMSTORE_PERIODIC_FLUSH_INTERVAL,
760 DEFAULT_CACHE_FLUSH_INTERVAL);
761 this.flushPerChanges = conf.getLong(MEMSTORE_FLUSH_PER_CHANGES, DEFAULT_FLUSH_PER_CHANGES);
762 if (this.flushPerChanges > MAX_FLUSH_PER_CHANGES) {
763 throw new IllegalArgumentException(MEMSTORE_FLUSH_PER_CHANGES + " can not exceed "
764 + MAX_FLUSH_PER_CHANGES);
765 }
766 int tmpRowLockDuration = conf.getInt("hbase.rowlock.wait.duration",
767 DEFAULT_ROWLOCK_WAIT_DURATION);
768 if (tmpRowLockDuration <= 0) {
769 LOG.info("Found hbase.rowlock.wait.duration set to " + tmpRowLockDuration + ". values <= 0 " +
770 "will cause all row locking to fail. Treating it as 1ms to avoid region failure.");
771 tmpRowLockDuration = 1;
772 }
773 this.rowLockWaitDuration = tmpRowLockDuration;
774
775 this.maxWaitForSeqId = conf.getInt(MAX_WAIT_FOR_SEQ_ID_KEY, DEFAULT_MAX_WAIT_FOR_SEQ_ID);
776 this.isLoadingCfsOnDemandDefault = conf.getBoolean(LOAD_CFS_ON_DEMAND_CONFIG_KEY, true);
777 this.htableDescriptor = htd;
778 this.rsServices = rsServices;
779 this.threadWakeFrequency = conf.getLong(HConstants.THREAD_WAKE_FREQUENCY, 10 * 1000);
780 setHTableSpecificConf();
781 this.scannerReadPoints = new ConcurrentHashMap<RegionScanner, Long>();
782
783 this.busyWaitDuration = conf.getLong(
784 "hbase.busy.wait.duration", DEFAULT_BUSY_WAIT_DURATION);
785 this.maxBusyWaitMultiplier = conf.getInt("hbase.busy.wait.multiplier.max", 2);
786 if (busyWaitDuration * maxBusyWaitMultiplier <= 0L) {
787 throw new IllegalArgumentException("Invalid hbase.busy.wait.duration ("
788 + busyWaitDuration + ") or hbase.busy.wait.multiplier.max ("
789 + maxBusyWaitMultiplier + "). Their product should be positive");
790 }
791 this.maxBusyWaitDuration = conf.getLong("hbase.ipc.client.call.purge.timeout",
792 2 * HConstants.DEFAULT_HBASE_RPC_TIMEOUT);
793
794
795
796
797
798
799
800 this.timestampSlop = conf.getLong(
801 "hbase.hregion.keyvalue.timestamp.slop.millisecs",
802 HConstants.LATEST_TIMESTAMP);
803
804
805
806
807
808 this.rowProcessorTimeout = conf.getLong(
809 "hbase.hregion.row.processor.timeout", DEFAULT_ROW_PROCESSOR_TIMEOUT);
810
811 boolean forceSync = conf.getBoolean(WAL_HSYNC_CONF_KEY, DEFAULT_WAL_HSYNC);
812
813
814
815
816 Durability defaultDurability = forceSync ? Durability.FSYNC_WAL : Durability.SYNC_WAL;
817 this.durability =
818 htd.getDurability() == Durability.USE_DEFAULT ? defaultDurability : htd.getDurability();
819
820 if (rsServices != null) {
821 this.rsAccounting = this.rsServices.getRegionServerAccounting();
822
823
824 this.coprocessorHost = new RegionCoprocessorHost(this, rsServices, conf);
825 this.metricsRegionWrapper = new MetricsRegionWrapperImpl(this);
826 this.metricsRegion = new MetricsRegion(this.metricsRegionWrapper);
827
828 Map<String, Region> recoveringRegions = rsServices.getRecoveringRegions();
829 String encodedName = getRegionInfo().getEncodedName();
830 if (recoveringRegions != null && recoveringRegions.containsKey(encodedName)) {
831 this.recovering = true;
832 recoveringRegions.put(encodedName, this);
833 }
834 } else {
835 this.metricsRegionWrapper = null;
836 this.metricsRegion = null;
837 }
838 if (LOG.isDebugEnabled()) {
839
840 LOG.debug("Instantiated " + this);
841 }
842
843
844 this.disallowWritesInRecovering =
845 conf.getBoolean(HConstants.DISALLOW_WRITES_IN_RECOVERING,
846 HConstants.DEFAULT_DISALLOW_WRITES_IN_RECOVERING_CONFIG);
847 configurationManager = Optional.absent();
848
849
850 this.regionStatsEnabled = htd.getTableName().getNamespaceAsString().equals(
851 NamespaceDescriptor.SYSTEM_NAMESPACE_NAME_STR) ?
852 false :
853 conf.getBoolean(HConstants.ENABLE_CLIENT_BACKPRESSURE,
854 HConstants.DEFAULT_ENABLE_CLIENT_BACKPRESSURE);
855
856 this.maxCellSize = conf.getLong(HBASE_MAX_CELL_SIZE_KEY, DEFAULT_MAX_CELL_SIZE);
857 }
858
859 void setHTableSpecificConf() {
860 if (this.htableDescriptor == null) return;
861 long flushSize = this.htableDescriptor.getMemStoreFlushSize();
862
863 if (flushSize <= 0) {
864 flushSize = conf.getLong(HConstants.HREGION_MEMSTORE_FLUSH_SIZE,
865 HTableDescriptor.DEFAULT_MEMSTORE_FLUSH_SIZE);
866 }
867 this.memstoreFlushSize = flushSize;
868 this.blockingMemStoreSize = this.memstoreFlushSize *
869 conf.getLong(HConstants.HREGION_MEMSTORE_BLOCK_MULTIPLIER,
870 HConstants.DEFAULT_HREGION_MEMSTORE_BLOCK_MULTIPLIER);
871 }
872
873
874
875
876
877
878
879
880
881 @Deprecated
882 public long initialize() throws IOException {
883 return initialize(null);
884 }
885
886
887
888
889
890
891
892
893 private long initialize(final CancelableProgressable reporter) throws IOException {
894
895
896 if (htableDescriptor.getColumnFamilies().length == 0) {
897 throw new DoNotRetryIOException("Table " + htableDescriptor.getNameAsString() +
898 " should have at least one column family.");
899 }
900
901 MonitoredTask status = TaskMonitor.get().createStatus("Initializing region " + this);
902 status.enableStatusJournal(true);
903 long nextSeqId = -1;
904 try {
905 nextSeqId = initializeRegionInternals(reporter, status);
906 return nextSeqId;
907 } finally {
908
909
910 if (nextSeqId == -1) {
911 status.abort("Exception during region " + getRegionInfo().getRegionNameAsString() +
912 " initialization.");
913 }
914 if (LOG.isDebugEnabled()) {
915 LOG.debug("Region open journal:\n" + status.prettyPrintJournal());
916 }
917 status.cleanup();
918 }
919 }
920
921 private long initializeRegionInternals(final CancelableProgressable reporter,
922 final MonitoredTask status) throws IOException {
923 if (coprocessorHost != null) {
924 status.setStatus("Running coprocessor pre-open hook");
925 coprocessorHost.preOpen();
926 }
927
928
929
930 if (this.getRegionInfo().getReplicaId() == HRegionInfo.DEFAULT_REPLICA_ID) {
931 status.setStatus("Writing region info on filesystem");
932 fs.checkRegionInfoOnFilesystem();
933 } else {
934 if (LOG.isDebugEnabled()) {
935 LOG.debug("Skipping creation of .regioninfo file for " + this.getRegionInfo());
936 }
937 }
938
939
940 status.setStatus("Initializing all the Stores");
941 long maxSeqId = initializeStores(reporter, status);
942 this.mvcc.advanceTo(maxSeqId);
943 if (ServerRegionReplicaUtil.shouldReplayRecoveredEdits(this)) {
944
945 maxSeqId = Math.max(maxSeqId,
946 replayRecoveredEditsIfAny(maxSeqIdInStores, reporter, status));
947
948 this.mvcc.advanceTo(maxSeqId);
949 }
950 this.lastReplayedOpenRegionSeqId = maxSeqId;
951
952 this.writestate.setReadOnly(ServerRegionReplicaUtil.isReadOnly(this));
953 this.writestate.flushRequested = false;
954 this.writestate.compacting.set(0);
955
956 if (this.writestate.writesEnabled) {
957
958 status.setStatus("Cleaning up temporary data from old regions");
959 fs.cleanupTempDir();
960 }
961
962 if (this.writestate.writesEnabled) {
963 status.setStatus("Cleaning up detritus from prior splits");
964
965
966
967 fs.cleanupAnySplitDetritus();
968 fs.cleanupMergesDir();
969 }
970
971
972 this.splitPolicy = RegionSplitPolicy.create(this, conf);
973
974
975 this.flushPolicy = FlushPolicyFactory.create(this, conf);
976
977 long lastFlushTime = EnvironmentEdgeManager.currentTime();
978 for (Store store: stores.values()) {
979 this.lastStoreFlushTimeMap.put(store, lastFlushTime);
980 }
981
982
983
984 long nextSeqid = maxSeqId;
985
986
987
988
989 if (this.writestate.writesEnabled) {
990 nextSeqid = WALSplitter.writeRegionSequenceIdFile(getWalFileSystem(), getWALRegionDir(),
991 nextSeqid, (this.recovering ? (this.flushPerChanges + 10000000) : 1));
992 } else {
993 nextSeqid++;
994 }
995
996 LOG.info("Onlined " + this.getRegionInfo().getShortNameToLog() +
997 "; next sequenceid=" + nextSeqid);
998
999
1000 this.closing.set(false);
1001 this.closed.set(false);
1002
1003 if (coprocessorHost != null) {
1004 status.setStatus("Running coprocessor post-open hooks");
1005 coprocessorHost.postOpen();
1006 }
1007
1008 status.markComplete("Region opened successfully");
1009 return nextSeqid;
1010 }
1011
1012
1013
1014
1015
1016
1017
1018
1019 private long initializeStores(final CancelableProgressable reporter, MonitoredTask status)
1020 throws IOException {
1021
1022
1023 long maxSeqId = -1;
1024
1025 long maxMemstoreTS = -1;
1026
1027 if (!htableDescriptor.getFamilies().isEmpty()) {
1028
1029 ThreadPoolExecutor storeOpenerThreadPool =
1030 getStoreOpenAndCloseThreadPool("StoreOpener-" + this.getRegionInfo().getShortNameToLog());
1031 CompletionService<HStore> completionService =
1032 new ExecutorCompletionService<HStore>(storeOpenerThreadPool);
1033
1034
1035 for (final HColumnDescriptor family : htableDescriptor.getFamilies()) {
1036 status.setStatus("Instantiating store for column family " + family);
1037 completionService.submit(new Callable<HStore>() {
1038 @Override
1039 public HStore call() throws IOException {
1040 return instantiateHStore(family);
1041 }
1042 });
1043 }
1044 boolean allStoresOpened = false;
1045 try {
1046 for (int i = 0; i < htableDescriptor.getFamilies().size(); i++) {
1047 Future<HStore> future = completionService.take();
1048 HStore store = future.get();
1049 this.stores.put(store.getFamily().getName(), store);
1050
1051 long storeMaxSequenceId = store.getMaxSequenceId();
1052 maxSeqIdInStores.put(Bytes.toBytes(store.getColumnFamilyName()),
1053 storeMaxSequenceId);
1054 if (maxSeqId == -1 || storeMaxSequenceId > maxSeqId) {
1055 maxSeqId = storeMaxSequenceId;
1056 }
1057 long maxStoreMemstoreTS = store.getMaxMemstoreTS();
1058 if (maxStoreMemstoreTS > maxMemstoreTS) {
1059 maxMemstoreTS = maxStoreMemstoreTS;
1060 }
1061 }
1062 allStoresOpened = true;
1063 } catch (InterruptedException e) {
1064 throw throwOnInterrupt(e);
1065 } catch (ExecutionException e) {
1066 throw new IOException(e.getCause());
1067 } finally {
1068 storeOpenerThreadPool.shutdownNow();
1069 if (!allStoresOpened) {
1070
1071 LOG.error("Could not initialize all stores for the region=" + this);
1072 for (Store store : this.stores.values()) {
1073 try {
1074 store.close();
1075 } catch (IOException e) {
1076 LOG.warn(e.getMessage());
1077 }
1078 }
1079 }
1080 }
1081 }
1082 return Math.max(maxSeqId, maxMemstoreTS + 1);
1083 }
1084
1085 private void initializeWarmup(final CancelableProgressable reporter) throws IOException {
1086 MonitoredTask status = TaskMonitor.get().createStatus("Initializing region " + this);
1087
1088 status.setStatus("Warming up all the Stores");
1089 try {
1090 initializeStores(reporter, status);
1091 } finally {
1092 status.markComplete("Done warming up.");
1093 }
1094 }
1095
1096
1097
1098
1099 private NavigableMap<byte[], List<Path>> getStoreFiles() {
1100 NavigableMap<byte[], List<Path>> allStoreFiles =
1101 new TreeMap<byte[], List<Path>>(Bytes.BYTES_COMPARATOR);
1102 for (Store store: getStores()) {
1103 Collection<StoreFile> storeFiles = store.getStorefiles();
1104 if (storeFiles == null) continue;
1105 List<Path> storeFileNames = new ArrayList<Path>();
1106 for (StoreFile storeFile: storeFiles) {
1107 storeFileNames.add(storeFile.getPath());
1108 }
1109 allStoreFiles.put(store.getFamily().getName(), storeFileNames);
1110 }
1111 return allStoreFiles;
1112 }
1113
1114 private void writeRegionOpenMarker(WAL wal, long openSeqId) throws IOException {
1115 Map<byte[], List<Path>> storeFiles = getStoreFiles();
1116 RegionEventDescriptor regionOpenDesc = ProtobufUtil.toRegionEventDescriptor(
1117 RegionEventDescriptor.EventType.REGION_OPEN, getRegionInfo(), openSeqId,
1118 getRegionServerServices().getServerName(), storeFiles);
1119 WALUtil.writeRegionEventMarker(wal, getTableDesc(), getRegionInfo(), regionOpenDesc, mvcc);
1120 }
1121
1122 private void writeRegionCloseMarker(WAL wal) throws IOException {
1123 Map<byte[], List<Path>> storeFiles = getStoreFiles();
1124 RegionEventDescriptor regionEventDesc = ProtobufUtil.toRegionEventDescriptor(
1125 RegionEventDescriptor.EventType.REGION_CLOSE, getRegionInfo(), mvcc.getReadPoint(),
1126 getRegionServerServices().getServerName(), storeFiles);
1127 WALUtil.writeRegionEventMarker(wal, getTableDesc(), getRegionInfo(), regionEventDesc, mvcc);
1128
1129
1130
1131
1132 if (getWalFileSystem().exists(getWALRegionDir())) {
1133 WALSplitter.writeRegionSequenceIdFile(getWalFileSystem(), getWALRegionDir(),
1134 mvcc.getReadPoint(), 0);
1135 }
1136 }
1137
1138
1139
1140
1141 public boolean hasReferences() {
1142 for (Store store : this.stores.values()) {
1143 if (store.hasReferences()) return true;
1144 }
1145 return false;
1146 }
1147
1148 @Override
1149 public HDFSBlocksDistribution getHDFSBlocksDistribution() {
1150 HDFSBlocksDistribution hdfsBlocksDistribution =
1151 new HDFSBlocksDistribution();
1152 synchronized (this.stores) {
1153 for (Store store : this.stores.values()) {
1154 Collection<StoreFile> storeFiles = store.getStorefiles();
1155 if (storeFiles == null) continue;
1156 for (StoreFile sf : storeFiles) {
1157 HDFSBlocksDistribution storeFileBlocksDistribution =
1158 sf.getHDFSBlockDistribution();
1159 hdfsBlocksDistribution.add(storeFileBlocksDistribution);
1160 }
1161 }
1162 }
1163 return hdfsBlocksDistribution;
1164 }
1165
1166
1167
1168
1169
1170
1171
1172
1173
1174 public static HDFSBlocksDistribution computeHDFSBlocksDistribution(final Configuration conf,
1175 final HTableDescriptor tableDescriptor, final HRegionInfo regionInfo) throws IOException {
1176 Path tablePath = FSUtils.getTableDir(FSUtils.getRootDir(conf), tableDescriptor.getTableName());
1177 return computeHDFSBlocksDistribution(conf, tableDescriptor, regionInfo, tablePath);
1178 }
1179
1180
1181
1182
1183
1184
1185
1186
1187
1188
1189 public static HDFSBlocksDistribution computeHDFSBlocksDistribution(final Configuration conf,
1190 final HTableDescriptor tableDescriptor, final HRegionInfo regionInfo, Path tablePath)
1191 throws IOException {
1192 HDFSBlocksDistribution hdfsBlocksDistribution = new HDFSBlocksDistribution();
1193 FileSystem fs = tablePath.getFileSystem(conf);
1194
1195 HRegionFileSystem regionFs = new HRegionFileSystem(conf, fs, tablePath, regionInfo);
1196 for (HColumnDescriptor family : tableDescriptor.getFamilies()) {
1197 List<LocatedFileStatus> locatedFileStatusList = HRegionFileSystem
1198 .getStoreFilesLocatedStatus(regionFs, family.getNameAsString(), true);
1199 if (locatedFileStatusList == null) {
1200 continue;
1201 }
1202
1203 for (LocatedFileStatus status : locatedFileStatusList) {
1204 Path p = status.getPath();
1205 if (StoreFileInfo.isReference(p) || HFileLink.isHFileLink(p)) {
1206
1207
1208 StoreFileInfo storeFileInfo = new StoreFileInfo(conf, fs, status);
1209 hdfsBlocksDistribution.add(storeFileInfo
1210 .computeHDFSBlocksDistribution(fs));
1211 } else if (StoreFileInfo.isHFile(p)) {
1212
1213
1214 FSUtils.addToHDFSBlocksDistribution(hdfsBlocksDistribution,
1215 status.getBlockLocations());
1216 } else {
1217 throw new IOException("path=" + p
1218 + " doesn't look like a valid StoreFile");
1219 }
1220 }
1221 }
1222 return hdfsBlocksDistribution;
1223 }
1224
1225
1226
1227
1228
1229
1230 public long addAndGetGlobalMemstoreSize(long memStoreSize) {
1231 if (this.rsAccounting != null) {
1232 rsAccounting.addAndGetGlobalMemstoreSize(memStoreSize);
1233 }
1234 long size = this.memstoreSize.addAndGet(memStoreSize);
1235
1236
1237 if (size < 0) {
1238 LOG.error("Asked to modify this region's (" + this.toString()
1239 + ") memstoreSize to a negative value which is incorrect. Current memstoreSize="
1240 + (size-memStoreSize) + ", delta=" + memStoreSize, new Exception());
1241 }
1242 return size;
1243 }
1244
1245 @Override
1246 public HRegionInfo getRegionInfo() {
1247 return this.fs.getRegionInfo();
1248 }
1249
1250
1251
1252
1253
1254 RegionServerServices getRegionServerServices() {
1255 return this.rsServices;
1256 }
1257
1258 @Override
1259 public long getReadRequestsCount() {
1260 return readRequestsCount.get();
1261 }
1262
1263 @Override
1264 public void updateReadRequestsCount(long i) {
1265 readRequestsCount.add(i);
1266 }
1267
1268 @Override
1269 public long getWriteRequestsCount() {
1270 return writeRequestsCount.get();
1271 }
1272
1273 @Override
1274 public void updateWriteRequestsCount(long i) {
1275 writeRequestsCount.add(i);
1276 }
1277
1278 @Override
1279 public long getMemstoreSize() {
1280 return memstoreSize.get();
1281 }
1282
1283 @Override
1284 public long getNumMutationsWithoutWAL() {
1285 return numMutationsWithoutWAL.get();
1286 }
1287
1288 @Override
1289 public long getDataInMemoryWithoutWAL() {
1290 return dataInMemoryWithoutWAL.get();
1291 }
1292
1293 @Override
1294 public long getBlockedRequestsCount() {
1295 return blockedRequestsCount.get();
1296 }
1297
1298 @Override
1299 public long getCheckAndMutateChecksPassed() {
1300 return checkAndMutateChecksPassed.get();
1301 }
1302
1303 @Override
1304 public long getCheckAndMutateChecksFailed() {
1305 return checkAndMutateChecksFailed.get();
1306 }
1307
1308 @Override
1309 public MetricsRegion getMetrics() {
1310 return metricsRegion;
1311 }
1312
1313 @Override
1314 public boolean isClosed() {
1315 return this.closed.get();
1316 }
1317
1318 @Override
1319 public boolean isClosing() {
1320 return this.closing.get();
1321 }
1322
1323 @Override
1324 public boolean isReadOnly() {
1325 return this.writestate.isReadOnly();
1326 }
1327
1328
1329
1330
1331 public void setRecovering(boolean newState) {
1332 boolean wasRecovering = this.recovering;
1333
1334
1335 if (wal != null && getRegionServerServices() != null && !writestate.readOnly
1336 && wasRecovering && !newState) {
1337
1338
1339 boolean forceFlush = getTableDesc().getRegionReplication() > 1;
1340
1341 MonitoredTask status = TaskMonitor.get().createStatus("Recovering region " + this);
1342
1343 try {
1344
1345 if (forceFlush) {
1346 status.setStatus("Flushing region " + this + " because recovery is finished");
1347 internalFlushcache(status);
1348 }
1349
1350 status.setStatus("Writing region open event marker to WAL because recovery is finished");
1351 try {
1352 long seqId = openSeqNum;
1353
1354 if (wal != null) {
1355 seqId = getNextSequenceId(wal);
1356 }
1357 writeRegionOpenMarker(wal, seqId);
1358 } catch (IOException e) {
1359
1360
1361 LOG.warn(getRegionInfo().getEncodedName() + " : was not able to write region opening "
1362 + "event to WAL, continuing", e);
1363 }
1364 } catch (IOException ioe) {
1365
1366
1367 LOG.warn(getRegionInfo().getEncodedName() + " : was not able to flush "
1368 + "event to WAL, continuing", ioe);
1369 } finally {
1370 status.cleanup();
1371 }
1372 }
1373
1374 this.recovering = newState;
1375 if (wasRecovering && !recovering) {
1376
1377 coprocessorHost.postLogReplay();
1378 }
1379 }
1380
1381 @Override
1382 public boolean isRecovering() {
1383 return this.recovering;
1384 }
1385
1386 @Override
1387 public boolean isAvailable() {
1388 return !isClosed() && !isClosing();
1389 }
1390
1391
1392 public boolean isSplittable() {
1393 return isAvailable() && !hasReferences();
1394 }
1395
1396
1397
1398
1399 public boolean isMergeable() {
1400 if (!isAvailable()) {
1401 LOG.debug("Region " + this
1402 + " is not mergeable because it is closing or closed");
1403 return false;
1404 }
1405 if (hasReferences()) {
1406 LOG.debug("Region " + this
1407 + " is not mergeable because it has references");
1408 return false;
1409 }
1410
1411 return true;
1412 }
1413
1414 public boolean areWritesEnabled() {
1415 synchronized(this.writestate) {
1416 return this.writestate.writesEnabled;
1417 }
1418 }
1419
1420 public MultiVersionConcurrencyControl getMVCC() {
1421 return mvcc;
1422 }
1423
1424 @Override
1425 public long getMaxFlushedSeqId() {
1426 return maxFlushedSeqId;
1427 }
1428
1429 @Override
1430 public long getReadpoint(IsolationLevel isolationLevel) {
1431 if (isolationLevel == IsolationLevel.READ_UNCOMMITTED) {
1432
1433 return Long.MAX_VALUE;
1434 }
1435 return mvcc.getReadPoint();
1436 }
1437
1438 @Override
1439 public boolean isLoadingCfsOnDemandDefault() {
1440 return this.isLoadingCfsOnDemandDefault;
1441 }
1442
1443
1444
1445
1446
1447
1448
1449
1450
1451
1452
1453
1454
1455
1456
1457
1458
1459 public Map<byte[], List<StoreFile>> close() throws IOException {
1460 return close(false);
1461 }
1462
1463 private final Object closeLock = new Object();
1464
1465
1466 public static final String FAIR_REENTRANT_CLOSE_LOCK =
1467 "hbase.regionserver.fair.region.close.lock";
1468 public static final boolean DEFAULT_FAIR_REENTRANT_CLOSE_LOCK = true;
1469
1470 public static final String MEMSTORE_PERIODIC_FLUSH_INTERVAL =
1471 "hbase.regionserver.optionalcacheflushinterval";
1472
1473 public static final int DEFAULT_CACHE_FLUSH_INTERVAL = 3600000;
1474
1475 public static final int SYSTEM_CACHE_FLUSH_INTERVAL = 300000;
1476
1477
1478 public static final String MEMSTORE_FLUSH_PER_CHANGES =
1479 "hbase.regionserver.flush.per.changes";
1480 public static final long DEFAULT_FLUSH_PER_CHANGES = 30000000;
1481
1482
1483
1484
1485 public static final long MAX_FLUSH_PER_CHANGES = 1000000000;
1486
1487 public static final String CLOSE_WAIT_ABORT = "hbase.regionserver.close.wait.abort";
1488 public static final boolean DEFAULT_CLOSE_WAIT_ABORT = false;
1489 public static final String CLOSE_WAIT_TIME = "hbase.regionserver.close.wait.time.ms";
1490 public static final long DEFAULT_CLOSE_WAIT_TIME = 60000;
1491 public static final String CLOSE_WAIT_INTERVAL = "hbase.regionserver.close.wait.interval.ms";
1492 public static final long DEFAULT_CLOSE_WAIT_INTERVAL = 10000;
1493
1494
1495
1496
1497
1498
1499
1500
1501
1502
1503
1504
1505
1506
1507
1508
1509
1510
1511 public Map<byte[], List<StoreFile>> close(final boolean abort) throws IOException {
1512
1513
1514 MonitoredTask status = TaskMonitor.get().createStatus(
1515 "Closing region " + this +
1516 (abort ? " due to abort" : ""));
1517 status.enableStatusJournal(true);
1518 status.setStatus("Waiting for close lock");
1519 try {
1520 synchronized (closeLock) {
1521 return doClose(abort, status);
1522 }
1523 } finally {
1524 if (LOG.isDebugEnabled()) {
1525 LOG.debug("Region close journal:\n" + status.prettyPrintJournal());
1526 }
1527 status.cleanup();
1528 }
1529 }
1530
1531
1532
1533
1534 public void setClosing(boolean closing) {
1535 this.closing.set(closing);
1536 }
1537
1538 @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="UL_UNRELEASED_LOCK_EXCEPTION_PATH",
1539 justification="I think FindBugs is confused")
1540 private Map<byte[], List<StoreFile>> doClose(final boolean abort, MonitoredTask status)
1541 throws IOException {
1542 if (isClosed()) {
1543 LOG.warn("Region " + this + " already closed");
1544 return null;
1545 }
1546
1547 if (coprocessorHost != null) {
1548 status.setStatus("Running coprocessor pre-close hooks");
1549 this.coprocessorHost.preClose(abort);
1550 }
1551 status.setStatus("Disabling compacts and flushes for region");
1552 boolean canFlush = true;
1553 synchronized (writestate) {
1554
1555
1556 canFlush = !writestate.readOnly;
1557 writestate.writesEnabled = false;
1558 LOG.debug("Closing " + this + ": disabling compactions & flushes");
1559 waitForFlushesAndCompactions();
1560 }
1561
1562
1563
1564 if (!abort && worthPreFlushing() && canFlush) {
1565 status.setStatus("Pre-flushing region before close");
1566 LOG.info("Running close preflush of " + this);
1567 try {
1568 internalFlushcache(status);
1569 } catch (IOException ioe) {
1570
1571 status.setStatus("Failed pre-flush " + this + "; " + ioe.getMessage());
1572 }
1573 }
1574
1575
1576
1577
1578 this.closing.set(true);
1579
1580
1581
1582
1583
1584
1585
1586
1587
1588 final boolean canAbort = conf.getBoolean(CLOSE_WAIT_ABORT, DEFAULT_CLOSE_WAIT_ABORT);
1589 final boolean useTimedWait = canAbort;
1590 if (LOG.isDebugEnabled()) {
1591 LOG.debug((useTimedWait ? "Time limited wait" : "Waiting without time limit") +
1592 " for close lock on " + this);
1593 }
1594 final long timeoutForWriteLock = conf.getLong(CLOSE_WAIT_TIME, DEFAULT_CLOSE_WAIT_TIME);
1595 final long closeWaitInterval = conf.getLong(CLOSE_WAIT_INTERVAL, DEFAULT_CLOSE_WAIT_INTERVAL);
1596 long elapsedWaitTime = 0;
1597 if (useTimedWait) {
1598
1599 long remainingWaitTime = timeoutForWriteLock;
1600 if (remainingWaitTime < closeWaitInterval) {
1601 LOG.warn("Time limit for close wait of " + timeoutForWriteLock +
1602 " ms is less than the configured lock acquisition wait interval " +
1603 closeWaitInterval + " ms, using wait interval as time limit");
1604 remainingWaitTime = closeWaitInterval;
1605 }
1606 boolean acquired = false;
1607 do {
1608 long start = EnvironmentEdgeManager.currentTime();
1609 try {
1610 acquired = lock.writeLock().tryLock(Math.min(remainingWaitTime, closeWaitInterval),
1611 TimeUnit.MILLISECONDS);
1612 } catch (InterruptedException e) {
1613
1614
1615
1616
1617 String msg = "Interrupted while waiting for close lock on " + this;
1618 LOG.warn(msg, e);
1619 throw (InterruptedIOException) new InterruptedIOException(msg).initCause(e);
1620 }
1621 long elapsed = EnvironmentEdgeManager.currentTime() - start;
1622 elapsedWaitTime += elapsed;
1623 remainingWaitTime -= elapsed;
1624 if (canAbort && !acquired && remainingWaitTime > 0) {
1625
1626
1627
1628
1629 if (LOG.isDebugEnabled()) {
1630 LOG.debug("Interrupting eligible region operations after waiting to close for " +
1631 elapsedWaitTime + " ms on " + this + ", " + remainingWaitTime +
1632 " ms remaining");
1633 }
1634 interruptRegionOperations();
1635 }
1636 } while (!acquired && remainingWaitTime > 0);
1637
1638
1639
1640 if (!acquired) {
1641 String msg = "Failed to acquire close lock on " + this + " after waiting " +
1642 elapsedWaitTime + " ms";
1643 LOG.fatal(msg);
1644 rsServices.abort(msg, null);
1645 throw new IOException(msg);
1646 }
1647
1648 } else {
1649
1650
1651
1652
1653 long start = EnvironmentEdgeManager.currentTime();
1654 lock.writeLock().lock();
1655 elapsedWaitTime = EnvironmentEdgeManager.currentTime() - start;
1656
1657 }
1658
1659 if (LOG.isDebugEnabled()) {
1660 LOG.debug("Acquired close lock on " + this + " after waiting " +
1661 elapsedWaitTime + " ms");
1662 }
1663
1664 status.setStatus("Disabling writes for close");
1665 try {
1666 if (this.isClosed()) {
1667 status.abort("Already got closed by another process");
1668
1669 return null;
1670 }
1671 LOG.debug("Updates disabled for region " + this);
1672
1673 if (!abort && canFlush) {
1674 int flushCount = 0;
1675 while (this.memstoreSize.get() > 0) {
1676 try {
1677 if (flushCount++ > 0) {
1678 int actualFlushes = flushCount - 1;
1679 if (actualFlushes > 5) {
1680
1681
1682 throw new DroppedSnapshotException("Failed clearing memory after " +
1683 actualFlushes + " attempts on region: " +
1684 Bytes.toStringBinary(getRegionInfo().getRegionName()));
1685 }
1686 LOG.info("Running extra flush, " + actualFlushes +
1687 " (carrying snapshot?) " + this);
1688 }
1689 internalFlushcache(status);
1690 } catch (IOException ioe) {
1691 status.setStatus("Failed flush " + this + ", putting online again");
1692 synchronized (writestate) {
1693 writestate.writesEnabled = true;
1694 }
1695
1696 throw ioe;
1697 }
1698 }
1699 }
1700
1701 Map<byte[], List<StoreFile>> result =
1702 new TreeMap<byte[], List<StoreFile>>(Bytes.BYTES_COMPARATOR);
1703 if (!stores.isEmpty()) {
1704
1705 ThreadPoolExecutor storeCloserThreadPool =
1706 getStoreOpenAndCloseThreadPool("StoreCloserThread-" +
1707 getRegionInfo().getRegionNameAsString());
1708 CompletionService<Pair<byte[], Collection<StoreFile>>> completionService =
1709 new ExecutorCompletionService<Pair<byte[], Collection<StoreFile>>>(storeCloserThreadPool);
1710
1711
1712 for (final Store store : stores.values()) {
1713 long flushableSize = store.getFlushableSize();
1714 if (!(abort || flushableSize == 0 || writestate.readOnly)) {
1715 if (getRegionServerServices() != null) {
1716 getRegionServerServices().abort("Assertion failed while closing store "
1717 + getRegionInfo().getRegionNameAsString() + " " + store
1718 + ". flushableSize expected=0, actual= " + flushableSize
1719 + ". Current memstoreSize=" + getMemstoreSize() + ". Maybe a coprocessor "
1720 + "operation failed and left the memstore in a partially updated state.", null);
1721 }
1722 }
1723 completionService
1724 .submit(new Callable<Pair<byte[], Collection<StoreFile>>>() {
1725 @Override
1726 public Pair<byte[], Collection<StoreFile>> call() throws IOException {
1727 return new Pair<byte[], Collection<StoreFile>>(
1728 store.getFamily().getName(), store.close());
1729 }
1730 });
1731 }
1732 try {
1733 for (int i = 0; i < stores.size(); i++) {
1734 Future<Pair<byte[], Collection<StoreFile>>> future = completionService.take();
1735 Pair<byte[], Collection<StoreFile>> storeFiles = future.get();
1736 List<StoreFile> familyFiles = result.get(storeFiles.getFirst());
1737 if (familyFiles == null) {
1738 familyFiles = new ArrayList<StoreFile>();
1739 result.put(storeFiles.getFirst(), familyFiles);
1740 }
1741 familyFiles.addAll(storeFiles.getSecond());
1742 }
1743 } catch (InterruptedException e) {
1744 throw throwOnInterrupt(e);
1745 } catch (ExecutionException e) {
1746 Throwable cause = e.getCause();
1747 if (cause instanceof IOException) {
1748 throw (IOException) cause;
1749 }
1750 throw new IOException(cause);
1751 } finally {
1752 storeCloserThreadPool.shutdownNow();
1753 }
1754 }
1755
1756 status.setStatus("Writing region close event to WAL");
1757 if (!abort && wal != null && getRegionServerServices() != null && !writestate.readOnly) {
1758 writeRegionCloseMarker(wal);
1759 }
1760
1761 this.closed.set(true);
1762 if (!canFlush) {
1763 addAndGetGlobalMemstoreSize(-memstoreSize.get());
1764 } else if (memstoreSize.get() != 0) {
1765 LOG.error("Memstore size is " + memstoreSize.get());
1766 }
1767 if (coprocessorHost != null) {
1768 status.setStatus("Running coprocessor post-close hooks");
1769 this.coprocessorHost.postClose(abort);
1770 }
1771 if (this.metricsRegion != null) {
1772 this.metricsRegion.close();
1773 }
1774 if (this.metricsRegionWrapper != null) {
1775 Closeables.closeQuietly(this.metricsRegionWrapper);
1776 }
1777 status.markComplete("Closed");
1778 LOG.info("Closed " + this);
1779 return result;
1780 } finally {
1781 lock.writeLock().unlock();
1782 }
1783 }
1784
1785 @Override
1786 public void waitForFlushesAndCompactions() {
1787 synchronized (writestate) {
1788 if (this.writestate.readOnly) {
1789
1790
1791 return;
1792 }
1793 boolean interrupted = false;
1794 try {
1795 while (writestate.compacting.get() > 0 || writestate.flushing) {
1796 LOG.debug("waiting for " + writestate.compacting + " compactions"
1797 + (writestate.flushing ? " & cache flush" : "") + " to complete for region " + this);
1798 try {
1799 writestate.wait();
1800 } catch (InterruptedException iex) {
1801
1802 LOG.warn("Interrupted while waiting");
1803 interrupted = true;
1804 }
1805 }
1806 } finally {
1807 if (interrupted) {
1808 Thread.currentThread().interrupt();
1809 }
1810 }
1811 }
1812 }
1813
1814 @Override
1815 public void waitForFlushes() {
1816 synchronized (writestate) {
1817 if (this.writestate.readOnly) {
1818
1819
1820 return;
1821 }
1822 if (!writestate.flushing) return;
1823 long start = System.currentTimeMillis();
1824 boolean interrupted = false;
1825 try {
1826 while (writestate.flushing) {
1827 LOG.debug("waiting for cache flush to complete for region " + this);
1828 try {
1829 writestate.wait();
1830 } catch (InterruptedException iex) {
1831
1832 LOG.warn("Interrupted while waiting");
1833 interrupted = true;
1834 break;
1835 }
1836 }
1837 } finally {
1838 if (interrupted) {
1839 Thread.currentThread().interrupt();
1840 }
1841 }
1842 long duration = System.currentTimeMillis() - start;
1843 LOG.debug("Waited " + duration + " ms for flush to complete");
1844 }
1845 }
1846 protected ThreadPoolExecutor getStoreOpenAndCloseThreadPool(
1847 final String threadNamePrefix) {
1848 int numStores = Math.max(1, this.htableDescriptor.getFamilies().size());
1849 int maxThreads = Math.min(numStores,
1850 conf.getInt(HConstants.HSTORE_OPEN_AND_CLOSE_THREADS_MAX,
1851 HConstants.DEFAULT_HSTORE_OPEN_AND_CLOSE_THREADS_MAX));
1852 return getOpenAndCloseThreadPool(maxThreads, threadNamePrefix);
1853 }
1854
1855 protected ThreadPoolExecutor getStoreFileOpenAndCloseThreadPool(
1856 final String threadNamePrefix) {
1857 int numStores = Math.max(1, this.htableDescriptor.getFamilies().size());
1858 int maxThreads = Math.max(1,
1859 conf.getInt(HConstants.HSTORE_OPEN_AND_CLOSE_THREADS_MAX,
1860 HConstants.DEFAULT_HSTORE_OPEN_AND_CLOSE_THREADS_MAX)
1861 / numStores);
1862 return getOpenAndCloseThreadPool(maxThreads, threadNamePrefix);
1863 }
1864
1865 static ThreadPoolExecutor getOpenAndCloseThreadPool(int maxThreads,
1866 final String threadNamePrefix) {
1867 return Threads.getBoundedCachedThreadPool(maxThreads, 30L, TimeUnit.SECONDS,
1868 new ThreadFactory() {
1869 private int count = 1;
1870
1871 @Override
1872 public Thread newThread(Runnable r) {
1873 return new Thread(r, threadNamePrefix + "-" + count++);
1874 }
1875 });
1876 }
1877
1878
1879
1880
1881 private boolean worthPreFlushing() {
1882 return this.memstoreSize.get() >
1883 this.conf.getLong("hbase.hregion.preclose.flush.size", 1024 * 1024 * 5);
1884 }
1885
1886
1887
1888
1889
1890 @Override
1891 public HTableDescriptor getTableDesc() {
1892 return this.htableDescriptor;
1893 }
1894
1895
1896 public WAL getWAL() {
1897 return this.wal;
1898 }
1899
1900
1901
1902
1903
1904
1905
1906
1907 Configuration getBaseConf() {
1908 return this.baseConf;
1909 }
1910
1911
1912 public FileSystem getFilesystem() {
1913 return fs.getFileSystem();
1914 }
1915
1916
1917 public HRegionFileSystem getRegionFileSystem() {
1918 return this.fs;
1919 }
1920
1921
1922 HRegionFileSystem getRegionWALFileSystem() throws IOException {
1923 return new HRegionFileSystem(conf, getWalFileSystem(),
1924 FSUtils.getWALTableDir(conf, htableDescriptor.getTableName()), fs.getRegionInfo());
1925 }
1926
1927
1928 FileSystem getWalFileSystem() throws IOException {
1929 if (walFS == null) {
1930 walFS = FSUtils.getWALFileSystem(conf);
1931 }
1932 return walFS;
1933 }
1934
1935
1936
1937
1938
1939 public Path getWALRegionDir() throws IOException {
1940 if (regionDir == null) {
1941 regionDir = FSUtils.getWALRegionDir(conf, fs.getRegionInfo());
1942 }
1943 return regionDir;
1944 }
1945
1946 @Override
1947 public long getEarliestFlushTimeForAllStores() {
1948 return Collections.min(lastStoreFlushTimeMap.values());
1949 }
1950
1951 @Override
1952 public long getOldestHfileTs(boolean majorCompactionOnly) throws IOException {
1953 long result = Long.MAX_VALUE;
1954 for (Store store : getStores()) {
1955 Collection<StoreFile> storeFiles = store.getStorefiles();
1956 if (storeFiles == null) continue;
1957 for (StoreFile file : storeFiles) {
1958 StoreFile.Reader sfReader = file.getReader();
1959 if (sfReader == null) continue;
1960 HFile.Reader reader = sfReader.getHFileReader();
1961 if (reader == null) continue;
1962 if (majorCompactionOnly) {
1963 byte[] val = reader.loadFileInfo().get(StoreFile.MAJOR_COMPACTION_KEY);
1964 if (val == null || !Bytes.toBoolean(val)) continue;
1965 }
1966 result = Math.min(result, reader.getFileContext().getFileCreateTime());
1967 }
1968 }
1969 return result == Long.MAX_VALUE ? 0 : result;
1970 }
1971
1972 RegionLoad.Builder setCompleteSequenceId(RegionLoad.Builder regionLoadBldr) {
1973 long lastFlushOpSeqIdLocal = this.lastFlushOpSeqId;
1974 byte[] encodedRegionName = this.getRegionInfo().getEncodedNameAsBytes();
1975 regionLoadBldr.clearStoreCompleteSequenceId();
1976 for (byte[] familyName : this.stores.keySet()) {
1977 long earliest = this.wal.getEarliestMemstoreSeqNum(encodedRegionName, familyName);
1978
1979
1980
1981 long csid = (earliest == HConstants.NO_SEQNUM)? lastFlushOpSeqIdLocal: earliest - 1;
1982 regionLoadBldr.addStoreCompleteSequenceId(StoreSequenceId.
1983 newBuilder().setFamilyName(ByteString.copyFrom(familyName)).setSequenceId(csid).build());
1984 }
1985 return regionLoadBldr.setCompleteSequenceId(getMaxFlushedSeqId());
1986 }
1987
1988
1989
1990
1991
1992
1993
1994
1995
1996 public long getLargestHStoreSize() {
1997 long size = 0;
1998 for (Store h : stores.values()) {
1999 long storeSize = h.getSize();
2000 if (storeSize > size) {
2001 size = storeSize;
2002 }
2003 }
2004 return size;
2005 }
2006
2007
2008
2009
2010 public KeyValue.KVComparator getComparator() {
2011 return this.comparator;
2012 }
2013
2014
2015
2016
2017
2018 protected void doRegionCompactionPrep() throws IOException {
2019 }
2020
2021 @Override
2022 public void triggerMajorCompaction() throws IOException {
2023 for (Store s : getStores()) {
2024 s.triggerMajorCompaction();
2025 }
2026 }
2027
2028 @Override
2029 public void compact(final boolean majorCompaction) throws IOException {
2030 if (majorCompaction) {
2031 triggerMajorCompaction();
2032 }
2033 for (Store s : getStores()) {
2034 CompactionContext compaction = s.requestCompaction();
2035 if (compaction != null) {
2036 ThroughputController controller = null;
2037 if (rsServices != null) {
2038 controller = CompactionThroughputControllerFactory.create(rsServices, conf);
2039 }
2040 if (controller == null) {
2041 controller = NoLimitThroughputController.INSTANCE;
2042 }
2043 compact(compaction, s, controller, null);
2044 }
2045 }
2046 }
2047
2048
2049
2050
2051
2052
2053
2054 public void compactStores() throws IOException {
2055 for (Store s : getStores()) {
2056 CompactionContext compaction = s.requestCompaction();
2057 if (compaction != null) {
2058 compact(compaction, s, NoLimitThroughputController.INSTANCE, null);
2059 }
2060 }
2061 }
2062
2063
2064
2065
2066
2067
2068
2069 void compactStore(byte[] family, ThroughputController throughputController)
2070 throws IOException {
2071 Store s = getStore(family);
2072 CompactionContext compaction = s.requestCompaction();
2073 if (compaction != null) {
2074 compact(compaction, s, throughputController, null);
2075 }
2076 }
2077
2078
2079
2080
2081
2082
2083
2084
2085
2086
2087
2088
2089
2090
2091
2092
2093 public boolean compact(CompactionContext compaction, Store store,
2094 ThroughputController throughputController) throws IOException {
2095 return compact(compaction, store, throughputController, null);
2096 }
2097
2098 public boolean compact(CompactionContext compaction, Store store,
2099 ThroughputController throughputController, User user) throws IOException {
2100 assert compaction != null && compaction.hasSelection();
2101 assert !compaction.getRequest().getFiles().isEmpty();
2102 if (this.closing.get() || this.closed.get()) {
2103 LOG.debug("Skipping compaction on " + this + " because closing/closed");
2104 store.cancelRequestedCompaction(compaction);
2105 return false;
2106 }
2107 MonitoredTask status = null;
2108 boolean requestNeedsCancellation = true;
2109
2110
2111
2112
2113
2114
2115
2116
2117
2118
2119
2120
2121
2122
2123
2124
2125
2126
2127
2128
2129
2130
2131
2132
2133
2134
2135
2136
2137
2138
2139
2140
2141
2142
2143
2144
2145
2146
2147
2148
2149
2150
2151
2152
2153
2154
2155
2156
2157
2158
2159
2160
2161
2162
2163
2164
2165
2166
2167
2168
2169
2170
2171
2172
2173
2174
2175
2176
2177
2178
2179
2180
2181
2182
2183 try {
2184 byte[] cf = Bytes.toBytes(store.getColumnFamilyName());
2185 if (stores.get(cf) != store) {
2186 LOG.warn("Store " + store.getColumnFamilyName() + " on region " + this
2187 + " has been re-instantiated, cancel this compaction request. "
2188 + " It may be caused by the roll back of split transaction");
2189 return false;
2190 }
2191
2192 status = TaskMonitor.get().createStatus("Compacting " + store + " in " + this);
2193 status.enableStatusJournal(false);
2194 if (this.closed.get()) {
2195 String msg = "Skipping compaction on " + this + " because closed";
2196 LOG.debug(msg);
2197 status.abort(msg);
2198 return false;
2199 }
2200 boolean wasStateSet = false;
2201 try {
2202 synchronized (writestate) {
2203 if (writestate.writesEnabled) {
2204 wasStateSet = true;
2205 writestate.compacting.incrementAndGet();
2206 } else {
2207 String msg = "NOT compacting region " + this + ". Writes disabled.";
2208 LOG.info(msg);
2209 status.abort(msg);
2210 return false;
2211 }
2212 }
2213 LOG.info("Starting compaction on " + store + " in region " + this
2214 + (compaction.getRequest().isOffPeak()?" as an off-peak compaction":""));
2215 doRegionCompactionPrep();
2216 try {
2217 status.setStatus("Compacting store " + store);
2218
2219
2220 requestNeedsCancellation = false;
2221 store.compact(compaction, throughputController, user);
2222 } catch (InterruptedIOException iioe) {
2223 String msg = "compaction interrupted";
2224 LOG.info(msg, iioe);
2225 status.abort(msg);
2226 return false;
2227 }
2228 } finally {
2229 if (wasStateSet) {
2230 synchronized (writestate) {
2231 writestate.compacting.decrementAndGet();
2232 if (writestate.compacting.get() <= 0) {
2233 writestate.notifyAll();
2234 }
2235 }
2236 }
2237 }
2238 status.markComplete("Compaction complete");
2239 return true;
2240 } finally {
2241 if (requestNeedsCancellation) store.cancelRequestedCompaction(compaction);
2242 if (status != null) {
2243 if (LOG.isDebugEnabled()) {
2244 LOG.debug("Compaction status journal:\n" + status.prettyPrintJournal());
2245 }
2246 status.cleanup();
2247 }
2248 }
2249 }
2250
2251 @Override
2252 public FlushResult flush(boolean force) throws IOException {
2253 return flushcache(force, false);
2254 }
2255
2256
2257
2258
2259
2260
2261
2262
2263
2264
2265
2266
2267
2268
2269
2270
2271
2272
2273
2274
2275
2276
2277
2278 public FlushResult flushcache(boolean forceFlushAllStores, boolean writeFlushRequestWalMarker)
2279 throws IOException {
2280
2281 if (this.closing.get()) {
2282 String msg = "Skipping flush on " + this + " because closing";
2283 LOG.debug(msg);
2284 return new FlushResultImpl(FlushResult.Result.CANNOT_FLUSH, msg, false);
2285 }
2286 MonitoredTask status = TaskMonitor.get().createStatus("Flushing " + this);
2287 status.enableStatusJournal(false);
2288 status.setStatus("Acquiring readlock on region");
2289
2290 lock.readLock().lock();
2291 try {
2292 if (this.closed.get()) {
2293 String msg = "Skipping flush on " + this + " because closed";
2294 LOG.debug(msg);
2295 status.abort(msg);
2296 return new FlushResultImpl(FlushResult.Result.CANNOT_FLUSH, msg, false);
2297 }
2298 if (coprocessorHost != null) {
2299 status.setStatus("Running coprocessor pre-flush hooks");
2300 coprocessorHost.preFlush();
2301 }
2302
2303
2304 if (numMutationsWithoutWAL.get() > 0) {
2305 numMutationsWithoutWAL.set(0);
2306 dataInMemoryWithoutWAL.set(0);
2307 }
2308 synchronized (writestate) {
2309 if (!writestate.flushing && writestate.writesEnabled) {
2310 this.writestate.flushing = true;
2311 } else {
2312 if (LOG.isDebugEnabled()) {
2313 LOG.debug("NOT flushing memstore for region " + this
2314 + ", flushing=" + writestate.flushing + ", writesEnabled="
2315 + writestate.writesEnabled);
2316 }
2317 String msg = "Not flushing since "
2318 + (writestate.flushing ? "already flushing"
2319 : "writes not enabled");
2320 status.abort(msg);
2321 return new FlushResultImpl(FlushResult.Result.CANNOT_FLUSH, msg, false);
2322 }
2323 }
2324
2325 try {
2326 Collection<Store> specificStoresToFlush =
2327 forceFlushAllStores ? stores.values() : flushPolicy.selectStoresToFlush();
2328 FlushResult fs = internalFlushcache(specificStoresToFlush,
2329 status, writeFlushRequestWalMarker);
2330
2331 if (coprocessorHost != null) {
2332 status.setStatus("Running post-flush coprocessor hooks");
2333 coprocessorHost.postFlush();
2334 }
2335
2336 if(fs.isFlushSucceeded()) {
2337 flushesQueued.set(0L);
2338 }
2339
2340 status.markComplete("Flush successful");
2341 return fs;
2342 } finally {
2343 synchronized (writestate) {
2344 writestate.flushing = false;
2345 this.writestate.flushRequested = false;
2346 writestate.notifyAll();
2347 }
2348 }
2349 } finally {
2350 lock.readLock().unlock();
2351 if (LOG.isDebugEnabled()) {
2352 LOG.debug("Flush status journal:\n" + status.prettyPrintJournal());
2353 }
2354 status.cleanup();
2355 }
2356 }
2357
2358
2359
2360
2361
2362
2363
2364
2365 boolean shouldFlushStore(Store store) {
2366 long earliest = this.wal.getEarliestMemstoreSeqNum(getRegionInfo().getEncodedNameAsBytes(),
2367 store.getFamily().getName()) - 1;
2368 if (earliest > 0 && earliest + flushPerChanges < mvcc.getReadPoint()) {
2369 if (LOG.isDebugEnabled()) {
2370 LOG.debug("Flush column family " + store.getColumnFamilyName() + " of " +
2371 getRegionInfo().getEncodedName() + " because unflushed sequenceid=" + earliest +
2372 " is > " + this.flushPerChanges + " from current=" + mvcc.getReadPoint());
2373 }
2374 return true;
2375 }
2376 if (this.flushCheckInterval <= 0) {
2377 return false;
2378 }
2379 long now = EnvironmentEdgeManager.currentTime();
2380 if (store.timeOfOldestEdit() < now - this.flushCheckInterval) {
2381 if (LOG.isDebugEnabled()) {
2382 LOG.debug("Flush column family: " + store.getColumnFamilyName() + " of " +
2383 getRegionInfo().getEncodedName() + " because time of oldest edit=" +
2384 store.timeOfOldestEdit() + " is > " + this.flushCheckInterval + " from now =" + now);
2385 }
2386 return true;
2387 }
2388 return false;
2389 }
2390
2391
2392
2393
2394 boolean shouldFlush(final StringBuffer whyFlush) {
2395 whyFlush.setLength(0);
2396
2397 if (this.maxFlushedSeqId > 0
2398 && (this.maxFlushedSeqId + this.flushPerChanges < this.mvcc.getReadPoint())) {
2399 whyFlush.append("more than max edits, " + this.flushPerChanges + ", since last flush");
2400 return true;
2401 }
2402 long modifiedFlushCheckInterval = flushCheckInterval;
2403 if (getRegionInfo().isSystemTable() &&
2404 getRegionInfo().getReplicaId() == HRegionInfo.DEFAULT_REPLICA_ID) {
2405 modifiedFlushCheckInterval = SYSTEM_CACHE_FLUSH_INTERVAL;
2406 }
2407 if (modifiedFlushCheckInterval <= 0) {
2408 return false;
2409 }
2410 long now = EnvironmentEdgeManager.currentTime();
2411
2412 if ((now - getEarliestFlushTimeForAllStores() < modifiedFlushCheckInterval)) {
2413 return false;
2414 }
2415
2416
2417 for (Store s : getStores()) {
2418 if (s.timeOfOldestEdit() < now - modifiedFlushCheckInterval) {
2419
2420 whyFlush.append(s.toString() + " has an old edit so flush to free WALs");
2421 return true;
2422 }
2423 }
2424 return false;
2425 }
2426
2427
2428
2429
2430
2431
2432 private FlushResult internalFlushcache(MonitoredTask status)
2433 throws IOException {
2434 return internalFlushcache(stores.values(), status, false);
2435 }
2436
2437
2438
2439
2440
2441
2442 private FlushResult internalFlushcache(final Collection<Store> storesToFlush,
2443 MonitoredTask status, boolean writeFlushWalMarker) throws IOException {
2444 return internalFlushcache(this.wal, HConstants.NO_SEQNUM, storesToFlush,
2445 status, writeFlushWalMarker);
2446 }
2447
2448
2449
2450
2451
2452
2453
2454
2455
2456
2457
2458
2459
2460
2461
2462
2463
2464
2465
2466
2467
2468
2469
2470
2471
2472
2473
2474
2475
2476 protected FlushResult internalFlushcache(final WAL wal, final long myseqid,
2477 final Collection<Store> storesToFlush, MonitoredTask status, boolean writeFlushWalMarker)
2478 throws IOException {
2479 PrepareFlushResult result
2480 = internalPrepareFlushCache(wal, myseqid, storesToFlush, status, writeFlushWalMarker);
2481 if (result.result == null) {
2482 return internalFlushCacheAndCommit(wal, status, result, storesToFlush);
2483 } else {
2484 return result.result;
2485 }
2486 }
2487
2488 @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="DLS_DEAD_LOCAL_STORE",
2489 justification="FindBugs seems confused about trxId")
2490 protected PrepareFlushResult internalPrepareFlushCache(final WAL wal, final long myseqid,
2491 final Collection<Store> storesToFlush, MonitoredTask status, boolean writeFlushWalMarker)
2492 throws IOException {
2493 if (this.rsServices != null && this.rsServices.isAborted()) {
2494
2495 throw new IOException("Aborting flush because server is aborted...");
2496 }
2497 final long startTime = EnvironmentEdgeManager.currentTime();
2498
2499 if (this.memstoreSize.get() <= 0) {
2500
2501
2502 MultiVersionConcurrencyControl.WriteEntry writeEntry = null;
2503 this.updatesLock.writeLock().lock();
2504 try {
2505 if (this.memstoreSize.get() <= 0) {
2506
2507
2508
2509
2510
2511
2512 if (wal != null) {
2513 writeEntry = mvcc.begin();
2514 long flushOpSeqId = writeEntry.getWriteNumber();
2515 FlushResult flushResult = new FlushResultImpl(
2516 FlushResult.Result.CANNOT_FLUSH_MEMSTORE_EMPTY,
2517 flushOpSeqId,
2518 "Nothing to flush",
2519 writeFlushRequestMarkerToWAL(wal, writeFlushWalMarker));
2520
2521
2522 mvcc.completeAndWait(writeEntry);
2523 writeEntry = null;
2524 return new PrepareFlushResult(flushResult, myseqid);
2525 } else {
2526 return new PrepareFlushResult(
2527 new FlushResultImpl(
2528 FlushResult.Result.CANNOT_FLUSH_MEMSTORE_EMPTY,
2529 "Nothing to flush",
2530 false),
2531 myseqid);
2532 }
2533 }
2534 } finally {
2535 this.updatesLock.writeLock().unlock();
2536 if (writeEntry != null) {
2537 mvcc.complete(writeEntry);
2538 }
2539 }
2540 }
2541
2542 if (LOG.isInfoEnabled()) {
2543
2544 StringBuilder perCfExtras = null;
2545 if (!isAllFamilies(storesToFlush)) {
2546 perCfExtras = new StringBuilder();
2547 for (Store store: storesToFlush) {
2548 perCfExtras.append("; ").append(store.getColumnFamilyName());
2549 perCfExtras.append("=").append(StringUtils.byteDesc(store.getMemStoreSize()));
2550 }
2551 }
2552 LOG.info("Flushing " + storesToFlush.size() + "/" + stores.size() +
2553 " column families, memstore=" + StringUtils.byteDesc(this.memstoreSize.get()) +
2554 ((perCfExtras != null && perCfExtras.length() > 0)? perCfExtras.toString(): "") +
2555 ((wal != null) ? "" : "; WAL is null, using passed sequenceid=" + myseqid));
2556 }
2557
2558
2559
2560
2561
2562
2563
2564 status.setStatus("Obtaining lock to block concurrent updates");
2565
2566 this.updatesLock.writeLock().lock();
2567 status.setStatus("Preparing to flush by snapshotting stores in " +
2568 getRegionInfo().getEncodedName());
2569 long totalFlushableSizeOfFlushableStores = 0;
2570
2571 Set<byte[]> flushedFamilyNames = new HashSet<byte[]>();
2572 for (Store store: storesToFlush) {
2573 flushedFamilyNames.add(store.getFamily().getName());
2574 }
2575
2576 TreeMap<byte[], StoreFlushContext> storeFlushCtxs
2577 = new TreeMap<byte[], StoreFlushContext>(Bytes.BYTES_COMPARATOR);
2578 TreeMap<byte[], List<Path>> committedFiles = new TreeMap<byte[], List<Path>>(
2579 Bytes.BYTES_COMPARATOR);
2580 TreeMap<byte[], Long> storeFlushableSize
2581 = new TreeMap<byte[], Long>(Bytes.BYTES_COMPARATOR);
2582
2583
2584
2585 long flushOpSeqId = HConstants.NO_SEQNUM;
2586
2587
2588 long flushedSeqId = HConstants.NO_SEQNUM;
2589 byte[] encodedRegionName = getRegionInfo().getEncodedNameAsBytes();
2590
2591 long trxId = 0;
2592 MultiVersionConcurrencyControl.WriteEntry writeEntry = mvcc.begin();
2593
2594
2595
2596
2597
2598 mvcc.completeAndWait(writeEntry);
2599
2600
2601 writeEntry = null;
2602 try {
2603 try {
2604 if (wal != null) {
2605 Long earliestUnflushedSequenceIdForTheRegion =
2606 wal.startCacheFlush(encodedRegionName, flushedFamilyNames);
2607 if (earliestUnflushedSequenceIdForTheRegion == null) {
2608
2609 String msg = this.getRegionInfo().getEncodedName() + " flush aborted; WAL closing.";
2610 status.setStatus(msg);
2611 return new PrepareFlushResult(
2612 new FlushResultImpl(FlushResult.Result.CANNOT_FLUSH, msg, false),
2613 myseqid);
2614 }
2615 flushOpSeqId = getNextSequenceId(wal);
2616
2617 flushedSeqId =
2618 earliestUnflushedSequenceIdForTheRegion.longValue() == HConstants.NO_SEQNUM?
2619 flushOpSeqId: earliestUnflushedSequenceIdForTheRegion.longValue() - 1;
2620 } else {
2621
2622 flushedSeqId = flushOpSeqId = myseqid;
2623 }
2624
2625 for (Store s : storesToFlush) {
2626 totalFlushableSizeOfFlushableStores += s.getFlushableSize();
2627 storeFlushCtxs.put(s.getFamily().getName(), s.createFlushContext(flushOpSeqId));
2628 committedFiles.put(s.getFamily().getName(), null);
2629 storeFlushableSize.put(s.getFamily().getName(), s.getFlushableSize());
2630 }
2631
2632
2633 if (wal != null && !writestate.readOnly) {
2634 FlushDescriptor desc = ProtobufUtil.toFlushDescriptor(FlushAction.START_FLUSH,
2635 getRegionInfo(), flushOpSeqId, committedFiles);
2636
2637 trxId = WALUtil.writeFlushMarker(wal, this.htableDescriptor, getRegionInfo(),
2638 desc, false, mvcc);
2639 }
2640
2641
2642 for (StoreFlushContext flush : storeFlushCtxs.values()) {
2643 flush.prepare();
2644 }
2645 } catch (IOException ex) {
2646 if (wal != null) {
2647 if (trxId > 0) {
2648 try {
2649 FlushDescriptor desc = ProtobufUtil.toFlushDescriptor(FlushAction.ABORT_FLUSH,
2650 getRegionInfo(), flushOpSeqId, committedFiles);
2651 WALUtil.writeFlushMarker(wal, this.htableDescriptor, getRegionInfo(),
2652 desc, false, mvcc);
2653 } catch (Throwable t) {
2654 LOG.warn("Received unexpected exception trying to write ABORT_FLUSH marker to WAL:" +
2655 StringUtils.stringifyException(t));
2656
2657 }
2658 }
2659
2660 wal.abortCacheFlush(this.getRegionInfo().getEncodedNameAsBytes());
2661 throw ex;
2662 }
2663 } finally {
2664 this.updatesLock.writeLock().unlock();
2665 }
2666 String s = "Finished memstore snapshotting " + this +
2667 ", syncing WAL and waiting on mvcc, flushsize=" + totalFlushableSizeOfFlushableStores;
2668 status.setStatus(s);
2669 if (LOG.isTraceEnabled()) LOG.trace(s);
2670
2671
2672 if (wal != null) {
2673 try {
2674 wal.sync();
2675 } catch (IOException ioe) {
2676 wal.abortCacheFlush(this.getRegionInfo().getEncodedNameAsBytes());
2677 throw ioe;
2678 }
2679 }
2680 } finally {
2681 if (writeEntry != null) {
2682
2683 mvcc.complete(writeEntry);
2684 }
2685 }
2686 return new PrepareFlushResult(storeFlushCtxs, committedFiles, storeFlushableSize, startTime,
2687 flushOpSeqId, flushedSeqId, totalFlushableSizeOfFlushableStores);
2688 }
2689
2690
2691
2692
2693
2694 private boolean isAllFamilies(final Collection<Store> families) {
2695 return families == null || this.stores.size() == families.size();
2696 }
2697
2698
2699
2700
2701
2702
2703
2704 private boolean writeFlushRequestMarkerToWAL(WAL wal, boolean writeFlushWalMarker) {
2705 if (writeFlushWalMarker && wal != null && !writestate.readOnly) {
2706 FlushDescriptor desc = ProtobufUtil.toFlushDescriptor(FlushAction.CANNOT_FLUSH,
2707 getRegionInfo(), -1, new TreeMap<byte[], List<Path>>(Bytes.BYTES_COMPARATOR));
2708 try {
2709 WALUtil.writeFlushMarker(wal, this.htableDescriptor, getRegionInfo(),
2710 desc, true, mvcc);
2711 return true;
2712 } catch (IOException e) {
2713 LOG.warn(getRegionInfo().getEncodedName() + " : "
2714 + "Received exception while trying to write the flush request to wal", e);
2715 }
2716 }
2717 return false;
2718 }
2719
2720 @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="NN_NAKED_NOTIFY",
2721 justification="Intentional; notify is about completed flush")
2722 protected FlushResult internalFlushCacheAndCommit(
2723 final WAL wal, MonitoredTask status, final PrepareFlushResult prepareResult,
2724 final Collection<Store> storesToFlush)
2725 throws IOException {
2726
2727
2728 TreeMap<byte[], StoreFlushContext> storeFlushCtxs = prepareResult.storeFlushCtxs;
2729 TreeMap<byte[], List<Path>> committedFiles = prepareResult.committedFiles;
2730 long startTime = prepareResult.startTime;
2731 long flushOpSeqId = prepareResult.flushOpSeqId;
2732 long flushedSeqId = prepareResult.flushedSeqId;
2733 long totalFlushableSizeOfFlushableStores = prepareResult.totalFlushableSize;
2734
2735 String s = "Flushing stores of " + this;
2736 status.setStatus(s);
2737 if (LOG.isTraceEnabled()) LOG.trace(s);
2738
2739
2740
2741
2742
2743 boolean compactionRequested = false;
2744 long flushedOutputFileSize = 0;
2745 try {
2746
2747
2748
2749
2750
2751 for (StoreFlushContext flush : storeFlushCtxs.values()) {
2752 flush.flushCache(status);
2753 }
2754
2755
2756
2757 Iterator<Store> it = storesToFlush.iterator();
2758
2759 for (StoreFlushContext flush : storeFlushCtxs.values()) {
2760 boolean needsCompaction = flush.commit(status);
2761 if (needsCompaction) {
2762 compactionRequested = true;
2763 }
2764 byte[] storeName = it.next().getFamily().getName();
2765 List<Path> storeCommittedFiles = flush.getCommittedFiles();
2766 committedFiles.put(storeName, storeCommittedFiles);
2767
2768 if (storeCommittedFiles == null || storeCommittedFiles.isEmpty()) {
2769 totalFlushableSizeOfFlushableStores -= prepareResult.storeFlushableSize.get(storeName);
2770 }
2771 flushedOutputFileSize += flush.getOutputFileSize();
2772 }
2773 storeFlushCtxs.clear();
2774
2775
2776 this.addAndGetGlobalMemstoreSize(-totalFlushableSizeOfFlushableStores);
2777
2778 if (wal != null) {
2779
2780 FlushDescriptor desc = ProtobufUtil.toFlushDescriptor(FlushAction.COMMIT_FLUSH,
2781 getRegionInfo(), flushOpSeqId, committedFiles);
2782 WALUtil.writeFlushMarker(wal, this.htableDescriptor, getRegionInfo(),
2783 desc, true, mvcc);
2784 }
2785 } catch (Throwable t) {
2786
2787
2788
2789
2790
2791
2792 if (wal != null) {
2793 try {
2794 FlushDescriptor desc = ProtobufUtil.toFlushDescriptor(FlushAction.ABORT_FLUSH,
2795 getRegionInfo(), flushOpSeqId, committedFiles);
2796 WALUtil.writeFlushMarker(wal, this.htableDescriptor, getRegionInfo(),
2797 desc, false, mvcc);
2798 } catch (Throwable ex) {
2799 LOG.warn(getRegionInfo().getEncodedName() + " : "
2800 + "failed writing ABORT_FLUSH marker to WAL", ex);
2801
2802 }
2803 wal.abortCacheFlush(this.getRegionInfo().getEncodedNameAsBytes());
2804 }
2805 DroppedSnapshotException dse = new DroppedSnapshotException("region: " +
2806 Bytes.toStringBinary(getRegionInfo().getRegionName()), t);
2807 status.abort("Flush failed: " + StringUtils.stringifyException(t));
2808
2809
2810
2811
2812
2813 this.closing.set(true);
2814
2815 if (rsServices != null) {
2816
2817 rsServices.abort("Replay of WAL required. Forcing server shutdown", dse);
2818 }
2819
2820 throw dse;
2821 }
2822
2823
2824 if (wal != null) {
2825 wal.completeCacheFlush(this.getRegionInfo().getEncodedNameAsBytes());
2826 }
2827
2828
2829 for (Store store: storesToFlush) {
2830 this.lastStoreFlushTimeMap.put(store, startTime);
2831 }
2832
2833 this.maxFlushedSeqId = flushedSeqId;
2834 this.lastFlushOpSeqId = flushOpSeqId;
2835
2836
2837
2838 synchronized (this) {
2839 notifyAll();
2840 }
2841
2842 long time = EnvironmentEdgeManager.currentTime() - startTime;
2843 long memstoresize = this.memstoreSize.get();
2844 String msg = "Finished memstore flush of ~"
2845 + StringUtils.byteDesc(totalFlushableSizeOfFlushableStores) + "/"
2846 + totalFlushableSizeOfFlushableStores + ", currentsize="
2847 + StringUtils.byteDesc(memstoresize) + "/" + memstoresize
2848 + " for region " + this + " in " + time + "ms, sequenceid="
2849 + flushOpSeqId + ", compaction requested=" + compactionRequested
2850 + ((wal == null) ? "; wal=null" : "");
2851 LOG.info(msg);
2852 status.setStatus(msg);
2853
2854 if (rsServices != null && rsServices.getMetrics() != null) {
2855 rsServices.getMetrics().updateFlush(time,
2856 totalFlushableSizeOfFlushableStores, flushedOutputFileSize);
2857 }
2858
2859 return new FlushResultImpl(compactionRequested ?
2860 FlushResult.Result.FLUSHED_COMPACTION_NEEDED :
2861 FlushResult.Result.FLUSHED_NO_COMPACTION_NEEDED, flushOpSeqId);
2862 }
2863
2864
2865
2866
2867
2868
2869 protected long getNextSequenceId(final WAL wal) throws IOException {
2870
2871
2872
2873
2874
2875
2876 WALKey key = this.appendEmptyEdit(wal);
2877 mvcc.complete(key.getWriteEntry());
2878 return key.getSequenceId();
2879 }
2880
2881
2882
2883
2884
2885 @Override
2886 public Result getClosestRowBefore(final byte [] row, final byte [] family) throws IOException {
2887 if (coprocessorHost != null) {
2888 Result result = new Result();
2889 if (coprocessorHost.preGetClosestRowBefore(row, family, result)) {
2890 return result;
2891 }
2892 }
2893
2894
2895 checkRow(row, "getClosestRowBefore");
2896 startRegionOperation(Operation.GET);
2897 try {
2898 Result result = null;
2899 Get get = new Get(row);
2900 get.addFamily(family);
2901 get.setClosestRowBefore(true);
2902 result = get(get);
2903
2904 result = result.isEmpty() ? null : result;
2905 if (coprocessorHost != null) {
2906 coprocessorHost.postGetClosestRowBefore(row, family, result);
2907 }
2908 return result;
2909 } finally {
2910 closeRegionOperation(Operation.GET);
2911 }
2912 }
2913
2914 @Override
2915 public RegionScanner getScanner(Scan scan) throws IOException {
2916 return getScanner(scan, null);
2917 }
2918
2919 @Override
2920 public RegionScanner getScanner(Scan scan, List<KeyValueScanner> additionalScanners)
2921 throws IOException {
2922 return getScanner(scan, additionalScanners, HConstants.NO_NONCE, HConstants.NO_NONCE);
2923 }
2924
2925 private RegionScanner getScanner(Scan scan, List<KeyValueScanner> additionalScanners,
2926 long nonceGroup, long nonce) throws IOException {
2927 startRegionOperation(Operation.SCAN);
2928 try {
2929
2930 if (!scan.hasFamilies()) {
2931
2932 for (byte[] family: this.htableDescriptor.getFamiliesKeys()) {
2933 scan.addFamily(family);
2934 }
2935 } else {
2936 for (byte [] family : scan.getFamilyMap().keySet()) {
2937 checkFamily(family);
2938 }
2939 }
2940 return instantiateRegionScanner(scan, additionalScanners, nonceGroup, nonce);
2941 } finally {
2942 closeRegionOperation(Operation.SCAN);
2943 }
2944 }
2945
2946 protected RegionScanner instantiateRegionScanner(Scan scan,
2947 List<KeyValueScanner> additionalScanners) throws IOException {
2948 return instantiateRegionScanner(scan, additionalScanners, HConstants.NO_NONCE,
2949 HConstants.NO_NONCE);
2950 }
2951
2952 protected RegionScanner instantiateRegionScanner(Scan scan,
2953 List<KeyValueScanner> additionalScanners, long nonceGroup, long nonce) throws IOException {
2954 if (scan.isReversed()) {
2955 if (scan.getFilter() != null) {
2956 scan.getFilter().setReversed(true);
2957 }
2958 return new ReversedRegionScannerImpl(scan, additionalScanners, this);
2959 }
2960 return new RegionScannerImpl(scan, additionalScanners, this, nonceGroup, nonce);
2961 }
2962
2963 @Override
2964 public void prepareDelete(Delete delete) throws IOException {
2965
2966 if(delete.getFamilyCellMap().isEmpty()){
2967 for(byte [] family : this.htableDescriptor.getFamiliesKeys()){
2968
2969 delete.addFamily(family, delete.getTimeStamp());
2970 }
2971 } else {
2972 for(byte [] family : delete.getFamilyCellMap().keySet()) {
2973 if(family == null) {
2974 throw new NoSuchColumnFamilyException("Empty family is invalid");
2975 }
2976 checkFamily(family);
2977 }
2978 }
2979 }
2980
2981 @Override
2982 public void delete(Delete delete) throws IOException {
2983 checkReadOnly();
2984 checkResources();
2985 startRegionOperation(Operation.DELETE);
2986 try {
2987
2988 doBatchMutate(delete);
2989 } finally {
2990 closeRegionOperation(Operation.DELETE);
2991 }
2992 }
2993
2994
2995
2996
2997 private static final byte [] FOR_UNIT_TESTS_ONLY = Bytes.toBytes("ForUnitTestsOnly");
2998
2999
3000
3001
3002
3003
3004 void delete(NavigableMap<byte[], List<Cell>> familyMap,
3005 Durability durability) throws IOException {
3006 Delete delete = new Delete(FOR_UNIT_TESTS_ONLY);
3007 delete.setFamilyCellMap(familyMap);
3008 delete.setDurability(durability);
3009 doBatchMutate(delete);
3010 }
3011
3012 @Override
3013 public void prepareDeleteTimestamps(Mutation mutation, Map<byte[], List<Cell>> familyMap,
3014 byte[] byteNow) throws IOException {
3015 for (Map.Entry<byte[], List<Cell>> e : familyMap.entrySet()) {
3016
3017 byte[] family = e.getKey();
3018 List<Cell> cells = e.getValue();
3019 assert cells instanceof RandomAccess;
3020
3021 Map<byte[], Integer> kvCount = new TreeMap<byte[], Integer>(Bytes.BYTES_COMPARATOR);
3022 int listSize = cells.size();
3023 for (int i=0; i < listSize; i++) {
3024 Cell cell = cells.get(i);
3025
3026
3027 if (cell.getTimestamp() == HConstants.LATEST_TIMESTAMP && CellUtil.isDeleteType(cell)) {
3028 byte[] qual = CellUtil.cloneQualifier(cell);
3029 if (qual == null) qual = HConstants.EMPTY_BYTE_ARRAY;
3030
3031 Integer count = kvCount.get(qual);
3032 if (count == null) {
3033 kvCount.put(qual, 1);
3034 } else {
3035 kvCount.put(qual, count + 1);
3036 }
3037 count = kvCount.get(qual);
3038
3039 Get get = new Get(CellUtil.cloneRow(cell));
3040 get.setMaxVersions(count);
3041 get.addColumn(family, qual);
3042 if (coprocessorHost != null) {
3043 if (!coprocessorHost.prePrepareTimeStampForDeleteVersion(mutation, cell,
3044 byteNow, get)) {
3045 updateDeleteLatestVersionTimeStamp(cell, get, count, byteNow);
3046 }
3047 } else {
3048 updateDeleteLatestVersionTimeStamp(cell, get, count, byteNow);
3049 }
3050 } else {
3051 CellUtil.updateLatestStamp(cell, byteNow, 0);
3052 }
3053 }
3054 }
3055 }
3056
3057 void updateDeleteLatestVersionTimeStamp(Cell cell, Get get, int count, byte[] byteNow)
3058 throws IOException {
3059 List<Cell> result = get(get, false);
3060
3061 if (result.size() < count) {
3062
3063 CellUtil.updateLatestStamp(cell, byteNow, 0);
3064 return;
3065 }
3066 if (result.size() > count) {
3067 throw new RuntimeException("Unexpected size: " + result.size());
3068 }
3069 Cell getCell = result.get(count - 1);
3070 CellUtil.setTimestamp(cell, getCell.getTimestamp());
3071 }
3072
3073 @Override
3074 public void put(Put put) throws IOException {
3075 checkReadOnly();
3076
3077
3078
3079
3080
3081 checkResources();
3082 startRegionOperation(Operation.PUT);
3083 try {
3084
3085 doBatchMutate(put);
3086 } finally {
3087 closeRegionOperation(Operation.PUT);
3088 }
3089 }
3090
3091
3092
3093
3094
3095
3096 private abstract static class BatchOperationInProgress<T> {
3097 T[] operations;
3098 int nextIndexToProcess = 0;
3099 OperationStatus[] retCodeDetails;
3100 WALEdit[] walEditsFromCoprocessors;
3101
3102 public BatchOperationInProgress(T[] operations) {
3103 this.operations = operations;
3104 this.retCodeDetails = new OperationStatus[operations.length];
3105 this.walEditsFromCoprocessors = new WALEdit[operations.length];
3106 Arrays.fill(this.retCodeDetails, OperationStatus.NOT_RUN);
3107 }
3108
3109 public abstract Mutation getMutation(int index);
3110 public abstract long getNonceGroup(int index);
3111 public abstract long getNonce(int index);
3112
3113 public abstract Mutation[] getMutationsForCoprocs();
3114 public abstract boolean isInReplay();
3115 public abstract long getReplaySequenceId();
3116
3117 public boolean isDone() {
3118 return nextIndexToProcess == operations.length;
3119 }
3120 }
3121
3122 private static class MutationBatch extends BatchOperationInProgress<Mutation> {
3123 private long nonceGroup;
3124 private long nonce;
3125 public MutationBatch(Mutation[] operations, long nonceGroup, long nonce) {
3126 super(operations);
3127 this.nonceGroup = nonceGroup;
3128 this.nonce = nonce;
3129 }
3130
3131 @Override
3132 public Mutation getMutation(int index) {
3133 return this.operations[index];
3134 }
3135
3136 @Override
3137 public long getNonceGroup(int index) {
3138 return nonceGroup;
3139 }
3140
3141 @Override
3142 public long getNonce(int index) {
3143 return nonce;
3144 }
3145
3146 @Override
3147 public Mutation[] getMutationsForCoprocs() {
3148 return this.operations;
3149 }
3150
3151 @Override
3152 public boolean isInReplay() {
3153 return false;
3154 }
3155
3156 @Override
3157 public long getReplaySequenceId() {
3158 return 0;
3159 }
3160 }
3161
3162 private static class ReplayBatch extends BatchOperationInProgress<MutationReplay> {
3163 private long replaySeqId = 0;
3164 public ReplayBatch(MutationReplay[] operations, long seqId) {
3165 super(operations);
3166 this.replaySeqId = seqId;
3167 }
3168
3169 @Override
3170 public Mutation getMutation(int index) {
3171 return this.operations[index].mutation;
3172 }
3173
3174 @Override
3175 public long getNonceGroup(int index) {
3176 return this.operations[index].nonceGroup;
3177 }
3178
3179 @Override
3180 public long getNonce(int index) {
3181 return this.operations[index].nonce;
3182 }
3183
3184 @Override
3185 public Mutation[] getMutationsForCoprocs() {
3186 assert false;
3187 throw new RuntimeException("Should not be called for replay batch");
3188 }
3189
3190 @Override
3191 public boolean isInReplay() {
3192 return true;
3193 }
3194
3195 @Override
3196 public long getReplaySequenceId() {
3197 return this.replaySeqId;
3198 }
3199 }
3200
3201 @Override
3202 public OperationStatus[] batchMutate(Mutation[] mutations, long nonceGroup, long nonce)
3203 throws IOException {
3204
3205
3206
3207
3208 return batchMutate(new MutationBatch(mutations, nonceGroup, nonce));
3209 }
3210
3211 public OperationStatus[] batchMutate(Mutation[] mutations) throws IOException {
3212 return batchMutate(mutations, HConstants.NO_NONCE, HConstants.NO_NONCE);
3213 }
3214
3215 @Override
3216 public OperationStatus[] batchReplay(MutationReplay[] mutations, long replaySeqId)
3217 throws IOException {
3218 if (!RegionReplicaUtil.isDefaultReplica(getRegionInfo())
3219 && replaySeqId < lastReplayedOpenRegionSeqId) {
3220
3221
3222 if (LOG.isTraceEnabled()) {
3223 LOG.trace(getRegionInfo().getEncodedName() + " : "
3224 + "Skipping " + mutations.length + " mutations with replaySeqId=" + replaySeqId
3225 + " which is < than lastReplayedOpenRegionSeqId=" + lastReplayedOpenRegionSeqId);
3226 for (MutationReplay mut : mutations) {
3227 LOG.trace(getRegionInfo().getEncodedName() + " : Skipping : " + mut.mutation);
3228 }
3229 }
3230
3231 OperationStatus[] statuses = new OperationStatus[mutations.length];
3232 for (int i = 0; i < statuses.length; i++) {
3233 statuses[i] = OperationStatus.SUCCESS;
3234 }
3235 return statuses;
3236 }
3237 return batchMutate(new ReplayBatch(mutations, replaySeqId));
3238 }
3239
3240
3241
3242
3243
3244
3245
3246
3247
3248 OperationStatus[] batchMutate(BatchOperationInProgress<?> batchOp) throws IOException {
3249 boolean initialized = false;
3250 Operation op = batchOp.isInReplay() ? Operation.REPLAY_BATCH_MUTATE : Operation.BATCH_MUTATE;
3251 startRegionOperation(op);
3252 try {
3253 while (!batchOp.isDone()) {
3254 if (!batchOp.isInReplay()) {
3255 checkReadOnly();
3256 }
3257 checkResources();
3258
3259 if (!initialized) {
3260 this.writeRequestsCount.add(batchOp.operations.length);
3261 if (!batchOp.isInReplay()) {
3262 doPreMutationHook(batchOp);
3263 }
3264 initialized = true;
3265 }
3266 doMiniBatchMutation(batchOp);
3267 long newSize = this.getMemstoreSize();
3268 if (isFlushSize(newSize)) {
3269 requestFlush();
3270 }
3271 }
3272 } finally {
3273 if (rsServices != null && rsServices.getMetrics() != null) {
3274 rsServices.getMetrics().updateWriteQueryMeter(this.htableDescriptor.
3275 getTableName(), batchOp.operations.length);
3276 }
3277 closeRegionOperation(op);
3278 }
3279 return batchOp.retCodeDetails;
3280 }
3281
3282
3283 private void doPreMutationHook(BatchOperationInProgress<?> batchOp)
3284 throws IOException {
3285
3286 WALEdit walEdit = new WALEdit();
3287 if (coprocessorHost != null) {
3288 for (int i = 0 ; i < batchOp.operations.length; i++) {
3289 Mutation m = batchOp.getMutation(i);
3290 if (m instanceof Put) {
3291 if (coprocessorHost.prePut((Put) m, walEdit, m.getDurability())) {
3292
3293
3294 batchOp.retCodeDetails[i] = OperationStatus.SUCCESS;
3295 }
3296 } else if (m instanceof Delete) {
3297 Delete curDel = (Delete) m;
3298 if (curDel.getFamilyCellMap().isEmpty()) {
3299
3300 prepareDelete(curDel);
3301 }
3302 if (coprocessorHost.preDelete(curDel, walEdit, m.getDurability())) {
3303
3304
3305 batchOp.retCodeDetails[i] = OperationStatus.SUCCESS;
3306 }
3307 } else {
3308
3309
3310
3311 batchOp.retCodeDetails[i] = new OperationStatus(OperationStatusCode.FAILURE,
3312 "Put/Delete mutations only supported in batchMutate() now");
3313 }
3314 if (!walEdit.isEmpty()) {
3315 batchOp.walEditsFromCoprocessors[i] = walEdit;
3316 walEdit = new WALEdit();
3317 }
3318 }
3319 }
3320 }
3321
3322
3323
3324
3325
3326
3327 private long doMiniBatchMutation(BatchOperationInProgress<?> batchOp) throws IOException {
3328 boolean isInReplay = batchOp.isInReplay();
3329
3330 boolean putsCfSetConsistent = true;
3331
3332 Set<byte[]> putsCfSet = null;
3333
3334 boolean deletesCfSetConsistent = true;
3335
3336 Set<byte[]> deletesCfSet = null;
3337
3338 long currentNonceGroup = HConstants.NO_NONCE, currentNonce = HConstants.NO_NONCE;
3339 WALEdit walEdit = null;
3340 MultiVersionConcurrencyControl.WriteEntry writeEntry = null;
3341 long txid = 0;
3342 boolean walSyncSuccess = true;
3343 boolean locked = false;
3344 int cellCount = 0;
3345
3346 List<RowLock> acquiredRowLocks = Lists.newArrayListWithCapacity(batchOp.operations.length);
3347
3348 Map<byte[], List<Cell>>[] familyMaps = new Map[batchOp.operations.length];
3349
3350 int firstIndex = batchOp.nextIndexToProcess;
3351 int lastIndexExclusive = firstIndex;
3352 RowLock prevRowLock = null;
3353 boolean success = false;
3354 int noOfPuts = 0, noOfDeletes = 0;
3355 WALKey walKey = null;
3356 long mvccNum = 0;
3357 long addedSize = 0;
3358 final ObservedExceptionsInBatch observedExceptions = new ObservedExceptionsInBatch();
3359
3360
3361
3362 checkInterrupt();
3363
3364 try {
3365
3366 disableInterrupts();
3367
3368
3369
3370
3371
3372 int numReadyToWrite = 0;
3373 long now = EnvironmentEdgeManager.currentTime();
3374 while (lastIndexExclusive < batchOp.operations.length) {
3375 Mutation mutation = batchOp.getMutation(lastIndexExclusive);
3376 boolean isPutMutation = mutation instanceof Put;
3377
3378 Map<byte[], List<Cell>> familyMap = mutation.getFamilyCellMap();
3379
3380 familyMaps[lastIndexExclusive] = familyMap;
3381
3382
3383 if (batchOp.retCodeDetails[lastIndexExclusive].getOperationStatusCode()
3384 != OperationStatusCode.NOT_RUN) {
3385 lastIndexExclusive++;
3386 continue;
3387 }
3388
3389 try {
3390 checkAndPrepareMutation(mutation, batchOp.isInReplay(), familyMap, now);
3391 } catch (NoSuchColumnFamilyException nscf) {
3392 final String msg = "No such column family in batch mutation. ";
3393 if (observedExceptions.hasSeenNoSuchFamily()) {
3394 LOG.warn(msg + nscf.getMessage());
3395 } else {
3396 LOG.warn(msg, nscf);
3397 observedExceptions.sawNoSuchFamily();
3398 }
3399 batchOp.retCodeDetails[lastIndexExclusive] = new OperationStatus(
3400 OperationStatusCode.BAD_FAMILY, nscf.getMessage());
3401 lastIndexExclusive++;
3402 continue;
3403 } catch (FailedSanityCheckException fsce) {
3404 final String msg = "Batch Mutation did not pass sanity check. ";
3405 if (observedExceptions.hasSeenFailedSanityCheck()) {
3406 LOG.warn(msg + fsce.getMessage());
3407 } else {
3408 LOG.warn(msg, fsce);
3409 observedExceptions.sawFailedSanityCheck();
3410 }
3411 batchOp.retCodeDetails[lastIndexExclusive] = new OperationStatus(
3412 OperationStatusCode.SANITY_CHECK_FAILURE, fsce.getMessage());
3413 lastIndexExclusive++;
3414 continue;
3415 } catch (WrongRegionException we) {
3416 final String msg = "Batch mutation had a row that does not belong to this region. ";
3417 if (observedExceptions.hasSeenWrongRegion()) {
3418 LOG.warn(msg + we.getMessage());
3419 } else {
3420 LOG.warn(msg, we);
3421 observedExceptions.sawWrongRegion();
3422 }
3423 batchOp.retCodeDetails[lastIndexExclusive] = new OperationStatus(
3424 OperationStatusCode.SANITY_CHECK_FAILURE, we.getMessage());
3425 lastIndexExclusive++;
3426 continue;
3427 }
3428
3429
3430
3431
3432
3433
3434
3435
3436
3437
3438
3439 boolean shouldBlock = numReadyToWrite == 0;
3440 RowLock rowLock = null;
3441 try {
3442 rowLock = getRowLockInternal(mutation.getRow(), true, shouldBlock, prevRowLock);
3443 } catch (TimeoutIOException e) {
3444
3445 throw e;
3446 } catch (IOException ioe) {
3447 LOG.warn("Failed getting lock in batch put, row="
3448 + Bytes.toStringBinary(mutation.getRow()), ioe);
3449 }
3450 if (rowLock == null) {
3451
3452
3453 break;
3454
3455 } else {
3456 if (rowLock != prevRowLock) {
3457
3458
3459 acquiredRowLocks.add(rowLock);
3460 prevRowLock = rowLock;
3461 }
3462 }
3463
3464 lastIndexExclusive++;
3465 numReadyToWrite++;
3466 if (isInReplay) {
3467 for (List<Cell> cells : mutation.getFamilyCellMap().values()) {
3468 cellCount += cells.size();
3469 }
3470 }
3471 if (isPutMutation) {
3472
3473
3474
3475 if (putsCfSet == null) {
3476 putsCfSet = mutation.getFamilyCellMap().keySet();
3477 } else {
3478 putsCfSetConsistent = putsCfSetConsistent
3479 && mutation.getFamilyCellMap().keySet().equals(putsCfSet);
3480 }
3481 } else {
3482 if (deletesCfSet == null) {
3483 deletesCfSet = mutation.getFamilyCellMap().keySet();
3484 } else {
3485 deletesCfSetConsistent = deletesCfSetConsistent
3486 && mutation.getFamilyCellMap().keySet().equals(deletesCfSet);
3487 }
3488 }
3489 }
3490
3491
3492
3493 now = EnvironmentEdgeManager.currentTime();
3494 byte[] byteNow = Bytes.toBytes(now);
3495
3496
3497 if (numReadyToWrite <= 0) return 0L;
3498
3499
3500
3501
3502
3503
3504 for (int i = firstIndex; !isInReplay && i < lastIndexExclusive; i++) {
3505
3506 if (batchOp.retCodeDetails[i].getOperationStatusCode()
3507 != OperationStatusCode.NOT_RUN) continue;
3508
3509 Mutation mutation = batchOp.getMutation(i);
3510 if (mutation instanceof Put) {
3511 updateCellTimestamps(familyMaps[i].values(), byteNow);
3512 noOfPuts++;
3513 } else {
3514 prepareDeleteTimestamps(mutation, familyMaps[i], byteNow);
3515 noOfDeletes++;
3516 }
3517 rewriteCellTags(familyMaps[i], mutation);
3518 WALEdit fromCP = batchOp.walEditsFromCoprocessors[i];
3519 if (fromCP != null) {
3520 cellCount += fromCP.size();
3521 }
3522 if (getEffectiveDurability(mutation.getDurability()) != Durability.SKIP_WAL) {
3523 for (List<Cell> cells : familyMaps[i].values()) {
3524 cellCount += cells.size();
3525 }
3526 }
3527 }
3528 lock(this.updatesLock.readLock(), numReadyToWrite);
3529 locked = true;
3530
3531
3532 if (!isInReplay && coprocessorHost != null) {
3533 MiniBatchOperationInProgress<Mutation> miniBatchOp =
3534 new MiniBatchOperationInProgress<Mutation>(batchOp.getMutationsForCoprocs(),
3535 batchOp.retCodeDetails, batchOp.walEditsFromCoprocessors, firstIndex, lastIndexExclusive);
3536 if (coprocessorHost.preBatchMutate(miniBatchOp)) {
3537 return 0L;
3538 } else {
3539 for (int i = firstIndex; i < lastIndexExclusive; i++) {
3540 if (batchOp.retCodeDetails[i].getOperationStatusCode() != OperationStatusCode.NOT_RUN) {
3541
3542 continue;
3543 }
3544
3545 Mutation[] cpMutations = miniBatchOp.getOperationsFromCoprocessors(i - firstIndex);
3546 if (cpMutations == null) {
3547 continue;
3548 }
3549 Mutation mutation = batchOp.getMutation(i);
3550 boolean skipWal = getEffectiveDurability(mutation.getDurability()) == Durability.SKIP_WAL;
3551
3552 for (int j = 0; j < cpMutations.length; j++) {
3553 Mutation cpMutation = cpMutations[j];
3554 Map<byte[], List<Cell>> cpFamilyMap = cpMutation.getFamilyCellMap();
3555 rewriteCellTags(cpFamilyMap, mutation);
3556 checkAndPrepareMutation(cpMutation, isInReplay, cpFamilyMap, now);
3557
3558
3559 acquiredRowLocks.add(getRowLockInternal(cpMutation.getRow(), true, true, null));
3560
3561
3562
3563 mergeFamilyMaps(familyMaps[i], cpFamilyMap);
3564
3565
3566
3567
3568 if (!skipWal) {
3569 for (List<Cell> cells : cpFamilyMap.values()) {
3570 cellCount += cells.size();
3571 }
3572 }
3573 }
3574 }
3575 }
3576 }
3577
3578
3579
3580
3581
3582
3583
3584
3585 checkInterrupt();
3586
3587 walEdit = new WALEdit(cellCount, isInReplay);
3588 Durability durability = Durability.USE_DEFAULT;
3589 for (int i = firstIndex; i < lastIndexExclusive; i++) {
3590
3591 if (batchOp.retCodeDetails[i].getOperationStatusCode() != OperationStatusCode.NOT_RUN) {
3592 continue;
3593 }
3594
3595 Mutation m = batchOp.getMutation(i);
3596 Durability tmpDur = getEffectiveDurability(m.getDurability());
3597 if (tmpDur.ordinal() > durability.ordinal()) {
3598 durability = tmpDur;
3599 }
3600
3601 if (tmpDur == Durability.SKIP_WAL) {
3602 recordMutationWithoutWal(m.getFamilyCellMap());
3603 continue;
3604 }
3605
3606 long nonceGroup = batchOp.getNonceGroup(i), nonce = batchOp.getNonce(i);
3607
3608
3609
3610 if (nonceGroup != currentNonceGroup || nonce != currentNonce) {
3611 if (walEdit.size() > 0) {
3612 if (!isInReplay) {
3613 throw new IOException("Multiple nonces per batch and not in replay");
3614 }
3615
3616
3617 walKey = new ReplayHLogKey(this.getRegionInfo().getEncodedNameAsBytes(),
3618 this.htableDescriptor.getTableName(), now, m.getClusterIds(),
3619 currentNonceGroup, currentNonce, mvcc);
3620 txid = this.wal.append(this.htableDescriptor, this.getRegionInfo(), walKey,
3621 walEdit, true);
3622 walEdit = new WALEdit(cellCount, isInReplay);
3623 walKey = null;
3624 }
3625 currentNonceGroup = nonceGroup;
3626 currentNonce = nonce;
3627 }
3628
3629
3630 WALEdit fromCP = batchOp.walEditsFromCoprocessors[i];
3631 if (fromCP != null) {
3632 for (Cell cell : fromCP.getCells()) {
3633 walEdit.add(cell);
3634 }
3635 }
3636 addFamilyMapToWALEdit(familyMaps[i], walEdit);
3637 }
3638
3639
3640
3641
3642
3643 Mutation mutation = batchOp.getMutation(firstIndex);
3644 if (isInReplay) {
3645
3646 walKey = new ReplayHLogKey(this.getRegionInfo().getEncodedNameAsBytes(),
3647 this.htableDescriptor.getTableName(), WALKey.NO_SEQUENCE_ID, now,
3648 mutation.getClusterIds(), currentNonceGroup, currentNonce, mvcc);
3649 long replaySeqId = batchOp.getReplaySequenceId();
3650 walKey.setOrigLogSeqNum(replaySeqId);
3651 if (walEdit.size() > 0) {
3652 txid =
3653 this.wal.append(this.htableDescriptor, this.getRegionInfo(), walKey, walEdit, true);
3654 }
3655 } else {
3656 if (walEdit.size() > 0) {
3657
3658 walKey = new HLogKey(this.getRegionInfo().getEncodedNameAsBytes(),
3659 this.htableDescriptor.getTableName(), WALKey.NO_SEQUENCE_ID, now,
3660 mutation.getClusterIds(), currentNonceGroup, currentNonce, mvcc);
3661 preWALAppend(walKey, walEdit);
3662 txid = this.wal
3663 .append(this.htableDescriptor, this.getRegionInfo(), walKey,
3664 walEdit, true);
3665 } else {
3666 walKey = appendEmptyEdit(wal);
3667 }
3668 }
3669
3670
3671
3672 if (!isInReplay) {
3673 if (writeEntry == null) {
3674
3675 writeEntry = walKey.getWriteEntry();
3676 }
3677 mvccNum = writeEntry.getWriteNumber();
3678 } else {
3679 mvccNum = batchOp.getReplaySequenceId();
3680 }
3681
3682
3683
3684
3685
3686
3687
3688
3689
3690
3691 for (int i = firstIndex; i < lastIndexExclusive; i++) {
3692 if (batchOp.retCodeDetails[i].getOperationStatusCode()
3693 != OperationStatusCode.NOT_RUN) {
3694 continue;
3695 }
3696
3697
3698
3699
3700
3701
3702
3703 boolean updateSeqId = isInReplay
3704 || batchOp.getMutation(i).getDurability() == Durability.SKIP_WAL;
3705 if (updateSeqId) {
3706 updateSequenceId(familyMaps[i].values(), mvccNum);
3707 }
3708 addedSize += applyFamilyMapToMemstore(familyMaps[i]);
3709 }
3710
3711
3712
3713
3714 if (locked) {
3715 this.updatesLock.readLock().unlock();
3716 locked = false;
3717 }
3718 releaseRowLocks(acquiredRowLocks);
3719
3720
3721
3722
3723 walSyncSuccess = false;
3724 if (txid != 0) {
3725 syncOrDefer(txid, durability);
3726 }
3727 walSyncSuccess = true;
3728
3729
3730 this.addAndGetGlobalMemstoreSize(addedSize);
3731
3732
3733 if (!isInReplay && coprocessorHost != null) {
3734 MiniBatchOperationInProgress<Mutation> miniBatchOp =
3735 new MiniBatchOperationInProgress<Mutation>(batchOp.getMutationsForCoprocs(),
3736 batchOp.retCodeDetails, batchOp.walEditsFromCoprocessors, firstIndex, lastIndexExclusive);
3737 coprocessorHost.postBatchMutate(miniBatchOp);
3738 }
3739
3740
3741
3742
3743 if (writeEntry != null) {
3744 mvcc.completeAndWait(writeEntry);
3745 writeEntry = null;
3746 } else if (isInReplay) {
3747
3748 mvcc.advanceTo(mvccNum);
3749 }
3750
3751 for (int i = firstIndex; i < lastIndexExclusive; i ++) {
3752 if (batchOp.retCodeDetails[i] == OperationStatus.NOT_RUN) {
3753 batchOp.retCodeDetails[i] = OperationStatus.SUCCESS;
3754 }
3755 }
3756
3757
3758
3759
3760
3761 if (!isInReplay && coprocessorHost != null) {
3762 for (int i = firstIndex; i < lastIndexExclusive; i++) {
3763
3764 if (batchOp.retCodeDetails[i].getOperationStatusCode()
3765 != OperationStatusCode.SUCCESS) {
3766 continue;
3767 }
3768 Mutation m = batchOp.getMutation(i);
3769 if (m instanceof Put) {
3770 coprocessorHost.postPut((Put) m, walEdit, m.getDurability());
3771 } else {
3772 coprocessorHost.postDelete((Delete) m, walEdit, m.getDurability());
3773 }
3774 }
3775 }
3776
3777 success = true;
3778 return addedSize;
3779 } catch (Throwable t) {
3780
3781
3782 if (!walSyncSuccess) {
3783 rsServices.abort("WAL sync failed, aborting to preserve WAL as source of truth", t);
3784 }
3785
3786 throw t;
3787 } finally {
3788
3789 if (!walSyncSuccess) {
3790 if (writeEntry != null) mvcc.complete(writeEntry);
3791 } else {
3792 if (writeEntry != null) {
3793 mvcc.completeAndWait(writeEntry);
3794 }
3795 }
3796
3797 if (locked) {
3798 this.updatesLock.readLock().unlock();
3799 }
3800 releaseRowLocks(acquiredRowLocks);
3801
3802
3803
3804
3805
3806
3807
3808 if (noOfPuts > 0) {
3809
3810 if (this.metricsRegion != null) {
3811 this.metricsRegion.updatePut();
3812 }
3813 }
3814 if (noOfDeletes > 0) {
3815
3816 if (this.metricsRegion != null) {
3817 this.metricsRegion.updateDelete();
3818 }
3819 }
3820
3821 enableInterrupts();
3822
3823 if (!success) {
3824 for (int i = firstIndex; i < lastIndexExclusive; i++) {
3825 if (batchOp.retCodeDetails[i].getOperationStatusCode() == OperationStatusCode.NOT_RUN) {
3826 batchOp.retCodeDetails[i] = OperationStatus.FAILURE;
3827 }
3828 }
3829 }
3830 if (coprocessorHost != null && !batchOp.isInReplay()) {
3831
3832
3833 MiniBatchOperationInProgress<Mutation> miniBatchOp =
3834 new MiniBatchOperationInProgress<Mutation>(batchOp.getMutationsForCoprocs(),
3835 batchOp.retCodeDetails, batchOp.walEditsFromCoprocessors, firstIndex, lastIndexExclusive);
3836 coprocessorHost.postBatchMutateIndispensably(miniBatchOp, success);
3837 }
3838
3839 batchOp.nextIndexToProcess = lastIndexExclusive;
3840 }
3841 }
3842
3843 private void mergeFamilyMaps(Map<byte[], List<Cell>> familyMap,
3844 Map<byte[], List<Cell>> toBeMerged) {
3845 for (Map.Entry<byte[], List<Cell>> entry : toBeMerged.entrySet()) {
3846 List<Cell> cells = familyMap.get(entry.getKey());
3847 if (cells == null) {
3848 familyMap.put(entry.getKey(), entry.getValue());
3849 } else {
3850 cells.addAll(entry.getValue());
3851 }
3852 }
3853 }
3854
3855
3856
3857
3858
3859 protected Durability getEffectiveDurability(Durability d) {
3860 return d == Durability.USE_DEFAULT ? this.durability : d;
3861 }
3862
3863
3864
3865
3866
3867
3868 @Override
3869 public boolean checkAndMutate(byte [] row, byte [] family, byte [] qualifier,
3870 CompareOp compareOp, ByteArrayComparable comparator, Mutation w,
3871 boolean writeToWAL)
3872 throws IOException{
3873 checkReadOnly();
3874
3875
3876 checkResources();
3877 boolean isPut = w instanceof Put;
3878 if (!isPut && !(w instanceof Delete))
3879 throw new org.apache.hadoop.hbase.DoNotRetryIOException("Action must " +
3880 "be Put or Delete");
3881 if (!Bytes.equals(row, w.getRow())) {
3882 throw new org.apache.hadoop.hbase.DoNotRetryIOException("Action's " +
3883 "getRow must match the passed row");
3884 }
3885
3886 startRegionOperation();
3887 try {
3888 Get get = new Get(row);
3889 checkFamily(family);
3890 get.addColumn(family, qualifier);
3891 checkRow(row, "checkAndMutate");
3892
3893 RowLock rowLock = getRowLockInternal(get.getRow());
3894 try {
3895
3896 mvcc.await();
3897 if (this.getCoprocessorHost() != null) {
3898 Boolean processed = null;
3899 if (w instanceof Put) {
3900 processed = this.getCoprocessorHost().preCheckAndPutAfterRowLock(row, family,
3901 qualifier, compareOp, comparator, (Put) w);
3902 } else if (w instanceof Delete) {
3903 processed = this.getCoprocessorHost().preCheckAndDeleteAfterRowLock(row, family,
3904 qualifier, compareOp, comparator, (Delete) w);
3905 }
3906 if (processed != null) {
3907 return processed;
3908 }
3909 }
3910 List<Cell> result = get(get, false);
3911
3912 boolean valueIsNull = comparator.getValue() == null ||
3913 comparator.getValue().length == 0;
3914 boolean matches = false;
3915 long cellTs = 0;
3916 if (result.size() == 0 && valueIsNull) {
3917 matches = true;
3918 } else if (result.size() > 0 && result.get(0).getValueLength() == 0 &&
3919 valueIsNull) {
3920 matches = true;
3921 cellTs = result.get(0).getTimestamp();
3922 } else if (result.size() == 1 && !valueIsNull) {
3923 Cell kv = result.get(0);
3924 cellTs = kv.getTimestamp();
3925 int compareResult = comparator.compareTo(kv.getValueArray(),
3926 kv.getValueOffset(), kv.getValueLength());
3927 switch (compareOp) {
3928 case LESS:
3929 matches = compareResult < 0;
3930 break;
3931 case LESS_OR_EQUAL:
3932 matches = compareResult <= 0;
3933 break;
3934 case EQUAL:
3935 matches = compareResult == 0;
3936 break;
3937 case NOT_EQUAL:
3938 matches = compareResult != 0;
3939 break;
3940 case GREATER_OR_EQUAL:
3941 matches = compareResult >= 0;
3942 break;
3943 case GREATER:
3944 matches = compareResult > 0;
3945 break;
3946 default:
3947 throw new RuntimeException("Unknown Compare op " + compareOp.name());
3948 }
3949 }
3950
3951 if (matches) {
3952
3953
3954
3955
3956 long now = EnvironmentEdgeManager.currentTime();
3957 long ts = Math.max(now, cellTs);
3958 byte[] byteTs = Bytes.toBytes(ts);
3959
3960 if (w instanceof Put) {
3961 updateCellTimestamps(w.getFamilyCellMap().values(), byteTs);
3962 }
3963
3964
3965
3966
3967
3968 doBatchMutate(w);
3969 this.checkAndMutateChecksPassed.increment();
3970 return true;
3971 }
3972 this.checkAndMutateChecksFailed.increment();
3973 return false;
3974 } finally {
3975 rowLock.release();
3976 }
3977 } finally {
3978 closeRegionOperation();
3979 }
3980 }
3981
3982
3983
3984
3985
3986
3987 @Override
3988 public boolean checkAndRowMutate(byte [] row, byte [] family, byte [] qualifier,
3989 CompareOp compareOp, ByteArrayComparable comparator, RowMutations rm,
3990 boolean writeToWAL) throws IOException {
3991 checkReadOnly();
3992
3993
3994 checkResources();
3995
3996 startRegionOperation();
3997 try {
3998 Get get = new Get(row);
3999 checkFamily(family);
4000 get.addColumn(family, qualifier);
4001 checkRow(row, "checkAndRowMutate");
4002
4003 RowLock rowLock = getRowLockInternal(get.getRow());
4004 try {
4005
4006 mvcc.await();
4007
4008 List<Cell> result = get(get, false);
4009
4010 boolean valueIsNull = comparator.getValue() == null ||
4011 comparator.getValue().length == 0;
4012 boolean matches = false;
4013 long cellTs = 0;
4014 if (result.size() == 0 && valueIsNull) {
4015 matches = true;
4016 } else if (result.size() > 0 && result.get(0).getValueLength() == 0 &&
4017 valueIsNull) {
4018 matches = true;
4019 cellTs = result.get(0).getTimestamp();
4020 } else if (result.size() == 1 && !valueIsNull) {
4021 Cell kv = result.get(0);
4022 cellTs = kv.getTimestamp();
4023 int compareResult = comparator.compareTo(kv.getValueArray(),
4024 kv.getValueOffset(), kv.getValueLength());
4025 switch (compareOp) {
4026 case LESS:
4027 matches = compareResult < 0;
4028 break;
4029 case LESS_OR_EQUAL:
4030 matches = compareResult <= 0;
4031 break;
4032 case EQUAL:
4033 matches = compareResult == 0;
4034 break;
4035 case NOT_EQUAL:
4036 matches = compareResult != 0;
4037 break;
4038 case GREATER_OR_EQUAL:
4039 matches = compareResult >= 0;
4040 break;
4041 case GREATER:
4042 matches = compareResult > 0;
4043 break;
4044 default:
4045 throw new RuntimeException("Unknown Compare op " + compareOp.name());
4046 }
4047 }
4048
4049 if (matches) {
4050
4051
4052
4053
4054 long now = EnvironmentEdgeManager.currentTime();
4055 long ts = Math.max(now, cellTs);
4056 byte[] byteTs = Bytes.toBytes(ts);
4057
4058 for (Mutation w : rm.getMutations()) {
4059 if (w instanceof Put) {
4060 updateCellTimestamps(w.getFamilyCellMap().values(), byteTs);
4061 }
4062
4063
4064 }
4065
4066
4067
4068 mutateRow(rm);
4069 this.checkAndMutateChecksPassed.increment();
4070 return true;
4071 }
4072 this.checkAndMutateChecksFailed.increment();
4073 return false;
4074 } finally {
4075 rowLock.release();
4076 }
4077 } finally {
4078 closeRegionOperation();
4079 }
4080 }
4081
4082 private void doBatchMutate(Mutation mutation) throws IOException {
4083
4084 OperationStatus[] batchMutate = this.batchMutate(new Mutation[]{mutation});
4085 if (batchMutate[0].getOperationStatusCode().equals(OperationStatusCode.SANITY_CHECK_FAILURE)) {
4086 throw new FailedSanityCheckException(batchMutate[0].getExceptionMsg());
4087 } else if (batchMutate[0].getOperationStatusCode().equals(OperationStatusCode.BAD_FAMILY)) {
4088 throw new NoSuchColumnFamilyException(batchMutate[0].getExceptionMsg());
4089 }
4090 }
4091
4092
4093
4094
4095
4096
4097
4098
4099
4100
4101
4102
4103
4104
4105 public void addRegionToSnapshot(SnapshotDescription desc,
4106 ForeignExceptionSnare exnSnare) throws IOException {
4107 Path rootDir = FSUtils.getRootDir(conf);
4108 Path snapshotDir = SnapshotDescriptionUtils.getWorkingSnapshotDir(desc, rootDir, conf);
4109
4110 SnapshotManifest manifest = SnapshotManifest.create(conf, getFilesystem(),
4111 snapshotDir, desc, exnSnare);
4112 manifest.addRegion(this);
4113 }
4114
4115 private void updateSequenceId(final Iterable<List<Cell>> cellItr, final long sequenceId)
4116 throws IOException {
4117 for (List<Cell> cells : cellItr) {
4118 if (cells == null) {
4119 return;
4120 }
4121 for (Cell cell : cells) {
4122 CellUtil.setSequenceId(cell, sequenceId);
4123 }
4124 }
4125 }
4126
4127 @Override
4128 public void updateCellTimestamps(final Iterable<List<Cell>> cellItr, final byte[] now)
4129 throws IOException {
4130 for (List<Cell> cells: cellItr) {
4131 if (cells == null) continue;
4132 assert cells instanceof RandomAccess;
4133 int listSize = cells.size();
4134 for (int i = 0; i < listSize; i++) {
4135 CellUtil.updateLatestStamp(cells.get(i), now, 0);
4136 }
4137 }
4138 }
4139
4140
4141
4142
4143 void rewriteCellTags(Map<byte[], List<Cell>> familyMap, final Mutation m) {
4144
4145
4146
4147 if (m.getTTL() == Long.MAX_VALUE) {
4148 return;
4149 }
4150
4151
4152
4153 for (Map.Entry<byte[], List<Cell>> e: familyMap.entrySet()) {
4154 List<Cell> cells = e.getValue();
4155 assert cells instanceof RandomAccess;
4156 int listSize = cells.size();
4157 for (int i = 0; i < listSize; i++) {
4158 Cell cell = cells.get(i);
4159 List<Tag> newTags = Tag.carryForwardTags(null, cell);
4160 newTags = carryForwardTTLTag(newTags, m);
4161
4162
4163 cells.set(i, new KeyValue(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength(),
4164 cell.getFamilyArray(), cell.getFamilyOffset(), cell.getFamilyLength(),
4165 cell.getQualifierArray(), cell.getQualifierOffset(), cell.getQualifierLength(),
4166 cell.getTimestamp(), KeyValue.Type.codeToType(cell.getTypeByte()),
4167 cell.getValueArray(), cell.getValueOffset(), cell.getValueLength(),
4168 newTags));
4169 }
4170 }
4171 }
4172
4173
4174
4175
4176
4177
4178
4179 private void checkResources() throws RegionTooBusyException {
4180
4181 if (this.getRegionInfo().isMetaRegion()) return;
4182
4183 if (this.memstoreSize.get() > this.blockingMemStoreSize) {
4184 blockedRequestsCount.increment();
4185 requestFlush();
4186 final String regionName =
4187 this.getRegionInfo() == null ? "unknown" : this.getRegionInfo().getRegionNameAsString();
4188 final String serverName = this.getRegionServerServices() == null ?
4189 "unknown" : (this.getRegionServerServices().getServerName() == null ? "unknown" :
4190 this.getRegionServerServices().getServerName().toString());
4191 RegionTooBusyException rtbe = new RegionTooBusyException(
4192 "Above memstore limit, " + "regionName=" + regionName + ", server=" + serverName
4193 + ", memstoreSize=" + memstoreSize.get()
4194 + ", blockingMemStoreSize=" + blockingMemStoreSize);
4195 LOG.warn("Region is too busy due to exceeding memstore size limit.", rtbe);
4196 throw rtbe;
4197 }
4198 }
4199
4200
4201
4202
4203 protected void checkReadOnly() throws IOException {
4204 if (isReadOnly()) {
4205 throw new DoNotRetryIOException("region is read only");
4206 }
4207 }
4208
4209 protected void checkReadsEnabled() throws IOException {
4210 if (!this.writestate.readsEnabled) {
4211 throw new IOException(getRegionInfo().getEncodedName()
4212 + ": The region's reads are disabled. Cannot serve the request");
4213 }
4214 }
4215
4216 public void setReadsEnabled(boolean readsEnabled) {
4217 if (readsEnabled && !this.writestate.readsEnabled) {
4218 LOG.info(getRegionInfo().getEncodedName() + " : Enabling reads for region.");
4219 }
4220 this.writestate.setReadsEnabled(readsEnabled);
4221 }
4222
4223
4224
4225
4226
4227
4228
4229 private void put(final byte [] row, byte [] family, List<Cell> edits)
4230 throws IOException {
4231 NavigableMap<byte[], List<Cell>> familyMap;
4232 familyMap = new TreeMap<byte[], List<Cell>>(Bytes.BYTES_COMPARATOR);
4233
4234 familyMap.put(family, edits);
4235 Put p = new Put(row);
4236 p.setFamilyCellMap(familyMap);
4237 doBatchMutate(p);
4238 }
4239
4240
4241
4242
4243
4244
4245
4246
4247
4248
4249
4250
4251 private long applyFamilyMapToMemstore(Map<byte[], List<Cell>> familyMap) throws IOException {
4252 long size = 0;
4253
4254 for (Map.Entry<byte[], List<Cell>> e : familyMap.entrySet()) {
4255 byte[] family = e.getKey();
4256 List<Cell> cells = e.getValue();
4257 assert cells instanceof RandomAccess;
4258 Store store = getStore(family);
4259 size += store.add(cells);
4260 }
4261 return size;
4262 }
4263
4264 @Override
4265 public void checkFamilies(Collection<byte[]> families) throws NoSuchColumnFamilyException {
4266 for (byte[] family : families) {
4267 checkFamily(family);
4268 }
4269 }
4270
4271 private void checkAndPrepareMutation(Mutation mutation, boolean replay,
4272 final Map<byte[], List<Cell>> familyMap, final long now)
4273 throws IOException {
4274 if (mutation instanceof Put) {
4275
4276 if (replay) {
4277 removeNonExistentColumnFamilyForReplay(familyMap);
4278 } else {
4279 checkFamilies(familyMap.keySet());
4280 }
4281 checkTimestamps(mutation.getFamilyCellMap(), now);
4282 } else {
4283 prepareDelete((Delete)mutation);
4284 }
4285 checkRow(mutation.getRow(), "doMiniBatchMutation");
4286 }
4287
4288
4289
4290
4291
4292 private void removeNonExistentColumnFamilyForReplay(
4293 final Map<byte[], List<Cell>> familyMap) {
4294 List<byte[]> nonExistentList = null;
4295 for (byte[] family : familyMap.keySet()) {
4296 if (!this.htableDescriptor.hasFamily(family)) {
4297 if (nonExistentList == null) {
4298 nonExistentList = new ArrayList<byte[]>();
4299 }
4300 nonExistentList.add(family);
4301 }
4302 }
4303 if (nonExistentList != null) {
4304 for (byte[] family : nonExistentList) {
4305
4306 LOG.info("No family for " + Bytes.toString(family) + " omit from replay.");
4307 familyMap.remove(family);
4308 }
4309 }
4310 }
4311
4312 @Override
4313 public void checkTimestamps(final Map<byte[], List<Cell>> familyMap, long now)
4314 throws FailedSanityCheckException {
4315 if (timestampSlop == HConstants.LATEST_TIMESTAMP) {
4316 return;
4317 }
4318 long maxTs = now + timestampSlop;
4319 for (List<Cell> kvs : familyMap.values()) {
4320 assert kvs instanceof RandomAccess;
4321 int listSize = kvs.size();
4322 for (int i=0; i < listSize; i++) {
4323 Cell cell = kvs.get(i);
4324
4325 long ts = cell.getTimestamp();
4326 if (ts != HConstants.LATEST_TIMESTAMP && ts > maxTs) {
4327 throw new FailedSanityCheckException("Timestamp for KV out of range "
4328 + cell + " (too.new=" + timestampSlop + ")");
4329 }
4330 }
4331 }
4332 }
4333
4334
4335
4336
4337
4338
4339
4340 private void addFamilyMapToWALEdit(Map<byte[], List<Cell>> familyMap,
4341 WALEdit walEdit) {
4342 for (List<Cell> edits : familyMap.values()) {
4343 assert edits instanceof RandomAccess;
4344 int listSize = edits.size();
4345 for (int i=0; i < listSize; i++) {
4346 Cell cell = edits.get(i);
4347 walEdit.add(cell);
4348 }
4349 }
4350 }
4351
4352 private void requestFlush() {
4353 if (this.rsServices == null) {
4354 return;
4355 }
4356 synchronized (writestate) {
4357 if (this.writestate.isFlushRequested()) {
4358 return;
4359 }
4360 writestate.flushRequested = true;
4361 }
4362
4363 this.rsServices.getFlushRequester().requestFlush(this, false);
4364 if (LOG.isDebugEnabled()) {
4365 LOG.debug("Flush requested on " + this.getRegionInfo().getEncodedName());
4366 }
4367 }
4368
4369
4370
4371
4372
4373 private boolean isFlushSize(final long size) {
4374 return size > this.memstoreFlushSize;
4375 }
4376
4377
4378
4379
4380
4381
4382
4383
4384
4385
4386
4387
4388
4389
4390
4391
4392
4393
4394
4395
4396
4397
4398
4399
4400
4401
4402
4403
4404
4405
4406
4407
4408
4409
4410 protected long replayRecoveredEditsIfAny(Map<byte[], Long> maxSeqIdInStores,
4411 final CancelableProgressable reporter, final MonitoredTask status)
4412 throws IOException {
4413 long minSeqIdForTheRegion = -1;
4414 for (Long maxSeqIdInStore : maxSeqIdInStores.values()) {
4415 if (maxSeqIdInStore < minSeqIdForTheRegion || minSeqIdForTheRegion == -1) {
4416 minSeqIdForTheRegion = maxSeqIdInStore;
4417 }
4418 }
4419 long seqId = minSeqIdForTheRegion;
4420
4421 FileSystem walFS = getWalFileSystem();
4422 FileSystem rootFS = getFilesystem();
4423 Path regionDir = FSUtils.getRegionDirFromRootDir(FSUtils.getRootDir(conf), getRegionInfo());
4424 Path regionWALDir = getWALRegionDir();
4425 Path wrongRegionWALDir = FSUtils.getWrongWALRegionDir(conf, getRegionInfo().getTable(),
4426 getRegionInfo().getEncodedName());
4427
4428
4429 NavigableSet<Path> filesUnderWrongRegionWALDir =
4430 WALSplitter.getSplitEditFilesSorted(walFS, wrongRegionWALDir);
4431 seqId = Math.max(seqId, replayRecoveredEditsForPaths(minSeqIdForTheRegion, walFS,
4432 filesUnderWrongRegionWALDir, reporter, regionDir));
4433
4434
4435 NavigableSet<Path> filesUnderRootDir = Sets.newTreeSet();
4436 if (!regionWALDir.equals(regionDir)) {
4437 filesUnderRootDir = WALSplitter.getSplitEditFilesSorted(rootFS, regionDir);
4438 seqId = Math.max(seqId, replayRecoveredEditsForPaths(minSeqIdForTheRegion, rootFS,
4439 filesUnderRootDir, reporter, regionDir));
4440 }
4441 NavigableSet<Path> files = WALSplitter.getSplitEditFilesSorted(walFS, regionWALDir);
4442 seqId = Math.max(seqId, replayRecoveredEditsForPaths(minSeqIdForTheRegion, walFS,
4443 files, reporter, regionWALDir));
4444 if (seqId > minSeqIdForTheRegion) {
4445
4446 internalFlushcache(null, seqId, stores.values(), status, false);
4447 }
4448
4449 if (conf.getBoolean("hbase.region.archive.recovered.edits", false)) {
4450
4451
4452
4453 String fakeFamilyName = WALSplitter.getRegionDirRecoveredEditsDir(regionWALDir).getName();
4454 Set<StoreFile> fakeStoreFiles = new HashSet<>();
4455 for (Path file: Iterables.concat(files, filesUnderWrongRegionWALDir)) {
4456 fakeStoreFiles.add(new StoreFile(walFS, file, conf, null, null));
4457 }
4458 for (Path file: filesUnderRootDir) {
4459 fakeStoreFiles.add(new StoreFile(rootFS, file, conf, null, null));
4460 }
4461 getRegionWALFileSystem().removeStoreFiles(fakeFamilyName, fakeStoreFiles);
4462 } else {
4463 for (Path file : filesUnderRootDir) {
4464 if (!rootFS.delete(file, false)) {
4465 LOG.error("Failed delete of " + file + " from under the root directory");
4466 } else {
4467 LOG.debug("Deleted recovered.edits under root directory, file=" + file);
4468 }
4469 }
4470 for (Path file : Iterables.concat(files, filesUnderWrongRegionWALDir)) {
4471 if (!walFS.delete(file, false)) {
4472 LOG.error("Failed delete of " + file);
4473 } else {
4474 LOG.debug("Deleted recovered.edits file=" + file);
4475 }
4476 }
4477 }
4478
4479
4480
4481 FileSystem walFs = getWalFileSystem();
4482 if (walFs.exists(wrongRegionWALDir)) {
4483 if (!walFs.delete(wrongRegionWALDir, true)) {
4484 LOG.warn("Unable to delete " + wrongRegionWALDir);
4485 }
4486 }
4487
4488 return seqId;
4489 }
4490
4491 private long replayRecoveredEditsForPaths(long minSeqIdForTheRegion, FileSystem fs,
4492 final NavigableSet<Path> files, final CancelableProgressable reporter, final Path regionDir)
4493 throws IOException {
4494 long seqid = minSeqIdForTheRegion;
4495 if (LOG.isDebugEnabled()) {
4496 LOG.debug("Found " + (files == null ? 0 : files.size()) +
4497 " recovered edits file(s) under " + regionDir);
4498 }
4499
4500 if (files == null || files.isEmpty()) {
4501 return seqid;
4502 }
4503
4504 for (Path edits : files) {
4505 if (edits == null || !fs.exists(edits)) {
4506 LOG.warn("Null or non-existent edits file: " + edits);
4507 continue;
4508 }
4509 if (isZeroLengthThenDelete(fs, edits)) {
4510 continue;
4511 }
4512
4513 long maxSeqId;
4514 String fileName = edits.getName();
4515 maxSeqId = Math.abs(Long.parseLong(fileName));
4516 if (maxSeqId <= minSeqIdForTheRegion) {
4517 if (LOG.isDebugEnabled()) {
4518 String msg = "Maximum sequenceid for this wal is " + maxSeqId +
4519 " and minimum sequenceid for the region is " + minSeqIdForTheRegion
4520 + ", skipped the whole file, path=" + edits;
4521 LOG.debug(msg);
4522 }
4523 continue;
4524 }
4525
4526 try {
4527
4528
4529 seqid = Math.max(seqid, replayRecoveredEdits(edits, maxSeqIdInStores, reporter, fs));
4530 } catch (IOException e) {
4531 boolean skipErrors = conf.getBoolean(HConstants.HREGION_EDITS_REPLAY_SKIP_ERRORS,
4532 conf.getBoolean("hbase.skip.errors",
4533 HConstants.DEFAULT_HREGION_EDITS_REPLAY_SKIP_ERRORS));
4534 if (conf.get("hbase.skip.errors") != null) {
4535 LOG.warn("The property 'hbase.skip.errors' has been deprecated. Please use " +
4536 HConstants.HREGION_EDITS_REPLAY_SKIP_ERRORS + " instead.");
4537 }
4538 if (skipErrors) {
4539 Path p = WALSplitter.moveAsideBadEditsFile(fs, edits);
4540 LOG.error(HConstants.HREGION_EDITS_REPLAY_SKIP_ERRORS +
4541 "=true so continuing. Renamed " + edits + " as " + p, e);
4542 } else {
4543 throw e;
4544 }
4545 }
4546 }
4547 return seqid;
4548 }
4549
4550
4551
4552
4553
4554
4555
4556
4557
4558
4559 private long replayRecoveredEdits(final Path edits, Map<byte[], Long> maxSeqIdInStores,
4560 final CancelableProgressable reporter, final FileSystem fs)
4561 throws IOException {
4562 String msg = "Replaying edits from " + edits;
4563 LOG.info(msg);
4564 MonitoredTask status = TaskMonitor.get().createStatus(msg);
4565
4566 status.setStatus("Opening recovered edits");
4567 WAL.Reader reader = null;
4568 try {
4569 reader = WALFactory.createReader(fs, edits, conf);
4570 long currentEditSeqId = -1;
4571 long currentReplaySeqId = -1;
4572 long firstSeqIdInLog = -1;
4573 long skippedEdits = 0;
4574 long editsCount = 0;
4575 long intervalEdits = 0;
4576 WAL.Entry entry;
4577 Store store = null;
4578 boolean reported_once = false;
4579 ServerNonceManager ng = this.rsServices == null ? null : this.rsServices.getNonceManager();
4580
4581 try {
4582
4583 int interval = this.conf.getInt("hbase.hstore.report.interval.edits", 2000);
4584
4585 int period = this.conf.getInt("hbase.hstore.report.period", 300000);
4586 long lastReport = EnvironmentEdgeManager.currentTime();
4587
4588 while ((entry = reader.next()) != null) {
4589 WALKey key = entry.getKey();
4590 WALEdit val = entry.getEdit();
4591
4592 if (ng != null) {
4593 ng.reportOperationFromWal(key.getNonceGroup(), key.getNonce(), key.getWriteTime());
4594 }
4595
4596 if (reporter != null) {
4597 intervalEdits += val.size();
4598 if (intervalEdits >= interval) {
4599
4600 intervalEdits = 0;
4601 long cur = EnvironmentEdgeManager.currentTime();
4602 if (lastReport + period <= cur) {
4603 status.setStatus("Replaying edits..." +
4604 " skipped=" + skippedEdits +
4605 " edits=" + editsCount);
4606
4607 if(!reporter.progress()) {
4608 msg = "Progressable reporter failed, stopping replay";
4609 LOG.warn(msg);
4610 status.abort(msg);
4611 throw new IOException(msg);
4612 }
4613 reported_once = true;
4614 lastReport = cur;
4615 }
4616 }
4617 }
4618
4619 if (firstSeqIdInLog == -1) {
4620 firstSeqIdInLog = key.getLogSeqNum();
4621 }
4622 if (currentEditSeqId > key.getLogSeqNum()) {
4623
4624
4625 LOG.error(getRegionInfo().getEncodedName() + " : "
4626 + "Found decreasing SeqId. PreId=" + currentEditSeqId + " key=" + key
4627 + "; edit=" + val);
4628 } else {
4629 currentEditSeqId = key.getLogSeqNum();
4630 }
4631 currentReplaySeqId = (key.getOrigLogSeqNum() > 0) ?
4632 key.getOrigLogSeqNum() : currentEditSeqId;
4633
4634
4635
4636 if (coprocessorHost != null) {
4637 status.setStatus("Running pre-WAL-restore hook in coprocessors");
4638 if (coprocessorHost.preWALRestore(this.getRegionInfo(), key, val)) {
4639
4640 continue;
4641 }
4642 }
4643 boolean checkRowWithinBoundary = false;
4644
4645 if (!Bytes.equals(key.getEncodedRegionName(),
4646 this.getRegionInfo().getEncodedNameAsBytes())) {
4647 checkRowWithinBoundary = true;
4648 }
4649
4650 boolean flush = false;
4651 for (Cell cell: val.getCells()) {
4652
4653
4654 if (CellUtil.matchingFamily(cell, WALEdit.METAFAMILY)) {
4655
4656 if (!checkRowWithinBoundary) {
4657
4658 CompactionDescriptor compaction = WALEdit.getCompaction(cell);
4659 if (compaction != null) {
4660
4661 replayWALCompactionMarker(compaction, false, true, Long.MAX_VALUE);
4662 }
4663 }
4664 skippedEdits++;
4665 continue;
4666 }
4667
4668 if (store == null || !CellUtil.matchingFamily(cell, store.getFamily().getName())) {
4669 store = getStore(cell);
4670 }
4671 if (store == null) {
4672
4673
4674 LOG.warn("No family for " + cell);
4675 skippedEdits++;
4676 continue;
4677 }
4678 if (checkRowWithinBoundary && !rowIsInRange(this.getRegionInfo(),
4679 cell.getRowArray(), cell.getRowOffset(), cell.getRowLength())) {
4680 LOG.warn("Row of " + cell + " is not within region boundary");
4681 skippedEdits++;
4682 continue;
4683 }
4684
4685 if (key.getLogSeqNum() <= maxSeqIdInStores.get(store.getFamily()
4686 .getName())) {
4687 skippedEdits++;
4688 continue;
4689 }
4690 CellUtil.setSequenceId(cell, currentReplaySeqId);
4691
4692
4693
4694
4695 flush |= restoreEdit(store, cell);
4696 editsCount++;
4697 }
4698 if (flush) {
4699 internalFlushcache(null, currentEditSeqId, stores.values(), status, false);
4700 }
4701
4702 if (coprocessorHost != null) {
4703 coprocessorHost.postWALRestore(this.getRegionInfo(), key, val);
4704 }
4705 }
4706 } catch (EOFException eof) {
4707 Path p = WALSplitter.moveAsideBadEditsFile(fs, edits);
4708 msg = "Encountered EOF. Most likely due to Master failure during " +
4709 "wal splitting, so we have this data in another edit. " +
4710 "Continuing, but renaming " + edits + " as " + p;
4711 LOG.warn(msg, eof);
4712 status.abort(msg);
4713 } catch (IOException ioe) {
4714
4715
4716 if (ioe.getCause() instanceof ParseException) {
4717 Path p = WALSplitter.moveAsideBadEditsFile(fs, edits);
4718 msg = "File corruption encountered! " +
4719 "Continuing, but renaming " + edits + " as " + p;
4720 LOG.warn(msg, ioe);
4721 status.setStatus(msg);
4722 } else {
4723 status.abort(StringUtils.stringifyException(ioe));
4724
4725
4726 throw ioe;
4727 }
4728 }
4729 if (reporter != null && !reported_once) {
4730 reporter.progress();
4731 }
4732 msg = "Applied " + editsCount + ", skipped " + skippedEdits +
4733 ", firstSequenceIdInLog=" + firstSeqIdInLog +
4734 ", maxSequenceIdInLog=" + currentEditSeqId + ", path=" + edits;
4735 status.markComplete(msg);
4736 LOG.debug(msg);
4737 return currentEditSeqId;
4738 } finally {
4739 status.cleanup();
4740 if (reader != null) {
4741 reader.close();
4742 }
4743 }
4744 }
4745
4746
4747
4748
4749
4750
4751 void replayWALCompactionMarker(CompactionDescriptor compaction, boolean pickCompactionFiles,
4752 boolean removeFiles, long replaySeqId)
4753 throws IOException {
4754 try {
4755 checkTargetRegion(compaction.getEncodedRegionName().toByteArray(),
4756 "Compaction marker from WAL ", compaction);
4757 } catch (WrongRegionException wre) {
4758 if (RegionReplicaUtil.isDefaultReplica(this.getRegionInfo())) {
4759
4760 return;
4761 }
4762 throw wre;
4763 }
4764
4765 synchronized (writestate) {
4766 if (replaySeqId < lastReplayedOpenRegionSeqId) {
4767 LOG.warn(getRegionInfo().getEncodedName() + " : "
4768 + "Skipping replaying compaction event :" + TextFormat.shortDebugString(compaction)
4769 + " because its sequence id " + replaySeqId + " is smaller than this regions "
4770 + "lastReplayedOpenRegionSeqId of " + lastReplayedOpenRegionSeqId);
4771 return;
4772 }
4773 if (replaySeqId < lastReplayedCompactionSeqId) {
4774 LOG.warn(getRegionInfo().getEncodedName() + " : "
4775 + "Skipping replaying compaction event :" + TextFormat.shortDebugString(compaction)
4776 + " because its sequence id " + replaySeqId + " is smaller than this regions "
4777 + "lastReplayedCompactionSeqId of " + lastReplayedCompactionSeqId);
4778 return;
4779 } else {
4780 lastReplayedCompactionSeqId = replaySeqId;
4781 }
4782
4783 if (LOG.isDebugEnabled()) {
4784 LOG.debug(getRegionInfo().getEncodedName() + " : "
4785 + "Replaying compaction marker " + TextFormat.shortDebugString(compaction)
4786 + " with seqId=" + replaySeqId + " and lastReplayedOpenRegionSeqId="
4787 + lastReplayedOpenRegionSeqId);
4788 }
4789
4790 startRegionOperation(Operation.REPLAY_EVENT);
4791 try {
4792 Store store = this.getStore(compaction.getFamilyName().toByteArray());
4793 if (store == null) {
4794 LOG.warn(getRegionInfo().getEncodedName() + " : "
4795 + "Found Compaction WAL edit for deleted family:"
4796 + Bytes.toString(compaction.getFamilyName().toByteArray()));
4797 return;
4798 }
4799 store.replayCompactionMarker(compaction, pickCompactionFiles, removeFiles);
4800 logRegionFiles();
4801 } catch (FileNotFoundException ex) {
4802 LOG.warn(getRegionInfo().getEncodedName() + " : "
4803 + "At least one of the store files in compaction: "
4804 + TextFormat.shortDebugString(compaction)
4805 + " doesn't exist any more. Skip loading the file(s)", ex);
4806 } finally {
4807 closeRegionOperation(Operation.REPLAY_EVENT);
4808 }
4809 }
4810 }
4811
4812 void replayWALFlushMarker(FlushDescriptor flush, long replaySeqId) throws IOException {
4813 checkTargetRegion(flush.getEncodedRegionName().toByteArray(),
4814 "Flush marker from WAL ", flush);
4815
4816 if (ServerRegionReplicaUtil.isDefaultReplica(this.getRegionInfo())) {
4817 return;
4818 }
4819
4820 if (LOG.isDebugEnabled()) {
4821 LOG.debug(getRegionInfo().getEncodedName() + " : "
4822 + "Replaying flush marker " + TextFormat.shortDebugString(flush));
4823 }
4824
4825 startRegionOperation(Operation.REPLAY_EVENT);
4826 try {
4827 FlushAction action = flush.getAction();
4828 switch (action) {
4829 case START_FLUSH:
4830 replayWALFlushStartMarker(flush);
4831 break;
4832 case COMMIT_FLUSH:
4833 replayWALFlushCommitMarker(flush);
4834 break;
4835 case ABORT_FLUSH:
4836 replayWALFlushAbortMarker(flush);
4837 break;
4838 case CANNOT_FLUSH:
4839 replayWALFlushCannotFlushMarker(flush, replaySeqId);
4840 break;
4841 default:
4842 LOG.warn(getRegionInfo().getEncodedName() + " : " +
4843 "Received a flush event with unknown action, ignoring. " +
4844 TextFormat.shortDebugString(flush));
4845 break;
4846 }
4847
4848 logRegionFiles();
4849 } finally {
4850 closeRegionOperation(Operation.REPLAY_EVENT);
4851 }
4852 }
4853
4854
4855
4856
4857
4858 PrepareFlushResult replayWALFlushStartMarker(FlushDescriptor flush) throws IOException {
4859 long flushSeqId = flush.getFlushSequenceNumber();
4860
4861 HashSet<Store> storesToFlush = new HashSet<Store>();
4862 for (StoreFlushDescriptor storeFlush : flush.getStoreFlushesList()) {
4863 byte[] family = storeFlush.getFamilyName().toByteArray();
4864 Store store = getStore(family);
4865 if (store == null) {
4866 LOG.warn(getRegionInfo().getEncodedName() + " : "
4867 + "Received a flush start marker from primary, but the family is not found. Ignoring"
4868 + " StoreFlushDescriptor:" + TextFormat.shortDebugString(storeFlush));
4869 continue;
4870 }
4871 storesToFlush.add(store);
4872 }
4873
4874 MonitoredTask status = TaskMonitor.get().createStatus("Preparing flush " + this);
4875
4876
4877
4878 synchronized (writestate) {
4879 try {
4880 if (flush.getFlushSequenceNumber() < lastReplayedOpenRegionSeqId) {
4881 LOG.warn(getRegionInfo().getEncodedName() + " : "
4882 + "Skipping replaying flush event :" + TextFormat.shortDebugString(flush)
4883 + " because its sequence id is smaller than this regions lastReplayedOpenRegionSeqId "
4884 + " of " + lastReplayedOpenRegionSeqId);
4885 return null;
4886 }
4887 if (numMutationsWithoutWAL.get() > 0) {
4888 numMutationsWithoutWAL.set(0);
4889 dataInMemoryWithoutWAL.set(0);
4890 }
4891
4892 if (!writestate.flushing) {
4893
4894
4895
4896
4897 PrepareFlushResult prepareResult = internalPrepareFlushCache(null,
4898 flushSeqId, storesToFlush, status, false);
4899 if (prepareResult.result == null) {
4900
4901 this.writestate.flushing = true;
4902 this.prepareFlushResult = prepareResult;
4903 status.markComplete("Flush prepare successful");
4904 if (LOG.isDebugEnabled()) {
4905 LOG.debug(getRegionInfo().getEncodedName() + " : "
4906 + " Prepared flush with seqId:" + flush.getFlushSequenceNumber());
4907 }
4908 } else {
4909
4910
4911 if (prepareResult.getResult().getResult() ==
4912 FlushResult.Result.CANNOT_FLUSH_MEMSTORE_EMPTY) {
4913 this.writestate.flushing = true;
4914 this.prepareFlushResult = prepareResult;
4915 if (LOG.isDebugEnabled()) {
4916 LOG.debug(getRegionInfo().getEncodedName() + " : "
4917 + " Prepared empty flush with seqId:" + flush.getFlushSequenceNumber());
4918 }
4919 }
4920 status.abort("Flush prepare failed with " + prepareResult.result);
4921
4922 }
4923 return prepareResult;
4924 } else {
4925
4926 if (flush.getFlushSequenceNumber() == this.prepareFlushResult.flushOpSeqId) {
4927
4928 LOG.warn(getRegionInfo().getEncodedName() + " : "
4929 + "Received a flush prepare marker with the same seqId: " +
4930 + flush.getFlushSequenceNumber() + " before clearing the previous one with seqId: "
4931 + prepareFlushResult.flushOpSeqId + ". Ignoring");
4932
4933 } else if (flush.getFlushSequenceNumber() < this.prepareFlushResult.flushOpSeqId) {
4934
4935
4936 LOG.warn(getRegionInfo().getEncodedName() + " : "
4937 + "Received a flush prepare marker with a smaller seqId: " +
4938 + flush.getFlushSequenceNumber() + " before clearing the previous one with seqId: "
4939 + prepareFlushResult.flushOpSeqId + ". Ignoring");
4940
4941 } else {
4942
4943 LOG.warn(getRegionInfo().getEncodedName() + " : "
4944 + "Received a flush prepare marker with a larger seqId: " +
4945 + flush.getFlushSequenceNumber() + " before clearing the previous one with seqId: "
4946 + prepareFlushResult.flushOpSeqId + ". Ignoring");
4947
4948
4949
4950
4951
4952
4953
4954
4955
4956
4957
4958 }
4959 }
4960 } finally {
4961 status.cleanup();
4962 writestate.notifyAll();
4963 }
4964 }
4965 return null;
4966 }
4967
4968 @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="NN_NAKED_NOTIFY",
4969 justification="Intentional; post memstore flush")
4970 void replayWALFlushCommitMarker(FlushDescriptor flush) throws IOException {
4971 MonitoredTask status = TaskMonitor.get().createStatus("Committing flush " + this);
4972
4973
4974
4975
4976
4977 synchronized (writestate) {
4978 try {
4979 if (flush.getFlushSequenceNumber() < lastReplayedOpenRegionSeqId) {
4980 LOG.warn(getRegionInfo().getEncodedName() + " : "
4981 + "Skipping replaying flush event :" + TextFormat.shortDebugString(flush)
4982 + " because its sequence id is smaller than this regions lastReplayedOpenRegionSeqId "
4983 + " of " + lastReplayedOpenRegionSeqId);
4984 return;
4985 }
4986
4987 if (writestate.flushing) {
4988 PrepareFlushResult prepareFlushResult = this.prepareFlushResult;
4989 if (flush.getFlushSequenceNumber() == prepareFlushResult.flushOpSeqId) {
4990 if (LOG.isDebugEnabled()) {
4991 LOG.debug(getRegionInfo().getEncodedName() + " : "
4992 + "Received a flush commit marker with seqId:" + flush.getFlushSequenceNumber()
4993 + " and a previous prepared snapshot was found");
4994 }
4995
4996
4997 replayFlushInStores(flush, prepareFlushResult, true);
4998
4999
5000 this.addAndGetGlobalMemstoreSize(-prepareFlushResult.totalFlushableSize);
5001
5002 this.prepareFlushResult = null;
5003 writestate.flushing = false;
5004 } else if (flush.getFlushSequenceNumber() < prepareFlushResult.flushOpSeqId) {
5005
5006
5007
5008
5009 LOG.warn(getRegionInfo().getEncodedName() + " : "
5010 + "Received a flush commit marker with smaller seqId: "
5011 + flush.getFlushSequenceNumber() + " than what we have prepared with seqId: "
5012 + prepareFlushResult.flushOpSeqId + ". Picking up new file, but not dropping"
5013 +" prepared memstore snapshot");
5014 replayFlushInStores(flush, prepareFlushResult, false);
5015
5016
5017
5018 } else {
5019
5020
5021
5022
5023
5024 LOG.warn(getRegionInfo().getEncodedName() + " : "
5025 + "Received a flush commit marker with larger seqId: "
5026 + flush.getFlushSequenceNumber() + " than what we have prepared with seqId: " +
5027 prepareFlushResult.flushOpSeqId + ". Picking up new file and dropping prepared"
5028 +" memstore snapshot");
5029
5030 replayFlushInStores(flush, prepareFlushResult, true);
5031
5032
5033 this.addAndGetGlobalMemstoreSize(-prepareFlushResult.totalFlushableSize);
5034
5035
5036
5037 dropMemstoreContentsForSeqId(flush.getFlushSequenceNumber(), null);
5038
5039 this.prepareFlushResult = null;
5040 writestate.flushing = false;
5041 }
5042
5043
5044
5045
5046
5047 this.setReadsEnabled(true);
5048 } else {
5049 LOG.warn(getRegionInfo().getEncodedName() + " : "
5050 + "Received a flush commit marker with seqId:" + flush.getFlushSequenceNumber()
5051 + ", but no previous prepared snapshot was found");
5052
5053
5054 replayFlushInStores(flush, null, false);
5055
5056
5057
5058 dropMemstoreContentsForSeqId(flush.getFlushSequenceNumber(), null);
5059 }
5060
5061 status.markComplete("Flush commit successful");
5062
5063
5064 this.maxFlushedSeqId = flush.getFlushSequenceNumber();
5065
5066
5067 mvcc.advanceTo(flush.getFlushSequenceNumber());
5068
5069 } catch (FileNotFoundException ex) {
5070 LOG.warn(getRegionInfo().getEncodedName() + " : "
5071 + "At least one of the store files in flush: " + TextFormat.shortDebugString(flush)
5072 + " doesn't exist any more. Skip loading the file(s)", ex);
5073 }
5074 finally {
5075 status.cleanup();
5076 writestate.notifyAll();
5077 }
5078 }
5079
5080
5081
5082 synchronized (this) {
5083 notifyAll();
5084 }
5085 }
5086
5087
5088
5089
5090
5091
5092
5093
5094
5095 private void replayFlushInStores(FlushDescriptor flush, PrepareFlushResult prepareFlushResult,
5096 boolean dropMemstoreSnapshot)
5097 throws IOException {
5098 for (StoreFlushDescriptor storeFlush : flush.getStoreFlushesList()) {
5099 byte[] family = storeFlush.getFamilyName().toByteArray();
5100 Store store = getStore(family);
5101 if (store == null) {
5102 LOG.warn(getRegionInfo().getEncodedName() + " : "
5103 + "Received a flush commit marker from primary, but the family is not found."
5104 + "Ignoring StoreFlushDescriptor:" + storeFlush);
5105 continue;
5106 }
5107 List<String> flushFiles = storeFlush.getFlushOutputList();
5108 StoreFlushContext ctx = null;
5109 long startTime = EnvironmentEdgeManager.currentTime();
5110 if (prepareFlushResult == null || prepareFlushResult.storeFlushCtxs == null) {
5111 ctx = store.createFlushContext(flush.getFlushSequenceNumber());
5112 } else {
5113 ctx = prepareFlushResult.storeFlushCtxs.get(family);
5114 startTime = prepareFlushResult.startTime;
5115 }
5116
5117 if (ctx == null) {
5118 LOG.warn(getRegionInfo().getEncodedName() + " : "
5119 + "Unexpected: flush commit marker received from store "
5120 + Bytes.toString(family) + " but no associated flush context. Ignoring");
5121 continue;
5122 }
5123
5124 ctx.replayFlush(flushFiles, dropMemstoreSnapshot);
5125
5126
5127 this.lastStoreFlushTimeMap.put(store, startTime);
5128 }
5129 }
5130
5131
5132
5133
5134
5135
5136 private long dropMemstoreContentsForSeqId(long seqId, Store store) throws IOException {
5137 long totalFreedSize = 0;
5138 this.updatesLock.writeLock().lock();
5139 try {
5140
5141 long currentSeqId = mvcc.getReadPoint();
5142 if (seqId >= currentSeqId) {
5143
5144 LOG.info(getRegionInfo().getEncodedName() + " : "
5145 + "Dropping memstore contents as well since replayed flush seqId: "
5146 + seqId + " is greater than current seqId:" + currentSeqId);
5147
5148
5149 if (store == null) {
5150 for (Store s : stores.values()) {
5151 totalFreedSize += doDropStoreMemstoreContentsForSeqId(s, currentSeqId);
5152 }
5153 } else {
5154 totalFreedSize += doDropStoreMemstoreContentsForSeqId(store, currentSeqId);
5155 }
5156 } else {
5157 LOG.info(getRegionInfo().getEncodedName() + " : "
5158 + "Not dropping memstore contents since replayed flush seqId: "
5159 + seqId + " is smaller than current seqId:" + currentSeqId);
5160 }
5161 } finally {
5162 this.updatesLock.writeLock().unlock();
5163 }
5164 return totalFreedSize;
5165 }
5166
5167 private long doDropStoreMemstoreContentsForSeqId(Store s, long currentSeqId) throws IOException {
5168 long snapshotSize = s.getFlushableSize();
5169 this.addAndGetGlobalMemstoreSize(-snapshotSize);
5170 StoreFlushContext ctx = s.createFlushContext(currentSeqId);
5171 ctx.prepare();
5172 ctx.abort();
5173 return snapshotSize;
5174 }
5175
5176 private void replayWALFlushAbortMarker(FlushDescriptor flush) {
5177
5178
5179
5180 }
5181
5182 private void replayWALFlushCannotFlushMarker(FlushDescriptor flush, long replaySeqId) {
5183 synchronized (writestate) {
5184 if (this.lastReplayedOpenRegionSeqId > replaySeqId) {
5185 LOG.warn(getRegionInfo().getEncodedName() + " : "
5186 + "Skipping replaying flush event :" + TextFormat.shortDebugString(flush)
5187 + " because its sequence id " + replaySeqId + " is smaller than this regions "
5188 + "lastReplayedOpenRegionSeqId of " + lastReplayedOpenRegionSeqId);
5189 return;
5190 }
5191
5192
5193
5194
5195
5196
5197 this.setReadsEnabled(true);
5198 }
5199 }
5200
5201 PrepareFlushResult getPrepareFlushResult() {
5202 return prepareFlushResult;
5203 }
5204
5205 @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="NN_NAKED_NOTIFY",
5206 justification="Intentional; cleared the memstore")
5207 void replayWALRegionEventMarker(RegionEventDescriptor regionEvent) throws IOException {
5208 checkTargetRegion(regionEvent.getEncodedRegionName().toByteArray(),
5209 "RegionEvent marker from WAL ", regionEvent);
5210
5211 startRegionOperation(Operation.REPLAY_EVENT);
5212 try {
5213 if (ServerRegionReplicaUtil.isDefaultReplica(this.getRegionInfo())) {
5214 return;
5215 }
5216
5217 if (regionEvent.getEventType() == EventType.REGION_CLOSE) {
5218
5219 return;
5220 }
5221 if (regionEvent.getEventType() != EventType.REGION_OPEN) {
5222 LOG.warn(getRegionInfo().getEncodedName() + " : "
5223 + "Unknown region event received, ignoring :"
5224 + TextFormat.shortDebugString(regionEvent));
5225 return;
5226 }
5227
5228 if (LOG.isDebugEnabled()) {
5229 LOG.debug(getRegionInfo().getEncodedName() + " : "
5230 + "Replaying region open event marker " + TextFormat.shortDebugString(regionEvent));
5231 }
5232
5233
5234 synchronized (writestate) {
5235
5236
5237
5238
5239
5240
5241 if (this.lastReplayedOpenRegionSeqId <= regionEvent.getLogSequenceNumber()) {
5242 this.lastReplayedOpenRegionSeqId = regionEvent.getLogSequenceNumber();
5243 } else {
5244 LOG.warn(getRegionInfo().getEncodedName() + " : "
5245 + "Skipping replaying region event :" + TextFormat.shortDebugString(regionEvent)
5246 + " because its sequence id is smaller than this regions lastReplayedOpenRegionSeqId "
5247 + " of " + lastReplayedOpenRegionSeqId);
5248 return;
5249 }
5250
5251
5252
5253 for (StoreDescriptor storeDescriptor : regionEvent.getStoresList()) {
5254
5255 byte[] family = storeDescriptor.getFamilyName().toByteArray();
5256 Store store = getStore(family);
5257 if (store == null) {
5258 LOG.warn(getRegionInfo().getEncodedName() + " : "
5259 + "Received a region open marker from primary, but the family is not found. "
5260 + "Ignoring. StoreDescriptor:" + storeDescriptor);
5261 continue;
5262 }
5263
5264 long storeSeqId = store.getMaxSequenceId();
5265 List<String> storeFiles = storeDescriptor.getStoreFileList();
5266 try {
5267 store.refreshStoreFiles(storeFiles);
5268 } catch (FileNotFoundException ex) {
5269 LOG.warn(getRegionInfo().getEncodedName() + " : "
5270 + "At least one of the store files: " + storeFiles
5271 + " doesn't exist any more. Skip loading the file(s)", ex);
5272 continue;
5273 }
5274 if (store.getMaxSequenceId() != storeSeqId) {
5275
5276 lastStoreFlushTimeMap.put(store, EnvironmentEdgeManager.currentTime());
5277 }
5278
5279 if (writestate.flushing) {
5280
5281 if (this.prepareFlushResult.flushOpSeqId <= regionEvent.getLogSequenceNumber()) {
5282 StoreFlushContext ctx = this.prepareFlushResult.storeFlushCtxs == null ?
5283 null : this.prepareFlushResult.storeFlushCtxs.get(family);
5284 if (ctx != null) {
5285 long snapshotSize = store.getFlushableSize();
5286 ctx.abort();
5287 this.addAndGetGlobalMemstoreSize(-snapshotSize);
5288 this.prepareFlushResult.storeFlushCtxs.remove(family);
5289 }
5290 }
5291 }
5292
5293
5294 dropMemstoreContentsForSeqId(regionEvent.getLogSequenceNumber(), store);
5295 if (storeSeqId > this.maxFlushedSeqId) {
5296 this.maxFlushedSeqId = storeSeqId;
5297 }
5298 }
5299
5300
5301
5302 dropPrepareFlushIfPossible();
5303
5304
5305 mvcc.await();
5306
5307
5308
5309 this.setReadsEnabled(true);
5310
5311
5312
5313 synchronized (this) {
5314 notifyAll();
5315 }
5316 }
5317 logRegionFiles();
5318 } finally {
5319 closeRegionOperation(Operation.REPLAY_EVENT);
5320 }
5321 }
5322
5323 void replayWALBulkLoadEventMarker(WALProtos.BulkLoadDescriptor bulkLoadEvent) throws IOException {
5324 checkTargetRegion(bulkLoadEvent.getEncodedRegionName().toByteArray(),
5325 "BulkLoad marker from WAL ", bulkLoadEvent);
5326
5327 if (ServerRegionReplicaUtil.isDefaultReplica(this.getRegionInfo())) {
5328 return;
5329 }
5330
5331 if (LOG.isDebugEnabled()) {
5332 LOG.debug(getRegionInfo().getEncodedName() + " : "
5333 + "Replaying bulkload event marker " + TextFormat.shortDebugString(bulkLoadEvent));
5334 }
5335
5336 boolean multipleFamilies = false;
5337 byte[] family = null;
5338 for (StoreDescriptor storeDescriptor : bulkLoadEvent.getStoresList()) {
5339 byte[] fam = storeDescriptor.getFamilyName().toByteArray();
5340 if (family == null) {
5341 family = fam;
5342 } else if (!Bytes.equals(family, fam)) {
5343 multipleFamilies = true;
5344 break;
5345 }
5346 }
5347
5348 startBulkRegionOperation(multipleFamilies);
5349 try {
5350
5351 synchronized (writestate) {
5352
5353
5354
5355
5356
5357
5358 if (bulkLoadEvent.getBulkloadSeqNum() >= 0
5359 && this.lastReplayedOpenRegionSeqId >= bulkLoadEvent.getBulkloadSeqNum()) {
5360 LOG.warn(getRegionInfo().getEncodedName() + " : "
5361 + "Skipping replaying bulkload event :"
5362 + TextFormat.shortDebugString(bulkLoadEvent)
5363 + " because its sequence id is smaller than this region's lastReplayedOpenRegionSeqId"
5364 + " =" + lastReplayedOpenRegionSeqId);
5365
5366 return;
5367 }
5368
5369 for (StoreDescriptor storeDescriptor : bulkLoadEvent.getStoresList()) {
5370
5371 family = storeDescriptor.getFamilyName().toByteArray();
5372 Store store = getStore(family);
5373 if (store == null) {
5374 LOG.warn(getRegionInfo().getEncodedName() + " : "
5375 + "Received a bulk load marker from primary, but the family is not found. "
5376 + "Ignoring. StoreDescriptor:" + storeDescriptor);
5377 continue;
5378 }
5379
5380 List<String> storeFiles = storeDescriptor.getStoreFileList();
5381 for (String storeFile : storeFiles) {
5382 StoreFileInfo storeFileInfo = null;
5383 try {
5384 storeFileInfo = fs.getStoreFileInfo(Bytes.toString(family), storeFile);
5385 store.bulkLoadHFile(storeFileInfo);
5386 } catch(FileNotFoundException ex) {
5387 LOG.warn(getRegionInfo().getEncodedName() + " : "
5388 + ((storeFileInfo != null) ? storeFileInfo.toString() :
5389 (new Path(Bytes.toString(family), storeFile)).toString())
5390 + " doesn't exist any more. Skip loading the file");
5391 }
5392 }
5393 }
5394 }
5395 if (bulkLoadEvent.getBulkloadSeqNum() > 0) {
5396 mvcc.advanceTo(bulkLoadEvent.getBulkloadSeqNum());
5397 }
5398 } finally {
5399 closeBulkRegionOperation();
5400 }
5401 }
5402
5403
5404
5405
5406 private void dropPrepareFlushIfPossible() {
5407 if (writestate.flushing) {
5408 boolean canDrop = true;
5409 if (prepareFlushResult.storeFlushCtxs != null) {
5410 for (Entry<byte[], StoreFlushContext> entry
5411 : prepareFlushResult.storeFlushCtxs.entrySet()) {
5412 Store store = getStore(entry.getKey());
5413 if (store == null) {
5414 continue;
5415 }
5416 if (store.getSnapshotSize() > 0) {
5417 canDrop = false;
5418 break;
5419 }
5420 }
5421 }
5422
5423
5424
5425 if (canDrop) {
5426 writestate.flushing = false;
5427 this.prepareFlushResult = null;
5428 }
5429 }
5430 }
5431
5432 @edu.umd.cs.findbugs.annotations.SuppressWarnings(value = "NN_NAKED_NOTIFY",
5433 justification = "Notify is about post replay. Intentional")
5434 @Override
5435 public boolean refreshStoreFiles() throws IOException {
5436 if (ServerRegionReplicaUtil.isDefaultReplica(this.getRegionInfo())) {
5437 return false;
5438 }
5439
5440 if (LOG.isDebugEnabled()) {
5441 LOG.debug(getRegionInfo().getEncodedName() + " : "
5442 + "Refreshing store files to see whether we can free up memstore");
5443 }
5444
5445 long totalFreedSize = 0;
5446
5447 long smallestSeqIdInStores = Long.MAX_VALUE;
5448
5449 startRegionOperation();
5450 try {
5451 Map<Store, Long> map = new HashMap<Store, Long>();
5452 synchronized (writestate) {
5453 for (Store store : getStores()) {
5454
5455
5456 long maxSeqIdBefore = store.getMaxSequenceId();
5457
5458
5459 store.refreshStoreFiles();
5460
5461 long storeSeqId = store.getMaxSequenceId();
5462 if (storeSeqId < smallestSeqIdInStores) {
5463 smallestSeqIdInStores = storeSeqId;
5464 }
5465
5466
5467 if (storeSeqId > maxSeqIdBefore) {
5468 if (writestate.flushing) {
5469
5470 if (this.prepareFlushResult.flushOpSeqId <= storeSeqId) {
5471 StoreFlushContext ctx = this.prepareFlushResult.storeFlushCtxs == null ?
5472 null : this.prepareFlushResult.storeFlushCtxs.get(store.getFamily().getName());
5473 if (ctx != null) {
5474 long snapshotSize = store.getFlushableSize();
5475 ctx.abort();
5476 this.addAndGetGlobalMemstoreSize(-snapshotSize);
5477 this.prepareFlushResult.storeFlushCtxs.remove(store.getFamily().getName());
5478 totalFreedSize += snapshotSize;
5479 }
5480 }
5481 }
5482
5483 map.put(store, storeSeqId);
5484 }
5485 }
5486
5487
5488
5489 dropPrepareFlushIfPossible();
5490
5491
5492
5493 for (Store s : getStores()) {
5494 mvcc.advanceTo(s.getMaxMemstoreTS());
5495 }
5496
5497
5498
5499
5500
5501
5502 if (this.lastReplayedOpenRegionSeqId < smallestSeqIdInStores) {
5503 this.lastReplayedOpenRegionSeqId = smallestSeqIdInStores;
5504 }
5505 }
5506 if (!map.isEmpty()) {
5507 for (Map.Entry<Store, Long> entry : map.entrySet()) {
5508
5509 totalFreedSize += dropMemstoreContentsForSeqId(entry.getValue(), entry.getKey());
5510 }
5511 }
5512
5513
5514 synchronized (this) {
5515 notifyAll();
5516 }
5517 return totalFreedSize > 0;
5518 } finally {
5519 closeRegionOperation();
5520 }
5521 }
5522
5523 private void logRegionFiles() {
5524 if (LOG.isTraceEnabled()) {
5525 LOG.trace(getRegionInfo().getEncodedName() + " : Store files for region: ");
5526 for (Store s : stores.values()) {
5527 Collection<StoreFile> storeFiles = s.getStorefiles();
5528 if (storeFiles == null) continue;
5529 for (StoreFile sf : storeFiles) {
5530 LOG.trace(getRegionInfo().getEncodedName() + " : " + sf);
5531 }
5532 }
5533 }
5534 }
5535
5536
5537
5538
5539 private void checkTargetRegion(byte[] encodedRegionName, String exceptionMsg, Object payload)
5540 throws WrongRegionException {
5541 if (Bytes.equals(this.getRegionInfo().getEncodedNameAsBytes(), encodedRegionName)) {
5542 return;
5543 }
5544
5545 if (!RegionReplicaUtil.isDefaultReplica(this.getRegionInfo()) &&
5546 Bytes.equals(encodedRegionName,
5547 this.fs.getRegionInfoForFS().getEncodedNameAsBytes())) {
5548 return;
5549 }
5550
5551 throw new WrongRegionException(exceptionMsg + payload
5552 + " targetted for region " + Bytes.toStringBinary(encodedRegionName)
5553 + " does not match this region: " + this.getRegionInfo());
5554 }
5555
5556
5557
5558
5559
5560
5561
5562 protected boolean restoreEdit(final Store s, final Cell cell) {
5563 long kvSize = s.add(cell);
5564 if (this.rsAccounting != null) {
5565 rsAccounting.addAndGetRegionReplayEditsSize(getRegionInfo().getRegionName(), kvSize);
5566 }
5567 return isFlushSize(this.addAndGetGlobalMemstoreSize(kvSize));
5568 }
5569
5570
5571
5572
5573
5574
5575
5576 private static boolean isZeroLengthThenDelete(final FileSystem fs, final Path p)
5577 throws IOException {
5578 FileStatus stat = fs.getFileStatus(p);
5579 if (stat.getLen() > 0) return false;
5580 LOG.warn("File " + p + " is zero-length, deleting.");
5581 fs.delete(p, false);
5582 return true;
5583 }
5584
5585 protected HStore instantiateHStore(final HColumnDescriptor family) throws IOException {
5586 return new HStore(this, family, this.conf);
5587 }
5588
5589 @Override
5590 public Store getStore(final byte[] column) {
5591 return this.stores.get(column);
5592 }
5593
5594
5595
5596
5597
5598 private Store getStore(Cell cell) {
5599 for (Map.Entry<byte[], Store> famStore : stores.entrySet()) {
5600 if (Bytes.equals(
5601 cell.getFamilyArray(), cell.getFamilyOffset(), cell.getFamilyLength(),
5602 famStore.getKey(), 0, famStore.getKey().length)) {
5603 return famStore.getValue();
5604 }
5605 }
5606
5607 return null;
5608 }
5609
5610 @Override
5611 public List<Store> getStores() {
5612 List<Store> list = new ArrayList<Store>(stores.size());
5613 list.addAll(stores.values());
5614 return list;
5615 }
5616
5617 @Override
5618 public List<String> getStoreFileList(final byte [][] columns)
5619 throws IllegalArgumentException {
5620 List<String> storeFileNames = new ArrayList<String>();
5621 synchronized(closeLock) {
5622 for(byte[] column : columns) {
5623 Store store = this.stores.get(column);
5624 if (store == null) {
5625 throw new IllegalArgumentException("No column family : " +
5626 new String(column, StandardCharsets.UTF_8) + " available");
5627 }
5628 Collection<StoreFile> storeFiles = store.getStorefiles();
5629 if (storeFiles == null) continue;
5630 for (StoreFile storeFile: storeFiles) {
5631 storeFileNames.add(storeFile.getPath().toString());
5632 }
5633
5634 logRegionFiles();
5635 }
5636 }
5637 return storeFileNames;
5638 }
5639
5640
5641
5642
5643
5644
5645 void checkRow(final byte [] row, String op) throws IOException {
5646 if (!rowIsInRange(getRegionInfo(), row)) {
5647 throw new WrongRegionException("Requested row out of range for " +
5648 op + " on HRegion " + this + ", startKey='" +
5649 Bytes.toStringBinary(getRegionInfo().getStartKey()) + "', getEndKey()='" +
5650 Bytes.toStringBinary(getRegionInfo().getEndKey()) + "', row='" +
5651 Bytes.toStringBinary(row) + "'");
5652 }
5653 }
5654
5655
5656
5657
5658
5659
5660
5661
5662 public RowLock getRowLock(byte[] row) throws IOException {
5663 return getRowLock(row, false);
5664 }
5665
5666 @Override
5667 public RowLock getRowLock(byte[] row, boolean readLock) throws IOException {
5668 return getRowLock(row, readLock, true);
5669 }
5670
5671
5672
5673
5674
5675
5676
5677
5678
5679
5680
5681
5682
5683
5684 public RowLock getRowLock(byte[] row, boolean readLock, boolean waitForLock) throws IOException {
5685
5686 checkRow(row, "row lock");
5687 return getRowLockInternal(row, readLock, waitForLock, null);
5688 }
5689
5690
5691 protected RowLock getRowLockInternal(byte[] row)
5692 throws IOException {
5693 return getRowLockInternal(row, false, true, null);
5694 }
5695
5696 protected RowLock getRowLockInternal(byte[] row, boolean readLock, boolean waitForLock,
5697 final RowLock prevRowLock) throws IOException {
5698
5699 HashedBytes rowKey = new HashedBytes(row);
5700
5701 RowLockContext rowLockContext = null;
5702 RowLockImpl result = null;
5703 TraceScope traceScope = null;
5704
5705
5706 if (Trace.isTracing()) {
5707 traceScope = Trace.startSpan("HRegion.getRowLock");
5708 traceScope.getSpan().addTimelineAnnotation("Getting a " + (readLock?"readLock":"writeLock"));
5709 }
5710
5711 boolean success = false;
5712 try {
5713
5714
5715 while (result == null) {
5716
5717
5718
5719 rowLockContext = new RowLockContext(rowKey);
5720 RowLockContext existingContext = lockedRows.putIfAbsent(rowKey, rowLockContext);
5721
5722
5723 if (existingContext != null) {
5724 rowLockContext = existingContext;
5725 }
5726
5727
5728
5729
5730 if (readLock) {
5731
5732
5733 RowLockImpl prevRowLockImpl = (RowLockImpl)prevRowLock;
5734 if ((prevRowLockImpl != null) && (prevRowLockImpl.getLock() ==
5735 rowLockContext.readWriteLock.readLock())) {
5736 success = true;
5737 return prevRowLock;
5738 }
5739 result = rowLockContext.newReadLock();
5740 } else {
5741 result = rowLockContext.newWriteLock();
5742 }
5743 }
5744
5745 int timeout = rowLockWaitDuration;
5746 boolean reachDeadlineFirst = false;
5747 RpcCallContext call = RpcServer.getCurrentCall();
5748 if (call != null && call.getDeadline() < Long.MAX_VALUE) {
5749 int timeToDeadline = (int)(call.getDeadline() - System.currentTimeMillis());
5750 if (timeToDeadline <= this.rowLockWaitDuration) {
5751 reachDeadlineFirst = true;
5752 timeout = timeToDeadline;
5753 }
5754 }
5755
5756 boolean lockAvailable = false;
5757 if (timeout > 0) {
5758 if (waitForLock) {
5759
5760 lockAvailable = result.getLock().tryLock(timeout, TimeUnit.MILLISECONDS);
5761 } else {
5762
5763 lockAvailable = result.getLock().tryLock();
5764 }
5765 }
5766 if (!lockAvailable) {
5767 if (traceScope != null) {
5768 traceScope.getSpan().addTimelineAnnotation("Failed to get row lock");
5769 }
5770 result = null;
5771 String message = "Timed out waiting for lock for row: " + rowKey + " in region " +
5772 getRegionInfo().getEncodedName() + ", timeout=" + timeout + ", deadlined=" +
5773 reachDeadlineFirst + ", waitForLock=" + waitForLock;
5774 if (waitForLock) {
5775 if (reachDeadlineFirst) {
5776 LOG.info("TIMEOUT: " + message);
5777 throw new TimeoutIOException(message);
5778 } else {
5779
5780 LOG.info("IOE " + message);
5781 throw new IOException(message);
5782 }
5783 } else {
5784
5785 return null;
5786 }
5787 }
5788 rowLockContext.setThreadName(Thread.currentThread().getName());
5789 success = true;
5790 return result;
5791 } catch (InterruptedException ie) {
5792 if (LOG.isDebugEnabled()) {
5793 LOG.debug("Thread interrupted waiting for lock on row: " + rowKey);
5794 }
5795 if (traceScope != null) {
5796 traceScope.getSpan().addTimelineAnnotation("Interrupted exception getting row lock");
5797 }
5798 throw throwOnInterrupt(ie);
5799 } catch (Error error) {
5800
5801
5802
5803 LOG.warn("Error to get row lock for " + Bytes.toStringBinary(row) + ", cause: " + error);
5804 IOException ioe = new IOException(error);
5805 if (traceScope != null) {
5806 traceScope.getSpan().addTimelineAnnotation("Error getting row lock");
5807 }
5808 throw ioe;
5809 } finally {
5810
5811 if (!success && rowLockContext != null) {
5812 rowLockContext.cleanUp();
5813 }
5814 if (traceScope != null) {
5815 traceScope.close();
5816 }
5817 }
5818 }
5819
5820 @Override
5821 public void releaseRowLocks(List<RowLock> rowLocks) {
5822 if (rowLocks != null) {
5823 for (int i = 0; i < rowLocks.size(); i++) {
5824 rowLocks.get(i).release();
5825 }
5826 rowLocks.clear();
5827 }
5828 }
5829
5830 public ConcurrentHashMap<HashedBytes, RowLockContext> getLockedRows() {
5831 return lockedRows;
5832 }
5833
5834 class RowLockContext {
5835 private final HashedBytes row;
5836 final ReadWriteLock readWriteLock = new ReentrantReadWriteLock(true);
5837 final AtomicBoolean usable = new AtomicBoolean(true);
5838 final AtomicInteger count = new AtomicInteger(0);
5839 final Object lock = new Object();
5840 private String threadName;
5841
5842 RowLockContext(HashedBytes row) {
5843 this.row = row;
5844 }
5845
5846 RowLockImpl newWriteLock() {
5847 Lock l = readWriteLock.writeLock();
5848 return getRowLock(l);
5849 }
5850 RowLockImpl newReadLock() {
5851 Lock l = readWriteLock.readLock();
5852 return getRowLock(l);
5853 }
5854
5855 private RowLockImpl getRowLock(Lock l) {
5856 count.incrementAndGet();
5857 synchronized (lock) {
5858 if (usable.get()) {
5859 return new RowLockImpl(this, l);
5860 } else {
5861 return null;
5862 }
5863 }
5864 }
5865
5866 void cleanUp() {
5867 long c = count.decrementAndGet();
5868 if (c <= 0) {
5869 synchronized (lock) {
5870 if (count.get() <= 0 && usable.get()){
5871 usable.set(false);
5872 RowLockContext removed = lockedRows.remove(row);
5873 assert removed == this: "we should never remove a different context";
5874 }
5875 }
5876 }
5877 }
5878
5879 public void setThreadName(String threadName) {
5880 this.threadName = threadName;
5881 }
5882
5883 @Override
5884 public String toString() {
5885 return "RowLockContext{" +
5886 "row=" + row +
5887 ", readWriteLock=" + readWriteLock +
5888 ", count=" + count +
5889 ", threadName=" + threadName +
5890 '}';
5891 }
5892 }
5893
5894
5895
5896
5897 public static class RowLockImpl implements RowLock {
5898 private final RowLockContext context;
5899 private final Lock lock;
5900
5901 public RowLockImpl(RowLockContext context, Lock lock) {
5902 this.context = context;
5903 this.lock = lock;
5904 }
5905
5906 public Lock getLock() {
5907 return lock;
5908 }
5909
5910 public RowLockContext getContext() {
5911 return context;
5912 }
5913
5914 @Override
5915 public void release() {
5916 lock.unlock();
5917 context.cleanUp();
5918 }
5919
5920 @Override
5921 public String toString() {
5922 return "RowLockImpl{" +
5923 "context=" + context +
5924 ", lock=" + lock +
5925 '}';
5926 }
5927 }
5928
5929
5930
5931
5932
5933
5934
5935 private static boolean hasMultipleColumnFamilies(Collection<Pair<byte[], String>> familyPaths) {
5936 boolean multipleFamilies = false;
5937 byte[] family = null;
5938 for (Pair<byte[], String> pair : familyPaths) {
5939 byte[] fam = pair.getFirst();
5940 if (family == null) {
5941 family = fam;
5942 } else if (!Bytes.equals(family, fam)) {
5943 multipleFamilies = true;
5944 break;
5945 }
5946 }
5947 return multipleFamilies;
5948 }
5949
5950 @Override
5951 public boolean bulkLoadHFiles(Collection<Pair<byte[], String>> familyPaths, boolean assignSeqId,
5952 BulkLoadListener bulkLoadListener, List<String> clusterIds) throws IOException {
5953 long seqId = -1;
5954 Map<byte[], List<Path>> storeFiles = new TreeMap<byte[], List<Path>>(Bytes.BYTES_COMPARATOR);
5955 Map<String, Long> storeFilesSizes = new HashMap<String, Long>();
5956 Preconditions.checkNotNull(familyPaths);
5957
5958 startBulkRegionOperation(hasMultipleColumnFamilies(familyPaths));
5959 boolean isSuccessful = false;
5960 try {
5961 this.writeRequestsCount.increment();
5962
5963
5964
5965
5966 List<IOException> ioes = new ArrayList<IOException>();
5967 List<Pair<byte[], String>> failures = new ArrayList<Pair<byte[], String>>();
5968 for (Pair<byte[], String> p : familyPaths) {
5969 byte[] familyName = p.getFirst();
5970 String path = p.getSecond();
5971
5972 Store store = getStore(familyName);
5973 if (store == null) {
5974 IOException ioe = new org.apache.hadoop.hbase.DoNotRetryIOException(
5975 "No such column family " + Bytes.toStringBinary(familyName));
5976 ioes.add(ioe);
5977 } else {
5978 try {
5979 store.assertBulkLoadHFileOk(new Path(path));
5980 } catch (WrongRegionException wre) {
5981
5982 failures.add(p);
5983 } catch (IOException ioe) {
5984
5985 ioes.add(ioe);
5986 }
5987 }
5988 }
5989
5990
5991
5992 checkInterrupt();
5993
5994
5995 if (ioes.size() != 0) {
5996 IOException e = MultipleIOException.createIOException(ioes);
5997 LOG.error("There were one or more IO errors when checking if the bulk load is ok.", e);
5998 throw e;
5999 }
6000
6001
6002 if (failures.size() != 0) {
6003 StringBuilder list = new StringBuilder();
6004 for (Pair<byte[], String> p : failures) {
6005 list.append("\n").append(Bytes.toString(p.getFirst())).append(" : ")
6006 .append(p.getSecond());
6007 }
6008
6009 LOG.warn("There was a recoverable bulk load failure likely due to a" +
6010 " split. These (family, HFile) pairs were not loaded: " + list);
6011 return isSuccessful;
6012 }
6013
6014
6015
6016
6017
6018
6019 if (assignSeqId) {
6020 FlushResult fs = flushcache(true, false);
6021 if (fs.isFlushSucceeded()) {
6022 seqId = ((FlushResultImpl)fs).flushSequenceId;
6023 } else if (fs.getResult() == FlushResult.Result.CANNOT_FLUSH_MEMSTORE_EMPTY) {
6024 seqId = ((FlushResultImpl)fs).flushSequenceId;
6025 } else if (fs.getResult() == FlushResult.Result.CANNOT_FLUSH) {
6026
6027
6028 waitForFlushes();
6029 } else {
6030 throw new IOException("Could not bulk load with an assigned sequential ID because the "+
6031 "flush didn't run. Reason for not flushing: " + ((FlushResultImpl)fs).failureReason);
6032 }
6033 }
6034
6035 Map<byte[], List<Pair<Path, Path>>> familyWithFinalPath =
6036 new TreeMap<>(Bytes.BYTES_COMPARATOR);
6037 for (Pair<byte[], String> p : familyPaths) {
6038 byte[] familyName = p.getFirst();
6039 String path = p.getSecond();
6040 Store store = getStore(familyName);
6041 if (!familyWithFinalPath.containsKey(familyName)) {
6042 familyWithFinalPath.put(familyName, new ArrayList<Pair<Path, Path>>());
6043 }
6044 List<Pair<Path, Path>> lst = familyWithFinalPath.get(familyName);
6045 try {
6046 String finalPath = path;
6047 if (bulkLoadListener != null) {
6048 finalPath = bulkLoadListener.prepareBulkLoad(familyName, path);
6049 }
6050 Pair<Path, Path> pair = ((HStore)store).preBulkLoadHFile(finalPath, seqId);
6051 lst.add(pair);
6052 } catch (IOException ioe) {
6053
6054
6055
6056
6057 LOG.error("There was a partial failure due to IO when attempting to" +
6058 " load " + Bytes.toString(p.getFirst()) + " : " + p.getSecond(), ioe);
6059 if (bulkLoadListener != null) {
6060 try {
6061 bulkLoadListener.failedBulkLoad(familyName, path);
6062 } catch (Exception ex) {
6063 LOG.error("Error while calling failedBulkLoad for family " +
6064 Bytes.toString(familyName) + " with path " + path, ex);
6065 }
6066 }
6067 throw ioe;
6068 }
6069 }
6070
6071 if (this.getCoprocessorHost() != null) {
6072 for (Map.Entry<byte[], List<Pair<Path, Path>>> entry : familyWithFinalPath.entrySet()) {
6073 this.getCoprocessorHost().preCommitStoreFile(entry.getKey(), entry.getValue());
6074 }
6075 }
6076 for (Map.Entry<byte[], List<Pair<Path, Path>>> entry : familyWithFinalPath.entrySet()) {
6077 byte[] familyName = entry.getKey();
6078 for (Pair<Path, Path> p : entry.getValue()) {
6079 String path = p.getFirst().toString();
6080 Path commitedStoreFile = p.getSecond();
6081 Store store = getStore(familyName);
6082 try {
6083 store.bulkLoadHFile(familyName, path, commitedStoreFile);
6084
6085 try {
6086 FileSystem fs = commitedStoreFile.getFileSystem(baseConf);
6087 storeFilesSizes.put(commitedStoreFile.getName(), fs.getFileStatus(commitedStoreFile)
6088 .getLen());
6089 } catch (IOException e) {
6090 LOG.warn("Failed to find the size of hfile " + commitedStoreFile);
6091 storeFilesSizes.put(commitedStoreFile.getName(), 0L);
6092 }
6093
6094 if(storeFiles.containsKey(familyName)) {
6095 storeFiles.get(familyName).add(commitedStoreFile);
6096 } else {
6097 List<Path> storeFileNames = new ArrayList<Path>();
6098 storeFileNames.add(commitedStoreFile);
6099 storeFiles.put(familyName, storeFileNames);
6100 }
6101 if (bulkLoadListener != null) {
6102 bulkLoadListener.doneBulkLoad(familyName, path);
6103 }
6104 } catch (IOException ioe) {
6105
6106
6107
6108
6109 LOG.error("There was a partial failure due to IO when attempting to" +
6110 " load " + Bytes.toString(familyName) + " : " + p.getSecond(), ioe);
6111 if (bulkLoadListener != null) {
6112 try {
6113 bulkLoadListener.failedBulkLoad(familyName, path);
6114 } catch (Exception ex) {
6115 LOG.error("Error while calling failedBulkLoad for family " +
6116 Bytes.toString(familyName) + " with path " + path, ex);
6117 }
6118 }
6119 throw ioe;
6120 }
6121 }
6122 }
6123
6124 isSuccessful = true;
6125 } finally {
6126 if (wal != null && !storeFiles.isEmpty()) {
6127
6128 try {
6129 WALProtos.BulkLoadDescriptor loadDescriptor =
6130 ProtobufUtil.toBulkLoadDescriptor(this.getRegionInfo().getTable(),
6131 ByteStringer.wrap(this.getRegionInfo().getEncodedNameAsBytes()), storeFiles,
6132 storeFilesSizes, seqId, clusterIds);
6133 WALUtil.writeBulkLoadMarkerAndSync(wal, this.htableDescriptor, getRegionInfo(),
6134 loadDescriptor, mvcc);
6135 } catch (IOException ioe) {
6136 if (this.rsServices != null) {
6137
6138
6139 isSuccessful = false;
6140 this.rsServices.abort("Failed to write bulk load event into WAL.", ioe);
6141 }
6142 }
6143 }
6144
6145 closeBulkRegionOperation();
6146 }
6147 return isSuccessful;
6148 }
6149
6150 @Override
6151 @Deprecated
6152 public boolean bulkLoadHFiles(Collection<Pair<byte[], String>> familyPaths, boolean assignSeqId,
6153 BulkLoadListener bulkLoadListener) throws IOException {
6154 LOG.warn("Deprecated bulkLoadHFiles invoked. This does not pass through source cluster ids." +
6155 " This is probably not what you want. See HBASE-22380.");
6156 return bulkLoadHFiles(familyPaths, assignSeqId, bulkLoadListener, EMPTY_CLUSTERID_LIST);
6157 }
6158
6159 @Override
6160 public boolean equals(Object o) {
6161 return o instanceof HRegion && Bytes.equals(getRegionInfo().getRegionName(),
6162 ((HRegion) o).getRegionInfo().getRegionName());
6163 }
6164
6165 @Override
6166 public int hashCode() {
6167 return Bytes.hashCode(getRegionInfo().getRegionName());
6168 }
6169
6170 @Override
6171 public String toString() {
6172 return getRegionInfo().getRegionNameAsString();
6173 }
6174
6175
6176
6177
6178 class RegionScannerImpl implements RegionScanner {
6179
6180 KeyValueHeap storeHeap = null;
6181
6182
6183 KeyValueHeap joinedHeap = null;
6184
6185
6186
6187 protected Cell joinedContinuationRow = null;
6188 private boolean filterClosed = false;
6189
6190 protected final byte[] stopRow;
6191 protected final boolean includeStopRow;
6192 protected final HRegion region;
6193
6194 private final long readPt;
6195 private final long maxResultSize;
6196 private final ScannerContext defaultScannerContext;
6197 private final FilterWrapper filter;
6198
6199 @Override
6200 public HRegionInfo getRegionInfo() {
6201 return region.getRegionInfo();
6202 }
6203
6204 RegionScannerImpl(Scan scan, List<KeyValueScanner> additionalScanners, HRegion region,
6205 long nonceGroup, long nonce) throws IOException {
6206 this.region = region;
6207 this.maxResultSize = scan.getMaxResultSize();
6208 if (scan.hasFilter()) {
6209 this.filter = new FilterWrapper(scan.getFilter());
6210 } else {
6211 this.filter = null;
6212 }
6213
6214
6215
6216
6217
6218
6219 defaultScannerContext = ScannerContext.newBuilder().setBatchLimit(scan.getBatch()).build();
6220
6221 this.stopRow = scan.getStopRow();
6222 this.includeStopRow = scan.includeStopRow();
6223
6224
6225 IsolationLevel isolationLevel = scan.getIsolationLevel();
6226 long mvccReadPoint = PackagePrivateFieldAccessor.getMvccReadPoint(scan);
6227 synchronized (scannerReadPoints) {
6228 if (mvccReadPoint > 0) {
6229 this.readPt = mvccReadPoint;
6230 } else if (nonce == HConstants.NO_NONCE || rsServices == null
6231 || rsServices.getNonceManager() == null) {
6232 this.readPt = getReadpoint(isolationLevel);
6233 } else {
6234 this.readPt = rsServices.getNonceManager().getMvccFromOperationContext(nonceGroup, nonce);
6235 }
6236 scannerReadPoints.put(this, this.readPt);
6237 }
6238 initializeScanners(scan, additionalScanners);
6239 }
6240
6241 RegionScannerImpl(Scan scan, List<KeyValueScanner> additionalScanners, HRegion region)
6242 throws IOException {
6243 this(scan, additionalScanners, region, HConstants.NO_NONCE, HConstants.NO_NONCE);
6244 }
6245
6246 protected void initializeScanners(Scan scan, List<KeyValueScanner> additionalScanners)
6247 throws IOException {
6248
6249
6250 List<KeyValueScanner> scanners = new ArrayList<KeyValueScanner>(scan.getFamilyMap().size());
6251 List<KeyValueScanner> joinedScanners
6252 = new ArrayList<KeyValueScanner>(scan.getFamilyMap().size());
6253
6254 List<KeyValueScanner> instantiatedScanners = new ArrayList<KeyValueScanner>();
6255
6256 if (additionalScanners != null && !additionalScanners.isEmpty()) {
6257 scanners.addAll(additionalScanners);
6258 instantiatedScanners.addAll(additionalScanners);
6259 }
6260
6261 try {
6262 for (Map.Entry<byte[], NavigableSet<byte[]>> entry : scan.getFamilyMap().entrySet()) {
6263 Store store = stores.get(entry.getKey());
6264 KeyValueScanner scanner = store.getScanner(scan, entry.getValue(), this.readPt);
6265 instantiatedScanners.add(scanner);
6266 if (this.filter == null || !scan.doLoadColumnFamiliesOnDemand()
6267 || this.filter.isFamilyEssential(entry.getKey())) {
6268 scanners.add(scanner);
6269 } else {
6270 joinedScanners.add(scanner);
6271 }
6272 }
6273 initializeKVHeap(scanners, joinedScanners, region);
6274 } catch (Throwable t) {
6275 throw handleException(instantiatedScanners, t);
6276 }
6277 }
6278
6279 protected void initializeKVHeap(List<KeyValueScanner> scanners,
6280 List<KeyValueScanner> joinedScanners, HRegion region)
6281 throws IOException {
6282 this.storeHeap = new KeyValueHeap(scanners, region.comparator);
6283 if (!joinedScanners.isEmpty()) {
6284 this.joinedHeap = new KeyValueHeap(joinedScanners, region.comparator);
6285 }
6286 }
6287
6288 private IOException handleException(List<KeyValueScanner> instantiatedScanners,
6289 Throwable t) {
6290
6291 scannerReadPoints.remove(this);
6292 if (storeHeap != null) {
6293 storeHeap.close();
6294 storeHeap = null;
6295 if (joinedHeap != null) {
6296 joinedHeap.close();
6297 joinedHeap = null;
6298 }
6299 } else {
6300
6301 for (KeyValueScanner scanner : instantiatedScanners) {
6302 scanner.close();
6303 }
6304 }
6305 return t instanceof IOException ? (IOException) t : new IOException(t);
6306 }
6307
6308 @Override
6309 public long getMaxResultSize() {
6310 return maxResultSize;
6311 }
6312
6313 @Override
6314 public long getMvccReadPoint() {
6315 return this.readPt;
6316 }
6317
6318 @Override
6319 public int getBatch() {
6320 return this.defaultScannerContext.getBatchLimit();
6321 }
6322
6323
6324
6325
6326
6327
6328 protected void resetFilters() throws IOException {
6329 if (filter != null) {
6330 filter.reset();
6331 }
6332 }
6333
6334 @Override
6335 public boolean next(List<Cell> outResults)
6336 throws IOException {
6337
6338 return next(outResults, defaultScannerContext);
6339 }
6340
6341 @Override
6342 public synchronized boolean next(List<Cell> outResults, ScannerContext scannerContext)
6343 throws IOException {
6344 if (this.filterClosed) {
6345 throw new UnknownScannerException("Scanner was closed (timed out?) " +
6346 "after we renewed it. Could be caused by a very slow scanner " +
6347 "or a lengthy garbage collection");
6348 }
6349 startRegionOperation(Operation.SCAN);
6350 try {
6351 return nextRaw(outResults, scannerContext);
6352 } finally {
6353 closeRegionOperation(Operation.SCAN);
6354 }
6355 }
6356
6357 @Override
6358 public boolean nextRaw(List<Cell> outResults) throws IOException {
6359
6360 return nextRaw(outResults, defaultScannerContext);
6361 }
6362
6363 @Override
6364 public boolean nextRaw(List<Cell> outResults, ScannerContext scannerContext)
6365 throws IOException {
6366 if (storeHeap == null) {
6367
6368 throw new UnknownScannerException("Scanner was closed");
6369 }
6370 boolean moreValues = false;
6371 if (outResults.isEmpty()) {
6372
6373
6374 moreValues = nextInternal(outResults, scannerContext);
6375 } else {
6376 List<Cell> tmpList = new ArrayList<Cell>();
6377 moreValues = nextInternal(tmpList, scannerContext);
6378 outResults.addAll(tmpList);
6379 }
6380
6381 if (!outResults.isEmpty()) {
6382 readRequestsCount.increment();
6383 }
6384
6385
6386
6387
6388 if (!scannerContext.mayHaveMoreCellsInRow()) {
6389 resetFilters();
6390 }
6391
6392 if (isFilterDoneInternal()) {
6393 moreValues = false;
6394 }
6395 return moreValues;
6396 }
6397
6398
6399
6400
6401 private boolean populateFromJoinedHeap(List<Cell> results, ScannerContext scannerContext)
6402 throws IOException {
6403 assert joinedContinuationRow != null;
6404 boolean moreValues =
6405 populateResult(results, this.joinedHeap, scannerContext,
6406 joinedContinuationRow.getRowArray(), joinedContinuationRow.getRowOffset(),
6407 joinedContinuationRow.getRowLength());
6408
6409 if (!scannerContext.checkAnyLimitReached(LimitScope.BETWEEN_CELLS)) {
6410
6411 joinedContinuationRow = null;
6412 }
6413
6414
6415 Collections.sort(results, comparator);
6416 return moreValues;
6417 }
6418
6419
6420
6421
6422
6423
6424
6425
6426
6427
6428
6429 private boolean populateResult(List<Cell> results, KeyValueHeap heap,
6430 ScannerContext scannerContext, byte[] currentRow, int offset, short length)
6431 throws IOException {
6432 Cell nextKv;
6433 boolean moreCellsInRow = false;
6434 boolean tmpKeepProgress = scannerContext.getKeepProgress();
6435
6436 LimitScope limitScope = LimitScope.BETWEEN_CELLS;
6437 do {
6438
6439
6440 checkInterrupt();
6441
6442
6443
6444
6445 scannerContext.setKeepProgress(true);
6446 heap.next(results, scannerContext);
6447 scannerContext.setKeepProgress(tmpKeepProgress);
6448
6449 nextKv = heap.peek();
6450 moreCellsInRow = moreCellsInRow(nextKv, currentRow, offset, length);
6451 if (!moreCellsInRow) incrementCountOfRowsScannedMetric(scannerContext);
6452
6453 if (moreCellsInRow && scannerContext.checkBatchLimit(limitScope)) {
6454 return scannerContext.setScannerState(NextState.BATCH_LIMIT_REACHED).hasMoreValues();
6455 } else if (scannerContext.checkSizeLimit(limitScope)) {
6456 ScannerContext.NextState state =
6457 moreCellsInRow ? NextState.SIZE_LIMIT_REACHED_MID_ROW : NextState.SIZE_LIMIT_REACHED;
6458 return scannerContext.setScannerState(state).hasMoreValues();
6459 } else if (scannerContext.checkTimeLimit(limitScope)) {
6460 ScannerContext.NextState state =
6461 moreCellsInRow ? NextState.TIME_LIMIT_REACHED_MID_ROW : NextState.TIME_LIMIT_REACHED;
6462 return scannerContext.setScannerState(state).hasMoreValues();
6463 }
6464 } while (moreCellsInRow);
6465 return nextKv != null;
6466 }
6467
6468
6469
6470
6471
6472
6473
6474
6475
6476
6477
6478 private boolean moreCellsInRow(final Cell nextKv, byte[] currentRow, int offset,
6479 short length) {
6480 return nextKv != null && CellUtil.matchingRow(nextKv, currentRow, offset, length);
6481 }
6482
6483
6484
6485
6486 @Override
6487 public synchronized boolean isFilterDone() throws IOException {
6488 return isFilterDoneInternal();
6489 }
6490
6491 private boolean isFilterDoneInternal() throws IOException {
6492 return this.filter != null && this.filter.filterAllRemaining();
6493 }
6494
6495 private boolean nextInternal(List<Cell> results, ScannerContext scannerContext)
6496 throws IOException {
6497 if (!results.isEmpty()) {
6498 throw new IllegalArgumentException("First parameter should be an empty list");
6499 }
6500 if (scannerContext == null) {
6501 throw new IllegalArgumentException("Scanner context cannot be null");
6502 }
6503 RpcCallContext rpcCall = RpcServer.getCurrentCall();
6504
6505
6506
6507
6508 int initialBatchProgress = scannerContext.getBatchProgress();
6509 long initialSizeProgress = scannerContext.getSizeProgress();
6510 long initialTimeProgress = scannerContext.getTimeProgress();
6511
6512
6513
6514
6515
6516
6517 while (true) {
6518
6519
6520 if (scannerContext.getKeepProgress()) {
6521
6522 scannerContext.setProgress(initialBatchProgress, initialSizeProgress,
6523 initialTimeProgress);
6524 } else {
6525 scannerContext.clearProgress();
6526 }
6527
6528 if (rpcCall != null) {
6529
6530
6531
6532
6533 long afterTime = rpcCall.disconnectSince();
6534 if (afterTime >= 0) {
6535 throw new CallerDisconnectedException(
6536 "Aborting on region " + getRegionInfo().getRegionNameAsString() + ", call " +
6537 this + " after " + afterTime + " ms, since " +
6538 "caller disconnected");
6539 }
6540 }
6541
6542
6543
6544 checkInterrupt();
6545
6546
6547 Cell current = this.storeHeap.peek();
6548
6549 byte[] currentRow = null;
6550 int offset = 0;
6551 short length = 0;
6552 if (current != null) {
6553 currentRow = current.getRowArray();
6554 offset = current.getRowOffset();
6555 length = current.getRowLength();
6556 }
6557
6558 boolean shouldStop = shouldStop(current);
6559
6560
6561
6562
6563 boolean hasFilterRow = this.filter != null && this.filter.hasFilterRow();
6564
6565
6566
6567
6568
6569 if (hasFilterRow) {
6570 if (LOG.isTraceEnabled()) {
6571 LOG.trace("filter#hasFilterRow is true which prevents partial results from being "
6572 + " formed. Changing scope of limits that may create partials");
6573 }
6574 scannerContext.setSizeLimitScope(LimitScope.BETWEEN_ROWS);
6575 scannerContext.setTimeLimitScope(LimitScope.BETWEEN_ROWS);
6576 }
6577
6578
6579
6580 if (joinedContinuationRow == null) {
6581
6582 if (shouldStop) {
6583 if (hasFilterRow) {
6584 filter.filterRowCells(results);
6585 }
6586 return scannerContext.setScannerState(NextState.NO_MORE_VALUES).hasMoreValues();
6587 }
6588
6589
6590
6591 if (filterRowKey(currentRow, offset, length)) {
6592 incrementCountOfRowsFilteredMetric(scannerContext);
6593
6594 if (isFilterDoneInternal()) {
6595 return scannerContext.setScannerState(NextState.NO_MORE_VALUES).hasMoreValues();
6596 }
6597
6598
6599
6600 incrementCountOfRowsScannedMetric(scannerContext);
6601 boolean moreRows = nextRow(scannerContext, currentRow, offset, length);
6602 if (!moreRows) {
6603 return scannerContext.setScannerState(NextState.NO_MORE_VALUES).hasMoreValues();
6604 }
6605 results.clear();
6606 continue;
6607 }
6608
6609
6610 populateResult(results, this.storeHeap, scannerContext, currentRow, offset, length);
6611
6612 if (scannerContext.checkAnyLimitReached(LimitScope.BETWEEN_CELLS)) {
6613 if (hasFilterRow) {
6614 throw new IncompatibleFilterException(
6615 "Filter whose hasFilterRow() returns true is incompatible with scans that must "
6616 + " stop mid-row because of a limit. ScannerContext:" + scannerContext);
6617 }
6618 return true;
6619 }
6620
6621
6622
6623 checkInterrupt();
6624
6625 Cell nextKv = this.storeHeap.peek();
6626 shouldStop = shouldStop(nextKv);
6627
6628 final boolean isEmptyRow = results.isEmpty();
6629
6630
6631
6632 FilterWrapper.FilterRowRetCode ret = FilterWrapper.FilterRowRetCode.NOT_CALLED;
6633 if (hasFilterRow) {
6634 ret = filter.filterRowCellsWithRet(results);
6635
6636
6637
6638
6639 long timeProgress = scannerContext.getTimeProgress();
6640 if (scannerContext.getKeepProgress()) {
6641 scannerContext.setProgress(initialBatchProgress, initialSizeProgress,
6642 initialTimeProgress);
6643 } else {
6644 scannerContext.clearProgress();
6645 }
6646 scannerContext.setTimeProgress(timeProgress);
6647 scannerContext.incrementBatchProgress(results.size());
6648 for (Cell cell : results) {
6649 scannerContext.incrementSizeProgress(CellUtil.estimatedHeapSizeOfWithoutTags(cell));
6650 }
6651 }
6652
6653 if (isEmptyRow || ret == FilterWrapper.FilterRowRetCode.EXCLUDE || filterRow()) {
6654 incrementCountOfRowsFilteredMetric(scannerContext);
6655 results.clear();
6656 boolean moreRows = nextRow(scannerContext, currentRow, offset, length);
6657 if (!moreRows) {
6658 return scannerContext.setScannerState(NextState.NO_MORE_VALUES).hasMoreValues();
6659 }
6660
6661
6662
6663 if (!shouldStop) continue;
6664 return scannerContext.setScannerState(NextState.NO_MORE_VALUES).hasMoreValues();
6665 }
6666
6667
6668
6669
6670
6671 if (this.joinedHeap != null) {
6672 boolean mayHaveData = joinedHeapMayHaveData(currentRow, offset, length);
6673 if (mayHaveData) {
6674 joinedContinuationRow = current;
6675 populateFromJoinedHeap(results, scannerContext);
6676
6677 if (scannerContext.checkAnyLimitReached(LimitScope.BETWEEN_CELLS)) {
6678 return true;
6679 }
6680 }
6681 }
6682 } else {
6683
6684 populateFromJoinedHeap(results, scannerContext);
6685 if (scannerContext.checkAnyLimitReached(LimitScope.BETWEEN_CELLS)) {
6686 return true;
6687 }
6688 }
6689
6690
6691 if (joinedContinuationRow != null) {
6692 return scannerContext.setScannerState(NextState.MORE_VALUES).hasMoreValues();
6693 }
6694
6695
6696
6697
6698 if (results.isEmpty()) {
6699 incrementCountOfRowsFilteredMetric(scannerContext);
6700 boolean moreRows = nextRow(scannerContext, currentRow, offset, length);
6701 if (!moreRows) {
6702 return scannerContext.setScannerState(NextState.NO_MORE_VALUES).hasMoreValues();
6703 }
6704 if (!shouldStop) continue;
6705 }
6706
6707 if (shouldStop) {
6708 return scannerContext.setScannerState(NextState.NO_MORE_VALUES).hasMoreValues();
6709 } else {
6710 return scannerContext.setScannerState(NextState.MORE_VALUES).hasMoreValues();
6711 }
6712 }
6713 }
6714
6715 protected void incrementCountOfRowsFilteredMetric(ScannerContext scannerContext) {
6716 if (scannerContext == null || !scannerContext.isTrackingMetrics()) return;
6717
6718 scannerContext.getMetrics().countOfRowsFiltered.incrementAndGet();
6719 }
6720
6721 protected void incrementCountOfRowsScannedMetric(ScannerContext scannerContext) {
6722 if (scannerContext == null || !scannerContext.isTrackingMetrics()) return;
6723
6724 scannerContext.getMetrics().countOfRowsScanned.incrementAndGet();
6725 }
6726
6727
6728
6729
6730
6731
6732
6733
6734 private boolean joinedHeapMayHaveData(byte[] currentRow, int offset, short length)
6735 throws IOException {
6736 Cell nextJoinedKv = joinedHeap.peek();
6737 boolean matchCurrentRow =
6738 nextJoinedKv != null && CellUtil.matchingRow(nextJoinedKv, currentRow, offset, length);
6739 boolean matchAfterSeek = false;
6740
6741
6742
6743 if (!matchCurrentRow) {
6744 Cell firstOnCurrentRow = KeyValueUtil.createFirstOnRow(currentRow, offset, length);
6745 boolean seekSuccessful = this.joinedHeap.requestSeek(firstOnCurrentRow, true, true);
6746 matchAfterSeek =
6747 seekSuccessful && joinedHeap.peek() != null
6748 && CellUtil.matchingRow(joinedHeap.peek(), currentRow, offset, length);
6749 }
6750
6751 return matchCurrentRow || matchAfterSeek;
6752 }
6753
6754
6755
6756
6757
6758
6759
6760
6761 private boolean filterRow() throws IOException {
6762
6763
6764 return filter != null && (!filter.hasFilterRow())
6765 && filter.filterRow();
6766 }
6767
6768 private boolean filterRowKey(byte[] row, int offset, short length) throws IOException {
6769 return filter != null
6770 && filter.filterRowKey(row, offset, length);
6771 }
6772
6773 protected boolean nextRow(ScannerContext scannerContext, byte[] currentRow, int offset,
6774 short length) throws IOException {
6775 assert this.joinedContinuationRow == null:
6776 "Trying to go to next row during joinedHeap read.";
6777 Cell next;
6778 while ((next = this.storeHeap.peek()) != null &&
6779 CellUtil.matchingRow(next, currentRow, offset, length)) {
6780
6781
6782 checkInterrupt();
6783 this.storeHeap.next(MOCKED_LIST);
6784 }
6785 resetFilters();
6786
6787
6788 return this.region.getCoprocessorHost() == null
6789 || this.region.getCoprocessorHost()
6790 .postScannerFilterRow(this, currentRow, offset, length);
6791 }
6792
6793 protected boolean shouldStop(Cell currentRowCell) {
6794 if (currentRowCell == null) {
6795 return true;
6796 }
6797 if (stopRow == null || Bytes.equals(stopRow, HConstants.EMPTY_START_ROW)) {
6798 return false;
6799 }
6800 int c = comparator.compareRows(currentRowCell, stopRow, 0, stopRow.length);
6801 return c > 0 || (c == 0 && !includeStopRow);
6802 }
6803
6804 @Override
6805 public synchronized void close() {
6806 if (storeHeap != null) {
6807 storeHeap.close();
6808 storeHeap = null;
6809 }
6810 if (joinedHeap != null) {
6811 joinedHeap.close();
6812 joinedHeap = null;
6813 }
6814
6815 scannerReadPoints.remove(this);
6816 this.filterClosed = true;
6817 }
6818
6819 KeyValueHeap getStoreHeapForTesting() {
6820 return storeHeap;
6821 }
6822
6823 @Override
6824 public synchronized boolean reseek(byte[] row) throws IOException {
6825 if (row == null) {
6826 throw new IllegalArgumentException("Row cannot be null.");
6827 }
6828 boolean result = false;
6829 startRegionOperation();
6830 KeyValue kv = KeyValueUtil.createFirstOnRow(row);
6831 try {
6832
6833 result = this.storeHeap.requestSeek(kv, true, true);
6834 if (this.joinedHeap != null) {
6835 result = this.joinedHeap.requestSeek(kv, true, true) || result;
6836 }
6837 } finally {
6838 closeRegionOperation();
6839 }
6840 return result;
6841 }
6842 }
6843
6844
6845
6846
6847
6848
6849
6850
6851
6852
6853
6854
6855
6856
6857
6858
6859
6860
6861
6862
6863 static HRegion newHRegion(Path tableDir, WAL wal, FileSystem fs,
6864 Configuration conf, HRegionInfo regionInfo, final HTableDescriptor htd,
6865 RegionServerServices rsServices) {
6866 try {
6867 @SuppressWarnings("unchecked")
6868 Class<? extends HRegion> regionClass =
6869 (Class<? extends HRegion>) conf.getClass(HConstants.REGION_IMPL, HRegion.class);
6870
6871 Constructor<? extends HRegion> c =
6872 regionClass.getConstructor(Path.class, WAL.class, FileSystem.class,
6873 Configuration.class, HRegionInfo.class, HTableDescriptor.class,
6874 RegionServerServices.class);
6875
6876 return c.newInstance(tableDir, wal, fs, conf, regionInfo, htd, rsServices);
6877 } catch (Throwable e) {
6878
6879 throw new IllegalStateException("Could not instantiate a region instance.", e);
6880 }
6881 }
6882
6883
6884
6885
6886
6887
6888
6889
6890
6891
6892
6893
6894
6895
6896
6897
6898
6899 public static HRegion createHRegion(final HRegionInfo info, final Path rootDir,
6900 final Configuration conf, final HTableDescriptor hTableDescriptor)
6901 throws IOException {
6902 return createHRegion(info, rootDir, conf, hTableDescriptor, null);
6903 }
6904
6905
6906
6907
6908
6909
6910
6911
6912
6913
6914
6915 public static void closeHRegion(final HRegion r) throws IOException {
6916 if (r == null) return;
6917 r.close();
6918 if (r.getWAL() == null) return;
6919 r.getWAL().close();
6920 }
6921
6922
6923
6924
6925
6926
6927
6928
6929
6930
6931
6932
6933
6934
6935 public static HRegion createHRegion(final HRegionInfo info, final Path rootDir,
6936 final Configuration conf,
6937 final HTableDescriptor hTableDescriptor,
6938 final WAL wal,
6939 final boolean initialize)
6940 throws IOException {
6941 return createHRegion(info, rootDir, conf, hTableDescriptor,
6942 wal, initialize, false);
6943 }
6944
6945
6946
6947
6948
6949
6950
6951
6952
6953
6954
6955
6956
6957
6958
6959 public static HRegion createHRegion(final HRegionInfo info, final Path rootDir,
6960 final Configuration conf,
6961 final HTableDescriptor hTableDescriptor,
6962 final WAL wal,
6963 final boolean initialize, final boolean ignoreWAL)
6964 throws IOException {
6965 Path tableDir = FSUtils.getTableDir(rootDir, info.getTable());
6966 return createHRegion(info, rootDir, tableDir, conf, hTableDescriptor, wal, initialize,
6967 ignoreWAL);
6968 }
6969
6970
6971
6972
6973
6974
6975
6976
6977
6978
6979
6980
6981
6982
6983
6984
6985 public static HRegion createHRegion(final HRegionInfo info, final Path rootDir,
6986 final Path tableDir, final Configuration conf, final HTableDescriptor hTableDescriptor,
6987 final WAL wal, final boolean initialize, final boolean ignoreWAL)
6988 throws IOException {
6989 LOG.info("creating HRegion " + info.getTable().getNameAsString()
6990 + " HTD == " + hTableDescriptor + " RootDir = " + rootDir +
6991 " Table name == " + info.getTable().getNameAsString());
6992 FileSystem fs = FileSystem.get(conf);
6993 HRegionFileSystem.createRegionOnFileSystem(conf, fs, tableDir, info);
6994 WAL effectiveWAL = wal;
6995 if (wal == null && !ignoreWAL) {
6996
6997
6998
6999 Configuration confForWAL = new Configuration(conf);
7000 FSUtils.setRootDir(confForWAL, rootDir);
7001 effectiveWAL = (new WALFactory(confForWAL,
7002 Collections.<WALActionsListener>singletonList(new MetricsWAL()),
7003 "hregion-" + RandomStringUtils.randomNumeric(8))).
7004 getWAL(info.getEncodedNameAsBytes(), info.getTable().getNamespace());
7005 }
7006 HRegion region = HRegion.newHRegion(tableDir,
7007 effectiveWAL, fs, conf, info, hTableDescriptor, null);
7008 if (initialize) region.initialize(null);
7009 return region;
7010 }
7011
7012 public static HRegion createHRegion(final HRegionInfo info, final Path rootDir,
7013 final Configuration conf,
7014 final HTableDescriptor hTableDescriptor,
7015 final WAL wal)
7016 throws IOException {
7017 return createHRegion(info, rootDir, conf, hTableDescriptor, wal, true);
7018 }
7019
7020
7021
7022
7023
7024
7025
7026
7027
7028
7029
7030
7031
7032 public static HRegion openHRegion(final HRegionInfo info,
7033 final HTableDescriptor htd, final WAL wal,
7034 final Configuration conf)
7035 throws IOException {
7036 return openHRegion(info, htd, wal, conf, null, null);
7037 }
7038
7039
7040
7041
7042
7043
7044
7045
7046
7047
7048
7049
7050
7051
7052
7053
7054 public static HRegion openHRegion(final HRegionInfo info,
7055 final HTableDescriptor htd, final WAL wal, final Configuration conf,
7056 final RegionServerServices rsServices,
7057 final CancelableProgressable reporter)
7058 throws IOException {
7059 return openHRegion(FSUtils.getRootDir(conf), info, htd, wal, conf, rsServices, reporter);
7060 }
7061
7062
7063
7064
7065
7066
7067
7068
7069
7070
7071
7072
7073
7074
7075 public static HRegion openHRegion(Path rootDir, final HRegionInfo info,
7076 final HTableDescriptor htd, final WAL wal, final Configuration conf)
7077 throws IOException {
7078 return openHRegion(rootDir, info, htd, wal, conf, null, null);
7079 }
7080
7081
7082
7083
7084
7085
7086
7087
7088
7089
7090
7091
7092
7093
7094
7095
7096 public static HRegion openHRegion(final Path rootDir, final HRegionInfo info,
7097 final HTableDescriptor htd, final WAL wal, final Configuration conf,
7098 final RegionServerServices rsServices,
7099 final CancelableProgressable reporter)
7100 throws IOException {
7101 FileSystem fs = null;
7102 if (rsServices != null) {
7103 fs = rsServices.getFileSystem();
7104 }
7105 if (fs == null) {
7106 fs = FileSystem.get(conf);
7107 }
7108 return openHRegion(conf, fs, rootDir, info, htd, wal, rsServices, reporter);
7109 }
7110
7111
7112
7113
7114
7115
7116
7117
7118
7119
7120
7121
7122
7123
7124
7125 public static HRegion openHRegion(final Configuration conf, final FileSystem fs,
7126 final Path rootDir, final HRegionInfo info, final HTableDescriptor htd, final WAL wal)
7127 throws IOException {
7128 return openHRegion(conf, fs, rootDir, info, htd, wal, null, null);
7129 }
7130
7131
7132
7133
7134
7135
7136
7137
7138
7139
7140
7141
7142
7143
7144
7145
7146
7147 public static HRegion openHRegion(final Configuration conf, final FileSystem fs,
7148 final Path rootDir, final HRegionInfo info, final HTableDescriptor htd, final WAL wal,
7149 final RegionServerServices rsServices, final CancelableProgressable reporter)
7150 throws IOException {
7151 Path tableDir = FSUtils.getTableDir(rootDir, info.getTable());
7152 return openHRegion(conf, fs, rootDir, tableDir, info, htd, wal, rsServices, reporter);
7153 }
7154
7155
7156
7157
7158
7159
7160
7161
7162
7163
7164
7165
7166
7167
7168
7169
7170
7171 public static HRegion openHRegion(final Configuration conf, final FileSystem fs,
7172 final Path rootDir, final Path tableDir, final HRegionInfo info, final HTableDescriptor htd,
7173 final WAL wal, final RegionServerServices rsServices,
7174 final CancelableProgressable reporter)
7175 throws IOException {
7176 if (info == null) throw new NullPointerException("Passed region info is null");
7177 if (LOG.isDebugEnabled()) {
7178 LOG.debug("Opening region: " + info);
7179 }
7180 HRegion r = HRegion.newHRegion(tableDir, wal, fs, conf, info, htd, rsServices);
7181 return r.openHRegion(reporter);
7182 }
7183
7184
7185
7186
7187
7188
7189
7190
7191
7192 public static HRegion openHRegion(final HRegion other, final CancelableProgressable reporter)
7193 throws IOException {
7194 HRegionFileSystem regionFs = other.getRegionFileSystem();
7195 HRegion r = newHRegion(regionFs.getTableDir(), other.getWAL(), regionFs.getFileSystem(),
7196 other.baseConf, other.getRegionInfo(), other.getTableDesc(), null);
7197 return r.openHRegion(reporter);
7198 }
7199
7200 public static Region openHRegion(final Region other, final CancelableProgressable reporter)
7201 throws IOException {
7202 return openHRegion((HRegion)other, reporter);
7203 }
7204
7205
7206
7207
7208
7209
7210
7211 protected HRegion openHRegion(final CancelableProgressable reporter)
7212 throws IOException {
7213
7214 checkCompressionCodecs();
7215
7216
7217 checkEncryption();
7218
7219 checkClassLoading();
7220 this.openSeqNum = initialize(reporter);
7221 this.mvcc.advanceTo(openSeqNum);
7222 if (wal != null && getRegionServerServices() != null && !writestate.readOnly
7223 && !recovering) {
7224
7225
7226
7227 writeRegionOpenMarker(wal, openSeqNum);
7228 }
7229 return this;
7230 }
7231
7232
7233
7234
7235
7236
7237
7238
7239
7240
7241 public static HRegion openReadOnlyFileSystemHRegion(final Configuration conf, final FileSystem fs,
7242 final Path tableDir, HRegionInfo info, final HTableDescriptor htd) throws IOException {
7243 if (info == null) {
7244 throw new NullPointerException("Passed region info is null");
7245 }
7246 if (LOG.isDebugEnabled()) {
7247 LOG.debug("Opening region (readOnly filesystem): " + info);
7248 }
7249 if (info.getReplicaId() <= 0) {
7250 info = new HRegionInfo((HRegionInfo) info, 1);
7251 }
7252 HRegion r = HRegion.newHRegion(tableDir, null, fs, conf, info, htd, null);
7253 r.writestate.setReadOnly(true);
7254 return r.openHRegion(null);
7255 }
7256
7257 public static void warmupHRegion(final HRegionInfo info,
7258 final HTableDescriptor htd, final WAL wal, final Configuration conf,
7259 final RegionServerServices rsServices,
7260 final CancelableProgressable reporter)
7261 throws IOException {
7262
7263 if (info == null) throw new NullPointerException("Passed region info is null");
7264
7265 if (LOG.isDebugEnabled()) {
7266 LOG.debug("HRegion.Warming up region: " + info);
7267 }
7268
7269 Path rootDir = FSUtils.getRootDir(conf);
7270 Path tableDir = FSUtils.getTableDir(rootDir, info.getTable());
7271
7272 FileSystem fs = null;
7273 if (rsServices != null) {
7274 fs = rsServices.getFileSystem();
7275 }
7276 if (fs == null) {
7277 fs = FileSystem.get(conf);
7278 }
7279
7280 HRegion r = HRegion.newHRegion(tableDir, wal, fs, conf, info, htd, null);
7281 r.initializeWarmup(reporter);
7282 }
7283
7284
7285 private void checkCompressionCodecs() throws IOException {
7286 for (HColumnDescriptor fam: this.htableDescriptor.getColumnFamilies()) {
7287 CompressionTest.testCompression(fam.getCompression());
7288 CompressionTest.testCompression(fam.getCompactionCompression());
7289 }
7290 }
7291
7292 private void checkEncryption() throws IOException {
7293 for (HColumnDescriptor fam: this.htableDescriptor.getColumnFamilies()) {
7294 EncryptionTest.testEncryption(conf, fam.getEncryptionType(), fam.getEncryptionKey());
7295 }
7296 }
7297
7298 private void checkClassLoading() throws IOException {
7299 RegionSplitPolicy.getSplitPolicyClass(this.htableDescriptor, conf);
7300 RegionCoprocessorHost.testTableCoprocessorAttrs(conf, this.htableDescriptor);
7301 }
7302
7303
7304
7305
7306
7307
7308 HRegion createDaughterRegionFromSplits(final HRegionInfo hri) throws IOException {
7309
7310 fs.commitDaughterRegion(hri);
7311
7312
7313 WAL daughterWAL = rsServices == null ? getWAL() :rsServices.getWAL(hri);
7314
7315 HRegion r = HRegion.newHRegion(this.fs.getTableDir(), daughterWAL,
7316 fs.getFileSystem(), this.getBaseConf(), hri, this.getTableDesc(), rsServices);
7317 r.readRequestsCount.set(this.getReadRequestsCount() / 2);
7318 r.writeRequestsCount.set(this.getWriteRequestsCount() / 2);
7319 return r;
7320 }
7321
7322
7323
7324
7325
7326
7327
7328 HRegion createMergedRegionFromMerges(final HRegionInfo mergedRegionInfo,
7329 final HRegion region_b) throws IOException {
7330 WAL mergedRegionWAL = rsServices == null ? getWAL() : rsServices.getWAL(mergedRegionInfo);
7331 HRegion r = HRegion.newHRegion(this.fs.getTableDir(), mergedRegionWAL,
7332 fs.getFileSystem(), this.getBaseConf(), mergedRegionInfo,
7333 this.getTableDesc(), this.rsServices);
7334 r.readRequestsCount.set(this.getReadRequestsCount()
7335 + region_b.getReadRequestsCount());
7336 r.writeRequestsCount.set(this.getWriteRequestsCount()
7337
7338 + region_b.getWriteRequestsCount());
7339 this.fs.commitMergedRegion(mergedRegionInfo);
7340 return r;
7341 }
7342
7343
7344
7345
7346
7347
7348
7349
7350
7351
7352
7353
7354 public static void addRegionToMETA(final HRegion meta, final HRegion r) throws IOException {
7355 meta.checkResources();
7356
7357 byte[] row = r.getRegionInfo().getRegionName();
7358 final long now = EnvironmentEdgeManager.currentTime();
7359 final List<Cell> cells = new ArrayList<Cell>(2);
7360 cells.add(new KeyValue(row, HConstants.CATALOG_FAMILY,
7361 HConstants.REGIONINFO_QUALIFIER, now,
7362 r.getRegionInfo().toByteArray()));
7363
7364 cells.add(new KeyValue(row, HConstants.CATALOG_FAMILY,
7365 HConstants.META_VERSION_QUALIFIER, now,
7366 Bytes.toBytes(HConstants.META_VERSION)));
7367 meta.put(row, HConstants.CATALOG_FAMILY, cells);
7368 }
7369
7370
7371
7372
7373
7374
7375
7376
7377
7378 public static boolean rowIsInRange(HRegionInfo info, final byte [] row) {
7379 return ((info.getStartKey().length == 0) ||
7380 (Bytes.compareTo(info.getStartKey(), row) <= 0)) &&
7381 ((info.getEndKey().length == 0) ||
7382 (Bytes.compareTo(info.getEndKey(), row) > 0));
7383 }
7384
7385 public static boolean rowIsInRange(HRegionInfo info, final byte [] row, final int offset,
7386 final short length) {
7387 return ((info.getStartKey().length == 0) ||
7388 (Bytes.compareTo(info.getStartKey(), 0, info.getStartKey().length,
7389 row, offset, length) <= 0)) &&
7390 ((info.getEndKey().length == 0) ||
7391 (Bytes.compareTo(info.getEndKey(), 0, info.getEndKey().length, row, offset, length) > 0));
7392 }
7393
7394
7395
7396
7397
7398
7399
7400 public static HRegion mergeAdjacent(final HRegion srcA, final HRegion srcB)
7401 throws IOException {
7402 HRegion a = srcA;
7403 HRegion b = srcB;
7404
7405
7406
7407 if (srcA.getRegionInfo().getStartKey() == null) {
7408 if (srcB.getRegionInfo().getStartKey() == null) {
7409 throw new IOException("Cannot merge two regions with null start key");
7410 }
7411
7412 } else if ((srcB.getRegionInfo().getStartKey() == null) ||
7413 (Bytes.compareTo(srcA.getRegionInfo().getStartKey(),
7414 srcB.getRegionInfo().getStartKey()) > 0)) {
7415 a = srcB;
7416 b = srcA;
7417 }
7418
7419 if (!(Bytes.compareTo(a.getRegionInfo().getEndKey(),
7420 b.getRegionInfo().getStartKey()) == 0)) {
7421 throw new IOException("Cannot merge non-adjacent regions");
7422 }
7423 return merge(a, b);
7424 }
7425
7426
7427
7428
7429
7430
7431
7432
7433
7434 public static HRegion merge(final HRegion a, final HRegion b) throws IOException {
7435 if (!a.getRegionInfo().getTable().equals(b.getRegionInfo().getTable())) {
7436 throw new IOException("Regions do not belong to the same table");
7437 }
7438
7439 FileSystem fs = a.getRegionFileSystem().getFileSystem();
7440
7441 a.flush(true);
7442 b.flush(true);
7443
7444
7445 a.compact(true);
7446 if (LOG.isDebugEnabled()) {
7447 LOG.debug("Files for region: " + a);
7448 a.getRegionFileSystem().logFileSystemState(LOG);
7449 }
7450 b.compact(true);
7451 if (LOG.isDebugEnabled()) {
7452 LOG.debug("Files for region: " + b);
7453 b.getRegionFileSystem().logFileSystemState(LOG);
7454 }
7455
7456 RegionMergeTransactionImpl rmt = new RegionMergeTransactionImpl(a, b, true);
7457 if (!rmt.prepare(null)) {
7458 throw new IOException("Unable to merge regions " + a + " and " + b);
7459 }
7460 HRegionInfo mergedRegionInfo = rmt.getMergedRegionInfo();
7461 LOG.info("starting merge of regions: " + a + " and " + b
7462 + " into new region " + mergedRegionInfo.getRegionNameAsString()
7463 + " with start key <"
7464 + Bytes.toStringBinary(mergedRegionInfo.getStartKey())
7465 + "> and end key <"
7466 + Bytes.toStringBinary(mergedRegionInfo.getEndKey()) + ">");
7467 HRegion dstRegion;
7468 try {
7469 dstRegion = (HRegion)rmt.execute(null, null);
7470 } catch (IOException ioe) {
7471 rmt.rollback(null, null);
7472 throw new IOException("Failed merging region " + a + " and " + b
7473 + ", and successfully rolled back");
7474 }
7475 dstRegion.compact(true);
7476
7477 if (LOG.isDebugEnabled()) {
7478 LOG.debug("Files for new region");
7479 dstRegion.getRegionFileSystem().logFileSystemState(LOG);
7480 }
7481
7482
7483 for (Store s : dstRegion.getStores()) {
7484 s.closeAndArchiveCompactedFiles();
7485 }
7486 if (dstRegion.getRegionFileSystem().hasReferences(dstRegion.getTableDesc())) {
7487 throw new IOException("Merged region " + dstRegion
7488 + " still has references after the compaction, is compaction canceled?");
7489 }
7490
7491
7492 HFileArchiver.archiveRegion(a.getBaseConf(), fs, a.getRegionInfo());
7493
7494 HFileArchiver.archiveRegion(b.getBaseConf(), fs, b.getRegionInfo());
7495
7496 LOG.info("merge completed. New region is " + dstRegion);
7497 return dstRegion;
7498 }
7499
7500 @Override
7501 public Result get(final Get get) throws IOException {
7502 checkRow(get.getRow(), "Get");
7503
7504 if (get.hasFamilies()) {
7505 for (byte [] family: get.familySet()) {
7506 checkFamily(family);
7507 }
7508 } else {
7509 for (byte[] family: this.htableDescriptor.getFamiliesKeys()) {
7510 get.addFamily(family);
7511 }
7512 }
7513 List<Cell> results = get(get, true);
7514 boolean stale = this.getRegionInfo().getReplicaId() != 0;
7515 return Result.create(results, get.isCheckExistenceOnly() ? !results.isEmpty() : null, stale);
7516 }
7517
7518 @Override
7519 public List<Cell> get(Get get, boolean withCoprocessor) throws IOException {
7520 return get(get, withCoprocessor, HConstants.NO_NONCE, HConstants.NO_NONCE);
7521 }
7522
7523 private Scan buildScanForGetWithClosestRowBefore(Get get) throws IOException {
7524 Scan scan = new Scan().withStartRow(get.getRow())
7525 .addFamily(get.getFamilyMap().keySet().iterator().next()).setReversed(true)
7526 .withStopRow(HConstants.EMPTY_END_ROW, false).setLimit(1);
7527 if (this.getRegionInfo().isMetaRegion()) {
7528 int delimiterIdx =
7529 KeyValue.getDelimiter(get.getRow(), 0, get.getRow().length, HConstants.DELIMITER);
7530 if (delimiterIdx >= 0) {
7531 scan.setFilter(new PrefixFilter(Bytes.copy(get.getRow(), 0, delimiterIdx + 1)));
7532 }
7533 }
7534 return scan;
7535 }
7536
7537 @Override
7538 public List<Cell> get(Get get, boolean withCoprocessor, long nonceGroup, long nonce)
7539 throws IOException {
7540 List<Cell> results = new ArrayList<Cell>();
7541
7542
7543 if (withCoprocessor && (coprocessorHost != null)) {
7544 if (coprocessorHost.preGet(get, results)) {
7545 return results;
7546 }
7547 }
7548 long before = EnvironmentEdgeManager.currentTime();
7549 Scan scan;
7550 if (get.isClosestRowBefore()) {
7551 scan = buildScanForGetWithClosestRowBefore(get);
7552 } else {
7553 scan = new Scan(get);
7554 }
7555
7556 if (scan.getLoadColumnFamiliesOnDemandValue() == null) {
7557 scan.setLoadColumnFamiliesOnDemand(isLoadingCfsOnDemandDefault());
7558 }
7559 RegionScanner scanner = null;
7560 try {
7561 scanner = getScanner(scan, null, nonceGroup, nonce);
7562 scanner.next(results);
7563 } finally {
7564 if (scanner != null) {
7565 scanner.close();
7566 }
7567 }
7568
7569
7570 if (withCoprocessor && (coprocessorHost != null)) {
7571 coprocessorHost.postGet(get, results);
7572 }
7573
7574 metricsUpdateForGet(results, before);
7575
7576 return results;
7577 }
7578
7579 void metricsUpdateForGet(List<Cell> results, long before) {
7580 if (this.metricsRegion != null) {
7581 this.metricsRegion.updateGet(EnvironmentEdgeManager.currentTime() - before);
7582 }
7583 if (this.rsServices != null && this.rsServices.getMetrics() != null) {
7584 rsServices.getMetrics().updateReadQueryMeter(getRegionInfo().getTable(), 1);
7585 }
7586 }
7587
7588 @Override
7589 public void mutateRow(RowMutations rm) throws IOException {
7590
7591 mutateRowsWithLocks(rm.getMutations(), Collections.singleton(rm.getRow()));
7592 }
7593
7594
7595
7596
7597
7598 public void mutateRowsWithLocks(Collection<Mutation> mutations,
7599 Collection<byte[]> rowsToLock) throws IOException {
7600 mutateRowsWithLocks(mutations, rowsToLock, HConstants.NO_NONCE, HConstants.NO_NONCE);
7601 }
7602
7603
7604
7605
7606
7607
7608
7609
7610
7611
7612
7613
7614
7615 @Override
7616 public void mutateRowsWithLocks(Collection<Mutation> mutations,
7617 Collection<byte[]> rowsToLock, long nonceGroup, long nonce) throws IOException {
7618 MultiRowMutationProcessor proc = new MultiRowMutationProcessor(mutations, rowsToLock);
7619 processRowsWithLocks(proc, -1, nonceGroup, nonce);
7620 }
7621
7622
7623
7624
7625 public ClientProtos.RegionLoadStats getLoadStatistics() {
7626 if (!regionStatsEnabled) {
7627 return null;
7628 }
7629 ClientProtos.RegionLoadStats.Builder stats = ClientProtos.RegionLoadStats.newBuilder();
7630 stats.setMemstoreLoad((int) (Math.min(100, (this.memstoreSize.get() * 100) / this
7631 .memstoreFlushSize)));
7632 if (rsServices.getHeapMemoryManager() != null) {
7633
7634
7635
7636
7637 final float occupancy = rsServices.getHeapMemoryManager().getHeapOccupancyPercent();
7638 if (occupancy != HeapMemoryManager.HEAP_OCCUPANCY_ERROR_VALUE) {
7639 stats.setHeapOccupancy((int)(occupancy * 100));
7640 }
7641 }
7642 stats.setCompactionPressure((int) (rsServices.getCompactionPressure() * 100 > 100 ? 100
7643 : rsServices.getCompactionPressure() * 100));
7644 return stats.build();
7645 }
7646
7647 @Override
7648 public void processRowsWithLocks(RowProcessor<?,?> processor) throws IOException {
7649 processRowsWithLocks(processor, rowProcessorTimeout, HConstants.NO_NONCE,
7650 HConstants.NO_NONCE);
7651 }
7652
7653 @Override
7654 public void processRowsWithLocks(RowProcessor<?,?> processor, long nonceGroup, long nonce)
7655 throws IOException {
7656 processRowsWithLocks(processor, rowProcessorTimeout, nonceGroup, nonce);
7657 }
7658
7659 @Override
7660 public void processRowsWithLocks(RowProcessor<?,?> processor, long timeout,
7661 long nonceGroup, long nonce) throws IOException {
7662
7663 for (byte[] row : processor.getRowsToLock()) {
7664 checkRow(row, "processRowsWithLocks");
7665 }
7666 if (!processor.readOnly()) {
7667 checkReadOnly();
7668 }
7669 checkResources();
7670
7671 startRegionOperation();
7672 WALEdit walEdit = new WALEdit();
7673
7674
7675 try {
7676 processor.preProcess(this, walEdit);
7677 } catch (IOException e) {
7678 closeRegionOperation();
7679 throw e;
7680 }
7681
7682 if (processor.readOnly()) {
7683 try {
7684 long now = EnvironmentEdgeManager.currentTime();
7685 doProcessRowWithTimeout(
7686 processor, now, this, null, null, timeout);
7687 processor.postProcess(this, walEdit, true);
7688 } finally {
7689 closeRegionOperation();
7690 }
7691 return;
7692 }
7693
7694 MultiVersionConcurrencyControl.WriteEntry writeEntry = null;
7695 boolean locked = false;
7696 boolean walSyncSuccessful = false;
7697 List<RowLock> acquiredRowLocks = null;
7698 long addedSize = 0;
7699 List<Mutation> mutations = new ArrayList<Mutation>();
7700 Collection<byte[]> rowsToLock = processor.getRowsToLock();
7701 long mvccNum = 0;
7702 WALKey walKey = null;
7703 try {
7704 try {
7705
7706 acquiredRowLocks = new ArrayList<RowLock>(rowsToLock.size());
7707 for (byte[] row : rowsToLock) {
7708
7709
7710 acquiredRowLocks.add(getRowLockInternal(row));
7711 }
7712
7713
7714
7715
7716
7717
7718 checkInterrupt();
7719
7720
7721 lock(this.updatesLock.readLock(), acquiredRowLocks.isEmpty() ? 1 : acquiredRowLocks.size());
7722 locked = true;
7723
7724
7725 disableInterrupts();
7726
7727 long now = EnvironmentEdgeManager.currentTime();
7728
7729
7730 doProcessRowWithTimeout(
7731 processor, now, this, mutations, walEdit, timeout);
7732
7733 if (!mutations.isEmpty()) {
7734 writeRequestsCount.add(mutations.size());
7735
7736 processor.preBatchMutate(this, walEdit);
7737
7738 long txid = 0;
7739
7740 if (!walEdit.isEmpty()) {
7741
7742 walKey = new HLogKey(this.getRegionInfo().getEncodedNameAsBytes(),
7743 this.htableDescriptor.getTableName(), WALKey.NO_SEQUENCE_ID, now,
7744 processor.getClusterIds(), nonceGroup, nonce, mvcc);
7745 preWALAppend(walKey, walEdit);
7746 txid = this.wal.append(this.htableDescriptor, this.getRegionInfo(),
7747 walKey, walEdit, true);
7748 }
7749 if(walKey == null){
7750
7751
7752 walKey = this.appendEmptyEdit(this.wal);
7753 }
7754
7755
7756 writeEntry = walKey.getWriteEntry();
7757 mvccNum = walKey.getSequenceId();
7758
7759
7760 for (Mutation m : mutations) {
7761
7762 rewriteCellTags(m.getFamilyCellMap(), m);
7763
7764 for (CellScanner cellScanner = m.cellScanner(); cellScanner.advance();) {
7765 Cell cell = cellScanner.current();
7766 CellUtil.setSequenceId(cell, mvccNum);
7767 Store store = getStore(cell);
7768 if (store == null) {
7769 checkFamily(CellUtil.cloneFamily(cell));
7770
7771 }
7772 addedSize += store.add(cell);
7773 }
7774 }
7775
7776
7777 if (locked) {
7778 this.updatesLock.readLock().unlock();
7779 locked = false;
7780 }
7781
7782
7783 releaseRowLocks(acquiredRowLocks);
7784
7785
7786 if (txid != 0) {
7787 syncOrDefer(txid, getEffectiveDurability(processor.useDurability()));
7788 }
7789
7790 if (rsServices != null && rsServices.getMetrics() != null) {
7791 rsServices.getMetrics().updateWriteQueryMeter(this.htableDescriptor.
7792 getTableName(), mutations.size());
7793 }
7794 walSyncSuccessful = true;
7795
7796 processor.postBatchMutate(this);
7797 }
7798 } finally {
7799
7800
7801
7802 if (!mutations.isEmpty() && !walSyncSuccessful) {
7803 LOG.warn("Wal sync failed. Roll back " + mutations.size() +
7804 " memstore keyvalues" + (processor.getRowsToLock().isEmpty() ? "" :
7805 (" for row(s):" + StringUtils.byteToHexString(
7806 processor.getRowsToLock().iterator().next()) + "...")));
7807 for (Mutation m : mutations) {
7808 for (CellScanner cellScanner = m.cellScanner(); cellScanner.advance();) {
7809 Cell cell = cellScanner.current();
7810 getStore(cell).rollback(cell);
7811 }
7812 }
7813 if (writeEntry != null) {
7814 mvcc.complete(writeEntry);
7815 writeEntry = null;
7816 }
7817 }
7818
7819 if (writeEntry != null) {
7820 mvcc.completeAndWait(writeEntry);
7821 }
7822 if (locked) {
7823 this.updatesLock.readLock().unlock();
7824 }
7825
7826 releaseRowLocks(acquiredRowLocks);
7827
7828 enableInterrupts();
7829 }
7830
7831
7832 processor.postProcess(this, walEdit, walSyncSuccessful);
7833
7834 } finally {
7835 closeRegionOperation();
7836 if (!mutations.isEmpty() && walSyncSuccessful &&
7837 isFlushSize(this.addAndGetGlobalMemstoreSize(addedSize))) {
7838 requestFlush();
7839 }
7840 }
7841 }
7842
7843 private void doProcessRowWithTimeout(final RowProcessor<?,?> processor,
7844 final long now,
7845 final HRegion region,
7846 final List<Mutation> mutations,
7847 final WALEdit walEdit,
7848 final long timeout) throws IOException {
7849
7850 if (timeout < 0) {
7851 try {
7852 processor.process(now, region, mutations, walEdit);
7853 } catch (IOException e) {
7854 String row = processor.getRowsToLock().isEmpty() ? "" :
7855 " on row(s):" + Bytes.toStringBinary(processor.getRowsToLock().iterator().next()) + "...";
7856 LOG.warn("RowProcessor:" + processor.getClass().getName() +
7857 " throws Exception" + row, e);
7858 throw e;
7859 }
7860 return;
7861 }
7862
7863
7864 FutureTask<Void> task =
7865 new FutureTask<Void>(new Callable<Void>() {
7866 @Override
7867 public Void call() throws IOException {
7868 try {
7869 processor.process(now, region, mutations, walEdit);
7870 return null;
7871 } catch (IOException e) {
7872 String row = processor.getRowsToLock().isEmpty() ? "" :
7873 " on row(s):" + Bytes.toStringBinary(processor.getRowsToLock().iterator().next()) + "...";
7874 LOG.warn("RowProcessor:" + processor.getClass().getName() +
7875 " throws Exception" + row, e);
7876 throw e;
7877 }
7878 }
7879 });
7880 rowProcessorExecutor.execute(task);
7881 try {
7882 task.get(timeout, TimeUnit.MILLISECONDS);
7883 } catch (InterruptedException ie) {
7884 throwOnInterrupt(ie);
7885 } catch (TimeoutException te) {
7886 String row = processor.getRowsToLock().isEmpty() ? "" :
7887 " on row(s):" + Bytes.toStringBinary(processor.getRowsToLock().iterator().next()) + "...";
7888 LOG.error("RowProcessor timeout:" + timeout + " ms" + row);
7889 throw new IOException(te);
7890 } catch (Exception e) {
7891 throw new IOException(e);
7892 }
7893 }
7894
7895
7896
7897
7898
7899
7900
7901
7902
7903
7904 private List<Cell> doGet(final Store store, final byte [] row,
7905 final Map.Entry<byte[], List<Cell>> family, final TimeRange tr)
7906 throws IOException {
7907
7908
7909
7910
7911 Collections.sort(family.getValue(), store.getComparator());
7912
7913 Get get = new Get(row);
7914 for (Cell cell : family.getValue()) {
7915 get.addColumn(family.getKey(), CellUtil.cloneQualifier(cell));
7916 }
7917 if (tr != null) get.setTimeRange(tr.getMin(), tr.getMax());
7918 return get(get, false);
7919 }
7920
7921 public Result append(Append append) throws IOException {
7922 return append(append, HConstants.NO_NONCE, HConstants.NO_NONCE);
7923 }
7924
7925 @Override
7926 public Result append(Append mutate, long nonceGroup, long nonce) throws IOException {
7927 Operation op = Operation.APPEND;
7928 byte[] row = mutate.getRow();
7929 checkRow(row, op.toString());
7930 checkFamilies(mutate.getFamilyCellMap().keySet());
7931 Durability durability = getEffectiveDurability(mutate.getDurability());
7932 boolean writeToWAL = durability != Durability.SKIP_WAL;
7933 WALEdit walEdits = null;
7934 List<Cell> allKVs = new ArrayList<Cell>(mutate.size());
7935 Map<Store, List<Cell>> tempMemstore = new LinkedHashMap<Store, List<Cell>>();
7936 Map<Store, List<Cell>> removedCellsForMemStore = new HashMap<>();
7937 long size = 0;
7938 long txid = 0;
7939 checkReadOnly();
7940 checkResources();
7941
7942 startRegionOperation(op);
7943 this.writeRequestsCount.increment();
7944 RowLock rowLock = null;
7945 WALKey walKey = null;
7946 boolean walSyncSuccess = true;
7947 try {
7948 rowLock = getRowLockInternal(row);
7949 assert rowLock != null;
7950
7951 disableInterrupts();
7952 try {
7953 lock(this.updatesLock.readLock());
7954 try {
7955
7956
7957 mvcc.await();
7958 if (this.coprocessorHost != null) {
7959 Result r = this.coprocessorHost.preAppendAfterRowLock(mutate);
7960 if (r!= null) {
7961 return r;
7962 }
7963 }
7964 long now = EnvironmentEdgeManager.currentTime();
7965
7966 for (Map.Entry<byte[], List<Cell>> family : mutate.getFamilyCellMap().entrySet()) {
7967 Store store = stores.get(family.getKey());
7968 List<Cell> kvs = new ArrayList<Cell>(family.getValue().size());
7969
7970 List<Cell> results = doGet(store, row, family, null);
7971
7972
7973
7974
7975
7976
7977
7978 int idx = 0;
7979 for (Cell cell : family.getValue()) {
7980 Cell newCell;
7981 Cell oldCell = null;
7982 if (idx < results.size()
7983 && CellUtil.matchingQualifier(results.get(idx), cell)) {
7984 oldCell = results.get(idx);
7985 long ts = Math.max(now, oldCell.getTimestamp() + 1);
7986
7987
7988
7989 List<Tag> tags = Tag.carryForwardTags(null, oldCell);
7990 tags = Tag.carryForwardTags(tags, cell);
7991 tags = carryForwardTTLTag(tags, mutate);
7992
7993 newCell = getNewCell(row, ts, cell, oldCell, Tag.fromList(tags));
7994 int newCellSize = CellUtil.estimatedSerializedSizeOf(newCell);
7995 if (newCellSize > this.maxCellSize) {
7996 String msg = "Cell with size " + newCellSize + " exceeds limit of " +
7997 this.maxCellSize + " bytes";
7998 if (LOG.isDebugEnabled()) {
7999 LOG.debug(msg);
8000 }
8001 throw new DoNotRetryIOException(msg);
8002 }
8003 idx++;
8004 } else {
8005
8006 CellUtil.updateLatestStamp(cell, now);
8007
8008
8009 newCell = getNewCell(mutate, cell);
8010 }
8011
8012
8013 if (coprocessorHost != null) {
8014 newCell = coprocessorHost.postMutationBeforeWAL(RegionObserver.MutationType.APPEND,
8015 mutate, oldCell, newCell);
8016 }
8017 kvs.add(newCell);
8018
8019
8020 if (writeToWAL) {
8021 if (walEdits == null) {
8022 walEdits = new WALEdit();
8023 }
8024 walEdits.add(newCell);
8025 }
8026 }
8027
8028
8029 tempMemstore.put(store, kvs);
8030 }
8031
8032
8033 if (walEdits != null && !walEdits.isEmpty()) {
8034 if (writeToWAL) {
8035
8036
8037
8038
8039 walKey = new HLogKey(
8040 getRegionInfo().getEncodedNameAsBytes(),
8041 this.htableDescriptor.getTableName(),
8042 WALKey.NO_SEQUENCE_ID,
8043 nonceGroup,
8044 nonce,
8045 mvcc);
8046 preWALAppend(walKey, walEdits);
8047 txid =
8048 this.wal.append(this.htableDescriptor, getRegionInfo(), walKey, walEdits, true);
8049 } else {
8050 recordMutationWithoutWal(mutate.getFamilyCellMap());
8051 }
8052 }
8053 boolean updateSeqId = false;
8054 if (walKey == null) {
8055
8056 walKey = this.appendEmptyEdit(this.wal);
8057
8058 updateSeqId = true;
8059 }
8060
8061
8062 WriteEntry writeEntry = walKey.getWriteEntry();
8063
8064 if (rsServices != null && rsServices.getNonceManager() != null) {
8065 rsServices.getNonceManager().addMvccToOperationContext(nonceGroup, nonce,
8066 writeEntry.getWriteNumber());
8067 }
8068
8069 if (updateSeqId) {
8070 updateSequenceId(tempMemstore.values(), writeEntry.getWriteNumber());
8071 }
8072
8073
8074 for (Map.Entry<Store, List<Cell>> entry : tempMemstore.entrySet()) {
8075 Store store = entry.getKey();
8076 if (store.getFamily().getMaxVersions() == 1) {
8077 List<Cell> removedCells = removedCellsForMemStore.get(store);
8078 if (removedCells == null) {
8079 removedCells = new ArrayList<>();
8080 removedCellsForMemStore.put(store, removedCells);
8081 }
8082
8083 size += store.upsert(entry.getValue(), getSmallestReadPoint(), removedCells);
8084 } else {
8085
8086 size += store.add(entry.getValue());
8087 }
8088
8089
8090 allKVs.addAll(entry.getValue());
8091 }
8092 } finally {
8093 this.updatesLock.readLock().unlock();
8094 }
8095
8096 } finally {
8097 rowLock.release();
8098 rowLock = null;
8099 }
8100
8101 walSyncSuccess = false;
8102 if(txid != 0){
8103 syncOrDefer(txid, durability);
8104 }
8105 walSyncSuccess = true;
8106 if (rsServices != null && rsServices.getMetrics() != null) {
8107 rsServices.getMetrics().updateWriteQueryMeter(this.htableDescriptor.
8108 getTableName());
8109 }
8110 } catch (Throwable t) {
8111
8112
8113 if (!walSyncSuccess) {
8114 rsServices.abort("WAL sync failed, aborting to preserve WAL as source of truth", t);
8115 }
8116
8117 throw t;
8118 } finally {
8119 if (rowLock != null) {
8120 rowLock.release();
8121 }
8122
8123 WriteEntry we = walKey != null? walKey.getWriteEntry(): null;
8124 if (!walSyncSuccess) {
8125 if (we != null) {
8126 mvcc.complete(we);
8127 }
8128 } else if (we != null) {
8129 mvcc.completeAndWait(we);
8130 }
8131
8132 enableInterrupts();
8133
8134 closeRegionOperation(op);
8135 }
8136
8137 if (this.metricsRegion != null) {
8138 this.metricsRegion.updateAppend();
8139 }
8140 if (isFlushSize(this.addAndGetGlobalMemstoreSize(size))) requestFlush();
8141 return mutate.isReturnResults() ? Result.create(allKVs) : null;
8142 }
8143
8144 private void preWALAppend(WALKey walKey, WALEdit walEdits) throws IOException {
8145 if (this.coprocessorHost != null && !walEdits.isMetaEdit()) {
8146 this.coprocessorHost.preWALAppend(walKey, walEdits);
8147 }
8148 }
8149
8150 private static Cell getNewCell(final byte [] row, final long ts, final Cell cell,
8151 final Cell oldCell, final byte [] tagBytes) {
8152
8153 Cell newCell = new KeyValue(row.length, cell.getFamilyLength(),
8154 cell.getQualifierLength(), ts, KeyValue.Type.Put,
8155 oldCell.getValueLength() + cell.getValueLength(),
8156 tagBytes == null? 0: tagBytes.length);
8157
8158 System.arraycopy(cell.getRowArray(), cell.getRowOffset(),
8159 newCell.getRowArray(), newCell.getRowOffset(), cell.getRowLength());
8160 System.arraycopy(cell.getFamilyArray(), cell.getFamilyOffset(),
8161 newCell.getFamilyArray(), newCell.getFamilyOffset(),
8162 cell.getFamilyLength());
8163 System.arraycopy(cell.getQualifierArray(), cell.getQualifierOffset(),
8164 newCell.getQualifierArray(), newCell.getQualifierOffset(),
8165 cell.getQualifierLength());
8166
8167 System.arraycopy(oldCell.getValueArray(), oldCell.getValueOffset(),
8168 newCell.getValueArray(), newCell.getValueOffset(),
8169 oldCell.getValueLength());
8170 System.arraycopy(cell.getValueArray(), cell.getValueOffset(),
8171 newCell.getValueArray(),
8172 newCell.getValueOffset() + oldCell.getValueLength(),
8173 cell.getValueLength());
8174
8175 if (tagBytes != null) {
8176 System.arraycopy(tagBytes, 0, newCell.getTagsArray(), newCell.getTagsOffset(),
8177 tagBytes.length);
8178 }
8179 return newCell;
8180 }
8181
8182 private static Cell getNewCell(final Mutation mutate, final Cell cell) {
8183 Cell newCell = null;
8184 if (mutate.getTTL() != Long.MAX_VALUE) {
8185
8186 newCell = new KeyValue(cell.getRowArray(), cell.getRowOffset(),
8187 cell.getRowLength(),
8188 cell.getFamilyArray(), cell.getFamilyOffset(),
8189 cell.getFamilyLength(),
8190 cell.getQualifierArray(), cell.getQualifierOffset(),
8191 cell.getQualifierLength(),
8192 cell.getTimestamp(), KeyValue.Type.codeToType(cell.getTypeByte()),
8193 cell.getValueArray(), cell.getValueOffset(), cell.getValueLength(),
8194 carryForwardTTLTag(mutate));
8195 } else {
8196 newCell = cell;
8197 }
8198 return newCell;
8199 }
8200
8201 public Result increment(Increment increment) throws IOException {
8202 return increment(increment, HConstants.NO_NONCE, HConstants.NO_NONCE);
8203 }
8204
8205
8206
8207
8208
8209
8210
8211
8212
8213 @Override
8214 public Result increment(Increment mutation, long nonceGroup, long nonce)
8215 throws IOException {
8216 Operation op = Operation.INCREMENT;
8217 checkReadOnly();
8218 checkResources();
8219 checkRow(mutation.getRow(), op.toString());
8220 checkFamilies(mutation.getFamilyCellMap().keySet());
8221 startRegionOperation(op);
8222 this.writeRequestsCount.increment();
8223 try {
8224
8225
8226
8227
8228
8229
8230
8231
8232
8233
8234
8235
8236
8237
8238
8239
8240 return doIncrement(mutation, nonceGroup, nonce);
8241 } finally {
8242 if (rsServices != null && rsServices.getMetrics() != null) {
8243 rsServices.getMetrics().updateWriteQueryMeter(this.htableDescriptor.
8244 getTableName());
8245 }
8246 if (this.metricsRegion != null) this.metricsRegion.updateIncrement();
8247 closeRegionOperation(op);
8248 }
8249 }
8250
8251 private Result doIncrement(Increment increment, long nonceGroup, long nonce) throws IOException {
8252 RowLock rowLock = null;
8253 WALKey walKey = null;
8254 boolean walSyncSuccess = true;
8255 long accumulatedResultSize = 0;
8256 List<Cell> allKVs = new ArrayList<Cell>(increment.size());
8257 Map<Store, List<Cell>> removedCellsForMemStore = new HashMap<>();
8258 Map<Store, List<Cell>> forMemStore = new HashMap<>();
8259 Durability effectiveDurability = getEffectiveDurability(increment.getDurability());
8260 try {
8261 rowLock = getRowLockInternal(increment.getRow());
8262
8263 disableInterrupts();
8264 long txid = 0;
8265 try {
8266 lock(this.updatesLock.readLock());
8267 try {
8268
8269
8270 this.mvcc.await();
8271 if (this.coprocessorHost != null) {
8272 Result r = this.coprocessorHost.preIncrementAfterRowLock(increment);
8273 if (r != null) return r;
8274 }
8275 long now = EnvironmentEdgeManager.currentTime();
8276 final boolean writeToWAL = effectiveDurability != Durability.SKIP_WAL;
8277 WALEdit walEdits = null;
8278
8279
8280 for (Map.Entry<byte [], List<Cell>> entry: increment.getFamilyCellMap().entrySet()) {
8281 byte [] columnFamilyName = entry.getKey();
8282 List<Cell> increments = entry.getValue();
8283 Store store = this.stores.get(columnFamilyName);
8284
8285
8286 List<Cell> results = applyIncrementsToColumnFamily(increment, columnFamilyName,
8287 sort(increments, store.getComparator()), now,
8288 MultiVersionConcurrencyControl.NO_WRITE_NUMBER, allKVs, null);
8289 if (!results.isEmpty()) {
8290 forMemStore.put(store, results);
8291
8292 if (writeToWAL) {
8293 if (walEdits == null) walEdits = new WALEdit();
8294 walEdits.getCells().addAll(results);
8295 }
8296 }
8297 }
8298 boolean updateSeqId = false;
8299
8300
8301
8302
8303 checkInterrupt();
8304
8305
8306 if (walEdits != null && !walEdits.isEmpty()) {
8307
8308
8309
8310 walKey = new HLogKey(this.getRegionInfo().getEncodedNameAsBytes(),
8311 this.htableDescriptor.getTableName(), WALKey.NO_SEQUENCE_ID, nonceGroup, nonce,
8312 getMVCC());
8313 preWALAppend(walKey, walEdits);
8314 txid =
8315 this.wal.append(this.htableDescriptor, this.getRegionInfo(), walKey, walEdits, true);
8316 } else {
8317
8318 walKey = this.appendEmptyEdit(this.wal);
8319
8320 updateSeqId = true;
8321 }
8322
8323 WriteEntry writeEntry = walKey.getWriteEntry();
8324
8325 if (rsServices != null && rsServices.getNonceManager() != null) {
8326 rsServices.getNonceManager().addMvccToOperationContext(nonceGroup, nonce,
8327 writeEntry.getWriteNumber());
8328 }
8329
8330 if (updateSeqId) {
8331 updateSequenceId(forMemStore.values(), writeEntry.getWriteNumber());
8332 }
8333
8334
8335 for (Map.Entry<Store, List<Cell>> entry: forMemStore.entrySet()) {
8336 Store store = entry.getKey();
8337 List<Cell> results = entry.getValue();
8338 if (store.getFamily().getMaxVersions() == 1) {
8339 List<Cell> removedCells = removedCellsForMemStore.get(store);
8340 if (removedCells == null) {
8341 removedCells = new ArrayList<>();
8342 removedCellsForMemStore.put(store, removedCells);
8343 }
8344
8345 accumulatedResultSize += store.upsert(results, getSmallestReadPoint(), removedCells);
8346 } else {
8347
8348 accumulatedResultSize += store.add(entry.getValue());
8349 }
8350 }
8351 } finally {
8352 this.updatesLock.readLock().unlock();
8353 }
8354 } finally {
8355 rowLock.release();
8356 rowLock = null;
8357 }
8358 walSyncSuccess = false;
8359
8360 if(txid != 0) {
8361 syncOrDefer(txid, effectiveDurability);
8362 }
8363 walSyncSuccess = true;
8364 } catch (Throwable t) {
8365
8366
8367 if (!walSyncSuccess) {
8368 rsServices.abort("WAL sync failed, aborting to preserve WAL as source of truth", t);
8369 }
8370
8371 throw t;
8372 } finally {
8373 if (rowLock != null) {
8374 rowLock.release();
8375 }
8376
8377 WriteEntry we = walKey != null ? walKey.getWriteEntry() : null;
8378 if (!walSyncSuccess) {
8379 if (we != null) {
8380 mvcc.complete(we);
8381 }
8382 } else {
8383 if (we != null) {
8384 mvcc.completeAndWait(we);
8385 }
8386 }
8387 enableInterrupts();
8388 }
8389
8390
8391 if (isFlushSize(this.addAndGetGlobalMemstoreSize(accumulatedResultSize))) requestFlush();
8392 return increment.isReturnResults() ? Result.create(allKVs) : null;
8393 }
8394
8395
8396
8397
8398 private static List<Cell> sort(List<Cell> cells, final Comparator<Cell> comparator) {
8399 Collections.sort(cells, comparator);
8400 return cells;
8401 }
8402
8403
8404
8405
8406
8407
8408
8409
8410
8411
8412
8413
8414 private List<Cell> applyIncrementsToColumnFamily(Increment increment, byte[] columnFamilyName,
8415 List<Cell> sortedIncrements, long now, long mvccNum, List<Cell> allKVs,
8416 final IsolationLevel isolation)
8417 throws IOException {
8418 List<Cell> results = new ArrayList<Cell>(sortedIncrements.size());
8419 byte [] row = increment.getRow();
8420
8421 List<Cell> currentValues =
8422 getIncrementCurrentValue(increment, columnFamilyName, sortedIncrements, isolation);
8423
8424
8425 int idx = 0;
8426 for (int i = 0; i < sortedIncrements.size(); i++) {
8427 Cell inc = sortedIncrements.get(i);
8428 long incrementAmount = getLongValue(inc);
8429
8430 boolean writeBack = (incrementAmount != 0);
8431
8432 List<Tag> tags = Tag.carryForwardTags(inc);
8433
8434 Cell currentValue = null;
8435 long ts = now;
8436 boolean firstWrite = false;
8437 if (idx < currentValues.size() && CellUtil.matchingQualifier(currentValues.get(idx), inc)) {
8438 currentValue = currentValues.get(idx);
8439 ts = Math.max(now, currentValue.getTimestamp() + 1);
8440 incrementAmount += getLongValue(currentValue);
8441
8442 tags = Tag.carryForwardTags(tags, currentValue);
8443 if (i < (sortedIncrements.size() - 1) &&
8444 !CellUtil.matchingQualifier(inc, sortedIncrements.get(i + 1))) idx++;
8445 } else {
8446 firstWrite = true;
8447 }
8448
8449
8450 byte [] incrementAmountInBytes = Bytes.toBytes(incrementAmount);
8451 tags = carryForwardTTLTag(tags, increment);
8452
8453 Cell newValue = new KeyValue(row, 0, row.length,
8454 columnFamilyName, 0, columnFamilyName.length,
8455 inc.getQualifierArray(), inc.getQualifierOffset(), inc.getQualifierLength(),
8456 ts, KeyValue.Type.Put,
8457 incrementAmountInBytes, 0, incrementAmountInBytes.length,
8458 tags);
8459
8460
8461
8462 if (mvccNum != MultiVersionConcurrencyControl.NO_WRITE_NUMBER) {
8463 CellUtil.setSequenceId(newValue, mvccNum);
8464 }
8465
8466
8467 if (coprocessorHost != null) {
8468 newValue = coprocessorHost.postMutationBeforeWAL(
8469 RegionObserver.MutationType.INCREMENT, increment, currentValue, newValue);
8470 }
8471 allKVs.add(newValue);
8472 if (writeBack || firstWrite) {
8473 results.add(newValue);
8474 }
8475 }
8476 return results;
8477 }
8478
8479
8480
8481
8482
8483 private static long getLongValue(final Cell cell) throws DoNotRetryIOException {
8484 int len = cell.getValueLength();
8485 if (len != Bytes.SIZEOF_LONG) {
8486
8487 throw new DoNotRetryIOException("Field is not a long, it's " + len + " bytes wide");
8488 }
8489 return Bytes.toLong(cell.getValueArray(), cell.getValueOffset(), len);
8490 }
8491
8492
8493
8494
8495
8496
8497
8498
8499
8500 private List<Cell> getIncrementCurrentValue(final Increment increment, byte [] columnFamily,
8501 final List<Cell> increments, final IsolationLevel isolation)
8502 throws IOException {
8503 Get get = new Get(increment.getRow());
8504 if (isolation != null) get.setIsolationLevel(isolation);
8505 for (Cell cell: increments) {
8506 get.addColumn(columnFamily, CellUtil.cloneQualifier(cell));
8507 }
8508 TimeRange tr = increment.getTimeRange();
8509 if (tr != null) {
8510 get.setTimeRange(tr.getMin(), tr.getMax());
8511 }
8512 return get(get, false);
8513 }
8514
8515 private static List<Tag> carryForwardTTLTag(final Mutation mutation) {
8516 return carryForwardTTLTag(null, mutation);
8517 }
8518
8519
8520
8521
8522 private static List<Tag> carryForwardTTLTag(final List<Tag> tagsOrNull,
8523 final Mutation mutation) {
8524 long ttl = mutation.getTTL();
8525 if (ttl == Long.MAX_VALUE) return tagsOrNull;
8526 List<Tag> tags = tagsOrNull;
8527
8528
8529
8530 if (tags == null) {
8531 tags = new ArrayList<Tag>(1);
8532 } else {
8533
8534 Iterator<Tag> tagsItr = tags.iterator();
8535 while (tagsItr.hasNext()) {
8536 Tag tag = tagsItr.next();
8537 if (tag.getType() == TagType.TTL_TAG_TYPE) {
8538 tagsItr.remove();
8539 break;
8540 }
8541 }
8542 }
8543 tags.add(new Tag(TagType.TTL_TAG_TYPE, Bytes.toBytes(ttl)));
8544 return tags;
8545 }
8546
8547
8548
8549
8550
8551 private void checkFamily(final byte [] family)
8552 throws NoSuchColumnFamilyException {
8553 if (!this.htableDescriptor.hasFamily(family)) {
8554 throw new NoSuchColumnFamilyException("Column family " +
8555 Bytes.toString(family) + " does not exist in region " + this
8556 + " in table " + this.htableDescriptor);
8557 }
8558 }
8559
8560 public static final long FIXED_OVERHEAD = ClassSize.align(
8561 ClassSize.OBJECT +
8562 ClassSize.ARRAY +
8563 50 * ClassSize.REFERENCE + 3 * Bytes.SIZEOF_INT +
8564 (15 * Bytes.SIZEOF_LONG) +
8565 5 * Bytes.SIZEOF_BOOLEAN);
8566
8567
8568
8569
8570
8571
8572
8573
8574
8575
8576
8577 public static final long DEEP_OVERHEAD = FIXED_OVERHEAD +
8578 ClassSize.OBJECT +
8579 (2 * ClassSize.ATOMIC_BOOLEAN) +
8580 (4 * ClassSize.ATOMIC_LONG) +
8581
8582 (3 * ClassSize.CONCURRENT_HASHMAP) +
8583 WriteState.HEAP_SIZE +
8584 ClassSize.CONCURRENT_SKIPLISTMAP + ClassSize.CONCURRENT_SKIPLISTMAP_ENTRY +
8585 (2 * ClassSize.REENTRANT_LOCK) +
8586 MultiVersionConcurrencyControl.FIXED_SIZE
8587 + ClassSize.TREEMAP
8588 + 2 * ClassSize.ATOMIC_INTEGER
8589 ;
8590
8591 @Override
8592 public long heapSize() {
8593 long heapSize = DEEP_OVERHEAD;
8594 for (Store store : this.stores.values()) {
8595 heapSize += store.heapSize();
8596 }
8597
8598 return heapSize;
8599 }
8600
8601
8602
8603
8604
8605 private static void printUsageAndExit(final String message) {
8606 if (message != null && message.length() > 0) System.out.println(message);
8607 System.out.println("Usage: HRegion CATALOG_TABLE_DIR [major_compact]");
8608 System.out.println("Options:");
8609 System.out.println(" major_compact Pass this option to major compact " +
8610 "passed region.");
8611 System.out.println("Default outputs scan of passed region.");
8612 System.exit(1);
8613 }
8614
8615 @Override
8616 public boolean registerService(Service instance) {
8617
8618
8619
8620 Descriptors.ServiceDescriptor serviceDesc = instance.getDescriptorForType();
8621 String serviceName = CoprocessorRpcUtils.getServiceName(serviceDesc);
8622 if (coprocessorServiceHandlers.containsKey(serviceName)) {
8623 LOG.error("Coprocessor service " + serviceName +
8624 " already registered, rejecting request from " + instance
8625 );
8626 return false;
8627 }
8628
8629 coprocessorServiceHandlers.put(serviceName, instance);
8630 if (LOG.isDebugEnabled()) {
8631 LOG.debug("Registered coprocessor service: region=" +
8632 Bytes.toStringBinary(getRegionInfo().getRegionName()) +
8633 " service=" + serviceName);
8634 }
8635 return true;
8636 }
8637
8638 @Override
8639 public Message execService(RpcController controller, CoprocessorServiceCall call)
8640 throws IOException {
8641 String serviceName = call.getServiceName();
8642 String methodName = call.getMethodName();
8643 if (!coprocessorServiceHandlers.containsKey(serviceName)) {
8644 throw new UnknownProtocolException(null,
8645 "No registered coprocessor service found for name "+serviceName+
8646 " in region "+Bytes.toStringBinary(getRegionInfo().getRegionName()));
8647 }
8648
8649 Service service = coprocessorServiceHandlers.get(serviceName);
8650 Descriptors.ServiceDescriptor serviceDesc = service.getDescriptorForType();
8651 Descriptors.MethodDescriptor methodDesc = serviceDesc.findMethodByName(methodName);
8652 if (methodDesc == null) {
8653 throw new UnknownProtocolException(service.getClass(),
8654 "Unknown method "+methodName+" called on service "+serviceName+
8655 " in region "+Bytes.toStringBinary(getRegionInfo().getRegionName()));
8656 }
8657
8658 Message.Builder builder = service.getRequestPrototype(methodDesc).newBuilderForType();
8659 ProtobufUtil.mergeFrom(builder, call.getRequest());
8660 Message request = builder.build();
8661
8662 if (coprocessorHost != null) {
8663 request = coprocessorHost.preEndpointInvocation(service, methodName, request);
8664 }
8665
8666 final Message.Builder responseBuilder =
8667 service.getResponsePrototype(methodDesc).newBuilderForType();
8668 service.callMethod(methodDesc, controller, request, new RpcCallback<Message>() {
8669 @Override
8670 public void run(Message message) {
8671 if (message != null) {
8672 responseBuilder.mergeFrom(message);
8673 }
8674 }
8675 });
8676
8677 if (coprocessorHost != null) {
8678 coprocessorHost.postEndpointInvocation(service, methodName, request, responseBuilder);
8679 }
8680
8681 IOException exception = ResponseConverter.getControllerException(controller);
8682 if (exception != null) {
8683 throw exception;
8684 }
8685
8686 return responseBuilder.build();
8687 }
8688
8689
8690
8691
8692
8693
8694 private static void processTable(final FileSystem fs, final Path p,
8695 final WALFactory walFactory, final Configuration c,
8696 final boolean majorCompact)
8697 throws IOException {
8698 HRegion region;
8699 FSTableDescriptors fst = new FSTableDescriptors(c);
8700
8701 if (FSUtils.getTableName(p).equals(TableName.META_TABLE_NAME)) {
8702 final WAL wal = walFactory.getMetaWAL(
8703 HRegionInfo.FIRST_META_REGIONINFO.getEncodedNameAsBytes());
8704 region = HRegion.newHRegion(p, wal, fs, c,
8705 HRegionInfo.FIRST_META_REGIONINFO, fst.get(TableName.META_TABLE_NAME), null);
8706 } else {
8707 throw new IOException("Not a known catalog table: " + p.toString());
8708 }
8709 try {
8710 region.mvcc.advanceTo(region.initialize(null));
8711 if (majorCompact) {
8712 region.compact(true);
8713 } else {
8714
8715 Scan scan = new Scan();
8716
8717 RegionScanner scanner = region.getScanner(scan);
8718 try {
8719 List<Cell> kvs = new ArrayList<Cell>();
8720 boolean done;
8721 do {
8722 kvs.clear();
8723 done = scanner.next(kvs);
8724 if (kvs.size() > 0) LOG.info(kvs);
8725 } while (done);
8726 } finally {
8727 scanner.close();
8728 }
8729 }
8730 } finally {
8731 region.close();
8732 }
8733 }
8734
8735 boolean shouldForceSplit() {
8736 return this.splitRequest;
8737 }
8738
8739 byte[] getExplicitSplitPoint() {
8740 return this.explicitSplitPoint;
8741 }
8742
8743 void forceSplit(byte[] sp) {
8744
8745
8746 this.splitRequest = true;
8747 if (sp != null) {
8748 this.explicitSplitPoint = sp;
8749 }
8750 }
8751
8752 void clearSplit() {
8753 this.splitRequest = false;
8754 this.explicitSplitPoint = null;
8755 }
8756
8757
8758
8759
8760 protected void prepareToSplit() {
8761
8762 }
8763
8764
8765
8766
8767
8768
8769
8770 public byte[] checkSplit() {
8771
8772 if (this.getRegionInfo().isMetaTable() ||
8773 TableName.NAMESPACE_TABLE_NAME.equals(this.getRegionInfo().getTable())) {
8774 if (shouldForceSplit()) {
8775 LOG.warn("Cannot split meta region in HBase 0.20 and above");
8776 }
8777 return null;
8778 }
8779
8780
8781 if (this.isRecovering()) {
8782 LOG.info("Cannot split region " + this.getRegionInfo().getEncodedName() + " in recovery.");
8783 return null;
8784 }
8785
8786 if (!splitPolicy.shouldSplit()) {
8787 return null;
8788 }
8789
8790 byte[] ret = splitPolicy.getSplitPoint();
8791
8792 if (ret != null) {
8793 try {
8794 checkRow(ret, "calculated split");
8795 } catch (IOException e) {
8796 LOG.error("Ignoring invalid split", e);
8797 return null;
8798 }
8799 }
8800 return ret;
8801 }
8802
8803
8804
8805
8806 public int getCompactPriority() {
8807 int count = Integer.MAX_VALUE;
8808 for (Store store : stores.values()) {
8809 count = Math.min(count, store.getCompactPriority());
8810 }
8811 return count;
8812 }
8813
8814
8815
8816 @Override
8817 public RegionCoprocessorHost getCoprocessorHost() {
8818 return coprocessorHost;
8819 }
8820
8821
8822 public void setCoprocessorHost(final RegionCoprocessorHost coprocessorHost) {
8823 this.coprocessorHost = coprocessorHost;
8824 }
8825
8826 @Override
8827 public void startRegionOperation() throws IOException {
8828 startRegionOperation(Operation.ANY);
8829 }
8830
8831 @Override
8832 @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="SF_SWITCH_FALLTHROUGH",
8833 justification="Intentional")
8834 public void startRegionOperation(Operation op) throws IOException {
8835 boolean isInterruptableOp = false;
8836 switch (op) {
8837 case GET:
8838 case SCAN:
8839 isInterruptableOp = true;
8840 checkReadsEnabled();
8841 break;
8842 case INCREMENT:
8843 case APPEND:
8844 case PUT:
8845 case DELETE:
8846 case BATCH_MUTATE:
8847 case CHECK_AND_MUTATE:
8848 isInterruptableOp = true;
8849 break;
8850 default:
8851 break;
8852 }
8853
8854 if (isRecovering() && (this.disallowWritesInRecovering ||
8855 (op != Operation.PUT && op != Operation.DELETE && op != Operation.BATCH_MUTATE))) {
8856 throw new RegionInRecoveryException(getRegionInfo().getRegionNameAsString() +
8857 " is recovering; cannot take reads");
8858 }
8859 if (op == Operation.MERGE_REGION || op == Operation.SPLIT_REGION
8860 || op == Operation.COMPACT_REGION || op == Operation.COMPACT_SWITCH) {
8861
8862
8863 return;
8864 }
8865 if (this.closing.get()) {
8866 throw new NotServingRegionException(getRegionInfo().getRegionNameAsString() + " is closing");
8867 }
8868 lock(lock.readLock());
8869
8870
8871 Thread thisThread = Thread.currentThread();
8872 if (isInterruptableOp) {
8873 regionLockHolders.put(thisThread, true);
8874 }
8875 if (this.closed.get()) {
8876 lock.readLock().unlock();
8877 throw new NotServingRegionException(getRegionInfo().getRegionNameAsString() + " is closed");
8878 }
8879
8880
8881 if (op == Operation.SNAPSHOT) {
8882 for (Store store : stores.values()) {
8883 if (store instanceof HStore) {
8884 ((HStore)store).preSnapshotOperation();
8885 }
8886 }
8887 }
8888 try {
8889 if (coprocessorHost != null) {
8890 coprocessorHost.postStartRegionOperation(op);
8891 }
8892 } catch (Exception e) {
8893 if (isInterruptableOp) {
8894
8895
8896 regionLockHolders.remove(thisThread);
8897 }
8898 lock.readLock().unlock();
8899 throw new IOException(e);
8900 }
8901 }
8902
8903 @Override
8904 public void closeRegionOperation() throws IOException {
8905 closeRegionOperation(Operation.ANY);
8906 }
8907
8908 @Override
8909 public void closeRegionOperation(Operation operation) throws IOException {
8910 if (operation == Operation.SNAPSHOT) {
8911 for (Store store: stores.values()) {
8912 if (store instanceof HStore) {
8913 ((HStore)store).postSnapshotOperation();
8914 }
8915 }
8916 }
8917 Thread thisThread = Thread.currentThread();
8918 regionLockHolders.remove(thisThread);
8919 lock.readLock().unlock();
8920 if (coprocessorHost != null) {
8921 coprocessorHost.postCloseRegionOperation(operation);
8922 }
8923 }
8924
8925
8926
8927
8928
8929
8930
8931
8932
8933
8934 private void startBulkRegionOperation(boolean writeLockNeeded) throws IOException {
8935 if (this.closing.get()) {
8936 throw new NotServingRegionException(getRegionInfo().getRegionNameAsString() + " is closing");
8937 }
8938 if (writeLockNeeded) lock(lock.writeLock());
8939 else lock(lock.readLock());
8940 if (this.closed.get()) {
8941 if (writeLockNeeded) lock.writeLock().unlock();
8942 else lock.readLock().unlock();
8943 throw new NotServingRegionException(getRegionInfo().getRegionNameAsString() + " is closed");
8944 }
8945 regionLockHolders.put(Thread.currentThread(), true);
8946 }
8947
8948
8949
8950
8951
8952 private void closeBulkRegionOperation(){
8953 regionLockHolders.remove(Thread.currentThread());
8954 if (lock.writeLock().isHeldByCurrentThread()) lock.writeLock().unlock();
8955 else lock.readLock().unlock();
8956 }
8957
8958
8959
8960
8961
8962 protected void disableInterrupts() {
8963
8964
8965
8966
8967
8968 synchronized (regionLockHolders) {
8969 Thread currentThread = Thread.currentThread();
8970 Boolean value = regionLockHolders.get(currentThread);
8971 if (value != null) {
8972 regionLockHolders.put(currentThread, false);
8973 }
8974 }
8975 }
8976
8977
8978
8979
8980
8981 protected void enableInterrupts() {
8982
8983
8984
8985
8986
8987 synchronized (regionLockHolders) {
8988 Thread currentThread = Thread.currentThread();
8989 Boolean value = regionLockHolders.get(currentThread);
8990 if (value != null) {
8991 regionLockHolders.put(currentThread, true);
8992 }
8993 }
8994 }
8995
8996
8997
8998
8999
9000
9001 private void interruptRegionOperations() {
9002 for (Map.Entry<Thread, Boolean> entry: regionLockHolders.entrySet()) {
9003
9004
9005 if (entry.getValue().booleanValue()) {
9006 entry.getKey().interrupt();
9007 }
9008 }
9009 }
9010
9011
9012
9013
9014
9015 private void recordMutationWithoutWal(final Map<byte [], List<Cell>> familyMap) {
9016 numMutationsWithoutWAL.increment();
9017 if (numMutationsWithoutWAL.get() <= 1) {
9018 LOG.info("writing data to region " + this +
9019 " with WAL disabled. Data may be lost in the event of a crash.");
9020 }
9021
9022 long mutationSize = 0;
9023 for (List<Cell> cells: familyMap.values()) {
9024 assert cells instanceof RandomAccess;
9025 int listSize = cells.size();
9026 for (int i=0; i < listSize; i++) {
9027 Cell cell = cells.get(i);
9028
9029 mutationSize += KeyValueUtil.length(cell);
9030 }
9031 }
9032
9033 dataInMemoryWithoutWAL.add(mutationSize);
9034 }
9035
9036 private void lock(final Lock lock) throws IOException {
9037 lock(lock, 1);
9038 }
9039
9040
9041
9042
9043
9044
9045 private void lock(final Lock lock, final int multiplier) throws IOException {
9046 try {
9047 final long waitTime = Math.min(maxBusyWaitDuration,
9048 busyWaitDuration * Math.min(multiplier, maxBusyWaitMultiplier));
9049 if (!lock.tryLock(waitTime, TimeUnit.MILLISECONDS)) {
9050 final String regionName =
9051 this.getRegionInfo() == null ? "unknown" : this.getRegionInfo().getRegionNameAsString();
9052 final String serverName = this.getRegionServerServices() == null ? "unknown" :
9053 (this.getRegionServerServices().getServerName() == null ? "unknown" :
9054 this.getRegionServerServices().getServerName().toString());
9055 RegionTooBusyException rtbe = new RegionTooBusyException(
9056 "failed to get a lock in " + waitTime + " ms. " + "regionName=" + regionName + ", server="
9057 + serverName);
9058 LOG.warn("Region is too busy to allow lock acquisition.", rtbe);
9059 throw rtbe;
9060 }
9061 } catch (InterruptedException ie) {
9062 if (LOG.isDebugEnabled()) {
9063 LOG.debug("Interrupted while waiting for a lock in region " + this);
9064 }
9065 throw throwOnInterrupt(ie);
9066 }
9067 }
9068
9069
9070
9071
9072
9073
9074
9075 private void syncOrDefer(long txid, Durability durability) throws IOException {
9076 if (this.getRegionInfo().isMetaRegion()) {
9077 this.wal.sync(txid);
9078 } else {
9079 switch(durability) {
9080 case USE_DEFAULT:
9081
9082 if (shouldSyncWAL()) {
9083 this.wal.sync(txid);
9084 }
9085 break;
9086 case SKIP_WAL:
9087
9088 break;
9089 case ASYNC_WAL:
9090
9091 break;
9092 case SYNC_WAL:
9093 this.wal.sync(txid, false);
9094 break;
9095 case FSYNC_WAL:
9096
9097 this.wal.sync(txid, true);
9098 break;
9099 default:
9100 throw new RuntimeException("Unknown durability " + durability);
9101 }
9102 }
9103 }
9104
9105
9106
9107
9108 private boolean shouldSyncWAL() {
9109 return durability.ordinal() > Durability.ASYNC_WAL.ordinal();
9110 }
9111
9112
9113
9114
9115 private static final List<Cell> MOCKED_LIST = new AbstractList<Cell>() {
9116
9117 @Override
9118 public void add(int index, Cell element) {
9119
9120 }
9121
9122 @Override
9123 public boolean addAll(int index, Collection<? extends Cell> c) {
9124 return false;
9125 }
9126
9127 @Override
9128 public KeyValue get(int index) {
9129 throw new UnsupportedOperationException();
9130 }
9131
9132 @Override
9133 public int size() {
9134 return 0;
9135 }
9136 };
9137
9138
9139
9140
9141
9142
9143
9144
9145
9146
9147 public static void main(String[] args) throws IOException {
9148 if (args.length < 1) {
9149 printUsageAndExit(null);
9150 }
9151 boolean majorCompact = false;
9152 if (args.length > 1) {
9153 if (!args[1].toLowerCase(Locale.ROOT).startsWith("major")) {
9154 printUsageAndExit("ERROR: Unrecognized option <" + args[1] + ">");
9155 }
9156 majorCompact = true;
9157 }
9158 final Path tableDir = new Path(args[0]);
9159 final Configuration c = HBaseConfiguration.create();
9160 final FileSystem fs = FileSystem.get(c);
9161 final Path logdir = new Path(c.get("hbase.tmp.dir"));
9162 final String logname = "wal" + FSUtils.getTableName(tableDir) + System.currentTimeMillis();
9163
9164 final Configuration walConf = new Configuration(c);
9165 FSUtils.setRootDir(walConf, logdir);
9166 final WALFactory wals = new WALFactory(walConf, null, logname);
9167 try {
9168 processTable(fs, tableDir, wals, c, majorCompact);
9169 } finally {
9170 wals.close();
9171
9172 BlockCache bc = new CacheConfig(c).getBlockCache();
9173 if (bc != null) bc.shutdown();
9174 }
9175 }
9176
9177 @Override
9178 public long getOpenSeqNum() {
9179 return this.openSeqNum;
9180 }
9181
9182 @Override
9183 public Map<byte[], Long> getMaxStoreSeqId() {
9184 return this.maxSeqIdInStores;
9185 }
9186
9187 @Override
9188 public long getOldestSeqIdOfStore(byte[] familyName) {
9189 return wal.getEarliestMemstoreSeqNum(getRegionInfo().getEncodedNameAsBytes(), familyName);
9190 }
9191
9192 @Override
9193 public CompactionState getCompactionState() {
9194 boolean hasMajor = majorInProgress.get() > 0, hasMinor = minorInProgress.get() > 0;
9195 return (hasMajor ? (hasMinor ? CompactionState.MAJOR_AND_MINOR : CompactionState.MAJOR)
9196 : (hasMinor ? CompactionState.MINOR : CompactionState.NONE));
9197 }
9198
9199 public void reportCompactionRequestStart(boolean isMajor){
9200 (isMajor ? majorInProgress : minorInProgress).incrementAndGet();
9201 }
9202
9203 public void reportCompactionRequestEnd(boolean isMajor, int numFiles, long filesSizeCompacted) {
9204 int newValue = (isMajor ? majorInProgress : minorInProgress).decrementAndGet();
9205
9206
9207 compactionsFinished.incrementAndGet();
9208 compactionNumFilesCompacted.addAndGet(numFiles);
9209 compactionNumBytesCompacted.addAndGet(filesSizeCompacted);
9210
9211 assert newValue >= 0;
9212 }
9213
9214 public void reportCompactionRequestFailure() {
9215 compactionsFailed.incrementAndGet();
9216 }
9217
9218 public void incrementCompactionsQueuedCount() {
9219 compactionsQueued.incrementAndGet();
9220 }
9221
9222 public void decrementCompactionsQueuedCount() {
9223 compactionsQueued.decrementAndGet();
9224 }
9225
9226 public void incrementFlushesQueuedCount() {
9227 flushesQueued.incrementAndGet();
9228 }
9229
9230 protected void decrementFlushesQueuedCount() {
9231 flushesQueued.decrementAndGet();
9232 }
9233
9234
9235
9236
9237
9238 public long getSequenceId() {
9239 return this.mvcc.getReadPoint();
9240 }
9241
9242
9243
9244
9245
9246
9247
9248
9249 private WALKey appendEmptyEdit(final WAL wal) throws IOException {
9250
9251 @SuppressWarnings("deprecation")
9252 WALKey key = new HLogKey(getRegionInfo().getEncodedNameAsBytes(),
9253 getRegionInfo().getTable(), WALKey.NO_SEQUENCE_ID, 0, null,
9254 HConstants.NO_NONCE, HConstants.NO_NONCE, getMVCC());
9255
9256
9257
9258 try {
9259 wal.append(getTableDesc(), getRegionInfo(), key, WALEdit.EMPTY_WALEDIT, false);
9260 } catch (Throwable t) {
9261
9262 getMVCC().complete(key.getWriteEntry());
9263 }
9264 return key;
9265 }
9266
9267
9268
9269
9270
9271
9272
9273 void checkInterrupt() throws NotServingRegionException, InterruptedIOException {
9274 if (Thread.interrupted()) {
9275 if (this.closing.get()) {
9276 throw new NotServingRegionException(
9277 getRegionInfo().getRegionNameAsString() + " is closing");
9278 }
9279 throw new InterruptedIOException();
9280 }
9281 }
9282
9283
9284
9285
9286
9287
9288 IOException throwOnInterrupt(Throwable t) {
9289 if (this.closing.get()) {
9290 return (NotServingRegionException) new NotServingRegionException(
9291 getRegionInfo().getRegionNameAsString() + " is closing")
9292 .initCause(t);
9293 }
9294 return (InterruptedIOException) new InterruptedIOException().initCause(t);
9295 }
9296
9297
9298
9299
9300 @Override
9301 public void onConfigurationChange(Configuration conf) {
9302
9303 }
9304
9305
9306
9307
9308 @Override
9309 public void registerChildren(ConfigurationManager manager) {
9310 configurationManager = Optional.of(manager);
9311 for (Store s : this.stores.values()) {
9312 configurationManager.get().registerObserver(s);
9313 }
9314 }
9315
9316
9317
9318
9319 @Override
9320 public void deregisterChildren(ConfigurationManager manager) {
9321 for (Store s : this.stores.values()) {
9322 configurationManager.get().deregisterObserver(s);
9323 }
9324 }
9325
9326
9327
9328
9329 public RegionSplitPolicy getSplitPolicy() {
9330 return this.splitPolicy;
9331 }
9332
9333 public void setRegionServerServices(RegionServerServices services) {
9334 this.rsServices = services;
9335 }
9336 }