You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by rm...@apache.org on 2011/01/13 03:09:56 UTC

svn commit: r1058390 [14/16] - in /lucene/dev/branches/bulkpostings: ./ dev-tools/ dev-tools/eclipse/ dev-tools/idea/ dev-tools/idea/.idea/ dev-tools/idea/.idea/libraries/ dev-tools/idea/lucene/ dev-tools/idea/lucene/contrib/ dev-tools/idea/lucene/cont...

Modified: lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/request/SimpleFacets.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/request/SimpleFacets.java?rev=1058390&r1=1058389&r2=1058390&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/request/SimpleFacets.java (original)
+++ lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/request/SimpleFacets.java Thu Jan 13 02:09:33 2011
@@ -71,7 +71,7 @@ public class SimpleFacets {
   protected SolrQueryRequest req;
   protected ResponseBuilder rb;
 
-  protected SimpleOrderedMap facetResponse;
+  protected SimpleOrderedMap<Object> facetResponse;
 
   // per-facet values
   SolrParams localParams; // localParams on this particular facet command
@@ -176,13 +176,13 @@ public class SimpleFacets {
    * @see FacetParams#FACET
    * @return a NamedList of Facet Count info or null
    */
-  public NamedList getFacetCounts() {
+  public NamedList<Object> getFacetCounts() {
 
     // if someone called this method, benefit of the doubt: assume true
     if (!params.getBool(FacetParams.FACET,true))
       return null;
 
-    facetResponse = new SimpleOrderedMap();
+    facetResponse = new SimpleOrderedMap<Object>();
     try {
       facetResponse.add("facet_queries", getFacetQueryCounts());
       facetResponse.add("facet_fields", getFacetFieldCounts());
@@ -197,9 +197,11 @@ public class SimpleFacets {
   }
 
   public void addException(String msg, Exception e) {
-    List exceptions = (List)facetResponse.get("exception");
+    @SuppressWarnings("unchecked") 
+      List<String> exceptions = (List<String>)facetResponse.get("exception");
+
     if (exceptions == null) {
-      exceptions = new ArrayList();
+      exceptions = new ArrayList<String>();
       facetResponse.add("exception", exceptions);
     }
 
@@ -213,9 +215,9 @@ public class SimpleFacets {
    *
    * @see FacetParams#FACET_QUERY
    */
-  public NamedList getFacetQueryCounts() throws IOException,ParseException {
+  public NamedList<Integer> getFacetQueryCounts() throws IOException,ParseException {
 
-    NamedList res = new SimpleOrderedMap();
+    NamedList<Integer> res = new SimpleOrderedMap<Integer>();
 
     /* Ignore CommonParams.DF - could have init param facet.query assuming
      * the schema default with query param DF intented to only affect Q.
@@ -247,10 +249,10 @@ public class SimpleFacets {
   }
 
 
-  public NamedList getTermCounts(String field) throws IOException {
+  public NamedList<Integer> getTermCounts(String field) throws IOException {
     int offset = params.getFieldInt(field, FacetParams.FACET_OFFSET, 0);
     int limit = params.getFieldInt(field, FacetParams.FACET_LIMIT, 100);
-    if (limit == 0) return new NamedList();
+    if (limit == 0) return new NamedList<Integer>();
     Integer mincount = params.getFieldInt(field, FacetParams.FACET_MINCOUNT);
     if (mincount==null) {
       Boolean zeros = params.getFieldBool(field, FacetParams.FACET_ZEROS);
@@ -264,7 +266,7 @@ public class SimpleFacets {
     String prefix = params.getFieldParam(field,FacetParams.FACET_PREFIX);
 
 
-    NamedList counts;
+    NamedList<Integer> counts;
     SchemaField sf = searcher.getSchema().getField(field);
     FieldType ft = sf.getType();
 
@@ -336,10 +338,10 @@ public class SimpleFacets {
    * @see #getFieldMissingCount
    * @see #getFacetTermEnumCounts
    */
-  public NamedList getFacetFieldCounts()
+  public NamedList<Object> getFacetFieldCounts()
           throws IOException, ParseException {
 
-    NamedList res = new SimpleOrderedMap();
+    NamedList<Object> res = new SimpleOrderedMap<Object>();
     String[] facetFs = params.getParams(FacetParams.FACET_FIELD);
     if (null != facetFs) {
       for (String f : facetFs) {
@@ -362,10 +364,10 @@ public class SimpleFacets {
   }
 
 
-  private NamedList getListedTermCounts(String field, String termList) throws IOException {
+  private NamedList<Integer> getListedTermCounts(String field, String termList) throws IOException {
     FieldType ft = searcher.getSchema().getFieldType(field);
     List<String> terms = StrUtils.splitSmart(termList, ",", true);
-    NamedList res = new NamedList();
+    NamedList<Integer> res = new NamedList<Integer>();
     Term t = new Term(field);
     for (String term : terms) {
       String internal = ft.toInternal(term);
@@ -395,7 +397,7 @@ public class SimpleFacets {
    * Use the Lucene FieldCache to get counts for each unique field value in <code>docs</code>.
    * The field must have at most one indexed token per document.
    */
-  public static NamedList getFieldCacheCounts(SolrIndexSearcher searcher, DocSet docs, String fieldName, int offset, int limit, int mincount, boolean missing, String sort, String prefix) throws IOException {
+  public static NamedList<Integer> getFieldCacheCounts(SolrIndexSearcher searcher, DocSet docs, String fieldName, int offset, int limit, int mincount, boolean missing, String sort, String prefix) throws IOException {
     // TODO: If the number of terms is high compared to docs.size(), and zeros==false,
     //  we should use an alternate strategy to avoid
     //  1) creating another huge int[] for the counts
@@ -410,7 +412,7 @@ public class SimpleFacets {
     // trying to pass all the various params around.
 
     FieldType ft = searcher.getSchema().getFieldType(fieldName);
-    NamedList res = new NamedList();
+    NamedList<Integer> res = new NamedList<Integer>();
 
     FieldCache.DocTermsIndex si = FieldCache.DEFAULT.getTermsIndex(searcher.getReader(), fieldName);
 
@@ -590,7 +592,7 @@ public class SimpleFacets {
    * @see FacetParams#FACET_ZEROS
    * @see FacetParams#FACET_MISSING
    */
-  public NamedList getFacetTermEnumCounts(SolrIndexSearcher searcher, DocSet docs, String field, int offset, int limit, int mincount, boolean missing, String sort, String prefix)
+  public NamedList<Integer> getFacetTermEnumCounts(SolrIndexSearcher searcher, DocSet docs, String field, int offset, int limit, int mincount, boolean missing, String sort, String prefix)
     throws IOException {
 
     /* :TODO: potential optimization...
@@ -616,7 +618,7 @@ public class SimpleFacets {
     boolean sortByCount = sort.equals("count") || sort.equals("true");
     final int maxsize = limit>=0 ? offset+limit : Integer.MAX_VALUE-1;
     final BoundedTreeSet<CountPair<BytesRef,Integer>> queue = sortByCount ? new BoundedTreeSet<CountPair<BytesRef,Integer>>(maxsize) : null;
-    final NamedList res = new NamedList();
+    final NamedList<Integer> res = new NamedList<Integer>();
 
     int min=mincount-1;  // the smallest value in the top 'N' values    
     int off=offset;
@@ -758,10 +760,10 @@ public class SimpleFacets {
    * @see FacetParams#FACET_DATE
    */
 
-  public NamedList getFacetDateCounts()
+  public NamedList<Object> getFacetDateCounts()
     throws IOException, ParseException {
 
-    final NamedList resOuter = new SimpleOrderedMap();
+    final NamedList<Object> resOuter = new SimpleOrderedMap<Object>();
     final String[] fields = params.getParams(FacetParams.FACET_DATE);
 
     if (null == fields || 0 == fields.length) return resOuter;
@@ -779,7 +781,7 @@ public class SimpleFacets {
     return resOuter;
   }
 
-  public void getFacetDateCounts(String dateFacet, NamedList resOuter)
+  public void getFacetDateCounts(String dateFacet, NamedList<Object> resOuter)
       throws IOException, ParseException {
 
     final IndexSchema schema = searcher.getSchema();
@@ -788,7 +790,7 @@ public class SimpleFacets {
     String f = facetValue;
 
 
-    final NamedList resInner = new SimpleOrderedMap();
+    final NamedList<Object> resInner = new SimpleOrderedMap<Object>();
     resOuter.add(key, resInner);
     final SchemaField sf = schema.getField(f);
     if (! (sf.getType() instanceof DateField)) {
@@ -930,8 +932,8 @@ public class SimpleFacets {
    * @see FacetParams#FACET_RANGE
    */
 
-  public NamedList getFacetRangeCounts() {
-    final NamedList resOuter = new SimpleOrderedMap();
+  public NamedList<Object> getFacetRangeCounts() {
+    final NamedList<Object> resOuter = new SimpleOrderedMap<Object>();
     final String[] fields = params.getParams(FacetParams.FACET_RANGE);
 
     if (null == fields || 0 == fields.length) return resOuter;
@@ -949,7 +951,7 @@ public class SimpleFacets {
     return resOuter;
   }
 
-  void getFacetRangeCounts(String facetRange, NamedList resOuter)
+  void getFacetRangeCounts(String facetRange, NamedList<Object> resOuter)
       throws IOException, ParseException {
 
     final IndexSchema schema = searcher.getSchema();
@@ -960,7 +962,7 @@ public class SimpleFacets {
     final SchemaField sf = schema.getField(f);
     final FieldType ft = sf.getType();
 
-    RangeEndpointCalculator calc = null;
+    RangeEndpointCalculator<?> calc = null;
 
     if (ft instanceof TrieField) {
       final TrieField trie = (TrieField)ft;
@@ -1007,8 +1009,8 @@ public class SimpleFacets {
      final RangeEndpointCalculator<T> calc) throws IOException {
     
     final String f = sf.getName();
-    final NamedList res = new SimpleOrderedMap();
-    final NamedList counts = new SimpleOrderedMap();
+    final NamedList<Object> res = new SimpleOrderedMap<Object>();
+    final NamedList<Integer> counts = new SimpleOrderedMap<Integer>();
     res.add("counts", counts);
 
     final T start = calc.getValue(required.getFieldParam(f,FacetParams.FACET_RANGE_START));
@@ -1158,8 +1160,9 @@ public class SimpleFacets {
       return key.hashCode() ^ val.hashCode();
     }
     public boolean equals(Object o) {
-      return (o instanceof CountPair)
-        && (0 == this.compareTo((CountPair<K,V>) o));
+      if (! (o instanceof CountPair)) return false;
+      CountPair<?,?> that = (CountPair<?,?>) o;
+      return (this.key.equals(that.key) && this.val.equals(that.val));
     }
     public int compareTo(CountPair<K,V> o) {
       int vc = o.val.compareTo(val);

Modified: lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/request/UnInvertedField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/request/UnInvertedField.java?rev=1058390&r1=1058389&r2=1058390&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/request/UnInvertedField.java (original)
+++ lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/request/UnInvertedField.java Thu Jan 13 02:09:33 2011
@@ -469,12 +469,12 @@ public class UnInvertedField {
 
 
 
-  public NamedList getCounts(SolrIndexSearcher searcher, DocSet baseDocs, int offset, int limit, Integer mincount, boolean missing, String sort, String prefix) throws IOException {
+  public NamedList<Integer> getCounts(SolrIndexSearcher searcher, DocSet baseDocs, int offset, int limit, Integer mincount, boolean missing, String sort, String prefix) throws IOException {
     use.incrementAndGet();
 
     FieldType ft = searcher.getSchema().getFieldType(field);
 
-    NamedList res = new NamedList();  // order is important
+    NamedList<Integer> res = new NamedList<Integer>();  // order is important
 
     DocSet docs = baseDocs;
     int baseSize = docs.size();
@@ -912,15 +912,15 @@ public class UnInvertedField {
   //////////////////////////// caching /////////////////////////////
   //////////////////////////////////////////////////////////////////
   public static UnInvertedField getUnInvertedField(String field, SolrIndexSearcher searcher) throws IOException {
-    SolrCache cache = searcher.getFieldValueCache();
+    SolrCache<String,UnInvertedField> cache = searcher.getFieldValueCache();
     if (cache == null) {
       return new UnInvertedField(field, searcher);
     }
 
-    UnInvertedField uif = (UnInvertedField)cache.get(field);
+    UnInvertedField uif = cache.get(field);
     if (uif == null) {
       synchronized (cache) {
-        uif = (UnInvertedField)cache.get(field);
+        uif = cache.get(field);
         if (uif == null) {
           uif = new UnInvertedField(field, searcher);
           cache.put(field, uif);
@@ -930,7 +930,6 @@ public class UnInvertedField {
 
     return uif;
   }
-
 }
 
 

Modified: lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/response/SolrQueryResponse.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/response/SolrQueryResponse.java?rev=1058390&r1=1058389&r2=1058390&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/response/SolrQueryResponse.java (original)
+++ lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/response/SolrQueryResponse.java Thu Jan 13 02:09:33 2011
@@ -66,12 +66,12 @@ public class SolrQueryResponse {
    * @see #setAllValues
    * @see <a href="#returnable_data">Note on Returnable Data</a>
    */
-  protected NamedList values = new SimpleOrderedMap();
+  protected NamedList<Object> values = new SimpleOrderedMap<Object>();
 
   /**
    * Container for storing information that should be logged by Solr before returning.
    */
-  protected NamedList toLog = new SimpleOrderedMap();
+  protected NamedList<Object> toLog = new SimpleOrderedMap<Object>();
 
   protected Set<String> defaultReturnFields;
 
@@ -103,7 +103,7 @@ public class SolrQueryResponse {
    * Sets data to be returned in this response
    * @see <a href="#returnable_data">Note on Returnable Data</a>
    */
-  public void setAllValues(NamedList nameValuePairs) {
+  public void setAllValues(NamedList<Object> nameValuePairs) {
     values=nameValuePairs;
   }
 
@@ -189,7 +189,8 @@ public class SolrQueryResponse {
   }
   
   /** Repsonse header to be logged */ 
-  public NamedList getResponseHeader() {
+  public NamedList<Object> getResponseHeader() {
+    @SuppressWarnings("unchecked")
 	  SimpleOrderedMap<Object> header = (SimpleOrderedMap<Object>) values.get("responseHeader");
 	  return header;
   }
@@ -207,7 +208,7 @@ public class SolrQueryResponse {
    * 
    * @return things to log
    */
-  public NamedList getToLog() {
+  public NamedList<Object> getToLog() {
 	  return toLog;
   }
   

Modified: lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/schema/BinaryField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/schema/BinaryField.java?rev=1058390&r1=1058389&r2=1058390&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/schema/BinaryField.java (original)
+++ lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/schema/BinaryField.java Thu Jan 13 02:09:33 2011
@@ -50,7 +50,7 @@ public class BinaryField extends FieldTy
     return  ByteBuffer.wrap(f.getBinaryValue(), f.getBinaryOffset(), f.getBinaryLength() ) ;
   }
 
-  public Field createField(SchemaField field, Object val, float boost) {
+  public Fieldable createField(SchemaField field, Object val, float boost) {
     if (val == null) return null;
     if (!field.stored()) {
       log.trace("Ignoring unstored binary field: " + field);

Modified: lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/schema/FieldType.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/schema/FieldType.java?rev=1058390&r1=1058389&r2=1058390&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/schema/FieldType.java (original)
+++ lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/schema/FieldType.java Thu Jan 13 02:09:33 2011
@@ -221,7 +221,7 @@ public abstract class FieldType extends 
    *
    *
    */
-  public Field createField(SchemaField field, String externalVal, float boost) {
+  public Fieldable createField(SchemaField field, String externalVal, float boost) {
     if (!field.indexed() && !field.stored()) {
       if (log.isTraceEnabled())
         log.trace("Ignoring unindexed/unstored field: " + field);
@@ -252,9 +252,9 @@ public abstract class FieldType extends 
    * @param omitNorms true if norms should be omitted
    * @param omitTFPos true if term freq and position should be omitted.
    * @param boost The boost value
-   * @return the {@link org.apache.lucene.document.Field}.
+   * @return the {@link org.apache.lucene.document.Fieldable}.
    */
-  protected Field createField(String name, String val, Field.Store storage, Field.Index index,
+  protected Fieldable createField(String name, String val, Field.Store storage, Field.Index index,
                                     Field.TermVector vec, boolean omitNorms, boolean omitTFPos, float boost){
     Field f = new Field(name,
                         val,
@@ -278,7 +278,7 @@ public abstract class FieldType extends 
    * @see #isPolyField()
    */
   public Fieldable[] createFields(SchemaField field, String externalVal, float boost) {
-    Field f = createField( field, externalVal, boost);
+    Fieldable f = createField( field, externalVal, boost);
     return f==null ? new Fieldable[]{} : new Fieldable[]{f};
   }
 
@@ -341,7 +341,7 @@ public abstract class FieldType extends 
   public Object toObject(SchemaField sf, BytesRef term) {
     CharArr ext = new CharArr(term.length);
     indexedToReadable(term, ext);
-    Field f = createField(sf, ext.toString(), 1.0f);
+    Fieldable f = createField(sf, ext.toString(), 1.0f);
     return toObject(f);
   }
 
@@ -511,7 +511,6 @@ public abstract class FieldType extends 
    * @param maxInclusive whether the maximum of the range is inclusive or not
    *  @return a Query instance to perform range search according to given parameters
    *
-   * @see org.apache.solr.search.SolrQueryParser#getRangeQuery(String, String, String, boolean)
    */
   public Query getRangeQuery(QParser parser, SchemaField field, String part1, String part2, boolean minInclusive, boolean maxInclusive) {
     // constant score mode is now enabled per default

Modified: lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/schema/IndexSchema.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/schema/IndexSchema.java?rev=1058390&r1=1058389&r2=1058390&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/schema/IndexSchema.java (original)
+++ lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/schema/IndexSchema.java Thu Jan 13 02:09:33 2011
@@ -92,17 +92,7 @@ public final class IndexSchema {
    */
   private Map<SchemaField, Integer> copyFieldTargetCounts
     = new HashMap<SchemaField, Integer>();
-  /**
-   * Constructs a schema using the specified file name using the normal
-   * Config path directory searching rules.
-   *
-   * @see Config#openResource
-   * @deprecated Use {@link #IndexSchema(SolrConfig, String, InputStream)} instead.
-   */
-  @Deprecated
-  public IndexSchema(SolrConfig solrConfig, String name) {
-    this(solrConfig, name, null);
-  }
+
     /**
    * Constructs a schema using the specified resource name and stream.
    * If the is stream is null, the resource loader will load the schema resource by name.
@@ -128,14 +118,6 @@ public final class IndexSchema {
     }
     loader.inform( loader );
   }
-
-  /**
-   * @deprecated -- get access to SolrConfig some other way...
-   */
-  @Deprecated
-  public SolrConfig getSolrConfig() {
-    return solrConfig;
-  }
   
   /**
    * @since solr 1.4
@@ -158,31 +140,7 @@ public final class IndexSchema {
   float getVersion() {
     return version;
   }
-  
-  /**
-   * Direct access to the InputStream for the schemaFile used by this instance.
-   * @see Config#openResource
-   * @deprecated Use {@link #getSolrConfig()} and open a resource input stream
-   *             for {@link #getResourceName()} instead.
-   */
-  @Deprecated
-  public InputStream getInputStream() {
-    return loader.openResource(resourceName);
-  }
 
-  /** Gets the name of the schema file.
-   * @deprecated Use {@link #getResourceName()} instead.
-   */
-  @Deprecated
-  public String getSchemaFile() {
-    return resourceName;
-  }
-
-  /** The Name of this schema (as specified in the schema file)
-   * @deprecated Use {@link #getSchemaName()} instead.
-   */
-  @Deprecated
-  public String getName() { return name; }
 
   /**
    * Provides direct access to the Map containing all explicit
@@ -265,24 +223,6 @@ public final class IndexSchema {
    */
   public Analyzer getQueryAnalyzer() { return queryAnalyzer; }
 
-
-
-  /**
-   * A SolrQueryParser linked to this IndexSchema for field datatype
-   * information, and populated with default options from the
-   * &lt;solrQueryParser&gt; configuration for this IndexSchema.
-   *
-   * @param defaultField if non-null overrides the schema default
-   * @deprecated
-   */
-  @Deprecated
-  public SolrQueryParser getSolrQueryParser(String defaultField) {
-    SolrQueryParser qp = new SolrQueryParser(this,defaultField);
-    String operator = getQueryParserDefaultOperator();
-    qp.setDefaultOperator("AND".equals(operator) ?
-                          QueryParser.Operator.AND : QueryParser.Operator.OR);
-    return qp;
-  }
   
   /**
    * Name of the default search field specified in the schema file
@@ -293,9 +233,7 @@ public final class IndexSchema {
 
   /**
    * default operator ("AND" or "OR") for QueryParser
-   * @deprecated use getSolrQueryParser().getDefaultOperator()
    */
-  @Deprecated
   public String getQueryParserDefaultOperator() {
     return queryParserDefaultOperator;
   }
@@ -1278,37 +1216,6 @@ public final class IndexSchema {
     }
     return sf.toArray(new SchemaField[sf.size()]);
   }
-  /**
-   * Get all copy fields, both the static and the dynamic ones.
-   * 
-   * @param sourceField
-   * @return Array of fields to copy to.
-   * @deprecated Use {@link #getCopyFieldsList(String)} instead.
-   */
-  @Deprecated
-  public SchemaField[] getCopyFields(String sourceField) {
-    // This is the List that holds all the results, dynamic or not.
-    List<SchemaField> matchCopyFields = new ArrayList<SchemaField>();
-
-    // Get the dynamic results into the list.
-    for(DynamicCopy dynamicCopy : dynamicCopyFields) {
-      if(dynamicCopy.matches(sourceField)) {
-        matchCopyFields.add(dynamicCopy.getTargetField(sourceField));
-      }
-    }
-
-    // Get the fixed ones, if there are any and add them.
-    final List<CopyField> copyFields = copyFieldsMap.get(sourceField);
-    if (copyFields!=null) {
-      final Iterator<CopyField> it = copyFields.iterator();
-      while (it.hasNext()) {
-        matchCopyFields.add(it.next().getDestination());
-      }
-    }
-
-    // Construct the results by transforming the list into an array.
-    return matchCopyFields.toArray(new SchemaField[matchCopyFields.size()]);
-  }
 
   /**
    * Get all copy fields for a specified source field, both static

Modified: lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/schema/LatLonType.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/schema/LatLonType.java?rev=1058390&r1=1058389&r2=1058390&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/schema/LatLonType.java (original)
+++ lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/schema/LatLonType.java Thu Jan 13 02:09:33 2011
@@ -19,6 +19,7 @@ package org.apache.solr.schema;
 import org.apache.lucene.document.Field;
 import org.apache.lucene.document.Fieldable;
 import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexReader.AtomicReaderContext;
 import org.apache.lucene.index.MultiFields;
 import org.apache.lucene.search.*;
 import org.apache.lucene.spatial.DistanceUtils;
@@ -27,7 +28,6 @@ import org.apache.lucene.util.Bits;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.response.TextResponseWriter;
 import org.apache.solr.search.QParser;
-import org.apache.solr.search.SolrIndexReader;
 import org.apache.solr.search.SpatialOptions;
 import org.apache.solr.search.function.DocValues;
 import org.apache.solr.search.function.ValueSource;
@@ -281,7 +281,7 @@ public class LatLonType extends Abstract
   //It never makes sense to create a single field, so make it impossible to happen
 
   @Override
-  public Field createField(SchemaField field, String externalVal, float boost) {
+  public Fieldable createField(SchemaField field, String externalVal, float boost) {
     throw new UnsupportedOperationException("LatLonType uses multiple fields.  field=" + field.getName());
   }
 
@@ -334,16 +334,16 @@ class SpatialDistanceQuery extends Query
   public void extractTerms(Set terms) {}
 
   protected class SpatialWeight extends Weight {
-    protected Searcher searcher;
+    protected IndexSearcher searcher;
     protected float queryNorm;
     protected float queryWeight;
     protected Map latContext;
     protected Map lonContext;
 
-    public SpatialWeight(Searcher searcher) throws IOException {
+    public SpatialWeight(IndexSearcher searcher) throws IOException {
       this.searcher = searcher;
-      this.latContext = latSource.newContext();
-      this.lonContext = lonSource.newContext();
+      this.latContext = latSource.newContext(searcher);
+      this.lonContext = lonSource.newContext(searcher);
       latSource.createWeight(latContext, searcher);
       lonSource.createWeight(lonContext, searcher);
     }
@@ -371,18 +371,13 @@ class SpatialDistanceQuery extends Query
     }
 
     @Override
-    public Scorer scorer(IndexReader reader, boolean scoreDocsInOrder, boolean topScorer) throws IOException {
-      return new SpatialScorer(getSimilarity(searcher), reader, this);
+    public Scorer scorer(AtomicReaderContext context, boolean scoreDocsInOrder, boolean topScorer) throws IOException {
+      return new SpatialScorer(getSimilarity(searcher), context.reader, this);
     }
 
     @Override
-    public Explanation explain(IndexReader reader, int doc) throws IOException {
-      SolrIndexReader topReader = (SolrIndexReader)reader;
-      SolrIndexReader[] subReaders = topReader.getLeafReaders();
-      int[] offsets = topReader.getLeafOffsets();
-      int readerPos = SolrIndexReader.readerIndex(doc, offsets);
-      int readerBase = offsets[readerPos];
-      return ((SpatialScorer)scorer(subReaders[readerPos], true, true)).explain(doc-readerBase);
+    public Explanation explain(AtomicReaderContext context, int doc) throws IOException {
+      return ((SpatialScorer)scorer(context, true, true)).explain(doc);
     }
   }
 
@@ -535,7 +530,7 @@ class SpatialDistanceQuery extends Query
 
 
   @Override
-  public Weight createWeight(Searcher searcher) throws IOException {
+  public Weight createWeight(IndexSearcher searcher) throws IOException {
     return new SpatialWeight(searcher);
   }
 

Modified: lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/schema/PointType.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/schema/PointType.java?rev=1058390&r1=1058389&r2=1058390&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/schema/PointType.java (original)
+++ lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/schema/PointType.java Thu Jan 13 02:09:33 2011
@@ -112,7 +112,7 @@ public class PointType extends Coordinat
    *
    */
   @Override
-  public Field createField(SchemaField field, String externalVal, float boost) {
+  public Fieldable createField(SchemaField field, String externalVal, float boost) {
     throw new UnsupportedOperationException("PointType uses multiple fields.  field=" + field.getName());
   }
 

Modified: lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/schema/SchemaField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/schema/SchemaField.java?rev=1058390&r1=1058389&r2=1058390&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/schema/SchemaField.java (original)
+++ lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/schema/SchemaField.java Thu Jan 13 02:09:33 2011
@@ -89,7 +89,7 @@ public final class SchemaField extends F
   boolean isBinary() { return (properties & BINARY)!=0; }
 
 
-  public Field createField(String val, float boost) {
+  public Fieldable createField(String val, float boost) {
     return type.createField(this,val,boost);
   }
   

Modified: lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/schema/TrieDateField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/schema/TrieDateField.java?rev=1058390&r1=1058389&r2=1058390&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/schema/TrieDateField.java (original)
+++ lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/schema/TrieDateField.java Thu Jan 13 02:09:33 2011
@@ -160,7 +160,7 @@ public class TrieDateField extends DateF
   }
 
   @Override
-  public Field createField(SchemaField field, String externalVal, float boost) {
+  public Fieldable createField(SchemaField field, String externalVal, float boost) {
     boolean indexed = field.indexed();
     boolean stored = field.stored();
 

Modified: lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/schema/TrieField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/schema/TrieField.java?rev=1058390&r1=1058389&r2=1058390&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/schema/TrieField.java (original)
+++ lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/schema/TrieField.java Thu Jan 13 02:09:33 2011
@@ -477,7 +477,7 @@ public class TrieField extends FieldType
   }
 
   @Override
-  public Field createField(SchemaField field, String externalVal, float boost) {
+  public Fieldable createField(SchemaField field, String externalVal, float boost) {
     boolean indexed = field.indexed();
     boolean stored = field.stored();
 

Modified: lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/search/DocSet.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/search/DocSet.java?rev=1058390&r1=1058389&r2=1058390&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/search/DocSet.java (original)
+++ lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/search/DocSet.java Thu Jan 13 02:09:33 2011
@@ -23,6 +23,7 @@ import org.apache.lucene.search.DocIdSet
 import org.apache.lucene.search.Filter;
 import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexReader.AtomicReaderContext;
 
 import java.io.IOException;
 
@@ -89,10 +90,7 @@ public interface DocSet /* extends Colle
    *
    * @return
    * An OpenBitSet with the bit number of every docid set in the set.
-   * 
-   * @deprecated Use {@link #iterator()} to access all docs instead.
    */
-  @Deprecated
   public OpenBitSet getBits();
 
   /**
@@ -251,17 +249,15 @@ abstract class DocSetBase implements Doc
 
     return new Filter() {
       @Override
-      public DocIdSet getDocIdSet(IndexReader reader) throws IOException {
-        int offset = 0;
-        SolrIndexReader r = (SolrIndexReader)reader;
-        while (r.getParent() != null) {
-          offset += r.getBase();
-          r = r.getParent();
-        }
+      public DocIdSet getDocIdSet(AtomicReaderContext ctx) throws IOException {
+        IndexReader.AtomicReaderContext context = (IndexReader.AtomicReaderContext)ctx;  // TODO: remove after lucene migration
+        IndexReader reader = ctx.reader;
 
-        if (r==reader) return bs;
+        if (context.isTopLevel) {
+          return bs;
+        }
 
-        final int base = offset;
+        final int base = context.docBase;
         final int maxDoc = reader.maxDoc();
         final int max = base + maxDoc;   // one past the max doc in this segment.
 

Modified: lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/search/Grouping.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/search/Grouping.java?rev=1058390&r1=1058389&r2=1058390&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/search/Grouping.java (original)
+++ lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/search/Grouping.java Thu Jan 13 02:09:33 2011
@@ -151,7 +151,7 @@ public class Grouping {
     
     @Override
     void prepare() throws IOException {
-        Map context = ValueSource.newContext();
+        Map context = ValueSource.newContext(searcher);
         groupBy.createWeight(context, searcher);
     }
 

Modified: lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/search/LuceneQueryOptimizer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/search/LuceneQueryOptimizer.java?rev=1058390&r1=1058389&r2=1058390&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/search/LuceneQueryOptimizer.java (original)
+++ lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/search/LuceneQueryOptimizer.java Thu Jan 13 02:09:33 2011
@@ -54,7 +54,7 @@ class LuceneQueryOptimizer {
   }
 
   public TopDocs optimize(BooleanQuery original,
-                          Searcher searcher,
+                          IndexSearcher searcher,
                           int numHits,
                           Query[] queryOut,
                           Filter[] filterOut

Modified: lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/search/QParser.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/search/QParser.java?rev=1058390&r1=1058389&r2=1058390&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/search/QParser.java (original)
+++ lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/search/QParser.java Thu Jan 13 02:09:33 2011
@@ -63,10 +63,11 @@ public abstract class QParser {
     if (localParams != null) {
       String tagStr = localParams.get(CommonParams.TAG);
       if (tagStr != null) {
-        Map context = req.getContext();
-        Map<String,Collection<Object>> tagMap = (Map<String, Collection<Object>>)req.getContext().get("tags");
+        Map<Object,Object> context = req.getContext();
+        @SuppressWarnings("unchecked")
+        Map<Object,Collection<Object>> tagMap = (Map<Object, Collection<Object>>)req.getContext().get("tags");
         if (tagMap == null) {
-          tagMap = new HashMap<String,Collection<Object>>();
+          tagMap = new HashMap<Object,Collection<Object>>();
           context.put("tags", tagMap);          
         }
         if (tagStr.indexOf(',') >= 0) {
@@ -85,10 +86,10 @@ public abstract class QParser {
   }
 
 
-  private static void addTag(Map tagMap, Object key, Object val) {
-    Collection lst = (Collection)tagMap.get(key);
+  private static void addTag(Map<Object,Collection<Object>> tagMap, Object key, Object val) {
+    Collection<Object> lst = tagMap.get(key);
     if (lst == null) {
-      lst = new ArrayList(2);
+      lst = new ArrayList<Object>(2);
       tagMap.put(key, lst);
     }
     lst.add(val);

Modified: lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/search/QueryParsing.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/search/QueryParsing.java?rev=1058390&r1=1058389&r2=1058390&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/search/QueryParsing.java (original)
+++ lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/search/QueryParsing.java Thu Jan 13 02:09:33 2011
@@ -83,68 +83,6 @@ public class QueryParsing {
     if (null == val) val = sch.getQueryParserDefaultOperator();
     return "AND".equals(val) ? Operator.AND : Operator.OR;
   }
-   
-
-
-  /**
-   * Helper utility for parsing a query using the Lucene QueryParser syntax.
-   *
-   * @param qs     query expression in standard Lucene syntax
-   * @param schema used for default operator (overridden by params) and passed to the query parser for field format analysis information
-   */
-  public static Query parseQuery(String qs, IndexSchema schema) {
-    return parseQuery(qs, null, schema);
-  }
-
-  /**
-   * Helper utility for parsing a query using the Lucene QueryParser syntax.
-   *
-   * @param qs           query expression in standard Lucene syntax
-   * @param defaultField default field used for unqualified search terms in the query expression
-   * @param schema       used for default operator (overridden by params) and passed to the query parser for field format analysis information
-   */
-  public static Query parseQuery(String qs, String defaultField, IndexSchema schema) {
-    try {
-      Query query = schema.getSolrQueryParser(defaultField).parse(qs);
-
-      if (SolrCore.log.isTraceEnabled()) {
-        SolrCore.log.trace("After QueryParser:" + query);
-      }
-
-      return query;
-
-    } catch (ParseException e) {
-      SolrCore.log(e);
-      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Error parsing Lucene query", e);
-    }
-  }
-
-  /**
-   * Helper utility for parsing a query using the Lucene QueryParser syntax.
-   *
-   * @param qs           query expression in standard Lucene syntax
-   * @param defaultField default field used for unqualified search terms in the query expression
-   * @param params       used to determine the default operator, overriding the schema specified operator
-   * @param schema       used for default operator (overridden by params) and passed to the query parser for field format analysis information
-   */
-  public static Query parseQuery(String qs, String defaultField, SolrParams params, IndexSchema schema) {
-    try {
-      SolrQueryParser parser = schema.getSolrQueryParser(defaultField);
-      parser.setDefaultOperator(getQueryParserDefaultOperator
-                                (schema, params.get(QueryParsing.OP)));
-      Query query = parser.parse(qs);
-
-      if (SolrCore.log.isTraceEnabled()) {
-        SolrCore.log.trace("After QueryParser:" + query);
-      }
-
-      return query;
-
-    } catch (ParseException e) {
-      SolrCore.log(e);
-      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Query parsing error: " + e.getMessage(), e);
-    }
-  }
 
 
   // note to self: something needs to detect infinite recursion when parsing queries
@@ -907,45 +845,4 @@ public class QueryParsing {
     return out;
   }
 
-  /**
-   * Parse a function, returning a FunctionQuery
-   * <p/>
-   * <p>
-   * Syntax Examples....
-   * </p>
-   * <p/>
-   * <pre>
-   * // Numeric fields default to correct type
-   * // (ie: IntFieldSource or FloatFieldSource)
-   * // Others use explicit ord(...) to generate numeric field value
-   * myfield
-   * <p/>
-   * // OrdFieldSource
-   * ord(myfield)
-   * <p/>
-   * // ReverseOrdFieldSource
-   * rord(myfield)
-   * <p/>
-   * // LinearFloatFunction on numeric field value
-   * linear(myfield,1,2)
-   * <p/>
-   * // MaxFloatFunction of LinearFloatFunction on numeric field value or constant
-   * max(linear(myfield,1,2),100)
-   * <p/>
-   * // ReciprocalFloatFunction on numeric field value
-   * recip(myfield,1,2,3)
-   * <p/>
-   * // ReciprocalFloatFunction on ReverseOrdFieldSource
-   * recip(rord(myfield),1,2,3)
-   * <p/>
-   * // ReciprocalFloatFunction on LinearFloatFunction on ReverseOrdFieldSource
-   * recip(linear(rord(myfield),1,2),3,4,5)
-   * </pre>
-   */
-  public static FunctionQuery parseFunction(String func, IndexSchema schema) throws ParseException {
-    SolrCore core = SolrCore.getSolrCore();
-    return (FunctionQuery) (QParser.getParser(func, "func", new LocalSolrQueryRequest(core, new HashMap())).parse());
-    // return new FunctionQuery(parseValSource(new StrParser(func), schema));
-  }
-
 }

Modified: lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/search/SolrConstantScoreQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/search/SolrConstantScoreQuery.java?rev=1058390&r1=1058389&r2=1058390&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/search/SolrConstantScoreQuery.java (original)
+++ lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/search/SolrConstantScoreQuery.java Thu Jan 13 02:09:33 2011
@@ -2,6 +2,7 @@ package org.apache.solr.search;
 
 import org.apache.lucene.search.*;
 import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexReader.AtomicReaderContext;
 import org.apache.solr.search.function.ValueSource;
 import org.apache.solr.common.SolrException;
 
@@ -59,9 +60,9 @@ public class SolrConstantScoreQuery exte
     private float queryWeight;
     private Map context;
 
-    public ConstantWeight(Searcher searcher) throws IOException {
+    public ConstantWeight(IndexSearcher searcher) throws IOException {
       this.similarity = getSimilarity(searcher);
-      this.context = ValueSource.newContext();
+      this.context = ValueSource.newContext(searcher);
       if (filter instanceof SolrFilter)
         ((SolrFilter)filter).createWeight(context, searcher);
     }
@@ -89,14 +90,14 @@ public class SolrConstantScoreQuery exte
     }
 
     @Override
-    public Scorer scorer(IndexReader reader, boolean scoreDocsInOrder, boolean topScorer) throws IOException {
-      return new ConstantScorer(similarity, reader, this);
+    public Scorer scorer(AtomicReaderContext context, boolean scoreDocsInOrder, boolean topScorer) throws IOException {
+      return new ConstantScorer(similarity, context, this);
     }
 
     @Override
-    public Explanation explain(IndexReader reader, int doc) throws IOException {
+    public Explanation explain(AtomicReaderContext context, int doc) throws IOException {
 
-      ConstantScorer cs = new ConstantScorer(similarity, reader, this);
+      ConstantScorer cs = new ConstantScorer(similarity, context, this);
       boolean exists = cs.docIdSetIterator.advance(doc) == doc;
 
       ComplexExplanation result = new ComplexExplanation();
@@ -123,10 +124,10 @@ public class SolrConstantScoreQuery exte
     final float theScore;
     int doc = -1;
 
-    public ConstantScorer(Similarity similarity, IndexReader reader, ConstantWeight w) throws IOException {
+    public ConstantScorer(Similarity similarity, AtomicReaderContext context, ConstantWeight w) throws IOException {
       super(similarity);
       theScore = w.getValue();
-      DocIdSet docIdSet = filter instanceof SolrFilter ? ((SolrFilter)filter).getDocIdSet(w.context, reader) : filter.getDocIdSet(reader);
+      DocIdSet docIdSet = filter instanceof SolrFilter ? ((SolrFilter)filter).getDocIdSet(w.context, context) : filter.getDocIdSet(context);
       if (docIdSet == null) {
         docIdSetIterator = DocIdSet.EMPTY_DOCIDSET.iterator();
       } else {
@@ -161,7 +162,7 @@ public class SolrConstantScoreQuery exte
   }
 
   @Override
-  public Weight createWeight(Searcher searcher) {
+  public Weight createWeight(IndexSearcher searcher) {
     try {
       return new SolrConstantScoreQuery.ConstantWeight(searcher);
     } catch (IOException e) {

Modified: lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/search/SolrFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/search/SolrFilter.java?rev=1058390&r1=1058389&r2=1058390&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/search/SolrFilter.java (original)
+++ lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/search/SolrFilter.java Thu Jan 13 02:09:33 2011
@@ -18,9 +18,11 @@
 package org.apache.solr.search;
 
 import org.apache.lucene.search.Filter;
-import org.apache.lucene.search.Searcher;
+import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.DocIdSet;
 import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexReader.AtomicReaderContext;
+import org.apache.lucene.index.IndexReader.ReaderContext;
 
 import java.util.Map;
 import java.io.IOException;
@@ -35,12 +37,12 @@ public abstract class SolrFilter extends
 
   /** Implementations should propagate createWeight to sub-ValueSources which can store weight info in the context.
    * The context object will be passed to getDocIdSet() where this info can be retrieved. */
-  public abstract void createWeight(Map context, Searcher searcher) throws IOException;
+  public abstract void createWeight(Map context, IndexSearcher searcher) throws IOException;
   
-  public abstract DocIdSet getDocIdSet(Map context, IndexReader reader) throws IOException;
+  public abstract DocIdSet getDocIdSet(Map context, ReaderContext readerContext) throws IOException;
 
   @Override
-  public DocIdSet getDocIdSet(IndexReader reader) throws IOException {
-    return getDocIdSet(null, reader);
+  public DocIdSet getDocIdSet(AtomicReaderContext context) throws IOException {
+    return getDocIdSet(null, context);
   }
 }

Modified: lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/search/SolrIndexReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/search/SolrIndexReader.java?rev=1058390&r1=1058389&r2=1058390&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/search/SolrIndexReader.java (original)
+++ lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/search/SolrIndexReader.java Thu Jan 13 02:09:33 2011
@@ -19,12 +19,14 @@ package org.apache.solr.search;
 
 
 import org.apache.lucene.index.*;
+import org.apache.lucene.index.IndexReader.ReaderContext;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.LockObtainFailedException;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.FieldSelector;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.ReaderUtil;
 
 import java.io.IOException;
 import java.util.Collection;
@@ -40,6 +42,7 @@ public class SolrIndexReader extends Fil
   private int[] leafOffsets;
   private final SolrIndexReader parent;
   private final int base; // docid offset of this reader within parent
+  private final ReaderContext topLevelContext;
 
   private static int[] zeroIntArray = new int[]{0};
 
@@ -79,7 +82,7 @@ public class SolrIndexReader extends Fil
       leafReaders = new SolrIndexReader[]{this};
       leafOffsets = zeroIntArray;
     }
-
+    topLevelContext = ReaderUtil.buildReaderContext(this);
   }
 
   private SolrIndexReader[] getLeaves(int numLeaves) {
@@ -364,11 +367,6 @@ public class SolrIndexReader extends Fil
   }
 
   @Override
-  public int getSubReaderDocBase(IndexReader subReader) {
-    return in.getSubReaderDocBase(subReader);
-  }
-
-  @Override
   public int hashCode() {
     return in.hashCode();
   }
@@ -493,6 +491,11 @@ public class SolrIndexReader extends Fil
   public int getTermInfosIndexDivisor() {
     return in.getTermInfosIndexDivisor();
   }
+  
+  @Override
+  public ReaderContext getTopReaderContext() {
+    return topLevelContext;
+  }
 }
 
 

Modified: lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/search/SolrIndexSearcher.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/search/SolrIndexSearcher.java?rev=1058390&r1=1058389&r2=1058390&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/search/SolrIndexSearcher.java (original)
+++ lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/search/SolrIndexSearcher.java Thu Jan 13 02:09:33 2011
@@ -85,7 +85,7 @@ public class SolrIndexSearcher extends I
   private final SolrCache<Query,DocSet> filterCache;
   private final SolrCache<QueryResultKey,DocList> queryResultCache;
   private final SolrCache<Integer,Document> documentCache;
-  private final SolrCache<String,Object> fieldValueCache;
+  private final SolrCache<String,UnInvertedField> fieldValueCache;
 
   private final LuceneQueryOptimizer optimizer;
   
@@ -100,14 +100,6 @@ public class SolrIndexSearcher extends I
   private final Collection<String> fieldNames;
   private Collection<String> storedHighlightFieldNames;
 
-  /** Creates a searcher searching the index in the named directory.
-   * 
-   * @deprecated use alternate constructor
-   */
-  @Deprecated
-  public SolrIndexSearcher(SolrCore core, IndexSchema schema, String name, String path, boolean enableCache) throws IOException {
-    this(core, schema,name, core.getIndexReaderFactory().newReader(core.getDirectoryFactory().open(path), false), true, enableCache);
-  }
 
   /*
    * Creates a searcher searching the index in the provided directory. Note:
@@ -498,7 +490,7 @@ public class SolrIndexSearcher extends I
   ////////////////////////////////////////////////////////////////////////////////
 
   /** expert: internal API, subject to change */
-  public SolrCache getFieldValueCache() {
+  public SolrCache<String,UnInvertedField> getFieldValueCache() {
     return fieldValueCache;
   }
 
@@ -914,22 +906,6 @@ public class SolrIndexSearcher extends I
     return positive ? getDocSetNC(absQ,filter) : filter.andNot(getPositiveDocSet(absQ));
   }
 
-
-  /**
-  * Converts a filter into a DocSet.
-  * This method is not cache-aware and no caches are checked.
-  */
-  public DocSet convertFilter(Filter lfilter) throws IOException {
-    DocIdSet docSet = lfilter.getDocIdSet(this.reader);
-    OpenBitSet obs = new OpenBitSet();
-    DocIdSetIterator it = docSet.iterator();
-    int doc;
-    while((doc = it.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
-      obs.fastSet(doc);
-    }
-    return new BitDocSet(obs);
-  }
-
   /**
    * Returns documents matching both <code>query</code> and <code>filter</code>
    * and sorted by <code>sort</code>.
@@ -1783,8 +1759,8 @@ public class SolrIndexSearcher extends I
     return null;
   }
 
-  public NamedList getStatistics() {
-    NamedList lst = new SimpleOrderedMap();
+  public NamedList<Object> getStatistics() {
+    NamedList<Object> lst = new SimpleOrderedMap<Object>();
     lst.add("searcherName", name);
     lst.add("caching", cachingEnabled);
     lst.add("numDocs", reader.numDocs());

Modified: lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/search/SolrQueryParser.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/search/SolrQueryParser.java?rev=1058390&r1=1058389&r2=1058390&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/search/SolrQueryParser.java (original)
+++ lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/search/SolrQueryParser.java Thu Jan 13 02:09:33 2011
@@ -39,9 +39,6 @@ import org.apache.solr.schema.IndexSchem
 import org.apache.solr.schema.SchemaField;
 import org.apache.solr.schema.TextField;
 
-// TODO: implement the analysis of simple fields with
-// FieldType.toInternal() instead of going through the
-// analyzer.  Should lead to faster query parsing.
 
 /**
  * A variation on the Lucene QueryParser which knows about the field 
@@ -57,8 +54,6 @@ import org.apache.solr.schema.TextField;
  * If the magic field name "<code>_val_</code>" is used in a term or 
  * phrase query, the value is parsed as a function.
  * </p>
- *
- * @see QueryParsing#parseFunction
  */
 public class SolrQueryParser extends QueryParser {
   protected final IndexSchema schema;
@@ -67,32 +62,12 @@ public class SolrQueryParser extends Que
   protected final Map<String, ReversedWildcardFilterFactory> leadingWildcards =
     new HashMap<String, ReversedWildcardFilterFactory>();
 
-  /**
-   * Constructs a SolrQueryParser using the schema to understand the
-   * formats and datatypes of each field.  Only the defaultSearchField
-   * will be used from the IndexSchema (unless overridden),
-   * &lt;solrQueryParser&gt; will not be used.
-   * 
-   * @param schema Used for default search field name if defaultField is null and field information is used for analysis
-   * @param defaultField default field used for unspecified search terms.  if null, the schema default field is used
-   * @see IndexSchema#getDefaultSearchFieldName()
-   */
-  public SolrQueryParser(IndexSchema schema, String defaultField) {
-    super(schema.getSolrConfig().getLuceneVersion("luceneMatchVersion", Version.LUCENE_30), defaultField == null ? schema.getDefaultSearchFieldName() : defaultField, schema.getQueryAnalyzer());
-    this.schema = schema;
-    this.parser  = null;
-    this.defaultField = defaultField;
-    setLowercaseExpandedTerms(false);
-    setEnablePositionIncrements(true);
-    checkAllowLeadingWildcards();
-  }
-
   public SolrQueryParser(QParser parser, String defaultField) {
     this(parser, defaultField, parser.getReq().getSchema().getQueryAnalyzer());
   }
 
   public SolrQueryParser(QParser parser, String defaultField, Analyzer analyzer) {
-    super(parser.getReq().getSchema().getSolrConfig().getLuceneVersion("luceneMatchVersion", Version.LUCENE_30), defaultField, analyzer);
+    super(parser.getReq().getCore().getSolrConfig().getLuceneVersion("luceneMatchVersion", Version.LUCENE_30), defaultField, analyzer);
     this.schema = parser.getReq().getSchema();
     this.parser = parser;
     this.defaultField = defaultField;
@@ -138,12 +113,8 @@ public class SolrQueryParser extends Que
     // own functions.
     if (field.charAt(0) == '_') {
       if ("_val_".equals(field)) {
-        if (parser==null) {
-          return QueryParsing.parseFunction(queryText, schema);
-        } else {
-          QParser nested = parser.subQuery(queryText, "func");
-          return nested.getQuery();
-        }
+        QParser nested = parser.subQuery(queryText, "func");
+        return nested.getQuery();
       } else if ("_query_".equals(field) && parser != null) {
         return parser.subQuery(queryText, null).getQuery();
       }

Modified: lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/search/SortedIntDocSet.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/search/SortedIntDocSet.java?rev=1058390&r1=1058389&r2=1058390&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/search/SortedIntDocSet.java (original)
+++ lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/search/SortedIntDocSet.java Thu Jan 13 02:09:33 2011
@@ -22,6 +22,7 @@ import org.apache.lucene.search.DocIdSet
 import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.search.Filter;
 import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexReader.AtomicReaderContext;
 
 import java.io.IOException;
 
@@ -551,14 +552,11 @@ public class SortedIntDocSet extends Doc
       int lastEndIdx = 0;
 
       @Override
-      public DocIdSet getDocIdSet(IndexReader reader) throws IOException {
-        int offset = 0;
-        SolrIndexReader r = (SolrIndexReader)reader;
-        while (r.getParent() != null) {
-          offset += r.getBase();
-          r = r.getParent();
-        }
-        final int base = offset;
+      public DocIdSet getDocIdSet(AtomicReaderContext contextX) throws IOException {
+        AtomicReaderContext context = (AtomicReaderContext)contextX;  // TODO: remove after lucene migration
+        IndexReader reader = context.reader;
+
+        final int base = context.docBase;
         final int maxDoc = reader.maxDoc();
         final int max = base + maxDoc;   // one past the max doc in this segment.
         int sidx = Math.max(0,lastEndIdx);

Modified: lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/search/ValueSourceParser.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/search/ValueSourceParser.java?rev=1058390&r1=1058389&r2=1058390&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/search/ValueSourceParser.java (original)
+++ lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/search/ValueSourceParser.java Thu Jan 13 02:09:33 2011
@@ -20,7 +20,7 @@ import org.apache.lucene.index.IndexRead
 import org.apache.lucene.index.Term;
 import org.apache.lucene.queryParser.ParseException;
 import org.apache.lucene.search.Query;
-import org.apache.lucene.search.Searcher;
+import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.TermQuery;
 import org.apache.lucene.search.spell.JaroWinklerDistance;
 import org.apache.lucene.search.spell.LevensteinDistance;
@@ -83,7 +83,7 @@ public abstract class ValueSourceParser 
     addParser("ord", new ValueSourceParser() {
       public ValueSource parse(FunctionQParser fp) throws ParseException {
         String field = fp.parseId();
-        return new TopValueSource(new OrdFieldSource(field));
+        return new OrdFieldSource(field);
       }
     });
     addParser("literal", new ValueSourceParser() {
@@ -94,15 +94,14 @@ public abstract class ValueSourceParser 
     addParser("rord", new ValueSourceParser() {
       public ValueSource parse(FunctionQParser fp) throws ParseException {
         String field = fp.parseId();
-        return new TopValueSource(new ReverseOrdFieldSource(field));
+        return new ReverseOrdFieldSource(field);
       }
     });
     addParser("top", new ValueSourceParser() {
       public ValueSource parse(FunctionQParser fp) throws ParseException {
+        // top(vs) is now a no-op
         ValueSource source = fp.parseValueSource();
-        // nested top is redundant, and ord and rord get automatically wrapped
-        if (source instanceof TopValueSource) return source;
-        return new TopValueSource(source);
+        return source;
       }
     });
     addParser("linear", new ValueSourceParser() {
@@ -134,7 +133,7 @@ public abstract class ValueSourceParser 
         ValueSource source = fp.parseValueSource();
         float min = fp.parseFloat();
         float max = fp.parseFloat();
-        return new TopValueSource(new ScaleFloatFunction(source, min, max));
+        return new ScaleFloatFunction(source, min, max);
       }
     });
     addParser("div", new ValueSourceParser() {
@@ -889,9 +888,7 @@ abstract class Double2Parser extends Nam
     }
 
     @Override
-    public void createWeight(Map context, Searcher searcher) throws IOException {
-      a.createWeight(context,searcher);
-      b.createWeight(context,searcher);
+    public void createWeight(Map context, IndexSearcher searcher) throws IOException {
     }
 
     public int hashCode() {

Modified: lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/search/function/BoostedQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/search/function/BoostedQuery.java?rev=1058390&r1=1058389&r2=1058390&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/search/function/BoostedQuery.java (original)
+++ lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/search/function/BoostedQuery.java Thu Jan 13 02:09:33 2011
@@ -19,8 +19,8 @@ package org.apache.solr.search.function;
 
 import org.apache.lucene.search.*;
 import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexReader.AtomicReaderContext;
 import org.apache.lucene.util.ToStringUtils;
-import org.apache.solr.search.SolrIndexReader;
 
 import java.io.IOException;
 import java.util.Set;
@@ -53,20 +53,20 @@ public class BoostedQuery extends Query 
     q.extractTerms(terms);
   }
 
-  public Weight createWeight(Searcher searcher) throws IOException {
+  public Weight createWeight(IndexSearcher searcher) throws IOException {
     return new BoostedQuery.BoostedWeight(searcher);
   }
 
   private class BoostedWeight extends Weight {
-    Searcher searcher;
+    IndexSearcher searcher;
     Weight qWeight;
-    Map context;
+    Map fcontext;
 
-    public BoostedWeight(Searcher searcher) throws IOException {
+    public BoostedWeight(IndexSearcher searcher) throws IOException {
       this.searcher = searcher;
       this.qWeight = q.weight(searcher);
-      this.context = boostVal.newContext();
-      boostVal.createWeight(context,searcher);
+      this.fcontext = boostVal.newContext(searcher);
+      boostVal.createWeight(fcontext,searcher);
     }
 
     public Query getQuery() {
@@ -91,33 +91,26 @@ public class BoostedQuery extends Query 
     }
 
     @Override
-    public Scorer scorer(IndexReader reader, boolean scoreDocsInOrder, boolean topScorer) throws IOException {
-      Scorer subQueryScorer = qWeight.scorer(reader, true, false);
+    public Scorer scorer(AtomicReaderContext context, boolean scoreDocsInOrder, boolean topScorer) throws IOException {
+      Scorer subQueryScorer = qWeight.scorer(context, true, false);
       if(subQueryScorer == null) {
         return null;
       }
-      return new BoostedQuery.CustomScorer(getSimilarity(searcher), searcher, reader, this, subQueryScorer, boostVal);
+      return new BoostedQuery.CustomScorer(getSimilarity(searcher), searcher, context.reader, this, subQueryScorer, boostVal);
     }
 
     @Override
-    public Explanation explain(IndexReader reader, int doc) throws IOException {
-      SolrIndexReader topReader = (SolrIndexReader)reader;
-      SolrIndexReader[] subReaders = topReader.getLeafReaders();
-      int[] offsets = topReader.getLeafOffsets();
-      int readerPos = SolrIndexReader.readerIndex(doc, offsets);
-      int readerBase = offsets[readerPos];
-
-      Explanation subQueryExpl = qWeight.explain(reader,doc);
+    public Explanation explain(AtomicReaderContext readerContext, int doc) throws IOException {
+      Explanation subQueryExpl = qWeight.explain(readerContext,doc);
       if (!subQueryExpl.isMatch()) {
         return subQueryExpl;
       }
-
-      DocValues vals = boostVal.getValues(context, subReaders[readerPos]);
-      float sc = subQueryExpl.getValue() * vals.floatVal(doc-readerBase);
+      DocValues vals = boostVal.getValues(fcontext, readerContext.reader);
+      float sc = subQueryExpl.getValue() * vals.floatVal(doc);
       Explanation res = new ComplexExplanation(
         true, sc, BoostedQuery.this.toString() + ", product of:");
       res.addDetail(subQueryExpl);
-      res.addDetail(vals.explain(doc-readerBase));
+      res.addDetail(vals.explain(doc));
       return res;
     }
   }
@@ -129,9 +122,9 @@ public class BoostedQuery extends Query 
     private final Scorer scorer;
     private final DocValues vals;
     private final IndexReader reader;
-    private final Searcher searcher;
+    private final IndexSearcher searcher;
 
-    private CustomScorer(Similarity similarity, Searcher searcher, IndexReader reader, BoostedQuery.BoostedWeight w,
+    private CustomScorer(Similarity similarity, IndexSearcher searcher, IndexReader reader, BoostedQuery.BoostedWeight w,
         Scorer scorer, ValueSource vs) throws IOException {
       super(similarity);
       this.weight = w;
@@ -139,7 +132,7 @@ public class BoostedQuery extends Query 
       this.scorer = scorer;
       this.reader = reader;
       this.searcher = searcher; // for explain
-      this.vals = vs.getValues(weight.context, reader);
+      this.vals = vs.getValues(weight.fcontext, reader);
     }
 
     @Override
@@ -168,7 +161,7 @@ public class BoostedQuery extends Query 
     }
 
     public Explanation explain(int doc) throws IOException {
-      Explanation subQueryExpl = weight.qWeight.explain(reader,doc);
+      Explanation subQueryExpl = weight.qWeight.explain(ValueSource.readerToContext(weight.fcontext,reader) ,doc);
       if (!subQueryExpl.isMatch()) {
         return subQueryExpl;
       }

Modified: lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/search/function/DocFreqValueSource.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/search/function/DocFreqValueSource.java?rev=1058390&r1=1058389&r2=1058390&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/search/function/DocFreqValueSource.java (original)
+++ lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/search/function/DocFreqValueSource.java Thu Jan 13 02:09:33 2011
@@ -19,7 +19,7 @@ package org.apache.solr.search.function;
 
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.Term;
-import org.apache.lucene.search.Searcher;
+import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.util.BytesRef;
 import org.apache.solr.search.MutableValueInt;
 import org.apache.solr.search.MutableValue;
@@ -240,13 +240,13 @@ public class DocFreqValueSource extends 
 
   @Override
   public DocValues getValues(Map context, IndexReader reader) throws IOException {
-    Searcher searcher = (Searcher)context.get("searcher");
+    IndexSearcher searcher = (IndexSearcher)context.get("searcher");
     int docfreq = searcher.docFreq(new Term(indexedField, indexedBytes));
     return new ConstIntDocValues(docfreq, this);
   }
 
   @Override
-  public void createWeight(Map context, Searcher searcher) throws IOException {
+  public void createWeight(Map context, IndexSearcher searcher) throws IOException {
     context.put("searcher",searcher);
   }
 

Modified: lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/search/function/DualFloatFunction.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/search/function/DualFloatFunction.java?rev=1058390&r1=1058389&r2=1058390&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/search/function/DualFloatFunction.java (original)
+++ lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/search/function/DualFloatFunction.java Thu Jan 13 02:09:33 2011
@@ -18,7 +18,7 @@
 package org.apache.solr.search.function;
 
 import org.apache.lucene.index.IndexReader;
-import org.apache.lucene.search.Searcher;
+import org.apache.lucene.search.IndexSearcher;
 
 import java.io.IOException;
 import java.util.Map;
@@ -69,7 +69,7 @@ public abstract class DualFloatFunction 
   }
 
   @Override
-  public void createWeight(Map context, Searcher searcher) throws IOException {
+  public void createWeight(Map context, IndexSearcher searcher) throws IOException {
     a.createWeight(context,searcher);
     b.createWeight(context,searcher);
   }

Modified: lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/search/function/FunctionQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/search/function/FunctionQuery.java?rev=1058390&r1=1058389&r2=1058390&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/search/function/FunctionQuery.java (original)
+++ lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/search/function/FunctionQuery.java Thu Jan 13 02:09:33 2011
@@ -18,10 +18,11 @@
 package org.apache.solr.search.function;
 
 import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.IndexReader.AtomicReaderContext;
+import org.apache.lucene.index.IndexReader.ReaderContext;
 import org.apache.lucene.search.*;
 import org.apache.lucene.index.MultiFields;
 import org.apache.lucene.util.Bits;
-import org.apache.solr.search.SolrIndexReader;
 
 import java.io.IOException;
 import java.util.Set;
@@ -60,14 +61,14 @@ public class FunctionQuery extends Query
   public void extractTerms(Set terms) {}
 
   protected class FunctionWeight extends Weight {
-    protected Searcher searcher;
+    protected IndexSearcher searcher;
     protected float queryNorm;
     protected float queryWeight;
     protected Map context;
 
-    public FunctionWeight(Searcher searcher) throws IOException {
+    public FunctionWeight(IndexSearcher searcher) throws IOException {
       this.searcher = searcher;
-      this.context = func.newContext();
+      this.context = func.newContext(searcher);
       func.createWeight(context, searcher);
     }
 
@@ -94,18 +95,13 @@ public class FunctionQuery extends Query
     }
 
     @Override
-    public Scorer scorer(IndexReader reader, boolean scoreDocsInOrder, boolean topScorer) throws IOException {
-      return new AllScorer(getSimilarity(searcher), reader, this);
+    public Scorer scorer(AtomicReaderContext context, boolean scoreDocsInOrder, boolean topScorer) throws IOException {
+      return new AllScorer(getSimilarity(searcher), context, this);
     }
 
     @Override
-    public Explanation explain(IndexReader reader, int doc) throws IOException {
-      SolrIndexReader topReader = (SolrIndexReader)reader;
-      SolrIndexReader[] subReaders = topReader.getLeafReaders();
-      int[] offsets = topReader.getLeafOffsets();
-      int readerPos = SolrIndexReader.readerIndex(doc, offsets);
-      int readerBase = offsets[readerPos];
-      return ((AllScorer)scorer(subReaders[readerPos], true, true)).explain(doc-readerBase);
+    public Explanation explain(AtomicReaderContext context, int doc) throws IOException {
+      return ((AllScorer)scorer(context, true, true)).explain(doc);
     }
   }
 
@@ -119,16 +115,18 @@ public class FunctionQuery extends Query
     final boolean hasDeletions;
     final Bits delDocs;
 
-    public AllScorer(Similarity similarity, IndexReader reader, FunctionWeight w) throws IOException {
+    public AllScorer(Similarity similarity, ReaderContext context, FunctionWeight w) throws IOException {
       super(similarity);
       this.weight = w;
       this.qWeight = w.getValue();
-      this.reader = reader;
+      this.reader = context.reader;
       this.maxDoc = reader.maxDoc();
       this.hasDeletions = reader.hasDeletions();
       this.delDocs = MultiFields.getDeletedDocs(reader);
       assert !hasDeletions || delDocs != null;
-      vals = func.getValues(weight.context, reader);
+      Map funcContext = weight.context;
+      funcContext.put(reader, context);
+      vals = func.getValues(funcContext, reader);
     }
 
     @Override
@@ -184,7 +182,7 @@ public class FunctionQuery extends Query
 
 
   @Override
-  public Weight createWeight(Searcher searcher) throws IOException {
+  public Weight createWeight(IndexSearcher searcher) throws IOException {
     return new FunctionQuery.FunctionWeight(searcher);
   }
 

Modified: lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/search/function/IDFValueSource.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/search/function/IDFValueSource.java?rev=1058390&r1=1058389&r2=1058390&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/search/function/IDFValueSource.java (original)
+++ lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/search/function/IDFValueSource.java Thu Jan 13 02:09:33 2011
@@ -18,7 +18,7 @@
 package org.apache.solr.search.function;
 
 import org.apache.lucene.index.*;
-import org.apache.lucene.search.Searcher;
+import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.Similarity;
 import org.apache.lucene.util.BytesRef;
 import org.apache.solr.util.ByteUtils;
@@ -39,7 +39,7 @@ public class IDFValueSource extends DocF
 
   @Override
   public DocValues getValues(Map context, IndexReader reader) throws IOException {
-    Searcher searcher = (Searcher)context.get("searcher");
+    IndexSearcher searcher = (IndexSearcher)context.get("searcher");
     Similarity sim = searcher.getSimilarity();
     // todo: we need docFreq that takes a BytesRef
     String strVal = ByteUtils.UTF8toUTF16(indexedBytes);

Modified: lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/search/function/LinearFloatFunction.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/search/function/LinearFloatFunction.java?rev=1058390&r1=1058389&r2=1058390&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/search/function/LinearFloatFunction.java (original)
+++ lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/search/function/LinearFloatFunction.java Thu Jan 13 02:09:33 2011
@@ -18,7 +18,7 @@
 package org.apache.solr.search.function;
 
 import org.apache.lucene.index.IndexReader;
-import org.apache.lucene.search.Searcher;
+import org.apache.lucene.search.IndexSearcher;
 
 import java.io.IOException;
 import java.util.Map;
@@ -71,7 +71,7 @@ public class LinearFloatFunction extends
   }
 
   @Override
-  public void createWeight(Map context, Searcher searcher) throws IOException {
+  public void createWeight(Map context, IndexSearcher searcher) throws IOException {
     source.createWeight(context, searcher);
   }
 

Modified: lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/search/function/MaxDocValueSource.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/search/function/MaxDocValueSource.java?rev=1058390&r1=1058389&r2=1058390&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/search/function/MaxDocValueSource.java (original)
+++ lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/search/function/MaxDocValueSource.java Thu Jan 13 02:09:33 2011
@@ -17,7 +17,7 @@
 package org.apache.solr.search.function;
 
 import org.apache.lucene.index.IndexReader;
-import org.apache.lucene.search.Searcher;
+import org.apache.lucene.search.IndexSearcher;
 
 import java.io.IOException;
 import java.util.Map;
@@ -33,13 +33,13 @@ public class MaxDocValueSource extends V
   }
 
   @Override
-  public void createWeight(Map context, Searcher searcher) throws IOException {
+  public void createWeight(Map context, IndexSearcher searcher) throws IOException {
     context.put("searcher",searcher);
   }
 
   @Override
   public DocValues getValues(Map context, IndexReader reader) throws IOException {
-    Searcher searcher = (Searcher)context.get("searcher");
+    IndexSearcher searcher = (IndexSearcher)context.get("searcher");
     return new ConstIntDocValues(searcher.maxDoc(), this);
   }
 

Modified: lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/search/function/MaxFloatFunction.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/search/function/MaxFloatFunction.java?rev=1058390&r1=1058389&r2=1058390&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/search/function/MaxFloatFunction.java (original)
+++ lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/search/function/MaxFloatFunction.java Thu Jan 13 02:09:33 2011
@@ -18,7 +18,7 @@
 package org.apache.solr.search.function;
 
 import org.apache.lucene.index.IndexReader;
-import org.apache.lucene.search.Searcher;
+import org.apache.lucene.search.IndexSearcher;
 
 import java.io.IOException;
 import java.util.Map;
@@ -71,7 +71,7 @@ public class MaxFloatFunction extends Va
   }
 
   @Override
-  public void createWeight(Map context, Searcher searcher) throws IOException {
+  public void createWeight(Map context, IndexSearcher searcher) throws IOException {
     source.createWeight(context, searcher);
   }
 

Modified: lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/search/function/MultiFloatFunction.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/search/function/MultiFloatFunction.java?rev=1058390&r1=1058389&r2=1058390&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/search/function/MultiFloatFunction.java (original)
+++ lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/search/function/MultiFloatFunction.java Thu Jan 13 02:09:33 2011
@@ -17,7 +17,7 @@ package org.apache.solr.search.function;
  */
 
 import org.apache.lucene.index.IndexReader;
-import org.apache.lucene.search.Searcher;
+import org.apache.lucene.search.IndexSearcher;
 
 import java.util.Map;
 import java.util.Arrays;
@@ -95,7 +95,7 @@ public abstract class MultiFloatFunction
   }
 
   @Override
-  public void createWeight(Map context, Searcher searcher) throws IOException {
+  public void createWeight(Map context, IndexSearcher searcher) throws IOException {
     for (ValueSource source : sources)
       source.createWeight(context, searcher);
   }

Modified: lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/search/function/NormValueSource.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/search/function/NormValueSource.java?rev=1058390&r1=1058389&r2=1058390&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/search/function/NormValueSource.java (original)
+++ lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/search/function/NormValueSource.java Thu Jan 13 02:09:33 2011
@@ -18,7 +18,7 @@
 package org.apache.solr.search.function;
 
 import org.apache.lucene.index.IndexReader;
-import org.apache.lucene.search.Searcher;
+import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.Similarity;
 import java.io.IOException;
 import java.util.Map;
@@ -39,13 +39,13 @@ public class NormValueSource extends Val
   }
 
   @Override
-  public void createWeight(Map context, Searcher searcher) throws IOException {
+  public void createWeight(Map context, IndexSearcher searcher) throws IOException {
     context.put("searcher",searcher);
   }
 
   @Override
   public DocValues getValues(Map context, IndexReader reader) throws IOException {
-    Searcher searcher = (Searcher)context.get("searcher");
+    IndexSearcher searcher = (IndexSearcher)context.get("searcher");
     final Similarity similarity = searcher.getSimilarity();
     final byte[] norms = reader.norms(field);
     if (norms == null) {

Modified: lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/search/function/OrdFieldSource.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/search/function/OrdFieldSource.java?rev=1058390&r1=1058389&r2=1058390&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/search/function/OrdFieldSource.java (original)
+++ lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/search/function/OrdFieldSource.java Thu Jan 13 02:09:33 2011
@@ -18,8 +18,10 @@
 package org.apache.solr.search.function;
 
 import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.search.FieldCache;
 import org.apache.solr.search.MutableValue;
 import org.apache.solr.search.MutableValueInt;
+import org.apache.solr.search.SolrIndexReader;
 
 import java.io.IOException;
 import java.util.Map;
@@ -55,38 +57,52 @@ public class OrdFieldSource extends Valu
 
 
   public DocValues getValues(Map context, IndexReader reader) throws IOException {
-    return new StringIndexDocValues(this, reader, field) {
+    int offset = 0;
+    IndexReader topReader = reader;
+    if (topReader instanceof SolrIndexReader) {
+      SolrIndexReader r = (SolrIndexReader)topReader;
+      while (r.getParent() != null) {
+        offset += r.getBase();
+        r = r.getParent();
+      }
+      topReader = r;
+    }
+    final int off = offset;
+
+    final FieldCache.DocTermsIndex sindex = FieldCache.DEFAULT.getTermsIndex(topReader, field);
+
+    return new DocValues() {
       protected String toTerm(String readableValue) {
         return readableValue;
       }
       
       public float floatVal(int doc) {
-        return (float)termsIndex.getOrd(doc);
+        return (float)sindex.getOrd(doc+off);
       }
 
       public int intVal(int doc) {
-        return termsIndex.getOrd(doc);
+        return sindex.getOrd(doc+off);
       }
 
       public long longVal(int doc) {
-        return (long)termsIndex.getOrd(doc);
+        return (long)sindex.getOrd(doc+off);
       }
 
       public double doubleVal(int doc) {
-        return (double)termsIndex.getOrd(doc);
+        return (double)sindex.getOrd(doc+off);
       }
 
       public int ordVal(int doc) {
-        return termsIndex.getOrd(doc);
+        return sindex.getOrd(doc+off);
       }
 
       public int numOrd() {
-        return termsIndex.numOrd();
+        return sindex.numOrd();
       }
 
       public String strVal(int doc) {
         // the string value of the ordinal, not the string itself
-        return Integer.toString(termsIndex.getOrd(doc));
+        return Integer.toString(sindex.getOrd(doc+off));
       }
 
       public String toString(int doc) {
@@ -105,7 +121,7 @@ public class OrdFieldSource extends Valu
 
           @Override
           public void fillValue(int doc) {
-            mval.value = termsIndex.getOrd(doc);
+            mval.value = sindex.getOrd(doc);
             mval.exists = mval.value!=0;
           }
         };

Modified: lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/search/function/QueryValueSource.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/search/function/QueryValueSource.java?rev=1058390&r1=1058389&r2=1058390&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/search/function/QueryValueSource.java (original)
+++ lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/search/function/QueryValueSource.java Thu Jan 13 02:09:33 2011
@@ -44,8 +44,8 @@ public class QueryValueSource extends Va
   }
 
   @Override
-  public DocValues getValues(Map context, IndexReader reader) throws IOException {
-    return new QueryDocValues(reader, q, defVal, context==null ? null : (Weight)context.get(this));
+  public DocValues getValues(Map fcontext, IndexReader reader) throws IOException {
+    return new QueryDocValues(reader, q, defVal, fcontext);
   }
 
   public int hashCode() {
@@ -59,7 +59,7 @@ public class QueryValueSource extends Va
   }
 
   @Override
-  public void createWeight(Map context, Searcher searcher) throws IOException {
+  public void createWeight(Map context, IndexSearcher searcher) throws IOException {
     Weight w = q.weight(searcher);
     context.put(this, w);
   }
@@ -71,6 +71,7 @@ class QueryDocValues extends DocValues {
   final IndexReader reader;
   final Weight weight;
   final float defVal;
+  final Map fcontext;
 
   Scorer scorer;
   int scorerDoc; // the document the scorer is on
@@ -79,18 +80,36 @@ class QueryDocValues extends DocValues {
   // to trigger a scorer reset on first access.
   int lastDocRequested=Integer.MAX_VALUE;
 
-  public QueryDocValues(IndexReader reader, Query q, float defVal, Weight w) throws IOException {
+  public QueryDocValues(IndexReader reader, Query q, float defVal, Map fcontext) throws IOException {
     this.reader = reader;
     this.q = q;
     this.defVal = defVal;
-    weight = w!=null ? w : q.weight(new IndexSearcher(reader));
+    this.fcontext = fcontext;
+
+    Weight w = fcontext==null ? null : (Weight)fcontext.get(q);
+    if (w == null) {
+       IndexSearcher weightSearcher = fcontext == null ? new IndexSearcher(reader) : (IndexSearcher)fcontext.get("searcher");
+
+       // TODO: sort by function doesn't weight (SOLR-1297 is open because of this bug)... so weightSearcher will currently be null
+       if (weightSearcher == null) weightSearcher = new IndexSearcher(reader);
+
+       w = q.weight(weightSearcher);
+    }
+    weight = w;
   }
 
   public float floatVal(int doc) {
     try {
       if (doc < lastDocRequested) {
         // out-of-order access.... reset scorer.
-        scorer = weight.scorer(reader, true, false);
+        IndexReader.AtomicReaderContext ctx = ValueSource.readerToContext(fcontext, reader);
+
+        if (ctx == null) {
+          // TODO: this is because SOLR-1297 does not weight
+          ctx = (IndexReader.AtomicReaderContext)reader.getTopReaderContext();  // this is the incorrect context
+        }
+
+        scorer = weight.scorer(ctx, true, false);
         if (scorer==null) return defVal;
         scorerDoc = -1;
       }

Modified: lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/search/function/RangeMapFloatFunction.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/search/function/RangeMapFloatFunction.java?rev=1058390&r1=1058389&r2=1058390&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/search/function/RangeMapFloatFunction.java (original)
+++ lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/search/function/RangeMapFloatFunction.java Thu Jan 13 02:09:33 2011
@@ -18,7 +18,7 @@
 package org.apache.solr.search.function;
 
 import org.apache.lucene.index.IndexReader;
-import org.apache.lucene.search.Searcher;
+import org.apache.lucene.search.IndexSearcher;
 
 import java.io.IOException;
 import java.util.Map;
@@ -76,7 +76,7 @@ public class RangeMapFloatFunction exten
   }
 
   @Override
-  public void createWeight(Map context, Searcher searcher) throws IOException {
+  public void createWeight(Map context, IndexSearcher searcher) throws IOException {
     source.createWeight(context, searcher);
   }
 

Modified: lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/search/function/ReciprocalFloatFunction.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/search/function/ReciprocalFloatFunction.java?rev=1058390&r1=1058389&r2=1058390&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/search/function/ReciprocalFloatFunction.java (original)
+++ lucene/dev/branches/bulkpostings/solr/src/java/org/apache/solr/search/function/ReciprocalFloatFunction.java Thu Jan 13 02:09:33 2011
@@ -18,7 +18,7 @@
 package org.apache.solr.search.function;
 
 import org.apache.lucene.index.IndexReader;
-import org.apache.lucene.search.Searcher;
+import org.apache.lucene.search.IndexSearcher;
 
 import java.io.IOException;
 import java.util.Map;
@@ -84,7 +84,7 @@ public class ReciprocalFloatFunction ext
   }
 
   @Override
-  public void createWeight(Map context, Searcher searcher) throws IOException {
+  public void createWeight(Map context, IndexSearcher searcher) throws IOException {
     source.createWeight(context, searcher);
   }