You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by yo...@apache.org on 2011/01/20 01:31:06 UTC

svn commit: r1061075 - in /lucene/dev/branches/branch_3x: ./ lucene/ solr/ solr/src/java/org/apache/solr/handler/component/ solr/src/java/org/apache/solr/search/ solr/src/java/org/apache/solr/search/function/ solr/src/test/org/apache/solr/ solr/src/tes...

Author: yonik
Date: Thu Jan 20 00:31:05 2011
New Revision: 1061075

URL: http://svn.apache.org/viewvc?rev=1061075&view=rev
Log:
SOLR-1297: fix weighting when sorting by function

Added:
    lucene/dev/branches/branch_3x/solr/src/java/org/apache/solr/search/SolrSortField.java   (with props)
Modified:
    lucene/dev/branches/branch_3x/   (props changed)
    lucene/dev/branches/branch_3x/lucene/   (props changed)
    lucene/dev/branches/branch_3x/solr/   (props changed)
    lucene/dev/branches/branch_3x/solr/src/java/org/apache/solr/handler/component/QueryComponent.java
    lucene/dev/branches/branch_3x/solr/src/java/org/apache/solr/search/SolrIndexSearcher.java
    lucene/dev/branches/branch_3x/solr/src/java/org/apache/solr/search/function/ValueSource.java
    lucene/dev/branches/branch_3x/solr/src/test/org/apache/solr/TestDistributedSearch.java
    lucene/dev/branches/branch_3x/solr/src/test/org/apache/solr/search/function/TestFunctionQuery.java

Modified: lucene/dev/branches/branch_3x/solr/src/java/org/apache/solr/handler/component/QueryComponent.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/solr/src/java/org/apache/solr/handler/component/QueryComponent.java?rev=1061075&r1=1061074&r2=1061075&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/solr/src/java/org/apache/solr/handler/component/QueryComponent.java (original)
+++ lucene/dev/branches/branch_3x/solr/src/java/org/apache/solr/handler/component/QueryComponent.java Thu Jan 20 00:31:05 2011
@@ -194,7 +194,7 @@ public class QueryComponent extends Sear
     // take the documents given and re-derive the sort values.
     boolean fsv = req.getParams().getBool(ResponseBuilder.FIELD_SORT_VALUES,false);
     if(fsv){
-      Sort sort = rb.getSortSpec().getSort();
+      Sort sort = searcher.weightSort(rb.getSortSpec().getSort());
       SortField[] sortFields = sort==null ? new SortField[]{SortField.FIELD_SCORE} : sort.getSort();
       NamedList sortVals = new NamedList(); // order is important for the sort fields
       Field field = new Field("dummy", "", Field.Store.YES, Field.Index.NO); // a dummy Field

Modified: lucene/dev/branches/branch_3x/solr/src/java/org/apache/solr/search/SolrIndexSearcher.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/solr/src/java/org/apache/solr/search/SolrIndexSearcher.java?rev=1061075&r1=1061074&r2=1061075&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/solr/src/java/org/apache/solr/search/SolrIndexSearcher.java (original)
+++ lucene/dev/branches/branch_3x/solr/src/java/org/apache/solr/search/SolrIndexSearcher.java Thu Jan 20 00:31:05 2011
@@ -481,6 +481,30 @@ public class SolrIndexSearcher extends I
     return fieldValueCache;
   }
 
+  /** Returns a weighted sort according to this searcher */
+  public Sort weightSort(Sort sort) throws IOException {
+    if (sort == null) return null;
+    SortField[] sorts = sort.getSort();
+
+    boolean needsWeighting = false;
+    for (SortField sf : sorts) {
+      if (sf instanceof SolrSortField) {
+        needsWeighting = true;
+        break;
+      }
+    }
+    if (!needsWeighting) return sort;
+
+    SortField[] newSorts = Arrays.copyOf(sorts, sorts.length);
+    for (int i=0; i<newSorts.length; i++) {
+      if (newSorts[i] instanceof SolrSortField) {
+        newSorts[i] = ((SolrSortField)newSorts[i]).weight(this);
+      }
+    }
+
+    return new Sort(newSorts);
+  }
+
 
   /**
    * Returns the first document number containing the term <code>t</code>
@@ -1131,7 +1155,7 @@ public class SolrIndexSearcher extends I
       if (cmd.getSort() == null) {
         topCollector = TopScoreDocCollector.create(len, true);
       } else {
-        topCollector = TopFieldCollector.create(cmd.getSort(), len, false, needScores, needScores, true);
+        topCollector = TopFieldCollector.create(weightSort(cmd.getSort()), len, false, needScores, needScores, true);
       }
       Collector collector = topCollector;
       if( timeAllowed > 0 ) {
@@ -1241,7 +1265,7 @@ public class SolrIndexSearcher extends I
       if (cmd.getSort() == null) {
         topCollector = TopScoreDocCollector.create(len, true);
       } else {
-        topCollector = TopFieldCollector.create(cmd.getSort(), len, false, needScores, needScores, true);
+        topCollector = TopFieldCollector.create(weightSort(cmd.getSort()), len, false, needScores, needScores, true);
       }
 
       DocSetCollector setCollector = new DocSetDelegateCollector(maxDoc>>6, maxDoc, topCollector);
@@ -1523,7 +1547,7 @@ public class SolrIndexSearcher extends I
     // bit of a hack to tell if a set is sorted - do it better in the futute.
     boolean inOrder = set instanceof BitDocSet || set instanceof SortedIntDocSet;
 
-    TopDocsCollector topCollector = TopFieldCollector.create(sort, nDocs, false, false, false, inOrder);
+    TopDocsCollector topCollector = TopFieldCollector.create(weightSort(sort), nDocs, false, false, false, inOrder);
 
     DocIterator iter = set.iterator();
     int base=0;

Added: lucene/dev/branches/branch_3x/solr/src/java/org/apache/solr/search/SolrSortField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/solr/src/java/org/apache/solr/search/SolrSortField.java?rev=1061075&view=auto
==============================================================================
--- lucene/dev/branches/branch_3x/solr/src/java/org/apache/solr/search/SolrSortField.java (added)
+++ lucene/dev/branches/branch_3x/solr/src/java/org/apache/solr/search/SolrSortField.java Thu Jan 20 00:31:05 2011
@@ -0,0 +1,31 @@
+/**
+ * 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.IndexSearcher;
+import org.apache.lucene.search.SortField;
+
+import java.io.IOException;
+
+/**@lucene.internal
+ *
+ */
+public interface SolrSortField {
+  public SortField weight(IndexSearcher searcher) throws IOException;
+}

Modified: lucene/dev/branches/branch_3x/solr/src/java/org/apache/solr/search/function/ValueSource.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/solr/src/java/org/apache/solr/search/function/ValueSource.java?rev=1061075&r1=1061074&r2=1061075&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/solr/src/java/org/apache/solr/search/function/ValueSource.java (original)
+++ lucene/dev/branches/branch_3x/solr/src/java/org/apache/solr/search/function/ValueSource.java Thu Jan 20 00:31:05 2011
@@ -18,12 +18,9 @@
 package org.apache.solr.search.function;
 
 import org.apache.lucene.index.IndexReader;
-import org.apache.lucene.search.Explanation;
-import org.apache.lucene.search.FieldComparator;
-import org.apache.lucene.search.FieldComparatorSource;
-import org.apache.lucene.search.Scorer;
-import org.apache.lucene.search.Searcher;
-import org.apache.lucene.search.SortField;
+import org.apache.lucene.search.*;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.search.SolrSortField;
 
 import java.io.IOException;
 import java.io.Serializable;
@@ -66,50 +63,70 @@ public abstract class ValueSource implem
     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();
+  }
+
+  //
+  // Sorting by function
+  //
+
   /**
    * EXPERIMENTAL: This method is subject to change.
    * <br>WARNING: Sorted function queries are not currently weighted.
    * <p>
-   * Get the SortField for this ValueSource.  Uses the {@link #getValues(java.util.Map, org.apache.lucene.index.IndexReader)}
+   * Get the SortField for this ValueSource.  Uses the {@link #getValues(java.util.Map, IndexReader)}
    * to populate the SortField.
-   * 
+   *
    * @param reverse true if this is a reverse sort.
    * @return The {@link org.apache.lucene.search.SortField} for the ValueSource
    * @throws IOException if there was a problem reading the values.
    */
   public SortField getSortField(boolean reverse) throws IOException {
-    //should we pass in the description for the field name?
-    //Hmm, Lucene is going to intern whatever we pass in, not sure I like that
-    //and we can't pass in null, either, as that throws an illegal arg. exception
-    return new SortField(description(), new ValueSourceComparatorSource(), reverse);
+    return new ValueSourceSortField(reverse);
   }
 
+  private static FieldComparatorSource dummyComparator = new FieldComparatorSource() {
+    @Override
+    public FieldComparator newComparator(String fieldname, int numHits, int sortPos, boolean reversed) throws IOException {
+      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Unweighted use of sort " + fieldname);
+    }
+  };
 
-  /**
-   * 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 {
-  }
+  class ValueSourceSortField extends SortField implements SolrSortField {
+    public ValueSourceSortField(boolean reverse) {
+      super(description(), dummyComparator, reverse);
+    }
 
-  /**
-   * Returns a new non-threadsafe context map.
-   */
-  public static Map newContext() {
-    return new IdentityHashMap();
+    @Override
+    public SortField weight(IndexSearcher searcher) throws IOException {
+      Map context = newContext();
+      createWeight(context, searcher);
+      return new SortField(getField(), new ValueSourceComparatorSource(context), getReverse());
+    }
   }
 
   class ValueSourceComparatorSource extends FieldComparatorSource {
+    private final Map context;
 
-
-    public ValueSourceComparatorSource() {
-
+    public ValueSourceComparatorSource(Map context) {
+      this.context = context;
     }
 
     public FieldComparator newComparator(String fieldname, int numHits,
                                          int sortPos, boolean reversed) throws IOException {
-      return new ValueSourceComparator(numHits);
+      return new ValueSourceComparator(context, numHits);
     }
   }
 
@@ -122,8 +139,10 @@ public abstract class ValueSource implem
     private final double[] values;
     private DocValues docVals;
     private double bottom;
+    private Map fcontext;
 
-    ValueSourceComparator(int numHits) {
+    ValueSourceComparator(Map fcontext, int numHits) {
+      this.fcontext = fcontext;
       values = new double[numHits];
     }
 
@@ -155,8 +174,9 @@ public abstract class ValueSource implem
       values[slot] = docVals.doubleVal(doc);
     }
 
+    @Override
     public void setNextReader(IndexReader reader, int docBase) throws IOException {
-      docVals = getValues(Collections.emptyMap(), reader);
+      docVals = getValues(fcontext, reader);
     }
 
     public void setBottom(final int bottom) {
@@ -164,7 +184,7 @@ public abstract class ValueSource implem
     }
 
     public Comparable value(int slot) {
-      return Double.valueOf(values[slot]);
+      return values[slot];
     }
   }
 }

Modified: lucene/dev/branches/branch_3x/solr/src/test/org/apache/solr/TestDistributedSearch.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/solr/src/test/org/apache/solr/TestDistributedSearch.java?rev=1061075&r1=1061074&r2=1061075&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/solr/src/test/org/apache/solr/TestDistributedSearch.java (original)
+++ lucene/dev/branches/branch_3x/solr/src/test/org/apache/solr/TestDistributedSearch.java Thu Jan 20 00:31:05 2011
@@ -95,6 +95,7 @@ public class TestDistributedSearch exten
 
     // these queries should be exactly ordered and scores should exactly match
     query("q","*:*", "sort",i1+" desc");
+    query("q","*:*", "sort","{!func}add("+i1+",5)"+" desc");
     query("q","*:*", "sort",i1+" asc");
     query("q","*:*", "sort",i1+" desc", "fl","*,score");
     query("q","*:*", "sort",tlong+" asc", "fl","score");  // test legacy behavior - "score"=="*,score"

Modified: lucene/dev/branches/branch_3x/solr/src/test/org/apache/solr/search/function/TestFunctionQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/solr/src/test/org/apache/solr/search/function/TestFunctionQuery.java?rev=1061075&r1=1061074&r2=1061075&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/solr/src/test/org/apache/solr/search/function/TestFunctionQuery.java (original)
+++ lucene/dev/branches/branch_3x/solr/src/test/org/apache/solr/search/function/TestFunctionQuery.java Thu Jan 20 00:31:05 2011
@@ -301,17 +301,18 @@ public class TestFunctionQuery extends S
       assertU(adoc("id",""+i, "text","batman"));
     }
     assertU(commit());
-    assertU(adoc("id","120", "text","batman superman"));   // in a segment by itself
+    assertU(adoc("id","120", "text","batman superman"));   // in a smaller segment
+    assertU(adoc("id","121", "text","superman"));
     assertU(commit());
 
-    // batman and superman have the same idf in single-doc segment, but very different in the complete index.
+    // superman has a higher df (thus lower idf) in one segment, but reversed 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']");
+    assertQ(req("fl","*,score", "fq",fq,  "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)";
@@ -335,6 +336,14 @@ public class TestFunctionQuery extends S
       // OK
     }
 
+    // test that sorting by function weights correctly.  superman should sort higher than batman due to idf of the whole index
+
+    assertQ(req("q", "*:*", "fq","id:120 OR id:121", "sort","{!func v=$sortfunc} desc", "sortfunc","query($qq)", "qq","text:(batman OR superman)")
+           ,"*//doc[1]/float[.='120.0']"
+           ,"*//doc[2]/float[.='121.0']"
+    );
+
+
     purgeFieldCache(FieldCache.DEFAULT);   // avoid FC insanity
   }