You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by sh...@apache.org on 2013/10/23 22:40:33 UTC

svn commit: r1535163 - in /lucene/dev/trunk/lucene: CHANGES.txt facet/src/java/org/apache/lucene/facet/search/OrdinalsCache.java facet/src/test/org/apache/lucene/facet/search/OrdinalsCacheTest.java

Author: shaie
Date: Wed Oct 23 20:40:33 2013
New Revision: 1535163

URL: http://svn.apache.org/r1535163
Log:
LUCENE-5303: OrdinalsCache should use reader.getCoreCacheKey()

Added:
    lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/search/OrdinalsCacheTest.java   (with props)
Modified:
    lucene/dev/trunk/lucene/CHANGES.txt
    lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/OrdinalsCache.java

Modified: lucene/dev/trunk/lucene/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/CHANGES.txt?rev=1535163&r1=1535162&r2=1535163&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/CHANGES.txt (original)
+++ lucene/dev/trunk/lucene/CHANGES.txt Wed Oct 23 20:40:33 2013
@@ -155,6 +155,9 @@ Bug Fixes
   when there were buffered delete-by-Term.  (Shalin Shekhar Mangar,
   Mike McCandless)
 
+* LUCENE-5303: OrdinalsCache did not use coreCacheKey, resulting in
+  over caching across multiple threads. (Mike McCandless, Shai Erera)
+
 API Changes:
 
 * LUCENE-5222: Add SortField.needsScores(). Previously it was not possible

Modified: lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/OrdinalsCache.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/OrdinalsCache.java?rev=1535163&r1=1535162&r2=1535163&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/OrdinalsCache.java (original)
+++ lucene/dev/trunk/lucene/facet/src/java/org/apache/lucene/facet/search/OrdinalsCache.java Wed Oct 23 20:40:33 2013
@@ -1,6 +1,7 @@
 package org.apache.lucene.facet.search;
 
 import java.io.IOException;
+import java.util.HashMap;
 import java.util.Map;
 import java.util.WeakHashMap;
 
@@ -12,6 +13,7 @@ import org.apache.lucene.index.BinaryDoc
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.IntsRef;
+import org.apache.lucene.util.RamUsageEstimator;
 
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
@@ -94,7 +96,10 @@ public class OrdinalsCache {
     }
   }
 
-  private static final Map<BinaryDocValues,CachedOrds> intsCache = new WeakHashMap<BinaryDocValues,CachedOrds>();
+  // outer map is a WeakHashMap which uses reader.getCoreCacheKey() as the weak
+  // reference. When it's no longer referenced, the entire inner map can be
+  // evicted.
+  private static final Map<Object,Map<String,CachedOrds>> ordsCache = new WeakHashMap<Object,Map<String,CachedOrds>>();
   
   /**
    * Returns the {@link CachedOrds} relevant to the given
@@ -107,12 +112,33 @@ public class OrdinalsCache {
     if (dv == null) {
       return null;
     }
-    CachedOrds ci = intsCache.get(dv);
-    if (ci == null) {
-      ci = new CachedOrds(dv, context.reader().maxDoc(), clp);
-      intsCache.put(dv, ci);
+    Map<String,CachedOrds> fieldCache = ordsCache.get(context.reader().getCoreCacheKey());
+    if (fieldCache == null) {
+      fieldCache = new HashMap<String,OrdinalsCache.CachedOrds>();
+      ordsCache.put(context.reader().getCoreCacheKey(), fieldCache);
     }
-    return ci;
+    CachedOrds co = fieldCache.get(clp.field);
+    if (co == null) {
+      co = new CachedOrds(dv, context.reader().maxDoc(), clp);
+      fieldCache.put(clp.field, co);
+    }
+    return co;
   }
 
+  /** Returns how many bytes the static ords cache is
+   *  consuming. */
+  public synchronized static long ramBytesUsed() {
+    long size = 0;
+    for (Map<String,CachedOrds> e : ordsCache.values()) {
+      for (CachedOrds co : e.values()) {
+        size += RamUsageEstimator.NUM_BYTES_OBJECT_REF              // CachedOrds reference in the map
+            + RamUsageEstimator.NUM_BYTES_OBJECT_HEADER             // CachedOrds object header
+            + RamUsageEstimator.NUM_BYTES_ARRAY_HEADER * 2          // 2 int[] (header)
+            + RamUsageEstimator.NUM_BYTES_OBJECT_REF * 2            // 2 int[] (ref)
+            + RamUsageEstimator.NUM_BYTES_INT * co.offsets.length   // sizeOf(offsets)
+            + RamUsageEstimator.NUM_BYTES_INT * co.ordinals.length; // sizeOf(ordinals)
+      }
+    }
+    return size;
+  }
 }

Added: lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/search/OrdinalsCacheTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/search/OrdinalsCacheTest.java?rev=1535163&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/search/OrdinalsCacheTest.java (added)
+++ lucene/dev/trunk/lucene/facet/src/test/org/apache/lucene/facet/search/OrdinalsCacheTest.java Wed Oct 23 20:40:33 2013
@@ -0,0 +1,92 @@
+package org.apache.lucene.facet.search;
+
+import java.io.IOException;
+import java.util.Arrays;
+
+import org.apache.lucene.analysis.MockAnalyzer;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.facet.FacetTestCase;
+import org.apache.lucene.facet.index.FacetFields;
+import org.apache.lucene.facet.params.FacetIndexingParams;
+import org.apache.lucene.facet.taxonomy.CategoryPath;
+import org.apache.lucene.facet.taxonomy.directory.DirectoryTaxonomyWriter;
+import org.apache.lucene.index.AtomicReaderContext;
+import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.IndexWriter;
+import org.apache.lucene.index.IndexWriterConfig;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.IOUtils;
+import org.junit.Test;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+public class OrdinalsCacheTest extends FacetTestCase {
+
+  @Test
+  public void testOrdinalsCacheWithThreads() throws Exception {
+    // LUCENE-5303: OrdinalsCache used the ThreadLocal BinaryDV instead of reader.getCoreCacheKey().
+    Directory indexDir = newDirectory();
+    Directory taxoDir = newDirectory();
+    IndexWriterConfig conf = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random()));
+    IndexWriter writer = new IndexWriter(indexDir, conf);
+    DirectoryTaxonomyWriter taxoWriter = new DirectoryTaxonomyWriter(taxoDir);
+    FacetFields facetFields = new FacetFields(taxoWriter);
+    
+    Document doc = new Document();
+    facetFields.addFields(doc, Arrays.asList(new CategoryPath("A", "1")));
+    writer.addDocument(doc);
+    doc = new Document();
+    facetFields.addFields(doc, Arrays.asList(new CategoryPath("A", "2")));
+    writer.addDocument(doc);
+    writer.close();
+    taxoWriter.close();
+    
+    final DirectoryReader reader = DirectoryReader.open(indexDir);
+    Thread[] threads = new Thread[3];
+    for (int i = 0; i < threads.length; i++) {
+      threads[i] = new Thread("CachedOrdsThread-" + i) {
+        @Override
+        public void run() {
+          for (AtomicReaderContext context : reader.leaves()) {
+            try {
+              OrdinalsCache.getCachedOrds(context, FacetIndexingParams.DEFAULT.getCategoryListParams(new CategoryPath("A")));
+            } catch (IOException e) {
+              throw new RuntimeException(e);
+            }
+          }
+        }
+      };
+    }
+    
+    long ramBytesUsed = 0;
+    for (Thread t : threads) {
+      t.start();
+      t.join();
+      if (ramBytesUsed == 0) {
+        ramBytesUsed = OrdinalsCache.ramBytesUsed();
+      } else {
+        assertEquals(ramBytesUsed, OrdinalsCache.ramBytesUsed());
+      }
+    }
+    
+    reader.close();
+    
+    IOUtils.close(indexDir, taxoDir);
+  }
+  
+}