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 yo...@apache.org on 2008/11/07 00:41:45 UTC

svn commit: r712014 - in /lucene/solr/trunk: ./ src/java/org/apache/solr/core/ src/java/org/apache/solr/schema/ src/java/org/apache/solr/search/ src/java/org/apache/solr/update/ src/java/org/apache/solr/util/plugin/

Author: yonik
Date: Thu Nov  6 15:41:39 2008
New Revision: 712014

URL: http://svn.apache.org/viewvc?rev=712014&view=rev
Log:
SOLR-465: Add configurable DirectoryProvider so that alternate Directory implementations can be specified via solrconfig.xml

Modified:
    lucene/solr/trunk/CHANGES.txt
    lucene/solr/trunk/src/java/org/apache/solr/core/SolrCore.java
    lucene/solr/trunk/src/java/org/apache/solr/schema/UUIDField.java
    lucene/solr/trunk/src/java/org/apache/solr/search/SolrIndexSearcher.java
    lucene/solr/trunk/src/java/org/apache/solr/update/SolrIndexWriter.java
    lucene/solr/trunk/src/java/org/apache/solr/util/plugin/AbstractPluginLoader.java

Modified: lucene/solr/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/solr/trunk/CHANGES.txt?rev=712014&r1=712013&r2=712014&view=diff
==============================================================================
--- lucene/solr/trunk/CHANGES.txt (original)
+++ lucene/solr/trunk/CHANGES.txt Thu Nov  6 15:41:39 2008
@@ -77,6 +77,11 @@
     contention and synchronization overhead, to utilize multiple CPU cores more effectively.
     (Fuad Efendi, Noble Paul, yonik via shalin)
 
+14. SOLR-465: Add configurable DirectoryProvider so that alternate Directory
+    implementations can be specified via solrconfig.xml.  The default
+    DirectoryProvider will use NIOFSDirectory for better concurrency
+    on non Windows platforms.  (Mark Miller, TJ Laurenzo via yonik)
+
 Optimizations
 ----------------------
  1. SOLR-374: Use IndexReader.reopen to save resources by re-using parts of the

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=712014&r1=712013&r2=712014&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 Thu Nov  6 15:41:39 2008
@@ -96,6 +96,7 @@
   private final Map<String,UpdateRequestProcessorChain> updateProcessorChains;
   private final Map<String, SolrInfoMBean> infoRegistry;
   private IndexDeletionPolicyWrapper solrDelPolicy;
+  private DirectoryFactory directoryFactory;
 
   public long getStartTime() { return startTime; }
 
@@ -217,7 +218,11 @@
     }
     return result;
   }
-
+  
+  public DirectoryFactory getDirectoryFactory() {
+    return directoryFactory;
+  }
+  
   public String getName() {
     return name;
   }
@@ -319,9 +324,28 @@
   
   // gets a non-caching searcher
   public SolrIndexSearcher newSearcher(String name, boolean readOnly) throws IOException {
-    return new SolrIndexSearcher(this, schema, "main", IndexReader.open(FSDirectory.getDirectory(getIndexDir()), readOnly), true, false);
+    return new SolrIndexSearcher(this, schema, name, directoryFactory.open(getIndexDir()), false);
+    //return new SolrIndexSearcher(this, schema, "main", IndexReader.open(FSDirectory.getDirectory(getIndexDir()), readOnly), true, false);
   }
 
+  private void initDirectoryFactory() {
+    String xpath = "directoryFactory";
+    Node node = (Node) solrConfig.evaluate(xpath, XPathConstants.NODE);
+    DirectoryFactory dirFactory;
+    if (node != null) {
+      Map<String, DirectoryFactory> registry = new HashMap<String, DirectoryFactory>();
+      NamedListPluginLoader<DirectoryFactory> indexReaderFactoryLoader = new NamedListPluginLoader<DirectoryFactory>(
+          "[solrconfig.xml] " + xpath, registry);
+
+      dirFactory = indexReaderFactoryLoader.loadSingle(solrConfig
+          .getResourceLoader(), node);
+    } else {
+      dirFactory = new StandardDirectoryFactory();
+    }
+
+    // And set it
+    directoryFactory = dirFactory;
+  }
 
   // protect via synchronized(SolrCore.class)
   private static Set<String> dirs = new HashSet<String>();
@@ -351,6 +375,8 @@
         SolrIndexWriter writer = new SolrIndexWriter("SolrCore.initIndex",getIndexDir(), true, schema, solrConfig.mainIndexConfig);
         writer.close();
       }
+      
+      initDirectoryFactory();
 
     } catch (IOException e) {
       throw new RuntimeException(e);
@@ -1011,22 +1037,22 @@
 
     try {
       newestSearcher = getNewestSearcher(false);
+      String newIndexDir = getNewIndexDir();
       if (newestSearcher != null) {
         IndexReader currentReader = newestSearcher.get().getReader();
-        String newIndexDir = getNewIndexDir();
         if(new File(getIndexDir()).equals(new File(newIndexDir)))  {
           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", newIndexDir, true);
+          tmp = new SolrIndexSearcher(this, schema, "main", getDirectoryFactory().open(newIndexDir), true, true);
         }
       } else {
-        tmp = new SolrIndexSearcher(this, schema, "main", getNewIndexDir(), true);
+          tmp = new SolrIndexSearcher(this, schema, "main", getDirectoryFactory().open(newIndexDir), true, true);
       }
     } catch (Throwable th) {
       synchronized(searcherLock) {

Modified: lucene/solr/trunk/src/java/org/apache/solr/schema/UUIDField.java
URL: http://svn.apache.org/viewvc/lucene/solr/trunk/src/java/org/apache/solr/schema/UUIDField.java?rev=712014&r1=712013&r2=712014&view=diff
==============================================================================
--- lucene/solr/trunk/src/java/org/apache/solr/schema/UUIDField.java (original)
+++ lucene/solr/trunk/src/java/org/apache/solr/schema/UUIDField.java Thu Nov  6 15:41:39 2008
@@ -33,7 +33,7 @@
  *
  * @see UUID#toString
  * @see UUID#randomUUID
- * @version $Id:$
+ * @version $Id$
  */
 public class UUIDField extends FieldType {
   private static final String NEW = "NEW";

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=712014&r1=712013&r2=712014&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 Thu Nov  6 15:41:39 2008
@@ -91,7 +91,10 @@
   private final SolrCache[] cacheList;
   private static final SolrCache[] noCaches = new SolrCache[0];
 
-  /** Creates a searcher searching the index in the named directory. */
+  /** Creates a searcher searching the index in the named directory.
+   * 
+   * @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);
   }
@@ -100,6 +103,11 @@
   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. */
+  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);
+  }
 
   /** Creates a searcher searching the provided index. */
   public SolrIndexSearcher(SolrCore core, IndexSchema schema, String name, IndexReader r, boolean enableCache) {

Modified: lucene/solr/trunk/src/java/org/apache/solr/update/SolrIndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/solr/trunk/src/java/org/apache/solr/update/SolrIndexWriter.java?rev=712014&r1=712013&r2=712014&view=diff
==============================================================================
--- lucene/solr/trunk/src/java/org/apache/solr/update/SolrIndexWriter.java (original)
+++ lucene/solr/trunk/src/java/org/apache/solr/update/SolrIndexWriter.java Thu Nov  6 15:41:39 2008
@@ -20,6 +20,7 @@
 import org.apache.lucene.index.*;
 import org.apache.lucene.store.*;
 import org.apache.solr.common.SolrException;
+import org.apache.solr.core.DirectoryFactory;
 import org.apache.solr.schema.IndexSchema;
 
 import org.slf4j.Logger;
@@ -75,12 +76,46 @@
 
   }
 
+  public static Directory getDirectory(String path, DirectoryFactory directoryFactory, SolrIndexConfig config) throws IOException {
+    
+    Directory d = directoryFactory.open(path);
+
+    String rawLockType = (null == config) ? null : config.lockType;
+    if (null == rawLockType) {
+      // we default to "simple" for backwards compatibility
+      log.warn("No lockType configured for " + path + " assuming 'simple'");
+      rawLockType = "simple";
+    }
+    final String lockType = rawLockType.toLowerCase().trim();
+
+    if ("simple".equals(lockType)) {
+      // multiple SimpleFSLockFactory instances should be OK
+      d.setLockFactory(new SimpleFSLockFactory(path));
+    } else if ("native".equals(lockType)) {
+      d.setLockFactory(new NativeFSLockFactory(path));
+    } else if ("single".equals(lockType)) {
+      if (!(d.getLockFactory() instanceof SingleInstanceLockFactory))
+        d.setLockFactory(new SingleInstanceLockFactory());
+    } else if ("none".equals(lockType)) {
+      // Recipe for disaster
+      log.error("CONFIGURATION WARNING: locks are disabled on " + path);      
+      d.setLockFactory(new NoLockFactory());
+    } else {
+      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
+              "Unrecognized lockType: " + rawLockType);
+    }
+    return d;
+  }
+  
+  /**
+   * @deprecated use getDirectory(DirectoryFactory directoryFactory, SolrIndexConfig config)
+   */
   public static Directory getDirectory(String path, SolrIndexConfig config) throws IOException {
     Directory d = FSDirectory.getDirectory(path);
 
     String rawLockType = (null == config) ? null : config.lockType;
     if (null == rawLockType) {
-      // we default to "simple" for backwards compatiblitiy
+      // we default to "simple" for backwards compatibility
       log.warn("No lockType configured for " + path + " assuming 'simple'");
       rawLockType = "simple";
     }
@@ -95,7 +130,7 @@
       if (!(d.getLockFactory() instanceof SingleInstanceLockFactory))
         d.setLockFactory(new SingleInstanceLockFactory());
     } else if ("none".equals(lockType)) {
-      // recipie for disaster
+      // Recipe for disaster
       log.error("CONFIGURATION WARNING: locks are disabled on " + path);      
       d.setLockFactory(new NoLockFactory());
     } else {
@@ -104,12 +139,34 @@
     }
     return d;
   }
+  
+  /**
+   *
+   */
+  public SolrIndexWriter(String name, String path, DirectoryFactory dirFactory, boolean create, IndexSchema schema) throws IOException {
+    super(getDirectory(path, dirFactory, null), false, schema.getAnalyzer(), create);
+    init(name, schema, null);
+  }
 
+  /**
+   *
+   */
+  public SolrIndexWriter(String name, String path, DirectoryFactory dirFactory, boolean create, IndexSchema schema, SolrIndexConfig config) throws IOException {
+    super(getDirectory(path, dirFactory, null), config.luceneAutoCommit, schema.getAnalyzer(), create);
+    init(name, schema, config);
+  }
+  
+  /**
+   * @deprecated
+   */
   public SolrIndexWriter(String name, String path, boolean create, IndexSchema schema) throws IOException {
     super(getDirectory(path, null), false, schema.getAnalyzer(), create);
     init(name, schema, null);
   }
 
+  /**
+   * @deprecated
+   */
   public SolrIndexWriter(String name, String path, boolean create, IndexSchema schema, SolrIndexConfig config) throws IOException {
     super(getDirectory(path, config), config.luceneAutoCommit, schema.getAnalyzer(), create);
     init(name, schema, config);

Modified: lucene/solr/trunk/src/java/org/apache/solr/util/plugin/AbstractPluginLoader.java
URL: http://svn.apache.org/viewvc/lucene/solr/trunk/src/java/org/apache/solr/util/plugin/AbstractPluginLoader.java?rev=712014&r1=712013&r2=712014&view=diff
==============================================================================
--- lucene/solr/trunk/src/java/org/apache/solr/util/plugin/AbstractPluginLoader.java (original)
+++ lucene/solr/trunk/src/java/org/apache/solr/util/plugin/AbstractPluginLoader.java Thu Nov  6 15:41:39 2008
@@ -183,6 +183,62 @@
     return defaultPlugin;
   }
   
+  /**
+   * Given a NodeList from XML in the form:
+   * 
+   * <plugin name="name1" class="solr.ClassName" > ... </plugin>
+   * 
+   * This will initialize and register a single plugin. A class will be
+   * generated for the plugin and registered to the given name.
+   * 
+   * If 'preRegister' is true, the plugin will be registered *before* it is
+   * initialized This may be useful for implementations that need to inspect
+   * other registered plugins at startup.
+   * 
+   * The created class for the plugin will be returned from this function.
+   * 
+   */
+  public T loadSingle(ResourceLoader loader, Node node) {
+    List<PluginInitInfo> info = new ArrayList<PluginInitInfo>();
+    T plugin = null;
+
+    try {
+      String name = DOMUtil.getAttr(node, "name", requireName ? type : null);
+      String className = DOMUtil.getAttr(node, "class", type);
+      plugin = create(loader, name, className, node);
+      log.info("created " + name + ": " + plugin.getClass().getName());
+
+      // Either initialize now or wait till everything has been registered
+      if (preRegister) {
+        info.add(new PluginInitInfo(plugin, node));
+      } else {
+        init(plugin, node);
+      }
+
+      T old = register(name, plugin);
+      if (old != null && !(name == null && !requireName)) {
+        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
+            "Multiple " + type + " registered to the same name: " + name
+                + " ignoring: " + old);
+      }
+
+    } catch (Exception e) {
+      SolrConfig.severeErrors.add(e);
+      SolrException.logOnce(log, null, e);
+    }
+
+    // If everything needs to be registered *first*, this will initialize later
+    for (PluginInitInfo pinfo : info) {
+      try {
+        init(pinfo.plugin, pinfo.node);
+      } catch (Exception ex) {
+        SolrConfig.severeErrors.add(ex);
+        SolrException.logOnce(log, null, ex);
+      }
+    }
+    return plugin;
+  }
+  
 
   /**
    * Internal class to hold onto initialization info so that it can be initialized