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

[43/50] [abbrv] lucene-solr:jira/solr-9045: LUCENE-7410: Make cache keys and close listeners less trappy.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/df6f8307/solr/core/src/test/org/apache/solr/search/TestSolr4Spatial2.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/search/TestSolr4Spatial2.java b/solr/core/src/test/org/apache/solr/search/TestSolr4Spatial2.java
index 7a7dc6b..1fcfe9a 100644
--- a/solr/core/src/test/org/apache/solr/search/TestSolr4Spatial2.java
+++ b/solr/core/src/test/org/apache/solr/search/TestSolr4Spatial2.java
@@ -157,7 +157,7 @@ public class TestSolr4Spatial2 extends SolrTestCaseJ4 {
 
 
   protected Object getFirstLeafReaderKey() {
-    return getSearcher().getRawReader().leaves().get(0).reader().getCoreCacheKey();
+    return getSearcher().getRawReader().leaves().get(0).reader().getCoreCacheHelper().getKey();
   }
 
   @Test// SOLR-8541

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/df6f8307/solr/core/src/test/org/apache/solr/uninverting/TestDocTermOrds.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/uninverting/TestDocTermOrds.java b/solr/core/src/test/org/apache/solr/uninverting/TestDocTermOrds.java
index 67b62cf..69b89b4 100644
--- a/solr/core/src/test/org/apache/solr/uninverting/TestDocTermOrds.java
+++ b/solr/core/src/test/org/apache/solr/uninverting/TestDocTermOrds.java
@@ -218,7 +218,7 @@ public class TestDocTermOrds extends LuceneTestCase {
     TestUtil.checkReader(slowR);
     verify(slowR, idToOrds, termsArray, null);
 
-    FieldCache.DEFAULT.purgeByCacheKey(slowR.getCoreCacheKey());
+    FieldCache.DEFAULT.purgeByCacheKey(slowR.getCoreCacheHelper().getKey());
 
     r.close();
     dir.close();
@@ -338,7 +338,7 @@ public class TestDocTermOrds extends LuceneTestCase {
       verify(slowR, idToOrdsPrefix, termsArray, prefixRef);
     }
 
-    FieldCache.DEFAULT.purgeByCacheKey(slowR.getCoreCacheKey());
+    FieldCache.DEFAULT.purgeByCacheKey(slowR.getCoreCacheHelper().getKey());
 
     r.close();
     dir.close();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/df6f8307/solr/core/src/test/org/apache/solr/uninverting/TestFieldCache.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/uninverting/TestFieldCache.java b/solr/core/src/test/org/apache/solr/uninverting/TestFieldCache.java
index 2d2c381..60bb9e8 100644
--- a/solr/core/src/test/org/apache/solr/uninverting/TestFieldCache.java
+++ b/solr/core/src/test/org/apache/solr/uninverting/TestFieldCache.java
@@ -267,7 +267,7 @@ public class TestFieldCache extends LuceneTestCase {
     termOrds = cache.getDocTermOrds(reader, "bogusfield", null);
     assertTrue(termOrds.getValueCount() == 0);
 
-    FieldCache.DEFAULT.purgeByCacheKey(reader.getCoreCacheKey());
+    FieldCache.DEFAULT.purgeByCacheKey(reader.getCoreCacheHelper().getKey());
   }
 
   public void testEmptyIndex() throws Exception {
@@ -279,7 +279,7 @@ public class TestFieldCache extends LuceneTestCase {
     TestUtil.checkReader(reader);
     FieldCache.DEFAULT.getTerms(reader, "foobar");
     FieldCache.DEFAULT.getTermsIndex(reader, "foobar");
-    FieldCache.DEFAULT.purgeByCacheKey(reader.getCoreCacheKey());
+    FieldCache.DEFAULT.purgeByCacheKey(reader.getCoreCacheHelper().getKey());
     r.close();
     dir.close();
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/df6f8307/solr/core/src/test/org/apache/solr/uninverting/TestFieldCacheSanityChecker.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/uninverting/TestFieldCacheSanityChecker.java b/solr/core/src/test/org/apache/solr/uninverting/TestFieldCacheSanityChecker.java
deleted file mode 100644
index b031681..0000000
--- a/solr/core/src/test/org/apache/solr/uninverting/TestFieldCacheSanityChecker.java
+++ /dev/null
@@ -1,164 +0,0 @@
-/*
- * 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.
- */
-package org.apache.solr.uninverting;
-
-import java.io.IOException;
-
-import org.apache.lucene.analysis.MockAnalyzer;
-import org.apache.lucene.document.Document;
-import org.apache.lucene.document.Field;
-import org.apache.lucene.index.DirectoryReader;
-import org.apache.lucene.index.IndexWriter;
-import org.apache.lucene.index.LeafReader;
-import org.apache.lucene.index.MultiReader;
-import org.apache.lucene.legacy.LegacyDoubleField;
-import org.apache.lucene.legacy.LegacyFloatField;
-import org.apache.lucene.legacy.LegacyIntField;
-import org.apache.lucene.legacy.LegacyLongField;
-import org.apache.lucene.store.Directory;
-import org.apache.lucene.util.LuceneTestCase;
-import org.apache.solr.index.SlowCompositeReaderWrapper;
-import org.apache.solr.uninverting.FieldCacheSanityChecker.Insanity;
-import org.apache.solr.uninverting.FieldCacheSanityChecker.InsanityType;
-
-public class TestFieldCacheSanityChecker extends LuceneTestCase {
-
-  protected LeafReader readerA;
-  protected LeafReader readerB;
-  protected LeafReader readerX;
-  protected LeafReader readerAclone;
-  protected Directory dirA, dirB;
-  private static final int NUM_DOCS = 1000;
-
-  @Override
-  public void setUp() throws Exception {
-    super.setUp();
-    dirA = newDirectory();
-    dirB = newDirectory();
-
-    IndexWriter wA = new IndexWriter(dirA, newIndexWriterConfig(new MockAnalyzer(random())));
-    IndexWriter wB = new IndexWriter(dirB, newIndexWriterConfig(new MockAnalyzer(random())));
-
-    long theLong = Long.MAX_VALUE;
-    double theDouble = Double.MAX_VALUE;
-    int theInt = Integer.MAX_VALUE;
-    float theFloat = Float.MAX_VALUE;
-    for (int i = 0; i < NUM_DOCS; i++){
-      Document doc = new Document();
-      doc.add(new LegacyLongField("theLong", theLong--, Field.Store.NO));
-      doc.add(new LegacyDoubleField("theDouble", theDouble--, Field.Store.NO));
-      doc.add(new LegacyIntField("theInt", theInt--, Field.Store.NO));
-      doc.add(new LegacyFloatField("theFloat", theFloat--, Field.Store.NO));
-      if (0 == i % 3) {
-        wA.addDocument(doc);
-      } else {
-        wB.addDocument(doc);
-      }
-    }
-    wA.close();
-    wB.close();
-    DirectoryReader rA = DirectoryReader.open(dirA);
-    readerA = SlowCompositeReaderWrapper.wrap(rA);
-    readerAclone = SlowCompositeReaderWrapper.wrap(rA);
-    readerA = SlowCompositeReaderWrapper.wrap(DirectoryReader.open(dirA));
-    readerB = SlowCompositeReaderWrapper.wrap(DirectoryReader.open(dirB));
-    readerX = SlowCompositeReaderWrapper.wrap(new MultiReader(readerA, readerB));
-  }
-
-  @Override
-  public void tearDown() throws Exception {
-    readerA.close();
-    readerAclone.close();
-    readerB.close();
-    readerX.close();
-    dirA.close();
-    dirB.close();
-    super.tearDown();
-  }
-
-  public void testSanity() throws IOException {
-    FieldCache cache = FieldCache.DEFAULT;
-    cache.purgeAllCaches();
-
-    cache.getNumerics(readerA, "theDouble", FieldCache.LEGACY_DOUBLE_PARSER);
-    cache.getNumerics(readerAclone, "theDouble", FieldCache.LEGACY_DOUBLE_PARSER);
-    cache.getNumerics(readerB, "theDouble", FieldCache.LEGACY_DOUBLE_PARSER);
-
-    cache.getNumerics(readerX, "theInt", FieldCache.LEGACY_INT_PARSER);
-
-    // // // 
-
-    Insanity[] insanity = 
-      FieldCacheSanityChecker.checkSanity(cache.getCacheEntries());
-    
-    if (0 < insanity.length)
-      dumpArray(getTestClass().getName() + "#" + getTestName() 
-          + " INSANITY", insanity, System.err);
-
-    assertEquals("shouldn't be any cache insanity", 0, insanity.length);
-    cache.purgeAllCaches();
-  }
-
-  public void testInsanity1() throws IOException {
-    FieldCache cache = FieldCache.DEFAULT;
-    cache.purgeAllCaches();
-
-    cache.getNumerics(readerX, "theInt", FieldCache.LEGACY_INT_PARSER);
-    cache.getTerms(readerX, "theInt");
-
-    // // // 
-
-    Insanity[] insanity = 
-      FieldCacheSanityChecker.checkSanity(cache.getCacheEntries());
-
-    assertEquals("wrong number of cache errors", 1, insanity.length);
-    assertEquals("wrong type of cache error", 
-                 InsanityType.VALUEMISMATCH,
-                 insanity[0].getType());
-    assertEquals("wrong number of entries in cache error", 2,
-                 insanity[0].getCacheEntries().length);
-
-    // we expect bad things, don't let tearDown complain about them
-    cache.purgeAllCaches();
-  }
-
-  public void testInsanity2() throws IOException {
-    FieldCache cache = FieldCache.DEFAULT;
-    cache.purgeAllCaches();
-
-    cache.getTerms(readerA, "theInt");
-    cache.getTerms(readerB, "theInt");
-    cache.getTerms(readerX, "theInt");
-
-
-    // // // 
-
-    Insanity[] insanity = 
-      FieldCacheSanityChecker.checkSanity(cache.getCacheEntries());
-    
-    assertEquals("wrong number of cache errors", 1, insanity.length);
-    assertEquals("wrong type of cache error", 
-                 InsanityType.SUBREADER,
-                 insanity[0].getType());
-    assertEquals("wrong number of entries in cache error", 3,
-                 insanity[0].getCacheEntries().length);
-
-    // we expect bad things, don't let tearDown complain about them
-    cache.purgeAllCaches();
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/df6f8307/solr/core/src/test/org/apache/solr/uninverting/TestLegacyFieldCache.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/uninverting/TestLegacyFieldCache.java b/solr/core/src/test/org/apache/solr/uninverting/TestLegacyFieldCache.java
index 9dc047b..e38e193 100644
--- a/solr/core/src/test/org/apache/solr/uninverting/TestLegacyFieldCache.java
+++ b/solr/core/src/test/org/apache/solr/uninverting/TestLegacyFieldCache.java
@@ -32,26 +32,20 @@ import org.apache.lucene.index.IndexWriterConfig;
 import org.apache.lucene.index.LeafReader;
 import org.apache.lucene.index.NumericDocValues;
 import org.apache.lucene.index.RandomIndexWriter;
-import org.apache.lucene.index.Terms;
-import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.legacy.LegacyDoubleField;
 import org.apache.lucene.legacy.LegacyFloatField;
 import org.apache.lucene.legacy.LegacyIntField;
 import org.apache.lucene.legacy.LegacyLongField;
-import org.apache.lucene.legacy.LegacyNumericUtils;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.TestUtil;
 import org.apache.solr.index.SlowCompositeReaderWrapper;
 import org.junit.AfterClass;
 import org.junit.BeforeClass;
 
-import java.io.ByteArrayOutputStream;
 import java.io.IOException;
-import java.io.PrintStream;
 import java.util.HashSet;
 import java.util.Set;
 import java.util.concurrent.CyclicBarrier;
@@ -106,31 +100,6 @@ public class TestLegacyFieldCache extends LuceneTestCase {
     directory.close();
     directory = null;
   }
-  
-  public void testInfoStream() throws Exception {
-    try {
-      FieldCache cache = FieldCache.DEFAULT;
-      ByteArrayOutputStream bos = new ByteArrayOutputStream(1024);
-      cache.setInfoStream(new PrintStream(bos, false, IOUtils.UTF_8));
-      cache.getNumerics(reader, "theDouble", FieldCache.LEGACY_DOUBLE_PARSER);
-      cache.getNumerics(reader, "theDouble", new FieldCache.Parser() {
-        @Override
-        public TermsEnum termsEnum(Terms terms) throws IOException {
-          return LegacyNumericUtils.filterPrefixCodedLongs(terms.iterator());
-        }
-        @Override
-        public long parseValue(BytesRef term) {
-          int val = (int) LegacyNumericUtils.prefixCodedToLong(term);
-          if (val<0) val ^= 0x7fffffff;
-          return val;
-        }
-      });
-      assertTrue(bos.toString(IOUtils.UTF_8).indexOf("WARNING") != -1);
-    } finally {
-      FieldCache.DEFAULT.setInfoStream(null);
-      FieldCache.DEFAULT.purgeAllCaches();
-    }
-  }
 
   public void test() throws IOException {
     FieldCache cache = FieldCache.DEFAULT;
@@ -174,7 +143,7 @@ public class TestLegacyFieldCache extends LuceneTestCase {
       assertEquals(i%2 == 0, docsWithField.get(i));
     }
 
-    FieldCache.DEFAULT.purgeByCacheKey(reader.getCoreCacheKey());
+    FieldCache.DEFAULT.purgeByCacheKey(reader.getCoreCacheHelper().getKey());
   }
 
   public void testEmptyIndex() throws Exception {
@@ -186,7 +155,7 @@ public class TestLegacyFieldCache extends LuceneTestCase {
     TestUtil.checkReader(reader);
     FieldCache.DEFAULT.getTerms(reader, "foobar");
     FieldCache.DEFAULT.getTermsIndex(reader, "foobar");
-    FieldCache.DEFAULT.purgeByCacheKey(reader.getCoreCacheKey());
+    FieldCache.DEFAULT.purgeByCacheKey(reader.getCoreCacheHelper().getKey());
     r.close();
     dir.close();
   }