You are viewing a plain text version of this content. The canonical link for it is here.
Posted to solr-commits@lucene.apache.org by yo...@apache.org on 2006/08/23 21:34:58 UTC

svn commit: r434136 - in /incubator/solr/trunk: example/solr/conf/ src/java/org/apache/solr/request/ src/java/org/apache/solr/util/ src/test/org/apache/solr/ src/test/test-files/solr/conf/ src/webapp/src/org/apache/solr/servlet/

Author: yonik
Date: Wed Aug 23 12:34:57 2006
New Revision: 434136

URL: http://svn.apache.org/viewvc?rev=434136&view=rev
Log:
SolrParams: SOLR-43

Added:
    incubator/solr/trunk/src/java/org/apache/solr/request/DefaultSolrParams.java   (with props)
    incubator/solr/trunk/src/java/org/apache/solr/request/MapSolrParams.java   (with props)
    incubator/solr/trunk/src/java/org/apache/solr/request/MultiMapSolrParams.java   (with props)
    incubator/solr/trunk/src/java/org/apache/solr/request/ServletSolrParams.java   (with props)
    incubator/solr/trunk/src/java/org/apache/solr/request/SolrParams.java   (with props)
Modified:
    incubator/solr/trunk/example/solr/conf/solrconfig.xml
    incubator/solr/trunk/src/java/org/apache/solr/request/DisMaxRequestHandler.java
    incubator/solr/trunk/src/java/org/apache/solr/request/LocalSolrQueryRequest.java
    incubator/solr/trunk/src/java/org/apache/solr/request/SolrQueryRequest.java
    incubator/solr/trunk/src/java/org/apache/solr/request/SolrQueryRequestBase.java
    incubator/solr/trunk/src/java/org/apache/solr/request/StandardRequestHandler.java
    incubator/solr/trunk/src/java/org/apache/solr/util/AbstractSolrTestCase.java
    incubator/solr/trunk/src/java/org/apache/solr/util/CommonParams.java
    incubator/solr/trunk/src/java/org/apache/solr/util/SolrPluginUtils.java
    incubator/solr/trunk/src/java/org/apache/solr/util/TestHarness.java
    incubator/solr/trunk/src/test/org/apache/solr/BasicFunctionalityTest.java
    incubator/solr/trunk/src/test/test-files/solr/conf/solrconfig.xml
    incubator/solr/trunk/src/webapp/src/org/apache/solr/servlet/SolrServletRequest.java

Modified: incubator/solr/trunk/example/solr/conf/solrconfig.xml
URL: http://svn.apache.org/viewvc/incubator/solr/trunk/example/solr/conf/solrconfig.xml?rev=434136&r1=434135&r2=434136&view=diff
==============================================================================
--- incubator/solr/trunk/example/solr/conf/solrconfig.xml (original)
+++ incubator/solr/trunk/example/solr/conf/solrconfig.xml Wed Aug 23 12:34:57 2006
@@ -178,7 +178,15 @@
      The "standard" request handler is the default and will be used if qt
      is not specified in the request.
   -->
-  <requestHandler name="standard" class="solr.StandardRequestHandler" />
+  <requestHandler name="standard" class="solr.StandardRequestHandler">
+    <!-- default values for query parameters may optionally be defined here
+     <lst name="defaults">
+       <int name="rows">10</int>
+       <str name="fl">*</str>
+       <str name="version">2.1</str>
+     <lst>
+    -->
+  </requestHandler>
 
   <!-- DisMaxRequestHandler is an example of a request handler that
        supports optional parameters which are passed to

Added: incubator/solr/trunk/src/java/org/apache/solr/request/DefaultSolrParams.java
URL: http://svn.apache.org/viewvc/incubator/solr/trunk/src/java/org/apache/solr/request/DefaultSolrParams.java?rev=434136&view=auto
==============================================================================
--- incubator/solr/trunk/src/java/org/apache/solr/request/DefaultSolrParams.java (added)
+++ incubator/solr/trunk/src/java/org/apache/solr/request/DefaultSolrParams.java Wed Aug 23 12:34:57 2006
@@ -0,0 +1,45 @@
+/**
+ * Copyright 2006 The Apache Software Foundation
+ *
+ * Licensed 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.request;
+
+/**
+ * @author yonik
+ * @version $Id$
+ */
+public class DefaultSolrParams extends SolrParams {
+  protected final SolrParams params;
+  protected final SolrParams defaults;
+
+  public DefaultSolrParams(SolrParams params, SolrParams defaults) {
+    this.params = params;
+    this.defaults = defaults;
+  }
+
+  public String get(String param) {
+    String val = params.get(param);
+    return val!=null ? val : defaults.get(param);
+  }
+
+  public String[] getParams(String param) {
+    String[] vals = params.getParams(param);
+    return vals!=null ? vals : defaults.getParams(param);
+  }
+
+  public String toString() {
+    return "{params("+params+"),defaults("+defaults+")}";
+  }
+}

Propchange: incubator/solr/trunk/src/java/org/apache/solr/request/DefaultSolrParams.java
------------------------------------------------------------------------------
    svn:eol-style = native

Modified: incubator/solr/trunk/src/java/org/apache/solr/request/DisMaxRequestHandler.java
URL: http://svn.apache.org/viewvc/incubator/solr/trunk/src/java/org/apache/solr/request/DisMaxRequestHandler.java?rev=434136&r1=434135&r2=434136&view=diff
==============================================================================
--- incubator/solr/trunk/src/java/org/apache/solr/request/DisMaxRequestHandler.java (original)
+++ incubator/solr/trunk/src/java/org/apache/solr/request/DisMaxRequestHandler.java Wed Aug 23 12:34:57 2006
@@ -21,61 +21,31 @@
 import org.apache.solr.core.SolrException;
 
 import org.apache.solr.search.SolrIndexSearcher;
-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.SolrCache;
 import org.apache.solr.search.SolrQueryParser;
 import org.apache.solr.search.QueryParsing;
-import org.apache.solr.search.CacheRegenerator;
 
-import org.apache.solr.request.StandardRequestHandler;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.request.SolrQueryResponse;
 import org.apache.solr.request.SolrRequestHandler;
 
 import org.apache.solr.schema.IndexSchema;
-import org.apache.solr.schema.FieldType;
 
-import org.apache.solr.util.StrUtils;
 import org.apache.solr.util.NamedList;
 import org.apache.solr.util.SolrPluginUtils;
 import org.apache.solr.util.DisMaxParams;
 
-import org.apache.lucene.document.Document;
-import org.apache.lucene.document.Field;
-import org.apache.lucene.index.Term;
 import org.apache.lucene.search.Query;
-import org.apache.lucene.search.TermQuery;
-import org.apache.lucene.search.DisjunctionMaxQuery;
 import org.apache.lucene.search.BooleanQuery;
 import org.apache.lucene.search.BooleanClause;
 import org.apache.lucene.search.BooleanClause.Occur;
-import org.apache.lucene.search.ConstantScoreRangeQuery;
-import org.apache.lucene.search.Sort;
-import org.apache.lucene.search.Explanation;
 import org.apache.lucene.queryParser.QueryParser;
-import org.apache.lucene.queryParser.ParseException;
-import org.apache.lucene.analysis.Analyzer;
-
-import org.xmlpull.v1.XmlPullParserException;
 
 /* this is the standard logging framework for Solr */
-import java.util.logging.Logger;
-import java.util.logging.Level;
-import java.util.logging.Handler;
 
-import java.util.Arrays;
 import java.util.ArrayList;
 import java.util.List;
-import java.util.Collection;
-import java.util.Set;
-import java.util.HashSet;
 import java.util.Map;
-import java.util.HashMap;
-import java.util.regex.Pattern;
-import java.io.IOException;
 import java.net.URL;
     
 /**
@@ -334,7 +304,7 @@
             
       /* * * Generate Main Results * * */
 
-      flags |= U.setReturnFields(U.getParam(req, params.FL, params.fl), rsp);
+      flags |= U.setReturnFields(U.getParam(req, SolrParams.FL, params.fl), rsp);
       DocList results = s.getDocList(query, restrictions,
                                      SolrPluginUtils.getSort(req),
                                      req.getStart(), req.getLimit(),
@@ -367,7 +337,7 @@
       }
 
       /* * * Highlighting/Summarizing  * * */
-      if(U.getBooleanParam(req, params.HIGHLIGHT, params.highlight)) {
+      if(U.getBooleanParam(req, SolrParams.HIGHLIGHT, params.highlight)) {
 
         BooleanQuery highlightQuery = new BooleanQuery();
         U.flattenBooleanQuery(highlightQuery, query);

Modified: incubator/solr/trunk/src/java/org/apache/solr/request/LocalSolrQueryRequest.java
URL: http://svn.apache.org/viewvc/incubator/solr/trunk/src/java/org/apache/solr/request/LocalSolrQueryRequest.java?rev=434136&r1=434135&r2=434136&view=diff
==============================================================================
--- incubator/solr/trunk/src/java/org/apache/solr/request/LocalSolrQueryRequest.java (original)
+++ incubator/solr/trunk/src/java/org/apache/solr/request/LocalSolrQueryRequest.java Wed Aug 23 12:34:57 2006
@@ -16,198 +16,49 @@
 
 package org.apache.solr.request;
 
-import org.apache.solr.search.SolrIndexSearcher;
-import org.apache.solr.util.RefCounted;
-import org.apache.solr.util.StrUtils;
 import org.apache.solr.util.NamedList;
-import org.apache.solr.schema.IndexSchema;
 import org.apache.solr.core.SolrCore;
 
 import java.util.Map;
 import java.util.HashMap;
+import java.util.Iterator;
+
+// With the addition of SolrParams, this class isn't needed for much anymore... it's currently
+// retained more for backward compatibility.
 
 /**
  * @author yonik
  * @version $Id$
  */
 public class LocalSolrQueryRequest extends SolrQueryRequestBase {
-  private final NamedList args;
-  private final String query;
-  private final String qtype;
-  private final int start;
-  private final int limit;
-
   public final static Map emptyArgs = new HashMap(0,1);
 
-  public LocalSolrQueryRequest(SolrCore core, String query, String qtype, int start, int limit, Map args) {
-    super(core);
-    this.query=query;
-    this.qtype=qtype;
-    this.start=start;
-    this.limit=limit;
-
-    this.args = new NamedList();
-    if (query!=null) this.args.add(SolrQueryRequestBase.QUERY_NAME, query);
-    if (qtype!=null) this.args.add(SolrQueryRequestBase.QUERYTYPE_NAME, qtype);
-    this.args.add(SolrQueryRequestBase.START_NAME, Integer.toString(start));
-    this.args.add(SolrQueryRequestBase.ROWS_NAME, Integer.toString(limit));
-
-    if (args!=null) this.args.addAll(args);
-  }
-
-
-  public LocalSolrQueryRequest(SolrCore core, NamedList args) {
-    super(core);
-    this.args=args;
-    this.query=getStrParam(QUERY_NAME,null);
-    this.qtype=getStrParam(QUERYTYPE_NAME,null);
-    this.start=getIntParam(START_NAME,0);
-    this.limit=getIntParam(ROWS_NAME,10);
-  }
-
-
-  public String getParam(String name) {
-    Object value = args.get(name);
-    if (value == null || value instanceof String) {
-      return (String) value;
-    }
-    else {
-      return ((String[]) value)[0];
-    }
-  }
-
-  public String[] getParams(String name) {
-    Object value = args.get(name);
-    if (value instanceof String) {
-      return new String[] {(String)value};
-    } else {
-      return (String[]) value;
-    }
-  }
-
-  public String getQueryString() {
-    return query;
-  }
-
-  // signifies the syntax and the handler that should be used
-  // to execute this query.
-  public String getQueryType() {
-    return qtype;
-  }
-
-
-  // starting position in matches to return to client
-  public int getStart() {
-    return start;
-  }
-
-  // number of matching documents to return
-  public int getLimit() {
-    return limit;
-  }
-
-  final long startTime=System.currentTimeMillis();
-  // Get the start time of this request in milliseconds
-  public long getStartTime() {
-    return startTime;
-  }
-
-  // The index searcher associated with this request
-  RefCounted<SolrIndexSearcher> searcherHolder;
-  public SolrIndexSearcher getSearcher() {
-    // should this reach out and get a searcher from the core singleton, or
-    // should the core populate one in a factory method to create requests?
-    // or there could be a setSearcher() method that Solr calls
-
-    if (searcherHolder==null) {
-      searcherHolder = core.getSearcher();
-    }
-
-    return searcherHolder.get();
-  }
-
-  // The solr core (coordinator, etc) associated with this request
-  public SolrCore getCore() {
-    return core;
+  protected static SolrParams makeParams(String query, String qtype, int start, int limit, Map args) {
+    Map<String,String[]> map = new HashMap<String,String[]>();
+    for (Iterator iter = args.entrySet().iterator(); iter.hasNext();) {
+      Map.Entry e = (Map.Entry)iter.next();
+      String k = e.getKey().toString();
+      Object v = e.getValue();
+      if (v instanceof String[]) map.put(k,(String[])v);
+      else map.put(k,new String[]{v.toString()});
+    }
+    if (query!=null) map.put(SolrParams.Q, new String[]{query});
+    if (qtype!=null) map.put(SolrParams.QT, new String[]{qtype});
+    map.put(SolrParams.START, new String[]{Integer.toString(start)});
+    map.put(SolrParams.ROWS, new String[]{Integer.toString(limit)});
+    return new MultiMapSolrParams(map);
   }
 
-  // The index schema associated with this request
-  public IndexSchema getSchema() {
-    return core.getSchema();
+  public LocalSolrQueryRequest(SolrCore core, String query, String qtype, int start, int limit, Map args) {
+    super(core,makeParams(query,qtype,start,limit,args));
   }
 
-  public String getParamString() {
-    StringBuilder sb = new StringBuilder(128);
-    try {
-
-      boolean first=true;
-      if (query!=null) {
-        if (!first) {
-          sb.append('&');
-        }
-        first=false;
-        sb.append("q=");
-        StrUtils.partialURLEncodeVal(sb,query);
-      }
-
-      // null, "", and "standard" are all the default query handler.
-      if (qtype!=null && !(qtype.equals("") || qtype.equals("standard"))) {
-        if (!first) {
-          sb.append('&');
-        }
-        first=false;
-        sb.append("qt=");
-        sb.append(qtype);
-      }
-
-      if (start!=0) {
-        if (!first) {
-          sb.append('&');
-        }
-        first=false;
-        sb.append("start=");
-        sb.append(start);
-      }
-
-      if (!first) {
-        sb.append('&');
-      }
-      first=false;
-      sb.append("rows=");
-      sb.append(limit);
-
-      if (args != null && args.size() > 0) {
-        for (int i=0; i<args.size(); i++) {
-          if (!first) {
-            sb.append('&');
-          }
-          first=false;
-
-          sb.append(args.getName(i));
-          sb.append('=');
-          StrUtils.partialURLEncodeVal(sb,args.getVal(i).toString());
-        }
-      }
-
-    } catch (Exception e) {
-      // should never happen... we only needed this because
-      // partialURLEncodeVal can throw an IOException, but it
-      // never will when adding to a StringBuilder.
-      throw new RuntimeException(e);
-    }
-
-    return sb.toString();
+  public LocalSolrQueryRequest(SolrCore core, NamedList args) {
+    super(core, SolrParams.toSolrParams(args));
   }
 
-
-  public void close() {
-    if (searcherHolder!=null) {
-      searcherHolder.decref();
-    }
+  public LocalSolrQueryRequest(SolrCore core, Map<String,String[]> args) {
+    super(core, new MultiMapSolrParams(args));
   }
-
-
-
 }
-
 

Added: incubator/solr/trunk/src/java/org/apache/solr/request/MapSolrParams.java
URL: http://svn.apache.org/viewvc/incubator/solr/trunk/src/java/org/apache/solr/request/MapSolrParams.java?rev=434136&view=auto
==============================================================================
--- incubator/solr/trunk/src/java/org/apache/solr/request/MapSolrParams.java (added)
+++ incubator/solr/trunk/src/java/org/apache/solr/request/MapSolrParams.java Wed Aug 23 12:34:57 2006
@@ -0,0 +1,66 @@
+/**
+ * Copyright 2006 The Apache Software Foundation
+ *
+ * Licensed 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.request;
+
+import org.apache.solr.util.StrUtils;
+
+import java.util.Map;
+import java.io.IOException;
+
+/**
+ * @author yonik
+ * @version $Id$
+ */
+public class MapSolrParams extends SolrParams {
+  protected final Map<String,String> map;
+
+  public MapSolrParams(Map<String,String> map) {
+    this.map = map;
+  }
+
+  public String get(String name) {
+    return map.get(name);
+  }
+
+  public String[] getParams(String name) {
+    String val = map.get(name);
+    return val==null ? null : new String[]{val};
+  }
+
+  public Map<String,String> getMap() { return map; }
+
+  public String toString() {
+    StringBuilder sb = new StringBuilder(128);
+    try {
+      boolean first=true;
+
+      for (Map.Entry<String,String> entry : map.entrySet()) {
+        String key = entry.getKey();
+        String val = entry.getValue();
+
+        if (!first) sb.append('&');
+        first=false;
+        sb.append(key);
+        sb.append('=');
+        StrUtils.partialURLEncodeVal(sb, val==null ? "" : val);
+      }
+    }
+    catch (IOException e) {throw new RuntimeException(e);}  // can't happen
+
+    return sb.toString();
+  }
+}

Propchange: incubator/solr/trunk/src/java/org/apache/solr/request/MapSolrParams.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: incubator/solr/trunk/src/java/org/apache/solr/request/MultiMapSolrParams.java
URL: http://svn.apache.org/viewvc/incubator/solr/trunk/src/java/org/apache/solr/request/MultiMapSolrParams.java?rev=434136&view=auto
==============================================================================
--- incubator/solr/trunk/src/java/org/apache/solr/request/MultiMapSolrParams.java (added)
+++ incubator/solr/trunk/src/java/org/apache/solr/request/MultiMapSolrParams.java Wed Aug 23 12:34:57 2006
@@ -0,0 +1,83 @@
+/**
+ * Copyright 2006 The Apache Software Foundation
+ *
+ * Licensed 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.request;
+
+import org.apache.solr.util.StrUtils;
+
+import java.util.Map;
+import java.io.IOException;
+
+/**
+ * @author yonik
+ * @version $Id$
+ */
+public class MultiMapSolrParams extends SolrParams {
+  protected final Map<String,String[]> map;
+
+  public static void addParam(String name, String val, Map<String,String[]> map) {
+      String[] arr = map.get(name);
+      if (arr ==null) {
+        arr =new String[]{val};
+      } else {
+        String[] newarr = new String[arr.length+1];
+        System.arraycopy(arr,0,newarr,0,arr.length);
+        newarr[arr.length]=val;
+        arr =newarr;
+      }
+      map.put(name, arr);
+  }
+
+  public MultiMapSolrParams(Map<String,String[]> map) {
+    this.map = map;
+  }
+
+  public String get(String name) {
+    String[] arr = map.get(name);
+    return arr==null ? null : arr[0];
+  }
+
+  public String[] getParams(String name) {
+    return map.get(name);
+  }
+
+  public Map<String,String[]> getMap() { return map; }
+
+  public String toString() {
+    StringBuilder sb = new StringBuilder(128);
+    try {
+      boolean first=true;
+
+      for (Map.Entry<String,String[]> entry : map.entrySet()) {
+        String key = entry.getKey();
+        String[] valarr = entry.getValue();
+
+        for (String val : valarr) {
+          if (!first) sb.append('&');
+          first=false;
+          sb.append(key);
+          sb.append('=');
+          StrUtils.partialURLEncodeVal(sb, val==null ? "" : val);
+        }
+      }
+    }
+    catch (IOException e) {throw new RuntimeException(e);}  // can't happen
+
+    return sb.toString();
+  }
+
+
+}

Propchange: incubator/solr/trunk/src/java/org/apache/solr/request/MultiMapSolrParams.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: incubator/solr/trunk/src/java/org/apache/solr/request/ServletSolrParams.java
URL: http://svn.apache.org/viewvc/incubator/solr/trunk/src/java/org/apache/solr/request/ServletSolrParams.java?rev=434136&view=auto
==============================================================================
--- incubator/solr/trunk/src/java/org/apache/solr/request/ServletSolrParams.java (added)
+++ incubator/solr/trunk/src/java/org/apache/solr/request/ServletSolrParams.java Wed Aug 23 12:34:57 2006
@@ -0,0 +1,21 @@
+package org.apache.solr.request;
+
+import javax.servlet.ServletRequest;
+
+/**
+ * @author yonik
+ * @version $Id$
+ */
+public class ServletSolrParams extends MultiMapSolrParams {
+  public ServletSolrParams(ServletRequest req) {
+    super(req.getParameterMap());
+  }
+
+  public String get(String name) {
+    String[] arr = map.get(name);
+    if (arr==null) return null;
+    String s = arr[0];
+    if (s.length()==0) return null;  // screen out blank parameters
+    return s;
+  }
+}

Propchange: incubator/solr/trunk/src/java/org/apache/solr/request/ServletSolrParams.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: incubator/solr/trunk/src/java/org/apache/solr/request/SolrParams.java
URL: http://svn.apache.org/viewvc/incubator/solr/trunk/src/java/org/apache/solr/request/SolrParams.java?rev=434136&view=auto
==============================================================================
--- incubator/solr/trunk/src/java/org/apache/solr/request/SolrParams.java (added)
+++ incubator/solr/trunk/src/java/org/apache/solr/request/SolrParams.java Wed Aug 23 12:34:57 2006
@@ -0,0 +1,162 @@
+/**
+ * Copyright 2006 The Apache Software Foundation
+ *
+ * Licensed 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.request;
+
+import org.apache.solr.util.NamedList;
+import org.apache.solr.util.StrUtils;
+
+import javax.servlet.ServletRequest;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+import java.io.IOException;
+
+/**  SolrParams hold request parameters.
+ *
+ * @author yonik
+ * @version $Id$
+ */
+public abstract class SolrParams {
+  /** the query type - which query handler should handle the request */
+  public static final String QT ="qt";
+  /** the response writer type - the format of the response */
+  public static final String WT ="wt";
+  /** query string */
+  public static final String Q ="q";
+  /** zero based offset of matching documents to retrieve */
+  public static final String START ="start";
+  /** number of documents to return starting at "start" */
+  public static final String ROWS ="rows";
+  /** stylesheet to apply to XML results */
+  public static final String XSL ="xsl";
+  /** stylesheet to apply to XML results */
+  public static final String VERSION ="version";
+  /** query and init param for field list */
+  public static final String FL = "fl";
+  /** default query field */
+  public static final String DF = "df";
+  /** whether to include debug data */
+  public static final String DEBUG_QUERY = "debugQuery";
+  /** another query to explain against */
+  public static final String EXPLAIN_OTHER = "explainOther";
+  /** wether to highlight */
+  public static final String HIGHLIGHT = "highlight";
+  /** fields to highlight */
+  public static final String HIGHLIGHT_FIELDS = "highlightFields";
+  /** maximum highlight fragments to return */
+  public static final String MAX_SNIPPETS = "maxSnippets";
+  /** override default highlight Formatter class */
+  public static final String HIGHLIGHT_FORMATTER_CLASS = "highlightFormatterClass";
+
+
+
+
+  /** returns the String value of a param, or null if not set */
+  public abstract String get(String param);
+
+  /** returns an array of the String values of a param, or null if none */
+  public abstract String[] getParams(String param);
+
+
+  protected String fpname(String field, String param) {
+    return "f."+field+'.'+param;
+  }
+
+  /** returns the String value of the field parameter, "f.field.param", or
+   *  the value for "param" if that is not set.
+   */
+  public String getFieldParam(String field, String param) {
+    String val = get(fpname(field,param));
+    return val!=null ? val : get(param);
+  }
+
+  /** Returns the Boolean value of the param, or null if not set */
+  public Boolean getBool(String param) {
+    String val = get(param);
+    return val==null ? null : parseBool(val);
+  }
+
+  /** Returns the boolean value of the param, or def if not set */
+  public boolean getBool(String param, boolean def) {
+    String val = get(param);
+    return val==null ? def : parseBool(val);
+  }
+
+  /** Returns the Integer value of the param, or null if not set */
+  public Integer getInt(String param) {
+    String val = get(param);
+    return val==null ? null : Integer.parseInt(val);
+  }
+
+  /** Returns the int value of the param, or def if not set */
+  public int getInt(String param, int def) {
+    String val = get(param);
+    return val==null ? def : Integer.parseInt(val);
+  }
+
+  /** Returns the Float value of the param, or null if not set */
+  public Float getFloat(String param) {
+    String val = get(param);
+    return val==null ? null : Float.parseFloat(val);
+  }
+
+  /** Returns the float value of the param, or def if not set */
+  public float getFloat(String param, float def) {
+    String val = get(param);
+    return val==null ? def : Float.parseFloat(val);
+  }
+
+  /** how to transform a String into a boolean... more flexible than
+   * Boolean.parseBoolean() to enable easier integration with html forms.
+   */
+  protected boolean parseBool(String s) {
+    return s.startsWith("true") || s.startsWith("on") || s.startsWith("yes");
+  }
+
+  /** Create a Map<String,String> from a NamedList given no keys are repeated */
+  public static Map<String,String> toMap(NamedList params) {
+    HashMap<String,String> map = new HashMap<String,String>();
+    for (int i=0; i<params.size(); i++) {
+      map.put(params.getName(i), params.getVal(i).toString());
+    }
+    return map;
+  }
+
+  /** Create a Map<String,String[]> from a NamedList */
+  public static Map<String,String[]> toMultiMap(NamedList params) {
+    HashMap<String,String[]> map = new HashMap<String,String[]>();
+    for (int i=0; i<params.size(); i++) {
+      String name = params.getName(i);
+      String val = params.getVal(i).toString();
+      MultiMapSolrParams.addParam(name,val,map);
+    }
+    return map;
+  }
+
+  /** Create SolrParams from NamedList. */
+  public static SolrParams toSolrParams(NamedList params) {
+    // if no keys are repeated use the faster MapSolrParams
+    HashMap<String,String> map = new HashMap<String,String>();
+    for (int i=0; i<params.size(); i++) {
+      String prev = map.put(params.getName(i), params.getVal(i).toString());
+      if (prev!=null) return new MultiMapSolrParams(toMultiMap(params));
+    }
+    return new MapSolrParams(map);
+  }
+}
+
+

Propchange: incubator/solr/trunk/src/java/org/apache/solr/request/SolrParams.java
------------------------------------------------------------------------------
    svn:eol-style = native

Modified: incubator/solr/trunk/src/java/org/apache/solr/request/SolrQueryRequest.java
URL: http://svn.apache.org/viewvc/incubator/solr/trunk/src/java/org/apache/solr/request/SolrQueryRequest.java?rev=434136&r1=434135&r2=434136&view=diff
==============================================================================
--- incubator/solr/trunk/src/java/org/apache/solr/request/SolrQueryRequest.java (original)
+++ incubator/solr/trunk/src/java/org/apache/solr/request/SolrQueryRequest.java Wed Aug 23 12:34:57 2006
@@ -27,6 +27,11 @@
  * @version $Id$
  */
 public interface SolrQueryRequest {
+  public SolrParams getParams();
+
+  public void setParams(SolrParams params);
+
+  public SolrParams getOriginalParams();
 
   /**
    * This method should be called when all uses of this request are
@@ -39,29 +44,35 @@
    * @return the value, or the first value if the parameter was
    * specified more then once; may be null.
    */
+  @Deprecated
   public String getParam(String name);
 
   /**
    * Returns the input parameter values for the specified name
    * @return the values; may be null or empty depending on implementation
    */
+  @Deprecated
   public String[] getParams(String name);
 
   /**
    * Returns the primary query string parameter of the request
    */
+  @Deprecated
   public String getQueryString();
 
   /**
    * Signifies the syntax and the handler that should be used
    * to execute this query.
    */
+  @Deprecated
   public String getQueryType();
 
   /** starting position in matches to return to client */
+  @Deprecated
   public int getStart();
 
   /** number of matching documents to return */
+  @Deprecated
   public int getLimit();
 
   /** The start time of this request in milliseconds */

Modified: incubator/solr/trunk/src/java/org/apache/solr/request/SolrQueryRequestBase.java
URL: http://svn.apache.org/viewvc/incubator/solr/trunk/src/java/org/apache/solr/request/SolrQueryRequestBase.java?rev=434136&r1=434135&r2=434136&view=diff
==============================================================================
--- incubator/solr/trunk/src/java/org/apache/solr/request/SolrQueryRequestBase.java (original)
+++ incubator/solr/trunk/src/java/org/apache/solr/request/SolrQueryRequestBase.java Wed Aug 23 12:34:57 2006
@@ -37,107 +37,148 @@
  * @version $Id$
  */
 public abstract class SolrQueryRequestBase implements SolrQueryRequest {
- // some standard query argument names
- public static final String QUERY_NAME="q";
- public static final String START_NAME="start";
- public static final String ROWS_NAME="rows";
- public static final String XSL_NAME="xsl";
- public static final String QUERYTYPE_NAME="qt";
-
-
- protected final SolrCore core;
-
- public SolrQueryRequestBase(SolrCore core) {
-   this.core=core;
- }
-
- public int getIntParam(String name) {
-   String s = getParam(name);
-   if (s==null) {
-     throw new SolrException(500,"Missing required parameter '"+name+"' from " + this);
-   }
-   return Integer.parseInt(s);
- }
-
- public int getIntParam(String name, int defval) {
-   String s = getParam(name);
-   return s==null ? defval : Integer.parseInt(s);
- }
-
- public String getStrParam(String name) {
-   String s = getParam(name);
-   if (s==null) {
-     throw new SolrException(500,"Missing required parameter '"+name+"' from " + this);
-   }
-   return s;
- }
-
- public String getStrParam(String name, String defval) {
-   String s = getParam(name);
-   return s==null ? defval : s;
- }
-
- public String getQueryString() {
-   return getParam(QUERY_NAME);
- }
-
- public String getQueryType() {
-   return getParam(QUERYTYPE_NAME);
- }
-
- // starting position in matches to return to client
- public int getStart() {
-   return getIntParam(START_NAME, 0);
- }
-
- // number of matching documents to return
- public int getLimit() {
-   return getIntParam(ROWS_NAME, 10);
- }
-
-
- protected final long startTime=System.currentTimeMillis();
- // Get the start time of this request in milliseconds
- public long getStartTime() {
-   return startTime;
- }
-
- // The index searcher associated with this request
- protected RefCounted<SolrIndexSearcher> searcherHolder;
- public SolrIndexSearcher getSearcher() {
-   // should this reach out and get a searcher from the core singleton, or
-   // should the core populate one in a factory method to create requests?
-   // or there could be a setSearcher() method that Solr calls
-
-   if (searcherHolder==null) {
-     searcherHolder = core.getSearcher();
-   }
-
-   return searcherHolder.get();
- }
-
- // The solr core (coordinator, etc) associated with this request
- public SolrCore getCore() {
-   return core;
- }
-
- // The index schema associated with this request
- public IndexSchema getSchema() {
-   return core.getSchema();
- }
-
- /**
-  * Frees resources associated with this request, this method <b>must</b>
-  * be called when the object is no longer in use.
-  */
- public void close() {
-   if (searcherHolder!=null) {
-     searcherHolder.decref();
-   }
- }
-
- public String toString() {
-   return this.getClass().getSimpleName() + '{' + getParamString() + '}';
- }
+  @Deprecated
+  public static final String QUERY_NAME="q";
+  @Deprecated
+  public static final String START_NAME="start";
+  @Deprecated
+  public static final String ROWS_NAME="rows";
+  @Deprecated
+  public static final String XSL_NAME="xsl";
+  @Deprecated
+  public static final String QUERYTYPE_NAME="qt";
+
+  protected final SolrCore core;
+  protected final SolrParams origParams;
+  protected SolrParams params;
+
+  public SolrQueryRequestBase(SolrCore core, SolrParams params) {
+    this.core = core;
+    this.params = this.origParams = params;
+  }
+
+  /** returns the current request parameters */
+  public SolrParams getParams() {
+    return params;
+  }
+
+  /** Returns the original request parameters.  As this
+   * does not normally include configured defaults
+   * it's more suitable for logging.
+   */
+  public SolrParams getOriginalParams() {
+    return origParams;
+  }
+
+  /** Change the parameters for this request.  This does not affect
+   *  the original parameters returned by getOriginalParams()
+   */
+  public void setParams(SolrParams params) {
+    this.params = params;
+  }
+
+  public String getParam(String name) {
+    return params.get(name);
+  }
+
+  public String[] getParams(String name) {
+    return params.getParams(name);
+  }
+
+
+  public int getIntParam(String name) {
+    String s = getParam(name);
+    if (s==null) {
+      throw new SolrException(500,"Missing required parameter '"+name+"' from " + this);
+    }
+    return Integer.parseInt(s);
+  }
+
+  public int getIntParam(String name, int defval) {
+    String s = getParam(name);
+    return s==null ? defval : Integer.parseInt(s);
+  }
+
+  public String getStrParam(String name) {
+    String s = getParam(name);
+    if (s==null) {
+      throw new SolrException(500,"Missing required parameter '"+name+"' from " + this);
+    }
+    return s;
+  }
+
+  public String getStrParam(String name, String defval) {
+    String s = getParam(name);
+    return s==null ? defval : s;
+  }
+
+  public String getQueryString() {
+    return params.get(SolrParams.Q);
+  }
+
+  public String getQueryType() {
+    return params.get(SolrParams.QT);
+  }
+
+  // starting position in matches to return to client
+  public int getStart() {
+    return params.getInt(SolrParams.START, 0);
+  }
+
+  // number of matching documents to return
+  @Deprecated
+  public int getLimit() {
+    return params.getInt(SolrParams.ROWS, 10);
+  }
+
+
+  protected final long startTime=System.currentTimeMillis();
+  // Get the start time of this request in milliseconds
+  public long getStartTime() {
+    return startTime;
+  }
+
+  // The index searcher associated with this request
+  protected RefCounted<SolrIndexSearcher> searcherHolder;
+  public SolrIndexSearcher getSearcher() {
+    // should this reach out and get a searcher from the core singleton, or
+    // should the core populate one in a factory method to create requests?
+    // or there could be a setSearcher() method that Solr calls
+
+    if (searcherHolder==null) {
+      searcherHolder = core.getSearcher();
+    }
+
+    return searcherHolder.get();
+  }
+
+  // The solr core (coordinator, etc) associated with this request
+  public SolrCore getCore() {
+    return core;
+  }
+
+  // The index schema associated with this request
+  public IndexSchema getSchema() {
+    return core.getSchema();
+  }
+
+  /**
+   * Frees resources associated with this request, this method <b>must</b>
+   * be called when the object is no longer in use.
+   */
+  public void close() {
+    if (searcherHolder!=null) {
+      searcherHolder.decref();
+    }
+  }
+
+
+  public String getParamString() {
+    return origParams.toString();
+  }
+
+  public String toString() {
+    return this.getClass().getSimpleName() + '{' + params + '}';
+  }
 
 }

Modified: incubator/solr/trunk/src/java/org/apache/solr/request/StandardRequestHandler.java
URL: http://svn.apache.org/viewvc/incubator/solr/trunk/src/java/org/apache/solr/request/StandardRequestHandler.java?rev=434136&r1=434135&r2=434136&view=diff
==============================================================================
--- incubator/solr/trunk/src/java/org/apache/solr/request/StandardRequestHandler.java (original)
+++ incubator/solr/trunk/src/java/org/apache/solr/request/StandardRequestHandler.java Wed Aug 23 12:34:57 2006
@@ -24,11 +24,11 @@
 import org.apache.solr.util.StrUtils;
 import org.apache.solr.util.NamedList;
 import org.apache.solr.util.SolrPluginUtils;
-import org.apache.solr.util.CommonParams;
 import org.apache.solr.search.*;
 import org.apache.solr.core.SolrCore;
 import org.apache.solr.core.SolrInfoMBean;
 import org.apache.solr.core.SolrException;
+import static org.apache.solr.request.SolrParams.*;
 
 /**
  * @author yonik
@@ -57,31 +57,32 @@
   // acceptable every million requests or so?
   long numRequests;
   long numErrors;
+  SolrParams defaults;
 
   /** shorten the class references for utilities */
   private static class U extends SolrPluginUtils {
     /* :NOOP */
   }
-  /** parameters garnered from config file */
-  protected final CommonParams params = new CommonParams();
-
 
   public void init(NamedList args) {
-    params.setValues(args);
+    Object o = args.get("defaults");
+    if (o != null && o instanceof NamedList) {
+      defaults = SolrParams.toSolrParams((NamedList)o);
+    }
   }
 
   public void handleRequest(SolrQueryRequest req, SolrQueryResponse rsp) {
     numRequests++;
 
-    // TODO: test if lucene will accept an escaped ';', otherwise
-    // we need to un-escape them before we pass to QueryParser
     try {
-      String sreq = req.getQueryString();
-      String debug = U.getParam(req, params.DEBUG_QUERY, params.debugQuery);
-      String defaultField = U.getParam(req, params.DF, params.df);
+      U.setDefaults(req,defaults);
+      SolrParams p = req.getParams();
+      String sreq = p.get(Q);
+
+      String defaultField = p.get(DF);
 
       // find fieldnames to return (fieldlist)
-      String fl = U.getParam(req, params.FL, params.fl);
+      String fl = p.get(SolrParams.FL);
       int flags = 0; 
       if (fl != null) {
         flags |= U.setReturnFields(fl, rsp);
@@ -105,11 +106,11 @@
         }
       }
 
-      DocList results = req.getSearcher().getDocList(query, null, sort, req.getStart(), req.getLimit(), flags);
+      DocList results = req.getSearcher().getDocList(query, null, sort, p.getInt(START,0), p.getInt(ROWS,10), flags);
       rsp.add(null,results);
 
       try {
-        NamedList dbg = U.doStandardDebug(req, qs, query, results, params);
+        NamedList dbg = U.doStandardDebug(req, qs, query, results);
         if (null != dbg) 
           rsp.add("debug", dbg);
       } catch (Exception e) {
@@ -118,7 +119,7 @@
       }
 
       NamedList sumData = SolrPluginUtils.doStandardHighlighting(
-        results, query, req, params, new String[]{defaultField});
+        results, query, req, new String[]{defaultField});
       if(sumData != null)
         rsp.add("highlighting", sumData);
 

Modified: incubator/solr/trunk/src/java/org/apache/solr/util/AbstractSolrTestCase.java
URL: http://svn.apache.org/viewvc/incubator/solr/trunk/src/java/org/apache/solr/util/AbstractSolrTestCase.java?rev=434136&r1=434135&r2=434136&view=diff
==============================================================================
--- incubator/solr/trunk/src/java/org/apache/solr/util/AbstractSolrTestCase.java (original)
+++ incubator/solr/trunk/src/java/org/apache/solr/util/AbstractSolrTestCase.java Wed Aug 23 12:34:57 2006
@@ -233,7 +233,7 @@
    * Generates a SolrQueryRequest using the LocalRequestFactory
    * @see #lrf
    */
-  public SolrQueryRequest req( String q ) {
+  public SolrQueryRequest req(String... q) {
     return lrf.makeRequest(q);
   }
     

Modified: incubator/solr/trunk/src/java/org/apache/solr/util/CommonParams.java
URL: http://svn.apache.org/viewvc/incubator/solr/trunk/src/java/org/apache/solr/util/CommonParams.java?rev=434136&r1=434135&r2=434136&view=diff
==============================================================================
--- incubator/solr/trunk/src/java/org/apache/solr/util/CommonParams.java (original)
+++ incubator/solr/trunk/src/java/org/apache/solr/util/CommonParams.java Wed Aug 23 12:34:57 2006
@@ -17,175 +17,168 @@
 package org.apache.solr.util;
 
 import org.apache.solr.core.SolrCore;
-import org.apache.solr.core.SolrInfoMBean;
-import org.apache.solr.core.SolrException;
 
-import org.apache.solr.util.StrUtils;
 import org.apache.solr.util.NamedList;
+import org.apache.solr.request.SolrParams;
 
-import java.util.logging.Logger;
-import java.util.logging.Level;
-import java.util.logging.Handler;
-
-import java.util.Arrays;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Collection;
-import java.util.Set;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.HashMap;
-import java.util.regex.Pattern;
-import java.io.IOException;
 
-    
+
+/**
+ * A collection on common params, both for Plugin initialization and
+ * for Requests.
+ */
+@Deprecated
+public class CommonParams {
+
+  @Deprecated
+  public static String FL = "fl";
+  /** default query field */
+  @Deprecated
+  public static String DF = "df";
+  /** whether to include debug data */
+  @Deprecated
+  public static String DEBUG_QUERY = "debugQuery";
+  /** another query to explain against */
+  @Deprecated
+  public static String EXPLAIN_OTHER = "explainOther";
+  /** wether to highlight */
+  @Deprecated
+  public static String HIGHLIGHT = "highlight";
+  /** fields to highlight */
+  @Deprecated
+  public static String HIGHLIGHT_FIELDS = "highlightFields";
+  /** maximum highlight fragments to return */
+  @Deprecated
+  public static String MAX_SNIPPETS = "maxSnippets";
+  /** override default highlight Formatter class */
+  @Deprecated
+  public static String HIGHLIGHT_FORMATTER_CLASS = "highlightFormatterClass";
+
+
+
+
+  /** the default field list to be used */
+  public String fl = null;
+  /** the default field to query */
+  public String df = null;
+  /** do not debug by default **/
+  public String debugQuery = null;
+  /** no default other explanation query **/
+  public String explainOther = null;
+  /** whether to highlight */
+  public boolean highlight = false;
+  /** fields to highlight */
+  public String highlightFields = null;
+  /** maximum highlight fragments to return */
+  public int maxSnippets = 1;
+  /** override default highlight Formatter class */
+  public String highlightFormatterClass = null;
+
+
+  public CommonParams() {
+    /* :NOOP: */
+  }
+
+  /** @see #setValues */
+  public CommonParams(NamedList args) {
+    this();
+    setValues(args);
+  }
 
   /**
-   * A collection on common params, both for Plugin initialization and
-   * for Requests.
+   * Sets the params using values from a NamedList, usefull in the
+   * init method for your handler.
+   *
+   * <p>
+   * If any param is not of the expected type, a severe error is
+   * logged,and the param is skipped.
+   * </p>
+   *
+   * <p>
+   * If any param is not of in the NamedList, it is skipped and the
+   * old value is left alone.
+   * </p>
+   *
    */
-  public class CommonParams {
+  public void setValues(NamedList args) {
+
+    Object tmp;
+
+    tmp = args.get(SolrParams.FL);
+    if (null != tmp) {
+      if (tmp instanceof String) {
+        fl = tmp.toString();
+      } else {
+        SolrCore.log.severe("init param is not a str: " + SolrParams.FL);
+      }
+    }
+
+    tmp = args.get(SolrParams.DF);
+    if (null != tmp) {
+      if (tmp instanceof String) {
+        df = tmp.toString();
+      } else {
+        SolrCore.log.severe("init param is not a str: " + SolrParams.DF);
+      }
+    }
 
-    /** query and init param for field list */
-    public static String FL = "fl";
-    /** default query field */
-    public static String DF = "df";
-    /** whether to include debug data */
-    public static String DEBUG_QUERY = "debugQuery";
-    /** another query to explain against */
-    public static String EXPLAIN_OTHER = "explainOther";
-    /** wether to highlight */
-    public static String HIGHLIGHT = "highlight";
-    /** fields to highlight */
-    public static String HIGHLIGHT_FIELDS = "highlightFields";
-    /** maximum highlight fragments to return */
-    public static String MAX_SNIPPETS = "maxSnippets";
-    /** override default highlight Formatter class */
-    public static String HIGHLIGHT_FORMATTER_CLASS = "highlightFormatterClass";
-
-
-    /** the default field list to be used */
-    public String fl = null;
-    /** the default field to query */
-    public String df = null;
-    /** do not debug by default **/
-    public String debugQuery = null;
-    /** no default other explanation query **/
-    public String explainOther = null;
-    /** whether to highlight */
-    public boolean highlight = false;
-    /** fields to highlight */
-    public String highlightFields = null;
-    /** maximum highlight fragments to return */
-    public int maxSnippets = 1;
-    /** override default highlight Formatter class */
-    public String highlightFormatterClass = null;
-
-
-    public CommonParams() {
-      /* :NOOP: */
-    }
-
-    /** @see #setValues */
-    public CommonParams(NamedList args) {
-      this();
-      setValues(args);
-    }
-
-    /**
-     * Sets the params using values from a NamedList, usefull in the
-     * init method for your handler.
-     *
-     * <p>
-     * If any param is not of the expected type, a severe error is
-     * logged,and the param is skipped.
-     * </p>
-     *
-     * <p>
-     * If any param is not of in the NamedList, it is skipped and the
-     * old value is left alone.
-     * </p>
-     *
-     */
-    public void setValues(NamedList args) {
-
-      Object tmp;
-        
-      tmp = args.get(FL);
-      if (null != tmp) {
-        if (tmp instanceof String) {
-          fl = tmp.toString();
-        } else {
-          SolrCore.log.severe("init param is not a str: " + FL);
-        }
-      }
-
-      tmp = args.get(DF);
-      if (null != tmp) {
-        if (tmp instanceof String) {
-          df = tmp.toString();
-        } else {
-          SolrCore.log.severe("init param is not a str: " + DF);
-        }
-      }
-
-      tmp = args.get(DEBUG_QUERY);
-      if (null != tmp) {
-        if (tmp instanceof String) {
-          debugQuery = tmp.toString();
-        } else {
-          SolrCore.log.severe("init param is not a str: " + DEBUG_QUERY);
-        }
-      }
-
-      tmp = args.get(EXPLAIN_OTHER);
-      if (null != tmp) {
-        if (tmp instanceof String) {
-          explainOther = tmp.toString();
-        } else {
-          SolrCore.log.severe("init param is not a str: " + EXPLAIN_OTHER);
-        }
-      }
-
-      tmp = args.get(HIGHLIGHT);
-      if (null != tmp) {
-        if (tmp instanceof String) {
-          // Any non-empty string other than 'false' implies highlighting
-          String val = tmp.toString().trim();
-          highlight = !(val.equals("") || val.equals("false"));
-        } else {
-          SolrCore.log.severe("init param is not a str: " + HIGHLIGHT);
-        }
-      }
-
-      tmp = args.get(HIGHLIGHT_FIELDS);
-      if (null != tmp) {
-        if (tmp instanceof String) {
-          highlightFields = tmp.toString();
-        } else {
-          SolrCore.log.severe("init param is not a str: " + HIGHLIGHT);
-        }
-      }
-
-      tmp = args.get(MAX_SNIPPETS);
-      if (null != tmp) {
-        if (tmp instanceof Integer) {
-          maxSnippets = ((Integer)tmp).intValue();
-        } else {
-          SolrCore.log.severe("init param is not an int: " + MAX_SNIPPETS);
-        }
-      }
-
-      tmp = args.get(HIGHLIGHT_FORMATTER_CLASS);
-      if (null != tmp) {
-        if (tmp instanceof String) {
-          highlightFormatterClass = tmp.toString();
-        } else {
-          SolrCore.log.severe("init param is not a str: " + HIGHLIGHT_FORMATTER_CLASS);
-        }
+    tmp = args.get(SolrParams.DEBUG_QUERY);
+    if (null != tmp) {
+      if (tmp instanceof String) {
+        debugQuery = tmp.toString();
+      } else {
+        SolrCore.log.severe("init param is not a str: " + SolrParams.DEBUG_QUERY);
+      }
+    }
+
+    tmp = args.get(SolrParams.EXPLAIN_OTHER);
+    if (null != tmp) {
+      if (tmp instanceof String) {
+        explainOther = tmp.toString();
+      } else {
+        SolrCore.log.severe("init param is not a str: " + SolrParams.EXPLAIN_OTHER);
+      }
+    }
+
+    tmp = args.get(SolrParams.HIGHLIGHT);
+    if (null != tmp) {
+      if (tmp instanceof String) {
+        // Any non-empty string other than 'false' implies highlighting
+        String val = tmp.toString().trim();
+        highlight = !(val.equals("") || val.equals("false"));
+      } else {
+        SolrCore.log.severe("init param is not a str: " + SolrParams.HIGHLIGHT);
+      }
+    }
+
+    tmp = args.get(SolrParams.HIGHLIGHT_FIELDS);
+    if (null != tmp) {
+      if (tmp instanceof String) {
+        highlightFields = tmp.toString();
+      } else {
+        SolrCore.log.severe("init param is not a str: " + SolrParams.HIGHLIGHT);
+      }
+    }
+
+    tmp = args.get(SolrParams.MAX_SNIPPETS);
+    if (null != tmp) {
+      if (tmp instanceof Integer) {
+        maxSnippets = ((Integer)tmp).intValue();
+      } else {
+        SolrCore.log.severe("init param is not an int: " + SolrParams.MAX_SNIPPETS);
+      }
+    }
+
+    tmp = args.get(SolrParams.HIGHLIGHT_FORMATTER_CLASS);
+    if (null != tmp) {
+      if (tmp instanceof String) {
+        highlightFormatterClass = tmp.toString();
+      } else {
+        SolrCore.log.severe("init param is not a str: " + SolrParams.HIGHLIGHT_FORMATTER_CLASS);
       }
-        
     }
 
   }
+
+}
 

Modified: incubator/solr/trunk/src/java/org/apache/solr/util/SolrPluginUtils.java
URL: http://svn.apache.org/viewvc/incubator/solr/trunk/src/java/org/apache/solr/util/SolrPluginUtils.java?rev=434136&r1=434135&r2=434136&view=diff
==============================================================================
--- incubator/solr/trunk/src/java/org/apache/solr/util/SolrPluginUtils.java (original)
+++ incubator/solr/trunk/src/java/org/apache/solr/util/SolrPluginUtils.java Wed Aug 23 12:34:57 2006
@@ -16,74 +16,32 @@
 
 package org.apache.solr.util;
 
-import org.apache.solr.core.Config; // highlighting
+import org.apache.lucene.analysis.Analyzer;
+import org.apache.lucene.analysis.Token;
+import org.apache.lucene.analysis.TokenFilter;
+import org.apache.lucene.analysis.TokenStream;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.queryParser.ParseException;
+import org.apache.lucene.queryParser.QueryParser;
+import org.apache.lucene.search.*;
+import org.apache.lucene.search.BooleanClause.Occur;
+import org.apache.lucene.search.highlight.Formatter;
+import org.apache.lucene.search.highlight.*;
+import org.apache.solr.core.Config;
 import org.apache.solr.core.SolrCore;
-import org.apache.solr.core.SolrInfoMBean;
 import org.apache.solr.core.SolrException;
-
-import org.apache.solr.search.SolrIndexSearcher;
-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.SolrCache;
-import org.apache.solr.search.SolrQueryParser;
-import org.apache.solr.search.QueryParsing;
-import org.apache.solr.search.CacheRegenerator;
-
-import org.apache.solr.request.StandardRequestHandler;
+import org.apache.solr.request.SolrParams;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.request.SolrQueryResponse;
-import org.apache.solr.request.SolrRequestHandler;
-
+import org.apache.solr.request.DefaultSolrParams;
 import org.apache.solr.schema.IndexSchema;
-import org.apache.solr.schema.FieldType;
-
-import org.apache.solr.util.StrUtils;
-import org.apache.solr.util.NamedList;
-import org.apache.solr.util.XML;
-
-import org.apache.lucene.document.Document;
-import org.apache.lucene.document.Field;
-import org.apache.lucene.index.Term;
-import org.apache.lucene.search.Query;
-import org.apache.lucene.search.TermQuery;
-import org.apache.lucene.search.DisjunctionMaxQuery;
-import org.apache.lucene.search.BooleanQuery;
-import org.apache.lucene.search.BooleanClause;
-import org.apache.lucene.search.BooleanClause.Occur;
-import org.apache.lucene.search.ConstantScoreRangeQuery;
-import org.apache.lucene.search.Sort;
-import org.apache.lucene.search.Explanation;
-import org.apache.lucene.search.highlight.Highlighter; // highlighting
-import org.apache.lucene.search.highlight.TokenSources;
-import org.apache.lucene.search.highlight.QueryScorer;
-import org.apache.lucene.search.highlight.Encoder;
-import org.apache.lucene.search.highlight.SimpleHTMLFormatter;
-import org.apache.lucene.search.highlight.Formatter;
-import org.apache.lucene.search.highlight.SimpleFragmenter;
-import org.apache.lucene.search.highlight.TextFragment;
-import org.apache.lucene.search.highlight.NullFragmenter;
-import org.apache.lucene.queryParser.QueryParser;
-import org.apache.lucene.queryParser.ParseException;
-import org.apache.lucene.analysis.Analyzer;
-import org.apache.lucene.analysis.TokenStream;
-import org.apache.lucene.analysis.TokenFilter;
-import org.apache.lucene.analysis.Token;
-
-
-import org.xmlpull.v1.XmlPullParserException;
+import org.apache.solr.search.*;
 
-import java.util.logging.Logger;
-import java.util.logging.Level;
-import java.util.logging.Handler;
-
-import java.util.*;
-import java.util.regex.Pattern;
 import java.io.IOException;
 import java.io.StringReader;
-import java.io.StringWriter; // highlighting
-import java.net.URL;
+import java.util.*;
+import java.util.logging.Level;
+import java.util.regex.Pattern;
     
 /**
  * <p>Utilities that may be of use to RequestHandlers.</p>
@@ -99,9 +57,21 @@
  * default parameter settings.  
  */
 public class SolrPluginUtils {
-    
+
+  /** set defaults on a SolrQueryRequest */
+  public static void setDefaults(SolrQueryRequest req, SolrParams defaults) {
+      SolrParams p = req.getParams();
+      if (defaults != null) {
+        p = new DefaultSolrParams(p,defaults);
+        // set params so they will be visible to other components such as the response writer
+        req.setParams(p);
+      }
+  }
+
+
   /** standard param for field list */
-  public static String FL = CommonParams.FL;
+  @Deprecated
+  public static String FL = SolrParams.FL;
 
   /**
    * SolrIndexSearch.numDocs(Query,Query) freaks out if the filtering
@@ -243,7 +213,7 @@
                                           CommonParams params)
     throws IOException {
         
-    String debug = getParam(req, params.DEBUG_QUERY, params.debugQuery);
+    String debug = getParam(req, SolrParams.DEBUG_QUERY, params.debugQuery);
 
     NamedList dbg = null;
     if (debug!=null) {
@@ -277,7 +247,85 @@
     return dbg;
   }
 
-    
+
+  /**
+   * <p>
+   * Returns a NamedList containing many "standard" pieces of debugging
+   * information.
+   * </p>
+   *
+   * <ul>
+   * <li>rawquerystring - the 'q' param exactly as specified by the client
+   * </li>
+   * <li>querystring - the 'q' param after any preprocessing done by the plugin
+   * </li>
+   * <li>parsedquery - the main query executed formated by the Solr
+   *     QueryParsing utils class (which knows about field types)
+   * </li>
+   * <li>parsedquery_toString - the main query executed formated by it's
+   *     own toString method (in case it has internal state Solr
+   *     doesn't know about)
+   * </li>
+   * <li>expain - the list of score explanations for each document in
+   *     results against query.
+   * </li>
+   * <li>otherQuery - the query string specified in 'explainOther' query param.
+   * </li>
+   * <li>explainOther - the list of score explanations for each document in
+   *     results against 'otherQuery'
+   * </li>
+   * </ul>
+   *
+   * @param req the request we are dealing with
+   * @param userQuery the users query as a string, after any basic
+   *                  preprocessing has been done
+   * @param query the query built from the userQuery
+   *              (and perhaps other clauses) that identifies the main
+   *              result set of the response.
+   * @param results the main result set of the response
+   */
+  public static NamedList doStandardDebug(SolrQueryRequest req,
+                                          String userQuery,
+                                          Query query,
+                                          DocList results)
+    throws IOException {
+
+    String debug = req.getParam(SolrParams.DEBUG_QUERY);
+
+    NamedList dbg = null;
+    if (debug!=null) {
+      dbg = new NamedList();
+
+      /* userQuery may have been pre-processes .. expose that */
+      dbg.add("rawquerystring", req.getQueryString());
+      dbg.add("querystring", userQuery);
+
+      /* QueryParsing.toString isn't perfect, use it to see converted
+       * values, use regular toString to see any attributes of the
+       * underlying Query it may have missed.
+       */
+      dbg.add("parsedquery",QueryParsing.toString(query, req.getSchema()));
+      dbg.add("parsedquery_toString", query.toString());
+
+      dbg.add("explain", getExplainList
+              (query, results, req.getSearcher(), req.getSchema()));
+      String otherQueryS = req.getParam("explainOther");
+      if (otherQueryS != null && otherQueryS.length() > 0) {
+        DocList otherResults = doSimpleQuery
+          (otherQueryS,req.getSearcher(), req.getSchema(),0,10);
+        dbg.add("otherQuery",otherQueryS);
+        dbg.add("explainOther", getExplainList
+                (query, otherResults,
+                 req.getSearcher(),
+                 req.getSchema()));
+      }
+    }
+
+    return dbg;
+  }
+
+
+
   /**
    * Generates an list of Explanations for each item in a list of docs.
    *
@@ -442,9 +490,9 @@
                                                  CommonParams params,
                                                  String[] defaultFields
                                                  ) throws IOException {
-    if(!getBooleanParam(req, params.HIGHLIGHT, params.highlight)) 
+    if(!getBooleanParam(req, SolrParams.HIGHLIGHT, params.highlight))
       return null;
-    String fieldParam = getParam(req, params.HIGHLIGHT_FIELDS, 
+    String fieldParam = getParam(req, SolrParams.HIGHLIGHT_FIELDS,
                                  params.highlightFields);
     String fields[];
     if(fieldParam == null || fieldParam.trim().equals("")) {
@@ -458,7 +506,7 @@
       fields = splitList.split(fieldParam.trim());
 
     Highlighter highlighter;
-    String formatterSpec = getParam(req, params.HIGHLIGHT_FORMATTER_CLASS,
+    String formatterSpec = getParam(req, SolrParams.HIGHLIGHT_FORMATTER_CLASS,
                                     params.highlightFormatterClass);
     if(formatterSpec == null || formatterSpec.equals("")) {
       highlighter = getDefaultHighlighter(query);
@@ -469,7 +517,7 @@
       highlighter.setTextFragmenter(new GapFragmenter());
     }
     
-    int numFragments = getNumberParam(req, params.MAX_SNIPPETS,
+    int numFragments = getNumberParam(req, SolrParams.MAX_SNIPPETS,
                                       params.maxSnippets).intValue();
 
     return getHighlights(
@@ -479,6 +527,61 @@
       highlighter,
       numFragments);
   }
+
+
+  /** TODO: API IN PROGRESS... SUBJECT TO CHANGE
+   * Perform highlighting of selected fields.
+   *
+   * @param docs query results
+   * @param query the (possibly re-written query)
+   * @param req associated SolrQueryRequest
+   * @param defaultFields default search field list
+   *
+   * @return NamedList containing summary data, or null if highlighting is
+   * disabled.
+   *
+   */
+  public static NamedList doStandardHighlighting(DocList docs,
+                                                 Query query,
+                                                 SolrQueryRequest req,
+                                                 String[] defaultFields
+                                                 ) throws IOException {
+    SolrParams p = req.getParams();
+    if (!p.getBool(SolrParams.HIGHLIGHT, false)) return null;
+    String fieldParam = p.get(SolrParams.HIGHLIGHT_FIELDS);
+    String fields[];
+    if(fieldParam == null || fieldParam.trim().equals("")) {
+      // use default search field if highlight fieldlist not specified.
+      if (defaultFields == null || defaultFields.length == 0 ||
+          defaultFields[0] == null) {
+        fields = new String[]{req.getSchema().getDefaultSearchFieldName()};
+      } else
+        fields = defaultFields;
+    } else
+      fields = splitList.split(fieldParam.trim());
+
+    Highlighter highlighter;
+    String formatterSpec = p.get(SolrParams.HIGHLIGHT_FORMATTER_CLASS);
+    if(formatterSpec == null) {
+      highlighter = getDefaultHighlighter(query);
+    } else {
+      highlighter = new Highlighter(
+        (Formatter)Config.newInstance(formatterSpec),
+        new QueryScorer(query));
+      highlighter.setTextFragmenter(new GapFragmenter());
+    }
+
+    int numFragments = p.getInt(SolrParams.MAX_SNIPPETS, 1);
+
+    return getHighlights(
+      docs,
+      fields,
+      req.getSearcher(),
+      highlighter,
+      numFragments);
+  }
+
+
 
   /**
    * Executes a basic query in lucene syntax

Modified: incubator/solr/trunk/src/java/org/apache/solr/util/TestHarness.java
URL: http://svn.apache.org/viewvc/incubator/solr/trunk/src/java/org/apache/solr/util/TestHarness.java?rev=434136&r1=434135&r2=434136&view=diff
==============================================================================
--- incubator/solr/trunk/src/java/org/apache/solr/util/TestHarness.java (original)
+++ incubator/solr/trunk/src/java/org/apache/solr/util/TestHarness.java Wed Aug 23 12:34:57 2006
@@ -387,9 +387,13 @@
     public Map<String,String> args = new HashMap<String,String>();
     public LocalRequestFactory() {
     }
-    public LocalSolrQueryRequest makeRequest(String q) {
+    public LocalSolrQueryRequest makeRequest(String ... q) {
+      if (q.length==1) {
       return new LocalSolrQueryRequest(TestHarness.this.getCore(),
-                                       q, qtype, start, limit, args);
+                                       q[0], qtype, start, limit, args);
+      }
+
+      return new LocalSolrQueryRequest(TestHarness.this.getCore(),new NamedList(Arrays.asList(q)));
     }
   }
 }

Modified: incubator/solr/trunk/src/test/org/apache/solr/BasicFunctionalityTest.java
URL: http://svn.apache.org/viewvc/incubator/solr/trunk/src/test/org/apache/solr/BasicFunctionalityTest.java?rev=434136&r1=434135&r2=434136&view=diff
==============================================================================
--- incubator/solr/trunk/src/test/org/apache/solr/BasicFunctionalityTest.java (original)
+++ incubator/solr/trunk/src/test/org/apache/solr/BasicFunctionalityTest.java Wed Aug 23 12:34:57 2006
@@ -245,6 +245,79 @@
     assertTrue(luf.isStoreOffsetWithTermVector() && luf.isStorePositionWithTermVector());
 
   }
+
+
+  public void testSolrParams() throws Exception {
+    NamedList nl = new NamedList();
+    nl.add("i",555);
+    nl.add("s","bbb");
+    nl.add("bt","true");
+    nl.add("bf","false");
+
+    Map<String,String> m = new HashMap<String,String>();
+    m.put("f.field1.i", "1000");
+    m.put("s", "BBB");
+    m.put("ss", "SSS");
+
+    LocalSolrQueryRequest req = new LocalSolrQueryRequest(null,nl);
+    SolrParams p = req.getParams();
+
+    assertEquals(p.get("i"), "555");
+    assertEquals(p.getInt("i").intValue(), 555);
+    assertEquals(p.getInt("i",5), 555);
+    assertEquals(p.getInt("iii",5), 5);
+    assertEquals(p.getFieldParam("field1","i"), "555");
+
+    req.setParams(new DefaultSolrParams(p, new MapSolrParams(m)));
+    p = req.getParams();
+    assertEquals(req.getOriginalParams().get("s"), "bbb");
+    assertEquals(p.get("i"), "555");
+    assertEquals(p.getInt("i").intValue(), 555);
+    assertEquals(p.getInt("i",5), 555);
+    assertEquals(p.getInt("iii",5), 5);
+
+    assertEquals(p.getFieldParam("field1","i"), "1000");
+    assertEquals(p.get("s"), "bbb");
+    assertEquals(p.get("ss"), "SSS");
+
+    assertEquals(!!p.getBool("bt"), !p.getBool("bf"));
+    assertEquals(p.getBool("foo",true), true);
+    assertEquals(p.getBool("foo",false), false);
+    assertEquals(!!p.getBool("bt"), !p.getBool("bf"));
+  }
+
+
+  public void testConfigDefaults() {
+    assertU(adoc("id", "42",
+                 "name", "Zapp Brannigan"));
+    assertU(adoc("id", "43",
+                 "title", "Democratic Order of Planets"));
+    assertU(adoc("id", "44",
+                 "name", "The Zapper"));
+    assertU(adoc("id", "45",
+                 "title", "25 star General"));
+    assertU(adoc("id", "46",
+                 "subject", "Defeated the pacifists of the Gandhi nebula"));
+    assertU(adoc("id", "47",
+                 "text", "line up and fly directly at the enemy death cannons, clogging them with wreckage!"));
+    assertU(commit());
+
+    assertQ("standard request handler returns all matches",
+            req("id:[42 TO 47]"),
+            "*[count(//doc)=6]"
+            );
+
+    assertQ("defaults handler returns fewer matches",
+            req("q", "id:[42 TO 47]",   "qt","defaults"),
+            "*[count(//doc)=4]"
+            );
+
+    assertQ("defaults handler includes highlighting",
+            req("q", "name:Zapp OR title:General",   "qt","defaults"),
+            "//lst[@name='highlighting']"
+            );
+
+  }
       
             
 

Modified: incubator/solr/trunk/src/test/test-files/solr/conf/solrconfig.xml
URL: http://svn.apache.org/viewvc/incubator/solr/trunk/src/test/test-files/solr/conf/solrconfig.xml?rev=434136&r1=434135&r2=434136&view=diff
==============================================================================
--- incubator/solr/trunk/src/test/test-files/solr/conf/solrconfig.xml (original)
+++ incubator/solr/trunk/src/test/test-files/solr/conf/solrconfig.xml Wed Aug 23 12:34:57 2006
@@ -197,6 +197,15 @@
 
   <requestHandler name="test" class="solr.tst.TestRequestHandler" />
 
+  <!-- test query parameter defaults --> 
+  <requestHandler name="defaults" class="solr.StandardRequestHandler">
+    <lst name="defaults">
+      <int name="rows">4</int>
+      <bool name="highlight">true</bool>
+      <str name="highlightFields">text,name,subject,title,whitetok</str>
+    </lst>
+  </requestHandler>
+
 
   <admin>
     <defaultQuery>solr</defaultQuery>

Modified: incubator/solr/trunk/src/webapp/src/org/apache/solr/servlet/SolrServletRequest.java
URL: http://svn.apache.org/viewvc/incubator/solr/trunk/src/webapp/src/org/apache/solr/servlet/SolrServletRequest.java?rev=434136&r1=434135&r2=434136&view=diff
==============================================================================
--- incubator/solr/trunk/src/webapp/src/org/apache/solr/servlet/SolrServletRequest.java (original)
+++ incubator/solr/trunk/src/webapp/src/org/apache/solr/servlet/SolrServletRequest.java Wed Aug 23 12:34:57 2006
@@ -1,6 +1,23 @@
+/**
+ * Copyright 2006 The Apache Software Foundation
+ *
+ * Licensed 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.servlet;
 
 import org.apache.solr.request.SolrQueryRequestBase;
+import org.apache.solr.request.ServletSolrParams;
 import org.apache.solr.core.SolrCore;
 import org.apache.solr.util.StrUtils;
 
@@ -13,49 +30,7 @@
  * @version $Id$
  */
 class SolrServletRequest extends SolrQueryRequestBase {
-
-  final HttpServletRequest req;
-
   public SolrServletRequest(SolrCore core, HttpServletRequest req) {
-    super(core);
-    this.req = req;
-  }
-
-  public String getParam(String name) {
-    return req.getParameter(name);
-  }
-
-  public String[] getParams(String name) {
-    return req.getParameterValues(name);
-  }
-
-
-  public String getParamString() {
-    StringBuilder sb = new StringBuilder(128);
-    try {
-      boolean first=true;
-
-      for (Map.Entry<String,String[]> entry : (Set<Map.Entry<String,String[]>>)req.getParameterMap().entrySet()) {
-        String key = entry.getKey();
-        String[] valarr = entry.getValue();
-
-        for (String val : valarr) {
-          if (!first) sb.append('&');
-          first=false;
-          sb.append(key);
-          sb.append('=');
-          StrUtils.partialURLEncodeVal(sb, val==null ? "" : val);
-        }
-      }
-    }
-    catch (Exception e) {
-      // should never happen... we only needed this because
-      // partialURLEncodeVal can throw an IOException, but it
-      // never will when adding to a StringBuilder.
-      throw new RuntimeException(e);
-    }
-
-    return sb.toString();
+    super(core, new ServletSolrParams(req));
   }
-
 }