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 2009/09/17 16:08:15 UTC

svn commit: r816202 - in /lucene/solr/trunk/src: java/org/apache/solr/schema/ java/org/apache/solr/search/ java/org/apache/solr/search/function/ test/org/apache/solr/search/function/

Author: yonik
Date: Thu Sep 17 14:08:13 2009
New Revision: 816202

URL: http://svn.apache.org/viewvc?rev=816202&view=rev
Log:
SOLR-1432: weight nested queries in function queries

Added:
    lucene/solr/trunk/src/java/org/apache/solr/search/SolrConstantScoreQuery.java   (with props)
    lucene/solr/trunk/src/java/org/apache/solr/search/SolrFilter.java
    lucene/solr/trunk/src/java/org/apache/solr/search/function/SingleFunction.java   (with props)
Modified:
    lucene/solr/trunk/src/java/org/apache/solr/schema/DateField.java
    lucene/solr/trunk/src/java/org/apache/solr/schema/RandomSortField.java
    lucene/solr/trunk/src/java/org/apache/solr/schema/SortableDoubleField.java
    lucene/solr/trunk/src/java/org/apache/solr/schema/SortableFloatField.java
    lucene/solr/trunk/src/java/org/apache/solr/schema/SortableIntField.java
    lucene/solr/trunk/src/java/org/apache/solr/schema/SortableLongField.java
    lucene/solr/trunk/src/java/org/apache/solr/schema/StrField.java
    lucene/solr/trunk/src/java/org/apache/solr/search/FunctionRangeQParserPlugin.java
    lucene/solr/trunk/src/java/org/apache/solr/search/QueryParsing.java
    lucene/solr/trunk/src/java/org/apache/solr/search/ValueSourceParser.java
    lucene/solr/trunk/src/java/org/apache/solr/search/function/BoostedQuery.java
    lucene/solr/trunk/src/java/org/apache/solr/search/function/ByteFieldSource.java
    lucene/solr/trunk/src/java/org/apache/solr/search/function/ConstValueSource.java
    lucene/solr/trunk/src/java/org/apache/solr/search/function/DoubleFieldSource.java
    lucene/solr/trunk/src/java/org/apache/solr/search/function/DualFloatFunction.java
    lucene/solr/trunk/src/java/org/apache/solr/search/function/FileFloatSource.java
    lucene/solr/trunk/src/java/org/apache/solr/search/function/FloatFieldSource.java
    lucene/solr/trunk/src/java/org/apache/solr/search/function/FunctionQuery.java
    lucene/solr/trunk/src/java/org/apache/solr/search/function/IntFieldSource.java
    lucene/solr/trunk/src/java/org/apache/solr/search/function/LinearFloatFunction.java
    lucene/solr/trunk/src/java/org/apache/solr/search/function/LongFieldSource.java
    lucene/solr/trunk/src/java/org/apache/solr/search/function/MaxFloatFunction.java
    lucene/solr/trunk/src/java/org/apache/solr/search/function/OrdFieldSource.java
    lucene/solr/trunk/src/java/org/apache/solr/search/function/QueryValueSource.java
    lucene/solr/trunk/src/java/org/apache/solr/search/function/RangeMapFloatFunction.java
    lucene/solr/trunk/src/java/org/apache/solr/search/function/ReciprocalFloatFunction.java
    lucene/solr/trunk/src/java/org/apache/solr/search/function/ReverseOrdFieldSource.java
    lucene/solr/trunk/src/java/org/apache/solr/search/function/ScaleFloatFunction.java
    lucene/solr/trunk/src/java/org/apache/solr/search/function/ShortFieldSource.java
    lucene/solr/trunk/src/java/org/apache/solr/search/function/SimpleFloatFunction.java
    lucene/solr/trunk/src/java/org/apache/solr/search/function/SumFloatFunction.java
    lucene/solr/trunk/src/java/org/apache/solr/search/function/TopValueSource.java
    lucene/solr/trunk/src/java/org/apache/solr/search/function/ValueSource.java
    lucene/solr/trunk/src/java/org/apache/solr/search/function/ValueSourceRangeFilter.java
    lucene/solr/trunk/src/test/org/apache/solr/search/function/TestFunctionQuery.java

Modified: lucene/solr/trunk/src/java/org/apache/solr/schema/DateField.java
URL: http://svn.apache.org/viewvc/lucene/solr/trunk/src/java/org/apache/solr/schema/DateField.java?rev=816202&r1=816201&r2=816202&view=diff
==============================================================================
--- lucene/solr/trunk/src/java/org/apache/solr/schema/DateField.java (original)
+++ lucene/solr/trunk/src/java/org/apache/solr/schema/DateField.java Thu Sep 17 14:08:13 2009
@@ -33,10 +33,7 @@
 
 import java.io.IOException;
 import java.text.*;
-import java.util.Date;
-import java.util.Locale;
-import java.util.Map;
-import java.util.TimeZone;
+import java.util.*;
 
 // TODO: make a FlexibleDateField that can accept dates in multiple
 // formats, better for human entered dates.
@@ -425,7 +422,7 @@
     return "date(" + field + ')';
   }
 
-  public DocValues getValues(IndexReader reader) throws IOException {
+  public DocValues getValues(Map context, IndexReader reader) throws IOException {
     return new StringIndexDocValues(this, reader, field) {
       protected String toTerm(String readableValue) {
         // needed for frange queries to work properly

Modified: lucene/solr/trunk/src/java/org/apache/solr/schema/RandomSortField.java
URL: http://svn.apache.org/viewvc/lucene/solr/trunk/src/java/org/apache/solr/schema/RandomSortField.java?rev=816202&r1=816201&r2=816202&view=diff
==============================================================================
--- lucene/solr/trunk/src/java/org/apache/solr/schema/RandomSortField.java (original)
+++ lucene/solr/trunk/src/java/org/apache/solr/schema/RandomSortField.java Thu Sep 17 14:08:13 2009
@@ -160,7 +160,7 @@
     }
 
     @Override
-    public DocValues getValues(final IndexReader reader) throws IOException {
+    public DocValues getValues(Map context, final IndexReader reader) throws IOException {
       return new DocValues() {
           private final int seed = getSeed(field, reader);
           @Override

Modified: lucene/solr/trunk/src/java/org/apache/solr/schema/SortableDoubleField.java
URL: http://svn.apache.org/viewvc/lucene/solr/trunk/src/java/org/apache/solr/schema/SortableDoubleField.java?rev=816202&r1=816201&r2=816202&view=diff
==============================================================================
--- lucene/solr/trunk/src/java/org/apache/solr/schema/SortableDoubleField.java (original)
+++ lucene/solr/trunk/src/java/org/apache/solr/schema/SortableDoubleField.java Thu Sep 17 14:08:13 2009
@@ -18,7 +18,6 @@
 package org.apache.solr.schema;
 
 import org.apache.lucene.search.SortField;
-import org.apache.lucene.search.FieldCache;
 import org.apache.solr.search.function.ValueSource;
 import org.apache.solr.search.function.FieldCacheSource;
 import org.apache.solr.search.function.DocValues;
@@ -93,7 +92,7 @@
     return "sdouble(" + field + ')';
   }
 
-  public DocValues getValues(IndexReader reader) throws IOException {
+  public DocValues getValues(Map context, IndexReader reader) throws IOException {
     final double def = defVal;
 
     return new StringIndexDocValues(this, reader, field) {

Modified: lucene/solr/trunk/src/java/org/apache/solr/schema/SortableFloatField.java
URL: http://svn.apache.org/viewvc/lucene/solr/trunk/src/java/org/apache/solr/schema/SortableFloatField.java?rev=816202&r1=816201&r2=816202&view=diff
==============================================================================
--- lucene/solr/trunk/src/java/org/apache/solr/schema/SortableFloatField.java (original)
+++ lucene/solr/trunk/src/java/org/apache/solr/schema/SortableFloatField.java Thu Sep 17 14:08:13 2009
@@ -18,7 +18,6 @@
 package org.apache.solr.schema;
 
 import org.apache.lucene.search.SortField;
-import org.apache.lucene.search.FieldCache;
 import org.apache.solr.search.function.ValueSource;
 import org.apache.solr.search.function.FieldCacheSource;
 import org.apache.solr.search.function.DocValues;
@@ -93,7 +92,7 @@
     return "sfloat(" + field + ')';
   }
 
-  public DocValues getValues(IndexReader reader) throws IOException {
+  public DocValues getValues(Map context, IndexReader reader) throws IOException {
     final float def = defVal;
 
     return new StringIndexDocValues(this, reader, field) {

Modified: lucene/solr/trunk/src/java/org/apache/solr/schema/SortableIntField.java
URL: http://svn.apache.org/viewvc/lucene/solr/trunk/src/java/org/apache/solr/schema/SortableIntField.java?rev=816202&r1=816201&r2=816202&view=diff
==============================================================================
--- lucene/solr/trunk/src/java/org/apache/solr/schema/SortableIntField.java (original)
+++ lucene/solr/trunk/src/java/org/apache/solr/schema/SortableIntField.java Thu Sep 17 14:08:13 2009
@@ -18,7 +18,6 @@
 package org.apache.solr.schema;
 
 import org.apache.lucene.search.SortField;
-import org.apache.lucene.search.FieldCache;
 import org.apache.solr.search.function.ValueSource;
 import org.apache.solr.search.function.FieldCacheSource;
 import org.apache.solr.search.function.DocValues;
@@ -97,7 +96,7 @@
     return "sint(" + field + ')';
   }
 
-  public DocValues getValues(IndexReader reader) throws IOException {
+  public DocValues getValues(Map context, IndexReader reader) throws IOException {
     final int def = defVal;
 
     return new StringIndexDocValues(this, reader, field) {

Modified: lucene/solr/trunk/src/java/org/apache/solr/schema/SortableLongField.java
URL: http://svn.apache.org/viewvc/lucene/solr/trunk/src/java/org/apache/solr/schema/SortableLongField.java?rev=816202&r1=816201&r2=816202&view=diff
==============================================================================
--- lucene/solr/trunk/src/java/org/apache/solr/schema/SortableLongField.java (original)
+++ lucene/solr/trunk/src/java/org/apache/solr/schema/SortableLongField.java Thu Sep 17 14:08:13 2009
@@ -18,7 +18,6 @@
 package org.apache.solr.schema;
 
 import org.apache.lucene.search.SortField;
-import org.apache.lucene.search.FieldCache;
 import org.apache.solr.search.function.ValueSource;
 import org.apache.solr.search.function.FieldCacheSource;
 import org.apache.solr.search.function.DocValues;
@@ -94,7 +93,7 @@
     return "slong(" + field + ')';
   }
 
-  public DocValues getValues(IndexReader reader) throws IOException {
+  public DocValues getValues(Map context, IndexReader reader) throws IOException {
     final long def = defVal;
 
     return new StringIndexDocValues(this, reader, field) {

Modified: lucene/solr/trunk/src/java/org/apache/solr/schema/StrField.java
URL: http://svn.apache.org/viewvc/lucene/solr/trunk/src/java/org/apache/solr/schema/StrField.java?rev=816202&r1=816201&r2=816202&view=diff
==============================================================================
--- lucene/solr/trunk/src/java/org/apache/solr/schema/StrField.java (original)
+++ lucene/solr/trunk/src/java/org/apache/solr/schema/StrField.java Thu Sep 17 14:08:13 2009
@@ -27,7 +27,6 @@
 import org.apache.solr.search.function.DocValues;
 import org.apache.solr.search.function.StringIndexDocValues;
 import org.apache.solr.search.QParser;
-import org.apache.solr.util.NumberUtils;
 
 import java.util.Map;
 import java.io.IOException;
@@ -67,7 +66,7 @@
     return "str(" + field + ')';
   }
 
-  public DocValues getValues(IndexReader reader) throws IOException {
+  public DocValues getValues(Map context, IndexReader reader) throws IOException {
     return new StringIndexDocValues(this, reader, field) {
       protected String toTerm(String readableValue) {
         return readableValue;

Modified: lucene/solr/trunk/src/java/org/apache/solr/search/FunctionRangeQParserPlugin.java
URL: http://svn.apache.org/viewvc/lucene/solr/trunk/src/java/org/apache/solr/search/FunctionRangeQParserPlugin.java?rev=816202&r1=816201&r2=816202&view=diff
==============================================================================
--- lucene/solr/trunk/src/java/org/apache/solr/search/FunctionRangeQParserPlugin.java (original)
+++ lucene/solr/trunk/src/java/org/apache/solr/search/FunctionRangeQParserPlugin.java Thu Sep 17 14:08:13 2009
@@ -19,6 +19,7 @@
 import org.apache.lucene.queryParser.ParseException;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.ConstantScoreQuery;
+import org.apache.lucene.search.Filter;
 import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.request.SolrQueryRequest;
@@ -60,10 +61,19 @@
 
         // TODO: add a score=val option to allow score to be the value
         ValueSourceRangeFilter rf = new ValueSourceRangeFilter(vs, l, u, includeLower, includeUpper);
-        ConstantScoreQuery csq = new ConstantScoreQuery(rf);
+        SolrConstantScoreQuery csq = new SolrConstantScoreQuery(rf);
         return csq;
       }
     };
   }
 
 }
+
+class FunctionConstantScoreQuery extends ConstantScoreQuery {
+  public FunctionConstantScoreQuery(Filter filter) {
+    super(filter);
+  }
+
+
+
+}
\ No newline at end of file

Modified: lucene/solr/trunk/src/java/org/apache/solr/search/QueryParsing.java
URL: http://svn.apache.org/viewvc/lucene/solr/trunk/src/java/org/apache/solr/search/QueryParsing.java?rev=816202&r1=816201&r2=816202&view=diff
==============================================================================
--- lucene/solr/trunk/src/java/org/apache/solr/search/QueryParsing.java (original)
+++ lucene/solr/trunk/src/java/org/apache/solr/search/QueryParsing.java Thu Sep 17 14:08:13 2009
@@ -17,7 +17,6 @@
 
 package org.apache.solr.search;
 
-import org.apache.lucene.document.Field;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.queryParser.ParseException;
 import org.apache.lucene.queryParser.QueryParser;
@@ -436,7 +435,7 @@
     } else if (query instanceof FuzzyQuery) {
       out.append(query.toString());
       writeBoost=false;      
-    } else if (query instanceof ConstantScoreQuery) {
+    } else if (query instanceof SolrConstantScoreQuery) {
       out.append(query.toString());
       writeBoost=false;
     } else {

Added: lucene/solr/trunk/src/java/org/apache/solr/search/SolrConstantScoreQuery.java
URL: http://svn.apache.org/viewvc/lucene/solr/trunk/src/java/org/apache/solr/search/SolrConstantScoreQuery.java?rev=816202&view=auto
==============================================================================
--- lucene/solr/trunk/src/java/org/apache/solr/search/SolrConstantScoreQuery.java (added)
+++ lucene/solr/trunk/src/java/org/apache/solr/search/SolrConstantScoreQuery.java Thu Sep 17 14:08:13 2009
@@ -0,0 +1,201 @@
+package org.apache.solr.search;
+
+import org.apache.lucene.search.*;
+import org.apache.lucene.index.IndexReader;
+import org.apache.solr.search.function.ValueSource;
+import org.apache.solr.common.SolrException;
+
+import java.io.IOException;
+import java.util.Set;
+import java.util.Map;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * A query that wraps a filter and simply returns a constant score equal to the
+ * query boost for every document in the filter.   This Solr extension also supports
+ * weighting of a SolrFilter.
+ *
+ * Experimental and subject to change.
+ */
+public class SolrConstantScoreQuery extends ConstantScoreQuery {
+
+  public SolrConstantScoreQuery(Filter filter) {
+    super(filter);
+  }
+
+  /** Returns the encapsulated filter */
+  public Filter getFilter() {
+    return filter;
+  }
+
+  public Query rewrite(IndexReader reader) throws IOException {
+    return this;
+  }
+
+  public void extractTerms(Set terms) {
+    // OK to not add any terms when used for MultiSearcher,
+    // but may not be OK for highlighting
+  }
+
+  protected class ConstantWeight extends Weight {
+    private Similarity similarity;
+    private float queryNorm;
+    private float queryWeight;
+    private Map context;
+
+    public ConstantWeight(Searcher searcher) throws IOException {
+      this.similarity = getSimilarity(searcher);
+      this.context = ValueSource.newContext();
+      if (filter instanceof SolrFilter)
+        ((SolrFilter)filter).createWeight(context, searcher);
+    }
+
+    public Query getQuery() {
+      return SolrConstantScoreQuery.this;
+    }
+
+    public float getValue() {
+      return queryWeight;
+    }
+
+    public float sumOfSquaredWeights() throws IOException {
+      queryWeight = getBoost();
+      return queryWeight * queryWeight;
+    }
+
+    public void normalize(float norm) {
+      this.queryNorm = norm;
+      queryWeight *= this.queryNorm;
+    }
+
+    public Scorer scorer(IndexReader reader, boolean scoreDocsInOrder, boolean topScorer) throws IOException {
+      return new ConstantScorer(similarity, reader, this);
+    }
+
+    public Explanation explain(IndexReader reader, int doc) throws IOException {
+
+      ConstantScorer cs = new ConstantScorer(similarity, reader, this);
+      boolean exists = cs.docIdSetIterator.advance(doc) == doc;
+
+      ComplexExplanation result = new ComplexExplanation();
+
+      if (exists) {
+        result.setDescription("ConstantScoreQuery(" + filter
+        + "), product of:");
+        result.setValue(queryWeight);
+        result.setMatch(Boolean.TRUE);
+        result.addDetail(new Explanation(getBoost(), "boost"));
+        result.addDetail(new Explanation(queryNorm,"queryNorm"));
+      } else {
+        result.setDescription("ConstantScoreQuery(" + filter
+        + ") doesn't match id " + doc);
+        result.setValue(0);
+        result.setMatch(Boolean.FALSE);
+      }
+      return result;
+    }
+  }
+
+  protected class ConstantScorer extends Scorer {
+    final DocIdSetIterator docIdSetIterator;
+    final float theScore;
+    int doc = -1;
+
+    public ConstantScorer(Similarity similarity, IndexReader reader, ConstantWeight w) throws IOException {
+      super(similarity);
+      theScore = w.getValue();
+      DocIdSet docIdSet = filter instanceof SolrFilter ? ((SolrFilter)filter).getDocIdSet(w.context, reader) : filter.getDocIdSet(reader);
+      if (docIdSet == null) {
+        docIdSetIterator = DocIdSet.EMPTY_DOCIDSET.iterator();
+      } else {
+        DocIdSetIterator iter = docIdSet.iterator();
+        if (iter == null) {
+          docIdSetIterator = DocIdSet.EMPTY_DOCIDSET.iterator();
+        } else {
+          docIdSetIterator = iter;
+        }
+      }
+    }
+
+    /** @deprecated use {@link #nextDoc()} instead. */
+    public boolean next() throws IOException {
+      return docIdSetIterator.nextDoc() != NO_MORE_DOCS;
+    }
+
+    public int nextDoc() throws IOException {
+      return docIdSetIterator.nextDoc();
+    }
+
+    /** @deprecated use {@link #docID()} instead. */
+    public int doc() {
+      return docIdSetIterator.doc();
+    }
+
+    public int docID() {
+      return docIdSetIterator.docID();
+    }
+
+    public float score() throws IOException {
+      return theScore;
+    }
+
+    /** @deprecated use {@link #advance(int)} instead. */
+    public boolean skipTo(int target) throws IOException {
+      return docIdSetIterator.advance(target) != NO_MORE_DOCS;
+    }
+
+    public int advance(int target) throws IOException {
+      return docIdSetIterator.advance(target);
+    }
+
+    public Explanation explain(int doc) throws IOException {
+      throw new UnsupportedOperationException();
+    }
+  }
+
+  public Weight createWeight(Searcher searcher) {
+    try {
+      return new SolrConstantScoreQuery.ConstantWeight(searcher);
+    } catch (IOException e) {
+      // TODO: remove this if ConstantScoreQuery.createWeight adds IOException
+      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
+    }
+  }
+
+  /** Prints a user-readable version of this query. */
+  public String toString(String field) {
+    return "ConstantScore(" + filter.toString()
+      + (getBoost()==1.0 ? ")" : "^" + getBoost());
+  }
+
+  /** Returns true if <code>o</code> is equal to this. */
+  public boolean equals(Object o) {
+    if (this == o) return true;
+    if (!(o instanceof SolrConstantScoreQuery)) return false;
+    SolrConstantScoreQuery other = (SolrConstantScoreQuery)o;
+    return this.getBoost()==other.getBoost() && filter.equals(other.filter);
+  }
+
+  /** Returns a hash code value for this object. */
+  public int hashCode() {
+    // Simple add is OK since no existing filter hashcode has a float component.
+    return filter.hashCode() + Float.floatToIntBits(getBoost());
+  }
+
+}

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

Propchange: lucene/solr/trunk/src/java/org/apache/solr/search/SolrConstantScoreQuery.java
------------------------------------------------------------------------------
    svn:executable = *

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

Added: lucene/solr/trunk/src/java/org/apache/solr/search/SolrFilter.java
URL: http://svn.apache.org/viewvc/lucene/solr/trunk/src/java/org/apache/solr/search/SolrFilter.java?rev=816202&view=auto
==============================================================================
--- lucene/solr/trunk/src/java/org/apache/solr/search/SolrFilter.java (added)
+++ lucene/solr/trunk/src/java/org/apache/solr/search/SolrFilter.java Thu Sep 17 14:08:13 2009
@@ -0,0 +1,46 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.solr.search;
+
+import org.apache.lucene.search.Filter;
+import org.apache.lucene.search.Searcher;
+import org.apache.lucene.search.DocIdSet;
+import org.apache.lucene.index.IndexReader;
+
+import java.util.Map;
+import java.io.IOException;
+
+
+/** A SolrFilter extends the Lucene Filter and adds extra semantics such as passing on
+ * weight context info for function queries.
+ *
+ * Experimental and subject to change.
+ */
+public abstract class SolrFilter extends Filter {
+
+  /** 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 DocIdSet getDocIdSet(Map context, IndexReader reader) throws IOException;
+
+  @Override
+  public DocIdSet getDocIdSet(IndexReader reader) throws IOException {
+    return getDocIdSet(null, reader);
+  }
+}

Modified: lucene/solr/trunk/src/java/org/apache/solr/search/ValueSourceParser.java
URL: http://svn.apache.org/viewvc/lucene/solr/trunk/src/java/org/apache/solr/search/ValueSourceParser.java?rev=816202&r1=816201&r2=816202&view=diff
==============================================================================
--- lucene/solr/trunk/src/java/org/apache/solr/search/ValueSourceParser.java (original)
+++ lucene/solr/trunk/src/java/org/apache/solr/search/ValueSourceParser.java Thu Sep 17 14:08:13 2009
@@ -16,10 +16,7 @@
  */
 package org.apache.solr.search;
 
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Date;
+import java.util.*;
 import java.io.IOException;
 
 import org.apache.lucene.queryParser.ParseException;
@@ -398,7 +395,7 @@
     return "const(" + constant + ")";
   }
 
-  public DocValues getValues(IndexReader reader) throws IOException {
+  public DocValues getValues(Map context, IndexReader reader) throws IOException {
     return new DocValues() {
       public float floatVal(int doc) {
         return constant;

Modified: lucene/solr/trunk/src/java/org/apache/solr/search/function/BoostedQuery.java
URL: http://svn.apache.org/viewvc/lucene/solr/trunk/src/java/org/apache/solr/search/function/BoostedQuery.java?rev=816202&r1=816201&r2=816202&view=diff
==============================================================================
--- lucene/solr/trunk/src/java/org/apache/solr/search/function/BoostedQuery.java (original)
+++ lucene/solr/trunk/src/java/org/apache/solr/search/function/BoostedQuery.java Thu Sep 17 14:08:13 2009
@@ -24,6 +24,7 @@
 
 import java.io.IOException;
 import java.util.Set;
+import java.util.Map;
 
 /**
  * Query that is boosted by a ValueSource
@@ -59,10 +60,13 @@
   private class BoostedWeight extends Weight {
     Searcher searcher;
     Weight qWeight;
+    Map context;
 
     public BoostedWeight(Searcher searcher) throws IOException {
       this.searcher = searcher;
       this.qWeight = q.weight(searcher);
+      this.context = boostVal.newContext();
+      boostVal.createWeight(context,searcher);
     }
 
     public Query getQuery() {
@@ -105,7 +109,7 @@
         return subQueryExpl;
       }
 
-      DocValues vals = boostVal.getValues(subReaders[readerPos]);
+      DocValues vals = boostVal.getValues(context, subReaders[readerPos]);
       float sc = subQueryExpl.getValue() * vals.floatVal(doc-readerBase);
       Explanation res = new ComplexExplanation(
         true, sc, BoostedQuery.this.toString() + ", product of:");
@@ -132,7 +136,7 @@
       this.scorer = scorer;
       this.reader = reader;
       this.searcher = searcher; // for explain
-      this.vals = vs.getValues(reader);
+      this.vals = vs.getValues(weight.context, reader);
     }
 
     @Override

Modified: lucene/solr/trunk/src/java/org/apache/solr/search/function/ByteFieldSource.java
URL: http://svn.apache.org/viewvc/lucene/solr/trunk/src/java/org/apache/solr/search/function/ByteFieldSource.java?rev=816202&r1=816201&r2=816202&view=diff
==============================================================================
--- lucene/solr/trunk/src/java/org/apache/solr/search/function/ByteFieldSource.java (original)
+++ lucene/solr/trunk/src/java/org/apache/solr/search/function/ByteFieldSource.java Thu Sep 17 14:08:13 2009
@@ -20,6 +20,7 @@
 import org.apache.lucene.search.FieldCache;
 
 import java.io.IOException;
+import java.util.Map;
 
 /**
  * Obtains int field values from the {@link org.apache.lucene.search.FieldCache}
@@ -45,7 +46,7 @@
     return "byte(" + field + ')';
   }
 
-  public DocValues getValues(IndexReader reader) throws IOException {
+  public DocValues getValues(Map context, IndexReader reader) throws IOException {
     final byte[] arr = (parser == null) ?
             cache.getBytes(reader, field) :
             cache.getBytes(reader, field, parser);

Modified: lucene/solr/trunk/src/java/org/apache/solr/search/function/ConstValueSource.java
URL: http://svn.apache.org/viewvc/lucene/solr/trunk/src/java/org/apache/solr/search/function/ConstValueSource.java?rev=816202&r1=816201&r2=816202&view=diff
==============================================================================
--- lucene/solr/trunk/src/java/org/apache/solr/search/function/ConstValueSource.java (original)
+++ lucene/solr/trunk/src/java/org/apache/solr/search/function/ConstValueSource.java Thu Sep 17 14:08:13 2009
@@ -20,6 +20,7 @@
 import org.apache.lucene.index.IndexReader;
 
 import java.io.IOException;
+import java.util.Map;
 
 /**
  * <code>ConstValueSource</code> returns a constant for all documents
@@ -35,7 +36,7 @@
     return "const(" + constant + ")";
   }
 
-  public DocValues getValues(IndexReader reader) throws IOException {
+  public DocValues getValues(Map context, IndexReader reader) throws IOException {
     return new DocValues() {
       public float floatVal(int doc) {
         return constant;

Modified: lucene/solr/trunk/src/java/org/apache/solr/search/function/DoubleFieldSource.java
URL: http://svn.apache.org/viewvc/lucene/solr/trunk/src/java/org/apache/solr/search/function/DoubleFieldSource.java?rev=816202&r1=816201&r2=816202&view=diff
==============================================================================
--- lucene/solr/trunk/src/java/org/apache/solr/search/function/DoubleFieldSource.java (original)
+++ lucene/solr/trunk/src/java/org/apache/solr/search/function/DoubleFieldSource.java Thu Sep 17 14:08:13 2009
@@ -21,6 +21,7 @@
 import org.apache.lucene.search.FieldCache;
 
 import java.io.IOException;
+import java.util.Map;
 
 /**
  * Obtains float field values from the {@link org.apache.lucene.search.FieldCache}
@@ -46,7 +47,7 @@
     return "double(" + field + ')';
   }
 
-  public DocValues getValues(IndexReader reader) throws IOException {
+  public DocValues getValues(Map context, IndexReader reader) throws IOException {
     final double[] arr = (parser == null) ?
             ((FieldCache) cache).getDoubles(reader, field) :
             ((FieldCache) cache).getDoubles(reader, field, parser);

Modified: lucene/solr/trunk/src/java/org/apache/solr/search/function/DualFloatFunction.java
URL: http://svn.apache.org/viewvc/lucene/solr/trunk/src/java/org/apache/solr/search/function/DualFloatFunction.java?rev=816202&r1=816201&r2=816202&view=diff
==============================================================================
--- lucene/solr/trunk/src/java/org/apache/solr/search/function/DualFloatFunction.java (original)
+++ lucene/solr/trunk/src/java/org/apache/solr/search/function/DualFloatFunction.java Thu Sep 17 14:08:13 2009
@@ -18,8 +18,10 @@
 package org.apache.solr.search.function;
 
 import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.search.Searcher;
 
 import java.io.IOException;
+import java.util.Map;
 
 public abstract class DualFloatFunction extends ValueSource {
   protected final ValueSource a;
@@ -41,9 +43,9 @@
     return name() + "(" + a.description() + "," + b.description() + ")";
   }
 
-  public DocValues getValues(IndexReader reader) throws IOException {
-    final DocValues aVals =  a.getValues(reader);
-    final DocValues bVals =  b.getValues(reader);
+  public DocValues getValues(Map context, IndexReader reader) throws IOException {
+    final DocValues aVals =  a.getValues(context, reader);
+    final DocValues bVals =  b.getValues(context, reader);
     return new DocValues() {
       public float floatVal(int doc) {
 	return func(doc, aVals, bVals);
@@ -66,6 +68,12 @@
     };
   }
 
+  @Override
+  public void createWeight(Map context, Searcher searcher) throws IOException {
+    a.createWeight(context,searcher);
+    b.createWeight(context,searcher);
+  }
+
   public int hashCode() {
     int h = a.hashCode();
     h ^= (h << 13) | (h >>> 20);

Modified: lucene/solr/trunk/src/java/org/apache/solr/search/function/FileFloatSource.java
URL: http://svn.apache.org/viewvc/lucene/solr/trunk/src/java/org/apache/solr/search/function/FileFloatSource.java?rev=816202&r1=816201&r2=816202&view=diff
==============================================================================
--- lucene/solr/trunk/src/java/org/apache/solr/search/function/FileFloatSource.java (original)
+++ lucene/solr/trunk/src/java/org/apache/solr/search/function/FileFloatSource.java Thu Sep 17 14:08:13 2009
@@ -54,7 +54,7 @@
     return "float(" + field + ')';
   }
 
-  public DocValues getValues(IndexReader reader) throws IOException {
+  public DocValues getValues(Map context, IndexReader reader) throws IOException {
     int offset = 0;
     if (reader instanceof SolrIndexReader) {
       SolrIndexReader r = (SolrIndexReader)reader;

Modified: lucene/solr/trunk/src/java/org/apache/solr/search/function/FloatFieldSource.java
URL: http://svn.apache.org/viewvc/lucene/solr/trunk/src/java/org/apache/solr/search/function/FloatFieldSource.java?rev=816202&r1=816201&r2=816202&view=diff
==============================================================================
--- lucene/solr/trunk/src/java/org/apache/solr/search/function/FloatFieldSource.java (original)
+++ lucene/solr/trunk/src/java/org/apache/solr/search/function/FloatFieldSource.java Thu Sep 17 14:08:13 2009
@@ -22,6 +22,7 @@
 import org.apache.lucene.search.FieldCache;
 
 import java.io.IOException;
+import java.util.Map;
 
 /**
  * Obtains float field values from the {@link org.apache.lucene.search.FieldCache}
@@ -47,7 +48,7 @@
     return "float(" + field + ')';
   }
 
-  public DocValues getValues(IndexReader reader) throws IOException {
+  public DocValues getValues(Map context, IndexReader reader) throws IOException {
     final float[] arr = (parser==null) ?
             cache.getFloats(reader, field) :
             cache.getFloats(reader, field, parser);

Modified: lucene/solr/trunk/src/java/org/apache/solr/search/function/FunctionQuery.java
URL: http://svn.apache.org/viewvc/lucene/solr/trunk/src/java/org/apache/solr/search/function/FunctionQuery.java?rev=816202&r1=816201&r2=816202&view=diff
==============================================================================
--- lucene/solr/trunk/src/java/org/apache/solr/search/function/FunctionQuery.java (original)
+++ lucene/solr/trunk/src/java/org/apache/solr/search/function/FunctionQuery.java Thu Sep 17 14:08:13 2009
@@ -23,6 +23,8 @@
 
 import java.io.IOException;
 import java.util.Set;
+import java.util.IdentityHashMap;
+import java.util.Map;
 
 
 /**
@@ -58,9 +60,12 @@
     protected Searcher searcher;
     protected float queryNorm;
     protected float queryWeight;
+    protected Map context;
 
-    public FunctionWeight(Searcher searcher) {
+    public FunctionWeight(Searcher searcher) throws IOException {
       this.searcher = searcher;
+      this.context = func.newContext();
+      func.createWeight(context, searcher);
     }
 
     public Query getQuery() {
@@ -115,7 +120,7 @@
       this.reader = reader;
       this.maxDoc = reader.maxDoc();
       this.hasDeletions = reader.hasDeletions();
-      vals = func.getValues(reader);
+      vals = func.getValues(weight.context, reader);
     }
 
     @Override
@@ -197,7 +202,7 @@
   }
 
 
-  public Weight createWeight(Searcher searcher) {
+  public Weight createWeight(Searcher searcher) throws IOException {
     return new FunctionQuery.FunctionWeight(searcher);
   }
 

Modified: lucene/solr/trunk/src/java/org/apache/solr/search/function/IntFieldSource.java
URL: http://svn.apache.org/viewvc/lucene/solr/trunk/src/java/org/apache/solr/search/function/IntFieldSource.java?rev=816202&r1=816201&r2=816202&view=diff
==============================================================================
--- lucene/solr/trunk/src/java/org/apache/solr/search/function/IntFieldSource.java (original)
+++ lucene/solr/trunk/src/java/org/apache/solr/search/function/IntFieldSource.java Thu Sep 17 14:08:13 2009
@@ -22,6 +22,7 @@
 import org.apache.lucene.search.FieldCache;
 
 import java.io.IOException;
+import java.util.Map;
 
 /**
  * Obtains int field values from the {@link org.apache.lucene.search.FieldCache}
@@ -47,7 +48,7 @@
   }
 
 
-  public DocValues getValues(IndexReader reader) throws IOException {
+  public DocValues getValues(Map context, IndexReader reader) throws IOException {
     final int[] arr = (parser==null) ?
             cache.getInts(reader, field) :
             cache.getInts(reader, field, parser);

Modified: lucene/solr/trunk/src/java/org/apache/solr/search/function/LinearFloatFunction.java
URL: http://svn.apache.org/viewvc/lucene/solr/trunk/src/java/org/apache/solr/search/function/LinearFloatFunction.java?rev=816202&r1=816201&r2=816202&view=diff
==============================================================================
--- lucene/solr/trunk/src/java/org/apache/solr/search/function/LinearFloatFunction.java (original)
+++ lucene/solr/trunk/src/java/org/apache/solr/search/function/LinearFloatFunction.java Thu Sep 17 14:08:13 2009
@@ -18,8 +18,10 @@
 package org.apache.solr.search.function;
 
 import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.search.Searcher;
 
 import java.io.IOException;
+import java.util.Map;
 
 /**
  * <code>LinearFloatFunction</code> implements a linear function over
@@ -44,8 +46,8 @@
     return slope + "*float(" + source.description() + ")+" + intercept;
   }
 
-  public DocValues getValues(IndexReader reader) throws IOException {
-    final DocValues vals =  source.getValues(reader);
+  public DocValues getValues(Map context, IndexReader reader) throws IOException {
+    final DocValues vals =  source.getValues(context, reader);
     return new DocValues() {
       public float floatVal(int doc) {
         return vals.floatVal(doc) * slope + intercept;
@@ -68,6 +70,11 @@
     };
   }
 
+  @Override
+  public void createWeight(Map context, Searcher searcher) throws IOException {
+    source.createWeight(context, searcher);
+  }
+
   public int hashCode() {
     int h = Float.floatToIntBits(slope);
     h = (h >>> 2) | (h << 30);

Modified: lucene/solr/trunk/src/java/org/apache/solr/search/function/LongFieldSource.java
URL: http://svn.apache.org/viewvc/lucene/solr/trunk/src/java/org/apache/solr/search/function/LongFieldSource.java?rev=816202&r1=816201&r2=816202&view=diff
==============================================================================
--- lucene/solr/trunk/src/java/org/apache/solr/search/function/LongFieldSource.java (original)
+++ lucene/solr/trunk/src/java/org/apache/solr/search/function/LongFieldSource.java Thu Sep 17 14:08:13 2009
@@ -22,6 +22,7 @@
 
 
 import java.io.IOException;
+import java.util.Map;
 
 /**
  * Obtains float field values from the {@link org.apache.lucene.search.FieldCache}
@@ -52,7 +53,7 @@
     return Long.parseLong(extVal);
   }
 
-  public DocValues getValues(IndexReader reader) throws IOException {
+  public DocValues getValues(Map context, IndexReader reader) throws IOException {
     final long[] arr = (parser == null) ?
             ((FieldCache) cache).getLongs(reader, field) :
             ((FieldCache) cache).getLongs(reader, field, parser);

Modified: lucene/solr/trunk/src/java/org/apache/solr/search/function/MaxFloatFunction.java
URL: http://svn.apache.org/viewvc/lucene/solr/trunk/src/java/org/apache/solr/search/function/MaxFloatFunction.java?rev=816202&r1=816201&r2=816202&view=diff
==============================================================================
--- lucene/solr/trunk/src/java/org/apache/solr/search/function/MaxFloatFunction.java (original)
+++ lucene/solr/trunk/src/java/org/apache/solr/search/function/MaxFloatFunction.java Thu Sep 17 14:08:13 2009
@@ -18,8 +18,10 @@
 package org.apache.solr.search.function;
 
 import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.search.Searcher;
 
 import java.io.IOException;
+import java.util.Map;
 
 /**
  * Returns the max of a ValueSource and a float
@@ -43,8 +45,8 @@
     return "max(" + source.description() + "," + fval + ")";
   }
 
-  public DocValues getValues(IndexReader reader) throws IOException {
-    final DocValues vals =  source.getValues(reader);
+  public DocValues getValues(Map context, IndexReader reader) throws IOException {
+    final DocValues vals =  source.getValues(context, reader);
     return new DocValues() {
       public float floatVal(int doc) {
 	float v = vals.floatVal(doc);
@@ -68,6 +70,11 @@
     };
   }
 
+  @Override
+  public void createWeight(Map context, Searcher searcher) throws IOException {
+    source.createWeight(context, searcher);
+  }
+
   public int hashCode() {
     int h = Float.floatToIntBits(fval);
     h = (h >>> 2) | (h << 30);

Modified: lucene/solr/trunk/src/java/org/apache/solr/search/function/OrdFieldSource.java
URL: http://svn.apache.org/viewvc/lucene/solr/trunk/src/java/org/apache/solr/search/function/OrdFieldSource.java?rev=816202&r1=816201&r2=816202&view=diff
==============================================================================
--- lucene/solr/trunk/src/java/org/apache/solr/search/function/OrdFieldSource.java (original)
+++ lucene/solr/trunk/src/java/org/apache/solr/search/function/OrdFieldSource.java Thu Sep 17 14:08:13 2009
@@ -20,9 +20,9 @@
 import org.apache.lucene.index.IndexReader;
 import org.apache.solr.search.function.DocValues;
 import org.apache.solr.search.function.ValueSource;
-import org.apache.lucene.search.FieldCache;
 
 import java.io.IOException;
+import java.util.Map;
 
 /**
  * Obtains the ordinal of the field value from the default Lucene {@link org.apache.lucene.search.FieldCache} using getStringIndex().
@@ -51,7 +51,7 @@
   }
 
 
-  public DocValues getValues(IndexReader reader) throws IOException {
+  public DocValues getValues(Map context, IndexReader reader) throws IOException {
     return new StringIndexDocValues(this, reader, field) {
       protected String toTerm(String readableValue) {
         return readableValue;

Modified: lucene/solr/trunk/src/java/org/apache/solr/search/function/QueryValueSource.java
URL: http://svn.apache.org/viewvc/lucene/solr/trunk/src/java/org/apache/solr/search/function/QueryValueSource.java?rev=816202&r1=816201&r2=816202&view=diff
==============================================================================
--- lucene/solr/trunk/src/java/org/apache/solr/search/function/QueryValueSource.java (original)
+++ lucene/solr/trunk/src/java/org/apache/solr/search/function/QueryValueSource.java Thu Sep 17 14:08:13 2009
@@ -18,13 +18,11 @@
 package org.apache.solr.search.function;
 
 import org.apache.lucene.index.IndexReader;
-import org.apache.lucene.search.Query;
-import org.apache.lucene.search.IndexSearcher;
-import org.apache.lucene.search.Weight;
-import org.apache.lucene.search.Scorer;
+import org.apache.lucene.search.*;
 import org.apache.solr.common.SolrException;
 
 import java.io.IOException;
+import java.util.Map;
 
 /**
  * <code>QueryValueSource</code> returns the relevance score of the query
@@ -45,8 +43,9 @@
     return "query(" + q + ",def=" + defVal + ")";
   }
 
-  public DocValues getValues(IndexReader reader) throws IOException {
-    return new QueryDocValues(reader, q, defVal);
+  @Override
+  public DocValues getValues(Map context, IndexReader reader) throws IOException {
+    return new QueryDocValues(reader, q, defVal, context==null ? null : (Weight)context.get(this));
   }
 
   public int hashCode() {
@@ -58,13 +57,18 @@
     QueryValueSource other = (QueryValueSource)o;
     return this.q.equals(other.q) && this.defVal==other.defVal;
   }
+
+  @Override
+  public void createWeight(Map context, Searcher searcher) throws IOException {
+    Weight w = q.weight(searcher);
+    context.put(this, w);
+  }
 }
 
 
 class QueryDocValues extends DocValues {
   final Query q;
   final IndexReader reader;
-  final IndexSearcher searcher;
   final Weight weight;
   final float defVal;
 
@@ -75,12 +79,11 @@
   // to trigger a scorer reset on first access.
   int lastDocRequested=Integer.MAX_VALUE;
 
-  public QueryDocValues(IndexReader reader, Query q, float defVal) throws IOException {
+  public QueryDocValues(IndexReader reader, Query q, float defVal, Weight w) throws IOException {
     this.reader = reader;
     this.q = q;
     this.defVal = defVal;
-    searcher = new IndexSearcher(reader);
-    weight = q.weight(searcher);
+    weight = w!=null ? w : q.weight(new IndexSearcher(reader));
   }
 
   public float floatVal(int doc) {
@@ -88,12 +91,12 @@
       if (doc < lastDocRequested) {
         // out-of-order access.... reset scorer.
         scorer = weight.scorer(reader, true, false);
-        scorerDoc = scorer.nextDoc();
+        scorerDoc = -1;
       }
       lastDocRequested = doc;
 
       if (scorerDoc < doc) {
-        scorerDoc = scorer.nextDoc();
+        scorerDoc = scorer.advance(doc);
       }
 
       if (scorerDoc > doc) {

Modified: lucene/solr/trunk/src/java/org/apache/solr/search/function/RangeMapFloatFunction.java
URL: http://svn.apache.org/viewvc/lucene/solr/trunk/src/java/org/apache/solr/search/function/RangeMapFloatFunction.java?rev=816202&r1=816201&r2=816202&view=diff
==============================================================================
--- lucene/solr/trunk/src/java/org/apache/solr/search/function/RangeMapFloatFunction.java (original)
+++ lucene/solr/trunk/src/java/org/apache/solr/search/function/RangeMapFloatFunction.java Thu Sep 17 14:08:13 2009
@@ -18,8 +18,10 @@
 package org.apache.solr.search.function;
 
 import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.search.Searcher;
 
 import java.io.IOException;
+import java.util.Map;
 
 /**
  * <code>LinearFloatFunction</code> implements a linear function over
@@ -48,8 +50,8 @@
     return "map(" + source.description() + "," + min + "," + max + "," + target + ")";
   }
 
-  public DocValues getValues(IndexReader reader) throws IOException {
-    final DocValues vals =  source.getValues(reader);
+  public DocValues getValues(Map context, IndexReader reader) throws IOException {
+    final DocValues vals =  source.getValues(context, reader);
     return new DocValues() {
       public float floatVal(int doc) {
         float val = vals.floatVal(doc);
@@ -73,6 +75,11 @@
     };
   }
 
+  @Override
+  public void createWeight(Map context, Searcher searcher) throws IOException {
+    source.createWeight(context, searcher);
+  }
+
   public int hashCode() {
     int h = source.hashCode();
     h ^= (h << 10) | (h >>> 23);

Modified: lucene/solr/trunk/src/java/org/apache/solr/search/function/ReciprocalFloatFunction.java
URL: http://svn.apache.org/viewvc/lucene/solr/trunk/src/java/org/apache/solr/search/function/ReciprocalFloatFunction.java?rev=816202&r1=816201&r2=816202&view=diff
==============================================================================
--- lucene/solr/trunk/src/java/org/apache/solr/search/function/ReciprocalFloatFunction.java (original)
+++ lucene/solr/trunk/src/java/org/apache/solr/search/function/ReciprocalFloatFunction.java Thu Sep 17 14:08:13 2009
@@ -18,8 +18,10 @@
 package org.apache.solr.search.function;
 
 import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.search.Searcher;
 
 import java.io.IOException;
+import java.util.Map;
 
 /**
  * <code>ReciprocalFloatFunction</code> implements a reciprocal function f(x) = a/(mx+b), based on
@@ -52,8 +54,8 @@
     this.b=b;
   }
 
-  public DocValues getValues(IndexReader reader) throws IOException {
-    final DocValues vals = source.getValues(reader);
+  public DocValues getValues(Map context, IndexReader reader) throws IOException {
+    final DocValues vals = source.getValues(context, reader);
     return new DocValues() {
       public float floatVal(int doc) {
         return a/(m*vals.floatVal(doc) + b);
@@ -78,6 +80,11 @@
     };
   }
 
+  @Override
+  public void createWeight(Map context, Searcher searcher) throws IOException {
+    source.createWeight(context, searcher);
+  }
+
   public String description() {
     return Float.toString(a) + "/("
            + m + "*float(" + source.description() + ")"

Modified: lucene/solr/trunk/src/java/org/apache/solr/search/function/ReverseOrdFieldSource.java
URL: http://svn.apache.org/viewvc/lucene/solr/trunk/src/java/org/apache/solr/search/function/ReverseOrdFieldSource.java?rev=816202&r1=816201&r2=816202&view=diff
==============================================================================
--- lucene/solr/trunk/src/java/org/apache/solr/search/function/ReverseOrdFieldSource.java (original)
+++ lucene/solr/trunk/src/java/org/apache/solr/search/function/ReverseOrdFieldSource.java Thu Sep 17 14:08:13 2009
@@ -23,6 +23,7 @@
 import org.apache.lucene.search.FieldCache;
 
 import java.io.IOException;
+import java.util.Map;
 
 /**
  * Obtains the ordinal of the field value from the default Lucene {@link org.apache.lucene.search.FieldCache} using getStringIndex()
@@ -51,7 +52,7 @@
     return "rord("+field+')';
   }
 
-  public DocValues getValues(IndexReader reader) throws IOException {
+  public DocValues getValues(Map context, IndexReader reader) throws IOException {
     final FieldCache.StringIndex sindex = FieldCache.DEFAULT.getStringIndex(reader, field);
 
     final int arr[] = sindex.order;

Modified: lucene/solr/trunk/src/java/org/apache/solr/search/function/ScaleFloatFunction.java
URL: http://svn.apache.org/viewvc/lucene/solr/trunk/src/java/org/apache/solr/search/function/ScaleFloatFunction.java?rev=816202&r1=816201&r2=816202&view=diff
==============================================================================
--- lucene/solr/trunk/src/java/org/apache/solr/search/function/ScaleFloatFunction.java (original)
+++ lucene/solr/trunk/src/java/org/apache/solr/search/function/ScaleFloatFunction.java Thu Sep 17 14:08:13 2009
@@ -18,8 +18,10 @@
 package org.apache.solr.search.function;
 
 import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.search.Searcher;
 
 import java.io.IOException;
+import java.util.Map;
 
 /**
  * Scales values to be between min and max.
@@ -47,8 +49,8 @@
     return "scale(" + source.description() + "," + min + "," + max + ")";
   }
 
-  public DocValues getValues(IndexReader reader) throws IOException {
-    final DocValues vals =  source.getValues(reader);
+  public DocValues getValues(Map context, IndexReader reader) throws IOException {
+    final DocValues vals =  source.getValues(context, reader);
     int maxDoc = reader.maxDoc();
 
     // this doesn't take into account deleted docs!
@@ -107,6 +109,11 @@
     };
   }
 
+  @Override
+  public void createWeight(Map context, Searcher searcher) throws IOException {
+    source.createWeight(context, searcher);
+  }
+
   public int hashCode() {
     int h = Float.floatToIntBits(min);
     h = h*29;

Modified: lucene/solr/trunk/src/java/org/apache/solr/search/function/ShortFieldSource.java
URL: http://svn.apache.org/viewvc/lucene/solr/trunk/src/java/org/apache/solr/search/function/ShortFieldSource.java?rev=816202&r1=816201&r2=816202&view=diff
==============================================================================
--- lucene/solr/trunk/src/java/org/apache/solr/search/function/ShortFieldSource.java (original)
+++ lucene/solr/trunk/src/java/org/apache/solr/search/function/ShortFieldSource.java Thu Sep 17 14:08:13 2009
@@ -20,6 +20,7 @@
 import org.apache.lucene.index.IndexReader;
 
 import java.io.IOException;
+import java.util.Map;
 
 
 /**
@@ -42,7 +43,7 @@
     return "short(" + field + ')';
   }
 
-  public DocValues getValues(IndexReader reader) throws IOException {
+  public DocValues getValues(Map context, IndexReader reader) throws IOException {
     final short[] arr = (parser == null) ?
             cache.getShorts(reader, field) :
             cache.getShorts(reader, field, parser);

Modified: lucene/solr/trunk/src/java/org/apache/solr/search/function/SimpleFloatFunction.java
URL: http://svn.apache.org/viewvc/lucene/solr/trunk/src/java/org/apache/solr/search/function/SimpleFloatFunction.java?rev=816202&r1=816201&r2=816202&view=diff
==============================================================================
--- lucene/solr/trunk/src/java/org/apache/solr/search/function/SimpleFloatFunction.java (original)
+++ lucene/solr/trunk/src/java/org/apache/solr/search/function/SimpleFloatFunction.java Thu Sep 17 14:08:13 2009
@@ -20,25 +20,20 @@
 import org.apache.lucene.index.IndexReader;
 
 import java.io.IOException;
+import java.util.Map;
 
 /** A simple float function with a single argument
  */
- public abstract class SimpleFloatFunction extends ValueSource {
-  protected final ValueSource source;
-
+ public abstract class SimpleFloatFunction extends SingleFunction {
   public SimpleFloatFunction(ValueSource source) {
-    this.source = source;
+    super(source);
   }
 
-  protected abstract String name();
   protected abstract float func(int doc, DocValues vals);
 
-  public String description() {
-    return name() + '(' + source.description() + ')';
-  }
-
-  public DocValues getValues(IndexReader reader) throws IOException {
-    final DocValues vals =  source.getValues(reader);
+  @Override
+  public DocValues getValues(Map context, IndexReader reader) throws IOException {
+    final DocValues vals =  source.getValues(context, reader);
     return new DocValues() {
       public float floatVal(int doc) {
 	return func(doc, vals);
@@ -60,15 +55,4 @@
       }
     };
   }
-
-  public int hashCode() {
-    return source.hashCode() + name().hashCode();
-  }
-
-  public boolean equals(Object o) {
-    if (this.getClass() != o.getClass()) return false;
-    SimpleFloatFunction other = (SimpleFloatFunction)o;
-    return this.name().equals(other.name())
-         && this.source.equals(other.source);
-  }
 }

Added: lucene/solr/trunk/src/java/org/apache/solr/search/function/SingleFunction.java
URL: http://svn.apache.org/viewvc/lucene/solr/trunk/src/java/org/apache/solr/search/function/SingleFunction.java?rev=816202&view=auto
==============================================================================
--- lucene/solr/trunk/src/java/org/apache/solr/search/function/SingleFunction.java (added)
+++ lucene/solr/trunk/src/java/org/apache/solr/search/function/SingleFunction.java Thu Sep 17 14:08:13 2009
@@ -0,0 +1,55 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.solr.search.function;
+
+import org.apache.lucene.search.Searcher;
+
+import java.io.IOException;
+import java.util.Map;
+
+/** A function with a single argument
+ */
+ public abstract class SingleFunction extends ValueSource {
+  protected final ValueSource source;
+
+  public SingleFunction(ValueSource source) {
+    this.source = source;
+  }
+
+  protected abstract String name();
+
+  public String description() {
+    return name() + '(' + source.description() + ')';
+  }
+
+  public int hashCode() {
+    return source.hashCode() + name().hashCode();
+  }
+
+  public boolean equals(Object o) {
+    if (this.getClass() != o.getClass()) return false;
+    SingleFunction other = (SingleFunction)o;
+    return this.name().equals(other.name())
+         && this.source.equals(other.source);
+  }
+
+  @Override
+  public void createWeight(Map context, Searcher searcher) throws IOException {
+    source.createWeight(context, searcher);
+  }
+}
\ No newline at end of file

Propchange: lucene/solr/trunk/src/java/org/apache/solr/search/function/SingleFunction.java
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: lucene/solr/trunk/src/java/org/apache/solr/search/function/SingleFunction.java
------------------------------------------------------------------------------
    svn:executable = *

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

Modified: lucene/solr/trunk/src/java/org/apache/solr/search/function/SumFloatFunction.java
URL: http://svn.apache.org/viewvc/lucene/solr/trunk/src/java/org/apache/solr/search/function/SumFloatFunction.java?rev=816202&r1=816201&r2=816202&view=diff
==============================================================================
--- lucene/solr/trunk/src/java/org/apache/solr/search/function/SumFloatFunction.java (original)
+++ lucene/solr/trunk/src/java/org/apache/solr/search/function/SumFloatFunction.java Thu Sep 17 14:08:13 2009
@@ -18,9 +18,11 @@
 package org.apache.solr.search.function;
 
 import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.search.Searcher;
 
 import java.io.IOException;
 import java.util.Arrays;
+import java.util.Map;
 
 /**
  * <code>SumFloatFunction</code> returns the sum of it's components.
@@ -71,10 +73,10 @@
     return sb.toString();
   }
 
-  public DocValues getValues(IndexReader reader) throws IOException {
+  public DocValues getValues(Map context, IndexReader reader) throws IOException {
     final DocValues[] valsArr = new DocValues[sources.length];
     for (int i=0; i<sources.length; i++) {
-      valsArr[i] = sources[i].getValues(reader);
+      valsArr[i] = sources[i].getValues(context, reader);
     }
 
     return new DocValues() {
@@ -111,6 +113,12 @@
     };
   }
 
+  @Override
+  public void createWeight(Map context, Searcher searcher) throws IOException {
+    for (ValueSource source : sources)
+      source.createWeight(context, searcher);
+  }
+
   public int hashCode() {
     return Arrays.hashCode(sources) + name().hashCode();
   }

Modified: lucene/solr/trunk/src/java/org/apache/solr/search/function/TopValueSource.java
URL: http://svn.apache.org/viewvc/lucene/solr/trunk/src/java/org/apache/solr/search/function/TopValueSource.java?rev=816202&r1=816201&r2=816202&view=diff
==============================================================================
--- lucene/solr/trunk/src/java/org/apache/solr/search/function/TopValueSource.java (original)
+++ lucene/solr/trunk/src/java/org/apache/solr/search/function/TopValueSource.java Thu Sep 17 14:08:13 2009
@@ -20,6 +20,7 @@
 import org.apache.lucene.index.IndexReader;
 
 import java.io.IOException;
+import java.util.Map;
 
 /**
  * A value source that wraps another and ensures that the top level reader
@@ -41,7 +42,7 @@
     return "top(" + vs.description() + ')';
   }
 
-  public DocValues getValues(IndexReader reader) throws IOException {
+  public DocValues getValues(Map context, IndexReader reader) throws IOException {
     int offset = 0;
     IndexReader topReader = reader;
     if (topReader instanceof SolrIndexReader) {
@@ -53,7 +54,7 @@
       topReader = r;
     }
     final int off = offset;
-    final DocValues vals = vs.getValues(topReader);
+    final DocValues vals = vs.getValues(context, topReader);
     if (topReader == reader) return vals;
 
     return new DocValues() {

Modified: lucene/solr/trunk/src/java/org/apache/solr/search/function/ValueSource.java
URL: http://svn.apache.org/viewvc/lucene/solr/trunk/src/java/org/apache/solr/search/function/ValueSource.java?rev=816202&r1=816201&r2=816202&view=diff
==============================================================================
--- lucene/solr/trunk/src/java/org/apache/solr/search/function/ValueSource.java (original)
+++ lucene/solr/trunk/src/java/org/apache/solr/search/function/ValueSource.java Thu Sep 17 14:08:13 2009
@@ -23,6 +23,8 @@
 
 import java.io.IOException;
 import java.io.Serializable;
+import java.util.IdentityHashMap;
+import java.util.Map;
 
 /**
  * Instantiates {@link org.apache.solr.search.function.DocValues} for a particular reader.
@@ -33,7 +35,16 @@
  */
 public abstract class ValueSource implements Serializable {
 
-  public abstract DocValues getValues(IndexReader reader) throws IOException;
+  public DocValues getValues(IndexReader reader) throws IOException {
+    return getValues(null, reader);
+  }
+
+  /** Gets the values for this reader and the context that was previously
+   * passed to createWeight()
+   */
+  public DocValues getValues(Map context, IndexReader reader) throws IOException {
+    return null;
+  }
 
   public abstract boolean equals(Object o);
 
@@ -46,6 +57,16 @@
     return description();
   }
 
+  /** Implementations should propagate createWeight to sub-ValueSources which can optionally store
+   * weight info in the context. The context object will be passed to getValues()
+   * where this info can be retrieved. */
+  public void createWeight(Map context, Searcher searcher) throws IOException {
+  }
+
+  /** Returns a new non-threadsafe context map. */
+  public static Map newContext() {
+    return new IdentityHashMap();
+  }
 }
 
 

Modified: lucene/solr/trunk/src/java/org/apache/solr/search/function/ValueSourceRangeFilter.java
URL: http://svn.apache.org/viewvc/lucene/solr/trunk/src/java/org/apache/solr/search/function/ValueSourceRangeFilter.java?rev=816202&r1=816201&r2=816202&view=diff
==============================================================================
--- lucene/solr/trunk/src/java/org/apache/solr/search/function/ValueSourceRangeFilter.java (original)
+++ lucene/solr/trunk/src/java/org/apache/solr/search/function/ValueSourceRangeFilter.java Thu Sep 17 14:08:13 2009
@@ -20,15 +20,18 @@
 import org.apache.lucene.search.Filter;
 import org.apache.lucene.search.DocIdSet;
 import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.Searcher;
 import org.apache.lucene.index.IndexReader;
+import org.apache.solr.search.SolrFilter;
 
 import java.io.IOException;
+import java.util.Map;
 
 
 /**
  * RangeFilter over a ValueSource.
  */
-public class ValueSourceRangeFilter extends Filter {
+public class ValueSourceRangeFilter extends SolrFilter {
   private final ValueSource valueSource;
   private final String lowerVal;
   private final String upperVal;
@@ -47,14 +50,19 @@
     this.includeUpper = upperVal != null && includeUpper;
   }
 
-  public DocIdSet getDocIdSet(final IndexReader reader) throws IOException {
+  public DocIdSet getDocIdSet(final Map context, final IndexReader reader) throws IOException {
      return new DocIdSet() {
        public DocIdSetIterator iterator() throws IOException {
-         return valueSource.getValues(reader).getRangeScorer(reader, lowerVal, upperVal, includeLower, includeUpper);
+         return valueSource.getValues(context, reader).getRangeScorer(reader, lowerVal, upperVal, includeLower, includeUpper);
        }
      };
   }
 
+  @Override
+  public void createWeight(Map context, Searcher searcher) throws IOException {
+    valueSource.createWeight(context, searcher);
+  }
+
   public String toString() {
     StringBuilder sb = new StringBuilder();
     sb.append("frange(");

Modified: lucene/solr/trunk/src/test/org/apache/solr/search/function/TestFunctionQuery.java
URL: http://svn.apache.org/viewvc/lucene/solr/trunk/src/test/org/apache/solr/search/function/TestFunctionQuery.java?rev=816202&r1=816201&r2=816202&view=diff
==============================================================================
--- lucene/solr/trunk/src/test/org/apache/solr/search/function/TestFunctionQuery.java (original)
+++ lucene/solr/trunk/src/test/org/apache/solr/search/function/TestFunctionQuery.java Thu Sep 17 14:08:13 2009
@@ -201,6 +201,7 @@
   public void testFunctions() {
     doTest("foo_pf");  // a plain float field
     doTest("foo_f");  // a sortable float field
+    doTest("foo_tf");  // a trie float field
   }
 
   public void testExternalField() {
@@ -304,5 +305,26 @@
     assertQ(req("fl","*,score","q", "{!func}ms(2009-08-31T12:10:10.125Z,b_tdt)", "fq","id:1"), "//float[@name='score']='1.0'");
 
     assertQ(req("fl","*,score","q", "{!func}ms(2009-08-31T12:10:10.125Z/SECOND,2009-08-31T12:10:10.124Z/SECOND)", "fq","id:1"), "//float[@name='score']='0.0'");
+
+    for (int i=100; i<112; i++) {
+      assertU(adoc("id",""+i, "text","batman"));
+    }
+    assertU(commit());
+    assertU(adoc("id","120", "text","batman superman"));   // in a segment by itself
+    assertU(commit());
+
+    // batman and superman have the same idf in single-doc segment, but very different in the complete index.
+    String q ="{!func}query($qq)";
+    String fq="id:120"; 
+    assertQ(req("fl","*,score","q", q, "qq","text:batman", "fq",fq), "//float[@name='score']<'1.0'");
+    assertQ(req("fl","*,score","q", q, "qq","text:superman", "fq",fq), "//float[@name='score']>'1.0'");
+
+    // test weighting through a function range query
+    assertQ(req("fl","*,score", "q", "{!frange l=1 u=10}query($qq)", "qq","text:superman"), "//*[@numFound='1']");
+
+    // test weighting through a complex function
+    q ="{!func}sub(div(sum(0.0,product(1,query($qq))),1),0)";
+    assertQ(req("fl","*,score","q", q, "qq","text:batman", "fq",fq), "//float[@name='score']<'1.0'");
+    assertQ(req("fl","*,score","q", q, "qq","text:superman", "fq",fq), "//float[@name='score']>'1.0'");
   }
 }
\ No newline at end of file



Re: svn commit: r816202 - in /lucene/solr/trunk/src: java/org/apache/solr/schema/ java/org/apache/solr/search/ java/org/apache/solr/search/function/ test/org/apache/solr/search/function/

Posted by Yonik Seeley <yo...@lucidimagination.com>.
2009/9/17 Noble Paul നോബിള്‍  नोब्ळ् <no...@corp.aol.com>:
> do we have some type info for the context param in
> SolrFilter#createWeight(Map context, Searcher searcher)

Nope... it's specifically opaque so we don't have to change it down
the road or force the creation of custom weight classes just to store
extra info, or force the creation of a fake/custom ValueSorce just to
use a different key.

-Yonik
http://www.lucidimagination.com

Re: svn commit: r816202 - in /lucene/solr/trunk/src: java/org/apache/solr/schema/ java/org/apache/solr/search/ java/org/apache/solr/search/function/ test/org/apache/solr/search/function/

Posted by Noble Paul നോബിള്‍ नोब्ळ् <no...@corp.aol.com>.
do we have some type info for the context param in
SolrFilter#createWeight(Map context, Searcher searcher)






On Thu, Sep 17, 2009 at 7:38 PM,  <yo...@apache.org> wrote:
> Author: yonik
> Date: Thu Sep 17 14:08:13 2009
> New Revision: 816202
>
> URL: http://svn.apache.org/viewvc?rev=816202&view=rev
> Log:
> SOLR-1432: weight nested queries in function queries
>
> Added:
>    lucene/solr/trunk/src/java/org/apache/solr/search/SolrConstantScoreQuery.java   (with props)
>    lucene/solr/trunk/src/java/org/apache/solr/search/SolrFilter.java
>    lucene/solr/trunk/src/java/org/apache/solr/search/function/SingleFunction.java   (with props)
> Modified:
>    lucene/solr/trunk/src/java/org/apache/solr/schema/DateField.java
>    lucene/solr/trunk/src/java/org/apache/solr/schema/RandomSortField.java
>    lucene/solr/trunk/src/java/org/apache/solr/schema/SortableDoubleField.java
>    lucene/solr/trunk/src/java/org/apache/solr/schema/SortableFloatField.java
>    lucene/solr/trunk/src/java/org/apache/solr/schema/SortableIntField.java
>    lucene/solr/trunk/src/java/org/apache/solr/schema/SortableLongField.java
>    lucene/solr/trunk/src/java/org/apache/solr/schema/StrField.java
>    lucene/solr/trunk/src/java/org/apache/solr/search/FunctionRangeQParserPlugin.java
>    lucene/solr/trunk/src/java/org/apache/solr/search/QueryParsing.java
>    lucene/solr/trunk/src/java/org/apache/solr/search/ValueSourceParser.java
>    lucene/solr/trunk/src/java/org/apache/solr/search/function/BoostedQuery.java
>    lucene/solr/trunk/src/java/org/apache/solr/search/function/ByteFieldSource.java
>    lucene/solr/trunk/src/java/org/apache/solr/search/function/ConstValueSource.java
>    lucene/solr/trunk/src/java/org/apache/solr/search/function/DoubleFieldSource.java
>    lucene/solr/trunk/src/java/org/apache/solr/search/function/DualFloatFunction.java
>    lucene/solr/trunk/src/java/org/apache/solr/search/function/FileFloatSource.java
>    lucene/solr/trunk/src/java/org/apache/solr/search/function/FloatFieldSource.java
>    lucene/solr/trunk/src/java/org/apache/solr/search/function/FunctionQuery.java
>    lucene/solr/trunk/src/java/org/apache/solr/search/function/IntFieldSource.java
>    lucene/solr/trunk/src/java/org/apache/solr/search/function/LinearFloatFunction.java
>    lucene/solr/trunk/src/java/org/apache/solr/search/function/LongFieldSource.java
>    lucene/solr/trunk/src/java/org/apache/solr/search/function/MaxFloatFunction.java
>    lucene/solr/trunk/src/java/org/apache/solr/search/function/OrdFieldSource.java
>    lucene/solr/trunk/src/java/org/apache/solr/search/function/QueryValueSource.java
>    lucene/solr/trunk/src/java/org/apache/solr/search/function/RangeMapFloatFunction.java
>    lucene/solr/trunk/src/java/org/apache/solr/search/function/ReciprocalFloatFunction.java
>    lucene/solr/trunk/src/java/org/apache/solr/search/function/ReverseOrdFieldSource.java
>    lucene/solr/trunk/src/java/org/apache/solr/search/function/ScaleFloatFunction.java
>    lucene/solr/trunk/src/java/org/apache/solr/search/function/ShortFieldSource.java
>    lucene/solr/trunk/src/java/org/apache/solr/search/function/SimpleFloatFunction.java
>    lucene/solr/trunk/src/java/org/apache/solr/search/function/SumFloatFunction.java
>    lucene/solr/trunk/src/java/org/apache/solr/search/function/TopValueSource.java
>    lucene/solr/trunk/src/java/org/apache/solr/search/function/ValueSource.java
>    lucene/solr/trunk/src/java/org/apache/solr/search/function/ValueSourceRangeFilter.java
>    lucene/solr/trunk/src/test/org/apache/solr/search/function/TestFunctionQuery.java
>
> Modified: lucene/solr/trunk/src/java/org/apache/solr/schema/DateField.java
> URL: http://svn.apache.org/viewvc/lucene/solr/trunk/src/java/org/apache/solr/schema/DateField.java?rev=816202&r1=816201&r2=816202&view=diff
> ==============================================================================
> --- lucene/solr/trunk/src/java/org/apache/solr/schema/DateField.java (original)
> +++ lucene/solr/trunk/src/java/org/apache/solr/schema/DateField.java Thu Sep 17 14:08:13 2009
> @@ -33,10 +33,7 @@
>
>  import java.io.IOException;
>  import java.text.*;
> -import java.util.Date;
> -import java.util.Locale;
> -import java.util.Map;
> -import java.util.TimeZone;
> +import java.util.*;
>
>  // TODO: make a FlexibleDateField that can accept dates in multiple
>  // formats, better for human entered dates.
> @@ -425,7 +422,7 @@
>     return "date(" + field + ')';
>   }
>
> -  public DocValues getValues(IndexReader reader) throws IOException {
> +  public DocValues getValues(Map context, IndexReader reader) throws IOException {
>     return new StringIndexDocValues(this, reader, field) {
>       protected String toTerm(String readableValue) {
>         // needed for frange queries to work properly
>
> Modified: lucene/solr/trunk/src/java/org/apache/solr/schema/RandomSortField.java
> URL: http://svn.apache.org/viewvc/lucene/solr/trunk/src/java/org/apache/solr/schema/RandomSortField.java?rev=816202&r1=816201&r2=816202&view=diff
> ==============================================================================
> --- lucene/solr/trunk/src/java/org/apache/solr/schema/RandomSortField.java (original)
> +++ lucene/solr/trunk/src/java/org/apache/solr/schema/RandomSortField.java Thu Sep 17 14:08:13 2009
> @@ -160,7 +160,7 @@
>     }
>
>     @Override
> -    public DocValues getValues(final IndexReader reader) throws IOException {
> +    public DocValues getValues(Map context, final IndexReader reader) throws IOException {
>       return new DocValues() {
>           private final int seed = getSeed(field, reader);
>           @Override
>
> Modified: lucene/solr/trunk/src/java/org/apache/solr/schema/SortableDoubleField.java
> URL: http://svn.apache.org/viewvc/lucene/solr/trunk/src/java/org/apache/solr/schema/SortableDoubleField.java?rev=816202&r1=816201&r2=816202&view=diff
> ==============================================================================
> --- lucene/solr/trunk/src/java/org/apache/solr/schema/SortableDoubleField.java (original)
> +++ lucene/solr/trunk/src/java/org/apache/solr/schema/SortableDoubleField.java Thu Sep 17 14:08:13 2009
> @@ -18,7 +18,6 @@
>  package org.apache.solr.schema;
>
>  import org.apache.lucene.search.SortField;
> -import org.apache.lucene.search.FieldCache;
>  import org.apache.solr.search.function.ValueSource;
>  import org.apache.solr.search.function.FieldCacheSource;
>  import org.apache.solr.search.function.DocValues;
> @@ -93,7 +92,7 @@
>     return "sdouble(" + field + ')';
>   }
>
> -  public DocValues getValues(IndexReader reader) throws IOException {
> +  public DocValues getValues(Map context, IndexReader reader) throws IOException {
>     final double def = defVal;
>
>     return new StringIndexDocValues(this, reader, field) {
>
> Modified: lucene/solr/trunk/src/java/org/apache/solr/schema/SortableFloatField.java
> URL: http://svn.apache.org/viewvc/lucene/solr/trunk/src/java/org/apache/solr/schema/SortableFloatField.java?rev=816202&r1=816201&r2=816202&view=diff
> ==============================================================================
> --- lucene/solr/trunk/src/java/org/apache/solr/schema/SortableFloatField.java (original)
> +++ lucene/solr/trunk/src/java/org/apache/solr/schema/SortableFloatField.java Thu Sep 17 14:08:13 2009
> @@ -18,7 +18,6 @@
>  package org.apache.solr.schema;
>
>  import org.apache.lucene.search.SortField;
> -import org.apache.lucene.search.FieldCache;
>  import org.apache.solr.search.function.ValueSource;
>  import org.apache.solr.search.function.FieldCacheSource;
>  import org.apache.solr.search.function.DocValues;
> @@ -93,7 +92,7 @@
>     return "sfloat(" + field + ')';
>   }
>
> -  public DocValues getValues(IndexReader reader) throws IOException {
> +  public DocValues getValues(Map context, IndexReader reader) throws IOException {
>     final float def = defVal;
>
>     return new StringIndexDocValues(this, reader, field) {
>
> Modified: lucene/solr/trunk/src/java/org/apache/solr/schema/SortableIntField.java
> URL: http://svn.apache.org/viewvc/lucene/solr/trunk/src/java/org/apache/solr/schema/SortableIntField.java?rev=816202&r1=816201&r2=816202&view=diff
> ==============================================================================
> --- lucene/solr/trunk/src/java/org/apache/solr/schema/SortableIntField.java (original)
> +++ lucene/solr/trunk/src/java/org/apache/solr/schema/SortableIntField.java Thu Sep 17 14:08:13 2009
> @@ -18,7 +18,6 @@
>  package org.apache.solr.schema;
>
>  import org.apache.lucene.search.SortField;
> -import org.apache.lucene.search.FieldCache;
>  import org.apache.solr.search.function.ValueSource;
>  import org.apache.solr.search.function.FieldCacheSource;
>  import org.apache.solr.search.function.DocValues;
> @@ -97,7 +96,7 @@
>     return "sint(" + field + ')';
>   }
>
> -  public DocValues getValues(IndexReader reader) throws IOException {
> +  public DocValues getValues(Map context, IndexReader reader) throws IOException {
>     final int def = defVal;
>
>     return new StringIndexDocValues(this, reader, field) {
>
> Modified: lucene/solr/trunk/src/java/org/apache/solr/schema/SortableLongField.java
> URL: http://svn.apache.org/viewvc/lucene/solr/trunk/src/java/org/apache/solr/schema/SortableLongField.java?rev=816202&r1=816201&r2=816202&view=diff
> ==============================================================================
> --- lucene/solr/trunk/src/java/org/apache/solr/schema/SortableLongField.java (original)
> +++ lucene/solr/trunk/src/java/org/apache/solr/schema/SortableLongField.java Thu Sep 17 14:08:13 2009
> @@ -18,7 +18,6 @@
>  package org.apache.solr.schema;
>
>  import org.apache.lucene.search.SortField;
> -import org.apache.lucene.search.FieldCache;
>  import org.apache.solr.search.function.ValueSource;
>  import org.apache.solr.search.function.FieldCacheSource;
>  import org.apache.solr.search.function.DocValues;
> @@ -94,7 +93,7 @@
>     return "slong(" + field + ')';
>   }
>
> -  public DocValues getValues(IndexReader reader) throws IOException {
> +  public DocValues getValues(Map context, IndexReader reader) throws IOException {
>     final long def = defVal;
>
>     return new StringIndexDocValues(this, reader, field) {
>
> Modified: lucene/solr/trunk/src/java/org/apache/solr/schema/StrField.java
> URL: http://svn.apache.org/viewvc/lucene/solr/trunk/src/java/org/apache/solr/schema/StrField.java?rev=816202&r1=816201&r2=816202&view=diff
> ==============================================================================
> --- lucene/solr/trunk/src/java/org/apache/solr/schema/StrField.java (original)
> +++ lucene/solr/trunk/src/java/org/apache/solr/schema/StrField.java Thu Sep 17 14:08:13 2009
> @@ -27,7 +27,6 @@
>  import org.apache.solr.search.function.DocValues;
>  import org.apache.solr.search.function.StringIndexDocValues;
>  import org.apache.solr.search.QParser;
> -import org.apache.solr.util.NumberUtils;
>
>  import java.util.Map;
>  import java.io.IOException;
> @@ -67,7 +66,7 @@
>     return "str(" + field + ')';
>   }
>
> -  public DocValues getValues(IndexReader reader) throws IOException {
> +  public DocValues getValues(Map context, IndexReader reader) throws IOException {
>     return new StringIndexDocValues(this, reader, field) {
>       protected String toTerm(String readableValue) {
>         return readableValue;
>
> Modified: lucene/solr/trunk/src/java/org/apache/solr/search/FunctionRangeQParserPlugin.java
> URL: http://svn.apache.org/viewvc/lucene/solr/trunk/src/java/org/apache/solr/search/FunctionRangeQParserPlugin.java?rev=816202&r1=816201&r2=816202&view=diff
> ==============================================================================
> --- lucene/solr/trunk/src/java/org/apache/solr/search/FunctionRangeQParserPlugin.java (original)
> +++ lucene/solr/trunk/src/java/org/apache/solr/search/FunctionRangeQParserPlugin.java Thu Sep 17 14:08:13 2009
> @@ -19,6 +19,7 @@
>  import org.apache.lucene.queryParser.ParseException;
>  import org.apache.lucene.search.Query;
>  import org.apache.lucene.search.ConstantScoreQuery;
> +import org.apache.lucene.search.Filter;
>  import org.apache.solr.common.params.SolrParams;
>  import org.apache.solr.common.util.NamedList;
>  import org.apache.solr.request.SolrQueryRequest;
> @@ -60,10 +61,19 @@
>
>         // TODO: add a score=val option to allow score to be the value
>         ValueSourceRangeFilter rf = new ValueSourceRangeFilter(vs, l, u, includeLower, includeUpper);
> -        ConstantScoreQuery csq = new ConstantScoreQuery(rf);
> +        SolrConstantScoreQuery csq = new SolrConstantScoreQuery(rf);
>         return csq;
>       }
>     };
>   }
>
>  }
> +
> +class FunctionConstantScoreQuery extends ConstantScoreQuery {
> +  public FunctionConstantScoreQuery(Filter filter) {
> +    super(filter);
> +  }
> +
> +
> +
> +}
> \ No newline at end of file
>
> Modified: lucene/solr/trunk/src/java/org/apache/solr/search/QueryParsing.java
> URL: http://svn.apache.org/viewvc/lucene/solr/trunk/src/java/org/apache/solr/search/QueryParsing.java?rev=816202&r1=816201&r2=816202&view=diff
> ==============================================================================
> --- lucene/solr/trunk/src/java/org/apache/solr/search/QueryParsing.java (original)
> +++ lucene/solr/trunk/src/java/org/apache/solr/search/QueryParsing.java Thu Sep 17 14:08:13 2009
> @@ -17,7 +17,6 @@
>
>  package org.apache.solr.search;
>
> -import org.apache.lucene.document.Field;
>  import org.apache.lucene.index.Term;
>  import org.apache.lucene.queryParser.ParseException;
>  import org.apache.lucene.queryParser.QueryParser;
> @@ -436,7 +435,7 @@
>     } else if (query instanceof FuzzyQuery) {
>       out.append(query.toString());
>       writeBoost=false;
> -    } else if (query instanceof ConstantScoreQuery) {
> +    } else if (query instanceof SolrConstantScoreQuery) {
>       out.append(query.toString());
>       writeBoost=false;
>     } else {
>
> Added: lucene/solr/trunk/src/java/org/apache/solr/search/SolrConstantScoreQuery.java
> URL: http://svn.apache.org/viewvc/lucene/solr/trunk/src/java/org/apache/solr/search/SolrConstantScoreQuery.java?rev=816202&view=auto
> ==============================================================================
> --- lucene/solr/trunk/src/java/org/apache/solr/search/SolrConstantScoreQuery.java (added)
> +++ lucene/solr/trunk/src/java/org/apache/solr/search/SolrConstantScoreQuery.java Thu Sep 17 14:08:13 2009
> @@ -0,0 +1,201 @@
> +package org.apache.solr.search;
> +
> +import org.apache.lucene.search.*;
> +import org.apache.lucene.index.IndexReader;
> +import org.apache.solr.search.function.ValueSource;
> +import org.apache.solr.common.SolrException;
> +
> +import java.io.IOException;
> +import java.util.Set;
> +import java.util.Map;
> +
> +/**
> + * Licensed to the Apache Software Foundation (ASF) under one or more
> + * contributor license agreements.  See the NOTICE file distributed with
> + * this work for additional information regarding copyright ownership.
> + * The ASF licenses this file to You under the Apache License, Version 2.0
> + * (the "License"); you may not use this file except in compliance with
> + * the License.  You may obtain a copy of the License at
> + *
> + *     http://www.apache.org/licenses/LICENSE-2.0
> + *
> + * Unless required by applicable law or agreed to in writing, software
> + * distributed under the License is distributed on an "AS IS" BASIS,
> + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
> + * See the License for the specific language governing permissions and
> + * limitations under the License.
> + */
> +
> +/**
> + * A query that wraps a filter and simply returns a constant score equal to the
> + * query boost for every document in the filter.   This Solr extension also supports
> + * weighting of a SolrFilter.
> + *
> + * Experimental and subject to change.
> + */
> +public class SolrConstantScoreQuery extends ConstantScoreQuery {
> +
> +  public SolrConstantScoreQuery(Filter filter) {
> +    super(filter);
> +  }
> +
> +  /** Returns the encapsulated filter */
> +  public Filter getFilter() {
> +    return filter;
> +  }
> +
> +  public Query rewrite(IndexReader reader) throws IOException {
> +    return this;
> +  }
> +
> +  public void extractTerms(Set terms) {
> +    // OK to not add any terms when used for MultiSearcher,
> +    // but may not be OK for highlighting
> +  }
> +
> +  protected class ConstantWeight extends Weight {
> +    private Similarity similarity;
> +    private float queryNorm;
> +    private float queryWeight;
> +    private Map context;
> +
> +    public ConstantWeight(Searcher searcher) throws IOException {
> +      this.similarity = getSimilarity(searcher);
> +      this.context = ValueSource.newContext();
> +      if (filter instanceof SolrFilter)
> +        ((SolrFilter)filter).createWeight(context, searcher);
> +    }
> +
> +    public Query getQuery() {
> +      return SolrConstantScoreQuery.this;
> +    }
> +
> +    public float getValue() {
> +      return queryWeight;
> +    }
> +
> +    public float sumOfSquaredWeights() throws IOException {
> +      queryWeight = getBoost();
> +      return queryWeight * queryWeight;
> +    }
> +
> +    public void normalize(float norm) {
> +      this.queryNorm = norm;
> +      queryWeight *= this.queryNorm;
> +    }
> +
> +    public Scorer scorer(IndexReader reader, boolean scoreDocsInOrder, boolean topScorer) throws IOException {
> +      return new ConstantScorer(similarity, reader, this);
> +    }
> +
> +    public Explanation explain(IndexReader reader, int doc) throws IOException {
> +
> +      ConstantScorer cs = new ConstantScorer(similarity, reader, this);
> +      boolean exists = cs.docIdSetIterator.advance(doc) == doc;
> +
> +      ComplexExplanation result = new ComplexExplanation();
> +
> +      if (exists) {
> +        result.setDescription("ConstantScoreQuery(" + filter
> +        + "), product of:");
> +        result.setValue(queryWeight);
> +        result.setMatch(Boolean.TRUE);
> +        result.addDetail(new Explanation(getBoost(), "boost"));
> +        result.addDetail(new Explanation(queryNorm,"queryNorm"));
> +      } else {
> +        result.setDescription("ConstantScoreQuery(" + filter
> +        + ") doesn't match id " + doc);
> +        result.setValue(0);
> +        result.setMatch(Boolean.FALSE);
> +      }
> +      return result;
> +    }
> +  }
> +
> +  protected class ConstantScorer extends Scorer {
> +    final DocIdSetIterator docIdSetIterator;
> +    final float theScore;
> +    int doc = -1;
> +
> +    public ConstantScorer(Similarity similarity, IndexReader reader, ConstantWeight w) throws IOException {
> +      super(similarity);
> +      theScore = w.getValue();
> +      DocIdSet docIdSet = filter instanceof SolrFilter ? ((SolrFilter)filter).getDocIdSet(w.context, reader) : filter.getDocIdSet(reader);
> +      if (docIdSet == null) {
> +        docIdSetIterator = DocIdSet.EMPTY_DOCIDSET.iterator();
> +      } else {
> +        DocIdSetIterator iter = docIdSet.iterator();
> +        if (iter == null) {
> +          docIdSetIterator = DocIdSet.EMPTY_DOCIDSET.iterator();
> +        } else {
> +          docIdSetIterator = iter;
> +        }
> +      }
> +    }
> +
> +    /** @deprecated use {@link #nextDoc()} instead. */
> +    public boolean next() throws IOException {
> +      return docIdSetIterator.nextDoc() != NO_MORE_DOCS;
> +    }
> +
> +    public int nextDoc() throws IOException {
> +      return docIdSetIterator.nextDoc();
> +    }
> +
> +    /** @deprecated use {@link #docID()} instead. */
> +    public int doc() {
> +      return docIdSetIterator.doc();
> +    }
> +
> +    public int docID() {
> +      return docIdSetIterator.docID();
> +    }
> +
> +    public float score() throws IOException {
> +      return theScore;
> +    }
> +
> +    /** @deprecated use {@link #advance(int)} instead. */
> +    public boolean skipTo(int target) throws IOException {
> +      return docIdSetIterator.advance(target) != NO_MORE_DOCS;
> +    }
> +
> +    public int advance(int target) throws IOException {
> +      return docIdSetIterator.advance(target);
> +    }
> +
> +    public Explanation explain(int doc) throws IOException {
> +      throw new UnsupportedOperationException();
> +    }
> +  }
> +
> +  public Weight createWeight(Searcher searcher) {
> +    try {
> +      return new SolrConstantScoreQuery.ConstantWeight(searcher);
> +    } catch (IOException e) {
> +      // TODO: remove this if ConstantScoreQuery.createWeight adds IOException
> +      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
> +    }
> +  }
> +
> +  /** Prints a user-readable version of this query. */
> +  public String toString(String field) {
> +    return "ConstantScore(" + filter.toString()
> +      + (getBoost()==1.0 ? ")" : "^" + getBoost());
> +  }
> +
> +  /** Returns true if <code>o</code> is equal to this. */
> +  public boolean equals(Object o) {
> +    if (this == o) return true;
> +    if (!(o instanceof SolrConstantScoreQuery)) return false;
> +    SolrConstantScoreQuery other = (SolrConstantScoreQuery)o;
> +    return this.getBoost()==other.getBoost() && filter.equals(other.filter);
> +  }
> +
> +  /** Returns a hash code value for this object. */
> +  public int hashCode() {
> +    // Simple add is OK since no existing filter hashcode has a float component.
> +    return filter.hashCode() + Float.floatToIntBits(getBoost());
> +  }
> +
> +}
>
> Propchange: lucene/solr/trunk/src/java/org/apache/solr/search/SolrConstantScoreQuery.java
> ------------------------------------------------------------------------------
>    svn:eol-style = native
>
> Propchange: lucene/solr/trunk/src/java/org/apache/solr/search/SolrConstantScoreQuery.java
> ------------------------------------------------------------------------------
>    svn:executable = *
>
> Propchange: lucene/solr/trunk/src/java/org/apache/solr/search/SolrConstantScoreQuery.java
> ------------------------------------------------------------------------------
>    svn:keywords = Date Author Id Revision HeadURL
>
> Added: lucene/solr/trunk/src/java/org/apache/solr/search/SolrFilter.java
> URL: http://svn.apache.org/viewvc/lucene/solr/trunk/src/java/org/apache/solr/search/SolrFilter.java?rev=816202&view=auto
> ==============================================================================
> --- lucene/solr/trunk/src/java/org/apache/solr/search/SolrFilter.java (added)
> +++ lucene/solr/trunk/src/java/org/apache/solr/search/SolrFilter.java Thu Sep 17 14:08:13 2009
> @@ -0,0 +1,46 @@
> +/**
> + * Licensed to the Apache Software Foundation (ASF) under one or more
> + * contributor license agreements.  See the NOTICE file distributed with
> + * this work for additional information regarding copyright ownership.
> + * The ASF licenses this file to You under the Apache License, Version 2.0
> + * (the "License"); you may not use this file except in compliance with
> + * the License.  You may obtain a copy of the License at
> + *
> + *     http://www.apache.org/licenses/LICENSE-2.0
> + *
> + * Unless required by applicable law or agreed to in writing, software
> + * distributed under the License is distributed on an "AS IS" BASIS,
> + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
> + * See the License for the specific language governing permissions and
> + * limitations under the License.
> + */
> +
> +package org.apache.solr.search;
> +
> +import org.apache.lucene.search.Filter;
> +import org.apache.lucene.search.Searcher;
> +import org.apache.lucene.search.DocIdSet;
> +import org.apache.lucene.index.IndexReader;
> +
> +import java.util.Map;
> +import java.io.IOException;
> +
> +
> +/** A SolrFilter extends the Lucene Filter and adds extra semantics such as passing on
> + * weight context info for function queries.
> + *
> + * Experimental and subject to change.
> + */
> +public abstract class SolrFilter extends Filter {
> +
> +  /** 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 DocIdSet getDocIdSet(Map context, IndexReader reader) throws IOException;
> +
> +  @Override
> +  public DocIdSet getDocIdSet(IndexReader reader) throws IOException {
> +    return getDocIdSet(null, reader);
> +  }
> +}
>
> Modified: lucene/solr/trunk/src/java/org/apache/solr/search/ValueSourceParser.java
> URL: http://svn.apache.org/viewvc/lucene/solr/trunk/src/java/org/apache/solr/search/ValueSourceParser.java?rev=816202&r1=816201&r2=816202&view=diff
> ==============================================================================
> --- lucene/solr/trunk/src/java/org/apache/solr/search/ValueSourceParser.java (original)
> +++ lucene/solr/trunk/src/java/org/apache/solr/search/ValueSourceParser.java Thu Sep 17 14:08:13 2009
> @@ -16,10 +16,7 @@
>  */
>  package org.apache.solr.search;
>
> -import java.util.HashMap;
> -import java.util.List;
> -import java.util.Map;
> -import java.util.Date;
> +import java.util.*;
>  import java.io.IOException;
>
>  import org.apache.lucene.queryParser.ParseException;
> @@ -398,7 +395,7 @@
>     return "const(" + constant + ")";
>   }
>
> -  public DocValues getValues(IndexReader reader) throws IOException {
> +  public DocValues getValues(Map context, IndexReader reader) throws IOException {
>     return new DocValues() {
>       public float floatVal(int doc) {
>         return constant;
>
> Modified: lucene/solr/trunk/src/java/org/apache/solr/search/function/BoostedQuery.java
> URL: http://svn.apache.org/viewvc/lucene/solr/trunk/src/java/org/apache/solr/search/function/BoostedQuery.java?rev=816202&r1=816201&r2=816202&view=diff
> ==============================================================================
> --- lucene/solr/trunk/src/java/org/apache/solr/search/function/BoostedQuery.java (original)
> +++ lucene/solr/trunk/src/java/org/apache/solr/search/function/BoostedQuery.java Thu Sep 17 14:08:13 2009
> @@ -24,6 +24,7 @@
>
>  import java.io.IOException;
>  import java.util.Set;
> +import java.util.Map;
>
>  /**
>  * Query that is boosted by a ValueSource
> @@ -59,10 +60,13 @@
>   private class BoostedWeight extends Weight {
>     Searcher searcher;
>     Weight qWeight;
> +    Map context;
>
>     public BoostedWeight(Searcher searcher) throws IOException {
>       this.searcher = searcher;
>       this.qWeight = q.weight(searcher);
> +      this.context = boostVal.newContext();
> +      boostVal.createWeight(context,searcher);
>     }
>
>     public Query getQuery() {
> @@ -105,7 +109,7 @@
>         return subQueryExpl;
>       }
>
> -      DocValues vals = boostVal.getValues(subReaders[readerPos]);
> +      DocValues vals = boostVal.getValues(context, subReaders[readerPos]);
>       float sc = subQueryExpl.getValue() * vals.floatVal(doc-readerBase);
>       Explanation res = new ComplexExplanation(
>         true, sc, BoostedQuery.this.toString() + ", product of:");
> @@ -132,7 +136,7 @@
>       this.scorer = scorer;
>       this.reader = reader;
>       this.searcher = searcher; // for explain
> -      this.vals = vs.getValues(reader);
> +      this.vals = vs.getValues(weight.context, reader);
>     }
>
>     @Override
>
> Modified: lucene/solr/trunk/src/java/org/apache/solr/search/function/ByteFieldSource.java
> URL: http://svn.apache.org/viewvc/lucene/solr/trunk/src/java/org/apache/solr/search/function/ByteFieldSource.java?rev=816202&r1=816201&r2=816202&view=diff
> ==============================================================================
> --- lucene/solr/trunk/src/java/org/apache/solr/search/function/ByteFieldSource.java (original)
> +++ lucene/solr/trunk/src/java/org/apache/solr/search/function/ByteFieldSource.java Thu Sep 17 14:08:13 2009
> @@ -20,6 +20,7 @@
>  import org.apache.lucene.search.FieldCache;
>
>  import java.io.IOException;
> +import java.util.Map;
>
>  /**
>  * Obtains int field values from the {@link org.apache.lucene.search.FieldCache}
> @@ -45,7 +46,7 @@
>     return "byte(" + field + ')';
>   }
>
> -  public DocValues getValues(IndexReader reader) throws IOException {
> +  public DocValues getValues(Map context, IndexReader reader) throws IOException {
>     final byte[] arr = (parser == null) ?
>             cache.getBytes(reader, field) :
>             cache.getBytes(reader, field, parser);
>
> Modified: lucene/solr/trunk/src/java/org/apache/solr/search/function/ConstValueSource.java
> URL: http://svn.apache.org/viewvc/lucene/solr/trunk/src/java/org/apache/solr/search/function/ConstValueSource.java?rev=816202&r1=816201&r2=816202&view=diff
> ==============================================================================
> --- lucene/solr/trunk/src/java/org/apache/solr/search/function/ConstValueSource.java (original)
> +++ lucene/solr/trunk/src/java/org/apache/solr/search/function/ConstValueSource.java Thu Sep 17 14:08:13 2009
> @@ -20,6 +20,7 @@
>  import org.apache.lucene.index.IndexReader;
>
>  import java.io.IOException;
> +import java.util.Map;
>
>  /**
>  * <code>ConstValueSource</code> returns a constant for all documents
> @@ -35,7 +36,7 @@
>     return "const(" + constant + ")";
>   }
>
> -  public DocValues getValues(IndexReader reader) throws IOException {
> +  public DocValues getValues(Map context, IndexReader reader) throws IOException {
>     return new DocValues() {
>       public float floatVal(int doc) {
>         return constant;
>
> Modified: lucene/solr/trunk/src/java/org/apache/solr/search/function/DoubleFieldSource.java
> URL: http://svn.apache.org/viewvc/lucene/solr/trunk/src/java/org/apache/solr/search/function/DoubleFieldSource.java?rev=816202&r1=816201&r2=816202&view=diff
> ==============================================================================
> --- lucene/solr/trunk/src/java/org/apache/solr/search/function/DoubleFieldSource.java (original)
> +++ lucene/solr/trunk/src/java/org/apache/solr/search/function/DoubleFieldSource.java Thu Sep 17 14:08:13 2009
> @@ -21,6 +21,7 @@
>  import org.apache.lucene.search.FieldCache;
>
>  import java.io.IOException;
> +import java.util.Map;
>
>  /**
>  * Obtains float field values from the {@link org.apache.lucene.search.FieldCache}
> @@ -46,7 +47,7 @@
>     return "double(" + field + ')';
>   }
>
> -  public DocValues getValues(IndexReader reader) throws IOException {
> +  public DocValues getValues(Map context, IndexReader reader) throws IOException {
>     final double[] arr = (parser == null) ?
>             ((FieldCache) cache).getDoubles(reader, field) :
>             ((FieldCache) cache).getDoubles(reader, field, parser);
>
> Modified: lucene/solr/trunk/src/java/org/apache/solr/search/function/DualFloatFunction.java
> URL: http://svn.apache.org/viewvc/lucene/solr/trunk/src/java/org/apache/solr/search/function/DualFloatFunction.java?rev=816202&r1=816201&r2=816202&view=diff
> ==============================================================================
> --- lucene/solr/trunk/src/java/org/apache/solr/search/function/DualFloatFunction.java (original)
> +++ lucene/solr/trunk/src/java/org/apache/solr/search/function/DualFloatFunction.java Thu Sep 17 14:08:13 2009
> @@ -18,8 +18,10 @@
>  package org.apache.solr.search.function;
>
>  import org.apache.lucene.index.IndexReader;
> +import org.apache.lucene.search.Searcher;
>
>  import java.io.IOException;
> +import java.util.Map;
>
>  public abstract class DualFloatFunction extends ValueSource {
>   protected final ValueSource a;
> @@ -41,9 +43,9 @@
>     return name() + "(" + a.description() + "," + b.description() + ")";
>   }
>
> -  public DocValues getValues(IndexReader reader) throws IOException {
> -    final DocValues aVals =  a.getValues(reader);
> -    final DocValues bVals =  b.getValues(reader);
> +  public DocValues getValues(Map context, IndexReader reader) throws IOException {
> +    final DocValues aVals =  a.getValues(context, reader);
> +    final DocValues bVals =  b.getValues(context, reader);
>     return new DocValues() {
>       public float floatVal(int doc) {
>        return func(doc, aVals, bVals);
> @@ -66,6 +68,12 @@
>     };
>   }
>
> +  @Override
> +  public void createWeight(Map context, Searcher searcher) throws IOException {
> +    a.createWeight(context,searcher);
> +    b.createWeight(context,searcher);
> +  }
> +
>   public int hashCode() {
>     int h = a.hashCode();
>     h ^= (h << 13) | (h >>> 20);
>
> Modified: lucene/solr/trunk/src/java/org/apache/solr/search/function/FileFloatSource.java
> URL: http://svn.apache.org/viewvc/lucene/solr/trunk/src/java/org/apache/solr/search/function/FileFloatSource.java?rev=816202&r1=816201&r2=816202&view=diff
> ==============================================================================
> --- lucene/solr/trunk/src/java/org/apache/solr/search/function/FileFloatSource.java (original)
> +++ lucene/solr/trunk/src/java/org/apache/solr/search/function/FileFloatSource.java Thu Sep 17 14:08:13 2009
> @@ -54,7 +54,7 @@
>     return "float(" + field + ')';
>   }
>
> -  public DocValues getValues(IndexReader reader) throws IOException {
> +  public DocValues getValues(Map context, IndexReader reader) throws IOException {
>     int offset = 0;
>     if (reader instanceof SolrIndexReader) {
>       SolrIndexReader r = (SolrIndexReader)reader;
>
> Modified: lucene/solr/trunk/src/java/org/apache/solr/search/function/FloatFieldSource.java
> URL: http://svn.apache.org/viewvc/lucene/solr/trunk/src/java/org/apache/solr/search/function/FloatFieldSource.java?rev=816202&r1=816201&r2=816202&view=diff
> ==============================================================================
> --- lucene/solr/trunk/src/java/org/apache/solr/search/function/FloatFieldSource.java (original)
> +++ lucene/solr/trunk/src/java/org/apache/solr/search/function/FloatFieldSource.java Thu Sep 17 14:08:13 2009
> @@ -22,6 +22,7 @@
>  import org.apache.lucene.search.FieldCache;
>
>  import java.io.IOException;
> +import java.util.Map;
>
>  /**
>  * Obtains float field values from the {@link org.apache.lucene.search.FieldCache}
> @@ -47,7 +48,7 @@
>     return "float(" + field + ')';
>   }
>
> -  public DocValues getValues(IndexReader reader) throws IOException {
> +  public DocValues getValues(Map context, IndexReader reader) throws IOException {
>     final float[] arr = (parser==null) ?
>             cache.getFloats(reader, field) :
>             cache.getFloats(reader, field, parser);
>
> Modified: lucene/solr/trunk/src/java/org/apache/solr/search/function/FunctionQuery.java
> URL: http://svn.apache.org/viewvc/lucene/solr/trunk/src/java/org/apache/solr/search/function/FunctionQuery.java?rev=816202&r1=816201&r2=816202&view=diff
> ==============================================================================
> --- lucene/solr/trunk/src/java/org/apache/solr/search/function/FunctionQuery.java (original)
> +++ lucene/solr/trunk/src/java/org/apache/solr/search/function/FunctionQuery.java Thu Sep 17 14:08:13 2009
> @@ -23,6 +23,8 @@
>
>  import java.io.IOException;
>  import java.util.Set;
> +import java.util.IdentityHashMap;
> +import java.util.Map;
>
>
>  /**
> @@ -58,9 +60,12 @@
>     protected Searcher searcher;
>     protected float queryNorm;
>     protected float queryWeight;
> +    protected Map context;
>
> -    public FunctionWeight(Searcher searcher) {
> +    public FunctionWeight(Searcher searcher) throws IOException {
>       this.searcher = searcher;
> +      this.context = func.newContext();
> +      func.createWeight(context, searcher);
>     }
>
>     public Query getQuery() {
> @@ -115,7 +120,7 @@
>       this.reader = reader;
>       this.maxDoc = reader.maxDoc();
>       this.hasDeletions = reader.hasDeletions();
> -      vals = func.getValues(reader);
> +      vals = func.getValues(weight.context, reader);
>     }
>
>     @Override
> @@ -197,7 +202,7 @@
>   }
>
>
> -  public Weight createWeight(Searcher searcher) {
> +  public Weight createWeight(Searcher searcher) throws IOException {
>     return new FunctionQuery.FunctionWeight(searcher);
>   }
>
>
> Modified: lucene/solr/trunk/src/java/org/apache/solr/search/function/IntFieldSource.java
> URL: http://svn.apache.org/viewvc/lucene/solr/trunk/src/java/org/apache/solr/search/function/IntFieldSource.java?rev=816202&r1=816201&r2=816202&view=diff
> ==============================================================================
> --- lucene/solr/trunk/src/java/org/apache/solr/search/function/IntFieldSource.java (original)
> +++ lucene/solr/trunk/src/java/org/apache/solr/search/function/IntFieldSource.java Thu Sep 17 14:08:13 2009
> @@ -22,6 +22,7 @@
>  import org.apache.lucene.search.FieldCache;
>
>  import java.io.IOException;
> +import java.util.Map;
>
>  /**
>  * Obtains int field values from the {@link org.apache.lucene.search.FieldCache}
> @@ -47,7 +48,7 @@
>   }
>
>
> -  public DocValues getValues(IndexReader reader) throws IOException {
> +  public DocValues getValues(Map context, IndexReader reader) throws IOException {
>     final int[] arr = (parser==null) ?
>             cache.getInts(reader, field) :
>             cache.getInts(reader, field, parser);
>
> Modified: lucene/solr/trunk/src/java/org/apache/solr/search/function/LinearFloatFunction.java
> URL: http://svn.apache.org/viewvc/lucene/solr/trunk/src/java/org/apache/solr/search/function/LinearFloatFunction.java?rev=816202&r1=816201&r2=816202&view=diff
> ==============================================================================
> --- lucene/solr/trunk/src/java/org/apache/solr/search/function/LinearFloatFunction.java (original)
> +++ lucene/solr/trunk/src/java/org/apache/solr/search/function/LinearFloatFunction.java Thu Sep 17 14:08:13 2009
> @@ -18,8 +18,10 @@
>  package org.apache.solr.search.function;
>
>  import org.apache.lucene.index.IndexReader;
> +import org.apache.lucene.search.Searcher;
>
>  import java.io.IOException;
> +import java.util.Map;
>
>  /**
>  * <code>LinearFloatFunction</code> implements a linear function over
> @@ -44,8 +46,8 @@
>     return slope + "*float(" + source.description() + ")+" + intercept;
>   }
>
> -  public DocValues getValues(IndexReader reader) throws IOException {
> -    final DocValues vals =  source.getValues(reader);
> +  public DocValues getValues(Map context, IndexReader reader) throws IOException {
> +    final DocValues vals =  source.getValues(context, reader);
>     return new DocValues() {
>       public float floatVal(int doc) {
>         return vals.floatVal(doc) * slope + intercept;
> @@ -68,6 +70,11 @@
>     };
>   }
>
> +  @Override
> +  public void createWeight(Map context, Searcher searcher) throws IOException {
> +    source.createWeight(context, searcher);
> +  }
> +
>   public int hashCode() {
>     int h = Float.floatToIntBits(slope);
>     h = (h >>> 2) | (h << 30);
>
> Modified: lucene/solr/trunk/src/java/org/apache/solr/search/function/LongFieldSource.java
> URL: http://svn.apache.org/viewvc/lucene/solr/trunk/src/java/org/apache/solr/search/function/LongFieldSource.java?rev=816202&r1=816201&r2=816202&view=diff
> ==============================================================================
> --- lucene/solr/trunk/src/java/org/apache/solr/search/function/LongFieldSource.java (original)
> +++ lucene/solr/trunk/src/java/org/apache/solr/search/function/LongFieldSource.java Thu Sep 17 14:08:13 2009
> @@ -22,6 +22,7 @@
>
>
>  import java.io.IOException;
> +import java.util.Map;
>
>  /**
>  * Obtains float field values from the {@link org.apache.lucene.search.FieldCache}
> @@ -52,7 +53,7 @@
>     return Long.parseLong(extVal);
>   }
>
> -  public DocValues getValues(IndexReader reader) throws IOException {
> +  public DocValues getValues(Map context, IndexReader reader) throws IOException {
>     final long[] arr = (parser == null) ?
>             ((FieldCache) cache).getLongs(reader, field) :
>             ((FieldCache) cache).getLongs(reader, field, parser);
>
> Modified: lucene/solr/trunk/src/java/org/apache/solr/search/function/MaxFloatFunction.java
> URL: http://svn.apache.org/viewvc/lucene/solr/trunk/src/java/org/apache/solr/search/function/MaxFloatFunction.java?rev=816202&r1=816201&r2=816202&view=diff
> ==============================================================================
> --- lucene/solr/trunk/src/java/org/apache/solr/search/function/MaxFloatFunction.java (original)
> +++ lucene/solr/trunk/src/java/org/apache/solr/search/function/MaxFloatFunction.java Thu Sep 17 14:08:13 2009
> @@ -18,8 +18,10 @@
>  package org.apache.solr.search.function;
>
>  import org.apache.lucene.index.IndexReader;
> +import org.apache.lucene.search.Searcher;
>
>  import java.io.IOException;
> +import java.util.Map;
>
>  /**
>  * Returns the max of a ValueSource and a float
> @@ -43,8 +45,8 @@
>     return "max(" + source.description() + "," + fval + ")";
>   }
>
> -  public DocValues getValues(IndexReader reader) throws IOException {
> -    final DocValues vals =  source.getValues(reader);
> +  public DocValues getValues(Map context, IndexReader reader) throws IOException {
> +    final DocValues vals =  source.getValues(context, reader);
>     return new DocValues() {
>       public float floatVal(int doc) {
>        float v = vals.floatVal(doc);
> @@ -68,6 +70,11 @@
>     };
>   }
>
> +  @Override
> +  public void createWeight(Map context, Searcher searcher) throws IOException {
> +    source.createWeight(context, searcher);
> +  }
> +
>   public int hashCode() {
>     int h = Float.floatToIntBits(fval);
>     h = (h >>> 2) | (h << 30);
>
> Modified: lucene/solr/trunk/src/java/org/apache/solr/search/function/OrdFieldSource.java
> URL: http://svn.apache.org/viewvc/lucene/solr/trunk/src/java/org/apache/solr/search/function/OrdFieldSource.java?rev=816202&r1=816201&r2=816202&view=diff
> ==============================================================================
> --- lucene/solr/trunk/src/java/org/apache/solr/search/function/OrdFieldSource.java (original)
> +++ lucene/solr/trunk/src/java/org/apache/solr/search/function/OrdFieldSource.java Thu Sep 17 14:08:13 2009
> @@ -20,9 +20,9 @@
>  import org.apache.lucene.index.IndexReader;
>  import org.apache.solr.search.function.DocValues;
>  import org.apache.solr.search.function.ValueSource;
> -import org.apache.lucene.search.FieldCache;
>
>  import java.io.IOException;
> +import java.util.Map;
>
>  /**
>  * Obtains the ordinal of the field value from the default Lucene {@link org.apache.lucene.search.FieldCache} using getStringIndex().
> @@ -51,7 +51,7 @@
>   }
>
>
> -  public DocValues getValues(IndexReader reader) throws IOException {
> +  public DocValues getValues(Map context, IndexReader reader) throws IOException {
>     return new StringIndexDocValues(this, reader, field) {
>       protected String toTerm(String readableValue) {
>         return readableValue;
>
> Modified: lucene/solr/trunk/src/java/org/apache/solr/search/function/QueryValueSource.java
> URL: http://svn.apache.org/viewvc/lucene/solr/trunk/src/java/org/apache/solr/search/function/QueryValueSource.java?rev=816202&r1=816201&r2=816202&view=diff
> ==============================================================================
> --- lucene/solr/trunk/src/java/org/apache/solr/search/function/QueryValueSource.java (original)
> +++ lucene/solr/trunk/src/java/org/apache/solr/search/function/QueryValueSource.java Thu Sep 17 14:08:13 2009
> @@ -18,13 +18,11 @@
>  package org.apache.solr.search.function;
>
>  import org.apache.lucene.index.IndexReader;
> -import org.apache.lucene.search.Query;
> -import org.apache.lucene.search.IndexSearcher;
> -import org.apache.lucene.search.Weight;
> -import org.apache.lucene.search.Scorer;
> +import org.apache.lucene.search.*;
>  import org.apache.solr.common.SolrException;
>
>  import java.io.IOException;
> +import java.util.Map;
>
>  /**
>  * <code>QueryValueSource</code> returns the relevance score of the query
> @@ -45,8 +43,9 @@
>     return "query(" + q + ",def=" + defVal + ")";
>   }
>
> -  public DocValues getValues(IndexReader reader) throws IOException {
> -    return new QueryDocValues(reader, q, defVal);
> +  @Override
> +  public DocValues getValues(Map context, IndexReader reader) throws IOException {
> +    return new QueryDocValues(reader, q, defVal, context==null ? null : (Weight)context.get(this));
>   }
>
>   public int hashCode() {
> @@ -58,13 +57,18 @@
>     QueryValueSource other = (QueryValueSource)o;
>     return this.q.equals(other.q) && this.defVal==other.defVal;
>   }
> +
> +  @Override
> +  public void createWeight(Map context, Searcher searcher) throws IOException {
> +    Weight w = q.weight(searcher);
> +    context.put(this, w);
> +  }
>  }
>
>
>  class QueryDocValues extends DocValues {
>   final Query q;
>   final IndexReader reader;
> -  final IndexSearcher searcher;
>   final Weight weight;
>   final float defVal;
>
> @@ -75,12 +79,11 @@
>   // to trigger a scorer reset on first access.
>   int lastDocRequested=Integer.MAX_VALUE;
>
> -  public QueryDocValues(IndexReader reader, Query q, float defVal) throws IOException {
> +  public QueryDocValues(IndexReader reader, Query q, float defVal, Weight w) throws IOException {
>     this.reader = reader;
>     this.q = q;
>     this.defVal = defVal;
> -    searcher = new IndexSearcher(reader);
> -    weight = q.weight(searcher);
> +    weight = w!=null ? w : q.weight(new IndexSearcher(reader));
>   }
>
>   public float floatVal(int doc) {
> @@ -88,12 +91,12 @@
>       if (doc < lastDocRequested) {
>         // out-of-order access.... reset scorer.
>         scorer = weight.scorer(reader, true, false);
> -        scorerDoc = scorer.nextDoc();
> +        scorerDoc = -1;
>       }
>       lastDocRequested = doc;
>
>       if (scorerDoc < doc) {
> -        scorerDoc = scorer.nextDoc();
> +        scorerDoc = scorer.advance(doc);
>       }
>
>       if (scorerDoc > doc) {
>
> Modified: lucene/solr/trunk/src/java/org/apache/solr/search/function/RangeMapFloatFunction.java
> URL: http://svn.apache.org/viewvc/lucene/solr/trunk/src/java/org/apache/solr/search/function/RangeMapFloatFunction.java?rev=816202&r1=816201&r2=816202&view=diff
> ==============================================================================
> --- lucene/solr/trunk/src/java/org/apache/solr/search/function/RangeMapFloatFunction.java (original)
> +++ lucene/solr/trunk/src/java/org/apache/solr/search/function/RangeMapFloatFunction.java Thu Sep 17 14:08:13 2009
> @@ -18,8 +18,10 @@
>  package org.apache.solr.search.function;
>
>  import org.apache.lucene.index.IndexReader;
> +import org.apache.lucene.search.Searcher;
>
>  import java.io.IOException;
> +import java.util.Map;
>
>  /**
>  * <code>LinearFloatFunction</code> implements a linear function over
> @@ -48,8 +50,8 @@
>     return "map(" + source.description() + "," + min + "," + max + "," + target + ")";
>   }
>
> -  public DocValues getValues(IndexReader reader) throws IOException {
> -    final DocValues vals =  source.getValues(reader);
> +  public DocValues getValues(Map context, IndexReader reader) throws IOException {
> +    final DocValues vals =  source.getValues(context, reader);
>     return new DocValues() {
>       public float floatVal(int doc) {
>         float val = vals.floatVal(doc);
> @@ -73,6 +75,11 @@
>     };
>   }
>
> +  @Override
> +  public void createWeight(Map context, Searcher searcher) throws IOException {
> +    source.createWeight(context, searcher);
> +  }
> +
>   public int hashCode() {
>     int h = source.hashCode();
>     h ^= (h << 10) | (h >>> 23);
>
> Modified: lucene/solr/trunk/src/java/org/apache/solr/search/function/ReciprocalFloatFunction.java
> URL: http://svn.apache.org/viewvc/lucene/solr/trunk/src/java/org/apache/solr/search/function/ReciprocalFloatFunction.java?rev=816202&r1=816201&r2=816202&view=diff
> ==============================================================================
> --- lucene/solr/trunk/src/java/org/apache/solr/search/function/ReciprocalFloatFunction.java (original)
> +++ lucene/solr/trunk/src/java/org/apache/solr/search/function/ReciprocalFloatFunction.java Thu Sep 17 14:08:13 2009
> @@ -18,8 +18,10 @@
>  package org.apache.solr.search.function;
>
>  import org.apache.lucene.index.IndexReader;
> +import org.apache.lucene.search.Searcher;
>
>  import java.io.IOException;
> +import java.util.Map;
>
>  /**
>  * <code>ReciprocalFloatFunction</code> implements a reciprocal function f(x) = a/(mx+b), based on
> @@ -52,8 +54,8 @@
>     this.b=b;
>   }
>
> -  public DocValues getValues(IndexReader reader) throws IOException {
> -    final DocValues vals = source.getValues(reader);
> +  public DocValues getValues(Map context, IndexReader reader) throws IOException {
> +    final DocValues vals = source.getValues(context, reader);
>     return new DocValues() {
>       public float floatVal(int doc) {
>         return a/(m*vals.floatVal(doc) + b);
> @@ -78,6 +80,11 @@
>     };
>   }
>
> +  @Override
> +  public void createWeight(Map context, Searcher searcher) throws IOException {
> +    source.createWeight(context, searcher);
> +  }
> +
>   public String description() {
>     return Float.toString(a) + "/("
>            + m + "*float(" + source.description() + ")"
>
> Modified: lucene/solr/trunk/src/java/org/apache/solr/search/function/ReverseOrdFieldSource.java
> URL: http://svn.apache.org/viewvc/lucene/solr/trunk/src/java/org/apache/solr/search/function/ReverseOrdFieldSource.java?rev=816202&r1=816201&r2=816202&view=diff
> ==============================================================================
> --- lucene/solr/trunk/src/java/org/apache/solr/search/function/ReverseOrdFieldSource.java (original)
> +++ lucene/solr/trunk/src/java/org/apache/solr/search/function/ReverseOrdFieldSource.java Thu Sep 17 14:08:13 2009
> @@ -23,6 +23,7 @@
>  import org.apache.lucene.search.FieldCache;
>
>  import java.io.IOException;
> +import java.util.Map;
>
>  /**
>  * Obtains the ordinal of the field value from the default Lucene {@link org.apache.lucene.search.FieldCache} using getStringIndex()
> @@ -51,7 +52,7 @@
>     return "rord("+field+')';
>   }
>
> -  public DocValues getValues(IndexReader reader) throws IOException {
> +  public DocValues getValues(Map context, IndexReader reader) throws IOException {
>     final FieldCache.StringIndex sindex = FieldCache.DEFAULT.getStringIndex(reader, field);
>
>     final int arr[] = sindex.order;
>
> Modified: lucene/solr/trunk/src/java/org/apache/solr/search/function/ScaleFloatFunction.java
> URL: http://svn.apache.org/viewvc/lucene/solr/trunk/src/java/org/apache/solr/search/function/ScaleFloatFunction.java?rev=816202&r1=816201&r2=816202&view=diff
> ==============================================================================
> --- lucene/solr/trunk/src/java/org/apache/solr/search/function/ScaleFloatFunction.java (original)
> +++ lucene/solr/trunk/src/java/org/apache/solr/search/function/ScaleFloatFunction.java Thu Sep 17 14:08:13 2009
> @@ -18,8 +18,10 @@
>  package org.apache.solr.search.function;
>
>  import org.apache.lucene.index.IndexReader;
> +import org.apache.lucene.search.Searcher;
>
>  import java.io.IOException;
> +import java.util.Map;
>
>  /**
>  * Scales values to be between min and max.
> @@ -47,8 +49,8 @@
>     return "scale(" + source.description() + "," + min + "," + max + ")";
>   }
>
> -  public DocValues getValues(IndexReader reader) throws IOException {
> -    final DocValues vals =  source.getValues(reader);
> +  public DocValues getValues(Map context, IndexReader reader) throws IOException {
> +    final DocValues vals =  source.getValues(context, reader);
>     int maxDoc = reader.maxDoc();
>
>     // this doesn't take into account deleted docs!
> @@ -107,6 +109,11 @@
>     };
>   }
>
> +  @Override
> +  public void createWeight(Map context, Searcher searcher) throws IOException {
> +    source.createWeight(context, searcher);
> +  }
> +
>   public int hashCode() {
>     int h = Float.floatToIntBits(min);
>     h = h*29;
>
> Modified: lucene/solr/trunk/src/java/org/apache/solr/search/function/ShortFieldSource.java
> URL: http://svn.apache.org/viewvc/lucene/solr/trunk/src/java/org/apache/solr/search/function/ShortFieldSource.java?rev=816202&r1=816201&r2=816202&view=diff
> ==============================================================================
> --- lucene/solr/trunk/src/java/org/apache/solr/search/function/ShortFieldSource.java (original)
> +++ lucene/solr/trunk/src/java/org/apache/solr/search/function/ShortFieldSource.java Thu Sep 17 14:08:13 2009
> @@ -20,6 +20,7 @@
>  import org.apache.lucene.index.IndexReader;
>
>  import java.io.IOException;
> +import java.util.Map;
>
>
>  /**
> @@ -42,7 +43,7 @@
>     return "short(" + field + ')';
>   }
>
> -  public DocValues getValues(IndexReader reader) throws IOException {
> +  public DocValues getValues(Map context, IndexReader reader) throws IOException {
>     final short[] arr = (parser == null) ?
>             cache.getShorts(reader, field) :
>             cache.getShorts(reader, field, parser);
>
> Modified: lucene/solr/trunk/src/java/org/apache/solr/search/function/SimpleFloatFunction.java
> URL: http://svn.apache.org/viewvc/lucene/solr/trunk/src/java/org/apache/solr/search/function/SimpleFloatFunction.java?rev=816202&r1=816201&r2=816202&view=diff
> ==============================================================================
> --- lucene/solr/trunk/src/java/org/apache/solr/search/function/SimpleFloatFunction.java (original)
> +++ lucene/solr/trunk/src/java/org/apache/solr/search/function/SimpleFloatFunction.java Thu Sep 17 14:08:13 2009
> @@ -20,25 +20,20 @@
>  import org.apache.lucene.index.IndexReader;
>
>  import java.io.IOException;
> +import java.util.Map;
>
>  /** A simple float function with a single argument
>  */
> - public abstract class SimpleFloatFunction extends ValueSource {
> -  protected final ValueSource source;
> -
> + public abstract class SimpleFloatFunction extends SingleFunction {
>   public SimpleFloatFunction(ValueSource source) {
> -    this.source = source;
> +    super(source);
>   }
>
> -  protected abstract String name();
>   protected abstract float func(int doc, DocValues vals);
>
> -  public String description() {
> -    return name() + '(' + source.description() + ')';
> -  }
> -
> -  public DocValues getValues(IndexReader reader) throws IOException {
> -    final DocValues vals =  source.getValues(reader);
> +  @Override
> +  public DocValues getValues(Map context, IndexReader reader) throws IOException {
> +    final DocValues vals =  source.getValues(context, reader);
>     return new DocValues() {
>       public float floatVal(int doc) {
>        return func(doc, vals);
> @@ -60,15 +55,4 @@
>       }
>     };
>   }
> -
> -  public int hashCode() {
> -    return source.hashCode() + name().hashCode();
> -  }
> -
> -  public boolean equals(Object o) {
> -    if (this.getClass() != o.getClass()) return false;
> -    SimpleFloatFunction other = (SimpleFloatFunction)o;
> -    return this.name().equals(other.name())
> -         && this.source.equals(other.source);
> -  }
>  }
>
> Added: lucene/solr/trunk/src/java/org/apache/solr/search/function/SingleFunction.java
> URL: http://svn.apache.org/viewvc/lucene/solr/trunk/src/java/org/apache/solr/search/function/SingleFunction.java?rev=816202&view=auto
> ==============================================================================
> --- lucene/solr/trunk/src/java/org/apache/solr/search/function/SingleFunction.java (added)
> +++ lucene/solr/trunk/src/java/org/apache/solr/search/function/SingleFunction.java Thu Sep 17 14:08:13 2009
> @@ -0,0 +1,55 @@
> +/**
> + * Licensed to the Apache Software Foundation (ASF) under one or more
> + * contributor license agreements.  See the NOTICE file distributed with
> + * this work for additional information regarding copyright ownership.
> + * The ASF licenses this file to You under the Apache License, Version 2.0
> + * (the "License"); you may not use this file except in compliance with
> + * the License.  You may obtain a copy of the License at
> + *
> + *     http://www.apache.org/licenses/LICENSE-2.0
> + *
> + * Unless required by applicable law or agreed to in writing, software
> + * distributed under the License is distributed on an "AS IS" BASIS,
> + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
> + * See the License for the specific language governing permissions and
> + * limitations under the License.
> + */
> +
> +package org.apache.solr.search.function;
> +
> +import org.apache.lucene.search.Searcher;
> +
> +import java.io.IOException;
> +import java.util.Map;
> +
> +/** A function with a single argument
> + */
> + public abstract class SingleFunction extends ValueSource {
> +  protected final ValueSource source;
> +
> +  public SingleFunction(ValueSource source) {
> +    this.source = source;
> +  }
> +
> +  protected abstract String name();
> +
> +  public String description() {
> +    return name() + '(' + source.description() + ')';
> +  }
> +
> +  public int hashCode() {
> +    return source.hashCode() + name().hashCode();
> +  }
> +
> +  public boolean equals(Object o) {
> +    if (this.getClass() != o.getClass()) return false;
> +    SingleFunction other = (SingleFunction)o;
> +    return this.name().equals(other.name())
> +         && this.source.equals(other.source);
> +  }
> +
> +  @Override
> +  public void createWeight(Map context, Searcher searcher) throws IOException {
> +    source.createWeight(context, searcher);
> +  }
> +}
> \ No newline at end of file
>
> Propchange: lucene/solr/trunk/src/java/org/apache/solr/search/function/SingleFunction.java
> ------------------------------------------------------------------------------
>    svn:eol-style = native
>
> Propchange: lucene/solr/trunk/src/java/org/apache/solr/search/function/SingleFunction.java
> ------------------------------------------------------------------------------
>    svn:executable = *
>
> Propchange: lucene/solr/trunk/src/java/org/apache/solr/search/function/SingleFunction.java
> ------------------------------------------------------------------------------
>    svn:keywords = Date Author Id Revision HeadURL
>
> Modified: lucene/solr/trunk/src/java/org/apache/solr/search/function/SumFloatFunction.java
> URL: http://svn.apache.org/viewvc/lucene/solr/trunk/src/java/org/apache/solr/search/function/SumFloatFunction.java?rev=816202&r1=816201&r2=816202&view=diff
> ==============================================================================
> --- lucene/solr/trunk/src/java/org/apache/solr/search/function/SumFloatFunction.java (original)
> +++ lucene/solr/trunk/src/java/org/apache/solr/search/function/SumFloatFunction.java Thu Sep 17 14:08:13 2009
> @@ -18,9 +18,11 @@
>  package org.apache.solr.search.function;
>
>  import org.apache.lucene.index.IndexReader;
> +import org.apache.lucene.search.Searcher;
>
>  import java.io.IOException;
>  import java.util.Arrays;
> +import java.util.Map;
>
>  /**
>  * <code>SumFloatFunction</code> returns the sum of it's components.
> @@ -71,10 +73,10 @@
>     return sb.toString();
>   }
>
> -  public DocValues getValues(IndexReader reader) throws IOException {
> +  public DocValues getValues(Map context, IndexReader reader) throws IOException {
>     final DocValues[] valsArr = new DocValues[sources.length];
>     for (int i=0; i<sources.length; i++) {
> -      valsArr[i] = sources[i].getValues(reader);
> +      valsArr[i] = sources[i].getValues(context, reader);
>     }
>
>     return new DocValues() {
> @@ -111,6 +113,12 @@
>     };
>   }
>
> +  @Override
> +  public void createWeight(Map context, Searcher searcher) throws IOException {
> +    for (ValueSource source : sources)
> +      source.createWeight(context, searcher);
> +  }
> +
>   public int hashCode() {
>     return Arrays.hashCode(sources) + name().hashCode();
>   }
>
> Modified: lucene/solr/trunk/src/java/org/apache/solr/search/function/TopValueSource.java
> URL: http://svn.apache.org/viewvc/lucene/solr/trunk/src/java/org/apache/solr/search/function/TopValueSource.java?rev=816202&r1=816201&r2=816202&view=diff
> ==============================================================================
> --- lucene/solr/trunk/src/java/org/apache/solr/search/function/TopValueSource.java (original)
> +++ lucene/solr/trunk/src/java/org/apache/solr/search/function/TopValueSource.java Thu Sep 17 14:08:13 2009
> @@ -20,6 +20,7 @@
>  import org.apache.lucene.index.IndexReader;
>
>  import java.io.IOException;
> +import java.util.Map;
>
>  /**
>  * A value source that wraps another and ensures that the top level reader
> @@ -41,7 +42,7 @@
>     return "top(" + vs.description() + ')';
>   }
>
> -  public DocValues getValues(IndexReader reader) throws IOException {
> +  public DocValues getValues(Map context, IndexReader reader) throws IOException {
>     int offset = 0;
>     IndexReader topReader = reader;
>     if (topReader instanceof SolrIndexReader) {
> @@ -53,7 +54,7 @@
>       topReader = r;
>     }
>     final int off = offset;
> -    final DocValues vals = vs.getValues(topReader);
> +    final DocValues vals = vs.getValues(context, topReader);
>     if (topReader == reader) return vals;
>
>     return new DocValues() {
>
> Modified: lucene/solr/trunk/src/java/org/apache/solr/search/function/ValueSource.java
> URL: http://svn.apache.org/viewvc/lucene/solr/trunk/src/java/org/apache/solr/search/function/ValueSource.java?rev=816202&r1=816201&r2=816202&view=diff
> ==============================================================================
> --- lucene/solr/trunk/src/java/org/apache/solr/search/function/ValueSource.java (original)
> +++ lucene/solr/trunk/src/java/org/apache/solr/search/function/ValueSource.java Thu Sep 17 14:08:13 2009
> @@ -23,6 +23,8 @@
>
>  import java.io.IOException;
>  import java.io.Serializable;
> +import java.util.IdentityHashMap;
> +import java.util.Map;
>
>  /**
>  * Instantiates {@link org.apache.solr.search.function.DocValues} for a particular reader.
> @@ -33,7 +35,16 @@
>  */
>  public abstract class ValueSource implements Serializable {
>
> -  public abstract DocValues getValues(IndexReader reader) throws IOException;
> +  public DocValues getValues(IndexReader reader) throws IOException {
> +    return getValues(null, reader);
> +  }
> +
> +  /** Gets the values for this reader and the context that was previously
> +   * passed to createWeight()
> +   */
> +  public DocValues getValues(Map context, IndexReader reader) throws IOException {
> +    return null;
> +  }
>
>   public abstract boolean equals(Object o);
>
> @@ -46,6 +57,16 @@
>     return description();
>   }
>
> +  /** Implementations should propagate createWeight to sub-ValueSources which can optionally store
> +   * weight info in the context. The context object will be passed to getValues()
> +   * where this info can be retrieved. */
> +  public void createWeight(Map context, Searcher searcher) throws IOException {
> +  }
> +
> +  /** Returns a new non-threadsafe context map. */
> +  public static Map newContext() {
> +    return new IdentityHashMap();
> +  }
>  }
>
>
>
> Modified: lucene/solr/trunk/src/java/org/apache/solr/search/function/ValueSourceRangeFilter.java
> URL: http://svn.apache.org/viewvc/lucene/solr/trunk/src/java/org/apache/solr/search/function/ValueSourceRangeFilter.java?rev=816202&r1=816201&r2=816202&view=diff
> ==============================================================================
> --- lucene/solr/trunk/src/java/org/apache/solr/search/function/ValueSourceRangeFilter.java (original)
> +++ lucene/solr/trunk/src/java/org/apache/solr/search/function/ValueSourceRangeFilter.java Thu Sep 17 14:08:13 2009
> @@ -20,15 +20,18 @@
>  import org.apache.lucene.search.Filter;
>  import org.apache.lucene.search.DocIdSet;
>  import org.apache.lucene.search.DocIdSetIterator;
> +import org.apache.lucene.search.Searcher;
>  import org.apache.lucene.index.IndexReader;
> +import org.apache.solr.search.SolrFilter;
>
>  import java.io.IOException;
> +import java.util.Map;
>
>
>  /**
>  * RangeFilter over a ValueSource.
>  */
> -public class ValueSourceRangeFilter extends Filter {
> +public class ValueSourceRangeFilter extends SolrFilter {
>   private final ValueSource valueSource;
>   private final String lowerVal;
>   private final String upperVal;
> @@ -47,14 +50,19 @@
>     this.includeUpper = upperVal != null && includeUpper;
>   }
>
> -  public DocIdSet getDocIdSet(final IndexReader reader) throws IOException {
> +  public DocIdSet getDocIdSet(final Map context, final IndexReader reader) throws IOException {
>      return new DocIdSet() {
>        public DocIdSetIterator iterator() throws IOException {
> -         return valueSource.getValues(reader).getRangeScorer(reader, lowerVal, upperVal, includeLower, includeUpper);
> +         return valueSource.getValues(context, reader).getRangeScorer(reader, lowerVal, upperVal, includeLower, includeUpper);
>        }
>      };
>   }
>
> +  @Override
> +  public void createWeight(Map context, Searcher searcher) throws IOException {
> +    valueSource.createWeight(context, searcher);
> +  }
> +
>   public String toString() {
>     StringBuilder sb = new StringBuilder();
>     sb.append("frange(");
>
> Modified: lucene/solr/trunk/src/test/org/apache/solr/search/function/TestFunctionQuery.java
> URL: http://svn.apache.org/viewvc/lucene/solr/trunk/src/test/org/apache/solr/search/function/TestFunctionQuery.java?rev=816202&r1=816201&r2=816202&view=diff
> ==============================================================================
> --- lucene/solr/trunk/src/test/org/apache/solr/search/function/TestFunctionQuery.java (original)
> +++ lucene/solr/trunk/src/test/org/apache/solr/search/function/TestFunctionQuery.java Thu Sep 17 14:08:13 2009
> @@ -201,6 +201,7 @@
>   public void testFunctions() {
>     doTest("foo_pf");  // a plain float field
>     doTest("foo_f");  // a sortable float field
> +    doTest("foo_tf");  // a trie float field
>   }
>
>   public void testExternalField() {
> @@ -304,5 +305,26 @@
>     assertQ(req("fl","*,score","q", "{!func}ms(2009-08-31T12:10:10.125Z,b_tdt)", "fq","id:1"), "//float[@name='score']='1.0'");
>
>     assertQ(req("fl","*,score","q", "{!func}ms(2009-08-31T12:10:10.125Z/SECOND,2009-08-31T12:10:10.124Z/SECOND)", "fq","id:1"), "//float[@name='score']='0.0'");
> +
> +    for (int i=100; i<112; i++) {
> +      assertU(adoc("id",""+i, "text","batman"));
> +    }
> +    assertU(commit());
> +    assertU(adoc("id","120", "text","batman superman"));   // in a segment by itself
> +    assertU(commit());
> +
> +    // batman and superman have the same idf in single-doc segment, but very different in the complete index.
> +    String q ="{!func}query($qq)";
> +    String fq="id:120";
> +    assertQ(req("fl","*,score","q", q, "qq","text:batman", "fq",fq), "//float[@name='score']<'1.0'");
> +    assertQ(req("fl","*,score","q", q, "qq","text:superman", "fq",fq), "//float[@name='score']>'1.0'");
> +
> +    // test weighting through a function range query
> +    assertQ(req("fl","*,score", "q", "{!frange l=1 u=10}query($qq)", "qq","text:superman"), "//*[@numFound='1']");
> +
> +    // test weighting through a complex function
> +    q ="{!func}sub(div(sum(0.0,product(1,query($qq))),1),0)";
> +    assertQ(req("fl","*,score","q", q, "qq","text:batman", "fq",fq), "//float[@name='score']<'1.0'");
> +    assertQ(req("fl","*,score","q", q, "qq","text:superman", "fq",fq), "//float[@name='score']>'1.0'");
>   }
>  }
> \ No newline at end of file
>
>
>



-- 
-----------------------------------------------------
Noble Paul | Principal Engineer| AOL | http://aol.com