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