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 doRollBackMemstore = false;
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 doRollBackMemstore = true;
3709 addedSize += applyFamilyMapToMemstore(familyMaps[i]);
3710 }
3711
3712
3713
3714
3715 if (locked) {
3716 this.updatesLock.readLock().unlock();
3717 locked = false;
3718 }
3719 releaseRowLocks(acquiredRowLocks);
3720
3721
3722
3723
3724 if (txid != 0) {
3725 syncOrDefer(txid, durability);
3726 }
3727
3728 doRollBackMemstore = false;
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 } finally {
3780
3781 if (doRollBackMemstore) {
3782 for (int j = 0; j < familyMaps.length; j++) {
3783 for(List<Cell> cells:familyMaps[j].values()) {
3784 rollbackMemstore(cells);
3785 }
3786 }
3787 if (writeEntry != null) mvcc.complete(writeEntry);
3788 } else {
3789 if (writeEntry != null) {
3790 mvcc.completeAndWait(writeEntry);
3791 }
3792 }
3793
3794 if (locked) {
3795 this.updatesLock.readLock().unlock();
3796 }
3797 releaseRowLocks(acquiredRowLocks);
3798
3799
3800
3801
3802
3803
3804
3805 if (noOfPuts > 0) {
3806
3807 if (this.metricsRegion != null) {
3808 this.metricsRegion.updatePut();
3809 }
3810 }
3811 if (noOfDeletes > 0) {
3812
3813 if (this.metricsRegion != null) {
3814 this.metricsRegion.updateDelete();
3815 }
3816 }
3817
3818 enableInterrupts();
3819
3820 if (!success) {
3821 for (int i = firstIndex; i < lastIndexExclusive; i++) {
3822 if (batchOp.retCodeDetails[i].getOperationStatusCode() == OperationStatusCode.NOT_RUN) {
3823 batchOp.retCodeDetails[i] = OperationStatus.FAILURE;
3824 }
3825 }
3826 }
3827 if (coprocessorHost != null && !batchOp.isInReplay()) {
3828
3829
3830 MiniBatchOperationInProgress<Mutation> miniBatchOp =
3831 new MiniBatchOperationInProgress<Mutation>(batchOp.getMutationsForCoprocs(),
3832 batchOp.retCodeDetails, batchOp.walEditsFromCoprocessors, firstIndex, lastIndexExclusive);
3833 coprocessorHost.postBatchMutateIndispensably(miniBatchOp, success);
3834 }
3835
3836 batchOp.nextIndexToProcess = lastIndexExclusive;
3837 }
3838 }
3839
3840 private void mergeFamilyMaps(Map<byte[], List<Cell>> familyMap,
3841 Map<byte[], List<Cell>> toBeMerged) {
3842 for (Map.Entry<byte[], List<Cell>> entry : toBeMerged.entrySet()) {
3843 List<Cell> cells = familyMap.get(entry.getKey());
3844 if (cells == null) {
3845 familyMap.put(entry.getKey(), entry.getValue());
3846 } else {
3847 cells.addAll(entry.getValue());
3848 }
3849 }
3850 }
3851
3852
3853
3854
3855
3856 protected Durability getEffectiveDurability(Durability d) {
3857 return d == Durability.USE_DEFAULT ? this.durability : d;
3858 }
3859
3860
3861
3862
3863
3864
3865 @Override
3866 public boolean checkAndMutate(byte [] row, byte [] family, byte [] qualifier,
3867 CompareOp compareOp, ByteArrayComparable comparator, Mutation w,
3868 boolean writeToWAL)
3869 throws IOException{
3870 checkReadOnly();
3871
3872
3873 checkResources();
3874 boolean isPut = w instanceof Put;
3875 if (!isPut && !(w instanceof Delete))
3876 throw new org.apache.hadoop.hbase.DoNotRetryIOException("Action must " +
3877 "be Put or Delete");
3878 if (!Bytes.equals(row, w.getRow())) {
3879 throw new org.apache.hadoop.hbase.DoNotRetryIOException("Action's " +
3880 "getRow must match the passed row");
3881 }
3882
3883 startRegionOperation();
3884 try {
3885 Get get = new Get(row);
3886 checkFamily(family);
3887 get.addColumn(family, qualifier);
3888 checkRow(row, "checkAndMutate");
3889
3890 RowLock rowLock = getRowLockInternal(get.getRow());
3891 try {
3892
3893 mvcc.await();
3894 if (this.getCoprocessorHost() != null) {
3895 Boolean processed = null;
3896 if (w instanceof Put) {
3897 processed = this.getCoprocessorHost().preCheckAndPutAfterRowLock(row, family,
3898 qualifier, compareOp, comparator, (Put) w);
3899 } else if (w instanceof Delete) {
3900 processed = this.getCoprocessorHost().preCheckAndDeleteAfterRowLock(row, family,
3901 qualifier, compareOp, comparator, (Delete) w);
3902 }
3903 if (processed != null) {
3904 return processed;
3905 }
3906 }
3907 List<Cell> result = get(get, false);
3908
3909 boolean valueIsNull = comparator.getValue() == null ||
3910 comparator.getValue().length == 0;
3911 boolean matches = false;
3912 long cellTs = 0;
3913 if (result.size() == 0 && valueIsNull) {
3914 matches = true;
3915 } else if (result.size() > 0 && result.get(0).getValueLength() == 0 &&
3916 valueIsNull) {
3917 matches = true;
3918 cellTs = result.get(0).getTimestamp();
3919 } else if (result.size() == 1 && !valueIsNull) {
3920 Cell kv = result.get(0);
3921 cellTs = kv.getTimestamp();
3922 int compareResult = comparator.compareTo(kv.getValueArray(),
3923 kv.getValueOffset(), kv.getValueLength());
3924 switch (compareOp) {
3925 case LESS:
3926 matches = compareResult < 0;
3927 break;
3928 case LESS_OR_EQUAL:
3929 matches = compareResult <= 0;
3930 break;
3931 case EQUAL:
3932 matches = compareResult == 0;
3933 break;
3934 case NOT_EQUAL:
3935 matches = compareResult != 0;
3936 break;
3937 case GREATER_OR_EQUAL:
3938 matches = compareResult >= 0;
3939 break;
3940 case GREATER:
3941 matches = compareResult > 0;
3942 break;
3943 default:
3944 throw new RuntimeException("Unknown Compare op " + compareOp.name());
3945 }
3946 }
3947
3948 if (matches) {
3949
3950
3951
3952
3953 long now = EnvironmentEdgeManager.currentTime();
3954 long ts = Math.max(now, cellTs);
3955 byte[] byteTs = Bytes.toBytes(ts);
3956
3957 if (w instanceof Put) {
3958 updateCellTimestamps(w.getFamilyCellMap().values(), byteTs);
3959 }
3960
3961
3962
3963
3964
3965 doBatchMutate(w);
3966 this.checkAndMutateChecksPassed.increment();
3967 return true;
3968 }
3969 this.checkAndMutateChecksFailed.increment();
3970 return false;
3971 } finally {
3972 rowLock.release();
3973 }
3974 } finally {
3975 closeRegionOperation();
3976 }
3977 }
3978
3979
3980
3981
3982
3983
3984 @Override
3985 public boolean checkAndRowMutate(byte [] row, byte [] family, byte [] qualifier,
3986 CompareOp compareOp, ByteArrayComparable comparator, RowMutations rm,
3987 boolean writeToWAL) throws IOException {
3988 checkReadOnly();
3989
3990
3991 checkResources();
3992
3993 startRegionOperation();
3994 try {
3995 Get get = new Get(row);
3996 checkFamily(family);
3997 get.addColumn(family, qualifier);
3998 checkRow(row, "checkAndRowMutate");
3999
4000 RowLock rowLock = getRowLockInternal(get.getRow());
4001 try {
4002
4003 mvcc.await();
4004
4005 List<Cell> result = get(get, false);
4006
4007 boolean valueIsNull = comparator.getValue() == null ||
4008 comparator.getValue().length == 0;
4009 boolean matches = false;
4010 long cellTs = 0;
4011 if (result.size() == 0 && valueIsNull) {
4012 matches = true;
4013 } else if (result.size() > 0 && result.get(0).getValueLength() == 0 &&
4014 valueIsNull) {
4015 matches = true;
4016 cellTs = result.get(0).getTimestamp();
4017 } else if (result.size() == 1 && !valueIsNull) {
4018 Cell kv = result.get(0);
4019 cellTs = kv.getTimestamp();
4020 int compareResult = comparator.compareTo(kv.getValueArray(),
4021 kv.getValueOffset(), kv.getValueLength());
4022 switch (compareOp) {
4023 case LESS:
4024 matches = compareResult < 0;
4025 break;
4026 case LESS_OR_EQUAL:
4027 matches = compareResult <= 0;
4028 break;
4029 case EQUAL:
4030 matches = compareResult == 0;
4031 break;
4032 case NOT_EQUAL:
4033 matches = compareResult != 0;
4034 break;
4035 case GREATER_OR_EQUAL:
4036 matches = compareResult >= 0;
4037 break;
4038 case GREATER:
4039 matches = compareResult > 0;
4040 break;
4041 default:
4042 throw new RuntimeException("Unknown Compare op " + compareOp.name());
4043 }
4044 }
4045
4046 if (matches) {
4047
4048
4049
4050
4051 long now = EnvironmentEdgeManager.currentTime();
4052 long ts = Math.max(now, cellTs);
4053 byte[] byteTs = Bytes.toBytes(ts);
4054
4055 for (Mutation w : rm.getMutations()) {
4056 if (w instanceof Put) {
4057 updateCellTimestamps(w.getFamilyCellMap().values(), byteTs);
4058 }
4059
4060
4061 }
4062
4063
4064
4065 mutateRow(rm);
4066 this.checkAndMutateChecksPassed.increment();
4067 return true;
4068 }
4069 this.checkAndMutateChecksFailed.increment();
4070 return false;
4071 } finally {
4072 rowLock.release();
4073 }
4074 } finally {
4075 closeRegionOperation();
4076 }
4077 }
4078
4079 private void doBatchMutate(Mutation mutation) throws IOException {
4080
4081 OperationStatus[] batchMutate = this.batchMutate(new Mutation[]{mutation});
4082 if (batchMutate[0].getOperationStatusCode().equals(OperationStatusCode.SANITY_CHECK_FAILURE)) {
4083 throw new FailedSanityCheckException(batchMutate[0].getExceptionMsg());
4084 } else if (batchMutate[0].getOperationStatusCode().equals(OperationStatusCode.BAD_FAMILY)) {
4085 throw new NoSuchColumnFamilyException(batchMutate[0].getExceptionMsg());
4086 }
4087 }
4088
4089
4090
4091
4092
4093
4094
4095
4096
4097
4098
4099
4100
4101
4102 public void addRegionToSnapshot(SnapshotDescription desc,
4103 ForeignExceptionSnare exnSnare) throws IOException {
4104 Path rootDir = FSUtils.getRootDir(conf);
4105 Path snapshotDir = SnapshotDescriptionUtils.getWorkingSnapshotDir(desc, rootDir, conf);
4106
4107 SnapshotManifest manifest = SnapshotManifest.create(conf, getFilesystem(),
4108 snapshotDir, desc, exnSnare);
4109 manifest.addRegion(this);
4110 }
4111
4112 private void updateSequenceId(final Iterable<List<Cell>> cellItr, final long sequenceId)
4113 throws IOException {
4114 for (List<Cell> cells : cellItr) {
4115 if (cells == null) {
4116 return;
4117 }
4118 for (Cell cell : cells) {
4119 CellUtil.setSequenceId(cell, sequenceId);
4120 }
4121 }
4122 }
4123
4124 @Override
4125 public void updateCellTimestamps(final Iterable<List<Cell>> cellItr, final byte[] now)
4126 throws IOException {
4127 for (List<Cell> cells: cellItr) {
4128 if (cells == null) continue;
4129 assert cells instanceof RandomAccess;
4130 int listSize = cells.size();
4131 for (int i = 0; i < listSize; i++) {
4132 CellUtil.updateLatestStamp(cells.get(i), now, 0);
4133 }
4134 }
4135 }
4136
4137
4138
4139
4140 void rewriteCellTags(Map<byte[], List<Cell>> familyMap, final Mutation m) {
4141
4142
4143
4144 if (m.getTTL() == Long.MAX_VALUE) {
4145 return;
4146 }
4147
4148
4149
4150 for (Map.Entry<byte[], List<Cell>> e: familyMap.entrySet()) {
4151 List<Cell> cells = e.getValue();
4152 assert cells instanceof RandomAccess;
4153 int listSize = cells.size();
4154 for (int i = 0; i < listSize; i++) {
4155 Cell cell = cells.get(i);
4156 List<Tag> newTags = Tag.carryForwardTags(null, cell);
4157 newTags = carryForwardTTLTag(newTags, m);
4158
4159
4160 cells.set(i, new KeyValue(cell.getRowArray(), cell.getRowOffset(), cell.getRowLength(),
4161 cell.getFamilyArray(), cell.getFamilyOffset(), cell.getFamilyLength(),
4162 cell.getQualifierArray(), cell.getQualifierOffset(), cell.getQualifierLength(),
4163 cell.getTimestamp(), KeyValue.Type.codeToType(cell.getTypeByte()),
4164 cell.getValueArray(), cell.getValueOffset(), cell.getValueLength(),
4165 newTags));
4166 }
4167 }
4168 }
4169
4170
4171
4172
4173
4174
4175
4176 private void checkResources() throws RegionTooBusyException {
4177
4178 if (this.getRegionInfo().isMetaRegion()) return;
4179
4180 if (this.memstoreSize.get() > this.blockingMemStoreSize) {
4181 blockedRequestsCount.increment();
4182 requestFlush();
4183 final String regionName =
4184 this.getRegionInfo() == null ? "unknown" : this.getRegionInfo().getRegionNameAsString();
4185 final String serverName = this.getRegionServerServices() == null ?
4186 "unknown" : (this.getRegionServerServices().getServerName() == null ? "unknown" :
4187 this.getRegionServerServices().getServerName().toString());
4188 RegionTooBusyException rtbe = new RegionTooBusyException(
4189 "Above memstore limit, " + "regionName=" + regionName + ", server=" + serverName
4190 + ", memstoreSize=" + memstoreSize.get()
4191 + ", blockingMemStoreSize=" + blockingMemStoreSize);
4192 LOG.warn("Region is too busy due to exceeding memstore size limit.", rtbe);
4193 throw rtbe;
4194 }
4195 }
4196
4197
4198
4199
4200 protected void checkReadOnly() throws IOException {
4201 if (isReadOnly()) {
4202 throw new DoNotRetryIOException("region is read only");
4203 }
4204 }
4205
4206 protected void checkReadsEnabled() throws IOException {
4207 if (!this.writestate.readsEnabled) {
4208 throw new IOException(getRegionInfo().getEncodedName()
4209 + ": The region's reads are disabled. Cannot serve the request");
4210 }
4211 }
4212
4213 public void setReadsEnabled(boolean readsEnabled) {
4214 if (readsEnabled && !this.writestate.readsEnabled) {
4215 LOG.info(getRegionInfo().getEncodedName() + " : Enabling reads for region.");
4216 }
4217 this.writestate.setReadsEnabled(readsEnabled);
4218 }
4219
4220
4221
4222
4223
4224
4225
4226 private void put(final byte [] row, byte [] family, List<Cell> edits)
4227 throws IOException {
4228 NavigableMap<byte[], List<Cell>> familyMap;
4229 familyMap = new TreeMap<byte[], List<Cell>>(Bytes.BYTES_COMPARATOR);
4230
4231 familyMap.put(family, edits);
4232 Put p = new Put(row);
4233 p.setFamilyCellMap(familyMap);
4234 doBatchMutate(p);
4235 }
4236
4237
4238
4239
4240
4241
4242
4243
4244
4245
4246
4247
4248 private long applyFamilyMapToMemstore(Map<byte[], List<Cell>> familyMap) throws IOException {
4249 long size = 0;
4250
4251 for (Map.Entry<byte[], List<Cell>> e : familyMap.entrySet()) {
4252 byte[] family = e.getKey();
4253 List<Cell> cells = e.getValue();
4254 assert cells instanceof RandomAccess;
4255 Store store = getStore(family);
4256 size += store.add(cells);
4257 }
4258
4259 return size;
4260 }
4261
4262
4263
4264
4265
4266
4267 private void rollbackMemstore(List<Cell> memstoreCells) {
4268 rollbackMemstore(null, memstoreCells);
4269 }
4270
4271 private void rollbackMemstore(final Store defaultStore, List<Cell> memstoreCells) {
4272 int kvsRolledback = 0;
4273 for (Cell cell : memstoreCells) {
4274 Store store = defaultStore;
4275 if (store == null) {
4276 byte[] family = CellUtil.cloneFamily(cell);
4277 store = getStore(family);
4278 }
4279 store.rollback(cell);
4280 kvsRolledback++;
4281 }
4282 LOG.debug("rollbackMemstore rolled back " + kvsRolledback);
4283 }
4284
4285 @Override
4286 public void checkFamilies(Collection<byte[]> families) throws NoSuchColumnFamilyException {
4287 for (byte[] family : families) {
4288 checkFamily(family);
4289 }
4290 }
4291
4292 private void checkAndPrepareMutation(Mutation mutation, boolean replay,
4293 final Map<byte[], List<Cell>> familyMap, final long now)
4294 throws IOException {
4295 if (mutation instanceof Put) {
4296
4297 if (replay) {
4298 removeNonExistentColumnFamilyForReplay(familyMap);
4299 } else {
4300 checkFamilies(familyMap.keySet());
4301 }
4302 checkTimestamps(mutation.getFamilyCellMap(), now);
4303 } else {
4304 prepareDelete((Delete)mutation);
4305 }
4306 checkRow(mutation.getRow(), "doMiniBatchMutation");
4307 }
4308
4309
4310
4311
4312
4313 private void removeNonExistentColumnFamilyForReplay(
4314 final Map<byte[], List<Cell>> familyMap) {
4315 List<byte[]> nonExistentList = null;
4316 for (byte[] family : familyMap.keySet()) {
4317 if (!this.htableDescriptor.hasFamily(family)) {
4318 if (nonExistentList == null) {
4319 nonExistentList = new ArrayList<byte[]>();
4320 }
4321 nonExistentList.add(family);
4322 }
4323 }
4324 if (nonExistentList != null) {
4325 for (byte[] family : nonExistentList) {
4326
4327 LOG.info("No family for " + Bytes.toString(family) + " omit from replay.");
4328 familyMap.remove(family);
4329 }
4330 }
4331 }
4332
4333 @Override
4334 public void checkTimestamps(final Map<byte[], List<Cell>> familyMap, long now)
4335 throws FailedSanityCheckException {
4336 if (timestampSlop == HConstants.LATEST_TIMESTAMP) {
4337 return;
4338 }
4339 long maxTs = now + timestampSlop;
4340 for (List<Cell> kvs : familyMap.values()) {
4341 assert kvs instanceof RandomAccess;
4342 int listSize = kvs.size();
4343 for (int i=0; i < listSize; i++) {
4344 Cell cell = kvs.get(i);
4345
4346 long ts = cell.getTimestamp();
4347 if (ts != HConstants.LATEST_TIMESTAMP && ts > maxTs) {
4348 throw new FailedSanityCheckException("Timestamp for KV out of range "
4349 + cell + " (too.new=" + timestampSlop + ")");
4350 }
4351 }
4352 }
4353 }
4354
4355
4356
4357
4358
4359
4360
4361 private void addFamilyMapToWALEdit(Map<byte[], List<Cell>> familyMap,
4362 WALEdit walEdit) {
4363 for (List<Cell> edits : familyMap.values()) {
4364 assert edits instanceof RandomAccess;
4365 int listSize = edits.size();
4366 for (int i=0; i < listSize; i++) {
4367 Cell cell = edits.get(i);
4368 walEdit.add(cell);
4369 }
4370 }
4371 }
4372
4373 private void requestFlush() {
4374 if (this.rsServices == null) {
4375 return;
4376 }
4377 synchronized (writestate) {
4378 if (this.writestate.isFlushRequested()) {
4379 return;
4380 }
4381 writestate.flushRequested = true;
4382 }
4383
4384 this.rsServices.getFlushRequester().requestFlush(this, false);
4385 if (LOG.isDebugEnabled()) {
4386 LOG.debug("Flush requested on " + this.getRegionInfo().getEncodedName());
4387 }
4388 }
4389
4390
4391
4392
4393
4394 private boolean isFlushSize(final long size) {
4395 return size > this.memstoreFlushSize;
4396 }
4397
4398
4399
4400
4401
4402
4403
4404
4405
4406
4407
4408
4409
4410
4411
4412
4413
4414
4415
4416
4417
4418
4419
4420
4421
4422
4423
4424
4425
4426
4427
4428
4429
4430
4431 protected long replayRecoveredEditsIfAny(Map<byte[], Long> maxSeqIdInStores,
4432 final CancelableProgressable reporter, final MonitoredTask status)
4433 throws IOException {
4434 long minSeqIdForTheRegion = -1;
4435 for (Long maxSeqIdInStore : maxSeqIdInStores.values()) {
4436 if (maxSeqIdInStore < minSeqIdForTheRegion || minSeqIdForTheRegion == -1) {
4437 minSeqIdForTheRegion = maxSeqIdInStore;
4438 }
4439 }
4440 long seqId = minSeqIdForTheRegion;
4441
4442 FileSystem walFS = getWalFileSystem();
4443 FileSystem rootFS = getFilesystem();
4444 Path regionDir = FSUtils.getRegionDirFromRootDir(FSUtils.getRootDir(conf), getRegionInfo());
4445 Path regionWALDir = getWALRegionDir();
4446 Path wrongRegionWALDir = FSUtils.getWrongWALRegionDir(conf, getRegionInfo().getTable(),
4447 getRegionInfo().getEncodedName());
4448
4449
4450 NavigableSet<Path> filesUnderWrongRegionWALDir =
4451 WALSplitter.getSplitEditFilesSorted(walFS, wrongRegionWALDir);
4452 seqId = Math.max(seqId, replayRecoveredEditsForPaths(minSeqIdForTheRegion, walFS,
4453 filesUnderWrongRegionWALDir, reporter, regionDir));
4454
4455
4456 NavigableSet<Path> filesUnderRootDir = Sets.newTreeSet();
4457 if (!regionWALDir.equals(regionDir)) {
4458 filesUnderRootDir = WALSplitter.getSplitEditFilesSorted(rootFS, regionDir);
4459 seqId = Math.max(seqId, replayRecoveredEditsForPaths(minSeqIdForTheRegion, rootFS,
4460 filesUnderRootDir, reporter, regionDir));
4461 }
4462 NavigableSet<Path> files = WALSplitter.getSplitEditFilesSorted(walFS, regionWALDir);
4463 seqId = Math.max(seqId, replayRecoveredEditsForPaths(minSeqIdForTheRegion, walFS,
4464 files, reporter, regionWALDir));
4465 if (seqId > minSeqIdForTheRegion) {
4466
4467 internalFlushcache(null, seqId, stores.values(), status, false);
4468 }
4469
4470 if (conf.getBoolean("hbase.region.archive.recovered.edits", false)) {
4471
4472
4473
4474 String fakeFamilyName = WALSplitter.getRegionDirRecoveredEditsDir(regionWALDir).getName();
4475 Set<StoreFile> fakeStoreFiles = new HashSet<>();
4476 for (Path file: Iterables.concat(files, filesUnderWrongRegionWALDir)) {
4477 fakeStoreFiles.add(new StoreFile(walFS, file, conf, null, null));
4478 }
4479 for (Path file: filesUnderRootDir) {
4480 fakeStoreFiles.add(new StoreFile(rootFS, file, conf, null, null));
4481 }
4482 getRegionWALFileSystem().removeStoreFiles(fakeFamilyName, fakeStoreFiles);
4483 } else {
4484 for (Path file : filesUnderRootDir) {
4485 if (!rootFS.delete(file, false)) {
4486 LOG.error("Failed delete of " + file + " from under the root directory");
4487 } else {
4488 LOG.debug("Deleted recovered.edits under root directory, file=" + file);
4489 }
4490 }
4491 for (Path file : Iterables.concat(files, filesUnderWrongRegionWALDir)) {
4492 if (!walFS.delete(file, false)) {
4493 LOG.error("Failed delete of " + file);
4494 } else {
4495 LOG.debug("Deleted recovered.edits file=" + file);
4496 }
4497 }
4498 }
4499
4500
4501
4502 FileSystem walFs = getWalFileSystem();
4503 if (walFs.exists(wrongRegionWALDir)) {
4504 if (!walFs.delete(wrongRegionWALDir, true)) {
4505 LOG.warn("Unable to delete " + wrongRegionWALDir);
4506 }
4507 }
4508
4509 return seqId;
4510 }
4511
4512 private long replayRecoveredEditsForPaths(long minSeqIdForTheRegion, FileSystem fs,
4513 final NavigableSet<Path> files, final CancelableProgressable reporter, final Path regionDir)
4514 throws IOException {
4515 long seqid = minSeqIdForTheRegion;
4516 if (LOG.isDebugEnabled()) {
4517 LOG.debug("Found " + (files == null ? 0 : files.size()) +
4518 " recovered edits file(s) under " + regionDir);
4519 }
4520
4521 if (files == null || files.isEmpty()) {
4522 return seqid;
4523 }
4524
4525 for (Path edits : files) {
4526 if (edits == null || !fs.exists(edits)) {
4527 LOG.warn("Null or non-existent edits file: " + edits);
4528 continue;
4529 }
4530 if (isZeroLengthThenDelete(fs, edits)) {
4531 continue;
4532 }
4533
4534 long maxSeqId;
4535 String fileName = edits.getName();
4536 maxSeqId = Math.abs(Long.parseLong(fileName));
4537 if (maxSeqId <= minSeqIdForTheRegion) {
4538 if (LOG.isDebugEnabled()) {
4539 String msg = "Maximum sequenceid for this wal is " + maxSeqId +
4540 " and minimum sequenceid for the region is " + minSeqIdForTheRegion
4541 + ", skipped the whole file, path=" + edits;
4542 LOG.debug(msg);
4543 }
4544 continue;
4545 }
4546
4547 try {
4548
4549
4550 seqid = Math.max(seqid, replayRecoveredEdits(edits, maxSeqIdInStores, reporter, fs));
4551 } catch (IOException e) {
4552 boolean skipErrors = conf.getBoolean(HConstants.HREGION_EDITS_REPLAY_SKIP_ERRORS,
4553 conf.getBoolean("hbase.skip.errors",
4554 HConstants.DEFAULT_HREGION_EDITS_REPLAY_SKIP_ERRORS));
4555 if (conf.get("hbase.skip.errors") != null) {
4556 LOG.warn("The property 'hbase.skip.errors' has been deprecated. Please use " +
4557 HConstants.HREGION_EDITS_REPLAY_SKIP_ERRORS + " instead.");
4558 }
4559 if (skipErrors) {
4560 Path p = WALSplitter.moveAsideBadEditsFile(fs, edits);
4561 LOG.error(HConstants.HREGION_EDITS_REPLAY_SKIP_ERRORS +
4562 "=true so continuing. Renamed " + edits + " as " + p, e);
4563 } else {
4564 throw e;
4565 }
4566 }
4567 }
4568 return seqid;
4569 }
4570
4571
4572
4573
4574
4575
4576
4577
4578
4579
4580 private long replayRecoveredEdits(final Path edits, Map<byte[], Long> maxSeqIdInStores,
4581 final CancelableProgressable reporter, final FileSystem fs)
4582 throws IOException {
4583 String msg = "Replaying edits from " + edits;
4584 LOG.info(msg);
4585 MonitoredTask status = TaskMonitor.get().createStatus(msg);
4586
4587 status.setStatus("Opening recovered edits");
4588 WAL.Reader reader = null;
4589 try {
4590 reader = WALFactory.createReader(fs, edits, conf);
4591 long currentEditSeqId = -1;
4592 long currentReplaySeqId = -1;
4593 long firstSeqIdInLog = -1;
4594 long skippedEdits = 0;
4595 long editsCount = 0;
4596 long intervalEdits = 0;
4597 WAL.Entry entry;
4598 Store store = null;
4599 boolean reported_once = false;
4600 ServerNonceManager ng = this.rsServices == null ? null : this.rsServices.getNonceManager();
4601
4602 try {
4603
4604 int interval = this.conf.getInt("hbase.hstore.report.interval.edits", 2000);
4605
4606 int period = this.conf.getInt("hbase.hstore.report.period", 300000);
4607 long lastReport = EnvironmentEdgeManager.currentTime();
4608
4609 while ((entry = reader.next()) != null) {
4610 WALKey key = entry.getKey();
4611 WALEdit val = entry.getEdit();
4612
4613 if (ng != null) {
4614 ng.reportOperationFromWal(key.getNonceGroup(), key.getNonce(), key.getWriteTime());
4615 }
4616
4617 if (reporter != null) {
4618 intervalEdits += val.size();
4619 if (intervalEdits >= interval) {
4620
4621 intervalEdits = 0;
4622 long cur = EnvironmentEdgeManager.currentTime();
4623 if (lastReport + period <= cur) {
4624 status.setStatus("Replaying edits..." +
4625 " skipped=" + skippedEdits +
4626 " edits=" + editsCount);
4627
4628 if(!reporter.progress()) {
4629 msg = "Progressable reporter failed, stopping replay";
4630 LOG.warn(msg);
4631 status.abort(msg);
4632 throw new IOException(msg);
4633 }
4634 reported_once = true;
4635 lastReport = cur;
4636 }
4637 }
4638 }
4639
4640 if (firstSeqIdInLog == -1) {
4641 firstSeqIdInLog = key.getLogSeqNum();
4642 }
4643 if (currentEditSeqId > key.getLogSeqNum()) {
4644
4645
4646 LOG.error(getRegionInfo().getEncodedName() + " : "
4647 + "Found decreasing SeqId. PreId=" + currentEditSeqId + " key=" + key
4648 + "; edit=" + val);
4649 } else {
4650 currentEditSeqId = key.getLogSeqNum();
4651 }
4652 currentReplaySeqId = (key.getOrigLogSeqNum() > 0) ?
4653 key.getOrigLogSeqNum() : currentEditSeqId;
4654
4655
4656
4657 if (coprocessorHost != null) {
4658 status.setStatus("Running pre-WAL-restore hook in coprocessors");
4659 if (coprocessorHost.preWALRestore(this.getRegionInfo(), key, val)) {
4660
4661 continue;
4662 }
4663 }
4664 boolean checkRowWithinBoundary = false;
4665
4666 if (!Bytes.equals(key.getEncodedRegionName(),
4667 this.getRegionInfo().getEncodedNameAsBytes())) {
4668 checkRowWithinBoundary = true;
4669 }
4670
4671 boolean flush = false;
4672 for (Cell cell: val.getCells()) {
4673
4674
4675 if (CellUtil.matchingFamily(cell, WALEdit.METAFAMILY)) {
4676
4677 if (!checkRowWithinBoundary) {
4678
4679 CompactionDescriptor compaction = WALEdit.getCompaction(cell);
4680 if (compaction != null) {
4681
4682 replayWALCompactionMarker(compaction, false, true, Long.MAX_VALUE);
4683 }
4684 }
4685 skippedEdits++;
4686 continue;
4687 }
4688
4689 if (store == null || !CellUtil.matchingFamily(cell, store.getFamily().getName())) {
4690 store = getStore(cell);
4691 }
4692 if (store == null) {
4693
4694
4695 LOG.warn("No family for " + cell);
4696 skippedEdits++;
4697 continue;
4698 }
4699 if (checkRowWithinBoundary && !rowIsInRange(this.getRegionInfo(),
4700 cell.getRowArray(), cell.getRowOffset(), cell.getRowLength())) {
4701 LOG.warn("Row of " + cell + " is not within region boundary");
4702 skippedEdits++;
4703 continue;
4704 }
4705
4706 if (key.getLogSeqNum() <= maxSeqIdInStores.get(store.getFamily()
4707 .getName())) {
4708 skippedEdits++;
4709 continue;
4710 }
4711 CellUtil.setSequenceId(cell, currentReplaySeqId);
4712
4713
4714
4715
4716 flush |= restoreEdit(store, cell);
4717 editsCount++;
4718 }
4719 if (flush) {
4720 internalFlushcache(null, currentEditSeqId, stores.values(), status, false);
4721 }
4722
4723 if (coprocessorHost != null) {
4724 coprocessorHost.postWALRestore(this.getRegionInfo(), key, val);
4725 }
4726 }
4727 } catch (EOFException eof) {
4728 Path p = WALSplitter.moveAsideBadEditsFile(fs, edits);
4729 msg = "Encountered EOF. Most likely due to Master failure during " +
4730 "wal splitting, so we have this data in another edit. " +
4731 "Continuing, but renaming " + edits + " as " + p;
4732 LOG.warn(msg, eof);
4733 status.abort(msg);
4734 } catch (IOException ioe) {
4735
4736
4737 if (ioe.getCause() instanceof ParseException) {
4738 Path p = WALSplitter.moveAsideBadEditsFile(fs, edits);
4739 msg = "File corruption encountered! " +
4740 "Continuing, but renaming " + edits + " as " + p;
4741 LOG.warn(msg, ioe);
4742 status.setStatus(msg);
4743 } else {
4744 status.abort(StringUtils.stringifyException(ioe));
4745
4746
4747 throw ioe;
4748 }
4749 }
4750 if (reporter != null && !reported_once) {
4751 reporter.progress();
4752 }
4753 msg = "Applied " + editsCount + ", skipped " + skippedEdits +
4754 ", firstSequenceIdInLog=" + firstSeqIdInLog +
4755 ", maxSequenceIdInLog=" + currentEditSeqId + ", path=" + edits;
4756 status.markComplete(msg);
4757 LOG.debug(msg);
4758 return currentEditSeqId;
4759 } finally {
4760 status.cleanup();
4761 if (reader != null) {
4762 reader.close();
4763 }
4764 }
4765 }
4766
4767
4768
4769
4770
4771
4772 void replayWALCompactionMarker(CompactionDescriptor compaction, boolean pickCompactionFiles,
4773 boolean removeFiles, long replaySeqId)
4774 throws IOException {
4775 try {
4776 checkTargetRegion(compaction.getEncodedRegionName().toByteArray(),
4777 "Compaction marker from WAL ", compaction);
4778 } catch (WrongRegionException wre) {
4779 if (RegionReplicaUtil.isDefaultReplica(this.getRegionInfo())) {
4780
4781 return;
4782 }
4783 throw wre;
4784 }
4785
4786 synchronized (writestate) {
4787 if (replaySeqId < lastReplayedOpenRegionSeqId) {
4788 LOG.warn(getRegionInfo().getEncodedName() + " : "
4789 + "Skipping replaying compaction event :" + TextFormat.shortDebugString(compaction)
4790 + " because its sequence id " + replaySeqId + " is smaller than this regions "
4791 + "lastReplayedOpenRegionSeqId of " + lastReplayedOpenRegionSeqId);
4792 return;
4793 }
4794 if (replaySeqId < lastReplayedCompactionSeqId) {
4795 LOG.warn(getRegionInfo().getEncodedName() + " : "
4796 + "Skipping replaying compaction event :" + TextFormat.shortDebugString(compaction)
4797 + " because its sequence id " + replaySeqId + " is smaller than this regions "
4798 + "lastReplayedCompactionSeqId of " + lastReplayedCompactionSeqId);
4799 return;
4800 } else {
4801 lastReplayedCompactionSeqId = replaySeqId;
4802 }
4803
4804 if (LOG.isDebugEnabled()) {
4805 LOG.debug(getRegionInfo().getEncodedName() + " : "
4806 + "Replaying compaction marker " + TextFormat.shortDebugString(compaction)
4807 + " with seqId=" + replaySeqId + " and lastReplayedOpenRegionSeqId="
4808 + lastReplayedOpenRegionSeqId);
4809 }
4810
4811 startRegionOperation(Operation.REPLAY_EVENT);
4812 try {
4813 Store store = this.getStore(compaction.getFamilyName().toByteArray());
4814 if (store == null) {
4815 LOG.warn(getRegionInfo().getEncodedName() + " : "
4816 + "Found Compaction WAL edit for deleted family:"
4817 + Bytes.toString(compaction.getFamilyName().toByteArray()));
4818 return;
4819 }
4820 store.replayCompactionMarker(compaction, pickCompactionFiles, removeFiles);
4821 logRegionFiles();
4822 } catch (FileNotFoundException ex) {
4823 LOG.warn(getRegionInfo().getEncodedName() + " : "
4824 + "At least one of the store files in compaction: "
4825 + TextFormat.shortDebugString(compaction)
4826 + " doesn't exist any more. Skip loading the file(s)", ex);
4827 } finally {
4828 closeRegionOperation(Operation.REPLAY_EVENT);
4829 }
4830 }
4831 }
4832
4833 void replayWALFlushMarker(FlushDescriptor flush, long replaySeqId) throws IOException {
4834 checkTargetRegion(flush.getEncodedRegionName().toByteArray(),
4835 "Flush marker from WAL ", flush);
4836
4837 if (ServerRegionReplicaUtil.isDefaultReplica(this.getRegionInfo())) {
4838 return;
4839 }
4840
4841 if (LOG.isDebugEnabled()) {
4842 LOG.debug(getRegionInfo().getEncodedName() + " : "
4843 + "Replaying flush marker " + TextFormat.shortDebugString(flush));
4844 }
4845
4846 startRegionOperation(Operation.REPLAY_EVENT);
4847 try {
4848 FlushAction action = flush.getAction();
4849 switch (action) {
4850 case START_FLUSH:
4851 replayWALFlushStartMarker(flush);
4852 break;
4853 case COMMIT_FLUSH:
4854 replayWALFlushCommitMarker(flush);
4855 break;
4856 case ABORT_FLUSH:
4857 replayWALFlushAbortMarker(flush);
4858 break;
4859 case CANNOT_FLUSH:
4860 replayWALFlushCannotFlushMarker(flush, replaySeqId);
4861 break;
4862 default:
4863 LOG.warn(getRegionInfo().getEncodedName() + " : " +
4864 "Received a flush event with unknown action, ignoring. " +
4865 TextFormat.shortDebugString(flush));
4866 break;
4867 }
4868
4869 logRegionFiles();
4870 } finally {
4871 closeRegionOperation(Operation.REPLAY_EVENT);
4872 }
4873 }
4874
4875
4876
4877
4878
4879 PrepareFlushResult replayWALFlushStartMarker(FlushDescriptor flush) throws IOException {
4880 long flushSeqId = flush.getFlushSequenceNumber();
4881
4882 HashSet<Store> storesToFlush = new HashSet<Store>();
4883 for (StoreFlushDescriptor storeFlush : flush.getStoreFlushesList()) {
4884 byte[] family = storeFlush.getFamilyName().toByteArray();
4885 Store store = getStore(family);
4886 if (store == null) {
4887 LOG.warn(getRegionInfo().getEncodedName() + " : "
4888 + "Received a flush start marker from primary, but the family is not found. Ignoring"
4889 + " StoreFlushDescriptor:" + TextFormat.shortDebugString(storeFlush));
4890 continue;
4891 }
4892 storesToFlush.add(store);
4893 }
4894
4895 MonitoredTask status = TaskMonitor.get().createStatus("Preparing flush " + this);
4896
4897
4898
4899 synchronized (writestate) {
4900 try {
4901 if (flush.getFlushSequenceNumber() < lastReplayedOpenRegionSeqId) {
4902 LOG.warn(getRegionInfo().getEncodedName() + " : "
4903 + "Skipping replaying flush event :" + TextFormat.shortDebugString(flush)
4904 + " because its sequence id is smaller than this regions lastReplayedOpenRegionSeqId "
4905 + " of " + lastReplayedOpenRegionSeqId);
4906 return null;
4907 }
4908 if (numMutationsWithoutWAL.get() > 0) {
4909 numMutationsWithoutWAL.set(0);
4910 dataInMemoryWithoutWAL.set(0);
4911 }
4912
4913 if (!writestate.flushing) {
4914
4915
4916
4917
4918 PrepareFlushResult prepareResult = internalPrepareFlushCache(null,
4919 flushSeqId, storesToFlush, status, false);
4920 if (prepareResult.result == null) {
4921
4922 this.writestate.flushing = true;
4923 this.prepareFlushResult = prepareResult;
4924 status.markComplete("Flush prepare successful");
4925 if (LOG.isDebugEnabled()) {
4926 LOG.debug(getRegionInfo().getEncodedName() + " : "
4927 + " Prepared flush with seqId:" + flush.getFlushSequenceNumber());
4928 }
4929 } else {
4930
4931
4932 if (prepareResult.getResult().getResult() ==
4933 FlushResult.Result.CANNOT_FLUSH_MEMSTORE_EMPTY) {
4934 this.writestate.flushing = true;
4935 this.prepareFlushResult = prepareResult;
4936 if (LOG.isDebugEnabled()) {
4937 LOG.debug(getRegionInfo().getEncodedName() + " : "
4938 + " Prepared empty flush with seqId:" + flush.getFlushSequenceNumber());
4939 }
4940 }
4941 status.abort("Flush prepare failed with " + prepareResult.result);
4942
4943 }
4944 return prepareResult;
4945 } else {
4946
4947 if (flush.getFlushSequenceNumber() == this.prepareFlushResult.flushOpSeqId) {
4948
4949 LOG.warn(getRegionInfo().getEncodedName() + " : "
4950 + "Received a flush prepare marker with the same seqId: " +
4951 + flush.getFlushSequenceNumber() + " before clearing the previous one with seqId: "
4952 + prepareFlushResult.flushOpSeqId + ". Ignoring");
4953
4954 } else if (flush.getFlushSequenceNumber() < this.prepareFlushResult.flushOpSeqId) {
4955
4956
4957 LOG.warn(getRegionInfo().getEncodedName() + " : "
4958 + "Received a flush prepare marker with a smaller seqId: " +
4959 + flush.getFlushSequenceNumber() + " before clearing the previous one with seqId: "
4960 + prepareFlushResult.flushOpSeqId + ". Ignoring");
4961
4962 } else {
4963
4964 LOG.warn(getRegionInfo().getEncodedName() + " : "
4965 + "Received a flush prepare marker with a larger seqId: " +
4966 + flush.getFlushSequenceNumber() + " before clearing the previous one with seqId: "
4967 + prepareFlushResult.flushOpSeqId + ". Ignoring");
4968
4969
4970
4971
4972
4973
4974
4975
4976
4977
4978
4979 }
4980 }
4981 } finally {
4982 status.cleanup();
4983 writestate.notifyAll();
4984 }
4985 }
4986 return null;
4987 }
4988
4989 @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="NN_NAKED_NOTIFY",
4990 justification="Intentional; post memstore flush")
4991 void replayWALFlushCommitMarker(FlushDescriptor flush) throws IOException {
4992 MonitoredTask status = TaskMonitor.get().createStatus("Committing flush " + this);
4993
4994
4995
4996
4997
4998 synchronized (writestate) {
4999 try {
5000 if (flush.getFlushSequenceNumber() < lastReplayedOpenRegionSeqId) {
5001 LOG.warn(getRegionInfo().getEncodedName() + " : "
5002 + "Skipping replaying flush event :" + TextFormat.shortDebugString(flush)
5003 + " because its sequence id is smaller than this regions lastReplayedOpenRegionSeqId "
5004 + " of " + lastReplayedOpenRegionSeqId);
5005 return;
5006 }
5007
5008 if (writestate.flushing) {
5009 PrepareFlushResult prepareFlushResult = this.prepareFlushResult;
5010 if (flush.getFlushSequenceNumber() == prepareFlushResult.flushOpSeqId) {
5011 if (LOG.isDebugEnabled()) {
5012 LOG.debug(getRegionInfo().getEncodedName() + " : "
5013 + "Received a flush commit marker with seqId:" + flush.getFlushSequenceNumber()
5014 + " and a previous prepared snapshot was found");
5015 }
5016
5017
5018 replayFlushInStores(flush, prepareFlushResult, true);
5019
5020
5021 this.addAndGetGlobalMemstoreSize(-prepareFlushResult.totalFlushableSize);
5022
5023 this.prepareFlushResult = null;
5024 writestate.flushing = false;
5025 } else if (flush.getFlushSequenceNumber() < prepareFlushResult.flushOpSeqId) {
5026
5027
5028
5029
5030 LOG.warn(getRegionInfo().getEncodedName() + " : "
5031 + "Received a flush commit marker with smaller seqId: "
5032 + flush.getFlushSequenceNumber() + " than what we have prepared with seqId: "
5033 + prepareFlushResult.flushOpSeqId + ". Picking up new file, but not dropping"
5034 +" prepared memstore snapshot");
5035 replayFlushInStores(flush, prepareFlushResult, false);
5036
5037
5038
5039 } else {
5040
5041
5042
5043
5044
5045 LOG.warn(getRegionInfo().getEncodedName() + " : "
5046 + "Received a flush commit marker with larger seqId: "
5047 + flush.getFlushSequenceNumber() + " than what we have prepared with seqId: " +
5048 prepareFlushResult.flushOpSeqId + ". Picking up new file and dropping prepared"
5049 +" memstore snapshot");
5050
5051 replayFlushInStores(flush, prepareFlushResult, true);
5052
5053
5054 this.addAndGetGlobalMemstoreSize(-prepareFlushResult.totalFlushableSize);
5055
5056
5057
5058 dropMemstoreContentsForSeqId(flush.getFlushSequenceNumber(), null);
5059
5060 this.prepareFlushResult = null;
5061 writestate.flushing = false;
5062 }
5063
5064
5065
5066
5067
5068 this.setReadsEnabled(true);
5069 } else {
5070 LOG.warn(getRegionInfo().getEncodedName() + " : "
5071 + "Received a flush commit marker with seqId:" + flush.getFlushSequenceNumber()
5072 + ", but no previous prepared snapshot was found");
5073
5074
5075 replayFlushInStores(flush, null, false);
5076
5077
5078
5079 dropMemstoreContentsForSeqId(flush.getFlushSequenceNumber(), null);
5080 }
5081
5082 status.markComplete("Flush commit successful");
5083
5084
5085 this.maxFlushedSeqId = flush.getFlushSequenceNumber();
5086
5087
5088 mvcc.advanceTo(flush.getFlushSequenceNumber());
5089
5090 } catch (FileNotFoundException ex) {
5091 LOG.warn(getRegionInfo().getEncodedName() + " : "
5092 + "At least one of the store files in flush: " + TextFormat.shortDebugString(flush)
5093 + " doesn't exist any more. Skip loading the file(s)", ex);
5094 }
5095 finally {
5096 status.cleanup();
5097 writestate.notifyAll();
5098 }
5099 }
5100
5101
5102
5103 synchronized (this) {
5104 notifyAll();
5105 }
5106 }
5107
5108
5109
5110
5111
5112
5113
5114
5115
5116 private void replayFlushInStores(FlushDescriptor flush, PrepareFlushResult prepareFlushResult,
5117 boolean dropMemstoreSnapshot)
5118 throws IOException {
5119 for (StoreFlushDescriptor storeFlush : flush.getStoreFlushesList()) {
5120 byte[] family = storeFlush.getFamilyName().toByteArray();
5121 Store store = getStore(family);
5122 if (store == null) {
5123 LOG.warn(getRegionInfo().getEncodedName() + " : "
5124 + "Received a flush commit marker from primary, but the family is not found."
5125 + "Ignoring StoreFlushDescriptor:" + storeFlush);
5126 continue;
5127 }
5128 List<String> flushFiles = storeFlush.getFlushOutputList();
5129 StoreFlushContext ctx = null;
5130 long startTime = EnvironmentEdgeManager.currentTime();
5131 if (prepareFlushResult == null || prepareFlushResult.storeFlushCtxs == null) {
5132 ctx = store.createFlushContext(flush.getFlushSequenceNumber());
5133 } else {
5134 ctx = prepareFlushResult.storeFlushCtxs.get(family);
5135 startTime = prepareFlushResult.startTime;
5136 }
5137
5138 if (ctx == null) {
5139 LOG.warn(getRegionInfo().getEncodedName() + " : "
5140 + "Unexpected: flush commit marker received from store "
5141 + Bytes.toString(family) + " but no associated flush context. Ignoring");
5142 continue;
5143 }
5144
5145 ctx.replayFlush(flushFiles, dropMemstoreSnapshot);
5146
5147
5148 this.lastStoreFlushTimeMap.put(store, startTime);
5149 }
5150 }
5151
5152
5153
5154
5155
5156
5157 private long dropMemstoreContentsForSeqId(long seqId, Store store) throws IOException {
5158 long totalFreedSize = 0;
5159 this.updatesLock.writeLock().lock();
5160 try {
5161
5162 long currentSeqId = mvcc.getReadPoint();
5163 if (seqId >= currentSeqId) {
5164
5165 LOG.info(getRegionInfo().getEncodedName() + " : "
5166 + "Dropping memstore contents as well since replayed flush seqId: "
5167 + seqId + " is greater than current seqId:" + currentSeqId);
5168
5169
5170 if (store == null) {
5171 for (Store s : stores.values()) {
5172 totalFreedSize += doDropStoreMemstoreContentsForSeqId(s, currentSeqId);
5173 }
5174 } else {
5175 totalFreedSize += doDropStoreMemstoreContentsForSeqId(store, currentSeqId);
5176 }
5177 } else {
5178 LOG.info(getRegionInfo().getEncodedName() + " : "
5179 + "Not dropping memstore contents since replayed flush seqId: "
5180 + seqId + " is smaller than current seqId:" + currentSeqId);
5181 }
5182 } finally {
5183 this.updatesLock.writeLock().unlock();
5184 }
5185 return totalFreedSize;
5186 }
5187
5188 private long doDropStoreMemstoreContentsForSeqId(Store s, long currentSeqId) throws IOException {
5189 long snapshotSize = s.getFlushableSize();
5190 this.addAndGetGlobalMemstoreSize(-snapshotSize);
5191 StoreFlushContext ctx = s.createFlushContext(currentSeqId);
5192 ctx.prepare();
5193 ctx.abort();
5194 return snapshotSize;
5195 }
5196
5197 private void replayWALFlushAbortMarker(FlushDescriptor flush) {
5198
5199
5200
5201 }
5202
5203 private void replayWALFlushCannotFlushMarker(FlushDescriptor flush, long replaySeqId) {
5204 synchronized (writestate) {
5205 if (this.lastReplayedOpenRegionSeqId > replaySeqId) {
5206 LOG.warn(getRegionInfo().getEncodedName() + " : "
5207 + "Skipping replaying flush event :" + TextFormat.shortDebugString(flush)
5208 + " because its sequence id " + replaySeqId + " is smaller than this regions "
5209 + "lastReplayedOpenRegionSeqId of " + lastReplayedOpenRegionSeqId);
5210 return;
5211 }
5212
5213
5214
5215
5216
5217
5218 this.setReadsEnabled(true);
5219 }
5220 }
5221
5222 PrepareFlushResult getPrepareFlushResult() {
5223 return prepareFlushResult;
5224 }
5225
5226 @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="NN_NAKED_NOTIFY",
5227 justification="Intentional; cleared the memstore")
5228 void replayWALRegionEventMarker(RegionEventDescriptor regionEvent) throws IOException {
5229 checkTargetRegion(regionEvent.getEncodedRegionName().toByteArray(),
5230 "RegionEvent marker from WAL ", regionEvent);
5231
5232 startRegionOperation(Operation.REPLAY_EVENT);
5233 try {
5234 if (ServerRegionReplicaUtil.isDefaultReplica(this.getRegionInfo())) {
5235 return;
5236 }
5237
5238 if (regionEvent.getEventType() == EventType.REGION_CLOSE) {
5239
5240 return;
5241 }
5242 if (regionEvent.getEventType() != EventType.REGION_OPEN) {
5243 LOG.warn(getRegionInfo().getEncodedName() + " : "
5244 + "Unknown region event received, ignoring :"
5245 + TextFormat.shortDebugString(regionEvent));
5246 return;
5247 }
5248
5249 if (LOG.isDebugEnabled()) {
5250 LOG.debug(getRegionInfo().getEncodedName() + " : "
5251 + "Replaying region open event marker " + TextFormat.shortDebugString(regionEvent));
5252 }
5253
5254
5255 synchronized (writestate) {
5256
5257
5258
5259
5260
5261
5262 if (this.lastReplayedOpenRegionSeqId <= regionEvent.getLogSequenceNumber()) {
5263 this.lastReplayedOpenRegionSeqId = regionEvent.getLogSequenceNumber();
5264 } else {
5265 LOG.warn(getRegionInfo().getEncodedName() + " : "
5266 + "Skipping replaying region event :" + TextFormat.shortDebugString(regionEvent)
5267 + " because its sequence id is smaller than this regions lastReplayedOpenRegionSeqId "
5268 + " of " + lastReplayedOpenRegionSeqId);
5269 return;
5270 }
5271
5272
5273
5274 for (StoreDescriptor storeDescriptor : regionEvent.getStoresList()) {
5275
5276 byte[] family = storeDescriptor.getFamilyName().toByteArray();
5277 Store store = getStore(family);
5278 if (store == null) {
5279 LOG.warn(getRegionInfo().getEncodedName() + " : "
5280 + "Received a region open marker from primary, but the family is not found. "
5281 + "Ignoring. StoreDescriptor:" + storeDescriptor);
5282 continue;
5283 }
5284
5285 long storeSeqId = store.getMaxSequenceId();
5286 List<String> storeFiles = storeDescriptor.getStoreFileList();
5287 try {
5288 store.refreshStoreFiles(storeFiles);
5289 } catch (FileNotFoundException ex) {
5290 LOG.warn(getRegionInfo().getEncodedName() + " : "
5291 + "At least one of the store files: " + storeFiles
5292 + " doesn't exist any more. Skip loading the file(s)", ex);
5293 continue;
5294 }
5295 if (store.getMaxSequenceId() != storeSeqId) {
5296
5297 lastStoreFlushTimeMap.put(store, EnvironmentEdgeManager.currentTime());
5298 }
5299
5300 if (writestate.flushing) {
5301
5302 if (this.prepareFlushResult.flushOpSeqId <= regionEvent.getLogSequenceNumber()) {
5303 StoreFlushContext ctx = this.prepareFlushResult.storeFlushCtxs == null ?
5304 null : this.prepareFlushResult.storeFlushCtxs.get(family);
5305 if (ctx != null) {
5306 long snapshotSize = store.getFlushableSize();
5307 ctx.abort();
5308 this.addAndGetGlobalMemstoreSize(-snapshotSize);
5309 this.prepareFlushResult.storeFlushCtxs.remove(family);
5310 }
5311 }
5312 }
5313
5314
5315 dropMemstoreContentsForSeqId(regionEvent.getLogSequenceNumber(), store);
5316 if (storeSeqId > this.maxFlushedSeqId) {
5317 this.maxFlushedSeqId = storeSeqId;
5318 }
5319 }
5320
5321
5322
5323 dropPrepareFlushIfPossible();
5324
5325
5326 mvcc.await();
5327
5328
5329
5330 this.setReadsEnabled(true);
5331
5332
5333
5334 synchronized (this) {
5335 notifyAll();
5336 }
5337 }
5338 logRegionFiles();
5339 } finally {
5340 closeRegionOperation(Operation.REPLAY_EVENT);
5341 }
5342 }
5343
5344 void replayWALBulkLoadEventMarker(WALProtos.BulkLoadDescriptor bulkLoadEvent) throws IOException {
5345 checkTargetRegion(bulkLoadEvent.getEncodedRegionName().toByteArray(),
5346 "BulkLoad marker from WAL ", bulkLoadEvent);
5347
5348 if (ServerRegionReplicaUtil.isDefaultReplica(this.getRegionInfo())) {
5349 return;
5350 }
5351
5352 if (LOG.isDebugEnabled()) {
5353 LOG.debug(getRegionInfo().getEncodedName() + " : "
5354 + "Replaying bulkload event marker " + TextFormat.shortDebugString(bulkLoadEvent));
5355 }
5356
5357 boolean multipleFamilies = false;
5358 byte[] family = null;
5359 for (StoreDescriptor storeDescriptor : bulkLoadEvent.getStoresList()) {
5360 byte[] fam = storeDescriptor.getFamilyName().toByteArray();
5361 if (family == null) {
5362 family = fam;
5363 } else if (!Bytes.equals(family, fam)) {
5364 multipleFamilies = true;
5365 break;
5366 }
5367 }
5368
5369 startBulkRegionOperation(multipleFamilies);
5370 try {
5371
5372 synchronized (writestate) {
5373
5374
5375
5376
5377
5378
5379 if (bulkLoadEvent.getBulkloadSeqNum() >= 0
5380 && this.lastReplayedOpenRegionSeqId >= bulkLoadEvent.getBulkloadSeqNum()) {
5381 LOG.warn(getRegionInfo().getEncodedName() + " : "
5382 + "Skipping replaying bulkload event :"
5383 + TextFormat.shortDebugString(bulkLoadEvent)
5384 + " because its sequence id is smaller than this region's lastReplayedOpenRegionSeqId"
5385 + " =" + lastReplayedOpenRegionSeqId);
5386
5387 return;
5388 }
5389
5390 for (StoreDescriptor storeDescriptor : bulkLoadEvent.getStoresList()) {
5391
5392 family = storeDescriptor.getFamilyName().toByteArray();
5393 Store store = getStore(family);
5394 if (store == null) {
5395 LOG.warn(getRegionInfo().getEncodedName() + " : "
5396 + "Received a bulk load marker from primary, but the family is not found. "
5397 + "Ignoring. StoreDescriptor:" + storeDescriptor);
5398 continue;
5399 }
5400
5401 List<String> storeFiles = storeDescriptor.getStoreFileList();
5402 for (String storeFile : storeFiles) {
5403 StoreFileInfo storeFileInfo = null;
5404 try {
5405 storeFileInfo = fs.getStoreFileInfo(Bytes.toString(family), storeFile);
5406 store.bulkLoadHFile(storeFileInfo);
5407 } catch(FileNotFoundException ex) {
5408 LOG.warn(getRegionInfo().getEncodedName() + " : "
5409 + ((storeFileInfo != null) ? storeFileInfo.toString() :
5410 (new Path(Bytes.toString(family), storeFile)).toString())
5411 + " doesn't exist any more. Skip loading the file");
5412 }
5413 }
5414 }
5415 }
5416 if (bulkLoadEvent.getBulkloadSeqNum() > 0) {
5417 mvcc.advanceTo(bulkLoadEvent.getBulkloadSeqNum());
5418 }
5419 } finally {
5420 closeBulkRegionOperation();
5421 }
5422 }
5423
5424
5425
5426
5427 private void dropPrepareFlushIfPossible() {
5428 if (writestate.flushing) {
5429 boolean canDrop = true;
5430 if (prepareFlushResult.storeFlushCtxs != null) {
5431 for (Entry<byte[], StoreFlushContext> entry
5432 : prepareFlushResult.storeFlushCtxs.entrySet()) {
5433 Store store = getStore(entry.getKey());
5434 if (store == null) {
5435 continue;
5436 }
5437 if (store.getSnapshotSize() > 0) {
5438 canDrop = false;
5439 break;
5440 }
5441 }
5442 }
5443
5444
5445
5446 if (canDrop) {
5447 writestate.flushing = false;
5448 this.prepareFlushResult = null;
5449 }
5450 }
5451 }
5452
5453 @edu.umd.cs.findbugs.annotations.SuppressWarnings(value = "NN_NAKED_NOTIFY",
5454 justification = "Notify is about post replay. Intentional")
5455 @Override
5456 public boolean refreshStoreFiles() throws IOException {
5457 if (ServerRegionReplicaUtil.isDefaultReplica(this.getRegionInfo())) {
5458 return false;
5459 }
5460
5461 if (LOG.isDebugEnabled()) {
5462 LOG.debug(getRegionInfo().getEncodedName() + " : "
5463 + "Refreshing store files to see whether we can free up memstore");
5464 }
5465
5466 long totalFreedSize = 0;
5467
5468 long smallestSeqIdInStores = Long.MAX_VALUE;
5469
5470 startRegionOperation();
5471 try {
5472 Map<Store, Long> map = new HashMap<Store, Long>();
5473 synchronized (writestate) {
5474 for (Store store : getStores()) {
5475
5476
5477 long maxSeqIdBefore = store.getMaxSequenceId();
5478
5479
5480 store.refreshStoreFiles();
5481
5482 long storeSeqId = store.getMaxSequenceId();
5483 if (storeSeqId < smallestSeqIdInStores) {
5484 smallestSeqIdInStores = storeSeqId;
5485 }
5486
5487
5488 if (storeSeqId > maxSeqIdBefore) {
5489 if (writestate.flushing) {
5490
5491 if (this.prepareFlushResult.flushOpSeqId <= storeSeqId) {
5492 StoreFlushContext ctx = this.prepareFlushResult.storeFlushCtxs == null ?
5493 null : this.prepareFlushResult.storeFlushCtxs.get(store.getFamily().getName());
5494 if (ctx != null) {
5495 long snapshotSize = store.getFlushableSize();
5496 ctx.abort();
5497 this.addAndGetGlobalMemstoreSize(-snapshotSize);
5498 this.prepareFlushResult.storeFlushCtxs.remove(store.getFamily().getName());
5499 totalFreedSize += snapshotSize;
5500 }
5501 }
5502 }
5503
5504 map.put(store, storeSeqId);
5505 }
5506 }
5507
5508
5509
5510 dropPrepareFlushIfPossible();
5511
5512
5513
5514 for (Store s : getStores()) {
5515 mvcc.advanceTo(s.getMaxMemstoreTS());
5516 }
5517
5518
5519
5520
5521
5522
5523 if (this.lastReplayedOpenRegionSeqId < smallestSeqIdInStores) {
5524 this.lastReplayedOpenRegionSeqId = smallestSeqIdInStores;
5525 }
5526 }
5527 if (!map.isEmpty()) {
5528 for (Map.Entry<Store, Long> entry : map.entrySet()) {
5529
5530 totalFreedSize += dropMemstoreContentsForSeqId(entry.getValue(), entry.getKey());
5531 }
5532 }
5533
5534
5535 synchronized (this) {
5536 notifyAll();
5537 }
5538 return totalFreedSize > 0;
5539 } finally {
5540 closeRegionOperation();
5541 }
5542 }
5543
5544 private void logRegionFiles() {
5545 if (LOG.isTraceEnabled()) {
5546 LOG.trace(getRegionInfo().getEncodedName() + " : Store files for region: ");
5547 for (Store s : stores.values()) {
5548 Collection<StoreFile> storeFiles = s.getStorefiles();
5549 if (storeFiles == null) continue;
5550 for (StoreFile sf : storeFiles) {
5551 LOG.trace(getRegionInfo().getEncodedName() + " : " + sf);
5552 }
5553 }
5554 }
5555 }
5556
5557
5558
5559
5560 private void checkTargetRegion(byte[] encodedRegionName, String exceptionMsg, Object payload)
5561 throws WrongRegionException {
5562 if (Bytes.equals(this.getRegionInfo().getEncodedNameAsBytes(), encodedRegionName)) {
5563 return;
5564 }
5565
5566 if (!RegionReplicaUtil.isDefaultReplica(this.getRegionInfo()) &&
5567 Bytes.equals(encodedRegionName,
5568 this.fs.getRegionInfoForFS().getEncodedNameAsBytes())) {
5569 return;
5570 }
5571
5572 throw new WrongRegionException(exceptionMsg + payload
5573 + " targetted for region " + Bytes.toStringBinary(encodedRegionName)
5574 + " does not match this region: " + this.getRegionInfo());
5575 }
5576
5577
5578
5579
5580
5581
5582
5583 protected boolean restoreEdit(final Store s, final Cell cell) {
5584 long kvSize = s.add(cell);
5585 if (this.rsAccounting != null) {
5586 rsAccounting.addAndGetRegionReplayEditsSize(getRegionInfo().getRegionName(), kvSize);
5587 }
5588 return isFlushSize(this.addAndGetGlobalMemstoreSize(kvSize));
5589 }
5590
5591
5592
5593
5594
5595
5596
5597 private static boolean isZeroLengthThenDelete(final FileSystem fs, final Path p)
5598 throws IOException {
5599 FileStatus stat = fs.getFileStatus(p);
5600 if (stat.getLen() > 0) return false;
5601 LOG.warn("File " + p + " is zero-length, deleting.");
5602 fs.delete(p, false);
5603 return true;
5604 }
5605
5606 protected HStore instantiateHStore(final HColumnDescriptor family) throws IOException {
5607 return new HStore(this, family, this.conf);
5608 }
5609
5610 @Override
5611 public Store getStore(final byte[] column) {
5612 return this.stores.get(column);
5613 }
5614
5615
5616
5617
5618
5619 private Store getStore(Cell cell) {
5620 for (Map.Entry<byte[], Store> famStore : stores.entrySet()) {
5621 if (Bytes.equals(
5622 cell.getFamilyArray(), cell.getFamilyOffset(), cell.getFamilyLength(),
5623 famStore.getKey(), 0, famStore.getKey().length)) {
5624 return famStore.getValue();
5625 }
5626 }
5627
5628 return null;
5629 }
5630
5631 @Override
5632 public List<Store> getStores() {
5633 List<Store> list = new ArrayList<Store>(stores.size());
5634 list.addAll(stores.values());
5635 return list;
5636 }
5637
5638 @Override
5639 public List<String> getStoreFileList(final byte [][] columns)
5640 throws IllegalArgumentException {
5641 List<String> storeFileNames = new ArrayList<String>();
5642 synchronized(closeLock) {
5643 for(byte[] column : columns) {
5644 Store store = this.stores.get(column);
5645 if (store == null) {
5646 throw new IllegalArgumentException("No column family : " +
5647 new String(column, StandardCharsets.UTF_8) + " available");
5648 }
5649 Collection<StoreFile> storeFiles = store.getStorefiles();
5650 if (storeFiles == null) continue;
5651 for (StoreFile storeFile: storeFiles) {
5652 storeFileNames.add(storeFile.getPath().toString());
5653 }
5654
5655 logRegionFiles();
5656 }
5657 }
5658 return storeFileNames;
5659 }
5660
5661
5662
5663
5664
5665
5666 void checkRow(final byte [] row, String op) throws IOException {
5667 if (!rowIsInRange(getRegionInfo(), row)) {
5668 throw new WrongRegionException("Requested row out of range for " +
5669 op + " on HRegion " + this + ", startKey='" +
5670 Bytes.toStringBinary(getRegionInfo().getStartKey()) + "', getEndKey()='" +
5671 Bytes.toStringBinary(getRegionInfo().getEndKey()) + "', row='" +
5672 Bytes.toStringBinary(row) + "'");
5673 }
5674 }
5675
5676
5677
5678
5679
5680
5681
5682
5683 public RowLock getRowLock(byte[] row) throws IOException {
5684 return getRowLock(row, false);
5685 }
5686
5687 @Override
5688 public RowLock getRowLock(byte[] row, boolean readLock) throws IOException {
5689 return getRowLock(row, readLock, true);
5690 }
5691
5692
5693
5694
5695
5696
5697
5698
5699
5700
5701
5702
5703
5704
5705 public RowLock getRowLock(byte[] row, boolean readLock, boolean waitForLock) throws IOException {
5706
5707 checkRow(row, "row lock");
5708 return getRowLockInternal(row, readLock, waitForLock, null);
5709 }
5710
5711
5712 protected RowLock getRowLockInternal(byte[] row)
5713 throws IOException {
5714 return getRowLockInternal(row, false, true, null);
5715 }
5716
5717 protected RowLock getRowLockInternal(byte[] row, boolean readLock, boolean waitForLock,
5718 final RowLock prevRowLock) throws IOException {
5719
5720 HashedBytes rowKey = new HashedBytes(row);
5721
5722 RowLockContext rowLockContext = null;
5723 RowLockImpl result = null;
5724 TraceScope traceScope = null;
5725
5726
5727 if (Trace.isTracing()) {
5728 traceScope = Trace.startSpan("HRegion.getRowLock");
5729 traceScope.getSpan().addTimelineAnnotation("Getting a " + (readLock?"readLock":"writeLock"));
5730 }
5731
5732 boolean success = false;
5733 try {
5734
5735
5736 while (result == null) {
5737
5738
5739
5740 rowLockContext = new RowLockContext(rowKey);
5741 RowLockContext existingContext = lockedRows.putIfAbsent(rowKey, rowLockContext);
5742
5743
5744 if (existingContext != null) {
5745 rowLockContext = existingContext;
5746 }
5747
5748
5749
5750
5751 if (readLock) {
5752
5753
5754 RowLockImpl prevRowLockImpl = (RowLockImpl)prevRowLock;
5755 if ((prevRowLockImpl != null) && (prevRowLockImpl.getLock() ==
5756 rowLockContext.readWriteLock.readLock())) {
5757 success = true;
5758 return prevRowLock;
5759 }
5760 result = rowLockContext.newReadLock();
5761 } else {
5762 result = rowLockContext.newWriteLock();
5763 }
5764 }
5765
5766 int timeout = rowLockWaitDuration;
5767 boolean reachDeadlineFirst = false;
5768 RpcCallContext call = RpcServer.getCurrentCall();
5769 if (call != null && call.getDeadline() < Long.MAX_VALUE) {
5770 int timeToDeadline = (int)(call.getDeadline() - System.currentTimeMillis());
5771 if (timeToDeadline <= this.rowLockWaitDuration) {
5772 reachDeadlineFirst = true;
5773 timeout = timeToDeadline;
5774 }
5775 }
5776
5777 boolean lockAvailable = false;
5778 if (timeout > 0) {
5779 if (waitForLock) {
5780
5781 lockAvailable = result.getLock().tryLock(timeout, TimeUnit.MILLISECONDS);
5782 } else {
5783
5784 lockAvailable = result.getLock().tryLock();
5785 }
5786 }
5787 if (!lockAvailable) {
5788 if (traceScope != null) {
5789 traceScope.getSpan().addTimelineAnnotation("Failed to get row lock");
5790 }
5791 result = null;
5792 String message = "Timed out waiting for lock for row: " + rowKey + " in region " +
5793 getRegionInfo().getEncodedName() + ", timeout=" + timeout + ", deadlined=" +
5794 reachDeadlineFirst + ", waitForLock=" + waitForLock;
5795 if (waitForLock) {
5796 if (reachDeadlineFirst) {
5797 LOG.info("TIMEOUT: " + message);
5798 throw new TimeoutIOException(message);
5799 } else {
5800
5801 LOG.info("IOE " + message);
5802 throw new IOException(message);
5803 }
5804 } else {
5805
5806 return null;
5807 }
5808 }
5809 rowLockContext.setThreadName(Thread.currentThread().getName());
5810 success = true;
5811 return result;
5812 } catch (InterruptedException ie) {
5813 if (LOG.isDebugEnabled()) {
5814 LOG.debug("Thread interrupted waiting for lock on row: " + rowKey);
5815 }
5816 if (traceScope != null) {
5817 traceScope.getSpan().addTimelineAnnotation("Interrupted exception getting row lock");
5818 }
5819 throw throwOnInterrupt(ie);
5820 } catch (Error error) {
5821
5822
5823
5824 LOG.warn("Error to get row lock for " + Bytes.toStringBinary(row) + ", cause: " + error);
5825 IOException ioe = new IOException(error);
5826 if (traceScope != null) {
5827 traceScope.getSpan().addTimelineAnnotation("Error getting row lock");
5828 }
5829 throw ioe;
5830 } finally {
5831
5832 if (!success && rowLockContext != null) {
5833 rowLockContext.cleanUp();
5834 }
5835 if (traceScope != null) {
5836 traceScope.close();
5837 }
5838 }
5839 }
5840
5841 @Override
5842 public void releaseRowLocks(List<RowLock> rowLocks) {
5843 if (rowLocks != null) {
5844 for (int i = 0; i < rowLocks.size(); i++) {
5845 rowLocks.get(i).release();
5846 }
5847 rowLocks.clear();
5848 }
5849 }
5850
5851 public ConcurrentHashMap<HashedBytes, RowLockContext> getLockedRows() {
5852 return lockedRows;
5853 }
5854
5855 class RowLockContext {
5856 private final HashedBytes row;
5857 final ReadWriteLock readWriteLock = new ReentrantReadWriteLock(true);
5858 final AtomicBoolean usable = new AtomicBoolean(true);
5859 final AtomicInteger count = new AtomicInteger(0);
5860 final Object lock = new Object();
5861 private String threadName;
5862
5863 RowLockContext(HashedBytes row) {
5864 this.row = row;
5865 }
5866
5867 RowLockImpl newWriteLock() {
5868 Lock l = readWriteLock.writeLock();
5869 return getRowLock(l);
5870 }
5871 RowLockImpl newReadLock() {
5872 Lock l = readWriteLock.readLock();
5873 return getRowLock(l);
5874 }
5875
5876 private RowLockImpl getRowLock(Lock l) {
5877 count.incrementAndGet();
5878 synchronized (lock) {
5879 if (usable.get()) {
5880 return new RowLockImpl(this, l);
5881 } else {
5882 return null;
5883 }
5884 }
5885 }
5886
5887 void cleanUp() {
5888 long c = count.decrementAndGet();
5889 if (c <= 0) {
5890 synchronized (lock) {
5891 if (count.get() <= 0 && usable.get()){
5892 usable.set(false);
5893 RowLockContext removed = lockedRows.remove(row);
5894 assert removed == this: "we should never remove a different context";
5895 }
5896 }
5897 }
5898 }
5899
5900 public void setThreadName(String threadName) {
5901 this.threadName = threadName;
5902 }
5903
5904 @Override
5905 public String toString() {
5906 return "RowLockContext{" +
5907 "row=" + row +
5908 ", readWriteLock=" + readWriteLock +
5909 ", count=" + count +
5910 ", threadName=" + threadName +
5911 '}';
5912 }
5913 }
5914
5915
5916
5917
5918 public static class RowLockImpl implements RowLock {
5919 private final RowLockContext context;
5920 private final Lock lock;
5921
5922 public RowLockImpl(RowLockContext context, Lock lock) {
5923 this.context = context;
5924 this.lock = lock;
5925 }
5926
5927 public Lock getLock() {
5928 return lock;
5929 }
5930
5931 public RowLockContext getContext() {
5932 return context;
5933 }
5934
5935 @Override
5936 public void release() {
5937 lock.unlock();
5938 context.cleanUp();
5939 }
5940
5941 @Override
5942 public String toString() {
5943 return "RowLockImpl{" +
5944 "context=" + context +
5945 ", lock=" + lock +
5946 '}';
5947 }
5948 }
5949
5950
5951
5952
5953
5954
5955
5956 private static boolean hasMultipleColumnFamilies(Collection<Pair<byte[], String>> familyPaths) {
5957 boolean multipleFamilies = false;
5958 byte[] family = null;
5959 for (Pair<byte[], String> pair : familyPaths) {
5960 byte[] fam = pair.getFirst();
5961 if (family == null) {
5962 family = fam;
5963 } else if (!Bytes.equals(family, fam)) {
5964 multipleFamilies = true;
5965 break;
5966 }
5967 }
5968 return multipleFamilies;
5969 }
5970
5971 @Override
5972 public boolean bulkLoadHFiles(Collection<Pair<byte[], String>> familyPaths, boolean assignSeqId,
5973 BulkLoadListener bulkLoadListener, List<String> clusterIds) throws IOException {
5974 long seqId = -1;
5975 Map<byte[], List<Path>> storeFiles = new TreeMap<byte[], List<Path>>(Bytes.BYTES_COMPARATOR);
5976 Map<String, Long> storeFilesSizes = new HashMap<String, Long>();
5977 Preconditions.checkNotNull(familyPaths);
5978
5979 startBulkRegionOperation(hasMultipleColumnFamilies(familyPaths));
5980 boolean isSuccessful = false;
5981 try {
5982 this.writeRequestsCount.increment();
5983
5984
5985
5986
5987 List<IOException> ioes = new ArrayList<IOException>();
5988 List<Pair<byte[], String>> failures = new ArrayList<Pair<byte[], String>>();
5989 for (Pair<byte[], String> p : familyPaths) {
5990 byte[] familyName = p.getFirst();
5991 String path = p.getSecond();
5992
5993 Store store = getStore(familyName);
5994 if (store == null) {
5995 IOException ioe = new org.apache.hadoop.hbase.DoNotRetryIOException(
5996 "No such column family " + Bytes.toStringBinary(familyName));
5997 ioes.add(ioe);
5998 } else {
5999 try {
6000 store.assertBulkLoadHFileOk(new Path(path));
6001 } catch (WrongRegionException wre) {
6002
6003 failures.add(p);
6004 } catch (IOException ioe) {
6005
6006 ioes.add(ioe);
6007 }
6008 }
6009 }
6010
6011
6012
6013 checkInterrupt();
6014
6015
6016 if (ioes.size() != 0) {
6017 IOException e = MultipleIOException.createIOException(ioes);
6018 LOG.error("There were one or more IO errors when checking if the bulk load is ok.", e);
6019 throw e;
6020 }
6021
6022
6023 if (failures.size() != 0) {
6024 StringBuilder list = new StringBuilder();
6025 for (Pair<byte[], String> p : failures) {
6026 list.append("\n").append(Bytes.toString(p.getFirst())).append(" : ")
6027 .append(p.getSecond());
6028 }
6029
6030 LOG.warn("There was a recoverable bulk load failure likely due to a" +
6031 " split. These (family, HFile) pairs were not loaded: " + list);
6032 return isSuccessful;
6033 }
6034
6035
6036
6037
6038
6039
6040 if (assignSeqId) {
6041 FlushResult fs = flushcache(true, false);
6042 if (fs.isFlushSucceeded()) {
6043 seqId = ((FlushResultImpl)fs).flushSequenceId;
6044 } else if (fs.getResult() == FlushResult.Result.CANNOT_FLUSH_MEMSTORE_EMPTY) {
6045 seqId = ((FlushResultImpl)fs).flushSequenceId;
6046 } else if (fs.getResult() == FlushResult.Result.CANNOT_FLUSH) {
6047
6048
6049 waitForFlushes();
6050 } else {
6051 throw new IOException("Could not bulk load with an assigned sequential ID because the "+
6052 "flush didn't run. Reason for not flushing: " + ((FlushResultImpl)fs).failureReason);
6053 }
6054 }
6055
6056 Map<byte[], List<Pair<Path, Path>>> familyWithFinalPath =
6057 new TreeMap<>(Bytes.BYTES_COMPARATOR);
6058 for (Pair<byte[], String> p : familyPaths) {
6059 byte[] familyName = p.getFirst();
6060 String path = p.getSecond();
6061 Store store = getStore(familyName);
6062 if (!familyWithFinalPath.containsKey(familyName)) {
6063 familyWithFinalPath.put(familyName, new ArrayList<Pair<Path, Path>>());
6064 }
6065 List<Pair<Path, Path>> lst = familyWithFinalPath.get(familyName);
6066 try {
6067 String finalPath = path;
6068 if (bulkLoadListener != null) {
6069 finalPath = bulkLoadListener.prepareBulkLoad(familyName, path);
6070 }
6071 Pair<Path, Path> pair = ((HStore)store).preBulkLoadHFile(finalPath, seqId);
6072 lst.add(pair);
6073 } catch (IOException ioe) {
6074
6075
6076
6077
6078 LOG.error("There was a partial failure due to IO when attempting to" +
6079 " load " + Bytes.toString(p.getFirst()) + " : " + p.getSecond(), ioe);
6080 if (bulkLoadListener != null) {
6081 try {
6082 bulkLoadListener.failedBulkLoad(familyName, path);
6083 } catch (Exception ex) {
6084 LOG.error("Error while calling failedBulkLoad for family " +
6085 Bytes.toString(familyName) + " with path " + path, ex);
6086 }
6087 }
6088 throw ioe;
6089 }
6090 }
6091
6092 if (this.getCoprocessorHost() != null) {
6093 for (Map.Entry<byte[], List<Pair<Path, Path>>> entry : familyWithFinalPath.entrySet()) {
6094 this.getCoprocessorHost().preCommitStoreFile(entry.getKey(), entry.getValue());
6095 }
6096 }
6097 for (Map.Entry<byte[], List<Pair<Path, Path>>> entry : familyWithFinalPath.entrySet()) {
6098 byte[] familyName = entry.getKey();
6099 for (Pair<Path, Path> p : entry.getValue()) {
6100 String path = p.getFirst().toString();
6101 Path commitedStoreFile = p.getSecond();
6102 Store store = getStore(familyName);
6103 try {
6104 store.bulkLoadHFile(familyName, path, commitedStoreFile);
6105
6106 try {
6107 FileSystem fs = commitedStoreFile.getFileSystem(baseConf);
6108 storeFilesSizes.put(commitedStoreFile.getName(), fs.getFileStatus(commitedStoreFile)
6109 .getLen());
6110 } catch (IOException e) {
6111 LOG.warn("Failed to find the size of hfile " + commitedStoreFile);
6112 storeFilesSizes.put(commitedStoreFile.getName(), 0L);
6113 }
6114
6115 if(storeFiles.containsKey(familyName)) {
6116 storeFiles.get(familyName).add(commitedStoreFile);
6117 } else {
6118 List<Path> storeFileNames = new ArrayList<Path>();
6119 storeFileNames.add(commitedStoreFile);
6120 storeFiles.put(familyName, storeFileNames);
6121 }
6122 if (bulkLoadListener != null) {
6123 bulkLoadListener.doneBulkLoad(familyName, path);
6124 }
6125 } catch (IOException ioe) {
6126
6127
6128
6129
6130 LOG.error("There was a partial failure due to IO when attempting to" +
6131 " load " + Bytes.toString(familyName) + " : " + p.getSecond(), ioe);
6132 if (bulkLoadListener != null) {
6133 try {
6134 bulkLoadListener.failedBulkLoad(familyName, path);
6135 } catch (Exception ex) {
6136 LOG.error("Error while calling failedBulkLoad for family " +
6137 Bytes.toString(familyName) + " with path " + path, ex);
6138 }
6139 }
6140 throw ioe;
6141 }
6142 }
6143 }
6144
6145 isSuccessful = true;
6146 } finally {
6147 if (wal != null && !storeFiles.isEmpty()) {
6148
6149 try {
6150 WALProtos.BulkLoadDescriptor loadDescriptor =
6151 ProtobufUtil.toBulkLoadDescriptor(this.getRegionInfo().getTable(),
6152 ByteStringer.wrap(this.getRegionInfo().getEncodedNameAsBytes()), storeFiles,
6153 storeFilesSizes, seqId, clusterIds);
6154 WALUtil.writeBulkLoadMarkerAndSync(wal, this.htableDescriptor, getRegionInfo(),
6155 loadDescriptor, mvcc);
6156 } catch (IOException ioe) {
6157 if (this.rsServices != null) {
6158
6159
6160 isSuccessful = false;
6161 this.rsServices.abort("Failed to write bulk load event into WAL.", ioe);
6162 }
6163 }
6164 }
6165
6166 closeBulkRegionOperation();
6167 }
6168 return isSuccessful;
6169 }
6170
6171 @Override
6172 @Deprecated
6173 public boolean bulkLoadHFiles(Collection<Pair<byte[], String>> familyPaths, boolean assignSeqId,
6174 BulkLoadListener bulkLoadListener) throws IOException {
6175 LOG.warn("Deprecated bulkLoadHFiles invoked. This does not pass through source cluster ids." +
6176 " This is probably not what you want. See HBASE-22380.");
6177 return bulkLoadHFiles(familyPaths, assignSeqId, bulkLoadListener, EMPTY_CLUSTERID_LIST);
6178 }
6179
6180 @Override
6181 public boolean equals(Object o) {
6182 return o instanceof HRegion && Bytes.equals(getRegionInfo().getRegionName(),
6183 ((HRegion) o).getRegionInfo().getRegionName());
6184 }
6185
6186 @Override
6187 public int hashCode() {
6188 return Bytes.hashCode(getRegionInfo().getRegionName());
6189 }
6190
6191 @Override
6192 public String toString() {
6193 return getRegionInfo().getRegionNameAsString();
6194 }
6195
6196
6197
6198
6199 class RegionScannerImpl implements RegionScanner {
6200
6201 KeyValueHeap storeHeap = null;
6202
6203
6204 KeyValueHeap joinedHeap = null;
6205
6206
6207
6208 protected Cell joinedContinuationRow = null;
6209 private boolean filterClosed = false;
6210
6211 protected final byte[] stopRow;
6212 protected final boolean includeStopRow;
6213 protected final HRegion region;
6214
6215 private final long readPt;
6216 private final long maxResultSize;
6217 private final ScannerContext defaultScannerContext;
6218 private final FilterWrapper filter;
6219
6220 @Override
6221 public HRegionInfo getRegionInfo() {
6222 return region.getRegionInfo();
6223 }
6224
6225 RegionScannerImpl(Scan scan, List<KeyValueScanner> additionalScanners, HRegion region,
6226 long nonceGroup, long nonce) throws IOException {
6227 this.region = region;
6228 this.maxResultSize = scan.getMaxResultSize();
6229 if (scan.hasFilter()) {
6230 this.filter = new FilterWrapper(scan.getFilter());
6231 } else {
6232 this.filter = null;
6233 }
6234
6235
6236
6237
6238
6239
6240 defaultScannerContext = ScannerContext.newBuilder().setBatchLimit(scan.getBatch()).build();
6241
6242 this.stopRow = scan.getStopRow();
6243 this.includeStopRow = scan.includeStopRow();
6244
6245
6246 IsolationLevel isolationLevel = scan.getIsolationLevel();
6247 long mvccReadPoint = PackagePrivateFieldAccessor.getMvccReadPoint(scan);
6248 synchronized (scannerReadPoints) {
6249 if (mvccReadPoint > 0) {
6250 this.readPt = mvccReadPoint;
6251 } else if (nonce == HConstants.NO_NONCE || rsServices == null
6252 || rsServices.getNonceManager() == null) {
6253 this.readPt = getReadpoint(isolationLevel);
6254 } else {
6255 this.readPt = rsServices.getNonceManager().getMvccFromOperationContext(nonceGroup, nonce);
6256 }
6257 scannerReadPoints.put(this, this.readPt);
6258 }
6259 initializeScanners(scan, additionalScanners);
6260 }
6261
6262 RegionScannerImpl(Scan scan, List<KeyValueScanner> additionalScanners, HRegion region)
6263 throws IOException {
6264 this(scan, additionalScanners, region, HConstants.NO_NONCE, HConstants.NO_NONCE);
6265 }
6266
6267 protected void initializeScanners(Scan scan, List<KeyValueScanner> additionalScanners)
6268 throws IOException {
6269
6270
6271 List<KeyValueScanner> scanners = new ArrayList<KeyValueScanner>(scan.getFamilyMap().size());
6272 List<KeyValueScanner> joinedScanners
6273 = new ArrayList<KeyValueScanner>(scan.getFamilyMap().size());
6274
6275 List<KeyValueScanner> instantiatedScanners = new ArrayList<KeyValueScanner>();
6276
6277 if (additionalScanners != null && !additionalScanners.isEmpty()) {
6278 scanners.addAll(additionalScanners);
6279 instantiatedScanners.addAll(additionalScanners);
6280 }
6281
6282 try {
6283 for (Map.Entry<byte[], NavigableSet<byte[]>> entry : scan.getFamilyMap().entrySet()) {
6284 Store store = stores.get(entry.getKey());
6285 KeyValueScanner scanner = store.getScanner(scan, entry.getValue(), this.readPt);
6286 instantiatedScanners.add(scanner);
6287 if (this.filter == null || !scan.doLoadColumnFamiliesOnDemand()
6288 || this.filter.isFamilyEssential(entry.getKey())) {
6289 scanners.add(scanner);
6290 } else {
6291 joinedScanners.add(scanner);
6292 }
6293 }
6294 initializeKVHeap(scanners, joinedScanners, region);
6295 } catch (Throwable t) {
6296 throw handleException(instantiatedScanners, t);
6297 }
6298 }
6299
6300 protected void initializeKVHeap(List<KeyValueScanner> scanners,
6301 List<KeyValueScanner> joinedScanners, HRegion region)
6302 throws IOException {
6303 this.storeHeap = new KeyValueHeap(scanners, region.comparator);
6304 if (!joinedScanners.isEmpty()) {
6305 this.joinedHeap = new KeyValueHeap(joinedScanners, region.comparator);
6306 }
6307 }
6308
6309 private IOException handleException(List<KeyValueScanner> instantiatedScanners,
6310 Throwable t) {
6311
6312 scannerReadPoints.remove(this);
6313 if (storeHeap != null) {
6314 storeHeap.close();
6315 storeHeap = null;
6316 if (joinedHeap != null) {
6317 joinedHeap.close();
6318 joinedHeap = null;
6319 }
6320 } else {
6321
6322 for (KeyValueScanner scanner : instantiatedScanners) {
6323 scanner.close();
6324 }
6325 }
6326 return t instanceof IOException ? (IOException) t : new IOException(t);
6327 }
6328
6329 @Override
6330 public long getMaxResultSize() {
6331 return maxResultSize;
6332 }
6333
6334 @Override
6335 public long getMvccReadPoint() {
6336 return this.readPt;
6337 }
6338
6339 @Override
6340 public int getBatch() {
6341 return this.defaultScannerContext.getBatchLimit();
6342 }
6343
6344
6345
6346
6347
6348
6349 protected void resetFilters() throws IOException {
6350 if (filter != null) {
6351 filter.reset();
6352 }
6353 }
6354
6355 @Override
6356 public boolean next(List<Cell> outResults)
6357 throws IOException {
6358
6359 return next(outResults, defaultScannerContext);
6360 }
6361
6362 @Override
6363 public synchronized boolean next(List<Cell> outResults, ScannerContext scannerContext)
6364 throws IOException {
6365 if (this.filterClosed) {
6366 throw new UnknownScannerException("Scanner was closed (timed out?) " +
6367 "after we renewed it. Could be caused by a very slow scanner " +
6368 "or a lengthy garbage collection");
6369 }
6370 startRegionOperation(Operation.SCAN);
6371 try {
6372 return nextRaw(outResults, scannerContext);
6373 } finally {
6374 closeRegionOperation(Operation.SCAN);
6375 }
6376 }
6377
6378 @Override
6379 public boolean nextRaw(List<Cell> outResults) throws IOException {
6380
6381 return nextRaw(outResults, defaultScannerContext);
6382 }
6383
6384 @Override
6385 public boolean nextRaw(List<Cell> outResults, ScannerContext scannerContext)
6386 throws IOException {
6387 if (storeHeap == null) {
6388
6389 throw new UnknownScannerException("Scanner was closed");
6390 }
6391 boolean moreValues = false;
6392 if (outResults.isEmpty()) {
6393
6394
6395 moreValues = nextInternal(outResults, scannerContext);
6396 } else {
6397 List<Cell> tmpList = new ArrayList<Cell>();
6398 moreValues = nextInternal(tmpList, scannerContext);
6399 outResults.addAll(tmpList);
6400 }
6401
6402 if (!outResults.isEmpty()) {
6403 readRequestsCount.increment();
6404 }
6405
6406
6407
6408
6409 if (!scannerContext.mayHaveMoreCellsInRow()) {
6410 resetFilters();
6411 }
6412
6413 if (isFilterDoneInternal()) {
6414 moreValues = false;
6415 }
6416 return moreValues;
6417 }
6418
6419
6420
6421
6422 private boolean populateFromJoinedHeap(List<Cell> results, ScannerContext scannerContext)
6423 throws IOException {
6424 assert joinedContinuationRow != null;
6425 boolean moreValues =
6426 populateResult(results, this.joinedHeap, scannerContext,
6427 joinedContinuationRow.getRowArray(), joinedContinuationRow.getRowOffset(),
6428 joinedContinuationRow.getRowLength());
6429
6430 if (!scannerContext.checkAnyLimitReached(LimitScope.BETWEEN_CELLS)) {
6431
6432 joinedContinuationRow = null;
6433 }
6434
6435
6436 Collections.sort(results, comparator);
6437 return moreValues;
6438 }
6439
6440
6441
6442
6443
6444
6445
6446
6447
6448
6449
6450 private boolean populateResult(List<Cell> results, KeyValueHeap heap,
6451 ScannerContext scannerContext, byte[] currentRow, int offset, short length)
6452 throws IOException {
6453 Cell nextKv;
6454 boolean moreCellsInRow = false;
6455 boolean tmpKeepProgress = scannerContext.getKeepProgress();
6456
6457 LimitScope limitScope = LimitScope.BETWEEN_CELLS;
6458 do {
6459
6460
6461 checkInterrupt();
6462
6463
6464
6465
6466 scannerContext.setKeepProgress(true);
6467 heap.next(results, scannerContext);
6468 scannerContext.setKeepProgress(tmpKeepProgress);
6469
6470 nextKv = heap.peek();
6471 moreCellsInRow = moreCellsInRow(nextKv, currentRow, offset, length);
6472 if (!moreCellsInRow) incrementCountOfRowsScannedMetric(scannerContext);
6473
6474 if (moreCellsInRow && scannerContext.checkBatchLimit(limitScope)) {
6475 return scannerContext.setScannerState(NextState.BATCH_LIMIT_REACHED).hasMoreValues();
6476 } else if (scannerContext.checkSizeLimit(limitScope)) {
6477 ScannerContext.NextState state =
6478 moreCellsInRow ? NextState.SIZE_LIMIT_REACHED_MID_ROW : NextState.SIZE_LIMIT_REACHED;
6479 return scannerContext.setScannerState(state).hasMoreValues();
6480 } else if (scannerContext.checkTimeLimit(limitScope)) {
6481 ScannerContext.NextState state =
6482 moreCellsInRow ? NextState.TIME_LIMIT_REACHED_MID_ROW : NextState.TIME_LIMIT_REACHED;
6483 return scannerContext.setScannerState(state).hasMoreValues();
6484 }
6485 } while (moreCellsInRow);
6486 return nextKv != null;
6487 }
6488
6489
6490
6491
6492
6493
6494
6495
6496
6497
6498
6499 private boolean moreCellsInRow(final Cell nextKv, byte[] currentRow, int offset,
6500 short length) {
6501 return nextKv != null && CellUtil.matchingRow(nextKv, currentRow, offset, length);
6502 }
6503
6504
6505
6506
6507 @Override
6508 public synchronized boolean isFilterDone() throws IOException {
6509 return isFilterDoneInternal();
6510 }
6511
6512 private boolean isFilterDoneInternal() throws IOException {
6513 return this.filter != null && this.filter.filterAllRemaining();
6514 }
6515
6516 private boolean nextInternal(List<Cell> results, ScannerContext scannerContext)
6517 throws IOException {
6518 if (!results.isEmpty()) {
6519 throw new IllegalArgumentException("First parameter should be an empty list");
6520 }
6521 if (scannerContext == null) {
6522 throw new IllegalArgumentException("Scanner context cannot be null");
6523 }
6524 RpcCallContext rpcCall = RpcServer.getCurrentCall();
6525
6526
6527
6528
6529 int initialBatchProgress = scannerContext.getBatchProgress();
6530 long initialSizeProgress = scannerContext.getSizeProgress();
6531 long initialTimeProgress = scannerContext.getTimeProgress();
6532
6533
6534
6535
6536
6537
6538 while (true) {
6539
6540
6541 if (scannerContext.getKeepProgress()) {
6542
6543 scannerContext.setProgress(initialBatchProgress, initialSizeProgress,
6544 initialTimeProgress);
6545 } else {
6546 scannerContext.clearProgress();
6547 }
6548
6549 if (rpcCall != null) {
6550
6551
6552
6553
6554 long afterTime = rpcCall.disconnectSince();
6555 if (afterTime >= 0) {
6556 throw new CallerDisconnectedException(
6557 "Aborting on region " + getRegionInfo().getRegionNameAsString() + ", call " +
6558 this + " after " + afterTime + " ms, since " +
6559 "caller disconnected");
6560 }
6561 }
6562
6563
6564
6565 checkInterrupt();
6566
6567
6568 Cell current = this.storeHeap.peek();
6569
6570 byte[] currentRow = null;
6571 int offset = 0;
6572 short length = 0;
6573 if (current != null) {
6574 currentRow = current.getRowArray();
6575 offset = current.getRowOffset();
6576 length = current.getRowLength();
6577 }
6578
6579 boolean shouldStop = shouldStop(current);
6580
6581
6582
6583
6584 boolean hasFilterRow = this.filter != null && this.filter.hasFilterRow();
6585
6586
6587
6588
6589
6590 if (hasFilterRow) {
6591 if (LOG.isTraceEnabled()) {
6592 LOG.trace("filter#hasFilterRow is true which prevents partial results from being "
6593 + " formed. Changing scope of limits that may create partials");
6594 }
6595 scannerContext.setSizeLimitScope(LimitScope.BETWEEN_ROWS);
6596 scannerContext.setTimeLimitScope(LimitScope.BETWEEN_ROWS);
6597 }
6598
6599
6600
6601 if (joinedContinuationRow == null) {
6602
6603 if (shouldStop) {
6604 if (hasFilterRow) {
6605 filter.filterRowCells(results);
6606 }
6607 return scannerContext.setScannerState(NextState.NO_MORE_VALUES).hasMoreValues();
6608 }
6609
6610
6611
6612 if (filterRowKey(currentRow, offset, length)) {
6613 incrementCountOfRowsFilteredMetric(scannerContext);
6614
6615 if (isFilterDoneInternal()) {
6616 return scannerContext.setScannerState(NextState.NO_MORE_VALUES).hasMoreValues();
6617 }
6618
6619
6620
6621 incrementCountOfRowsScannedMetric(scannerContext);
6622 boolean moreRows = nextRow(scannerContext, currentRow, offset, length);
6623 if (!moreRows) {
6624 return scannerContext.setScannerState(NextState.NO_MORE_VALUES).hasMoreValues();
6625 }
6626 results.clear();
6627 continue;
6628 }
6629
6630
6631 populateResult(results, this.storeHeap, scannerContext, currentRow, offset, length);
6632
6633 if (scannerContext.checkAnyLimitReached(LimitScope.BETWEEN_CELLS)) {
6634 if (hasFilterRow) {
6635 throw new IncompatibleFilterException(
6636 "Filter whose hasFilterRow() returns true is incompatible with scans that must "
6637 + " stop mid-row because of a limit. ScannerContext:" + scannerContext);
6638 }
6639 return true;
6640 }
6641
6642
6643
6644 checkInterrupt();
6645
6646 Cell nextKv = this.storeHeap.peek();
6647 shouldStop = shouldStop(nextKv);
6648
6649 final boolean isEmptyRow = results.isEmpty();
6650
6651
6652
6653 FilterWrapper.FilterRowRetCode ret = FilterWrapper.FilterRowRetCode.NOT_CALLED;
6654 if (hasFilterRow) {
6655 ret = filter.filterRowCellsWithRet(results);
6656
6657
6658
6659
6660 long timeProgress = scannerContext.getTimeProgress();
6661 if (scannerContext.getKeepProgress()) {
6662 scannerContext.setProgress(initialBatchProgress, initialSizeProgress,
6663 initialTimeProgress);
6664 } else {
6665 scannerContext.clearProgress();
6666 }
6667 scannerContext.setTimeProgress(timeProgress);
6668 scannerContext.incrementBatchProgress(results.size());
6669 for (Cell cell : results) {
6670 scannerContext.incrementSizeProgress(CellUtil.estimatedHeapSizeOfWithoutTags(cell));
6671 }
6672 }
6673
6674 if (isEmptyRow || ret == FilterWrapper.FilterRowRetCode.EXCLUDE || filterRow()) {
6675 incrementCountOfRowsFilteredMetric(scannerContext);
6676 results.clear();
6677 boolean moreRows = nextRow(scannerContext, currentRow, offset, length);
6678 if (!moreRows) {
6679 return scannerContext.setScannerState(NextState.NO_MORE_VALUES).hasMoreValues();
6680 }
6681
6682
6683
6684 if (!shouldStop) continue;
6685 return scannerContext.setScannerState(NextState.NO_MORE_VALUES).hasMoreValues();
6686 }
6687
6688
6689
6690
6691
6692 if (this.joinedHeap != null) {
6693 boolean mayHaveData = joinedHeapMayHaveData(currentRow, offset, length);
6694 if (mayHaveData) {
6695 joinedContinuationRow = current;
6696 populateFromJoinedHeap(results, scannerContext);
6697
6698 if (scannerContext.checkAnyLimitReached(LimitScope.BETWEEN_CELLS)) {
6699 return true;
6700 }
6701 }
6702 }
6703 } else {
6704
6705 populateFromJoinedHeap(results, scannerContext);
6706 if (scannerContext.checkAnyLimitReached(LimitScope.BETWEEN_CELLS)) {
6707 return true;
6708 }
6709 }
6710
6711
6712 if (joinedContinuationRow != null) {
6713 return scannerContext.setScannerState(NextState.MORE_VALUES).hasMoreValues();
6714 }
6715
6716
6717
6718
6719 if (results.isEmpty()) {
6720 incrementCountOfRowsFilteredMetric(scannerContext);
6721 boolean moreRows = nextRow(scannerContext, currentRow, offset, length);
6722 if (!moreRows) {
6723 return scannerContext.setScannerState(NextState.NO_MORE_VALUES).hasMoreValues();
6724 }
6725 if (!shouldStop) continue;
6726 }
6727
6728 if (shouldStop) {
6729 return scannerContext.setScannerState(NextState.NO_MORE_VALUES).hasMoreValues();
6730 } else {
6731 return scannerContext.setScannerState(NextState.MORE_VALUES).hasMoreValues();
6732 }
6733 }
6734 }
6735
6736 protected void incrementCountOfRowsFilteredMetric(ScannerContext scannerContext) {
6737 if (scannerContext == null || !scannerContext.isTrackingMetrics()) return;
6738
6739 scannerContext.getMetrics().countOfRowsFiltered.incrementAndGet();
6740 }
6741
6742 protected void incrementCountOfRowsScannedMetric(ScannerContext scannerContext) {
6743 if (scannerContext == null || !scannerContext.isTrackingMetrics()) return;
6744
6745 scannerContext.getMetrics().countOfRowsScanned.incrementAndGet();
6746 }
6747
6748
6749
6750
6751
6752
6753
6754
6755 private boolean joinedHeapMayHaveData(byte[] currentRow, int offset, short length)
6756 throws IOException {
6757 Cell nextJoinedKv = joinedHeap.peek();
6758 boolean matchCurrentRow =
6759 nextJoinedKv != null && CellUtil.matchingRow(nextJoinedKv, currentRow, offset, length);
6760 boolean matchAfterSeek = false;
6761
6762
6763
6764 if (!matchCurrentRow) {
6765 Cell firstOnCurrentRow = KeyValueUtil.createFirstOnRow(currentRow, offset, length);
6766 boolean seekSuccessful = this.joinedHeap.requestSeek(firstOnCurrentRow, true, true);
6767 matchAfterSeek =
6768 seekSuccessful && joinedHeap.peek() != null
6769 && CellUtil.matchingRow(joinedHeap.peek(), currentRow, offset, length);
6770 }
6771
6772 return matchCurrentRow || matchAfterSeek;
6773 }
6774
6775
6776
6777
6778
6779
6780
6781
6782 private boolean filterRow() throws IOException {
6783
6784
6785 return filter != null && (!filter.hasFilterRow())
6786 && filter.filterRow();
6787 }
6788
6789 private boolean filterRowKey(byte[] row, int offset, short length) throws IOException {
6790 return filter != null
6791 && filter.filterRowKey(row, offset, length);
6792 }
6793
6794 protected boolean nextRow(ScannerContext scannerContext, byte[] currentRow, int offset,
6795 short length) throws IOException {
6796 assert this.joinedContinuationRow == null:
6797 "Trying to go to next row during joinedHeap read.";
6798 Cell next;
6799 while ((next = this.storeHeap.peek()) != null &&
6800 CellUtil.matchingRow(next, currentRow, offset, length)) {
6801
6802
6803 checkInterrupt();
6804 this.storeHeap.next(MOCKED_LIST);
6805 }
6806 resetFilters();
6807
6808
6809 return this.region.getCoprocessorHost() == null
6810 || this.region.getCoprocessorHost()
6811 .postScannerFilterRow(this, currentRow, offset, length);
6812 }
6813
6814 protected boolean shouldStop(Cell currentRowCell) {
6815 if (currentRowCell == null) {
6816 return true;
6817 }
6818 if (stopRow == null || Bytes.equals(stopRow, HConstants.EMPTY_START_ROW)) {
6819 return false;
6820 }
6821 int c = comparator.compareRows(currentRowCell, stopRow, 0, stopRow.length);
6822 return c > 0 || (c == 0 && !includeStopRow);
6823 }
6824
6825 @Override
6826 public synchronized void close() {
6827 if (storeHeap != null) {
6828 storeHeap.close();
6829 storeHeap = null;
6830 }
6831 if (joinedHeap != null) {
6832 joinedHeap.close();
6833 joinedHeap = null;
6834 }
6835
6836 scannerReadPoints.remove(this);
6837 this.filterClosed = true;
6838 }
6839
6840 KeyValueHeap getStoreHeapForTesting() {
6841 return storeHeap;
6842 }
6843
6844 @Override
6845 public synchronized boolean reseek(byte[] row) throws IOException {
6846 if (row == null) {
6847 throw new IllegalArgumentException("Row cannot be null.");
6848 }
6849 boolean result = false;
6850 startRegionOperation();
6851 KeyValue kv = KeyValueUtil.createFirstOnRow(row);
6852 try {
6853
6854 result = this.storeHeap.requestSeek(kv, true, true);
6855 if (this.joinedHeap != null) {
6856 result = this.joinedHeap.requestSeek(kv, true, true) || result;
6857 }
6858 } finally {
6859 closeRegionOperation();
6860 }
6861 return result;
6862 }
6863 }
6864
6865
6866
6867
6868
6869
6870
6871
6872
6873
6874
6875
6876
6877
6878
6879
6880
6881
6882
6883
6884 static HRegion newHRegion(Path tableDir, WAL wal, FileSystem fs,
6885 Configuration conf, HRegionInfo regionInfo, final HTableDescriptor htd,
6886 RegionServerServices rsServices) {
6887 try {
6888 @SuppressWarnings("unchecked")
6889 Class<? extends HRegion> regionClass =
6890 (Class<? extends HRegion>) conf.getClass(HConstants.REGION_IMPL, HRegion.class);
6891
6892 Constructor<? extends HRegion> c =
6893 regionClass.getConstructor(Path.class, WAL.class, FileSystem.class,
6894 Configuration.class, HRegionInfo.class, HTableDescriptor.class,
6895 RegionServerServices.class);
6896
6897 return c.newInstance(tableDir, wal, fs, conf, regionInfo, htd, rsServices);
6898 } catch (Throwable e) {
6899
6900 throw new IllegalStateException("Could not instantiate a region instance.", e);
6901 }
6902 }
6903
6904
6905
6906
6907
6908
6909
6910
6911
6912
6913
6914
6915
6916
6917
6918
6919
6920 public static HRegion createHRegion(final HRegionInfo info, final Path rootDir,
6921 final Configuration conf, final HTableDescriptor hTableDescriptor)
6922 throws IOException {
6923 return createHRegion(info, rootDir, conf, hTableDescriptor, null);
6924 }
6925
6926
6927
6928
6929
6930
6931
6932
6933
6934
6935
6936 public static void closeHRegion(final HRegion r) throws IOException {
6937 if (r == null) return;
6938 r.close();
6939 if (r.getWAL() == null) return;
6940 r.getWAL().close();
6941 }
6942
6943
6944
6945
6946
6947
6948
6949
6950
6951
6952
6953
6954
6955
6956 public static HRegion createHRegion(final HRegionInfo info, final Path rootDir,
6957 final Configuration conf,
6958 final HTableDescriptor hTableDescriptor,
6959 final WAL wal,
6960 final boolean initialize)
6961 throws IOException {
6962 return createHRegion(info, rootDir, conf, hTableDescriptor,
6963 wal, initialize, false);
6964 }
6965
6966
6967
6968
6969
6970
6971
6972
6973
6974
6975
6976
6977
6978
6979
6980 public static HRegion createHRegion(final HRegionInfo info, final Path rootDir,
6981 final Configuration conf,
6982 final HTableDescriptor hTableDescriptor,
6983 final WAL wal,
6984 final boolean initialize, final boolean ignoreWAL)
6985 throws IOException {
6986 Path tableDir = FSUtils.getTableDir(rootDir, info.getTable());
6987 return createHRegion(info, rootDir, tableDir, conf, hTableDescriptor, wal, initialize,
6988 ignoreWAL);
6989 }
6990
6991
6992
6993
6994
6995
6996
6997
6998
6999
7000
7001
7002
7003
7004
7005
7006 public static HRegion createHRegion(final HRegionInfo info, final Path rootDir,
7007 final Path tableDir, final Configuration conf, final HTableDescriptor hTableDescriptor,
7008 final WAL wal, final boolean initialize, final boolean ignoreWAL)
7009 throws IOException {
7010 LOG.info("creating HRegion " + info.getTable().getNameAsString()
7011 + " HTD == " + hTableDescriptor + " RootDir = " + rootDir +
7012 " Table name == " + info.getTable().getNameAsString());
7013 FileSystem fs = FileSystem.get(conf);
7014 HRegionFileSystem.createRegionOnFileSystem(conf, fs, tableDir, info);
7015 WAL effectiveWAL = wal;
7016 if (wal == null && !ignoreWAL) {
7017
7018
7019
7020 Configuration confForWAL = new Configuration(conf);
7021 FSUtils.setRootDir(confForWAL, rootDir);
7022 effectiveWAL = (new WALFactory(confForWAL,
7023 Collections.<WALActionsListener>singletonList(new MetricsWAL()),
7024 "hregion-" + RandomStringUtils.randomNumeric(8))).
7025 getWAL(info.getEncodedNameAsBytes(), info.getTable().getNamespace());
7026 }
7027 HRegion region = HRegion.newHRegion(tableDir,
7028 effectiveWAL, fs, conf, info, hTableDescriptor, null);
7029 if (initialize) region.initialize(null);
7030 return region;
7031 }
7032
7033 public static HRegion createHRegion(final HRegionInfo info, final Path rootDir,
7034 final Configuration conf,
7035 final HTableDescriptor hTableDescriptor,
7036 final WAL wal)
7037 throws IOException {
7038 return createHRegion(info, rootDir, conf, hTableDescriptor, wal, true);
7039 }
7040
7041
7042
7043
7044
7045
7046
7047
7048
7049
7050
7051
7052
7053 public static HRegion openHRegion(final HRegionInfo info,
7054 final HTableDescriptor htd, final WAL wal,
7055 final Configuration conf)
7056 throws IOException {
7057 return openHRegion(info, htd, wal, conf, null, null);
7058 }
7059
7060
7061
7062
7063
7064
7065
7066
7067
7068
7069
7070
7071
7072
7073
7074
7075 public static HRegion openHRegion(final HRegionInfo info,
7076 final HTableDescriptor htd, final WAL wal, final Configuration conf,
7077 final RegionServerServices rsServices,
7078 final CancelableProgressable reporter)
7079 throws IOException {
7080 return openHRegion(FSUtils.getRootDir(conf), info, htd, wal, conf, rsServices, reporter);
7081 }
7082
7083
7084
7085
7086
7087
7088
7089
7090
7091
7092
7093
7094
7095
7096 public static HRegion openHRegion(Path rootDir, final HRegionInfo info,
7097 final HTableDescriptor htd, final WAL wal, final Configuration conf)
7098 throws IOException {
7099 return openHRegion(rootDir, info, htd, wal, conf, null, null);
7100 }
7101
7102
7103
7104
7105
7106
7107
7108
7109
7110
7111
7112
7113
7114
7115
7116
7117 public static HRegion openHRegion(final Path rootDir, final HRegionInfo info,
7118 final HTableDescriptor htd, final WAL wal, final Configuration conf,
7119 final RegionServerServices rsServices,
7120 final CancelableProgressable reporter)
7121 throws IOException {
7122 FileSystem fs = null;
7123 if (rsServices != null) {
7124 fs = rsServices.getFileSystem();
7125 }
7126 if (fs == null) {
7127 fs = FileSystem.get(conf);
7128 }
7129 return openHRegion(conf, fs, rootDir, info, htd, wal, rsServices, reporter);
7130 }
7131
7132
7133
7134
7135
7136
7137
7138
7139
7140
7141
7142
7143
7144
7145
7146 public static HRegion openHRegion(final Configuration conf, final FileSystem fs,
7147 final Path rootDir, final HRegionInfo info, final HTableDescriptor htd, final WAL wal)
7148 throws IOException {
7149 return openHRegion(conf, fs, rootDir, info, htd, wal, null, null);
7150 }
7151
7152
7153
7154
7155
7156
7157
7158
7159
7160
7161
7162
7163
7164
7165
7166
7167
7168 public static HRegion openHRegion(final Configuration conf, final FileSystem fs,
7169 final Path rootDir, final HRegionInfo info, final HTableDescriptor htd, final WAL wal,
7170 final RegionServerServices rsServices, final CancelableProgressable reporter)
7171 throws IOException {
7172 Path tableDir = FSUtils.getTableDir(rootDir, info.getTable());
7173 return openHRegion(conf, fs, rootDir, tableDir, info, htd, wal, rsServices, reporter);
7174 }
7175
7176
7177
7178
7179
7180
7181
7182
7183
7184
7185
7186
7187
7188
7189
7190
7191
7192 public static HRegion openHRegion(final Configuration conf, final FileSystem fs,
7193 final Path rootDir, final Path tableDir, final HRegionInfo info, final HTableDescriptor htd,
7194 final WAL wal, final RegionServerServices rsServices,
7195 final CancelableProgressable reporter)
7196 throws IOException {
7197 if (info == null) throw new NullPointerException("Passed region info is null");
7198 if (LOG.isDebugEnabled()) {
7199 LOG.debug("Opening region: " + info);
7200 }
7201 HRegion r = HRegion.newHRegion(tableDir, wal, fs, conf, info, htd, rsServices);
7202 return r.openHRegion(reporter);
7203 }
7204
7205
7206
7207
7208
7209
7210
7211
7212
7213 public static HRegion openHRegion(final HRegion other, final CancelableProgressable reporter)
7214 throws IOException {
7215 HRegionFileSystem regionFs = other.getRegionFileSystem();
7216 HRegion r = newHRegion(regionFs.getTableDir(), other.getWAL(), regionFs.getFileSystem(),
7217 other.baseConf, other.getRegionInfo(), other.getTableDesc(), null);
7218 return r.openHRegion(reporter);
7219 }
7220
7221 public static Region openHRegion(final Region other, final CancelableProgressable reporter)
7222 throws IOException {
7223 return openHRegion((HRegion)other, reporter);
7224 }
7225
7226
7227
7228
7229
7230
7231
7232 protected HRegion openHRegion(final CancelableProgressable reporter)
7233 throws IOException {
7234
7235 checkCompressionCodecs();
7236
7237
7238 checkEncryption();
7239
7240 checkClassLoading();
7241 this.openSeqNum = initialize(reporter);
7242 this.mvcc.advanceTo(openSeqNum);
7243 if (wal != null && getRegionServerServices() != null && !writestate.readOnly
7244 && !recovering) {
7245
7246
7247
7248 writeRegionOpenMarker(wal, openSeqNum);
7249 }
7250 return this;
7251 }
7252
7253
7254
7255
7256
7257
7258
7259
7260
7261
7262 public static HRegion openReadOnlyFileSystemHRegion(final Configuration conf, final FileSystem fs,
7263 final Path tableDir, HRegionInfo info, final HTableDescriptor htd) throws IOException {
7264 if (info == null) {
7265 throw new NullPointerException("Passed region info is null");
7266 }
7267 if (LOG.isDebugEnabled()) {
7268 LOG.debug("Opening region (readOnly filesystem): " + info);
7269 }
7270 if (info.getReplicaId() <= 0) {
7271 info = new HRegionInfo((HRegionInfo) info, 1);
7272 }
7273 HRegion r = HRegion.newHRegion(tableDir, null, fs, conf, info, htd, null);
7274 r.writestate.setReadOnly(true);
7275 return r.openHRegion(null);
7276 }
7277
7278 public static void warmupHRegion(final HRegionInfo info,
7279 final HTableDescriptor htd, final WAL wal, final Configuration conf,
7280 final RegionServerServices rsServices,
7281 final CancelableProgressable reporter)
7282 throws IOException {
7283
7284 if (info == null) throw new NullPointerException("Passed region info is null");
7285
7286 if (LOG.isDebugEnabled()) {
7287 LOG.debug("HRegion.Warming up region: " + info);
7288 }
7289
7290 Path rootDir = FSUtils.getRootDir(conf);
7291 Path tableDir = FSUtils.getTableDir(rootDir, info.getTable());
7292
7293 FileSystem fs = null;
7294 if (rsServices != null) {
7295 fs = rsServices.getFileSystem();
7296 }
7297 if (fs == null) {
7298 fs = FileSystem.get(conf);
7299 }
7300
7301 HRegion r = HRegion.newHRegion(tableDir, wal, fs, conf, info, htd, null);
7302 r.initializeWarmup(reporter);
7303 }
7304
7305
7306 private void checkCompressionCodecs() throws IOException {
7307 for (HColumnDescriptor fam: this.htableDescriptor.getColumnFamilies()) {
7308 CompressionTest.testCompression(fam.getCompression());
7309 CompressionTest.testCompression(fam.getCompactionCompression());
7310 }
7311 }
7312
7313 private void checkEncryption() throws IOException {
7314 for (HColumnDescriptor fam: this.htableDescriptor.getColumnFamilies()) {
7315 EncryptionTest.testEncryption(conf, fam.getEncryptionType(), fam.getEncryptionKey());
7316 }
7317 }
7318
7319 private void checkClassLoading() throws IOException {
7320 RegionSplitPolicy.getSplitPolicyClass(this.htableDescriptor, conf);
7321 RegionCoprocessorHost.testTableCoprocessorAttrs(conf, this.htableDescriptor);
7322 }
7323
7324
7325
7326
7327
7328
7329 HRegion createDaughterRegionFromSplits(final HRegionInfo hri) throws IOException {
7330
7331 fs.commitDaughterRegion(hri);
7332
7333
7334 WAL daughterWAL = rsServices == null ? getWAL() :rsServices.getWAL(hri);
7335
7336 HRegion r = HRegion.newHRegion(this.fs.getTableDir(), daughterWAL,
7337 fs.getFileSystem(), this.getBaseConf(), hri, this.getTableDesc(), rsServices);
7338 r.readRequestsCount.set(this.getReadRequestsCount() / 2);
7339 r.writeRequestsCount.set(this.getWriteRequestsCount() / 2);
7340 return r;
7341 }
7342
7343
7344
7345
7346
7347
7348
7349 HRegion createMergedRegionFromMerges(final HRegionInfo mergedRegionInfo,
7350 final HRegion region_b) throws IOException {
7351 WAL mergedRegionWAL = rsServices == null ? getWAL() : rsServices.getWAL(mergedRegionInfo);
7352 HRegion r = HRegion.newHRegion(this.fs.getTableDir(), mergedRegionWAL,
7353 fs.getFileSystem(), this.getBaseConf(), mergedRegionInfo,
7354 this.getTableDesc(), this.rsServices);
7355 r.readRequestsCount.set(this.getReadRequestsCount()
7356 + region_b.getReadRequestsCount());
7357 r.writeRequestsCount.set(this.getWriteRequestsCount()
7358
7359 + region_b.getWriteRequestsCount());
7360 this.fs.commitMergedRegion(mergedRegionInfo);
7361 return r;
7362 }
7363
7364
7365
7366
7367
7368
7369
7370
7371
7372
7373
7374
7375 public static void addRegionToMETA(final HRegion meta, final HRegion r) throws IOException {
7376 meta.checkResources();
7377
7378 byte[] row = r.getRegionInfo().getRegionName();
7379 final long now = EnvironmentEdgeManager.currentTime();
7380 final List<Cell> cells = new ArrayList<Cell>(2);
7381 cells.add(new KeyValue(row, HConstants.CATALOG_FAMILY,
7382 HConstants.REGIONINFO_QUALIFIER, now,
7383 r.getRegionInfo().toByteArray()));
7384
7385 cells.add(new KeyValue(row, HConstants.CATALOG_FAMILY,
7386 HConstants.META_VERSION_QUALIFIER, now,
7387 Bytes.toBytes(HConstants.META_VERSION)));
7388 meta.put(row, HConstants.CATALOG_FAMILY, cells);
7389 }
7390
7391
7392
7393
7394
7395
7396
7397
7398
7399 public static boolean rowIsInRange(HRegionInfo info, final byte [] row) {
7400 return ((info.getStartKey().length == 0) ||
7401 (Bytes.compareTo(info.getStartKey(), row) <= 0)) &&
7402 ((info.getEndKey().length == 0) ||
7403 (Bytes.compareTo(info.getEndKey(), row) > 0));
7404 }
7405
7406 public static boolean rowIsInRange(HRegionInfo info, final byte [] row, final int offset,
7407 final short length) {
7408 return ((info.getStartKey().length == 0) ||
7409 (Bytes.compareTo(info.getStartKey(), 0, info.getStartKey().length,
7410 row, offset, length) <= 0)) &&
7411 ((info.getEndKey().length == 0) ||
7412 (Bytes.compareTo(info.getEndKey(), 0, info.getEndKey().length, row, offset, length) > 0));
7413 }
7414
7415
7416
7417
7418
7419
7420
7421 public static HRegion mergeAdjacent(final HRegion srcA, final HRegion srcB)
7422 throws IOException {
7423 HRegion a = srcA;
7424 HRegion b = srcB;
7425
7426
7427
7428 if (srcA.getRegionInfo().getStartKey() == null) {
7429 if (srcB.getRegionInfo().getStartKey() == null) {
7430 throw new IOException("Cannot merge two regions with null start key");
7431 }
7432
7433 } else if ((srcB.getRegionInfo().getStartKey() == null) ||
7434 (Bytes.compareTo(srcA.getRegionInfo().getStartKey(),
7435 srcB.getRegionInfo().getStartKey()) > 0)) {
7436 a = srcB;
7437 b = srcA;
7438 }
7439
7440 if (!(Bytes.compareTo(a.getRegionInfo().getEndKey(),
7441 b.getRegionInfo().getStartKey()) == 0)) {
7442 throw new IOException("Cannot merge non-adjacent regions");
7443 }
7444 return merge(a, b);
7445 }
7446
7447
7448
7449
7450
7451
7452
7453
7454
7455 public static HRegion merge(final HRegion a, final HRegion b) throws IOException {
7456 if (!a.getRegionInfo().getTable().equals(b.getRegionInfo().getTable())) {
7457 throw new IOException("Regions do not belong to the same table");
7458 }
7459
7460 FileSystem fs = a.getRegionFileSystem().getFileSystem();
7461
7462 a.flush(true);
7463 b.flush(true);
7464
7465
7466 a.compact(true);
7467 if (LOG.isDebugEnabled()) {
7468 LOG.debug("Files for region: " + a);
7469 a.getRegionFileSystem().logFileSystemState(LOG);
7470 }
7471 b.compact(true);
7472 if (LOG.isDebugEnabled()) {
7473 LOG.debug("Files for region: " + b);
7474 b.getRegionFileSystem().logFileSystemState(LOG);
7475 }
7476
7477 RegionMergeTransactionImpl rmt = new RegionMergeTransactionImpl(a, b, true);
7478 if (!rmt.prepare(null)) {
7479 throw new IOException("Unable to merge regions " + a + " and " + b);
7480 }
7481 HRegionInfo mergedRegionInfo = rmt.getMergedRegionInfo();
7482 LOG.info("starting merge of regions: " + a + " and " + b
7483 + " into new region " + mergedRegionInfo.getRegionNameAsString()
7484 + " with start key <"
7485 + Bytes.toStringBinary(mergedRegionInfo.getStartKey())
7486 + "> and end key <"
7487 + Bytes.toStringBinary(mergedRegionInfo.getEndKey()) + ">");
7488 HRegion dstRegion;
7489 try {
7490 dstRegion = (HRegion)rmt.execute(null, null);
7491 } catch (IOException ioe) {
7492 rmt.rollback(null, null);
7493 throw new IOException("Failed merging region " + a + " and " + b
7494 + ", and successfully rolled back");
7495 }
7496 dstRegion.compact(true);
7497
7498 if (LOG.isDebugEnabled()) {
7499 LOG.debug("Files for new region");
7500 dstRegion.getRegionFileSystem().logFileSystemState(LOG);
7501 }
7502
7503
7504 for (Store s : dstRegion.getStores()) {
7505 s.closeAndArchiveCompactedFiles();
7506 }
7507 if (dstRegion.getRegionFileSystem().hasReferences(dstRegion.getTableDesc())) {
7508 throw new IOException("Merged region " + dstRegion
7509 + " still has references after the compaction, is compaction canceled?");
7510 }
7511
7512
7513 HFileArchiver.archiveRegion(a.getBaseConf(), fs, a.getRegionInfo());
7514
7515 HFileArchiver.archiveRegion(b.getBaseConf(), fs, b.getRegionInfo());
7516
7517 LOG.info("merge completed. New region is " + dstRegion);
7518 return dstRegion;
7519 }
7520
7521 @Override
7522 public Result get(final Get get) throws IOException {
7523 checkRow(get.getRow(), "Get");
7524
7525 if (get.hasFamilies()) {
7526 for (byte [] family: get.familySet()) {
7527 checkFamily(family);
7528 }
7529 } else {
7530 for (byte[] family: this.htableDescriptor.getFamiliesKeys()) {
7531 get.addFamily(family);
7532 }
7533 }
7534 List<Cell> results = get(get, true);
7535 boolean stale = this.getRegionInfo().getReplicaId() != 0;
7536 return Result.create(results, get.isCheckExistenceOnly() ? !results.isEmpty() : null, stale);
7537 }
7538
7539 @Override
7540 public List<Cell> get(Get get, boolean withCoprocessor) throws IOException {
7541 return get(get, withCoprocessor, HConstants.NO_NONCE, HConstants.NO_NONCE);
7542 }
7543
7544 private Scan buildScanForGetWithClosestRowBefore(Get get) throws IOException {
7545 Scan scan = new Scan().withStartRow(get.getRow())
7546 .addFamily(get.getFamilyMap().keySet().iterator().next()).setReversed(true)
7547 .withStopRow(HConstants.EMPTY_END_ROW, false).setLimit(1);
7548 if (this.getRegionInfo().isMetaRegion()) {
7549 int delimiterIdx =
7550 KeyValue.getDelimiter(get.getRow(), 0, get.getRow().length, HConstants.DELIMITER);
7551 if (delimiterIdx >= 0) {
7552 scan.setFilter(new PrefixFilter(Bytes.copy(get.getRow(), 0, delimiterIdx + 1)));
7553 }
7554 }
7555 return scan;
7556 }
7557
7558 @Override
7559 public List<Cell> get(Get get, boolean withCoprocessor, long nonceGroup, long nonce)
7560 throws IOException {
7561 List<Cell> results = new ArrayList<Cell>();
7562
7563
7564 if (withCoprocessor && (coprocessorHost != null)) {
7565 if (coprocessorHost.preGet(get, results)) {
7566 return results;
7567 }
7568 }
7569 long before = EnvironmentEdgeManager.currentTime();
7570 Scan scan;
7571 if (get.isClosestRowBefore()) {
7572 scan = buildScanForGetWithClosestRowBefore(get);
7573 } else {
7574 scan = new Scan(get);
7575 }
7576
7577 if (scan.getLoadColumnFamiliesOnDemandValue() == null) {
7578 scan.setLoadColumnFamiliesOnDemand(isLoadingCfsOnDemandDefault());
7579 }
7580 RegionScanner scanner = null;
7581 try {
7582 scanner = getScanner(scan, null, nonceGroup, nonce);
7583 scanner.next(results);
7584 } finally {
7585 if (scanner != null) {
7586 scanner.close();
7587 }
7588 }
7589
7590
7591 if (withCoprocessor && (coprocessorHost != null)) {
7592 coprocessorHost.postGet(get, results);
7593 }
7594
7595 metricsUpdateForGet(results, before);
7596
7597 return results;
7598 }
7599
7600 void metricsUpdateForGet(List<Cell> results, long before) {
7601 if (this.metricsRegion != null) {
7602 this.metricsRegion.updateGet(EnvironmentEdgeManager.currentTime() - before);
7603 }
7604 if (this.rsServices != null && this.rsServices.getMetrics() != null) {
7605 rsServices.getMetrics().updateReadQueryMeter(getRegionInfo().getTable(), 1);
7606 }
7607 }
7608
7609 @Override
7610 public void mutateRow(RowMutations rm) throws IOException {
7611
7612 mutateRowsWithLocks(rm.getMutations(), Collections.singleton(rm.getRow()));
7613 }
7614
7615
7616
7617
7618
7619 public void mutateRowsWithLocks(Collection<Mutation> mutations,
7620 Collection<byte[]> rowsToLock) throws IOException {
7621 mutateRowsWithLocks(mutations, rowsToLock, HConstants.NO_NONCE, HConstants.NO_NONCE);
7622 }
7623
7624
7625
7626
7627
7628
7629
7630
7631
7632
7633
7634
7635
7636 @Override
7637 public void mutateRowsWithLocks(Collection<Mutation> mutations,
7638 Collection<byte[]> rowsToLock, long nonceGroup, long nonce) throws IOException {
7639 MultiRowMutationProcessor proc = new MultiRowMutationProcessor(mutations, rowsToLock);
7640 processRowsWithLocks(proc, -1, nonceGroup, nonce);
7641 }
7642
7643
7644
7645
7646 public ClientProtos.RegionLoadStats getLoadStatistics() {
7647 if (!regionStatsEnabled) {
7648 return null;
7649 }
7650 ClientProtos.RegionLoadStats.Builder stats = ClientProtos.RegionLoadStats.newBuilder();
7651 stats.setMemstoreLoad((int) (Math.min(100, (this.memstoreSize.get() * 100) / this
7652 .memstoreFlushSize)));
7653 if (rsServices.getHeapMemoryManager() != null) {
7654
7655
7656
7657
7658 final float occupancy = rsServices.getHeapMemoryManager().getHeapOccupancyPercent();
7659 if (occupancy != HeapMemoryManager.HEAP_OCCUPANCY_ERROR_VALUE) {
7660 stats.setHeapOccupancy((int)(occupancy * 100));
7661 }
7662 }
7663 stats.setCompactionPressure((int) (rsServices.getCompactionPressure() * 100 > 100 ? 100
7664 : rsServices.getCompactionPressure() * 100));
7665 return stats.build();
7666 }
7667
7668 @Override
7669 public void processRowsWithLocks(RowProcessor<?,?> processor) throws IOException {
7670 processRowsWithLocks(processor, rowProcessorTimeout, HConstants.NO_NONCE,
7671 HConstants.NO_NONCE);
7672 }
7673
7674 @Override
7675 public void processRowsWithLocks(RowProcessor<?,?> processor, long nonceGroup, long nonce)
7676 throws IOException {
7677 processRowsWithLocks(processor, rowProcessorTimeout, nonceGroup, nonce);
7678 }
7679
7680 @Override
7681 public void processRowsWithLocks(RowProcessor<?,?> processor, long timeout,
7682 long nonceGroup, long nonce) throws IOException {
7683
7684 for (byte[] row : processor.getRowsToLock()) {
7685 checkRow(row, "processRowsWithLocks");
7686 }
7687 if (!processor.readOnly()) {
7688 checkReadOnly();
7689 }
7690 checkResources();
7691
7692 startRegionOperation();
7693 WALEdit walEdit = new WALEdit();
7694
7695
7696 try {
7697 processor.preProcess(this, walEdit);
7698 } catch (IOException e) {
7699 closeRegionOperation();
7700 throw e;
7701 }
7702
7703 if (processor.readOnly()) {
7704 try {
7705 long now = EnvironmentEdgeManager.currentTime();
7706 doProcessRowWithTimeout(
7707 processor, now, this, null, null, timeout);
7708 processor.postProcess(this, walEdit, true);
7709 } finally {
7710 closeRegionOperation();
7711 }
7712 return;
7713 }
7714
7715 MultiVersionConcurrencyControl.WriteEntry writeEntry = null;
7716 boolean locked = false;
7717 boolean walSyncSuccessful = false;
7718 List<RowLock> acquiredRowLocks = null;
7719 long addedSize = 0;
7720 List<Mutation> mutations = new ArrayList<Mutation>();
7721 Collection<byte[]> rowsToLock = processor.getRowsToLock();
7722 long mvccNum = 0;
7723 WALKey walKey = null;
7724 try {
7725 try {
7726
7727 acquiredRowLocks = new ArrayList<RowLock>(rowsToLock.size());
7728 for (byte[] row : rowsToLock) {
7729
7730
7731 acquiredRowLocks.add(getRowLockInternal(row));
7732 }
7733
7734
7735
7736
7737
7738
7739 checkInterrupt();
7740
7741
7742 lock(this.updatesLock.readLock(), acquiredRowLocks.isEmpty() ? 1 : acquiredRowLocks.size());
7743 locked = true;
7744
7745
7746 disableInterrupts();
7747
7748 long now = EnvironmentEdgeManager.currentTime();
7749
7750
7751 doProcessRowWithTimeout(
7752 processor, now, this, mutations, walEdit, timeout);
7753
7754 if (!mutations.isEmpty()) {
7755 writeRequestsCount.add(mutations.size());
7756
7757 processor.preBatchMutate(this, walEdit);
7758
7759 long txid = 0;
7760
7761 if (!walEdit.isEmpty()) {
7762
7763 walKey = new HLogKey(this.getRegionInfo().getEncodedNameAsBytes(),
7764 this.htableDescriptor.getTableName(), WALKey.NO_SEQUENCE_ID, now,
7765 processor.getClusterIds(), nonceGroup, nonce, mvcc);
7766 preWALAppend(walKey, walEdit);
7767 txid = this.wal.append(this.htableDescriptor, this.getRegionInfo(),
7768 walKey, walEdit, true);
7769 }
7770 if(walKey == null){
7771
7772
7773 walKey = this.appendEmptyEdit(this.wal);
7774 }
7775
7776
7777 writeEntry = walKey.getWriteEntry();
7778 mvccNum = walKey.getSequenceId();
7779
7780
7781 for (Mutation m : mutations) {
7782
7783 rewriteCellTags(m.getFamilyCellMap(), m);
7784
7785 for (CellScanner cellScanner = m.cellScanner(); cellScanner.advance();) {
7786 Cell cell = cellScanner.current();
7787 CellUtil.setSequenceId(cell, mvccNum);
7788 Store store = getStore(cell);
7789 if (store == null) {
7790 checkFamily(CellUtil.cloneFamily(cell));
7791
7792 }
7793 addedSize += store.add(cell);
7794 }
7795 }
7796
7797
7798 if (locked) {
7799 this.updatesLock.readLock().unlock();
7800 locked = false;
7801 }
7802
7803
7804 releaseRowLocks(acquiredRowLocks);
7805
7806
7807 if (txid != 0) {
7808 syncOrDefer(txid, getEffectiveDurability(processor.useDurability()));
7809 }
7810
7811 if (rsServices != null && rsServices.getMetrics() != null) {
7812 rsServices.getMetrics().updateWriteQueryMeter(this.htableDescriptor.
7813 getTableName(), mutations.size());
7814 }
7815 walSyncSuccessful = true;
7816
7817 processor.postBatchMutate(this);
7818 }
7819 } finally {
7820
7821
7822
7823 if (!mutations.isEmpty() && !walSyncSuccessful) {
7824 LOG.warn("Wal sync failed. Roll back " + mutations.size() +
7825 " memstore keyvalues" + (processor.getRowsToLock().isEmpty() ? "" :
7826 (" for row(s):" + StringUtils.byteToHexString(
7827 processor.getRowsToLock().iterator().next()) + "...")));
7828 for (Mutation m : mutations) {
7829 for (CellScanner cellScanner = m.cellScanner(); cellScanner.advance();) {
7830 Cell cell = cellScanner.current();
7831 getStore(cell).rollback(cell);
7832 }
7833 }
7834 if (writeEntry != null) {
7835 mvcc.complete(writeEntry);
7836 writeEntry = null;
7837 }
7838 }
7839
7840 if (writeEntry != null) {
7841 mvcc.completeAndWait(writeEntry);
7842 }
7843 if (locked) {
7844 this.updatesLock.readLock().unlock();
7845 }
7846
7847 releaseRowLocks(acquiredRowLocks);
7848
7849 enableInterrupts();
7850 }
7851
7852
7853 processor.postProcess(this, walEdit, walSyncSuccessful);
7854
7855 } finally {
7856 closeRegionOperation();
7857 if (!mutations.isEmpty() && walSyncSuccessful &&
7858 isFlushSize(this.addAndGetGlobalMemstoreSize(addedSize))) {
7859 requestFlush();
7860 }
7861 }
7862 }
7863
7864 private void doProcessRowWithTimeout(final RowProcessor<?,?> processor,
7865 final long now,
7866 final HRegion region,
7867 final List<Mutation> mutations,
7868 final WALEdit walEdit,
7869 final long timeout) throws IOException {
7870
7871 if (timeout < 0) {
7872 try {
7873 processor.process(now, region, mutations, walEdit);
7874 } catch (IOException e) {
7875 String row = processor.getRowsToLock().isEmpty() ? "" :
7876 " on row(s):" + Bytes.toStringBinary(processor.getRowsToLock().iterator().next()) + "...";
7877 LOG.warn("RowProcessor:" + processor.getClass().getName() +
7878 " throws Exception" + row, e);
7879 throw e;
7880 }
7881 return;
7882 }
7883
7884
7885 FutureTask<Void> task =
7886 new FutureTask<Void>(new Callable<Void>() {
7887 @Override
7888 public Void call() throws IOException {
7889 try {
7890 processor.process(now, region, mutations, walEdit);
7891 return null;
7892 } catch (IOException e) {
7893 String row = processor.getRowsToLock().isEmpty() ? "" :
7894 " on row(s):" + Bytes.toStringBinary(processor.getRowsToLock().iterator().next()) + "...";
7895 LOG.warn("RowProcessor:" + processor.getClass().getName() +
7896 " throws Exception" + row, e);
7897 throw e;
7898 }
7899 }
7900 });
7901 rowProcessorExecutor.execute(task);
7902 try {
7903 task.get(timeout, TimeUnit.MILLISECONDS);
7904 } catch (InterruptedException ie) {
7905 throwOnInterrupt(ie);
7906 } catch (TimeoutException te) {
7907 String row = processor.getRowsToLock().isEmpty() ? "" :
7908 " on row(s):" + Bytes.toStringBinary(processor.getRowsToLock().iterator().next()) + "...";
7909 LOG.error("RowProcessor timeout:" + timeout + " ms" + row);
7910 throw new IOException(te);
7911 } catch (Exception e) {
7912 throw new IOException(e);
7913 }
7914 }
7915
7916
7917
7918
7919
7920
7921
7922
7923
7924
7925 private List<Cell> doGet(final Store store, final byte [] row,
7926 final Map.Entry<byte[], List<Cell>> family, final TimeRange tr)
7927 throws IOException {
7928
7929
7930
7931
7932 Collections.sort(family.getValue(), store.getComparator());
7933
7934 Get get = new Get(row);
7935 for (Cell cell : family.getValue()) {
7936 get.addColumn(family.getKey(), CellUtil.cloneQualifier(cell));
7937 }
7938 if (tr != null) get.setTimeRange(tr.getMin(), tr.getMax());
7939 return get(get, false);
7940 }
7941
7942 public Result append(Append append) throws IOException {
7943 return append(append, HConstants.NO_NONCE, HConstants.NO_NONCE);
7944 }
7945
7946 @Override
7947 public Result append(Append mutate, long nonceGroup, long nonce) throws IOException {
7948 Operation op = Operation.APPEND;
7949 byte[] row = mutate.getRow();
7950 checkRow(row, op.toString());
7951 checkFamilies(mutate.getFamilyCellMap().keySet());
7952 Durability durability = getEffectiveDurability(mutate.getDurability());
7953 boolean writeToWAL = durability != Durability.SKIP_WAL;
7954 WALEdit walEdits = null;
7955 List<Cell> allKVs = new ArrayList<Cell>(mutate.size());
7956 Map<Store, List<Cell>> tempMemstore = new LinkedHashMap<Store, List<Cell>>();
7957 Map<Store, List<Cell>> removedCellsForMemStore = new HashMap<>();
7958 long size = 0;
7959 long txid = 0;
7960 checkReadOnly();
7961 checkResources();
7962
7963 startRegionOperation(op);
7964 this.writeRequestsCount.increment();
7965 RowLock rowLock = null;
7966 WALKey walKey = null;
7967 boolean doRollBackMemstore = false;
7968 try {
7969 rowLock = getRowLockInternal(row);
7970 assert rowLock != null;
7971
7972 disableInterrupts();
7973 try {
7974 lock(this.updatesLock.readLock());
7975 try {
7976
7977
7978 mvcc.await();
7979 if (this.coprocessorHost != null) {
7980 Result r = this.coprocessorHost.preAppendAfterRowLock(mutate);
7981 if (r!= null) {
7982 return r;
7983 }
7984 }
7985 long now = EnvironmentEdgeManager.currentTime();
7986
7987 for (Map.Entry<byte[], List<Cell>> family : mutate.getFamilyCellMap().entrySet()) {
7988 Store store = stores.get(family.getKey());
7989 List<Cell> kvs = new ArrayList<Cell>(family.getValue().size());
7990
7991 List<Cell> results = doGet(store, row, family, null);
7992
7993
7994
7995
7996
7997
7998
7999 int idx = 0;
8000 for (Cell cell : family.getValue()) {
8001 Cell newCell;
8002 Cell oldCell = null;
8003 if (idx < results.size()
8004 && CellUtil.matchingQualifier(results.get(idx), cell)) {
8005 oldCell = results.get(idx);
8006 long ts = Math.max(now, oldCell.getTimestamp() + 1);
8007
8008
8009
8010 List<Tag> tags = Tag.carryForwardTags(null, oldCell);
8011 tags = Tag.carryForwardTags(tags, cell);
8012 tags = carryForwardTTLTag(tags, mutate);
8013
8014 newCell = getNewCell(row, ts, cell, oldCell, Tag.fromList(tags));
8015 int newCellSize = CellUtil.estimatedSerializedSizeOf(newCell);
8016 if (newCellSize > this.maxCellSize) {
8017 String msg = "Cell with size " + newCellSize + " exceeds limit of " +
8018 this.maxCellSize + " bytes";
8019 if (LOG.isDebugEnabled()) {
8020 LOG.debug(msg);
8021 }
8022 throw new DoNotRetryIOException(msg);
8023 }
8024 idx++;
8025 } else {
8026
8027 CellUtil.updateLatestStamp(cell, now);
8028
8029
8030 newCell = getNewCell(mutate, cell);
8031 }
8032
8033
8034 if (coprocessorHost != null) {
8035 newCell = coprocessorHost.postMutationBeforeWAL(RegionObserver.MutationType.APPEND,
8036 mutate, oldCell, newCell);
8037 }
8038 kvs.add(newCell);
8039
8040
8041 if (writeToWAL) {
8042 if (walEdits == null) {
8043 walEdits = new WALEdit();
8044 }
8045 walEdits.add(newCell);
8046 }
8047 }
8048
8049
8050 tempMemstore.put(store, kvs);
8051 }
8052
8053
8054 if (walEdits != null && !walEdits.isEmpty()) {
8055 if (writeToWAL) {
8056
8057
8058
8059
8060 walKey = new HLogKey(
8061 getRegionInfo().getEncodedNameAsBytes(),
8062 this.htableDescriptor.getTableName(),
8063 WALKey.NO_SEQUENCE_ID,
8064 nonceGroup,
8065 nonce,
8066 mvcc);
8067 preWALAppend(walKey, walEdits);
8068 txid =
8069 this.wal.append(this.htableDescriptor, getRegionInfo(), walKey, walEdits, true);
8070 } else {
8071 recordMutationWithoutWal(mutate.getFamilyCellMap());
8072 }
8073 }
8074 boolean updateSeqId = false;
8075 if (walKey == null) {
8076
8077 walKey = this.appendEmptyEdit(this.wal);
8078
8079 updateSeqId = true;
8080 }
8081
8082
8083 WriteEntry writeEntry = walKey.getWriteEntry();
8084
8085 if (rsServices != null && rsServices.getNonceManager() != null) {
8086 rsServices.getNonceManager().addMvccToOperationContext(nonceGroup, nonce,
8087 writeEntry.getWriteNumber());
8088 }
8089
8090 if (updateSeqId) {
8091 updateSequenceId(tempMemstore.values(), writeEntry.getWriteNumber());
8092 }
8093
8094
8095 doRollBackMemstore = !tempMemstore.isEmpty();
8096 for (Map.Entry<Store, List<Cell>> entry : tempMemstore.entrySet()) {
8097 Store store = entry.getKey();
8098 if (store.getFamily().getMaxVersions() == 1) {
8099 List<Cell> removedCells = removedCellsForMemStore.get(store);
8100 if (removedCells == null) {
8101 removedCells = new ArrayList<>();
8102 removedCellsForMemStore.put(store, removedCells);
8103 }
8104
8105 size += store.upsert(entry.getValue(), getSmallestReadPoint(), removedCells);
8106 } else {
8107
8108 size += store.add(entry.getValue());
8109 }
8110
8111
8112 allKVs.addAll(entry.getValue());
8113 }
8114 } finally {
8115 this.updatesLock.readLock().unlock();
8116 }
8117
8118 } finally {
8119 rowLock.release();
8120 rowLock = null;
8121 }
8122
8123 if(txid != 0){
8124 syncOrDefer(txid, durability);
8125 }
8126 if (rsServices != null && rsServices.getMetrics() != null) {
8127 rsServices.getMetrics().updateWriteQueryMeter(this.htableDescriptor.
8128 getTableName());
8129 }
8130 doRollBackMemstore = false;
8131 } finally {
8132 if (rowLock != null) {
8133 rowLock.release();
8134 }
8135
8136 WriteEntry we = walKey != null? walKey.getWriteEntry(): null;
8137 if (doRollBackMemstore) {
8138 for (Map.Entry<Store, List<Cell>> entry: tempMemstore.entrySet()) {
8139 rollbackMemstore(entry.getKey(), entry.getValue());
8140 }
8141 for (Map.Entry<Store, List<Cell>> entry: removedCellsForMemStore.entrySet()) {
8142 entry.getKey().add(entry.getValue());
8143 }
8144 if (we != null) {
8145 mvcc.complete(we);
8146 }
8147 } else if (we != null) {
8148 mvcc.completeAndWait(we);
8149 }
8150
8151 enableInterrupts();
8152
8153 closeRegionOperation(op);
8154 }
8155
8156 if (this.metricsRegion != null) {
8157 this.metricsRegion.updateAppend();
8158 }
8159 if (isFlushSize(this.addAndGetGlobalMemstoreSize(size))) requestFlush();
8160 return mutate.isReturnResults() ? Result.create(allKVs) : null;
8161 }
8162
8163 private void preWALAppend(WALKey walKey, WALEdit walEdits) throws IOException {
8164 if (this.coprocessorHost != null && !walEdits.isMetaEdit()) {
8165 this.coprocessorHost.preWALAppend(walKey, walEdits);
8166 }
8167 }
8168
8169 private static Cell getNewCell(final byte [] row, final long ts, final Cell cell,
8170 final Cell oldCell, final byte [] tagBytes) {
8171
8172 Cell newCell = new KeyValue(row.length, cell.getFamilyLength(),
8173 cell.getQualifierLength(), ts, KeyValue.Type.Put,
8174 oldCell.getValueLength() + cell.getValueLength(),
8175 tagBytes == null? 0: tagBytes.length);
8176
8177 System.arraycopy(cell.getRowArray(), cell.getRowOffset(),
8178 newCell.getRowArray(), newCell.getRowOffset(), cell.getRowLength());
8179 System.arraycopy(cell.getFamilyArray(), cell.getFamilyOffset(),
8180 newCell.getFamilyArray(), newCell.getFamilyOffset(),
8181 cell.getFamilyLength());
8182 System.arraycopy(cell.getQualifierArray(), cell.getQualifierOffset(),
8183 newCell.getQualifierArray(), newCell.getQualifierOffset(),
8184 cell.getQualifierLength());
8185
8186 System.arraycopy(oldCell.getValueArray(), oldCell.getValueOffset(),
8187 newCell.getValueArray(), newCell.getValueOffset(),
8188 oldCell.getValueLength());
8189 System.arraycopy(cell.getValueArray(), cell.getValueOffset(),
8190 newCell.getValueArray(),
8191 newCell.getValueOffset() + oldCell.getValueLength(),
8192 cell.getValueLength());
8193
8194 if (tagBytes != null) {
8195 System.arraycopy(tagBytes, 0, newCell.getTagsArray(), newCell.getTagsOffset(),
8196 tagBytes.length);
8197 }
8198 return newCell;
8199 }
8200
8201 private static Cell getNewCell(final Mutation mutate, final Cell cell) {
8202 Cell newCell = null;
8203 if (mutate.getTTL() != Long.MAX_VALUE) {
8204
8205 newCell = new KeyValue(cell.getRowArray(), cell.getRowOffset(),
8206 cell.getRowLength(),
8207 cell.getFamilyArray(), cell.getFamilyOffset(),
8208 cell.getFamilyLength(),
8209 cell.getQualifierArray(), cell.getQualifierOffset(),
8210 cell.getQualifierLength(),
8211 cell.getTimestamp(), KeyValue.Type.codeToType(cell.getTypeByte()),
8212 cell.getValueArray(), cell.getValueOffset(), cell.getValueLength(),
8213 carryForwardTTLTag(mutate));
8214 } else {
8215 newCell = cell;
8216 }
8217 return newCell;
8218 }
8219
8220 public Result increment(Increment increment) throws IOException {
8221 return increment(increment, HConstants.NO_NONCE, HConstants.NO_NONCE);
8222 }
8223
8224
8225
8226
8227
8228
8229
8230
8231
8232 @Override
8233 public Result increment(Increment mutation, long nonceGroup, long nonce)
8234 throws IOException {
8235 Operation op = Operation.INCREMENT;
8236 checkReadOnly();
8237 checkResources();
8238 checkRow(mutation.getRow(), op.toString());
8239 checkFamilies(mutation.getFamilyCellMap().keySet());
8240 startRegionOperation(op);
8241 this.writeRequestsCount.increment();
8242 try {
8243
8244
8245
8246
8247
8248
8249
8250
8251
8252
8253
8254
8255
8256
8257
8258
8259 return doIncrement(mutation, nonceGroup, nonce);
8260 } finally {
8261 if (rsServices != null && rsServices.getMetrics() != null) {
8262 rsServices.getMetrics().updateWriteQueryMeter(this.htableDescriptor.
8263 getTableName());
8264 }
8265 if (this.metricsRegion != null) this.metricsRegion.updateIncrement();
8266 closeRegionOperation(op);
8267 }
8268 }
8269
8270 private Result doIncrement(Increment increment, long nonceGroup, long nonce) throws IOException {
8271 RowLock rowLock = null;
8272 WALKey walKey = null;
8273 boolean doRollBackMemstore = false;
8274 long accumulatedResultSize = 0;
8275 List<Cell> allKVs = new ArrayList<Cell>(increment.size());
8276 Map<Store, List<Cell>> removedCellsForMemStore = new HashMap<>();
8277 Map<Store, List<Cell>> forMemStore = new HashMap<>();
8278 Durability effectiveDurability = getEffectiveDurability(increment.getDurability());
8279 try {
8280 rowLock = getRowLockInternal(increment.getRow());
8281
8282 disableInterrupts();
8283 long txid = 0;
8284 try {
8285 lock(this.updatesLock.readLock());
8286 try {
8287
8288
8289 this.mvcc.await();
8290 if (this.coprocessorHost != null) {
8291 Result r = this.coprocessorHost.preIncrementAfterRowLock(increment);
8292 if (r != null) return r;
8293 }
8294 long now = EnvironmentEdgeManager.currentTime();
8295 final boolean writeToWAL = effectiveDurability != Durability.SKIP_WAL;
8296 WALEdit walEdits = null;
8297
8298
8299 for (Map.Entry<byte [], List<Cell>> entry: increment.getFamilyCellMap().entrySet()) {
8300 byte [] columnFamilyName = entry.getKey();
8301 List<Cell> increments = entry.getValue();
8302 Store store = this.stores.get(columnFamilyName);
8303
8304
8305 List<Cell> results = applyIncrementsToColumnFamily(increment, columnFamilyName,
8306 sort(increments, store.getComparator()), now,
8307 MultiVersionConcurrencyControl.NO_WRITE_NUMBER, allKVs, null);
8308 if (!results.isEmpty()) {
8309 forMemStore.put(store, results);
8310
8311 if (writeToWAL) {
8312 if (walEdits == null) walEdits = new WALEdit();
8313 walEdits.getCells().addAll(results);
8314 }
8315 }
8316 }
8317 boolean updateSeqId = false;
8318
8319
8320
8321
8322 checkInterrupt();
8323
8324
8325 if (walEdits != null && !walEdits.isEmpty()) {
8326
8327
8328
8329 walKey = new HLogKey(this.getRegionInfo().getEncodedNameAsBytes(),
8330 this.htableDescriptor.getTableName(), WALKey.NO_SEQUENCE_ID, nonceGroup, nonce,
8331 getMVCC());
8332 preWALAppend(walKey, walEdits);
8333 txid =
8334 this.wal.append(this.htableDescriptor, this.getRegionInfo(), walKey, walEdits, true);
8335 } else {
8336
8337 walKey = this.appendEmptyEdit(this.wal);
8338
8339 updateSeqId = true;
8340 }
8341
8342 WriteEntry writeEntry = walKey.getWriteEntry();
8343
8344 if (rsServices != null && rsServices.getNonceManager() != null) {
8345 rsServices.getNonceManager().addMvccToOperationContext(nonceGroup, nonce,
8346 writeEntry.getWriteNumber());
8347 }
8348
8349 if (updateSeqId) {
8350 updateSequenceId(forMemStore.values(), writeEntry.getWriteNumber());
8351 }
8352
8353
8354 doRollBackMemstore = !forMemStore.isEmpty();
8355 for (Map.Entry<Store, List<Cell>> entry: forMemStore.entrySet()) {
8356 Store store = entry.getKey();
8357 List<Cell> results = entry.getValue();
8358 if (store.getFamily().getMaxVersions() == 1) {
8359 List<Cell> removedCells = removedCellsForMemStore.get(store);
8360 if (removedCells == null) {
8361 removedCells = new ArrayList<>();
8362 removedCellsForMemStore.put(store, removedCells);
8363 }
8364
8365 accumulatedResultSize += store.upsert(results, getSmallestReadPoint(), removedCells);
8366 } else {
8367
8368 accumulatedResultSize += store.add(entry.getValue());
8369 }
8370 }
8371 } finally {
8372 this.updatesLock.readLock().unlock();
8373 }
8374 } finally {
8375 rowLock.release();
8376 rowLock = null;
8377 }
8378
8379 if(txid != 0) {
8380 syncOrDefer(txid, effectiveDurability);
8381 }
8382 doRollBackMemstore = false;
8383 } finally {
8384 if (rowLock != null) {
8385 rowLock.release();
8386 }
8387
8388 WriteEntry we = walKey != null ? walKey.getWriteEntry() : null;
8389 if (doRollBackMemstore) {
8390 for (Map.Entry<Store, List<Cell>> entry: forMemStore.entrySet()) {
8391 rollbackMemstore(entry.getKey(), entry.getValue());
8392 }
8393 for (Map.Entry<Store, List<Cell>> entry: removedCellsForMemStore.entrySet()) {
8394 entry.getKey().add(entry.getValue());
8395 }
8396 if (we != null) {
8397 mvcc.complete(we);
8398 }
8399 } else {
8400 if (we != null) {
8401 mvcc.completeAndWait(we);
8402 }
8403 }
8404 enableInterrupts();
8405 }
8406
8407
8408 if (isFlushSize(this.addAndGetGlobalMemstoreSize(accumulatedResultSize))) requestFlush();
8409 return increment.isReturnResults() ? Result.create(allKVs) : null;
8410 }
8411
8412
8413
8414
8415 private static List<Cell> sort(List<Cell> cells, final Comparator<Cell> comparator) {
8416 Collections.sort(cells, comparator);
8417 return cells;
8418 }
8419
8420
8421
8422
8423
8424
8425
8426
8427
8428
8429
8430
8431 private List<Cell> applyIncrementsToColumnFamily(Increment increment, byte[] columnFamilyName,
8432 List<Cell> sortedIncrements, long now, long mvccNum, List<Cell> allKVs,
8433 final IsolationLevel isolation)
8434 throws IOException {
8435 List<Cell> results = new ArrayList<Cell>(sortedIncrements.size());
8436 byte [] row = increment.getRow();
8437
8438 List<Cell> currentValues =
8439 getIncrementCurrentValue(increment, columnFamilyName, sortedIncrements, isolation);
8440
8441
8442 int idx = 0;
8443 for (int i = 0; i < sortedIncrements.size(); i++) {
8444 Cell inc = sortedIncrements.get(i);
8445 long incrementAmount = getLongValue(inc);
8446
8447 boolean writeBack = (incrementAmount != 0);
8448
8449 List<Tag> tags = Tag.carryForwardTags(inc);
8450
8451 Cell currentValue = null;
8452 long ts = now;
8453 boolean firstWrite = false;
8454 if (idx < currentValues.size() && CellUtil.matchingQualifier(currentValues.get(idx), inc)) {
8455 currentValue = currentValues.get(idx);
8456 ts = Math.max(now, currentValue.getTimestamp() + 1);
8457 incrementAmount += getLongValue(currentValue);
8458
8459 tags = Tag.carryForwardTags(tags, currentValue);
8460 if (i < (sortedIncrements.size() - 1) &&
8461 !CellUtil.matchingQualifier(inc, sortedIncrements.get(i + 1))) idx++;
8462 } else {
8463 firstWrite = true;
8464 }
8465
8466
8467 byte [] incrementAmountInBytes = Bytes.toBytes(incrementAmount);
8468 tags = carryForwardTTLTag(tags, increment);
8469
8470 Cell newValue = new KeyValue(row, 0, row.length,
8471 columnFamilyName, 0, columnFamilyName.length,
8472 inc.getQualifierArray(), inc.getQualifierOffset(), inc.getQualifierLength(),
8473 ts, KeyValue.Type.Put,
8474 incrementAmountInBytes, 0, incrementAmountInBytes.length,
8475 tags);
8476
8477
8478
8479 if (mvccNum != MultiVersionConcurrencyControl.NO_WRITE_NUMBER) {
8480 CellUtil.setSequenceId(newValue, mvccNum);
8481 }
8482
8483
8484 if (coprocessorHost != null) {
8485 newValue = coprocessorHost.postMutationBeforeWAL(
8486 RegionObserver.MutationType.INCREMENT, increment, currentValue, newValue);
8487 }
8488 allKVs.add(newValue);
8489 if (writeBack || firstWrite) {
8490 results.add(newValue);
8491 }
8492 }
8493 return results;
8494 }
8495
8496
8497
8498
8499
8500 private static long getLongValue(final Cell cell) throws DoNotRetryIOException {
8501 int len = cell.getValueLength();
8502 if (len != Bytes.SIZEOF_LONG) {
8503
8504 throw new DoNotRetryIOException("Field is not a long, it's " + len + " bytes wide");
8505 }
8506 return Bytes.toLong(cell.getValueArray(), cell.getValueOffset(), len);
8507 }
8508
8509
8510
8511
8512
8513
8514
8515
8516
8517 private List<Cell> getIncrementCurrentValue(final Increment increment, byte [] columnFamily,
8518 final List<Cell> increments, final IsolationLevel isolation)
8519 throws IOException {
8520 Get get = new Get(increment.getRow());
8521 if (isolation != null) get.setIsolationLevel(isolation);
8522 for (Cell cell: increments) {
8523 get.addColumn(columnFamily, CellUtil.cloneQualifier(cell));
8524 }
8525 TimeRange tr = increment.getTimeRange();
8526 if (tr != null) {
8527 get.setTimeRange(tr.getMin(), tr.getMax());
8528 }
8529 return get(get, false);
8530 }
8531
8532 private static List<Tag> carryForwardTTLTag(final Mutation mutation) {
8533 return carryForwardTTLTag(null, mutation);
8534 }
8535
8536
8537
8538
8539 private static List<Tag> carryForwardTTLTag(final List<Tag> tagsOrNull,
8540 final Mutation mutation) {
8541 long ttl = mutation.getTTL();
8542 if (ttl == Long.MAX_VALUE) return tagsOrNull;
8543 List<Tag> tags = tagsOrNull;
8544
8545
8546
8547 if (tags == null) {
8548 tags = new ArrayList<Tag>(1);
8549 } else {
8550
8551 Iterator<Tag> tagsItr = tags.iterator();
8552 while (tagsItr.hasNext()) {
8553 Tag tag = tagsItr.next();
8554 if (tag.getType() == TagType.TTL_TAG_TYPE) {
8555 tagsItr.remove();
8556 break;
8557 }
8558 }
8559 }
8560 tags.add(new Tag(TagType.TTL_TAG_TYPE, Bytes.toBytes(ttl)));
8561 return tags;
8562 }
8563
8564
8565
8566
8567
8568 private void checkFamily(final byte [] family)
8569 throws NoSuchColumnFamilyException {
8570 if (!this.htableDescriptor.hasFamily(family)) {
8571 throw new NoSuchColumnFamilyException("Column family " +
8572 Bytes.toString(family) + " does not exist in region " + this
8573 + " in table " + this.htableDescriptor);
8574 }
8575 }
8576
8577 public static final long FIXED_OVERHEAD = ClassSize.align(
8578 ClassSize.OBJECT +
8579 ClassSize.ARRAY +
8580 50 * ClassSize.REFERENCE + 3 * Bytes.SIZEOF_INT +
8581 (15 * Bytes.SIZEOF_LONG) +
8582 5 * Bytes.SIZEOF_BOOLEAN);
8583
8584
8585
8586
8587
8588
8589
8590
8591
8592
8593
8594 public static final long DEEP_OVERHEAD = FIXED_OVERHEAD +
8595 ClassSize.OBJECT +
8596 (2 * ClassSize.ATOMIC_BOOLEAN) +
8597 (4 * ClassSize.ATOMIC_LONG) +
8598
8599 (3 * ClassSize.CONCURRENT_HASHMAP) +
8600 WriteState.HEAP_SIZE +
8601 ClassSize.CONCURRENT_SKIPLISTMAP + ClassSize.CONCURRENT_SKIPLISTMAP_ENTRY +
8602 (2 * ClassSize.REENTRANT_LOCK) +
8603 MultiVersionConcurrencyControl.FIXED_SIZE
8604 + ClassSize.TREEMAP
8605 + 2 * ClassSize.ATOMIC_INTEGER
8606 ;
8607
8608 @Override
8609 public long heapSize() {
8610 long heapSize = DEEP_OVERHEAD;
8611 for (Store store : this.stores.values()) {
8612 heapSize += store.heapSize();
8613 }
8614
8615 return heapSize;
8616 }
8617
8618
8619
8620
8621
8622 private static void printUsageAndExit(final String message) {
8623 if (message != null && message.length() > 0) System.out.println(message);
8624 System.out.println("Usage: HRegion CATALOG_TABLE_DIR [major_compact]");
8625 System.out.println("Options:");
8626 System.out.println(" major_compact Pass this option to major compact " +
8627 "passed region.");
8628 System.out.println("Default outputs scan of passed region.");
8629 System.exit(1);
8630 }
8631
8632 @Override
8633 public boolean registerService(Service instance) {
8634
8635
8636
8637 Descriptors.ServiceDescriptor serviceDesc = instance.getDescriptorForType();
8638 String serviceName = CoprocessorRpcUtils.getServiceName(serviceDesc);
8639 if (coprocessorServiceHandlers.containsKey(serviceName)) {
8640 LOG.error("Coprocessor service " + serviceName +
8641 " already registered, rejecting request from " + instance
8642 );
8643 return false;
8644 }
8645
8646 coprocessorServiceHandlers.put(serviceName, instance);
8647 if (LOG.isDebugEnabled()) {
8648 LOG.debug("Registered coprocessor service: region=" +
8649 Bytes.toStringBinary(getRegionInfo().getRegionName()) +
8650 " service=" + serviceName);
8651 }
8652 return true;
8653 }
8654
8655 @Override
8656 public Message execService(RpcController controller, CoprocessorServiceCall call)
8657 throws IOException {
8658 String serviceName = call.getServiceName();
8659 String methodName = call.getMethodName();
8660 if (!coprocessorServiceHandlers.containsKey(serviceName)) {
8661 throw new UnknownProtocolException(null,
8662 "No registered coprocessor service found for name "+serviceName+
8663 " in region "+Bytes.toStringBinary(getRegionInfo().getRegionName()));
8664 }
8665
8666 Service service = coprocessorServiceHandlers.get(serviceName);
8667 Descriptors.ServiceDescriptor serviceDesc = service.getDescriptorForType();
8668 Descriptors.MethodDescriptor methodDesc = serviceDesc.findMethodByName(methodName);
8669 if (methodDesc == null) {
8670 throw new UnknownProtocolException(service.getClass(),
8671 "Unknown method "+methodName+" called on service "+serviceName+
8672 " in region "+Bytes.toStringBinary(getRegionInfo().getRegionName()));
8673 }
8674
8675 Message.Builder builder = service.getRequestPrototype(methodDesc).newBuilderForType();
8676 ProtobufUtil.mergeFrom(builder, call.getRequest());
8677 Message request = builder.build();
8678
8679 if (coprocessorHost != null) {
8680 request = coprocessorHost.preEndpointInvocation(service, methodName, request);
8681 }
8682
8683 final Message.Builder responseBuilder =
8684 service.getResponsePrototype(methodDesc).newBuilderForType();
8685 service.callMethod(methodDesc, controller, request, new RpcCallback<Message>() {
8686 @Override
8687 public void run(Message message) {
8688 if (message != null) {
8689 responseBuilder.mergeFrom(message);
8690 }
8691 }
8692 });
8693
8694 if (coprocessorHost != null) {
8695 coprocessorHost.postEndpointInvocation(service, methodName, request, responseBuilder);
8696 }
8697
8698 IOException exception = ResponseConverter.getControllerException(controller);
8699 if (exception != null) {
8700 throw exception;
8701 }
8702
8703 return responseBuilder.build();
8704 }
8705
8706
8707
8708
8709
8710
8711 private static void processTable(final FileSystem fs, final Path p,
8712 final WALFactory walFactory, final Configuration c,
8713 final boolean majorCompact)
8714 throws IOException {
8715 HRegion region;
8716 FSTableDescriptors fst = new FSTableDescriptors(c);
8717
8718 if (FSUtils.getTableName(p).equals(TableName.META_TABLE_NAME)) {
8719 final WAL wal = walFactory.getMetaWAL(
8720 HRegionInfo.FIRST_META_REGIONINFO.getEncodedNameAsBytes());
8721 region = HRegion.newHRegion(p, wal, fs, c,
8722 HRegionInfo.FIRST_META_REGIONINFO, fst.get(TableName.META_TABLE_NAME), null);
8723 } else {
8724 throw new IOException("Not a known catalog table: " + p.toString());
8725 }
8726 try {
8727 region.mvcc.advanceTo(region.initialize(null));
8728 if (majorCompact) {
8729 region.compact(true);
8730 } else {
8731
8732 Scan scan = new Scan();
8733
8734 RegionScanner scanner = region.getScanner(scan);
8735 try {
8736 List<Cell> kvs = new ArrayList<Cell>();
8737 boolean done;
8738 do {
8739 kvs.clear();
8740 done = scanner.next(kvs);
8741 if (kvs.size() > 0) LOG.info(kvs);
8742 } while (done);
8743 } finally {
8744 scanner.close();
8745 }
8746 }
8747 } finally {
8748 region.close();
8749 }
8750 }
8751
8752 boolean shouldForceSplit() {
8753 return this.splitRequest;
8754 }
8755
8756 byte[] getExplicitSplitPoint() {
8757 return this.explicitSplitPoint;
8758 }
8759
8760 void forceSplit(byte[] sp) {
8761
8762
8763 this.splitRequest = true;
8764 if (sp != null) {
8765 this.explicitSplitPoint = sp;
8766 }
8767 }
8768
8769 void clearSplit() {
8770 this.splitRequest = false;
8771 this.explicitSplitPoint = null;
8772 }
8773
8774
8775
8776
8777 protected void prepareToSplit() {
8778
8779 }
8780
8781
8782
8783
8784
8785
8786
8787 public byte[] checkSplit() {
8788
8789 if (this.getRegionInfo().isMetaTable() ||
8790 TableName.NAMESPACE_TABLE_NAME.equals(this.getRegionInfo().getTable())) {
8791 if (shouldForceSplit()) {
8792 LOG.warn("Cannot split meta region in HBase 0.20 and above");
8793 }
8794 return null;
8795 }
8796
8797
8798 if (this.isRecovering()) {
8799 LOG.info("Cannot split region " + this.getRegionInfo().getEncodedName() + " in recovery.");
8800 return null;
8801 }
8802
8803 if (!splitPolicy.shouldSplit()) {
8804 return null;
8805 }
8806
8807 byte[] ret = splitPolicy.getSplitPoint();
8808
8809 if (ret != null) {
8810 try {
8811 checkRow(ret, "calculated split");
8812 } catch (IOException e) {
8813 LOG.error("Ignoring invalid split", e);
8814 return null;
8815 }
8816 }
8817 return ret;
8818 }
8819
8820
8821
8822
8823 public int getCompactPriority() {
8824 int count = Integer.MAX_VALUE;
8825 for (Store store : stores.values()) {
8826 count = Math.min(count, store.getCompactPriority());
8827 }
8828 return count;
8829 }
8830
8831
8832
8833 @Override
8834 public RegionCoprocessorHost getCoprocessorHost() {
8835 return coprocessorHost;
8836 }
8837
8838
8839 public void setCoprocessorHost(final RegionCoprocessorHost coprocessorHost) {
8840 this.coprocessorHost = coprocessorHost;
8841 }
8842
8843 @Override
8844 public void startRegionOperation() throws IOException {
8845 startRegionOperation(Operation.ANY);
8846 }
8847
8848 @Override
8849 @edu.umd.cs.findbugs.annotations.SuppressWarnings(value="SF_SWITCH_FALLTHROUGH",
8850 justification="Intentional")
8851 public void startRegionOperation(Operation op) throws IOException {
8852 boolean isInterruptableOp = false;
8853 switch (op) {
8854 case GET:
8855 case SCAN:
8856 isInterruptableOp = true;
8857 checkReadsEnabled();
8858 break;
8859 case INCREMENT:
8860 case APPEND:
8861 case PUT:
8862 case DELETE:
8863 case BATCH_MUTATE:
8864 case CHECK_AND_MUTATE:
8865 isInterruptableOp = true;
8866 break;
8867 default:
8868 break;
8869 }
8870
8871 if (isRecovering() && (this.disallowWritesInRecovering ||
8872 (op != Operation.PUT && op != Operation.DELETE && op != Operation.BATCH_MUTATE))) {
8873 throw new RegionInRecoveryException(getRegionInfo().getRegionNameAsString() +
8874 " is recovering; cannot take reads");
8875 }
8876 if (op == Operation.MERGE_REGION || op == Operation.SPLIT_REGION
8877 || op == Operation.COMPACT_REGION || op == Operation.COMPACT_SWITCH) {
8878
8879
8880 return;
8881 }
8882 if (this.closing.get()) {
8883 throw new NotServingRegionException(getRegionInfo().getRegionNameAsString() + " is closing");
8884 }
8885 lock(lock.readLock());
8886
8887
8888 Thread thisThread = Thread.currentThread();
8889 if (isInterruptableOp) {
8890 regionLockHolders.put(thisThread, true);
8891 }
8892 if (this.closed.get()) {
8893 lock.readLock().unlock();
8894 throw new NotServingRegionException(getRegionInfo().getRegionNameAsString() + " is closed");
8895 }
8896
8897
8898 if (op == Operation.SNAPSHOT) {
8899 for (Store store : stores.values()) {
8900 if (store instanceof HStore) {
8901 ((HStore)store).preSnapshotOperation();
8902 }
8903 }
8904 }
8905 try {
8906 if (coprocessorHost != null) {
8907 coprocessorHost.postStartRegionOperation(op);
8908 }
8909 } catch (Exception e) {
8910 if (isInterruptableOp) {
8911
8912
8913 regionLockHolders.remove(thisThread);
8914 }
8915 lock.readLock().unlock();
8916 throw new IOException(e);
8917 }
8918 }
8919
8920 @Override
8921 public void closeRegionOperation() throws IOException {
8922 closeRegionOperation(Operation.ANY);
8923 }
8924
8925 @Override
8926 public void closeRegionOperation(Operation operation) throws IOException {
8927 if (operation == Operation.SNAPSHOT) {
8928 for (Store store: stores.values()) {
8929 if (store instanceof HStore) {
8930 ((HStore)store).postSnapshotOperation();
8931 }
8932 }
8933 }
8934 Thread thisThread = Thread.currentThread();
8935 regionLockHolders.remove(thisThread);
8936 lock.readLock().unlock();
8937 if (coprocessorHost != null) {
8938 coprocessorHost.postCloseRegionOperation(operation);
8939 }
8940 }
8941
8942
8943
8944
8945
8946
8947
8948
8949
8950
8951 private void startBulkRegionOperation(boolean writeLockNeeded) throws IOException {
8952 if (this.closing.get()) {
8953 throw new NotServingRegionException(getRegionInfo().getRegionNameAsString() + " is closing");
8954 }
8955 if (writeLockNeeded) lock(lock.writeLock());
8956 else lock(lock.readLock());
8957 if (this.closed.get()) {
8958 if (writeLockNeeded) lock.writeLock().unlock();
8959 else lock.readLock().unlock();
8960 throw new NotServingRegionException(getRegionInfo().getRegionNameAsString() + " is closed");
8961 }
8962 regionLockHolders.put(Thread.currentThread(), true);
8963 }
8964
8965
8966
8967
8968
8969 private void closeBulkRegionOperation(){
8970 regionLockHolders.remove(Thread.currentThread());
8971 if (lock.writeLock().isHeldByCurrentThread()) lock.writeLock().unlock();
8972 else lock.readLock().unlock();
8973 }
8974
8975
8976
8977
8978
8979 protected void disableInterrupts() {
8980
8981
8982
8983
8984
8985 synchronized (regionLockHolders) {
8986 Thread currentThread = Thread.currentThread();
8987 Boolean value = regionLockHolders.get(currentThread);
8988 if (value != null) {
8989 regionLockHolders.put(currentThread, false);
8990 }
8991 }
8992 }
8993
8994
8995
8996
8997
8998 protected void enableInterrupts() {
8999
9000
9001
9002
9003
9004 synchronized (regionLockHolders) {
9005 Thread currentThread = Thread.currentThread();
9006 Boolean value = regionLockHolders.get(currentThread);
9007 if (value != null) {
9008 regionLockHolders.put(currentThread, true);
9009 }
9010 }
9011 }
9012
9013
9014
9015
9016
9017
9018 private void interruptRegionOperations() {
9019 for (Map.Entry<Thread, Boolean> entry: regionLockHolders.entrySet()) {
9020
9021
9022 if (entry.getValue().booleanValue()) {
9023 entry.getKey().interrupt();
9024 }
9025 }
9026 }
9027
9028
9029
9030
9031
9032 private void recordMutationWithoutWal(final Map<byte [], List<Cell>> familyMap) {
9033 numMutationsWithoutWAL.increment();
9034 if (numMutationsWithoutWAL.get() <= 1) {
9035 LOG.info("writing data to region " + this +
9036 " with WAL disabled. Data may be lost in the event of a crash.");
9037 }
9038
9039 long mutationSize = 0;
9040 for (List<Cell> cells: familyMap.values()) {
9041 assert cells instanceof RandomAccess;
9042 int listSize = cells.size();
9043 for (int i=0; i < listSize; i++) {
9044 Cell cell = cells.get(i);
9045
9046 mutationSize += KeyValueUtil.length(cell);
9047 }
9048 }
9049
9050 dataInMemoryWithoutWAL.add(mutationSize);
9051 }
9052
9053 private void lock(final Lock lock) throws IOException {
9054 lock(lock, 1);
9055 }
9056
9057
9058
9059
9060
9061
9062 private void lock(final Lock lock, final int multiplier) throws IOException {
9063 try {
9064 final long waitTime = Math.min(maxBusyWaitDuration,
9065 busyWaitDuration * Math.min(multiplier, maxBusyWaitMultiplier));
9066 if (!lock.tryLock(waitTime, TimeUnit.MILLISECONDS)) {
9067 final String regionName =
9068 this.getRegionInfo() == null ? "unknown" : this.getRegionInfo().getRegionNameAsString();
9069 final String serverName = this.getRegionServerServices() == null ? "unknown" :
9070 (this.getRegionServerServices().getServerName() == null ? "unknown" :
9071 this.getRegionServerServices().getServerName().toString());
9072 RegionTooBusyException rtbe = new RegionTooBusyException(
9073 "failed to get a lock in " + waitTime + " ms. " + "regionName=" + regionName + ", server="
9074 + serverName);
9075 LOG.warn("Region is too busy to allow lock acquisition.", rtbe);
9076 throw rtbe;
9077 }
9078 } catch (InterruptedException ie) {
9079 if (LOG.isDebugEnabled()) {
9080 LOG.debug("Interrupted while waiting for a lock in region " + this);
9081 }
9082 throw throwOnInterrupt(ie);
9083 }
9084 }
9085
9086
9087
9088
9089
9090
9091
9092 private void syncOrDefer(long txid, Durability durability) throws IOException {
9093 if (this.getRegionInfo().isMetaRegion()) {
9094 this.wal.sync(txid);
9095 } else {
9096 switch(durability) {
9097 case USE_DEFAULT:
9098
9099 if (shouldSyncWAL()) {
9100 this.wal.sync(txid);
9101 }
9102 break;
9103 case SKIP_WAL:
9104
9105 break;
9106 case ASYNC_WAL:
9107
9108 break;
9109 case SYNC_WAL:
9110 this.wal.sync(txid, false);
9111 break;
9112 case FSYNC_WAL:
9113
9114 this.wal.sync(txid, true);
9115 break;
9116 default:
9117 throw new RuntimeException("Unknown durability " + durability);
9118 }
9119 }
9120 }
9121
9122
9123
9124
9125 private boolean shouldSyncWAL() {
9126 return durability.ordinal() > Durability.ASYNC_WAL.ordinal();
9127 }
9128
9129
9130
9131
9132 private static final List<Cell> MOCKED_LIST = new AbstractList<Cell>() {
9133
9134 @Override
9135 public void add(int index, Cell element) {
9136
9137 }
9138
9139 @Override
9140 public boolean addAll(int index, Collection<? extends Cell> c) {
9141 return false;
9142 }
9143
9144 @Override
9145 public KeyValue get(int index) {
9146 throw new UnsupportedOperationException();
9147 }
9148
9149 @Override
9150 public int size() {
9151 return 0;
9152 }
9153 };
9154
9155
9156
9157
9158
9159
9160
9161
9162
9163
9164 public static void main(String[] args) throws IOException {
9165 if (args.length < 1) {
9166 printUsageAndExit(null);
9167 }
9168 boolean majorCompact = false;
9169 if (args.length > 1) {
9170 if (!args[1].toLowerCase(Locale.ROOT).startsWith("major")) {
9171 printUsageAndExit("ERROR: Unrecognized option <" + args[1] + ">");
9172 }
9173 majorCompact = true;
9174 }
9175 final Path tableDir = new Path(args[0]);
9176 final Configuration c = HBaseConfiguration.create();
9177 final FileSystem fs = FileSystem.get(c);
9178 final Path logdir = new Path(c.get("hbase.tmp.dir"));
9179 final String logname = "wal" + FSUtils.getTableName(tableDir) + System.currentTimeMillis();
9180
9181 final Configuration walConf = new Configuration(c);
9182 FSUtils.setRootDir(walConf, logdir);
9183 final WALFactory wals = new WALFactory(walConf, null, logname);
9184 try {
9185 processTable(fs, tableDir, wals, c, majorCompact);
9186 } finally {
9187 wals.close();
9188
9189 BlockCache bc = new CacheConfig(c).getBlockCache();
9190 if (bc != null) bc.shutdown();
9191 }
9192 }
9193
9194 @Override
9195 public long getOpenSeqNum() {
9196 return this.openSeqNum;
9197 }
9198
9199 @Override
9200 public Map<byte[], Long> getMaxStoreSeqId() {
9201 return this.maxSeqIdInStores;
9202 }
9203
9204 @Override
9205 public long getOldestSeqIdOfStore(byte[] familyName) {
9206 return wal.getEarliestMemstoreSeqNum(getRegionInfo().getEncodedNameAsBytes(), familyName);
9207 }
9208
9209 @Override
9210 public CompactionState getCompactionState() {
9211 boolean hasMajor = majorInProgress.get() > 0, hasMinor = minorInProgress.get() > 0;
9212 return (hasMajor ? (hasMinor ? CompactionState.MAJOR_AND_MINOR : CompactionState.MAJOR)
9213 : (hasMinor ? CompactionState.MINOR : CompactionState.NONE));
9214 }
9215
9216 public void reportCompactionRequestStart(boolean isMajor){
9217 (isMajor ? majorInProgress : minorInProgress).incrementAndGet();
9218 }
9219
9220 public void reportCompactionRequestEnd(boolean isMajor, int numFiles, long filesSizeCompacted) {
9221 int newValue = (isMajor ? majorInProgress : minorInProgress).decrementAndGet();
9222
9223
9224 compactionsFinished.incrementAndGet();
9225 compactionNumFilesCompacted.addAndGet(numFiles);
9226 compactionNumBytesCompacted.addAndGet(filesSizeCompacted);
9227
9228 assert newValue >= 0;
9229 }
9230
9231 public void reportCompactionRequestFailure() {
9232 compactionsFailed.incrementAndGet();
9233 }
9234
9235 public void incrementCompactionsQueuedCount() {
9236 compactionsQueued.incrementAndGet();
9237 }
9238
9239 public void decrementCompactionsQueuedCount() {
9240 compactionsQueued.decrementAndGet();
9241 }
9242
9243 public void incrementFlushesQueuedCount() {
9244 flushesQueued.incrementAndGet();
9245 }
9246
9247
9248
9249
9250
9251 public long getSequenceId() {
9252 return this.mvcc.getReadPoint();
9253 }
9254
9255
9256
9257
9258
9259
9260
9261
9262 private WALKey appendEmptyEdit(final WAL wal) throws IOException {
9263
9264 @SuppressWarnings("deprecation")
9265 WALKey key = new HLogKey(getRegionInfo().getEncodedNameAsBytes(),
9266 getRegionInfo().getTable(), WALKey.NO_SEQUENCE_ID, 0, null,
9267 HConstants.NO_NONCE, HConstants.NO_NONCE, getMVCC());
9268
9269
9270
9271 try {
9272 wal.append(getTableDesc(), getRegionInfo(), key, WALEdit.EMPTY_WALEDIT, false);
9273 } catch (Throwable t) {
9274
9275 getMVCC().complete(key.getWriteEntry());
9276 }
9277 return key;
9278 }
9279
9280
9281
9282
9283
9284
9285
9286 void checkInterrupt() throws NotServingRegionException, InterruptedIOException {
9287 if (Thread.interrupted()) {
9288 if (this.closing.get()) {
9289 throw new NotServingRegionException(
9290 getRegionInfo().getRegionNameAsString() + " is closing");
9291 }
9292 throw new InterruptedIOException();
9293 }
9294 }
9295
9296
9297
9298
9299
9300
9301 IOException throwOnInterrupt(Throwable t) {
9302 if (this.closing.get()) {
9303 return (NotServingRegionException) new NotServingRegionException(
9304 getRegionInfo().getRegionNameAsString() + " is closing")
9305 .initCause(t);
9306 }
9307 return (InterruptedIOException) new InterruptedIOException().initCause(t);
9308 }
9309
9310
9311
9312
9313 @Override
9314 public void onConfigurationChange(Configuration conf) {
9315
9316 }
9317
9318
9319
9320
9321 @Override
9322 public void registerChildren(ConfigurationManager manager) {
9323 configurationManager = Optional.of(manager);
9324 for (Store s : this.stores.values()) {
9325 configurationManager.get().registerObserver(s);
9326 }
9327 }
9328
9329
9330
9331
9332 @Override
9333 public void deregisterChildren(ConfigurationManager manager) {
9334 for (Store s : this.stores.values()) {
9335 configurationManager.get().deregisterObserver(s);
9336 }
9337 }
9338
9339
9340
9341
9342 public RegionSplitPolicy getSplitPolicy() {
9343 return this.splitPolicy;
9344 }
9345 }