mirror of https://github.com/apache/lucene.git
SOLR-9335: Switch Solr cache/search/update stats counters to use LongAdder
This commit is contained in:
parent
aba731a285
commit
f906862afe
|
@ -174,6 +174,9 @@ Optimizations
|
|||
* SOLR-9264: Optimize ZkController.publishAndWaitForDownStates to not read all collection states and
|
||||
watch relevant collections instead. (Hrishikesh Gadre, shalin)
|
||||
|
||||
* SOLR-9335: Solr cache/search/update stats counters now use LongAdder which are supposed to have higher throughput
|
||||
under high contention. (Varun Thacker)
|
||||
|
||||
Other Changes
|
||||
----------------------
|
||||
|
||||
|
|
|
@ -16,13 +16,17 @@
|
|||
*/
|
||||
package org.apache.solr.handler;
|
||||
|
||||
import java.lang.invoke.MethodHandles;
|
||||
import java.net.URL;
|
||||
import java.util.concurrent.atomic.LongAdder;
|
||||
|
||||
import org.apache.solr.common.SolrException;
|
||||
import org.apache.solr.common.params.SolrParams;
|
||||
import org.apache.solr.common.util.NamedList;
|
||||
import org.apache.solr.common.util.SimpleOrderedMap;
|
||||
import org.apache.solr.common.util.SuppressForbidden;
|
||||
import org.apache.solr.core.PluginInfo;
|
||||
import org.apache.solr.core.PluginBag;
|
||||
import org.apache.solr.core.PluginInfo;
|
||||
import org.apache.solr.core.SolrInfoMBean;
|
||||
import org.apache.solr.request.SolrQueryRequest;
|
||||
import org.apache.solr.request.SolrRequestHandler;
|
||||
|
@ -35,10 +39,6 @@ import org.apache.solr.util.stats.TimerContext;
|
|||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import java.lang.invoke.MethodHandles;
|
||||
import java.net.URL;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
|
||||
import static org.apache.solr.core.RequestParams.USEPARAM;
|
||||
|
||||
/**
|
||||
|
@ -53,10 +53,10 @@ public abstract class RequestHandlerBase implements SolrRequestHandler, SolrInfo
|
|||
protected boolean httpCaching = true;
|
||||
|
||||
// Statistics
|
||||
private final AtomicLong numRequests = new AtomicLong();
|
||||
private final AtomicLong numServerErrors = new AtomicLong();
|
||||
private final AtomicLong numClientErrors = new AtomicLong();
|
||||
private final AtomicLong numTimeouts = new AtomicLong();
|
||||
private final LongAdder numRequests = new LongAdder();
|
||||
private final LongAdder numServerErrors = new LongAdder();
|
||||
private final LongAdder numClientErrors = new LongAdder();
|
||||
private final LongAdder numTimeouts = new LongAdder();
|
||||
private final Timer requestTimes = new Timer();
|
||||
|
||||
private final long handlerStart;
|
||||
|
@ -144,7 +144,7 @@ public abstract class RequestHandlerBase implements SolrRequestHandler, SolrInfo
|
|||
|
||||
@Override
|
||||
public void handleRequest(SolrQueryRequest req, SolrQueryResponse rsp) {
|
||||
numRequests.incrementAndGet();
|
||||
numRequests.increment();
|
||||
TimerContext timer = requestTimes.time();
|
||||
try {
|
||||
if(pluginInfo != null && pluginInfo.attributes.containsKey(USEPARAM)) req.getContext().put(USEPARAM,pluginInfo.attributes.get(USEPARAM));
|
||||
|
@ -158,7 +158,7 @@ public abstract class RequestHandlerBase implements SolrRequestHandler, SolrInfo
|
|||
Object partialResults = header.get(SolrQueryResponse.RESPONSE_HEADER_PARTIAL_RESULTS_KEY);
|
||||
boolean timedOut = partialResults == null ? false : (Boolean)partialResults;
|
||||
if( timedOut ) {
|
||||
numTimeouts.incrementAndGet();
|
||||
numTimeouts.increment();
|
||||
rsp.setHttpCaching(false);
|
||||
}
|
||||
}
|
||||
|
@ -185,9 +185,9 @@ public abstract class RequestHandlerBase implements SolrRequestHandler, SolrInfo
|
|||
SolrException.log(log, e);
|
||||
|
||||
if (isServerError) {
|
||||
numServerErrors.incrementAndGet();
|
||||
numServerErrors.increment();
|
||||
} else {
|
||||
numClientErrors.incrementAndGet();
|
||||
numClientErrors.increment();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -23,7 +23,7 @@ import java.util.Iterator;
|
|||
import java.util.LinkedHashMap;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
import java.util.concurrent.atomic.LongAdder;
|
||||
|
||||
import org.apache.lucene.util.Accountable;
|
||||
import org.apache.lucene.util.Accountables;
|
||||
|
@ -61,11 +61,11 @@ public class LRUCache<K,V> extends SolrCacheBase implements SolrCache<K,V>, Acco
|
|||
* of an LRUCache at the same time. Make sure everything is thread safe.
|
||||
*/
|
||||
private static class CumulativeStats {
|
||||
AtomicLong lookups = new AtomicLong();
|
||||
AtomicLong hits = new AtomicLong();
|
||||
AtomicLong inserts = new AtomicLong();
|
||||
AtomicLong evictions = new AtomicLong();
|
||||
AtomicLong evictionsRamUsage = new AtomicLong();
|
||||
LongAdder lookups = new LongAdder();
|
||||
LongAdder hits = new LongAdder();
|
||||
LongAdder inserts = new LongAdder();
|
||||
LongAdder evictions = new LongAdder();
|
||||
LongAdder evictionsRamUsage = new LongAdder();
|
||||
}
|
||||
|
||||
private CumulativeStats stats;
|
||||
|
@ -124,8 +124,8 @@ public class LRUCache<K,V> extends SolrCacheBase implements SolrCache<K,V>, Acco
|
|||
iterator.remove();
|
||||
evictions++;
|
||||
evictionsRamUsage++;
|
||||
stats.evictions.incrementAndGet();
|
||||
stats.evictionsRamUsage.incrementAndGet();
|
||||
stats.evictions.increment();
|
||||
stats.evictionsRamUsage.increment();
|
||||
} while (iterator.hasNext() && ramBytesUsed > maxRamBytes);
|
||||
// must return false according to javadocs of removeEldestEntry if we're modifying
|
||||
// the map ourselves
|
||||
|
@ -135,7 +135,7 @@ public class LRUCache<K,V> extends SolrCacheBase implements SolrCache<K,V>, Acco
|
|||
// this doesn't need to be synchronized because it will
|
||||
// only be called in the context of a higher level synchronized block.
|
||||
evictions++;
|
||||
stats.evictions.incrementAndGet();
|
||||
stats.evictions.increment();
|
||||
return true;
|
||||
}
|
||||
}
|
||||
|
@ -180,7 +180,7 @@ public class LRUCache<K,V> extends SolrCacheBase implements SolrCache<K,V>, Acco
|
|||
public V put(K key, V value) {
|
||||
synchronized (map) {
|
||||
if (getState() == State.LIVE) {
|
||||
stats.inserts.incrementAndGet();
|
||||
stats.inserts.increment();
|
||||
}
|
||||
|
||||
// increment local inserts regardless of state???
|
||||
|
@ -232,10 +232,10 @@ public class LRUCache<K,V> extends SolrCacheBase implements SolrCache<K,V>, Acco
|
|||
if (getState() == State.LIVE) {
|
||||
// only increment lookups and hits if we are live.
|
||||
lookups++;
|
||||
stats.lookups.incrementAndGet();
|
||||
stats.lookups.increment();
|
||||
if (val!=null) {
|
||||
hits++;
|
||||
stats.hits.incrementAndGet();
|
||||
stats.hits.increment();
|
||||
}
|
||||
}
|
||||
return val;
|
||||
|
@ -341,15 +341,15 @@ public class LRUCache<K,V> extends SolrCacheBase implements SolrCache<K,V>, Acco
|
|||
}
|
||||
lst.add("warmupTime", warmupTime);
|
||||
|
||||
long clookups = stats.lookups.get();
|
||||
long chits = stats.hits.get();
|
||||
long clookups = stats.lookups.longValue();
|
||||
long chits = stats.hits.longValue();
|
||||
lst.add("cumulative_lookups", clookups);
|
||||
lst.add("cumulative_hits", chits);
|
||||
lst.add("cumulative_hitratio", calcHitRatio(clookups, chits));
|
||||
lst.add("cumulative_inserts", stats.inserts.get());
|
||||
lst.add("cumulative_evictions", stats.evictions.get());
|
||||
lst.add("cumulative_inserts", stats.inserts.longValue());
|
||||
lst.add("cumulative_evictions", stats.evictions.longValue());
|
||||
if (maxRamBytes != Long.MAX_VALUE) {
|
||||
lst.add("cumulative_evictionsRamUsage", stats.evictionsRamUsage.get());
|
||||
lst.add("cumulative_evictionsRamUsage", stats.evictionsRamUsage.longValue());
|
||||
}
|
||||
|
||||
return lst;
|
||||
|
|
|
@ -26,7 +26,7 @@ import java.util.Locale;
|
|||
import java.util.Map;
|
||||
import java.util.concurrent.ExecutionException;
|
||||
import java.util.concurrent.Future;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
import java.util.concurrent.atomic.LongAdder;
|
||||
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.index.CodecReader;
|
||||
|
@ -76,20 +76,20 @@ public class DirectUpdateHandler2 extends UpdateHandler implements SolrCoreState
|
|||
protected final SolrCoreState solrCoreState;
|
||||
|
||||
// stats
|
||||
AtomicLong addCommands = new AtomicLong();
|
||||
AtomicLong addCommandsCumulative = new AtomicLong();
|
||||
AtomicLong deleteByIdCommands= new AtomicLong();
|
||||
AtomicLong deleteByIdCommandsCumulative= new AtomicLong();
|
||||
AtomicLong deleteByQueryCommands= new AtomicLong();
|
||||
AtomicLong deleteByQueryCommandsCumulative= new AtomicLong();
|
||||
AtomicLong expungeDeleteCommands = new AtomicLong();
|
||||
AtomicLong mergeIndexesCommands = new AtomicLong();
|
||||
AtomicLong commitCommands= new AtomicLong();
|
||||
AtomicLong optimizeCommands= new AtomicLong();
|
||||
AtomicLong rollbackCommands= new AtomicLong();
|
||||
AtomicLong numDocsPending= new AtomicLong();
|
||||
AtomicLong numErrors = new AtomicLong();
|
||||
AtomicLong numErrorsCumulative = new AtomicLong();
|
||||
LongAdder addCommands = new LongAdder();
|
||||
LongAdder addCommandsCumulative = new LongAdder();
|
||||
LongAdder deleteByIdCommands= new LongAdder();
|
||||
LongAdder deleteByIdCommandsCumulative= new LongAdder();
|
||||
LongAdder deleteByQueryCommands= new LongAdder();
|
||||
LongAdder deleteByQueryCommandsCumulative= new LongAdder();
|
||||
LongAdder expungeDeleteCommands = new LongAdder();
|
||||
LongAdder mergeIndexesCommands = new LongAdder();
|
||||
LongAdder commitCommands= new LongAdder();
|
||||
LongAdder optimizeCommands= new LongAdder();
|
||||
LongAdder rollbackCommands= new LongAdder();
|
||||
LongAdder numDocsPending= new LongAdder();
|
||||
LongAdder numErrors = new LongAdder();
|
||||
LongAdder numErrorsCumulative = new LongAdder();
|
||||
|
||||
// tracks when auto-commit should occur
|
||||
protected final CommitTracker commitTracker;
|
||||
|
@ -158,7 +158,7 @@ public class DirectUpdateHandler2 extends UpdateHandler implements SolrCoreState
|
|||
}
|
||||
|
||||
protected void rollbackWriter() throws IOException {
|
||||
numDocsPending.set(0);
|
||||
numDocsPending.reset();
|
||||
solrCoreState.rollbackIndexWriter(core);
|
||||
|
||||
}
|
||||
|
@ -192,8 +192,8 @@ public class DirectUpdateHandler2 extends UpdateHandler implements SolrCoreState
|
|||
private int addDoc0(AddUpdateCommand cmd) throws IOException {
|
||||
int rc = -1;
|
||||
|
||||
addCommands.incrementAndGet();
|
||||
addCommandsCumulative.incrementAndGet();
|
||||
addCommands.increment();
|
||||
addCommandsCumulative.increment();
|
||||
|
||||
// if there is no ID field, don't overwrite
|
||||
if (idField == null) {
|
||||
|
@ -230,10 +230,10 @@ public class DirectUpdateHandler2 extends UpdateHandler implements SolrCoreState
|
|||
rc = 1;
|
||||
} finally {
|
||||
if (rc != 1) {
|
||||
numErrors.incrementAndGet();
|
||||
numErrorsCumulative.incrementAndGet();
|
||||
numErrors.increment();
|
||||
numErrorsCumulative.increment();
|
||||
} else {
|
||||
numDocsPending.incrementAndGet();
|
||||
numDocsPending.increment();
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -368,8 +368,8 @@ public class DirectUpdateHandler2 extends UpdateHandler implements SolrCoreState
|
|||
// we don't return the number of docs deleted because it's not always possible to quickly know that info.
|
||||
@Override
|
||||
public void delete(DeleteUpdateCommand cmd) throws IOException {
|
||||
deleteByIdCommands.incrementAndGet();
|
||||
deleteByIdCommandsCumulative.incrementAndGet();
|
||||
deleteByIdCommands.increment();
|
||||
deleteByIdCommandsCumulative.increment();
|
||||
|
||||
Term deleteTerm = new Term(idField.getName(), cmd.getIndexedId());
|
||||
// SolrCore.verbose("deleteDocuments",deleteTerm,writer);
|
||||
|
@ -426,8 +426,8 @@ public class DirectUpdateHandler2 extends UpdateHandler implements SolrCoreState
|
|||
// we don't return the number of docs deleted because it's not always possible to quickly know that info.
|
||||
@Override
|
||||
public void deleteByQuery(DeleteUpdateCommand cmd) throws IOException {
|
||||
deleteByQueryCommands.incrementAndGet();
|
||||
deleteByQueryCommandsCumulative.incrementAndGet();
|
||||
deleteByQueryCommands.increment();
|
||||
deleteByQueryCommandsCumulative.increment();
|
||||
boolean madeIt=false;
|
||||
try {
|
||||
Query q = getQuery(cmd);
|
||||
|
@ -473,8 +473,8 @@ public class DirectUpdateHandler2 extends UpdateHandler implements SolrCoreState
|
|||
|
||||
} finally {
|
||||
if (!madeIt) {
|
||||
numErrors.incrementAndGet();
|
||||
numErrorsCumulative.incrementAndGet();
|
||||
numErrors.increment();
|
||||
numErrorsCumulative.increment();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -482,7 +482,7 @@ public class DirectUpdateHandler2 extends UpdateHandler implements SolrCoreState
|
|||
|
||||
@Override
|
||||
public int mergeIndexes(MergeIndexesCommand cmd) throws IOException {
|
||||
mergeIndexesCommands.incrementAndGet();
|
||||
mergeIndexesCommands.increment();
|
||||
int rc;
|
||||
|
||||
log.info("start " + cmd);
|
||||
|
@ -545,7 +545,7 @@ public class DirectUpdateHandler2 extends UpdateHandler implements SolrCoreState
|
|||
error=false;
|
||||
}
|
||||
finally {
|
||||
if (error) numErrors.incrementAndGet();
|
||||
if (error) numErrors.increment();
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -557,10 +557,10 @@ public class DirectUpdateHandler2 extends UpdateHandler implements SolrCoreState
|
|||
}
|
||||
|
||||
if (cmd.optimize) {
|
||||
optimizeCommands.incrementAndGet();
|
||||
optimizeCommands.increment();
|
||||
} else {
|
||||
commitCommands.incrementAndGet();
|
||||
if (cmd.expungeDeletes) expungeDeleteCommands.incrementAndGet();
|
||||
commitCommands.increment();
|
||||
if (cmd.expungeDeletes) expungeDeleteCommands.increment();
|
||||
}
|
||||
|
||||
Future[] waitSearcher = null;
|
||||
|
@ -622,7 +622,7 @@ public class DirectUpdateHandler2 extends UpdateHandler implements SolrCoreState
|
|||
}
|
||||
|
||||
// SolrCore.verbose("writer.commit() end");
|
||||
numDocsPending.set(0);
|
||||
numDocsPending.reset();
|
||||
callPostCommitCallbacks();
|
||||
}
|
||||
} finally {
|
||||
|
@ -676,10 +676,10 @@ public class DirectUpdateHandler2 extends UpdateHandler implements SolrCoreState
|
|||
solrCoreState.getCommitLock().unlock();
|
||||
}
|
||||
|
||||
addCommands.set(0);
|
||||
deleteByIdCommands.set(0);
|
||||
deleteByQueryCommands.set(0);
|
||||
if (error) numErrors.incrementAndGet();
|
||||
addCommands.reset();
|
||||
deleteByIdCommands.reset();
|
||||
deleteByQueryCommands.reset();
|
||||
if (error) numErrors.increment();
|
||||
}
|
||||
|
||||
// if we are supposed to wait for the searcher to be registered, then we should do it
|
||||
|
@ -707,7 +707,7 @@ public class DirectUpdateHandler2 extends UpdateHandler implements SolrCoreState
|
|||
throw new UnsupportedOperationException("Rollback is currently not supported in SolrCloud mode. (SOLR-4895)");
|
||||
}
|
||||
|
||||
rollbackCommands.incrementAndGet();
|
||||
rollbackCommands.increment();
|
||||
|
||||
boolean error=true;
|
||||
|
||||
|
@ -727,13 +727,10 @@ public class DirectUpdateHandler2 extends UpdateHandler implements SolrCoreState
|
|||
error=false;
|
||||
}
|
||||
finally {
|
||||
addCommandsCumulative.set(
|
||||
addCommandsCumulative.get() - addCommands.getAndSet( 0 ) );
|
||||
deleteByIdCommandsCumulative.set(
|
||||
deleteByIdCommandsCumulative.get() - deleteByIdCommands.getAndSet( 0 ) );
|
||||
deleteByQueryCommandsCumulative.set(
|
||||
deleteByQueryCommandsCumulative.get() - deleteByQueryCommands.getAndSet( 0 ) );
|
||||
if (error) numErrors.incrementAndGet();
|
||||
addCommandsCumulative.add(-addCommands.sumThenReset());
|
||||
deleteByIdCommandsCumulative.add(-deleteByIdCommands.sumThenReset());
|
||||
deleteByQueryCommandsCumulative.add(-deleteByQueryCommands.sumThenReset());
|
||||
if (error) numErrors.increment();
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -749,7 +746,7 @@ public class DirectUpdateHandler2 extends UpdateHandler implements SolrCoreState
|
|||
commitTracker.close();
|
||||
softCommitTracker.close();
|
||||
|
||||
numDocsPending.set(0);
|
||||
numDocsPending.reset();
|
||||
}
|
||||
|
||||
|
||||
|
@ -882,7 +879,7 @@ public class DirectUpdateHandler2 extends UpdateHandler implements SolrCoreState
|
|||
@Override
|
||||
public NamedList getStatistics() {
|
||||
NamedList lst = new SimpleOrderedMap();
|
||||
lst.add("commits", commitCommands.get());
|
||||
lst.add("commits", commitCommands.longValue());
|
||||
if (commitTracker.getDocsUpperBound() > 0) {
|
||||
lst.add("autocommit maxDocs", commitTracker.getDocsUpperBound());
|
||||
}
|
||||
|
@ -897,20 +894,20 @@ public class DirectUpdateHandler2 extends UpdateHandler implements SolrCoreState
|
|||
lst.add("soft autocommit maxTime", "" + softCommitTracker.getTimeUpperBound() + "ms");
|
||||
}
|
||||
lst.add("soft autocommits", softCommitTracker.getCommitCount());
|
||||
lst.add("optimizes", optimizeCommands.get());
|
||||
lst.add("rollbacks", rollbackCommands.get());
|
||||
lst.add("expungeDeletes", expungeDeleteCommands.get());
|
||||
lst.add("docsPending", numDocsPending.get());
|
||||
lst.add("optimizes", optimizeCommands.longValue());
|
||||
lst.add("rollbacks", rollbackCommands.longValue());
|
||||
lst.add("expungeDeletes", expungeDeleteCommands.longValue());
|
||||
lst.add("docsPending", numDocsPending.longValue());
|
||||
// pset.size() not synchronized, but it should be fine to access.
|
||||
// lst.add("deletesPending", pset.size());
|
||||
lst.add("adds", addCommands.get());
|
||||
lst.add("deletesById", deleteByIdCommands.get());
|
||||
lst.add("deletesByQuery", deleteByQueryCommands.get());
|
||||
lst.add("errors", numErrors.get());
|
||||
lst.add("cumulative_adds", addCommandsCumulative.get());
|
||||
lst.add("cumulative_deletesById", deleteByIdCommandsCumulative.get());
|
||||
lst.add("cumulative_deletesByQuery", deleteByQueryCommandsCumulative.get());
|
||||
lst.add("cumulative_errors", numErrorsCumulative.get());
|
||||
lst.add("adds", addCommands.longValue());
|
||||
lst.add("deletesById", deleteByIdCommands.longValue());
|
||||
lst.add("deletesByQuery", deleteByQueryCommands.longValue());
|
||||
lst.add("errors", numErrors.longValue());
|
||||
lst.add("cumulative_adds", addCommandsCumulative.longValue());
|
||||
lst.add("cumulative_deletesById", deleteByIdCommandsCumulative.longValue());
|
||||
lst.add("cumulative_deletesByQuery", deleteByQueryCommandsCumulative.longValue());
|
||||
lst.add("cumulative_errors", numErrorsCumulative.longValue());
|
||||
if (this.ulog != null) {
|
||||
lst.add("transaction_logs_total_size", ulog.getTotalLogsSize());
|
||||
lst.add("transaction_logs_total_number", ulog.getTotalLogsNumber());
|
||||
|
|
|
@ -28,6 +28,7 @@ import java.util.TreeSet;
|
|||
import java.util.concurrent.ConcurrentHashMap;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
import java.util.concurrent.atomic.LongAdder;
|
||||
import java.util.concurrent.locks.ReentrantLock;
|
||||
import java.lang.invoke.MethodHandles;
|
||||
import java.lang.ref.WeakReference;
|
||||
|
@ -90,7 +91,7 @@ public class ConcurrentLRUCache<K,V> implements Cache<K,V> {
|
|||
public V get(K key) {
|
||||
CacheEntry<K,V> e = map.get(key);
|
||||
if (e == null) {
|
||||
if (islive) stats.missCounter.incrementAndGet();
|
||||
if (islive) stats.missCounter.increment();
|
||||
return null;
|
||||
}
|
||||
if (islive) e.lastAccessed = stats.accessCounter.incrementAndGet();
|
||||
|
@ -119,9 +120,9 @@ public class ConcurrentLRUCache<K,V> implements Cache<K,V> {
|
|||
currentSize = stats.size.get();
|
||||
}
|
||||
if (islive) {
|
||||
stats.putCounter.incrementAndGet();
|
||||
stats.putCounter.increment();
|
||||
} else {
|
||||
stats.nonLivePutCounter.incrementAndGet();
|
||||
stats.nonLivePutCounter.increment();
|
||||
}
|
||||
|
||||
// Check if we need to clear out old entries from the cache.
|
||||
|
@ -172,7 +173,7 @@ public class ConcurrentLRUCache<K,V> implements Cache<K,V> {
|
|||
isCleaning = true;
|
||||
this.oldestEntry = oldestEntry; // volatile write to make isCleaning visible
|
||||
|
||||
long timeCurrent = stats.accessCounter.get();
|
||||
long timeCurrent = stats.accessCounter.longValue();
|
||||
int sz = stats.size.get();
|
||||
|
||||
int numRemoved = 0;
|
||||
|
@ -532,23 +533,23 @@ public class ConcurrentLRUCache<K,V> implements Cache<K,V> {
|
|||
|
||||
|
||||
public static class Stats {
|
||||
private final AtomicLong accessCounter = new AtomicLong(0),
|
||||
putCounter = new AtomicLong(0),
|
||||
nonLivePutCounter = new AtomicLong(0),
|
||||
missCounter = new AtomicLong();
|
||||
private final AtomicLong accessCounter = new AtomicLong(0);
|
||||
private final LongAdder putCounter = new LongAdder();
|
||||
private final LongAdder nonLivePutCounter = new LongAdder();
|
||||
private final LongAdder missCounter = new LongAdder();
|
||||
private final AtomicInteger size = new AtomicInteger();
|
||||
private AtomicLong evictionCounter = new AtomicLong();
|
||||
|
||||
public long getCumulativeLookups() {
|
||||
return (accessCounter.get() - putCounter.get() - nonLivePutCounter.get()) + missCounter.get();
|
||||
return (accessCounter.longValue() - putCounter.longValue() - nonLivePutCounter.longValue()) + missCounter.longValue();
|
||||
}
|
||||
|
||||
public long getCumulativeHits() {
|
||||
return accessCounter.get() - putCounter.get() - nonLivePutCounter.get();
|
||||
return accessCounter.longValue() - putCounter.longValue() - nonLivePutCounter.longValue();
|
||||
}
|
||||
|
||||
public long getCumulativePuts() {
|
||||
return putCounter.get();
|
||||
return putCounter.longValue();
|
||||
}
|
||||
|
||||
public long getCumulativeEvictions() {
|
||||
|
@ -560,18 +561,18 @@ public class ConcurrentLRUCache<K,V> implements Cache<K,V> {
|
|||
}
|
||||
|
||||
public long getCumulativeNonLivePuts() {
|
||||
return nonLivePutCounter.get();
|
||||
return nonLivePutCounter.longValue();
|
||||
}
|
||||
|
||||
public long getCumulativeMisses() {
|
||||
return missCounter.get();
|
||||
return missCounter.longValue();
|
||||
}
|
||||
|
||||
public void add(Stats other) {
|
||||
accessCounter.addAndGet(other.accessCounter.get());
|
||||
putCounter.addAndGet(other.putCounter.get());
|
||||
nonLivePutCounter.addAndGet(other.nonLivePutCounter.get());
|
||||
missCounter.addAndGet(other.missCounter.get());
|
||||
putCounter.add(other.putCounter.longValue());
|
||||
nonLivePutCounter.add(other.nonLivePutCounter.longValue());
|
||||
missCounter.add(other.missCounter.longValue());
|
||||
evictionCounter.addAndGet(other.evictionCounter.get());
|
||||
size.set(Math.max(size.get(), other.size.get()));
|
||||
}
|
||||
|
|
|
@ -160,13 +160,13 @@ public class DirectUpdateHandlerTest extends SolrTestCaseJ4 {
|
|||
SolrQueryRequest ureq = req();
|
||||
CommitUpdateCommand cmtCmd = new CommitUpdateCommand(ureq, false);
|
||||
cmtCmd.waitSearcher = true;
|
||||
assertEquals( 1, duh2.addCommands.get() );
|
||||
assertEquals( 1, duh2.addCommandsCumulative.get() );
|
||||
assertEquals( 0, duh2.commitCommands.get() );
|
||||
assertEquals( 1, duh2.addCommands.longValue() );
|
||||
assertEquals( 1, duh2.addCommandsCumulative.longValue() );
|
||||
assertEquals( 0, duh2.commitCommands.longValue() );
|
||||
updater.commit(cmtCmd);
|
||||
assertEquals( 0, duh2.addCommands.get() );
|
||||
assertEquals( 1, duh2.addCommandsCumulative.get() );
|
||||
assertEquals( 1, duh2.commitCommands.get() );
|
||||
assertEquals( 0, duh2.addCommands.longValue() );
|
||||
assertEquals( 1, duh2.addCommandsCumulative.longValue() );
|
||||
assertEquals( 1, duh2.commitCommands.longValue() );
|
||||
ureq.close();
|
||||
|
||||
assertU(adoc("id","B"));
|
||||
|
@ -174,13 +174,13 @@ public class DirectUpdateHandlerTest extends SolrTestCaseJ4 {
|
|||
// rollback "B"
|
||||
ureq = req();
|
||||
RollbackUpdateCommand rbkCmd = new RollbackUpdateCommand(ureq);
|
||||
assertEquals( 1, duh2.addCommands.get() );
|
||||
assertEquals( 2, duh2.addCommandsCumulative.get() );
|
||||
assertEquals( 0, duh2.rollbackCommands.get() );
|
||||
assertEquals( 1, duh2.addCommands.longValue() );
|
||||
assertEquals( 2, duh2.addCommandsCumulative.longValue() );
|
||||
assertEquals( 0, duh2.rollbackCommands.longValue() );
|
||||
updater.rollback(rbkCmd);
|
||||
assertEquals( 0, duh2.addCommands.get() );
|
||||
assertEquals( 1, duh2.addCommandsCumulative.get() );
|
||||
assertEquals( 1, duh2.rollbackCommands.get() );
|
||||
assertEquals( 0, duh2.addCommands.longValue() );
|
||||
assertEquals( 1, duh2.addCommandsCumulative.longValue() );
|
||||
assertEquals( 1, duh2.rollbackCommands.longValue() );
|
||||
ureq.close();
|
||||
|
||||
// search - "B" should not be found.
|
||||
|
@ -220,13 +220,13 @@ public class DirectUpdateHandlerTest extends SolrTestCaseJ4 {
|
|||
SolrQueryRequest ureq = req();
|
||||
CommitUpdateCommand cmtCmd = new CommitUpdateCommand(ureq, false);
|
||||
cmtCmd.waitSearcher = true;
|
||||
assertEquals( 2, duh2.addCommands.get() );
|
||||
assertEquals( 2, duh2.addCommandsCumulative.get() );
|
||||
assertEquals( 0, duh2.commitCommands.get() );
|
||||
assertEquals( 2, duh2.addCommands.longValue() );
|
||||
assertEquals( 2, duh2.addCommandsCumulative.longValue() );
|
||||
assertEquals( 0, duh2.commitCommands.longValue() );
|
||||
updater.commit(cmtCmd);
|
||||
assertEquals( 0, duh2.addCommands.get() );
|
||||
assertEquals( 2, duh2.addCommandsCumulative.get() );
|
||||
assertEquals( 1, duh2.commitCommands.get() );
|
||||
assertEquals( 0, duh2.addCommands.longValue() );
|
||||
assertEquals( 2, duh2.addCommandsCumulative.longValue() );
|
||||
assertEquals( 1, duh2.commitCommands.longValue() );
|
||||
ureq.close();
|
||||
|
||||
// search - "A","B" should be found.
|
||||
|
@ -253,14 +253,14 @@ public class DirectUpdateHandlerTest extends SolrTestCaseJ4 {
|
|||
// rollback "B"
|
||||
ureq = req();
|
||||
RollbackUpdateCommand rbkCmd = new RollbackUpdateCommand(ureq);
|
||||
assertEquals( 1, duh2.deleteByIdCommands.get() );
|
||||
assertEquals( 1, duh2.deleteByIdCommandsCumulative.get() );
|
||||
assertEquals( 0, duh2.rollbackCommands.get() );
|
||||
assertEquals( 1, duh2.deleteByIdCommands.longValue() );
|
||||
assertEquals( 1, duh2.deleteByIdCommandsCumulative.longValue() );
|
||||
assertEquals( 0, duh2.rollbackCommands.longValue() );
|
||||
updater.rollback(rbkCmd);
|
||||
ureq.close();
|
||||
assertEquals( 0, duh2.deleteByIdCommands.get() );
|
||||
assertEquals( 0, duh2.deleteByIdCommandsCumulative.get() );
|
||||
assertEquals( 1, duh2.rollbackCommands.get() );
|
||||
assertEquals( 0, duh2.deleteByIdCommands.longValue() );
|
||||
assertEquals( 0, duh2.deleteByIdCommandsCumulative.longValue() );
|
||||
assertEquals( 1, duh2.rollbackCommands.longValue() );
|
||||
|
||||
// search - "B" should be found.
|
||||
assertQ("\"B\" should be found.", req
|
||||
|
|
Loading…
Reference in New Issue