HBASE-7612 [JDK8] Replace use of high-scale-lib counters with intrinsic facilities

This commit is contained in:
zhangduo 2016-09-18 15:34:37 +08:00
parent 9c58d26d3b
commit 6eb6225456
20 changed files with 234 additions and 247 deletions

View File

@ -44,8 +44,8 @@ public class SpanReceiverHost {
private static enum SingletonHolder {
INSTANCE;
Object lock = new Object();
SpanReceiverHost host = null;
transient Object lock = new Object();
transient SpanReceiverHost host = null;
}
public static SpanReceiverHost getInstance(Configuration conf) {

View File

@ -26,9 +26,11 @@ import org.apache.hadoop.hbase.classification.InterfaceStability;
/**
* High scalable counter. Thread safe.
* @deprecated use {@link java.util.concurrent.atomic.LongAdder} instead.
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
@Deprecated
public class Counter {
private static final int MAX_CELLS_LENGTH = 1 << 20;

View File

@ -17,8 +17,10 @@
*/
package org.apache.hadoop.hbase.util;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.Arrays;
import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.atomic.LongAdder;
import java.util.stream.Stream;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.classification.InterfaceStability;
@ -43,7 +45,7 @@ public class FastLongHistogram {
* Bins is a class containing a list of buckets(or bins) for estimation histogram of some data.
*/
private static class Bins {
private final Counter[] counts;
private final LongAdder[] counts;
// inclusive
private final long binsMin;
// exclusive
@ -52,18 +54,18 @@ public class FastLongHistogram {
private final AtomicLong min = new AtomicLong(Long.MAX_VALUE);
private final AtomicLong max = new AtomicLong(0L);
private final Counter count = new Counter(0);
private final Counter total = new Counter(0);
private final LongAdder count = new LongAdder();
private final LongAdder total = new LongAdder();
// set to true when any of data has been inserted to the Bins. It is set after the counts are
// updated.
private final AtomicBoolean hasData = new AtomicBoolean(false);
private volatile boolean hasData = false;
/**
* The constructor for creating a Bins without any prior data.
*/
public Bins(int numBins) {
counts = createCounters(numBins + 3);
counts = createCounters(numBins);
this.binsMin = 1L;
// These two numbers are total guesses
@ -75,25 +77,21 @@ public class FastLongHistogram {
/**
* The constructor for creating a Bins with last Bins.
*/
public Bins(Bins last, int numOfBins, double minQ, double maxQ) {
public Bins(Bins last, int numBins, double minQ, double maxQ) {
long[] values = last.getQuantiles(new double[] { minQ, maxQ });
long wd = values[1] - values[0] + 1;
// expand minQ and maxQ in two ends back assuming uniform distribution
this.binsMin = Math.max(0L, (long) (values[0] - wd * minQ));
long binsMax = (long) (values[1] + wd * (1 - maxQ)) + 1;
// make sure each of bins is at least of width 1
this.binsMax = Math.max(binsMax, this.binsMin + numOfBins);
this.binsMax = Math.max(binsMax, this.binsMin + numBins);
this.bins10XMax = Math.max((long) (values[1] + (binsMax - 1) * 9), this.binsMax + 1);
this.counts = createCounters(numOfBins + 3);
this.counts = createCounters(numBins);
}
private Counter[] createCounters(int num) {
Counter[] counters = new Counter[num];
for (int i = 0; i < num; i++) {
counters[i] = new Counter();
}
return counters;
private LongAdder[] createCounters(int numBins) {
return Stream.generate(LongAdder::new).limit(numBins + 3).toArray(LongAdder[]::new);
}
private int getIndex(long value) {
@ -132,14 +130,14 @@ public class FastLongHistogram {
this.counts[pos].add(count);
// hasData needs to be updated as last
this.hasData.set(true);
this.hasData = true;
}
/**
* Computes the quantiles give the ratios.
*/
public long[] getQuantiles(double[] quantiles) {
if (!this.hasData.get()) {
if (!hasData) {
// No data yet.
return new long[quantiles.length];
}
@ -150,7 +148,7 @@ public class FastLongHistogram {
long[] counts = new long[this.counts.length];
long total = 0L;
for (int i = 0; i < this.counts.length; i++) {
counts[i] = this.counts[i].get();
counts[i] = this.counts[i].sum();
total += counts[i];
}
@ -213,14 +211,8 @@ public class FastLongHistogram {
return res;
}
long getNumAtOrBelow(long val) {
final int targetIndex = getIndex(val);
long totalToCurrentIndex = 0;
for (int i = 0; i <= targetIndex; i++) {
totalToCurrentIndex += this.counts[i].get();
}
return totalToCurrentIndex;
return Arrays.stream(counts).mapToLong(c -> c.sum()).limit(getIndex(val) + 1).sum();
}
}
@ -290,13 +282,13 @@ public class FastLongHistogram {
}
public long getCount() {
return this.bins.count.get();
return this.bins.count.sum();
}
public long getMean() {
Bins bins = this.bins;
long count = bins.count.get();
long total = bins.total.get();
long count = bins.count.sum();
long total = bins.total.sum();
if (count == 0) {
return 0;
}

View File

@ -48,7 +48,7 @@ public class MetricsExecutorImpl implements MetricsExecutor {
private enum ExecutorSingleton {
INSTANCE;
private final ScheduledExecutorService scheduler = new ScheduledThreadPoolExecutor(1,
private final transient ScheduledExecutorService scheduler = new ScheduledThreadPoolExecutor(1,
new ThreadPoolExecutorThreadFactory("HBase-Metrics2-"));
}

View File

@ -18,17 +18,19 @@
package org.apache.hadoop.metrics2.lib;
import org.apache.hadoop.hbase.util.Counter;
import java.util.concurrent.atomic.LongAdder;
import org.apache.hadoop.metrics2.MetricsInfo;
import org.apache.hadoop.metrics2.MetricsRecordBuilder;
public class MutableFastCounter extends MutableCounter {
private final Counter counter;
private final LongAdder counter;
protected MutableFastCounter(MetricsInfo info, long iVal) {
super(info);
counter = new Counter(iVal);
counter = new LongAdder();
counter.add(iVal);
}
@Override
@ -55,6 +57,6 @@ public class MutableFastCounter extends MutableCounter {
}
public long value() {
return counter.get();
return counter.sum();
}
}

View File

@ -18,10 +18,11 @@
package org.apache.hadoop.metrics2.lib;
import java.util.concurrent.atomic.LongAdder;
import org.apache.commons.lang.StringUtils;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.metrics.Interns;
import org.apache.hadoop.hbase.util.Counter;
import org.apache.hadoop.hbase.util.FastLongHistogram;
import org.apache.hadoop.metrics2.MetricHistogram;
import org.apache.hadoop.metrics2.MetricsInfo;
@ -39,7 +40,7 @@ public class MutableHistogram extends MutableMetric implements MetricHistogram {
protected final String name;
protected final String desc;
protected final Counter counter = new Counter(0);
protected final LongAdder counter = new LongAdder();
private boolean metricsInfoStringInited = false;
private String NUM_OPS_METRIC;
@ -103,7 +104,7 @@ public class MutableHistogram extends MutableMetric implements MetricHistogram {
metricsInfoStringInited = true;
}
metricsRecordBuilder.addCounter(Interns.info(NUM_OPS_METRIC, desc), counter.get());
metricsRecordBuilder.addCounter(Interns.info(NUM_OPS_METRIC, desc), counter.sum());
metricsRecordBuilder.addGauge(Interns.info(MIN_METRIC, desc), histo.getMin());
metricsRecordBuilder.addGauge(Interns.info(MAX_METRIC, desc), histo.getMax());
metricsRecordBuilder.addGauge(Interns.info(MEAN_METRIC, desc), histo.getMean());

View File

@ -18,11 +18,11 @@
*/
package org.apache.hadoop.hbase.io.hfile;
import java.util.Arrays;
import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.atomic.LongAdder;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.util.Counter;
import org.apache.hadoop.hbase.util.FastLongHistogram;
@ -38,64 +38,64 @@ public class CacheStats {
static final int DEFAULT_WINDOW_PERIODS = 5;
/** The number of getBlock requests that were cache hits */
private final Counter hitCount = new Counter();
private final LongAdder hitCount = new LongAdder();
/** The number of getBlock requests that were cache hits from primary replica */
private final Counter primaryHitCount = new Counter();
private final LongAdder primaryHitCount = new LongAdder();
/**
* The number of getBlock requests that were cache hits, but only from
* requests that were set to use the block cache. This is because all reads
* attempt to read from the block cache even if they will not put new blocks
* into the block cache. See HBASE-2253 for more information.
*/
private final Counter hitCachingCount = new Counter();
private final LongAdder hitCachingCount = new LongAdder();
/** The number of getBlock requests that were cache misses */
private final Counter missCount = new Counter();
private final LongAdder missCount = new LongAdder();
/** The number of getBlock requests for primary replica that were cache misses */
private final Counter primaryMissCount = new Counter();
private final LongAdder primaryMissCount = new LongAdder();
/**
* The number of getBlock requests that were cache misses, but only from
* requests that were set to use the block cache.
*/
private final Counter missCachingCount = new Counter();
private final LongAdder missCachingCount = new LongAdder();
/** The number of times an eviction has occurred */
private final Counter evictionCount = new Counter();
private final LongAdder evictionCount = new LongAdder();
/** The total number of blocks that have been evicted */
private final Counter evictedBlockCount = new Counter();
private final LongAdder evictedBlockCount = new LongAdder();
/** The total number of blocks for primary replica that have been evicted */
private final Counter primaryEvictedBlockCount = new Counter();
private final LongAdder primaryEvictedBlockCount = new LongAdder();
/** The total number of blocks that were not inserted. */
private final AtomicLong failedInserts = new AtomicLong(0);
/** Per Block Type Counts */
private final Counter dataMissCount = new Counter(0);
private final Counter leafIndexMissCount = new Counter(0);
private final Counter bloomChunkMissCount = new Counter(0);
private final Counter metaMissCount = new Counter(0);
private final Counter rootIndexMissCount = new Counter(0);
private final Counter intermediateIndexMissCount = new Counter(0);
private final Counter fileInfoMissCount = new Counter(0);
private final Counter generalBloomMetaMissCount = new Counter(0);
private final Counter deleteFamilyBloomMissCount = new Counter(0);
private final Counter trailerMissCount = new Counter(0);
private final LongAdder dataMissCount = new LongAdder();
private final LongAdder leafIndexMissCount = new LongAdder();
private final LongAdder bloomChunkMissCount = new LongAdder();
private final LongAdder metaMissCount = new LongAdder();
private final LongAdder rootIndexMissCount = new LongAdder();
private final LongAdder intermediateIndexMissCount = new LongAdder();
private final LongAdder fileInfoMissCount = new LongAdder();
private final LongAdder generalBloomMetaMissCount = new LongAdder();
private final LongAdder deleteFamilyBloomMissCount = new LongAdder();
private final LongAdder trailerMissCount = new LongAdder();
private final Counter dataHitCount = new Counter(0);
private final Counter leafIndexHitCount = new Counter(0);
private final Counter bloomChunkHitCount = new Counter(0);
private final Counter metaHitCount = new Counter(0);
private final Counter rootIndexHitCount = new Counter(0);
private final Counter intermediateIndexHitCount = new Counter(0);
private final Counter fileInfoHitCount = new Counter(0);
private final Counter generalBloomMetaHitCount = new Counter(0);
private final Counter deleteFamilyBloomHitCount = new Counter(0);
private final Counter trailerHitCount = new Counter(0);
private final LongAdder dataHitCount = new LongAdder();
private final LongAdder leafIndexHitCount = new LongAdder();
private final LongAdder bloomChunkHitCount = new LongAdder();
private final LongAdder metaHitCount = new LongAdder();
private final LongAdder rootIndexHitCount = new LongAdder();
private final LongAdder intermediateIndexHitCount = new LongAdder();
private final LongAdder fileInfoHitCount = new LongAdder();
private final LongAdder generalBloomMetaHitCount = new LongAdder();
private final LongAdder deleteFamilyBloomHitCount = new LongAdder();
private final LongAdder trailerHitCount = new LongAdder();
/** The number of metrics periods to include in window */
private final int numPeriodsInWindow;
@ -129,10 +129,10 @@ public class CacheStats {
public CacheStats(final String name, int numPeriodsInWindow) {
this.numPeriodsInWindow = numPeriodsInWindow;
this.hitCounts = initializeZeros(numPeriodsInWindow);
this.hitCachingCounts = initializeZeros(numPeriodsInWindow);
this.requestCounts = initializeZeros(numPeriodsInWindow);
this.requestCachingCounts = initializeZeros(numPeriodsInWindow);
this.hitCounts = new long[numPeriodsInWindow];
this.hitCachingCounts = new long[numPeriodsInWindow];
this.requestCounts = new long[numPeriodsInWindow];
this.requestCachingCounts = new long[numPeriodsInWindow];
this.ageAtEviction = new FastLongHistogram();
}
@ -264,83 +264,83 @@ public class CacheStats {
// All of the counts of misses and hits.
public long getDataMissCount() {
return dataMissCount.get();
return dataMissCount.sum();
}
public long getLeafIndexMissCount() {
return leafIndexMissCount.get();
return leafIndexMissCount.sum();
}
public long getBloomChunkMissCount() {
return bloomChunkMissCount.get();
return bloomChunkMissCount.sum();
}
public long getMetaMissCount() {
return metaMissCount.get();
return metaMissCount.sum();
}
public long getRootIndexMissCount() {
return rootIndexMissCount.get();
return rootIndexMissCount.sum();
}
public long getIntermediateIndexMissCount() {
return intermediateIndexMissCount.get();
return intermediateIndexMissCount.sum();
}
public long getFileInfoMissCount() {
return fileInfoMissCount.get();
return fileInfoMissCount.sum();
}
public long getGeneralBloomMetaMissCount() {
return generalBloomMetaMissCount.get();
return generalBloomMetaMissCount.sum();
}
public long getDeleteFamilyBloomMissCount() {
return deleteFamilyBloomMissCount.get();
return deleteFamilyBloomMissCount.sum();
}
public long getTrailerMissCount() {
return trailerMissCount.get();
return trailerMissCount.sum();
}
public long getDataHitCount() {
return dataHitCount.get();
return dataHitCount.sum();
}
public long getLeafIndexHitCount() {
return leafIndexHitCount.get();
return leafIndexHitCount.sum();
}
public long getBloomChunkHitCount() {
return bloomChunkHitCount.get();
return bloomChunkHitCount.sum();
}
public long getMetaHitCount() {
return metaHitCount.get();
return metaHitCount.sum();
}
public long getRootIndexHitCount() {
return rootIndexHitCount.get();
return rootIndexHitCount.sum();
}
public long getIntermediateIndexHitCount() {
return intermediateIndexHitCount.get();
return intermediateIndexHitCount.sum();
}
public long getFileInfoHitCount() {
return fileInfoHitCount.get();
return fileInfoHitCount.sum();
}
public long getGeneralBloomMetaHitCount() {
return generalBloomMetaHitCount.get();
return generalBloomMetaHitCount.sum();
}
public long getDeleteFamilyBloomHitCount() {
return deleteFamilyBloomHitCount.get();
return deleteFamilyBloomHitCount.sum();
}
public long getTrailerHitCount() {
return trailerHitCount.get();
return trailerHitCount.sum();
}
public long getRequestCount() {
@ -352,59 +352,59 @@ public class CacheStats {
}
public long getMissCount() {
return missCount.get();
return missCount.sum();
}
public long getPrimaryMissCount() {
return primaryMissCount.get();
return primaryMissCount.sum();
}
public long getMissCachingCount() {
return missCachingCount.get();
return missCachingCount.sum();
}
public long getHitCount() {
return hitCount.get();
return hitCount.sum();
}
public long getPrimaryHitCount() {
return primaryHitCount.get();
return primaryHitCount.sum();
}
public long getHitCachingCount() {
return hitCachingCount.get();
return hitCachingCount.sum();
}
public long getEvictionCount() {
return evictionCount.get();
return evictionCount.sum();
}
public long getEvictedCount() {
return this.evictedBlockCount.get();
return this.evictedBlockCount.sum();
}
public long getPrimaryEvictedCount() {
return primaryEvictedBlockCount.get();
return primaryEvictedBlockCount.sum();
}
public double getHitRatio() {
return ((float)getHitCount()/(float)getRequestCount());
return ((double) getHitCount() / (double) getRequestCount());
}
public double getHitCachingRatio() {
return ((float)getHitCachingCount()/(float)getRequestCachingCount());
return ((double) getHitCachingCount() / (double) getRequestCachingCount());
}
public double getMissRatio() {
return ((float)getMissCount()/(float)getRequestCount());
return ((double) getMissCount() / (double) getRequestCount());
}
public double getMissCachingRatio() {
return ((float)getMissCachingCount()/(float)getRequestCachingCount());
return ((double) getMissCachingCount() / (double) getRequestCachingCount());
}
public double evictedPerEviction() {
return ((float)getEvictedCount()/(float)getEvictionCount());
return ((double) getEvictedCount() / (double) getEvictionCount());
}
public long getFailedInserts() {
@ -457,17 +457,7 @@ public class CacheStats {
return new AgeSnapshot(this.ageAtEviction);
}
private static long sum(long [] counts) {
long sum = 0;
for (long count : counts) sum += count;
return sum;
}
private static long [] initializeZeros(int n) {
long [] zeros = new long [n];
for (int i=0; i<n; i++) {
zeros[i] = 0L;
}
return zeros;
private static long sum(long[] counts) {
return Arrays.stream(counts).sum();
}
}

View File

@ -18,6 +18,9 @@
*/
package org.apache.hadoop.hbase.io.hfile;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
import java.io.ByteArrayInputStream;
import java.io.Closeable;
import java.io.DataInput;
@ -34,11 +37,10 @@ import java.util.Map;
import java.util.Set;
import java.util.SortedMap;
import java.util.TreeMap;
import java.util.concurrent.atomic.LongAdder;
import org.apache.hadoop.hbase.util.ByteStringer;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
@ -49,6 +51,7 @@ import org.apache.hadoop.fs.PathFilter;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellComparator;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.fs.HFileSystem;
import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper;
import org.apache.hadoop.hbase.io.compress.Compression;
@ -59,14 +62,11 @@ import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos;
import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair;
import org.apache.hadoop.hbase.protobuf.generated.HFileProtos;
import org.apache.hadoop.hbase.util.BloomFilterWriter;
import org.apache.hadoop.hbase.util.ByteStringer;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Counter;
import org.apache.hadoop.hbase.util.FSUtils;
import org.apache.hadoop.io.Writable;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
/**
* File format for hbase.
* A file of sorted key/value pairs. Both keys and values are byte arrays.
@ -180,19 +180,17 @@ public class HFile {
public static final int DEFAULT_BYTES_PER_CHECKSUM = 16 * 1024;
// For measuring number of checksum failures
static final Counter CHECKSUM_FAILURES = new Counter();
static final LongAdder CHECKSUM_FAILURES = new LongAdder();
// For tests. Gets incremented when we read a block whether from HDFS or from Cache.
public static final Counter DATABLOCK_READ_COUNT = new Counter();
public static final LongAdder DATABLOCK_READ_COUNT = new LongAdder();
/**
* Number of checksum verification failures. It also
* clears the counter.
*/
public static final long getChecksumFailuresCount() {
long count = CHECKSUM_FAILURES.get();
CHECKSUM_FAILURES.set(0);
return count;
return CHECKSUM_FAILURES.sumThenReset();
}
/** API required to write an {@link HFile} */

View File

@ -18,9 +18,11 @@
*/
package org.apache.hadoop.hbase.io.hfile.bucket;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.LongAdder;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.io.hfile.CacheStats;
import org.apache.hadoop.hbase.util.Counter;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
/**
@ -28,9 +30,9 @@ import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
*/
@InterfaceAudience.Private
public class BucketCacheStats extends CacheStats {
private final Counter ioHitCount = new Counter(0);
private final Counter ioHitTime = new Counter(0);
private final static int nanoTime = 1000000;
private final LongAdder ioHitCount = new LongAdder();
private final LongAdder ioHitTime = new LongAdder();
private static final long NANO_TIME = TimeUnit.MILLISECONDS.toNanos(1);
private long lastLogTime = EnvironmentEdgeManager.currentTime();
BucketCacheStats() {
@ -52,17 +54,17 @@ public class BucketCacheStats extends CacheStats {
long now = EnvironmentEdgeManager.currentTime();
long took = (now - lastLogTime) / 1000;
lastLogTime = now;
return took == 0? 0: ioHitCount.get() / took;
return took == 0 ? 0 : ioHitCount.sum() / took;
}
public double getIOTimePerHit() {
long time = ioHitTime.get() / nanoTime;
long count = ioHitCount.get();
long time = ioHitTime.sum() / NANO_TIME;
long count = ioHitCount.sum();
return ((float) time / (float) count);
}
public void reset() {
ioHitCount.set(0);
ioHitTime.set(0);
ioHitCount.reset();
ioHitTime.reset();
}
}

View File

@ -36,7 +36,7 @@ public class MetricsHBaseServerWrapperImpl implements MetricsHBaseServerWrapper
if (!isServerStarted()) {
return 0;
}
return server.callQueueSizeInBytes.get();
return server.callQueueSizeInBytes.sum();
}
@Override

View File

@ -20,6 +20,15 @@ package org.apache.hadoop.hbase.ipc;
import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHORIZATION;
import com.google.common.util.concurrent.ThreadFactoryBuilder;
import com.google.protobuf.BlockingService;
import com.google.protobuf.CodedInputStream;
import com.google.protobuf.CodedOutputStream;
import com.google.protobuf.Descriptors.MethodDescriptor;
import com.google.protobuf.Message;
import com.google.protobuf.ServiceException;
import com.google.protobuf.TextFormat;
import java.io.ByteArrayInputStream;
import java.io.ByteArrayOutputStream;
import java.io.DataOutputStream;
@ -59,6 +68,7 @@ import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.LinkedBlockingQueue;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.LongAdder;
import java.util.concurrent.locks.Lock;
import java.util.concurrent.locks.ReentrantLock;
@ -108,7 +118,6 @@ import org.apache.hadoop.hbase.security.User;
import org.apache.hadoop.hbase.security.UserProvider;
import org.apache.hadoop.hbase.security.token.AuthenticationTokenSecretManager;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Counter;
import org.apache.hadoop.hbase.util.Pair;
import org.apache.hadoop.hbase.util.Threads;
import org.apache.hadoop.io.BytesWritable;
@ -130,15 +139,6 @@ import org.apache.hadoop.util.StringUtils;
import org.apache.htrace.TraceInfo;
import org.codehaus.jackson.map.ObjectMapper;
import com.google.common.util.concurrent.ThreadFactoryBuilder;
import com.google.protobuf.BlockingService;
import com.google.protobuf.CodedInputStream;
import com.google.protobuf.CodedOutputStream;
import com.google.protobuf.Descriptors.MethodDescriptor;
import com.google.protobuf.Message;
import com.google.protobuf.ServiceException;
import com.google.protobuf.TextFormat;
/**
* An RPC server that hosts protobuf described Services.
*
@ -227,7 +227,7 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
* This is a running count of the size in bytes of all outstanding calls whether currently
* executing or queued waiting to be run.
*/
protected final Counter callQueueSizeInBytes = new Counter();
protected final LongAdder callQueueSizeInBytes = new LongAdder();
protected int socketSendBufferSize;
protected final boolean tcpNoDelay; // if T then disable Nagle's Algorithm
@ -1204,7 +1204,7 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
private ByteBuffer dataLengthBuffer;
protected final ConcurrentLinkedDeque<Call> responseQueue = new ConcurrentLinkedDeque<Call>();
private final Lock responseWriteLock = new ReentrantLock();
private Counter rpcCount = new Counter(); // number of outstanding rpcs
private LongAdder rpcCount = new LongAdder(); // number of outstanding rpcs
private long lastContact;
private InetAddress addr;
protected Socket socket;
@ -1310,7 +1310,7 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
/* Return true if the connection has no outstanding rpc */
private boolean isIdle() {
return rpcCount.get() == 0;
return rpcCount.sum() == 0;
}
/* Decrement the outstanding RPC count */
@ -1832,7 +1832,7 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
}
// Enforcing the call queue size, this triggers a retry in the client
// This is a bit late to be doing this check - we have already read in the total request.
if ((totalRequestSize + callQueueSizeInBytes.get()) > maxQueueSizeInBytes) {
if ((totalRequestSize + callQueueSizeInBytes.sum()) > maxQueueSizeInBytes) {
final Call callTooBig =
new Call(id, this.service, null, null, null, null, this,
responder, totalRequestSize, null, null, 0);
@ -1959,7 +1959,6 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
LOG.trace("Ignored exception", ignored);
}
}
rpcCount.destroy();
}
private UserGroupInformation createUser(ConnectionHeader head) {
@ -2685,7 +2684,7 @@ public class RpcServer implements RpcServerInterface, ConfigurationObserver {
if (LOG.isDebugEnabled()) {
LOG.debug("Server connection from " + connection +
"; connections=" + size() +
", queued calls size (bytes)=" + callQueueSizeInBytes.get() +
", queued calls size (bytes)=" + callQueueSizeInBytes.sum() +
", general queued calls=" + scheduler.getGeneralQueueLength() +
", priority queued calls=" + scheduler.getPriorityQueueLength());
}

View File

@ -32,6 +32,7 @@ import com.google.protobuf.RpcCallback;
import com.google.protobuf.RpcController;
import com.google.protobuf.Service;
import com.google.protobuf.TextFormat;
import java.io.EOFException;
import java.io.FileNotFoundException;
import java.io.IOException;
@ -74,6 +75,7 @@ import java.util.concurrent.TimeoutException;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicInteger;
import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.atomic.LongAdder;
import java.util.concurrent.locks.Lock;
import java.util.concurrent.locks.ReadWriteLock;
import java.util.concurrent.locks.ReentrantReadWriteLock;
@ -176,7 +178,6 @@ import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.CancelableProgressable;
import org.apache.hadoop.hbase.util.ClassSize;
import org.apache.hadoop.hbase.util.CompressionTest;
import org.apache.hadoop.hbase.util.Counter;
import org.apache.hadoop.hbase.util.EncryptionTest;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.FSUtils;
@ -264,22 +265,22 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
private final RegionServicesForStores regionServicesForStores = new RegionServicesForStores(this);
// Debug possible data loss due to WAL off
final Counter numMutationsWithoutWAL = new Counter();
final Counter dataInMemoryWithoutWAL = new Counter();
final LongAdder numMutationsWithoutWAL = new LongAdder();
final LongAdder dataInMemoryWithoutWAL = new LongAdder();
// Debug why CAS operations are taking a while.
final Counter checkAndMutateChecksPassed = new Counter();
final Counter checkAndMutateChecksFailed = new Counter();
final LongAdder checkAndMutateChecksPassed = new LongAdder();
final LongAdder checkAndMutateChecksFailed = new LongAdder();
// Number of requests
final Counter readRequestsCount = new Counter();
final Counter filteredReadRequestsCount = new Counter();
final Counter writeRequestsCount = new Counter();
final LongAdder readRequestsCount = new LongAdder();
final LongAdder filteredReadRequestsCount = new LongAdder();
final LongAdder writeRequestsCount = new LongAdder();
// Number of requests blocked by memstore size.
private final Counter blockedRequestsCount = new Counter();
private final LongAdder blockedRequestsCount = new LongAdder();
// Compaction counters
// Compaction LongAdders
final AtomicLong compactionsFinished = new AtomicLong(0L);
final AtomicLong compactionsFailed = new AtomicLong(0L);
final AtomicLong compactionNumFilesCompacted = new AtomicLong(0L);
@ -318,7 +319,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
private final ConcurrentHashMap<RegionScanner, Long> scannerReadPoints;
/**
* The sequence ID that was encountered when this region was opened.
* The sequence ID that was enLongAddered when this region was opened.
*/
private long openSeqNum = HConstants.NO_SEQNUM;
@ -1146,7 +1147,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
@Override
public long getReadRequestsCount() {
return readRequestsCount.get();
return readRequestsCount.sum();
}
@Override
@ -1156,12 +1157,12 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
@Override
public long getFilteredReadRequestsCount() {
return filteredReadRequestsCount.get();
return filteredReadRequestsCount.sum();
}
@Override
public long getWriteRequestsCount() {
return writeRequestsCount.get();
return writeRequestsCount.sum();
}
@Override
@ -1181,27 +1182,27 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
@Override
public long getNumMutationsWithoutWAL() {
return numMutationsWithoutWAL.get();
return numMutationsWithoutWAL.sum();
}
@Override
public long getDataInMemoryWithoutWAL() {
return dataInMemoryWithoutWAL.get();
return dataInMemoryWithoutWAL.sum();
}
@Override
public long getBlockedRequestsCount() {
return blockedRequestsCount.get();
return blockedRequestsCount.sum();
}
@Override
public long getCheckAndMutateChecksPassed() {
return checkAndMutateChecksPassed.get();
return checkAndMutateChecksPassed.sum();
}
@Override
public long getCheckAndMutateChecksFailed() {
return checkAndMutateChecksFailed.get();
return checkAndMutateChecksFailed.sum();
}
@Override
@ -2055,9 +2056,9 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
}
// TODO: this should be managed within memstore with the snapshot, updated only after flush
// successful
if (numMutationsWithoutWAL.get() > 0) {
numMutationsWithoutWAL.set(0);
dataInMemoryWithoutWAL.set(0);
if (numMutationsWithoutWAL.sum() > 0) {
numMutationsWithoutWAL.reset();
dataInMemoryWithoutWAL.reset();
}
synchronized (writestate) {
if (!writestate.flushing && writestate.writesEnabled) {
@ -4223,7 +4224,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
}
} catch (EOFException eof) {
Path p = WALSplitter.moveAsideBadEditsFile(fs, edits);
msg = "Encountered EOF. Most likely due to Master failure during " +
msg = "EnLongAddered EOF. Most likely due to Master failure during " +
"wal splitting, so we have this data in another edit. " +
"Continuing, but renaming " + edits + " as " + p;
LOG.warn(msg, eof);
@ -4233,7 +4234,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
// then this problem is idempotent and retrying won't help
if (ioe.getCause() instanceof ParseException) {
Path p = WALSplitter.moveAsideBadEditsFile(fs, edits);
msg = "File corruption encountered! " +
msg = "File corruption enLongAddered! " +
"Continuing, but renaming " + edits + " as " + p;
LOG.warn(msg, ioe);
status.setStatus(msg);
@ -4403,9 +4404,9 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
+ " of " + lastReplayedOpenRegionSeqId);
return null;
}
if (numMutationsWithoutWAL.get() > 0) {
numMutationsWithoutWAL.set(0);
dataInMemoryWithoutWAL.set(0);
if (numMutationsWithoutWAL.sum() > 0) {
numMutationsWithoutWAL.reset();
dataInMemoryWithoutWAL.reset();
}
if (!writestate.flushing) {
@ -5961,7 +5962,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
boolean stopRow = isStopRow(current);
// When has filter row is true it means that the all the cells for a particular row must be
// read before a filtering decision can be made. This means that filters where hasFilterRow
// run the risk of encountering out of memory errors in the case that they are applied to a
// run the risk of enLongAddering out of memory errors in the case that they are applied to a
// table that has very large rows.
boolean hasFilterRow = this.filter != null && this.filter.hasFilterRow();
@ -6616,9 +6617,9 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
// Create the daughter HRegion instance
HRegion r = HRegion.newHRegion(this.fs.getTableDir(), this.getWAL(), fs.getFileSystem(),
this.getBaseConf(), hri, this.getTableDesc(), rsServices);
r.readRequestsCount.set(this.getReadRequestsCount() / 2);
r.filteredReadRequestsCount.set(this.getFilteredReadRequestsCount() / 2);
r.writeRequestsCount.set(this.getWriteRequestsCount() / 2);
r.readRequestsCount.add(this.getReadRequestsCount() / 2);
r.filteredReadRequestsCount.add(this.getFilteredReadRequestsCount() / 2);
r.writeRequestsCount.add(this.getWriteRequestsCount() / 2);
return r;
}
@ -6633,11 +6634,11 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
HRegion r = HRegion.newHRegion(this.fs.getTableDir(), this.getWAL(),
fs.getFileSystem(), this.getBaseConf(), mergedRegionInfo,
this.getTableDesc(), this.rsServices);
r.readRequestsCount.set(this.getReadRequestsCount()
r.readRequestsCount.add(this.getReadRequestsCount()
+ region_b.getReadRequestsCount());
r.filteredReadRequestsCount.set(this.getFilteredReadRequestsCount()
r.filteredReadRequestsCount.add(this.getFilteredReadRequestsCount()
+ region_b.getFilteredReadRequestsCount());
r.writeRequestsCount.set(this.getWriteRequestsCount()
r.writeRequestsCount.add(this.getWriteRequestsCount()
+ region_b.getWriteRequestsCount());
this.fs.commitMergedRegion(mergedRegionInfo);
@ -7225,7 +7226,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
if (walEdit != null && !walEdit.isEmpty()) {
writeEntry = doWALAppend(walEdit, durability, nonceGroup, nonce);
} else {
// If walEdits is empty, it means we skipped the WAL; update counters and start an mvcc
// If walEdits is empty, it means we skipped the WAL; update LongAdders and start an mvcc
// transaction.
recordMutationWithoutWal(mutation.getFamilyCellMap());
writeEntry = mvcc.begin();
@ -7594,7 +7595,7 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
// woefully out of date - currently missing:
// 1 x HashMap - coprocessorServiceHandlers
// 6 x Counter - numMutationsWithoutWAL, dataInMemoryWithoutWAL,
// 6 x LongAdder - numMutationsWithoutWAL, dataInMemoryWithoutWAL,
// checkAndMutateChecksPassed, checkAndMutateChecksFailed, readRequestsCount,
// writeRequestsCount
// 1 x HRegion$WriteState - writestate
@ -7897,12 +7898,12 @@ public class HRegion implements HeapSize, PropagatingConfigurationObserver, Regi
}
/**
* Update counters for number of puts without wal and the size of possible data loss.
* Update LongAdders for number of puts without wal and the size of possible data loss.
* These information are exposed by the region server metrics.
*/
private void recordMutationWithoutWal(final Map<byte [], List<Cell>> familyMap) {
numMutationsWithoutWAL.increment();
if (numMutationsWithoutWAL.get() <= 1) {
if (numMutationsWithoutWAL.sum() <= 1) {
LOG.info("writing data to region " + this +
" with WAL disabled. Data may be lost in the event of a crash.");
}

View File

@ -18,6 +18,17 @@
*/
package org.apache.hadoop.hbase.regionserver;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
import com.google.common.collect.Maps;
import com.google.protobuf.BlockingRpcChannel;
import com.google.protobuf.Descriptors;
import com.google.protobuf.Message;
import com.google.protobuf.RpcCallback;
import com.google.protobuf.RpcController;
import com.google.protobuf.Service;
import com.google.protobuf.ServiceException;
import java.io.IOException;
import java.io.InterruptedIOException;
import java.lang.Thread.UncaughtExceptionHandler;
@ -186,17 +197,6 @@ import org.apache.zookeeper.KeeperException;
import org.apache.zookeeper.KeeperException.NoNodeException;
import org.apache.zookeeper.data.Stat;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
import com.google.common.collect.Maps;
import com.google.protobuf.BlockingRpcChannel;
import com.google.protobuf.Descriptors;
import com.google.protobuf.Message;
import com.google.protobuf.RpcCallback;
import com.google.protobuf.RpcController;
import com.google.protobuf.Service;
import com.google.protobuf.ServiceException;
import sun.misc.Signal;
import sun.misc.SignalHandler;
@ -2373,11 +2373,11 @@ public class HRegionServer extends HasThread implements
if (masterServerName == null) return null;
RegionServerStartupResponse result = null;
try {
rpcServices.requestCount.set(0);
rpcServices.rpcGetRequestCount.set(0);
rpcServices.rpcScanRequestCount.set(0);
rpcServices.rpcMultiRequestCount.set(0);
rpcServices.rpcMutateRequestCount.set(0);
rpcServices.requestCount.reset();
rpcServices.rpcGetRequestCount.reset();
rpcServices.rpcScanRequestCount.reset();
rpcServices.rpcMultiRequestCount.reset();
rpcServices.rpcMutateRequestCount.reset();
LOG.info("reportForDuty to master=" + masterServerName + " with port="
+ rpcServices.isa.getPort() + ", startcode=" + this.startcode);
long now = EnvironmentEdgeManager.currentTime();

View File

@ -20,6 +20,8 @@ package org.apache.hadoop.hbase.regionserver;
import static org.apache.hadoop.util.StringUtils.humanReadableInt;
import com.google.common.base.Preconditions;
import java.io.IOException;
import java.lang.Thread.UncaughtExceptionHandler;
import java.lang.management.ManagementFactory;
@ -37,6 +39,7 @@ import java.util.concurrent.Delayed;
import java.util.concurrent.ThreadFactory;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.LongAdder;
import java.util.concurrent.locks.ReentrantReadWriteLock;
import org.apache.commons.logging.Log;
@ -49,7 +52,6 @@ import org.apache.hadoop.hbase.client.RegionReplicaUtil;
import org.apache.hadoop.hbase.io.util.HeapMemorySizeUtil;
import org.apache.hadoop.hbase.regionserver.Region.FlushResult;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Counter;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.HasThread;
import org.apache.hadoop.hbase.util.ServerRegionReplicaUtil;
@ -60,8 +62,6 @@ import org.apache.hadoop.util.StringUtils.TraditionalBinaryPrefix;
import org.apache.htrace.Trace;
import org.apache.htrace.TraceScope;
import com.google.common.base.Preconditions;
/**
* Thread that flushes cache on request
*
@ -94,7 +94,7 @@ class MemStoreFlusher implements FlushRequester {
protected long globalMemStoreLimitLowMark;
private long blockingWaitTime;
private final Counter updatesBlockedMsHighWater = new Counter();
private final LongAdder updatesBlockedMsHighWater = new LongAdder();
private final FlushHandler[] flushHandlers;
private List<FlushRequestListener> flushRequestListeners = new ArrayList<FlushRequestListener>(1);
@ -129,7 +129,7 @@ class MemStoreFlusher implements FlushRequester {
+ ", maxHeap=" + TraditionalBinaryPrefix.long2String(max, "", 1));
}
public Counter getUpdatesBlockedMsHighWater() {
public LongAdder getUpdatesBlockedMsHighWater() {
return this.updatesBlockedMsHighWater;
}

View File

@ -26,12 +26,12 @@ import java.util.concurrent.TimeUnit;
import org.apache.commons.lang.StringUtils;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.CompatibilitySingletonFactory;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.HDFSBlocksDistribution;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.io.hfile.BlockCache;
import org.apache.hadoop.hbase.io.hfile.CacheConfig;
import org.apache.hadoop.hbase.io.hfile.CacheStats;
@ -215,7 +215,7 @@ class MetricsRegionServerWrapperImpl
@Override
public long getTotalRequestCount() {
return regionServer.rpcServices.requestCount.get();
return regionServer.rpcServices.requestCount.sum();
}
@Override
@ -448,22 +448,22 @@ class MetricsRegionServerWrapperImpl
@Override
public long getRpcGetRequestsCount() {
return regionServer.rpcServices.rpcGetRequestCount.get();
return regionServer.rpcServices.rpcGetRequestCount.sum();
}
@Override
public long getRpcScanRequestsCount() {
return regionServer.rpcServices.rpcScanRequestCount.get();
return regionServer.rpcServices.rpcScanRequestCount.sum();
}
@Override
public long getRpcMultiRequestsCount() {
return regionServer.rpcServices.rpcMultiRequestCount.get();
return regionServer.rpcServices.rpcMultiRequestCount.sum();
}
@Override
public long getRpcMutateRequestsCount() {
return regionServer.rpcServices.rpcMutateRequestCount.get();
return regionServer.rpcServices.rpcMutateRequestCount.sum();
}
@Override
@ -516,7 +516,7 @@ class MetricsRegionServerWrapperImpl
if (this.regionServer.cacheFlusher == null) {
return 0;
}
return this.regionServer.cacheFlusher.getUpdatesBlockedMsHighWater().get();
return this.regionServer.cacheFlusher.getUpdatesBlockedMsHighWater().sum();
}
@Override

View File

@ -18,6 +18,13 @@
*/
package org.apache.hadoop.hbase.regionserver;
import com.google.common.annotations.VisibleForTesting;
import com.google.protobuf.ByteString;
import com.google.protobuf.Message;
import com.google.protobuf.RpcController;
import com.google.protobuf.ServiceException;
import com.google.protobuf.TextFormat;
import java.io.IOException;
import java.io.InterruptedIOException;
import java.net.BindException;
@ -36,6 +43,7 @@ import java.util.Set;
import java.util.TreeSet;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.atomic.AtomicLong;
import java.util.concurrent.atomic.LongAdder;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
@ -177,7 +185,6 @@ import org.apache.hadoop.hbase.regionserver.handler.OpenRegionHandler;
import org.apache.hadoop.hbase.regionserver.wal.WALEdit;
import org.apache.hadoop.hbase.security.User;
import org.apache.hadoop.hbase.util.Bytes;
import org.apache.hadoop.hbase.util.Counter;
import org.apache.hadoop.hbase.util.DNS;
import org.apache.hadoop.hbase.util.EnvironmentEdgeManager;
import org.apache.hadoop.hbase.util.Pair;
@ -189,13 +196,6 @@ import org.apache.hadoop.hbase.wal.WALSplitter;
import org.apache.hadoop.hbase.zookeeper.ZKSplitLog;
import org.apache.zookeeper.KeeperException;
import com.google.common.annotations.VisibleForTesting;
import com.google.protobuf.ByteString;
import com.google.protobuf.Message;
import com.google.protobuf.RpcController;
import com.google.protobuf.ServiceException;
import com.google.protobuf.TextFormat;
/**
* Implements the regionserver RPC services.
*/
@ -223,19 +223,19 @@ public class RSRpcServices implements HBaseRPCErrorHandler,
private static final long DEFAULT_REGION_SERVER_RPC_MINIMUM_SCAN_TIME_LIMIT_DELTA = 10;
// Request counter. (Includes requests that are not serviced by regions.)
final Counter requestCount = new Counter();
final LongAdder requestCount = new LongAdder();
// Request counter for rpc get
final Counter rpcGetRequestCount = new Counter();
final LongAdder rpcGetRequestCount = new LongAdder();
// Request counter for rpc scan
final Counter rpcScanRequestCount = new Counter();
final LongAdder rpcScanRequestCount = new LongAdder();
// Request counter for rpc multi
final Counter rpcMultiRequestCount = new Counter();
final LongAdder rpcMultiRequestCount = new LongAdder();
// Request counter for rpc mutate
final Counter rpcMutateRequestCount = new Counter();
final LongAdder rpcMutateRequestCount = new LongAdder();
// Server to handle client requests.
final RpcServerInterface rpcServer;

View File

@ -25,19 +25,19 @@ import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.List;
import java.util.concurrent.atomic.LongAdder;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.Cell;
import org.apache.hadoop.hbase.CellComparator;
import org.apache.hadoop.hbase.CellUtil;
import org.apache.hadoop.hbase.HConstants;
import org.apache.hadoop.hbase.KeyValue;
import org.apache.hadoop.hbase.KeyValueUtil;
import org.apache.hadoop.hbase.classification.InterfaceAudience;
import org.apache.hadoop.hbase.client.Scan;
import org.apache.hadoop.hbase.io.TimeRange;
import org.apache.hadoop.hbase.io.hfile.HFileScanner;
import org.apache.hadoop.hbase.regionserver.querymatcher.ScanQueryMatcher;
import org.apache.hadoop.hbase.util.Counter;
/**
* KeyValueScanner adaptor over the Reader. It also provides hooks into
@ -61,7 +61,7 @@ public class StoreFileScanner implements KeyValueScanner {
// if have encountered the next row. Only used for reversed scan
private boolean stopSkippingKVsIfNextRow = false;
private static Counter seekCount;
private static LongAdder seekCount;
private final boolean canOptimizeForNonNullColumn;
@ -429,12 +429,12 @@ public class StoreFileScanner implements KeyValueScanner {
}
// Test methods
static final long getSeekCount() {
return seekCount.get();
return seekCount.sum();
}
static final void instrument() {
seekCount = new Counter();
seekCount = new LongAdder();
}
@Override

View File

@ -83,7 +83,7 @@ public class TestFilterListOrOperatorWithBlkCnt {
}
private static long getBlkAccessCount() {
return HFile.DATABLOCK_READ_COUNT.get();
return HFile.DATABLOCK_READ_COUNT.sum();
}
@Test

View File

@ -97,7 +97,7 @@ public class TestForceCacheImportantBlocks {
public void setup() {
// Make sure we make a new one each time.
CacheConfig.GLOBAL_BLOCK_CACHE_INSTANCE = null;
HFile.DATABLOCK_READ_COUNT.set(0);
HFile.DATABLOCK_READ_COUNT.reset();
}
@Test
@ -114,12 +114,12 @@ public class TestForceCacheImportantBlocks {
CacheStats stats = cache.getStats();
writeTestData(region);
assertEquals(0, stats.getHitCount());
assertEquals(0, HFile.DATABLOCK_READ_COUNT.get());
assertEquals(0, HFile.DATABLOCK_READ_COUNT.sum());
// Do a single get, take count of caches. If we are NOT caching DATA blocks, the miss
// count should go up. Otherwise, all should be cached and the miss count should not rise.
region.get(new Get(Bytes.toBytes("row" + 0)));
assertTrue(stats.getHitCount() > 0);
assertTrue(HFile.DATABLOCK_READ_COUNT.get() > 0);
assertTrue(HFile.DATABLOCK_READ_COUNT.sum() > 0);
long missCount = stats.getMissCount();
region.get(new Get(Bytes.toBytes("row" + 0)));
if (this.cfCacheEnabled) assertEquals(missCount, stats.getMissCount());

View File

@ -196,7 +196,7 @@ public class TestBlocksRead {
}
private static long getBlkAccessCount(byte[] cf) {
return HFile.DATABLOCK_READ_COUNT.get();
return HFile.DATABLOCK_READ_COUNT.sum();
}
private static long getBlkCount() {