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