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 2007/11/12 20:39:12 UTC

svn commit: r594268 - in /lucene/solr/trunk: ./ example/solr/conf/ src/java/org/apache/solr/common/util/ src/java/org/apache/solr/handler/ src/java/org/apache/solr/handler/component/ src/java/org/apache/solr/search/

Author: ryan
Date: Mon Nov 12 11:39:01 2007
New Revision: 594268

URL: http://svn.apache.org/viewvc?rev=594268&view=rev
Log:
SOLR-281 -- adding a search component framework

Added:
    lucene/solr/trunk/src/java/org/apache/solr/common/util/RTimer.java   (with props)
    lucene/solr/trunk/src/java/org/apache/solr/handler/SearchHandler.java   (with props)
    lucene/solr/trunk/src/java/org/apache/solr/handler/component/
    lucene/solr/trunk/src/java/org/apache/solr/handler/component/DebugComponent.java   (with props)
    lucene/solr/trunk/src/java/org/apache/solr/handler/component/FacetComponent.java   (with props)
    lucene/solr/trunk/src/java/org/apache/solr/handler/component/HighlightComponent.java   (with props)
    lucene/solr/trunk/src/java/org/apache/solr/handler/component/MoreLikeThisComponent.java   (with props)
    lucene/solr/trunk/src/java/org/apache/solr/handler/component/QueryComponent.java   (with props)
    lucene/solr/trunk/src/java/org/apache/solr/handler/component/ResponseBuilder.java   (with props)
    lucene/solr/trunk/src/java/org/apache/solr/handler/component/SearchComponent.java   (with props)
Modified:
    lucene/solr/trunk/CHANGES.txt
    lucene/solr/trunk/example/solr/conf/solrconfig.xml
    lucene/solr/trunk/src/java/org/apache/solr/handler/DisMaxRequestHandler.java
    lucene/solr/trunk/src/java/org/apache/solr/handler/MoreLikeThisHandler.java
    lucene/solr/trunk/src/java/org/apache/solr/handler/StandardRequestHandler.java
    lucene/solr/trunk/src/java/org/apache/solr/search/BoostQParserPlugin.java
    lucene/solr/trunk/src/java/org/apache/solr/search/DisMaxQParserPlugin.java
    lucene/solr/trunk/src/java/org/apache/solr/search/QParser.java
    lucene/solr/trunk/src/java/org/apache/solr/search/QueryParsing.java
    lucene/solr/trunk/src/java/org/apache/solr/search/SolrIndexSearcher.java

Modified: lucene/solr/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/solr/trunk/CHANGES.txt?rev=594268&r1=594267&r2=594268&view=diff
==============================================================================
--- lucene/solr/trunk/CHANGES.txt (original)
+++ lucene/solr/trunk/CHANGES.txt Mon Nov 12 11:39:01 2007
@@ -153,6 +153,13 @@
     solrconfig.xml rather then using the <pingQuery></pingQuery> syntax.
     (Karsten Sperling via ryan)
 
+30. SOLR-281: Added a 'Search Component' interface and converted StandardRequestHandler
+    and DisMaxRequestHandler to use this framework.  
+    (Sharad Agarwal, Henri Biestro, yonik, ryan) 
+    
+31. SOLR-176: Add detailed timing data to query response output.  The SearchHandler
+    interface now returns how long each section takes.  (klaas)
+    
 
 Changes in runtime behavior
 

Modified: lucene/solr/trunk/example/solr/conf/solrconfig.xml
URL: http://svn.apache.org/viewvc/lucene/solr/trunk/example/solr/conf/solrconfig.xml?rev=594268&r1=594267&r2=594268&view=diff
==============================================================================
--- lucene/solr/trunk/example/solr/conf/solrconfig.xml (original)
+++ lucene/solr/trunk/example/solr/conf/solrconfig.xml Mon Nov 12 11:39:01 2007
@@ -287,6 +287,20 @@
      </lst>
   </requestHandler>
 
+
+  <requestHandler name="/search" class="solr.SearchHandler">
+    <lst name="defaults">
+      <str name="echoParams">explicit</str>
+    </lst>
+    <arr name="components">
+      <str>org.apache.solr.handler.component.QueryComponent</str>
+      <str>org.apache.solr.handler.component.FacetComponent</str>
+      <str>org.apache.solr.handler.component.MoreLikeThisComponent</str>
+      <str>org.apache.solr.handler.component.HighlightComponent</str>
+      <str>org.apache.solr.handler.component.DebugComponent</str>
+    </arr>
+  </requestHandler>
+
   <!-- DisMaxRequestHandler allows easy searching across multiple fields
        for simple user-entered phrases. 
        see http://wiki.apache.org/solr/DisMaxRequestHandler

Added: lucene/solr/trunk/src/java/org/apache/solr/common/util/RTimer.java
URL: http://svn.apache.org/viewvc/lucene/solr/trunk/src/java/org/apache/solr/common/util/RTimer.java?rev=594268&view=auto
==============================================================================
--- lucene/solr/trunk/src/java/org/apache/solr/common/util/RTimer.java (added)
+++ lucene/solr/trunk/src/java/org/apache/solr/common/util/RTimer.java Mon Nov 12 11:39:01 2007
@@ -0,0 +1,146 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.solr.common.util;
+
+import java.lang.System;
+import java.lang.Thread;
+import java.util.*;
+
+/** A recursive timer.
+ * 
+ * RTimers are started automatically when instantiated; subtimers are also
+ * started automatically when created.
+ *
+ * @since solr 1.3
+ * @revision $Id$
+ */
+public class RTimer {
+
+  public static final int STARTED = 0;
+  public static final int STOPPED = 1;
+  public static final int PAUSED = 2;
+
+  protected int state;
+  protected double startTime;
+  protected double time;
+  protected double culmTime;
+  protected SimpleOrderedMap<RTimer> children;
+
+  public RTimer() {
+    time = 0;
+    culmTime = 0;
+    children = new SimpleOrderedMap<RTimer>();
+    startTime = now();
+    state = STARTED;
+  }
+
+  /** Get current time
+   *
+   * May override to implement a different timer (CPU time, etc).
+   */
+  protected double now() { return System.currentTimeMillis(); }
+
+  /** Recursively stop timer and sub timers */
+  public double stop() {
+    assert state == STARTED || state == PAUSED;
+    time = culmTime;
+    if(state == STARTED) 
+      time += now() - startTime;
+    state = STOPPED;
+    
+    for( Map.Entry<String,RTimer> entry : children ) {
+      RTimer child = entry.getValue();
+      if(child.state == STARTED || child.state == PAUSED) 
+        child.stop();
+    }
+    return time;
+  }
+
+  public void pause() {
+    assert state == STARTED;
+    culmTime += now() - startTime;
+    state = PAUSED;
+  }
+  
+  public void resume() {
+    if(state == STARTED)
+      return;
+    assert state == PAUSED;
+    state = STARTED;
+    startTime = now();
+  }
+
+  /** Get total elapsed time for this timer.
+   *
+   * Timer must be STOPped.
+   */
+  public double getTime() {
+    assert state == STOPPED;
+    return time;
+  }
+
+  /** Create new subtimer with given name
+   *
+   * Subtimer will be started.
+   */
+  public RTimer sub(String desc) {
+    RTimer child = children.get( desc );
+    if( child == null ) {
+      child = new RTimer();
+      children.add(desc, child);
+    }
+    return child;
+  }
+
+  @Override
+  public String toString() {
+    return asNamedList().toString();
+  }
+
+  public NamedList asNamedList() {
+    NamedList<Object> m = new SimpleOrderedMap<Object>();
+    m.add( "time", time );
+    if( children.size() > 0 ) {
+      for( Map.Entry<String, RTimer> entry : children ) {
+        m.add( entry.getKey(), entry.getValue().asNamedList() );
+      }
+    }
+    return m;
+  }
+
+  /*************** Testing *******/
+  public static void main(String []argv) throws InterruptedException {
+    RTimer rt = new RTimer(), subt, st;
+    Thread.sleep(100);
+
+    subt = rt.sub("sub1");
+    Thread.sleep(50);
+    st = subt.sub("sub1.1");
+    st.resume();
+    Thread.sleep(10);
+    st.pause();
+    Thread.sleep(50);
+    st.resume();
+    Thread.sleep(10);
+    st.pause();
+    subt.stop();
+    rt.stop();
+
+    System.out.println( rt.toString());
+  }
+}

Propchange: lucene/solr/trunk/src/java/org/apache/solr/common/util/RTimer.java
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: lucene/solr/trunk/src/java/org/apache/solr/common/util/RTimer.java
------------------------------------------------------------------------------
    svn:keywords = Date Author Id Revision HeadURL

Modified: lucene/solr/trunk/src/java/org/apache/solr/handler/DisMaxRequestHandler.java
URL: http://svn.apache.org/viewvc/lucene/solr/trunk/src/java/org/apache/solr/handler/DisMaxRequestHandler.java?rev=594268&r1=594267&r2=594268&view=diff
==============================================================================
--- lucene/solr/trunk/src/java/org/apache/solr/handler/DisMaxRequestHandler.java (original)
+++ lucene/solr/trunk/src/java/org/apache/solr/handler/DisMaxRequestHandler.java Mon Nov 12 11:39:01 2007
@@ -17,34 +17,15 @@
 
 package org.apache.solr.handler;
 
-import java.net.MalformedURLException;
-import java.net.URL;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.lucene.queryParser.QueryParser;
-import org.apache.lucene.search.BooleanClause;
-import org.apache.lucene.search.BooleanQuery;
-import org.apache.lucene.search.Query;
-import org.apache.lucene.search.BooleanClause.Occur;
-import org.apache.solr.common.SolrException;
-import org.apache.solr.common.params.CommonParams;
-import org.apache.solr.common.params.DisMaxParams;
-import org.apache.solr.common.params.FacetParams;
 import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.util.NamedList;
-import org.apache.solr.core.SolrCore;
-import org.apache.solr.highlight.SolrHighlighter;
-import org.apache.solr.request.SimpleFacets;
-import org.apache.solr.request.SolrQueryRequest;
-import org.apache.solr.request.SolrQueryResponse;
-import org.apache.solr.schema.IndexSchema;
-import org.apache.solr.search.DocListAndSet;
-import org.apache.solr.search.DocSet;
+import org.apache.solr.search.DisMaxQParserPlugin;
 import org.apache.solr.search.QueryParsing;
-import org.apache.solr.search.SolrIndexSearcher;
 import org.apache.solr.util.SolrPluginUtils;
-    
+
+import java.net.MalformedURLException;
+import java.net.URL;
+
 /**
  * <p>
  * A Generic query plugin designed to be given a simple query expression
@@ -133,259 +114,32 @@
  *
  * </pre>
  */
-public class DisMaxRequestHandler extends RequestHandlerBase  {
+@Deprecated
+public class DisMaxRequestHandler extends StandardRequestHandler  
+{
 
-  /**
-   * A field we can't ever find in any schema, so we can safely tell
-   * DisjunctionMaxQueryParser to use it as our defaultField, and
-   * map aliases from it to any field in our schema.
-   */
-  private static String IMPOSSIBLE_FIELD_NAME = "\uFFFC\uFFFC\uFFFC";
-    
-    
-  /** shorten the class references for utilities */
-  private static class U extends SolrPluginUtils {
-    /* :NOOP */
-  }
-  /** shorten the class references for utilities */
-  private static interface DMP extends DisMaxParams {
-    /* :NOOP */
-  }
-
-  public DisMaxRequestHandler() {
-    super();
-  }
-  
-  /** Sets the default variables for any useful info it finds in the config.
-   * If a config option is not in the format expected, logs a warning
-   * and ignores it.
-   */
+  @Override
   public void init(NamedList args) {
-	// Handle an old format
-    if (-1 == args.indexOf("defaults",0)) {
+    super.init( args );
+    NamedList def = null;
+    
+    // redo "defaults"
+    Object o = args.get("defaults");
+    if (o != null && o instanceof NamedList) {
+      def = (NamedList)o;
+    } else {
       // no explict defaults list, use all args implicitly
       // indexOf so "<null name="defaults"/> is valid indicator of no defaults
-      defaults = SolrParams.toSolrParams(args);
-    } else {
-      // otherwise use the new one.
-      super.init( args );
+      def = args;
+    }
+    
+    //  Make the default query type "dismax" if not specified
+    if (def.get(QueryParsing.DEFTYPE) == null) {
+      def = def.clone();
+      def.add(QueryParsing.DEFTYPE, DisMaxQParserPlugin.NAME);
+      defaults = SolrParams.toSolrParams( def );
     }
   }
-
-  public void handleRequestBody(SolrQueryRequest req, SolrQueryResponse rsp) throws Exception
-  {
-      SolrParams params = req.getParams();
-      
-      int flags = 0;
-      
-      SolrIndexSearcher s = req.getSearcher();
-      IndexSchema schema = req.getSchema();
-            
-      Map<String,Float> queryFields = U.parseFieldBoosts(params.getParams(DMP.QF));
-      Map<String,Float> phraseFields = U.parseFieldBoosts(params.getParams(DMP.PF));
-
-      float tiebreaker = params.getFloat(DMP.TIE, 0.0f);
-            
-      int pslop = params.getInt(DMP.PS, 0);
-      int qslop = params.getInt(DMP.QS, 0);
-
-      /* a generic parser for parsing regular lucene queries */
-      QueryParser p = schema.getSolrQueryParser(null);
-
-      /* a parser for dealing with user input, which will convert
-       * things to DisjunctionMaxQueries
-       */
-      U.DisjunctionMaxQueryParser up =
-        new U.DisjunctionMaxQueryParser(schema, IMPOSSIBLE_FIELD_NAME);
-      up.addAlias(IMPOSSIBLE_FIELD_NAME,
-                  tiebreaker, queryFields);
-      up.setPhraseSlop(qslop);
-      
-      /* for parsing sloppy phrases using DisjunctionMaxQueries */
-      U.DisjunctionMaxQueryParser pp =
-        new U.DisjunctionMaxQueryParser(schema, IMPOSSIBLE_FIELD_NAME);
-      pp.addAlias(IMPOSSIBLE_FIELD_NAME,
-                  tiebreaker, phraseFields);
-      pp.setPhraseSlop(pslop);
-            
-            
-      /* the main query we will execute.  we disable the coord because
-       * this query is an artificial construct
-       */
-      BooleanQuery query = new BooleanQuery(true);
-
-      /* * * Main User Query * * */
-      Query parsedUserQuery = null;
-      String userQuery = params.get( CommonParams.Q );
-      Query altUserQuery = null;
-      if( userQuery == null || userQuery.trim().length() < 1 ) {
-        // If no query is specified, we may have an alternate
-        String altQ = params.get( DMP.ALTQ );
-        if (altQ != null) {
-          altUserQuery = p.parse(altQ);
-          query.add( altUserQuery , Occur.MUST );
-        } else {
-          throw new SolrException( SolrException.ErrorCode.BAD_REQUEST, "missing query string" );
-        }
-      }
-      else {
-        // There is a valid query string
-        userQuery = U.partialEscape(U.stripUnbalancedQuotes(userQuery)).toString();
-            
-        String minShouldMatch = params.get(DMP.MM, "100%");
-        Query dis = up.parse(userQuery);
-        parsedUserQuery = dis;
-  
-        if (dis instanceof BooleanQuery) {
-          BooleanQuery t = new BooleanQuery();
-          U.flattenBooleanQuery(t, (BooleanQuery)dis);
-          U.setMinShouldMatch(t, minShouldMatch);                
-          parsedUserQuery = t;
-        } 
-        query.add(parsedUserQuery, Occur.MUST);
-        
-
-        /* * * Add on Phrases for the Query * * */
-              
-        /* build up phrase boosting queries */
-
-        /* if the userQuery already has some quotes, stip them out.
-         * we've already done the phrases they asked for in the main
-         * part of the query, this is to boost docs that may not have
-         * matched those phrases but do match looser phrases.
-         */
-        String userPhraseQuery = userQuery.replace("\"","");
-        Query phrase = pp.parse("\"" + userPhraseQuery + "\"");
-        if (null != phrase) {
-          query.add(phrase, Occur.SHOULD);
-        }
-      }
-
-            
-      /* * * Boosting Query * * */
-      String[] boostParams = params.getParams(DMP.BQ);
-      List<Query> boostQueries = U.parseQueryStrings(req, boostParams);
-      if (null != boostQueries) {
-        if(1 == boostQueries.size() && 1 == boostParams.length) {
-          /* legacy logic */
-          Query f = boostQueries.get(0);
-          if (1.0f == f.getBoost() && f instanceof BooleanQuery) {
-            /* if the default boost was used, and we've got a BooleanQuery
-             * extract the subqueries out and use them directly
-             */
-            for (Object c : ((BooleanQuery)f).clauses()) {
-              query.add((BooleanClause)c);
-            }
-          } else {
-            query.add(f, BooleanClause.Occur.SHOULD);
-          }
-        } else {
-          for(Query f : boostQueries) {
-            query.add(f, BooleanClause.Occur.SHOULD);
-          }
-        }
-      }
-
-      /* * * Boosting Functions * * */
-
-      String[] boostFuncs = params.getParams(DMP.BF);
-      if (null != boostFuncs && 0 != boostFuncs.length) {
-        for (String boostFunc : boostFuncs) {
-          if(null == boostFunc || "".equals(boostFunc)) continue;
-          List<Query> funcs = U.parseFuncs(schema, boostFunc);
-          for (Query f : funcs) {
-            query.add(f, Occur.SHOULD);          
-          }
-        }
-      }
-            
-      /* * * Restrict Results * * */
-
-      List<Query> restrictions = U.parseFilterQueries(req);
-            
-      /* * * Generate Main Results * * */
-
-      flags |= U.setReturnFields(req,rsp);
-      
-      DocListAndSet results = new DocListAndSet();
-      NamedList facetInfo = null;
-      if (params.getBool(FacetParams.FACET,false)) {
-        results = s.getDocListAndSet(query, restrictions,
-                                     SolrPluginUtils.getSort(req),
-                                     req.getStart(), req.getLimit(),
-                                     flags);
-        facetInfo = getFacetInfo(req, rsp, results.docSet);
-      } else {
-        results.docList = s.getDocList(query, restrictions,
-                                       SolrPluginUtils.getSort(req),
-                                       req.getStart(), req.getLimit(),
-                                       flags);
-      }
-      rsp.add("response",results.docList);
-      // pre-fetch returned documents
-      U.optimizePreFetchDocs(results.docList, query, req, rsp);
-
-      
-      if (null != facetInfo) rsp.add("facet_counts", facetInfo);
-
-
-            
-      /* * * Debugging Info * * */
-
-      try {
-        NamedList debug = U.doStandardDebug(req, userQuery, query, results.docList);
-        if (null != debug) {
-          debug.add("altquerystring", altUserQuery);
-          if (null != boostQueries) {
-            debug.add("boost_queries", boostParams);
-            debug.add("parsed_boost_queries", 
-                      QueryParsing.toString(boostQueries, req.getSchema()));
-          }
-          debug.add("boostfuncs", params.getParams(DMP.BF));
-          if (null != restrictions) {
-            debug.add("filter_queries", params.getParams(CommonParams.FQ));
-            debug.add("parsed_filter_queries", 
-                      QueryParsing.toString(restrictions, req.getSchema()));
-          }
-          rsp.add("debug", debug);
-        }
-
-      } catch (Exception e) {
-        SolrException.logOnce(SolrCore.log, "Exception during debug", e);
-        rsp.add("exception_during_debug", SolrException.toStr(e));
-      }
-
-      /* * * Highlighting/Summarizing  * * */
-      SolrHighlighter highlighter = req.getCore().getHighlighter();
-      if(highlighter.isHighlightingEnabled( params ) && parsedUserQuery != null) {
-        String[] highFields = queryFields.keySet().toArray(new String[0]);
-        NamedList sumData = highlighter.doHighlighting(
-  	       results.docList, 
-  	       parsedUserQuery.rewrite(req.getSearcher().getReader()), 
-  	       req, 
-  	       highFields);
-        if(sumData != null)
-          rsp.add("highlighting", sumData);
-      }
-  }
-
-  /**
-   * Fetches information about Facets for this request.
-   *
-   * Subclasses may with to override this method to provide more 
-   * advanced faceting behavior.
-   * @see SimpleFacets#getFacetCounts
-   */
-  protected NamedList getFacetInfo(SolrQueryRequest req, 
-                                   SolrQueryResponse rsp, 
-                                   DocSet mainSet) {
-
-    SimpleFacets f = new SimpleFacets(req.getSearcher(), 
-                                      mainSet, 
-                                      req.getParams());
-    return f.getFacetCounts();
-  }
-  
 
 	//////////////////////// SolrInfoMBeans methods //////////////////////
 

Modified: lucene/solr/trunk/src/java/org/apache/solr/handler/MoreLikeThisHandler.java
URL: http://svn.apache.org/viewvc/lucene/solr/trunk/src/java/org/apache/solr/handler/MoreLikeThisHandler.java?rev=594268&r1=594267&r2=594268&view=diff
==============================================================================
--- lucene/solr/trunk/src/java/org/apache/solr/handler/MoreLikeThisHandler.java (original)
+++ lucene/solr/trunk/src/java/org/apache/solr/handler/MoreLikeThisHandler.java Mon Nov 12 11:39:01 2007
@@ -51,6 +51,7 @@
 import org.apache.solr.schema.IndexSchema;
 import org.apache.solr.schema.SchemaField;
 import org.apache.solr.search.DocIterator;
+import org.apache.solr.search.DocSet;
 import org.apache.solr.search.DocList;
 import org.apache.solr.search.DocListAndSet;
 import org.apache.solr.search.QueryParsing;

Added: lucene/solr/trunk/src/java/org/apache/solr/handler/SearchHandler.java
URL: http://svn.apache.org/viewvc/lucene/solr/trunk/src/java/org/apache/solr/handler/SearchHandler.java?rev=594268&view=auto
==============================================================================
--- lucene/solr/trunk/src/java/org/apache/solr/handler/SearchHandler.java (added)
+++ lucene/solr/trunk/src/java/org/apache/solr/handler/SearchHandler.java Mon Nov 12 11:39:01 2007
@@ -0,0 +1,159 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.solr.handler;
+
+import org.apache.lucene.queryParser.ParseException;
+import org.apache.solr.common.params.CommonParams;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.common.util.RTimer;
+import org.apache.solr.common.util.SimpleOrderedMap;
+import org.apache.solr.handler.component.ResponseBuilder;
+import org.apache.solr.handler.component.SearchComponent;
+import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.request.SolrQueryResponse;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.logging.Logger;
+
+/**
+ *
+ * Refer SOLR-281
+ *
+ */
+public class SearchHandler extends RequestHandlerBase
+{
+  private static final String RESPONSE_BUILDER_CONTEXT_KEY = "ResponseBuilder";
+  
+  protected static Logger log = Logger.getLogger(SearchHandler.class.getName());
+  
+  protected Collection<SearchComponent> components;
+  
+  @Override
+  public void init(NamedList args) {
+    super.init( args );
+    initComponents(args);
+  }
+  
+  // TODO: should there be a way to append components from solrconfig w/o having to
+  // know the complete standard list (which may expand over time?)
+  protected void initComponents(NamedList args){
+    if( args != null ) {
+      try {
+        Object declaredComponents = args.get("components");
+        if (declaredComponents != null && declaredComponents instanceof List) {
+          List list = (List) declaredComponents;
+          components = new ArrayList<SearchComponent>(list.size());
+          for(Object c : list){
+            // TODO: an init() with args for components?
+            SearchComponent comp = (SearchComponent) Class.forName((String) c).newInstance();
+            components.add(comp);
+            log.info("Adding  component:"+comp);
+          }
+        }
+      } catch (Exception e) {
+        throw new RuntimeException(e);
+      } 
+    }
+  }
+
+  public static ResponseBuilder getResponseBuilder(SolrQueryRequest req) 
+  {
+    return (ResponseBuilder) req.getContext().get( RESPONSE_BUILDER_CONTEXT_KEY );
+  }
+  
+  @Override
+  public void handleRequestBody(SolrQueryRequest req, SolrQueryResponse rsp) throws IOException, ParseException, InstantiationException, IllegalAccessException 
+  {
+    ResponseBuilder builder = new ResponseBuilder();
+    req.getContext().put( RESPONSE_BUILDER_CONTEXT_KEY, builder );
+    
+    // The semantics of debugging vs not debugging are different enough that 
+    // it makes sense to have two control loops
+    if( !req.getParams().getBool( CommonParams.DEBUG_QUERY, false ) ) {
+      // Prepare
+      for( SearchComponent c : components ) {
+        c.prepare( req, rsp );
+      }
+  
+      // Process
+      for( SearchComponent c : components ) {
+        c.process( req, rsp );
+      }
+    }
+    else {
+      builder.setDebug( true );
+      RTimer timer = new RTimer();
+      
+      // Prepare
+      RTimer subt = timer.sub( "prepare" );
+      for( SearchComponent c : components ) {
+        builder.setTimer( subt.sub( c.getName() ) );
+        c.prepare( req, rsp );
+        builder.getTimer().stop();
+      }
+      subt.stop();
+  
+      // Process
+      subt = timer.sub( "process" );
+      for( SearchComponent c : components ) {
+        builder.setTimer( subt.sub( c.getName() ) );
+        c.process( req, rsp );
+        builder.getTimer().stop();
+      }
+      subt.stop();
+      timer.stop();
+      
+      // add the timing info
+      if( builder.getDebugInfo() == null ) {
+        builder.setDebugInfo( new SimpleOrderedMap<Object>() );
+      }
+      builder.getDebugInfo().add( "timing", timer.asNamedList() );
+    }
+  }
+
+  //////////////////////// SolrInfoMBeans methods //////////////////////
+
+  @Override
+  public String getDescription() {
+    StringBuilder sb = new StringBuilder();
+    sb.append("Search using components: ");
+    for(SearchComponent c : components){
+      sb.append(c.getName());
+      sb.append(",");
+    }
+    return sb.toString();
+  }
+
+  @Override
+  public String getVersion() {
+    return "$Revision$";
+  }
+
+  @Override
+  public String getSourceId() {
+    return "$Id$";
+  }
+
+  @Override
+  public String getSource() {
+    return "$URL$";
+  }
+}

Propchange: lucene/solr/trunk/src/java/org/apache/solr/handler/SearchHandler.java
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: lucene/solr/trunk/src/java/org/apache/solr/handler/SearchHandler.java
------------------------------------------------------------------------------
    svn:keywords = Date Author Id Revision HeadURL

Modified: lucene/solr/trunk/src/java/org/apache/solr/handler/StandardRequestHandler.java
URL: http://svn.apache.org/viewvc/lucene/solr/trunk/src/java/org/apache/solr/handler/StandardRequestHandler.java?rev=594268&r1=594267&r2=594268&view=diff
==============================================================================
--- lucene/solr/trunk/src/java/org/apache/solr/handler/StandardRequestHandler.java (original)
+++ lucene/solr/trunk/src/java/org/apache/solr/handler/StandardRequestHandler.java Mon Nov 12 11:39:01 2007
@@ -17,26 +17,17 @@
 
 package org.apache.solr.handler;
 
-import org.apache.lucene.search.*;
-
 import java.net.MalformedURLException;
 import java.net.URL;
 import java.util.ArrayList;
-import java.util.List;
 
-import org.apache.solr.util.SolrPluginUtils;
-import org.apache.solr.request.SimpleFacets;
-import org.apache.solr.request.SolrQueryRequest;
-import org.apache.solr.request.SolrQueryResponse;
-import org.apache.solr.search.*;
-import org.apache.solr.common.SolrException;
-import org.apache.solr.common.params.CommonParams;
-import org.apache.solr.common.params.FacetParams;
-import org.apache.solr.common.params.MoreLikeThisParams;
-import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.util.NamedList;
-import org.apache.solr.core.SolrCore;
-import org.apache.solr.highlight.SolrHighlighter;
+import org.apache.solr.handler.component.DebugComponent;
+import org.apache.solr.handler.component.FacetComponent;
+import org.apache.solr.handler.component.HighlightComponent;
+import org.apache.solr.handler.component.MoreLikeThisComponent;
+import org.apache.solr.handler.component.QueryComponent;
+import org.apache.solr.handler.component.SearchComponent;
 
 /**
  * @version $Id$
@@ -57,111 +48,17 @@
  * </ul>
  *
  */
+public class StandardRequestHandler extends SearchHandler {
 
-public class StandardRequestHandler extends RequestHandlerBase {
-
-  /** shorten the class references for utilities */
-  private static class U extends SolrPluginUtils {
-    /* :NOOP */
-  }
-
-
-  public void handleRequestBody(SolrQueryRequest req, SolrQueryResponse rsp) throws Exception
-  {
-    
-      SolrParams p = req.getParams();
-      String qstr = p.required().get(CommonParams.Q);
-
-      // find fieldnames to return (fieldlist)
-      // TODO: make this per-query and add method to QParser to get?
-      String fl = p.get(CommonParams.FL);
-      int flags = 0; 
-      if (fl != null) {
-        flags |= U.setReturnFields(fl, rsp);
-      }
-
-      QParser parser = QParser.getParser(qstr, OldLuceneQParserPlugin.NAME, req);
-      Query query = parser.getQuery();
-      QueryParsing.SortSpec sortSpec = parser.getSort(true);
-
-      DocListAndSet results = new DocListAndSet();
-      NamedList facetInfo = null;
-      List<Query> filters = U.parseFilterQueries(req);
-      SolrIndexSearcher s = req.getSearcher();
-
-      if (p.getBool(FacetParams.FACET,false)) {
-        results = s.getDocListAndSet(query, filters, sortSpec.getSort(),
-                                     sortSpec.getOffset(), sortSpec.getCount(),
-                                     flags);
-        facetInfo = getFacetInfo(req, rsp, results.docSet);
-      } else {
-        results.docList = s.getDocList(query, filters, sortSpec.getSort(),
-                                       sortSpec.getOffset(), sortSpec.getCount(),
-                                       flags);
-      }
-
-      // pre-fetch returned documents
-      U.optimizePreFetchDocs(results.docList, query, req, rsp);
-      
-      rsp.add("response",results.docList);
-
-      if (null != facetInfo) rsp.add("facet_counts", facetInfo);
-
-      // Include "More Like This" results for *each* result
-      if( p.getBool( MoreLikeThisParams.MLT, false ) ) {
-        MoreLikeThisHandler.MoreLikeThisHelper mlt 
-          = new MoreLikeThisHandler.MoreLikeThisHelper( p, s );
-        int mltcount = p.getInt( MoreLikeThisParams.DOC_COUNT, 5 );
-        rsp.add( "moreLikeThis", mlt.getMoreLikeThese(results.docList, mltcount, flags));
-      }
-      
-      try {
-        NamedList dbg = U.doStandardDebug(req, qstr, query, results.docList);
-        if (null != dbg) {
-          if (null != filters) {
-            dbg.add("filter_queries",req.getParams().getParams(CommonParams.FQ));
-            List<String> fqs = new ArrayList<String>(filters.size());
-            for (Query fq : filters) {
-              fqs.add(QueryParsing.toString(fq, req.getSchema()));
-            }
-            dbg.add("parsed_filter_queries",fqs);
-          }
-          rsp.add("debug", dbg);
-        }
-      } catch (Exception e) {
-        SolrException.logOnce(SolrCore.log, "Exception during debug", e);
-        rsp.add("exception_during_debug", SolrException.toStr(e));
-      }
-      
-
-      SolrHighlighter highlighter = req.getCore().getHighlighter();
-      NamedList sumData = highlighter.doHighlighting(
-            results.docList,
-            parser.getHighlightQuery().rewrite(req.getSearcher().getReader()),
-            req,
-            parser.getDefaultHighlightFields());
-      if(sumData != null)
-        rsp.add("highlighting", sumData);
+  @Override
+  protected void initComponents(NamedList args) {
+    components = new ArrayList<SearchComponent>(5);
+    components.add( new QueryComponent() );
+    components.add( new FacetComponent() );
+    components.add( new MoreLikeThisComponent() );
+    components.add( new HighlightComponent() );
+    components.add( new DebugComponent() );
   }
-
-  /**
-   * Fetches information about Facets for this request.
-   *
-   * Subclasses may with to override this method to provide more 
-   * advanced faceting behavior.
-   * @see SimpleFacets#getFacetCounts
-   */
-  protected NamedList getFacetInfo(SolrQueryRequest req, 
-                                   SolrQueryResponse rsp, 
-                                   DocSet mainSet) {
-
-    SimpleFacets f = new SimpleFacets(req.getSearcher(), 
-                                      mainSet, 
-                                      req.getParams());
-    return f.getFacetCounts();
-  }
-
-
 
   //////////////////////// SolrInfoMBeans methods //////////////////////
 

Added: lucene/solr/trunk/src/java/org/apache/solr/handler/component/DebugComponent.java
URL: http://svn.apache.org/viewvc/lucene/solr/trunk/src/java/org/apache/solr/handler/component/DebugComponent.java?rev=594268&view=auto
==============================================================================
--- lucene/solr/trunk/src/java/org/apache/solr/handler/component/DebugComponent.java (added)
+++ lucene/solr/trunk/src/java/org/apache/solr/handler/component/DebugComponent.java Mon Nov 12 11:39:01 2007
@@ -0,0 +1,105 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.solr.handler.component;
+
+import static org.apache.solr.common.params.CommonParams.FQ;
+
+import java.io.IOException;
+import java.net.URL;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.lucene.queryParser.ParseException;
+import org.apache.lucene.search.Query;
+import org.apache.solr.handler.SearchHandler;
+import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.request.SolrQueryResponse;
+import org.apache.solr.search.QueryParsing;
+import org.apache.solr.util.SolrPluginUtils;
+
+/**
+ * TODO!
+ * 
+ * @version $Id$
+ * @since solr 1.3
+ */
+public class DebugComponent extends SearchComponent
+{
+  @Override
+  public void prepare(SolrQueryRequest req, SolrQueryResponse rsp) throws IOException, ParseException 
+  {
+    
+  }
+
+  @Override
+  public void process(SolrQueryRequest req, SolrQueryResponse rsp) throws IOException 
+  {
+    ResponseBuilder builder = SearchHandler.getResponseBuilder( req );
+    if( builder.isDebug() ) {
+      builder.setDebugInfo( SolrPluginUtils.doStandardDebug( req, 
+          builder.getQueryString(), builder.getQuery(), builder.getResults().docList) );
+
+      if (builder.getQparser() != null) {
+        builder.getQparser().addDebugInfo(builder.getDebugInfo());
+      }
+
+      if (null != builder.getDebugInfo() ) {
+        if (null != builder.getFilters() ) {
+          builder.getDebugInfo().add("filter_queries",req.getParams().getParams(FQ));
+          List<String> fqs = new ArrayList<String>(builder.getFilters().size());
+          for (Query fq : builder.getFilters()) {
+            fqs.add(QueryParsing.toString(fq, req.getSchema()));
+          }
+          builder.getDebugInfo().add("parsed_filter_queries",fqs);
+        }
+        
+        // Add this directly here?
+        rsp.add("debug", builder.getDebugInfo() );
+      }
+    }
+  }
+  
+  /////////////////////////////////////////////
+  ///  SolrInfoMBean
+  ////////////////////////////////////////////
+
+  @Override
+  public String getDescription() {
+    return "Debug Information";
+  }
+
+  @Override
+  public String getVersion() {
+    return "$Revision$";
+  }
+
+  @Override
+  public String getSourceId() {
+    return "$Id$";
+  }
+
+  @Override
+  public String getSource() {
+    return "$URL$";
+  }
+
+  @Override
+  public URL[] getDocs() {
+    return null;
+  }
+}

Propchange: lucene/solr/trunk/src/java/org/apache/solr/handler/component/DebugComponent.java
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: lucene/solr/trunk/src/java/org/apache/solr/handler/component/DebugComponent.java
------------------------------------------------------------------------------
    svn:keywords = Date Author Id Revision HeadURL

Added: lucene/solr/trunk/src/java/org/apache/solr/handler/component/FacetComponent.java
URL: http://svn.apache.org/viewvc/lucene/solr/trunk/src/java/org/apache/solr/handler/component/FacetComponent.java?rev=594268&view=auto
==============================================================================
--- lucene/solr/trunk/src/java/org/apache/solr/handler/component/FacetComponent.java (added)
+++ lucene/solr/trunk/src/java/org/apache/solr/handler/component/FacetComponent.java Mon Nov 12 11:39:01 2007
@@ -0,0 +1,96 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.solr.handler.component;
+
+import java.io.IOException;
+import java.net.URL;
+
+import org.apache.lucene.queryParser.ParseException;
+import org.apache.solr.common.params.FacetParams;
+import org.apache.solr.common.params.SolrParams;
+import org.apache.solr.handler.SearchHandler;
+import org.apache.solr.request.SimpleFacets;
+import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.request.SolrQueryResponse;
+
+/**
+ * TODO!
+ * 
+ * @version $Id$
+ * @since solr 1.3
+ */
+public class FacetComponent extends SearchComponent
+{
+  @Override
+  public void prepare(SolrQueryRequest req, SolrQueryResponse rsp) throws IOException, ParseException 
+  {
+    SolrParams params = req.getParams();
+    if (params.getBool(FacetParams.FACET,false)) {
+      ResponseBuilder builder = SearchHandler.getResponseBuilder( req );
+      builder.setNeedDocSet( true );
+    }
+  }
+
+  /**
+   * Actually run the query
+   */
+  @Override
+  public void process(SolrQueryRequest req, SolrQueryResponse rsp) throws IOException 
+  {
+    SolrParams params = req.getParams();
+    if (params.getBool(FacetParams.FACET,false)) {
+      ResponseBuilder builder = SearchHandler.getResponseBuilder( req );
+      
+      SimpleFacets f = new SimpleFacets(req.getSearcher(), 
+          builder.getResults().docSet, 
+          params );
+
+      // TODO ???? add this directly to the response?
+      rsp.add( "facet_counts", f.getFacetCounts() );
+    }
+  }
+
+  /////////////////////////////////////////////
+  ///  SolrInfoMBean
+  ////////////////////////////////////////////
+
+  @Override
+  public String getDescription() {
+    return "Handle Faceting";
+  }
+
+  @Override
+  public String getVersion() {
+    return "$Revision$";
+  }
+
+  @Override
+  public String getSourceId() {
+    return "$Id$";
+  }
+
+  @Override
+  public String getSource() {
+    return "$URL$";
+  }
+
+  @Override
+  public URL[] getDocs() {
+    return null;
+  }
+}

Propchange: lucene/solr/trunk/src/java/org/apache/solr/handler/component/FacetComponent.java
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: lucene/solr/trunk/src/java/org/apache/solr/handler/component/FacetComponent.java
------------------------------------------------------------------------------
    svn:keywords = Date Author Id Revision HeadURL

Added: lucene/solr/trunk/src/java/org/apache/solr/handler/component/HighlightComponent.java
URL: http://svn.apache.org/viewvc/lucene/solr/trunk/src/java/org/apache/solr/handler/component/HighlightComponent.java?rev=594268&view=auto
==============================================================================
--- lucene/solr/trunk/src/java/org/apache/solr/handler/component/HighlightComponent.java (added)
+++ lucene/solr/trunk/src/java/org/apache/solr/handler/component/HighlightComponent.java Mon Nov 12 11:39:01 2007
@@ -0,0 +1,112 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.solr.handler.component;
+
+import org.apache.lucene.queryParser.ParseException;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.params.CommonParams;
+import org.apache.solr.common.params.SolrParams;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.handler.SearchHandler;
+import org.apache.solr.highlight.SolrHighlighter;
+import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.request.SolrQueryResponse;
+
+import java.io.IOException;
+import java.net.URL;
+
+/**
+ * TODO!
+ *
+ * @version $Id$
+ * @since solr 1.3
+ */
+public class HighlightComponent extends SearchComponent {
+  @Override
+  public void prepare(SolrQueryRequest req, SolrQueryResponse rsp) throws IOException, ParseException {
+    
+  }
+  
+  @Override
+  public void process(SolrQueryRequest req, SolrQueryResponse rsp) throws IOException {
+    SolrHighlighter highlighter = req.getCore().getHighlighter();
+    if (highlighter.isHighlightingEnabled(req.getParams())) {
+      ResponseBuilder builder = SearchHandler.getResponseBuilder( req );
+      SolrParams params = req.getParams();
+
+      String[] defaultHighlightFields;  //TODO: get from builder by default?
+
+      if (builder.getQparser() != null) {
+        defaultHighlightFields = builder.getQparser().getDefaultHighlightFields();
+      } else {
+        defaultHighlightFields = params.getParams(CommonParams.DF);
+      }
+      
+      if(builder.getHighlightQuery()==null) {
+        if (builder.getQparser() != null) {
+          try {
+            builder.setHighlightQuery( builder.getQparser().getHighlightQuery() );
+          } catch (Exception e) {
+            throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, e);
+          }
+        } else {
+          builder.setHighlightQuery( builder.getQuery() );
+        }
+      }
+      
+      NamedList sumData = highlighter.doHighlighting(
+              builder.getResults().docList,
+              builder.getHighlightQuery().rewrite(req.getSearcher().getReader()),
+              req, defaultHighlightFields );
+      
+      if(sumData != null) {
+        // TODO ???? add this directly to the response?
+        rsp.add("highlighting", sumData);
+      }
+    }
+  }
+  
+  /////////////////////////////////////////////
+  ///  SolrInfoMBean
+  ////////////////////////////////////////////
+  
+  @Override
+  public String getDescription() {
+    return "Highlighting";
+  }
+  
+  @Override
+  public String getVersion() {
+    return "$Revision$";
+  }
+  
+  @Override
+  public String getSourceId() {
+    return "$Id$";
+  }
+  
+  @Override
+  public String getSource() {
+    return "$URL$";
+  }
+  
+  @Override
+  public URL[] getDocs() {
+    return null;
+  }
+}

Propchange: lucene/solr/trunk/src/java/org/apache/solr/handler/component/HighlightComponent.java
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: lucene/solr/trunk/src/java/org/apache/solr/handler/component/HighlightComponent.java
------------------------------------------------------------------------------
    svn:keywords = Date Author Id Revision HeadURL

Added: lucene/solr/trunk/src/java/org/apache/solr/handler/component/MoreLikeThisComponent.java
URL: http://svn.apache.org/viewvc/lucene/solr/trunk/src/java/org/apache/solr/handler/component/MoreLikeThisComponent.java?rev=594268&view=auto
==============================================================================
--- lucene/solr/trunk/src/java/org/apache/solr/handler/component/MoreLikeThisComponent.java (added)
+++ lucene/solr/trunk/src/java/org/apache/solr/handler/component/MoreLikeThisComponent.java Mon Nov 12 11:39:01 2007
@@ -0,0 +1,96 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.solr.handler.component;
+
+import java.io.IOException;
+import java.net.URL;
+
+import org.apache.lucene.queryParser.ParseException;
+import org.apache.solr.common.params.MoreLikeThisParams;
+import org.apache.solr.common.params.SolrParams;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.handler.MoreLikeThisHandler;
+import org.apache.solr.handler.SearchHandler;
+import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.request.SolrQueryResponse;
+import org.apache.solr.search.DocList;
+import org.apache.solr.search.SolrIndexSearcher;
+
+/**
+ * TODO!
+ * 
+ * @version $Id$
+ * @since solr 1.3
+ */
+public class MoreLikeThisComponent extends SearchComponent
+{
+  @Override
+  public void prepare(SolrQueryRequest req, SolrQueryResponse rsp) throws IOException, ParseException 
+  {
+    
+  }
+
+  @Override
+  public void process(SolrQueryRequest req, SolrQueryResponse rsp) throws IOException 
+  {
+    SolrParams p = req.getParams();
+    if( p.getBool( MoreLikeThisParams.MLT, false ) ) {
+      ResponseBuilder builder = SearchHandler.getResponseBuilder( req );
+      SolrIndexSearcher searcher = req.getSearcher();
+      
+      MoreLikeThisHandler.MoreLikeThisHelper mlt 
+        = new MoreLikeThisHandler.MoreLikeThisHelper( p, searcher );
+      
+      int mltcount = p.getInt( MoreLikeThisParams.DOC_COUNT, 5 );
+      NamedList<DocList> sim = mlt.getMoreLikeThese(
+          builder.getResults().docList, mltcount, builder.getFieldFlags() );
+
+      // TODO ???? add this directly to the response?
+      rsp.add( "moreLikeThis", sim );
+    }
+  }
+
+  /////////////////////////////////////////////
+  ///  SolrInfoMBean
+  ////////////////////////////////////////////
+
+  @Override
+  public String getDescription() {
+    return "More Like This";
+  }
+
+  @Override
+  public String getVersion() {
+    return "$Revision$";
+  }
+
+  @Override
+  public String getSourceId() {
+    return "$Id$";
+  }
+
+  @Override
+  public String getSource() {
+    return "$URL$";
+  }
+
+  @Override
+  public URL[] getDocs() {
+    return null;
+  }
+}

Propchange: lucene/solr/trunk/src/java/org/apache/solr/handler/component/MoreLikeThisComponent.java
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: lucene/solr/trunk/src/java/org/apache/solr/handler/component/MoreLikeThisComponent.java
------------------------------------------------------------------------------
    svn:keywords = Date Author Id Revision HeadURL

Added: lucene/solr/trunk/src/java/org/apache/solr/handler/component/QueryComponent.java
URL: http://svn.apache.org/viewvc/lucene/solr/trunk/src/java/org/apache/solr/handler/component/QueryComponent.java?rev=594268&view=auto
==============================================================================
--- lucene/solr/trunk/src/java/org/apache/solr/handler/component/QueryComponent.java (added)
+++ lucene/solr/trunk/src/java/org/apache/solr/handler/component/QueryComponent.java Mon Nov 12 11:39:01 2007
@@ -0,0 +1,146 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.solr.handler.component;
+
+import org.apache.lucene.queryParser.ParseException;
+import org.apache.lucene.search.Query;
+import org.apache.solr.common.params.CommonParams;
+import org.apache.solr.common.params.SolrParams;
+import org.apache.solr.handler.SearchHandler;
+import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.request.SolrQueryResponse;
+import org.apache.solr.search.*;
+import org.apache.solr.util.SolrPluginUtils;
+
+import java.io.IOException;
+import java.net.URL;
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * TODO!
+ * 
+ * @version $Id$
+ * @since solr 1.3
+ */
+public class QueryComponent extends SearchComponent
+{
+  @Override
+  public void prepare(SolrQueryRequest req, SolrQueryResponse rsp) throws IOException, ParseException 
+  {
+    ResponseBuilder builder = SearchHandler.getResponseBuilder( req );
+    SolrParams params = req.getParams();
+    
+    // Set field flags
+    String fl = params.get(CommonParams.FL);
+    int fieldFlags = 0;
+    if (fl != null) {
+      fieldFlags |= SolrPluginUtils.setReturnFields(fl, rsp);
+    }
+    builder.setFieldFlags( fieldFlags ); 
+
+    String defType = params.get(QueryParsing.DEFTYPE);
+    defType = defType==null ? OldLuceneQParserPlugin.NAME : defType;
+
+    if (builder.getQueryString() == null) {
+      builder.setQueryString( params.get( CommonParams.Q ) );
+    }
+
+    QParser parser = QParser.getParser(builder.getQueryString(), defType, req);
+    builder.setQuery( parser.getQuery() );
+    builder.setSortSpec( parser.getSort(true) );
+    
+    String[] fqs = req.getParams().getParams(org.apache.solr.common.params.CommonParams.FQ);
+    if (fqs!=null && fqs.length!=0) {
+      List<Query> filters = builder.getFilters();
+      if (filters==null) {
+        filters = new ArrayList<Query>();
+        builder.setFilters( filters );
+      }
+      for (String fq : fqs) {
+        if (fq != null && fq.trim().length()!=0) {
+          QParser fqp = QParser.getParser(fq, null, req);
+          filters.add(fqp.getQuery());
+        }
+      }
+    }
+  }
+
+  /**
+   * Actually run the query
+   */
+  @Override
+  public void process(SolrQueryRequest req, SolrQueryResponse rsp) throws IOException 
+  {
+    ResponseBuilder builder = SearchHandler.getResponseBuilder( req );
+    SolrIndexSearcher searcher = req.getSearcher();
+    SolrParams params = req.getParams();
+    
+    if( builder.isNeedDocSet() ) {
+      builder.setResults( searcher.getDocListAndSet(
+          builder.getQuery(), builder.getFilters(), builder.getSortSpec().getSort(),
+          builder.getSortSpec().getOffset(), builder.getSortSpec().getCount(),
+          builder.getFieldFlags() ) );
+    }
+    else {
+      DocListAndSet results = new DocListAndSet();
+      results.docList = searcher.getDocList(
+          builder.getQuery(), builder.getFilters(), builder.getSortSpec().getSort(),
+          builder.getSortSpec().getOffset(), builder.getSortSpec().getCount(),
+          builder.getFieldFlags() );
+      builder.setResults( results );
+    }
+
+    //pre-fetch returned documents
+    if (builder.getResults().docList != null && builder.getResults().docList.size()<=50) {
+      // TODO: this may depend on the highlighter component (or other components?)
+      SolrPluginUtils.optimizePreFetchDocs(builder.getResults().docList, builder.getQuery(), req, rsp);
+    }
+    rsp.add("response",builder.getResults().docList);
+  }
+  
+
+  /////////////////////////////////////////////
+  ///  SolrInfoMBean
+  ////////////////////////////////////////////
+
+  @Override
+  public String getDescription() {
+    return "query";
+  }
+
+  @Override
+  public String getVersion() {
+    return "$Revision$";
+  }
+
+  @Override
+  public String getSourceId() {
+    return "$Id$";
+  }
+
+  @Override
+  public String getSource() {
+    return "$URL$";
+  }
+
+  @Override
+  public URL[] getDocs() {
+    return null;
+  }
+}

Propchange: lucene/solr/trunk/src/java/org/apache/solr/handler/component/QueryComponent.java
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: lucene/solr/trunk/src/java/org/apache/solr/handler/component/QueryComponent.java
------------------------------------------------------------------------------
    svn:keywords = Date Author Id Revision HeadURL

Added: lucene/solr/trunk/src/java/org/apache/solr/handler/component/ResponseBuilder.java
URL: http://svn.apache.org/viewvc/lucene/solr/trunk/src/java/org/apache/solr/handler/component/ResponseBuilder.java?rev=594268&view=auto
==============================================================================
--- lucene/solr/trunk/src/java/org/apache/solr/handler/component/ResponseBuilder.java (added)
+++ lucene/solr/trunk/src/java/org/apache/solr/handler/component/ResponseBuilder.java Mon Nov 12 11:39:01 2007
@@ -0,0 +1,159 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.solr.handler.component;
+
+import org.apache.lucene.search.Query;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.common.util.RTimer;
+import org.apache.solr.search.DocListAndSet;
+import org.apache.solr.search.QParser;
+import org.apache.solr.search.QueryParsing;
+
+import java.util.List;
+
+/**
+ * TODO!
+ * 
+ * @version $Id$
+ * @since solr 1.3
+ */
+public class ResponseBuilder 
+{
+  private boolean needDocList = false;
+  private boolean needDocSet = false;
+  private int fieldFlags = 0;
+  private boolean debug = false;
+
+  private QParser qparser = null;
+  private String queryString = null;
+  private Query query = null;
+  private List<Query> filters = null;
+  private QueryParsing.SortSpec sortSpec = null;
+  
+  private DocListAndSet results = null;
+  private NamedList<Object> debugInfo = null;
+  private RTimer timer = null;
+  
+  private Query highlightQuery = null;
+
+  //-------------------------------------------------------------------------
+  
+  public boolean isDebug() {
+    return debug;
+  }
+
+  public void setDebug(boolean debug) {
+    this.debug = debug;
+  }
+
+  public NamedList<Object> getDebugInfo() {
+    return debugInfo;
+  }
+
+  public void setDebugInfo(NamedList<Object> debugInfo) {
+    this.debugInfo = debugInfo;
+  }
+
+  public int getFieldFlags() {
+    return fieldFlags;
+  }
+
+  public void setFieldFlags(int fieldFlags) {
+    this.fieldFlags = fieldFlags;
+  }
+
+  public List<Query> getFilters() {
+    return filters;
+  }
+
+  public void setFilters(List<Query> filters) {
+    this.filters = filters;
+  }
+
+  public Query getHighlightQuery() {
+    return highlightQuery;
+  }
+
+  public void setHighlightQuery(Query highlightQuery) {
+    this.highlightQuery = highlightQuery;
+  }
+
+  public boolean isNeedDocList() {
+    return needDocList;
+  }
+
+  public void setNeedDocList(boolean needDocList) {
+    this.needDocList = needDocList;
+  }
+
+  public boolean isNeedDocSet() {
+    return needDocSet;
+  }
+
+  public void setNeedDocSet(boolean needDocSet) {
+    this.needDocSet = needDocSet;
+  }
+
+  public QParser getQparser() {
+    return qparser;
+  }
+
+  public void setQparser(QParser qparser) {
+    this.qparser = qparser;
+  }
+
+  public String getQueryString() {
+    return queryString;
+  }
+
+  public void setQueryString(String qstr) {
+    this.queryString = qstr;
+  }
+
+  public Query getQuery() {
+    return query;
+  }
+
+  public void setQuery(Query query) {
+    this.query = query;
+  }
+
+  public DocListAndSet getResults() {
+    return results;
+  }
+
+  public void setResults(DocListAndSet results) {
+    this.results = results;
+  }
+
+  public QueryParsing.SortSpec getSortSpec() {
+    return sortSpec;
+  }
+
+  public void setSortSpec(QueryParsing.SortSpec sort) {
+    this.sortSpec = sort;
+  }
+
+  public RTimer getTimer() {
+    return timer;
+  }
+
+  public void setTimer(RTimer timer) {
+    this.timer = timer;
+  }
+}

Propchange: lucene/solr/trunk/src/java/org/apache/solr/handler/component/ResponseBuilder.java
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: lucene/solr/trunk/src/java/org/apache/solr/handler/component/ResponseBuilder.java
------------------------------------------------------------------------------
    svn:keywords = Date Author Id Revision HeadURL

Added: lucene/solr/trunk/src/java/org/apache/solr/handler/component/SearchComponent.java
URL: http://svn.apache.org/viewvc/lucene/solr/trunk/src/java/org/apache/solr/handler/component/SearchComponent.java?rev=594268&view=auto
==============================================================================
--- lucene/solr/trunk/src/java/org/apache/solr/handler/component/SearchComponent.java (added)
+++ lucene/solr/trunk/src/java/org/apache/solr/handler/component/SearchComponent.java Mon Nov 12 11:39:01 2007
@@ -0,0 +1,64 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.solr.handler.component;
+
+import java.io.IOException;
+import java.net.URL;
+
+import org.apache.lucene.queryParser.ParseException;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.common.util.SimpleOrderedMap;
+import org.apache.solr.core.SolrInfoMBean;
+import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.request.SolrQueryResponse;
+
+/**
+ * TODO!
+ * 
+ * @version $Id$
+ * @since solr 1.3
+ */
+public abstract class SearchComponent implements SolrInfoMBean
+{
+  public abstract void prepare( SolrQueryRequest req, SolrQueryResponse rsp ) throws IOException, ParseException;
+  public abstract void process( SolrQueryRequest req, SolrQueryResponse rsp ) throws IOException;
+  
+  //////////////////////// SolrInfoMBeans methods //////////////////////
+
+  public String getName() {
+    return this.getClass().getName();
+  }
+
+  public abstract String getDescription();
+  public abstract String getSourceId();
+  public abstract String getSource();
+  public abstract String getVersion();
+  
+  public Category getCategory() {
+    return Category.OTHER;
+  }
+
+  public URL[] getDocs() {
+    return null;  // this can be overridden, but not required
+  }
+
+  public NamedList getStatistics() {
+    NamedList lst = new SimpleOrderedMap();
+    return lst;
+  }
+}

Propchange: lucene/solr/trunk/src/java/org/apache/solr/handler/component/SearchComponent.java
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: lucene/solr/trunk/src/java/org/apache/solr/handler/component/SearchComponent.java
------------------------------------------------------------------------------
    svn:keywords = Date Author Id Revision HeadURL

Modified: lucene/solr/trunk/src/java/org/apache/solr/search/BoostQParserPlugin.java
URL: http://svn.apache.org/viewvc/lucene/solr/trunk/src/java/org/apache/solr/search/BoostQParserPlugin.java?rev=594268&r1=594267&r2=594268&view=diff
==============================================================================
--- lucene/solr/trunk/src/java/org/apache/solr/search/BoostQParserPlugin.java (original)
+++ lucene/solr/trunk/src/java/org/apache/solr/search/BoostQParserPlugin.java Mon Nov 12 11:39:01 2007
@@ -43,15 +43,16 @@
   public QParser createParser(String qstr, SolrParams localParams, SolrParams params, SolrQueryRequest req) {
     return new QParser(qstr, localParams, params, req) {
       QParser baseParser;
+      ValueSource vs;
+      String b;
 
       public Query parse() throws ParseException {
-        String b = localParams.get(BOOSTFUNC);
+        b = localParams.get(BOOSTFUNC);
         baseParser = subQuery(localParams.get(QueryParsing.V), null);
         Query q = baseParser.parse();
 
         if (b == null) return q;
         Query bq = subQuery(b, FunctionQParserPlugin.NAME).parse();
-        ValueSource vs;
         if (bq instanceof FunctionQuery) {
           vs = ((FunctionQuery)bq).getValueSource();
         } else {
@@ -63,6 +64,17 @@
 
       public String[] getDefaultHighlightFields() {
         return baseParser.getDefaultHighlightFields();
+      }
+
+      public Query getHighlightQuery() throws ParseException {
+        return baseParser.getHighlightQuery();
+      }
+
+      public void addDebugInfo(NamedList<Object> debugInfo) {
+        // encapsulate base debug info in a sub-list?
+        baseParser.addDebugInfo(debugInfo);
+        debugInfo.add("boost_str",b);
+        debugInfo.add("boost_parsed",vs);
       }
     };
   }

Modified: lucene/solr/trunk/src/java/org/apache/solr/search/DisMaxQParserPlugin.java
URL: http://svn.apache.org/viewvc/lucene/solr/trunk/src/java/org/apache/solr/search/DisMaxQParserPlugin.java?rev=594268&r1=594267&r2=594268&view=diff
==============================================================================
--- lucene/solr/trunk/src/java/org/apache/solr/search/DisMaxQParserPlugin.java (original)
+++ lucene/solr/trunk/src/java/org/apache/solr/search/DisMaxQParserPlugin.java Mon Nov 12 11:39:01 2007
@@ -79,6 +79,13 @@
   Map<String,Float> queryFields;
   Query parsedUserQuery;
 
+
+  private String[] boostParams;
+  private List<Query> boostQueries;
+  private Query altUserQuery;
+  private QParser altQParser;
+  
+
   public Query parse() throws ParseException {
     SolrParams solrParams = localParams == null ? params : new DefaultSolrParams(localParams, params);
 
@@ -120,12 +127,13 @@
     /* * * Main User Query * * */
     parsedUserQuery = null;
     String userQuery = getString();
-    Query altUserQuery = null;
+    altUserQuery = null;
     if( userQuery == null || userQuery.trim().length() < 1 ) {
       // If no query is specified, we may have an alternate
       String altQ = solrParams.get( DMP.ALTQ );
       if (altQ != null) {
-        altUserQuery = p.parse(altQ);
+        altQParser = subQuery(altQ, null);
+        altUserQuery = altQParser.parse();
         query.add( altUserQuery , BooleanClause.Occur.MUST );
       } else {
         throw new SolrException( SolrException.ErrorCode.BAD_REQUEST, "missing query string" );
@@ -166,12 +174,13 @@
 
 
     /* * * Boosting Query * * */
-    String[] boostParams = solrParams.getParams(DMP.BQ);
+    boostParams = solrParams.getParams(DMP.BQ);
     //List<Query> boostQueries = U.parseQueryStrings(req, boostParams);
-    List<Query> boostQueries=null;
+    boostQueries=null;
     if (boostParams!=null && boostParams.length>0) {
       boostQueries = new ArrayList<Query>();
       for (String qs : boostParams) {
+        if (qs.trim().length()==0) continue;
         Query q = subQuery(qs, null).parse();
         boostQueries.add(q);
       }
@@ -227,5 +236,16 @@
   @Override
   public Query getHighlightQuery() throws ParseException {
     return parsedUserQuery;
+  }
+
+  public void addDebugInfo(NamedList<Object> debugInfo) {
+    super.addDebugInfo(debugInfo);
+    debugInfo.add("altquerystring", altUserQuery);
+    if (null != boostQueries) {
+      debugInfo.add("boost_queries", boostParams);
+      debugInfo.add("parsed_boost_queries",
+                QueryParsing.toString(boostQueries, req.getSchema()));
+    }
+    debugInfo.add("boostfuncs", req.getParams().getParams(DisMaxParams.BF));   
   }
 }

Modified: lucene/solr/trunk/src/java/org/apache/solr/search/QParser.java
URL: http://svn.apache.org/viewvc/lucene/solr/trunk/src/java/org/apache/solr/search/QParser.java?rev=594268&r1=594267&r2=594268&view=diff
==============================================================================
--- lucene/solr/trunk/src/java/org/apache/solr/search/QParser.java (original)
+++ lucene/solr/trunk/src/java/org/apache/solr/search/QParser.java Mon Nov 12 11:39:01 2007
@@ -20,6 +20,7 @@
 import org.apache.lucene.search.Query;
 import org.apache.solr.common.params.CommonParams;
 import org.apache.solr.common.params.SolrParams;
+import org.apache.solr.common.util.NamedList;
 import org.apache.solr.request.SolrQueryRequest;
 
 public abstract class QParser {
@@ -174,6 +175,10 @@
 
   public Query getHighlightQuery() throws ParseException {
     return getQuery();
+  }
+
+  public void addDebugInfo(NamedList<Object> debugInfo) {
+    debugInfo.add("QParser", this.getClass().getSimpleName());
   }
 
   /** Create a <code>QParser</code> to parse <code>qstr</code>,

Modified: lucene/solr/trunk/src/java/org/apache/solr/search/QueryParsing.java
URL: http://svn.apache.org/viewvc/lucene/solr/trunk/src/java/org/apache/solr/search/QueryParsing.java?rev=594268&r1=594267&r2=594268&view=diff
==============================================================================
--- lucene/solr/trunk/src/java/org/apache/solr/search/QueryParsing.java (original)
+++ lucene/solr/trunk/src/java/org/apache/solr/search/QueryParsing.java Mon Nov 12 11:39:01 2007
@@ -183,7 +183,7 @@
    *  "<!prefix f=myfield v=$p>" returns type="prefix",f="myfield",v=params.get("p")
    */
   public static SolrParams getLocalParams(String txt, SolrParams params) throws ParseException {
-    if (!txt.startsWith("<!")) {
+    if (txt==null || !txt.startsWith("<!")) {
       return null;      
     }
     Map<String,String> localParams = new HashMap<String,String>();

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=594268&r1=594267&r2=594268&view=diff
==============================================================================
--- lucene/solr/trunk/src/java/org/apache/solr/search/SolrIndexSearcher.java (original)
+++ lucene/solr/trunk/src/java/org/apache/solr/search/SolrIndexSearcher.java Mon Nov 12 11:39:01 2007
@@ -29,7 +29,6 @@
 import org.apache.solr.core.SolrConfig;
 import org.apache.solr.core.SolrCore;
 import org.apache.solr.core.SolrInfoMBean;
-import org.apache.solr.core.SolrInfoRegistry;
 import org.apache.solr.schema.IndexSchema;
 import org.apache.solr.util.OpenBitSet;
 
@@ -38,6 +37,7 @@
 import java.util.*;
 import java.util.logging.Level;
 import java.util.logging.Logger;
+import org.apache.solr.core.SolrInfoMBean.Category;
 
 
 /**