LUCENE-1695: Update the Highlighter to use the new TokenStream API. This issue breaks backwards compatibility with some public classes. If you have implemented custom Fregmenters or Scorers, you will need to adjust them to work with the new TokenStream API. Rather than getting passed a Token at a time, you will be given a TokenStream to init your impl with - store the Attributes you are interested in locally and access them on each call to the method that used to pass a new Token. Look at the included updated impls for examples.

git-svn-id: https://svn.apache.org/repos/asf/lucene/java/trunk@799455 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Mark Robert Miller 2009-07-30 22:00:47 +00:00
parent 7ecaa8c990
commit f73a4f4324
13 changed files with 569 additions and 401 deletions

View File

@ -11,7 +11,12 @@ Changes in runtime behavior
API Changes API Changes
(None) 1. LUCENE-1695: Update the Highlighter to use the new TokenStream API. This issue breaks backwards
compatibility with some public classes. If you have implemented custom Fregmenters or Scorers,
you will need to adjust them to work with the new TokenStream API. Rather than getting passed a
Token at a time, you will be given a TokenStream to init your impl with - store the Attributes
you are interested in locally and access them on each call to the method that used to pass a new
Token. Look at the included updated impls for examples. (Mark Miller)
Bug fixes Bug fixes
@ -41,9 +46,6 @@ Bug fixes
8. LUCENE-1491: EdgeNGramTokenFilter no longer stops on tokens shorter than minimum n-gram size. 8. LUCENE-1491: EdgeNGramTokenFilter no longer stops on tokens shorter than minimum n-gram size.
(Todd Teak via Otis Gospodnetic) (Todd Teak via Otis Gospodnetic)
9. LUCENE-1752: Missing highlights when terms were repeated in separate, nested, boolean or
disjunction queries. (Koji Sekiguchi, Mark Miller)
New features New features

View File

@ -16,24 +16,31 @@ package org.apache.lucene.search.highlight;
* limitations under the License. * limitations under the License.
*/ */
import org.apache.lucene.analysis.Token; import org.apache.lucene.analysis.TokenStream;
/** /**
* Implements the policy for breaking text into multiple fragments for consideration * Implements the policy for breaking text into multiple fragments for
* by the {@link Highlighter} class. A sophisticated implementation may do this on the basis * consideration by the {@link Highlighter} class. A sophisticated
* of detecting end of sentences in the text. * implementation may do this on the basis of detecting end of sentences in the
* text.
*/ */
public interface Fragmenter public interface Fragmenter {
{
/**
* Initializes the Fragmenter
* @param originalText
*/
public void start(String originalText);
/** /**
* Test to see if this token from the stream should be held in a new TextFragment * Initializes the Fragmenter. You can grab references to the Attributes you are
* @param nextToken * interested in from tokenStream and then access the values in isNewFragment.
*/ *
public boolean isNewFragment(Token nextToken); * @param originalText
* @param tokenStream
*/
public void start(String originalText, TokenStream tokenStream);
/**
* Test to see if this token from the stream should be held in a new
* TextFragment. Every time this is called, the TokenStream
* passed to start(String, TokenStream) will have been incremented.
*
*/
public boolean isNewFragment();
} }

View File

@ -22,8 +22,10 @@ import java.util.ArrayList;
import java.util.Iterator; import java.util.Iterator;
import org.apache.lucene.analysis.Analyzer; import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.analysis.Token;
import org.apache.lucene.analysis.TokenStream; import org.apache.lucene.analysis.TokenStream;
import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
import org.apache.lucene.analysis.tokenattributes.TermAttribute;
import org.apache.lucene.util.PriorityQueue; import org.apache.lucene.util.PriorityQueue;
/** /**
@ -214,8 +216,14 @@ public class Highlighter
{ {
ArrayList docFrags = new ArrayList(); ArrayList docFrags = new ArrayList();
StringBuffer newText=new StringBuffer(); StringBuffer newText=new StringBuffer();
TermAttribute termAtt = (TermAttribute) tokenStream.addAttribute(TermAttribute.class);
OffsetAttribute offsetAtt = (OffsetAttribute) tokenStream.addAttribute(OffsetAttribute.class);
tokenStream.addAttribute(PositionIncrementAttribute.class);
tokenStream.reset();
TextFragment currentFrag = new TextFragment(newText,newText.length(), docFrags.size()); TextFragment currentFrag = new TextFragment(newText,newText.length(), docFrags.size());
fragmentScorer.init(tokenStream);
fragmentScorer.startFragment(currentFrag); fragmentScorer.startFragment(currentFrag);
docFrags.add(currentFrag); docFrags.add(currentFrag);
@ -223,28 +231,27 @@ public class Highlighter
try try
{ {
final Token reusableToken = new Token();
String tokenText; String tokenText;
int startOffset; int startOffset;
int endOffset; int endOffset;
int lastEndOffset = 0; int lastEndOffset = 0;
textFragmenter.start(text); textFragmenter.start(text, tokenStream);
TokenGroup tokenGroup=new TokenGroup(); TokenGroup tokenGroup=new TokenGroup(tokenStream);
for (Token nextToken = tokenStream.next(reusableToken); for (boolean next = tokenStream.incrementToken(); next && (offsetAtt.startOffset()< maxDocCharsToAnalyze);
(nextToken!= null)&&(nextToken.startOffset()< maxDocCharsToAnalyze); next = tokenStream.incrementToken())
nextToken = tokenStream.next(reusableToken))
{ {
if( (nextToken.endOffset()>text.length()) if( (offsetAtt.endOffset()>text.length())
|| ||
(nextToken.startOffset()>text.length()) (offsetAtt.startOffset()>text.length())
) )
{ {
throw new InvalidTokenOffsetsException("Token "+nextToken.toString() throw new InvalidTokenOffsetsException("Token "+ termAtt.term()
+" exceeds length of provided text sized "+text.length()); +" exceeds length of provided text sized "+text.length());
} }
if((tokenGroup.numTokens>0)&&(tokenGroup.isDistinct(nextToken))) if((tokenGroup.numTokens>0)&&(tokenGroup.isDistinct()))
{ {
//the current token is distinct from previous tokens - //the current token is distinct from previous tokens -
// markup the cached token group info // markup the cached token group info
@ -260,7 +267,7 @@ public class Highlighter
tokenGroup.clear(); tokenGroup.clear();
//check if current token marks the start of a new fragment //check if current token marks the start of a new fragment
if(textFragmenter.isNewFragment(nextToken)) if(textFragmenter.isNewFragment())
{ {
currentFrag.setScore(fragmentScorer.getFragmentScore()); currentFrag.setScore(fragmentScorer.getFragmentScore());
//record stats for a new fragment //record stats for a new fragment
@ -271,7 +278,7 @@ public class Highlighter
} }
} }
tokenGroup.addToken(nextToken,fragmentScorer.getTokenScore(nextToken)); tokenGroup.addToken(fragmentScorer.getTokenScore());
// if(lastEndOffset>maxDocBytesToAnalyze) // if(lastEndOffset>maxDocBytesToAnalyze)
// { // {
@ -332,7 +339,7 @@ public class Highlighter
//The above code caused a problem as a result of Christoph Goller's 11th Sept 03 //The above code caused a problem as a result of Christoph Goller's 11th Sept 03
//fix to PriorityQueue. The correct method to use here is the new "insert" method //fix to PriorityQueue. The correct method to use here is the new "insert" method
// USE ABOVE CODE IF THIS DOES NOT COMPILE! // USE ABOVE CODE IF THIS DOES NOT COMPILE!
fragQueue.insert(currentFrag); fragQueue.insertWithOverflow(currentFrag);
} }
//return the most relevant fragments //return the most relevant fragments

View File

@ -16,17 +16,18 @@ package org.apache.lucene.search.highlight;
* limitations under the License. * limitations under the License.
*/ */
import org.apache.lucene.analysis.Token; import org.apache.lucene.analysis.TokenStream;
/** /**
* {@link Fragmenter} implementation which does not fragment the text. * {@link Fragmenter} implementation which does not fragment the text.
* This is useful for highlighting the entire content of a document or field. * This is useful for highlighting the entire content of a document or field.
*/ */
public class NullFragmenter implements Fragmenter { public class NullFragmenter implements Fragmenter {
public void start(String s) { public void start(String s, TokenStream tokenStream) {
} }
public boolean isNewFragment(Token token) { public boolean isNewFragment() {
return false; return false;
} }
} }

View File

@ -1,4 +1,5 @@
package org.apache.lucene.search.highlight; package org.apache.lucene.search.highlight;
/** /**
* Licensed to the Apache Software Foundation (ASF) under one or more * Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with * contributor license agreements. See the NOTICE file distributed with
@ -19,134 +20,142 @@ package org.apache.lucene.search.highlight;
import java.util.HashMap; import java.util.HashMap;
import java.util.HashSet; import java.util.HashSet;
import org.apache.lucene.analysis.Token; import org.apache.lucene.analysis.TokenStream;
import org.apache.lucene.analysis.tokenattributes.TermAttribute;
import org.apache.lucene.index.IndexReader; import org.apache.lucene.index.IndexReader;
import org.apache.lucene.search.Query; import org.apache.lucene.search.Query;
/** /**
* {@link Scorer} implementation which scores text fragments by the number of unique query terms found. * {@link Scorer} implementation which scores text fragments by the number of
* This class uses the {@link QueryTermExtractor} class to process determine the query terms and * unique query terms found. This class uses the {@link QueryTermExtractor}
* their boosts to be used. * class to process determine the query terms and their boosts to be used.
*/ */
//TODO: provide option to boost score of fragments near beginning of document // TODO: provide option to boost score of fragments near beginning of document
// based on fragment.getFragNum() // based on fragment.getFragNum()
public class QueryScorer implements Scorer public class QueryScorer implements Scorer {
{
TextFragment currentTextFragment=null; TextFragment currentTextFragment = null;
HashSet uniqueTermsInFragment; HashSet uniqueTermsInFragment;
float totalScore=0;
float maxTermWeight=0;
private HashMap termsToFind;
/** float totalScore = 0;
* float maxTermWeight = 0;
* @param query a Lucene query (ideally rewritten using query.rewrite private HashMap termsToFind;
* before being passed to this class and the searcher)
*/
public QueryScorer(Query query)
{
this(QueryTermExtractor.getTerms(query));
}
/**
*
* @param query a Lucene query (ideally rewritten using query.rewrite
* before being passed to this class and the searcher)
* @param fieldName the Field name which is used to match Query terms
*/
public QueryScorer(Query query, String fieldName)
{
this(QueryTermExtractor.getTerms(query, false,fieldName));
}
/** private TermAttribute termAtt;
*
* @param query a Lucene query (ideally rewritten using query.rewrite
* before being passed to this class and the searcher)
* @param reader used to compute IDF which can be used to a) score selected fragments better
* b) use graded highlights eg set font color intensity
* @param fieldName the field on which Inverse Document Frequency (IDF) calculations are based
*/
public QueryScorer(Query query, IndexReader reader, String fieldName)
{
this(QueryTermExtractor.getIdfWeightedTerms(query, reader, fieldName));
}
public QueryScorer(WeightedTerm []weightedTerms ) /**
{ *
termsToFind = new HashMap(); * @param query a Lucene query (ideally rewritten using query.rewrite before
for (int i = 0; i < weightedTerms.length; i++) * being passed to this class and the searcher)
{ */
WeightedTerm existingTerm=(WeightedTerm) termsToFind.get(weightedTerms[i].term); public QueryScorer(Query query) {
if( (existingTerm==null) ||(existingTerm.weight<weightedTerms[i].weight) ) this(QueryTermExtractor.getTerms(query));
{ }
//if a term is defined more than once, always use the highest scoring weight
termsToFind.put(weightedTerms[i].term,weightedTerms[i]);
maxTermWeight=Math.max(maxTermWeight,weightedTerms[i].getWeight());
}
}
}
/* (non-Javadoc) /**
* @see org.apache.lucene.search.highlight.FragmentScorer#startFragment(org.apache.lucene.search.highlight.TextFragment) *
*/ * @param query a Lucene query (ideally rewritten using query.rewrite before
public void startFragment(TextFragment newFragment) * being passed to this class and the searcher)
{ * @param fieldName the Field name which is used to match Query terms
uniqueTermsInFragment = new HashSet(); */
currentTextFragment=newFragment; public QueryScorer(Query query, String fieldName) {
totalScore=0; this(QueryTermExtractor.getTerms(query, false, fieldName));
}
}
/* (non-Javadoc)
* @see org.apache.lucene.search.highlight.FragmentScorer#scoreToken(org.apache.lucene.analysis.Token)
*/
public float getTokenScore(Token token)
{
String termText=token.term();
WeightedTerm queryTerm=(WeightedTerm) termsToFind.get(termText);
if(queryTerm==null)
{
//not a query term - return
return 0;
}
//found a query term - is it unique in this doc?
if(!uniqueTermsInFragment.contains(termText))
{
totalScore+=queryTerm.getWeight();
uniqueTermsInFragment.add(termText);
}
return queryTerm.getWeight();
}
/* (non-Javadoc)
* @see org.apache.lucene.search.highlight.FragmentScorer#endFragment(org.apache.lucene.search.highlight.TextFragment)
*/
public float getFragmentScore()
{
return totalScore;
}
/**
*
* @param query a Lucene query (ideally rewritten using query.rewrite before
* being passed to this class and the searcher)
* @param reader used to compute IDF which can be used to a) score selected
* fragments better b) use graded highlights eg set font color
* intensity
* @param fieldName the field on which Inverse Document Frequency (IDF)
* calculations are based
*/
public QueryScorer(Query query, IndexReader reader, String fieldName) {
this(QueryTermExtractor.getIdfWeightedTerms(query, reader, fieldName));
}
/* (non-Javadoc) public QueryScorer(WeightedTerm[] weightedTerms) {
* @see org.apache.lucene.search.highlight.FragmentScorer#allFragmentsProcessed() termsToFind = new HashMap();
*/ for (int i = 0; i < weightedTerms.length; i++) {
public void allFragmentsProcessed() WeightedTerm existingTerm = (WeightedTerm) termsToFind
{ .get(weightedTerms[i].term);
//this class has no special operations to perform at end of processing if ((existingTerm == null)
} || (existingTerm.weight < weightedTerms[i].weight)) {
// if a term is defined more than once, always use the highest scoring
/** // weight
* termsToFind.put(weightedTerms[i].term, weightedTerms[i]);
* @return The highest weighted term (useful for passing to GradientFormatter to set maxTermWeight = Math.max(maxTermWeight, weightedTerms[i].getWeight());
* top end of coloring scale. }
*/
public float getMaxTermWeight()
{
return maxTermWeight;
} }
}
/* (non-Javadoc)
* @see org.apache.lucene.search.highlight.Scorer#init(org.apache.lucene.analysis.TokenStream)
*/
public void init(TokenStream tokenStream) {
termAtt = (TermAttribute) tokenStream.getAttribute(TermAttribute.class);
}
/*
* (non-Javadoc)
*
* @see
* org.apache.lucene.search.highlight.FragmentScorer#startFragment(org.apache
* .lucene.search.highlight.TextFragment)
*/
public void startFragment(TextFragment newFragment) {
uniqueTermsInFragment = new HashSet();
currentTextFragment = newFragment;
totalScore = 0;
}
/* (non-Javadoc)
* @see org.apache.lucene.search.highlight.Scorer#getTokenScore()
*/
public float getTokenScore() {
String termText = termAtt.term();
WeightedTerm queryTerm = (WeightedTerm) termsToFind.get(termText);
if (queryTerm == null) {
// not a query term - return
return 0;
}
// found a query term - is it unique in this doc?
if (!uniqueTermsInFragment.contains(termText)) {
totalScore += queryTerm.getWeight();
uniqueTermsInFragment.add(termText);
}
return queryTerm.getWeight();
}
/* (non-Javadoc)
* @see org.apache.lucene.search.highlight.Scorer#getFragmentScore()
*/
public float getFragmentScore() {
return totalScore;
}
/*
* (non-Javadoc)
*
* @see
* org.apache.lucene.search.highlight.FragmentScorer#allFragmentsProcessed()
*/
public void allFragmentsProcessed() {
// this class has no special operations to perform at end of processing
}
/**
*
* @return The highest weighted term (useful for passing to GradientFormatter
* to set top end of coloring scale.
*/
public float getMaxTermWeight() {
return maxTermWeight;
}
} }

View File

@ -1,4 +1,5 @@
package org.apache.lucene.search.highlight; package org.apache.lucene.search.highlight;
/** /**
* Licensed to the Apache Software Foundation (ASF) under one or more * Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with * contributor license agreements. See the NOTICE file distributed with
@ -16,34 +17,45 @@ package org.apache.lucene.search.highlight;
* limitations under the License. * limitations under the License.
*/ */
import org.apache.lucene.analysis.Token; import org.apache.lucene.analysis.TokenStream;
/** /**
* Adds to the score for a fragment based on its tokens * Adds to the score for a fragment based on its tokens
*/ */
public interface Scorer public interface Scorer {
{
/**
* called when a new fragment is started for consideration
* @param newFragment
*/
public void startFragment(TextFragment newFragment);
/** /**
* Called for each token in the current fragment * Called to init the Scorer with a TokenStream. You can grab references to
* @param token The token to be scored * the attributes you are interested in here and access them from
* @return a score which is passed to the Highlighter class to influence the mark-up of the text * getTokenScore().
* (this return value is NOT used to score the fragment) *
*/ * @param tokenStream
public float getTokenScore(Token token); */
public void init(TokenStream tokenStream);
/** /**
* Called when the highlighter has no more tokens for the current fragment - the scorer returns * called when a new fragment is started for consideration
* the weighting it has derived for the most recent fragment, typically based on the tokens *
* passed to getTokenScore(). * @param newFragment
* */
*/ public void startFragment(TextFragment newFragment);
public float getFragmentScore();
/**
* Called for each token in the current fragment. The Highlighter will
* increment the TokenStream passed to init on every call.
*
* @return a score which is passed to the Highlighter class to influence the
* mark-up of the text (this return value is NOT used to score the
* fragment)
*/
public float getTokenScore();
/**
* Called when the highlighter has no more tokens for the current fragment -
* the scorer returns the weighting it has derived for the most recent
* fragment, typically based on the tokens passed to getTokenScore().
*
*/
public float getFragmentScore();
} }

View File

@ -1,4 +1,5 @@
package org.apache.lucene.search.highlight; package org.apache.lucene.search.highlight;
/** /**
* Licensed to the Apache Software Foundation (ASF) under one or more * Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with * contributor license agreements. See the NOTICE file distributed with
@ -16,69 +17,64 @@ package org.apache.lucene.search.highlight;
* limitations under the License. * limitations under the License.
*/ */
import org.apache.lucene.analysis.Token; import org.apache.lucene.analysis.TokenStream;
import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
/** /**
* {@link Fragmenter} implementation which breaks text up into same-size * {@link Fragmenter} implementation which breaks text up into same-size
* fragments with no concerns over spotting sentence boundaries. * fragments with no concerns over spotting sentence boundaries.
*/ */
public class SimpleFragmenter implements Fragmenter public class SimpleFragmenter implements Fragmenter {
{ private static final int DEFAULT_FRAGMENT_SIZE = 100;
private static final int DEFAULT_FRAGMENT_SIZE =100; private int currentNumFrags;
private int currentNumFrags; private int fragmentSize;
private int fragmentSize; private OffsetAttribute offsetAtt;
public SimpleFragmenter() {
this(DEFAULT_FRAGMENT_SIZE);
}
/**
*
* @param fragmentSize size in number of characters of each fragment
*/
public SimpleFragmenter(int fragmentSize) {
this.fragmentSize = fragmentSize;
}
public SimpleFragmenter() /* (non-Javadoc)
{ * @see org.apache.lucene.search.highlight.Fragmenter#start(java.lang.String, org.apache.lucene.analysis.TokenStream)
this(DEFAULT_FRAGMENT_SIZE); */
} public void start(String originalText, TokenStream stream) {
offsetAtt = (OffsetAttribute) stream.getAttribute(OffsetAttribute.class);
currentNumFrags = 1;
}
/** /* (non-Javadoc)
* * @see org.apache.lucene.search.highlight.Fragmenter#isNewFragment()
* @param fragmentSize size in number of characters of each fragment */
*/ public boolean isNewFragment() {
public SimpleFragmenter(int fragmentSize) boolean isNewFrag = offsetAtt.endOffset() >= (fragmentSize * currentNumFrags);
{ if (isNewFrag) {
this.fragmentSize=fragmentSize; currentNumFrags++;
} }
return isNewFrag;
}
/* (non-Javadoc) /**
* @see org.apache.lucene.search.highlight.TextFragmenter#start(java.lang.String) * @return size in number of characters of each fragment
*/ */
public void start(String originalText) public int getFragmentSize() {
{ return fragmentSize;
currentNumFrags=1; }
}
/* (non-Javadoc) /**
* @see org.apache.lucene.search.highlight.TextFragmenter#isNewFragment(org.apache.lucene.analysis.Token) * @param size size in characters of each fragment
*/ */
public boolean isNewFragment(Token token) public void setFragmentSize(int size) {
{ fragmentSize = size;
boolean isNewFrag= token.endOffset()>=(fragmentSize*currentNumFrags); }
if(isNewFrag)
{
currentNumFrags++;
}
return isNewFrag;
}
/**
* @return size in number of characters of each fragment
*/
public int getFragmentSize()
{
return fragmentSize;
}
/**
* @param size size in characters of each fragment
*/
public void setFragmentSize(int size)
{
fragmentSize = size;
}
} }

View File

@ -17,10 +17,13 @@ package org.apache.lucene.search.highlight;
* See the License for the specific language governing permissions and * See the License for the specific language governing permissions and
* limitations under the License. * limitations under the License.
*/ */
import org.apache.lucene.analysis.Token;
import java.util.List; import java.util.List;
import org.apache.lucene.analysis.TokenStream;
import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
import org.apache.lucene.analysis.tokenattributes.TermAttribute;
/** /**
* {@link Fragmenter} implementation which breaks text up into same-size * {@link Fragmenter} implementation which breaks text up into same-size
@ -34,6 +37,9 @@ public class SimpleSpanFragmenter implements Fragmenter {
private SpanScorer spanScorer; private SpanScorer spanScorer;
private int waitForPos = -1; private int waitForPos = -1;
private int textSize; private int textSize;
private TermAttribute termAtt;
private PositionIncrementAttribute posIncAtt;
private OffsetAttribute offsetAtt;
/** /**
* @param spanscorer SpanScorer that was used to score hits * @param spanscorer SpanScorer that was used to score hits
@ -50,12 +56,12 @@ public class SimpleSpanFragmenter implements Fragmenter {
this.fragmentSize = fragmentSize; this.fragmentSize = fragmentSize;
this.spanScorer = spanscorer; this.spanScorer = spanscorer;
} }
/* (non-Javadoc) /* (non-Javadoc)
* @see org.apache.lucene.search.highlight.Fragmenter#isNewFragment(org.apache.lucene.analysis.Token) * @see org.apache.lucene.search.highlight.Fragmenter#isNewFragment()
*/ */
public boolean isNewFragment(Token token) { public boolean isNewFragment() {
position += token.getPositionIncrement(); position += posIncAtt.getPositionIncrement();
if (waitForPos == position) { if (waitForPos == position) {
waitForPos = -1; waitForPos = -1;
@ -63,7 +69,7 @@ public class SimpleSpanFragmenter implements Fragmenter {
return false; return false;
} }
WeightedSpanTerm wSpanTerm = spanScorer.getWeightedSpanTerm(token.term()); WeightedSpanTerm wSpanTerm = spanScorer.getWeightedSpanTerm(termAtt.term());
if (wSpanTerm != null) { if (wSpanTerm != null) {
List positionSpans = wSpanTerm.getPositionSpans(); List positionSpans = wSpanTerm.getPositionSpans();
@ -76,8 +82,8 @@ public class SimpleSpanFragmenter implements Fragmenter {
} }
} }
boolean isNewFrag = token.endOffset() >= (fragmentSize * currentNumFrags) boolean isNewFrag = offsetAtt.endOffset() >= (fragmentSize * currentNumFrags)
&& (textSize - token.endOffset()) >= (fragmentSize >>> 1); && (textSize - offsetAtt.endOffset()) >= (fragmentSize >>> 1);
if (isNewFrag) { if (isNewFrag) {
currentNumFrags++; currentNumFrags++;
@ -86,12 +92,16 @@ public class SimpleSpanFragmenter implements Fragmenter {
return isNewFrag; return isNewFrag;
} }
/* (non-Javadoc) /* (non-Javadoc)
* @see org.apache.lucene.search.highlight.Fragmenter#start(java.lang.String) * @see org.apache.lucene.search.highlight.Fragmenter#start(java.lang.String, org.apache.lucene.analysis.TokenStream)
*/ */
public void start(String originalText) { public void start(String originalText, TokenStream tokenStream) {
position = -1; position = -1;
currentNumFrags = 1; currentNumFrags = 1;
textSize = originalText.length(); textSize = originalText.length();
termAtt = (TermAttribute) tokenStream.getAttribute(TermAttribute.class);
posIncAtt = (PositionIncrementAttribute) tokenStream.getAttribute(PositionIncrementAttribute.class);
offsetAtt = (OffsetAttribute) tokenStream.getAttribute(OffsetAttribute.class);
} }
} }

View File

@ -7,9 +7,10 @@ import java.util.Map;
import java.util.Set; import java.util.Set;
import org.apache.lucene.analysis.CachingTokenFilter; import org.apache.lucene.analysis.CachingTokenFilter;
import org.apache.lucene.analysis.Token; import org.apache.lucene.analysis.TokenStream;
import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
import org.apache.lucene.analysis.tokenattributes.TermAttribute;
import org.apache.lucene.index.IndexReader; import org.apache.lucene.index.IndexReader;
import org.apache.lucene.search.ConstantScoreRangeQuery;
import org.apache.lucene.search.Query; import org.apache.lucene.search.Query;
@ -26,6 +27,8 @@ public class SpanScorer implements Scorer {
private float maxTermWeight; private float maxTermWeight;
private int position = -1; private int position = -1;
private String defaultField; private String defaultField;
private TermAttribute termAtt;
private PositionIncrementAttribute posIncAtt;
private static boolean highlightCnstScrRngQuery; private static boolean highlightCnstScrRngQuery;
/** /**
@ -176,9 +179,9 @@ public class SpanScorer implements Scorer {
* @see org.apache.lucene.search.highlight.Scorer#getTokenScore(org.apache.lucene.analysis.Token, * @see org.apache.lucene.search.highlight.Scorer#getTokenScore(org.apache.lucene.analysis.Token,
* int) * int)
*/ */
public float getTokenScore(Token token) { public float getTokenScore() {
position += token.getPositionIncrement(); position += posIncAtt.getPositionIncrement();
String termText = token.term(); String termText = termAtt.term();
WeightedSpanTerm weightedSpanTerm; WeightedSpanTerm weightedSpanTerm;
@ -203,6 +206,11 @@ public class SpanScorer implements Scorer {
return score; return score;
} }
public void init(TokenStream tokenStream) {
termAtt = (TermAttribute) tokenStream.getAttribute(TermAttribute.class);
posIncAtt = (PositionIncrementAttribute) tokenStream.getAttribute(PositionIncrementAttribute.class);
}
/** /**
* Retrieve the WeightedSpanTerm for the specified token. Useful for passing * Retrieve the WeightedSpanTerm for the specified token. Useful for passing
* Span information to a Fragmenter. * Span information to a Fragmenter.

View File

@ -1,4 +1,5 @@
package org.apache.lucene.search.highlight; package org.apache.lucene.search.highlight;
/** /**
* Licensed to the Apache Software Foundation (ASF) under one or more * Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with * contributor license agreements. See the NOTICE file distributed with
@ -15,118 +16,117 @@ package org.apache.lucene.search.highlight;
* See the License for the specific language governing permissions and * See the License for the specific language governing permissions and
* limitations under the License. * limitations under the License.
*/ */
import org.apache.lucene.analysis.Token; import org.apache.lucene.analysis.Token;
import org.apache.lucene.analysis.TokenStream;
import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
import org.apache.lucene.analysis.tokenattributes.TermAttribute;
/** /**
* One, or several overlapping tokens, along with the score(s) and the * One, or several overlapping tokens, along with the score(s) and the scope of
* scope of the original text * the original text
*/ */
public class TokenGroup public class TokenGroup {
{
private static final int MAX_NUM_TOKENS_PER_GROUP=50;
Token [] tokens=new Token[MAX_NUM_TOKENS_PER_GROUP];
float [] scores=new float[MAX_NUM_TOKENS_PER_GROUP];
int numTokens=0;
int startOffset=0;
int endOffset=0;
float tot;
private static final int MAX_NUM_TOKENS_PER_GROUP = 50;
Token [] tokens=new Token[MAX_NUM_TOKENS_PER_GROUP];
float[] scores = new float[MAX_NUM_TOKENS_PER_GROUP];
int numTokens = 0;
int startOffset = 0;
int endOffset = 0;
float tot;
int matchStartOffset, matchEndOffset; int matchStartOffset, matchEndOffset;
private OffsetAttribute offsetAtt;
private TermAttribute termAtt;
void addToken(Token token, float score) public TokenGroup(TokenStream tokenStream) {
{ offsetAtt = (OffsetAttribute) tokenStream.getAttribute(OffsetAttribute.class);
if(numTokens < MAX_NUM_TOKENS_PER_GROUP) termAtt = (TermAttribute) tokenStream.getAttribute(TermAttribute.class);
{ }
if(numTokens==0)
{ void addToken(float score) {
startOffset=matchStartOffset=token.startOffset(); if (numTokens < MAX_NUM_TOKENS_PER_GROUP) {
endOffset=matchEndOffset=token.endOffset(); int termStartOffset = offsetAtt.startOffset();
tot += score; int termEndOffset = offsetAtt.endOffset();
} if (numTokens == 0) {
else startOffset = matchStartOffset = termStartOffset;
{ endOffset = matchEndOffset = termEndOffset;
startOffset=Math.min(startOffset,token.startOffset()); tot += score;
endOffset=Math.max(endOffset,token.endOffset()); } else {
if (score>0) { startOffset = Math.min(startOffset, termStartOffset);
if (tot==0) { endOffset = Math.max(endOffset, termEndOffset);
matchStartOffset=token.startOffset(); if (score > 0) {
matchEndOffset=token.endOffset(); if (tot == 0) {
matchStartOffset = offsetAtt.startOffset();
matchEndOffset = offsetAtt.endOffset();
} else { } else {
matchStartOffset=Math.min(matchStartOffset,token.startOffset()); matchStartOffset = Math.min(matchStartOffset, termStartOffset);
matchEndOffset=Math.max(matchEndOffset,token.endOffset()); matchEndOffset = Math.max(matchEndOffset, termEndOffset);
} }
tot+=score; tot += score;
} }
} }
tokens[numTokens]= (Token) token.clone(); Token token = new Token(termStartOffset, termEndOffset);
scores[numTokens]=score; token.setTermBuffer(termAtt.term());
numTokens++; tokens[numTokens] = token;
} scores[numTokens] = score;
} numTokens++;
}
}
boolean isDistinct(Token token) boolean isDistinct() {
{ return offsetAtt.startOffset() >= endOffset;
return token.startOffset()>=endOffset; }
}
void clear() {
numTokens = 0;
tot = 0;
}
/*
* @param index a value between 0 and numTokens -1
* @return the "n"th token
*/
public Token getToken(int index)
{
return tokens[index];
}
void clear() /**
{ *
numTokens=0; * @param index a value between 0 and numTokens -1
tot=0; * @return the "n"th score
} */
public float getScore(int index) {
/** return scores[index];
* }
* @param index a value between 0 and numTokens -1
* @return the "n"th token
*/
public Token getToken(int index)
{
return tokens[index];
}
/** /**
* * @return the end position in the original text
* @param index a value between 0 and numTokens -1 */
* @return the "n"th score public int getEndOffset() {
*/ return endOffset;
public float getScore(int index) }
{
return scores[index];
}
/** /**
* @return the end position in the original text * @return the number of tokens in this group
*/ */
public int getEndOffset() public int getNumTokens() {
{ return numTokens;
return endOffset; }
}
/** /**
* @return the number of tokens in this group * @return the start position in the original text
*/ */
public int getNumTokens() public int getStartOffset() {
{ return startOffset;
return numTokens; }
}
/** /**
* @return the start position in the original text * @return all tokens' scores summed up
*/ */
public int getStartOffset() public float getTotalScore() {
{ return tot;
return startOffset; }
}
/**
* @return all tokens' scores summed up
*/
public float getTotalScore()
{
return tot;
}
} }

View File

@ -29,6 +29,8 @@ import java.util.Comparator;
import org.apache.lucene.analysis.Analyzer; import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.analysis.Token; import org.apache.lucene.analysis.Token;
import org.apache.lucene.analysis.TokenStream; import org.apache.lucene.analysis.TokenStream;
import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
import org.apache.lucene.analysis.tokenattributes.TermAttribute;
import org.apache.lucene.document.Document; import org.apache.lucene.document.Document;
import org.apache.lucene.index.IndexReader; import org.apache.lucene.index.IndexReader;
import org.apache.lucene.index.TermFreqVector; import org.apache.lucene.index.TermFreqVector;
@ -135,32 +137,45 @@ public class TokenSources
* @param tokenPositionsGuaranteedContiguous true if the token position numbers have no overlaps or gaps. If looking * @param tokenPositionsGuaranteedContiguous true if the token position numbers have no overlaps or gaps. If looking
* to eek out the last drops of performance, set to true. If in doubt, set to false. * to eek out the last drops of performance, set to true. If in doubt, set to false.
*/ */
public static TokenStream getTokenStream(TermPositionVector tpv, boolean tokenPositionsGuaranteedContiguous) public static TokenStream getTokenStream(TermPositionVector tpv, boolean tokenPositionsGuaranteedContiguous) {
{
//an object used to iterate across an array of tokens //an object used to iterate across an array of tokens
class StoredTokenStream extends TokenStream class StoredTokenStream extends TokenStream {
{ Token tokens[];
Token tokens[]; int currentToken = 0;
int currentToken=0; TermAttribute termAtt;
StoredTokenStream(Token tokens[]) OffsetAttribute offsetAtt;
{
this.tokens=tokens; StoredTokenStream(Token tokens[]) {
this.tokens = tokens;
termAtt = (TermAttribute) addAttribute(TermAttribute.class);
offsetAtt = (OffsetAttribute) addAttribute(OffsetAttribute.class);
}
public Token next(final Token reusableToken) {
System.out.println("next token");
assert reusableToken != null;
if (currentToken >= tokens.length) {
return null;
} }
public Token next(final Token reusableToken) return tokens[currentToken++];
{ }
assert reusableToken != null;
if(currentToken>=tokens.length) public boolean incrementToken() throws IOException {
{ System.out.println("inc token");
return null; if (currentToken >= tokens.length) {
} return false;
return tokens[currentToken++]; }
} Token token = tokens[currentToken++];
} termAtt.setTermBuffer(token.term());
offsetAtt.setOffset(token.startOffset(), token.endOffset());
return true;
}
}
//code to reconstruct the original sequence of Tokens //code to reconstruct the original sequence of Tokens
String[] terms=tpv.getTerms(); String[] terms=tpv.getTerms();
int[] freq=tpv.getTermFrequencies(); int[] freq=tpv.getTermFrequencies();
int totalTokens=0; int totalTokens=0;
Token newToken = new Token();
for (int t = 0; t < freq.length; t++) for (int t = 0; t < freq.length; t++)
{ {
totalTokens+=freq[t]; totalTokens+=freq[t];
@ -190,8 +205,9 @@ public class TokenSources
} }
for (int tp = 0; tp < offsets.length; tp++) for (int tp = 0; tp < offsets.length; tp++)
{ {
newToken.reinit(terms[t], offsets[tp].getStartOffset(), offsets[tp].getEndOffset()); Token token = new Token(offsets[tp].getStartOffset(), offsets[tp].getEndOffset());
unsortedTokens.add(newToken.clone()); token.setTermBuffer(terms[t]);
unsortedTokens.add(token);
} }
} }
else else
@ -204,8 +220,8 @@ public class TokenSources
//tokens stored with positions - can use this to index straight into sorted array //tokens stored with positions - can use this to index straight into sorted array
for (int tp = 0; tp < pos.length; tp++) for (int tp = 0; tp < pos.length; tp++)
{ {
newToken.reinit(terms[t], offsets[tp].getStartOffset(), offsets[tp].getEndOffset()); Token token = new Token(terms[t], offsets[tp].getStartOffset(), offsets[tp].getEndOffset());
tokensInOriginalOrder[pos[tp]] = (Token) newToken.clone(); tokensInOriginalOrder[pos[tp]] = token;
} }
} }
} }
@ -218,7 +234,7 @@ public class TokenSources
{ {
Token t1=(Token) o1; Token t1=(Token) o1;
Token t2=(Token) o2; Token t2=(Token) o2;
if(t1.startOffset()>t2.startOffset()) if(t1.startOffset()>t2.endOffset())
return 1; return 1;
if(t1.startOffset()<t2.startOffset()) if(t1.startOffset()<t2.startOffset())
return -1; return -1;

View File

@ -42,8 +42,8 @@ import org.apache.lucene.search.MultiTermQuery;
import org.apache.lucene.search.PhraseQuery; import org.apache.lucene.search.PhraseQuery;
import org.apache.lucene.search.PrefixQuery; import org.apache.lucene.search.PrefixQuery;
import org.apache.lucene.search.Query; import org.apache.lucene.search.Query;
import org.apache.lucene.search.TermRangeQuery;
import org.apache.lucene.search.TermQuery; import org.apache.lucene.search.TermQuery;
import org.apache.lucene.search.TermRangeQuery;
import org.apache.lucene.search.WildcardQuery; import org.apache.lucene.search.WildcardQuery;
import org.apache.lucene.search.spans.SpanNearQuery; import org.apache.lucene.search.spans.SpanNearQuery;
import org.apache.lucene.search.spans.SpanOrQuery; import org.apache.lucene.search.spans.SpanOrQuery;
@ -98,7 +98,7 @@ public class WeightedSpanTermExtractor {
private void extract(Query query, Map terms) throws IOException { private void extract(Query query, Map terms) throws IOException {
if (query instanceof BooleanQuery) { if (query instanceof BooleanQuery) {
BooleanClause[] queryClauses = ((BooleanQuery) query).getClauses(); BooleanClause[] queryClauses = ((BooleanQuery) query).getClauses();
for (int i = 0; i < queryClauses.length; i++) { for (int i = 0; i < queryClauses.length; i++) {
if (!queryClauses[i].isProhibited()) { if (!queryClauses[i].isProhibited()) {
extract(queryClauses[i].getQuery(), terms); extract(queryClauses[i].getQuery(), terms);
@ -441,7 +441,7 @@ public class WeightedSpanTermExtractor {
* This class makes sure that if both position sensitive and insensitive * This class makes sure that if both position sensitive and insensitive
* versions of the same term are added, the position insensitive one wins. * versions of the same term are added, the position insensitive one wins.
*/ */
private class PositionCheckingMap extends HashMap { static private class PositionCheckingMap extends HashMap {
public void putAll(Map m) { public void putAll(Map m) {
Iterator it = m.keySet().iterator(); Iterator it = m.keySet().iterator();

View File

@ -38,10 +38,14 @@ import junit.framework.TestCase;
import org.apache.lucene.analysis.Analyzer; import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.analysis.CachingTokenFilter; import org.apache.lucene.analysis.CachingTokenFilter;
import org.apache.lucene.analysis.LowerCaseTokenizer; import org.apache.lucene.analysis.LowerCaseTokenizer;
import org.apache.lucene.analysis.SimpleAnalyzer;
import org.apache.lucene.analysis.Token; import org.apache.lucene.analysis.Token;
import org.apache.lucene.analysis.TokenStream; import org.apache.lucene.analysis.TokenStream;
import org.apache.lucene.analysis.WhitespaceAnalyzer; import org.apache.lucene.analysis.WhitespaceAnalyzer;
import org.apache.lucene.analysis.standard.StandardAnalyzer; import org.apache.lucene.analysis.standard.StandardAnalyzer;
import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
import org.apache.lucene.analysis.tokenattributes.TermAttribute;
import org.apache.lucene.document.Document; import org.apache.lucene.document.Document;
import org.apache.lucene.document.Field; import org.apache.lucene.document.Field;
import org.apache.lucene.document.Field.Index; import org.apache.lucene.document.Field.Index;
@ -62,9 +66,8 @@ import org.apache.lucene.search.MultiSearcher;
import org.apache.lucene.search.MultiTermQuery; import org.apache.lucene.search.MultiTermQuery;
import org.apache.lucene.search.PhraseQuery; import org.apache.lucene.search.PhraseQuery;
import org.apache.lucene.search.Query; import org.apache.lucene.search.Query;
import org.apache.lucene.search.TermRangeFilter;
import org.apache.lucene.search.Searcher;
import org.apache.lucene.search.TermQuery; import org.apache.lucene.search.TermQuery;
import org.apache.lucene.search.TermRangeFilter;
import org.apache.lucene.search.TopDocs; import org.apache.lucene.search.TopDocs;
import org.apache.lucene.search.WildcardQuery; import org.apache.lucene.search.WildcardQuery;
import org.apache.lucene.search.BooleanClause.Occur; import org.apache.lucene.search.BooleanClause.Occur;
@ -75,6 +78,7 @@ import org.apache.lucene.search.spans.SpanQuery;
import org.apache.lucene.search.spans.SpanTermQuery; import org.apache.lucene.search.spans.SpanTermQuery;
import org.apache.lucene.store.Directory; import org.apache.lucene.store.Directory;
import org.apache.lucene.store.RAMDirectory; import org.apache.lucene.store.RAMDirectory;
import org.apache.lucene.util.Version;
import org.w3c.dom.Element; import org.w3c.dom.Element;
import org.w3c.dom.NodeList; import org.w3c.dom.NodeList;
@ -87,7 +91,7 @@ public class HighlighterTest extends TestCase implements Formatter {
static final String FIELD_NAME = "contents"; static final String FIELD_NAME = "contents";
private Query query; private Query query;
RAMDirectory ramDir; RAMDirectory ramDir;
public Searcher searcher = null; public IndexSearcher searcher = null;
public Hits hits = null; public Hits hits = null;
int numHighlights = 0; int numHighlights = 0;
Analyzer analyzer = new StandardAnalyzer(); Analyzer analyzer = new StandardAnalyzer();
@ -108,11 +112,40 @@ public class HighlighterTest extends TestCase implements Formatter {
super(arg0); super(arg0);
} }
public void testHits() throws Exception {
Analyzer analyzer = new SimpleAnalyzer();
QueryParser qp = new QueryParser(FIELD_NAME, analyzer);
query = qp.parse("\"very long\"");
searcher = new IndexSearcher(ramDir, false);
TopDocs hits = searcher.search(query, 10);
Highlighter highlighter = new Highlighter(null);
for (int i = 0; i < hits.scoreDocs.length; i++) {
Document doc = searcher.doc(hits.scoreDocs[i].doc);
String storedField = doc.get(FIELD_NAME);
TokenStream stream = TokenSources.getAnyTokenStream(searcher
.getIndexReader(), hits.scoreDocs[i].doc, FIELD_NAME, doc, analyzer);
CachingTokenFilter ctf = new CachingTokenFilter(stream);
SpanScorer scorer = new SpanScorer(query, FIELD_NAME, ctf);
// ctf.reset();
Fragmenter fragmenter = new SimpleSpanFragmenter(scorer);
highlighter.setFragmentScorer(scorer);
highlighter.setTextFragmenter(fragmenter);
String fragment = highlighter.getBestFragment(ctf, storedField);
System.out.println(fragment);
}
}
public void testHighlightingWithDefaultField() throws Exception { public void testHighlightingWithDefaultField() throws Exception {
String s1 = "I call our world Flatland, not because we call it so,"; String s1 = "I call our world Flatland, not because we call it so,";
QueryParser parser = new QueryParser(FIELD_NAME, new StandardAnalyzer()); QueryParser parser = new QueryParser(FIELD_NAME, new StandardAnalyzer(Version.LUCENE_CURRENT));
// Verify that a query against the default field results in text being // Verify that a query against the default field results in text being
// highlighted // highlighted
@ -144,7 +177,7 @@ public class HighlighterTest extends TestCase implements Formatter {
*/ */
private static String highlightField(Query query, String fieldName, String text) private static String highlightField(Query query, String fieldName, String text)
throws IOException, InvalidTokenOffsetsException { throws IOException, InvalidTokenOffsetsException {
CachingTokenFilter tokenStream = new CachingTokenFilter(new StandardAnalyzer().tokenStream( CachingTokenFilter tokenStream = new CachingTokenFilter(new StandardAnalyzer(Version.LUCENE_CURRENT).tokenStream(
fieldName, new StringReader(text))); fieldName, new StringReader(text)));
// Assuming "<B>", "</B>" used to highlight // Assuming "<B>", "</B>" used to highlight
SimpleHTMLFormatter formatter = new SimpleHTMLFormatter(); SimpleHTMLFormatter formatter = new SimpleHTMLFormatter();
@ -908,10 +941,12 @@ public class HighlighterTest extends TestCase implements Formatter {
Query query = parser.parse(srchkey); Query query = parser.parse(srchkey);
TokenStream tokenStream = analyzer.tokenStream(null, new StringReader(s)); TokenStream tokenStream = analyzer.tokenStream(null, new StringReader(s));
Highlighter highlighter = getHighlighter(query, null, tokenStream, HighlighterTest.this); Highlighter highlighter = getHighlighter(query, null, tokenStream, HighlighterTest.this);
// Get 3 best fragments and seperate with a "..." // Get 3 best fragments and seperate with a "..."
tokenStream = analyzer.tokenStream(null, new StringReader(s)); tokenStream = analyzer.tokenStream(null, new StringReader(s));
String result = highlighter.getBestFragments(tokenStream, s, 3, "..."); String result = highlighter.getBestFragments(tokenStream, s, 3, "...");
String expectedResult = "<B>football</B>-<B>soccer</B> in the euro 2004 <B>footie</B> competition"; String expectedResult = "<B>football</B>-<B>soccer</B> in the euro 2004 <B>footie</B> competition";
assertTrue("overlapping analyzer should handle highlights OK, expected:" + expectedResult assertTrue("overlapping analyzer should handle highlights OK, expected:" + expectedResult
@ -1075,10 +1110,11 @@ public class HighlighterTest extends TestCase implements Formatter {
} }
public void testUnRewrittenQuery() throws Exception { public void testUnRewrittenQuery() throws Exception {
TestHighlightRunner helper = new TestHighlightRunner() { final TestHighlightRunner helper = new TestHighlightRunner() {
public void run() throws Exception { public void run() throws Exception {
numHighlights = 0; numHighlights = 0;
SpanScorer.setHighlightCnstScrRngQuery(false);
// test to show how rewritten query can still be used // test to show how rewritten query can still be used
searcher = new IndexSearcher(ramDir); searcher = new IndexSearcher(ramDir);
Analyzer analyzer = new StandardAnalyzer(); Analyzer analyzer = new StandardAnalyzer();
@ -1154,13 +1190,17 @@ public class HighlighterTest extends TestCase implements Formatter {
public void startFragment(TextFragment newFragment) { public void startFragment(TextFragment newFragment) {
} }
public float getTokenScore(Token token) { public float getTokenScore() {
return 0; return 0;
} }
public float getFragmentScore() { public float getFragmentScore() {
return 1; return 1;
} }
public void init(TokenStream tokenStream) {
}
}); });
highlighter.setTextFragmenter(new SimpleFragmenter(2000)); highlighter.setTextFragmenter(new SimpleFragmenter(2000));
TokenStream tokenStream = analyzer.tokenStream(FIELD_NAME, new StringReader(rawDocContent)); TokenStream tokenStream = analyzer.tokenStream(FIELD_NAME, new StringReader(rawDocContent));
@ -1292,27 +1332,44 @@ public class HighlighterTest extends TestCase implements Formatter {
return new TokenStream() { return new TokenStream() {
Iterator iter; Iterator iter;
List lst; List lst;
private TermAttribute termAtt;
private PositionIncrementAttribute posIncrAtt;
private OffsetAttribute offsetAtt;
{ {
termAtt = (TermAttribute) addAttribute(TermAttribute.class);
posIncrAtt = (PositionIncrementAttribute) addAttribute(PositionIncrementAttribute.class);
offsetAtt = (OffsetAttribute) addAttribute(OffsetAttribute.class);
lst = new ArrayList(); lst = new ArrayList();
Token t; Token t;
t = createToken("hi", 0, 2); t = createToken("hi", 0, 2);
t.setPositionIncrement(1);
lst.add(t); lst.add(t);
t = createToken("hispeed", 0, 8); t = createToken("hispeed", 0, 8);
t.setPositionIncrement(1);
lst.add(t); lst.add(t);
t = createToken("speed", 3, 8); t = createToken("speed", 3, 8);
t.setPositionIncrement(0); t.setPositionIncrement(0);
lst.add(t); lst.add(t);
t = createToken("10", 8, 10); t = createToken("10", 8, 10);
t.setPositionIncrement(1);
lst.add(t); lst.add(t);
t = createToken("foo", 11, 14); t = createToken("foo", 11, 14);
t.setPositionIncrement(1);
lst.add(t); lst.add(t);
iter = lst.iterator(); iter = lst.iterator();
} }
public Token next(final Token reusableToken) throws IOException { public boolean incrementToken() throws IOException {
assert reusableToken != null; if(iter.hasNext()) {
return iter.hasNext() ? (Token) iter.next() : null; Token token = (Token) iter.next();
termAtt.setTermBuffer(token.term());
posIncrAtt.setPositionIncrement(token.getPositionIncrement());
offsetAtt.setOffset(token.startOffset(), token.endOffset());
return true;
}
return false;
} }
}; };
} }
@ -1322,26 +1379,42 @@ public class HighlighterTest extends TestCase implements Formatter {
return new TokenStream() { return new TokenStream() {
Iterator iter; Iterator iter;
List lst; List lst;
private TermAttribute termAtt;
private PositionIncrementAttribute posIncrAtt;
private OffsetAttribute offsetAtt;
{ {
termAtt = (TermAttribute) addAttribute(TermAttribute.class);
posIncrAtt = (PositionIncrementAttribute) addAttribute(PositionIncrementAttribute.class);
offsetAtt = (OffsetAttribute) addAttribute(OffsetAttribute.class);
lst = new ArrayList(); lst = new ArrayList();
Token t; Token t;
t = createToken("hispeed", 0, 8); t = createToken("hispeed", 0, 8);
t.setPositionIncrement(1);
lst.add(t); lst.add(t);
t = createToken("hi", 0, 2); t = createToken("hi", 0, 2);
t.setPositionIncrement(0); t.setPositionIncrement(0);
lst.add(t); lst.add(t);
t = createToken("speed", 3, 8); t = createToken("speed", 3, 8);
t.setPositionIncrement(1);
lst.add(t); lst.add(t);
t = createToken("10", 8, 10); t = createToken("10", 8, 10);
t.setPositionIncrement(1);
lst.add(t); lst.add(t);
t = createToken("foo", 11, 14); t = createToken("foo", 11, 14);
t.setPositionIncrement(1);
lst.add(t); lst.add(t);
iter = lst.iterator(); iter = lst.iterator();
} }
public Token next(final Token reusableToken) throws IOException { public boolean incrementToken() throws IOException {
assert reusableToken != null; if(iter.hasNext()) {
return iter.hasNext() ? (Token) iter.next() : null; Token token = (Token) iter.next();
termAtt.setTermBuffer(token.term());
posIncrAtt.setPositionIncrement(token.getPositionIncrement());
offsetAtt.setOffset(token.startOffset(), token.endOffset());
return true;
}
return false;
} }
}; };
} }
@ -1611,7 +1684,11 @@ class SynonymAnalyzer extends Analyzer {
* java.io.Reader) * java.io.Reader)
*/ */
public TokenStream tokenStream(String arg0, Reader arg1) { public TokenStream tokenStream(String arg0, Reader arg1) {
return new SynonymTokenizer(new LowerCaseTokenizer(arg1), synonyms); LowerCaseTokenizer stream = new LowerCaseTokenizer(arg1);
stream.addAttribute(TermAttribute.class);
stream.addAttribute(PositionIncrementAttribute.class);
stream.addAttribute(OffsetAttribute.class);
return new SynonymTokenizer(stream, synonyms);
} }
} }
@ -1622,47 +1699,70 @@ class SynonymAnalyzer extends Analyzer {
class SynonymTokenizer extends TokenStream { class SynonymTokenizer extends TokenStream {
private TokenStream realStream; private TokenStream realStream;
private Token currentRealToken = null; private Token currentRealToken = null;
private org.apache.lucene.analysis.Token cRealToken = null;
private Map synonyms; private Map synonyms;
StringTokenizer st = null; StringTokenizer st = null;
private TermAttribute realTermAtt;
private PositionIncrementAttribute realPosIncrAtt;
private OffsetAttribute realOffsetAtt;
private TermAttribute termAtt;
private PositionIncrementAttribute posIncrAtt;
private OffsetAttribute offsetAtt;
public SynonymTokenizer(TokenStream realStream, Map synonyms) { public SynonymTokenizer(TokenStream realStream, Map synonyms) {
this.realStream = realStream; this.realStream = realStream;
this.synonyms = synonyms; this.synonyms = synonyms;
realTermAtt = (TermAttribute) realStream.getAttribute(TermAttribute.class);
realPosIncrAtt = (PositionIncrementAttribute) realStream.getAttribute(PositionIncrementAttribute.class);
realOffsetAtt = (OffsetAttribute) realStream.getAttribute(OffsetAttribute.class);
termAtt = (TermAttribute) addAttribute(TermAttribute.class);
posIncrAtt = (PositionIncrementAttribute) addAttribute(PositionIncrementAttribute.class);
offsetAtt = (OffsetAttribute) addAttribute(OffsetAttribute.class);
} }
public Token next(final Token reusableToken) throws IOException { public boolean incrementToken() throws IOException {
assert reusableToken != null;
if (currentRealToken == null) { if (currentRealToken == null) {
Token nextRealToken = realStream.next(reusableToken); boolean next = realStream.incrementToken();
if (nextRealToken == null) { if (!next) {
return null; return false;
} }
String expansions = (String) synonyms.get(nextRealToken.term()); //Token nextRealToken = new Token(, offsetAtt.startOffset(), offsetAtt.endOffset());
termAtt.setTermBuffer(realTermAtt.term());
offsetAtt.setOffset(realOffsetAtt.startOffset(), realOffsetAtt.endOffset());
posIncrAtt.setPositionIncrement(realPosIncrAtt.getPositionIncrement());
String expansions = (String) synonyms.get(realTermAtt.term());
if (expansions == null) { if (expansions == null) {
return nextRealToken; return true;
} }
st = new StringTokenizer(expansions, ","); st = new StringTokenizer(expansions, ",");
if (st.hasMoreTokens()) { if (st.hasMoreTokens()) {
currentRealToken = (Token) nextRealToken.clone(); currentRealToken = new Token(realOffsetAtt.startOffset(), realOffsetAtt.endOffset());
currentRealToken.setTermBuffer(realTermAtt.term());
} }
return currentRealToken;
return true;
} else { } else {
reusableToken.reinit(st.nextToken(), String tok = st.nextToken();
currentRealToken.startOffset(), termAtt.setTermBuffer(tok);
currentRealToken.endOffset()); offsetAtt.setOffset(currentRealToken.startOffset(), currentRealToken.endOffset());
reusableToken.setPositionIncrement(0); posIncrAtt.setPositionIncrement(0);
if (!st.hasMoreTokens()) { if (!st.hasMoreTokens()) {
currentRealToken = null; currentRealToken = null;
st = null; st = null;
} }
return reusableToken; return true;
} }
} }
static abstract class TestHighlightRunner { static abstract class TestHighlightRunner {
static final int STANDARD = 0; static final int STANDARD = 0;
static final int SPAN = 1; static final int SPAN = 1;
int mode = STANDARD; int mode = STANDARD;
Fragmenter frag = new SimpleFragmenter(20);
public Highlighter getHighlighter(Query query, String fieldName, TokenStream stream, public Highlighter getHighlighter(Query query, String fieldName, TokenStream stream,
Formatter formatter) { Formatter formatter) {
@ -1725,7 +1825,7 @@ class SynonymTokenizer extends TokenStream {
if (mode == SPAN) { if (mode == SPAN) {
((CachingTokenFilter) tokenStream).reset(); ((CachingTokenFilter) tokenStream).reset();
} }
highlighter.setTextFragmenter(new SimpleFragmenter(20)); highlighter.setTextFragmenter(frag);
String result = highlighter.getBestFragments(tokenStream, text, maxNumFragmentsRequired, String result = highlighter.getBestFragments(tokenStream, text, maxNumFragmentsRequired,
fragmentSeparator); fragmentSeparator);