HBASE-3647 Distinguish read and write request count in region -- reversed the patch because missing a file
git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1087953 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
371f963359
commit
9bf60eb011
|
@ -27,6 +27,7 @@ import java.util.Comparator;
|
|||
import java.util.Set;
|
||||
|
||||
import org.apache.hadoop.hbase.regionserver.HRegionServer;
|
||||
import org.apache.hadoop.io.VersionedWritable;
|
||||
import org.apache.hadoop.io.Writable;
|
||||
import org.apache.hadoop.io.WritableComparable;
|
||||
|
||||
|
@ -43,7 +44,10 @@ import org.apache.hadoop.io.WritableComparable;
|
|||
* by. In subsequent communications, the regionserver will pass a HServerInfo
|
||||
* with the master-supplied address.
|
||||
*/
|
||||
public class HServerInfo implements WritableComparable<HServerInfo> {
|
||||
public class HServerInfo extends VersionedWritable
|
||||
implements WritableComparable<HServerInfo> {
|
||||
private static final byte VERSION = 0;
|
||||
|
||||
/*
|
||||
* This character is used as separator between server hostname and port and
|
||||
* its startcode. Servername is formatted as
|
||||
|
@ -61,6 +65,11 @@ public class HServerInfo implements WritableComparable<HServerInfo> {
|
|||
private String hostname;
|
||||
private String cachedHostnamePort = null;
|
||||
|
||||
/** @return the object version number */
|
||||
public byte getVersion() {
|
||||
return VERSION;
|
||||
}
|
||||
|
||||
public HServerInfo() {
|
||||
this(new HServerAddress(), 0, HConstants.DEFAULT_REGIONSERVER_INFOPORT,
|
||||
"default name");
|
||||
|
|
|
@ -28,13 +28,17 @@ import java.util.TreeMap;
|
|||
|
||||
import org.apache.hadoop.hbase.util.Bytes;
|
||||
import org.apache.hadoop.hbase.util.Strings;
|
||||
import org.apache.hadoop.io.VersionedWritable;
|
||||
import org.apache.hadoop.io.Writable;
|
||||
import org.apache.hadoop.io.WritableComparable;
|
||||
|
||||
/**
|
||||
* This class encapsulates metrics for determining the load on a HRegionServer
|
||||
*/
|
||||
public class HServerLoad implements WritableComparable<HServerLoad> {
|
||||
public class HServerLoad extends VersionedWritable
|
||||
implements WritableComparable<HServerLoad> {
|
||||
private static final byte VERSION = 0;
|
||||
|
||||
/** number of regions */
|
||||
// could just use regionLoad.size() but master.RegionManager likes to play
|
||||
// around with this value while passing HServerLoad objects around during
|
||||
|
@ -49,6 +53,11 @@ public class HServerLoad implements WritableComparable<HServerLoad> {
|
|||
/** per-region load metrics */
|
||||
private Map<byte[], RegionLoad> regionLoad = new TreeMap<byte[], RegionLoad>(Bytes.BYTES_COMPARATOR);
|
||||
|
||||
/** @return the object version number */
|
||||
public byte getVersion() {
|
||||
return VERSION;
|
||||
}
|
||||
|
||||
/**
|
||||
* Encapsulates per-region loading metrics.
|
||||
*/
|
||||
|
@ -65,8 +74,10 @@ public class HServerLoad implements WritableComparable<HServerLoad> {
|
|||
private int memstoreSizeMB;
|
||||
/** the current total size of storefile indexes for the region, in MB */
|
||||
private int storefileIndexSizeMB;
|
||||
/** the current total request made to region */
|
||||
private long requestsCount;
|
||||
/** the current total read requests made to region */
|
||||
private int readRequestsCount;
|
||||
/** the current total write requests made to region */
|
||||
private int writeRequestsCount;
|
||||
|
||||
/**
|
||||
* Constructor, for Writable
|
||||
|
@ -82,18 +93,21 @@ public class HServerLoad implements WritableComparable<HServerLoad> {
|
|||
* @param storefileSizeMB
|
||||
* @param memstoreSizeMB
|
||||
* @param storefileIndexSizeMB
|
||||
* @param requestsCount
|
||||
* @param readRequestsCount
|
||||
* @param writeRequestsCount
|
||||
*/
|
||||
public RegionLoad(final byte[] name, final int stores,
|
||||
final int storefiles, final int storefileSizeMB,
|
||||
final int memstoreSizeMB, final int storefileIndexSizeMB,final long requestsCount) {
|
||||
final int memstoreSizeMB, final int storefileIndexSizeMB,
|
||||
final int readRequestsCount, final int writeRequestsCount) {
|
||||
this.name = name;
|
||||
this.stores = stores;
|
||||
this.storefiles = storefiles;
|
||||
this.storefileSizeMB = storefileSizeMB;
|
||||
this.memstoreSizeMB = memstoreSizeMB;
|
||||
this.storefileIndexSizeMB = storefileIndexSizeMB;
|
||||
this.requestsCount = requestsCount;
|
||||
this.readRequestsCount = readRequestsCount;
|
||||
this.writeRequestsCount = writeRequestsCount;
|
||||
}
|
||||
|
||||
// Getters
|
||||
|
@ -146,12 +160,26 @@ public class HServerLoad implements WritableComparable<HServerLoad> {
|
|||
public int getStorefileIndexSizeMB() {
|
||||
return storefileIndexSizeMB;
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* @return the number of requests made to region
|
||||
*/
|
||||
public long getRequestsCount() {
|
||||
return requestsCount;
|
||||
return readRequestsCount + writeRequestsCount;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return the number of read requests made to region
|
||||
*/
|
||||
public long getReadRequestsCount() {
|
||||
return readRequestsCount;
|
||||
}
|
||||
|
||||
/**
|
||||
* @return the number of read requests made to region
|
||||
*/
|
||||
public long getWriteRequestsCount() {
|
||||
return writeRequestsCount;
|
||||
}
|
||||
|
||||
// Setters
|
||||
|
@ -193,10 +221,17 @@ public class HServerLoad implements WritableComparable<HServerLoad> {
|
|||
}
|
||||
|
||||
/**
|
||||
* @param requestsCount the number of requests to region
|
||||
* @param requestsCount the number of read requests to region
|
||||
*/
|
||||
public void setRequestsCount(long requestsCount) {
|
||||
this.requestsCount = requestsCount;
|
||||
public void setReadRequestsCount(int requestsCount) {
|
||||
this.readRequestsCount = requestsCount;
|
||||
}
|
||||
|
||||
/**
|
||||
* @param requestsCount the number of write requests to region
|
||||
*/
|
||||
public void setWriteRequestsCount(int requestsCount) {
|
||||
this.writeRequestsCount = requestsCount;
|
||||
}
|
||||
|
||||
// Writable
|
||||
|
@ -209,7 +244,8 @@ public class HServerLoad implements WritableComparable<HServerLoad> {
|
|||
this.storefileSizeMB = in.readInt();
|
||||
this.memstoreSizeMB = in.readInt();
|
||||
this.storefileIndexSizeMB = in.readInt();
|
||||
this.requestsCount = in.readLong();
|
||||
this.readRequestsCount = in.readInt();
|
||||
this.writeRequestsCount = in.readInt();
|
||||
}
|
||||
|
||||
public void write(DataOutput out) throws IOException {
|
||||
|
@ -220,7 +256,8 @@ public class HServerLoad implements WritableComparable<HServerLoad> {
|
|||
out.writeInt(storefileSizeMB);
|
||||
out.writeInt(memstoreSizeMB);
|
||||
out.writeInt(storefileIndexSizeMB);
|
||||
out.writeLong(requestsCount);
|
||||
out.writeInt(readRequestsCount);
|
||||
out.writeInt(writeRequestsCount);
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -238,8 +275,10 @@ public class HServerLoad implements WritableComparable<HServerLoad> {
|
|||
Integer.valueOf(this.memstoreSizeMB));
|
||||
sb = Strings.appendKeyValue(sb, "storefileIndexSizeMB",
|
||||
Integer.valueOf(this.storefileIndexSizeMB));
|
||||
sb = Strings.appendKeyValue(sb, "requestsCount",
|
||||
Long.valueOf(this.requestsCount));
|
||||
sb = Strings.appendKeyValue(sb, "readRequestsCount",
|
||||
Long.valueOf(this.readRequestsCount));
|
||||
sb = Strings.appendKeyValue(sb, "writeRequestsCount",
|
||||
Long.valueOf(this.writeRequestsCount));
|
||||
return sb.toString();
|
||||
}
|
||||
}
|
||||
|
@ -483,9 +522,9 @@ public class HServerLoad implements WritableComparable<HServerLoad> {
|
|||
public void addRegionInfo(final byte[] name, final int stores,
|
||||
final int storefiles, final int storefileSizeMB,
|
||||
final int memstoreSizeMB, final int storefileIndexSizeMB,
|
||||
final long requestsCount) {
|
||||
final int readRequestsCount, final int writeRequestsCount) {
|
||||
this.regionLoad.put(name, new HServerLoad.RegionLoad(name, stores, storefiles,
|
||||
storefileSizeMB, memstoreSizeMB, storefileIndexSizeMB, requestsCount));
|
||||
storefileSizeMB, memstoreSizeMB, storefileIndexSizeMB, readRequestsCount, writeRequestsCount));
|
||||
}
|
||||
|
||||
// Writable
|
||||
|
|
|
@ -177,7 +177,8 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
|
||||
final AtomicLong memstoreSize = new AtomicLong(0);
|
||||
|
||||
final Counter requestsCount = new Counter();
|
||||
final Counter readRequestsCount = new Counter();
|
||||
final Counter writeRequestsCount = new Counter();
|
||||
|
||||
/**
|
||||
* The directory for the table this region is part of.
|
||||
|
@ -462,7 +463,17 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
|
||||
/** @return requestsCount for this region */
|
||||
public long getRequestsCount() {
|
||||
return this.requestsCount.get();
|
||||
return this.readRequestsCount.get() + this.writeRequestsCount.get();
|
||||
}
|
||||
|
||||
/** @return readRequestsCount for this region */
|
||||
public long getReadRequestsCount() {
|
||||
return this.readRequestsCount.get();
|
||||
}
|
||||
|
||||
/** @return writeRequestsCount for this region */
|
||||
public long getWriteRequestsCount() {
|
||||
return this.writeRequestsCount.get();
|
||||
}
|
||||
|
||||
/** @return true if region is closed */
|
||||
|
@ -1131,6 +1142,7 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
// closest key is across all column families, since the data may be sparse
|
||||
checkRow(row);
|
||||
startRegionOperation();
|
||||
this.readRequestsCount.increment();
|
||||
try {
|
||||
Store store = getStore(family);
|
||||
KeyValue kv = new KeyValue(row, HConstants.LATEST_TIMESTAMP);
|
||||
|
@ -1167,6 +1179,7 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
|
||||
protected InternalScanner getScanner(Scan scan, List<KeyValueScanner> additionalScanners) throws IOException {
|
||||
startRegionOperation();
|
||||
this.readRequestsCount.increment();
|
||||
try {
|
||||
// Verify families are all valid
|
||||
if(scan.hasFamilies()) {
|
||||
|
@ -1235,6 +1248,7 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
checkResources();
|
||||
Integer lid = null;
|
||||
startRegionOperation();
|
||||
this.writeRequestsCount.increment();
|
||||
try {
|
||||
byte [] row = delete.getRow();
|
||||
// If we did not pass an existing row lock, obtain a new one
|
||||
|
@ -1388,6 +1402,7 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
// will be extremely rare; we'll deal with it when it happens.
|
||||
checkResources();
|
||||
startRegionOperation();
|
||||
this.writeRequestsCount.increment();
|
||||
try {
|
||||
// We obtain a per-row lock, so other clients will block while one client
|
||||
// performs an update. The read lock is released by the client calling
|
||||
|
@ -1460,6 +1475,7 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
|
||||
long newSize;
|
||||
startRegionOperation();
|
||||
this.writeRequestsCount.increment();
|
||||
try {
|
||||
long addedSize = doMiniBatchPut(batchOp);
|
||||
newSize = memstoreSize.addAndGet(addedSize);
|
||||
|
@ -1661,6 +1677,7 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
}
|
||||
|
||||
startRegionOperation();
|
||||
this.writeRequestsCount.increment();
|
||||
try {
|
||||
RowLock lock = isPut ? ((Put)w).getRowLock() : ((Delete)w).getRowLock();
|
||||
Get get = new Get(row, lock);
|
||||
|
@ -2229,6 +2246,7 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
*/
|
||||
public Integer obtainRowLock(final byte [] row) throws IOException {
|
||||
startRegionOperation();
|
||||
this.writeRequestsCount.increment();
|
||||
try {
|
||||
return internalObtainRowLock(row, true);
|
||||
} finally {
|
||||
|
@ -2236,21 +2254,6 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Tries to obtain a row lock on the given row, but does not block if the
|
||||
* row lock is not available. If the lock is not available, returns false.
|
||||
* Otherwise behaves the same as the above method.
|
||||
* @see HRegion#obtainRowLock(byte[])
|
||||
*/
|
||||
public Integer tryObtainRowLock(final byte[] row) throws IOException {
|
||||
startRegionOperation();
|
||||
try {
|
||||
return internalObtainRowLock(row, false);
|
||||
} finally {
|
||||
closeRegionOperation();
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Obtains or tries to obtain the given row lock.
|
||||
* @param waitForLock if true, will block until the lock is available.
|
||||
|
@ -2362,6 +2365,7 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
public void bulkLoadHFile(String hfilePath, byte[] familyName)
|
||||
throws IOException {
|
||||
startRegionOperation();
|
||||
this.writeRequestsCount.increment();
|
||||
try {
|
||||
Store store = getStore(familyName);
|
||||
if (store == null) {
|
||||
|
@ -2467,6 +2471,7 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
"or a lengthy garbage collection");
|
||||
}
|
||||
startRegionOperation();
|
||||
readRequestsCount.increment();
|
||||
try {
|
||||
|
||||
// This could be a new thread from the last time we called next().
|
||||
|
@ -2973,7 +2978,8 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
listPaths(fs, newRegionDir);
|
||||
}
|
||||
HRegion dstRegion = HRegion.newHRegion(tableDir, log, fs, conf, newRegionInfo, null);
|
||||
dstRegion.requestsCount.set(a.requestsCount.get() + b.requestsCount.get());
|
||||
dstRegion.readRequestsCount.set(a.readRequestsCount.get() + b.readRequestsCount.get());
|
||||
dstRegion.writeRequestsCount.set(a.writeRequestsCount.get() + b.writeRequestsCount.get());
|
||||
dstRegion.initialize();
|
||||
dstRegion.compactStores();
|
||||
if (LOG.isDebugEnabled()) {
|
||||
|
@ -3219,6 +3225,7 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
|
||||
// Lock row
|
||||
startRegionOperation();
|
||||
this.writeRequestsCount.increment();
|
||||
try {
|
||||
Integer lid = getLock(lockid, row, true);
|
||||
this.updatesLock.readLock().lock();
|
||||
|
@ -3309,6 +3316,7 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
// Lock row
|
||||
long result = amount;
|
||||
startRegionOperation();
|
||||
this.writeRequestsCount.increment();
|
||||
try {
|
||||
Integer lid = obtainRowLock(row);
|
||||
this.updatesLock.readLock().lock();
|
||||
|
@ -3383,7 +3391,7 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
|
||||
public static final long FIXED_OVERHEAD = ClassSize.align(
|
||||
(4 * Bytes.SIZEOF_LONG) + ClassSize.ARRAY +
|
||||
ClassSize.align(25 * ClassSize.REFERENCE) + ClassSize.OBJECT +
|
||||
ClassSize.align(26 * ClassSize.REFERENCE) + ClassSize.OBJECT +
|
||||
ClassSize.align(Bytes.SIZEOF_INT));
|
||||
|
||||
public static final long DEEP_OVERHEAD = ClassSize.align(FIXED_OVERHEAD +
|
||||
|
@ -3563,7 +3571,6 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
} finally {
|
||||
scanner.close();
|
||||
}
|
||||
// System.out.println(region.getClosestRowBefore(Bytes.toBytes("GeneratedCSVContent2,E3652782193BC8D66A0BA1629D0FAAAB,9993372036854775807")));
|
||||
}
|
||||
} finally {
|
||||
region.close();
|
||||
|
@ -3647,7 +3654,6 @@ public class HRegion implements HeapSize { // , Writable{
|
|||
throw new NotServingRegionException(regionInfo.getRegionNameAsString() +
|
||||
" is closed");
|
||||
}
|
||||
this.requestsCount.increment();
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -59,7 +59,6 @@ import org.apache.hadoop.hbase.ClockOutOfSyncException;
|
|||
import org.apache.hadoop.hbase.DoNotRetryIOException;
|
||||
import org.apache.hadoop.hbase.HBaseConfiguration;
|
||||
import org.apache.hadoop.hbase.HConstants;
|
||||
import org.apache.hadoop.hbase.HConstants.OperationStatusCode;
|
||||
import org.apache.hadoop.hbase.HMsg;
|
||||
import org.apache.hadoop.hbase.HRegionInfo;
|
||||
import org.apache.hadoop.hbase.HServerAddress;
|
||||
|
@ -74,6 +73,7 @@ import org.apache.hadoop.hbase.Stoppable;
|
|||
import org.apache.hadoop.hbase.UnknownRowLockException;
|
||||
import org.apache.hadoop.hbase.UnknownScannerException;
|
||||
import org.apache.hadoop.hbase.YouAreDeadException;
|
||||
import org.apache.hadoop.hbase.HConstants.OperationStatusCode;
|
||||
import org.apache.hadoop.hbase.catalog.CatalogTracker;
|
||||
import org.apache.hadoop.hbase.catalog.MetaEditor;
|
||||
import org.apache.hadoop.hbase.catalog.RootLocationEditor;
|
||||
|
@ -897,7 +897,6 @@ public class HRegionServer implements HRegionInterface, HBaseRPCErrorHandler,
|
|||
int storefileSizeMB = 0;
|
||||
int memstoreSizeMB = (int) (r.memstoreSize.get() / 1024 / 1024);
|
||||
int storefileIndexSizeMB = 0;
|
||||
long requestsCount = r.requestsCount.get();
|
||||
synchronized (r.stores) {
|
||||
stores += r.stores.size();
|
||||
for (Store store : r.stores.values()) {
|
||||
|
@ -907,7 +906,8 @@ public class HRegionServer implements HRegionInterface, HBaseRPCErrorHandler,
|
|||
}
|
||||
}
|
||||
return new HServerLoad.RegionLoad(name,stores, storefiles,
|
||||
storefileSizeMB, memstoreSizeMB, storefileIndexSizeMB, requestsCount);
|
||||
storefileSizeMB, memstoreSizeMB, storefileIndexSizeMB,
|
||||
(int) r.readRequestsCount.get(), (int) r.writeRequestsCount.get());
|
||||
}
|
||||
|
||||
/**
|
||||
|
@ -1142,12 +1142,14 @@ public class HRegionServer implements HRegionInterface, HBaseRPCErrorHandler,
|
|||
int stores = 0;
|
||||
int storefiles = 0;
|
||||
long memstoreSize = 0;
|
||||
long requestsCount = 0;
|
||||
int readRequestsCount = 0;
|
||||
int writeRequestsCount = 0;
|
||||
long storefileIndexSize = 0;
|
||||
for (Map.Entry<String, HRegion> e : this.onlineRegions.entrySet()) {
|
||||
HRegion r = e.getValue();
|
||||
memstoreSize += r.memstoreSize.get();
|
||||
requestsCount += r.requestsCount.get();
|
||||
readRequestsCount += r.readRequestsCount.get();
|
||||
writeRequestsCount += r.writeRequestsCount.get();
|
||||
synchronized (r.stores) {
|
||||
stores += r.stores.size();
|
||||
for (Map.Entry<byte[], Store> ee : r.stores.entrySet()) {
|
||||
|
@ -1160,7 +1162,8 @@ public class HRegionServer implements HRegionInterface, HBaseRPCErrorHandler,
|
|||
this.metrics.stores.set(stores);
|
||||
this.metrics.storefiles.set(storefiles);
|
||||
this.metrics.memstoreSizeMB.set((int) (memstoreSize / (1024 * 1024)));
|
||||
this.metrics.requestsCount.set(requestsCount);
|
||||
this.metrics.readRequestsCount.set(readRequestsCount);
|
||||
this.metrics.writeRequestsCount.set(writeRequestsCount);
|
||||
this.metrics.storefileIndexSizeMB
|
||||
.set((int) (storefileIndexSize / (1024 * 1024)));
|
||||
this.metrics.compactionQueueSize.set(compactSplitThread
|
||||
|
|
|
@ -543,7 +543,8 @@ public class SplitTransaction {
|
|||
HRegion r = HRegion.newHRegion(this.parent.getTableDir(),
|
||||
this.parent.getLog(), fs, this.parent.getConf(),
|
||||
hri, rsServices);
|
||||
r.requestsCount.set(this.parent.getRequestsCount() / 2);
|
||||
r.readRequestsCount.set(this.parent.getReadRequestsCount() / 2);
|
||||
r.writeRequestsCount.set(this.parent.getWriteRequestsCount() / 2);
|
||||
HRegion.moveInitialFilesIntoPlace(fs, regionDir, r.getRegionDir());
|
||||
return r;
|
||||
}
|
||||
|
|
|
@ -128,9 +128,14 @@ public class RegionServerMetrics implements Updater {
|
|||
public final MetricsIntValue storefiles = new MetricsIntValue("storefiles", registry);
|
||||
|
||||
/**
|
||||
* Count of requests
|
||||
* Count of read requests
|
||||
*/
|
||||
public final MetricsLongValue requestsCount = new MetricsLongValue("requestsCount", registry);
|
||||
public final MetricsLongValue readRequestsCount = new MetricsLongValue("readRequestsCount", registry);
|
||||
|
||||
/**
|
||||
* Count of write requests
|
||||
*/
|
||||
public final MetricsLongValue writeRequestsCount = new MetricsLongValue("writeRequestsCount", registry);
|
||||
|
||||
/**
|
||||
* Sum of all the storefile index sizes in this regionserver in MB
|
||||
|
@ -248,7 +253,8 @@ public class RegionServerMetrics implements Updater {
|
|||
this.storefiles.pushMetric(this.metricsRecord);
|
||||
this.storefileIndexSizeMB.pushMetric(this.metricsRecord);
|
||||
this.memstoreSizeMB.pushMetric(this.metricsRecord);
|
||||
this.requestsCount.pushMetric(this.metricsRecord);
|
||||
this.readRequestsCount.pushMetric(this.metricsRecord);
|
||||
this.writeRequestsCount.pushMetric(this.metricsRecord);
|
||||
this.regions.pushMetric(this.metricsRecord);
|
||||
this.requests.pushMetric(this.metricsRecord);
|
||||
this.compactionQueueSize.pushMetric(this.metricsRecord);
|
||||
|
@ -351,8 +357,10 @@ public class RegionServerMetrics implements Updater {
|
|||
Integer.valueOf(this.storefileIndexSizeMB.get()));
|
||||
sb = Strings.appendKeyValue(sb, "memstoreSize",
|
||||
Integer.valueOf(this.memstoreSizeMB.get()));
|
||||
sb = Strings.appendKeyValue(sb, "requestsCount",
|
||||
Long.valueOf(this.requestsCount.get()));
|
||||
sb = Strings.appendKeyValue(sb, "readRequestsCount",
|
||||
Long.valueOf(this.readRequestsCount.get()));
|
||||
sb = Strings.appendKeyValue(sb, "writeRequestsCount",
|
||||
Long.valueOf(this.writeRequestsCount.get()));
|
||||
sb = Strings.appendKeyValue(sb, "compactionQueueSize",
|
||||
Integer.valueOf(this.compactionQueueSize.get()));
|
||||
sb = Strings.appendKeyValue(sb, "flushQueueSize",
|
||||
|
|
Loading…
Reference in New Issue