lucene-java-user mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Jeremy Long <jeremy.l...@gmail.com>
Subject Re: TokenFilter state question
Date Wed, 26 Dec 2012 22:52:40 GMT
Actually, I had thought the same thing and had played around with the reset
method. However, in my example where I called the custom analyzer the
"reset" method was called after every token, not after the end of the
stream (which implies each token was treated as its own TokenStream?).

        String querystr = "product:(Spring Framework Core)
vendor:(SpringSource)";

        Map<String, Analyzer> fieldAnalyzers = new HashMap<String,
Analyzer>();

        fieldAnalyzers.put("product", new
SearchFieldAnalyzer(Version.LUCENE_40));
        fieldAnalyzers.put("vendor", new
SearchFieldAnalyzer(Version.LUCENE_40));

        PerFieldAnalyzerWrapper wrapper = new PerFieldAnalyzerWrapper(
                new StandardAnalyzer(Version.LUCENE_40), fieldAnalyzers);
        QueryParser parser = new QueryParser(Version.LUCENE_40, field1,
wrapper);

        Query q = parser.parse(querystr);

In the above example (from the code snippets in the original email), if I
were to add a reset method it would be called 4 times (yes, I have tried
this). Reset gets called for each token "Spring" "Framework" "Core" and
"SpringSource". Thus, if I reset my internal state I would not achieve the
goal of have "Spring Framework Core" result in the tokens "Spring
SpringFramework Framework FrameworkCore Core". My question is - why would
these be treated as separate token streams?

--Jeremy

On Wed, Dec 26, 2012 at 10:54 AM, Jack Krupansky <jack@basetechnology.com>wrote:

> You need a "reset" method that calls the super reset to reset the parent
> state and then reset your own state.
>
> http://lucene.apache.org/core/**4_0_0/core/org/apache/lucene/**
> analysis/TokenStream.html#**reset()<http://lucene.apache.org/core/4_0_0/core/org/apache/lucene/analysis/TokenStream.html#reset()>
>
> You probably don't have one, so only the parent state gets reset.
>
> -- Jack Krupansky
>
> -----Original Message----- From: Jeremy Long
> Sent: Wednesday, December 26, 2012 9:08 AM
> To: java-user@lucene.apache.org
> Subject: TokenFilter state question
>
>
> Hello,
>
> I'm still trying to figure out some of the nuances of Lucene and I have run
> into a small issue. I have created my own custom analyzer which uses the
> WhitespaceTokenizer and chains together the LowercaseFilter,
> StopwordFilter, and my own custom filter (below). I am using this analyzer
> when searching (i.e. it is the analyzer used in a QueryParser).
>
> The custom analyzers purpose is to add tokens by concatenating the previous
> word with the current word. So that if you were given "Spring Framework
> Core" the resulting tokens would be "Spring SpringFramework Framework
> FrameworkCore Core". My problem is that when my query text is "Spring
> Framework Core" I end up with left-over state in
> my TokenPairConcatenatingFilter (the previousWord is a member field). So if
> I end up re-using my query parser on a subsequent search for "Apache
> Struts" I end up with the token stream of "CoreApache Apache ApacheStruts
> Struts". The Initial "core" was left over state.
>
> The left over state from the initial query appears to arise because in my
> initial loop that collects all of the tokens from the underlying stream
> only collects a single token. So the processing is - we collect the token
> "spring", we write "spring" out to the stream and move it to the
> previousWord. Next, we are at the end of the stream and we have no more
> words in the list so the filter returns false. At this time, the filter is
> called again and "Framework" is collected... repeat until end of tokens
> from the query is reached; however, "Core" is left in the previousWord
> field. The filter would work correctly with no state being left over if all
> of the tokens were collected at the beginning (i.e. the first call to
> incrementToken). Can anyone explain why all of the tokens would not be
> collected and/or a work around so that when
> QueryParser.parse("field:(**Spring Framework Core)") is called residual
> state
> is not left over in my token filter? I have two hack solutions - 1) don't
> reuse the analyzer/QueryParser for subsequent queries or 2) build in a
> reset mechanism to clear the previousWord field. I don't like either
> solution and was hoping someone from the list might have a suggestion as to
> what I've done wrong or some feature of Lucene I've missed. The code is
> below.
>
> Thanks in advance,
>
> Jeremy
>
>
>
> //----------------------------**------------
> // TokenPairConcatenatingFilter
>
> import java.io.IOException;
> import java.util.LinkedList;
> import org.apache.lucene.analysis.**TokenFilter;
> import org.apache.lucene.analysis.**TokenStream;
> import org.apache.lucene.analysis.**tokenattributes.**CharTermAttribute;
> import
> org.apache.lucene.analysis.**tokenattributes.**PositionIncrementAttribute;
>
> /**
> * <p>Takes a TokenStream and adds additional tokens by concatenating pairs
> of words.</p>
> * <p><b>Example:</b> "Spring Framework Core" -> "Spring SpringFramework
> Framework FrameworkCore Core".</p>
> *
> * @author Jeremy Long (jeremy.long@gmail.com)
> */
> public final class TokenPairConcatenatingFilter extends TokenFilter {
>
>    private final CharTermAttribute termAtt =
> addAttribute(**CharTermAttribute.class);
>    private final PositionIncrementAttribute posIncAtt =
> addAttribute(**PositionIncrementAttribute.**class);
>    private String previousWord = null;
>    private LinkedList<String> words = null;
>
>    public TokenPairConcatenatingFilter(**TokenStream stream) {
>        super(stream);
>        words = new LinkedList<String>();
>    }
>
>    /**
>     * Increments the underlying TokenStream and sets CharTermAtttributes to
>     * construct an expanded set of tokens by concatenting tokens with the
>     * previous token.
>     *
>     * @return whether or not we have hit the end of the TokenStream
>     * @throws IOException is thrown when an IOException occurs
>     */
>    @Override
>    public boolean incrementToken() throws IOException {
>
>        //collect all the terms into the words collaction
>        while (input.incrementToken()) {
>            String word = new String(termAtt.buffer(), 0, termAtt.length());
>            words.add(word);
>        }
>
>        //if we have a previousTerm - write it out as its own token
> concatonated
>        // with the current word (if one is available).
>        if (previousWord != null && words.size() > 0) {
>            String word = words.getFirst();
>            clearAttributes();
>            termAtt.append(previousWord).**append(word);
>            posIncAtt.**setPositionIncrement(0);
>            previousWord = null;
>            return true;
>        }
>        //if we have words, write it out as a single token
>        if (words.size() > 0) {
>            String word = words.removeFirst();
>            clearAttributes();
>            termAtt.append(word);
>            previousWord = word;
>            return true;
>        }
>        return false;
>    }
> }
>
>
> //----------------------------**------------
> // SearchFieldAnalyzer
>
> import java.io.Reader;
> import org.apache.lucene.analysis.**Analyzer;
> import org.apache.lucene.analysis.**TokenStream;
> import org.apache.lucene.analysis.**Tokenizer;
> import org.apache.lucene.analysis.**core.WhitespaceTokenizer;
> import org.apache.lucene.analysis.**core.LowerCaseFilter;
> import org.apache.lucene.analysis.**core.StopAnalyzer;
> import org.apache.lucene.analysis.**core.StopFilter;
> import org.apache.lucene.util.**Version;
>
> /**
> *
> * @author Jeremy Long (jeremy.long@gmail.com)
> */
> public class SearchFieldAnalyzer extends Analyzer {
>
>    private Version version = null;
>
>    public SearchFieldAnalyzer(Version version) {
>        this.version = version;
>    }
>
>    @Override
>    protected TokenStreamComponents createComponents(String fieldName,
> Reader reader) {
>        Tokenizer source = new WhitespaceTokenizer(version, reader);
>
>        TokenStream stream = source;
>
>        stream = new LowerCaseFilter(version, stream);
>        stream = new TokenPairConcatenatingFilter(**stream);
>        stream = new StopFilter(version, stream,
> StopAnalyzer.ENGLISH_STOP_**WORDS_SET);
>
>        return new TokenStreamComponents(source, stream);
>    }
> }
>
>
> //----------------------------**------------
> // The following is a unit test to exercise  the above methods and to show
> the issue:
>
> import org.apache.lucene.analysis.**miscellaneous.**
> PerFieldAnalyzerWrapper;
> import java.util.Map;
> import java.util.HashMap;
> import org.apache.lucene.queryparser.**classic.QueryParser;
> import org.apache.lucene.analysis.**standard.StandardAnalyzer;
> import org.apache.lucene.analysis.**Analyzer;
> import org.apache.lucene.search.**ScoreDoc;
> import org.apache.lucene.search.**TopScoreDocCollector;
> import org.apache.lucene.search.**IndexSearcher;
> import org.apache.lucene.index.**IndexReader;
> import org.apache.lucene.search.**Query;
> import java.io.IOException;
> import org.apache.lucene.document.**Field;
> import org.apache.lucene.document.**TextField;
> import org.apache.lucene.document.**Document;
> import org.apache.lucene.index.**DirectoryReader;
> import org.apache.lucene.index.**IndexWriter;
> import org.apache.lucene.index.**IndexWriterConfig;
> import org.apache.lucene.store.**RAMDirectory;
> import org.apache.lucene.store.**Directory;
> import org.apache.lucene.util.**Version;
> import org.junit.After;
> import org.junit.AfterClass;
> import org.junit.Before;
> import org.junit.BeforeClass;
> import org.junit.Test;
> import static org.junit.Assert.*;
>
> /**
> *
> * @author Jeremy Long (jeremy.long@gmail.com)
> */
> public class FieldAnalyzerTest {
>
>    public FieldAnalyzerTest() {
>    }
>
>    @BeforeClass
>    public static void setUpClass() throws Exception {
>    }
>
>    @AfterClass
>    public static void tearDownClass() throws Exception {
>    }
>
>    @Before
>    public void setUp() {
>    }
>
>    @After
>    public void tearDown() {
>    }
>
>    @Test
>    public void testAnalyzers() throws Exception {
>        Analyzer analyzer = new FieldAnalyzer(Version.LUCENE_**40);
>
>        Directory index = new RAMDirectory();
>
>        String field1 = "product";
>        String text1 = "springframework";
>
>        String field2 = "vendor";
>        String text2 = "springsource";
>
>        createIndex(analyzer, index, field1, text1, field2, text2);
>
>        //Analyzer searchingAnalyzer = new
> SearchFieldAnalyzer(Version.**LUCENE_40);
>        String querystr = "product:(Spring Framework Core)
> vendor:(SpringSource)";
>
>        Map<String, Analyzer> fieldAnalyzers = new HashMap<String,
> Analyzer>();
>
>        fieldAnalyzers.put("product", new
> SearchFieldAnalyzer(Version.**LUCENE_40));
>        fieldAnalyzers.put("vendor", new
> SearchFieldAnalyzer(Version.**LUCENE_40));
>
>        PerFieldAnalyzerWrapper wrapper = new PerFieldAnalyzerWrapper(
>                new StandardAnalyzer(Version.**LUCENE_40), fieldAnalyzers);
>        QueryParser parser = new QueryParser(Version.LUCENE_40, field1,
> wrapper);
>
>        Query q = parser.parse(querystr);
>        System.out.println(q.toString(**));
>
>        querystr = "product:(Apache Struts) vendor:(Apache)";
>        q = parser.parse(querystr);
>        System.out.println(q.toString(**));
>
>        int hitsPerPage = 10;
>
>        IndexReader reader = DirectoryReader.open(index);
>        IndexSearcher searcher = new IndexSearcher(reader);
>        TopScoreDocCollector collector =
> TopScoreDocCollector.create(**hitsPerPage, true);
>        searcher.search(q, collector);
>        ScoreDoc[] hits = collector.topDocs().scoreDocs;
>
>        assertEquals("Did not find 1 document", 1, hits.length);
>    }
>
>    private void createIndex(Analyzer analyzer, Directory index, String
> field1, String text1, String field2, String text2) throws IOException {
>        IndexWriterConfig config = new IndexWriterConfig(Version.**
> LUCENE_40,
> analyzer);
>        IndexWriter w = new IndexWriter(index, config);
>        addDoc(w, field1, text1, field2, text2);
>        w.close();
>    }
>
>    private static void addDoc(IndexWriter w, String field1, String text1,
> String field2, String text2) throws IOException {
>        Document doc = new Document();
>        doc.add(new TextField(field1, text1, Field.Store.YES));
>        doc.add(new TextField(field2, text2, Field.Store.YES));
>        w.addDocument(doc);
>    }
> }
>
>
> //----------------------------**---------
> // The following is my "Fieldanalyzer" used in the above test case.
>
> import java.io.Reader;
> import org.apache.lucene.analysis.**Analyzer;
> import org.apache.lucene.analysis.**TokenStream;
> import org.apache.lucene.analysis.**Tokenizer;
> import org.apache.lucene.analysis.**core.WhitespaceTokenizer;
> import org.apache.lucene.analysis.**core.LowerCaseFilter;
> import org.apache.lucene.analysis.**core.StopAnalyzer;
> import org.apache.lucene.analysis.**core.StopFilter;
> import org.apache.lucene.analysis.**miscellaneous.**WordDelimiterFilter;
> import org.apache.lucene.util.**Version;
>
> /**
> *
> * @author Jeremy Long (jeremy.long@gmail.com)
> */
> public class FieldAnalyzer extends Analyzer {
>
>    private Version version = null;
>
>    public FieldAnalyzer(Version version) {
>        this.version = version;
>    }
>
>    @Override
>    protected TokenStreamComponents createComponents(String fieldName,
> Reader reader) {
>        Tokenizer source = new WhitespaceTokenizer(version, reader);
>
>        TokenStream stream = source;
>
>        stream = new WordDelimiterFilter(stream,
>                WordDelimiterFilter.CATENATE_**WORDS
>                | WordDelimiterFilter.GENERATE_**WORD_PARTS
>                | WordDelimiterFilter.GENERATE_**NUMBER_PARTS
>                | WordDelimiterFilter.PRESERVE_**ORIGINAL
>                | WordDelimiterFilter.SPLIT_ON_**CASE_CHANGE
>                | WordDelimiterFilter.SPLIT_ON_**NUMERICS
>                | WordDelimiterFilter.STEM_**ENGLISH_POSSESSIVE, null);
>
>
>        stream = new LowerCaseFilter(version, stream);
>        //stream = new ConcatenateFilter(stream);
>        stream = new StopFilter(version, stream,
> StopAnalyzer.ENGLISH_STOP_**WORDS_SET);
>
>        return new TokenStreamComponents(source, stream);
>    }
> }
>
> ------------------------------**------------------------------**---------
> To unsubscribe, e-mail: java-user-unsubscribe@lucene.**apache.org<java-user-unsubscribe@lucene.apache.org>
> For additional commands, e-mail: java-user-help@lucene.apache.**org<java-user-help@lucene.apache.org>
>
>

Mime
  • Unnamed multipart/alternative (inline, None, 0 bytes)
View raw message