You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@lucene.apache.org by mattweber <gi...@git.apache.org> on 2016/12/26 17:09:32 UTC

[GitHub] lucene-solr pull request #129: LUCENE-7603: Support Graph Token Streams in Q...

GitHub user mattweber opened a pull request:

    https://github.com/apache/lucene-solr/pull/129

    LUCENE-7603: Support Graph Token Streams in QueryBuilder

    Adds support for handling graph token streams inside the
    QueryBuilder util class used by query parsers.

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/mattweber/lucene-solr LUCENE-7603

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/lucene-solr/pull/129.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #129
    
----
commit 568cb43d6af1aeef96cc7b6cabb7237de9058f36
Author: Matt Weber <ma...@mattweber.org>
Date:   2016-12-26T15:50:58Z

    Support Graph Token Streams in QueryBuilder
    
    Adds support for handling graph token streams inside the
    QueryBuilder util class used by query parsers.

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] lucene-solr pull request #129: LUCENE-7603: Support Graph Token Streams in Q...

Posted by dsmiley <gi...@git.apache.org>.
Github user dsmiley commented on a diff in the pull request:

    https://github.com/apache/lucene-solr/pull/129#discussion_r94243375
  
    --- Diff: lucene/core/src/java/org/apache/lucene/util/graph/GraphTokenStreamFiniteStrings.java ---
    @@ -210,85 +199,41 @@ private void finish() {
        */
       private void finish(int maxDeterminizedStates) {
         Automaton automaton = builder.finish();
    -
    --- End diff --
    
    So all this code here removed wasn't needed after all?  It's nice to see it all go away (less to maintain / less complexity) :-)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] lucene-solr pull request #129: LUCENE-7603: Support Graph Token Streams in Q...

Posted by dsmiley <gi...@git.apache.org>.
Github user dsmiley commented on a diff in the pull request:

    https://github.com/apache/lucene-solr/pull/129#discussion_r94244009
  
    --- Diff: lucene/core/src/java/org/apache/lucene/util/graph/GraphTokenStreamFiniteStrings.java ---
    @@ -80,22 +77,41 @@ public boolean incrementToken() throws IOException {
         }
       }
     
    +  private GraphTokenStreamFiniteStrings() {
    +    this.builder = new Automaton.Builder();
    +  }
    +
       /**
        * Gets the list of finite string token streams from the given input graph token stream.
        */
    -  public List<TokenStream> getTokenStreams(final TokenStream in) throws IOException {
    -    // build automation
    +  public static List<TokenStream> getTokenStreams(final TokenStream in) throws IOException {
    +    GraphTokenStreamFiniteStrings gfs = new GraphTokenStreamFiniteStrings();
    +    return gfs.process(in);
    +  }
    +
    +  /**
    +   * Builds automaton and builds the finite string token streams.
    +   */
    +  private List<TokenStream> process(final TokenStream in) throws IOException {
         build(in);
     
         List<TokenStream> tokenStreams = new ArrayList<>();
         final FiniteStringsIterator finiteStrings = new FiniteStringsIterator(det);
         for (IntsRef string; (string = finiteStrings.next()) != null; ) {
           final BytesRef[] tokens = new BytesRef[string.length];
    --- End diff --
    
    Hmm; rather than materializing an array of tokens and increments, maybe you could simply give the IntsRefString  to BytesRefArrayTokenStream (and make BRATS not static) so that it could do this on the fly?  Not a big deal either way (current or my proposal).  If you do as I suggest then BRATS would no longer be a suitable name; maybe simply FiniteStringTokenStream or CustomTokenStream.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] lucene-solr pull request #129: LUCENE-7603: Support Graph Token Streams in Q...

Posted by mikemccand <gi...@git.apache.org>.
Github user mikemccand commented on a diff in the pull request:

    https://github.com/apache/lucene-solr/pull/129#discussion_r94216469
  
    --- Diff: lucene/core/src/java/org/apache/lucene/util/graph/GraphTokenStreamFiniteStrings.java ---
    @@ -210,82 +215,20 @@ private void finish() {
        */
       private void finish(int maxDeterminizedStates) {
         Automaton automaton = builder.finish();
    -
    -    // System.out.println("before det:\n" + automaton.toDot());
    -
    -    Transition t = new Transition();
    -
    -    // TODO: should we add "eps back to initial node" for all states,
    -    // and det that?  then we don't need to revisit initial node at
    -    // every position?  but automaton could blow up?  And, this makes it
    -    // harder to skip useless positions at search time?
    -
    -    if (anyTermID != -1) {
    -
    -      // Make sure there are no leading or trailing ANY:
    -      int count = automaton.initTransition(0, t);
    -      for (int i = 0; i < count; i++) {
    -        automaton.getNextTransition(t);
    -        if (anyTermID >= t.min && anyTermID <= t.max) {
    -          throw new IllegalStateException("automaton cannot lead with an ANY transition");
    -        }
    -      }
    -
    -      int numStates = automaton.getNumStates();
    -      for (int i = 0; i < numStates; i++) {
    -        count = automaton.initTransition(i, t);
    -        for (int j = 0; j < count; j++) {
    -          automaton.getNextTransition(t);
    -          if (automaton.isAccept(t.dest) && anyTermID >= t.min && anyTermID <= t.max) {
    -            throw new IllegalStateException("automaton cannot end with an ANY transition");
    -          }
    -        }
    -      }
    -
    -      int termCount = termToID.size();
    -
    -      // We have to carefully translate these transitions so automaton
    -      // realizes they also match all other terms:
    -      Automaton newAutomaton = new Automaton();
    -      for (int i = 0; i < numStates; i++) {
    -        newAutomaton.createState();
    -        newAutomaton.setAccept(i, automaton.isAccept(i));
    -      }
    -
    -      for (int i = 0; i < numStates; i++) {
    -        count = automaton.initTransition(i, t);
    -        for (int j = 0; j < count; j++) {
    -          automaton.getNextTransition(t);
    -          int min, max;
    -          if (t.min <= anyTermID && anyTermID <= t.max) {
    -            // Match any term
    -            min = 0;
    -            max = termCount - 1;
    -          } else {
    -            min = t.min;
    -            max = t.max;
    -          }
    -          newAutomaton.addTransition(t.source, t.dest, min, max);
    -        }
    -      }
    -      newAutomaton.finishState();
    -      automaton = newAutomaton;
    -    }
    -
         det = Operations.removeDeadStates(Operations.determinize(automaton, maxDeterminizedStates));
       }
     
    -  private int getTermID(BytesRef term) {
    +  private int getTermID(int incr, BytesRef term) {
         Integer id = termToID.get(term);
    -    if (id == null) {
    +    if (incr > 1 || id == null) {
    --- End diff --
    
    Hmm doesn't this mean that if the same term shows up, but with different `incr`, that it will get different `id` assigned?  But I think that is actually fine, since nowhere here do we depend on / expect that the same term must have the same id.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] lucene-solr pull request #129: LUCENE-7603: Support Graph Token Streams in Q...

Posted by mattweber <gi...@git.apache.org>.
Github user mattweber commented on a diff in the pull request:

    https://github.com/apache/lucene-solr/pull/129#discussion_r94171262
  
    --- Diff: lucene/core/src/java/org/apache/lucene/util/graph/GraphTokenStreamFiniteStrings.java ---
    @@ -0,0 +1,294 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.lucene.util.graph;
    +
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +
    +import org.apache.lucene.analysis.TokenStream;
    +import org.apache.lucene.analysis.tokenattributes.BytesTermAttribute;
    +import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
    +import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
    +import org.apache.lucene.analysis.tokenattributes.PositionLengthAttribute;
    +import org.apache.lucene.analysis.tokenattributes.TermToBytesRefAttribute;
    +import org.apache.lucene.util.BytesRef;
    +import org.apache.lucene.util.IntsRef;
    +import org.apache.lucene.util.automaton.Automaton;
    +import org.apache.lucene.util.automaton.FiniteStringsIterator;
    +import org.apache.lucene.util.automaton.Operations;
    +import org.apache.lucene.util.automaton.Transition;
    +
    +import static org.apache.lucene.util.automaton.Operations.DEFAULT_MAX_DETERMINIZED_STATES;
    +
    +/**
    + * Creates a list of {@link TokenStream} where each stream is the tokens that make up a finite string in graph token stream.  To do this,
    + * the graph token stream is converted to an {@link Automaton} and from there we use a {@link FiniteStringsIterator} to collect the various
    + * token streams for each finite string.
    + */
    +public class GraphTokenStreamFiniteStrings {
    +  /* TODO:
    +     Most of this is a combination of code from TermAutomatonQuery and TokenStreamToTermAutomatonQuery. Would be
    +     good to make this so it could be shared. */
    +  private final Automaton.Builder builder;
    +  Automaton det;
    +  private final Map<BytesRef, Integer> termToID = new HashMap<>();
    +  private final Map<Integer, BytesRef> idToTerm = new HashMap<>();
    +  private int anyTermID = -1;
    +
    +  public GraphTokenStreamFiniteStrings() {
    +    this.builder = new Automaton.Builder();
    +  }
    +
    +  private static class BytesRefArrayTokenStream extends TokenStream {
    +    private final BytesTermAttribute termAtt = addAttribute(BytesTermAttribute.class);
    +    private final BytesRef[] terms;
    +    private int offset;
    +
    +    BytesRefArrayTokenStream(BytesRef[] terms) {
    +      this.terms = terms;
    +      offset = 0;
    +    }
    +
    +    @Override
    +    public boolean incrementToken() throws IOException {
    +      if (offset < terms.length) {
    +        clearAttributes();
    +        termAtt.setBytesRef(terms[offset]);
    +        offset = offset + 1;
    +        return true;
    +      }
    +
    +      return false;
    +    }
    +  }
    +
    +  /**
    +   * Gets the list of finite string token streams from the given input graph token stream.
    +   */
    +  public List<TokenStream> getTokenStreams(final TokenStream in) throws IOException {
    +    // build automation
    +    build(in);
    +
    +    List<TokenStream> tokenStreams = new ArrayList<>();
    +    final FiniteStringsIterator finiteStrings = new FiniteStringsIterator(det);
    +    for (IntsRef string; (string = finiteStrings.next()) != null; ) {
    +      final BytesRef[] tokens = new BytesRef[string.length];
    +      for (int idx = string.offset, len = string.offset + string.length; idx < len; idx++) {
    +        tokens[idx - string.offset] = idToTerm.get(string.ints[idx]);
    +      }
    +
    +      tokenStreams.add(new BytesRefArrayTokenStream(tokens));
    +    }
    +
    +    return tokenStreams;
    +  }
    +
    +  private void build(final TokenStream in) throws IOException {
    +    if (det != null) {
    +      throw new IllegalStateException("Automation already built");
    +    }
    +
    +    final TermToBytesRefAttribute termBytesAtt = in.addAttribute(TermToBytesRefAttribute.class);
    +    final PositionIncrementAttribute posIncAtt = in.addAttribute(PositionIncrementAttribute.class);
    +    final PositionLengthAttribute posLengthAtt = in.addAttribute(PositionLengthAttribute.class);
    +    final OffsetAttribute offsetAtt = in.addAttribute(OffsetAttribute.class);
    +
    +    in.reset();
    +
    +    int pos = -1;
    +    int lastPos = 0;
    +    int maxOffset = 0;
    +    int maxPos = -1;
    +    int state = -1;
    +    while (in.incrementToken()) {
    +      int posInc = posIncAtt.getPositionIncrement();
    +      assert pos > -1 || posInc > 0;
    +
    +      if (posInc > 1) {
    --- End diff --
    
    @dsmiley That was actually pulled out of the existing [TokenStreamToTermAutomatonQuery.java](https://github.com/apache/lucene-solr/blob/master/lucene/sandbox/src/java/org/apache/lucene/search/TokenStreamToTermAutomatonQuery.java#L77).  Let me look into it more.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] lucene-solr pull request #129: LUCENE-7603: Support Graph Token Streams in Q...

Posted by mattweber <gi...@git.apache.org>.
Github user mattweber closed the pull request at:

    https://github.com/apache/lucene-solr/pull/129


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] lucene-solr pull request #129: LUCENE-7603: Support Graph Token Streams in Q...

Posted by dsmiley <gi...@git.apache.org>.
Github user dsmiley commented on a diff in the pull request:

    https://github.com/apache/lucene-solr/pull/129#discussion_r94241922
  
    --- Diff: lucene/core/src/java/org/apache/lucene/util/graph/GraphTokenStreamFiniteStrings.java ---
    @@ -80,22 +77,41 @@ public boolean incrementToken() throws IOException {
         }
       }
     
    +  private GraphTokenStreamFiniteStrings() {
    +    this.builder = new Automaton.Builder();
    --- End diff --
    
    The other fields are initialized at the declaration; might as well move this here too?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] lucene-solr pull request #129: LUCENE-7603: Support Graph Token Streams in Q...

Posted by mikemccand <gi...@git.apache.org>.
Github user mikemccand commented on a diff in the pull request:

    https://github.com/apache/lucene-solr/pull/129#discussion_r94216511
  
    --- Diff: lucene/core/src/java/org/apache/lucene/util/graph/GraphTokenStreamFiniteStrings.java ---
    @@ -174,25 +191,13 @@ private void setAccept(int state, boolean accept) {
       /**
        * Adds a transition to the automaton.
        */
    -  private void addTransition(int source, int dest, String term) {
    -    addTransition(source, dest, new BytesRef(term));
    -  }
    -
    -  /**
    -   * Adds a transition to the automaton.
    -   */
    -  private void addTransition(int source, int dest, BytesRef term) {
    +  private int addTransition(int source, int dest, int incr, BytesRef term) {
         if (term == null) {
    --- End diff --
    
    This can become an assert?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] lucene-solr issue #129: LUCENE-7603: Support Graph Token Streams in QueryBui...

Posted by mattweber <gi...@git.apache.org>.
Github user mattweber commented on the issue:

    https://github.com/apache/lucene-solr/pull/129
  
    @mikemccand I addressed you comments.  I also added some more tests and fixed a bug that would yield wrong increment when a term that had previously been seen was found again with an increment of 0.  Tests were added.  I have squashed these changes with the previous commit so it is clear to see the difference between the original PR which did not support position increments and the new one that does.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] lucene-solr issue #129: LUCENE-7603: Support Graph Token Streams in QueryBui...

Posted by mikemccand <gi...@git.apache.org>.
Github user mikemccand commented on the issue:

    https://github.com/apache/lucene-solr/pull/129
  
    I've merged this into Lucene's master (7.0), and I'm working on 6.x (#130) now.  Thanks @mattweber! Can you close this?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] lucene-solr issue #129: LUCENE-7603: Support Graph Token Streams in QueryBui...

Posted by mikemccand <gi...@git.apache.org>.
Github user mikemccand commented on the issue:

    https://github.com/apache/lucene-solr/pull/129
  
    This change looks great to me!  What an awesome improvement, to properly use graph token streams at search time so multi-token synonyms are correct.
    
    I'll push this in a few days once I'm back home unless someone pushes first (@dsmiley feel free)...
    
    Thank you @mattweber!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] lucene-solr pull request #129: LUCENE-7603: Support Graph Token Streams in Q...

Posted by dsmiley <gi...@git.apache.org>.
Github user dsmiley commented on a diff in the pull request:

    https://github.com/apache/lucene-solr/pull/129#discussion_r94243010
  
    --- Diff: lucene/core/src/java/org/apache/lucene/util/graph/GraphTokenStreamFiniteStrings.java ---
    @@ -210,85 +199,41 @@ private void finish() {
        */
       private void finish(int maxDeterminizedStates) {
         Automaton automaton = builder.finish();
    -
    -    // System.out.println("before det:\n" + automaton.toDot());
    -
    -    Transition t = new Transition();
    -
    -    // TODO: should we add "eps back to initial node" for all states,
    -    // and det that?  then we don't need to revisit initial node at
    -    // every position?  but automaton could blow up?  And, this makes it
    -    // harder to skip useless positions at search time?
    -
    -    if (anyTermID != -1) {
    -
    -      // Make sure there are no leading or trailing ANY:
    -      int count = automaton.initTransition(0, t);
    -      for (int i = 0; i < count; i++) {
    -        automaton.getNextTransition(t);
    -        if (anyTermID >= t.min && anyTermID <= t.max) {
    -          throw new IllegalStateException("automaton cannot lead with an ANY transition");
    -        }
    -      }
    -
    -      int numStates = automaton.getNumStates();
    -      for (int i = 0; i < numStates; i++) {
    -        count = automaton.initTransition(i, t);
    -        for (int j = 0; j < count; j++) {
    -          automaton.getNextTransition(t);
    -          if (automaton.isAccept(t.dest) && anyTermID >= t.min && anyTermID <= t.max) {
    -            throw new IllegalStateException("automaton cannot end with an ANY transition");
    -          }
    -        }
    -      }
    -
    -      int termCount = termToID.size();
    -
    -      // We have to carefully translate these transitions so automaton
    -      // realizes they also match all other terms:
    -      Automaton newAutomaton = new Automaton();
    -      for (int i = 0; i < numStates; i++) {
    -        newAutomaton.createState();
    -        newAutomaton.setAccept(i, automaton.isAccept(i));
    -      }
    -
    -      for (int i = 0; i < numStates; i++) {
    -        count = automaton.initTransition(i, t);
    -        for (int j = 0; j < count; j++) {
    -          automaton.getNextTransition(t);
    -          int min, max;
    -          if (t.min <= anyTermID && anyTermID <= t.max) {
    -            // Match any term
    -            min = 0;
    -            max = termCount - 1;
    -          } else {
    -            min = t.min;
    -            max = t.max;
    -          }
    -          newAutomaton.addTransition(t.source, t.dest, min, max);
    -        }
    -      }
    -      newAutomaton.finishState();
    -      automaton = newAutomaton;
    -    }
    -
         det = Operations.removeDeadStates(Operations.determinize(automaton, maxDeterminizedStates));
       }
     
    -  private int getTermID(BytesRef term) {
    -    Integer id = termToID.get(term);
    -    if (id == null) {
    -      id = termToID.size();
    -      if (term != null) {
    -        term = BytesRef.deepCopyOf(term);
    -      }
    -      termToID.put(term, id);
    +  /**
    +   * Gets an integer id for a given term.
    +   *
    +   * If there is no position gaps for this token then we can reuse the id for the same term if it appeared at another
    +   * position without a gap.  If we have a position gap generate a new id so we can keep track of the position
    +   * increment.
    +   */
    +  private int getTermID(int incr, int prevIncr, BytesRef term) {
    +    assert term != null;
    +    boolean isStackedGap = incr == 0 && prevIncr > 1;
    +    boolean hasGap = incr > 1;
    +    term = BytesRef.deepCopyOf(term);
    --- End diff --
    
    The deepCopyOf is only needed if you generate a new ID, not for an existing one.  
    
    BTW... have you seen BytesRefHash?  I think re-using that could minimize the code here to deal with this stuff.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] lucene-solr pull request #129: LUCENE-7603: Support Graph Token Streams in Q...

Posted by mikemccand <gi...@git.apache.org>.
Github user mikemccand commented on a diff in the pull request:

    https://github.com/apache/lucene-solr/pull/129#discussion_r94217475
  
    --- Diff: lucene/core/src/java/org/apache/lucene/util/graph/GraphTokenStreamFiniteStrings.java ---
    @@ -0,0 +1,237 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.lucene.util.graph;
    +
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +
    +import org.apache.lucene.analysis.TokenStream;
    +import org.apache.lucene.analysis.tokenattributes.BytesTermAttribute;
    +import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
    +import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
    +import org.apache.lucene.analysis.tokenattributes.PositionLengthAttribute;
    +import org.apache.lucene.analysis.tokenattributes.TermToBytesRefAttribute;
    +import org.apache.lucene.util.BytesRef;
    +import org.apache.lucene.util.IntsRef;
    +import org.apache.lucene.util.automaton.Automaton;
    +import org.apache.lucene.util.automaton.FiniteStringsIterator;
    +import org.apache.lucene.util.automaton.Operations;
    +
    +import static org.apache.lucene.util.automaton.Operations.DEFAULT_MAX_DETERMINIZED_STATES;
    +
    +/**
    + * Creates a list of {@link TokenStream} where each stream is the tokens that make up a finite string in graph token stream.  To do this,
    + * the graph token stream is converted to an {@link Automaton} and from there we use a {@link FiniteStringsIterator} to collect the various
    + * token streams for each finite string.
    + */
    +public class GraphTokenStreamFiniteStrings {
    +  /* TODO:
    +     Most of this is a combination of code from TermAutomatonQuery and TokenStreamToTermAutomatonQuery. Would be
    +     good to make this so it could be shared. */
    +  private final Automaton.Builder builder;
    +  Automaton det;
    +  private final Map<BytesRef, Integer> termToID = new HashMap<>();
    +  private final Map<Integer, BytesRef> idToTerm = new HashMap<>();
    +  private final Map<Integer, Integer> idToInc = new HashMap<>();
    +
    +  public GraphTokenStreamFiniteStrings() {
    +    this.builder = new Automaton.Builder();
    +  }
    +
    +  private static class BytesRefArrayTokenStream extends TokenStream {
    +    private final BytesTermAttribute termAtt = addAttribute(BytesTermAttribute.class);
    +    private final PositionIncrementAttribute posIncAtt = addAttribute(PositionIncrementAttribute.class);
    +
    +    private final BytesRef[] terms;
    +    private final int[] increments;
    +    private int offset;
    +
    +    BytesRefArrayTokenStream(BytesRef[] terms, int[] increments) {
    +      this.terms = terms;
    +      this.increments = increments;
    +      assert terms.length == increments.length;
    +      offset = 0;
    +    }
    +
    +    @Override
    +    public boolean incrementToken() throws IOException {
    +      if (offset < terms.length) {
    +        clearAttributes();
    +        termAtt.setBytesRef(terms[offset]);
    +        posIncAtt.setPositionIncrement(increments[offset]);
    +        offset = offset + 1;
    +        return true;
    +      }
    +
    +      return false;
    +    }
    +  }
    +
    +  /**
    +   * Gets the list of finite string token streams from the given input graph token stream.
    +   */
    +  public List<TokenStream> getTokenStreams(final TokenStream in) throws IOException {
    +    // build automation
    +    build(in);
    +
    +    List<TokenStream> tokenStreams = new ArrayList<>();
    +    final FiniteStringsIterator finiteStrings = new FiniteStringsIterator(det);
    +    for (IntsRef string; (string = finiteStrings.next()) != null; ) {
    +      final BytesRef[] tokens = new BytesRef[string.length];
    +      final int[] increments = new int[string.length];
    +      for (int idx = string.offset, len = string.offset + string.length; idx < len; idx++) {
    +        int id = string.ints[idx];
    +        int offset = idx - string.offset;
    +        tokens[offset] = idToTerm.get(id);
    +        if (idToInc.containsKey(id)) {
    +          increments[offset] = idToInc.get(id);
    +        } else {
    +          increments[offset] = 1;
    +        }
    +      }
    +
    +      tokenStreams.add(new BytesRefArrayTokenStream(tokens, increments));
    +    }
    +
    +    return tokenStreams;
    +  }
    +
    +  private void build(final TokenStream in) throws IOException {
    +    if (det != null) {
    +      throw new IllegalStateException("Automation already built");
    +    }
    +
    +    final TermToBytesRefAttribute termBytesAtt = in.addAttribute(TermToBytesRefAttribute.class);
    +    final PositionIncrementAttribute posIncAtt = in.addAttribute(PositionIncrementAttribute.class);
    +    final PositionLengthAttribute posLengthAtt = in.addAttribute(PositionLengthAttribute.class);
    +    final OffsetAttribute offsetAtt = in.addAttribute(OffsetAttribute.class);
    +
    +    in.reset();
    +
    +    int pos = -1;
    +    int lastIncr = 1;
    +    int maxOffset = 0;
    +    int maxPos = -1;
    +    int state = -1;
    +    while (in.incrementToken()) {
    +      int posInc = posIncAtt.getPositionIncrement();
    +
    +      // always use inc 1 while building, but save original increment
    +      int fakePosInc = posInc > 1 ? 1 : posInc;
    +
    +      assert pos > -1 || fakePosInc > 0;
    --- End diff --
    
    Can we upgrade this to a real `if`?  I.e. we need a well-formed `TokenStream` input ... it cannot have `posInc=0` on its first token.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] lucene-solr issue #129: LUCENE-7603: Support Graph Token Streams in QueryBui...

Posted by mattweber <gi...@git.apache.org>.
Github user mattweber commented on the issue:

    https://github.com/apache/lucene-solr/pull/129
  
    Rebased against master, added missing ASF header.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] lucene-solr issue #129: LUCENE-7603: Support Graph Token Streams in QueryBui...

Posted by mattweber <gi...@git.apache.org>.
Github user mattweber commented on the issue:

    https://github.com/apache/lucene-solr/pull/129
  
    Thanks @dsmiley!  I have just pushed up code with your suggestions except for using `BytesRefHash` due to the fact we might have the same `BytesRef` but need a different id because we have position gap.
    
    This has been great, love the feedback!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] lucene-solr pull request #129: LUCENE-7603: Support Graph Token Streams in Q...

Posted by mikemccand <gi...@git.apache.org>.
Github user mikemccand commented on a diff in the pull request:

    https://github.com/apache/lucene-solr/pull/129#discussion_r94215751
  
    --- Diff: lucene/core/src/java/org/apache/lucene/util/graph/GraphTokenStreamFiniteStrings.java ---
    @@ -114,21 +127,20 @@ private void build(final TokenStream in) throws IOException {
         in.reset();
     
         int pos = -1;
    -    int lastPos = 0;
    +    int lastIncr = 1;
         int maxOffset = 0;
         int maxPos = -1;
         int state = -1;
         while (in.incrementToken()) {
           int posInc = posIncAtt.getPositionIncrement();
    -      assert pos > -1 || posInc > 0;
     
    -      if (posInc > 1) {
    -        throw new IllegalArgumentException("cannot handle holes; to accept any term, use '*' term");
    -      }
    +      // always use inc 1 while building, but save original increment
    +      int fakePosInc = posInc > 1 ? 1 : posInc;
    --- End diff --
    
    Maybe just `Math.min(1, posInc)` instead?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] lucene-solr pull request #129: LUCENE-7603: Support Graph Token Streams in Q...

Posted by mikemccand <gi...@git.apache.org>.
Github user mikemccand commented on a diff in the pull request:

    https://github.com/apache/lucene-solr/pull/129#discussion_r94217160
  
    --- Diff: lucene/core/src/java/org/apache/lucene/util/graph/GraphTokenStreamFiniteStrings.java ---
    @@ -0,0 +1,237 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.lucene.util.graph;
    +
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +
    +import org.apache.lucene.analysis.TokenStream;
    +import org.apache.lucene.analysis.tokenattributes.BytesTermAttribute;
    +import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
    +import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
    +import org.apache.lucene.analysis.tokenattributes.PositionLengthAttribute;
    +import org.apache.lucene.analysis.tokenattributes.TermToBytesRefAttribute;
    +import org.apache.lucene.util.BytesRef;
    +import org.apache.lucene.util.IntsRef;
    +import org.apache.lucene.util.automaton.Automaton;
    +import org.apache.lucene.util.automaton.FiniteStringsIterator;
    +import org.apache.lucene.util.automaton.Operations;
    +
    +import static org.apache.lucene.util.automaton.Operations.DEFAULT_MAX_DETERMINIZED_STATES;
    +
    +/**
    + * Creates a list of {@link TokenStream} where each stream is the tokens that make up a finite string in graph token stream.  To do this,
    + * the graph token stream is converted to an {@link Automaton} and from there we use a {@link FiniteStringsIterator} to collect the various
    + * token streams for each finite string.
    + */
    +public class GraphTokenStreamFiniteStrings {
    +  /* TODO:
    +     Most of this is a combination of code from TermAutomatonQuery and TokenStreamToTermAutomatonQuery. Would be
    +     good to make this so it could be shared. */
    +  private final Automaton.Builder builder;
    +  Automaton det;
    +  private final Map<BytesRef, Integer> termToID = new HashMap<>();
    +  private final Map<Integer, BytesRef> idToTerm = new HashMap<>();
    +  private final Map<Integer, Integer> idToInc = new HashMap<>();
    +
    +  public GraphTokenStreamFiniteStrings() {
    +    this.builder = new Automaton.Builder();
    +  }
    +
    +  private static class BytesRefArrayTokenStream extends TokenStream {
    +    private final BytesTermAttribute termAtt = addAttribute(BytesTermAttribute.class);
    +    private final PositionIncrementAttribute posIncAtt = addAttribute(PositionIncrementAttribute.class);
    +
    +    private final BytesRef[] terms;
    +    private final int[] increments;
    +    private int offset;
    +
    +    BytesRefArrayTokenStream(BytesRef[] terms, int[] increments) {
    +      this.terms = terms;
    +      this.increments = increments;
    +      assert terms.length == increments.length;
    +      offset = 0;
    +    }
    +
    +    @Override
    +    public boolean incrementToken() throws IOException {
    +      if (offset < terms.length) {
    +        clearAttributes();
    +        termAtt.setBytesRef(terms[offset]);
    +        posIncAtt.setPositionIncrement(increments[offset]);
    +        offset = offset + 1;
    +        return true;
    +      }
    +
    +      return false;
    +    }
    +  }
    +
    +  /**
    +   * Gets the list of finite string token streams from the given input graph token stream.
    +   */
    +  public List<TokenStream> getTokenStreams(final TokenStream in) throws IOException {
    --- End diff --
    
    Could we make this method private, make this class's constructor private, and add a `static` method here, the sole public method on this class, that receives the incoming `TokenStream` and returns the resulting `TokenStream[]`?  Otherwise the API is sort of awkard, since e.g. this method seems like a getter yet it's doing lots of side-effects under the hood ...


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] lucene-solr pull request: Adding now() support to expressions

Posted by rmuir <gi...@git.apache.org>.
Github user rmuir commented on a diff in the pull request:

    https://github.com/apache/lucene-solr/pull/129#discussion_r25448647
  
    --- Diff: lucene/expressions/src/java/org/apache/lucene/expressions/js/JavascriptCompiler.java ---
    @@ -565,11 +566,16 @@ private static int findSingleQuoteStringEnd(String text, int start) {
             checkFunction(method, JavascriptCompiler.class.getClassLoader());
             map.put(call, method);
           }
    +      map.put("now", JavascriptCompiler.class.getMethod("getNow"));
         } catch (ReflectiveOperationException | IOException e) {
           throw new Error("Cannot resolve function", e);
         }
         DEFAULT_FUNCTIONS = Collections.unmodifiableMap(map);
       }
    +
    --- End diff --
    
    I am not sure we should add this function, for example it will not work in a distributed setup (where now needs to be 'set' to a value). Also these functions need to be idempotent, and return the same thing for the same inputs.
    
    If someone wants this functionality they can currently expose with bindings, a ConstantValueSource named "_now" or "now" that they initialize to the time up front that they want, and its consistent across the entire expression, and that allows it to work in a distributed case as well.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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


[GitHub] lucene-solr pull request #129: LUCENE-7603: Support Graph Token Streams in Q...

Posted by dsmiley <gi...@git.apache.org>.
Github user dsmiley commented on a diff in the pull request:

    https://github.com/apache/lucene-solr/pull/129#discussion_r94148633
  
    --- Diff: lucene/core/src/java/org/apache/lucene/util/graph/GraphTokenStreamFiniteStrings.java ---
    @@ -0,0 +1,294 @@
    +/*
    + * Licensed to the Apache Software Foundation (ASF) under one or more
    + * contributor license agreements.  See the NOTICE file distributed with
    + * this work for additional information regarding copyright ownership.
    + * The ASF licenses this file to You under the Apache License, Version 2.0
    + * (the "License"); you may not use this file except in compliance with
    + * the License.  You may obtain a copy of the License at
    + *
    + *     http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing, software
    + * distributed under the License is distributed on an "AS IS" BASIS,
    + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    + * See the License for the specific language governing permissions and
    + * limitations under the License.
    + */
    +
    +package org.apache.lucene.util.graph;
    +
    +import java.io.IOException;
    +import java.util.ArrayList;
    +import java.util.HashMap;
    +import java.util.List;
    +import java.util.Map;
    +
    +import org.apache.lucene.analysis.TokenStream;
    +import org.apache.lucene.analysis.tokenattributes.BytesTermAttribute;
    +import org.apache.lucene.analysis.tokenattributes.OffsetAttribute;
    +import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
    +import org.apache.lucene.analysis.tokenattributes.PositionLengthAttribute;
    +import org.apache.lucene.analysis.tokenattributes.TermToBytesRefAttribute;
    +import org.apache.lucene.util.BytesRef;
    +import org.apache.lucene.util.IntsRef;
    +import org.apache.lucene.util.automaton.Automaton;
    +import org.apache.lucene.util.automaton.FiniteStringsIterator;
    +import org.apache.lucene.util.automaton.Operations;
    +import org.apache.lucene.util.automaton.Transition;
    +
    +import static org.apache.lucene.util.automaton.Operations.DEFAULT_MAX_DETERMINIZED_STATES;
    +
    +/**
    + * Creates a list of {@link TokenStream} where each stream is the tokens that make up a finite string in graph token stream.  To do this,
    + * the graph token stream is converted to an {@link Automaton} and from there we use a {@link FiniteStringsIterator} to collect the various
    + * token streams for each finite string.
    + */
    +public class GraphTokenStreamFiniteStrings {
    +  /* TODO:
    +     Most of this is a combination of code from TermAutomatonQuery and TokenStreamToTermAutomatonQuery. Would be
    +     good to make this so it could be shared. */
    +  private final Automaton.Builder builder;
    +  Automaton det;
    +  private final Map<BytesRef, Integer> termToID = new HashMap<>();
    +  private final Map<Integer, BytesRef> idToTerm = new HashMap<>();
    +  private int anyTermID = -1;
    +
    +  public GraphTokenStreamFiniteStrings() {
    +    this.builder = new Automaton.Builder();
    +  }
    +
    +  private static class BytesRefArrayTokenStream extends TokenStream {
    +    private final BytesTermAttribute termAtt = addAttribute(BytesTermAttribute.class);
    +    private final BytesRef[] terms;
    +    private int offset;
    +
    +    BytesRefArrayTokenStream(BytesRef[] terms) {
    +      this.terms = terms;
    +      offset = 0;
    +    }
    +
    +    @Override
    +    public boolean incrementToken() throws IOException {
    +      if (offset < terms.length) {
    +        clearAttributes();
    +        termAtt.setBytesRef(terms[offset]);
    +        offset = offset + 1;
    +        return true;
    +      }
    +
    +      return false;
    +    }
    +  }
    +
    +  /**
    +   * Gets the list of finite string token streams from the given input graph token stream.
    +   */
    +  public List<TokenStream> getTokenStreams(final TokenStream in) throws IOException {
    +    // build automation
    +    build(in);
    +
    +    List<TokenStream> tokenStreams = new ArrayList<>();
    +    final FiniteStringsIterator finiteStrings = new FiniteStringsIterator(det);
    +    for (IntsRef string; (string = finiteStrings.next()) != null; ) {
    +      final BytesRef[] tokens = new BytesRef[string.length];
    +      for (int idx = string.offset, len = string.offset + string.length; idx < len; idx++) {
    +        tokens[idx - string.offset] = idToTerm.get(string.ints[idx]);
    +      }
    +
    +      tokenStreams.add(new BytesRefArrayTokenStream(tokens));
    +    }
    +
    +    return tokenStreams;
    +  }
    +
    +  private void build(final TokenStream in) throws IOException {
    +    if (det != null) {
    +      throw new IllegalStateException("Automation already built");
    +    }
    +
    +    final TermToBytesRefAttribute termBytesAtt = in.addAttribute(TermToBytesRefAttribute.class);
    +    final PositionIncrementAttribute posIncAtt = in.addAttribute(PositionIncrementAttribute.class);
    +    final PositionLengthAttribute posLengthAtt = in.addAttribute(PositionLengthAttribute.class);
    +    final OffsetAttribute offsetAtt = in.addAttribute(OffsetAttribute.class);
    +
    +    in.reset();
    +
    +    int pos = -1;
    +    int lastPos = 0;
    +    int maxOffset = 0;
    +    int maxPos = -1;
    +    int state = -1;
    +    while (in.incrementToken()) {
    +      int posInc = posIncAtt.getPositionIncrement();
    +      assert pos > -1 || posInc > 0;
    +
    +      if (posInc > 1) {
    --- End diff --
    
    This seems like a notable limitation that should be documented in javadocs somewhere.  Can't we support holes without demanding the stream use '*' ?  And might there be a test for this?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

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