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 doRollBackMemstore = 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 doRollBackMemstore = true;
2413 addedSize += applyFamilyMapToMemstore(familyMaps[i], w);
2414 }
2415
2416
2417
2418
2419 Durability durability = Durability.USE_DEFAULT;
2420
2421 for (int i = firstIndex; i < lastIndexExclusive; i++) {
2422
2423 if (batchOp.retCodeDetails[i].getOperationStatusCode()
2424 != OperationStatusCode.NOT_RUN) {
2425 continue;
2426 }
2427 batchOp.retCodeDetails[i] = OperationStatus.SUCCESS;
2428
2429 Mutation m = batchOp.operations[i].getFirst();
2430 Durability tmpDur = m.getDurability();
2431 if (tmpDur.ordinal() > durability.ordinal()) {
2432 durability = tmpDur;
2433 }
2434 if (tmpDur == Durability.SKIP_WAL) {
2435 if (m instanceof Put) {
2436 recordPutWithoutWal(m.getFamilyMap());
2437 }
2438 continue;
2439 }
2440
2441
2442 WALEdit fromCP = batchOp.walEditsFromCoprocessors[i];
2443 if (fromCP != null) {
2444 for (KeyValue kv : fromCP.getKeyValues()) {
2445 walEdit.add(kv);
2446 }
2447 }
2448 addFamilyMapToWALEdit(familyMaps[i], walEdit);
2449 }
2450
2451
2452
2453
2454 Mutation first = batchOp.operations[firstIndex].getFirst();
2455 walEdit.addClusterIds(first.getClusterIds());
2456 txid = this.log.appendNoSync(regionInfo, this.htableDescriptor.getName(),
2457 walEdit, first.getClusterId(), now, this.htableDescriptor);
2458
2459
2460
2461
2462 if (locked) {
2463 this.updatesLock.readLock().unlock();
2464 locked = false;
2465 }
2466 if (acquiredLocks != null) {
2467 for (Integer toRelease : acquiredLocks) {
2468 releaseRowLock(toRelease);
2469 }
2470 acquiredLocks = null;
2471 rowsAlreadyLocked = null;
2472 }
2473
2474
2475
2476 if (walEdit.size() > 0) {
2477 syncOrDefer(txid, durability);
2478 }
2479 doRollBackMemstore = false;
2480
2481 if (coprocessorHost != null) {
2482 MiniBatchOperationInProgress<Pair<Mutation, Integer>> miniBatchOp =
2483 new MiniBatchOperationInProgress<Pair<Mutation, Integer>>(batchOp.operations,
2484 batchOp.retCodeDetails, batchOp.walEditsFromCoprocessors, firstIndex, lastIndexExclusive);
2485 coprocessorHost.postBatchMutate(miniBatchOp);
2486 }
2487
2488
2489
2490
2491 if (w != null) {
2492 mvcc.completeMemstoreInsert(w);
2493 w = null;
2494 }
2495
2496
2497
2498
2499
2500 if (coprocessorHost != null) {
2501 for (int i = firstIndex; i < lastIndexExclusive; i++) {
2502
2503 if (batchOp.retCodeDetails[i].getOperationStatusCode()
2504 != OperationStatusCode.SUCCESS) {
2505 continue;
2506 }
2507 Mutation m = batchOp.operations[i].getFirst();
2508 if (m instanceof Put) {
2509 coprocessorHost.postPut((Put) m, walEdit, m.getWriteToWAL());
2510 } else {
2511 coprocessorHost.postDelete((Delete) m, walEdit, m.getWriteToWAL());
2512 }
2513 }
2514 }
2515 success = true;
2516 return addedSize;
2517 } finally {
2518
2519
2520 if (doRollBackMemstore) {
2521 rollbackMemstore(batchOp, familyMaps, firstIndex, lastIndexExclusive);
2522 }
2523 if (w != null) mvcc.completeMemstoreInsert(w);
2524
2525 if (locked) {
2526 this.updatesLock.readLock().unlock();
2527 }
2528
2529 if (acquiredLocks != null) {
2530 for (Integer toRelease : acquiredLocks) {
2531 releaseRowLock(toRelease);
2532 }
2533 }
2534
2535
2536 final long netTimeMs = EnvironmentEdgeManager.currentTimeMillis()- startTimeMs;
2537
2538
2539
2540
2541
2542
2543 long timeTakenForPuts = 0;
2544 if (noOfPuts > 0) {
2545
2546 double noOfMutations = noOfPuts + noOfDeletes;
2547 timeTakenForPuts = (long) (netTimeMs * (noOfPuts / noOfMutations));
2548 final Set<byte[]> keptCfs = putsCfSetConsistent ? putsCfSet : null;
2549 this.opMetrics.updateMultiPutMetrics(keptCfs, timeTakenForPuts);
2550 }
2551 if (noOfDeletes > 0) {
2552
2553 final Set<byte[]> keptCfs = deletesCfSetConsistent ? deletesCfSet : null;
2554 this.opMetrics.updateMultiDeleteMetrics(keptCfs, netTimeMs - timeTakenForPuts);
2555 }
2556 if (!success) {
2557 for (int i = firstIndex; i < lastIndexExclusive; i++) {
2558 if (batchOp.retCodeDetails[i].getOperationStatusCode() == OperationStatusCode.NOT_RUN) {
2559 batchOp.retCodeDetails[i] = OperationStatus.FAILURE;
2560 }
2561 }
2562 }
2563 batchOp.nextIndexToProcess = lastIndexExclusive;
2564 }
2565 }
2566
2567
2568
2569
2570
2571
2572
2573
2574
2575
2576
2577
2578
2579
2580
2581
2582 public boolean checkAndMutate(byte [] row, byte [] family, byte [] qualifier,
2583 CompareOp compareOp, WritableByteArrayComparable comparator, Writable w,
2584 boolean writeToWAL)
2585 throws IOException {
2586 return checkAndMutate(row, family, qualifier, compareOp, comparator, w, null, writeToWAL);
2587 }
2588
2589
2590
2591
2592
2593
2594
2595
2596
2597
2598
2599
2600
2601
2602 public boolean checkAndMutate(byte [] row, byte [] family, byte [] qualifier,
2603 CompareOp compareOp, WritableByteArrayComparable comparator, Writable w,
2604 Integer lockId, boolean writeToWAL)
2605 throws IOException{
2606 checkReadOnly();
2607
2608
2609 checkResources();
2610 boolean isPut = w instanceof Put;
2611 if (!isPut && !(w instanceof Delete))
2612 throw new DoNotRetryIOException("Action must be Put or Delete");
2613 Row r = (Row)w;
2614 if (!Bytes.equals(row, r.getRow())) {
2615 throw new DoNotRetryIOException("Action's getRow must match the passed row");
2616 }
2617
2618 startRegionOperation();
2619 this.writeRequestsCount.increment();
2620 this.opMetrics.setWriteRequestCountMetrics(this.writeRequestsCount.get());
2621 try {
2622 RowLock lock = isPut ? ((Put)w).getRowLock() : ((Delete)w).getRowLock();
2623 Get get = new Get(row, lock);
2624 checkFamily(family);
2625 get.addColumn(family, qualifier);
2626
2627
2628 Integer lid = getLock(lockId, get.getRow(), true);
2629
2630 mvcc.completeMemstoreInsert(mvcc.beginMemstoreInsert());
2631 List<KeyValue> result = new ArrayList<KeyValue>();
2632 try {
2633 result = get(get, false);
2634
2635 boolean valueIsNull = comparator.getValue() == null ||
2636 comparator.getValue().length == 0;
2637 boolean matches = false;
2638 if (result.size() == 0 && valueIsNull) {
2639 matches = true;
2640 } else if (result.size() > 0 && result.get(0).getValue().length == 0 &&
2641 valueIsNull) {
2642 matches = true;
2643 } else if (result.size() == 1 && !valueIsNull) {
2644 KeyValue kv = result.get(0);
2645 int compareResult = comparator.compareTo(kv.getBuffer(),
2646 kv.getValueOffset(), kv.getValueLength());
2647 switch (compareOp) {
2648 case LESS:
2649 matches = compareResult <= 0;
2650 break;
2651 case LESS_OR_EQUAL:
2652 matches = compareResult < 0;
2653 break;
2654 case EQUAL:
2655 matches = compareResult == 0;
2656 break;
2657 case NOT_EQUAL:
2658 matches = compareResult != 0;
2659 break;
2660 case GREATER_OR_EQUAL:
2661 matches = compareResult > 0;
2662 break;
2663 case GREATER:
2664 matches = compareResult >= 0;
2665 break;
2666 default:
2667 throw new RuntimeException("Unknown Compare op " + compareOp.name());
2668 }
2669 }
2670
2671 if (matches) {
2672
2673
2674
2675
2676
2677
2678 if (isPut) {
2679 internalPut(((Put) w), HConstants.DEFAULT_CLUSTER_ID, writeToWAL);
2680 } else {
2681 Delete d = (Delete)w;
2682 prepareDelete(d);
2683 internalDelete(d, HConstants.DEFAULT_CLUSTER_ID, writeToWAL);
2684 }
2685 return true;
2686 }
2687 return false;
2688 } finally {
2689 if(lockId == null) releaseRowLock(lid);
2690 }
2691 } finally {
2692 closeRegionOperation();
2693 }
2694 }
2695
2696
2697
2698
2699
2700
2701
2702
2703
2704
2705
2706
2707
2708
2709
2710 public void addRegionToSnapshot(SnapshotDescription desc,
2711 ForeignExceptionSnare exnSnare) throws IOException {
2712
2713
2714 Path rootDir = FSUtils.getRootDir(this.rsServices.getConfiguration());
2715 Path snapshotRegionDir = TakeSnapshotUtils.getRegionSnapshotDirectory(desc, rootDir,
2716 regionInfo.getEncodedName());
2717
2718
2719 LOG.debug("Storing region-info for snapshot.");
2720 checkRegioninfoOnFilesystem(snapshotRegionDir);
2721
2722
2723 LOG.debug("Creating references for hfiles");
2724
2725
2726
2727
2728
2729
2730 for (Store store : stores.values()) {
2731
2732 Path dstStoreDir = TakeSnapshotUtils.getStoreSnapshotDirectory(snapshotRegionDir,
2733 Bytes.toString(store.getFamily().getName()));
2734 List<StoreFile> storeFiles = store.getStorefiles();
2735 if (LOG.isDebugEnabled()) {
2736 LOG.debug("Adding snapshot references for " + storeFiles + " hfiles");
2737 }
2738
2739
2740 int sz = storeFiles.size();
2741 for (int i = 0; i < sz; i++) {
2742 if (exnSnare != null) {
2743 exnSnare.rethrowException();
2744 }
2745 StoreFile storeFile = storeFiles.get(i);
2746 Path file = storeFile.getPath();
2747
2748 LOG.debug("Creating reference for file (" + (i+1) + "/" + sz + ") : " + file);
2749 Path referenceFile = new Path(dstStoreDir, file.getName());
2750 boolean success = true;
2751 if (storeFile.isReference()) {
2752
2753 storeFile.getReference().write(fs, referenceFile);
2754 } else {
2755
2756
2757
2758
2759 success = HBaseFileSystem.createNewFileOnFileSystem(fs, referenceFile);
2760 }
2761 if (!success) {
2762 throw new IOException("Failed to create reference file:" + referenceFile);
2763 }
2764 }
2765 }
2766 }
2767
2768
2769
2770
2771
2772 private void updateKVTimestamps(
2773 final Iterable<List<KeyValue>> keyLists, final byte[] now) {
2774 for (List<KeyValue> keys: keyLists) {
2775 if (keys == null) continue;
2776 for (KeyValue key : keys) {
2777 key.updateLatestStamp(now);
2778 }
2779 }
2780 }
2781
2782
2783
2784
2785
2786
2787
2788
2789
2790
2791 private void checkResources()
2792 throws RegionTooBusyException, InterruptedIOException {
2793
2794
2795 if (this.getRegionInfo().isMetaRegion()) return;
2796
2797 boolean blocked = false;
2798 long startTime = 0;
2799 while (this.memstoreSize.get() > this.blockingMemStoreSize) {
2800 requestFlush();
2801 if (!blocked) {
2802 startTime = EnvironmentEdgeManager.currentTimeMillis();
2803 LOG.info("Blocking updates for '" + Thread.currentThread().getName() +
2804 "' on region " + Bytes.toStringBinary(getRegionName()) +
2805 ": memstore size " +
2806 StringUtils.humanReadableInt(this.memstoreSize.get()) +
2807 " is >= than blocking " +
2808 StringUtils.humanReadableInt(this.blockingMemStoreSize) + " size");
2809 }
2810 long now = EnvironmentEdgeManager.currentTimeMillis();
2811 long timeToWait = startTime + busyWaitDuration - now;
2812 if (timeToWait <= 0L) {
2813 final long totalTime = now - startTime;
2814 this.updatesBlockedMs.add(totalTime);
2815 LOG.info("Failed to unblock updates for region " + this + " '"
2816 + Thread.currentThread().getName() + "' in " + totalTime
2817 + "ms. The region is still busy.");
2818 throw new RegionTooBusyException("region is flushing");
2819 }
2820 blocked = true;
2821 synchronized(this) {
2822 try {
2823 wait(Math.min(timeToWait, threadWakeFrequency));
2824 } catch (InterruptedException ie) {
2825 final long totalTime = EnvironmentEdgeManager.currentTimeMillis() - startTime;
2826 if (totalTime > 0) {
2827 this.updatesBlockedMs.add(totalTime);
2828 }
2829 LOG.info("Interrupted while waiting to unblock updates for region "
2830 + this + " '" + Thread.currentThread().getName() + "'");
2831 InterruptedIOException iie = new InterruptedIOException();
2832 iie.initCause(ie);
2833 throw iie;
2834 }
2835 }
2836 }
2837 if (blocked) {
2838
2839 final long totalTime = EnvironmentEdgeManager.currentTimeMillis() - startTime;
2840 if(totalTime > 0 ){
2841 this.updatesBlockedMs.add(totalTime);
2842 }
2843 LOG.info("Unblocking updates for region " + this + " '"
2844 + Thread.currentThread().getName() + "'");
2845 }
2846 }
2847
2848
2849
2850
2851 protected void checkReadOnly() throws IOException {
2852 if (this.writestate.isReadOnly()) {
2853 throw new IOException("region is read only");
2854 }
2855 }
2856
2857
2858
2859
2860
2861
2862
2863
2864
2865 private void put(byte [] family, List<KeyValue> edits)
2866 throws IOException {
2867 Map<byte[], List<KeyValue>> familyMap;
2868 familyMap = new HashMap<byte[], List<KeyValue>>();
2869
2870 familyMap.put(family, edits);
2871 Put p = new Put();
2872 p.setFamilyMap(familyMap);
2873 p.setClusterId(HConstants.DEFAULT_CLUSTER_ID);
2874 p.setWriteToWAL(true);
2875 this.internalPut(p, HConstants.DEFAULT_CLUSTER_ID, true);
2876 }
2877
2878
2879
2880
2881
2882
2883
2884
2885
2886 private void internalPut(Put put, UUID clusterId, boolean writeToWAL) throws IOException {
2887 Map<byte[], List<KeyValue>> familyMap = put.getFamilyMap();
2888 WALEdit walEdit = new WALEdit();
2889
2890 if (coprocessorHost != null) {
2891 if (coprocessorHost.prePut(put, walEdit, writeToWAL)) {
2892 return;
2893 }
2894 }
2895
2896 long now = EnvironmentEdgeManager.currentTimeMillis();
2897 byte[] byteNow = Bytes.toBytes(now);
2898 boolean flush = false;
2899
2900 lock(this.updatesLock.readLock());
2901 try {
2902 checkFamilies(familyMap.keySet());
2903 checkTimestamps(familyMap, now);
2904 updateKVTimestamps(familyMap.values(), byteNow);
2905
2906
2907
2908
2909
2910 if (writeToWAL) {
2911 addFamilyMapToWALEdit(familyMap, walEdit);
2912 walEdit.addClusterIds(put.getClusterIds());
2913 this.log.append(regionInfo, this.htableDescriptor.getName(),
2914 walEdit, clusterId, now, this.htableDescriptor);
2915 } else {
2916 recordPutWithoutWal(familyMap);
2917 }
2918
2919 long addedSize = applyFamilyMapToMemstore(familyMap, null);
2920 flush = isFlushSize(this.addAndGetGlobalMemstoreSize(addedSize));
2921 } finally {
2922 this.updatesLock.readLock().unlock();
2923 }
2924
2925 if (coprocessorHost != null) {
2926 coprocessorHost.postPut(put, walEdit, writeToWAL);
2927 }
2928
2929
2930 final long after = EnvironmentEdgeManager.currentTimeMillis();
2931 this.opMetrics.updatePutMetrics(familyMap.keySet(), after - now);
2932
2933 if (flush) {
2934
2935 requestFlush();
2936 }
2937 }
2938
2939
2940
2941
2942
2943
2944
2945
2946
2947
2948
2949
2950
2951 private long applyFamilyMapToMemstore(Map<byte[], List<KeyValue>> familyMap,
2952 MultiVersionConsistencyControl.WriteEntry localizedWriteEntry) {
2953 long size = 0;
2954 boolean freemvcc = false;
2955
2956 try {
2957 if (localizedWriteEntry == null) {
2958 localizedWriteEntry = mvcc.beginMemstoreInsert();
2959 freemvcc = true;
2960 }
2961
2962 for (Map.Entry<byte[], List<KeyValue>> e : familyMap.entrySet()) {
2963 byte[] family = e.getKey();
2964 List<KeyValue> edits = e.getValue();
2965
2966 Store store = getStore(family);
2967 for (KeyValue kv: edits) {
2968 kv.setMemstoreTS(localizedWriteEntry.getWriteNumber());
2969 size += store.add(kv);
2970 }
2971 }
2972 } finally {
2973 if (freemvcc) {
2974 mvcc.completeMemstoreInsert(localizedWriteEntry);
2975 }
2976 }
2977
2978 return size;
2979 }
2980
2981
2982
2983
2984
2985
2986 private void rollbackMemstore(BatchOperationInProgress<Pair<Mutation, Integer>> batchOp,
2987 Map<byte[], List<KeyValue>>[] familyMaps,
2988 int start, int end) {
2989 int kvsRolledback = 0;
2990 for (int i = start; i < end; i++) {
2991
2992 if (batchOp.retCodeDetails[i].getOperationStatusCode()
2993 != OperationStatusCode.SUCCESS) {
2994 continue;
2995 }
2996
2997
2998 Map<byte[], List<KeyValue>> familyMap = familyMaps[i];
2999 for (Map.Entry<byte[], List<KeyValue>> e : familyMap.entrySet()) {
3000 byte[] family = e.getKey();
3001 List<KeyValue> edits = e.getValue();
3002
3003
3004
3005
3006 Store store = getStore(family);
3007 for (KeyValue kv: edits) {
3008 store.rollback(kv);
3009 kvsRolledback++;
3010 }
3011 }
3012 }
3013 LOG.debug("rollbackMemstore rolled back " + kvsRolledback +
3014 " keyvalues from start:" + start + " to end:" + end);
3015 }
3016
3017
3018
3019
3020
3021 private void checkFamilies(Collection<byte[]> families)
3022 throws NoSuchColumnFamilyException {
3023 for (byte[] family : families) {
3024 checkFamily(family);
3025 }
3026 }
3027
3028 private void checkTimestamps(final Map<byte[], List<KeyValue>> familyMap,
3029 long now) throws DoNotRetryIOException {
3030 if (timestampSlop == HConstants.LATEST_TIMESTAMP) {
3031 return;
3032 }
3033 long maxTs = now + timestampSlop;
3034 for (List<KeyValue> kvs : familyMap.values()) {
3035 for (KeyValue kv : kvs) {
3036
3037 if (!kv.isLatestTimestamp() && kv.getTimestamp() > maxTs) {
3038 throw new DoNotRetryIOException("Timestamp for KV out of range "
3039 + kv + " (too.new=" + timestampSlop + ")");
3040 }
3041 }
3042 }
3043 }
3044
3045
3046
3047
3048
3049
3050
3051 private void addFamilyMapToWALEdit(Map<byte[], List<KeyValue>> familyMap,
3052 WALEdit walEdit) {
3053 for (List<KeyValue> edits : familyMap.values()) {
3054 for (KeyValue kv : edits) {
3055 walEdit.add(kv);
3056 }
3057 }
3058 }
3059
3060 private void requestFlush() {
3061 if (this.rsServices == null) {
3062 return;
3063 }
3064 synchronized (writestate) {
3065 if (this.writestate.isFlushRequested()) {
3066 return;
3067 }
3068 writestate.flushRequested = true;
3069 }
3070
3071 this.rsServices.getFlushRequester().requestFlush(this);
3072 if (LOG.isDebugEnabled()) {
3073 LOG.debug("Flush requested on " + this);
3074 }
3075 }
3076
3077
3078
3079
3080
3081 private boolean isFlushSize(final long size) {
3082 return size > this.memstoreFlushSize;
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
3121 protected long replayRecoveredEditsIfAny(final Path regiondir,
3122 Map<byte[], Long> maxSeqIdInStores,
3123 final CancelableProgressable reporter, final MonitoredTask status)
3124 throws UnsupportedEncodingException, IOException {
3125 long minSeqIdForTheRegion = -1;
3126 for (Long maxSeqIdInStore : maxSeqIdInStores.values()) {
3127 if (maxSeqIdInStore < minSeqIdForTheRegion || minSeqIdForTheRegion == -1) {
3128 minSeqIdForTheRegion = maxSeqIdInStore;
3129 }
3130 }
3131 long seqid = minSeqIdForTheRegion;
3132 NavigableSet<Path> files = HLog.getSplitEditFilesSorted(this.fs, regiondir);
3133 if (files == null || files.isEmpty()) return seqid;
3134
3135 for (Path edits: files) {
3136 if (edits == null || !this.fs.exists(edits)) {
3137 LOG.warn("Null or non-existent edits file: " + edits);
3138 continue;
3139 }
3140 if (isZeroLengthThenDelete(this.fs, edits)) continue;
3141
3142 long maxSeqId = Long.MAX_VALUE;
3143 String fileName = edits.getName();
3144 maxSeqId = Math.abs(Long.parseLong(fileName));
3145 if (maxSeqId <= minSeqIdForTheRegion) {
3146 String msg = "Maximum sequenceid for this log is " + maxSeqId
3147 + " and minimum sequenceid for the region is " + minSeqIdForTheRegion
3148 + ", skipped the whole file, path=" + edits;
3149 LOG.debug(msg);
3150 continue;
3151 }
3152
3153 try {
3154 seqid = replayRecoveredEdits(edits, maxSeqIdInStores, reporter);
3155 } catch (IOException e) {
3156 boolean skipErrors = conf.getBoolean("hbase.skip.errors", false);
3157 if (skipErrors) {
3158 Path p = HLog.moveAsideBadEditsFile(fs, edits);
3159 LOG.error("hbase.skip.errors=true so continuing. Renamed " + edits +
3160 " as " + p, e);
3161 } else {
3162 throw e;
3163 }
3164 }
3165
3166
3167 if (this.rsAccounting != null) {
3168 this.rsAccounting.clearRegionReplayEditsSize(this.regionInfo.getRegionName());
3169 }
3170 }
3171 if (seqid > minSeqIdForTheRegion) {
3172
3173 internalFlushcache(null, seqid, status);
3174 }
3175
3176 for (Path file: files) {
3177 if (!HBaseFileSystem.deleteFileFromFileSystem(fs, file)) {
3178 LOG.error("Failed delete of " + file);
3179 } else {
3180 LOG.debug("Deleted recovered.edits file=" + file);
3181 }
3182 }
3183 return seqid;
3184 }
3185
3186
3187
3188
3189
3190
3191
3192
3193
3194
3195 private long replayRecoveredEdits(final Path edits,
3196 Map<byte[], Long> maxSeqIdInStores, final CancelableProgressable reporter)
3197 throws IOException {
3198 String msg = "Replaying edits from " + edits;
3199 LOG.info(msg);
3200 MonitoredTask status = TaskMonitor.get().createStatus(msg);
3201
3202 status.setStatus("Opening logs");
3203 HLog.Reader reader = null;
3204 try {
3205 reader = HLog.getReader(this.fs, edits, conf);
3206 long currentEditSeqId = -1;
3207 long firstSeqIdInLog = -1;
3208 long skippedEdits = 0;
3209 long editsCount = 0;
3210 long intervalEdits = 0;
3211 HLog.Entry entry;
3212 Store store = null;
3213 boolean reported_once = false;
3214
3215 try {
3216
3217 int interval = this.conf.getInt("hbase.hstore.report.interval.edits",
3218 2000);
3219
3220 int period = this.conf.getInt("hbase.hstore.report.period",
3221 this.conf.getInt("hbase.master.assignment.timeoutmonitor.timeout",
3222 180000) / 2);
3223 long lastReport = EnvironmentEdgeManager.currentTimeMillis();
3224
3225 while ((entry = reader.next()) != null) {
3226 HLogKey key = entry.getKey();
3227 WALEdit val = entry.getEdit();
3228
3229 if (reporter != null) {
3230 intervalEdits += val.size();
3231 if (intervalEdits >= interval) {
3232
3233 intervalEdits = 0;
3234 long cur = EnvironmentEdgeManager.currentTimeMillis();
3235 if (lastReport + period <= cur) {
3236 status.setStatus("Replaying edits..." +
3237 " skipped=" + skippedEdits +
3238 " edits=" + editsCount);
3239
3240 if(!reporter.progress()) {
3241 msg = "Progressable reporter failed, stopping replay";
3242 LOG.warn(msg);
3243 status.abort(msg);
3244 throw new IOException(msg);
3245 }
3246 reported_once = true;
3247 lastReport = cur;
3248 }
3249 }
3250 }
3251
3252
3253
3254 if (coprocessorHost != null) {
3255 status.setStatus("Running pre-WAL-restore hook in coprocessors");
3256 if (coprocessorHost.preWALRestore(this.getRegionInfo(), key, val)) {
3257
3258 continue;
3259 }
3260 }
3261
3262 if (firstSeqIdInLog == -1) {
3263 firstSeqIdInLog = key.getLogSeqNum();
3264 }
3265 boolean flush = false;
3266 for (KeyValue kv: val.getKeyValues()) {
3267
3268
3269 if (kv.matchingFamily(HLog.METAFAMILY) ||
3270 !Bytes.equals(key.getEncodedRegionName(), this.regionInfo.getEncodedNameAsBytes())) {
3271 skippedEdits++;
3272 continue;
3273 }
3274
3275 if (store == null || !kv.matchingFamily(store.getFamily().getName())) {
3276 store = this.stores.get(kv.getFamily());
3277 }
3278 if (store == null) {
3279
3280
3281 LOG.warn("No family for " + kv);
3282 skippedEdits++;
3283 continue;
3284 }
3285
3286 if (key.getLogSeqNum() <= maxSeqIdInStores.get(store.getFamily()
3287 .getName())) {
3288 skippedEdits++;
3289 continue;
3290 }
3291 currentEditSeqId = key.getLogSeqNum();
3292
3293
3294
3295 flush = restoreEdit(store, kv);
3296 editsCount++;
3297 }
3298 if (flush) internalFlushcache(null, currentEditSeqId, status);
3299
3300 if (coprocessorHost != null) {
3301 coprocessorHost.postWALRestore(this.getRegionInfo(), key, val);
3302 }
3303 }
3304 } catch (EOFException eof) {
3305 Path p = HLog.moveAsideBadEditsFile(fs, edits);
3306 msg = "Encountered EOF. Most likely due to Master failure during " +
3307 "log spliting, so we have this data in another edit. " +
3308 "Continuing, but renaming " + edits + " as " + p;
3309 LOG.warn(msg, eof);
3310 status.abort(msg);
3311 } catch (IOException ioe) {
3312
3313
3314 if (ioe.getCause() instanceof ParseException) {
3315 Path p = HLog.moveAsideBadEditsFile(fs, edits);
3316 msg = "File corruption encountered! " +
3317 "Continuing, but renaming " + edits + " as " + p;
3318 LOG.warn(msg, ioe);
3319 status.setStatus(msg);
3320 } else {
3321 status.abort(StringUtils.stringifyException(ioe));
3322
3323
3324 throw ioe;
3325 }
3326 }
3327 if (reporter != null && !reported_once) {
3328 reporter.progress();
3329 }
3330 msg = "Applied " + editsCount + ", skipped " + skippedEdits +
3331 ", firstSequenceidInLog=" + firstSeqIdInLog +
3332 ", maxSequenceidInLog=" + currentEditSeqId + ", path=" + edits;
3333 status.markComplete(msg);
3334 LOG.debug(msg);
3335 return currentEditSeqId;
3336 } finally {
3337 status.cleanup();
3338 if (reader != null) {
3339 reader.close();
3340 }
3341 }
3342 }
3343
3344
3345
3346
3347
3348
3349
3350 protected boolean restoreEdit(final Store s, final KeyValue kv) {
3351 long kvSize = s.add(kv);
3352 if (this.rsAccounting != null) {
3353 rsAccounting.addAndGetRegionReplayEditsSize(this.regionInfo.getRegionName(), kvSize);
3354 }
3355 return isFlushSize(this.addAndGetGlobalMemstoreSize(kvSize));
3356 }
3357
3358
3359
3360
3361
3362
3363
3364 private static boolean isZeroLengthThenDelete(final FileSystem fs, final Path p)
3365 throws IOException {
3366 FileStatus stat = fs.getFileStatus(p);
3367 if (stat.getLen() > 0) return false;
3368 LOG.warn("File " + p + " is zero-length, deleting.");
3369 HBaseFileSystem.deleteFileFromFileSystem(fs, p);
3370 return true;
3371 }
3372
3373 protected Store instantiateHStore(Path tableDir, HColumnDescriptor c)
3374 throws IOException {
3375 return new Store(tableDir, this, c, this.fs, this.conf);
3376 }
3377
3378
3379
3380
3381
3382
3383
3384
3385 public Store getStore(final byte [] column) {
3386 return this.stores.get(column);
3387 }
3388
3389 public Map<byte[], Store> getStores() {
3390 return this.stores;
3391 }
3392
3393
3394
3395
3396
3397
3398
3399
3400 public List<String> getStoreFileList(final byte [][] columns)
3401 throws IllegalArgumentException {
3402 List<String> storeFileNames = new ArrayList<String>();
3403 synchronized(closeLock) {
3404 for(byte[] column : columns) {
3405 Store store = this.stores.get(column);
3406 if (store == null) {
3407 throw new IllegalArgumentException("No column family : " +
3408 new String(column) + " available");
3409 }
3410 List<StoreFile> storeFiles = store.getStorefiles();
3411 for (StoreFile storeFile: storeFiles) {
3412 storeFileNames.add(storeFile.getPath().toString());
3413 }
3414 }
3415 }
3416 return storeFileNames;
3417 }
3418
3419
3420
3421
3422
3423 void checkRow(final byte [] row, String op) throws IOException {
3424 if(!rowIsInRange(regionInfo, row)) {
3425 throw new WrongRegionException("Requested row out of range for " +
3426 op + " on HRegion " + this + ", startKey='" +
3427 Bytes.toStringBinary(regionInfo.getStartKey()) + "', getEndKey()='" +
3428 Bytes.toStringBinary(regionInfo.getEndKey()) + "', row='" +
3429 Bytes.toStringBinary(row) + "'");
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
3456 public Integer obtainRowLock(final byte [] row) throws IOException {
3457 startRegionOperation();
3458 this.writeRequestsCount.increment();
3459 this.opMetrics.setWriteRequestCountMetrics( this.writeRequestsCount.get());
3460 try {
3461 return internalObtainRowLock(new HashedBytes(row), true);
3462 } finally {
3463 closeRegionOperation();
3464 }
3465 }
3466
3467
3468
3469
3470
3471
3472
3473 private Integer internalObtainRowLock(final HashedBytes rowKey, boolean waitForLock)
3474 throws IOException {
3475 checkRow(rowKey.getBytes(), "row lock");
3476 startRegionOperation();
3477 try {
3478 CountDownLatch rowLatch = new CountDownLatch(1);
3479
3480
3481 while (true) {
3482 CountDownLatch existingLatch = lockedRows.putIfAbsent(rowKey, rowLatch);
3483 if (existingLatch == null) {
3484 break;
3485 } else {
3486
3487 if (!waitForLock) {
3488 return null;
3489 }
3490 try {
3491 if (!existingLatch.await(this.rowLockWaitDuration,
3492 TimeUnit.MILLISECONDS)) {
3493 throw new IOException("Timed out on getting lock for row=" + rowKey);
3494 }
3495 } catch (InterruptedException ie) {
3496
3497 }
3498 }
3499 }
3500
3501
3502 while (true) {
3503 Integer lockId = lockIdGenerator.incrementAndGet();
3504 HashedBytes existingRowKey = lockIds.putIfAbsent(lockId, rowKey);
3505 if (existingRowKey == null) {
3506 return lockId;
3507 } else {
3508
3509 lockIdGenerator.set(rand.nextInt());
3510 }
3511 }
3512 } finally {
3513 closeRegionOperation();
3514 }
3515 }
3516
3517
3518
3519
3520
3521
3522 byte[] getRowFromLock(final Integer lockid) {
3523 HashedBytes rowKey = lockIds.get(lockid);
3524 return rowKey == null ? null : rowKey.getBytes();
3525 }
3526
3527
3528
3529
3530
3531 public void releaseRowLock(final Integer lockId) {
3532 if (lockId == null) return;
3533 HashedBytes rowKey = lockIds.remove(lockId);
3534 if (rowKey == null) {
3535 LOG.warn("Release unknown lockId: " + lockId);
3536 return;
3537 }
3538 CountDownLatch rowLatch = lockedRows.remove(rowKey);
3539 if (rowLatch == null) {
3540 LOG.error("Releases row not locked, lockId: " + lockId + " row: "
3541 + rowKey);
3542 return;
3543 }
3544 rowLatch.countDown();
3545 }
3546
3547
3548
3549
3550
3551
3552 boolean isRowLocked(final Integer lockId) {
3553 return lockIds.containsKey(lockId);
3554 }
3555
3556
3557
3558
3559
3560
3561
3562
3563
3564
3565 public Integer getLock(Integer lockid, byte [] row, boolean waitForLock)
3566 throws IOException {
3567 return getLock(lockid, new HashedBytes(row), waitForLock);
3568 }
3569
3570
3571
3572
3573
3574
3575
3576
3577
3578
3579 protected Integer getLock(Integer lockid, HashedBytes row, boolean waitForLock)
3580 throws IOException {
3581 Integer lid;
3582 if (lockid == null) {
3583 lid = internalObtainRowLock(row, waitForLock);
3584 } else {
3585 HashedBytes rowFromLock = lockIds.get(lockid);
3586 if (!row.equals(rowFromLock)) {
3587 throw new IOException("Invalid row lock: LockId: " + lockid + " holds the lock for row: " + rowFromLock + " but wanted lock for row: " + row);
3588 }
3589 lid = lockid;
3590 }
3591 return lid;
3592 }
3593
3594
3595
3596
3597
3598
3599
3600 private static boolean hasMultipleColumnFamilies(
3601 List<Pair<byte[], String>> familyPaths) {
3602 boolean multipleFamilies = false;
3603 byte[] family = null;
3604 for (Pair<byte[], String> pair : familyPaths) {
3605 byte[] fam = pair.getFirst();
3606 if (family == null) {
3607 family = fam;
3608 } else if (!Bytes.equals(family, fam)) {
3609 multipleFamilies = true;
3610 break;
3611 }
3612 }
3613 return multipleFamilies;
3614 }
3615
3616
3617
3618
3619
3620
3621
3622
3623
3624 public boolean bulkLoadHFiles(List<Pair<byte[], String>> familyPaths) throws IOException {
3625 return bulkLoadHFiles(familyPaths, false);
3626 }
3627
3628
3629
3630
3631
3632
3633
3634
3635
3636 public boolean bulkLoadHFiles(List<Pair<byte[], String>> familyPaths, boolean assignSeqId)
3637 throws IOException {
3638 return bulkLoadHFiles(familyPaths, null, assignSeqId);
3639 }
3640
3641
3642
3643
3644
3645
3646
3647
3648
3649
3650
3651 public boolean bulkLoadHFiles(List<Pair<byte[], String>> familyPaths,
3652 BulkLoadListener bulkLoadListener) throws IOException {
3653 return bulkLoadHFiles(familyPaths, bulkLoadListener, false);
3654 }
3655
3656
3657
3658
3659
3660
3661
3662
3663
3664
3665 public boolean bulkLoadHFiles(List<Pair<byte[], String>> familyPaths,
3666 BulkLoadListener bulkLoadListener, boolean assignSeqId) throws IOException {
3667 Preconditions.checkNotNull(familyPaths);
3668
3669 startBulkRegionOperation(hasMultipleColumnFamilies(familyPaths));
3670 try {
3671 this.writeRequestsCount.increment();
3672 this.opMetrics.setWriteRequestCountMetrics( this.writeRequestsCount.get());
3673
3674
3675
3676
3677 List<IOException> ioes = new ArrayList<IOException>();
3678 List<Pair<byte[], String>> failures = new ArrayList<Pair<byte[], String>>();
3679 for (Pair<byte[], String> p : familyPaths) {
3680 byte[] familyName = p.getFirst();
3681 String path = p.getSecond();
3682
3683 Store store = getStore(familyName);
3684 if (store == null) {
3685 IOException ioe = new DoNotRetryIOException(
3686 "No such column family " + Bytes.toStringBinary(familyName));
3687 ioes.add(ioe);
3688 } else {
3689 try {
3690 store.assertBulkLoadHFileOk(new Path(path));
3691 } catch (WrongRegionException wre) {
3692
3693 failures.add(p);
3694 } catch (IOException ioe) {
3695
3696 ioes.add(ioe);
3697 }
3698 }
3699 }
3700
3701
3702 if (ioes.size() != 0) {
3703 IOException e = MultipleIOException.createIOException(ioes);
3704 LOG.error("There were one or more IO errors when checking if the bulk load is ok.", e);
3705 throw e;
3706 }
3707
3708
3709 if (failures.size() != 0) {
3710 StringBuilder list = new StringBuilder();
3711 for (Pair<byte[], String> p : failures) {
3712 list.append("\n").append(Bytes.toString(p.getFirst())).append(" : ")
3713 .append(p.getSecond());
3714 }
3715
3716 LOG.warn("There was a recoverable bulk load failure likely due to a" +
3717 " split. These (family, HFile) pairs were not loaded: " + list);
3718 return false;
3719 }
3720
3721 for (Pair<byte[], String> p : familyPaths) {
3722 byte[] familyName = p.getFirst();
3723 String path = p.getSecond();
3724 Store store = getStore(familyName);
3725 try {
3726 String finalPath = path;
3727 if(bulkLoadListener != null) {
3728 finalPath = bulkLoadListener.prepareBulkLoad(familyName, path);
3729 }
3730 store.bulkLoadHFile(finalPath, assignSeqId ? this.log.obtainSeqNum() : -1);
3731 if(bulkLoadListener != null) {
3732 bulkLoadListener.doneBulkLoad(familyName, path);
3733 }
3734 } catch (IOException ioe) {
3735
3736
3737
3738
3739 LOG.error("There was a partial failure due to IO when attempting to" +
3740 " load " + Bytes.toString(p.getFirst()) + " : "+ p.getSecond());
3741 if(bulkLoadListener != null) {
3742 try {
3743 bulkLoadListener.failedBulkLoad(familyName, path);
3744 } catch (Exception ex) {
3745 LOG.error("Error while calling failedBulkLoad for family "+
3746 Bytes.toString(familyName)+" with path "+path, ex);
3747 }
3748 }
3749 throw ioe;
3750 }
3751 }
3752 return true;
3753 } finally {
3754 closeBulkRegionOperation();
3755 }
3756 }
3757
3758 @Override
3759 public boolean equals(Object o) {
3760 if (!(o instanceof HRegion)) {
3761 return false;
3762 }
3763 return Bytes.equals(this.getRegionName(), ((HRegion) o).getRegionName());
3764 }
3765
3766 @Override
3767 public int hashCode() {
3768 return Bytes.hashCode(this.getRegionName());
3769 }
3770
3771 @Override
3772 public String toString() {
3773 return this.regionInfo.getRegionNameAsString();
3774 }
3775
3776
3777 public Path getTableDir() {
3778 return this.tableDir;
3779 }
3780
3781
3782
3783
3784 class RegionScannerImpl implements RegionScanner {
3785
3786 KeyValueHeap storeHeap = null;
3787
3788
3789 KeyValueHeap joinedHeap = null;
3790
3791
3792
3793 private KeyValue joinedContinuationRow = null;
3794
3795 private final KeyValue KV_LIMIT = new KeyValue();
3796 private final byte [] stopRow;
3797 private final Filter filter;
3798 private int batch;
3799 private int isScan;
3800 private boolean filterClosed = false;
3801 private long readPt;
3802 private HRegion region;
3803
3804 public HRegionInfo getRegionInfo() {
3805 return regionInfo;
3806 }
3807
3808 RegionScannerImpl(Scan scan, List<KeyValueScanner> additionalScanners, HRegion region)
3809 throws IOException {
3810
3811 this.region = region;
3812 this.filter = scan.getFilter();
3813 this.batch = scan.getBatch();
3814 if (Bytes.equals(scan.getStopRow(), HConstants.EMPTY_END_ROW)) {
3815 this.stopRow = null;
3816 } else {
3817 this.stopRow = scan.getStopRow();
3818 }
3819
3820
3821 this.isScan = scan.isGetScan() ? -1 : 0;
3822
3823
3824
3825 IsolationLevel isolationLevel = scan.getIsolationLevel();
3826 synchronized(scannerReadPoints) {
3827 if (isolationLevel == IsolationLevel.READ_UNCOMMITTED) {
3828
3829 this.readPt = Long.MAX_VALUE;
3830 MultiVersionConsistencyControl.setThreadReadPoint(this.readPt);
3831 } else {
3832 this.readPt = MultiVersionConsistencyControl.resetThreadReadPoint(mvcc);
3833 }
3834 scannerReadPoints.put(this, this.readPt);
3835 }
3836
3837
3838
3839 List<KeyValueScanner> scanners = new ArrayList<KeyValueScanner>();
3840 List<KeyValueScanner> joinedScanners = new ArrayList<KeyValueScanner>();
3841 if (additionalScanners != null) {
3842 scanners.addAll(additionalScanners);
3843 }
3844
3845 for (Map.Entry<byte[], NavigableSet<byte[]>> entry :
3846 scan.getFamilyMap().entrySet()) {
3847 Store store = stores.get(entry.getKey());
3848 KeyValueScanner scanner = store.getScanner(scan, entry.getValue());
3849 if (this.filter == null || !scan.doLoadColumnFamiliesOnDemand()
3850 || FilterBase.isFamilyEssential(this.filter, entry.getKey())) {
3851 scanners.add(scanner);
3852 } else {
3853 joinedScanners.add(scanner);
3854 }
3855 }
3856 this.storeHeap = new KeyValueHeap(scanners, comparator);
3857 if (!joinedScanners.isEmpty()) {
3858 this.joinedHeap = new KeyValueHeap(joinedScanners, comparator);
3859 }
3860 }
3861
3862 RegionScannerImpl(Scan scan, HRegion region) throws IOException {
3863 this(scan, null, region);
3864 }
3865
3866 @Override
3867 public long getMvccReadPoint() {
3868 return this.readPt;
3869 }
3870
3871
3872
3873 protected void resetFilters() {
3874 if (filter != null) {
3875 filter.reset();
3876 }
3877 }
3878
3879 @Override
3880 public boolean next(List<KeyValue> outResults, int limit)
3881 throws IOException {
3882 return next(outResults, limit, null);
3883 }
3884
3885 @Override
3886 public synchronized boolean next(List<KeyValue> outResults, int limit,
3887 String metric) throws IOException {
3888 if (this.filterClosed) {
3889 throw new UnknownScannerException("Scanner was closed (timed out?) " +
3890 "after we renewed it. Could be caused by a very slow scanner " +
3891 "or a lengthy garbage collection");
3892 }
3893 startRegionOperation();
3894 readRequestsCount.increment();
3895 opMetrics.setReadRequestCountMetrics(readRequestsCount.get());
3896 try {
3897
3898
3899 MultiVersionConsistencyControl.setThreadReadPoint(this.readPt);
3900
3901 return nextRaw(outResults, limit, metric);
3902 } finally {
3903 closeRegionOperation();
3904 }
3905 }
3906
3907 @Override
3908 public boolean nextRaw(List<KeyValue> outResults, String metric)
3909 throws IOException {
3910 return nextRaw(outResults, batch, metric);
3911 }
3912
3913 @Override
3914 public boolean nextRaw(List<KeyValue> outResults, int limit,
3915 String metric) throws IOException {
3916 boolean returnResult;
3917 if (outResults.isEmpty()) {
3918
3919
3920 returnResult = nextInternal(outResults, limit, metric);
3921 } else {
3922 List<KeyValue> tmpList = new ArrayList<KeyValue>();
3923 returnResult = nextInternal(tmpList, limit, metric);
3924 outResults.addAll(tmpList);
3925 }
3926 resetFilters();
3927 if (isFilterDoneInternal()) {
3928 return false;
3929 }
3930 return returnResult;
3931 }
3932
3933 @Override
3934 public boolean next(List<KeyValue> outResults)
3935 throws IOException {
3936
3937 return next(outResults, batch, null);
3938 }
3939
3940 @Override
3941 public boolean next(List<KeyValue> outResults, String metric)
3942 throws IOException {
3943
3944 return next(outResults, batch, metric);
3945 }
3946
3947 private void populateFromJoinedHeap(List<KeyValue> results, int limit, String metric)
3948 throws IOException {
3949 assert joinedContinuationRow != null;
3950 KeyValue kv = populateResult(results, this.joinedHeap, limit,
3951 joinedContinuationRow.getBuffer(), joinedContinuationRow.getRowOffset(),
3952 joinedContinuationRow.getRowLength(), metric);
3953 if (kv != KV_LIMIT) {
3954
3955 joinedContinuationRow = null;
3956 }
3957
3958
3959 Collections.sort(results, comparator);
3960 }
3961
3962
3963
3964
3965
3966
3967
3968
3969
3970
3971
3972
3973 private KeyValue populateResult(List<KeyValue> results, KeyValueHeap heap, int limit,
3974 byte[] currentRow, int offset, short length, String metric) throws IOException {
3975 KeyValue nextKv;
3976 do {
3977 heap.next(results, limit - results.size(), metric);
3978 if (limit > 0 && results.size() == limit) {
3979 return KV_LIMIT;
3980 }
3981 nextKv = heap.peek();
3982 } while (nextKv != null && nextKv.matchingRow(currentRow, offset, length));
3983 return nextKv;
3984 }
3985
3986
3987
3988
3989 public synchronized boolean isFilterDone() {
3990 return isFilterDoneInternal();
3991 }
3992
3993 private boolean isFilterDoneInternal() {
3994 return this.filter != null && this.filter.filterAllRemaining();
3995 }
3996
3997 private boolean nextInternal(List<KeyValue> results, int limit, String metric)
3998 throws IOException {
3999 if (!results.isEmpty()) {
4000 throw new IllegalArgumentException("First parameter should be an empty list");
4001 }
4002 RpcCallContext rpcCall = HBaseServer.getCurrentCall();
4003
4004
4005
4006
4007
4008 while (true) {
4009 if (rpcCall != null) {
4010
4011
4012
4013
4014 rpcCall.throwExceptionIfCallerDisconnected();
4015 }
4016
4017
4018 KeyValue current = this.storeHeap.peek();
4019
4020 byte[] currentRow = null;
4021 int offset = 0;
4022 short length = 0;
4023 if (current != null) {
4024 currentRow = current.getBuffer();
4025 offset = current.getRowOffset();
4026 length = current.getRowLength();
4027 }
4028 boolean stopRow = isStopRow(currentRow, offset, length);
4029
4030
4031 if (joinedContinuationRow == null) {
4032
4033 if (stopRow) {
4034 if (filter != null && filter.hasFilterRow()) {
4035 filter.filterRow(results);
4036 }
4037 if (filter != null && filter.filterRow()) {
4038 results.clear();
4039 }
4040 return false;
4041 }
4042
4043
4044
4045 if (filterRowKey(currentRow, offset, length)) {
4046 results.clear();
4047 boolean moreRows = nextRow(currentRow, offset, length);
4048 if (!moreRows) return false;
4049 continue;
4050 }
4051
4052
4053 KeyValue nextKv = populateResult(results, this.storeHeap, limit, currentRow, offset,
4054 length, metric);
4055 if (nextKv == KV_LIMIT) {
4056 if (this.filter != null && filter.hasFilterRow()) {
4057 throw new IncompatibleFilterException(
4058 "Filter whose hasFilterRow() returns true is incompatible with scan with limit!");
4059 }
4060 return true;
4061 }
4062 stopRow = nextKv == null
4063 || isStopRow(nextKv.getBuffer(), nextKv.getRowOffset(), nextKv.getRowLength());
4064
4065 final boolean isEmptyRow = results.isEmpty();
4066
4067
4068
4069 if (filter != null && filter.hasFilterRow()) {
4070 filter.filterRow(results);
4071 }
4072
4073 if (isEmptyRow || filterRow()) {
4074 results.clear();
4075 boolean moreRows = nextRow(currentRow, offset, length);
4076 if (!moreRows) return false;
4077
4078
4079
4080 if (!stopRow) continue;
4081 return false;
4082 }
4083
4084
4085
4086
4087
4088 if (this.joinedHeap != null) {
4089 KeyValue nextJoinedKv = joinedHeap.peek();
4090
4091 boolean mayHaveData =
4092 (nextJoinedKv != null && nextJoinedKv.matchingRow(currentRow, offset, length))
4093 || (this.joinedHeap.requestSeek(
4094 KeyValue.createFirstOnRow(currentRow, offset, length), true, true)
4095 && joinedHeap.peek() != null
4096 && joinedHeap.peek().matchingRow(currentRow, offset, length));
4097 if (mayHaveData) {
4098 joinedContinuationRow = current;
4099 populateFromJoinedHeap(results, limit, metric);
4100 }
4101 }
4102 } else {
4103
4104 populateFromJoinedHeap(results, limit, metric);
4105 }
4106
4107
4108
4109 if (joinedContinuationRow != null) {
4110 return true;
4111 }
4112
4113
4114
4115
4116 if (results.isEmpty()) {
4117 boolean moreRows = nextRow(currentRow, offset, length);
4118 if (!moreRows) return false;
4119 if (!stopRow) continue;
4120 }
4121
4122
4123 return !stopRow;
4124 }
4125 }
4126
4127 private boolean filterRow() {
4128 return filter != null
4129 && filter.filterRow();
4130 }
4131 private boolean filterRowKey(byte[] row, int offset, short length) {
4132 return filter != null
4133 && filter.filterRowKey(row, offset, length);
4134 }
4135
4136 protected boolean nextRow(byte [] currentRow, int offset, short length) throws IOException {
4137 KeyValue next;
4138 while((next = this.storeHeap.peek()) != null && next.matchingRow(currentRow, offset, length)) {
4139 this.storeHeap.next(MOCKED_LIST);
4140 }
4141 resetFilters();
4142
4143 if (this.region.getCoprocessorHost() != null) {
4144 return this.region.getCoprocessorHost().postScannerFilterRow(this, currentRow);
4145 }
4146 return true;
4147 }
4148
4149 private boolean isStopRow(byte [] currentRow, int offset, short length) {
4150 return currentRow == null ||
4151 (stopRow != null &&
4152 comparator.compareRows(stopRow, 0, stopRow.length,
4153 currentRow, offset, length) <= isScan);
4154 }
4155
4156 @Override
4157 public synchronized void close() {
4158 if (storeHeap != null) {
4159 storeHeap.close();
4160 storeHeap = null;
4161 }
4162 if (joinedHeap != null) {
4163 joinedHeap.close();
4164 joinedHeap = null;
4165 }
4166
4167 scannerReadPoints.remove(this);
4168 this.filterClosed = true;
4169 }
4170
4171 KeyValueHeap getStoreHeapForTesting() {
4172 return storeHeap;
4173 }
4174
4175 @Override
4176 public synchronized boolean reseek(byte[] row) throws IOException {
4177 if (row == null) {
4178 throw new IllegalArgumentException("Row cannot be null.");
4179 }
4180 boolean result = false;
4181 startRegionOperation();
4182 try {
4183
4184 MultiVersionConsistencyControl.setThreadReadPoint(this.readPt);
4185 KeyValue kv = KeyValue.createFirstOnRow(row);
4186
4187 result = this.storeHeap.requestSeek(kv, true, true);
4188 if (this.joinedHeap != null) {
4189 result = this.joinedHeap.requestSeek(kv, true, true) || result;
4190 }
4191 } finally {
4192 closeRegionOperation();
4193 }
4194 return result;
4195 }
4196 }
4197
4198
4199
4200
4201
4202
4203
4204
4205
4206
4207
4208
4209
4210
4211
4212
4213
4214
4215
4216
4217
4218
4219 public static HRegion newHRegion(Path tableDir, HLog log, FileSystem fs,
4220 Configuration conf, HRegionInfo regionInfo, final HTableDescriptor htd,
4221 RegionServerServices rsServices) {
4222 try {
4223 @SuppressWarnings("unchecked")
4224 Class<? extends HRegion> regionClass =
4225 (Class<? extends HRegion>) conf.getClass(HConstants.REGION_IMPL, HRegion.class);
4226
4227 Constructor<? extends HRegion> c =
4228 regionClass.getConstructor(Path.class, HLog.class, FileSystem.class,
4229 Configuration.class, HRegionInfo.class, HTableDescriptor.class,
4230 RegionServerServices.class);
4231
4232 return c.newInstance(tableDir, log, fs, conf, regionInfo, htd, rsServices);
4233 } catch (Throwable e) {
4234
4235 throw new IllegalStateException("Could not instantiate a region instance.", e);
4236 }
4237 }
4238
4239
4240
4241
4242
4243
4244
4245
4246
4247
4248
4249
4250
4251
4252
4253
4254
4255
4256
4257 public static HRegion createHRegion(final HRegionInfo info, final Path rootDir,
4258 final Configuration conf, final HTableDescriptor hTableDescriptor)
4259 throws IOException {
4260 return createHRegion(info, rootDir, conf, hTableDescriptor, null);
4261 }
4262
4263
4264
4265
4266
4267
4268
4269
4270
4271
4272
4273 public static void closeHRegion(final HRegion r) throws IOException {
4274 if (r == null) return;
4275 r.close();
4276 if (r.getLog() == null) return;
4277 r.getLog().closeAndDelete();
4278 }
4279
4280
4281
4282
4283
4284
4285
4286
4287
4288
4289
4290
4291
4292
4293
4294
4295 public static HRegion createHRegion(final HRegionInfo info, final Path rootDir,
4296 final Configuration conf,
4297 final HTableDescriptor hTableDescriptor,
4298 final HLog hlog,
4299 final boolean initialize)
4300 throws IOException {
4301 return createHRegion(info, rootDir, conf, hTableDescriptor,
4302 hlog, initialize, false);
4303 }
4304
4305
4306
4307
4308
4309
4310
4311
4312
4313
4314
4315
4316
4317
4318
4319
4320
4321
4322
4323 public static HRegion createHRegion(final HRegionInfo info, final Path rootDir,
4324 final Configuration conf,
4325 final HTableDescriptor hTableDescriptor,
4326 final HLog hlog,
4327 final boolean initialize, final boolean ignoreHLog)
4328 throws IOException {
4329 LOG.info("creating HRegion " + info.getTableNameAsString()
4330 + " HTD == " + hTableDescriptor + " RootDir = " + rootDir +
4331 " Table name == " + info.getTableNameAsString());
4332
4333 Path tableDir =
4334 HTableDescriptor.getTableDir(rootDir, info.getTableName());
4335 Path regionDir = HRegion.getRegionDir(tableDir, info.getEncodedName());
4336 FileSystem fs = FileSystem.get(conf);
4337 HBaseFileSystem.makeDirOnFileSystem(fs, regionDir);
4338
4339 writeRegioninfoOnFilesystem(info, regionDir, fs, conf);
4340 HLog effectiveHLog = hlog;
4341 if (hlog == null && !ignoreHLog) {
4342 effectiveHLog = new HLog(fs, new Path(regionDir, HConstants.HREGION_LOGDIR_NAME),
4343 new Path(regionDir, HConstants.HREGION_OLDLOGDIR_NAME), conf);
4344 }
4345 HRegion region = HRegion.newHRegion(tableDir,
4346 effectiveHLog, fs, conf, info, hTableDescriptor, null);
4347 if (initialize) {
4348 region.initialize();
4349 }
4350 return region;
4351 }
4352
4353 public static HRegion createHRegion(final HRegionInfo info, final Path rootDir,
4354 final Configuration conf,
4355 final HTableDescriptor hTableDescriptor,
4356 final HLog hlog)
4357 throws IOException {
4358 return createHRegion(info, rootDir, conf, hTableDescriptor, hlog, true);
4359 }
4360
4361
4362
4363
4364
4365
4366
4367
4368
4369
4370
4371
4372
4373 public static HRegion openHRegion(final HRegionInfo info,
4374 final HTableDescriptor htd, final HLog wal,
4375 final Configuration conf)
4376 throws IOException {
4377 return openHRegion(info, htd, wal, conf, null, null);
4378 }
4379
4380
4381
4382
4383
4384
4385
4386
4387
4388
4389
4390
4391
4392
4393
4394
4395 public static HRegion openHRegion(final HRegionInfo info,
4396 final HTableDescriptor htd, final HLog wal, final Configuration conf,
4397 final RegionServerServices rsServices,
4398 final CancelableProgressable reporter)
4399 throws IOException {
4400 if (LOG.isDebugEnabled()) {
4401 LOG.debug("Opening region: " + info);
4402 }
4403 if (info == null) {
4404 throw new NullPointerException("Passed region info is null");
4405 }
4406 Path dir = HTableDescriptor.getTableDir(FSUtils.getRootDir(conf),
4407 info.getTableName());
4408 FileSystem fs = null;
4409 if (rsServices != null) {
4410 fs = rsServices.getFileSystem();
4411 }
4412 if (fs == null) {
4413 fs = FileSystem.get(conf);
4414 }
4415 HRegion r = HRegion.newHRegion(dir, wal, fs, conf, info,
4416 htd, rsServices);
4417 return r.openHRegion(reporter);
4418 }
4419
4420 public static HRegion openHRegion(Path tableDir, final HRegionInfo info,
4421 final HTableDescriptor htd, final HLog wal, final Configuration conf)
4422 throws IOException {
4423 return openHRegion(tableDir, info, htd, wal, conf, null, null);
4424 }
4425
4426
4427
4428
4429
4430
4431
4432
4433
4434
4435
4436
4437
4438
4439
4440 public static HRegion openHRegion(final Path tableDir, final HRegionInfo info,
4441 final HTableDescriptor htd, final HLog wal, final Configuration conf,
4442 final RegionServerServices rsServices,
4443 final CancelableProgressable reporter)
4444 throws IOException {
4445 if (info == null) throw new NullPointerException("Passed region info is null");
4446 LOG.info("HRegion.openHRegion Region name ==" + info.getRegionNameAsString());
4447 if (LOG.isDebugEnabled()) {
4448 LOG.debug("Opening region: " + info);
4449 }
4450 Path dir = HTableDescriptor.getTableDir(tableDir,
4451 info.getTableName());
4452 HRegion r = HRegion.newHRegion(dir, wal, FileSystem.get(conf), conf, info,
4453 htd, rsServices);
4454 return r.openHRegion(reporter);
4455 }
4456
4457
4458
4459
4460
4461
4462
4463
4464
4465 protected HRegion openHRegion(final CancelableProgressable reporter)
4466 throws IOException {
4467 checkCompressionCodecs();
4468
4469 long seqid = initialize(reporter);
4470 if (this.log != null) {
4471 this.log.setSequenceNumber(seqid);
4472 }
4473 return this;
4474 }
4475
4476 private void checkCompressionCodecs() throws IOException {
4477 for (HColumnDescriptor fam: this.htableDescriptor.getColumnFamilies()) {
4478 CompressionTest.testCompression(fam.getCompression());
4479 CompressionTest.testCompression(fam.getCompactionCompression());
4480 }
4481 }
4482
4483
4484
4485
4486
4487
4488
4489
4490
4491
4492
4493 public static void addRegionToMETA(HRegion meta, HRegion r)
4494 throws IOException {
4495 meta.checkResources();
4496
4497 byte[] row = r.getRegionName();
4498 Integer lid = meta.obtainRowLock(row);
4499 try {
4500 final long now = EnvironmentEdgeManager.currentTimeMillis();
4501 final List<KeyValue> edits = new ArrayList<KeyValue>(2);
4502 edits.add(new KeyValue(row, HConstants.CATALOG_FAMILY,
4503 HConstants.REGIONINFO_QUALIFIER, now,
4504 Writables.getBytes(r.getRegionInfo())));
4505
4506 edits.add(new KeyValue(row, HConstants.CATALOG_FAMILY,
4507 HConstants.META_VERSION_QUALIFIER, now,
4508 Bytes.toBytes(HConstants.META_VERSION)));
4509 meta.put(HConstants.CATALOG_FAMILY, edits);
4510 } finally {
4511 meta.releaseRowLock(lid);
4512 }
4513 }
4514
4515
4516
4517
4518
4519
4520
4521
4522
4523 public static void deleteRegion(FileSystem fs, Path rootdir, HRegionInfo info)
4524 throws IOException {
4525 deleteRegion(fs, HRegion.getRegionDir(rootdir, info));
4526 }
4527
4528 private static void deleteRegion(FileSystem fs, Path regiondir)
4529 throws IOException {
4530 if (LOG.isDebugEnabled()) {
4531 LOG.debug("DELETING region " + regiondir.toString());
4532 }
4533 if (!HBaseFileSystem.deleteDirFromFileSystem(fs, regiondir)) {
4534 LOG.warn("Failed delete of " + regiondir);
4535 }
4536 }
4537
4538
4539
4540
4541
4542
4543
4544
4545 public static Path getRegionDir(final Path rootdir, final HRegionInfo info) {
4546 return new Path(
4547 HTableDescriptor.getTableDir(rootdir, info.getTableName()),
4548 info.getEncodedName());
4549 }
4550
4551
4552
4553
4554
4555
4556
4557
4558
4559 public static boolean rowIsInRange(HRegionInfo info, final byte [] row) {
4560 return ((info.getStartKey().length == 0) ||
4561 (Bytes.compareTo(info.getStartKey(), row) <= 0)) &&
4562 ((info.getEndKey().length == 0) ||
4563 (Bytes.compareTo(info.getEndKey(), row) > 0));
4564 }
4565
4566
4567
4568
4569
4570
4571
4572
4573
4574
4575 public static void makeColumnFamilyDirs(FileSystem fs, Path tabledir,
4576 final HRegionInfo hri, byte [] colFamily)
4577 throws IOException {
4578 Path dir = Store.getStoreHomedir(tabledir, hri.getEncodedName(), colFamily);
4579 if (!HBaseFileSystem.makeDirOnFileSystem(fs, dir)) {
4580 LOG.warn("Failed to create " + dir);
4581 }
4582 }
4583
4584
4585
4586
4587
4588
4589
4590
4591
4592 public static HRegion mergeAdjacent(final HRegion srcA, final HRegion srcB)
4593 throws IOException {
4594 HRegion a = srcA;
4595 HRegion b = srcB;
4596
4597
4598
4599 if (srcA.getStartKey() == null) {
4600 if (srcB.getStartKey() == null) {
4601 throw new IOException("Cannot merge two regions with null start key");
4602 }
4603
4604 } else if ((srcB.getStartKey() == null) ||
4605 (Bytes.compareTo(srcA.getStartKey(), srcB.getStartKey()) > 0)) {
4606 a = srcB;
4607 b = srcA;
4608 }
4609
4610 if (!(Bytes.compareTo(a.getEndKey(), b.getStartKey()) == 0)) {
4611 throw new IOException("Cannot merge non-adjacent regions");
4612 }
4613 return merge(a, b);
4614 }
4615
4616
4617
4618
4619
4620
4621
4622
4623
4624 public static HRegion merge(HRegion a, HRegion b)
4625 throws IOException {
4626 if (!a.getRegionInfo().getTableNameAsString().equals(
4627 b.getRegionInfo().getTableNameAsString())) {
4628 throw new IOException("Regions do not belong to the same table");
4629 }
4630
4631 FileSystem fs = a.getFilesystem();
4632
4633
4634
4635 a.flushcache();
4636 b.flushcache();
4637
4638
4639
4640 a.compactStores(true);
4641 if (LOG.isDebugEnabled()) {
4642 LOG.debug("Files for region: " + a);
4643 listPaths(fs, a.getRegionDir());
4644 }
4645 b.compactStores(true);
4646 if (LOG.isDebugEnabled()) {
4647 LOG.debug("Files for region: " + b);
4648 listPaths(fs, b.getRegionDir());
4649 }
4650
4651 Configuration conf = a.getBaseConf();
4652 HTableDescriptor tabledesc = a.getTableDesc();
4653 HLog log = a.getLog();
4654 Path tableDir = a.getTableDir();
4655
4656
4657 final byte[] startKey =
4658 (a.comparator.matchingRows(a.getStartKey(), 0, a.getStartKey().length,
4659 HConstants.EMPTY_BYTE_ARRAY, 0, HConstants.EMPTY_BYTE_ARRAY.length)
4660 || b.comparator.matchingRows(b.getStartKey(), 0,
4661 b.getStartKey().length, HConstants.EMPTY_BYTE_ARRAY, 0,
4662 HConstants.EMPTY_BYTE_ARRAY.length))
4663 ? HConstants.EMPTY_BYTE_ARRAY
4664 : (a.comparator.compareRows(a.getStartKey(), 0, a.getStartKey().length,
4665 b.getStartKey(), 0, b.getStartKey().length) <= 0
4666 ? a.getStartKey()
4667 : b.getStartKey());
4668 final byte[] endKey =
4669 (a.comparator.matchingRows(a.getEndKey(), 0, a.getEndKey().length,
4670 HConstants.EMPTY_BYTE_ARRAY, 0, HConstants.EMPTY_BYTE_ARRAY.length)
4671 || a.comparator.matchingRows(b.getEndKey(), 0, b.getEndKey().length,
4672 HConstants.EMPTY_BYTE_ARRAY, 0,
4673 HConstants.EMPTY_BYTE_ARRAY.length))
4674 ? HConstants.EMPTY_BYTE_ARRAY
4675 : (a.comparator.compareRows(a.getEndKey(), 0, a.getEndKey().length,
4676 b.getEndKey(), 0, b.getEndKey().length) <= 0
4677 ? b.getEndKey()
4678 : a.getEndKey());
4679
4680 HRegionInfo newRegionInfo =
4681 new HRegionInfo(tabledesc.getName(), startKey, endKey);
4682 LOG.info("Creating new region " + newRegionInfo.toString());
4683 String encodedName = newRegionInfo.getEncodedName();
4684 Path newRegionDir = HRegion.getRegionDir(a.getTableDir(), encodedName);
4685 if(fs.exists(newRegionDir)) {
4686 throw new IOException("Cannot merge; target file collision at " +
4687 newRegionDir);
4688 }
4689 HBaseFileSystem.makeDirOnFileSystem(fs, newRegionDir);
4690
4691 LOG.info("starting merge of regions: " + a + " and " + b +
4692 " into new region " + newRegionInfo.toString() +
4693 " with start key <" + Bytes.toStringBinary(startKey) + "> and end key <" +
4694 Bytes.toStringBinary(endKey) + ">");
4695
4696
4697 Map<byte [], List<StoreFile>> byFamily =
4698 new TreeMap<byte [], List<StoreFile>>(Bytes.BYTES_COMPARATOR);
4699 byFamily = filesByFamily(byFamily, a.close());
4700 byFamily = filesByFamily(byFamily, b.close());
4701 for (Map.Entry<byte [], List<StoreFile>> es : byFamily.entrySet()) {
4702 byte [] colFamily = es.getKey();
4703 makeColumnFamilyDirs(fs, tableDir, newRegionInfo, colFamily);
4704
4705
4706 List<StoreFile> srcFiles = es.getValue();
4707 for (StoreFile hsf: srcFiles) {
4708 StoreFile.rename(fs, hsf.getPath(),
4709 StoreFile.getUniqueFile(fs, Store.getStoreHomedir(tableDir,
4710 newRegionInfo.getEncodedName(), colFamily)));
4711 }
4712 }
4713 if (LOG.isDebugEnabled()) {
4714 LOG.debug("Files for new region");
4715 listPaths(fs, newRegionDir);
4716 }
4717 HRegion dstRegion = HRegion.newHRegion(tableDir, log, fs, conf,
4718 newRegionInfo, a.getTableDesc(), null);
4719 long totalReadRequestCount = a.readRequestsCount.get() + b.readRequestsCount.get();
4720 dstRegion.readRequestsCount.set(totalReadRequestCount);
4721 dstRegion.opMetrics.setReadRequestCountMetrics(totalReadRequestCount);
4722
4723 long totalWriteRequestCount = a.writeRequestsCount.get() + b.writeRequestsCount.get();
4724 dstRegion.writeRequestsCount.set(totalWriteRequestCount);
4725 dstRegion.opMetrics.setWriteRequestCountMetrics(totalWriteRequestCount);
4726
4727 dstRegion.initialize();
4728 dstRegion.compactStores();
4729 if (LOG.isDebugEnabled()) {
4730 LOG.debug("Files for new region");
4731 listPaths(fs, dstRegion.getRegionDir());
4732 }
4733
4734
4735 HFileArchiver.archiveRegion(fs, FSUtils.getRootDir(a.getConf()),
4736 a.getTableDir(), a.getRegionDir());
4737
4738 HFileArchiver.archiveRegion(fs, FSUtils.getRootDir(b.getConf()),
4739 b.getTableDir(), b.getRegionDir());
4740
4741 LOG.info("merge completed. New region is " + dstRegion);
4742
4743 return dstRegion;
4744 }
4745
4746
4747
4748
4749
4750
4751
4752
4753 private static Map<byte [], List<StoreFile>> filesByFamily(
4754 Map<byte [], List<StoreFile>> byFamily, List<StoreFile> storeFiles) {
4755 for (StoreFile src: storeFiles) {
4756 byte [] family = src.getFamily();
4757 List<StoreFile> v = byFamily.get(family);
4758 if (v == null) {
4759 v = new ArrayList<StoreFile>();
4760 byFamily.put(family, v);
4761 }
4762 v.add(src);
4763 }
4764 return byFamily;
4765 }
4766
4767
4768
4769
4770
4771 boolean isMajorCompaction() throws IOException {
4772 for (Store store: this.stores.values()) {
4773 if (store.isMajorCompaction()) {
4774 return true;
4775 }
4776 }
4777 return false;
4778 }
4779
4780
4781
4782
4783
4784
4785
4786
4787 private static void listPaths(FileSystem fs, Path dir) throws IOException {
4788 if (LOG.isDebugEnabled()) {
4789 FileStatus[] stats = FSUtils.listStatus(fs, dir, null);
4790 if (stats == null || stats.length == 0) {
4791 return;
4792 }
4793 for (int i = 0; i < stats.length; i++) {
4794 String path = stats[i].getPath().toString();
4795 if (stats[i].isDir()) {
4796 LOG.debug("d " + path);
4797 listPaths(fs, stats[i].getPath());
4798 } else {
4799 LOG.debug("f " + path + " size=" + stats[i].getLen());
4800 }
4801 }
4802 }
4803 }
4804
4805
4806
4807
4808
4809
4810
4811
4812
4813
4814 public Result get(final Get get) throws IOException {
4815 return get(get, null);
4816 }
4817
4818
4819
4820
4821
4822
4823
4824
4825 public Result get(final Get get, final Integer lockid) throws IOException {
4826 checkRow(get.getRow(), "Get");
4827
4828 if (get.hasFamilies()) {
4829 for (byte [] family: get.familySet()) {
4830 checkFamily(family);
4831 }
4832 } else {
4833 for (byte[] family: this.htableDescriptor.getFamiliesKeys()) {
4834 get.addFamily(family);
4835 }
4836 }
4837 List<KeyValue> results = get(get, true);
4838 return new Result(results);
4839 }
4840
4841
4842
4843
4844
4845
4846 private List<KeyValue> get(Get get, boolean withCoprocessor)
4847 throws IOException {
4848 long now = EnvironmentEdgeManager.currentTimeMillis();
4849
4850 List<KeyValue> results = new ArrayList<KeyValue>();
4851
4852
4853 if (withCoprocessor && (coprocessorHost != null)) {
4854 if (coprocessorHost.preGet(get, results)) {
4855 return results;
4856 }
4857 }
4858
4859 Scan scan = new Scan(get);
4860
4861 RegionScanner scanner = null;
4862 try {
4863 scanner = getScanner(scan);
4864 scanner.next(results, SchemaMetrics.METRIC_GETSIZE);
4865 } finally {
4866 if (scanner != null)
4867 scanner.close();
4868 }
4869
4870
4871 if (withCoprocessor && (coprocessorHost != null)) {
4872 coprocessorHost.postGet(get, results);
4873 }
4874
4875
4876 final long after = EnvironmentEdgeManager.currentTimeMillis();
4877 this.opMetrics.updateGetMetrics(get.familySet(), after - now);
4878
4879 return results;
4880 }
4881
4882 public void mutateRow(RowMutations rm) throws IOException {
4883 mutateRowsWithLocks(rm.getMutations(), Collections.singleton(rm.getRow()));
4884 }
4885
4886
4887
4888
4889
4890
4891
4892
4893
4894
4895
4896 public void mutateRowsWithLocks(Collection<Mutation> mutations,
4897 Collection<byte[]> rowsToLock) throws IOException {
4898 boolean flush = false;
4899
4900 checkReadOnly();
4901 checkResources();
4902
4903 startRegionOperation();
4904 List<Integer> acquiredLocks = null;
4905 try {
4906
4907
4908
4909
4910 WALEdit walEdit = new WALEdit();
4911 if (coprocessorHost != null) {
4912 for (Mutation m : mutations) {
4913 if (m instanceof Put) {
4914 if (coprocessorHost.prePut((Put) m, walEdit, m.getWriteToWAL())) {
4915
4916 return;
4917 }
4918 } else if (m instanceof Delete) {
4919 Delete d = (Delete) m;
4920 prepareDelete(d);
4921 if (coprocessorHost.preDelete(d, walEdit, d.getWriteToWAL())) {
4922
4923 return;
4924 }
4925 }
4926 }
4927 }
4928
4929 long txid = 0;
4930 boolean walSyncSuccessful = false;
4931 boolean memstoreUpdated = false;
4932 boolean locked = false;
4933
4934
4935 acquiredLocks = new ArrayList<Integer>(rowsToLock.size());
4936 for (byte[] row : rowsToLock) {
4937
4938 Integer lid = getLock(null, row, true);
4939 if (lid == null) {
4940 throw new IOException("Failed to acquire lock on "
4941 + Bytes.toStringBinary(row));
4942 }
4943 acquiredLocks.add(lid);
4944 }
4945
4946
4947 lock(this.updatesLock.readLock(), acquiredLocks.size());
4948 locked = true;
4949
4950
4951 MultiVersionConsistencyControl.WriteEntry w = mvcc.beginMemstoreInsert();
4952
4953 long now = EnvironmentEdgeManager.currentTimeMillis();
4954 byte[] byteNow = Bytes.toBytes(now);
4955 Durability durability = Durability.USE_DEFAULT;
4956 try {
4957
4958 for (Mutation m : mutations) {
4959 if (m instanceof Put) {
4960 Map<byte[], List<KeyValue>> familyMap = m.getFamilyMap();
4961 checkFamilies(familyMap.keySet());
4962 checkTimestamps(familyMap, now);
4963 updateKVTimestamps(familyMap.values(), byteNow);
4964 } else if (m instanceof Delete) {
4965 Delete d = (Delete) m;
4966 prepareDelete(d);
4967 prepareDeleteTimestamps(d.getFamilyMap(), byteNow);
4968 } else {
4969 throw new DoNotRetryIOException(
4970 "Action must be Put or Delete. But was: "
4971 + m.getClass().getName());
4972 }
4973 Durability tmpDur = m.getDurability();
4974 if (tmpDur.ordinal() > durability.ordinal()) {
4975 durability = tmpDur;
4976 }
4977 if (tmpDur != Durability.SKIP_WAL) {
4978 addFamilyMapToWALEdit(m.getFamilyMap(), walEdit);
4979 }
4980 }
4981
4982
4983 if (walEdit.size() > 0) {
4984 txid = this.log.appendNoSync(regionInfo,
4985 this.htableDescriptor.getName(), walEdit,
4986 HConstants.DEFAULT_CLUSTER_ID, now, this.htableDescriptor);
4987 }
4988
4989
4990 long addedSize = 0;
4991 memstoreUpdated = true;
4992 for (Mutation m : mutations) {
4993 addedSize += applyFamilyMapToMemstore(m.getFamilyMap(), w);
4994 }
4995 flush = isFlushSize(this.addAndGetGlobalMemstoreSize(addedSize));
4996
4997
4998 this.updatesLock.readLock().unlock();
4999 locked = false;
5000 if (acquiredLocks != null) {
5001 for (Integer lid : acquiredLocks) {
5002 releaseRowLock(lid);
5003 }
5004 acquiredLocks = null;
5005 }
5006
5007
5008 if (walEdit.size() > 0) {
5009 syncOrDefer(txid, durability);
5010 }
5011 walSyncSuccessful = true;
5012
5013
5014 mvcc.completeMemstoreInsert(w);
5015 w = null;
5016
5017
5018
5019
5020 if (coprocessorHost != null) {
5021 for (Mutation m : mutations) {
5022 if (m instanceof Put) {
5023 coprocessorHost.postPut((Put) m, walEdit, m.getWriteToWAL());
5024 } else if (m instanceof Delete) {
5025 coprocessorHost.postDelete((Delete) m, walEdit, m.getWriteToWAL());
5026 }
5027 }
5028 }
5029 } finally {
5030
5031 if (memstoreUpdated && !walSyncSuccessful) {
5032 int kvsRolledback = 0;
5033 for (Mutation m : mutations) {
5034 for (Map.Entry<byte[], List<KeyValue>> e : m.getFamilyMap()
5035 .entrySet()) {
5036 List<KeyValue> kvs = e.getValue();
5037 byte[] family = e.getKey();
5038 Store store = getStore(family);
5039
5040 for (KeyValue kv : kvs) {
5041 store.rollback(kv);
5042 kvsRolledback++;
5043 }
5044 }
5045 }
5046 LOG.info("mutateRowWithLocks: rolled back " + kvsRolledback
5047 + " KeyValues");
5048 }
5049
5050 if (w != null) {
5051 mvcc.completeMemstoreInsert(w);
5052 }
5053
5054 if (locked) {
5055 this.updatesLock.readLock().unlock();
5056 }
5057
5058 if (acquiredLocks != null) {
5059 for (Integer lid : acquiredLocks) {
5060 releaseRowLock(lid);
5061 }
5062 }
5063 }
5064 } finally {
5065 if (flush) {
5066
5067 requestFlush();
5068 }
5069 closeRegionOperation();
5070 }
5071 }
5072
5073
5074
5075
5076
5077
5078
5079
5080
5081
5082
5083
5084
5085
5086
5087
5088 public Result append(Append append, boolean writeToWAL)
5089 throws IOException {
5090 return append(append, null, writeToWAL);
5091 }
5092
5093
5094
5095
5096
5097
5098
5099
5100
5101
5102
5103
5104
5105
5106 public Result append(Append append, Integer lockid, boolean writeToWAL)
5107 throws IOException {
5108
5109 byte[] row = append.getRow();
5110 checkRow(row, "append");
5111 boolean flush = false;
5112 WALEdit walEdits = null;
5113 List<KeyValue> allKVs = new ArrayList<KeyValue>(append.size());
5114 Map<Store, List<KeyValue>> tempMemstore = new HashMap<Store, List<KeyValue>>();
5115 long before = EnvironmentEdgeManager.currentTimeMillis();
5116 long size = 0;
5117 long txid = 0;
5118
5119 checkReadOnly();
5120
5121 startRegionOperation();
5122 this.writeRequestsCount.increment();
5123 this.opMetrics.setWriteRequestCountMetrics(this.writeRequestsCount.get());
5124 try {
5125 Integer lid = getLock(lockid, row, true);
5126 lock(this.updatesLock.readLock());
5127 try {
5128 long now = EnvironmentEdgeManager.currentTimeMillis();
5129
5130 for (Map.Entry<byte[], List<KeyValue>> family : append.getFamilyMap()
5131 .entrySet()) {
5132
5133 Store store = stores.get(family.getKey());
5134 Collections.sort(family.getValue(), store.getComparator());
5135 List<KeyValue> kvs = new ArrayList<KeyValue>(family.getValue().size());
5136
5137
5138 Get get = new Get(row);
5139 for (KeyValue kv : family.getValue()) {
5140 get.addColumn(family.getKey(), kv.getQualifier());
5141 }
5142 List<KeyValue> results = get(get, false);
5143
5144
5145
5146
5147
5148
5149
5150 int idx = 0;
5151 for (KeyValue kv : family.getValue()) {
5152 KeyValue newKV;
5153 if (idx < results.size()
5154 && results.get(idx).matchingQualifier(kv.getBuffer(),
5155 kv.getQualifierOffset(), kv.getQualifierLength())) {
5156 KeyValue oldKv = results.get(idx);
5157
5158 newKV = new KeyValue(row.length, kv.getFamilyLength(),
5159 kv.getQualifierLength(), now, KeyValue.Type.Put,
5160 oldKv.getValueLength() + kv.getValueLength());
5161
5162 System.arraycopy(oldKv.getBuffer(), oldKv.getValueOffset(),
5163 newKV.getBuffer(), newKV.getValueOffset(),
5164 oldKv.getValueLength());
5165 System.arraycopy(kv.getBuffer(), kv.getValueOffset(),
5166 newKV.getBuffer(),
5167 newKV.getValueOffset() + oldKv.getValueLength(),
5168 kv.getValueLength());
5169 idx++;
5170 } else {
5171
5172 newKV = new KeyValue(row.length, kv.getFamilyLength(),
5173 kv.getQualifierLength(), now, KeyValue.Type.Put,
5174 kv.getValueLength());
5175
5176 System.arraycopy(kv.getBuffer(), kv.getValueOffset(),
5177 newKV.getBuffer(), newKV.getValueOffset(),
5178 kv.getValueLength());
5179 }
5180
5181 System.arraycopy(kv.getBuffer(), kv.getRowOffset(),
5182 newKV.getBuffer(), newKV.getRowOffset(), kv.getRowLength());
5183 System.arraycopy(kv.getBuffer(), kv.getFamilyOffset(),
5184 newKV.getBuffer(), newKV.getFamilyOffset(),
5185 kv.getFamilyLength());
5186 System.arraycopy(kv.getBuffer(), kv.getQualifierOffset(),
5187 newKV.getBuffer(), newKV.getQualifierOffset(),
5188 kv.getQualifierLength());
5189
5190 kvs.add(newKV);
5191
5192
5193 if (writeToWAL) {
5194 if (walEdits == null) {
5195 walEdits = new WALEdit();
5196 }
5197 walEdits.add(newKV);
5198 }
5199 }
5200
5201
5202 tempMemstore.put(store, kvs);
5203 }
5204
5205
5206 if (writeToWAL) {
5207
5208
5209
5210 txid = this.log.appendNoSync(regionInfo,
5211 this.htableDescriptor.getName(), walEdits,
5212 HConstants.DEFAULT_CLUSTER_ID, EnvironmentEdgeManager.currentTimeMillis(),
5213 this.htableDescriptor);
5214 }
5215
5216 for (Map.Entry<Store, List<KeyValue>> entry : tempMemstore.entrySet()) {
5217 Store store = entry.getKey();
5218 size += store.upsert(entry.getValue());
5219 allKVs.addAll(entry.getValue());
5220 }
5221 size = this.addAndGetGlobalMemstoreSize(size);
5222 flush = isFlushSize(size);
5223 } finally {
5224 this.updatesLock.readLock().unlock();
5225 releaseRowLock(lid);
5226 }
5227 if (writeToWAL) {
5228
5229 syncOrDefer(txid, append.getDurability());
5230 }
5231 } finally {
5232 closeRegionOperation();
5233 }
5234
5235
5236 long after = EnvironmentEdgeManager.currentTimeMillis();
5237 this.opMetrics.updateAppendMetrics(append.getFamilyMap().keySet(), after - before);
5238
5239 if (flush) {
5240
5241 requestFlush();
5242 }
5243
5244 return append.isReturnResults() ? new Result(allKVs) : null;
5245 }
5246
5247
5248
5249
5250
5251
5252
5253
5254
5255
5256
5257
5258 public Result increment(Increment increment, boolean writeToWAL)
5259 throws IOException {
5260 return increment(increment, null, writeToWAL);
5261 }
5262
5263
5264
5265
5266
5267
5268
5269
5270
5271
5272
5273
5274
5275
5276
5277 public Result increment(Increment increment, Integer lockid,
5278 boolean writeToWAL)
5279 throws IOException {
5280
5281 byte [] row = increment.getRow();
5282 checkRow(row, "increment");
5283 TimeRange tr = increment.getTimeRange();
5284 boolean flush = false;
5285 WALEdit walEdits = null;
5286 List<KeyValue> allKVs = new ArrayList<KeyValue>(increment.numColumns());
5287 Map<Store, List<KeyValue>> tempMemstore = new HashMap<Store, List<KeyValue>>();
5288 long before = EnvironmentEdgeManager.currentTimeMillis();
5289 long size = 0;
5290 long txid = 0;
5291
5292 checkReadOnly();
5293
5294 startRegionOperation();
5295 this.writeRequestsCount.increment();
5296 this.opMetrics.setWriteRequestCountMetrics(this.writeRequestsCount.get());
5297 try {
5298 Integer lid = getLock(lockid, row, true);
5299 lock(this.updatesLock.readLock());
5300 try {
5301 long now = EnvironmentEdgeManager.currentTimeMillis();
5302
5303 for (Map.Entry<byte [], NavigableMap<byte [], Long>> family :
5304 increment.getFamilyMap().entrySet()) {
5305
5306 Store store = stores.get(family.getKey());
5307 List<KeyValue> kvs = new ArrayList<KeyValue>(family.getValue().size());
5308
5309
5310 Get get = new Get(row);
5311 for (Map.Entry<byte [], Long> column : family.getValue().entrySet()) {
5312 get.addColumn(family.getKey(), column.getKey());
5313 }
5314 get.setTimeRange(tr.getMin(), tr.getMax());
5315 List<KeyValue> results = get(get, false);
5316
5317
5318
5319 int idx = 0;
5320 for (Map.Entry<byte [], Long> column : family.getValue().entrySet()) {
5321 long amount = column.getValue();
5322 if (idx < results.size() &&
5323 results.get(idx).matchingQualifier(column.getKey())) {
5324 KeyValue kv = results.get(idx);
5325 if(kv.getValueLength() == Bytes.SIZEOF_LONG) {
5326 amount += Bytes.toLong(kv.getBuffer(), kv.getValueOffset(), Bytes.SIZEOF_LONG);
5327 } else {
5328
5329 throw new DoNotRetryIOException(
5330 "Attempted to increment field that isn't 64 bits wide");
5331 }
5332 idx++;
5333 }
5334
5335
5336 KeyValue newKV = new KeyValue(row, family.getKey(), column.getKey(),
5337 now, Bytes.toBytes(amount));
5338 kvs.add(newKV);
5339
5340
5341 if (writeToWAL) {
5342 if (walEdits == null) {
5343 walEdits = new WALEdit();
5344 }
5345 walEdits.add(newKV);
5346 }
5347 }
5348
5349
5350 tempMemstore.put(store, kvs);
5351 }
5352
5353
5354 if (writeToWAL) {
5355
5356
5357
5358 txid = this.log.appendNoSync(regionInfo, this.htableDescriptor.getName(),
5359 walEdits, HConstants.DEFAULT_CLUSTER_ID, EnvironmentEdgeManager.currentTimeMillis(),
5360 this.htableDescriptor);
5361 }
5362
5363
5364 for (Map.Entry<Store, List<KeyValue>> entry : tempMemstore.entrySet()) {
5365 Store store = entry.getKey();
5366 size += store.upsert(entry.getValue());
5367 allKVs.addAll(entry.getValue());
5368 }
5369 size = this.addAndGetGlobalMemstoreSize(size);
5370 flush = isFlushSize(size);
5371 } finally {
5372 this.updatesLock.readLock().unlock();
5373 releaseRowLock(lid);
5374 }
5375 if (writeToWAL) {
5376
5377 syncOrDefer(txid, Durability.USE_DEFAULT);
5378 }
5379 } finally {
5380 closeRegionOperation();
5381 long after = EnvironmentEdgeManager.currentTimeMillis();
5382 this.opMetrics.updateIncrementMetrics(increment.getFamilyMap().keySet(), after - before);
5383 }
5384
5385 if (flush) {
5386
5387 requestFlush();
5388 }
5389
5390 return new Result(allKVs);
5391 }
5392
5393
5394
5395
5396
5397
5398
5399
5400
5401
5402 public long incrementColumnValue(byte [] row, byte [] family,
5403 byte [] qualifier, long amount, boolean writeToWAL)
5404 throws IOException {
5405
5406 long before = EnvironmentEdgeManager.currentTimeMillis();
5407
5408 checkRow(row, "increment");
5409 boolean flush = false;
5410 boolean wrongLength = false;
5411 long txid = 0;
5412
5413 long result = amount;
5414 startRegionOperation();
5415 this.writeRequestsCount.increment();
5416 this.opMetrics.setWriteRequestCountMetrics(this.writeRequestsCount.get());
5417 try {
5418 Integer lid = obtainRowLock(row);
5419 lock(this.updatesLock.readLock());
5420 try {
5421 Store store = stores.get(family);
5422
5423
5424 Get get = new Get(row);
5425 get.addColumn(family, qualifier);
5426
5427
5428
5429 List<KeyValue> results = get(get, false);
5430
5431 if (!results.isEmpty()) {
5432 KeyValue kv = results.get(0);
5433 if(kv.getValueLength() == Bytes.SIZEOF_LONG){
5434 byte [] buffer = kv.getBuffer();
5435 int valueOffset = kv.getValueOffset();
5436 result += Bytes.toLong(buffer, valueOffset, Bytes.SIZEOF_LONG);
5437 }
5438 else{
5439 wrongLength = true;
5440 }
5441 }
5442 if(!wrongLength){
5443
5444 KeyValue newKv = new KeyValue(row, family,
5445 qualifier, EnvironmentEdgeManager.currentTimeMillis(),
5446 Bytes.toBytes(result));
5447
5448
5449 if (writeToWAL) {
5450 long now = EnvironmentEdgeManager.currentTimeMillis();
5451 WALEdit walEdit = new WALEdit();
5452 walEdit.add(newKv);
5453
5454
5455
5456 txid = this.log.appendNoSync(regionInfo, this.htableDescriptor.getName(),
5457 walEdit, HConstants.DEFAULT_CLUSTER_ID, now,
5458 this.htableDescriptor);
5459 }
5460
5461
5462
5463
5464 long size = store.updateColumnValue(row, family, qualifier, result);
5465
5466 size = this.addAndGetGlobalMemstoreSize(size);
5467 flush = isFlushSize(size);
5468 }
5469 } finally {
5470 this.updatesLock.readLock().unlock();
5471 releaseRowLock(lid);
5472 }
5473 if (writeToWAL) {
5474
5475 syncOrDefer(txid, Durability.USE_DEFAULT);
5476 }
5477 } finally {
5478 closeRegionOperation();
5479 }
5480
5481
5482 long after = EnvironmentEdgeManager.currentTimeMillis();
5483 this.opMetrics.updateIncrementColumnValueMetrics(family, after - before);
5484
5485 if (flush) {
5486
5487 requestFlush();
5488 }
5489 if(wrongLength){
5490 throw new DoNotRetryIOException(
5491 "Attempted to increment field that isn't 64 bits wide");
5492 }
5493 return result;
5494 }
5495
5496
5497
5498
5499
5500
5501 private void checkFamily(final byte [] family)
5502 throws NoSuchColumnFamilyException {
5503 if (!this.htableDescriptor.hasFamily(family)) {
5504 throw new NoSuchColumnFamilyException("Column family " +
5505 Bytes.toString(family) + " does not exist in region " + this
5506 + " in table " + this.htableDescriptor);
5507 }
5508 }
5509
5510 public static final long FIXED_OVERHEAD = ClassSize.align(
5511 ClassSize.OBJECT +
5512 ClassSize.ARRAY +
5513 36 * ClassSize.REFERENCE + 2 * Bytes.SIZEOF_INT +
5514 (8 * Bytes.SIZEOF_LONG) +
5515 Bytes.SIZEOF_BOOLEAN);
5516
5517 public static final long DEEP_OVERHEAD = FIXED_OVERHEAD +
5518 ClassSize.OBJECT +
5519 (2 * ClassSize.ATOMIC_BOOLEAN) +
5520 (3 * ClassSize.ATOMIC_LONG) +
5521 ClassSize.ATOMIC_INTEGER +
5522 (3 * ClassSize.CONCURRENT_HASHMAP) +
5523 WriteState.HEAP_SIZE +
5524 ClassSize.CONCURRENT_SKIPLISTMAP + ClassSize.CONCURRENT_SKIPLISTMAP_ENTRY +
5525 (2 * ClassSize.REENTRANT_LOCK) +
5526 ClassSize.ARRAYLIST +
5527 MultiVersionConsistencyControl.FIXED_SIZE
5528 ;
5529
5530 @Override
5531 public long heapSize() {
5532 long heapSize = DEEP_OVERHEAD;
5533 for(Store store : this.stores.values()) {
5534 heapSize += store.heapSize();
5535 }
5536
5537 return heapSize;
5538 }
5539
5540
5541
5542
5543
5544 private static void printUsageAndExit(final String message) {
5545 if (message != null && message.length() > 0) System.out.println(message);
5546 System.out.println("Usage: HRegion CATLALOG_TABLE_DIR [major_compact]");
5547 System.out.println("Options:");
5548 System.out.println(" major_compact Pass this option to major compact " +
5549 "passed region.");
5550 System.out.println("Default outputs scan of passed region.");
5551 System.exit(1);
5552 }
5553
5554
5555
5556
5557
5558
5559
5560
5561
5562
5563
5564
5565
5566
5567
5568
5569
5570
5571 public <T extends CoprocessorProtocol> boolean registerProtocol(
5572 Class<T> protocol, T handler) {
5573
5574
5575
5576
5577 if (protocolHandlers.containsKey(protocol)) {
5578 LOG.error("Protocol "+protocol.getName()+
5579 " already registered, rejecting request from "+
5580 handler
5581 );
5582 return false;
5583 }
5584
5585 protocolHandlers.putInstance(protocol, handler);
5586 protocolHandlerNames.put(protocol.getName(), protocol);
5587 if (LOG.isDebugEnabled()) {
5588 LOG.debug("Registered protocol handler: region="+
5589 Bytes.toStringBinary(getRegionName())+" protocol="+protocol.getName());
5590 }
5591 return true;
5592 }
5593
5594
5595
5596
5597
5598
5599
5600
5601
5602
5603
5604
5605
5606
5607
5608
5609 public ExecResult exec(Exec call)
5610 throws IOException {
5611 Class<? extends CoprocessorProtocol> protocol = call.getProtocol();
5612 if (protocol == null) {
5613 String protocolName = call.getProtocolName();
5614 if (LOG.isTraceEnabled()) {
5615 LOG.trace("Received dynamic protocol exec call with protocolName " + protocolName);
5616 }
5617
5618 protocol = protocolHandlerNames.get(protocolName);
5619 if (protocol == null) {
5620 throw new HBaseRPC.UnknownProtocolException(protocol,
5621 "No matching handler for protocol "+protocolName+
5622 " in region "+Bytes.toStringBinary(getRegionName()));
5623 }
5624 }
5625 if (!protocolHandlers.containsKey(protocol)) {
5626 throw new HBaseRPC.UnknownProtocolException(protocol,
5627 "No matching handler for protocol "+protocol.getName()+
5628 " in region "+Bytes.toStringBinary(getRegionName()));
5629 }
5630
5631 CoprocessorProtocol handler = protocolHandlers.getInstance(protocol);
5632 Object value;
5633
5634 try {
5635 Method method = protocol.getMethod(
5636 call.getMethodName(), call.getParameterClasses());
5637 method.setAccessible(true);
5638
5639 value = method.invoke(handler, call.getParameters());
5640 } catch (InvocationTargetException e) {
5641 Throwable target = e.getTargetException();
5642 if (target instanceof IOException) {
5643 throw (IOException)target;
5644 }
5645 IOException ioe = new IOException(target.toString());
5646 ioe.setStackTrace(target.getStackTrace());
5647 throw ioe;
5648 } catch (Throwable e) {
5649 if (!(e instanceof IOException)) {
5650 LOG.error("Unexpected throwable object ", e);
5651 }
5652 IOException ioe = new IOException(e.toString());
5653 ioe.setStackTrace(e.getStackTrace());
5654 throw ioe;
5655 }
5656
5657 return new ExecResult(getRegionName(), value);
5658 }
5659
5660
5661
5662
5663
5664
5665
5666
5667
5668
5669
5670 private static void processTable(final FileSystem fs, final Path p,
5671 final HLog log, final Configuration c,
5672 final boolean majorCompact)
5673 throws IOException {
5674 HRegion region = null;
5675 String rootStr = Bytes.toString(HConstants.ROOT_TABLE_NAME);
5676 String metaStr = Bytes.toString(HConstants.META_TABLE_NAME);
5677
5678 if (p.getName().startsWith(rootStr)) {
5679 region = HRegion.newHRegion(p, log, fs, c, HRegionInfo.ROOT_REGIONINFO,
5680 HTableDescriptor.ROOT_TABLEDESC, null);
5681 } else if (p.getName().startsWith(metaStr)) {
5682 region = HRegion.newHRegion(p, log, fs, c,
5683 HRegionInfo.FIRST_META_REGIONINFO, HTableDescriptor.META_TABLEDESC, null);
5684 } else {
5685 throw new IOException("Not a known catalog table: " + p.toString());
5686 }
5687 try {
5688 region.initialize();
5689 if (majorCompact) {
5690 region.compactStores(true);
5691 } else {
5692
5693 Scan scan = new Scan();
5694
5695 RegionScanner scanner = region.getScanner(scan);
5696 try {
5697 List<KeyValue> kvs = new ArrayList<KeyValue>();
5698 boolean done = false;
5699 do {
5700 kvs.clear();
5701 done = scanner.next(kvs);
5702 if (kvs.size() > 0) LOG.info(kvs);
5703 } while (done);
5704 } finally {
5705 scanner.close();
5706 }
5707 }
5708 } finally {
5709 region.close();
5710 }
5711 }
5712
5713 boolean shouldForceSplit() {
5714 return this.splitRequest;
5715 }
5716
5717 byte[] getExplicitSplitPoint() {
5718 return this.explicitSplitPoint;
5719 }
5720
5721 void forceSplit(byte[] sp) {
5722
5723
5724 this.splitRequest = true;
5725 if (sp != null) {
5726 this.explicitSplitPoint = sp;
5727 }
5728 }
5729
5730 void clearSplit_TESTS_ONLY() {
5731 this.splitRequest = false;
5732 }
5733
5734
5735
5736
5737 protected void prepareToSplit() {
5738
5739 }
5740
5741
5742
5743
5744
5745
5746
5747 public byte[] checkSplit() {
5748
5749 if (this.regionInfo.isMetaTable()) {
5750 if (shouldForceSplit()) {
5751 LOG.warn("Cannot split root/meta regions in HBase 0.20 and above");
5752 }
5753 return null;
5754 }
5755
5756 if (!splitPolicy.shouldSplit()) {
5757 return null;
5758 }
5759
5760 byte[] ret = splitPolicy.getSplitPoint();
5761
5762 if (ret != null) {
5763 try {
5764 checkRow(ret, "calculated split");
5765 } catch (IOException e) {
5766 LOG.error("Ignoring invalid split", e);
5767 return null;
5768 }
5769 }
5770 return ret;
5771 }
5772
5773
5774
5775
5776 public int getCompactPriority() {
5777 int count = Integer.MAX_VALUE;
5778 for(Store store : stores.values()) {
5779 count = Math.min(count, store.getCompactPriority());
5780 }
5781 return count;
5782 }
5783
5784
5785
5786
5787
5788
5789 public boolean needsCompaction() {
5790 for(Store store : stores.values()) {
5791 if(store.needsCompaction()) {
5792 return true;
5793 }
5794 }
5795 return false;
5796 }
5797
5798
5799 public RegionCoprocessorHost getCoprocessorHost() {
5800 return coprocessorHost;
5801 }
5802
5803
5804
5805
5806
5807 public void setOpMetricsReadRequestCount(long value)
5808 {
5809 this.opMetrics.setReadRequestCountMetrics(value);
5810 }
5811
5812
5813
5814
5815
5816 public void setOpMetricsWriteRequestCount(long value)
5817 {
5818 this.opMetrics.setWriteRequestCountMetrics(value);
5819 }
5820
5821
5822 public void setCoprocessorHost(final RegionCoprocessorHost coprocessorHost) {
5823 this.coprocessorHost = coprocessorHost;
5824 }
5825
5826
5827
5828
5829
5830
5831
5832
5833
5834
5835 public void startRegionOperation()
5836 throws NotServingRegionException, RegionTooBusyException, InterruptedIOException {
5837 if (this.closing.get()) {
5838 throw new NotServingRegionException(regionInfo.getRegionNameAsString() +
5839 " is closing");
5840 }
5841 lock(lock.readLock());
5842 if (this.closed.get()) {
5843 lock.readLock().unlock();
5844 throw new NotServingRegionException(regionInfo.getRegionNameAsString() +
5845 " is closed");
5846 }
5847 }
5848
5849
5850
5851
5852
5853 public void closeRegionOperation(){
5854 lock.readLock().unlock();
5855 }
5856
5857
5858
5859
5860
5861
5862
5863
5864
5865
5866 private void startBulkRegionOperation(boolean writeLockNeeded)
5867 throws NotServingRegionException, RegionTooBusyException, InterruptedIOException {
5868 if (this.closing.get()) {
5869 throw new NotServingRegionException(regionInfo.getRegionNameAsString() +
5870 " is closing");
5871 }
5872 if (writeLockNeeded) lock(lock.writeLock());
5873 else lock(lock.readLock());
5874 if (this.closed.get()) {
5875 if (writeLockNeeded) lock.writeLock().unlock();
5876 else lock.readLock().unlock();
5877 throw new NotServingRegionException(regionInfo.getRegionNameAsString() +
5878 " is closed");
5879 }
5880 }
5881
5882
5883
5884
5885
5886 private void closeBulkRegionOperation() {
5887 if (lock.writeLock().isHeldByCurrentThread()) lock.writeLock().unlock();
5888 else lock.readLock().unlock();
5889 }
5890
5891
5892
5893
5894
5895 private void recordPutWithoutWal(final Map<byte [], List<KeyValue>> familyMap) {
5896 if (numPutsWithoutWAL.getAndIncrement() == 0) {
5897 LOG.info("writing data to region " + this +
5898 " with WAL disabled. Data may be lost in the event of a crash.");
5899 }
5900
5901 long putSize = 0;
5902 for (List<KeyValue> edits : familyMap.values()) {
5903 for (KeyValue kv : edits) {
5904 putSize += kv.getKeyLength() + kv.getValueLength();
5905 }
5906 }
5907
5908 dataInMemoryWithoutWAL.addAndGet(putSize);
5909 }
5910
5911 private void lock(final Lock lock)
5912 throws RegionTooBusyException, InterruptedIOException {
5913 lock(lock, 1);
5914 }
5915
5916
5917
5918
5919
5920
5921 private void lock(final Lock lock, final int multiplier)
5922 throws RegionTooBusyException, InterruptedIOException {
5923 try {
5924 final long waitTime = Math.min(maxBusyWaitDuration,
5925 busyWaitDuration * Math.min(multiplier, maxBusyWaitMultiplier));
5926 if (!lock.tryLock(waitTime, TimeUnit.MILLISECONDS)) {
5927 throw new RegionTooBusyException(
5928 "failed to get a lock in " + waitTime + " ms. " +
5929 "regionName=" + (this.getRegionInfo() == null ? "unknown" :
5930 this.getRegionInfo().getRegionNameAsString()) +
5931 ", server=" + (this.getRegionServerServices() == null ? "unknown" :
5932 this.getRegionServerServices().getServerName()));
5933 }
5934 } catch (InterruptedException ie) {
5935 LOG.info("Interrupted while waiting for a lock");
5936 InterruptedIOException iie = new InterruptedIOException();
5937 iie.initCause(ie);
5938 throw iie;
5939 }
5940 }
5941
5942
5943
5944
5945
5946
5947
5948 private void syncOrDefer(long txid, Durability durability) throws IOException {
5949 if (this.getRegionInfo().isMetaRegion()) {
5950 this.log.sync(txid);
5951 } else {
5952 switch(durability) {
5953 case USE_DEFAULT:
5954
5955 if (!isDeferredLogSyncEnabled()) {
5956 this.log.sync(txid);
5957 }
5958 break;
5959 case SKIP_WAL:
5960
5961 break;
5962 case ASYNC_WAL:
5963
5964 if (this.deferredLogSyncDisabled) {
5965 this.log.sync(txid);
5966 }
5967 break;
5968 case SYNC_WAL:
5969 case FSYNC_WAL:
5970
5971 this.log.sync(txid);
5972 break;
5973 }
5974 }
5975 }
5976
5977
5978
5979
5980 private boolean isDeferredLogSyncEnabled() {
5981 return (this.htableDescriptor.isDeferredLogFlush() && !this.deferredLogSyncDisabled);
5982 }
5983
5984
5985
5986
5987 private static final List<KeyValue> MOCKED_LIST = new AbstractList<KeyValue>() {
5988
5989 @Override
5990 public void add(int index, KeyValue element) {
5991
5992 }
5993
5994 @Override
5995 public boolean addAll(int index, Collection<? extends KeyValue> c) {
5996 return false;
5997 }
5998
5999 @Override
6000 public KeyValue get(int index) {
6001 throw new UnsupportedOperationException();
6002 }
6003
6004 @Override
6005 public int size() {
6006 return 0;
6007 }
6008 };
6009
6010
6011
6012
6013
6014
6015
6016
6017
6018
6019
6020 public static void main(String[] args) throws IOException {
6021 if (args.length < 1) {
6022 printUsageAndExit(null);
6023 }
6024 boolean majorCompact = false;
6025 if (args.length > 1) {
6026 if (!args[1].toLowerCase().startsWith("major")) {
6027 printUsageAndExit("ERROR: Unrecognized option <" + args[1] + ">");
6028 }
6029 majorCompact = true;
6030 }
6031 final Path tableDir = new Path(args[0]);
6032 final Configuration c = HBaseConfiguration.create();
6033 final FileSystem fs = FileSystem.get(c);
6034 final Path logdir = new Path(c.get("hbase.tmp.dir"),
6035 "hlog" + tableDir.getName()
6036 + EnvironmentEdgeManager.currentTimeMillis());
6037 final Path oldLogDir = new Path(c.get("hbase.tmp.dir"),
6038 HConstants.HREGION_OLDLOGDIR_NAME);
6039 final HLog log = new HLog(fs, logdir, oldLogDir, c);
6040 try {
6041 processTable(fs, tableDir, log, c, majorCompact);
6042 } finally {
6043 log.close();
6044
6045 BlockCache bc = new CacheConfig(c).getBlockCache();
6046 if (bc != null) bc.shutdown();
6047 }
6048 }
6049
6050
6051
6052
6053
6054
6055 public static interface BulkLoadListener {
6056
6057
6058
6059
6060
6061
6062
6063
6064 String prepareBulkLoad(byte[] family, String srcPath) throws IOException;
6065
6066
6067
6068
6069
6070
6071
6072 void doneBulkLoad(byte[] family, String srcPath) throws IOException;
6073
6074
6075
6076
6077
6078
6079
6080 void failedBulkLoad(byte[] family, String srcPath) throws IOException;
6081
6082 }
6083 }