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