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