You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by yo...@apache.org on 2017/02/28 17:36:14 UTC

lucene-solr:master: SOLR-10214: clean up BlockCache Metrics, add storeFails and counts

Repository: lucene-solr
Updated Branches:
  refs/heads/master 2adc11c70 -> 34bb7f31e


SOLR-10214: clean up BlockCache Metrics, add storeFails and counts


Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/34bb7f31
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/34bb7f31
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/34bb7f31

Branch: refs/heads/master
Commit: 34bb7f31e546856094cb378b9d12c9ac7540e7e2
Parents: 2adc11c
Author: yonik <yo...@apache.org>
Authored: Tue Feb 28 12:35:13 2017 -0500
Committer: yonik <yo...@apache.org>
Committed: Tue Feb 28 12:36:08 2017 -0500

----------------------------------------------------------------------
 solr/CHANGES.txt                                |   4 +
 .../solr/store/blockcache/BlockCache.java       |   6 +
 .../store/blockcache/BlockDirectoryCache.java   |   5 -
 .../apache/solr/store/blockcache/Metrics.java   | 118 ++++++++-----------
 4 files changed, 62 insertions(+), 71 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/34bb7f31/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 07f1c4e..47f190b 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -256,6 +256,10 @@ Other Changes
 
 * SOLR-7453: Remove replication & backup scripts in the solr/scripts directory of the checkout (Varun Thacker)
 
+* SOLR-10214: Remove unused HDFS BlockCache metrics and add storeFails, as well as adding total
+  counts for lookups, hits, and evictions. (yonik)
+  
+
 ==================  6.4.2 ==================
 
 Consult the LUCENE_CHANGES.txt file for additional, low level, changes in this release.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/34bb7f31/solr/core/src/java/org/apache/solr/store/blockcache/BlockCache.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/store/blockcache/BlockCache.java b/solr/core/src/java/org/apache/solr/store/blockcache/BlockCache.java
index b774782..ad5b2f4 100644
--- a/solr/core/src/java/org/apache/solr/store/blockcache/BlockCache.java
+++ b/solr/core/src/java/org/apache/solr/store/blockcache/BlockCache.java
@@ -133,6 +133,7 @@ public class BlockCache {
         // YCS: it looks like when the cache is full (a normal scenario), then two concurrent writes will result in one of them failing
         // because no eviction is done first.  The code seems to rely on leaving just a single block empty.
         // TODO: simplest fix would be to leave more than one block empty
+        metrics.blockCacheStoreFail.incrementAndGet();
         return false;
       }
     } else {
@@ -141,6 +142,8 @@ public class BlockCache {
       // purpose (and then our write may overwrite that).  This can happen even if clients never try to update existing blocks,
       // since two clients can try to cache the same block concurrently.  Because of this, the ability to update an existing
       // block has been removed for the time being (see SOLR-10121).
+
+      // No metrics to update: we don't count a redundant store as a store fail.
       return false;
     }
 
@@ -168,6 +171,7 @@ public class BlockCache {
       int blockOffset, int off, int length) {
     BlockCacheLocation location = cache.getIfPresent(blockCacheKey);
     if (location == null) {
+      metrics.blockCacheMiss.incrementAndGet();
       return false;
     }
 
@@ -181,9 +185,11 @@ public class BlockCache {
     if (location.isRemoved()) {
       // must check *after* the read is done since the bank may have been reused for another block
       // before or during the read.
+      metrics.blockCacheMiss.incrementAndGet();
       return false;
     }
 
+    metrics.blockCacheHit.incrementAndGet();
     return true;
   }
   

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/34bb7f31/solr/core/src/java/org/apache/solr/store/blockcache/BlockDirectoryCache.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/store/blockcache/BlockDirectoryCache.java b/solr/core/src/java/org/apache/solr/store/blockcache/BlockDirectoryCache.java
index e8a9f43..6e999d5 100644
--- a/solr/core/src/java/org/apache/solr/store/blockcache/BlockDirectoryCache.java
+++ b/solr/core/src/java/org/apache/solr/store/blockcache/BlockDirectoryCache.java
@@ -104,11 +104,6 @@ public class BlockDirectoryCache implements Cache {
     blockCacheKey.setFile(file);
     boolean fetch = blockCache.fetch(blockCacheKey, b, blockOffset, off,
         lengthToReadInBlock);
-    if (fetch) {
-      metrics.blockCacheHit.incrementAndGet();
-    } else {
-      metrics.blockCacheMiss.incrementAndGet();
-    }
     return fetch;
   }
   

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/34bb7f31/solr/core/src/java/org/apache/solr/store/blockcache/Metrics.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/store/blockcache/Metrics.java b/solr/core/src/java/org/apache/solr/store/blockcache/Metrics.java
index cfab89e..3dc8947 100644
--- a/solr/core/src/java/org/apache/solr/store/blockcache/Metrics.java
+++ b/solr/core/src/java/org/apache/solr/store/blockcache/Metrics.java
@@ -33,92 +33,78 @@ import org.apache.solr.search.SolrCacheBase;
  * @lucene.experimental
  */
 public class Metrics extends SolrCacheBase implements SolrInfoMBean {
-  
-  public static class MethodCall {
-    public AtomicLong invokes = new AtomicLong();
-    public AtomicLong times = new AtomicLong();
-  }
 
+
+  public AtomicLong blockCacheSize = new AtomicLong(0);
   public AtomicLong blockCacheHit = new AtomicLong(0);
   public AtomicLong blockCacheMiss = new AtomicLong(0);
   public AtomicLong blockCacheEviction = new AtomicLong(0);
-  public AtomicLong blockCacheSize = new AtomicLong(0);
-  public AtomicLong rowReads = new AtomicLong(0);
-  public AtomicLong rowWrites = new AtomicLong(0);
-  public AtomicLong recordReads = new AtomicLong(0);
-  public AtomicLong recordWrites = new AtomicLong(0);
-  public AtomicLong queriesExternal = new AtomicLong(0);
-  public AtomicLong queriesInternal = new AtomicLong(0);
+  public AtomicLong blockCacheStoreFail = new AtomicLong(0);
+
+  // since the last call
+  private AtomicLong blockCacheHit_last = new AtomicLong(0);
+  private AtomicLong blockCacheMiss_last = new AtomicLong(0);
+  private AtomicLong blockCacheEviction_last = new AtomicLong(0);
+  public AtomicLong blockCacheStoreFail_last = new AtomicLong(0);
+
+
+  // These are used by the BufferStore (just a generic cache of byte[]).
+  // TODO: If this (the Store) is a good idea, we should make it more general and use it across more places in Solr.
   public AtomicLong shardBuffercacheAllocate = new AtomicLong(0);
   public AtomicLong shardBuffercacheLost = new AtomicLong(0);
-  public Map<String,MethodCall> methodCalls = new ConcurrentHashMap<>();
-  
-  public AtomicLong tableCount = new AtomicLong(0);
-  public AtomicLong rowCount = new AtomicLong(0);
-  public AtomicLong recordCount = new AtomicLong(0);
-  public AtomicLong indexCount = new AtomicLong(0);
-  public AtomicLong indexMemoryUsage = new AtomicLong(0);
-  public AtomicLong segmentCount = new AtomicLong(0);
+
 
   private long previous = System.nanoTime();
 
-  public static void main(String[] args) throws InterruptedException {
-    Metrics metrics = new Metrics();
-    MethodCall methodCall = new MethodCall();
-    metrics.methodCalls.put("test", methodCall);
-    for (int i = 0; i < 100; i++) {
-      metrics.blockCacheHit.incrementAndGet();
-      metrics.blockCacheMiss.incrementAndGet();
-      methodCall.invokes.incrementAndGet();
-      methodCall.times.addAndGet(56000000);
-      Thread.sleep(500);
-    }
-  }
 
   public NamedList<Number> getStatistics() {
     NamedList<Number> stats = new SimpleOrderedMap<>(21); // room for one method call before growing
-    
+
     long now = System.nanoTime();
-    float seconds = (now - previous) / 1000000000.0f;
-    
-    long hits = blockCacheHit.getAndSet(0);
-    long lookups = hits + blockCacheMiss.getAndSet(0);
-    
-    stats.add("lookups", getPerSecond(lookups, seconds));
-    stats.add("hits", getPerSecond(hits, seconds));
-    stats.add("hitratio", calcHitRatio(lookups, hits));
-    stats.add("evictions", getPerSecond(blockCacheEviction.getAndSet(0), seconds));
+    long delta = Math.max(now - previous, 1);
+    double seconds = delta / 1000000000.0;
+
+    long hits_total = blockCacheHit.get();
+    long hits_delta = hits_total - blockCacheHit_last.get();
+    blockCacheHit_last.set(hits_total);
+
+    long miss_total = blockCacheMiss.get();
+    long miss_delta = miss_total - blockCacheMiss_last.get();
+    blockCacheMiss_last.set(miss_total);
+
+    long evict_total = blockCacheEviction.get();
+    long evict_delta = evict_total - blockCacheEviction_last.get();
+    blockCacheEviction_last.set(evict_total);
+
+    long storeFail_total = blockCacheStoreFail.get();
+    long storeFail_delta = storeFail_total - blockCacheStoreFail_last.get();
+    blockCacheStoreFail_last.set(storeFail_total);
+
+    long lookups_delta = hits_delta + miss_delta;
+    long lookups_total = hits_total + miss_total;
+
     stats.add("size", blockCacheSize.get());
-    stats.add("row.reads", getPerSecond(rowReads.getAndSet(0), seconds));
-    stats.add("row.writes", getPerSecond(rowWrites.getAndSet(0), seconds));
-    stats.add("record.reads", getPerSecond(recordReads.getAndSet(0), seconds));
-    stats.add("record.writes", getPerSecond(recordWrites.getAndSet(0), seconds));
-    stats.add("query.external", getPerSecond(queriesExternal.getAndSet(0), seconds));
-    stats.add("query.internal", getPerSecond(queriesInternal.getAndSet(0), seconds));
+    stats.add("lookups", lookups_total);
+    stats.add("hits", hits_total);
+    stats.add("evictions", evict_total);
+    stats.add("storeFails", storeFail_total);
+    stats.add("hitratio_current", calcHitRatio(lookups_delta, hits_delta));  // hit ratio since the last call
+    stats.add("lookups_persec", getPerSecond(lookups_delta, seconds)); // lookups per second since the last call
+    stats.add("hits_persec", getPerSecond(hits_delta, seconds));       // hits per second since the last call
+    stats.add("evictions_persec", getPerSecond(evict_delta, seconds));  // evictions per second since the last call
+    stats.add("storeFails_persec", getPerSecond(storeFail_delta, seconds));  // evictions per second since the last call
+    stats.add("time_delta", seconds);  // seconds since last call
+
+    // TODO: these aren't really related to the BlockCache
     stats.add("buffercache.allocations", getPerSecond(shardBuffercacheAllocate.getAndSet(0), seconds));
     stats.add("buffercache.lost", getPerSecond(shardBuffercacheLost.getAndSet(0), seconds));
-    for (Entry<String,MethodCall> entry : methodCalls.entrySet()) {
-      String key = entry.getKey();
-      MethodCall value = entry.getValue();
-      long invokes = value.invokes.getAndSet(0);
-      long times = value.times.getAndSet(0);
-      
-      float avgTimes = (times / (float) invokes) / 1000000000.0f;
-      stats.add("methodcalls." + key + ".count", getPerSecond(invokes, seconds));
-      stats.add("methodcalls." + key + ".time", avgTimes);
-    }
-    stats.add("tables", tableCount.get());
-    stats.add("rows", rowCount.get());
-    stats.add("records", recordCount.get());
-    stats.add("index.count", indexCount.get());
-    stats.add("index.memoryusage", indexMemoryUsage.get());
-    stats.add("index.segments", segmentCount.get());
+
     previous = now;
-    
+
     return stats;
   }
 
-  private float getPerSecond(long value, float seconds) {
+  private float getPerSecond(long value, double seconds) {
     return (float) (value / seconds);
   }