You are viewing a plain text version of this content. The canonical link for it is here.
Posted to solr-dev@lucene.apache.org by Yonik Seeley <ys...@gmail.com> on 2006/05/10 16:31:19 UTC

BufferingTokenStream and RemoveDups

Just brainstorming...
Here's a completely untested prototype of what a BufferingTokenStream
might look like, and a possible implementation of removing duplicate
tokens on top of it.


-Yonik



class RemoveDuplicatesTokenFilter extends BufferedTokenStream {
  public RemoveDuplicatesTokenFilter(TokenStream input) {super(input);}
  public Token process(Token t) throws IOException {
    Token tok = read();
    while (tok!=null && tok.getPositionIncrement()==0) {
      boolean dup=false;
      for (Token outTok : output()) {
        if (outTok.termText().equals(tok.termText())) {
          dup=true;
          break;
        }
      }
      if (!dup) write(tok);
    }
    if (tok != null) pushBack(tok);
    return t;
  }
}


/**
 * Handles input and output buffering of TokenStream
 *
 *
 * // Example of a class implementing the rule "A" "B" => "Q"
 * class MyTokenStream extends BufferedTokenStream {
 * public MyTokenStream(TokenStream input) {super(input);}
 * public Token process(Token t) throws IOException {
 *   if ("A".equals(t.termText())) {
 *     Token t2 = read();
 *     if (t2!=null && "B".equals(t2.termText())) t.setTermText("Q");
 *     if (t2!=null) pushBack(t2);
 *   }
 *   return t;
 * }
 *}
 *
 * <pre>
 * // Example of a class implementing "A" "B" => "A" "A" "B"
 *class MyTokenStream extends BufferedTokenStream {
 *  public MyTokenStream(TokenStream input) {super(input);}
 *  public Token process(Token t) throws IOException {
 *    if ("A".equals(t.termText()) && "B".equals(peek(1).termText()) write(t);
 *    return t;
 *  }
 *}
 * </pre>
 *
 *
 * @author yonik
 * @version $Id$
*/
abstract class BufferedTokenStream extends TokenStream {
  // in the futute, might be faster if we implemented as an array
based CircularQueue
  private final LinkedList<Token> inQueue = new LinkedList<Token>();
  private final LinkedList<Token> outQueue = new LinkedList<Token>();
  private final TokenStream input;

  public BufferedTokenStream(TokenStream input) {
    this.input = input;
  }

  /** Process a token.  Subclasses may read more tokens from the input stream,
   * write more tokens to the output stream, or simply return the next token
   * to be output.  Subclasses may return null if the token is to be dropped.
   * If a subclass writes tokens to the output stream and returns a
non-null Token,
   * the returned Token is considered to be at the head of the token
output stream.
   */
  public abstract Token process(Token t) throws IOException;

  public final Token next() throws IOException {
    while (true) {
      if (!outQueue.isEmpty()) return outQueue.removeFirst();
      Token t = inQueue.isEmpty() ? input.next() : inQueue.removeFirst();
      if (t==null) return null;
      Token out = process(t);
      if (out!=null) return out;
      // loop back to top in case process() put something on the output queue
    }
  }

  /** read a token from the input stream */
  public Token read() throws IOException {
    if (inQueue.size()==0) {
      Token t = input.next();
      return t;
    }
    return inQueue.getFirst();
  }

  /** push a token back into the input stream */
  public void pushBack(Token t) {
    inQueue.addFirst(t);
  }

  /** peek n tokens ahead in the stream (1 based... 0 is invalid) */
  public Token peek(int n) throws IOException {
    int fillCount = n-inQueue.size();
    for (int i=0; i<fillCount; i++) {
      Token t = input.next();
      if (t==null) return null;
      inQueue.add(t);
    }
    return inQueue.get(n-1);
  }

  /** write a token to the output stream */
  public void write(Token t) {
    outQueue.add(t);
  }

  Iterable<Token> output() {
    return outQueue;
  }

}

Re: BufferingTokenStream and RemoveDups

Posted by Doug Cutting <cu...@apache.org>.
This would be useful for implementing an N-gram filter.  I'd support 
adding something like this to the Lucene core.

Doug

Yonik Seeley wrote:
> Just brainstorming...
> Here's a completely untested prototype of what a BufferingTokenStream
> might look like, and a possible implementation of removing duplicate
> tokens on top of it.
> 
> 
> -Yonik
> 
> 
> 
> class RemoveDuplicatesTokenFilter extends BufferedTokenStream {
>  public RemoveDuplicatesTokenFilter(TokenStream input) {super(input);}
>  public Token process(Token t) throws IOException {
>    Token tok = read();
>    while (tok!=null && tok.getPositionIncrement()==0) {
>      boolean dup=false;
>      for (Token outTok : output()) {
>        if (outTok.termText().equals(tok.termText())) {
>          dup=true;
>          break;
>        }
>      }
>      if (!dup) write(tok);
>    }
>    if (tok != null) pushBack(tok);
>    return t;
>  }
> }
> 
> 
> /**
> * Handles input and output buffering of TokenStream
> *
> *
> * // Example of a class implementing the rule "A" "B" => "Q"
> * class MyTokenStream extends BufferedTokenStream {
> * public MyTokenStream(TokenStream input) {super(input);}
> * public Token process(Token t) throws IOException {
> *   if ("A".equals(t.termText())) {
> *     Token t2 = read();
> *     if (t2!=null && "B".equals(t2.termText())) t.setTermText("Q");
> *     if (t2!=null) pushBack(t2);
> *   }
> *   return t;
> * }
> *}
> *
> * <pre>
> * // Example of a class implementing "A" "B" => "A" "A" "B"
> *class MyTokenStream extends BufferedTokenStream {
> *  public MyTokenStream(TokenStream input) {super(input);}
> *  public Token process(Token t) throws IOException {
> *    if ("A".equals(t.termText()) && "B".equals(peek(1).termText()) 
> write(t);
> *    return t;
> *  }
> *}
> * </pre>
> *
> *
> * @author yonik
> * @version $Id$
> */
> abstract class BufferedTokenStream extends TokenStream {
>  // in the futute, might be faster if we implemented as an array
> based CircularQueue
>  private final LinkedList<Token> inQueue = new LinkedList<Token>();
>  private final LinkedList<Token> outQueue = new LinkedList<Token>();
>  private final TokenStream input;
> 
>  public BufferedTokenStream(TokenStream input) {
>    this.input = input;
>  }
> 
>  /** Process a token.  Subclasses may read more tokens from the input 
> stream,
>   * write more tokens to the output stream, or simply return the next token
>   * to be output.  Subclasses may return null if the token is to be 
> dropped.
>   * If a subclass writes tokens to the output stream and returns a
> non-null Token,
>   * the returned Token is considered to be at the head of the token
> output stream.
>   */
>  public abstract Token process(Token t) throws IOException;
> 
>  public final Token next() throws IOException {
>    while (true) {
>      if (!outQueue.isEmpty()) return outQueue.removeFirst();
>      Token t = inQueue.isEmpty() ? input.next() : inQueue.removeFirst();
>      if (t==null) return null;
>      Token out = process(t);
>      if (out!=null) return out;
>      // loop back to top in case process() put something on the output 
> queue
>    }
>  }
> 
>  /** read a token from the input stream */
>  public Token read() throws IOException {
>    if (inQueue.size()==0) {
>      Token t = input.next();
>      return t;
>    }
>    return inQueue.getFirst();
>  }
> 
>  /** push a token back into the input stream */
>  public void pushBack(Token t) {
>    inQueue.addFirst(t);
>  }
> 
>  /** peek n tokens ahead in the stream (1 based... 0 is invalid) */
>  public Token peek(int n) throws IOException {
>    int fillCount = n-inQueue.size();
>    for (int i=0; i<fillCount; i++) {
>      Token t = input.next();
>      if (t==null) return null;
>      inQueue.add(t);
>    }
>    return inQueue.get(n-1);
>  }
> 
>  /** write a token to the output stream */
>  public void write(Token t) {
>    outQueue.add(t);
>  }
> 
>  Iterable<Token> output() {
>    return outQueue;
>  }
> 
> }

Re: BufferingTokenStream and RemoveDups

Posted by Chris Hostetter <ho...@fucit.org>.
: That would work, but in that case, the buffering isn't needed.  If you
: wanted to use BufferedTokenStream for some reason it would be:

Yeah, you're right ... in the back of my mind i was thinking of more
obscure cases i couldn't articulate and tried using Trey's case as an
example.

Instead: imaging you wanted a TokenFilter that could do balanced paren
matching whil parsing source code ... you want to loop over hte input
stream untill you find something yo uare looking for, and on the way some
things you encounter can be put directly in the output strem, while other
tokens you hang on to because your not ready to toss them on and you're
not sure how many more tokens you need to read before you will be ready.

: +1 for the general idea of flushing state (but I'm not sure how hard
: implementing the exact semantics w.r.t. "before the output stream is
: exhausted").

wouldn't something like this work...

  public final Token next() throws IOException {
    boolean finshed = false;
    while (true) {
      if (!outQueue.isEmpty()) return outQueue.removeFirst();
      Token t = inQueue.isEmpty() ? input.next() : inQueue.removeFirst();
      if (t==null) {
        if (finished)
          return null;
        done();
        finished = true;
        continue;
      }
      Token out = process(t);
      if (out!=null) return out;
      // loop back to top in case process() put something on the output queue
    }
  }

...allthough, now that i think about it, we probably don't need a done
method .. and sufficiently complex filter that needs to know when the
input stream is totally exhausted so it can flush it's state could allways
just use the peek method in it's process method before returning whatever
it is it's going to return.  ie, any class that could be implimented be...

  public class Foo extends BufferingTokenFilter{
    public Token process(Token t) {
      return doSomethingComplicated(t);
    }
    public void done() {
      flushInternalState();
    }
  }

...could ust as easily be implimented as...

  public class Foo extends BufferingTokenFilter{
    public Token process(Token t) {
      Token r = doSomethingComplicated(t);
      if (null == inQueue.peek(1))
        flushInternalState()
      return r;
    }
  }


...right?


-Hoss


Re: BufferingTokenStream and RemoveDups

Posted by Yonik Seeley <ys...@gmail.com>.
On 5/12/06, Chris Hostetter <ho...@fucit.org> wrote:
> Is your expectation that for situations like Trey's "remove all dups,
> regardless of position" type TokenFilter, the implimentation of process
> would be such that it compltely emptied out the input queue, writing all
> non-dups to the output queue (effectively insuring that it will only ever
> be called once)
> ie...
>
> cladd RemoveAllDups extends BufferedTokenStream {
>    Set<String> m = new HashSet<String>(23);
>    public Token process(Token t) throws IOException {
>      while (null != t) {
>        if (! m.contains(t.termText()))
>           write(t);
>        m.add(t.termText());
>        t = read();
>      }
>      return null;
>    }
> }


That would work, but in that case, the buffering isn't needed.  If you
wanted to use BufferedTokenStream for some reason it would be:

cladd RemoveAllDupsRegardlessOfPosition extends BufferedTokenStream {
     Set<String> m = new HashSet<String>(23);
    public Token process(Token t) throws IOException {
      if (!m.contains(t.termText()) {
        m.add(t.termText)
        return t;
      } else return null
    }}

One might as well just throw in a for loop and not use
BufferedTokenStream for that (rather unique) case.

>    /**
>     * This method is garunteed to be called at least once after the input
>     * stream is  exhausted, but before the output stream is exhausted.
>     * By default it is a No-Op.
>     */
>    protected void done() throw IOException { /* NOOP*/ }

+1 for the general idea of flushing state (but I'm not sure how hard
implementing the exact semantics w.r.t. "before the output stream is
exhausted").

-Yonik

Re: BufferingTokenStream and RemoveDups

Posted by Chris Hostetter <ho...@fucit.org>.
I finally took some time to look at this.

It looks good, but i woner if some of those public methods should be
protected?

Is your expectation that for situations like Trey's "remove all dups,
regardless of position" type TokenFilter, the implimentation of process
would be such that it compltely emptied out the input queue, writing all
non-dups to the output queue (effectively insuring that it will only ever
be called once)

ie...

cladd RemoveAllDups extends BufferedTokenStream {
   Set<String> m = new HashSet<String>(23);
   public Token process(Token t) throws IOException {
     while (null != t) {
       if (! m.contains(t.termText()))
          write(t);
       m.add(t.termText());
       t = read();
     }
     return null;
   }
}

Thinking about it some more ... the line "if (t==null) return null;"
should probably be removed from the next method so that at the end of the
input stream, process is called at least once with a null value -- That
way we ensure that subclasses which find a need for maintining their own
state (seperate from the output queue) will allways have at least one
opportunity to "flush" their internal state.

Either that, or we could add something like this to the base
class...
   /**
    * This method is garunteed to be called at least once after the input
    * stream is  exhausted, but before the output stream is exhausted.
    * By default it is a No-Op.
    */
   protected void done() throw IOException { /* NOOP*/ }

...that way "simple" subclasses wouldn't have to write anythign special in
the process method for handling a null parameter, but complex subclasses
could override the method to flush any internal state they may have.


After talking it through, i think i like the "done()" option best.


: Date: Wed, 10 May 2006 10:31:19 -0400
: From: Yonik Seeley <ys...@gmail.com>
: Reply-To: solr-dev@lucene.apache.org
: To: solr-dev@lucene.apache.org
: Subject: BufferingTokenStream and RemoveDups
:
: Just brainstorming...
: Here's a completely untested prototype of what a BufferingTokenStream
: might look like, and a possible implementation of removing duplicate
: tokens on top of it.
:
:
: -Yonik
:
:
:
: class RemoveDuplicatesTokenFilter extends BufferedTokenStream {
:   public RemoveDuplicatesTokenFilter(TokenStream input) {super(input);}
:   public Token process(Token t) throws IOException {
:     Token tok = read();
:     while (tok!=null && tok.getPositionIncrement()==0) {
:       boolean dup=false;
:       for (Token outTok : output()) {
:         if (outTok.termText().equals(tok.termText())) {
:           dup=true;
:           break;
:         }
:       }
:       if (!dup) write(tok);
:     }
:     if (tok != null) pushBack(tok);
:     return t;
:   }
: }
:
:
: /**
:  * Handles input and output buffering of TokenStream
:  *
:  *
:  * // Example of a class implementing the rule "A" "B" => "Q"
:  * class MyTokenStream extends BufferedTokenStream {
:  * public MyTokenStream(TokenStream input) {super(input);}
:  * public Token process(Token t) throws IOException {
:  *   if ("A".equals(t.termText())) {
:  *     Token t2 = read();
:  *     if (t2!=null && "B".equals(t2.termText())) t.setTermText("Q");
:  *     if (t2!=null) pushBack(t2);
:  *   }
:  *   return t;
:  * }
:  *}
:  *
:  * <pre>
:  * // Example of a class implementing "A" "B" => "A" "A" "B"
:  *class MyTokenStream extends BufferedTokenStream {
:  *  public MyTokenStream(TokenStream input) {super(input);}
:  *  public Token process(Token t) throws IOException {
:  *    if ("A".equals(t.termText()) && "B".equals(peek(1).termText()) write(t);
:  *    return t;
:  *  }
:  *}
:  * </pre>
:  *
:  *
:  * @author yonik
:  * @version $Id$
: */
: abstract class BufferedTokenStream extends TokenStream {
:   // in the futute, might be faster if we implemented as an array
: based CircularQueue
:   private final LinkedList<Token> inQueue = new LinkedList<Token>();
:   private final LinkedList<Token> outQueue = new LinkedList<Token>();
:   private final TokenStream input;
:
:   public BufferedTokenStream(TokenStream input) {
:     this.input = input;
:   }
:
:   /** Process a token.  Subclasses may read more tokens from the input stream,
:    * write more tokens to the output stream, or simply return the next token
:    * to be output.  Subclasses may return null if the token is to be dropped.
:    * If a subclass writes tokens to the output stream and returns a
: non-null Token,
:    * the returned Token is considered to be at the head of the token
: output stream.
:    */
:   public abstract Token process(Token t) throws IOException;
:
:   public final Token next() throws IOException {
:     while (true) {
:       if (!outQueue.isEmpty()) return outQueue.removeFirst();
:       Token t = inQueue.isEmpty() ? input.next() : inQueue.removeFirst();
:       if (t==null) return null;
:       Token out = process(t);
:       if (out!=null) return out;
:       // loop back to top in case process() put something on the output queue
:     }
:   }
:
:   /** read a token from the input stream */
:   public Token read() throws IOException {
:     if (inQueue.size()==0) {
:       Token t = input.next();
:       return t;
:     }
:     return inQueue.getFirst();
:   }
:
:   /** push a token back into the input stream */
:   public void pushBack(Token t) {
:     inQueue.addFirst(t);
:   }
:
:   /** peek n tokens ahead in the stream (1 based... 0 is invalid) */
:   public Token peek(int n) throws IOException {
:     int fillCount = n-inQueue.size();
:     for (int i=0; i<fillCount; i++) {
:       Token t = input.next();
:       if (t==null) return null;
:       inQueue.add(t);
:     }
:     return inQueue.get(n-1);
:   }
:
:   /** write a token to the output stream */
:   public void write(Token t) {
:     outQueue.add(t);
:   }
:
:   Iterable<Token> output() {
:     return outQueue;
:   }
:
: }
:



-Hoss