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