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 ry...@apache.org on 2008/10/04 05:55:02 UTC

svn commit: r701592 [2/2] - in /lucene/solr/branches/sandbox/ryan: contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/ src/java/org/apache/solr/config/ src/java/org/apache/solr/core/ src/java/org/apache/solr/handler/admin/ src/j...

Modified: lucene/solr/branches/sandbox/ryan/src/java/org/apache/solr/core/SolrCore.java
URL: http://svn.apache.org/viewvc/lucene/solr/branches/sandbox/ryan/src/java/org/apache/solr/core/SolrCore.java?rev=701592&r1=701591&r2=701592&view=diff
==============================================================================
--- lucene/solr/branches/sandbox/ryan/src/java/org/apache/solr/core/SolrCore.java (original)
+++ lucene/solr/branches/sandbox/ryan/src/java/org/apache/solr/core/SolrCore.java Fri Oct  3 20:55:00 2008
@@ -17,22 +17,23 @@
 
 package org.apache.solr.core;
 
-import org.apache.lucene.index.IndexDeletionPolicy;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.IndexWriter;
-import org.apache.lucene.search.BooleanQuery;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.FSDirectory;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.params.CommonParams;
 import org.apache.solr.common.params.CommonParams.EchoParamStyle;
 import org.apache.solr.common.params.SolrParams;
-import org.apache.solr.common.util.DOMUtil;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.SimpleOrderedMap;
-import org.apache.solr.handler.admin.ShowFileRequestHandler;
+import org.apache.solr.config.CoreDescriptor;
+import org.apache.solr.config.CoreInitalizer;
+import org.apache.solr.config.CoreVars;
+import org.apache.solr.config.SolrConfig;
+import org.apache.solr.config.SolrConfiguraion;
+import org.apache.solr.config.SolrResourceLoader;
 import org.apache.solr.handler.component.*;
-import org.apache.solr.highlight.DefaultSolrHighlighter;
 import org.apache.solr.highlight.SolrHighlighter;
 import org.apache.solr.request.*;
 import org.apache.solr.schema.IndexSchema;
@@ -42,24 +43,9 @@
 import org.apache.solr.update.DirectUpdateHandler2;
 import org.apache.solr.update.SolrIndexWriter;
 import org.apache.solr.update.UpdateHandler;
-import org.apache.solr.update.processor.LogUpdateProcessorFactory;
-import org.apache.solr.update.processor.RunUpdateProcessorFactory;
 import org.apache.solr.update.processor.UpdateRequestProcessorChain;
-import org.apache.solr.update.processor.UpdateRequestProcessorFactory;
 import org.apache.solr.util.RefCounted;
-import org.apache.solr.util.plugin.AbstractPluginLoader;
-import org.apache.solr.util.plugin.NamedListInitializedPlugin;
-import org.apache.solr.util.plugin.NamedListPluginLoader;
 import org.apache.solr.util.plugin.SolrCoreAware;
-import org.w3c.dom.Node;
-import org.w3c.dom.NodeList;
-import org.xml.sax.SAXException;
-
-import javax.xml.parsers.ParserConfigurationException;
-import javax.xml.xpath.XPathConstants;
-import javax.xml.xpath.XPath;
-import javax.xml.xpath.XPathExpressionException;
-
 import java.io.File;
 import java.io.IOException;
 import java.util.*;
@@ -78,92 +64,69 @@
 
   public static Logger log = LoggerFactory.getLogger(SolrCore.class);
 
+  // TODO -- get rid of me...
+  private final CoreInitalizer initalizer;
+  private final SolrConfiguraion configuration;
+  
+  @Deprecated
+  public static SolrCore INSTANCE;
+  
   private String name;
-  private String logid; // used to show what name is set
-  private final CoreDescriptor coreDescriptor;
+  private String logid;
 
-  private final SolrConfig solrConfig;
   private final IndexSchema schema;
   private final String dataDir;
   private final UpdateHandler updateHandler;
-  private final long startTime;
+  
   private final RequestHandlers reqHandlers;
   private final SolrHighlighter highlighter;
   private final Map<String,SearchComponent> searchComponents;
   private final Map<String,UpdateRequestProcessorChain> updateProcessorChains;
   private final Map<String, SolrInfoMBean> infoRegistry;
-  private IndexDeletionPolicyWrapper solrDelPolicy;
+  private final IndexDeletionPolicyWrapper solrDelPolicy;
+  private final QueryResponseWriter defaultResponseWriter;
 
-  public long getStartTime() { return startTime; }
+  private final Map<String, QueryResponseWriter> responseWriters;
+  private final Map<String, QParserPlugin> qParserPlugins;
+  private final HashMap<String, ValueSourceParser> valueSourceParsers;
 
-  /**
-   * @deprecated Use {@link CoreContainer#getCore(String)} instead.
-   */
-  @Deprecated
-  private static SolrCore instance;
+  final List<SolrEventListener> firstSearcherListeners;
+  final List<SolrEventListener> newSearcherListeners;
+  
+  
+  private final long startTime = System.currentTimeMillis();
 
-  static int boolean_query_max_clause_count = Integer.MIN_VALUE;
-  // only change the BooleanQuery maxClauseCount once for ALL cores...
-  void booleanQueryMaxClauseCount()  {
-    synchronized(SolrCore.class) {
-      if (boolean_query_max_clause_count == Integer.MIN_VALUE) {
-        boolean_query_max_clause_count = solrConfig.booleanQueryMaxClauseCount;
-        BooleanQuery.setMaxClauseCount(boolean_query_max_clause_count);
-      } else if (boolean_query_max_clause_count != solrConfig.booleanQueryMaxClauseCount ) {
-        log.debug("BooleanQuery.maxClauseCount= " +boolean_query_max_clause_count+ ", ignoring " +solrConfig.booleanQueryMaxClauseCount);
-      }
-    }
-  }
+
+  private List<CloseHook> closeHooks = null;
 
   
   /**
    * The SolrResourceLoader used to load all resources for this core.
    * @since solr 1.3
    */
+  @Deprecated
   public SolrResourceLoader getResourceLoader() {
-    return solrConfig.getResourceLoader();
-  }
-
-  /**
-   * Gets the configuration resource name used by this core instance.
-   * @since solr 1.3
-   */
-  public String getConfigResource() {
-    return solrConfig.getResourceName();
+    return initalizer.getSolrConfig().getResourceLoader();
   }
   
   /**
-   * Gets the configuration resource name used by this core instance.
-   * @deprecated Use {@link #getConfigResource()} instead.
-   */
-  @Deprecated
-  public String getConfigFile() {
-    return solrConfig.getResourceName();
-  }
-  /**
    * Gets the configuration object used by this core instance.
    */
+  @Deprecated
   public SolrConfig getSolrConfig() {
-    return solrConfig;
-  }
-  
-  /**
-   * Gets the schema resource name used by this core instance.
-   * @since solr 1.3
-   */
-  public String getSchemaResource() {
-    return schema.getResourceName();
+    return initalizer.getSolrConfig();
   }
 
+  
   /**
-   * Gets the schema resource name used by this core instance.
-   * @deprecated Use {@link #getSchemaResource()} instead.
+   * Gets the configuration object used by this core instance.
    */
   @Deprecated
-  public String getSchemaFile() {
-    return schema.getResourceName();
+  public CoreInitalizer getInitalizer() {
+    return initalizer;
   }
   
+  
   /**
    * Gets the schema object used by this core instance.
    */
@@ -201,41 +164,7 @@
     return infoRegistry;
   }
 
-  private void initDeletionPolicy() {
-    String className = solrConfig.get("mainIndex/deletionPolicy/@class", SolrDeletionPolicy.class.getName());
-    IndexDeletionPolicy delPolicy = createInstance(className, IndexDeletionPolicy.class, "Deletion Policy for SOLR");
-
-    Node node = (Node) solrConfig.evaluate("mainIndex/deletionPolicy", XPathConstants.NODE);
-    if (node != null) {
-      if (delPolicy instanceof NamedListInitializedPlugin)
-        ((NamedListInitializedPlugin) delPolicy).init(DOMUtil.childNodesToNamedList(node));
-    }
-    solrDelPolicy = new IndexDeletionPolicyWrapper(delPolicy);
-  }
-
-  public List<SolrEventListener> parseListener(String path) {
-    List<SolrEventListener> lst = new ArrayList<SolrEventListener>();
-    log.info( logid+"Searching for listeners: " +path);
-    NodeList nodes = (NodeList)solrConfig.evaluate(path, XPathConstants.NODESET);
-    if (nodes!=null) {
-      for (int i=0; i<nodes.getLength(); i++) {
-        Node node = nodes.item(i);
-        String className = DOMUtil.getAttr(node,"class");
-        SolrEventListener listener = createEventListener(className);
-        listener.init(DOMUtil.childNodesToNamedList(node));
-        lst.add(listener);
-        log.info( logid+"Added SolrEventListener: " + listener);
-      }
-    }
-    return lst;
-  }
 
-  List<SolrEventListener> firstSearcherListeners;
-  List<SolrEventListener> newSearcherListeners;
-  private void parseListeners() {
-    firstSearcherListeners = parseListener("//listener[@event=\"firstSearcher\"]");
-    newSearcherListeners = parseListener("//listener[@event=\"newSearcher\"]");
-  }
   
   /**
    * NOTE: this function is not thread safe.  However, it is safe to call within the
@@ -268,11 +197,10 @@
    * 
    * @see SolrCoreAware
    */
-  public void registerResponseWriter( String name, QueryResponseWriter responseWriter ){
-    responseWriters.put(name, responseWriter);
+  public QueryResponseWriter registerResponseWriter( String name, QueryResponseWriter responseWriter ){
+    return responseWriters.put(name, responseWriter);
   }
 
-
   // gets a non-caching searcher
   public SolrIndexSearcher newSearcher(String name) throws IOException {
     return newSearcher(name, false);
@@ -284,20 +212,97 @@
   }
 
 
+
+  
+  /**
+   * Creates a new core and register it in the list of cores.
+   * If a core with the same name already exists, it will be stopped and replaced by this one.
+   *@param dataDir the index directory
+   *@param config a solr config instance
+   *@param schema a solr schema instance
+   *
+   *@since solr 1.3
+   */
+  public SolrCore(String name, CoreVars vars ) {
+    INSTANCE = this;
+    this.initalizer = vars.getInitalizer();
+    this.configuration = vars.getConfiguration();
+    this.setName( name );
+    
+    this.dataDir = configuration.getDataDir();
+    log.info(logid+"Opening new SolrCore at: " + dataDir);
+
+    
+    //Initialize JMX
+    this.schema = vars.getSchema();
+    if (schema==null) {
+      throw new NullPointerException( "schema can't be null" );
+    }
+
+    this.reqHandlers = vars.getReqHandlers();
+    this.highlighter = vars.getHighlighter();
+    this.searchComponents = vars.getSearchComponents();
+    this.updateProcessorChains = vars.getUpdateProcessorChains();
+    this.infoRegistry = vars.getInfoRegistry();
+    this.solrDelPolicy = vars.getSolrDelPolicy();
+    this.defaultResponseWriter = vars.getDefaultResponseWriter();
+
+    this.responseWriters = vars.getResponseWriters();
+    this.qParserPlugins = vars.getQParserPlugins();
+    this.valueSourceParsers = vars.getValueSourceParsers();
+
+    this.firstSearcherListeners = vars.getFirstSearcherListeners();
+    this.newSearcherListeners = vars.getNewSearcherListeners();
+    
+    initIndex();
+    
+    final CountDownLatch latch = new CountDownLatch(1);
+
+    try {
+      // cause the executor to stall so firstSearcher events won't fire
+      // until after inform() has been called for all components.
+      // searchExecutor must be single-threaded for this to work
+      searcherExecutor.submit(new Callable() {
+        public Object call() throws Exception {
+          latch.await();
+          return null;
+        }
+      });
+
+      // Open the searcher *before* the update handler so we don't end up opening
+      // one in the middle.
+      // With lockless commits in Lucene now, this probably shouldn't be an issue anymore
+      getSearcher(false,false,null);
+
+      // HACK -- for now, this is hard coded since it needs the update handler...
+      updateHandler = new DirectUpdateHandler2(this, 
+          vars.getCommitCallbacks(), vars.getOptimizeCallbacks() );
+      infoRegistry.put("updateHandler", updateHandler);
+    } 
+    catch (IOException e) {
+      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
+    } 
+    finally {
+      // allow firstSearcher events to fire
+      latch.countDown();
+    }
+    infoRegistry.put("core", this);
+  }
+
+
   // protect via synchronized(SolrCore.class)
   private static Set<String> dirs = new HashSet<String>();
-
+  
   // currently only called with SolrCore.class lock held
   void initIndex() {
     try {
       File dirFile = new File(getIndexDir());
       boolean indexExists = dirFile.canRead();
       boolean firstTime = dirs.add(dirFile.getCanonicalPath());
-      boolean removeLocks = solrConfig.getBool("mainIndex/unlockOnStartup", false);
-      if (indexExists && firstTime && removeLocks) {
+      if (indexExists && firstTime && configuration.isUnlockOnStartup() ) {
         // to remove locks, the directory must already exist... so we create it
         // if it didn't exist already...
-        Directory dir = SolrIndexWriter.getDirectory(getIndexDir(), solrConfig.mainIndexConfig);
+        Directory dir = SolrIndexWriter.getDirectory(getIndexDir(), configuration.getMainIndexConfig() );
         if (dir != null && IndexWriter.isLocked(dir)) {
           log.warn(logid+"WARNING: Solr index directory '" + getIndexDir() + "' is locked.  Unlocking...");
           IndexWriter.unlock(dir);
@@ -309,7 +314,9 @@
         log.warn(logid+"Solr index directory '" + dirFile + "' doesn't exist."
                 + " Creating new index...");
 
-        SolrIndexWriter writer = new SolrIndexWriter("SolrCore.initIndex",getIndexDir(), true, schema, solrConfig.mainIndexConfig);
+        SolrIndexWriter writer = new SolrIndexWriter(
+            "SolrCore.initIndex",getIndexDir(), true, schema, 
+            configuration.getMainIndexConfig() );
         writer.close();
       }
 
@@ -317,270 +324,8 @@
       throw new RuntimeException(e);
     }
   }
-
-  /** Creates an instance by trying a constructor that accepts a SolrCore before
-   *  trying the default (no arg) constructor.
-   *@param className the instance class to create
-   *@cast the class or interface that the instance should extend or implement
-   *@param msg a message helping compose the exception error if any occurs.
-   *@return the desired instance
-   *@throws SolrException if the object could not be instantiated
-   */
-  private <T extends Object> T createInstance(String className, Class<T> cast, String msg) {
-    Class clazz = null;
-    if (msg == null) msg = "SolrCore Object";
-    try {
-      try {
-        clazz = solrConfig.getResourceLoader().findClass(className);
-        if (cast != null && !cast.isAssignableFrom(clazz))
-          throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,"Error Instantiating "+msg+", "+className+ " is not a " +cast.getName());
-        
-        java.lang.reflect.Constructor cons = clazz.getConstructor(new Class[]{SolrCore.class});
-        return (T) cons.newInstance(new Object[]{this});
-      } catch(NoSuchMethodException xnomethod) {
-        return (T) clazz.newInstance();
-      }
-    } catch (SolrException e) {
-      throw e;
-    } catch (Exception e) {
-      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,"Error Instantiating "+msg+", "+className+ " failed to instantiate " +cast.getName(), e);
-    }
-  }
-
-  public SolrEventListener createEventListener(String className) {
-    return createInstance(className, SolrEventListener.class, "Event Listener");
-  }
-
-  public SolrRequestHandler createRequestHandler(String className) {
-    return createInstance(className, SolrRequestHandler.class, "Request Handler");
-  }
-
-  private UpdateHandler createUpdateHandler(String className) {
-    return createInstance(className, UpdateHandler.class, "Update Handler");
-  }
-  
-  private SolrHighlighter createHighlighter(String className) {
-	return createInstance(className, SolrHighlighter.class, "Highlighter");
-  }
-  
-  /** 
-   * @return the last core initialized.  If you are using multiple cores, 
-   * this is not a function to use.
-   * 
-   * @deprecated Use {@link CoreContainer#getCore(String)} instead.
-   */
-  @Deprecated
-  public static SolrCore getSolrCore() {
-    synchronized( SolrCore.class ) {
-      if( instance == null ) {
-        try {
-          // sets 'instance' to the latest solr core
-          instance = new SolrCore( null, null, new SolrConfig(), null, null);
-        } catch(Exception xany) {
-          throw new SolrException( SolrException.ErrorCode.SERVER_ERROR,
-              "error creating core", xany );
-        }
-      }
-    }
-    return instance;
-  }
-  
-  /**
-   * 
-   * @param dataDir
-   * @param schema
-   * @throws SAXException 
-   * @throws IOException 
-   * @throws ParserConfigurationException 
-   * 
-   * @since solr 1.0
-   */
-  public SolrCore(String dataDir, IndexSchema schema) throws ParserConfigurationException, IOException, SAXException {
-    this(null, dataDir, new SolrConfig(), schema, null );
-  }
-  
-  /**
-   * Creates a new core and register it in the list of cores.
-   * If a core with the same name already exists, it will be stopped and replaced by this one.
-   *@param dataDir the index directory
-   *@param config a solr config instance
-   *@param schema a solr schema instance
-   *
-   *@since solr 1.3
-   */
-  public SolrCore(String name, String dataDir, SolrConfig config, IndexSchema schema, CoreDescriptor cd) {
-    synchronized (SolrCore.class) {
-      coreDescriptor = cd;
-      // this is for backward compatibility (and also the reason
-      // the sync block is needed)
-      instance = this;   // set singleton
-      this.setName( name );
-      SolrResourceLoader loader = config.getResourceLoader();
-      if (dataDir == null)
-        dataDir = config.get("dataDir",loader.getInstanceDir()+"data/");
-
-      dataDir = SolrResourceLoader.normalizeDir(dataDir);
-
-      log.info(logid+"Opening new SolrCore at " + loader.getInstanceDir() + ", dataDir="+dataDir);
-
-      if (schema==null) {
-        schema = new IndexSchema(config, IndexSchema.DEFAULT_SCHEMA_FILE, null);
-      }
-      
-      //Initialize JMX
-      if (config.jmxConfig.enabled) {
-        infoRegistry = new JmxMonitoredMap<String, SolrInfoMBean>(name, config.jmxConfig);
-      } else  {
-        log.info("JMX monitoring not detected for core: " + name);
-        infoRegistry = new LinkedHashMap<String, SolrInfoMBean>();
-      }
-
-      this.schema = schema;
-      this.dataDir = dataDir;
-      this.solrConfig = config;
-      this.startTime = System.currentTimeMillis();
-      this.maxWarmingSearchers = config.getInt("query/maxWarmingSearchers",Integer.MAX_VALUE);
-
-      booleanQueryMaxClauseCount();
   
-      parseListeners();
-
-      initDeletionPolicy();
-
-      initIndex();
-      
-      initWriters();
-      initQParsers();
-      initValueSourceParsers();
-      
-      this.searchComponents = loadSearchComponents( config );
-
-      // Processors initialized before the handlers
-      updateProcessorChains = loadUpdateProcessorChains();
-      reqHandlers = new RequestHandlers(this);
-      reqHandlers.initHandlersFromConfig( solrConfig );
-  
-      highlighter = createHighlighter(
-    		  solrConfig.get("highlighting/@class", DefaultSolrHighlighter.class.getName())
-      );
-      highlighter.initalize( solrConfig );
-      
-      // Handle things that should eventually go away
-      initDeprecatedSupport();
-
-      final CountDownLatch latch = new CountDownLatch(1);
-
-      try {
-        // cause the executor to stall so firstSearcher events won't fire
-        // until after inform() has been called for all components.
-        // searchExecutor must be single-threaded for this to work
-        searcherExecutor.submit(new Callable() {
-          public Object call() throws Exception {
-            latch.await();
-            return null;
-          }
-        });
-
-        // Open the searcher *before* the update handler so we don't end up opening
-        // one in the middle.
-        // With lockless commits in Lucene now, this probably shouldn't be an issue anymore
-        getSearcher(false,false,null);
   
-        updateHandler = createUpdateHandler(
-          solrConfig.get("updateHandler/@class", DirectUpdateHandler2.class.getName())
-        );
-
-        infoRegistry.put("updateHandler", updateHandler);
-
-        // Finally tell anyone who wants to know
-        loader.inform( loader );
-        loader.inform( this );
-
-      } catch (IOException e) {
-        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
-      } finally {
-        // allow firstSearcher events to fire
-        latch.countDown();
-      }
-    } // end synchronized
-
-    infoRegistry.put("core", this);
-  }
-
-
-  /**
-   * Load the request processors configured in solrconfig.xml
-   */
-  private Map<String,UpdateRequestProcessorChain> loadUpdateProcessorChains() {
-    final Map<String,UpdateRequestProcessorChain> map = new HashMap<String, UpdateRequestProcessorChain>();
-    
-    final String parsingErrorText = "Parsing Update Request Processor Chain";
-    UpdateRequestProcessorChain def = null;
-    
-    // This is kinda ugly, but at least it keeps the xpath logic in one place
-    // away from the Processors themselves.  
-    XPath xpath = solrConfig.getXPath();
-    NodeList nodes = (NodeList)solrConfig.evaluate("updateRequestProcessorChain", XPathConstants.NODESET);
-    boolean requireName = nodes.getLength() > 1;
-    if (nodes !=null ) {
-      for (int i=0; i<nodes.getLength(); i++) {
-        Node node = nodes.item(i);
-        String name       = DOMUtil.getAttr(node,"name", requireName?parsingErrorText:null);
-        boolean isDefault = "true".equals( DOMUtil.getAttr(node,"default", null ) );
-        
-        NodeList links = null;
-        try {
-          links = (NodeList)xpath.evaluate("processor", node, XPathConstants.NODESET);
-        } 
-        catch (XPathExpressionException e) {
-          throw new SolrException( SolrException.ErrorCode.SERVER_ERROR,"Error reading processors",e,false);
-        }
-        if( links == null || links.getLength() < 1 ) {
-          throw new RuntimeException( "updateRequestProcessorChain require at least one processor");
-        }
-        
-        // keep a list of the factories...
-        final ArrayList<UpdateRequestProcessorFactory> factories = new ArrayList<UpdateRequestProcessorFactory>(links.getLength());
-        // Load and initialize the plugin chain
-        AbstractPluginLoader<UpdateRequestProcessorFactory> loader 
-            = new AbstractPluginLoader<UpdateRequestProcessorFactory>( "processor chain", false, false ) {
-          @Override
-          protected void init(UpdateRequestProcessorFactory plugin, Node node) throws Exception {
-            plugin.init( (node==null)?null:DOMUtil.childNodesToNamedList(node) );
-          }
-    
-          @Override
-          protected UpdateRequestProcessorFactory register(String name, UpdateRequestProcessorFactory plugin) throws Exception {
-            factories.add( plugin );
-            return null;
-          }
-        };
-        loader.load( solrConfig.getResourceLoader(), links );
-        
-        
-        UpdateRequestProcessorChain chain = new UpdateRequestProcessorChain( 
-            factories.toArray( new UpdateRequestProcessorFactory[factories.size()] ) );
-        if( isDefault || nodes.getLength()==1 ) {
-          def = chain;
-        }
-        if( name != null ) {
-          map.put(name, chain);
-        }
-      }
-    }
-    
-    if( def == null ) {
-      // construct the default chain
-      UpdateRequestProcessorFactory[] factories = new UpdateRequestProcessorFactory[] {
-        new RunUpdateProcessorFactory(),
-        new LogUpdateProcessorFactory()
-      };
-      def = new UpdateRequestProcessorChain( factories );
-    }
-    map.put( null, def );
-    map.put( "", def );
-    return map;
-  }
   
   /**
    * @return an update processor registered to the given name.  Throw an exception if this chain is undefined
@@ -668,14 +413,13 @@
       return refCount.get() <= 0;
   }
   
+  @Override
   protected void finalize() {
     if (getOpenCount() != 0) {
       log.error("REFCOUNT ERROR: unreferenced " + this + " (" + getName() + ") has a reference count of " + getOpenCount());
     }
   }
 
-  private List<CloseHook> closeHooks = null;
-
    /**
     * Add a close callback hook
     */
@@ -736,39 +480,6 @@
     return reqHandlers.register(handlerName,handler);
   }
   
-  /**
-   * Register the default search components
-   */
-  private static Map<String, SearchComponent> loadSearchComponents( SolrConfig config )
-  {
-    Map<String, SearchComponent> components = new HashMap<String, SearchComponent>();
-  
-    String xpath = "searchComponent";
-    NamedListPluginLoader<SearchComponent> loader = new NamedListPluginLoader<SearchComponent>( xpath, components );
-    loader.load( config.getResourceLoader(), (NodeList)config.evaluate( xpath, XPathConstants.NODESET ) );
-  
-    final Map<String,Class<? extends SearchComponent>> standardcomponents 
-        = new HashMap<String, Class<? extends SearchComponent>>();
-    standardcomponents.put( QueryComponent.COMPONENT_NAME,        QueryComponent.class        );
-    standardcomponents.put( FacetComponent.COMPONENT_NAME,        FacetComponent.class        );
-    standardcomponents.put( MoreLikeThisComponent.COMPONENT_NAME, MoreLikeThisComponent.class );
-    standardcomponents.put( HighlightComponent.COMPONENT_NAME,    HighlightComponent.class    );
-    standardcomponents.put( DebugComponent.COMPONENT_NAME,        DebugComponent.class        );
-    for( Map.Entry<String, Class<? extends SearchComponent>> entry : standardcomponents.entrySet() ) {
-      if( components.get( entry.getKey() ) == null ) {
-        try {
-          SearchComponent comp = entry.getValue().newInstance();
-          comp.init( null ); // default components initialized with nothing
-          components.put( entry.getKey(), comp );
-        }
-        catch (Exception e) {
-          SolrConfig.severeErrors.add( e );
-          SolrException.logOnce(log,null,e);
-        }
-      }
-    }
-    return components;
-  }
   
   /**
    * @return a Search Component registered to a given name.  Throw an exception if the component is undefined
@@ -819,7 +530,6 @@
   final ExecutorService searcherExecutor = Executors.newSingleThreadExecutor();
   private int onDeckSearchers;  // number of searchers preparing
   private Object searcherLock = new Object();  // the sync object for the searcher
-  private final int maxWarmingSearchers;  // max number of on-deck searchers allowed
 
   /**
   * Return a registered {@link RefCounted}&lt;{@link SolrIndexSearcher}&gt; with
@@ -940,9 +650,9 @@
         // should never happen... just a sanity check
         log.error(logid+"ERROR!!! onDeckSearchers is " + onDeckSearchers);
         onDeckSearchers=1;  // reset
-      } else if (onDeckSearchers > maxWarmingSearchers) {
+      } else if (onDeckSearchers > configuration.getMaxWarmingSearchers() ) {
         onDeckSearchers--;
-        String msg="Error opening new searcher. exceeded limit of maxWarmingSearchers="+maxWarmingSearchers + ", try again later.";
+        String msg="Error opening new searcher. exceeded limit of maxWarmingSearchers="+configuration.getMaxWarmingSearchers() + ", try again later.";
         log.warn(logid+""+ msg);
         // HTTP 503==service unavailable, or 409==Conflict
         throw new SolrException(SolrException.ErrorCode.SERVICE_UNAVAILABLE,msg,true);
@@ -1001,11 +711,11 @@
       boolean alreadyRegistered = false;
       synchronized (searcherLock) {
         _searchers.add(newSearchHolder);
-
+       
         if (_searcher == null) {
           // if there isn't a current searcher then we may
           // want to register this one before warming is complete instead of waiting.
-          if (solrConfig.getBool("query/useColdSearcher",false)) {
+          if ( configuration.isUseColdSearcher() ) {
             registerSearcher(newSearchHolder);
             decrementOnDeckCount[0]=false;
             alreadyRegistered=true;
@@ -1145,6 +855,7 @@
 
   private RefCounted<SolrIndexSearcher> newHolder(SolrIndexSearcher newSearcher) {
     RefCounted<SolrIndexSearcher> holder = new RefCounted<SolrIndexSearcher>(newSearcher) {
+      @Override
       public void close() {
         try {
           synchronized(searcherLock) {
@@ -1277,46 +988,6 @@
 
   
   
-  private QueryResponseWriter defaultResponseWriter;
-  private final Map<String, QueryResponseWriter> responseWriters = new HashMap<String, QueryResponseWriter>();
-  
-  /** Configure the query response writers. There will always be a default writer; additional 
-   * writers may also be configured. */
-  private void initWriters() {
-    String xpath = "queryResponseWriter";
-    NodeList nodes = (NodeList) solrConfig.evaluate(xpath, XPathConstants.NODESET);
-    
-    NamedListPluginLoader<QueryResponseWriter> loader = 
-      new NamedListPluginLoader<QueryResponseWriter>( "[solrconfig.xml] "+xpath, responseWriters );
-    
-    defaultResponseWriter = loader.load( solrConfig.getResourceLoader(), nodes );
-    
-    // configure the default response writer; this one should never be null
-    if (defaultResponseWriter == null) {
-      defaultResponseWriter = responseWriters.get("standard");
-      if( defaultResponseWriter == null ) {
-        defaultResponseWriter = new XMLResponseWriter();
-      }
-    }
-
-    // make JSON response writers available by default
-    if (responseWriters.get("json")==null) {
-      responseWriters.put("json", new JSONResponseWriter());
-    }
-    if (responseWriters.get("python")==null) {
-      responseWriters.put("python", new PythonResponseWriter());
-    }
-    if (responseWriters.get("ruby")==null) {
-      responseWriters.put("ruby", new RubyResponseWriter());
-    }
-    if (responseWriters.get("raw")==null) {
-      responseWriters.put("raw", new RawResponseWriter());
-    }
-    if (responseWriters.get("javabin") == null) {
-      responseWriters.put("javabin", new BinaryResponseWriter());
-    }
-  }
-  
   /** Finds a writer by name, or returns the default writer if not found. */
   public final QueryResponseWriter getQueryResponseWriter(String writerName) {
     if (writerName != null) {
@@ -1335,114 +1006,19 @@
     return getQueryResponseWriter(request.getParams().get(CommonParams.WT)); 
   }
 
-  private final Map<String, QParserPlugin> qParserPlugins = new HashMap<String, QParserPlugin>();
-
-  /** Configure the query parsers. */
-  private void initQParsers() {
-    String xpath = "queryParser";
-    NodeList nodes = (NodeList) solrConfig.evaluate(xpath, XPathConstants.NODESET);
-
-    NamedListPluginLoader<QParserPlugin> loader =
-      new NamedListPluginLoader<QParserPlugin>( "[solrconfig.xml] "+xpath, qParserPlugins);
-
-    loader.load( solrConfig.getResourceLoader(), nodes );
-
-    // default parsers
-    for (int i=0; i<QParserPlugin.standardPlugins.length; i+=2) {
-     try {
-       String name = (String)QParserPlugin.standardPlugins[i];
-       if (null == qParserPlugins.get(name)) {
-         Class<QParserPlugin> clazz = (Class<QParserPlugin>)QParserPlugin.standardPlugins[i+1];
-         QParserPlugin plugin = clazz.newInstance();
-         qParserPlugins.put(name, plugin);
-         plugin.init(null);
-       }
-     } catch (Exception e) {
-       throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
-     }
-    }
-  }
-
   public QParserPlugin getQueryPlugin(String parserName) {
     QParserPlugin plugin = qParserPlugins.get(parserName);
     if (plugin != null) return plugin;
-    throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Unknown query type '"+parserName+"'");
-  }
-  
-  private final HashMap<String, ValueSourceParser> valueSourceParsers = new HashMap<String, ValueSourceParser>();
-  
-  /** Configure the ValueSource (function) plugins */
-  private void initValueSourceParsers() {
-    String xpath = "valueSourceParser";
-    NodeList nodes = (NodeList) solrConfig.evaluate(xpath, XPathConstants.NODESET);
-
-    NamedListPluginLoader<ValueSourceParser> loader =
-      new NamedListPluginLoader<ValueSourceParser>( "[solrconfig.xml] "+xpath, valueSourceParsers);
-
-    loader.load( solrConfig.getResourceLoader(), nodes );
-
-    // default value source parsers
-    for (Map.Entry<String, ValueSourceParser> entry : ValueSourceParser.standardValueSourceParsers.entrySet()) {
-      try {
-        String name = entry.getKey();
-        if (null == valueSourceParsers.get(name)) {
-          ValueSourceParser valueSourceParser = entry.getValue();
-          valueSourceParsers.put(name, valueSourceParser);
-          valueSourceParser.init(null);
-        }
-      } catch (Exception e) {
-        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
-      }
-    }
+    throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Unknown query parser '"+parserName+"'");
   }
   
   public ValueSourceParser getValueSourceParser(String parserName) {
     return valueSourceParsers.get(parserName);
   }
-  
-  /**
-   * Manage anything that should be taken care of in case configs change
-   */
-  private void initDeprecatedSupport()
-  {
-    // TODO -- this should be removed in deprecation release...
-    String gettable = solrConfig.get("admin/gettableFiles", null );
-    if( gettable != null ) {
-      log.warn( 
-          "solrconfig.xml uses deprecated <admin/gettableFiles>, Please "+
-          "update your config to use the ShowFileRequestHandler." );
-      if( getRequestHandler( "/admin/file" ) == null ) {
-        NamedList<String> invariants = new NamedList<String>();
-        
-        // Hide everything...
-        Set<String> hide = new HashSet<String>();
-        File configdir = new File( solrConfig.getResourceLoader().getConfigDir() ); 
-        for( String file : configdir.list() ) {
-          hide.add( file.toUpperCase() );
-        }
-        
-        // except the "gettable" list
-        StringTokenizer st = new StringTokenizer( gettable );
-        while( st.hasMoreTokens() ) {
-          hide.remove( st.nextToken().toUpperCase() );
-        }
-        for( String s : hide ) {
-          invariants.add( ShowFileRequestHandler.HIDDEN, s );
-        }
-        
-        NamedList<Object> args = new NamedList<Object>();
-        args.add( "invariants", invariants );
-        ShowFileRequestHandler handler = new ShowFileRequestHandler();
-        handler.init( args );
-        reqHandlers.register("/admin/file", handler);
-
-        log.warn( "adding ShowFileRequestHandler with hidden files: "+hide );
-      }
-    }
-  } 
 
+  @Deprecated
   public CoreDescriptor getCoreDescriptor() {
-    return coreDescriptor;
+    return initalizer.getCoreDescriptor();
   }
 
   public IndexDeletionPolicyWrapper getDeletionPolicy(){
@@ -1486,6 +1062,13 @@
     return lst;
   }
 
+  public long getStartTime() {
+    return startTime;
+  }
+
+  public SolrConfiguraion getConfiguration() {
+    return configuration;
+  }
 }
 
 

Modified: lucene/solr/branches/sandbox/ryan/src/java/org/apache/solr/handler/admin/CoreAdminHandler.java
URL: http://svn.apache.org/viewvc/lucene/solr/branches/sandbox/ryan/src/java/org/apache/solr/handler/admin/CoreAdminHandler.java?rev=701592&r1=701591&r2=701592&view=diff
==============================================================================
--- lucene/solr/branches/sandbox/ryan/src/java/org/apache/solr/handler/admin/CoreAdminHandler.java (original)
+++ lucene/solr/branches/sandbox/ryan/src/java/org/apache/solr/handler/admin/CoreAdminHandler.java Fri Oct  3 20:55:00 2008
@@ -27,9 +27,9 @@
 import org.apache.solr.common.params.CoreAdminParams.CoreAdminAction;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.SimpleOrderedMap;
+import org.apache.solr.config.CoreDescriptor;
 import org.apache.solr.core.CoreContainer;
 import org.apache.solr.core.SolrCore;
-import org.apache.solr.core.CoreDescriptor;
 import org.apache.solr.handler.RequestHandlerBase;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.request.SolrQueryResponse;

Modified: lucene/solr/branches/sandbox/ryan/src/java/org/apache/solr/handler/admin/ShowFileRequestHandler.java
URL: http://svn.apache.org/viewvc/lucene/solr/branches/sandbox/ryan/src/java/org/apache/solr/handler/admin/ShowFileRequestHandler.java?rev=701592&r1=701591&r2=701592&view=diff
==============================================================================
--- lucene/solr/branches/sandbox/ryan/src/java/org/apache/solr/handler/admin/ShowFileRequestHandler.java (original)
+++ lucene/solr/branches/sandbox/ryan/src/java/org/apache/solr/handler/admin/ShowFileRequestHandler.java Fri Oct  3 20:55:00 2008
@@ -30,7 +30,7 @@
 import org.apache.solr.common.util.ContentStreamBase;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.SimpleOrderedMap;
-import org.apache.solr.core.SolrResourceLoader;
+import org.apache.solr.config.SolrResourceLoader;
 import org.apache.solr.handler.RequestHandlerBase;
 import org.apache.solr.request.RawResponseWriter;
 import org.apache.solr.request.SolrQueryRequest;

Modified: lucene/solr/branches/sandbox/ryan/src/java/org/apache/solr/handler/component/QueryElevationComponent.java
URL: http://svn.apache.org/viewvc/lucene/solr/branches/sandbox/ryan/src/java/org/apache/solr/handler/component/QueryElevationComponent.java?rev=701592&r1=701591&r2=701592&view=diff
==============================================================================
--- lucene/solr/branches/sandbox/ryan/src/java/org/apache/solr/handler/component/QueryElevationComponent.java (original)
+++ lucene/solr/branches/sandbox/ryan/src/java/org/apache/solr/handler/component/QueryElevationComponent.java Fri Oct  3 20:55:00 2008
@@ -57,7 +57,7 @@
 import org.apache.solr.common.util.DOMUtil;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.SimpleOrderedMap;
-import org.apache.solr.core.Config;
+import org.apache.solr.config.Config;
 import org.apache.solr.core.SolrCore;
 import org.apache.solr.schema.FieldType;
 import org.apache.solr.schema.SchemaField;

Modified: lucene/solr/branches/sandbox/ryan/src/java/org/apache/solr/handler/component/SearchComponent.java
URL: http://svn.apache.org/viewvc/lucene/solr/branches/sandbox/ryan/src/java/org/apache/solr/handler/component/SearchComponent.java?rev=701592&r1=701591&r2=701592&view=diff
==============================================================================
--- lucene/solr/branches/sandbox/ryan/src/java/org/apache/solr/handler/component/SearchComponent.java (original)
+++ lucene/solr/branches/sandbox/ryan/src/java/org/apache/solr/handler/component/SearchComponent.java Fri Oct  3 20:55:00 2008
@@ -33,6 +33,12 @@
  */
 public abstract class SearchComponent implements SolrInfoMBean, NamedListInitializedPlugin
 {
+  //////////////////////// NamedListInitializedPlugin methods //////////////////////
+  public void init( NamedList args )
+  {
+    // By default do nothing
+  }
+  
   /**
    * Prepare the response.  Guaranteed to be called before any SearchComponent {@link #process(org.apache.solr.handler.component.ResponseBuilder)} method.
    * Called for every incoming request.
@@ -71,13 +77,6 @@
    */
   public void finishStage(ResponseBuilder rb) {
   }
-
-
-  //////////////////////// NamedListInitializedPlugin methods //////////////////////
-  public void init( NamedList args )
-  {
-    // By default do nothing
-  }
   
   //////////////////////// SolrInfoMBeans methods //////////////////////
 

Modified: lucene/solr/branches/sandbox/ryan/src/java/org/apache/solr/handler/component/SpellCheckComponent.java
URL: http://svn.apache.org/viewvc/lucene/solr/branches/sandbox/ryan/src/java/org/apache/solr/handler/component/SpellCheckComponent.java?rev=701592&r1=701591&r2=701592&view=diff
==============================================================================
--- lucene/solr/branches/sandbox/ryan/src/java/org/apache/solr/handler/component/SpellCheckComponent.java (original)
+++ lucene/solr/branches/sandbox/ryan/src/java/org/apache/solr/handler/component/SpellCheckComponent.java Fri Oct  3 20:55:00 2008
@@ -43,10 +43,10 @@
 import org.apache.solr.common.params.SpellingParams;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.SimpleOrderedMap;
-import org.apache.solr.core.SolrConfig;
+import org.apache.solr.config.SolrConfig;
+import org.apache.solr.config.SolrResourceLoader;
 import org.apache.solr.core.SolrCore;
 import org.apache.solr.core.SolrEventListener;
-import org.apache.solr.core.SolrResourceLoader;
 import org.apache.solr.schema.FieldType;
 import org.apache.solr.schema.IndexSchema;
 import org.apache.solr.search.SolrIndexSearcher;

Modified: lucene/solr/branches/sandbox/ryan/src/java/org/apache/solr/highlight/DefaultSolrHighlighter.java
URL: http://svn.apache.org/viewvc/lucene/solr/branches/sandbox/ryan/src/java/org/apache/solr/highlight/DefaultSolrHighlighter.java?rev=701592&r1=701591&r2=701592&view=diff
==============================================================================
--- lucene/solr/branches/sandbox/ryan/src/java/org/apache/solr/highlight/DefaultSolrHighlighter.java (original)
+++ lucene/solr/branches/sandbox/ryan/src/java/org/apache/solr/highlight/DefaultSolrHighlighter.java Fri Oct  3 20:55:00 2008
@@ -47,7 +47,7 @@
 import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.SimpleOrderedMap;
-import org.apache.solr.core.Config;
+import org.apache.solr.config.Config;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.schema.IndexSchema;
 import org.apache.solr.schema.SchemaField;

Modified: lucene/solr/branches/sandbox/ryan/src/java/org/apache/solr/highlight/SolrHighlighter.java
URL: http://svn.apache.org/viewvc/lucene/solr/branches/sandbox/ryan/src/java/org/apache/solr/highlight/SolrHighlighter.java?rev=701592&r1=701591&r2=701592&view=diff
==============================================================================
--- lucene/solr/branches/sandbox/ryan/src/java/org/apache/solr/highlight/SolrHighlighter.java (original)
+++ lucene/solr/branches/sandbox/ryan/src/java/org/apache/solr/highlight/SolrHighlighter.java Fri Oct  3 20:55:00 2008
@@ -27,7 +27,7 @@
 import org.apache.solr.common.params.HighlightParams;
 import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.util.NamedList;
-import org.apache.solr.core.Config;
+import org.apache.solr.config.Config;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.search.DocList;
 import org.apache.solr.util.SolrPluginUtils;

Modified: lucene/solr/branches/sandbox/ryan/src/java/org/apache/solr/request/XSLTResponseWriter.java
URL: http://svn.apache.org/viewvc/lucene/solr/branches/sandbox/ryan/src/java/org/apache/solr/request/XSLTResponseWriter.java?rev=701592&r1=701591&r2=701592&view=diff
==============================================================================
--- lucene/solr/branches/sandbox/ryan/src/java/org/apache/solr/request/XSLTResponseWriter.java (original)
+++ lucene/solr/branches/sandbox/ryan/src/java/org/apache/solr/request/XSLTResponseWriter.java Fri Oct  3 20:55:00 2008
@@ -32,9 +32,9 @@
 import javax.xml.transform.stream.StreamResult;
 import javax.xml.transform.stream.StreamSource;
 
-import org.apache.solr.core.SolrConfig;
 import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.util.NamedList;
+import org.apache.solr.config.SolrConfig;
 import org.apache.solr.util.xslt.TransformerProvider;
 
 /** QueryResponseWriter which captures the output of the XMLWriter

Modified: lucene/solr/branches/sandbox/ryan/src/java/org/apache/solr/schema/IndexSchema.java
URL: http://svn.apache.org/viewvc/lucene/solr/branches/sandbox/ryan/src/java/org/apache/solr/schema/IndexSchema.java?rev=701592&r1=701591&r2=701592&view=diff
==============================================================================
--- lucene/solr/branches/sandbox/ryan/src/java/org/apache/solr/schema/IndexSchema.java (original)
+++ lucene/solr/branches/sandbox/ryan/src/java/org/apache/solr/schema/IndexSchema.java Fri Oct  3 20:55:00 2008
@@ -26,9 +26,10 @@
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.util.DOMUtil;
-import org.apache.solr.core.SolrConfig;
-import org.apache.solr.core.Config;
-import org.apache.solr.core.SolrResourceLoader;
+import org.apache.solr.config.Config;
+import org.apache.solr.config.SolrConfig;
+import org.apache.solr.config.SolrConfiguraion;
+import org.apache.solr.config.SolrResourceLoader;
 import org.apache.solr.analysis.TokenFilterFactory;
 import org.apache.solr.analysis.TokenizerChain;
 import org.apache.solr.analysis.TokenizerFactory;
@@ -59,6 +60,7 @@
 
   final static Logger log = LoggerFactory.getLogger(IndexSchema.class);
   private final SolrConfig solrConfig;
+  private final SolrConfiguraion config;
   private final String resourceName;
   private String name;
   private float version;
@@ -70,10 +72,12 @@
    * By default, this follows the normal config path directory searching rules.
    * @see Config#openResource
    */
-  public IndexSchema(SolrConfig solrConfig, String name, InputStream is) {
+  public IndexSchema(SolrConfig solrConfig, SolrConfiguraion config, String name, InputStream is) {
     this.solrConfig = solrConfig;
     if (name == null)
       name = DEFAULT_SCHEMA_FILE;
+    
+    this.config = config;
     this.resourceName = name;
     SolrResourceLoader loader = solrConfig.getResourceLoader();
     InputStream lis = is;
@@ -341,7 +345,7 @@
   }
 
   private void readSchema(InputStream is) {
-    log.info("Reading Solr Schema");
+    log.info("Reading Solr Schema: "+this );
 
     try {
       // pass the config resource loader to avoid building an empty one for no reason:
@@ -1112,4 +1116,8 @@
     return  name.startsWith("*") || name.endsWith("*");
   }
 
+  public SolrConfiguraion getConfig() {
+    return config;
+  }
+
 }

Modified: lucene/solr/branches/sandbox/ryan/src/java/org/apache/solr/search/CacheConfig.java
URL: http://svn.apache.org/viewvc/lucene/solr/branches/sandbox/ryan/src/java/org/apache/solr/search/CacheConfig.java?rev=701592&r1=701591&r2=701592&view=diff
==============================================================================
--- lucene/solr/branches/sandbox/ryan/src/java/org/apache/solr/search/CacheConfig.java (original)
+++ lucene/solr/branches/sandbox/ryan/src/java/org/apache/solr/search/CacheConfig.java Fri Oct  3 20:55:00 2008
@@ -24,8 +24,8 @@
 
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.util.DOMUtil;
-import org.apache.solr.core.SolrConfig;
-import org.apache.solr.core.SolrResourceLoader;
+import org.apache.solr.config.SolrConfig;
+import org.apache.solr.config.SolrResourceLoader;
 
 import javax.xml.xpath.XPathConstants;
 

Modified: lucene/solr/branches/sandbox/ryan/src/java/org/apache/solr/search/DisMaxQParserPlugin.java
URL: http://svn.apache.org/viewvc/lucene/solr/branches/sandbox/ryan/src/java/org/apache/solr/search/DisMaxQParserPlugin.java?rev=701592&r1=701591&r2=701592&view=diff
==============================================================================
--- lucene/solr/branches/sandbox/ryan/src/java/org/apache/solr/search/DisMaxQParserPlugin.java (original)
+++ lucene/solr/branches/sandbox/ryan/src/java/org/apache/solr/search/DisMaxQParserPlugin.java Fri Oct  3 20:55:00 2008
@@ -41,7 +41,7 @@
  * across myfield and mytitle, with a higher weight on mytitle.
  */
 public class DisMaxQParserPlugin extends QParserPlugin {
-  public static String NAME = "dismax";
+  public static final String NAME = "dismax";
 
   public void init(NamedList args) {
   }

Modified: lucene/solr/branches/sandbox/ryan/src/java/org/apache/solr/search/QueryParsing.java
URL: http://svn.apache.org/viewvc/lucene/solr/branches/sandbox/ryan/src/java/org/apache/solr/search/QueryParsing.java?rev=701592&r1=701591&r2=701592&view=diff
==============================================================================
--- lucene/solr/branches/sandbox/ryan/src/java/org/apache/solr/search/QueryParsing.java (original)
+++ lucene/solr/branches/sandbox/ryan/src/java/org/apache/solr/search/QueryParsing.java Fri Oct  3 20:55:00 2008
@@ -654,9 +654,10 @@
    * </pre>
    */
   public static FunctionQuery parseFunction(String func, IndexSchema schema) throws ParseException {
-    SolrCore core = SolrCore.getSolrCore();
+    //SolrCore core = SolrCore.getSolrCore();
+    SolrCore core = SolrCore.INSTANCE;
     return (FunctionQuery)(QParser.getParser(func,"func",new LocalSolrQueryRequest(core,new HashMap())).parse());
-    // return new FunctionQuery(parseValSource(new StrParser(func), schema));
+    //return new FunctionQuery(parseValSourc(new StrParser(func), schema));
   }
 
 }

Modified: lucene/solr/branches/sandbox/ryan/src/java/org/apache/solr/search/SolrIndexSearcher.java
URL: http://svn.apache.org/viewvc/lucene/solr/branches/sandbox/ryan/src/java/org/apache/solr/search/SolrIndexSearcher.java?rev=701592&r1=701591&r2=701592&view=diff
==============================================================================
--- lucene/solr/branches/sandbox/ryan/src/java/org/apache/solr/search/SolrIndexSearcher.java (original)
+++ lucene/solr/branches/sandbox/ryan/src/java/org/apache/solr/search/SolrIndexSearcher.java Fri Oct  3 20:55:00 2008
@@ -26,7 +26,7 @@
 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;
+import org.apache.solr.config.SolrConfiguraion;
 import org.apache.solr.core.SolrCore;
 import org.apache.solr.core.SolrInfoMBean;
 import org.apache.solr.schema.IndexSchema;
@@ -116,27 +116,27 @@
     this.closeReader = closeReader;
     searcher.setSimilarity(schema.getSimilarity());
 
-    SolrConfig solrConfig = schema.getSolrConfig();
-    queryResultWindowSize = solrConfig.queryResultWindowSize;
-    queryResultMaxDocsCached = solrConfig.queryResultMaxDocsCached;
-    useFilterForSortedQuery = solrConfig.useFilterForSortedQuery;
-    enableLazyFieldLoading = solrConfig.enableLazyFieldLoading;
+    SolrConfiguraion solrConfig = schema.getConfig();
+    queryResultWindowSize = solrConfig.getQueryResultWindowSize();
+    queryResultMaxDocsCached = solrConfig.getQueryResultMaxDocsCached();
+    useFilterForSortedQuery = solrConfig.isUseFilterForSortedQuery();
+    enableLazyFieldLoading = solrConfig.isEnableLazyFieldLoading();
     
     cachingEnabled=enableCache;
     if (cachingEnabled) {
       ArrayList<SolrCache> clist = new ArrayList<SolrCache>();
-      filterCache= solrConfig.filterCacheConfig==null ? null : solrConfig.filterCacheConfig.newInstance();
+      filterCache= solrConfig.getFilterCacheConfig()==null ? null : solrConfig.getFilterCacheConfig().newInstance();
       if (filterCache!=null) clist.add(filterCache);
-      queryResultCache = solrConfig.queryResultCacheConfig==null ? null : solrConfig.queryResultCacheConfig.newInstance();
+      queryResultCache = solrConfig.getQueryResultCacheConfig() ==null ? null : solrConfig.getQueryResultCacheConfig().newInstance();
       if (queryResultCache!=null) clist.add(queryResultCache);
-      documentCache = solrConfig.documentCacheConfig==null ? null : solrConfig.documentCacheConfig.newInstance();
+      documentCache = solrConfig.getDocumentCacheConfig() ==null ? null : solrConfig.getDocumentCacheConfig().newInstance();
       if (documentCache!=null) clist.add(documentCache);
 
-      if (solrConfig.userCacheConfigs == null) {
+      if (solrConfig.getUserCacheConfigs() == null) {
         cacheMap = noGenericCaches;
       } else {
-        cacheMap = new HashMap<String,SolrCache>(solrConfig.userCacheConfigs.length);
-        for (CacheConfig userCacheConfig : solrConfig.userCacheConfigs) {
+        cacheMap = new HashMap<String,SolrCache>(solrConfig.getUserCacheConfigs().length);
+        for (CacheConfig userCacheConfig : solrConfig.getUserCacheConfigs()) {
           SolrCache cache = null;
           if (userCacheConfig != null) cache = userCacheConfig.newInstance();
           if (cache != null) {
@@ -154,11 +154,12 @@
       cacheMap = noGenericCaches;
       cacheList= noCaches;
     }
-    optimizer = solrConfig.filtOptEnabled ? new LuceneQueryOptimizer(solrConfig.filtOptCacheSize,solrConfig.filtOptThreshold) : null;
+    optimizer = solrConfig.isFiltOptEnabled()  
+        ? new LuceneQueryOptimizer(solrConfig.getFiltOptCacheSize(),solrConfig.getFiltOptThreshold()) : null;
 
     // for DocSets
-    HASHSET_INVERSE_LOAD_FACTOR = solrConfig.hashSetInverseLoadFactor;
-    HASHDOCSET_MAXSIZE = solrConfig.hashDocSetMaxSize;
+    HASHSET_INVERSE_LOAD_FACTOR = solrConfig.getHashSetInverseLoadFactor();
+    HASHDOCSET_MAXSIZE = solrConfig.getHashDocSetMaxSize();
   }
 
 
@@ -218,9 +219,10 @@
   //
   // Set default regenerators on filter and query caches if they don't have any
   //
-  public static void initRegenerators(SolrConfig solrConfig) {
-    if (solrConfig.filterCacheConfig != null && solrConfig.filterCacheConfig.getRegenerator() == null) {
-      solrConfig.filterCacheConfig.setRegenerator(
+  public static void initRegenerators(SolrConfiguraion config) {
+    CacheConfig cacheConfig = config.getFilterCacheConfig();
+    if (cacheConfig != null && cacheConfig.getRegenerator() == null) {
+      cacheConfig.setRegenerator(
               new CacheRegenerator() {
                 public boolean regenerateItem(SolrIndexSearcher newSearcher, SolrCache newCache, SolrCache oldCache, Object oldKey, Object oldVal) throws IOException {
                   newSearcher.cacheDocSet((Query)oldKey, null, false);
@@ -230,9 +232,10 @@
       );
     }
 
-    if (solrConfig.queryResultCacheConfig != null && solrConfig.queryResultCacheConfig.getRegenerator() == null) {
-      final int queryResultWindowSize = solrConfig.queryResultWindowSize;
-      solrConfig.queryResultCacheConfig.setRegenerator(
+    cacheConfig = config.getQueryResultCacheConfig();
+    if (cacheConfig != null && cacheConfig.getRegenerator() == null) {
+      final int queryResultWindowSize = config.getQueryResultWindowSize();
+      cacheConfig.setRegenerator(
               new CacheRegenerator() {
                 public boolean regenerateItem(SolrIndexSearcher newSearcher, SolrCache newCache, SolrCache oldCache, Object oldKey, Object oldVal) throws IOException {
                   QueryResultKey key = (QueryResultKey)oldKey;
@@ -1570,6 +1573,11 @@
   public long getOpenTime() {
     return openTime;
   }
+  
+  public SolrCore getCore()
+  {
+    return core;
+  }
 
   /////////////////////////////////////////////////////////////////////
   // SolrInfoMBean stuff: Statistics and Module Info

Modified: lucene/solr/branches/sandbox/ryan/src/java/org/apache/solr/spelling/FileBasedSpellChecker.java
URL: http://svn.apache.org/viewvc/lucene/solr/branches/sandbox/ryan/src/java/org/apache/solr/spelling/FileBasedSpellChecker.java?rev=701592&r1=701591&r2=701592&view=diff
==============================================================================
--- lucene/solr/branches/sandbox/ryan/src/java/org/apache/solr/spelling/FileBasedSpellChecker.java (original)
+++ lucene/solr/branches/sandbox/ryan/src/java/org/apache/solr/spelling/FileBasedSpellChecker.java Fri Oct  3 20:55:00 2008
@@ -30,8 +30,8 @@
 import org.apache.lucene.search.spell.PlainTextDictionary;
 import org.apache.lucene.store.RAMDirectory;
 import org.apache.solr.common.util.NamedList;
+import org.apache.solr.config.SolrResourceLoader;
 import org.apache.solr.core.SolrCore;
-import org.apache.solr.core.SolrResourceLoader;
 import org.apache.solr.schema.FieldType;
 import org.apache.solr.schema.IndexSchema;
 import org.apache.solr.util.HighFrequencyDictionary;

Modified: lucene/solr/branches/sandbox/ryan/src/java/org/apache/solr/update/DirectUpdateHandler2.java
URL: http://svn.apache.org/viewvc/lucene/solr/branches/sandbox/ryan/src/java/org/apache/solr/update/DirectUpdateHandler2.java?rev=701592&r1=701591&r2=701592&view=diff
==============================================================================
--- lucene/solr/branches/sandbox/ryan/src/java/org/apache/solr/update/DirectUpdateHandler2.java (original)
+++ lucene/solr/branches/sandbox/ryan/src/java/org/apache/solr/update/DirectUpdateHandler2.java Fri Oct  3 20:55:00 2008
@@ -24,6 +24,7 @@
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.MatchAllDocsQuery;
 
+import java.util.List;
 import java.util.concurrent.Executors;
 import java.util.concurrent.Future;
 import java.util.concurrent.ExecutionException;
@@ -42,6 +43,7 @@
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.SimpleOrderedMap;
 import org.apache.solr.core.SolrCore;
+import org.apache.solr.core.SolrEventListener;
 
 /**
  * <code>DirectUpdateHandler2</code> implements an UpdateHandler where documents are added
@@ -140,8 +142,10 @@
 
   protected IndexWriter writer;
 
-  public DirectUpdateHandler2(SolrCore core) throws IOException {
-    super(core);
+  public DirectUpdateHandler2(SolrCore core, 
+      List<SolrEventListener> commitCallbacks,
+      List<SolrEventListener> optimizeCallbacks ) throws IOException {
+    super(core, commitCallbacks, optimizeCallbacks);
 
     ReadWriteLock rwl = new ReentrantReadWriteLock();
     iwAccess = rwl.readLock();

Modified: lucene/solr/branches/sandbox/ryan/src/java/org/apache/solr/update/SolrIndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/solr/branches/sandbox/ryan/src/java/org/apache/solr/update/SolrIndexWriter.java?rev=701592&r1=701591&r2=701592&view=diff
==============================================================================
--- lucene/solr/branches/sandbox/ryan/src/java/org/apache/solr/update/SolrIndexWriter.java (original)
+++ lucene/solr/branches/sandbox/ryan/src/java/org/apache/solr/update/SolrIndexWriter.java Fri Oct  3 20:55:00 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.config.SolrIndexConfig;
 import org.apache.solr.schema.IndexSchema;
 
 import org.slf4j.Logger;

Modified: lucene/solr/branches/sandbox/ryan/src/java/org/apache/solr/update/UpdateHandler.java
URL: http://svn.apache.org/viewvc/lucene/solr/branches/sandbox/ryan/src/java/org/apache/solr/update/UpdateHandler.java?rev=701592&r1=701591&r2=701592&view=diff
==============================================================================
--- lucene/solr/branches/sandbox/ryan/src/java/org/apache/solr/update/UpdateHandler.java (original)
+++ lucene/solr/branches/sandbox/ryan/src/java/org/apache/solr/update/UpdateHandler.java Fri Oct  3 20:55:00 2008
@@ -23,12 +23,10 @@
 import org.apache.lucene.document.Field;
 import org.apache.lucene.document.Fieldable;
 import org.apache.lucene.search.HitCollector;
-import org.w3c.dom.NodeList;
-import org.w3c.dom.Node;
-
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-import java.util.Vector;
+
+import java.util.List;
 import java.io.IOException;
 
 import org.apache.solr.search.SolrIndexSearcher;
@@ -36,12 +34,9 @@
 import org.apache.solr.schema.SchemaField;
 import org.apache.solr.schema.FieldType;
 import org.apache.solr.common.SolrException;
-import org.apache.solr.common.util.DOMUtil;
 import org.apache.solr.util.plugin.SolrCoreAware;
 import org.apache.solr.core.*;
 
-import javax.xml.xpath.XPathConstants;
-
 /**
  * <code>UpdateHandler</code> handles requests to change the index
  * (adds, deletes, commits, optimizes, etc).
@@ -60,44 +55,9 @@
   protected final FieldType idFieldType;
   protected final Term idTerm; // prototype term to avoid interning fieldname
 
-  protected Vector<SolrEventListener> commitCallbacks = new Vector<SolrEventListener>();
-  protected Vector<SolrEventListener> optimizeCallbacks = new Vector<SolrEventListener>();
-
-  private void parseEventListeners() {
-    final SolrConfig solrConfig = core.getSolrConfig();
-    NodeList nodes = (NodeList) solrConfig.evaluate("updateHandler/listener[@event=\"postCommit\"]", XPathConstants.NODESET);
-    if (nodes!=null) {
-      for (int i=0; i<nodes.getLength(); i++) {
-        Node node = nodes.item(i);
-        try {
-          String className = DOMUtil.getAttr(node,"class");
-          SolrEventListener listener = core.createEventListener(className);
-          listener.init(DOMUtil.childNodesToNamedList(node));
-          // listener.init(DOMUtil.toMapExcept(node.getAttributes(),"class","synchronized"));
-          commitCallbacks.add(listener);
-          log.info("added SolrEventListener for postCommit: " + listener);
-        } catch (Exception e) {
-          throw new SolrException( SolrException.ErrorCode.SERVER_ERROR,"error parsing event listevers", e, false);
-        }
-      }
-    }
-    nodes = (NodeList) solrConfig.evaluate("updateHandler/listener[@event=\"postOptimize\"]", XPathConstants.NODESET);
-    if (nodes!=null) {
-      for (int i=0; i<nodes.getLength(); i++) {
-        Node node = nodes.item(i);
-        try {
-          String className = DOMUtil.getAttr(node,"class");
-          SolrEventListener listener = core.createEventListener(className);
-          listener.init(DOMUtil.childNodesToNamedList(node));
-          optimizeCallbacks.add(listener);
-          log.info("added SolarEventListener for postOptimize: " + listener);
-        } catch (Exception e) {
-          throw new SolrException( SolrException.ErrorCode.SERVER_ERROR,"error parsing event listeners", e, false);
-        }
-      }
-    }
-  }
-
+  protected final List<SolrEventListener> commitCallbacks;
+  protected final List<SolrEventListener> optimizeCallbacks;
+  
   protected void callPostCommitCallbacks() {
     for (SolrEventListener listener : commitCallbacks) {
       listener.postCommit();
@@ -110,17 +70,22 @@
     }
   }
 
-  public UpdateHandler(SolrCore core)  {
+  public UpdateHandler(SolrCore core, 
+      List<SolrEventListener> commitCallbacks,
+      List<SolrEventListener> optimizeCallbacks )  {
     this.core=core;
     schema = core.getSchema();
     idField = schema.getUniqueKeyField();
     idFieldType = idField!=null ? idField.getType() : null;
     idTerm = idField!=null ? new Term(idField.getName(),"") : null;
-    parseEventListeners();
+    
+    this.commitCallbacks = commitCallbacks;
+    this.optimizeCallbacks = optimizeCallbacks;
   }
 
   protected SolrIndexWriter createMainIndexWriter(String name, boolean removeAllExisting) throws IOException {
-    return new SolrIndexWriter(name,core.getIndexDir(), removeAllExisting, schema, core.getSolrConfig().mainIndexConfig, core.getDeletionPolicy());
+    return new SolrIndexWriter(name,core.getIndexDir(), removeAllExisting, schema, 
+        core.getConfiguration().getMainIndexConfig(), core.getDeletionPolicy());
   }
 
   protected final Term idTerm(String readableId) {

Modified: lucene/solr/branches/sandbox/ryan/src/java/org/apache/solr/util/AbstractSolrTestCase.java
URL: http://svn.apache.org/viewvc/lucene/solr/branches/sandbox/ryan/src/java/org/apache/solr/util/AbstractSolrTestCase.java?rev=701592&r1=701591&r2=701592&view=diff
==============================================================================
--- lucene/solr/branches/sandbox/ryan/src/java/org/apache/solr/util/AbstractSolrTestCase.java (original)
+++ lucene/solr/branches/sandbox/ryan/src/java/org/apache/solr/util/AbstractSolrTestCase.java Fri Oct  3 20:55:00 2008
@@ -19,9 +19,9 @@
 package org.apache.solr.util;
 
 
-import org.apache.solr.core.SolrConfig;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.util.XML;
+import org.apache.solr.config.SolrConfig;
 import org.apache.solr.request.*;
 import org.apache.solr.util.TestHarness;
 

Modified: lucene/solr/branches/sandbox/ryan/src/java/org/apache/solr/util/TestHarness.java
URL: http://svn.apache.org/viewvc/lucene/solr/branches/sandbox/ryan/src/java/org/apache/solr/util/TestHarness.java?rev=701592&r1=701591&r2=701592&view=diff
==============================================================================
--- lucene/solr/branches/sandbox/ryan/src/java/org/apache/solr/util/TestHarness.java (original)
+++ lucene/solr/branches/sandbox/ryan/src/java/org/apache/solr/util/TestHarness.java Fri Oct  3 20:55:00 2008
@@ -24,11 +24,12 @@
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.SimpleOrderedMap;
 import org.apache.solr.common.util.XML;
-import org.apache.solr.core.SolrConfig;
+import org.apache.solr.config.CoreDescriptor;
+import org.apache.solr.config.CoreInitalizer;
+import org.apache.solr.config.SolrConfig;
+import org.apache.solr.config.SolrResourceLoader;
 import org.apache.solr.core.SolrCore;
 import org.apache.solr.core.CoreContainer;
-import org.apache.solr.core.CoreDescriptor;
-import org.apache.solr.core.SolrResourceLoader;
 import org.apache.solr.handler.XmlUpdateRequestHandler;
 import org.apache.solr.request.LocalSolrQueryRequest;
 import org.apache.solr.request.QueryResponseWriter;
@@ -122,7 +123,7 @@
       public TestHarness( String dataDirectory,
                           SolrConfig solrConfig,
                           String schemaFile) {
-     this( dataDirectory, solrConfig, new IndexSchema(solrConfig, schemaFile, null));
+     this( dataDirectory, solrConfig, new IndexSchema(solrConfig, solrConfig.vars, schemaFile, null));
    }
    /**
     * @param dataDirectory path for index data, will not be cleaned up
@@ -180,7 +181,8 @@
       CoreDescriptor dcore = new CoreDescriptor(container, coreName, solrConfig.getResourceLoader().getInstanceDir());
       dcore.setConfigName(solrConfig.getResourceName());
       dcore.setSchemaName(indexSchema.getResourceName());
-      SolrCore core = new SolrCore( null, dataDirectory, solrConfig, indexSchema, dcore);
+      CoreInitalizer initalizer = new CoreInitalizer(null, dataDirectory, solrConfig, indexSchema, dcore);
+      SolrCore core = initalizer.initalizeCore( coreName );
       container.register(coreName, core, false);
       return container;
     }

Modified: lucene/solr/branches/sandbox/ryan/src/java/org/apache/solr/util/plugin/AbstractPluginLoader.java
URL: http://svn.apache.org/viewvc/lucene/solr/branches/sandbox/ryan/src/java/org/apache/solr/util/plugin/AbstractPluginLoader.java?rev=701592&r1=701591&r2=701592&view=diff
==============================================================================
--- lucene/solr/branches/sandbox/ryan/src/java/org/apache/solr/util/plugin/AbstractPluginLoader.java (original)
+++ lucene/solr/branches/sandbox/ryan/src/java/org/apache/solr/util/plugin/AbstractPluginLoader.java Fri Oct  3 20:55:00 2008
@@ -25,7 +25,7 @@
 import org.apache.solr.common.ResourceLoader;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.util.DOMUtil;
-import org.apache.solr.core.SolrConfig;
+import org.apache.solr.config.SolrConfig;
 import org.w3c.dom.Node;
 import org.w3c.dom.NodeList;
 

Modified: lucene/solr/branches/sandbox/ryan/src/java/org/apache/solr/util/xslt/TransformerProvider.java
URL: http://svn.apache.org/viewvc/lucene/solr/branches/sandbox/ryan/src/java/org/apache/solr/util/xslt/TransformerProvider.java?rev=701592&r1=701591&r2=701592&view=diff
==============================================================================
--- lucene/solr/branches/sandbox/ryan/src/java/org/apache/solr/util/xslt/TransformerProvider.java (original)
+++ lucene/solr/branches/sandbox/ryan/src/java/org/apache/solr/util/xslt/TransformerProvider.java Fri Oct  3 20:55:00 2008
@@ -29,7 +29,7 @@
 import javax.xml.transform.stream.StreamSource;
 
 import org.apache.solr.common.ResourceLoader;
-import org.apache.solr.core.SolrConfig;
+import org.apache.solr.config.SolrConfig;
 
 /** Singleton that creates a Transformer for the XSLTServletFilter.
  *  For now, only caches the last created Transformer, but

Modified: lucene/solr/branches/sandbox/ryan/src/test/org/apache/solr/BasicFunctionalityTest.java
URL: http://svn.apache.org/viewvc/lucene/solr/branches/sandbox/ryan/src/test/org/apache/solr/BasicFunctionalityTest.java?rev=701592&r1=701591&r2=701592&view=diff
==============================================================================
--- lucene/solr/branches/sandbox/ryan/src/test/org/apache/solr/BasicFunctionalityTest.java (original)
+++ lucene/solr/branches/sandbox/ryan/src/test/org/apache/solr/BasicFunctionalityTest.java Fri Oct  3 20:55:00 2008
@@ -336,7 +336,7 @@
   
   public void testTermVectorFields() {
     
-    IndexSchema ischema = new IndexSchema(solrConfig, getSchemaFile(), null);
+    IndexSchema ischema = new IndexSchema(solrConfig, h.getCore().getConfiguration(), getSchemaFile(), null);
     SchemaField f; // Solr field type
     Field luf; // Lucene field
 
@@ -505,7 +505,7 @@
   }   
   public void testCompressableFieldType() {
     
-    IndexSchema ischema = new IndexSchema(solrConfig, getSchemaFile(), null);
+    IndexSchema ischema = new IndexSchema(solrConfig, h.getCore().getConfiguration(), getSchemaFile(), null);
     SchemaField f; // Solr field type
     Field luf; // Lucene field
 

Modified: lucene/solr/branches/sandbox/ryan/src/test/org/apache/solr/analysis/AnalysisTestCase.java
URL: http://svn.apache.org/viewvc/lucene/solr/branches/sandbox/ryan/src/test/org/apache/solr/analysis/AnalysisTestCase.java?rev=701592&r1=701591&r2=701592&view=diff
==============================================================================
--- lucene/solr/branches/sandbox/ryan/src/test/org/apache/solr/analysis/AnalysisTestCase.java (original)
+++ lucene/solr/branches/sandbox/ryan/src/test/org/apache/solr/analysis/AnalysisTestCase.java Fri Oct  3 20:55:00 2008
@@ -16,7 +16,7 @@
  */
 
 package org.apache.solr.analysis;
-import org.apache.solr.core.SolrConfig;
+import org.apache.solr.config.SolrConfig;
 import org.apache.solr.util.TestHarness;
 import junit.framework.TestCase;
 

Modified: lucene/solr/branches/sandbox/ryan/src/test/org/apache/solr/core/ResourceLoaderTest.java
URL: http://svn.apache.org/viewvc/lucene/solr/branches/sandbox/ryan/src/test/org/apache/solr/core/ResourceLoaderTest.java?rev=701592&r1=701591&r2=701592&view=diff
==============================================================================
--- lucene/solr/branches/sandbox/ryan/src/test/org/apache/solr/core/ResourceLoaderTest.java (original)
+++ lucene/solr/branches/sandbox/ryan/src/test/org/apache/solr/core/ResourceLoaderTest.java Fri Oct  3 20:55:00 2008
@@ -24,6 +24,7 @@
 import org.apache.solr.analysis.KeywordTokenizerFactory;
 import org.apache.solr.analysis.NGramFilterFactory;
 import org.apache.solr.common.SolrException;
+import org.apache.solr.config.SolrResourceLoader;
 import org.apache.solr.handler.admin.LukeRequestHandler;
 import org.apache.solr.handler.component.FacetComponent;
 import org.apache.solr.request.JSONResponseWriter;

Modified: lucene/solr/branches/sandbox/ryan/src/test/org/apache/solr/core/TestBadConfig.java
URL: http://svn.apache.org/viewvc/lucene/solr/branches/sandbox/ryan/src/test/org/apache/solr/core/TestBadConfig.java?rev=701592&r1=701591&r2=701592&view=diff
==============================================================================
--- lucene/solr/branches/sandbox/ryan/src/test/org/apache/solr/core/TestBadConfig.java (original)
+++ lucene/solr/branches/sandbox/ryan/src/test/org/apache/solr/core/TestBadConfig.java Fri Oct  3 20:55:00 2008
@@ -17,6 +17,7 @@
 
 package org.apache.solr.core;
 
+import org.apache.solr.config.SolrConfig;
 import org.apache.solr.util.AbstractSolrTestCase;
 import org.apache.solr.util.TestHarness;
 import java.io.File;

Modified: lucene/solr/branches/sandbox/ryan/src/test/org/apache/solr/core/TestConfig.java
URL: http://svn.apache.org/viewvc/lucene/solr/branches/sandbox/ryan/src/test/org/apache/solr/core/TestConfig.java?rev=701592&r1=701591&r2=701592&view=diff
==============================================================================
--- lucene/solr/branches/sandbox/ryan/src/test/org/apache/solr/core/TestConfig.java (original)
+++ lucene/solr/branches/sandbox/ryan/src/test/org/apache/solr/core/TestConfig.java Fri Oct  3 20:55:00 2008
@@ -17,9 +17,8 @@
 
 package org.apache.solr.core;
 
-import org.apache.solr.handler.admin.ShowFileRequestHandler;
+import org.apache.solr.config.SolrIndexConfig;
 import org.apache.solr.util.AbstractSolrTestCase;
-import org.apache.solr.update.SolrIndexConfig;
 import org.w3c.dom.Node;
 import org.w3c.dom.NodeList;
 
@@ -65,17 +64,4 @@
     boolean luceneAutoCommit = solrConfig.getBool("indexDefaults/luceneAutoCommit");
     assertTrue(luceneAutoCommit + " does not equal: " + false, luceneAutoCommit == false);
   }
-
-  // sometime if the config referes to old things, it must be replaced with new stuff
-  public void testAutomaticDeprecationSupport()
-  {
-    // make sure the "admin/file" handler is registered
-    ShowFileRequestHandler handler = (ShowFileRequestHandler) h.getCore().getRequestHandler( "/admin/file" );
-    assertTrue( "file handler should have been automatically registered", handler!=null );
-
-    //System.out.println( handler.getHiddenFiles() );
-    // should not contain: <gettableFiles>solrconfig.xml scheam.xml admin-extra.html</gettableFiles>
-    assertFalse( handler.getHiddenFiles().contains( "scheam.xml".toUpperCase() ) );
-    assertTrue( handler.getHiddenFiles().contains( "PROTWORDS.TXT" ) );
-  }
 }

Modified: lucene/solr/branches/sandbox/ryan/src/test/org/apache/solr/core/TestJmxMonitoredMap.java
URL: http://svn.apache.org/viewvc/lucene/solr/branches/sandbox/ryan/src/test/org/apache/solr/core/TestJmxMonitoredMap.java?rev=701592&r1=701591&r2=701592&view=diff
==============================================================================
--- lucene/solr/branches/sandbox/ryan/src/test/org/apache/solr/core/TestJmxMonitoredMap.java (original)
+++ lucene/solr/branches/sandbox/ryan/src/test/org/apache/solr/core/TestJmxMonitoredMap.java Fri Oct  3 20:55:00 2008
@@ -17,7 +17,7 @@
 package org.apache.solr.core;
 
 import org.apache.solr.common.util.NamedList;
-import org.apache.solr.core.SolrConfig.JmxConfiguration;
+import org.apache.solr.config.JmxConfiguration;
 import org.junit.After;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;

Modified: lucene/solr/branches/sandbox/ryan/src/test/org/apache/solr/highlight/DummyHighlighter.java
URL: http://svn.apache.org/viewvc/lucene/solr/branches/sandbox/ryan/src/test/org/apache/solr/highlight/DummyHighlighter.java?rev=701592&r1=701591&r2=701592&view=diff
==============================================================================
--- lucene/solr/branches/sandbox/ryan/src/test/org/apache/solr/highlight/DummyHighlighter.java (original)
+++ lucene/solr/branches/sandbox/ryan/src/test/org/apache/solr/highlight/DummyHighlighter.java Fri Oct  3 20:55:00 2008
@@ -5,7 +5,7 @@
 import org.apache.lucene.search.Query;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.SimpleOrderedMap;
-import org.apache.solr.core.Config;
+import org.apache.solr.config.Config;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.search.DocList;
 

Modified: lucene/solr/branches/sandbox/ryan/src/test/org/apache/solr/schema/BadIndexSchemaTest.java
URL: http://svn.apache.org/viewvc/lucene/solr/branches/sandbox/ryan/src/test/org/apache/solr/schema/BadIndexSchemaTest.java?rev=701592&r1=701591&r2=701592&view=diff
==============================================================================
--- lucene/solr/branches/sandbox/ryan/src/test/org/apache/solr/schema/BadIndexSchemaTest.java (original)
+++ lucene/solr/branches/sandbox/ryan/src/test/org/apache/solr/schema/BadIndexSchemaTest.java Fri Oct  3 20:55:00 2008
@@ -20,7 +20,7 @@
 import java.util.LinkedList;
 import java.util.List;
 
-import org.apache.solr.core.SolrConfig;
+import org.apache.solr.config.SolrConfig;
 import org.apache.solr.core.SolrCore;
 import org.apache.solr.util.AbstractSolrTestCase;
 

Modified: lucene/solr/branches/sandbox/ryan/src/test/org/apache/solr/servlet/SolrRequestParserTest.java
URL: http://svn.apache.org/viewvc/lucene/solr/branches/sandbox/ryan/src/test/org/apache/solr/servlet/SolrRequestParserTest.java?rev=701592&r1=701591&r2=701592&view=diff
==============================================================================
--- lucene/solr/branches/sandbox/ryan/src/test/org/apache/solr/servlet/SolrRequestParserTest.java (original)
+++ lucene/solr/branches/sandbox/ryan/src/test/org/apache/solr/servlet/SolrRequestParserTest.java Fri Oct  3 20:55:00 2008
@@ -50,7 +50,7 @@
   @Override
   public void setUp() throws Exception {
     super.setUp();
-    parser = new SolrRequestParsers( h.getCore().getSolrConfig() );
+    parser = new SolrRequestParsers( h.getCore().getConfiguration().getDispatcherConfig() );
   }
   
   public void testStreamBody() throws Exception

Modified: lucene/solr/branches/sandbox/ryan/src/webapp/src/org/apache/solr/servlet/DirectSolrConnection.java
URL: http://svn.apache.org/viewvc/lucene/solr/branches/sandbox/ryan/src/webapp/src/org/apache/solr/servlet/DirectSolrConnection.java?rev=701592&r1=701591&r2=701592&view=diff
==============================================================================
--- lucene/solr/branches/sandbox/ryan/src/webapp/src/org/apache/solr/servlet/DirectSolrConnection.java (original)
+++ lucene/solr/branches/sandbox/ryan/src/webapp/src/org/apache/solr/servlet/DirectSolrConnection.java Fri Oct  3 20:55:00 2008
@@ -17,7 +17,6 @@
 
 package org.apache.solr.servlet;
 
-import java.io.File;
 import java.io.StringWriter;
 import java.util.ArrayList;
 import java.util.HashMap;
@@ -29,13 +28,11 @@
 import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.util.ContentStream;
 import org.apache.solr.common.util.ContentStreamBase;
-import org.apache.solr.core.SolrConfig;
 import org.apache.solr.core.SolrCore;
 import org.apache.solr.request.QueryResponseWriter;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.request.SolrQueryResponse;
 import org.apache.solr.request.SolrRequestHandler;
-import org.apache.solr.schema.IndexSchema;
 
 /**
  * DirectSolrConnection provides an interface to solr that is similar to 
@@ -58,52 +55,7 @@
   public DirectSolrConnection( SolrCore c )
   {
     core = c;
-    parser = new SolrRequestParsers( c.getSolrConfig() );
-  }
-
-  /**
-   * This constructor is designed to make it easy for JNI embedded applications 
-   * to setup the entire solr environment with a simple interface.  It takes three parameters:
-   * 
-   * <code>instanceDir:</code> The solr instance directory.  If null, it will check the standard 
-   * places first (JNDI,properties,"solr" directory)
-   * 
-   * <code>dataDir:</code> where the index is stored. 
-   * 
-   * <code>loggingPath:</code> Path to a java.util.logging.config.file.  If the path represents
-   * an absolute path or is relative to the CWD, it will use that.  Next it will try a path 
-   * relative to the instanceDir.  If none of these files exist, it will error.
-   */
-  public DirectSolrConnection( String instanceDir, String dataDir, String loggingPath )
-  {
-    // If a loggingPath is specified, try using that (this needs to happen first)
-    if( loggingPath != null ) {
-      File loggingConfig = new File( loggingPath );
-      if( !loggingConfig.exists() && instanceDir != null ) {
-        loggingConfig = new File( new File(instanceDir), loggingPath  );
-      }
-      if( loggingConfig.exists() ) {
-        System.setProperty("java.util.logging.config.file", loggingConfig.getAbsolutePath() ); 
-      }
-      else {
-        throw new SolrException( SolrException.ErrorCode.SERVER_ERROR, "can not find logging file: "+loggingConfig );
-      }
-    }
-    
-    // Initialize SolrConfig
-    SolrConfig config = null;
-    try {
-      config = new SolrConfig(instanceDir, SolrConfig.DEFAULT_CONF_FILE, null);
-      instanceDir = config.getResourceLoader().getInstanceDir();
-
-      // If the Data directory is specified, initialize SolrCore directly
-      IndexSchema schema = new IndexSchema(config, instanceDir+"/conf/schema.xml", null);
-      core = new SolrCore( null, dataDir, config, schema, null );
-      parser = new SolrRequestParsers( config );
-    } 
-    catch (Exception ee) {
-      throw new RuntimeException(ee);
-    }
+    parser = new SolrRequestParsers( null );
   }
   
 

Modified: lucene/solr/branches/sandbox/ryan/src/webapp/src/org/apache/solr/servlet/SolrDispatchFilter.java
URL: http://svn.apache.org/viewvc/lucene/solr/branches/sandbox/ryan/src/webapp/src/org/apache/solr/servlet/SolrDispatchFilter.java?rev=701592&r1=701591&r2=701592&view=diff
==============================================================================
--- lucene/solr/branches/sandbox/ryan/src/webapp/src/org/apache/solr/servlet/SolrDispatchFilter.java (original)
+++ lucene/solr/branches/sandbox/ryan/src/webapp/src/org/apache/solr/servlet/SolrDispatchFilter.java Fri Oct  3 20:55:00 2008
@@ -35,6 +35,8 @@
 
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.params.CommonParams;
+import org.apache.solr.config.SolrConfig;
+import org.apache.solr.config.SolrConfiguraion;
 import org.apache.solr.core.*;
 import org.apache.solr.request.*;
 import org.apache.solr.servlet.cache.HttpCacheHeaderUtil;
@@ -181,12 +183,12 @@
 
         // With a valid core...
         if( core != null ) {
-          final SolrConfig config = core.getSolrConfig();
+          final SolrConfiguraion config = core.getConfiguration();
           // get or create/cache the parser for the core
           SolrRequestParsers parser = null;
           parser = parsers.get(core);
           if( parser == null ) {
-            parser = new SolrRequestParsers(config);
+            parser = new SolrRequestParsers(config.getDispatcherConfig());
             parsers.put( core, parser );
           }
 

Modified: lucene/solr/branches/sandbox/ryan/src/webapp/src/org/apache/solr/servlet/SolrRequestParsers.java
URL: http://svn.apache.org/viewvc/lucene/solr/branches/sandbox/ryan/src/webapp/src/org/apache/solr/servlet/SolrRequestParsers.java?rev=701592&r1=701591&r2=701592&view=diff
==============================================================================
--- lucene/solr/branches/sandbox/ryan/src/webapp/src/org/apache/solr/servlet/SolrRequestParsers.java (original)
+++ lucene/solr/branches/sandbox/ryan/src/webapp/src/org/apache/solr/servlet/SolrRequestParsers.java Fri Oct  3 20:55:00 2008
@@ -43,7 +43,7 @@
 import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.util.ContentStream;
 import org.apache.solr.common.util.ContentStreamBase;
-import org.apache.solr.core.Config;
+import org.apache.solr.config.RequestDispatcherConfiguration;
 import org.apache.solr.core.SolrCore;
 import org.apache.solr.request.ServletSolrParams;
 import org.apache.solr.request.SolrQueryRequest;
@@ -69,24 +69,18 @@
    * Pass in an xml configuration.  A null configuration will enable
    * everythign with maximum values.
    */
-  public SolrRequestParsers( Config globalConfig )
+  public SolrRequestParsers( RequestDispatcherConfiguration config )
   {
     long uploadLimitKB = 1048;  // 2MB default
-    if( globalConfig == null ) {
+    if( config == null ) {
       uploadLimitKB = Long.MAX_VALUE; 
       enableRemoteStreams = true;
       handleSelect = true;
     }
     else {
-      uploadLimitKB = globalConfig.getInt( 
-          "requestDispatcher/requestParsers/@multipartUploadLimitInKB", (int)uploadLimitKB );
-      
-      enableRemoteStreams = globalConfig.getBool( 
-          "requestDispatcher/requestParsers/@enableRemoteStreaming", false ); 
-  
-      // Let this filter take care of /select?xxx format
-      handleSelect = globalConfig.getBool( 
-          "requestDispatcher/@handleSelect", handleSelect ); 
+      uploadLimitKB = config.getUploadLimitKB();
+      enableRemoteStreams = config.isEnableRemoteStreams();
+      handleSelect = config.isHandleSelect();
     }
        
     MultipartRequestParser multi = new MultipartRequestParser( uploadLimitKB );

Modified: lucene/solr/branches/sandbox/ryan/src/webapp/src/org/apache/solr/servlet/cache/HttpCacheHeaderUtil.java
URL: http://svn.apache.org/viewvc/lucene/solr/branches/sandbox/ryan/src/webapp/src/org/apache/solr/servlet/cache/HttpCacheHeaderUtil.java?rev=701592&r1=701591&r2=701592&view=diff
==============================================================================
--- lucene/solr/branches/sandbox/ryan/src/webapp/src/org/apache/solr/servlet/cache/HttpCacheHeaderUtil.java (original)
+++ lucene/solr/branches/sandbox/ryan/src/webapp/src/org/apache/solr/servlet/cache/HttpCacheHeaderUtil.java Fri Oct  3 20:55:00 2008
@@ -29,10 +29,10 @@
 import org.apache.lucene.index.IndexReader;
 
 import org.apache.solr.core.SolrCore;
-import org.apache.solr.core.SolrConfig;
-import org.apache.solr.core.SolrConfig.HttpCachingConfig.LastModFrom;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrException.ErrorCode;
+import org.apache.solr.config.SolrConfiguraion;
+import org.apache.solr.config.HttpCachingConfiguration.LastModFrom;
 import org.apache.solr.search.SolrIndexSearcher;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.request.SolrQueryResponse;
@@ -99,7 +99,7 @@
     EtagCacheVal etagCache = etagCoreCache.get(core);
     if (null == etagCache) {
       final String etagSeed
-        = core.getSolrConfig().getHttpCachingConfig().getEtagSeed();
+        = core.getConfiguration().getHttpCachingConfig().getEtagSeed();
       etagCache = new EtagCacheVal(etagSeed);
       etagCoreCache.put(core, etagCache);
     }
@@ -144,7 +144,7 @@
     final SolrIndexSearcher searcher = solrReq.getSearcher();
     
     final LastModFrom lastModFrom
-      = core.getSolrConfig().getHttpCachingConfig().getLastModFrom();
+      = core.getConfiguration().getHttpCachingConfig().getLastModFrom();
 
     long lastMod;
     try {
@@ -170,7 +170,7 @@
    * @param resp The servlet response object to modify
    * @param method The request method (GET, POST, ...) used by this request
    */
-  public static void setCacheControlHeader(final SolrConfig conf,
+  public static void setCacheControlHeader(final SolrConfiguraion conf,
                                            final HttpServletResponse resp, final Method method) {
     // We do not emit HTTP header for POST and OTHER request types
     if (Method.POST==method || Method.OTHER==method) {

Modified: lucene/solr/branches/sandbox/ryan/src/webapp/web/admin/_info.jsp
URL: http://svn.apache.org/viewvc/lucene/solr/branches/sandbox/ryan/src/webapp/web/admin/_info.jsp?rev=701592&r1=701591&r2=701592&view=diff
==============================================================================
--- lucene/solr/branches/sandbox/ryan/src/webapp/web/admin/_info.jsp (original)
+++ lucene/solr/branches/sandbox/ryan/src/webapp/web/admin/_info.jsp Fri Oct  3 20:55:00 2008
@@ -14,13 +14,13 @@
  See the License for the specific language governing permissions and
  limitations under the License.
 --%>
-<%@ page import="org.apache.solr.core.SolrConfig,
+<%@ page import="org.apache.solr.config.SolrConfig,
                  org.apache.solr.core.SolrCore,
                  org.apache.solr.schema.IndexSchema,
                  java.io.File"%>
 <%@ page import="java.net.InetAddress"%>
 <%@ page import="java.io.StringWriter"%>
-<%@ page import="org.apache.solr.core.Config"%>
+<%@ page import="org.apache.solr.config.Config"%>
 <%@ page import="org.apache.solr.common.util.XML"%>
 <%@ page import="org.apache.solr.common.SolrException"%>
 <%@ page import="org.apache.lucene.LucenePackage"%>

Modified: lucene/solr/branches/sandbox/ryan/src/webapp/web/admin/index.jsp
URL: http://svn.apache.org/viewvc/lucene/solr/branches/sandbox/ryan/src/webapp/web/admin/index.jsp?rev=701592&r1=701591&r2=701592&view=diff
==============================================================================
--- lucene/solr/branches/sandbox/ryan/src/webapp/web/admin/index.jsp (original)
+++ lucene/solr/branches/sandbox/ryan/src/webapp/web/admin/index.jsp Fri Oct  3 20:55:00 2008
@@ -36,12 +36,8 @@
 	<h3>Solr</h3>
   </td>
   <td>
-    <% if (null != core.getSchemaResource()) { %>
-    [<a href="file/?file=<%=core.getSchemaResource()%>">Schema</a>]
-    <% }
-       if (null != core.getConfigResource()) { %>
-    [<a href="file/?file=<%=core.getConfigResource()%>">Config</a>]
-    <% } %>
+    [<a href="file/?file=schema.xml">Schema</a>]
+    [<a href="file/?file=solrconfig.xml">Config</a>]
     [<a href="analysis.jsp?highlight=on">Analysis</a>]
     [<a href="schema.jsp">Schema Browser</a>]
     <br>