diff --git a/hbase-client/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java b/hbase-client/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java index c62ab1d7000..c9807c3528a 100644 --- a/hbase-client/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java +++ b/hbase-client/src/main/java/org/apache/hadoop/hbase/HTableDescriptor.java @@ -562,12 +562,30 @@ public class HTableDescriptor implements TableDescriptor, Comparable regions = + Iterator regions = node.getRegions().iterator(); StorageClusterStatusModel.Node.Region region = regions.next(); assertTrue(Bytes.toString(region.getName()).equals( @@ -106,7 +106,7 @@ public class TestStorageClusterStatusModel extends TestModelBase <% TraditionalBinaryPrefix.long2String(sl.getMaxHeapMB() * TraditionalBinaryPrefix.MEGA.value, "B", 1) %> - <% TraditionalBinaryPrefix.long2String(sl.getMemstoreSizeMB() + <% TraditionalBinaryPrefix.long2String(sl.getMemStoreSizeMB() * TraditionalBinaryPrefix.MEGA.value, "B", 1) %> diff --git a/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/RegionListTmpl.jamon b/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/RegionListTmpl.jamon index cf0e8ad1466..d16ce06fbd0 100644 --- a/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/RegionListTmpl.jamon +++ b/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/RegionListTmpl.jamon @@ -255,7 +255,7 @@ <% displayName %> <%if load != null %> <% TraditionalBinaryPrefix.long2String( - load.getMemstoreSizeMB() * TraditionalBinaryPrefix.MEGA.value, "B", 1) %> + load.getMemStoreSizeMB() * TraditionalBinaryPrefix.MEGA.value, "B", 1) %> diff --git a/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/ServerMetricsTmpl.jamon b/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/ServerMetricsTmpl.jamon index 1a8b274febd..2e99d5b8332 100644 --- a/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/ServerMetricsTmpl.jamon +++ b/hbase-server/src/main/jamon/org/apache/hadoop/hbase/tmpl/regionserver/ServerMetricsTmpl.jamon @@ -129,10 +129,10 @@ MetricsRegionServerWrapper mWrap; <% TraditionalBinaryPrefix.long2String(DirectMemoryUtils.getDirectMemorySize(), "B", 1) %> - <% TraditionalBinaryPrefix.long2String(mWrap.getMemstoreSize(), "B", 1) %> + <% TraditionalBinaryPrefix.long2String(mWrap.getMemStoreSize(), "B", 1) %> - <% TraditionalBinaryPrefix.long2String(mWrap.getMemstoreLimit(), "B", 1) %> + <% TraditionalBinaryPrefix.long2String(mWrap.getMemStoreLimit(), "B", 1) %> diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java index 9cf00b33740..35b741d07d5 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java @@ -489,9 +489,9 @@ public class HFile { boolean isPrimaryReplicaReader(); - boolean shouldIncludeMemstoreTS(); + boolean shouldIncludeMemStoreTS(); - boolean isDecodeMemstoreTS(); + boolean isDecodeMemStoreTS(); DataBlockEncoding getEffectiveEncodingInCache(boolean isCompaction); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlock.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlock.java index 24557ad6935..c6c7446f3fc 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlock.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlock.java @@ -1388,7 +1388,7 @@ public class HFileBlock implements Cacheable { /** Get the default decoder for blocks from this file. */ HFileBlockDecodingContext getDefaultBlockDecodingContext(); - void setIncludesMemstoreTS(boolean includesMemstoreTS); + void setIncludesMemStoreTS(boolean includesMemstoreTS); void setDataBlockEncoder(HFileDataBlockEncoder encoder); /** @@ -1810,7 +1810,7 @@ public class HFileBlock implements Cacheable { } @Override - public void setIncludesMemstoreTS(boolean includesMemstoreTS) { + public void setIncludesMemStoreTS(boolean includesMemstoreTS) { this.fileContext.setIncludesMvcc(includesMemstoreTS); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderImpl.java index 381279aa664..039f499f95d 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderImpl.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderImpl.java @@ -236,7 +236,7 @@ public class HFileReaderImpl implements HFile.Reader, Configurable { byte [] keyValueFormatVersion = fileInfo.get(HFileWriterImpl.KEY_VALUE_VERSION); includesMemstoreTS = keyValueFormatVersion != null && Bytes.toInt(keyValueFormatVersion) == HFileWriterImpl.KEY_VALUE_VER_WITH_MEMSTORE; - fsBlockReader.setIncludesMemstoreTS(includesMemstoreTS); + fsBlockReader.setIncludesMemStoreTS(includesMemstoreTS); if (includesMemstoreTS) { decodeMemstoreTS = Bytes.toLong(fileInfo.get(HFileWriterImpl.MAX_MEMSTORE_TS_KEY)) > 0; } @@ -645,8 +645,8 @@ public class HFileReaderImpl implements HFile.Reader, Configurable { */ protected void readMvccVersion(final int offsetFromPos) { // See if we even need to decode mvcc. - if (!this.reader.shouldIncludeMemstoreTS()) return; - if (!this.reader.isDecodeMemstoreTS()) { + if (!this.reader.shouldIncludeMemStoreTS()) return; + if (!this.reader.isDecodeMemStoreTS()) { currMemstoreTS = 0; currMemstoreTSLen = 1; return; @@ -741,7 +741,7 @@ public class HFileReaderImpl implements HFile.Reader, Configurable { // add the two bytes read for the tags. offsetFromPos += tlen + (Bytes.SIZEOF_SHORT); } - if (this.reader.shouldIncludeMemstoreTS()) { + if (this.reader.shouldIncludeMemStoreTS()) { // Directly read the mvcc based on current position readMvccVersion(offsetFromPos); } @@ -873,7 +873,7 @@ public class HFileReaderImpl implements HFile.Reader, Configurable { return false; } - // The first key in the current block 'seekToBlock' is greater than the given + // The first key in the current block 'seekToBlock' is greater than the given // seekBefore key. We will go ahead by reading the next block that satisfies the // given key. Return the current block before reading the next one. reader.returnBlock(seekToBlock); @@ -945,7 +945,7 @@ public class HFileReaderImpl implements HFile.Reader, Configurable { Cell ret; int cellBufSize = getKVBufSize(); long seqId = 0l; - if (this.reader.shouldIncludeMemstoreTS()) { + if (this.reader.shouldIncludeMemStoreTS()) { seqId = currMemstoreTS; } if (blockBuffer.hasArray()) { @@ -987,7 +987,7 @@ public class HFileReaderImpl implements HFile.Reader, Configurable { + keyPair.getSecond(), currKeyLen); } else { // Better to do a copy here instead of holding on to this BB so that - // we could release the blocks referring to this key. This key is specifically used + // we could release the blocks referring to this key. This key is specifically used // in HalfStoreFileReader to get the firstkey and lastkey by creating a new scanner // every time. So holding onto the BB (incase of DBB) is not advised here. byte[] key = new byte[currKeyLen]; @@ -1275,11 +1275,11 @@ public class HFileReaderImpl implements HFile.Reader, Configurable { protected boolean decodeMemstoreTS = false; - public boolean isDecodeMemstoreTS() { + public boolean isDecodeMemStoreTS() { return this.decodeMemstoreTS; } - public boolean shouldIncludeMemstoreTS() { + public boolean shouldIncludeMemStoreTS() { return includesMemstoreTS; } @@ -1783,7 +1783,7 @@ public class HFileReaderImpl implements HFile.Reader, Configurable { protected HFileContext createHFileContext(FSDataInputStreamWrapper fsdis, long fileSize, HFileSystem hfs, Path path, FixedFileTrailer trailer) throws IOException { HFileContextBuilder builder = new HFileContextBuilder() - .withIncludesMvcc(shouldIncludeMemstoreTS()) + .withIncludesMvcc(shouldIncludeMemStoreTS()) .withHBaseCheckSum(true) .withHFileName(this.getName()) .withCompression(this.compressAlgo); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/util/MemorySizeUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/util/MemorySizeUtil.java index 545c54d2ba8..dbfb295eea3 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/util/MemorySizeUtil.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/util/MemorySizeUtil.java @@ -159,7 +159,7 @@ public class MemorySizeUtil { /** * @return Pair of global memstore size and memory type(ie. on heap or off heap). */ - public static Pair getGlobalMemstoreSize(Configuration conf) { + public static Pair getGlobalMemStoreSize(Configuration conf) { long offheapMSGlobal = conf.getLong(OFFHEAP_MEMSTORE_SIZE_KEY, 0);// Size in MBs if (offheapMSGlobal > 0) { // Off heap memstore size has not relevance when MSLAB is turned OFF. We will go with making @@ -178,7 +178,7 @@ public class MemorySizeUtil { + " Going with on heap global memstore size ('" + MEMSTORE_SIZE_KEY + "')"); } } - return new Pair<>(getOnheapGlobalMemstoreSize(conf), MemoryType.HEAP); + return new Pair<>(getOnheapGlobalMemStoreSize(conf), MemoryType.HEAP); } /** @@ -187,7 +187,7 @@ public class MemorySizeUtil { * @param conf * @return the onheap global memstore limt */ - public static long getOnheapGlobalMemstoreSize(Configuration conf) { + public static long getOnheapGlobalMemStoreSize(Configuration conf) { long max = -1L; final MemoryUsage usage = safeGetHeapMemoryUsage(); if (usage != null) { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/StochasticLoadBalancer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/StochasticLoadBalancer.java index 258f68aebf6..23d9cb47c69 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/StochasticLoadBalancer.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/master/balancer/StochasticLoadBalancer.java @@ -192,7 +192,7 @@ public class StochasticLoadBalancer extends BaseLoadBalancer { regionLoadFunctions = new CostFromRegionLoadFunction[] { new ReadRequestCostFunction(conf), new WriteRequestCostFunction(conf), - new MemstoreSizeCostFunction(conf), + new MemStoreSizeCostFunction(conf), new StoreFileCostFunction(conf) }; regionReplicaHostCostFunction = new RegionReplicaHostCostFunction(conf); @@ -1668,13 +1668,13 @@ public class StochasticLoadBalancer extends BaseLoadBalancer { * Compute the cost of total memstore size. The more unbalanced the higher the * computed cost will be. This uses a rolling average of regionload. */ - static class MemstoreSizeCostFunction extends CostFromRegionLoadAsRateFunction { + static class MemStoreSizeCostFunction extends CostFromRegionLoadAsRateFunction { private static final String MEMSTORE_SIZE_COST_KEY = "hbase.master.balancer.stochastic.memstoreSizeCost"; private static final float DEFAULT_MEMSTORE_SIZE_COST = 5; - MemstoreSizeCostFunction(Configuration conf) { + MemStoreSizeCostFunction(Configuration conf) { super(conf); this.setMultiplier(conf.getFloat(MEMSTORE_SIZE_COST_KEY, DEFAULT_MEMSTORE_SIZE_COST)); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobFile.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobFile.java index 91702c1986d..1d0d5fff113 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobFile.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/mob/MobFile.java @@ -58,7 +58,7 @@ public class MobFile { List sfs = new ArrayList<>(); sfs.add(sf); List sfScanners = StoreFileScanner.getScannersForStoreFiles(sfs, false, true, - false, false, sf.getMaxMemstoreTS()); + false, false, sf.getMaxMemStoreTS()); return sfScanners.get(0); } @@ -71,7 +71,7 @@ public class MobFile { * @throws IOException */ public Cell readCell(Cell search, boolean cacheMobBlocks) throws IOException { - return readCell(search, cacheMobBlocks, sf.getMaxMemstoreTS()); + return readCell(search, cacheMobBlocks, sf.getMaxMemStoreTS()); } /** diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/AbstractMemStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/AbstractMemStore.java index a5c585554b6..4cb9ed1f4be 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/AbstractMemStore.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/AbstractMemStore.java @@ -96,14 +96,14 @@ public abstract class AbstractMemStore implements MemStore { public abstract void updateLowestUnflushedSequenceIdInWAL(boolean onlyIfMoreRecent); @Override - public void add(Iterable cells, MemstoreSize memstoreSize) { + public void add(Iterable cells, MemStoreSize memstoreSize) { for (Cell cell : cells) { add(cell, memstoreSize); } } @Override - public void add(Cell cell, MemstoreSize memstoreSize) { + public void add(Cell cell, MemStoreSize memstoreSize) { Cell toAdd = maybeCloneWithAllocator(cell); boolean mslabUsed = (toAdd != cell); // This cell data is backed by the same byte[] where we read request in RPC(See HBASE-15180). By @@ -129,7 +129,7 @@ public abstract class AbstractMemStore implements MemStore { } @Override - public void upsert(Iterable cells, long readpoint, MemstoreSize memstoreSize) { + public void upsert(Iterable cells, long readpoint, MemStoreSize memstoreSize) { for (Cell cell : cells) { upsert(cell, readpoint, memstoreSize); } @@ -166,8 +166,8 @@ public abstract class AbstractMemStore implements MemStore { } @Override - public MemstoreSize getSnapshotSize() { - return new MemstoreSize(this.snapshot.keySize(), this.snapshot.heapSize()); + public MemStoreSize getSnapshotSize() { + return new MemStoreSize(this.snapshot.keySize(), this.snapshot.heapSize()); } @Override @@ -210,7 +210,7 @@ public abstract class AbstractMemStore implements MemStore { * @param readpoint readpoint below which we can safely remove duplicate KVs * @param memstoreSize */ - private void upsert(Cell cell, long readpoint, MemstoreSize memstoreSize) { + private void upsert(Cell cell, long readpoint, MemStoreSize memstoreSize) { // Add the Cell to the MemStore // Use the internalAdd method here since we (a) already have a lock // and (b) cannot safely use the MSLAB here without potentially @@ -277,7 +277,7 @@ public abstract class AbstractMemStore implements MemStore { * @param mslabUsed whether using MSLAB * @param memstoreSize */ - private void internalAdd(final Cell toAdd, final boolean mslabUsed, MemstoreSize memstoreSize) { + private void internalAdd(final Cell toAdd, final boolean mslabUsed, MemStoreSize memstoreSize) { active.add(toAdd, mslabUsed, memstoreSize); setOldestEditTimeToNow(); checkActiveSize(); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CellArrayImmutableSegment.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CellArrayImmutableSegment.java index fc3a652ec97..a4b8c7b8352 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CellArrayImmutableSegment.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CellArrayImmutableSegment.java @@ -54,7 +54,7 @@ public class CellArrayImmutableSegment extends ImmutableSegment { * of CSLMImmutableSegment * The given iterator returns the Cells that "survived" the compaction. */ - protected CellArrayImmutableSegment(CSLMImmutableSegment segment, MemstoreSize memstoreSize) { + protected CellArrayImmutableSegment(CSLMImmutableSegment segment, MemStoreSize memstoreSize) { super(segment); // initiailize the upper class incSize(0, DEEP_OVERHEAD_CAM - CSLMImmutableSegment.DEEP_OVERHEAD_CSLM); int numOfCells = segment.getCellsCount(); @@ -64,7 +64,7 @@ public class CellArrayImmutableSegment extends ImmutableSegment { // add sizes of CellArrayMap entry (reinitializeCellSet doesn't take the care for the sizes) long newSegmentSizeDelta = numOfCells*(indexEntrySize()-ClassSize.CONCURRENT_SKIPLISTMAP_ENTRY); incSize(0, newSegmentSizeDelta); - memstoreSize.incMemstoreSize(0, newSegmentSizeDelta); + memstoreSize.incMemStoreSize(0, newSegmentSizeDelta); } @Override diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CellChunkImmutableSegment.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CellChunkImmutableSegment.java index d4667e1c627..62b62da54af 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CellChunkImmutableSegment.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CellChunkImmutableSegment.java @@ -60,7 +60,7 @@ public class CellChunkImmutableSegment extends ImmutableSegment { * of CSLMImmutableSegment * The given iterator returns the Cells that "survived" the compaction. */ - protected CellChunkImmutableSegment(CSLMImmutableSegment segment, MemstoreSize memstoreSize) { + protected CellChunkImmutableSegment(CSLMImmutableSegment segment, MemStoreSize memstoreSize) { super(segment); // initiailize the upper class incSize(0,-CSLMImmutableSegment.DEEP_OVERHEAD_CSLM+ CellChunkImmutableSegment.DEEP_OVERHEAD_CCM); int numOfCells = segment.getCellsCount(); @@ -72,7 +72,7 @@ public class CellChunkImmutableSegment extends ImmutableSegment { long newSegmentSizeDelta = numOfCells*(indexEntrySize()-ClassSize.CONCURRENT_SKIPLISTMAP_ENTRY); incSize(0, newSegmentSizeDelta); - memstoreSize.incMemstoreSize(0, newSegmentSizeDelta); + memstoreSize.incMemStoreSize(0, newSegmentSizeDelta); } @Override diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactingMemStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactingMemStore.java index da502c2fb85..01138df92ad 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactingMemStore.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactingMemStore.java @@ -124,7 +124,7 @@ public class CompactingMemStore extends AbstractMemStore { } private void initInmemoryFlushSize(Configuration conf) { - long memstoreFlushSize = getRegionServices().getMemstoreFlushSize(); + long memstoreFlushSize = getRegionServices().getMemStoreFlushSize(); int numStores = getRegionServices().getNumStores(); if (numStores <= 1) { // Family number might also be zero in some of our unit test case @@ -145,11 +145,11 @@ public class CompactingMemStore extends AbstractMemStore { * caller to make sure this doesn't happen. */ @Override - public MemstoreSize size() { - MemstoreSize memstoreSize = new MemstoreSize(); - memstoreSize.incMemstoreSize(this.active.keySize(), this.active.heapSize()); + public MemStoreSize size() { + MemStoreSize memstoreSize = new MemStoreSize(); + memstoreSize.incMemStoreSize(this.active.keySize(), this.active.heapSize()); for (Segment item : pipeline.getSegments()) { - memstoreSize.incMemstoreSize(item.keySize(), item.heapSize()); + memstoreSize.incMemStoreSize(item.keySize(), item.heapSize()); } return memstoreSize; } @@ -214,19 +214,19 @@ public class CompactingMemStore extends AbstractMemStore { * @return size of data that is going to be flushed */ @Override - public MemstoreSize getFlushableSize() { - MemstoreSize snapshotSize = getSnapshotSize(); + public MemStoreSize getFlushableSize() { + MemStoreSize snapshotSize = getSnapshotSize(); if (snapshotSize.getDataSize() == 0) { // if snapshot is empty the tail of the pipeline (or everything in the memstore) is flushed if (compositeSnapshot) { snapshotSize = pipeline.getPipelineSize(); - snapshotSize.incMemstoreSize(this.active.keySize(), this.active.heapSize()); + snapshotSize.incMemStoreSize(this.active.keySize(), this.active.heapSize()); } else { snapshotSize = pipeline.getTailSize(); } } return snapshotSize.getDataSize() > 0 ? snapshotSize - : new MemstoreSize(this.active.keySize(), this.active.heapSize()); + : new MemStoreSize(this.active.keySize(), this.active.heapSize()); } @Override diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactionPipeline.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactionPipeline.java index 83f641e96bf..ccbfbe3ba31 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactionPipeline.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompactionPipeline.java @@ -149,7 +149,7 @@ public class CompactionPipeline { long newHeapSize = 0; if(segment != null) newHeapSize = segment.heapSize(); long heapSizeDelta = suffixHeapSize - newHeapSize; - region.addMemstoreSize(new MemstoreSize(-dataSizeDelta, -heapSizeDelta)); + region.addMemStoreSize(new MemStoreSize(-dataSizeDelta, -heapSizeDelta)); if (LOG.isDebugEnabled()) { LOG.debug("Suffix data size: " + suffixDataSize + " new segment data size: " + newDataSize + ". Suffix heap size: " + suffixHeapSize @@ -199,14 +199,14 @@ public class CompactionPipeline { int i = 0; for (ImmutableSegment s : pipeline) { if ( s.canBeFlattened() ) { - MemstoreSize newMemstoreSize = new MemstoreSize(); // the size to be updated + MemStoreSize newMemstoreSize = new MemStoreSize(); // the size to be updated ImmutableSegment newS = SegmentFactory.instance().createImmutableSegmentByFlattening( (CSLMImmutableSegment)s,idxType,newMemstoreSize); replaceAtIndex(i,newS); if(region != null) { // update the global memstore size counter // upon flattening there is no change in the data size - region.addMemstoreSize(new MemstoreSize(0, newMemstoreSize.getHeapSize())); + region.addMemStoreSize(new MemStoreSize(0, newMemstoreSize.getHeapSize())); } LOG.debug("Compaction pipeline segment " + s + " was flattened"); return true; @@ -241,22 +241,22 @@ public class CompactionPipeline { return minSequenceId; } - public MemstoreSize getTailSize() { + public MemStoreSize getTailSize() { LinkedList localCopy = readOnlyCopy; - if (localCopy.isEmpty()) return new MemstoreSize(true); - return new MemstoreSize(localCopy.peekLast().keySize(), localCopy.peekLast().heapSize()); + if (localCopy.isEmpty()) return new MemStoreSize(true); + return new MemStoreSize(localCopy.peekLast().keySize(), localCopy.peekLast().heapSize()); } - public MemstoreSize getPipelineSize() { + public MemStoreSize getPipelineSize() { long keySize = 0; long heapSize = 0; LinkedList localCopy = readOnlyCopy; - if (localCopy.isEmpty()) return new MemstoreSize(true); + if (localCopy.isEmpty()) return new MemStoreSize(true); for (Segment segment : localCopy) { keySize += segment.keySize(); heapSize += segment.heapSize(); } - return new MemstoreSize(keySize, heapSize); + return new MemStoreSize(keySize, heapSize); } private void swapSuffix(List suffix, ImmutableSegment segment, diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompositeImmutableSegment.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompositeImmutableSegment.java index 5c1410fa297..0d2608f6c79 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompositeImmutableSegment.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/CompositeImmutableSegment.java @@ -268,13 +268,13 @@ public class CompositeImmutableSegment extends ImmutableSegment { } @Override - protected void internalAdd(Cell cell, boolean mslabUsed, MemstoreSize memstoreSize) { + protected void internalAdd(Cell cell, boolean mslabUsed, MemStoreSize memstoreSize) { throw new IllegalStateException("Not supported by CompositeImmutableScanner"); } @Override protected void updateMetaInfo(Cell cellToAdd, boolean succ, boolean mslabUsed, - MemstoreSize memstoreSize) { + MemStoreSize memstoreSize) { throw new IllegalStateException("Not supported by CompositeImmutableScanner"); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultHeapMemoryTuner.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultHeapMemoryTuner.java index 17e79196c4f..13c344150ca 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultHeapMemoryTuner.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultHeapMemoryTuner.java @@ -142,7 +142,7 @@ class DefaultHeapMemoryTuner implements HeapMemoryTuner { long blockedFlushCount = context.getBlockedFlushCount(); long unblockedFlushCount = context.getUnblockedFlushCount(); long totalOnheapFlushCount = blockedFlushCount + unblockedFlushCount; - boolean offheapMemstore = context.isOffheapMemstore(); + boolean offheapMemstore = context.isOffheapMemStore(); float newMemstoreSize; float newBlockCacheSize; @@ -223,7 +223,7 @@ class DefaultHeapMemoryTuner implements HeapMemoryTuner { newBlockCacheSize = blockCachePercentMinRange; } TUNER_RESULT.setBlockCacheSize(newBlockCacheSize); - TUNER_RESULT.setMemstoreSize(newMemstoreSize); + TUNER_RESULT.setMemStoreSize(newMemstoreSize); prevTuneDirection = newTuneDirection; return TUNER_RESULT; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultMemStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultMemStore.java index 086a2911571..7883d1243cb 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultMemStore.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/DefaultMemStore.java @@ -103,10 +103,10 @@ public class DefaultMemStore extends AbstractMemStore { * @return size of data that is going to be flushed from active set */ @Override - public MemstoreSize getFlushableSize() { - MemstoreSize snapshotSize = getSnapshotSize(); + public MemStoreSize getFlushableSize() { + MemStoreSize snapshotSize = getSnapshotSize(); return snapshotSize.getDataSize() > 0 ? snapshotSize - : new MemstoreSize(keySize(), heapSize()); + : new MemStoreSize(keySize(), heapSize()); } @Override @@ -154,8 +154,8 @@ public class DefaultMemStore extends AbstractMemStore { } @Override - public MemstoreSize size() { - return new MemstoreSize(this.active.keySize(), this.active.heapSize()); + public MemStoreSize size() { + return new MemStoreSize(this.active.keySize(), this.active.heapSize()); } /** @@ -194,7 +194,7 @@ public class DefaultMemStore extends AbstractMemStore { byte [] fam = Bytes.toBytes("col"); byte [] qf = Bytes.toBytes("umn"); byte [] empty = new byte[0]; - MemstoreSize memstoreSize = new MemstoreSize(); + MemStoreSize memstoreSize = new MemStoreSize(); for (int i = 0; i < count; i++) { // Give each its own ts memstore1.add(new KeyValue(Bytes.toBytes(i), fam, qf, i, empty), memstoreSize); @@ -208,7 +208,7 @@ public class DefaultMemStore extends AbstractMemStore { + (memstoreSize.getDataSize() + memstoreSize.getHeapSize())); // Make a variably sized memstore. DefaultMemStore memstore2 = new DefaultMemStore(); - memstoreSize = new MemstoreSize(); + memstoreSize = new MemStoreSize(); for (int i = 0; i < count; i++) { memstore2.add(new KeyValue(Bytes.toBytes(i), fam, qf, i, new byte[i]), memstoreSize); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/FlushLargeStoresPolicy.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/FlushLargeStoresPolicy.java index 4777e620a66..1610fd882f8 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/FlushLargeStoresPolicy.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/FlushLargeStoresPolicy.java @@ -46,7 +46,7 @@ public abstract class FlushLargeStoresPolicy extends FlushPolicy { int familyNumber = region.getTableDescriptor().getColumnFamilyCount(); // For multiple families, lower bound is the "average flush size" by default // unless setting in configuration is larger. - long flushSizeLowerBound = region.getMemstoreFlushSize() / familyNumber; + long flushSizeLowerBound = region.getMemStoreFlushSize() / familyNumber; long minimumLowerBound = getConf().getLong(HREGION_COLUMNFAMILY_FLUSH_SIZE_LOWER_BOUND_MIN, DEFAULT_HREGION_COLUMNFAMILY_FLUSH_SIZE_LOWER_BOUND_MIN); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/FlushNonSloppyStoresFirstPolicy.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/FlushNonSloppyStoresFirstPolicy.java index c779ce36abc..ed23e3d9d8d 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/FlushNonSloppyStoresFirstPolicy.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/FlushNonSloppyStoresFirstPolicy.java @@ -65,7 +65,7 @@ public class FlushNonSloppyStoresFirstPolicy extends FlushLargeStoresPolicy { super.configureForRegion(region); this.flushSizeLowerBound = getFlushSizeLowerBound(region); for (HStore store : region.stores.values()) { - if (store.isSloppyMemstore()) { + if (store.isSloppyMemStore()) { sloppyStores.add(store); } else { regularStores.add(store); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/FlushRequester.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/FlushRequester.java index 5e1fc6dc94e..e9f5d769f58 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/FlushRequester.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/FlushRequester.java @@ -65,5 +65,5 @@ public interface FlushRequester { * * @param globalMemStoreSize */ - public void setGlobalMemstoreLimit(long globalMemStoreSize); + public void setGlobalMemStoreLimit(long globalMemStoreSize); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java index d0599776c84..80c043379f0 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegion.java @@ -518,23 +518,23 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi final FlushResultImpl result; // indicating a failure result from prepare final TreeMap storeFlushCtxs; final TreeMap> committedFiles; - final TreeMap storeFlushableSize; + final TreeMap storeFlushableSize; final long startTime; final long flushOpSeqId; final long flushedSeqId; - final MemstoreSize totalFlushableSize; + final MemStoreSize totalFlushableSize; /** Constructs an early exit case */ PrepareFlushResult(FlushResultImpl result, long flushSeqId) { - this(result, null, null, null, Math.max(0, flushSeqId), 0, 0, new MemstoreSize()); + this(result, null, null, null, Math.max(0, flushSeqId), 0, 0, new MemStoreSize()); } /** Constructs a successful prepare flush result */ PrepareFlushResult( TreeMap storeFlushCtxs, TreeMap> committedFiles, - TreeMap storeFlushableSize, long startTime, long flushSeqId, - long flushedSeqId, MemstoreSize totalFlushableSize) { + TreeMap storeFlushableSize, long startTime, long flushSeqId, + long flushedSeqId, MemStoreSize totalFlushableSize) { this(null, storeFlushCtxs, committedFiles, storeFlushableSize, startTime, flushSeqId, flushedSeqId, totalFlushableSize); } @@ -543,8 +543,8 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi FlushResultImpl result, TreeMap storeFlushCtxs, TreeMap> committedFiles, - TreeMap storeFlushableSize, long startTime, long flushSeqId, - long flushedSeqId, MemstoreSize totalFlushableSize) { + TreeMap storeFlushableSize, long startTime, long flushSeqId, + long flushedSeqId, MemStoreSize totalFlushableSize) { this.result = result; this.storeFlushCtxs = storeFlushCtxs; this.committedFiles = committedFiles; @@ -1007,7 +1007,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi Future future = completionService.take(); HStore store = future.get(); this.stores.put(store.getColumnFamilyDescriptor().getName(), store); - if (store.isSloppyMemstore()) { + if (store.isSloppyMemStore()) { hasSloppyStores = true; } @@ -1017,7 +1017,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi if (maxSeqId == -1 || storeMaxSequenceId > maxSeqId) { maxSeqId = storeMaxSequenceId; } - long maxStoreMemstoreTS = store.getMaxMemstoreTS().orElse(0L); + long maxStoreMemstoreTS = store.getMaxMemStoreTS().orElse(0L); if (maxStoreMemstoreTS > maxMemstoreTS) { maxMemstoreTS = maxStoreMemstoreTS; } @@ -1194,24 +1194,24 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi * store * @return the size of memstore in this region */ - public long addAndGetMemstoreSize(MemstoreSize memstoreSize) { + public long addAndGetMemStoreSize(MemStoreSize memstoreSize) { if (this.rsAccounting != null) { - rsAccounting.incGlobalMemstoreSize(memstoreSize); + rsAccounting.incGlobalMemStoreSize(memstoreSize); } long size = this.memstoreDataSize.addAndGet(memstoreSize.getDataSize()); - checkNegativeMemstoreDataSize(size, memstoreSize.getDataSize()); + checkNegativeMemStoreDataSize(size, memstoreSize.getDataSize()); return size; } - public void decrMemstoreSize(MemstoreSize memstoreSize) { + public void decrMemStoreSize(MemStoreSize memstoreSize) { if (this.rsAccounting != null) { - rsAccounting.decGlobalMemstoreSize(memstoreSize); + rsAccounting.decGlobalMemStoreSize(memstoreSize); } long size = this.memstoreDataSize.addAndGet(-memstoreSize.getDataSize()); - checkNegativeMemstoreDataSize(size, -memstoreSize.getDataSize()); + checkNegativeMemStoreDataSize(size, -memstoreSize.getDataSize()); } - private void checkNegativeMemstoreDataSize(long memstoreDataSize, long delta) { + private void checkNegativeMemStoreDataSize(long memstoreDataSize, long delta) { // This is extremely bad if we make memstoreSize negative. Log as much info on the offending // caller as possible. (memStoreSize might be a negative value already -- freeing memory) if (memstoreDataSize < 0) { @@ -1260,7 +1260,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi } @Override - public long getMemstoreSize() { + public long getMemStoreSize() { return memstoreDataSize.get(); } @@ -1645,13 +1645,13 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi // close each store in parallel for (HStore store : stores.values()) { - MemstoreSize flushableSize = store.getFlushableSize(); + MemStoreSize flushableSize = store.getFlushableSize(); if (!(abort || flushableSize.getDataSize() == 0 || writestate.readOnly)) { if (getRegionServerServices() != null) { getRegionServerServices().abort("Assertion failed while closing store " + getRegionInfo().getRegionNameAsString() + " " + store + ". flushableSize expected=0, actual= " + flushableSize - + ". Current memstoreSize=" + getMemstoreSize() + ". Maybe a coprocessor " + + ". Current memstoreSize=" + getMemStoreSize() + ". Maybe a coprocessor " + "operation failed and left the memstore in a partially updated state.", null); } } @@ -1694,7 +1694,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi this.closed.set(true); if (!canFlush) { - this.decrMemstoreSize(new MemstoreSize(memstoreDataSize.get(), getMemstoreHeapSize())); + this.decrMemStoreSize(new MemStoreSize(memstoreDataSize.get(), getMemStoreHeapSize())); } else if (memstoreDataSize.get() != 0) { LOG.error("Memstore size is " + memstoreDataSize.get()); } @@ -1716,7 +1716,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi } } - private long getMemstoreHeapSize() { + private long getMemStoreHeapSize() { return stores.values().stream().mapToLong(s -> s.getMemStoreSize().getHeapSize()).sum(); } @@ -1906,7 +1906,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi byte[] encodedRegionName = this.getRegionInfo().getEncodedNameAsBytes(); regionLoadBldr.clearStoreCompleteSequenceId(); for (byte[] familyName : this.stores.keySet()) { - long earliest = this.wal.getEarliestMemstoreSeqNum(encodedRegionName, familyName); + long earliest = this.wal.getEarliestMemStoreSeqNum(encodedRegionName, familyName); // Subtract - 1 to go earlier than the current oldest, unflushed edit in memstore; this will // give us a sequence id that is for sure flushed. We want edit replay to start after this // sequence id in this region. If NO_SEQNUM, use the regions maximum flush id. @@ -2269,7 +2269,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi * returns true which will make a lot of flush requests. */ boolean shouldFlushStore(HStore store) { - long earliest = this.wal.getEarliestMemstoreSeqNum(getRegionInfo().getEncodedNameAsBytes(), + long earliest = this.wal.getEarliestMemStoreSeqNum(getRegionInfo().getEncodedNameAsBytes(), store.getColumnFamilyDescriptor().getName()) - 1; if (earliest > 0 && earliest + flushPerChanges < mvcc.getReadPoint()) { if (LOG.isDebugEnabled()) { @@ -2439,7 +2439,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi // block waiting for the lock for internal flush this.updatesLock.writeLock().lock(); status.setStatus("Preparing flush snapshotting stores in " + getRegionInfo().getEncodedName()); - MemstoreSize totalSizeOfFlushableStores = new MemstoreSize(); + MemStoreSize totalSizeOfFlushableStores = new MemStoreSize(); Map flushedFamilyNamesToSeq = new HashMap<>(); for (HStore store : storesToFlush) { @@ -2449,7 +2449,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi TreeMap storeFlushCtxs = new TreeMap<>(Bytes.BYTES_COMPARATOR); TreeMap> committedFiles = new TreeMap<>(Bytes.BYTES_COMPARATOR); - TreeMap storeFlushableSize = new TreeMap<>(Bytes.BYTES_COMPARATOR); + TreeMap storeFlushableSize = new TreeMap<>(Bytes.BYTES_COMPARATOR); // The sequence id of this flush operation which is used to log FlushMarker and pass to // createFlushContext to use as the store file's sequence id. It can be in advance of edits // still in the memstore, edits that are in other column families yet to be flushed. @@ -2481,8 +2481,8 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi } for (HStore s : storesToFlush) { - MemstoreSize flushableSize = s.getFlushableSize(); - totalSizeOfFlushableStores.incMemstoreSize(flushableSize); + MemStoreSize flushableSize = s.getFlushableSize(); + totalSizeOfFlushableStores.incMemStoreSize(flushableSize); storeFlushCtxs.put(s.getColumnFamilyDescriptor().getName(), s.createFlushContext(flushOpSeqId)); committedFiles.put(s.getColumnFamilyDescriptor().getName(), null); // for writing stores to WAL storeFlushableSize.put(s.getColumnFamilyDescriptor().getName(), flushableSize); @@ -2645,15 +2645,15 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi committedFiles.put(storeName, storeCommittedFiles); // Flush committed no files, indicating flush is empty or flush was canceled if (storeCommittedFiles == null || storeCommittedFiles.isEmpty()) { - MemstoreSize storeFlushableSize = prepareResult.storeFlushableSize.get(storeName); - prepareResult.totalFlushableSize.decMemstoreSize(storeFlushableSize); + MemStoreSize storeFlushableSize = prepareResult.storeFlushableSize.get(storeName); + prepareResult.totalFlushableSize.decMemStoreSize(storeFlushableSize); } flushedOutputFileSize += flush.getOutputFileSize(); } storeFlushCtxs.clear(); // Set down the memstore size by amount of flush. - this.decrMemstoreSize(prepareResult.totalFlushableSize); + this.decrMemStoreSize(prepareResult.totalFlushableSize); if (wal != null) { // write flush marker to WAL. If fail, we should throw DroppedSnapshotException @@ -3110,7 +3110,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi initialized = true; } doMiniBatchMutate(batchOp); - long newSize = this.getMemstoreSize(); + long newSize = this.getMemStoreSize(); requestFlushIfNeeded(newSize); } } finally { @@ -3182,7 +3182,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi int cellCount = 0; /** Keep track of the locks we hold so we can release them in finally clause */ List acquiredRowLocks = Lists.newArrayListWithCapacity(batchOp.operations.length); - MemstoreSize memstoreSize = new MemstoreSize(); + MemStoreSize memstoreSize = new MemStoreSize(); final ObservedExceptionsInBatch observedExceptions = new ObservedExceptionsInBatch(); try { // STEP 1. Try to acquire as many locks as we can, and ensure we acquire at least one. @@ -3414,11 +3414,11 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi this.updateSequenceId(familyMaps[i].values(), replay? batchOp.getReplaySequenceId(): writeEntry.getWriteNumber()); } - applyFamilyMapToMemstore(familyMaps[i], memstoreSize); + applyFamilyMapToMemStore(familyMaps[i], memstoreSize); } // update memstore size - this.addAndGetMemstoreSize(memstoreSize); + this.addAndGetMemStoreSize(memstoreSize); // calling the post CP hook for batch mutation if (!replay && coprocessorHost != null) { @@ -3952,13 +3952,13 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi * @param familyMap Map of Cells by family * @param memstoreSize */ - private void applyFamilyMapToMemstore(Map> familyMap, - MemstoreSize memstoreSize) throws IOException { + private void applyFamilyMapToMemStore(Map> familyMap, + MemStoreSize memstoreSize) throws IOException { for (Map.Entry> e : familyMap.entrySet()) { byte[] family = e.getKey(); List cells = e.getValue(); assert cells instanceof RandomAccess; - applyToMemstore(getStore(family), cells, false, memstoreSize); + applyToMemStore(getStore(family), cells, false, memstoreSize); } } @@ -3966,11 +3966,11 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi * @param delta If we are doing delta changes -- e.g. increment/append -- then this flag will be * set; when set we will run operations that make sense in the increment/append scenario * but that do not make sense otherwise. - * @see #applyToMemstore(HStore, Cell, long) + * @see #applyToMemStore(HStore, Cell, MemStoreSize) */ - private void applyToMemstore(HStore store, List cells, boolean delta, - MemstoreSize memstoreSize) throws IOException { - // Any change in how we update Store/MemStore needs to also be done in other applyToMemstore!!!! + private void applyToMemStore(HStore store, List cells, boolean delta, + MemStoreSize memstoreSize) throws IOException { + // Any change in how we update Store/MemStore needs to also be done in other applyToMemStore!!!! boolean upsert = delta && store.getColumnFamilyDescriptor().getMaxVersions() == 1; if (upsert) { store.upsert(cells, getSmallestReadPoint(), memstoreSize); @@ -3980,11 +3980,11 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi } /** - * @see #applyToMemstore(HStore, List, boolean, boolean, long) + * @see #applyToMemStore(HStore, List, boolean, MemStoreSize) */ - private void applyToMemstore(HStore store, Cell cell, MemstoreSize memstoreSize) + private void applyToMemStore(HStore store, Cell cell, MemStoreSize memstoreSize) throws IOException { - // Any change in how we update Store/MemStore needs to also be done in other applyToMemstore!!!! + // Any change in how we update Store/MemStore needs to also be done in other applyToMemStore!!!! if (store == null) { checkFamily(CellUtil.cloneFamily(cell)); // Unreachable because checkFamily will throw exception @@ -4040,7 +4040,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi } private void requestFlushIfNeeded(long memstoreTotalSize) throws RegionTooBusyException { - if(memstoreTotalSize > this.getMemstoreFlushSize()) { + if(memstoreTotalSize > this.getMemStoreFlushSize()) { requestFlush(); } } @@ -4308,7 +4308,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi } boolean flush = false; - MemstoreSize memstoreSize = new MemstoreSize(); + MemStoreSize memstoreSize = new MemStoreSize(); for (Cell cell: val.getCells()) { // Check this edit is for me. Also, guard against writing the special // METACOLUMN info such as HBASE::CACHEFLUSH entries @@ -4358,7 +4358,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi rsAccounting.addRegionReplayEditsSize(getRegionInfo().getRegionName(), memstoreSize); } - flush = isFlushSize(this.addAndGetMemstoreSize(memstoreSize)); + flush = isFlushSize(this.addAndGetMemStoreSize(memstoreSize)); if (flush) { internalFlushcache(null, currentEditSeqId, stores.values(), status, false); } @@ -4667,7 +4667,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi replayFlushInStores(flush, prepareFlushResult, true); // Set down the memstore size by amount of flush. - this.decrMemstoreSize(prepareFlushResult.totalFlushableSize); + this.decrMemStoreSize(prepareFlushResult.totalFlushableSize); this.prepareFlushResult = null; writestate.flushing = false; @@ -4700,11 +4700,11 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi replayFlushInStores(flush, prepareFlushResult, true); // Set down the memstore size by amount of flush. - this.decrMemstoreSize(prepareFlushResult.totalFlushableSize); + this.decrMemStoreSize(prepareFlushResult.totalFlushableSize); // Inspect the memstore contents to see whether the memstore contains only edits // with seqId smaller than the flush seqId. If so, we can discard those edits. - dropMemstoreContentsForSeqId(flush.getFlushSequenceNumber(), null); + dropMemStoreContentsForSeqId(flush.getFlushSequenceNumber(), null); this.prepareFlushResult = null; writestate.flushing = false; @@ -4725,7 +4725,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi // Inspect the memstore contents to see whether the memstore contains only edits // with seqId smaller than the flush seqId. If so, we can discard those edits. - dropMemstoreContentsForSeqId(flush.getFlushSequenceNumber(), null); + dropMemStoreContentsForSeqId(flush.getFlushSequenceNumber(), null); } status.markComplete("Flush commit successful"); @@ -4803,8 +4803,8 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi * if the memstore edits have seqNums smaller than the given seq id * @throws IOException */ - private MemstoreSize dropMemstoreContentsForSeqId(long seqId, HStore store) throws IOException { - MemstoreSize totalFreedSize = new MemstoreSize(); + private MemStoreSize dropMemStoreContentsForSeqId(long seqId, HStore store) throws IOException { + MemStoreSize totalFreedSize = new MemStoreSize(); this.updatesLock.writeLock().lock(); try { @@ -4818,10 +4818,10 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi // Prepare flush (take a snapshot) and then abort (drop the snapshot) if (store == null) { for (HStore s : stores.values()) { - totalFreedSize.incMemstoreSize(doDropStoreMemstoreContentsForSeqId(s, currentSeqId)); + totalFreedSize.incMemStoreSize(doDropStoreMemStoreContentsForSeqId(s, currentSeqId)); } } else { - totalFreedSize.incMemstoreSize(doDropStoreMemstoreContentsForSeqId(store, currentSeqId)); + totalFreedSize.incMemStoreSize(doDropStoreMemStoreContentsForSeqId(store, currentSeqId)); } } else { LOG.info(getRegionInfo().getEncodedName() + " : " @@ -4834,10 +4834,10 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi return totalFreedSize; } - private MemstoreSize doDropStoreMemstoreContentsForSeqId(HStore s, long currentSeqId) + private MemStoreSize doDropStoreMemStoreContentsForSeqId(HStore s, long currentSeqId) throws IOException { - MemstoreSize flushableSize = s.getFlushableSize(); - this.decrMemstoreSize(flushableSize); + MemStoreSize flushableSize = s.getFlushableSize(); + this.decrMemStoreSize(flushableSize); StoreFlushContext ctx = s.createFlushContext(currentSeqId); ctx.prepare(); ctx.abort(); @@ -4954,16 +4954,16 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi StoreFlushContext ctx = this.prepareFlushResult.storeFlushCtxs == null ? null : this.prepareFlushResult.storeFlushCtxs.get(family); if (ctx != null) { - MemstoreSize snapshotSize = store.getFlushableSize(); + MemStoreSize snapshotSize = store.getFlushableSize(); ctx.abort(); - this.decrMemstoreSize(snapshotSize); + this.decrMemStoreSize(snapshotSize); this.prepareFlushResult.storeFlushCtxs.remove(family); } } } // Drop the memstore contents if they are now smaller than the latest seen flushed file - dropMemstoreContentsForSeqId(regionEvent.getLogSequenceNumber(), store); + dropMemStoreContentsForSeqId(regionEvent.getLogSequenceNumber(), store); if (storeSeqId > this.maxFlushedSeqId) { this.maxFlushedSeqId = storeSeqId; } @@ -5148,9 +5148,9 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi null : this.prepareFlushResult.storeFlushCtxs.get( store.getColumnFamilyDescriptor().getName()); if (ctx != null) { - MemstoreSize snapshotSize = store.getFlushableSize(); + MemStoreSize snapshotSize = store.getFlushableSize(); ctx.abort(); - this.decrMemstoreSize(snapshotSize); + this.decrMemStoreSize(snapshotSize); this.prepareFlushResult.storeFlushCtxs.remove( store.getColumnFamilyDescriptor().getName()); totalFreedDataSize += snapshotSize.getDataSize(); @@ -5169,7 +5169,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi // advance the mvcc read point so that the new flushed files are visible. // either greater than flush seq number or they were already picked up via flush. for (HStore s : stores.values()) { - mvcc.advanceTo(s.getMaxMemstoreTS().orElse(0L)); + mvcc.advanceTo(s.getMaxMemStoreTS().orElse(0L)); } @@ -5184,7 +5184,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi if (!map.isEmpty()) { for (Map.Entry entry : map.entrySet()) { // Drop the memstore contents if they are now smaller than the latest seen flushed file - totalFreedDataSize += dropMemstoreContentsForSeqId(entry.getValue(), entry.getKey()) + totalFreedDataSize += dropMemStoreContentsForSeqId(entry.getValue(), entry.getKey()) .getDataSize(); } } @@ -5235,7 +5235,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi * @param memstoreSize */ @VisibleForTesting - protected void restoreEdit(HStore s, Cell cell, MemstoreSize memstoreSize) { + protected void restoreEdit(HStore s, Cell cell, MemStoreSize memstoreSize) { s.add(cell, memstoreSize); } @@ -6958,7 +6958,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi return null; } ClientProtos.RegionLoadStats.Builder stats = ClientProtos.RegionLoadStats.newBuilder(); - stats.setMemstoreLoad((int) (Math.min(100, (this.memstoreDataSize.get() * 100) / this + stats.setMemStoreLoad((int) (Math.min(100, (this.memstoreDataSize.get() * 100) / this .memstoreFlushSize))); if (rsServices.getHeapMemoryManager() != null) { // the HeapMemoryManager uses -0.0 to signal a problem asking the JVM, @@ -7021,7 +7021,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi // This is assigned by mvcc either explicity in the below or in the guts of the WAL append // when it assigns the edit a sequencedid (A.K.A the mvcc write number). WriteEntry writeEntry = null; - MemstoreSize memstoreSize = new MemstoreSize(); + MemStoreSize memstoreSize = new MemStoreSize(); try { boolean success = false; try { @@ -7055,7 +7055,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi long sequenceId = writeEntry.getWriteNumber(); for (Mutation m : mutations) { // Handle any tag based cell features. - // TODO: Do we need to call rewriteCellTags down in applyToMemstore()? Why not before + // TODO: Do we need to call rewriteCellTags down in applyToMemStore()? Why not before // so tags go into WAL? rewriteCellTags(m.getFamilyCellMap(), m); for (CellScanner cellScanner = m.cellScanner(); cellScanner.advance();) { @@ -7065,7 +7065,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi // If no WAL, need to stamp it here. CellUtil.setSequenceId(cell, sequenceId); } - applyToMemstore(getStore(cell), cell, memstoreSize); + applyToMemStore(getStore(cell), cell, memstoreSize); } } @@ -7101,7 +7101,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi } finally { closeRegionOperation(); if (!mutations.isEmpty()) { - long newSize = this.addAndGetMemstoreSize(memstoreSize); + long newSize = this.addAndGetMemStoreSize(memstoreSize); requestFlushIfNeeded(newSize); } } @@ -7206,7 +7206,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi startRegionOperation(op); List results = returnResults? new ArrayList<>(mutation.size()): null; RowLock rowLock = null; - MemstoreSize memstoreSize = new MemstoreSize(); + MemStoreSize memstoreSize = new MemStoreSize(); try { rowLock = getRowLockInternal(mutation.getRow(), false); lock(this.updatesLock.readLock()); @@ -7232,7 +7232,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi } // Now write to MemStore. Do it a column family at a time. for (Map.Entry> e : forMemStore.entrySet()) { - applyToMemstore(e.getKey(), e.getValue(), true, memstoreSize); + applyToMemStore(e.getKey(), e.getValue(), true, memstoreSize); } mvcc.completeAndWait(writeEntry); if (rsServices != null && rsServices.getNonceManager() != null) { @@ -7255,7 +7255,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi rowLock.release(); } // Request a cache flush if over the limit. Do it outside update lock. - if (isFlushSize(addAndGetMemstoreSize(memstoreSize))) { + if (isFlushSize(addAndGetMemStoreSize(memstoreSize))) { requestFlush(); } closeRegionOperation(op); @@ -7982,7 +7982,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi @Override public long getOldestSeqIdOfStore(byte[] familyName) { - return wal.getEarliestMemstoreSeqNum(getRegionInfo().getEncodedNameAsBytes(), familyName); + return wal.getEarliestMemStoreSeqNum(getRegionInfo().getEncodedNameAsBytes(), familyName); } @Override @@ -8059,7 +8059,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi : CellComparator.COMPARATOR; } - public long getMemstoreFlushSize() { + public long getMemStoreFlushSize() { return this.memstoreFlushSize; } @@ -8079,7 +8079,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi } buf.append("end-of-stores"); buf.append(", memstore size "); - buf.append(getMemstoreSize()); + buf.append(getMemStoreSize()); if (getRegionInfo().getRegionNameAsString().startsWith(regionName)) { throw new RuntimeException(buf.toString()); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java index 5ef0358f469..f69695aa0d3 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java @@ -949,7 +949,7 @@ public class HRegionServer extends HasThread implements // Background thread to check for compactions; needed if region has not gotten updates // in a while. It will take care of not checking too frequently on store-by-store basis. this.compactionChecker = new CompactionChecker(this, this.threadWakeFrequency, this); - this.periodicFlusher = new PeriodicMemstoreFlusher(this.threadWakeFrequency, this); + this.periodicFlusher = new PeriodicMemStoreFlusher(this.threadWakeFrequency, this); this.leases = new Leases(this.threadWakeFrequency); // Create the thread to clean the moved regions list @@ -1622,7 +1622,7 @@ public class HRegionServer extends HasThread implements // MSLAB is enabled. So initialize MemStoreChunkPool // By this time, the MemstoreFlusher is already initialized. We can get the global limits from // it. - Pair pair = MemorySizeUtil.getGlobalMemstoreSize(conf); + Pair pair = MemorySizeUtil.getGlobalMemStoreSize(conf); long globalMemStoreSize = pair.getFirst(); boolean offheap = this.regionServerAccounting.isOffheap(); // When off heap memstore in use, take full area for chunk pool. @@ -1679,7 +1679,7 @@ public class HRegionServer extends HasThread implements int storefiles = 0; int storeUncompressedSizeMB = 0; int storefileSizeMB = 0; - int memstoreSizeMB = (int) (r.getMemstoreSize() / 1024 / 1024); + int memstoreSizeMB = (int) (r.getMemStoreSize() / 1024 / 1024); long storefileIndexSizeKB = 0; int rootIndexSizeKB = 0; int totalStaticIndexSizeKB = 0; @@ -1718,7 +1718,7 @@ public class HRegionServer extends HasThread implements .setStorefiles(storefiles) .setStoreUncompressedSizeMB(storeUncompressedSizeMB) .setStorefileSizeMB(storefileSizeMB) - .setMemstoreSizeMB(memstoreSizeMB) + .setMemStoreSizeMB(memstoreSizeMB) .setStorefileIndexSizeKB(storefileIndexSizeKB) .setRootIndexSizeKB(rootIndexSizeKB) .setTotalStaticIndexSizeKB(totalStaticIndexSizeKB) @@ -1809,11 +1809,11 @@ public class HRegionServer extends HasThread implements } } - static class PeriodicMemstoreFlusher extends ScheduledChore { + static class PeriodicMemStoreFlusher extends ScheduledChore { final HRegionServer server; final static int RANGE_OF_DELAY = 5 * 60 * 1000; // 5 min in milliseconds final static int MIN_DELAY_TIME = 0; // millisec - public PeriodicMemstoreFlusher(int cacheFlushInterval, final HRegionServer server) { + public PeriodicMemStoreFlusher(int cacheFlushInterval, final HRegionServer server) { super("MemstoreFlusherChore", server, cacheFlushInterval); this.server = server; } @@ -2778,7 +2778,7 @@ public class HRegionServer extends HasThread implements }); // Copy over all regions. Regions are sorted by size with biggest first. for (Region region : this.onlineRegions.values()) { - sortedRegions.put(region.getMemstoreSize(), region); + sortedRegions.put(region.getMemStoreSize(), region); } return sortedRegions; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java index f232dddd3f0..86686493490 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStore.java @@ -393,18 +393,18 @@ public class HStore implements Store, HeapSize, StoreConfigInformation, Propagat } @Override - public long getMemstoreFlushSize() { + public long getMemStoreFlushSize() { // TODO: Why is this in here? The flushsize of the region rather than the store? St.Ack return this.region.memstoreFlushSize; } @Override - public MemstoreSize getFlushableSize() { + public MemStoreSize getFlushableSize() { return this.memstore.getFlushableSize(); } @Override - public MemstoreSize getSnapshotSize() { + public MemStoreSize getSnapshotSize() { return this.memstore.getSnapshotSize(); } @@ -461,8 +461,8 @@ public class HStore implements Store, HeapSize, StoreConfigInformation, Propagat } @Override - public OptionalLong getMaxMemstoreTS() { - return StoreUtils.getMaxMemstoreTSInList(this.getStorefiles()); + public OptionalLong getMaxMemStoreTS() { + return StoreUtils.getMaxMemStoreTSInList(this.getStorefiles()); } /** @@ -684,7 +684,7 @@ public class HStore implements Store, HeapSize, StoreConfigInformation, Propagat * @param cell * @param memstoreSize */ - public void add(final Cell cell, MemstoreSize memstoreSize) { + public void add(final Cell cell, MemStoreSize memstoreSize) { lock.readLock().lock(); try { this.memstore.add(cell, memstoreSize); @@ -698,7 +698,7 @@ public class HStore implements Store, HeapSize, StoreConfigInformation, Propagat * @param cells * @param memstoreSize */ - public void add(final Iterable cells, MemstoreSize memstoreSize) { + public void add(final Iterable cells, MemStoreSize memstoreSize) { lock.readLock().lock(); try { memstore.add(cells, memstoreSize); @@ -2115,7 +2115,7 @@ public class HStore implements Store, HeapSize, StoreConfigInformation, Propagat } @Override - public MemstoreSize getMemStoreSize() { + public MemStoreSize getMemStoreSize() { return this.memstore.size(); } @@ -2169,7 +2169,7 @@ public class HStore implements Store, HeapSize, StoreConfigInformation, Propagat * @param memstoreSize * @throws IOException */ - public void upsert(Iterable cells, long readpoint, MemstoreSize memstoreSize) + public void upsert(Iterable cells, long readpoint, MemStoreSize memstoreSize) throws IOException { this.lock.readLock().lock(); try { @@ -2343,7 +2343,7 @@ public class HStore implements Store, HeapSize, StoreConfigInformation, Propagat @Override public long heapSize() { - MemstoreSize memstoreSize = this.memstore.size(); + MemStoreSize memstoreSize = this.memstore.size(); return DEEP_OVERHEAD + memstoreSize.getHeapSize(); } @@ -2578,7 +2578,7 @@ public class HStore implements Store, HeapSize, StoreConfigInformation, Propagat } @Override - public boolean isSloppyMemstore() { + public boolean isSloppyMemStore() { return this.memstore.isSloppy(); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStoreFile.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStoreFile.java index a79af136d4a..5301922e6f1 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStoreFile.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HStoreFile.java @@ -159,7 +159,7 @@ public class HStoreFile implements StoreFile { } @Override - public long getMaxMemstoreTS() { + public long getMaxMemStoreTS() { return maxMemstoreTS; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HeapMemoryManager.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HeapMemoryManager.java index 0ee03eba71a..14021ffb781 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HeapMemoryManager.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HeapMemoryManager.java @@ -49,19 +49,19 @@ import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTe public class HeapMemoryManager { private static final Log LOG = LogFactory.getLog(HeapMemoryManager.class); private static final int CONVERT_TO_PERCENTAGE = 100; - private static final int CLUSTER_MINIMUM_MEMORY_THRESHOLD = + private static final int CLUSTER_MINIMUM_MEMORY_THRESHOLD = (int) (CONVERT_TO_PERCENTAGE * HConstants.HBASE_CLUSTER_MINIMUM_MEMORY_THRESHOLD); public static final String BLOCK_CACHE_SIZE_MAX_RANGE_KEY = "hfile.block.cache.size.max.range"; public static final String BLOCK_CACHE_SIZE_MIN_RANGE_KEY = "hfile.block.cache.size.min.range"; - public static final String MEMSTORE_SIZE_MAX_RANGE_KEY = + public static final String MEMSTORE_SIZE_MAX_RANGE_KEY = "hbase.regionserver.global.memstore.size.max.range"; - public static final String MEMSTORE_SIZE_MIN_RANGE_KEY = + public static final String MEMSTORE_SIZE_MIN_RANGE_KEY = "hbase.regionserver.global.memstore.size.min.range"; - public static final String HBASE_RS_HEAP_MEMORY_TUNER_PERIOD = + public static final String HBASE_RS_HEAP_MEMORY_TUNER_PERIOD = "hbase.regionserver.heapmemory.tuner.period"; public static final int HBASE_RS_HEAP_MEMORY_TUNER_DEFAULT_PERIOD = 60 * 1000; - public static final String HBASE_RS_HEAP_MEMORY_TUNER_CLASS = + public static final String HBASE_RS_HEAP_MEMORY_TUNER_CLASS = "hbase.regionserver.heapmemory.tuner.class"; public static final float HEAP_OCCUPANCY_ERROR_VALUE = -0.0f; @@ -255,7 +255,7 @@ public class HeapMemoryManager { HBASE_RS_HEAP_MEMORY_TUNER_CLASS, DefaultHeapMemoryTuner.class, HeapMemoryTuner.class); heapMemTuner = ReflectionUtils.newInstance(tunerKlass, server.getConfiguration()); tunerContext - .setOffheapMemstore(regionServerAccounting.isOffheap()); + .setOffheapMemStore(regionServerAccounting.isOffheap()); } @Override @@ -324,7 +324,7 @@ public class HeapMemoryManager { // TODO : add support for offheap metrics tunerContext.setCurBlockCacheUsed((float) blockCache.getCurrentSize() / maxHeapSize); metricsHeapMemoryManager.setCurBlockCacheSizeGauge(blockCache.getCurrentSize()); - long globalMemstoreHeapSize = regionServerAccounting.getGlobalMemstoreHeapSize(); + long globalMemstoreHeapSize = regionServerAccounting.getGlobalMemStoreHeapSize(); tunerContext.setCurMemStoreUsed((float) globalMemstoreHeapSize / maxHeapSize); metricsHeapMemoryManager.setCurMemStoreSizeGauge(globalMemstoreHeapSize); tunerContext.setCurBlockCacheSize(blockCachePercent); @@ -336,7 +336,7 @@ public class HeapMemoryManager { LOG.error("Exception thrown from the HeapMemoryTuner implementation", t); } if (result != null && result.needsTuning()) { - float memstoreSize = result.getMemstoreSize(); + float memstoreSize = result.getMemStoreSize(); float blockCacheSize = result.getBlockCacheSize(); LOG.debug("From HeapMemoryTuner new memstoreSize: " + memstoreSize + ". new blockCacheSize: " + blockCacheSize); @@ -388,7 +388,7 @@ public class HeapMemoryManager { globalMemStorePercent = memstoreSize; // Internally sets it to RegionServerAccounting // TODO : Set directly on RSAccounting?? - memStoreFlusher.setGlobalMemstoreLimit(newMemstoreSize); + memStoreFlusher.setGlobalMemStoreLimit(newMemstoreSize); for (HeapMemoryTuneObserver observer : tuneObservers) { // Risky.. If this newMemstoreSize decreases we reduce the count in offheap chunk pool observer.onHeapMemoryTune(newMemstoreSize, newBlockCacheSize); @@ -500,11 +500,11 @@ public class HeapMemoryManager { this.curMemStoreUsed = d; } - public void setOffheapMemstore(boolean offheapMemstore) { + public void setOffheapMemStore(boolean offheapMemstore) { this.offheapMemstore = offheapMemstore; } - public boolean isOffheapMemstore() { + public boolean isOffheapMemStore() { return this.offheapMemstore; } } @@ -522,11 +522,11 @@ public class HeapMemoryManager { this.needsTuning = needsTuning; } - public float getMemstoreSize() { + public float getMemStoreSize() { return memstoreSize; } - public void setMemstoreSize(float memstoreSize) { + public void setMemStoreSize(float memstoreSize) { this.memstoreSize = memstoreSize; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStore.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStore.java index cd80c298609..324afbe220c 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStore.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStore.java @@ -57,13 +57,13 @@ public interface MemStore { * * @return size of data that is going to be flushed */ - MemstoreSize getFlushableSize(); + MemStoreSize getFlushableSize(); /** * Return the size of the snapshot(s) if any * @return size of the memstore snapshot */ - MemstoreSize getSnapshotSize(); + MemStoreSize getSnapshotSize(); /** * Write an update @@ -71,7 +71,7 @@ public interface MemStore { * @param memstoreSize The delta in memstore size will be passed back via this. * This will include both data size and heap overhead delta. */ - void add(final Cell cell, MemstoreSize memstoreSize); + void add(final Cell cell, MemStoreSize memstoreSize); /** * Write the updates @@ -79,7 +79,7 @@ public interface MemStore { * @param memstoreSize The delta in memstore size will be passed back via this. * This will include both data size and heap overhead delta. */ - void add(Iterable cells, MemstoreSize memstoreSize); + void add(Iterable cells, MemStoreSize memstoreSize); /** * @return Oldest timestamp of all the Cells in the MemStore @@ -102,7 +102,7 @@ public interface MemStore { * @param memstoreSize The delta in memstore size will be passed back via this. * This will include both data size and heap overhead delta. */ - void upsert(Iterable cells, long readpoint, MemstoreSize memstoreSize); + void upsert(Iterable cells, long readpoint, MemStoreSize memstoreSize); /** * @return scanner over the memstore. This might include scanner over the snapshot when one is @@ -116,7 +116,7 @@ public interface MemStore { * the memstore may be changed while computing its size. It is the responsibility of the * caller to make sure this doesn't happen. */ - MemstoreSize size(); + MemStoreSize size(); /** * This method is called before the flush is executed. diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreFlusher.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreFlusher.java index 8fa686c71cf..f37f8f6cbb4 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreFlusher.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreFlusher.java @@ -107,10 +107,10 @@ class MemStoreFlusher implements FlushRequester { this.flushHandlers = new FlushHandler[handlerCount]; LOG.info("globalMemStoreLimit=" + TraditionalBinaryPrefix - .long2String(this.server.getRegionServerAccounting().getGlobalMemstoreLimit(), "", 1) + .long2String(this.server.getRegionServerAccounting().getGlobalMemStoreLimit(), "", 1) + ", globalMemStoreLimitLowMark=" + TraditionalBinaryPrefix.long2String( - this.server.getRegionServerAccounting().getGlobalMemstoreLimitLowMark(), "", 1) + this.server.getRegionServerAccounting().getGlobalMemStoreLimitLowMark(), "", 1) + ", Offheap=" + (this.server.getRegionServerAccounting().isOffheap())); } @@ -136,12 +136,12 @@ class MemStoreFlusher implements FlushRequester { while (!flushedOne) { // Find the biggest region that doesn't have too many storefiles // (might be null!) - Region bestFlushableRegion = getBiggestMemstoreRegion(regionsBySize, excludedRegions, true); + Region bestFlushableRegion = getBiggestMemStoreRegion(regionsBySize, excludedRegions, true); // Find the biggest region, total, even if it might have too many flushes. - Region bestAnyRegion = getBiggestMemstoreRegion( + Region bestAnyRegion = getBiggestMemStoreRegion( regionsBySize, excludedRegions, false); // Find the biggest region that is a secondary region - Region bestRegionReplica = getBiggestMemstoreOfRegionReplica(regionsBySize, + Region bestRegionReplica = getBiggestMemStoreOfRegionReplica(regionsBySize, excludedRegions); if (bestAnyRegion == null && bestRegionReplica == null) { @@ -151,7 +151,7 @@ class MemStoreFlusher implements FlushRequester { Region regionToFlush; if (bestFlushableRegion != null && - bestAnyRegion.getMemstoreSize() > 2 * bestFlushableRegion.getMemstoreSize()) { + bestAnyRegion.getMemStoreSize() > 2 * bestFlushableRegion.getMemStoreSize()) { // Even if it's not supposed to be flushed, pick a region if it's more than twice // as big as the best flushable one - otherwise when we're under pressure we make // lots of little flushes and cause lots of compactions, etc, which just makes @@ -160,9 +160,9 @@ class MemStoreFlusher implements FlushRequester { LOG.debug("Under global heap pressure: " + "Region " + bestAnyRegion.getRegionInfo().getRegionNameAsString() + " has too many " + "store files, but is " - + TraditionalBinaryPrefix.long2String(bestAnyRegion.getMemstoreSize(), "", 1) + + TraditionalBinaryPrefix.long2String(bestAnyRegion.getMemStoreSize(), "", 1) + " vs best flushable region's " - + TraditionalBinaryPrefix.long2String(bestFlushableRegion.getMemstoreSize(), "", 1) + + TraditionalBinaryPrefix.long2String(bestFlushableRegion.getMemStoreSize(), "", 1) + ". Choosing the bigger."); } regionToFlush = bestAnyRegion; @@ -175,20 +175,20 @@ class MemStoreFlusher implements FlushRequester { } Preconditions.checkState( - (regionToFlush != null && regionToFlush.getMemstoreSize() > 0) || - (bestRegionReplica != null && bestRegionReplica.getMemstoreSize() > 0)); + (regionToFlush != null && regionToFlush.getMemStoreSize() > 0) || + (bestRegionReplica != null && bestRegionReplica.getMemStoreSize() > 0)); if (regionToFlush == null || (bestRegionReplica != null && ServerRegionReplicaUtil.isRegionReplicaStoreFileRefreshEnabled(conf) && - (bestRegionReplica.getMemstoreSize() - > secondaryMultiplier * regionToFlush.getMemstoreSize()))) { + (bestRegionReplica.getMemStoreSize() + > secondaryMultiplier * regionToFlush.getMemStoreSize()))) { LOG.info("Refreshing storefiles of region " + bestRegionReplica + " due to global heap pressure. Total memstore datasize=" + StringUtils - .humanReadableInt(server.getRegionServerAccounting().getGlobalMemstoreDataSize()) + .humanReadableInt(server.getRegionServerAccounting().getGlobalMemStoreDataSize()) + " memstore heap size=" + StringUtils.humanReadableInt( - server.getRegionServerAccounting().getGlobalMemstoreHeapSize())); + server.getRegionServerAccounting().getGlobalMemStoreHeapSize())); flushedOne = refreshStoreFilesAndReclaimMemory(bestRegionReplica); if (!flushedOne) { LOG.info("Excluding secondary region " + bestRegionReplica + @@ -198,9 +198,9 @@ class MemStoreFlusher implements FlushRequester { } else { LOG.info("Flush of region " + regionToFlush + " due to global heap pressure. " + "Total Memstore size=" - + humanReadableInt(server.getRegionServerAccounting().getGlobalMemstoreDataSize()) + + humanReadableInt(server.getRegionServerAccounting().getGlobalMemStoreDataSize()) + ", Region memstore size=" - + humanReadableInt(regionToFlush.getMemstoreSize())); + + humanReadableInt(regionToFlush.getMemStoreSize())); flushedOne = flushRegion(regionToFlush, true, false); if (!flushedOne) { @@ -231,7 +231,7 @@ class MemStoreFlusher implements FlushRequester { if (type != FlushType.NORMAL) { LOG.debug("Flush thread woke up because memory above low water=" + TraditionalBinaryPrefix.long2String( - server.getRegionServerAccounting().getGlobalMemstoreLimitLowMark(), "", 1)); + server.getRegionServerAccounting().getGlobalMemStoreLimitLowMark(), "", 1)); // For offheap memstore, even if the lower water mark was breached due to heap overhead // we still select the regions based on the region's memstore data size. // TODO : If we want to decide based on heap over head it can be done without tracking @@ -283,7 +283,7 @@ class MemStoreFlusher implements FlushRequester { } } - private Region getBiggestMemstoreRegion( + private Region getBiggestMemStoreRegion( SortedMap regionsBySize, Set excludedRegions, boolean checkStoreFileCount) { @@ -307,7 +307,7 @@ class MemStoreFlusher implements FlushRequester { return null; } - private Region getBiggestMemstoreOfRegionReplica(SortedMap regionsBySize, + private Region getBiggestMemStoreOfRegionReplica(SortedMap regionsBySize, Set excludedRegions) { synchronized (regionsInQueue) { for (Region region : regionsBySize.values()) { @@ -588,19 +588,19 @@ class MemStoreFlusher implements FlushRequester { startTime = EnvironmentEdgeManager.currentTime(); if (!server.getRegionServerAccounting().isOffheap()) { logMsg("global memstore heapsize", - server.getRegionServerAccounting().getGlobalMemstoreHeapSize(), - server.getRegionServerAccounting().getGlobalMemstoreLimit()); + server.getRegionServerAccounting().getGlobalMemStoreHeapSize(), + server.getRegionServerAccounting().getGlobalMemStoreLimit()); } else { switch (flushType) { case ABOVE_OFFHEAP_HIGHER_MARK: logMsg("the global offheap memstore datasize", - server.getRegionServerAccounting().getGlobalMemstoreDataSize(), - server.getRegionServerAccounting().getGlobalMemstoreLimit()); + server.getRegionServerAccounting().getGlobalMemStoreDataSize(), + server.getRegionServerAccounting().getGlobalMemStoreLimit()); break; case ABOVE_ONHEAP_HIGHER_MARK: logMsg("global memstore heapsize", - server.getRegionServerAccounting().getGlobalMemstoreHeapSize(), - server.getRegionServerAccounting().getGlobalOnHeapMemstoreLimit()); + server.getRegionServerAccounting().getGlobalMemStoreHeapSize(), + server.getRegionServerAccounting().getGlobalOnHeapMemStoreLimit()); break; default: break; @@ -691,8 +691,8 @@ class MemStoreFlusher implements FlushRequester { * @param globalMemStoreSize */ @Override - public void setGlobalMemstoreLimit(long globalMemStoreSize) { - this.server.getRegionServerAccounting().setGlobalMemstoreLimits(globalMemStoreSize); + public void setGlobalMemStoreLimit(long globalMemStoreSize) { + this.server.getRegionServerAccounting().setGlobalMemStoreLimits(globalMemStoreSize); reclaimMemStoreMemory(); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemstoreSize.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreSize.java similarity index 84% rename from hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemstoreSize.java rename to hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreSize.java index 8693598991a..a588b2049ef 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemstoreSize.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MemStoreSize.java @@ -23,7 +23,7 @@ import org.apache.yetus.audience.InterfaceAudience; * Wraps the data size part and total heap space occupied by the memstore. */ @InterfaceAudience.Private -public class MemstoreSize { +public class MemStoreSize { // 'dataSize' tracks the Cell's data bytes size alone (Key bytes, value bytes). A cell's data can // be in on heap or off heap area depending on the MSLAB and its configuration to be using on heap @@ -34,13 +34,13 @@ public class MemstoreSize { private long heapSize; final private boolean isEmpty; - public MemstoreSize() { + public MemStoreSize() { dataSize = 0; heapSize = 0; isEmpty = false; } - public MemstoreSize(boolean isEmpty) { + public MemStoreSize(boolean isEmpty) { dataSize = 0; heapSize = 0; this.isEmpty = isEmpty; @@ -50,28 +50,28 @@ public class MemstoreSize { return isEmpty; } - public MemstoreSize(long dataSize, long heapSize) { + public MemStoreSize(long dataSize, long heapSize) { this.dataSize = dataSize; this.heapSize = heapSize; this.isEmpty = false; } - public void incMemstoreSize(long dataSizeDelta, long heapSizeDelta) { + public void incMemStoreSize(long dataSizeDelta, long heapSizeDelta) { this.dataSize += dataSizeDelta; this.heapSize += heapSizeDelta; } - public void incMemstoreSize(MemstoreSize delta) { + public void incMemStoreSize(MemStoreSize delta) { this.dataSize += delta.dataSize; this.heapSize += delta.heapSize; } - public void decMemstoreSize(long dataSizeDelta, long heapSizeDelta) { + public void decMemStoreSize(long dataSizeDelta, long heapSizeDelta) { this.dataSize -= dataSizeDelta; this.heapSize -= heapSizeDelta; } - public void decMemstoreSize(MemstoreSize delta) { + public void decMemStoreSize(MemStoreSize delta) { this.dataSize -= delta.dataSize; this.heapSize -= delta.heapSize; } @@ -86,10 +86,10 @@ public class MemstoreSize { @Override public boolean equals(Object obj) { - if (obj == null || !(obj instanceof MemstoreSize)) { + if (obj == null || !(obj instanceof MemStoreSize)) { return false; } - MemstoreSize other = (MemstoreSize) obj; + MemStoreSize other = (MemStoreSize) obj; return this.dataSize == other.dataSize && this.heapSize == other.heapSize; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServer.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServer.java index 00ca8ec6129..1d0a899c1ce 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServer.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServer.java @@ -148,7 +148,7 @@ public class MetricsRegionServer { public void updateFlush(long t, long memstoreSize, long fileSize) { serverSource.updateFlushTime(t); - serverSource.updateFlushMemstoreSize(memstoreSize); + serverSource.updateFlushMemStoreSize(memstoreSize); serverSource.updateFlushOutputSize(fileSize); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperImpl.java index a99dcd6bffb..a7feb946830 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperImpl.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperImpl.java @@ -284,8 +284,8 @@ class MetricsRegionServerWrapperImpl } @Override - public long getMemstoreLimit() { - return this.regionServer.getRegionServerAccounting().getGlobalMemstoreLimit(); + public long getMemStoreLimit() { + return this.regionServer.getRegionServerAccounting().getGlobalMemStoreLimit(); } @Override @@ -458,7 +458,7 @@ class MetricsRegionServerWrapperImpl public long getNumStores() { return numStores; } - + @Override public long getNumWALFiles() { return numWALFiles; @@ -473,7 +473,7 @@ class MetricsRegionServerWrapperImpl public long getNumWALSlowAppend() { return metricsWALSource.getSlowAppendCount(); } - + @Override public long getNumStoreFiles() { return numStoreFiles; @@ -500,7 +500,7 @@ class MetricsRegionServerWrapperImpl } @Override - public long getMemstoreSize() { + public long getMemStoreSize() { return memstoreSize; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionWrapperImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionWrapperImpl.java index 24e1ae5d6c3..7f37bbf9b25 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionWrapperImpl.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsRegionWrapperImpl.java @@ -110,7 +110,7 @@ public class MetricsRegionWrapperImpl implements MetricsRegionWrapper, Closeable } @Override - public long getMemstoreSize() { + public long getMemStoreSize() { return memstoreSize; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsTableWrapperAggregateImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsTableWrapperAggregateImpl.java index 4eea146a449..10656fefa80 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsTableWrapperAggregateImpl.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MetricsTableWrapperAggregateImpl.java @@ -72,9 +72,9 @@ public class MetricsTableWrapperAggregateImpl implements MetricsTableWrapperAggr for (Store store : r.getStores()) { tempStorefilesSize += store.getStorefilesSize(); } - metricsTable.setMemstoresSize(metricsTable.getMemstoresSize() + r.getMemstoreSize()); + metricsTable.setMemStoresSize(metricsTable.getMemStoresSize() + r.getMemStoreSize()); metricsTable.setStoreFilesSize(metricsTable.getStoreFilesSize() + tempStorefilesSize); - metricsTable.setTableSize(metricsTable.getMemstoresSize() + metricsTable.getStoreFilesSize()); + metricsTable.setTableSize(metricsTable.getMemStoresSize() + metricsTable.getStoreFilesSize()); metricsTable.setReadRequestsCount(metricsTable.getReadRequestsCount() + r.getReadRequestsCount()); metricsTable.setWriteRequestsCount(metricsTable.getWriteRequestsCount() + r.getWriteRequestsCount()); metricsTable.setTotalRequestsCount(metricsTable.getReadRequestsCount() + metricsTable.getWriteRequestsCount()); @@ -133,12 +133,12 @@ public class MetricsTableWrapperAggregateImpl implements MetricsTableWrapperAggr } @Override - public long getMemstoresSize(String table) { + public long getMemStoresSize(String table) { MetricsTableValues metricsTable = metricsTableMap.get(TableName.valueOf(table)); if (metricsTable == null) return 0; else - return metricsTable.getMemstoresSize(); + return metricsTable.getMemStoresSize(); } @Override @@ -197,11 +197,11 @@ public class MetricsTableWrapperAggregateImpl implements MetricsTableWrapperAggr this.writeRequestsCount = writeRequestsCount; } - public long getMemstoresSize() { + public long getMemStoresSize() { return memstoresSize; } - public void setMemstoresSize(long memstoresSize) { + public void setMemStoresSize(long memstoresSize) { this.memstoresSize = memstoresSize; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MutableSegment.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MutableSegment.java index ea8dfde0c58..856f40e0062 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MutableSegment.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/MutableSegment.java @@ -51,11 +51,11 @@ public class MutableSegment extends Segment { * @param mslabUsed whether using MSLAB * @param memstoreSize */ - public void add(Cell cell, boolean mslabUsed, MemstoreSize memstoreSize) { + public void add(Cell cell, boolean mslabUsed, MemStoreSize memstoreSize) { internalAdd(cell, mslabUsed, memstoreSize); } - public void upsert(Cell cell, long readpoint, MemstoreSize memstoreSize) { + public void upsert(Cell cell, long readpoint, MemStoreSize memstoreSize) { internalAdd(cell, false, memstoreSize); // Get the Cells for the row/family/qualifier regardless of timestamp. @@ -89,7 +89,7 @@ public class MutableSegment extends Segment { long heapSize = heapSizeChange(cur, true); this.incSize(-cellLen, -heapSize); if (memstoreSize != null) { - memstoreSize.decMemstoreSize(cellLen, heapSize); + memstoreSize.decMemStoreSize(cellLen, heapSize); } it.remove(); } else { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Region.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Region.java index f410e578ad2..e3ba2fa8da4 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Region.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Region.java @@ -211,7 +211,7 @@ public interface Region extends ConfigurationObserver { * the memstores of this Region. Means size in bytes for key, value and tags within Cells. * It wont consider any java heap overhead for the cell objects or any other. */ - long getMemstoreSize(); + long getMemStoreSize(); /** @return store services for this region, to access services required by store level needs */ RegionServicesForStores getRegionServicesForStores(); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerAccounting.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerAccounting.java index bd8ff170159..8d1656f85b7 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerAccounting.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServerAccounting.java @@ -46,7 +46,7 @@ public class RegionServerAccounting { // Store the edits size during replaying WAL. Use this to roll back the // global memstore size once a region opening failed. - private final ConcurrentMap replayEditsPerRegion = + private final ConcurrentMap replayEditsPerRegion = new ConcurrentSkipListMap<>(Bytes.BYTES_COMPARATOR); private long globalMemStoreLimit; @@ -57,7 +57,7 @@ public class RegionServerAccounting { private long globalOnHeapMemstoreLimitLowMark; public RegionServerAccounting(Configuration conf) { - Pair globalMemstoreSizePair = MemorySizeUtil.getGlobalMemstoreSize(conf); + Pair globalMemstoreSizePair = MemorySizeUtil.getGlobalMemStoreSize(conf); this.globalMemStoreLimit = globalMemstoreSizePair.getFirst(); this.memType = globalMemstoreSizePair.getSecond(); this.globalMemStoreLimitLowMarkPercent = @@ -73,21 +73,21 @@ public class RegionServerAccounting { // "hbase.regionserver.global.memstore.lowerLimit". Can get rid of this boolean passing then. this.globalMemStoreLimitLowMark = (long) (this.globalMemStoreLimit * this.globalMemStoreLimitLowMarkPercent); - this.globalOnHeapMemstoreLimit = MemorySizeUtil.getOnheapGlobalMemstoreSize(conf); + this.globalOnHeapMemstoreLimit = MemorySizeUtil.getOnheapGlobalMemStoreSize(conf); this.globalOnHeapMemstoreLimitLowMark = (long) (this.globalOnHeapMemstoreLimit * this.globalMemStoreLimitLowMarkPercent); } - long getGlobalMemstoreLimit() { + long getGlobalMemStoreLimit() { return this.globalMemStoreLimit; } - long getGlobalOnHeapMemstoreLimit() { + long getGlobalOnHeapMemStoreLimit() { return this.globalOnHeapMemstoreLimit; } // Called by the tuners. - void setGlobalMemstoreLimits(long newGlobalMemstoreLimit) { + void setGlobalMemStoreLimits(long newGlobalMemstoreLimit) { if (this.memType == MemoryType.HEAP) { this.globalMemStoreLimit = newGlobalMemstoreLimit; this.globalMemStoreLimitLowMark = @@ -103,38 +103,38 @@ public class RegionServerAccounting { return this.memType == MemoryType.NON_HEAP; } - long getGlobalMemstoreLimitLowMark() { + long getGlobalMemStoreLimitLowMark() { return this.globalMemStoreLimitLowMark; } - float getGlobalMemstoreLimitLowMarkPercent() { + float getGlobalMemStoreLimitLowMarkPercent() { return this.globalMemStoreLimitLowMarkPercent; } /** * @return the global Memstore data size in the RegionServer */ - public long getGlobalMemstoreDataSize() { + public long getGlobalMemStoreDataSize() { return globalMemstoreDataSize.sum(); } /** * @return the global memstore heap size in the RegionServer */ - public long getGlobalMemstoreHeapSize() { + public long getGlobalMemStoreHeapSize() { return this.globalMemstoreHeapSize.sum(); } /** - * @param memStoreSize the Memstore size will be added to - * the global Memstore size + * @param memStoreSize the Memstore size will be added to + * the global Memstore size */ - public void incGlobalMemstoreSize(MemstoreSize memStoreSize) { + public void incGlobalMemStoreSize(MemStoreSize memStoreSize) { globalMemstoreDataSize.add(memStoreSize.getDataSize()); globalMemstoreHeapSize.add(memStoreSize.getHeapSize()); } - public void decGlobalMemstoreSize(MemstoreSize memStoreSize) { + public void decGlobalMemStoreSize(MemStoreSize memStoreSize) { globalMemstoreDataSize.add(-memStoreSize.getDataSize()); globalMemstoreHeapSize.add(-memStoreSize.getHeapSize()); } @@ -147,7 +147,7 @@ public class RegionServerAccounting { // for onheap memstore we check if the global memstore size and the // global heap overhead is greater than the global memstore limit if (memType == MemoryType.HEAP) { - if (getGlobalMemstoreHeapSize() >= globalMemStoreLimit) { + if (getGlobalMemStoreHeapSize() >= globalMemStoreLimit) { return FlushType.ABOVE_ONHEAP_HIGHER_MARK; } } else { @@ -158,11 +158,11 @@ public class RegionServerAccounting { // global memstore limit 'hbase.regionserver.global.memstore.size'. // We do this to avoid OOME incase of scenarios where the heap is occupied with // lot of onheap references to the cells in memstore - if (getGlobalMemstoreDataSize() >= globalMemStoreLimit) { + if (getGlobalMemStoreDataSize() >= globalMemStoreLimit) { // Indicates that global memstore size is above the configured // 'hbase.regionserver.offheap.global.memstore.size' return FlushType.ABOVE_OFFHEAP_HIGHER_MARK; - } else if (getGlobalMemstoreHeapSize() >= this.globalOnHeapMemstoreLimit) { + } else if (getGlobalMemStoreHeapSize() >= this.globalOnHeapMemstoreLimit) { // Indicates that the offheap memstore's heap overhead is greater than the // configured 'hbase.regionserver.global.memstore.size'. return FlushType.ABOVE_ONHEAP_HIGHER_MARK; @@ -178,15 +178,15 @@ public class RegionServerAccounting { // for onheap memstore we check if the global memstore size and the // global heap overhead is greater than the global memstore lower mark limit if (memType == MemoryType.HEAP) { - if (getGlobalMemstoreHeapSize() >= globalMemStoreLimitLowMark) { + if (getGlobalMemStoreHeapSize() >= globalMemStoreLimitLowMark) { return FlushType.ABOVE_ONHEAP_LOWER_MARK; } } else { - if (getGlobalMemstoreDataSize() >= globalMemStoreLimitLowMark) { + if (getGlobalMemStoreDataSize() >= globalMemStoreLimitLowMark) { // Indicates that the offheap memstore's data size is greater than the global memstore // lower limit return FlushType.ABOVE_OFFHEAP_LOWER_MARK; - } else if (getGlobalMemstoreHeapSize() >= globalOnHeapMemstoreLimitLowMark) { + } else if (getGlobalMemStoreHeapSize() >= globalOnHeapMemstoreLimitLowMark) { // Indicates that the offheap memstore's heap overhead is greater than the global memstore // onheap lower limit return FlushType.ABOVE_ONHEAP_LOWER_MARK; @@ -202,10 +202,10 @@ public class RegionServerAccounting { */ public double getFlushPressure() { if (memType == MemoryType.HEAP) { - return (getGlobalMemstoreHeapSize()) * 1.0 / globalMemStoreLimitLowMark; + return (getGlobalMemStoreHeapSize()) * 1.0 / globalMemStoreLimitLowMark; } else { - return Math.max(getGlobalMemstoreDataSize() * 1.0 / globalMemStoreLimitLowMark, - getGlobalMemstoreHeapSize() * 1.0 / globalOnHeapMemstoreLimitLowMark); + return Math.max(getGlobalMemStoreDataSize() * 1.0 / globalMemStoreLimitLowMark, + getGlobalMemStoreHeapSize() * 1.0 / globalOnHeapMemstoreLimitLowMark); } } @@ -215,37 +215,37 @@ public class RegionServerAccounting { * @param regionName region name. * @param memStoreSize the Memstore size will be added to replayEditsPerRegion. */ - public void addRegionReplayEditsSize(byte[] regionName, MemstoreSize memStoreSize) { - MemstoreSize replayEdistsSize = replayEditsPerRegion.get(regionName); - // All ops on the same MemstoreSize object is going to be done by single thread, sequentially + public void addRegionReplayEditsSize(byte[] regionName, MemStoreSize memStoreSize) { + MemStoreSize replayEdistsSize = replayEditsPerRegion.get(regionName); + // All ops on the same MemStoreSize object is going to be done by single thread, sequentially // only. First calls to this method to increment the per region reply edits size and then call // to either rollbackRegionReplayEditsSize or clearRegionReplayEditsSize as per the result of // the region open operation. No need to handle multi thread issues on one region's entry in // this Map. if (replayEdistsSize == null) { - replayEdistsSize = new MemstoreSize(); + replayEdistsSize = new MemStoreSize(); replayEditsPerRegion.put(regionName, replayEdistsSize); } - replayEdistsSize.incMemstoreSize(memStoreSize); + replayEdistsSize.incMemStoreSize(memStoreSize); } /** * Roll back the global MemStore size for a specified region when this region * can't be opened. - * + * * @param regionName the region which could not open. */ public void rollbackRegionReplayEditsSize(byte[] regionName) { - MemstoreSize replayEditsSize = replayEditsPerRegion.get(regionName); + MemStoreSize replayEditsSize = replayEditsPerRegion.get(regionName); if (replayEditsSize != null) { clearRegionReplayEditsSize(regionName); - decGlobalMemstoreSize(replayEditsSize); + decGlobalMemStoreSize(replayEditsSize); } } /** * Clear a region from replayEditsPerRegion. - * + * * @param regionName region name. */ public void clearRegionReplayEditsSize(byte[] regionName) { diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServicesForStores.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServicesForStores.java index aea92f88c37..61815a540df 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServicesForStores.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/RegionServicesForStores.java @@ -27,6 +27,8 @@ import org.apache.hadoop.hbase.client.RegionInfo; import org.apache.hadoop.hbase.wal.WAL; import org.apache.yetus.audience.InterfaceAudience; +import org.apache.hadoop.hbase.shaded.com.google.common.annotations.VisibleForTesting; + /** * Services a Store needs from a Region. * RegionServicesForStores class is the interface through which memstore access services at the @@ -63,8 +65,8 @@ public class RegionServicesForStores { region.unblockUpdates(); } - public void addMemstoreSize(MemstoreSize size) { - region.addAndGetMemstoreSize(size); + public void addMemStoreSize(MemStoreSize size) { + region.addAndGetMemStoreSize(size); } public RegionInfo getRegionInfo() { @@ -77,16 +79,16 @@ public class RegionServicesForStores { public ThreadPoolExecutor getInMemoryCompactionPool() { return INMEMORY_COMPACTION_POOL; } - public long getMemstoreFlushSize() { - return region.getMemstoreFlushSize(); + public long getMemStoreFlushSize() { + return region.getMemStoreFlushSize(); } public int getNumStores() { return region.getTableDescriptor().getColumnFamilyCount(); } - // methods for tests - long getMemstoreSize() { - return region.getMemstoreSize(); + @VisibleForTesting + long getMemStoreSize() { + return region.getMemStoreSize(); } } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Segment.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Segment.java index 0b5d81a2817..23b386f3df0 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Segment.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Segment.java @@ -275,13 +275,13 @@ public abstract class Segment { return comparator; } - protected void internalAdd(Cell cell, boolean mslabUsed, MemstoreSize memstoreSize) { + protected void internalAdd(Cell cell, boolean mslabUsed, MemStoreSize memstoreSize) { boolean succ = getCellSet().add(cell); updateMetaInfo(cell, succ, mslabUsed, memstoreSize); } protected void updateMetaInfo(Cell cellToAdd, boolean succ, boolean mslabUsed, - MemstoreSize memstoreSize) { + MemStoreSize memstoreSize) { long cellSize = 0; // If there's already a same cell in the CellSet and we are using MSLAB, we must count in the // MSLAB allocation size as well, or else there will be memory leak (occupied heap size larger @@ -292,7 +292,7 @@ public abstract class Segment { long heapSize = heapSizeChange(cellToAdd, succ); incSize(cellSize, heapSize); if (memstoreSize != null) { - memstoreSize.incMemstoreSize(cellSize, heapSize); + memstoreSize.incMemStoreSize(cellSize, heapSize); } getTimeRangeTracker().includeTimestamp(cellToAdd); minSequenceId = Math.min(minSequenceId, cellToAdd.getSequenceId()); @@ -305,7 +305,7 @@ public abstract class Segment { } } - protected void updateMetaInfo(Cell cellToAdd, boolean succ, MemstoreSize memstoreSize) { + protected void updateMetaInfo(Cell cellToAdd, boolean succ, MemStoreSize memstoreSize) { updateMetaInfo(cellToAdd, succ, (getMemStoreLAB()!=null), memstoreSize); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SegmentFactory.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SegmentFactory.java index d609e6f231f..63d1baae221 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SegmentFactory.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/SegmentFactory.java @@ -18,7 +18,6 @@ */ package org.apache.hadoop.hbase.regionserver; -import org.apache.hadoop.hbase.shaded.com.google.common.base.Preconditions; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.CellComparator; import org.apache.yetus.audience.InterfaceAudience; @@ -96,7 +95,7 @@ public final class SegmentFactory { // create flat immutable segment from non-flat immutable segment // for flattening public ImmutableSegment createImmutableSegmentByFlattening( - CSLMImmutableSegment segment, CompactingMemStore.IndexType idxType, MemstoreSize memstoreSize) { + CSLMImmutableSegment segment, CompactingMemStore.IndexType idxType, MemStoreSize memstoreSize) { ImmutableSegment res = null; switch (idxType) { case CHUNK_MAP: diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java index 9c01301b24f..6f6f31c3320 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/Store.java @@ -97,19 +97,19 @@ public interface Store { /** * @return The size of this store's memstore. */ - MemstoreSize getMemStoreSize(); + MemStoreSize getMemStoreSize(); /** * @return The amount of memory we could flush from this memstore; usually this is equal to * {@link #getMemStoreSize()} unless we are carrying snapshots and then it will be the size of * outstanding snapshots. */ - MemstoreSize getFlushableSize(); + MemStoreSize getFlushableSize(); /** * @return size of the memstore snapshot */ - MemstoreSize getSnapshotSize(); + MemStoreSize getSnapshotSize(); ColumnFamilyDescriptor getColumnFamilyDescriptor(); @@ -121,7 +121,7 @@ public interface Store { /** * @return The maximum memstoreTS in all store files. */ - OptionalLong getMaxMemstoreTS(); + OptionalLong getMaxMemStoreTS(); /** @return aggregate size of all HStores used in the last compaction */ long getLastCompactSize(); @@ -289,5 +289,5 @@ public interface Store { /** * @return true if the memstore may need some extra memory space */ - boolean isSloppyMemstore(); + boolean isSloppyMemStore(); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreConfigInformation.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreConfigInformation.java index 63493b77b38..f9cce7f85d9 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreConfigInformation.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreConfigInformation.java @@ -35,7 +35,7 @@ public interface StoreConfigInformation { */ // TODO: Why is this in here? It should be in Store and it should return the Store flush size, // not the Regions. St.Ack - long getMemstoreFlushSize(); + long getMemStoreFlushSize(); /** * @return Gets the cf-specific time-to-live for store files. diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java index 0097bd76477..c11c8a3a24c 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreFile.java @@ -59,7 +59,7 @@ public interface StoreFile { /** * Get max of the MemstoreTS in the KV's in this store file. */ - long getMaxMemstoreTS(); + long getMaxMemStoreTS(); /** * @return Path or null if this StoreFile was made with a Stream. diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreUtils.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreUtils.java index 354b056f21b..2ada5a99f9c 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreUtils.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StoreUtils.java @@ -81,8 +81,8 @@ public class StoreUtils { * were created by a mapreduce bulk load are ignored, as they do not correspond to any specific * put operation, and thus do not have a memstoreTS associated with them. */ - public static OptionalLong getMaxMemstoreTSInList(Collection sfs) { - return sfs.stream().filter(sf -> !sf.isBulkLoadResult()).mapToLong(HStoreFile::getMaxMemstoreTS) + public static OptionalLong getMaxMemStoreTSInList(Collection sfs) { + return sfs.stream().filter(sf -> !sf.isBulkLoadResult()).mapToLong(HStoreFile::getMaxMemStoreTS) .max(); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StripeStoreConfig.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StripeStoreConfig.java index 169d1d87eb6..eb2a9b6d96e 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StripeStoreConfig.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/StripeStoreConfig.java @@ -96,7 +96,7 @@ public class StripeStoreConfig { this.splitPartCount = splitPartCount; // Arbitrary default split size - 4 times the size of one L0 compaction. // If we flush into L0 there's no split compaction, but for default value it is ok. - double flushSize = sci.getMemstoreFlushSize(); + double flushSize = sci.getMemStoreFlushSize(); if (flushSize == 0) { flushSize = 128 * 1024 * 1024; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CompactionConfiguration.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CompactionConfiguration.java index fe9ae306ed2..b8194eb55bd 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CompactionConfiguration.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/CompactionConfiguration.java @@ -115,10 +115,10 @@ public class CompactionConfiguration { this.storeConfigInfo = storeConfigInfo; maxCompactSize = conf.getLong(HBASE_HSTORE_COMPACTION_MAX_SIZE_KEY, Long.MAX_VALUE); - offPeakMaxCompactSize = conf.getLong(HBASE_HSTORE_COMPACTION_MAX_SIZE_OFFPEAK_KEY, - maxCompactSize); + offPeakMaxCompactSize = conf.getLong(HBASE_HSTORE_COMPACTION_MAX_SIZE_OFFPEAK_KEY, + maxCompactSize); minCompactSize = conf.getLong(HBASE_HSTORE_COMPACTION_MIN_SIZE_KEY, - storeConfigInfo.getMemstoreFlushSize()); + storeConfigInfo.getMemStoreFlushSize()); minFilesToCompact = Math.max(2, conf.getInt(HBASE_HSTORE_COMPACTION_MIN_KEY, /*old name*/ conf.getInt("hbase.hstore.compactionThreshold", 3))); maxFilesToCompact = conf.getInt(HBASE_HSTORE_COMPACTION_MAX_KEY, 10); @@ -126,7 +126,7 @@ public class CompactionConfiguration { offPeakCompactionRatio = conf.getFloat(HBASE_HSTORE_COMPACTION_RATIO_OFFPEAK_KEY, 5.0F); throttlePoint = conf.getLong("hbase.regionserver.thread.compaction.throttle", - 2 * maxFilesToCompact * storeConfigInfo.getMemstoreFlushSize()); + 2 * maxFilesToCompact * storeConfigInfo.getMemStoreFlushSize()); majorCompactionPeriod = conf.getLong(HConstants.MAJOR_COMPACTION_PERIOD, 1000*60*60*24*7); // Make it 0.5 so jitter has us fall evenly either side of when the compaction should run majorCompactionJitter = conf.getFloat("hbase.hregion.majorcompaction.jitter", 0.50F); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/Compactor.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/Compactor.java index 2c9a519a486..5865ed56651 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/Compactor.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/compactions/Compactor.java @@ -77,7 +77,7 @@ public abstract class Compactor { protected final int compactionKVMax; protected final Compression.Algorithm compactionCompression; - /** specify how many days to keep MVCC values during major compaction **/ + /** specify how many days to keep MVCC values during major compaction **/ protected int keepSeqIdPeriod; // Configs that drive whether we drop page cache behind compactions @@ -141,15 +141,15 @@ public abstract class Compactor { protected FileDetails getFileDetails( Collection filesToCompact, boolean allFiles) throws IOException { FileDetails fd = new FileDetails(); - long oldestHFileTimeStampToKeepMVCC = System.currentTimeMillis() - - (1000L * 60 * 60 * 24 * this.keepSeqIdPeriod); + long oldestHFileTimeStampToKeepMVCC = System.currentTimeMillis() - + (1000L * 60 * 60 * 24 * this.keepSeqIdPeriod); for (HStoreFile file : filesToCompact) { if(allFiles && (file.getModificationTimeStamp() < oldestHFileTimeStampToKeepMVCC)) { - // when isAllFiles is true, all files are compacted so we can calculate the smallest + // when isAllFiles is true, all files are compacted so we can calculate the smallest // MVCC value to keep - if(fd.minSeqIdToKeep < file.getMaxMemstoreTS()) { - fd.minSeqIdToKeep = file.getMaxMemstoreTS(); + if(fd.minSeqIdToKeep < file.getMaxMemStoreTS()) { + fd.minSeqIdToKeep = file.getMaxMemStoreTS(); } } long seqNum = file.getMaxSequenceId(); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java index 3868ba75acd..84559682af5 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/AbstractFSWAL.java @@ -302,7 +302,7 @@ public abstract class AbstractFSWAL implements WAL { } private int calculateMaxLogFiles(Configuration conf, long logRollSize) { - Pair globalMemstoreSize = MemorySizeUtil.getGlobalMemstoreSize(conf); + Pair globalMemstoreSize = MemorySizeUtil.getGlobalMemStoreSize(conf); return (int) ((globalMemstoreSize.getFirst() * 2) / logRollSize); } @@ -468,13 +468,13 @@ public abstract class AbstractFSWAL implements WAL { } @Override - public long getEarliestMemstoreSeqNum(byte[] encodedRegionName) { + public long getEarliestMemStoreSeqNum(byte[] encodedRegionName) { // Used by tests. Deprecated as too subtle for general usage. return this.sequenceIdAccounting.getLowestSequenceId(encodedRegionName); } @Override - public long getEarliestMemstoreSeqNum(byte[] encodedRegionName, byte[] familyName) { + public long getEarliestMemStoreSeqNum(byte[] encodedRegionName, byte[] familyName) { // This method is used by tests and for figuring if we should flush or not because our // sequenceids are too old. It is also used reporting the master our oldest sequenceid for use // figuring what edits can be skipped during log recovery. getEarliestMemStoreSequenceId @@ -924,7 +924,7 @@ public abstract class AbstractFSWAL implements WAL { assert highestUnsyncedTxid < entry.getTxid(); highestUnsyncedTxid = entry.getTxid(); sequenceIdAccounting.update(encodedRegionName, entry.getFamilyNames(), regionSequenceId, - entry.isInMemstore()); + entry.isInMemStore()); coprocessorHost.postWALWrite(entry.getRegionInfo(), entry.getKey(), entry.getEdit()); // Update metrics. postAppend(entry, EnvironmentEdgeManager.currentTime() - start); diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSWALEntry.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSWALEntry.java index 0c833745297..f9374d86f1c 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSWALEntry.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/wal/FSWALEntry.java @@ -92,7 +92,7 @@ class FSWALEntry extends Entry { return "sequence=" + this.txid + ", " + super.toString(); }; - boolean isInMemstore() { + boolean isInMemStore() { return this.inMemstore; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RegionReplicaReplicationEndpoint.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RegionReplicaReplicationEndpoint.java index e1e2514683d..69365d804a5 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RegionReplicaReplicationEndpoint.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/replication/regionserver/RegionReplicaReplicationEndpoint.java @@ -400,7 +400,7 @@ public class RegionReplicaReplicationEndpoint extends HBaseReplicationEndpoint { // check if the table requires memstore replication // some unit-test drop the table, so we should do a bypass check and always replicate. TableDescriptor htd = tableDescriptors.get(tableName); - requiresReplication = htd == null || htd.hasRegionMemstoreReplication(); + requiresReplication = htd == null || htd.hasRegionMemStoreReplication(); memstoreReplicationEnabled.put(tableName, requiresReplication); } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/DisabledWALProvider.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/DisabledWALProvider.java index 652aa2fd90f..14586318c42 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/DisabledWALProvider.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/DisabledWALProvider.java @@ -220,12 +220,12 @@ class DisabledWALProvider implements WALProvider { } @Override - public long getEarliestMemstoreSeqNum(byte[] encodedRegionName) { + public long getEarliestMemStoreSeqNum(byte[] encodedRegionName) { return HConstants.NO_SEQNUM; } @Override - public long getEarliestMemstoreSeqNum(byte[] encodedRegionName, byte[] familyName) { + public long getEarliestMemStoreSeqNum(byte[] encodedRegionName, byte[] familyName) { return HConstants.NO_SEQNUM; } diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WAL.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WAL.java index 886ec78496c..e3192559518 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WAL.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/wal/WAL.java @@ -190,11 +190,11 @@ public interface WAL extends Closeable, WALFileLengthProvider { * @param encodedRegionName The region to get the number for. * @return The earliest/lowest/oldest sequence id if present, HConstants.NO_SEQNUM if absent. * @deprecated Since version 1.2.0. Removing because not used and exposes subtle internal - * workings. Use {@link #getEarliestMemstoreSeqNum(byte[], byte[])} + * workings. Use {@link #getEarliestMemStoreSeqNum(byte[], byte[])} */ @VisibleForTesting @Deprecated - long getEarliestMemstoreSeqNum(byte[] encodedRegionName); + long getEarliestMemStoreSeqNum(byte[] encodedRegionName); /** * Gets the earliest unflushed sequence id in the memstore for the store. @@ -202,7 +202,7 @@ public interface WAL extends Closeable, WALFileLengthProvider { * @param familyName The family to get the number for. * @return The earliest/lowest/oldest sequence id if present, HConstants.NO_SEQNUM if absent. */ - long getEarliestMemstoreSeqNum(byte[] encodedRegionName, byte[] familyName); + long getEarliestMemStoreSeqNum(byte[] encodedRegionName, byte[] familyName); /** * Human readable identifying information about the state of this WAL. diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestGlobalMemStoreSize.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestGlobalMemStoreSize.java index 21b36b7baa1..202ea4b8ad6 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/TestGlobalMemStoreSize.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/TestGlobalMemStoreSize.java @@ -95,9 +95,9 @@ public class TestGlobalMemStoreSize { long globalMemStoreSize = 0; for (RegionInfo regionInfo : ProtobufUtil.getOnlineRegions(null, server.getRSRpcServices())) { - globalMemStoreSize += server.getRegion(regionInfo.getEncodedName()).getMemstoreSize(); + globalMemStoreSize += server.getRegion(regionInfo.getEncodedName()).getMemStoreSize(); } - assertEquals(server.getRegionServerAccounting().getGlobalMemstoreDataSize(), + assertEquals(server.getRegionServerAccounting().getGlobalMemStoreDataSize(), globalMemStoreSize); } @@ -105,7 +105,7 @@ public class TestGlobalMemStoreSize { int i = 0; for (HRegionServer server : getOnlineRegionServers()) { LOG.info("Starting flushes on " + server.getServerName() + - ", size=" + server.getRegionServerAccounting().getGlobalMemstoreDataSize()); + ", size=" + server.getRegionServerAccounting().getGlobalMemStoreDataSize()); for (RegionInfo regionInfo : ProtobufUtil.getOnlineRegions(null, server.getRSRpcServices())) { @@ -115,18 +115,18 @@ public class TestGlobalMemStoreSize { LOG.info("Post flush on " + server.getServerName()); long now = System.currentTimeMillis(); long timeout = now + 1000; - while(server.getRegionServerAccounting().getGlobalMemstoreDataSize() != 0 && + while(server.getRegionServerAccounting().getGlobalMemStoreDataSize() != 0 && timeout < System.currentTimeMillis()) { Threads.sleep(10); } - long size = server.getRegionServerAccounting().getGlobalMemstoreDataSize(); + long size = server.getRegionServerAccounting().getGlobalMemStoreDataSize(); if (size > 0) { // If size > 0, see if its because the meta region got edits while // our test was running.... for (RegionInfo regionInfo : ProtobufUtil.getOnlineRegions(null, server.getRSRpcServices())) { Region r = server.getRegion(regionInfo.getEncodedName()); - long l = r.getMemstoreSize(); + long l = r.getMemStoreSize(); if (l > 0) { // Only meta could have edits at this stage. Give it another flush // clear them. @@ -136,7 +136,7 @@ public class TestGlobalMemStoreSize { } } } - size = server.getRegionServerAccounting().getGlobalMemstoreDataSize(); + size = server.getRegionServerAccounting().getGlobalMemStoreDataSize(); assertEquals("Server=" + server.getServerName() + ", i=" + i++, 0, size); } @@ -154,7 +154,7 @@ public class TestGlobalMemStoreSize { throws IOException { LOG.info("Flush " + r.toString() + " on " + server.getServerName() + ", " + r.flush(true) + ", size=" + - server.getRegionServerAccounting().getGlobalMemstoreDataSize()); + server.getRegionServerAccounting().getGlobalMemStoreDataSize()); } private List getOnlineRegionServers() { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionAdminApi.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionAdminApi.java index 9775b86e310..3966cf27b00 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionAdminApi.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestAsyncRegionAdminApi.java @@ -325,30 +325,30 @@ public class TestAsyncRegionAdminApi extends TestAsyncAdminBase { ASYNC_CONN.getRawTable(tableName) .put(new Put(hri.getStartKey()).addColumn(FAMILY, FAMILY_0, Bytes.toBytes("value-1"))) .join(); - Assert.assertTrue(regionServer.getOnlineRegion(hri.getRegionName()).getMemstoreSize() > 0); + Assert.assertTrue(regionServer.getOnlineRegion(hri.getRegionName()).getMemStoreSize() > 0); // flush region and wait flush operation finished. LOG.info("flushing region: " + Bytes.toStringBinary(hri.getRegionName())); admin.flushRegion(hri.getRegionName()).get(); LOG.info("blocking until flush is complete: " + Bytes.toStringBinary(hri.getRegionName())); Threads.sleepWithoutInterrupt(500); - while (regionServer.getOnlineRegion(hri.getRegionName()).getMemstoreSize() > 0) { + while (regionServer.getOnlineRegion(hri.getRegionName()).getMemStoreSize() > 0) { Threads.sleep(50); } // check the memstore. - Assert.assertEquals(regionServer.getOnlineRegion(hri.getRegionName()).getMemstoreSize(), 0); + Assert.assertEquals(regionServer.getOnlineRegion(hri.getRegionName()).getMemStoreSize(), 0); // write another put into the specific region ASYNC_CONN.getRawTable(tableName) .put(new Put(hri.getStartKey()).addColumn(FAMILY, FAMILY_0, Bytes.toBytes("value-2"))) .join(); - Assert.assertTrue(regionServer.getOnlineRegion(hri.getRegionName()).getMemstoreSize() > 0); + Assert.assertTrue(regionServer.getOnlineRegion(hri.getRegionName()).getMemStoreSize() > 0); admin.flush(tableName).get(); Threads.sleepWithoutInterrupt(500); - while (regionServer.getOnlineRegion(hri.getRegionName()).getMemstoreSize() > 0) { + while (regionServer.getOnlineRegion(hri.getRegionName()).getMemStoreSize() > 0) { Threads.sleep(50); } // check the memstore. - Assert.assertEquals(regionServer.getOnlineRegion(hri.getRegionName()).getMemstoreSize(), 0); + Assert.assertEquals(regionServer.getOnlineRegion(hri.getRegionName()).getMemStoreSize(), 0); } @Test diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientPushback.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientPushback.java index 1b1805e210f..12c7faeb394 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientPushback.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/client/TestClientPushback.java @@ -28,14 +28,13 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hbase.*; -import org.apache.hadoop.hbase.client.AsyncProcessTask; import org.apache.hadoop.hbase.client.backoff.ClientBackoffPolicy; import org.apache.hadoop.hbase.client.backoff.ExponentialClientBackoffPolicy; import org.apache.hadoop.hbase.client.backoff.ServerStatistics; import org.apache.hadoop.hbase.client.coprocessor.Batch; import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.HRegionServer; -import org.apache.hadoop.hbase.regionserver.MemstoreSize; +import org.apache.hadoop.hbase.regionserver.MemStoreSize; import org.apache.hadoop.hbase.regionserver.Region; import org.apache.hadoop.hbase.testclassification.MediumTests; import org.apache.hadoop.hbase.util.Bytes; @@ -106,7 +105,7 @@ public class TestClientPushback { mutator.flush(); // get the current load on RS. Hopefully memstore isn't flushed since we wrote the the data - int load = (int) ((((HRegion) region).addAndGetMemstoreSize(new MemstoreSize(0, 0)) * 100) + int load = (int) ((((HRegion) region).addAndGetMemStoreSize(new MemStoreSize(0, 0)) * 100) / flushSizeBytes); LOG.debug("Done writing some data to "+tableName); @@ -114,7 +113,7 @@ public class TestClientPushback { ClientBackoffPolicy backoffPolicy = conn.getBackoffPolicy(); assertTrue("Backoff policy is not correctly configured", backoffPolicy instanceof ExponentialClientBackoffPolicy); - + ServerStatisticTracker stats = conn.getStatisticsTracker(); assertNotNull( "No stats configured for the client!", stats); // get the names so we can query the stats @@ -125,7 +124,7 @@ public class TestClientPushback { ServerStatistics serverStats = stats.getServerStatsForTesting(server); ServerStatistics.RegionStatistics regionStats = serverStats.getStatsForRegion(regionName); assertEquals("We did not find some load on the memstore", load, - regionStats.getMemstoreLoadPercent()); + regionStats.getMemStoreLoadPercent()); // check that the load reported produces a nonzero delay long backoffTime = backoffPolicy.getBackoffTime(server, regionName, serverStats); assertNotEquals("Reported load does not produce a backoff", backoffTime, 0); @@ -163,7 +162,7 @@ public class TestClientPushback { assertEquals(rsStats.heapOccupancyHist.getSnapshot().getMean(), (double)regionStats.getHeapOccupancyPercent(), 0.1 ); assertEquals(rsStats.memstoreLoadHist.getSnapshot().getMean(), - (double)regionStats.getMemstoreLoadPercent(), 0.1); + (double)regionStats.getMemStoreLoadPercent(), 0.1); MetricsConnection.RunnerStats runnerStats = conn.getConnectionMetrics().runnerStats; @@ -202,6 +201,6 @@ public class TestClientPushback { ServerStatistics.RegionStatistics regionStats = serverStats.getStatsForRegion(regionName); assertNotNull(regionStats); - assertTrue(regionStats.getMemstoreLoadPercent() > 0); + assertTrue(regionStats.getMemStoreLoadPercent() > 0); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestNegativeMemstoreSizeWithSlowCoprocessor.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestNegativeMemStoreSizeWithSlowCoprocessor.java similarity index 94% rename from hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestNegativeMemstoreSizeWithSlowCoprocessor.java rename to hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestNegativeMemStoreSizeWithSlowCoprocessor.java index 42a8ee408b0..30b3d71a854 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestNegativeMemstoreSizeWithSlowCoprocessor.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestNegativeMemStoreSizeWithSlowCoprocessor.java @@ -22,7 +22,7 @@ import org.apache.hadoop.hbase.client.Durability; import org.apache.hadoop.hbase.client.Put; import org.apache.hadoop.hbase.client.Table; import org.apache.hadoop.hbase.regionserver.HRegion; -import org.apache.hadoop.hbase.regionserver.MemstoreSize; +import org.apache.hadoop.hbase.regionserver.MemStoreSize; import org.apache.hadoop.hbase.wal.WALEdit; import org.apache.hadoop.hbase.testclassification.LargeTests; import org.apache.hadoop.hbase.util.Bytes; @@ -38,9 +38,9 @@ import org.junit.experimental.categories.Category; * simulate this we call flush from the coprocessor itself */ @Category(LargeTests.class) -public class TestNegativeMemstoreSizeWithSlowCoprocessor { +public class TestNegativeMemStoreSizeWithSlowCoprocessor { - static final Log LOG = LogFactory.getLog(TestNegativeMemstoreSizeWithSlowCoprocessor.class); + static final Log LOG = LogFactory.getLog(TestNegativeMemStoreSizeWithSlowCoprocessor.class); private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); private static final byte[] tableName = Bytes.toBytes("test_table"); private static final byte[] family = Bytes.toBytes("f"); @@ -97,7 +97,7 @@ public class TestNegativeMemstoreSizeWithSlowCoprocessor { if (Bytes.equals(put.getRow(), Bytes.toBytes("row2"))) { region.flush(false); - Assert.assertTrue(region.addAndGetMemstoreSize(new MemstoreSize()) >= 0); + Assert.assertTrue(region.addAndGetMemStoreSize(new MemStoreSize()) >= 0); } } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlock.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlock.java index c5bf58179e8..334127c48ec 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlock.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/io/hfile/TestHFileBlock.java @@ -418,7 +418,7 @@ public class TestHFileBlock { .build(); HFileBlock.FSReaderImpl hbr = new HFileBlock.FSReaderImpl(is, totalSize, meta); hbr.setDataBlockEncoder(dataBlockEncoder); - hbr.setIncludesMemstoreTS(includesMemstoreTS); + hbr.setIncludesMemStoreTS(includesMemstoreTS); HFileBlock blockFromHFile, blockUnpacked; int pos = 0; for (int blockId = 0; blockId < numBlocks; ++blockId) { diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancer.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancer.java index e547f87192d..68d009dfa01 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancer.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/master/balancer/TestStochasticLoadBalancer.java @@ -343,7 +343,7 @@ public class TestStochasticLoadBalancer extends BalancerTestBase { public void testCostFromArray() { Configuration conf = HBaseConfiguration.create(); StochasticLoadBalancer.CostFromRegionLoadFunction - costFunction = new StochasticLoadBalancer.MemstoreSizeCostFunction(conf); + costFunction = new StochasticLoadBalancer.MemStoreSizeCostFunction(conf); costFunction.init(mockCluster(new int[]{0, 0, 0, 0, 1})); double[] statOne = new double[100]; diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperStub.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperStub.java index e2b4ab398ff..d56823e0ddb 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperStub.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/MetricsRegionServerWrapperStub.java @@ -61,7 +61,7 @@ public class MetricsRegionServerWrapperStub implements MetricsRegionServerWrappe } @Override - public long getMemstoreSize() { + public long getMemStoreSize() { return 1025; } @@ -206,7 +206,7 @@ public class MetricsRegionServerWrapperStub implements MetricsRegionServerWrappe } @Override - public long getMemstoreLimit() { + public long getMemStoreLimit() { return 419; } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/MetricsRegionWrapperStub.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/MetricsRegionWrapperStub.java index 6f32000a3df..524d03b8b50 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/MetricsRegionWrapperStub.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/MetricsRegionWrapperStub.java @@ -61,7 +61,7 @@ public class MetricsRegionWrapperStub implements MetricsRegionWrapper { } @Override - public long getMemstoreSize() { + public long getMemStoreSize() { return 103; } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/MetricsTableWrapperStub.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/MetricsTableWrapperStub.java index 6fd8dd7135b..ba333a5bb85 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/MetricsTableWrapperStub.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/MetricsTableWrapperStub.java @@ -42,7 +42,7 @@ public class MetricsTableWrapperStub implements MetricsTableWrapperAggregate { } @Override - public long getMemstoresSize(String table) { + public long getMemStoresSize(String table) { return 1000; } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactingMemStore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactingMemStore.java index dc3cf4d9023..0886fd124a2 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactingMemStore.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactingMemStore.java @@ -260,7 +260,7 @@ public class TestCompactingMemStore extends TestDefaultMemStore { @Override @Test public void testUpsertMemstoreSize() throws Exception { - MemstoreSize oldSize = memstore.size(); + MemStoreSize oldSize = memstore.size(); List l = new ArrayList<>(); KeyValue kv1 = KeyValueTestUtil.create("r", "f", "q", 100, "v"); @@ -275,7 +275,7 @@ public class TestCompactingMemStore extends TestDefaultMemStore { l.add(kv3); this.memstore.upsert(l, 2, null);// readpoint is 2 - MemstoreSize newSize = this.memstore.size(); + MemStoreSize newSize = this.memstore.size(); assert (newSize.getDataSize() > oldSize.getDataSize()); //The kv1 should be removed. assert (memstore.getActive().getCellsCount() == 2); @@ -593,7 +593,7 @@ public class TestCompactingMemStore extends TestDefaultMemStore { .length(kv)); long totalHeapSize = numOfCells * oneCellOnCSLMHeapSize + MutableSegment.DEEP_OVERHEAD; - assertEquals(totalCellsLen, regionServicesForStores.getMemstoreSize()); + assertEquals(totalCellsLen, regionServicesForStores.getMemStoreSize()); assertEquals(totalHeapSize, ((CompactingMemStore)memstore).heapSize()); ((CompactingMemStore)memstore).flushInMemory(); // push keys to pipeline and flatten @@ -605,15 +605,15 @@ public class TestCompactingMemStore extends TestDefaultMemStore { totalHeapSize = MutableSegment.DEEP_OVERHEAD + CellChunkImmutableSegment.DEEP_OVERHEAD_CCM + numOfCells * oneCellOnCCMHeapSize; - assertEquals(totalCellsLen, regionServicesForStores.getMemstoreSize()); + assertEquals(totalCellsLen, regionServicesForStores.getMemStoreSize()); assertEquals(totalHeapSize, ((CompactingMemStore)memstore).heapSize()); - MemstoreSize size = memstore.getFlushableSize(); + MemStoreSize size = memstore.getFlushableSize(); MemStoreSnapshot snapshot = memstore.snapshot(); // push keys to snapshot - region.decrMemstoreSize(size); // simulate flusher + region.decrMemStoreSize(size); // simulate flusher ImmutableSegment s = memstore.getSnapshot(); assertEquals(numOfCells, s.getCellsCount()); - assertEquals(0, regionServicesForStores.getMemstoreSize()); + assertEquals(0, regionServicesForStores.getMemStoreSize()); memstore.clearSnapshot(snapshot.getId()); } @@ -638,7 +638,7 @@ public class TestCompactingMemStore extends TestDefaultMemStore { int oneCellOnCSLMHeapSize = 120; int oneCellOnCAHeapSize = 88; long totalHeapSize = MutableSegment.DEEP_OVERHEAD + 4 * oneCellOnCSLMHeapSize; - assertEquals(totalCellsLen, regionServicesForStores.getMemstoreSize()); + assertEquals(totalCellsLen, regionServicesForStores.getMemStoreSize()); assertEquals(totalHeapSize, ((CompactingMemStore)memstore).heapSize()); ((CompactingMemStore)memstore).flushInMemory(); // push keys to pipeline and compact @@ -647,15 +647,15 @@ public class TestCompactingMemStore extends TestDefaultMemStore { // totalCellsLen remains the same totalHeapSize = MutableSegment.DEEP_OVERHEAD + CellArrayImmutableSegment.DEEP_OVERHEAD_CAM + 4 * oneCellOnCAHeapSize; - assertEquals(totalCellsLen, regionServicesForStores.getMemstoreSize()); + assertEquals(totalCellsLen, regionServicesForStores.getMemStoreSize()); assertEquals(totalHeapSize, ((CompactingMemStore)memstore).heapSize()); - MemstoreSize size = memstore.getFlushableSize(); + MemStoreSize size = memstore.getFlushableSize(); MemStoreSnapshot snapshot = memstore.snapshot(); // push keys to snapshot - region.decrMemstoreSize(size); // simulate flusher + region.decrMemStoreSize(size); // simulate flusher ImmutableSegment s = memstore.getSnapshot(); assertEquals(4, s.getCellsCount()); - assertEquals(0, regionServicesForStores.getMemstoreSize()); + assertEquals(0, regionServicesForStores.getMemStoreSize()); memstore.clearSnapshot(snapshot.getId()); } @@ -677,7 +677,7 @@ public class TestCompactingMemStore extends TestDefaultMemStore { int oneCellOnCAHeapSize = 88; long totalHeapSize = MutableSegment.DEEP_OVERHEAD + 4 * oneCellOnCSLMHeapSize; - assertEquals(totalCellsLen1, regionServicesForStores.getMemstoreSize()); + assertEquals(totalCellsLen1, regionServicesForStores.getMemStoreSize()); assertEquals(totalHeapSize, ((CompactingMemStore)memstore).heapSize()); ((CompactingMemStore)memstore).flushInMemory(); // push keys to pipeline and compact @@ -689,30 +689,30 @@ public class TestCompactingMemStore extends TestDefaultMemStore { assertEquals(0, memstore.getSnapshot().getCellsCount()); // There is no compaction, as the compacting memstore type is basic. // totalCellsLen remains the same - assertEquals(totalCellsLen1, regionServicesForStores.getMemstoreSize()); + assertEquals(totalCellsLen1, regionServicesForStores.getMemStoreSize()); totalHeapSize = MutableSegment.DEEP_OVERHEAD + CellArrayImmutableSegment.DEEP_OVERHEAD_CAM + 4 * oneCellOnCAHeapSize; assertEquals(totalHeapSize, ((CompactingMemStore)memstore).heapSize()); int totalCellsLen2 = addRowsByKeys(memstore, keys2); totalHeapSize += 3 * oneCellOnCSLMHeapSize; - assertEquals(totalCellsLen1 + totalCellsLen2, regionServicesForStores.getMemstoreSize()); + assertEquals(totalCellsLen1 + totalCellsLen2, regionServicesForStores.getMemStoreSize()); assertEquals(totalHeapSize, ((CompactingMemStore) memstore).heapSize()); - MemstoreSize size = memstore.getFlushableSize(); + MemStoreSize size = memstore.getFlushableSize(); ((CompactingMemStore)memstore).flushInMemory(); // push keys to pipeline and compact assertEquals(0, memstore.getSnapshot().getCellsCount()); - assertEquals(totalCellsLen1 + totalCellsLen2, regionServicesForStores.getMemstoreSize()); + assertEquals(totalCellsLen1 + totalCellsLen2, regionServicesForStores.getMemStoreSize()); totalHeapSize = MutableSegment.DEEP_OVERHEAD + CellArrayImmutableSegment.DEEP_OVERHEAD_CAM + 7 * oneCellOnCAHeapSize; assertEquals(totalHeapSize, ((CompactingMemStore)memstore).heapSize()); size = memstore.getFlushableSize(); MemStoreSnapshot snapshot = memstore.snapshot(); // push keys to snapshot - region.decrMemstoreSize(size); // simulate flusher + region.decrMemStoreSize(size); // simulate flusher ImmutableSegment s = memstore.getSnapshot(); assertEquals(7, s.getCellsCount()); - assertEquals(0, regionServicesForStores.getMemstoreSize()); + assertEquals(0, regionServicesForStores.getMemStoreSize()); memstore.clearSnapshot(snapshot.getId()); } @@ -732,7 +732,7 @@ public class TestCompactingMemStore extends TestDefaultMemStore { int totalCellsLen1 = addRowsByKeys(memstore, keys1);// Adding 4 cells. int oneCellOnCSLMHeapSize = 120; int oneCellOnCAHeapSize = 88; - assertEquals(totalCellsLen1, region.getMemstoreSize()); + assertEquals(totalCellsLen1, region.getMemStoreSize()); long totalHeapSize = MutableSegment.DEEP_OVERHEAD + 4 * oneCellOnCSLMHeapSize; assertEquals(totalHeapSize, ((CompactingMemStore)memstore).heapSize()); ((CompactingMemStore)memstore).flushInMemory(); // push keys to pipeline and compact @@ -741,7 +741,7 @@ public class TestCompactingMemStore extends TestDefaultMemStore { // One cell is duplicated and the compaction will remove it. All cells of same time so adjusting // totalCellsLen totalCellsLen1 = (totalCellsLen1 * 3) / 4; - assertEquals(totalCellsLen1, regionServicesForStores.getMemstoreSize()); + assertEquals(totalCellsLen1, regionServicesForStores.getMemStoreSize()); // In memory flush to make a CellArrayMap instead of CSLM. See the overhead diff. totalHeapSize = MutableSegment.DEEP_OVERHEAD + CellArrayImmutableSegment.DEEP_OVERHEAD_CAM + 3 * oneCellOnCAHeapSize; @@ -750,21 +750,21 @@ public class TestCompactingMemStore extends TestDefaultMemStore { int totalCellsLen2 = addRowsByKeys(memstore, keys2);// Adding 3 more cells. long totalHeapSize2 = totalHeapSize + 3 * oneCellOnCSLMHeapSize; - assertEquals(totalCellsLen1 + totalCellsLen2, regionServicesForStores.getMemstoreSize()); + assertEquals(totalCellsLen1 + totalCellsLen2, regionServicesForStores.getMemStoreSize()); assertEquals(totalHeapSize2, ((CompactingMemStore) memstore).heapSize()); ((CompactingMemStore) memstore).disableCompaction(); - MemstoreSize size = memstore.getFlushableSize(); + MemStoreSize size = memstore.getFlushableSize(); ((CompactingMemStore)memstore).flushInMemory(); // push keys to pipeline without compaction assertEquals(0, memstore.getSnapshot().getCellsCount()); // No change in the cells data size. ie. memstore size. as there is no compaction. - assertEquals(totalCellsLen1 + totalCellsLen2, regionServicesForStores.getMemstoreSize()); + assertEquals(totalCellsLen1 + totalCellsLen2, regionServicesForStores.getMemStoreSize()); assertEquals(totalHeapSize2 + CellArrayImmutableSegment.DEEP_OVERHEAD_CAM, ((CompactingMemStore) memstore).heapSize()); int totalCellsLen3 = addRowsByKeys(memstore, keys3);// 3 more cells added assertEquals(totalCellsLen1 + totalCellsLen2 + totalCellsLen3, - regionServicesForStores.getMemstoreSize()); + regionServicesForStores.getMemStoreSize()); long totalHeapSize3 = totalHeapSize2 + CellArrayImmutableSegment.DEEP_OVERHEAD_CAM + 3 * oneCellOnCSLMHeapSize; assertEquals(totalHeapSize3, ((CompactingMemStore) memstore).heapSize()); @@ -778,17 +778,17 @@ public class TestCompactingMemStore extends TestDefaultMemStore { totalCellsLen2 = totalCellsLen2 / 3;// 2 out of 3 cells are duplicated totalCellsLen3 = 0;// All duplicated cells. assertEquals(totalCellsLen1 + totalCellsLen2 + totalCellsLen3, - regionServicesForStores.getMemstoreSize()); + regionServicesForStores.getMemStoreSize()); // Only 4 unique cells left assertEquals(4 * oneCellOnCAHeapSize + MutableSegment.DEEP_OVERHEAD + CellArrayImmutableSegment.DEEP_OVERHEAD_CAM, ((CompactingMemStore) memstore).heapSize()); size = memstore.getFlushableSize(); MemStoreSnapshot snapshot = memstore.snapshot(); // push keys to snapshot - region.decrMemstoreSize(size); // simulate flusher + region.decrMemStoreSize(size); // simulate flusher ImmutableSegment s = memstore.getSnapshot(); assertEquals(4, s.getCellsCount()); - assertEquals(0, regionServicesForStores.getMemstoreSize()); + assertEquals(0, regionServicesForStores.getMemStoreSize()); memstore.clearSnapshot(snapshot.getId()); } @@ -809,7 +809,7 @@ public class TestCompactingMemStore extends TestDefaultMemStore { hmc.add(kv, null); LOG.debug("added kv: " + kv.getKeyString() + ", timestamp:" + kv.getTimestamp()); } - regionServicesForStores.addMemstoreSize(new MemstoreSize(hmc.getActive().keySize() - size, + regionServicesForStores.addMemStoreSize(new MemStoreSize(hmc.getActive().keySize() - size, hmc.getActive().heapSize() - heapOverhead)); return totalLen; } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactingToCellFlatMapMemStore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactingToCellFlatMapMemStore.java index 6011af7808c..3fa5cd0ed9f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactingToCellFlatMapMemStore.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestCompactingToCellFlatMapMemStore.java @@ -99,7 +99,7 @@ public class TestCompactingToCellFlatMapMemStore extends TestCompactingMemStore long cellAfterFlushSize = cellAfterFlushSize(); long totalHeapSize = MutableSegment.DEEP_OVERHEAD + 4 * cellBeforeFlushSize; - assertEquals(totalCellsLen, regionServicesForStores.getMemstoreSize()); + assertEquals(totalCellsLen, regionServicesForStores.getMemStoreSize()); assertEquals(totalHeapSize, ((CompactingMemStore)memstore).heapSize()); assertEquals(4, memstore.getActive().getCellsCount()); @@ -108,7 +108,7 @@ public class TestCompactingToCellFlatMapMemStore extends TestCompactingMemStore // One cell is duplicated and the compaction will remove it. All cells of same size so adjusting // totalCellsLen totalCellsLen = (totalCellsLen * 3) / 4; - assertEquals(totalCellsLen, regionServicesForStores.getMemstoreSize()); + assertEquals(totalCellsLen, regionServicesForStores.getMemStoreSize()); totalHeapSize = 3 * cellAfterFlushSize + MutableSegment.DEEP_OVERHEAD @@ -120,12 +120,12 @@ public class TestCompactingToCellFlatMapMemStore extends TestCompactingMemStore counter += s.getCellsCount(); } assertEquals(3, counter); - MemstoreSize size = memstore.getFlushableSize(); + MemStoreSize size = memstore.getFlushableSize(); MemStoreSnapshot snapshot = memstore.snapshot(); // push keys to snapshot - region.decrMemstoreSize(size); // simulate flusher + region.decrMemStoreSize(size); // simulate flusher ImmutableSegment s = memstore.getSnapshot(); assertEquals(3, s.getCellsCount()); - assertEquals(0, regionServicesForStores.getMemstoreSize()); + assertEquals(0, regionServicesForStores.getMemStoreSize()); memstore.clearSnapshot(snapshot.getId()); } @@ -144,7 +144,7 @@ public class TestCompactingToCellFlatMapMemStore extends TestCompactingMemStore long cellBeforeFlushSize = cellBeforeFlushSize(); long cellAfterFlushSize = cellAfterFlushSize(); long totalHeapSize1 = MutableSegment.DEEP_OVERHEAD + 4 * cellBeforeFlushSize; - assertEquals(totalCellsLen1, regionServicesForStores.getMemstoreSize()); + assertEquals(totalCellsLen1, regionServicesForStores.getMemStoreSize()); assertEquals(totalHeapSize1, ((CompactingMemStore) memstore).heapSize()); ((CompactingMemStore) memstore).flushInMemory(); // push keys to pipeline and compact @@ -161,12 +161,12 @@ public class TestCompactingToCellFlatMapMemStore extends TestCompactingMemStore + (toCellChunkMap ? CellChunkImmutableSegment.DEEP_OVERHEAD_CCM : CellArrayImmutableSegment.DEEP_OVERHEAD_CAM); - assertEquals(totalCellsLen1, regionServicesForStores.getMemstoreSize()); + assertEquals(totalCellsLen1, regionServicesForStores.getMemStoreSize()); assertEquals(totalHeapSize1, ((CompactingMemStore) memstore).heapSize()); long totalCellsLen2 = addRowsByKeys(memstore, keys2); // INSERT 3 (3+3=6) long totalHeapSize2 = 3 * cellBeforeFlushSize; - assertEquals(totalCellsLen1 + totalCellsLen2, regionServicesForStores.getMemstoreSize()); + assertEquals(totalCellsLen1 + totalCellsLen2, regionServicesForStores.getMemStoreSize()); assertEquals(totalHeapSize1 + totalHeapSize2, ((CompactingMemStore) memstore).heapSize()); ((CompactingMemStore) memstore).flushInMemory(); // push keys to pipeline and compact @@ -177,16 +177,16 @@ public class TestCompactingToCellFlatMapMemStore extends TestCompactingMemStore } assertEquals(4,counter); totalCellsLen2 = totalCellsLen2 / 3;// 2 cells duplicated in set 2 - assertEquals(totalCellsLen1 + totalCellsLen2, regionServicesForStores.getMemstoreSize()); + assertEquals(totalCellsLen1 + totalCellsLen2, regionServicesForStores.getMemStoreSize()); totalHeapSize2 = 1 * cellAfterFlushSize; assertEquals(totalHeapSize1 + totalHeapSize2, ((CompactingMemStore) memstore).heapSize()); - MemstoreSize size = memstore.getFlushableSize(); + MemStoreSize size = memstore.getFlushableSize(); MemStoreSnapshot snapshot = memstore.snapshot(); // push keys to snapshot - region.decrMemstoreSize(size); // simulate flusher + region.decrMemStoreSize(size); // simulate flusher ImmutableSegment s = memstore.getSnapshot(); assertEquals(4, s.getCellsCount()); - assertEquals(0, regionServicesForStores.getMemstoreSize()); + assertEquals(0, regionServicesForStores.getMemStoreSize()); memstore.clearSnapshot(snapshot.getId()); } @@ -206,10 +206,10 @@ public class TestCompactingToCellFlatMapMemStore extends TestCompactingMemStore long cellBeforeFlushSize = cellBeforeFlushSize(); long cellAfterFlushSize = cellAfterFlushSize(); long totalHeapSize1 = MutableSegment.DEEP_OVERHEAD + 4 * cellBeforeFlushSize; - assertEquals(totalCellsLen1, region.getMemstoreSize()); + assertEquals(totalCellsLen1, region.getMemStoreSize()); assertEquals(totalHeapSize1, ((CompactingMemStore) memstore).heapSize()); - MemstoreSize size = memstore.getFlushableSize(); + MemStoreSize size = memstore.getFlushableSize(); ((CompactingMemStore) memstore).flushInMemory(); // push keys to pipeline and compact assertEquals(0, memstore.getSnapshot().getCellsCount()); @@ -220,13 +220,13 @@ public class TestCompactingToCellFlatMapMemStore extends TestCompactingMemStore + (toCellChunkMap ? CellChunkImmutableSegment.DEEP_OVERHEAD_CCM : CellArrayImmutableSegment.DEEP_OVERHEAD_CAM); - assertEquals(totalCellsLen1, regionServicesForStores.getMemstoreSize()); + assertEquals(totalCellsLen1, regionServicesForStores.getMemStoreSize()); assertEquals(totalHeapSize1, ((CompactingMemStore) memstore).heapSize()); long totalCellsLen2 = addRowsByKeys(memstore, keys2); long totalHeapSize2 = 3 * cellBeforeFlushSize; - assertEquals(totalCellsLen1 + totalCellsLen2, regionServicesForStores.getMemstoreSize()); + assertEquals(totalCellsLen1 + totalCellsLen2, regionServicesForStores.getMemStoreSize()); assertEquals(totalHeapSize1 + totalHeapSize2, ((CompactingMemStore) memstore).heapSize()); ((CompactingMemStore) memstore).disableCompaction(); @@ -234,13 +234,13 @@ public class TestCompactingToCellFlatMapMemStore extends TestCompactingMemStore ((CompactingMemStore) memstore).flushInMemory(); // push keys to pipeline without compaction totalHeapSize2 = totalHeapSize2 + CSLMImmutableSegment.DEEP_OVERHEAD_CSLM; assertEquals(0, memstore.getSnapshot().getCellsCount()); - assertEquals(totalCellsLen1 + totalCellsLen2, regionServicesForStores.getMemstoreSize()); + assertEquals(totalCellsLen1 + totalCellsLen2, regionServicesForStores.getMemStoreSize()); assertEquals(totalHeapSize1 + totalHeapSize2, ((CompactingMemStore) memstore).heapSize()); long totalCellsLen3 = addRowsByKeys(memstore, keys3); long totalHeapSize3 = 3 * cellBeforeFlushSize; assertEquals(totalCellsLen1 + totalCellsLen2 + totalCellsLen3, - regionServicesForStores.getMemstoreSize()); + regionServicesForStores.getMemStoreSize()); assertEquals(totalHeapSize1 + totalHeapSize2 + totalHeapSize3, ((CompactingMemStore) memstore).heapSize()); @@ -256,7 +256,7 @@ public class TestCompactingToCellFlatMapMemStore extends TestCompactingMemStore totalCellsLen2 = totalCellsLen2 / 3;// 2 out of 3 cells are duplicated totalCellsLen3 = 0;// All duplicated cells. assertEquals(totalCellsLen1 + totalCellsLen2 + totalCellsLen3, - regionServicesForStores.getMemstoreSize()); + regionServicesForStores.getMemStoreSize()); // Only 4 unique cells left long totalHeapSize4 = 4 * cellAfterFlushSize + MutableSegment.DEEP_OVERHEAD + (toCellChunkMap ? @@ -266,10 +266,10 @@ public class TestCompactingToCellFlatMapMemStore extends TestCompactingMemStore size = memstore.getFlushableSize(); MemStoreSnapshot snapshot = memstore.snapshot(); // push keys to snapshot - region.decrMemstoreSize(size); // simulate flusher + region.decrMemStoreSize(size); // simulate flusher ImmutableSegment s = memstore.getSnapshot(); assertEquals(4, s.getCellsCount()); - assertEquals(0, regionServicesForStores.getMemstoreSize()); + assertEquals(0, regionServicesForStores.getMemStoreSize()); memstore.clearSnapshot(snapshot.getId()); @@ -524,7 +524,7 @@ public class TestCompactingToCellFlatMapMemStore extends TestCompactingMemStore private long addRowsByKeys(final AbstractMemStore hmc, String[] keys) { byte[] fam = Bytes.toBytes("testfamily"); byte[] qf = Bytes.toBytes("testqualifier"); - MemstoreSize memstoreSize = new MemstoreSize(); + MemStoreSize memstoreSize = new MemStoreSize(); for (int i = 0; i < keys.length; i++) { long timestamp = System.currentTimeMillis(); Threads.sleep(1); // to make sure each kv gets a different ts @@ -534,7 +534,7 @@ public class TestCompactingToCellFlatMapMemStore extends TestCompactingMemStore hmc.add(kv, memstoreSize); LOG.debug("added kv: " + kv.getKeyString() + ", timestamp" + kv.getTimestamp()); } - regionServicesForStores.addMemstoreSize(memstoreSize); + regionServicesForStores.addMemStoreSize(memstoreSize); return memstoreSize.getDataSize(); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultMemStore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultMemStore.java index e40ff8eb2bc..eb9efabbdbf 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultMemStore.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestDefaultMemStore.java @@ -126,9 +126,9 @@ public class TestDefaultMemStore { public void testPutSameCell() { byte[] bytes = Bytes.toBytes(getName()); KeyValue kv = new KeyValue(bytes, bytes, bytes, bytes); - MemstoreSize sizeChangeForFirstCell = new MemstoreSize(); + MemStoreSize sizeChangeForFirstCell = new MemStoreSize(); this.memstore.add(kv, sizeChangeForFirstCell); - MemstoreSize sizeChangeForSecondCell = new MemstoreSize(); + MemStoreSize sizeChangeForSecondCell = new MemStoreSize(); this.memstore.add(kv, sizeChangeForSecondCell); // make sure memstore size increase won't double-count MSLAB chunk size assertEquals(Segment.getCellLength(kv), sizeChangeForFirstCell.getDataSize()); @@ -826,7 +826,7 @@ public class TestDefaultMemStore { public void testUpsertMemstoreSize() throws Exception { Configuration conf = HBaseConfiguration.create(); memstore = new DefaultMemStore(conf, CellComparator.COMPARATOR); - MemstoreSize oldSize = memstore.size(); + MemStoreSize oldSize = memstore.size(); List l = new ArrayList<>(); KeyValue kv1 = KeyValueTestUtil.create("r", "f", "q", 100, "v"); @@ -837,7 +837,7 @@ public class TestDefaultMemStore { l.add(kv1); l.add(kv2); l.add(kv3); this.memstore.upsert(l, 2, null);// readpoint is 2 - MemstoreSize newSize = this.memstore.size(); + MemStoreSize newSize = this.memstore.size(); assert (newSize.getDataSize() > oldSize.getDataSize()); //The kv1 should be removed. assert(memstore.getActive().getCellsCount() == 2); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEndToEndSplitTransaction.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEndToEndSplitTransaction.java index 6a41742b765..38f3060eda1 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEndToEndSplitTransaction.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestEndToEndSplitTransaction.java @@ -65,7 +65,6 @@ import org.junit.experimental.categories.Category; import org.junit.rules.TestName; import org.apache.hadoop.hbase.shaded.com.google.common.collect.Iterators; -import org.apache.hadoop.hbase.shaded.com.google.common.collect.Sets; @Category(LargeTests.class) public class TestEndToEndSplitTransaction { @@ -316,7 +315,7 @@ public class TestEndToEndSplitTransaction { admin.flushRegion(regionName); log("blocking until flush is complete: " + Bytes.toStringBinary(regionName)); Threads.sleepWithoutInterrupt(500); - while (rs.getOnlineRegion(regionName).getMemstoreSize() > 0) { + while (rs.getOnlineRegion(regionName).getMemStoreSize() > 0) { Threads.sleep(50); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java index 4d557b92de5..a7793f6045e 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegion.java @@ -300,7 +300,7 @@ public class TestHRegion { region.put(put); // Close with something in memstore and something in the snapshot. Make sure all is cleared. region.close(); - assertEquals(0, region.getMemstoreSize()); + assertEquals(0, region.getMemStoreSize()); HBaseTestingUtility.closeRegionAndWAL(region); } @@ -384,17 +384,17 @@ public class TestHRegion { HRegion region = initHRegion(tableName, null, null, false, Durability.SYNC_WAL, hLog, COLUMN_FAMILY_BYTES); HStore store = region.getStore(COLUMN_FAMILY_BYTES); - assertEquals(0, region.getMemstoreSize()); + assertEquals(0, region.getMemStoreSize()); // Put some value and make sure flush could be completed normally byte [] value = Bytes.toBytes(method); Put put = new Put(value); put.addColumn(COLUMN_FAMILY_BYTES, Bytes.toBytes("abc"), value); region.put(put); - long onePutSize = region.getMemstoreSize(); + long onePutSize = region.getMemStoreSize(); assertTrue(onePutSize > 0); region.flush(true); - assertEquals("memstoreSize should be zero", 0, region.getMemstoreSize()); + assertEquals("memstoreSize should be zero", 0, region.getMemStoreSize()); assertEquals("flushable size should be zero", 0, store.getFlushableSize().getDataSize()); // save normalCPHost and replaced by mockedCPHost, which will cancel flush requests @@ -405,14 +405,14 @@ public class TestHRegion { region.setCoprocessorHost(mockedCPHost); region.put(put); region.flush(true); - assertEquals("memstoreSize should NOT be zero", onePutSize, region.getMemstoreSize()); + assertEquals("memstoreSize should NOT be zero", onePutSize, region.getMemStoreSize()); assertEquals("flushable size should NOT be zero", onePutSize, store.getFlushableSize().getDataSize()); // set normalCPHost and flush again, the snapshot will be flushed region.setCoprocessorHost(normalCPHost); region.flush(true); - assertEquals("memstoreSize should be zero", 0, region.getMemstoreSize()); + assertEquals("memstoreSize should be zero", 0, region.getMemStoreSize()); assertEquals("flushable size should be zero", 0, store.getFlushableSize().getDataSize()); HBaseTestingUtility.closeRegionAndWAL(region); } @@ -426,14 +426,14 @@ public class TestHRegion { HRegion region = initHRegion(tableName, null, null, false, Durability.SYNC_WAL, hLog, COLUMN_FAMILY_BYTES); HStore store = region.getStore(COLUMN_FAMILY_BYTES); - assertEquals(0, region.getMemstoreSize()); + assertEquals(0, region.getMemStoreSize()); // Put one value byte [] value = Bytes.toBytes(method); Put put = new Put(value); put.addColumn(COLUMN_FAMILY_BYTES, Bytes.toBytes("abc"), value); region.put(put); - long onePutSize = region.getMemstoreSize(); + long onePutSize = region.getMemStoreSize(); assertTrue(onePutSize > 0); RegionCoprocessorHost mockedCPHost = Mockito.mock(RegionCoprocessorHost.class); @@ -449,7 +449,7 @@ public class TestHRegion { } catch (IOException expected) { } long expectedSize = onePutSize * 2; - assertEquals("memstoreSize should be incremented", expectedSize, region.getMemstoreSize()); + assertEquals("memstoreSize should be incremented", expectedSize, region.getMemStoreSize()); assertEquals("flushable size should be incremented", expectedSize, store.getFlushableSize().getDataSize()); @@ -494,13 +494,13 @@ public class TestHRegion { // Initialize region region = initHRegion(tableName, null, null, false, Durability.SYNC_WAL, wal, COLUMN_FAMILY_BYTES); - long size = region.getMemstoreSize(); + long size = region.getMemStoreSize(); Assert.assertEquals(0, size); // Put one item into memstore. Measure the size of one item in memstore. Put p1 = new Put(row); p1.add(new KeyValue(row, COLUMN_FAMILY_BYTES, qual1, 1, (byte[]) null)); region.put(p1); - final long sizeOfOnePut = region.getMemstoreSize(); + final long sizeOfOnePut = region.getMemStoreSize(); // Fail a flush which means the current memstore will hang out as memstore 'snapshot'. try { LOG.info("Flushing"); @@ -513,7 +513,7 @@ public class TestHRegion { // Make it so all writes succeed from here on out ffs.fault.set(false); // Check sizes. Should still be the one entry. - Assert.assertEquals(sizeOfOnePut, region.getMemstoreSize()); + Assert.assertEquals(sizeOfOnePut, region.getMemStoreSize()); // Now add two entries so that on this next flush that fails, we can see if we // subtract the right amount, the snapshot size only. Put p2 = new Put(row); @@ -521,13 +521,13 @@ public class TestHRegion { p2.add(new KeyValue(row, COLUMN_FAMILY_BYTES, qual3, 3, (byte[])null)); region.put(p2); long expectedSize = sizeOfOnePut * 3; - Assert.assertEquals(expectedSize, region.getMemstoreSize()); + Assert.assertEquals(expectedSize, region.getMemStoreSize()); // Do a successful flush. It will clear the snapshot only. Thats how flushes work. // If already a snapshot, we clear it else we move the memstore to be snapshot and flush // it region.flush(true); // Make sure our memory accounting is right. - Assert.assertEquals(sizeOfOnePut * 2, region.getMemstoreSize()); + Assert.assertEquals(sizeOfOnePut * 2, region.getMemStoreSize()); } finally { HBaseTestingUtility.closeRegionAndWAL(region); } @@ -559,7 +559,7 @@ public class TestHRegion { // Initialize region region = initHRegion(tableName, null, null, false, Durability.SYNC_WAL, wal, COLUMN_FAMILY_BYTES); - long size = region.getMemstoreSize(); + long size = region.getMemStoreSize(); Assert.assertEquals(0, size); // Put one item into memstore. Measure the size of one item in memstore. Put p1 = new Put(row); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionReplayEvents.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionReplayEvents.java index 63f5dfc7230..fe684bb60c7 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionReplayEvents.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHRegionReplayEvents.java @@ -282,12 +282,12 @@ public class TestHRegionReplayEvents { } } - assertTrue(rss.getRegionServerAccounting().getGlobalMemstoreDataSize() > 0); + assertTrue(rss.getRegionServerAccounting().getGlobalMemStoreDataSize() > 0); // now close the region which should not cause hold because of un-committed flush secondaryRegion.close(); // verify that the memstore size is back to what it was - assertEquals(0, rss.getRegionServerAccounting().getGlobalMemstoreDataSize()); + assertEquals(0, rss.getRegionServerAccounting().getGlobalMemStoreDataSize()); } static int replayEdit(HRegion region, WAL.Entry entry) throws IOException { @@ -341,7 +341,7 @@ public class TestHRegionReplayEvents { verifyData(secondaryRegion, 0, lastReplayed, cq, families); HStore store = secondaryRegion.getStore(Bytes.toBytes("cf1")); long storeMemstoreSize = store.getMemStoreSize().getHeapSize(); - long regionMemstoreSize = secondaryRegion.getMemstoreSize(); + long regionMemstoreSize = secondaryRegion.getMemStoreSize(); long storeFlushableSize = store.getFlushableSize().getHeapSize(); long storeSize = store.getSize(); long storeSizeUncompressed = store.getStoreSizeUncompressed(); @@ -370,7 +370,7 @@ public class TestHRegionReplayEvents { assertTrue(storeFlushableSize > newFlushableSize); // assert that the region memstore is smaller now - long newRegionMemstoreSize = secondaryRegion.getMemstoreSize(); + long newRegionMemstoreSize = secondaryRegion.getMemStoreSize(); assertTrue(regionMemstoreSize > newRegionMemstoreSize); // assert that the store sizes are bigger @@ -440,7 +440,7 @@ public class TestHRegionReplayEvents { // first verify that everything is replayed and visible before flush event replay HStore store = secondaryRegion.getStore(Bytes.toBytes("cf1")); long storeMemstoreSize = store.getMemStoreSize().getHeapSize(); - long regionMemstoreSize = secondaryRegion.getMemstoreSize(); + long regionMemstoreSize = secondaryRegion.getMemStoreSize(); long storeFlushableSize = store.getFlushableSize().getHeapSize(); if (flushDesc.getAction() == FlushAction.START_FLUSH) { @@ -480,7 +480,7 @@ public class TestHRegionReplayEvents { assertNotNull(secondaryRegion.getPrepareFlushResult()); assertEquals(secondaryRegion.getPrepareFlushResult().flushOpSeqId, startFlushDesc.getFlushSequenceNumber()); - assertTrue(secondaryRegion.getMemstoreSize() > 0); // memstore is not empty + assertTrue(secondaryRegion.getMemStoreSize() > 0); // memstore is not empty verifyData(secondaryRegion, 0, numRows, cq, families); // Test case 2: replay a flush start marker with a smaller seqId @@ -493,7 +493,7 @@ public class TestHRegionReplayEvents { assertNotNull(secondaryRegion.getPrepareFlushResult()); assertEquals(secondaryRegion.getPrepareFlushResult().flushOpSeqId, startFlushDesc.getFlushSequenceNumber()); - assertTrue(secondaryRegion.getMemstoreSize() > 0); // memstore is not empty + assertTrue(secondaryRegion.getMemStoreSize() > 0); // memstore is not empty verifyData(secondaryRegion, 0, numRows, cq, families); // Test case 3: replay a flush start marker with a larger seqId @@ -506,7 +506,7 @@ public class TestHRegionReplayEvents { assertNotNull(secondaryRegion.getPrepareFlushResult()); assertEquals(secondaryRegion.getPrepareFlushResult().flushOpSeqId, startFlushDesc.getFlushSequenceNumber()); - assertTrue(secondaryRegion.getMemstoreSize() > 0); // memstore is not empty + assertTrue(secondaryRegion.getMemStoreSize() > 0); // memstore is not empty verifyData(secondaryRegion, 0, numRows, cq, families); LOG.info("-- Verifying edits from secondary"); @@ -575,7 +575,7 @@ public class TestHRegionReplayEvents { for (HStore s : secondaryRegion.getStores()) { assertEquals(expectedStoreFileCount, s.getStorefilesCount()); } - long regionMemstoreSize = secondaryRegion.getMemstoreSize(); + long regionMemstoreSize = secondaryRegion.getMemStoreSize(); // Test case 1: replay the a flush commit marker smaller than what we have prepared LOG.info("Testing replaying flush COMMIT " + commitFlushDesc + " on top of flush START" @@ -595,7 +595,7 @@ public class TestHRegionReplayEvents { assertTrue(newFlushableSize > 0); // assert that the memstore is not dropped // assert that the region memstore is same as before - long newRegionMemstoreSize = secondaryRegion.getMemstoreSize(); + long newRegionMemstoreSize = secondaryRegion.getMemStoreSize(); assertEquals(regionMemstoreSize, newRegionMemstoreSize); assertNotNull(secondaryRegion.getPrepareFlushResult()); // not dropped @@ -665,7 +665,7 @@ public class TestHRegionReplayEvents { for (HStore s : secondaryRegion.getStores()) { assertEquals(expectedStoreFileCount, s.getStorefilesCount()); } - long regionMemstoreSize = secondaryRegion.getMemstoreSize(); + long regionMemstoreSize = secondaryRegion.getMemStoreSize(); // Test case 1: replay the a flush commit marker larger than what we have prepared LOG.info("Testing replaying flush COMMIT " + commitFlushDesc + " on top of flush START" @@ -685,7 +685,7 @@ public class TestHRegionReplayEvents { assertTrue(newFlushableSize > 0); // assert that the memstore is not dropped // assert that the region memstore is smaller than before, but not empty - long newRegionMemstoreSize = secondaryRegion.getMemstoreSize(); + long newRegionMemstoreSize = secondaryRegion.getMemStoreSize(); assertTrue(newRegionMemstoreSize > 0); assertTrue(regionMemstoreSize > newRegionMemstoreSize); @@ -766,7 +766,7 @@ public class TestHRegionReplayEvents { for (HStore s : secondaryRegion.getStores()) { assertEquals(expectedStoreFileCount, s.getStorefilesCount()); } - long regionMemstoreSize = secondaryRegion.getMemstoreSize(); + long regionMemstoreSize = secondaryRegion.getMemStoreSize(); // Test case 1: replay a flush commit marker without start flush marker assertNull(secondaryRegion.getPrepareFlushResult()); @@ -795,7 +795,7 @@ public class TestHRegionReplayEvents { } // assert that the region memstore is same as before (we could not drop) - long newRegionMemstoreSize = secondaryRegion.getMemstoreSize(); + long newRegionMemstoreSize = secondaryRegion.getMemStoreSize(); if (droppableMemstore) { assertTrue(0 == newRegionMemstoreSize); } else { @@ -865,7 +865,7 @@ public class TestHRegionReplayEvents { for (HStore s : secondaryRegion.getStores()) { assertEquals(expectedStoreFileCount, s.getStorefilesCount()); } - long regionMemstoreSize = secondaryRegion.getMemstoreSize(); + long regionMemstoreSize = secondaryRegion.getMemStoreSize(); assertTrue(regionMemstoreSize == 0); // now replay the region open event that should contain new file locations @@ -882,7 +882,7 @@ public class TestHRegionReplayEvents { assertTrue(newFlushableSize == MutableSegment.DEEP_OVERHEAD); // assert that the region memstore is empty - long newRegionMemstoreSize = secondaryRegion.getMemstoreSize(); + long newRegionMemstoreSize = secondaryRegion.getMemStoreSize(); assertTrue(newRegionMemstoreSize == 0); assertNull(secondaryRegion.getPrepareFlushResult()); //prepare snapshot should be dropped if any @@ -957,11 +957,11 @@ public class TestHRegionReplayEvents { assertEquals(expectedStoreFileCount, s.getStorefilesCount()); } HStore store = secondaryRegion.getStore(Bytes.toBytes("cf1")); - MemstoreSize newSnapshotSize = store.getSnapshotSize(); + MemStoreSize newSnapshotSize = store.getSnapshotSize(); assertTrue(newSnapshotSize.getDataSize() == 0); // assert that the region memstore is empty - long newRegionMemstoreSize = secondaryRegion.getMemstoreSize(); + long newRegionMemstoreSize = secondaryRegion.getMemStoreSize(); assertTrue(newRegionMemstoreSize == 0); assertNull(secondaryRegion.getPrepareFlushResult()); //prepare snapshot should be dropped if any @@ -1409,7 +1409,7 @@ public class TestHRegionReplayEvents { LOG.info("-- Replaying edits in secondary"); // Test case 4: replay some edits, ensure that memstore is dropped. - assertTrue(secondaryRegion.getMemstoreSize() == 0); + assertTrue(secondaryRegion.getMemStoreSize() == 0); putDataWithFlushes(primaryRegion, 400, 400, 0); numRows = 400; @@ -1427,11 +1427,11 @@ public class TestHRegionReplayEvents { } } - assertTrue(secondaryRegion.getMemstoreSize() > 0); + assertTrue(secondaryRegion.getMemStoreSize() > 0); secondaryRegion.refreshStoreFiles(); - assertTrue(secondaryRegion.getMemstoreSize() == 0); + assertTrue(secondaryRegion.getMemStoreSize() == 0); LOG.info("-- Verifying edits from primary"); verifyData(primaryRegion, 0, numRows, cq, families); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHStore.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHStore.java index 815166bec38..7c9b822b225 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHStore.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHStore.java @@ -249,13 +249,13 @@ public class TestHStore { // Initialize region init(name.getMethodName(), conf); - MemstoreSize size = store.memstore.getFlushableSize(); + MemStoreSize size = store.memstore.getFlushableSize(); assertEquals(0, size.getDataSize()); LOG.info("Adding some data"); - MemstoreSize kvSize = new MemstoreSize(); + MemStoreSize kvSize = new MemStoreSize(); store.add(new KeyValue(row, family, qf1, 1, (byte[]) null), kvSize); // add the heap size of active (mutable) segment - kvSize.incMemstoreSize(0, MutableSegment.DEEP_OVERHEAD); + kvSize.incMemStoreSize(0, MutableSegment.DEEP_OVERHEAD); size = store.memstore.getFlushableSize(); assertEquals(kvSize, size); // Flush. Bug #1 from HBASE-10466. Make sure size calculation on failed flush is right. @@ -267,13 +267,13 @@ public class TestHStore { assertTrue(ioe.getMessage().contains("Fault injected")); } // due to snapshot, change mutable to immutable segment - kvSize.incMemstoreSize(0, + kvSize.incMemStoreSize(0, CSLMImmutableSegment.DEEP_OVERHEAD_CSLM-MutableSegment.DEEP_OVERHEAD); size = store.memstore.getFlushableSize(); assertEquals(kvSize, size); - MemstoreSize kvSize2 = new MemstoreSize(); + MemStoreSize kvSize2 = new MemStoreSize(); store.add(new KeyValue(row, family, qf2, 2, (byte[])null), kvSize2); - kvSize2.incMemstoreSize(0, MutableSegment.DEEP_OVERHEAD); + kvSize2.incMemStoreSize(0, MutableSegment.DEEP_OVERHEAD); // Even though we add a new kv, we expect the flushable size to be 'same' since we have // not yet cleared the snapshot -- the above flush failed. assertEquals(kvSize, size); @@ -1182,7 +1182,7 @@ public class TestHStore { byte[] value0 = Bytes.toBytes("value0"); byte[] value1 = Bytes.toBytes("value1"); byte[] value2 = Bytes.toBytes("value2"); - MemstoreSize memStoreSize = new MemstoreSize(); + MemStoreSize memStoreSize = new MemStoreSize(); long ts = EnvironmentEdgeManager.currentTime(); long seqId = 100; init(name.getMethodName(), conf, TableDescriptorBuilder.newBuilder(TableName.valueOf(table)), @@ -1241,7 +1241,7 @@ public class TestHStore { init(name.getMethodName(), conf, ColumnFamilyDescriptorBuilder.newBuilder(family) .setInMemoryCompaction(MemoryCompactionPolicy.BASIC).build()); byte[] value = Bytes.toBytes("value"); - MemstoreSize memStoreSize = new MemstoreSize(); + MemStoreSize memStoreSize = new MemStoreSize(); long ts = EnvironmentEdgeManager.currentTime(); long seqId = 100; // older data whihc shouldn't be "seen" by client @@ -1319,7 +1319,7 @@ public class TestHStore { }); byte[] oldValue = Bytes.toBytes("oldValue"); byte[] currentValue = Bytes.toBytes("currentValue"); - MemstoreSize memStoreSize = new MemstoreSize(); + MemStoreSize memStoreSize = new MemStoreSize(); long ts = EnvironmentEdgeManager.currentTime(); long seqId = 100; // older data whihc shouldn't be "seen" by client @@ -1432,7 +1432,7 @@ public class TestHStore { init(name.getMethodName(), conf, ColumnFamilyDescriptorBuilder.newBuilder(family) .setInMemoryCompaction(MemoryCompactionPolicy.BASIC).build()); byte[] value = Bytes.toBytes("thisisavarylargevalue"); - MemstoreSize memStoreSize = new MemstoreSize(); + MemStoreSize memStoreSize = new MemStoreSize(); long ts = EnvironmentEdgeManager.currentTime(); long seqId = 100; // older data whihc shouldn't be "seen" by client @@ -1554,7 +1554,7 @@ public class TestHStore { conf.setLong(StoreScanner.STORESCANNER_PREAD_MAX_BYTES, 0); // Set the lower threshold to invoke the "MERGE" policy MyStore store = initMyStore(name.getMethodName(), conf, new MyStoreHook() {}); - MemstoreSize memStoreSize = new MemstoreSize(); + MemStoreSize memStoreSize = new MemStoreSize(); long ts = System.currentTimeMillis(); long seqID = 1l; // Add some data to the region and do some flushes diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHeapMemoryManager.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHeapMemoryManager.java index 95a94b4ecd9..ba0d309bf7b 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHeapMemoryManager.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestHeapMemoryManager.java @@ -824,7 +824,7 @@ public class TestHeapMemoryManager { } @Override - public void setGlobalMemstoreLimit(long globalMemStoreSize) { + public void setGlobalMemStoreLimit(long globalMemStoreSize) { this.memstoreSize = globalMemStoreSize; } } @@ -917,7 +917,7 @@ public class TestHeapMemoryManager { public TunerResult tune(TunerContext context) { TunerResult result = new TunerResult(true); result.setBlockCacheSize(blockCacheSize); - result.setMemstoreSize(memstoreSize); + result.setMemStoreSize(memstoreSize); return result; } } @@ -937,12 +937,12 @@ public class TestHeapMemoryManager { private long testMemstoreSize = 0; @Override - public long getGlobalMemstoreDataSize() { + public long getGlobalMemStoreDataSize() { return testMemstoreSize; } @Override - public long getGlobalMemstoreHeapSize() { + public long getGlobalMemStoreHeapSize() { return testMemstoreSize; } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestPerColumnFamilyFlush.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestPerColumnFamilyFlush.java index a0d953e3092..e4f7663269f 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestPerColumnFamilyFlush.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestPerColumnFamilyFlush.java @@ -46,7 +46,6 @@ import org.apache.hadoop.hbase.util.JVMClusterUtil; import org.apache.hadoop.hbase.util.Pair; import org.apache.hadoop.hbase.wal.AbstractFSWALProvider; import org.apache.hadoop.hbase.wal.WAL; -import org.junit.Before; import org.junit.Ignore; import org.junit.Test; import org.junit.experimental.categories.Category; @@ -144,7 +143,7 @@ public class TestPerColumnFamilyFlush { } } - long totalMemstoreSize = region.getMemstoreSize(); + long totalMemstoreSize = region.getMemStoreSize(); // Find the smallest LSNs for edits wrt to each CF. long smallestSeqCF1 = region.getOldestSeqIdOfStore(FAMILY1); @@ -152,13 +151,13 @@ public class TestPerColumnFamilyFlush { long smallestSeqCF3 = region.getOldestSeqIdOfStore(FAMILY3); // Find the sizes of the memstores of each CF. - MemstoreSize cf1MemstoreSize = region.getStore(FAMILY1).getMemStoreSize(); - MemstoreSize cf2MemstoreSize = region.getStore(FAMILY2).getMemStoreSize(); - MemstoreSize cf3MemstoreSize = region.getStore(FAMILY3).getMemStoreSize(); + MemStoreSize cf1MemstoreSize = region.getStore(FAMILY1).getMemStoreSize(); + MemStoreSize cf2MemstoreSize = region.getStore(FAMILY2).getMemStoreSize(); + MemStoreSize cf3MemstoreSize = region.getStore(FAMILY3).getMemStoreSize(); // Get the overall smallest LSN in the region's memstores. long smallestSeqInRegionCurrentMemstore = getWAL(region) - .getEarliestMemstoreSeqNum(region.getRegionInfo().getEncodedNameAsBytes()); + .getEarliestMemStoreSeqNum(region.getRegionInfo().getEncodedNameAsBytes()); // The overall smallest LSN in the region's memstores should be the same as // the LSN of the smallest edit in CF1 @@ -180,16 +179,16 @@ public class TestPerColumnFamilyFlush { region.flush(false); // Will use these to check if anything changed. - MemstoreSize oldCF2MemstoreSize = cf2MemstoreSize; - MemstoreSize oldCF3MemstoreSize = cf3MemstoreSize; + MemStoreSize oldCF2MemstoreSize = cf2MemstoreSize; + MemStoreSize oldCF3MemstoreSize = cf3MemstoreSize; // Recalculate everything cf1MemstoreSize = region.getStore(FAMILY1).getMemStoreSize(); cf2MemstoreSize = region.getStore(FAMILY2).getMemStoreSize(); cf3MemstoreSize = region.getStore(FAMILY3).getMemStoreSize(); - totalMemstoreSize = region.getMemstoreSize(); + totalMemstoreSize = region.getMemStoreSize(); smallestSeqInRegionCurrentMemstore = getWAL(region) - .getEarliestMemstoreSeqNum(region.getRegionInfo().getEncodedNameAsBytes()); + .getEarliestMemStoreSeqNum(region.getRegionInfo().getEncodedNameAsBytes()); // We should have cleared out only CF1, since we chose the flush thresholds // and number of puts accordingly. @@ -225,9 +224,9 @@ public class TestPerColumnFamilyFlush { cf1MemstoreSize = region.getStore(FAMILY1).getMemStoreSize(); cf2MemstoreSize = region.getStore(FAMILY2).getMemStoreSize(); cf3MemstoreSize = region.getStore(FAMILY3).getMemStoreSize(); - totalMemstoreSize = region.getMemstoreSize(); + totalMemstoreSize = region.getMemStoreSize(); smallestSeqInRegionCurrentMemstore = getWAL(region) - .getEarliestMemstoreSeqNum(region.getRegionInfo().getEncodedNameAsBytes()); + .getEarliestMemStoreSeqNum(region.getRegionInfo().getEncodedNameAsBytes()); // CF1 and CF2, both should be absent. assertEquals(0, cf1MemstoreSize.getDataSize()); @@ -261,7 +260,7 @@ public class TestPerColumnFamilyFlush { // Since we won't find any CF above the threshold, and hence no specific // store to flush, we should flush all the memstores. - assertEquals(0, region.getMemstoreSize()); + assertEquals(0, region.getMemStoreSize()); HBaseTestingUtility.closeRegionAndWAL(region); } @@ -285,12 +284,12 @@ public class TestPerColumnFamilyFlush { } } - long totalMemstoreSize = region.getMemstoreSize(); + long totalMemstoreSize = region.getMemStoreSize(); // Find the sizes of the memstores of each CF. - MemstoreSize cf1MemstoreSize = region.getStore(FAMILY1).getMemStoreSize(); - MemstoreSize cf2MemstoreSize = region.getStore(FAMILY2).getMemStoreSize(); - MemstoreSize cf3MemstoreSize = region.getStore(FAMILY3).getMemStoreSize(); + MemStoreSize cf1MemstoreSize = region.getStore(FAMILY1).getMemStoreSize(); + MemStoreSize cf2MemstoreSize = region.getStore(FAMILY2).getMemStoreSize(); + MemStoreSize cf3MemstoreSize = region.getStore(FAMILY3).getMemStoreSize(); // Some other sanity checks. assertTrue(cf1MemstoreSize.getDataSize() > 0); @@ -308,9 +307,9 @@ public class TestPerColumnFamilyFlush { cf1MemstoreSize = region.getStore(FAMILY1).getMemStoreSize(); cf2MemstoreSize = region.getStore(FAMILY2).getMemStoreSize(); cf3MemstoreSize = region.getStore(FAMILY3).getMemStoreSize(); - totalMemstoreSize = region.getMemstoreSize(); + totalMemstoreSize = region.getMemStoreSize(); long smallestSeqInRegionCurrentMemstore = - region.getWAL().getEarliestMemstoreSeqNum(region.getRegionInfo().getEncodedNameAsBytes()); + region.getWAL().getEarliestMemStoreSeqNum(region.getRegionInfo().getEncodedNameAsBytes()); // Everything should have been cleared assertEquals(0, cf1MemstoreSize.getDataSize()); @@ -377,7 +376,7 @@ public class TestPerColumnFamilyFlush { long totalMemstoreSize; long cf1MemstoreSize, cf2MemstoreSize, cf3MemstoreSize; - totalMemstoreSize = desiredRegion.getMemstoreSize(); + totalMemstoreSize = desiredRegion.getMemStoreSize(); // Find the sizes of the memstores of each CF. cf1MemstoreSize = desiredRegion.getStore(FAMILY1).getMemStoreSize().getDataSize(); @@ -511,12 +510,12 @@ public class TestPerColumnFamilyFlush { @Override public boolean evaluate() throws Exception { - return desiredRegion.getMemstoreSize() == 0; + return desiredRegion.getMemStoreSize() == 0; } @Override public String explainFailure() throws Exception { - long memstoreSize = desiredRegion.getMemstoreSize(); + long memstoreSize = desiredRegion.getMemStoreSize(); if (memstoreSize > 0) { return "Still have unflushed entries in memstore, memstore size is " + memstoreSize; } @@ -558,7 +557,7 @@ public class TestPerColumnFamilyFlush { put.addColumn(FAMILY3, qf, value3); table.put(put); // slow down to let regionserver flush region. - while (region.getMemstoreSize() > memstoreFlushSize) { + while (region.getMemStoreSize() > memstoreFlushSize) { Thread.sleep(100); } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerAccounting.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerAccounting.java index fd45a126cec..0122674ce15 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerAccounting.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestRegionServerAccounting.java @@ -36,9 +36,9 @@ public class TestRegionServerAccounting { conf.setFloat(MemorySizeUtil.MEMSTORE_SIZE_KEY, 0.2f); // try for default cases RegionServerAccounting regionServerAccounting = new RegionServerAccounting(conf); - MemstoreSize memstoreSize = - new MemstoreSize((long) (3l * 1024l * 1024l * 1024l), (long) (1l * 1024l * 1024l * 1024l)); - regionServerAccounting.incGlobalMemstoreSize(memstoreSize); + MemStoreSize memstoreSize = + new MemStoreSize((long) (3l * 1024l * 1024l * 1024l), (long) (1l * 1024l * 1024l * 1024l)); + regionServerAccounting.incGlobalMemStoreSize(memstoreSize); assertEquals(FlushType.ABOVE_ONHEAP_HIGHER_MARK, regionServerAccounting.isAboveHighWaterMark()); } @@ -49,9 +49,9 @@ public class TestRegionServerAccounting { conf.setFloat(MemorySizeUtil.MEMSTORE_SIZE_KEY, 0.2f); // try for default cases RegionServerAccounting regionServerAccounting = new RegionServerAccounting(conf); - MemstoreSize memstoreSize = - new MemstoreSize((long) (3l * 1024l * 1024l * 1024l), (long) (1l * 1024l * 1024l * 1024l)); - regionServerAccounting.incGlobalMemstoreSize(memstoreSize); + MemStoreSize memstoreSize = + new MemStoreSize((long) (3l * 1024l * 1024l * 1024l), (long) (1l * 1024l * 1024l * 1024l)); + regionServerAccounting.incGlobalMemStoreSize(memstoreSize); assertEquals(FlushType.ABOVE_ONHEAP_LOWER_MARK, regionServerAccounting.isAboveLowWaterMark()); } @@ -64,9 +64,9 @@ public class TestRegionServerAccounting { // try for default cases RegionServerAccounting regionServerAccounting = new RegionServerAccounting(conf); // this will breach offheap limit as data size is higher and not due to heap size - MemstoreSize memstoreSize = - new MemstoreSize((long) (3l * 1024l * 1024l * 1024l), (long) (1l * 1024l * 1024l * 1024l)); - regionServerAccounting.incGlobalMemstoreSize(memstoreSize); + MemStoreSize memstoreSize = + new MemStoreSize((long) (3l * 1024l * 1024l * 1024l), (long) (1l * 1024l * 1024l * 1024l)); + regionServerAccounting.incGlobalMemStoreSize(memstoreSize); assertEquals(FlushType.ABOVE_OFFHEAP_HIGHER_MARK, regionServerAccounting.isAboveHighWaterMark()); } @@ -80,9 +80,9 @@ public class TestRegionServerAccounting { // try for default cases RegionServerAccounting regionServerAccounting = new RegionServerAccounting(conf); // this will breach higher limit as heap size is higher and not due to offheap size - MemstoreSize memstoreSize = - new MemstoreSize((long) (3l * 1024l * 1024l), (long) (2l * 1024l * 1024l * 1024l)); - regionServerAccounting.incGlobalMemstoreSize(memstoreSize); + MemStoreSize memstoreSize = + new MemStoreSize((long) (3l * 1024l * 1024l), (long) (2l * 1024l * 1024l * 1024l)); + regionServerAccounting.incGlobalMemStoreSize(memstoreSize); assertEquals(FlushType.ABOVE_ONHEAP_HIGHER_MARK, regionServerAccounting.isAboveHighWaterMark()); } @@ -95,9 +95,9 @@ public class TestRegionServerAccounting { // try for default cases RegionServerAccounting regionServerAccounting = new RegionServerAccounting(conf); // this will breach offheap limit as data size is higher and not due to heap size - MemstoreSize memstoreSize = - new MemstoreSize((long) (3l * 1024l * 1024l * 1024l), (long) (1l * 1024l * 1024l * 1024l)); - regionServerAccounting.incGlobalMemstoreSize(memstoreSize); + MemStoreSize memstoreSize = + new MemStoreSize((long) (3l * 1024l * 1024l * 1024l), (long) (1l * 1024l * 1024l * 1024l)); + regionServerAccounting.incGlobalMemStoreSize(memstoreSize); assertEquals(FlushType.ABOVE_OFFHEAP_LOWER_MARK, regionServerAccounting.isAboveLowWaterMark()); } @@ -111,9 +111,9 @@ public class TestRegionServerAccounting { // try for default cases RegionServerAccounting regionServerAccounting = new RegionServerAccounting(conf); // this will breach higher limit as heap size is higher and not due to offheap size - MemstoreSize memstoreSize = - new MemstoreSize((long) (3l * 1024l * 1024l), (long) (2l * 1024l * 1024l * 1024l)); - regionServerAccounting.incGlobalMemstoreSize(memstoreSize); + MemStoreSize memstoreSize = + new MemStoreSize((long) (3l * 1024l * 1024l), (long) (2l * 1024l * 1024l * 1024l)); + regionServerAccounting.incGlobalMemStoreSize(memstoreSize); assertEquals(FlushType.ABOVE_ONHEAP_LOWER_MARK, regionServerAccounting.isAboveLowWaterMark()); } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java index d692a9a4319..a8a60abb89b 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWALLockup.java @@ -258,8 +258,8 @@ public class TestWALLockup { Thread t = new Thread ("Flusher") { public void run() { try { - if (region.getMemstoreSize() <= 0) { - throw new IOException("memstore size=" + region.getMemstoreSize()); + if (region.getMemStoreSize() <= 0) { + throw new IOException("memstore size=" + region.getMemStoreSize()); } region.flush(false); } catch (IOException e) { @@ -272,7 +272,7 @@ public class TestWALLockup { }; t.setDaemon(true); t.start(); - // Wait until + // Wait until while (dodgyWAL.latch.getCount() > 0) Threads.sleep(1); // Now assert I got a new WAL file put in place even though loads of errors above. assertTrue(originalWAL != dodgyWAL.getCurrentFileName()); diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWalAndCompactingMemStoreFlush.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWalAndCompactingMemStoreFlush.java index 6dc1a1b7da7..4d0a16868c8 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWalAndCompactingMemStoreFlush.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/TestWalAndCompactingMemStoreFlush.java @@ -161,7 +161,7 @@ public class TestWalAndCompactingMemStoreFlush { region.put(createPut(2, i)); } - long totalMemstoreSize = region.getMemstoreSize(); + long totalMemstoreSize = region.getMemStoreSize(); // Find the smallest LSNs for edits wrt to each CF. long smallestSeqCF1PhaseI = region.getOldestSeqIdOfStore(FAMILY1); @@ -169,22 +169,22 @@ public class TestWalAndCompactingMemStoreFlush { long smallestSeqCF3PhaseI = region.getOldestSeqIdOfStore(FAMILY3); // Find the sizes of the memstores of each CF. - MemstoreSize cf1MemstoreSizePhaseI = region.getStore(FAMILY1).getMemStoreSize(); - MemstoreSize cf2MemstoreSizePhaseI = region.getStore(FAMILY2).getMemStoreSize(); - MemstoreSize cf3MemstoreSizePhaseI = region.getStore(FAMILY3).getMemStoreSize(); + MemStoreSize cf1MemstoreSizePhaseI = region.getStore(FAMILY1).getMemStoreSize(); + MemStoreSize cf2MemstoreSizePhaseI = region.getStore(FAMILY2).getMemStoreSize(); + MemStoreSize cf3MemstoreSizePhaseI = region.getStore(FAMILY3).getMemStoreSize(); // Get the overall smallest LSN in the region's memstores. long smallestSeqInRegionCurrentMemstorePhaseI = getWAL(region) - .getEarliestMemstoreSeqNum(region.getRegionInfo().getEncodedNameAsBytes()); + .getEarliestMemStoreSeqNum(region.getRegionInfo().getEncodedNameAsBytes()); String s = "\n\n----------------------------------\n" + "Upon initial insert and before any flush, size of CF1 is:" + cf1MemstoreSizePhaseI + ", is CF1 compacted memstore?:" - + region.getStore(FAMILY1).isSloppyMemstore() + ". Size of CF2 is:" + + region.getStore(FAMILY1).isSloppyMemStore() + ". Size of CF2 is:" + cf2MemstoreSizePhaseI + ", is CF2 compacted memstore?:" - + region.getStore(FAMILY2).isSloppyMemstore() + ". Size of CF3 is:" + + region.getStore(FAMILY2).isSloppyMemStore() + ". Size of CF3 is:" + cf3MemstoreSizePhaseI + ", is CF3 compacted memstore?:" - + region.getStore(FAMILY3).isSloppyMemstore() + "\n"; + + region.getStore(FAMILY3).isSloppyMemStore() + "\n"; // The overall smallest LSN in the region's memstores should be the same as // the LSN of the smallest edit in CF1 @@ -220,12 +220,12 @@ public class TestWalAndCompactingMemStoreFlush { region.flush(false); // Recalculate everything - MemstoreSize cf1MemstoreSizePhaseII = region.getStore(FAMILY1).getMemStoreSize(); - MemstoreSize cf2MemstoreSizePhaseII = region.getStore(FAMILY2).getMemStoreSize(); - MemstoreSize cf3MemstoreSizePhaseII = region.getStore(FAMILY3).getMemStoreSize(); + MemStoreSize cf1MemstoreSizePhaseII = region.getStore(FAMILY1).getMemStoreSize(); + MemStoreSize cf2MemstoreSizePhaseII = region.getStore(FAMILY2).getMemStoreSize(); + MemStoreSize cf3MemstoreSizePhaseII = region.getStore(FAMILY3).getMemStoreSize(); long smallestSeqInRegionCurrentMemstorePhaseII = getWAL(region) - .getEarliestMemstoreSeqNum(region.getRegionInfo().getEncodedNameAsBytes()); + .getEarliestMemStoreSeqNum(region.getRegionInfo().getEncodedNameAsBytes()); // Find the smallest LSNs for edits wrt to each CF. long smallestSeqCF1PhaseII = region.getOldestSeqIdOfStore(FAMILY1); long smallestSeqCF2PhaseII = region.getOldestSeqIdOfStore(FAMILY2); @@ -264,7 +264,7 @@ public class TestWalAndCompactingMemStoreFlush { + smallestSeqCF2PhaseII +", the smallest sequence in CF3:" + smallestSeqCF3PhaseII + "\n"; // How much does the CF1 memstore occupy? Will be used later. - MemstoreSize cf1MemstoreSizePhaseIII = region.getStore(FAMILY1).getMemStoreSize(); + MemStoreSize cf1MemstoreSizePhaseIII = region.getStore(FAMILY1).getMemStoreSize(); long smallestSeqCF1PhaseIII = region.getOldestSeqIdOfStore(FAMILY1); s = s + "----After more puts into CF1 its size is:" + cf1MemstoreSizePhaseIII @@ -278,12 +278,12 @@ public class TestWalAndCompactingMemStoreFlush { region.flush(false); // Recalculate everything - MemstoreSize cf1MemstoreSizePhaseIV = region.getStore(FAMILY1).getMemStoreSize(); - MemstoreSize cf2MemstoreSizePhaseIV = region.getStore(FAMILY2).getMemStoreSize(); - MemstoreSize cf3MemstoreSizePhaseIV = region.getStore(FAMILY3).getMemStoreSize(); + MemStoreSize cf1MemstoreSizePhaseIV = region.getStore(FAMILY1).getMemStoreSize(); + MemStoreSize cf2MemstoreSizePhaseIV = region.getStore(FAMILY2).getMemStoreSize(); + MemStoreSize cf3MemstoreSizePhaseIV = region.getStore(FAMILY3).getMemStoreSize(); long smallestSeqInRegionCurrentMemstorePhaseIV = getWAL(region) - .getEarliestMemstoreSeqNum(region.getRegionInfo().getEncodedNameAsBytes()); + .getEarliestMemStoreSeqNum(region.getRegionInfo().getEncodedNameAsBytes()); long smallestSeqCF1PhaseIV = region.getOldestSeqIdOfStore(FAMILY1); long smallestSeqCF2PhaseIV = region.getOldestSeqIdOfStore(FAMILY2); long smallestSeqCF3PhaseIV = region.getOldestSeqIdOfStore(FAMILY3); @@ -319,11 +319,11 @@ public class TestWalAndCompactingMemStoreFlush { region.flush(true); // Recalculate everything - MemstoreSize cf1MemstoreSizePhaseV = region.getStore(FAMILY1).getMemStoreSize(); - MemstoreSize cf2MemstoreSizePhaseV = region.getStore(FAMILY2).getMemStoreSize(); - MemstoreSize cf3MemstoreSizePhaseV = region.getStore(FAMILY3).getMemStoreSize(); + MemStoreSize cf1MemstoreSizePhaseV = region.getStore(FAMILY1).getMemStoreSize(); + MemStoreSize cf2MemstoreSizePhaseV = region.getStore(FAMILY2).getMemStoreSize(); + MemStoreSize cf3MemstoreSizePhaseV = region.getStore(FAMILY3).getMemStoreSize(); long smallestSeqInRegionCurrentMemstorePhaseV = getWAL(region) - .getEarliestMemstoreSeqNum(region.getRegionInfo().getEncodedNameAsBytes()); + .getEarliestMemStoreSeqNum(region.getRegionInfo().getEncodedNameAsBytes()); assertEquals(0, cf1MemstoreSizePhaseV.getDataSize()); assertEquals(MutableSegment.DEEP_OVERHEAD, cf1MemstoreSizePhaseV.getHeapSize()); @@ -352,13 +352,13 @@ public class TestWalAndCompactingMemStoreFlush { s = s + "----AFTER THIRD AND FORTH FLUSH, The smallest sequence in region WAL is: " + smallestSeqInRegionCurrentMemstorePhaseV + ". After additional inserts and last flush, the entire region size is:" + region - .getMemstoreSize() + .getMemStoreSize() + "\n----------------------------------\n"; // Since we won't find any CF above the threshold, and hence no specific // store to flush, we should flush all the memstores // Also compacted memstores are flushed to disk. - assertEquals(0, region.getMemstoreSize()); + assertEquals(0, region.getMemStoreSize()); System.out.println(s); HBaseTestingUtility.closeRegionAndWAL(region); } @@ -404,18 +404,18 @@ public class TestWalAndCompactingMemStoreFlush { /*------------------------------------------------------------------------------*/ /*------------------------------------------------------------------------------*/ /* PHASE I - collect sizes */ - long totalMemstoreSizePhaseI = region.getMemstoreSize(); + long totalMemstoreSizePhaseI = region.getMemStoreSize(); // Find the smallest LSNs for edits wrt to each CF. long smallestSeqCF1PhaseI = region.getOldestSeqIdOfStore(FAMILY1); long smallestSeqCF2PhaseI = region.getOldestSeqIdOfStore(FAMILY2); long smallestSeqCF3PhaseI = region.getOldestSeqIdOfStore(FAMILY3); // Find the sizes of the memstores of each CF. - MemstoreSize cf1MemstoreSizePhaseI = region.getStore(FAMILY1).getMemStoreSize(); - MemstoreSize cf2MemstoreSizePhaseI = region.getStore(FAMILY2).getMemStoreSize(); - MemstoreSize cf3MemstoreSizePhaseI = region.getStore(FAMILY3).getMemStoreSize(); + MemStoreSize cf1MemstoreSizePhaseI = region.getStore(FAMILY1).getMemStoreSize(); + MemStoreSize cf2MemstoreSizePhaseI = region.getStore(FAMILY2).getMemStoreSize(); + MemStoreSize cf3MemstoreSizePhaseI = region.getStore(FAMILY3).getMemStoreSize(); // Get the overall smallest LSN in the region's memstores. long smallestSeqInRegionCurrentMemstorePhaseI = getWAL(region) - .getEarliestMemstoreSeqNum(region.getRegionInfo().getEncodedNameAsBytes()); + .getEarliestMemStoreSeqNum(region.getRegionInfo().getEncodedNameAsBytes()); /*------------------------------------------------------------------------------*/ /* PHASE I - validation */ @@ -460,14 +460,14 @@ public class TestWalAndCompactingMemStoreFlush { /*------------------------------------------------------------------------------*/ /* PHASE II - collect sizes */ // Recalculate everything - MemstoreSize cf1MemstoreSizePhaseII = region.getStore(FAMILY1).getMemStoreSize(); - MemstoreSize cf2MemstoreSizePhaseII = region.getStore(FAMILY2).getMemStoreSize(); - MemstoreSize cf3MemstoreSizePhaseII = region.getStore(FAMILY3).getMemStoreSize(); + MemStoreSize cf1MemstoreSizePhaseII = region.getStore(FAMILY1).getMemStoreSize(); + MemStoreSize cf2MemstoreSizePhaseII = region.getStore(FAMILY2).getMemStoreSize(); + MemStoreSize cf3MemstoreSizePhaseII = region.getStore(FAMILY3).getMemStoreSize(); long smallestSeqInRegionCurrentMemstorePhaseII = getWAL(region) - .getEarliestMemstoreSeqNum(region.getRegionInfo().getEncodedNameAsBytes()); + .getEarliestMemStoreSeqNum(region.getRegionInfo().getEncodedNameAsBytes()); // Find the smallest LSNs for edits wrt to each CF. long smallestSeqCF3PhaseII = region.getOldestSeqIdOfStore(FAMILY3); - long totalMemstoreSizePhaseII = region.getMemstoreSize(); + long totalMemstoreSizePhaseII = region.getMemStoreSize(); /*------------------------------------------------------------------------------*/ /* PHASE II - validation */ @@ -509,8 +509,8 @@ public class TestWalAndCompactingMemStoreFlush { /*------------------------------------------------------------------------------*/ /* PHASE III - collect sizes */ // How much does the CF1 memstore occupy now? Will be used later. - MemstoreSize cf1MemstoreSizePhaseIII = region.getStore(FAMILY1).getMemStoreSize(); - long totalMemstoreSizePhaseIII = region.getMemstoreSize(); + MemStoreSize cf1MemstoreSizePhaseIII = region.getStore(FAMILY1).getMemStoreSize(); + long totalMemstoreSizePhaseIII = region.getMemStoreSize(); /*------------------------------------------------------------------------------*/ /* PHASE III - validation */ @@ -531,11 +531,11 @@ public class TestWalAndCompactingMemStoreFlush { /*------------------------------------------------------------------------------*/ /* PHASE IV - collect sizes */ // Recalculate everything - MemstoreSize cf1MemstoreSizePhaseIV = region.getStore(FAMILY1).getMemStoreSize(); - MemstoreSize cf2MemstoreSizePhaseIV = region.getStore(FAMILY2).getMemStoreSize(); - MemstoreSize cf3MemstoreSizePhaseIV = region.getStore(FAMILY3).getMemStoreSize(); + MemStoreSize cf1MemstoreSizePhaseIV = region.getStore(FAMILY1).getMemStoreSize(); + MemStoreSize cf2MemstoreSizePhaseIV = region.getStore(FAMILY2).getMemStoreSize(); + MemStoreSize cf3MemstoreSizePhaseIV = region.getStore(FAMILY3).getMemStoreSize(); long smallestSeqInRegionCurrentMemstorePhaseIV = getWAL(region) - .getEarliestMemstoreSeqNum(region.getRegionInfo().getEncodedNameAsBytes()); + .getEarliestMemStoreSeqNum(region.getRegionInfo().getEncodedNameAsBytes()); long smallestSeqCF3PhaseIV = region.getOldestSeqIdOfStore(FAMILY3); /*------------------------------------------------------------------------------*/ @@ -563,12 +563,12 @@ public class TestWalAndCompactingMemStoreFlush { /*------------------------------------------------------------------------------*/ /* PHASE V - collect sizes */ // Recalculate everything - MemstoreSize cf1MemstoreSizePhaseV = region.getStore(FAMILY1).getMemStoreSize(); - MemstoreSize cf2MemstoreSizePhaseV = region.getStore(FAMILY2).getMemStoreSize(); - MemstoreSize cf3MemstoreSizePhaseV = region.getStore(FAMILY3).getMemStoreSize(); + MemStoreSize cf1MemstoreSizePhaseV = region.getStore(FAMILY1).getMemStoreSize(); + MemStoreSize cf2MemstoreSizePhaseV = region.getStore(FAMILY2).getMemStoreSize(); + MemStoreSize cf3MemstoreSizePhaseV = region.getStore(FAMILY3).getMemStoreSize(); long smallestSeqInRegionCurrentMemstorePhaseV = getWAL(region) - .getEarliestMemstoreSeqNum(region.getRegionInfo().getEncodedNameAsBytes()); - long totalMemstoreSizePhaseV = region.getMemstoreSize(); + .getEarliestMemStoreSeqNum(region.getRegionInfo().getEncodedNameAsBytes()); + long totalMemstoreSizePhaseV = region.getMemStoreSize(); /*------------------------------------------------------------------------------*/ /* PHASE V - validation */ @@ -601,9 +601,9 @@ public class TestWalAndCompactingMemStoreFlush { region.put(createPut(5, i)); } - MemstoreSize cf1ActiveSizePhaseVI = region.getStore(FAMILY1).getMemStoreSize(); - MemstoreSize cf3ActiveSizePhaseVI = region.getStore(FAMILY3).getMemStoreSize(); - MemstoreSize cf5ActiveSizePhaseVI = region.getStore(FAMILIES[4]).getMemStoreSize(); + MemStoreSize cf1ActiveSizePhaseVI = region.getStore(FAMILY1).getMemStoreSize(); + MemStoreSize cf3ActiveSizePhaseVI = region.getStore(FAMILY3).getMemStoreSize(); + MemStoreSize cf5ActiveSizePhaseVI = region.getStore(FAMILIES[4]).getMemStoreSize(); /*------------------------------------------------------------------------------*/ /* PHASE VI - Flush */ @@ -614,9 +614,9 @@ public class TestWalAndCompactingMemStoreFlush { // Since we won't find any CF above the threshold, and hence no specific // store to flush, we should flush all the memstores // Also compacted memstores are flushed to disk, but not entirely emptied - MemstoreSize cf1ActiveSizePhaseVII = region.getStore(FAMILY1).getMemStoreSize(); - MemstoreSize cf3ActiveSizePhaseVII = region.getStore(FAMILY3).getMemStoreSize(); - MemstoreSize cf5ActiveSizePhaseVII = region.getStore(FAMILIES[4]).getMemStoreSize(); + MemStoreSize cf1ActiveSizePhaseVII = region.getStore(FAMILY1).getMemStoreSize(); + MemStoreSize cf3ActiveSizePhaseVII = region.getStore(FAMILY3).getMemStoreSize(); + MemStoreSize cf5ActiveSizePhaseVII = region.getStore(FAMILIES[4]).getMemStoreSize(); assertTrue(cf1ActiveSizePhaseVII.getDataSize() < cf1ActiveSizePhaseVI.getDataSize()); assertTrue(cf3ActiveSizePhaseVII.getDataSize() < cf3ActiveSizePhaseVI.getDataSize()); @@ -660,12 +660,12 @@ public class TestWalAndCompactingMemStoreFlush { ((CompactingMemStore) ((HStore) region.getStore(FAMILY1)).memstore).setCompositeSnapshot(false); ((CompactingMemStore) ((HStore) region.getStore(FAMILY3)).memstore).setCompositeSnapshot(false); - long totalMemstoreSize = region.getMemstoreSize(); + long totalMemstoreSize = region.getMemStoreSize(); // Find the sizes of the memstores of each CF. - MemstoreSize cf1MemstoreSizePhaseI = region.getStore(FAMILY1).getMemStoreSize(); - MemstoreSize cf2MemstoreSizePhaseI = region.getStore(FAMILY2).getMemStoreSize(); - MemstoreSize cf3MemstoreSizePhaseI = region.getStore(FAMILY3).getMemStoreSize(); + MemStoreSize cf1MemstoreSizePhaseI = region.getStore(FAMILY1).getMemStoreSize(); + MemStoreSize cf2MemstoreSizePhaseI = region.getStore(FAMILY2).getMemStoreSize(); + MemStoreSize cf3MemstoreSizePhaseI = region.getStore(FAMILY3).getMemStoreSize(); // Some other sanity checks. assertTrue(cf1MemstoreSizePhaseI.getDataSize() > 0); @@ -689,10 +689,10 @@ public class TestWalAndCompactingMemStoreFlush { cms3.flushInMemory(); region.flush(false); - MemstoreSize cf2MemstoreSizePhaseII = region.getStore(FAMILY2).getMemStoreSize(); + MemStoreSize cf2MemstoreSizePhaseII = region.getStore(FAMILY2).getMemStoreSize(); long smallestSeqInRegionCurrentMemstorePhaseII = - region.getWAL().getEarliestMemstoreSeqNum(region.getRegionInfo().getEncodedNameAsBytes()); + region.getWAL().getEarliestMemStoreSeqNum(region.getRegionInfo().getEncodedNameAsBytes()); long smallestSeqCF1PhaseII = region.getOldestSeqIdOfStore(FAMILY1); long smallestSeqCF2PhaseII = region.getOldestSeqIdOfStore(FAMILY2); long smallestSeqCF3PhaseII = region.getOldestSeqIdOfStore(FAMILY3); @@ -724,7 +724,7 @@ public class TestWalAndCompactingMemStoreFlush { } long smallestSeqInRegionCurrentMemstorePhaseIII = - region.getWAL().getEarliestMemstoreSeqNum(region.getRegionInfo().getEncodedNameAsBytes()); + region.getWAL().getEarliestMemStoreSeqNum(region.getRegionInfo().getEncodedNameAsBytes()); long smallestSeqCF1PhaseIII = region.getOldestSeqIdOfStore(FAMILY1); long smallestSeqCF2PhaseIII = region.getOldestSeqIdOfStore(FAMILY2); long smallestSeqCF3PhaseIII = region.getOldestSeqIdOfStore(FAMILY3); @@ -742,7 +742,7 @@ public class TestWalAndCompactingMemStoreFlush { region.flush(false); long smallestSeqInRegionCurrentMemstorePhaseIV = - region.getWAL().getEarliestMemstoreSeqNum(region.getRegionInfo().getEncodedNameAsBytes()); + region.getWAL().getEarliestMemStoreSeqNum(region.getRegionInfo().getEncodedNameAsBytes()); long smallestSeqCF1PhaseIV = region.getOldestSeqIdOfStore(FAMILY1); long smallestSeqCF2PhaseIV = region.getOldestSeqIdOfStore(FAMILY2); long smallestSeqCF3PhaseIV = region.getOldestSeqIdOfStore(FAMILY3); @@ -795,12 +795,12 @@ public class TestWalAndCompactingMemStoreFlush { region.put(createPut(2, i)); } - long totalMemstoreSize = region.getMemstoreSize(); + long totalMemstoreSize = region.getMemStoreSize(); // Find the sizes of the memstores of each CF. - MemstoreSize cf1MemstoreSizePhaseI = region.getStore(FAMILY1).getMemStoreSize(); - MemstoreSize cf2MemstoreSizePhaseI = region.getStore(FAMILY2).getMemStoreSize(); - MemstoreSize cf3MemstoreSizePhaseI = region.getStore(FAMILY3).getMemStoreSize(); + MemStoreSize cf1MemstoreSizePhaseI = region.getStore(FAMILY1).getMemStoreSize(); + MemStoreSize cf2MemstoreSizePhaseI = region.getStore(FAMILY2).getMemStoreSize(); + MemStoreSize cf3MemstoreSizePhaseI = region.getStore(FAMILY3).getMemStoreSize(); // Some other sanity checks. assertTrue(cf1MemstoreSizePhaseI.getDataSize() > 0); @@ -829,9 +829,9 @@ public class TestWalAndCompactingMemStoreFlush { // Flush-to-disk! CF2 only should be flushed region.flush(false); - MemstoreSize cf1MemstoreSizePhaseII = region.getStore(FAMILY1).getMemStoreSize(); - MemstoreSize cf2MemstoreSizePhaseII = region.getStore(FAMILY2).getMemStoreSize(); - MemstoreSize cf3MemstoreSizePhaseII = region.getStore(FAMILY3).getMemStoreSize(); + MemStoreSize cf1MemstoreSizePhaseII = region.getStore(FAMILY1).getMemStoreSize(); + MemStoreSize cf2MemstoreSizePhaseII = region.getStore(FAMILY2).getMemStoreSize(); + MemStoreSize cf3MemstoreSizePhaseII = region.getStore(FAMILY3).getMemStoreSize(); // CF1 should be flushed in memory and just flattened, so CF1 heap overhead should be smaller assertTrue(cf1MemstoreSizePhaseI.getHeapSize() > cf1MemstoreSizePhaseII.getHeapSize()); @@ -855,7 +855,7 @@ public class TestWalAndCompactingMemStoreFlush { region.put(createPut(2, i)); } - MemstoreSize cf1MemstoreSizePhaseIII = region.getStore(FAMILY1).getMemStoreSize(); + MemStoreSize cf1MemstoreSizePhaseIII = region.getStore(FAMILY1).getMemStoreSize(); // Flush in memory! ((CompactingMemStore) ((HStore)region.getStore(FAMILY1)).memstore).flushInMemory(); @@ -871,8 +871,8 @@ public class TestWalAndCompactingMemStoreFlush { } region.flush(false); - MemstoreSize cf1MemstoreSizePhaseIV = region.getStore(FAMILY1).getMemStoreSize(); - MemstoreSize cf2MemstoreSizePhaseIV = region.getStore(FAMILY2).getMemStoreSize(); + MemStoreSize cf1MemstoreSizePhaseIV = region.getStore(FAMILY1).getMemStoreSize(); + MemStoreSize cf2MemstoreSizePhaseIV = region.getStore(FAMILY2).getMemStoreSize(); assertEquals(2*cf1MemstoreSizePhaseI.getDataSize(), cf1MemstoreSizePhaseIV.getDataSize()); // the decrease in the heap size due to usage of CellArrayMap instead of CSLM diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestWALReplay.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestWALReplay.java index 0598e342706..096d3cacd9d 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestWALReplay.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/AbstractTestWALReplay.java @@ -78,12 +78,11 @@ import org.apache.hadoop.hbase.regionserver.HRegion; import org.apache.hadoop.hbase.regionserver.HRegionServer; import org.apache.hadoop.hbase.regionserver.HStore; import org.apache.hadoop.hbase.regionserver.MemStoreSnapshot; -import org.apache.hadoop.hbase.regionserver.MemstoreSize; +import org.apache.hadoop.hbase.regionserver.MemStoreSize; import org.apache.hadoop.hbase.regionserver.MultiVersionConcurrencyControl; import org.apache.hadoop.hbase.regionserver.Region; import org.apache.hadoop.hbase.regionserver.RegionScanner; import org.apache.hadoop.hbase.regionserver.RegionServerServices; -import org.apache.hadoop.hbase.regionserver.Store; import org.apache.hadoop.hbase.regionserver.throttle.ThroughputController; import org.apache.hadoop.hbase.security.User; import org.apache.hadoop.hbase.util.Bytes; @@ -544,7 +543,7 @@ public abstract class AbstractTestWALReplay { final AtomicInteger countOfRestoredEdits = new AtomicInteger(0); HRegion region3 = new HRegion(basedir, wal3, newFS, newConf, hri, htd, null) { @Override - protected void restoreEdit(HStore s, Cell cell, MemstoreSize memstoreSize) { + protected void restoreEdit(HStore s, Cell cell, MemStoreSize memstoreSize) { super.restoreEdit(s, cell, memstoreSize); countOfRestoredEdits.incrementAndGet(); } @@ -1143,7 +1142,7 @@ public abstract class AbstractTestWALReplay { } @Override - public void setGlobalMemstoreLimit(long globalMemStoreSize) { + public void setGlobalMemStoreLimit(long globalMemStoreSize) { } } diff --git a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestFSHLog.java b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestFSHLog.java index 1585e792bc5..17416ba53e4 100644 --- a/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestFSHLog.java +++ b/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestFSHLog.java @@ -206,7 +206,7 @@ public class TestFSHLog extends AbstractTestFSWAL { assertEquals("Region did not flush?", 1, region.getStoreFileList(new byte[][]{b}).size()); // now check the region's unflushed seqIds. - long seqId = log.getEarliestMemstoreSeqNum(hri.getEncodedNameAsBytes()); + long seqId = log.getEarliestMemStoreSeqNum(hri.getEncodedNameAsBytes()); assertEquals("Found seqId for the region which is already flushed", HConstants.NO_SEQNUM, seqId);