HBASE-18897 Substitute MemStore for Memstore

This commit is contained in:
Chia-Ping Tsai 2017-10-02 21:10:11 +08:00
parent 8c6ed571bd
commit e047f518ef
99 changed files with 647 additions and 622 deletions

View File

@ -562,12 +562,30 @@ public class HTableDescriptor implements TableDescriptor, Comparable<HTableDescr
return this;
}
/**
* @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0.
* Use {@link #hasRegionMemStoreReplication()} instead
*/
@Deprecated
public boolean hasRegionMemstoreReplication() {
return hasRegionMemStoreReplication();
}
/**
* @return true if the read-replicas memstore replication is enabled.
*/
@Override
public boolean hasRegionMemstoreReplication() {
return delegatee.hasRegionMemstoreReplication();
public boolean hasRegionMemStoreReplication() {
return delegatee.hasRegionMemStoreReplication();
}
/**
* @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0.
* Use {@link #setRegionMemStoreReplication(boolean)} instead
*/
@Deprecated
public HTableDescriptor setRegionMemstoreReplication(boolean memstoreReplication) {
return setRegionMemStoreReplication(memstoreReplication);
}
/**
@ -579,8 +597,8 @@ public class HTableDescriptor implements TableDescriptor, Comparable<HTableDescr
* false if the secondaries can tollerate to have new
* data only when the primary flushes the memstore.
*/
public HTableDescriptor setRegionMemstoreReplication(boolean memstoreReplication) {
getDelegateeForModification().setRegionMemstoreReplication(memstoreReplication);
public HTableDescriptor setRegionMemStoreReplication(boolean memstoreReplication) {
getDelegateeForModification().setRegionMemStoreReplication(memstoreReplication);
return this;
}

View File

@ -80,7 +80,7 @@ public class RegionLoad {
* @return the memstore size, in MB
*/
public int getMemStoreSizeMB() {
return regionLoadPB.getMemstoreSizeMB();
return regionLoadPB.getMemStoreSizeMB();
}
/**

View File

@ -65,7 +65,7 @@ public class ServerLoad {
storefiles += rl.getStorefiles();
storeUncompressedSizeMB += rl.getStoreUncompressedSizeMB();
storefileSizeMB += rl.getStorefileSizeMB();
memstoreSizeMB += rl.getMemstoreSizeMB();
memstoreSizeMB += rl.getMemStoreSizeMB();
storefileIndexSizeKB += rl.getStorefileIndexSizeKB();
readRequestsCount += rl.getReadRequestsCount();
filteredReadRequestsCount += rl.getFilteredReadRequestsCount();
@ -147,14 +147,14 @@ public class ServerLoad {
/**
* @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0
* Use {@link #getMemstoreSizeMB()} instead.
* Use {@link #getMemStoreSizeMB()} instead.
*/
@Deprecated
public int getMemstoreSizeInMB() {
return memstoreSizeMB;
}
public int getMemstoreSizeMB() {
public int getMemStoreSizeMB() {
return memstoreSizeMB;
}

View File

@ -167,7 +167,7 @@ public class MetricsConnection implements StatisticTrackable {
}
public void update(RegionLoadStats regionStatistics) {
this.memstoreLoadHist.update(regionStatistics.getMemstoreLoad());
this.memstoreLoadHist.update(regionStatistics.getMemStoreLoad());
this.heapOccupancyHist.update(regionStatistics.getHeapOccupancy());
}
}

View File

@ -34,10 +34,19 @@ public class RegionLoadStats {
this.compactionPressure = compactionPressure;
}
/**
* @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0.
* Use {@link #getMemStoreLoad()} instead
*/
@Deprecated
public int getMemstoreLoad() {
return this.memstoreLoad;
}
public int getMemStoreLoad() {
return this.memstoreLoad;
}
public int getHeapOccupancy() {
return this.heapOccupancy;
}

View File

@ -220,7 +220,7 @@ public interface TableDescriptor {
/**
* @return true if the read-replicas memstore replication is enabled.
*/
boolean hasRegionMemstoreReplication();
boolean hasRegionMemStoreReplication();
/**
* @return true if there are at least one cf whose replication scope is

View File

@ -379,8 +379,8 @@ public class TableDescriptorBuilder {
return this;
}
public TableDescriptorBuilder setRegionMemstoreReplication(boolean memstoreReplication) {
desc.setRegionMemstoreReplication(memstoreReplication);
public TableDescriptorBuilder setRegionMemStoreReplication(boolean memstoreReplication) {
desc.setRegionMemStoreReplication(memstoreReplication);
return this;
}
@ -1067,7 +1067,7 @@ public class TableDescriptorBuilder {
* @return true if the read-replicas memstore replication is enabled.
*/
@Override
public boolean hasRegionMemstoreReplication() {
public boolean hasRegionMemStoreReplication() {
return getOrDefault(REGION_MEMSTORE_REPLICATION_KEY, Boolean::valueOf, DEFAULT_REGION_MEMSTORE_REPLICATION);
}
@ -1081,7 +1081,7 @@ public class TableDescriptorBuilder {
* have new data only when the primary flushes the memstore.
* @return the modifyable TD
*/
public ModifyableTableDescriptor setRegionMemstoreReplication(boolean memstoreReplication) {
public ModifyableTableDescriptor setRegionMemStoreReplication(boolean memstoreReplication) {
setValue(REGION_MEMSTORE_REPLICATION_KEY, Boolean.toString(memstoreReplication));
// If the memstore replication is setup, we do not have to wait for observing a flush event
// from primary before starting to serve reads, because gaps from replication is not applicable

View File

@ -64,7 +64,7 @@ public class ExponentialClientBackoffPolicy implements ClientBackoffPolicy {
}
// Factor in memstore load
double percent = regionStats.getMemstoreLoadPercent() / 100.0;
double percent = regionStats.getMemStoreLoadPercent() / 100.0;
// Factor in heap occupancy
float heapOccupancy = regionStats.getHeapOccupancyPercent() / 100.0f;

View File

@ -59,12 +59,12 @@ public class ServerStatistics {
private int compactionPressure = 0;
public void update(RegionLoadStats currentStats) {
this.memstoreLoad = currentStats.getMemstoreLoad();
this.memstoreLoad = currentStats.getMemStoreLoad();
this.heapOccupancy = currentStats.getHeapOccupancy();
this.compactionPressure = currentStats.getCompactionPressure();
}
public int getMemstoreLoadPercent(){
public int getMemStoreLoadPercent(){
return this.memstoreLoad;
}

View File

@ -3155,7 +3155,7 @@ public final class ProtobufUtil {
}
public static RegionLoadStats createRegionLoadStats(ClientProtos.RegionLoadStats stats) {
return new RegionLoadStats(stats.getMemstoreLoad(), stats.getHeapOccupancy(),
return new RegionLoadStats(stats.getMemStoreLoad(), stats.getHeapOccupancy(),
stats.getCompactionPressure());
}

View File

@ -156,7 +156,7 @@ public class TestClientExponentialBackoff {
private void update(ServerStatistics stats, int load) {
ClientProtos.RegionLoadStats stat = ClientProtos.RegionLoadStats.newBuilder()
.setMemstoreLoad
.setMemStoreLoad
(load).build();
stats.update(regionname, ProtobufUtil.createRegionLoadStats(stat));
}
@ -164,7 +164,7 @@ public class TestClientExponentialBackoff {
private void update(ServerStatistics stats, int memstoreLoad, int heapOccupancy,
int compactionPressure) {
ClientProtos.RegionLoadStats stat = ClientProtos.RegionLoadStats.newBuilder()
.setMemstoreLoad(memstoreLoad)
.setMemStoreLoad(memstoreLoad)
.setHeapOccupancy(heapOccupancy)
.setCompactionPressure(compactionPressure)
.build();

View File

@ -175,7 +175,7 @@ public interface MetricsRegionServerSource extends BaseSource, JvmPauseMonitorSo
* Update the flush memstore size histogram
* @param bytes the number of bytes in the memstore
*/
void updateFlushMemstoreSize(long bytes);
void updateFlushMemStoreSize(long bytes);
/**
* Update the flush output file size histogram

View File

@ -71,7 +71,7 @@ public interface MetricsRegionServerWrapper {
* Get the number of WAL files of this region server.
*/
long getNumWALFiles();
/**
* Get the size of WAL files of this region server.
*/
@ -90,7 +90,7 @@ public interface MetricsRegionServerWrapper {
/**
* Get the size of the memstore on this region server.
*/
long getMemstoreSize();
long getMemStoreSize();
/**
* Get the total size of the store files this region server is serving from.
@ -207,7 +207,7 @@ public interface MetricsRegionServerWrapper {
*/
int getFlushQueueSize();
public long getMemstoreLimit();
public long getMemStoreLimit();
/**
* Get the size (in bytes) of the block cache that is free.
*/

View File

@ -57,7 +57,7 @@ public interface MetricsRegionWrapper {
/**
* Get the size of the memstore on this region server.
*/
long getMemstoreSize();
long getMemStoreSize();
/**
* Get the total size of the store files this region server is serving from.

View File

@ -43,7 +43,7 @@ public interface MetricsTableWrapperAggregate {
/**
* Get the memory store size against this table
*/
long getMemstoresSize(String table);
long getMemStoresSize(String table);
/**
* Get the store file size against this table

View File

@ -264,7 +264,7 @@ public class MetricsRegionServerSourceImpl
}
@Override
public void updateFlushMemstoreSize(long bytes) {
public void updateFlushMemStoreSize(long bytes) {
flushMemstoreSizeHisto.add(bytes);
flushedMemstoreBytes.incr(bytes);
}
@ -339,7 +339,7 @@ public class MetricsRegionServerSourceImpl
.addGauge(Interns.info(WALFILE_COUNT, WALFILE_COUNT_DESC), rsWrap.getNumWALFiles())
.addGauge(Interns.info(WALFILE_SIZE, WALFILE_SIZE_DESC), rsWrap.getWALFileSize())
.addGauge(Interns.info(STOREFILE_COUNT, STOREFILE_COUNT_DESC), rsWrap.getNumStoreFiles())
.addGauge(Interns.info(MEMSTORE_SIZE, MEMSTORE_SIZE_DESC), rsWrap.getMemstoreSize())
.addGauge(Interns.info(MEMSTORE_SIZE, MEMSTORE_SIZE_DESC), rsWrap.getMemStoreSize())
.addGauge(Interns.info(STOREFILE_SIZE, STOREFILE_SIZE_DESC), rsWrap.getStoreFileSize())
.addGauge(Interns.info(MAX_STORE_FILE_AGE, MAX_STORE_FILE_AGE_DESC),
rsWrap.getMaxStoreFileAge())

View File

@ -216,7 +216,7 @@ public class MetricsRegionSourceImpl implements MetricsRegionSource {
mrb.addGauge(Interns.info(
regionNamePrefix + MetricsRegionServerSource.MEMSTORE_SIZE,
MetricsRegionServerSource.MEMSTORE_SIZE_DESC),
this.regionWrapper.getMemstoreSize());
this.regionWrapper.getMemStoreSize());
mrb.addGauge(Interns.info(
regionNamePrefix + MetricsRegionServerSource.MAX_STORE_FILE_AGE,
MetricsRegionServerSource.MAX_STORE_FILE_AGE_DESC),

View File

@ -133,7 +133,7 @@ public class MetricsTableSourceImpl implements MetricsTableSource {
tableWrapperAgg.getTotalRequestsCount(tableName.getNameAsString()));
mrb.addGauge(Interns.info(tableNamePrefix + MetricsTableSource.MEMSTORE_SIZE,
MetricsTableSource.MEMSTORE_SIZE_DESC),
tableWrapperAgg.getMemstoresSize(tableName.getNameAsString()));
tableWrapperAgg.getMemStoresSize(tableName.getNameAsString()));
mrb.addGauge(Interns.info(tableNamePrefix + MetricsTableSource.STORE_FILE_SIZE,
MetricsTableSource.STORE_FILE_SIZE_DESC),
tableWrapperAgg.getStoreFilesSize(tableName.getNameAsString()));

View File

@ -92,7 +92,7 @@ public class TestMetricsRegionSourceImpl {
}
@Override
public long getMemstoreSize() {
public long getMemStoreSize() {
return 0;
}

View File

@ -92,7 +92,7 @@ public class TestMetricsTableSourceImpl {
}
@Override
public long getMemstoresSize(String table) {
public long getMemStoresSize(String table) {
return 1000;
}

View File

@ -102,7 +102,7 @@ message Result {
// in the query.
optional bool exists = 3;
// Whether or not the results are coming from possibly stale data
// Whether or not the results are coming from possibly stale data
optional bool stale = 4 [default = false];
// Whether or not the entire result could be returned. Results will be split when
@ -452,7 +452,7 @@ message RegionAction {
*/
message RegionLoadStats {
// Percent load on the memstore. Guaranteed to be positive, between 0 and 100.
optional int32 memstoreLoad = 1 [default = 0];
optional int32 memStoreLoad = 1 [default = 0];
// Percent JVM heap occupancy. Guaranteed to be positive, between 0 and 100.
// We can move this to "ServerLoadStats" should we develop them.
optional int32 heapOccupancy = 2 [default = 0];
@ -533,7 +533,7 @@ service ClientService {
rpc ExecService(CoprocessorServiceRequest)
returns(CoprocessorServiceResponse);
rpc ExecRegionServerService(CoprocessorServiceRequest)
returns(CoprocessorServiceResponse);

View File

@ -94,7 +94,7 @@ message RegionLoad {
optional uint32 storefile_size_MB = 5;
/** the current size of the memstore for the region, in MB */
optional uint32 memstore_size_MB = 6;
optional uint32 mem_store_size_MB = 6;
/**
* The current total size of root-level store file indexes for the region,

View File

@ -103,7 +103,7 @@ message Result {
// in the query.
optional bool exists = 3;
// Whether or not the results are coming from possibly stale data
// Whether or not the results are coming from possibly stale data
optional bool stale = 4 [default = false];
// Whether or not the entire result could be returned. Results will be split when
@ -453,7 +453,7 @@ message RegionAction {
*/
message RegionLoadStats {
// Percent load on the memstore. Guaranteed to be positive, between 0 and 100.
optional int32 memstoreLoad = 1 [default = 0];
optional int32 memStoreLoad = 1 [default = 0];
// Percent JVM heap occupancy. Guaranteed to be positive, between 0 and 100.
// We can move this to "ServerLoadStats" should we develop them.
optional int32 heapOccupancy = 2 [default = 0];
@ -534,7 +534,7 @@ service ClientService {
rpc ExecService(CoprocessorServiceRequest)
returns(CoprocessorServiceResponse);
rpc ExecRegionServerService(CoprocessorServiceRequest)
returns(CoprocessorServiceResponse);

View File

@ -203,7 +203,7 @@ public class StorageClusterStatusModel
* @return memstore size, in MB
*/
@XmlAttribute
public int getMemstoreSizeMB() {
public int getMemStoreSizeMB() {
return memstoreSizeMB;
}
@ -356,7 +356,7 @@ public class StorageClusterStatusModel
/**
* @param memstoreSizeMB memstore size, in MB
*/
public void setMemstoreSizeMB(int memstoreSizeMB) {
public void setMemStoreSizeMB(int memstoreSizeMB) {
this.memstoreSizeMB = memstoreSizeMB;
}
@ -728,7 +728,7 @@ public class StorageClusterStatusModel
regionBuilder.setStores(region.stores);
regionBuilder.setStorefiles(region.storefiles);
regionBuilder.setStorefileSizeMB(region.storefileSizeMB);
regionBuilder.setMemstoreSizeMB(region.memstoreSizeMB);
regionBuilder.setMemStoreSizeMB(region.memstoreSizeMB);
regionBuilder.setStorefileIndexSizeKB(region.storefileIndexSizeKB);
regionBuilder.setReadRequestsCount(region.readRequestsCount);
regionBuilder.setWriteRequestsCount(region.writeRequestsCount);
@ -774,7 +774,7 @@ public class StorageClusterStatusModel
region.getStores(),
region.getStorefiles(),
region.getStorefileSizeMB(),
region.getMemstoreSizeMB(),
region.getMemStoreSizeMB(),
region.getStorefileIndexSizeKB(),
region.getReadRequestsCount(),
region.getWriteRequestsCount(),

View File

@ -23,7 +23,7 @@ message StorageClusterStatus {
optional int32 stores = 2;
optional int32 storefiles = 3;
optional int32 storefileSizeMB = 4;
optional int32 memstoreSizeMB = 5;
optional int32 memStoreSizeMB = 5;
optional int64 storefileIndexSizeKB = 6;
optional int64 readRequestsCount = 7;
optional int64 writeRequestsCount = 8;
@ -45,7 +45,7 @@ message StorageClusterStatus {
repeated Node liveNodes = 1;
repeated string deadNodes = 2;
// summary statistics
optional int32 regions = 3;
optional int32 regions = 3;
optional int64 requests = 4;
optional double averageLoad = 5;
}

View File

@ -61,13 +61,13 @@ public class TestStorageClusterStatusModel extends TestModelBase<StorageClusterS
AS_JSON =
"{\"regions\":2,\"requests\":0,\"averageLoad\":1.0,\"LiveNodes\":[{\"name\":\"test1\"," +
"\"Region\":[{\"name\":\"aGJhc2U6cm9vdCwsMA==\",\"stores\":1,\"storefiles\":1," +
"\"storefileSizeMB\":0,\"memstoreSizeMB\":0,\"storefileIndexSizeKB\":0," +
"\"storefileSizeMB\":0,\"memStoreSizeMB\":0,\"storefileIndexSizeKB\":0," +
"\"readRequestsCount\":1,\"writeRequestsCount\":2,\"rootIndexSizeKB\":1," +
"\"totalStaticIndexSizeKB\":1,\"totalStaticBloomSizeKB\":1,\"totalCompactingKVs\":1," +
"\"currentCompactedKVs\":1}],\"requests\":0,\"startCode\":1245219839331," +
"\"heapSizeMB\":128,\"maxHeapSizeMB\":1024},{\"name\":\"test2\"," +
"\"Region\":[{\"name\":\"aGJhc2U6bWV0YSwsMTI0NjAwMDA0MzcyNA==\",\"stores\":1," +
"\"storefiles\":1,\"storefileSizeMB\":0,\"memstoreSizeMB\":0,\"storefileIndexSizeKB\":0," +
"\"storefiles\":1,\"storefileSizeMB\":0,\"memStoreSizeMB\":0,\"storefileIndexSizeKB\":0," +
"\"readRequestsCount\":1,\"writeRequestsCount\":2,\"rootIndexSizeKB\":1," +
"\"totalStaticIndexSizeKB\":1,\"totalStaticBloomSizeKB\":1,\"totalCompactingKVs\":1," +
"\"currentCompactedKVs\":1}],\"requests\":0,\"startCode\":1245239331198," +
@ -98,7 +98,7 @@ public class TestStorageClusterStatusModel extends TestModelBase<StorageClusterS
assertEquals(node.getStartCode(), 1245219839331L);
assertEquals(node.getHeapSizeMB(), 128);
assertEquals(node.getMaxHeapSizeMB(), 1024);
Iterator<StorageClusterStatusModel.Node.Region> regions =
Iterator<StorageClusterStatusModel.Node.Region> 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<StorageClusterS
assertEquals(region.getStores(), 1);
assertEquals(region.getStorefiles(), 1);
assertEquals(region.getStorefileSizeMB(), 0);
assertEquals(region.getMemstoreSizeMB(), 0);
assertEquals(region.getMemStoreSizeMB(), 0);
assertEquals(region.getStorefileIndexSizeKB(), 0);
assertEquals(region.getReadRequestsCount(), 1);
assertEquals(region.getWriteRequestsCount(), 2);
@ -128,7 +128,7 @@ public class TestStorageClusterStatusModel extends TestModelBase<StorageClusterS
assertEquals(region.getStores(), 1);
assertEquals(region.getStorefiles(), 1);
assertEquals(region.getStorefileSizeMB(), 0);
assertEquals(region.getMemstoreSizeMB(), 0);
assertEquals(region.getMemStoreSizeMB(), 0);
assertEquals(region.getStorefileIndexSizeKB(), 0);
assertEquals(region.getReadRequestsCount(), 1);
assertEquals(region.getWriteRequestsCount(), 2);
@ -137,7 +137,7 @@ public class TestStorageClusterStatusModel extends TestModelBase<StorageClusterS
assertEquals(region.getTotalStaticBloomSizeKB(), 1);
assertEquals(region.getTotalCompactingKVs(), 1);
assertEquals(region.getCurrentCompactedKVs(), 1);
assertFalse(regions.hasNext());
assertFalse(nodes.hasNext());
}

View File

@ -94,7 +94,7 @@ Arrays.sort(serverNames);
if (!masterVersion.equals(version)) {
inconsistentNodeNum ++;
}
double requestsPerSecond = 0.0;
int numRegionsOnline = 0;
long lastContact = 0;
@ -158,7 +158,7 @@ for (ServerName serverName: serverNames) {
* TraditionalBinaryPrefix.MEGA.value, "B", 1) %></td>
<td><% TraditionalBinaryPrefix.long2String(sl.getMaxHeapMB()
* TraditionalBinaryPrefix.MEGA.value, "B", 1) %></td>
<td><% TraditionalBinaryPrefix.long2String(sl.getMemstoreSizeMB()
<td><% TraditionalBinaryPrefix.long2String(sl.getMemStoreSizeMB()
* TraditionalBinaryPrefix.MEGA.value, "B", 1) %></td>
</tr>

View File

@ -255,7 +255,7 @@
<td><a href="region.jsp?name=<% r.getEncodedName() %>"><% displayName %></a></td>
<%if load != null %>
<td><% TraditionalBinaryPrefix.long2String(
load.getMemstoreSizeMB() * TraditionalBinaryPrefix.MEGA.value, "B", 1) %></td>
load.getMemStoreSizeMB() * TraditionalBinaryPrefix.MEGA.value, "B", 1) %></td>
</%if>
</tr>
</%for>

View File

@ -129,10 +129,10 @@ MetricsRegionServerWrapper mWrap;
<% TraditionalBinaryPrefix.long2String(DirectMemoryUtils.getDirectMemorySize(), "B", 1) %>
</td>
<td>
<% TraditionalBinaryPrefix.long2String(mWrap.getMemstoreSize(), "B", 1) %>
<% TraditionalBinaryPrefix.long2String(mWrap.getMemStoreSize(), "B", 1) %>
</td>
<td>
<% TraditionalBinaryPrefix.long2String(mWrap.getMemstoreLimit(), "B", 1) %>
<% TraditionalBinaryPrefix.long2String(mWrap.getMemStoreLimit(), "B", 1) %>
</td>
</tr>
</table>

View File

@ -489,9 +489,9 @@ public class HFile {
boolean isPrimaryReplicaReader();
boolean shouldIncludeMemstoreTS();
boolean shouldIncludeMemStoreTS();
boolean isDecodeMemstoreTS();
boolean isDecodeMemStoreTS();
DataBlockEncoding getEffectiveEncodingInCache(boolean isCompaction);

View File

@ -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);
}

View File

@ -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);

View File

@ -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<Long, MemoryType> getGlobalMemstoreSize(Configuration conf) {
public static Pair<Long, MemoryType> 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) {

View File

@ -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));
}

View File

@ -58,7 +58,7 @@ public class MobFile {
List<HStoreFile> sfs = new ArrayList<>();
sfs.add(sf);
List<StoreFileScanner> 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());
}
/**

View File

@ -96,14 +96,14 @@ public abstract class AbstractMemStore implements MemStore {
public abstract void updateLowestUnflushedSequenceIdInWAL(boolean onlyIfMoreRecent);
@Override
public void add(Iterable<Cell> cells, MemstoreSize memstoreSize) {
public void add(Iterable<Cell> 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<Cell> cells, long readpoint, MemstoreSize memstoreSize) {
public void upsert(Iterable<Cell> 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();

View File

@ -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

View File

@ -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

View File

@ -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

View File

@ -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<? extends Segment> 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<? extends Segment> 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<? extends Segment> suffix, ImmutableSegment segment,

View File

@ -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");
}

View File

@ -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;
}

View File

@ -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);
}

View File

@ -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);

View File

@ -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);

View File

@ -65,5 +65,5 @@ public interface FlushRequester {
*
* @param globalMemStoreSize
*/
public void setGlobalMemstoreLimit(long globalMemStoreSize);
public void setGlobalMemStoreLimit(long globalMemStoreSize);
}

View File

@ -518,23 +518,23 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
final FlushResultImpl result; // indicating a failure result from prepare
final TreeMap<byte[], StoreFlushContext> storeFlushCtxs;
final TreeMap<byte[], List<Path>> committedFiles;
final TreeMap<byte[], MemstoreSize> storeFlushableSize;
final TreeMap<byte[], MemStoreSize> 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<byte[], StoreFlushContext> storeFlushCtxs,
TreeMap<byte[], List<Path>> committedFiles,
TreeMap<byte[], MemstoreSize> storeFlushableSize, long startTime, long flushSeqId,
long flushedSeqId, MemstoreSize totalFlushableSize) {
TreeMap<byte[], MemStoreSize> 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<byte[], StoreFlushContext> storeFlushCtxs,
TreeMap<byte[], List<Path>> committedFiles,
TreeMap<byte[], MemstoreSize> storeFlushableSize, long startTime, long flushSeqId,
long flushedSeqId, MemstoreSize totalFlushableSize) {
TreeMap<byte[], MemStoreSize> 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<HStore> 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<byte[], Long> flushedFamilyNamesToSeq = new HashMap<>();
for (HStore store : storesToFlush) {
@ -2449,7 +2449,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
TreeMap<byte[], StoreFlushContext> storeFlushCtxs = new TreeMap<>(Bytes.BYTES_COMPARATOR);
TreeMap<byte[], List<Path>> committedFiles = new TreeMap<>(Bytes.BYTES_COMPARATOR);
TreeMap<byte[], MemstoreSize> storeFlushableSize = new TreeMap<>(Bytes.BYTES_COMPARATOR);
TreeMap<byte[], MemStoreSize> 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<RowLock> 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<byte[], List<Cell>> familyMap,
MemstoreSize memstoreSize) throws IOException {
private void applyFamilyMapToMemStore(Map<byte[], List<Cell>> familyMap,
MemStoreSize memstoreSize) throws IOException {
for (Map.Entry<byte[], List<Cell>> e : familyMap.entrySet()) {
byte[] family = e.getKey();
List<Cell> 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<Cell> 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<Cell> 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<HStore, Long> 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<Cell> 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<HStore, List<Cell>> 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());
}

View File

@ -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<Long, MemoryType> pair = MemorySizeUtil.getGlobalMemstoreSize(conf);
Pair<Long, MemoryType> 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;
}

View File

@ -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<Cell> cells, MemstoreSize memstoreSize) {
public void add(final Iterable<Cell> 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<Cell> cells, long readpoint, MemstoreSize memstoreSize)
public void upsert(Iterable<Cell> 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();
}

View File

@ -159,7 +159,7 @@ public class HStoreFile implements StoreFile {
}
@Override
public long getMaxMemstoreTS() {
public long getMaxMemStoreTS() {
return maxMemstoreTS;
}

View File

@ -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;
}

View File

@ -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<Cell> cells, MemstoreSize memstoreSize);
void add(Iterable<Cell> 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<Cell> cells, long readpoint, MemstoreSize memstoreSize);
void upsert(Iterable<Cell> 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.

View File

@ -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<Long, Region> regionsBySize,
Set<Region> excludedRegions,
boolean checkStoreFileCount) {
@ -307,7 +307,7 @@ class MemStoreFlusher implements FlushRequester {
return null;
}
private Region getBiggestMemstoreOfRegionReplica(SortedMap<Long, Region> regionsBySize,
private Region getBiggestMemStoreOfRegionReplica(SortedMap<Long, Region> regionsBySize,
Set<Region> 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();
}

View File

@ -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;
}

View File

@ -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);
}

View File

@ -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;
}

View File

@ -110,7 +110,7 @@ public class MetricsRegionWrapperImpl implements MetricsRegionWrapper, Closeable
}
@Override
public long getMemstoreSize() {
public long getMemStoreSize() {
return memstoreSize;
}

View File

@ -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;
}

View File

@ -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 {

View File

@ -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();

View File

@ -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<byte[], MemstoreSize> replayEditsPerRegion =
private final ConcurrentMap<byte[], MemStoreSize> replayEditsPerRegion =
new ConcurrentSkipListMap<>(Bytes.BYTES_COMPARATOR);
private long globalMemStoreLimit;
@ -57,7 +57,7 @@ public class RegionServerAccounting {
private long globalOnHeapMemstoreLimitLowMark;
public RegionServerAccounting(Configuration conf) {
Pair<Long, MemoryType> globalMemstoreSizePair = MemorySizeUtil.getGlobalMemstoreSize(conf);
Pair<Long, MemoryType> 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) {

View File

@ -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();
}
}

View File

@ -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);
}

View File

@ -95,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:

View File

@ -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();
}

View File

@ -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.

View File

@ -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.

View File

@ -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<HStoreFile> sfs) {
return sfs.stream().filter(sf -> !sf.isBulkLoadResult()).mapToLong(HStoreFile::getMaxMemstoreTS)
public static OptionalLong getMaxMemStoreTSInList(Collection<HStoreFile> sfs) {
return sfs.stream().filter(sf -> !sf.isBulkLoadResult()).mapToLong(HStoreFile::getMaxMemStoreTS)
.max();
}

View File

@ -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;
}

View File

@ -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);

View File

@ -77,7 +77,7 @@ public abstract class Compactor<T extends CellSink> {
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<T extends CellSink> {
protected FileDetails getFileDetails(
Collection<HStoreFile> 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();

View File

@ -302,7 +302,7 @@ public abstract class AbstractFSWAL<W extends WriterBase> implements WAL {
}
private int calculateMaxLogFiles(Configuration conf, long logRollSize) {
Pair<Long, MemoryType> globalMemstoreSize = MemorySizeUtil.getGlobalMemstoreSize(conf);
Pair<Long, MemoryType> globalMemstoreSize = MemorySizeUtil.getGlobalMemStoreSize(conf);
return (int) ((globalMemstoreSize.getFirst() * 2) / logRollSize);
}
@ -468,13 +468,13 @@ public abstract class AbstractFSWAL<W extends WriterBase> 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<W extends WriterBase> 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);

View File

@ -92,7 +92,7 @@ class FSWALEntry extends Entry {
return "sequence=" + this.txid + ", " + super.toString();
};
boolean isInMemstore() {
boolean isInMemStore() {
return this.inMemstore;
}

View File

@ -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);
}

View File

@ -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;
}

View File

@ -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.

View File

@ -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<HRegionServer> getOnlineRegionServers() {

View File

@ -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

View File

@ -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);
}
}

View File

@ -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);
}
}
}

View File

@ -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) {

View File

@ -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];

View File

@ -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;
}

View File

@ -61,7 +61,7 @@ public class MetricsRegionWrapperStub implements MetricsRegionWrapper {
}
@Override
public long getMemstoreSize() {
public long getMemStoreSize() {
return 103;
}

View File

@ -42,7 +42,7 @@ public class MetricsTableWrapperStub implements MetricsTableWrapperAggregate {
}
@Override
public long getMemstoresSize(String table) {
public long getMemStoresSize(String table) {
return 1000;
}

View File

@ -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<Cell> 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;
}

View File

@ -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();
}

View File

@ -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<Cell> 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);

View File

@ -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);
}
}

View File

@ -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);

View File

@ -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);

View File

@ -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

View File

@ -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;
}

View File

@ -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);
}
}

View File

@ -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());
}

View File

@ -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());

View File

@ -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

View File

@ -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) {
}
}

View File

@ -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);