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