You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@lucene.apache.org by Grant Ingersoll <gs...@apache.org> on 2009/08/05 16:02:39 UTC

SpanQuery and BoostingTermQuery oddities

A BoostingTermQuery (BTQ) is a SpanQuery.

If I run:
     IndexSearcher searcher = new IndexSearcher(dir, true);
     searcher.setSimilarity(payloadSimilarity);//set the similarity.   
Very important
     BoostingTermQuery btq = new BoostingTermQuery(new Term("body",  
"fox"));
     TopDocs topDocs = searcher.search(btq, 10);
     printResults(searcher, btq, topDocs);

I get, as expected, documents that contain "fox" with a payload  
boosted higher than those containing fox without a boost.  (See [1]  
for full code)
Output is:
Doc: doc=0 score=4.2344446
Explain: 4.234444 = (MATCH) fieldWeight(body:fox in 0), product of:
   7.071068 = (MATCH) btq, product of:
     0.70710677 = tf(phraseFreq=0.5)
     10.0 = scorePayload(...)
   1.9162908 = idf(body: fox=3)
   0.3125 = fieldNorm(field=body, doc=0)

Doc: doc=2 score=4.2344446
Explain: 4.234444 = (MATCH) fieldWeight(body:fox in 2), product of:
   7.071068 = (MATCH) btq, product of:
     0.70710677 = tf(phraseFreq=0.5)
     10.0 = scorePayload(...)
   1.9162908 = idf(body: fox=3)
   0.3125 = fieldNorm(field=body, doc=2)

Doc: doc=1 score=0.42344445
Explain: 0.42344445 = (MATCH) fieldWeight(body:fox in 1), product of:
   0.70710677 = (MATCH) btq, product of:
     0.70710677 = tf(phraseFreq=0.5)
     1.0 = scorePayload(...)
   1.9162908 = idf(body: fox=3)
   0.3125 = fieldNorm(field=body, doc=1)



However, if I then add the BTQ to a SpanNearQuery, I do not get the  
expected results:
     SpanQuery[] queries = new SpanQuery[2];
     queries[0] = new BoostingTermQuery(new Term("body", "red"));
     queries[1] = new BoostingTermQuery(new Term("body", "fox"));
     SpanNearQuery near = new SpanNearQuery(queries, 2, true);
     topDocs = searcher.search(near, 10);
     printResults(searcher, near, topDocs);

Output is:
Doc: doc=0 score=0.6914818
Explain: 0.6914818 = (MATCH) fieldWeight(body:spanNear([red, fox], 2,  
true) in 0), product of:
   0.57735026 = tf(phraseFreq=0.33333334)
   3.8325815 = idf(body: fox=3 red=3)
   0.3125 = fieldNorm(field=body, doc=0)

Doc: doc=1 score=0.6914818
Explain: 0.6914818 = (MATCH) fieldWeight(body:spanNear([red, fox], 2,  
true) in 1), product of:
   0.57735026 = tf(phraseFreq=0.33333334)
   3.8325815 = idf(body: fox=3 red=3)
   0.3125 = fieldNorm(field=body, doc=1)

Doc: doc=2 score=0.6914818
Explain: 0.6914818 = (MATCH) fieldWeight(body:spanNear([red, fox], 2,  
true) in 2), product of:
   0.57735026 = tf(phraseFreq=0.33333334)
   3.8325815 = idf(body: fox=3 red=3)
   0.3125 = fieldNorm(field=body, doc=2)


It seems the BTQ score method is not being called.  One of the main  
points of the SpanNearQuery is that it can take in complex subclauses,  
presumably rolling up scores from the subclauses.  Yet that appears to  
not be the case.  Instead it just seems to rely on the matches that  
get produced by those subclauses, but not the scoring.  Is my  
understanding correct?  If so, is that the correct functionality?

I'm not a spans expert (SpanNearQuery always confuses me with the  
NearSpansOrdered/Unordered), but it seems like the SpanNearQuery (and  
likely others that take clauses) needs to create a QueryWeight object  
that is made up of the QueryWeight objects from it's subclauses, right?

Thoughts?

Thanks,
Grant

[1]
import junit.framework.TestCase;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.RAMDirectory;
import org.apache.lucene.index.IndexWriter;
import org.apache.lucene.index.Term;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field;
import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.analysis.TokenStream;
import org.apache.lucene.analysis.WhitespaceTokenizer;
import org.apache.lucene.analysis.LowerCaseFilter;
import org.apache.lucene.analysis.payloads.DelimitedPayloadTokenFilter;
import org.apache.lucene.analysis.payloads.PayloadEncoder;
import org.apache.lucene.analysis.payloads.FloatEncoder;
import org.apache.lucene.analysis.payloads.PayloadHelper;
import org.apache.lucene.search.DefaultSimilarity;
import org.apache.lucene.search.IndexSearcher;
import org.apache.lucene.search.TopDocs;
import org.apache.lucene.search.ScoreDoc;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.payloads.BoostingTermQuery;
import org.apache.lucene.search.spans.SpanNearQuery;
import org.apache.lucene.search.spans.SpanQuery;

import java.io.Reader;
import java.io.IOException;


/**
  *
  *
  **/
public class PayloadTest extends TestCase {
Directory dir;



   public static String[] DOCS = {
           "The quick|2.0 red|2.0 fox|10.0 jumped|5.0 over the lazy| 
2.0 brown|2.0 dogs|10.0",
           "The quick red fox jumped over the lazy brown dogs",//no  
boosts
           "The quick|2.0 red|2.0 fox|10.0 jumped|5.0 over the old|2.0  
brown|2.0 box|10.0",
           "Mary|10.0 had a little|2.0 lamb|10.0 whose fleece|10.0 was| 
5.0 white|2.0 as snow|10.0",
           "Mary had a little lamb whose fleece was white as snow",
           "Mary|10.0 takes on Wolf|10.0 Restoration|10.0 project|10.0  
despite ties|10.0 to sheep|10.0 farming|10.0",
           "Mary|10.0 who lives|5.0 on a farm|10.0 is|5.0 happy|2.0  
that she|10.0 takes|5.0 a walk|10.0 every day|10.0",
           "Moby|10.0 Dick|10.0 is|5.0 a story|10.0 of a whale|10.0  
and a man|10.0 obsessed|10.0",
           "The robber|10.0 wore|5.0 a black|2.0 fleece|10.0 jacket| 
10.0 and a baseball|10.0 cap|10.0",
           "The English|10.0 Springer|10.0 Spaniel|10.0 is|5.0 the  
best|2.0 of all dogs|10.0"
   };
   protected PayloadSimilarity payloadSimilarity;

   @Override
   protected void setUp() throws Exception {
     dir = new RAMDirectory();

     PayloadEncoder encoder = new FloatEncoder();
     IndexWriter writer = new IndexWriter(dir, new  
PayloadAnalyzer(encoder), true, IndexWriter.MaxFieldLength.UNLIMITED);
     payloadSimilarity = new PayloadSimilarity();
     writer.setSimilarity(payloadSimilarity);
     for (int i = 0; i < DOCS.length; i++) {
       Document doc = new Document();
       Field id = new Field("id", "doc_" + i, Field.Store.YES,  
Field.Index.NOT_ANALYZED_NO_NORMS);
       doc.add(id);
       //Store both position and offset information
       Field text = new Field("body", DOCS[i], Field.Store.NO,  
Field.Index.ANALYZED);
       doc.add(text);
       writer.addDocument(doc);
     }
     writer.close();
   }


   public void testPayloads() throws Exception {
     IndexSearcher searcher = new IndexSearcher(dir, true);
     searcher.setSimilarity(payloadSimilarity);//set the similarity.   
Very important
     BoostingTermQuery btq = new BoostingTermQuery(new Term("body",  
"fox"));
     TopDocs topDocs = searcher.search(btq, 10);
     printResults(searcher, btq, topDocs);
     System.out.println("-----------");
     System.out.println("Try out some Spans");
     SpanQuery[] queries = new SpanQuery[2];
     queries[0] = new BoostingTermQuery(new Term("body", "red"));
     queries[1] = new BoostingTermQuery(new Term("body", "fox"));
     SpanNearQuery near = new SpanNearQuery(queries, 2, true);
     topDocs = searcher.search(near, 10);
     printResults(searcher, near, topDocs);

   }

   private void printResults(IndexSearcher searcher, Query btq,  
TopDocs topDocs) throws IOException {
     for (int i = 0; i < topDocs.scoreDocs.length; i++) {
       ScoreDoc doc = topDocs.scoreDocs[i];
       System.out.println("Doc: " + doc.toString());
       System.out.println("Explain: " + searcher.explain(btq, doc.doc));
     }
   }

   class PayloadSimilarity extends DefaultSimilarity {
     @Override
     public float scorePayload(String fieldName, byte[] bytes, int  
offset, int length) {
       return PayloadHelper.decodeFloat(bytes, offset);//we can ignore  
length here, because we know it is encoded as 4 bytes
     }
   }

   class PayloadAnalyzer extends Analyzer {
     private PayloadEncoder encoder;

     PayloadAnalyzer(PayloadEncoder encoder) {
       this.encoder = encoder;
     }

     public TokenStream tokenStream(String fieldName, Reader reader) {
       TokenStream result = new WhitespaceTokenizer(reader);
       result = new LowerCaseFilter(result);
       result = new DelimitedPayloadTokenFilter(result, '|', encoder);
       return result;
     }
   }
}


---------------------------------------------------------------------
To unsubscribe, e-mail: java-dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: java-dev-help@lucene.apache.org


Re: SpanQuery and BoostingTermQuery oddities

Posted by Mark Miller <ma...@gmail.com>.
If we end up keeping Weight, I don't think the Changes entry on back 
compat says enough? If you overrode   protected Weight 
createWeight(Query query) throws IOException, its not called anymore so 
you are dead in the water? And just overall, it appears to be a break, 
but the instructions don't tell me enough about how to handle - just the 
Wrapper mention is not enough?

- Mark

Mark Miller wrote:
> This is a little different than what Grant was talking about.
>
> For the old variants to work, I think you have to call the Searcher 
> methods that take an old style weight.
> Its not an ideal break, I know.
>
> I think we might consider just dropping Weight? If we make the issue 
> with 1771 a compile time break (which I think makes sense) I think 
> that leads to just dropping Weight for QueryWeight.
>
> - Mark
>
> Peter Keegan wrote:
>> I got it to work by adding new overrides:
>>
>>   public QueryWeight createQueryWeight(Searcher searcher) throws 
>> IOException
>>   public Scorer scorer(IndexReader reader, boolean scoreDocsInOrder, 
>> boolean topScorer) throws IOException
>>
>> But, it seems like it should still work with the old overrides:
>>
>> protected Weight createWeight(Searcher searcher) throws IOException
>> public Scorer scorer(IndexReader reader) throws IOException
>>
>> Peter
>>
>>
>> On Wed, Aug 5, 2009 at 12:29 PM, Peter Keegan <peterlkeegan@gmail.com 
>> <ma...@gmail.com>> wrote:
>>
>>     I ran into the same problem trying to update the BoostingNearQuery
>>     patch in *LUCENE-1341
>>     <https://issues.apache.org/jira/browse/LUCENE-1341>.
>>     *The scorer never gets called. This used to work in 2.3.2
>>
>>     Peter
>>
>>
>>     On Wed, Aug 5, 2009 at 11:01 AM, Mark Miller
>>     <markrmiller@gmail.com <ma...@gmail.com>> wrote:
>>
>>         Grant Ingersoll wrote:
>>
>>
>>             On Aug 5, 2009, at 10:07 AM, Mark Miller wrote:
>>
>>                 Yeah - SpanQuery's don't use the boosts from subspans
>>                 - it just uses the idf for the query terms and the
>>                 span length I believe - and the boost for the top
>>                 level Query.
>>
>>                 Is that the right way to go? I guess Doug seemed to
>>                 think so? I don't know. It is sort of a bug that lower
>>                 boosts would be ignored right? There is an issue for
>>                 it somewhere.
>>
>>                 It gets complicated quick to change it - all of a
>>                 sudden you need something like BooleanQuery ...
>>
>>
>>             Not sure it needs BooleanQuery, but it does seem like it
>>             should take into account the scores of the subclauses
>>             (regardless of BoostingTermQuery).  There is a spot in
>>             creating the SpanScorer where it gets the value from the
>>             QueryWeight, but this QueryWeight does not account for the
>>             subclauses QueryWeights.
>>
>>
>>         It doesn't need BooleanQuery - it needs BooleanQuery type
>>         logic - which is fairly complicated. At least to do it right I
>>         think. I don't have a clear memory of it, but I started to try
>>         and address this once and ...
>>         well I didn't continue.
>>
>>
>>         --         - Mark
>>
>>         http://www.lucidimagination.com
>>
>>
>>
>>
>>         
>> ---------------------------------------------------------------------
>>         To unsubscribe, e-mail: java-dev-unsubscribe@lucene.apache.org
>>         <ma...@lucene.apache.org>
>>         For additional commands, e-mail:
>>         java-dev-help@lucene.apache.org
>>         <ma...@lucene.apache.org>
>>
>>
>>
>
>


-- 
- Mark

http://www.lucidimagination.com




---------------------------------------------------------------------
To unsubscribe, e-mail: java-dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: java-dev-help@lucene.apache.org


Re: SpanQuery and BoostingTermQuery oddities

Posted by Mark Miller <ma...@gmail.com>.
This is a little different than what Grant was talking about.

For the old variants to work, I think you have to call the Searcher 
methods that take an old style weight.
Its not an ideal break, I know.

I think we might consider just dropping Weight? If we make the issue 
with 1771 a compile time break (which I think makes sense) I think that 
leads to just dropping Weight for QueryWeight.

- Mark

Peter Keegan wrote:
> I got it to work by adding new overrides:
>
>   public QueryWeight createQueryWeight(Searcher searcher) throws 
> IOException
>   public Scorer scorer(IndexReader reader, boolean scoreDocsInOrder, 
> boolean topScorer) throws IOException
>
> But, it seems like it should still work with the old overrides:
>
> protected Weight createWeight(Searcher searcher) throws IOException
> public Scorer scorer(IndexReader reader) throws IOException
>
> Peter
>
>
> On Wed, Aug 5, 2009 at 12:29 PM, Peter Keegan <peterlkeegan@gmail.com 
> <ma...@gmail.com>> wrote:
>
>     I ran into the same problem trying to update the BoostingNearQuery
>     patch in *LUCENE-1341
>     <https://issues.apache.org/jira/browse/LUCENE-1341>.
>     *The scorer never gets called. This used to work in 2.3.2
>
>     Peter
>
>
>     On Wed, Aug 5, 2009 at 11:01 AM, Mark Miller
>     <markrmiller@gmail.com <ma...@gmail.com>> wrote:
>
>         Grant Ingersoll wrote:
>
>
>             On Aug 5, 2009, at 10:07 AM, Mark Miller wrote:
>
>                 Yeah - SpanQuery's don't use the boosts from subspans
>                 - it just uses the idf for the query terms and the
>                 span length I believe - and the boost for the top
>                 level Query.
>
>                 Is that the right way to go? I guess Doug seemed to
>                 think so? I don't know. It is sort of a bug that lower
>                 boosts would be ignored right? There is an issue for
>                 it somewhere.
>
>                 It gets complicated quick to change it - all of a
>                 sudden you need something like BooleanQuery ...
>
>
>             Not sure it needs BooleanQuery, but it does seem like it
>             should take into account the scores of the subclauses
>             (regardless of BoostingTermQuery).  There is a spot in
>             creating the SpanScorer where it gets the value from the
>             QueryWeight, but this QueryWeight does not account for the
>             subclauses QueryWeights.
>
>
>         It doesn't need BooleanQuery - it needs BooleanQuery type
>         logic - which is fairly complicated. At least to do it right I
>         think. I don't have a clear memory of it, but I started to try
>         and address this once and ...
>         well I didn't continue.
>
>
>         -- 
>         - Mark
>
>         http://www.lucidimagination.com
>
>
>
>
>         ---------------------------------------------------------------------
>         To unsubscribe, e-mail: java-dev-unsubscribe@lucene.apache.org
>         <ma...@lucene.apache.org>
>         For additional commands, e-mail:
>         java-dev-help@lucene.apache.org
>         <ma...@lucene.apache.org>
>
>
>


-- 
- Mark

http://www.lucidimagination.com




---------------------------------------------------------------------
To unsubscribe, e-mail: java-dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: java-dev-help@lucene.apache.org


Re: SpanQuery and BoostingTermQuery oddities

Posted by Peter Keegan <pe...@gmail.com>.
I got it to work by adding new overrides:

  public QueryWeight createQueryWeight(Searcher searcher) throws IOException

  public Scorer scorer(IndexReader reader, boolean scoreDocsInOrder, boolean
topScorer) throws IOException

But, it seems like it should still work with the old overrides:

protected Weight createWeight(Searcher searcher) throws IOException
public Scorer scorer(IndexReader reader) throws IOException

Peter


On Wed, Aug 5, 2009 at 12:29 PM, Peter Keegan <pe...@gmail.com>wrote:

> I ran into the same problem trying to update the BoostingNearQuery patch in
> *LUCENE-1341 <https://issues.apache.org/jira/browse/LUCENE-1341>.
> *The scorer never gets called. This used to work in 2.3.2
>
> Peter
>
>
> On Wed, Aug 5, 2009 at 11:01 AM, Mark Miller <ma...@gmail.com>wrote:
>
>> Grant Ingersoll wrote:
>>
>>>
>>> On Aug 5, 2009, at 10:07 AM, Mark Miller wrote:
>>>
>>>> Yeah - SpanQuery's don't use the boosts from subspans - it just uses the
>>>> idf for the query terms and the span length I believe - and the boost for
>>>> the top level Query.
>>>>
>>>> Is that the right way to go? I guess Doug seemed to think so? I don't
>>>> know. It is sort of a bug that lower boosts would be ignored right? There is
>>>> an issue for it somewhere.
>>>>
>>>> It gets complicated quick to change it - all of a sudden you need
>>>> something like BooleanQuery ...
>>>>
>>>>
>>> Not sure it needs BooleanQuery, but it does seem like it should take into
>>> account the scores of the subclauses (regardless of BoostingTermQuery).
>>>  There is a spot in creating the SpanScorer where it gets the value from the
>>> QueryWeight, but this QueryWeight does not account for the subclauses
>>> QueryWeights.
>>>
>>>
>> It doesn't need BooleanQuery - it needs BooleanQuery type logic - which is
>> fairly complicated. At least to do it right I think. I don't have a clear
>> memory of it, but I started to try and address this once and ...
>> well I didn't continue.
>>
>> --
>> - Mark
>>
>> http://www.lucidimagination.com
>>
>>
>>
>>
>> ---------------------------------------------------------------------
>> To unsubscribe, e-mail: java-dev-unsubscribe@lucene.apache.org
>> For additional commands, e-mail: java-dev-help@lucene.apache.org
>>
>>
>

Re: SpanQuery and BoostingTermQuery oddities

Posted by Peter Keegan <pe...@gmail.com>.
I ran into the same problem trying to update the BoostingNearQuery patch in
*LUCENE-1341 <https://issues.apache.org/jira/browse/LUCENE-1341>.
*The scorer never gets called. This used to work in 2.3.2

Peter

On Wed, Aug 5, 2009 at 11:01 AM, Mark Miller <ma...@gmail.com> wrote:

> Grant Ingersoll wrote:
>
>>
>> On Aug 5, 2009, at 10:07 AM, Mark Miller wrote:
>>
>>> Yeah - SpanQuery's don't use the boosts from subspans - it just uses the
>>> idf for the query terms and the span length I believe - and the boost for
>>> the top level Query.
>>>
>>> Is that the right way to go? I guess Doug seemed to think so? I don't
>>> know. It is sort of a bug that lower boosts would be ignored right? There is
>>> an issue for it somewhere.
>>>
>>> It gets complicated quick to change it - all of a sudden you need
>>> something like BooleanQuery ...
>>>
>>>
>> Not sure it needs BooleanQuery, but it does seem like it should take into
>> account the scores of the subclauses (regardless of BoostingTermQuery).
>>  There is a spot in creating the SpanScorer where it gets the value from the
>> QueryWeight, but this QueryWeight does not account for the subclauses
>> QueryWeights.
>>
>>
> It doesn't need BooleanQuery - it needs BooleanQuery type logic - which is
> fairly complicated. At least to do it right I think. I don't have a clear
> memory of it, but I started to try and address this once and ...
> well I didn't continue.
>
> --
> - Mark
>
> http://www.lucidimagination.com
>
>
>
>
> ---------------------------------------------------------------------
> To unsubscribe, e-mail: java-dev-unsubscribe@lucene.apache.org
> For additional commands, e-mail: java-dev-help@lucene.apache.org
>
>

Re: SpanQuery and BoostingTermQuery oddities

Posted by Eran Sevi <er...@gmail.com>.
I've managed to create some sort of solution to this problem -

The result is that we can get an equal score for a SpanOrQuery as a regular
BooleanQuery with only SHOULD clauses.
We can also get an equal score for a SpanNotQuery as a regular BooleanQuery
with only MUST clauses.

The good is that the score is calculated recursively and the boosts of the
inner queries are taken into account.
The bad in my solution is that the span distance is not taken into account
and that the spans are fetched for each sub query which can really affect
performance.

My solution is as follows:

1. Create a derived class for each "complex" span*Query that inherit from
SpanWeight (e.g. SpanNearWeight).
2. The new weight class is initialized with the SpanNearQuery and creates a
weight for each of the query's clauses - this gives us the recursive pass.
3. override the "SumOfSquaredWeights","Normalize" methods as the
BooleanWeight implementation.
4.  override the "Scorer" method as follows: create a BooleanScorer and add
the scorers from the weights of the sub queries. for SpanOrQuery add them as
not required and not prohibited. for SpanNearQuery add them as required and
not prohibited.
5. Override the "CreateWeight" method in the Span*Query to return the new
Weight class instead of the old SpanWeight class (the SpanWeight class will
still be returned for SpanTermQuery which doesn't contain any sub queries
and shouldn't be overriden).
6. optional - change the "SetFreqCurrentDoc" method in SpanScorer to sum the
freq in each doc instead of running SloppyFreq.

I hope you can understand the main idea from my complicated description.
The problem with the current spans implementation is that by the time you
have the spans you don't know how they were created - the span of a
complicated query or a simple query looks the same and treated the same.

With this method you can at least get a score for span queries which is not
the most accurate but at least take into account sub queries and boosts.
I haven't dealt with SpanNotQuery yet but I guess it can follow the same
base idea - create sub scorers with MUST and MUST_NOT for the
inclusive/exclusive sub queries of SpanNotQuery.

* I can attach my code for the above changes but I use Lucene.Net so the
code will be in c# and the Lucene version is 2.3.2.

Eran.
On Wed, Aug 12, 2009 at 1:25 PM, Michael McCandless <
lucene@mikemccandless.com> wrote:

> All Span*Query seem to rely on the SpanQuery.createWeight (which
> returns SpanWeight/SpanScorer) to make their weight/scorer.
> SpanScorer in turn simply enumerates all spans summing up their
> "sloppy freq" and always scoring with that, regardless of the sub
> queries.
>
> So SpanNearQuery (or any composite span query, eg even SpanFirstQuery
> I think will do this), disregards the scores of its child query/ies.
>
> I agree it's odd... it seems like composite span queries ought to take
> their child query scoring into account.  This would be a benefit of
> merging into the normal Query*, since these composite queries already
> factor in scoring from their sub queries.
>
> Mike
>
> On Wed, Aug 5, 2009 at 11:01 AM, Mark Miller<ma...@gmail.com> wrote:
> > Grant Ingersoll wrote:
> >>
> >> On Aug 5, 2009, at 10:07 AM, Mark Miller wrote:
> >>>
> >>> Yeah - SpanQuery's don't use the boosts from subspans - it just uses
> the
> >>> idf for the query terms and the span length I believe - and the boost
> for
> >>> the top level Query.
> >>>
> >>> Is that the right way to go? I guess Doug seemed to think so? I don't
> >>> know. It is sort of a bug that lower boosts would be ignored right?
> There is
> >>> an issue for it somewhere.
> >>>
> >>> It gets complicated quick to change it - all of a sudden you need
> >>> something like BooleanQuery ...
> >>>
> >>
> >> Not sure it needs BooleanQuery, but it does seem like it should take
> into
> >> account the scores of the subclauses (regardless of BoostingTermQuery).
> >>  There is a spot in creating the SpanScorer where it gets the value from
> the
> >> QueryWeight, but this QueryWeight does not account for the subclauses
> >> QueryWeights.
> >>
> >
> > It doesn't need BooleanQuery - it needs BooleanQuery type logic - which
> is
> > fairly complicated. At least to do it right I think. I don't have a clear
> > memory of it, but I started to try and address this once and ...
> > well I didn't continue.
> >
> > --
> > - Mark
> >
> > http://www.lucidimagination.com
> >
> >
> >
> >
> > ---------------------------------------------------------------------
> > To unsubscribe, e-mail: java-dev-unsubscribe@lucene.apache.org
> > For additional commands, e-mail: java-dev-help@lucene.apache.org
> >
> >
>
> ---------------------------------------------------------------------
> To unsubscribe, e-mail: java-dev-unsubscribe@lucene.apache.org
> For additional commands, e-mail: java-dev-help@lucene.apache.org
>
>

Re: SpanQuery and BoostingTermQuery oddities

Posted by Michael McCandless <lu...@mikemccandless.com>.
All Span*Query seem to rely on the SpanQuery.createWeight (which
returns SpanWeight/SpanScorer) to make their weight/scorer.
SpanScorer in turn simply enumerates all spans summing up their
"sloppy freq" and always scoring with that, regardless of the sub
queries.

So SpanNearQuery (or any composite span query, eg even SpanFirstQuery
I think will do this), disregards the scores of its child query/ies.

I agree it's odd... it seems like composite span queries ought to take
their child query scoring into account.  This would be a benefit of
merging into the normal Query*, since these composite queries already
factor in scoring from their sub queries.

Mike

On Wed, Aug 5, 2009 at 11:01 AM, Mark Miller<ma...@gmail.com> wrote:
> Grant Ingersoll wrote:
>>
>> On Aug 5, 2009, at 10:07 AM, Mark Miller wrote:
>>>
>>> Yeah - SpanQuery's don't use the boosts from subspans - it just uses the
>>> idf for the query terms and the span length I believe - and the boost for
>>> the top level Query.
>>>
>>> Is that the right way to go? I guess Doug seemed to think so? I don't
>>> know. It is sort of a bug that lower boosts would be ignored right? There is
>>> an issue for it somewhere.
>>>
>>> It gets complicated quick to change it - all of a sudden you need
>>> something like BooleanQuery ...
>>>
>>
>> Not sure it needs BooleanQuery, but it does seem like it should take into
>> account the scores of the subclauses (regardless of BoostingTermQuery).
>>  There is a spot in creating the SpanScorer where it gets the value from the
>> QueryWeight, but this QueryWeight does not account for the subclauses
>> QueryWeights.
>>
>
> It doesn't need BooleanQuery - it needs BooleanQuery type logic - which is
> fairly complicated. At least to do it right I think. I don't have a clear
> memory of it, but I started to try and address this once and ...
> well I didn't continue.
>
> --
> - Mark
>
> http://www.lucidimagination.com
>
>
>
>
> ---------------------------------------------------------------------
> To unsubscribe, e-mail: java-dev-unsubscribe@lucene.apache.org
> For additional commands, e-mail: java-dev-help@lucene.apache.org
>
>

---------------------------------------------------------------------
To unsubscribe, e-mail: java-dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: java-dev-help@lucene.apache.org


Re: SpanQuery and BoostingTermQuery oddities

Posted by Mark Miller <ma...@gmail.com>.
Grant Ingersoll wrote:
>
> On Aug 5, 2009, at 10:07 AM, Mark Miller wrote:
>> Yeah - SpanQuery's don't use the boosts from subspans - it just uses 
>> the idf for the query terms and the span length I believe - and the 
>> boost for the top level Query.
>>
>> Is that the right way to go? I guess Doug seemed to think so? I don't 
>> know. It is sort of a bug that lower boosts would be ignored right? 
>> There is an issue for it somewhere.
>>
>> It gets complicated quick to change it - all of a sudden you need 
>> something like BooleanQuery ...
>>
>
> Not sure it needs BooleanQuery, but it does seem like it should take 
> into account the scores of the subclauses (regardless of 
> BoostingTermQuery).  There is a spot in creating the SpanScorer where 
> it gets the value from the QueryWeight, but this QueryWeight does not 
> account for the subclauses QueryWeights.
>

It doesn't need BooleanQuery - it needs BooleanQuery type logic - which 
is fairly complicated. At least to do it right I think. I don't have a 
clear memory of it, but I started to try and address this once and ...
well I didn't continue.

-- 
- Mark

http://www.lucidimagination.com




---------------------------------------------------------------------
To unsubscribe, e-mail: java-dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: java-dev-help@lucene.apache.org


Re: SpanQuery and BoostingTermQuery oddities

Posted by Grant Ingersoll <gs...@apache.org>.
On Aug 5, 2009, at 10:07 AM, Mark Miller wrote:
> Yeah - SpanQuery's don't use the boosts from subspans - it just uses  
> the idf for the query terms and the span length I believe - and the  
> boost for the top level Query.
>
> Is that the right way to go? I guess Doug seemed to think so? I  
> don't know. It is sort of a bug that lower boosts would be ignored  
> right? There is an issue for it somewhere.
>
> It gets complicated quick to change it - all of a sudden you need  
> something like BooleanQuery ...
>

Not sure it needs BooleanQuery, but it does seem like it should take  
into account the scores of the subclauses (regardless of  
BoostingTermQuery).  There is a spot in creating the SpanScorer where  
it gets the value from the QueryWeight, but this QueryWeight does not  
account for the subclauses QueryWeights.




---------------------------------------------------------------------
To unsubscribe, e-mail: java-dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: java-dev-help@lucene.apache.org


Re: SpanQuery and BoostingTermQuery oddities

Posted by Mark Miller <ma...@gmail.com>.
Grant Ingersoll wrote:
> A BoostingTermQuery (BTQ) is a SpanQuery.
>
> If I run:
>     IndexSearcher searcher = new IndexSearcher(dir, true);
>     searcher.setSimilarity(payloadSimilarity);//set the similarity.  
> Very important
>     BoostingTermQuery btq = new BoostingTermQuery(new Term("body", 
> "fox"));
>     TopDocs topDocs = searcher.search(btq, 10);
>     printResults(searcher, btq, topDocs);
>
> I get, as expected, documents that contain "fox" with a payload 
> boosted higher than those containing fox without a boost.  (See [1] 
> for full code)
> Output is:
> Doc: doc=0 score=4.2344446
> Explain: 4.234444 = (MATCH) fieldWeight(body:fox in 0), product of:
>   7.071068 = (MATCH) btq, product of:
>     0.70710677 = tf(phraseFreq=0.5)
>     10.0 = scorePayload(...)
>   1.9162908 = idf(body: fox=3)
>   0.3125 = fieldNorm(field=body, doc=0)
>
> Doc: doc=2 score=4.2344446
> Explain: 4.234444 = (MATCH) fieldWeight(body:fox in 2), product of:
>   7.071068 = (MATCH) btq, product of:
>     0.70710677 = tf(phraseFreq=0.5)
>     10.0 = scorePayload(...)
>   1.9162908 = idf(body: fox=3)
>   0.3125 = fieldNorm(field=body, doc=2)
>
> Doc: doc=1 score=0.42344445
> Explain: 0.42344445 = (MATCH) fieldWeight(body:fox in 1), product of:
>   0.70710677 = (MATCH) btq, product of:
>     0.70710677 = tf(phraseFreq=0.5)
>     1.0 = scorePayload(...)
>   1.9162908 = idf(body: fox=3)
>   0.3125 = fieldNorm(field=body, doc=1)
>
>
>
> However, if I then add the BTQ to a SpanNearQuery, I do not get the 
> expected results:
>     SpanQuery[] queries = new SpanQuery[2];
>     queries[0] = new BoostingTermQuery(new Term("body", "red"));
>     queries[1] = new BoostingTermQuery(new Term("body", "fox"));
>     SpanNearQuery near = new SpanNearQuery(queries, 2, true);
>     topDocs = searcher.search(near, 10);
>     printResults(searcher, near, topDocs);
>
> Output is:
> Doc: doc=0 score=0.6914818
> Explain: 0.6914818 = (MATCH) fieldWeight(body:spanNear([red, fox], 2, 
> true) in 0), product of:
>   0.57735026 = tf(phraseFreq=0.33333334)
>   3.8325815 = idf(body: fox=3 red=3)
>   0.3125 = fieldNorm(field=body, doc=0)
>
> Doc: doc=1 score=0.6914818
> Explain: 0.6914818 = (MATCH) fieldWeight(body:spanNear([red, fox], 2, 
> true) in 1), product of:
>   0.57735026 = tf(phraseFreq=0.33333334)
>   3.8325815 = idf(body: fox=3 red=3)
>   0.3125 = fieldNorm(field=body, doc=1)
>
> Doc: doc=2 score=0.6914818
> Explain: 0.6914818 = (MATCH) fieldWeight(body:spanNear([red, fox], 2, 
> true) in 2), product of:
>   0.57735026 = tf(phraseFreq=0.33333334)
>   3.8325815 = idf(body: fox=3 red=3)
>   0.3125 = fieldNorm(field=body, doc=2)
>
>
> It seems the BTQ score method is not being called.  One of the main 
> points of the SpanNearQuery is that it can take in complex subclauses, 
> presumably rolling up scores from the subclauses.  Yet that appears to 
> not be the case.  Instead it just seems to rely on the matches that 
> get produced by those subclauses, but not the scoring.  Is my 
> understanding correct?  If so, is that the correct functionality?
>
> I'm not a spans expert (SpanNearQuery always confuses me with the 
> NearSpansOrdered/Unordered), but it seems like the SpanNearQuery (and 
> likely others that take clauses) needs to create a QueryWeight object 
> that is made up of the QueryWeight objects from it's subclauses, right?
>
> Thoughts?
>
> Thanks,
> Grant
>
> [1]
> import junit.framework.TestCase;
> import org.apache.lucene.store.Directory;
> import org.apache.lucene.store.RAMDirectory;
> import org.apache.lucene.index.IndexWriter;
> import org.apache.lucene.index.Term;
> import org.apache.lucene.document.Document;
> import org.apache.lucene.document.Field;
> import org.apache.lucene.analysis.Analyzer;
> import org.apache.lucene.analysis.TokenStream;
> import org.apache.lucene.analysis.WhitespaceTokenizer;
> import org.apache.lucene.analysis.LowerCaseFilter;
> import org.apache.lucene.analysis.payloads.DelimitedPayloadTokenFilter;
> import org.apache.lucene.analysis.payloads.PayloadEncoder;
> import org.apache.lucene.analysis.payloads.FloatEncoder;
> import org.apache.lucene.analysis.payloads.PayloadHelper;
> import org.apache.lucene.search.DefaultSimilarity;
> import org.apache.lucene.search.IndexSearcher;
> import org.apache.lucene.search.TopDocs;
> import org.apache.lucene.search.ScoreDoc;
> import org.apache.lucene.search.Query;
> import org.apache.lucene.search.payloads.BoostingTermQuery;
> import org.apache.lucene.search.spans.SpanNearQuery;
> import org.apache.lucene.search.spans.SpanQuery;
>
> import java.io.Reader;
> import java.io.IOException;
>
>
> /**
>  *
>  *
>  **/
> public class PayloadTest extends TestCase {
> Directory dir;
>
>
>
>   public static String[] DOCS = {
>           "The quick|2.0 red|2.0 fox|10.0 jumped|5.0 over the lazy|2.0 
> brown|2.0 dogs|10.0",
>           "The quick red fox jumped over the lazy brown dogs",//no boosts
>           "The quick|2.0 red|2.0 fox|10.0 jumped|5.0 over the old|2.0 
> brown|2.0 box|10.0",
>           "Mary|10.0 had a little|2.0 lamb|10.0 whose fleece|10.0 
> was|5.0 white|2.0 as snow|10.0",
>           "Mary had a little lamb whose fleece was white as snow",
>           "Mary|10.0 takes on Wolf|10.0 Restoration|10.0 project|10.0 
> despite ties|10.0 to sheep|10.0 farming|10.0",
>           "Mary|10.0 who lives|5.0 on a farm|10.0 is|5.0 happy|2.0 
> that she|10.0 takes|5.0 a walk|10.0 every day|10.0",
>           "Moby|10.0 Dick|10.0 is|5.0 a story|10.0 of a whale|10.0 and 
> a man|10.0 obsessed|10.0",
>           "The robber|10.0 wore|5.0 a black|2.0 fleece|10.0 
> jacket|10.0 and a baseball|10.0 cap|10.0",
>           "The English|10.0 Springer|10.0 Spaniel|10.0 is|5.0 the 
> best|2.0 of all dogs|10.0"
>   };
>   protected PayloadSimilarity payloadSimilarity;
>
>   @Override
>   protected void setUp() throws Exception {
>     dir = new RAMDirectory();
>
>     PayloadEncoder encoder = new FloatEncoder();
>     IndexWriter writer = new IndexWriter(dir, new 
> PayloadAnalyzer(encoder), true, IndexWriter.MaxFieldLength.UNLIMITED);
>     payloadSimilarity = new PayloadSimilarity();
>     writer.setSimilarity(payloadSimilarity);
>     for (int i = 0; i < DOCS.length; i++) {
>       Document doc = new Document();
>       Field id = new Field("id", "doc_" + i, Field.Store.YES, 
> Field.Index.NOT_ANALYZED_NO_NORMS);
>       doc.add(id);
>       //Store both position and offset information
>       Field text = new Field("body", DOCS[i], Field.Store.NO, 
> Field.Index.ANALYZED);
>       doc.add(text);
>       writer.addDocument(doc);
>     }
>     writer.close();
>   }
>
>
>   public void testPayloads() throws Exception {
>     IndexSearcher searcher = new IndexSearcher(dir, true);
>     searcher.setSimilarity(payloadSimilarity);//set the similarity.  
> Very important
>     BoostingTermQuery btq = new BoostingTermQuery(new Term("body", 
> "fox"));
>     TopDocs topDocs = searcher.search(btq, 10);
>     printResults(searcher, btq, topDocs);
>     System.out.println("-----------");
>     System.out.println("Try out some Spans");
>     SpanQuery[] queries = new SpanQuery[2];
>     queries[0] = new BoostingTermQuery(new Term("body", "red"));
>     queries[1] = new BoostingTermQuery(new Term("body", "fox"));
>     SpanNearQuery near = new SpanNearQuery(queries, 2, true);
>     topDocs = searcher.search(near, 10);
>     printResults(searcher, near, topDocs);
>
>   }
>
>   private void printResults(IndexSearcher searcher, Query btq, TopDocs 
> topDocs) throws IOException {
>     for (int i = 0; i < topDocs.scoreDocs.length; i++) {
>       ScoreDoc doc = topDocs.scoreDocs[i];
>       System.out.println("Doc: " + doc.toString());
>       System.out.println("Explain: " + searcher.explain(btq, doc.doc));
>     }
>   }
>
>   class PayloadSimilarity extends DefaultSimilarity {
>     @Override
>     public float scorePayload(String fieldName, byte[] bytes, int 
> offset, int length) {
>       return PayloadHelper.decodeFloat(bytes, offset);//we can ignore 
> length here, because we know it is encoded as 4 bytes
>     }
>   }
>
>   class PayloadAnalyzer extends Analyzer {
>     private PayloadEncoder encoder;
>
>     PayloadAnalyzer(PayloadEncoder encoder) {
>       this.encoder = encoder;
>     }
>
>     public TokenStream tokenStream(String fieldName, Reader reader) {
>       TokenStream result = new WhitespaceTokenizer(reader);
>       result = new LowerCaseFilter(result);
>       result = new DelimitedPayloadTokenFilter(result, '|', encoder);
>       return result;
>     }
>   }
> }
>
>
> ---------------------------------------------------------------------
> To unsubscribe, e-mail: java-dev-unsubscribe@lucene.apache.org
> For additional commands, e-mail: java-dev-help@lucene.apache.org
>
Yeah - SpanQuery's don't use the boosts from subspans - it just uses the 
idf for the query terms and the span length I believe - and the boost 
for the top level Query.

Is that the right way to go? I guess Doug seemed to think so? I don't 
know. It is sort of a bug that lower boosts would be ignored right? 
There is an issue for it somewhere.

It gets complicated quick to change it - all of a sudden you need 
something like BooleanQuery ...

-- 
- Mark

http://www.lucidimagination.com




---------------------------------------------------------------------
To unsubscribe, e-mail: java-dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: java-dev-help@lucene.apache.org