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