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

[2/4] lucene-solr:branch_6x: SOLR-6246: SolrSuggester.build() now throws SolrCoreState.CoreIsClosedException when a core reload/shutdown happens; add a random test lookup dictionary with configurable size; add {Analyzing,Blended}InfixSuggester reload/bui

SOLR-6246: SolrSuggester.build() now throws SolrCoreState.CoreIsClosedException when a core reload/shutdown happens; add a random test lookup dictionary with configurable size; add {Analyzing,Blended}InfixSuggester reload/build tests; add a wrapped-exception expectThrows() variant to LuceneTestCase


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

Branch: refs/heads/branch_6x
Commit: 7b081e468a97b353a5e096ed69163ee9c3044925
Parents: 64b1d24
Author: Steve Rowe <sa...@apache.org>
Authored: Fri Feb 3 18:25:38 2017 -0500
Committer: Steve Rowe <sa...@apache.org>
Committed: Fri Feb 3 18:26:10 2017 -0500

----------------------------------------------------------------------
 .../org/apache/lucene/util/LuceneTestCase.java  |  28 ++++
 .../solr/spelling/suggest/SolrSuggester.java    |  11 +-
 .../conf/solrconfig-infixsuggesters.xml         | 101 ++++++++++++++
 .../handler/component/InfixSuggestersTest.java  | 136 +++++++++++++++++++
 .../suggest/RandomTestDictionaryFactory.java    | 117 ++++++++++++++++
 5 files changed, 392 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7b081e46/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java
----------------------------------------------------------------------
diff --git a/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java b/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java
index 23c47a6..3034980 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java
@@ -2704,6 +2704,34 @@ public abstract class LuceneTestCase extends Assert {
     throw new AssertionFailedError("Expected exception " + expectedType.getSimpleName());
   }
 
+  /**
+   * Checks that specific wrapped and outer exception classes are thrown
+   * by the given runnable, and returns the wrapped exception. 
+   */
+  public static <TO extends Throwable, TW extends Throwable> TW expectThrows
+  (Class<TO> expectedOuterType, Class<TW> expectedWrappedType, ThrowingRunnable runnable) {
+    try {
+      runnable.run();
+    } catch (Throwable e) {
+      if (expectedOuterType.isInstance(e)) {
+        Throwable cause = e.getCause();
+        if (expectedWrappedType.isInstance(cause)) {
+          return expectedWrappedType.cast(cause);
+        } else {
+          AssertionFailedError assertion = new AssertionFailedError
+              ("Unexpected wrapped exception type, expected " + expectedWrappedType.getSimpleName());
+          assertion.initCause(e);
+          throw assertion;
+        }
+      }
+      AssertionFailedError assertion = new AssertionFailedError
+          ("Unexpected outer exception type, expected " + expectedOuterType.getSimpleName());
+      assertion.initCause(e);
+      throw assertion;
+    }
+    throw new AssertionFailedError("Expected outer exception " + expectedOuterType.getSimpleName());
+  }
+
   /** Returns true if the file exists (can be opened), false
    *  if it cannot be opened, and (unlike Java's
    *  File.exists) throws IOException if there's some

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7b081e46/solr/core/src/java/org/apache/solr/spelling/suggest/SolrSuggester.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/spelling/suggest/SolrSuggester.java b/solr/core/src/java/org/apache/solr/spelling/suggest/SolrSuggester.java
index b4ca4da..267d9ad 100644
--- a/solr/core/src/java/org/apache/solr/spelling/suggest/SolrSuggester.java
+++ b/solr/core/src/java/org/apache/solr/spelling/suggest/SolrSuggester.java
@@ -36,6 +36,7 @@ import org.apache.lucene.search.Query;
 import org.apache.lucene.search.spell.Dictionary;
 import org.apache.lucene.search.suggest.Lookup;
 import org.apache.lucene.search.suggest.Lookup.LookupResult;
+import org.apache.lucene.store.AlreadyClosedException;
 import org.apache.lucene.util.Accountable;
 import org.apache.lucene.util.IOUtils;
 import org.apache.solr.analysis.TokenizerChain;
@@ -43,6 +44,7 @@ import org.apache.solr.common.util.NamedList;
 import org.apache.solr.core.CloseHook;
 import org.apache.solr.core.SolrCore;
 import org.apache.solr.search.SolrIndexSearcher;
+import org.apache.solr.update.SolrCoreState;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -175,7 +177,14 @@ public class SolrSuggester implements Accountable {
     LOG.info("SolrSuggester.build(" + name + ")");
 
     dictionary = dictionaryFactory.create(core, searcher);
-    lookup.build(dictionary);
+    try {
+      lookup.build(dictionary);
+    } catch (AlreadyClosedException e) {
+      RuntimeException e2 = new SolrCoreState.CoreIsClosedException
+          ("Suggester build has been interrupted by a core reload or shutdown.");
+      e2.initCause(e);
+      throw e2;
+    }
     if (storeDir != null) {
       File target = getStoreFile();
       if(!lookup.store(new FileOutputStream(target))) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7b081e46/solr/core/src/test-files/solr/collection1/conf/solrconfig-infixsuggesters.xml
----------------------------------------------------------------------
diff --git a/solr/core/src/test-files/solr/collection1/conf/solrconfig-infixsuggesters.xml b/solr/core/src/test-files/solr/collection1/conf/solrconfig-infixsuggesters.xml
new file mode 100644
index 0000000..8fc42ff
--- /dev/null
+++ b/solr/core/src/test-files/solr/collection1/conf/solrconfig-infixsuggesters.xml
@@ -0,0 +1,101 @@
+<?xml version="1.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.
+-->
+
+<config>
+  <xi:include href="solrconfig.snippet.randomindexconfig.xml" xmlns:xi="http://www.w3.org/2001/XInclude"/>
+  <luceneMatchVersion>${tests.luceneMatchVersion:LATEST}</luceneMatchVersion>
+  <dataDir>${solr.data.dir:}</dataDir>
+  <directoryFactory name="DirectoryFactory" class="solr.NRTCachingDirectoryFactory"/>
+  <schemaFactory class="ClassicIndexSchemaFactory"/>
+
+  <updateHandler class="solr.DirectUpdateHandler2"/>
+
+  <requestHandler name="standard" class="solr.StandardRequestHandler" />
+
+  <searchComponent class="solr.SuggestComponent" name="analyzing_infix_suggester_random_short_dictionary">
+    <lst name="suggester">
+      <str name="name">shortRandomAnalyzingInfixSuggester</str>
+      <str name="lookupImpl">AnalyzingInfixLookupFactory</str>
+      <str name="dictionaryImpl">RandomTestDictionaryFactory</str>
+      <long name="randDictMaxItems">100</long>
+      <str name="indexPath">short_random_analyzing_infix_suggester</str>
+      <str name="suggestAnalyzerFieldType">text</str>
+      <str name="buildOnCommit">false</str>
+      <str name="buildOnStartup">false</str>
+    </lst>
+  </searchComponent>
+  <requestHandler name="/suggest_analyzing_infix_short_dictionary" class="solr.SearchHandler">
+    <lst name="defaults">
+      <str name="suggest">true</str>
+    </lst>
+    <arr name="components">
+      <str>analyzing_infix_suggester_random_short_dictionary</str>
+    </arr>
+  </requestHandler>
+
+  <searchComponent class="solr.SuggestComponent" name="analyzing_infix_suggester_random_long_dictionary">
+    <lst name="suggester">
+      <str name="name">longRandomAnalyzingInfixSuggester</str>
+      <str name="lookupImpl">AnalyzingInfixLookupFactory</str>
+      <str name="dictionaryImpl">RandomTestDictionaryFactory</str>
+      <long name="randDictMaxItems">100000</long>
+      <str name="indexPath">long_random_analyzing_infix_suggester</str>
+      <str name="suggestAnalyzerFieldType">text</str>
+      <str name="buildOnCommit">false</str>
+      <str name="buildOnStartup">false</str>
+    </lst>
+  </searchComponent>
+  <requestHandler name="/suggest_analyzing_infix_long_dictionary" class="solr.SearchHandler">
+    <lst name="defaults">
+      <str name="suggest">true</str>
+    </lst>
+    <arr name="components">
+      <str>analyzing_infix_suggester_random_long_dictionary</str>
+    </arr>
+  </requestHandler>
+
+  <searchComponent class="solr.SuggestComponent" name="blended_infix_suggester_random_short_dictionary">
+    <lst name="suggester">
+      <str name="name">shortRandomBlendedInfixSuggester</str>
+      <str name="lookupImpl">BlendedInfixLookupFactory</str>
+      <str name="dictionaryImpl">RandomTestDictionaryFactory</str>
+      <long name="randDictMaxItems">100</long>
+      <str name="blenderType">position_linear</str>
+      <str name="suggestAnalyzerFieldType">text</str>
+      <str name="indexPath">short_random_blended_infix_suggester</str>
+      <str name="highlight">true</str>
+      <str name="buildOnStartup">false</str>
+      <str name="buildOnCommit">false</str>
+    </lst>
+  </searchComponent>
+  <requestHandler name="/suggest_blended_infix_short_dictionary" class="solr.SearchHandler">
+    <lst name="defaults">
+      <str name="suggest">true</str>
+      <str name="suggest.dictionary">shortRandomBlendedInfixSuggester</str>
+      <str name="suggest.onlyMorePopular">true</str>
+      <str name="suggest.count">10</str>
+      <str name="suggest.collate">true</str>
+    </lst>
+    <arr name="components">
+      <str>blended_infix_suggester_random_short_dictionary</str>
+    </arr>
+  </requestHandler>
+
+  <query><useColdSearcher>false</useColdSearcher></query>
+
+</config>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7b081e46/solr/core/src/test/org/apache/solr/handler/component/InfixSuggestersTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/handler/component/InfixSuggestersTest.java b/solr/core/src/test/org/apache/solr/handler/component/InfixSuggestersTest.java
new file mode 100644
index 0000000..a8188bb
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/handler/component/InfixSuggestersTest.java
@@ -0,0 +1,136 @@
+/*
+ * 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.handler.component;
+
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.common.util.ExecutorUtil;
+import org.apache.solr.spelling.suggest.RandomTestDictionaryFactory;
+import org.apache.solr.spelling.suggest.SuggesterParams;
+import org.apache.solr.update.SolrCoreState;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class InfixSuggestersTest extends SolrTestCaseJ4 {
+  private static final String rh_analyzing_short = "/suggest_analyzing_infix_short_dictionary";
+  private static final String rh_analyzing_long = "/suggest_analyzing_infix_long_dictionary";
+  private static final String rh_blended_short = "/suggest_blended_infix_short_dictionary";
+
+  @BeforeClass
+  public static void beforeClass() throws Exception {
+    initCore("solrconfig-infixsuggesters.xml","schema.xml");
+  }
+
+  @Test
+  public void test2xBuildReload() throws Exception {
+    for (int i = 0 ; i < 2 ; ++i) {
+      assertQ(req("qt", rh_analyzing_short,
+          SuggesterParams.SUGGEST_BUILD_ALL, "true"),
+          "//str[@name='command'][.='buildAll']"
+      );
+      h.reload();
+    }
+  }
+
+  @Test
+  public void testTwoSuggestersBuildThenReload() throws Exception {
+    assertQ(req("qt", rh_analyzing_short,
+        SuggesterParams.SUGGEST_BUILD_ALL, "true"),
+        "//str[@name='command'][.='buildAll']"
+    );
+    h.reload();
+
+    assertQ(req("qt", rh_blended_short,
+        SuggesterParams.SUGGEST_BUILD_ALL, "true"),
+        "//str[@name='command'][.='buildAll']"
+    );
+    h.reload();
+  }
+
+  @Test
+  public void testBuildThen2xReload() throws Exception {
+    assertQ(req("qt", rh_analyzing_short,
+        SuggesterParams.SUGGEST_BUILD_ALL, "true"),
+        "//str[@name='command'][.='buildAll']"
+    );
+    h.reload();
+    h.reload();
+  }
+
+  @Test
+  public void testAnalyzingInfixSuggesterBuildThenReload() throws Exception {
+    assertQ(req("qt", rh_analyzing_short,
+        SuggesterParams.SUGGEST_BUILD_ALL, "true"),
+        "//str[@name='command'][.='buildAll']"
+    );
+    h.reload();
+  }
+
+  @Test
+  public void testBlendedInfixSuggesterBuildThenReload() throws Exception {
+    assertQ(req("qt", rh_blended_short,
+        SuggesterParams.SUGGEST_BUILD_ALL, "true"),
+        "//str[@name='command'][.='buildAll']"
+    );
+    h.reload();
+  }
+
+  @Test
+  public void testReloadDuringBuild() throws Exception {
+    ExecutorService executor = ExecutorUtil.newMDCAwareCachedThreadPool("AnalyzingInfixSuggesterTest");
+    try {
+      // Build the suggester in the background with a long dictionary
+      Future job = executor.submit(() ->
+          expectThrows(RuntimeException.class, SolrCoreState.CoreIsClosedException.class,
+              () -> assertQ(req("qt", rh_analyzing_long,
+                  SuggesterParams.SUGGEST_BUILD_ALL, "true"),
+                  "//str[@name='command'][.='buildAll']")));
+      h.reload();
+      // Stop the dictionary's input iterator
+      System.clearProperty(RandomTestDictionaryFactory.RandomTestDictionary
+          .getEnabledSysProp("longRandomAnalyzingInfixSuggester"));
+      job.get();
+    } finally {
+      ExecutorUtil.shutdownAndAwaitTermination(executor);
+    }
+  }
+
+  @Test
+  public void testShutdownDuringBuild() throws Exception {
+    ExecutorService executor = ExecutorUtil.newMDCAwareCachedThreadPool("AnalyzingInfixSuggesterTest");
+    try {
+      // Build the suggester in the background with a long dictionary
+      Future job = executor.submit(() -> 
+          expectThrows(RuntimeException.class, SolrCoreState.CoreIsClosedException.class,
+              () -> assertQ(req("qt", rh_analyzing_long,
+                  SuggesterParams.SUGGEST_BUILD_ALL, "true"),
+                  "//str[@name='command'][.='buildAll']")));
+      Thread.sleep(100); // TODO: is there a better way to ensure that the build has begun?
+      h.close();
+      // Stop the dictionary's input iterator
+      System.clearProperty(RandomTestDictionaryFactory.RandomTestDictionary
+          .getEnabledSysProp("longRandomAnalyzingInfixSuggester"));
+      job.get();
+    } finally {
+      ExecutorUtil.shutdownAndAwaitTermination(executor);
+      initCore("solrconfig-infixsuggesters.xml","schema.xml"); // put the core back for other tests
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/7b081e46/solr/core/src/test/org/apache/solr/spelling/suggest/RandomTestDictionaryFactory.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/spelling/suggest/RandomTestDictionaryFactory.java b/solr/core/src/test/org/apache/solr/spelling/suggest/RandomTestDictionaryFactory.java
new file mode 100644
index 0000000..6e3172d
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/spelling/suggest/RandomTestDictionaryFactory.java
@@ -0,0 +1,117 @@
+/*
+ * 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.spelling.suggest;
+
+import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+
+import org.apache.lucene.search.spell.Dictionary;
+import org.apache.lucene.search.suggest.InputIterator;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefIterator;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.TestUtil;
+import org.apache.solr.common.params.CommonParams;
+import org.apache.solr.core.SolrCore;
+import org.apache.solr.search.SolrIndexSearcher;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Factory for a dictionary with an iterator over bounded-length random strings (with fixed
+ * weight of 1 and null payloads) that only operates when RandomDictionary.enabledSysProp
+ * is set; this will be true from the time a RandomDictionary has been constructed until
+ * its enabledSysProp has been cleared.
+ */
+public class RandomTestDictionaryFactory extends DictionaryFactory {
+  public static final String RAND_DICT_MAX_ITEMS = "randDictMaxItems";
+
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+  private static final long DEFAULT_MAX_ITEMS = 100_000_000;
+
+  @Override
+  public RandomTestDictionary create(SolrCore core, SolrIndexSearcher searcher) {
+    if(params == null) {
+      // should not happen; implies setParams was not called
+      throw new IllegalStateException("Value of params not set");
+    }
+    String name = (String)params.get(CommonParams.NAME);
+    if (name == null) { // Shouldn't happen since this is the component name
+      throw new IllegalArgumentException(CommonParams.NAME + " is a mandatory parameter");
+    }
+    long maxItems = DEFAULT_MAX_ITEMS;
+    Object specifiedMaxItems = params.get(RAND_DICT_MAX_ITEMS);
+    if (specifiedMaxItems != null) {
+      maxItems = Long.parseLong(specifiedMaxItems.toString());
+    }
+    return new RandomTestDictionary(name, maxItems);
+  }
+
+  public static class RandomTestDictionary implements Dictionary {
+    private static final String SYS_PROP_PREFIX = RandomTestDictionary.class.getName() + ".enabled.";
+    private final String enabledSysProp; // Clear this property to stop the input iterator
+    private final long maxItems;
+    private long emittedItems = 0L;
+
+    RandomTestDictionary(String name, long maxItems) {
+      enabledSysProp = getEnabledSysProp(name);
+      this.maxItems = maxItems;
+      synchronized (RandomTestDictionary.class) {
+        if (System.getProperty(enabledSysProp) != null) {
+          throw new RuntimeException("System property '" + enabledSysProp + "' is already in use.");
+        }
+        System.setProperty(enabledSysProp, "true");
+      }
+    }
+
+    public static String getEnabledSysProp(String suggesterName) {
+      return SYS_PROP_PREFIX + suggesterName;
+    }
+
+    @Override
+    public InputIterator getEntryIterator() throws IOException {
+      return new InputIterator.InputIteratorWrapper(new RandomByteRefIterator());
+    }
+
+    private class RandomByteRefIterator implements BytesRefIterator {
+      private static final int MAX_LENGTH = 100;
+
+      @Override
+      public BytesRef next() throws IOException {
+        BytesRef next = null;
+        if (System.getProperty(enabledSysProp) != null) {
+          if (emittedItems < maxItems) {
+            ++emittedItems;
+            next = new BytesRef(TestUtil.randomUnicodeString(LuceneTestCase.random(), MAX_LENGTH));
+            if (emittedItems % 1000 == 0) {
+              log.info(enabledSysProp + " emitted " + emittedItems + " items.");
+            }
+          } else {
+            log.info(enabledSysProp + " disabled after emitting " + emittedItems + " items.");
+            System.clearProperty(enabledSysProp); // disable once maxItems has been reached
+            emittedItems = 0L;
+          }
+        } else {
+          log.warn(enabledSysProp + " invoked when disabled");
+          emittedItems = 0L;
+        }
+        return next;
+      }
+    }
+  }
+}