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