You are viewing a plain text version of this content. The canonical link for it is here.
Posted to solr-commits@lucene.apache.org by ma...@apache.org on 2009/06/08 16:56:31 UTC

svn commit: r782660 - in /lucene/solr/trunk: ./ example/solr/conf/ src/java/org/apache/solr/core/ src/java/org/apache/solr/search/ src/test/org/apache/solr/core/ src/test/test-files/solr/conf/

Author: markrmiller
Date: Mon Jun  8 14:56:31 2009
New Revision: 782660

URL: http://svn.apache.org/viewvc?rev=782660&view=rev
Log:
SOLR-243: Add configurable IndexReaderFactory so that alternate IndexReader implementations can be specified via solrconfig.xml. 

Added:
    lucene/solr/trunk/src/java/org/apache/solr/core/IndexReaderFactory.java
    lucene/solr/trunk/src/java/org/apache/solr/core/StandardIndexReaderFactory.java
    lucene/solr/trunk/src/test/org/apache/solr/core/AlternateIndexReaderTest.java
Modified:
    lucene/solr/trunk/CHANGES.txt
    lucene/solr/trunk/example/solr/conf/schema.xml
    lucene/solr/trunk/example/solr/conf/solrconfig.xml
    lucene/solr/trunk/src/java/org/apache/solr/core/SolrCore.java
    lucene/solr/trunk/src/java/org/apache/solr/search/SolrIndexSearcher.java
    lucene/solr/trunk/src/test/test-files/solr/conf/solrconfig-altdirectory.xml

Modified: lucene/solr/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/solr/trunk/CHANGES.txt?rev=782660&r1=782659&r2=782660&view=diff
==============================================================================
--- lucene/solr/trunk/CHANGES.txt (original)
+++ lucene/solr/trunk/CHANGES.txt Mon Jun  8 14:56:31 2009
@@ -234,6 +234,9 @@
 
 59. SOLR-1189: Support providing username and password for basic HTTP authentication in Java replication
     (Matthew Gregg, shalin)
+    
+60. SOLR-243: Add configurable IndexReaderFactory so that alternate IndexReader implementations 
+    can be specified via solrconfig.xml. (Andrzej Bialecki, hossman, Mark Miller, John Wang)
 
 Optimizations
 ----------------------

Modified: lucene/solr/trunk/example/solr/conf/schema.xml
URL: http://svn.apache.org/viewvc/lucene/solr/trunk/example/solr/conf/schema.xml?rev=782660&r1=782659&r2=782660&view=diff
==============================================================================
--- lucene/solr/trunk/example/solr/conf/schema.xml (original)
+++ lucene/solr/trunk/example/solr/conf/schema.xml Mon Jun  8 14:56:31 2009
@@ -435,10 +435,17 @@
    <copyField source="manu" dest="text"/>
    <copyField source="features" dest="text"/>
    <copyField source="includes" dest="text"/>
+	
+   <!-- Above, multiple specific fields are copied to the [text] field.
+	   Another option is to use the dynamic field syntax. A maxChars to
+	   copy setting is also available.  -->
+	   
+   <!-- <copyField source="*" dest="text" maxChars="3000"/> -->
+  
 
    <copyField source="manu" dest="manu_exact"/>
 
-  <copyField source="name" dest="spell"/>
+   <copyField source="name" dest="spell"/>
 
  <!-- Similarity is the scoring routine for each document vs. a query.
       A custom similarity may be specified here, but the default is fine

Modified: lucene/solr/trunk/example/solr/conf/solrconfig.xml
URL: http://svn.apache.org/viewvc/lucene/solr/trunk/example/solr/conf/solrconfig.xml?rev=782660&r1=782659&r2=782660&view=diff
==============================================================================
--- lucene/solr/trunk/example/solr/conf/solrconfig.xml (original)
+++ lucene/solr/trunk/example/solr/conf/solrconfig.xml Mon Jun  8 14:56:31 2009
@@ -205,6 +205,13 @@
     -->
 
   </updateHandler>
+  
+  <!-- Use the following format to specify a custom IndexReaderFactory - allows for alternate
+       IndexReader implementations.
+  <indexReaderFactory name="IndexReaderFactory" class="package.class">
+    Parameters as required by the implementation
+  </indexReaderFactory >
+  -->
 
 
   <query>

Added: lucene/solr/trunk/src/java/org/apache/solr/core/IndexReaderFactory.java
URL: http://svn.apache.org/viewvc/lucene/solr/trunk/src/java/org/apache/solr/core/IndexReaderFactory.java?rev=782660&view=auto
==============================================================================
--- lucene/solr/trunk/src/java/org/apache/solr/core/IndexReaderFactory.java (added)
+++ lucene/solr/trunk/src/java/org/apache/solr/core/IndexReaderFactory.java Mon Jun  8 14:56:31 2009
@@ -0,0 +1,51 @@
+package org.apache.solr.core;
+/**
+ * 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.
+ */
+
+import java.io.IOException;
+
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.store.Directory;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.util.plugin.NamedListInitializedPlugin;
+
+/**
+ * Factory used to build a new IndexReader instance.
+ */
+public abstract class IndexReaderFactory implements NamedListInitializedPlugin {
+
+  /**
+   * <code>init</code> will be called just once, immediately after creation.
+   * <p>
+   * The args are user-level initialization parameters that may be specified
+   * when declaring an indexReaderFactory in solrconfig.xml
+   */
+  public void init(NamedList args) {
+    /* :NOOP: */
+  }
+
+  /**
+   * Creates a new IndexReader instance using the given Directory.
+   * 
+   * @param indexDir indexDir index location
+   * @param readOnly return readOnly IndexReader
+   * @return An IndexReader instance
+   * @throws IOException
+   */
+  public abstract IndexReader newReader(Directory indexDir, boolean readOnly)
+      throws IOException;
+}

Modified: lucene/solr/trunk/src/java/org/apache/solr/core/SolrCore.java
URL: http://svn.apache.org/viewvc/lucene/solr/trunk/src/java/org/apache/solr/core/SolrCore.java?rev=782660&r1=782659&r2=782660&view=diff
==============================================================================
--- lucene/solr/trunk/src/java/org/apache/solr/core/SolrCore.java (original)
+++ lucene/solr/trunk/src/java/org/apache/solr/core/SolrCore.java Mon Jun  8 14:56:31 2009
@@ -97,6 +97,7 @@
   private final Map<String, SolrInfoMBean> infoRegistry;
   private IndexDeletionPolicyWrapper solrDelPolicy;
   private DirectoryFactory directoryFactory;
+  private IndexReaderFactory indexReaderFactory;
 
   public long getStartTime() { return startTime; }
 
@@ -223,6 +224,10 @@
     return directoryFactory;
   }
   
+  public IndexReaderFactory getIndexReaderFactory() {
+    return indexReaderFactory;
+  }
+  
   public String getName() {
     return name;
   }
@@ -341,6 +346,24 @@
     directoryFactory = dirFactory;
   }
 
+  private void initIndexReaderFactory() {
+    String xpath = "indexReaderFactory";
+    Node node = (Node) solrConfig.evaluate(xpath, XPathConstants.NODE);
+    IndexReaderFactory indexReaderFactory;
+    if (node != null) {
+      Map<String, IndexReaderFactory> registry = new HashMap<String, IndexReaderFactory>();
+      NamedListPluginLoader<IndexReaderFactory> indexReaderFactoryLoader = new NamedListPluginLoader<IndexReaderFactory>(
+          "[solrconfig.xml] " + xpath, registry);
+
+      indexReaderFactory = indexReaderFactoryLoader.loadSingle(solrConfig
+          .getResourceLoader(), node);
+    } else {
+      indexReaderFactory = new StandardIndexReaderFactory();
+    }
+
+    this.indexReaderFactory = indexReaderFactory;
+  }
+  
   // protect via synchronized(SolrCore.class)
   private static Set<String> dirs = new HashSet<String>();
 
@@ -355,6 +378,7 @@
       boolean removeLocks = solrConfig.unlockOnStartup;
 
       initDirectoryFactory();
+      initIndexReaderFactory();
 
       if (indexExists && firstTime && removeLocks) {
         // to remove locks, the directory must already exist... so we create it
@@ -1048,21 +1072,22 @@
     try {
       newestSearcher = getNewestSearcher(false);
       String newIndexDir = getNewIndexDir();
-      if (newestSearcher != null) {
+      File indexDirFile = new File(getIndexDir()).getCanonicalFile();
+      File newIndexDirFile = new File(newIndexDir).getCanonicalFile();
+      
+      if (newestSearcher != null && solrConfig.reopenReaders
+          && indexDirFile.equals(newIndexDirFile)) {
         IndexReader currentReader = newestSearcher.get().getReader();
-        if(solrConfig.reopenReaders && new File(getIndexDir()).getCanonicalFile().equals(new File(newIndexDir).getCanonicalFile()))  {
-          IndexReader newReader = currentReader.reopen();
+        IndexReader newReader = currentReader.reopen();
 
-          if(newReader == currentReader) {
-            currentReader.incRef();
-          }
-          
-          tmp = new SolrIndexSearcher(this, schema, "main", newReader, true, true);
-        } else  {
-          tmp = new SolrIndexSearcher(this, schema, "main", getDirectoryFactory().open(newIndexDir), true, true);
+        if (newReader == currentReader) {
+          currentReader.incRef();
         }
+
+        tmp = new SolrIndexSearcher(this, schema, "main", newReader, true, true);
       } else {
-          tmp = new SolrIndexSearcher(this, schema, "main", getDirectoryFactory().open(newIndexDir), true, true);
+        IndexReader reader = getIndexReaderFactory().newReader(getDirectoryFactory().open(newIndexDir), true);
+        tmp = new SolrIndexSearcher(this, schema, "main", reader, true, true);
       }
     } catch (Throwable th) {
       synchronized(searcherLock) {

Added: lucene/solr/trunk/src/java/org/apache/solr/core/StandardIndexReaderFactory.java
URL: http://svn.apache.org/viewvc/lucene/solr/trunk/src/java/org/apache/solr/core/StandardIndexReaderFactory.java?rev=782660&view=auto
==============================================================================
--- lucene/solr/trunk/src/java/org/apache/solr/core/StandardIndexReaderFactory.java (added)
+++ lucene/solr/trunk/src/java/org/apache/solr/core/StandardIndexReaderFactory.java Mon Jun  8 14:56:31 2009
@@ -0,0 +1,39 @@
+package org.apache.solr.core;
+/**
+ * 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.
+ */
+
+import java.io.IOException;
+
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.store.Directory;
+
+/**
+ * Default IndexReaderFactory implementation. Returns a standard Lucene
+ * IndexReader.
+ * 
+ * @see IndexReader#open(Directory)
+ */
+public class StandardIndexReaderFactory extends IndexReaderFactory {
+
+  /* (non-Javadoc)
+   * @see org.apache.solr.core.IndexReaderFactory#newReader(org.apache.lucene.store.Directory, boolean)
+   */
+  public IndexReader newReader(Directory indexDir, boolean readOnly)
+      throws IOException {
+    return IndexReader.open(indexDir, readOnly);
+  }
+}

Modified: lucene/solr/trunk/src/java/org/apache/solr/search/SolrIndexSearcher.java
URL: http://svn.apache.org/viewvc/lucene/solr/trunk/src/java/org/apache/solr/search/SolrIndexSearcher.java?rev=782660&r1=782659&r2=782660&view=diff
==============================================================================
--- lucene/solr/trunk/src/java/org/apache/solr/search/SolrIndexSearcher.java (original)
+++ lucene/solr/trunk/src/java/org/apache/solr/search/SolrIndexSearcher.java Mon Jun  8 14:56:31 2009
@@ -24,7 +24,6 @@
 import org.apache.lucene.search.*;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.FSDirectory;
-import org.apache.lucene.util.PriorityQueue;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.SimpleOrderedMap;
 import org.apache.solr.core.SolrConfig;
@@ -33,7 +32,6 @@
 import org.apache.solr.schema.IndexSchema;
 import org.apache.solr.schema.SchemaField;
 import org.apache.solr.request.UnInvertedField;
-import org.apache.solr.search.function.BoostedQuery;
 import org.apache.lucene.util.OpenBitSet;
 
 import java.io.IOException;
@@ -92,17 +90,23 @@
    * @deprecated use alternate constructor
    */
   public SolrIndexSearcher(SolrCore core, IndexSchema schema, String name, String path, boolean enableCache) throws IOException {
-    this(core, schema,name,IndexReader.open(path), true, enableCache);
+    this(core, schema,name, core.getIndexReaderFactory().newReader(core.getDirectoryFactory().open(path), false), true, enableCache);
   }
 
-  /** Creates a searcher searching the index in the provided directory. */
-  public SolrIndexSearcher(SolrCore core, IndexSchema schema, String name, Directory directory, boolean enableCache) throws IOException {
-    this(core, schema,name,IndexReader.open(directory), true, enableCache);
+  /*
+   * Creates a searcher searching the index in the provided directory. Note:
+   * uses the main IndexReaderFactory for the specified SolrCore.
+   * 
+   * @see SolrCore#getMainIndexReaderFactory
+   */
+  public SolrIndexSearcher(SolrCore core, IndexSchema schema, String name,
+      Directory directory, boolean enableCache) throws IOException {
+    this(core, schema,name, core.getIndexReaderFactory().newReader(directory, false), true, enableCache);
   }
   
   /** Creates a searcher searching the index in the provided directory. */
   public SolrIndexSearcher(SolrCore core, IndexSchema schema, String name, Directory directory, boolean readOnly, boolean enableCache) throws IOException {
-    this(core, schema,name,IndexReader.open(directory, readOnly), true, enableCache);
+    this(core, schema,name, core.getIndexReaderFactory().newReader(directory, readOnly), true, enableCache);
   }
 
   /** Creates a searcher searching the provided index. */

Added: lucene/solr/trunk/src/test/org/apache/solr/core/AlternateIndexReaderTest.java
URL: http://svn.apache.org/viewvc/lucene/solr/trunk/src/test/org/apache/solr/core/AlternateIndexReaderTest.java?rev=782660&view=auto
==============================================================================
--- lucene/solr/trunk/src/test/org/apache/solr/core/AlternateIndexReaderTest.java (added)
+++ lucene/solr/trunk/src/test/org/apache/solr/core/AlternateIndexReaderTest.java Mon Jun  8 14:56:31 2009
@@ -0,0 +1,61 @@
+package org.apache.solr.core;
+/**
+ * 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.
+ */
+
+import java.io.IOException;
+
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.store.Directory;
+import org.apache.solr.util.AbstractSolrTestCase;
+
+public class AlternateIndexReaderTest extends AbstractSolrTestCase {
+
+  public String getSchemaFile() {
+    return "schema.xml";
+  }
+
+  public String getSolrConfigFile() {
+    return "solrconfig-altdirectory.xml";
+  }
+
+  /**
+   * Simple test to ensure that alternate IndexReaderFactory is being used.
+   * 
+   * @throws Exception
+   */
+  public void testAltReaderUsed() throws Exception {
+    assertTrue(TestIndexReaderFactory.newReaderCalled);
+  }
+
+  static public class TestIndexReaderFactory extends IndexReaderFactory {
+
+    static boolean newReaderCalled = false;
+
+    public IndexReader newReader(Directory indexDir) throws IOException {
+      TestIndexReaderFactory.newReaderCalled = true;
+      return IndexReader.open(indexDir);
+    }
+
+    public IndexReader newReader(Directory indexDir, boolean readOnly)
+        throws IOException {
+      TestIndexReaderFactory.newReaderCalled = true;
+      return IndexReader.open(indexDir, readOnly);
+    }
+
+  }
+
+}

Modified: lucene/solr/trunk/src/test/test-files/solr/conf/solrconfig-altdirectory.xml
URL: http://svn.apache.org/viewvc/lucene/solr/trunk/src/test/test-files/solr/conf/solrconfig-altdirectory.xml?rev=782660&r1=782659&r2=782660&view=diff
==============================================================================
--- lucene/solr/trunk/src/test/test-files/solr/conf/solrconfig-altdirectory.xml (original)
+++ lucene/solr/trunk/src/test/test-files/solr/conf/solrconfig-altdirectory.xml Mon Jun  8 14:56:31 2009
@@ -133,6 +133,11 @@
   <directoryFactory name="DirectoryFactory" class="org.apache.solr.core.AlternateDirectoryTest$TestFSDirectoryFactory">
     <!-- Parameters as required by the implementation -->
   </directoryFactory>
+  
+  
+  <indexReaderFactory name="IndexReaderFactory" class="org.apache.solr.core.AlternateIndexReaderTest$TestIndexReaderFactory">
+    <!-- Parameters as required by the implementation -->
+  </indexReaderFactory >
 
   <query>
     <!-- Maximum number of clauses in a boolean query... can affect