You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ad...@apache.org on 2014/01/18 06:36:28 UTC

svn commit: r1559318 - in /lucene/dev/branches/lucene_solr_4_6/lucene: ./ queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/processors/ queryparser/src/test/org/apache/lucene/queryparser/util/

Author: adrianocrestani
Date: Sat Jan 18 05:36:28 2014
New Revision: 1559318

URL: http://svn.apache.org/r1559318
Log:
LUCENE-Flexible StandardQueryParser behaves differently than ClassicQueryParser (4.6 branch)

Modified:
    lucene/dev/branches/lucene_solr_4_6/lucene/CHANGES.txt
    lucene/dev/branches/lucene_solr_4_6/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/processors/AnalyzerQueryNodeProcessor.java
    lucene/dev/branches/lucene_solr_4_6/lucene/queryparser/src/test/org/apache/lucene/queryparser/util/QueryParserTestBase.java

Modified: lucene/dev/branches/lucene_solr_4_6/lucene/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene_solr_4_6/lucene/CHANGES.txt?rev=1559318&r1=1559317&r2=1559318&view=diff
==============================================================================
--- lucene/dev/branches/lucene_solr_4_6/lucene/CHANGES.txt (original)
+++ lucene/dev/branches/lucene_solr_4_6/lucene/CHANGES.txt Sat Jan 18 05:36:28 2014
@@ -22,6 +22,9 @@ Bug fixes
 
 * LUCENE-5394: Fix TokenSources.getTokenStream to return payloads if
   they were indexed with the term vectors. (Mike McCandless)
+  
+* LUCENE-5344: Flexible StandardQueryParser behaves differently than 
+  ClassicQueryParser. (Adriano Crestani)
 
 * LUCENE-5375: ToChildBlockJoinQuery works harder to detect mis-use,
   when the parent query incorrectly returns child documents, and throw
@@ -139,7 +142,7 @@ Bug Fixes
 * LUCENE-5342: Fixed bulk-merge issue in CompressingStoredFieldsFormat which
   created corrupted segments when mixing chunk sizes.
   Lucene41StoredFieldsFormat is not impacted. (Adrien Grand, Robert Muir)
-
+  
 API Changes
 
 * LUCENE-5222: Add SortField.needsScores(). Previously it was not possible

Modified: lucene/dev/branches/lucene_solr_4_6/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/processors/AnalyzerQueryNodeProcessor.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene_solr_4_6/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/processors/AnalyzerQueryNodeProcessor.java?rev=1559318&r1=1559317&r2=1559318&view=diff
==============================================================================
--- lucene/dev/branches/lucene_solr_4_6/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/processors/AnalyzerQueryNodeProcessor.java (original)
+++ lucene/dev/branches/lucene_solr_4_6/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/processors/AnalyzerQueryNodeProcessor.java Sat Jan 18 05:36:28 2014
@@ -19,6 +19,7 @@ package org.apache.lucene.queryparser.fl
 
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.Collections;
 import java.util.LinkedList;
 import java.util.List;
 
@@ -27,23 +28,32 @@ import org.apache.lucene.analysis.Cachin
 import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
 import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
+import org.apache.lucene.index.Term;
 import org.apache.lucene.queryparser.flexible.core.QueryNodeException;
 import org.apache.lucene.queryparser.flexible.core.config.QueryConfigHandler;
+import org.apache.lucene.queryparser.flexible.core.nodes.BooleanQueryNode;
 import org.apache.lucene.queryparser.flexible.core.nodes.FieldQueryNode;
 import org.apache.lucene.queryparser.flexible.core.nodes.FuzzyQueryNode;
 import org.apache.lucene.queryparser.flexible.core.nodes.GroupQueryNode;
+import org.apache.lucene.queryparser.flexible.core.nodes.ModifierQueryNode;
 import org.apache.lucene.queryparser.flexible.core.nodes.NoTokenFoundQueryNode;
 import org.apache.lucene.queryparser.flexible.core.nodes.QueryNode;
 import org.apache.lucene.queryparser.flexible.core.nodes.QuotedFieldQueryNode;
 import org.apache.lucene.queryparser.flexible.core.nodes.RangeQueryNode;
 import org.apache.lucene.queryparser.flexible.core.nodes.TextableQueryNode;
 import org.apache.lucene.queryparser.flexible.core.nodes.TokenizedPhraseQueryNode;
+import org.apache.lucene.queryparser.flexible.core.nodes.ModifierQueryNode.Modifier;
 import org.apache.lucene.queryparser.flexible.core.processors.QueryNodeProcessorImpl;
 import org.apache.lucene.queryparser.flexible.standard.config.StandardQueryConfigHandler.ConfigurationKeys;
+import org.apache.lucene.queryparser.flexible.standard.config.StandardQueryConfigHandler.Operator;
 import org.apache.lucene.queryparser.flexible.standard.nodes.MultiPhraseQueryNode;
 import org.apache.lucene.queryparser.flexible.standard.nodes.RegexpQueryNode;
 import org.apache.lucene.queryparser.flexible.standard.nodes.StandardBooleanQueryNode;
 import org.apache.lucene.queryparser.flexible.standard.nodes.WildcardQueryNode;
+import org.apache.lucene.search.BooleanClause;
+import org.apache.lucene.search.BooleanQuery;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.IOUtils;
 
 /**
@@ -73,6 +83,8 @@ public class AnalyzerQueryNodeProcessor 
   private Analyzer analyzer;
 
   private boolean positionIncrementsEnabled;
+  
+  private Operator defaultOperator;
 
   public AnalyzerQueryNodeProcessor() {
     // empty constructor
@@ -86,6 +98,8 @@ public class AnalyzerQueryNodeProcessor 
       this.analyzer = analyzer;
       this.positionIncrementsEnabled = false;
       Boolean positionIncrementsEnabled = getQueryConfigHandler().get(ConfigurationKeys.ENABLE_POSITION_INCREMENTS);
+      Operator defaultOperator = getQueryConfigHandler().get(ConfigurationKeys.DEFAULT_OPERATOR);
+      this.defaultOperator = defaultOperator != null ? defaultOperator : Operator.OR;
 
       if (positionIncrementsEnabled != null) {
           this.positionIncrementsEnabled = positionIncrementsEnabled;
@@ -185,24 +199,70 @@ public class AnalyzerQueryNodeProcessor 
       } else if (severalTokensAtSamePosition || !(node instanceof QuotedFieldQueryNode)) {
         if (positionCount == 1 || !(node instanceof QuotedFieldQueryNode)) {
           // no phrase query:
-          LinkedList<QueryNode> children = new LinkedList<QueryNode>();
-
-          for (int i = 0; i < numTokens; i++) {
-            String term = null;
-            try {
-              boolean hasNext = buffer.incrementToken();
-              assert hasNext == true;
-              term = termAtt.toString();
-
-            } catch (IOException e) {
-              // safe to ignore, because we know the number of tokens
+          
+          if (positionCount == 1) { 
+            // simple case: only one position, with synonyms
+            LinkedList<QueryNode> children = new LinkedList<QueryNode>();
+            
+            for (int i = 0; i < numTokens; i++) {
+              String term = null;
+              try {
+                boolean hasNext = buffer.incrementToken();
+                assert hasNext == true;
+                term = termAtt.toString();
+                
+              } catch (IOException e) {
+                // safe to ignore, because we know the number of tokens
+              }
+              
+              children.add(new FieldQueryNode(field, term, -1, -1));
+              
             }
-
-            children.add(new FieldQueryNode(field, term, -1, -1));
-
+            return new GroupQueryNode(
+                new StandardBooleanQueryNode(children, positionCount==1));
+            
+          } else {
+            // multiple positions
+            QueryNode q = new StandardBooleanQueryNode(Collections.<QueryNode>emptyList(),false);
+            QueryNode currentQuery = null;
+            for (int i = 0; i < numTokens; i++) {
+              String term = null;
+              try {
+                boolean hasNext = buffer.incrementToken();
+                assert hasNext == true;
+                term = termAtt.toString();
+              } catch (IOException e) {
+                // safe to ignore, because we know the number of tokens
+              }
+              if (posIncrAtt != null && posIncrAtt.getPositionIncrement() == 0) {
+                if (!(currentQuery instanceof BooleanQueryNode)) {
+                  QueryNode t = currentQuery;
+                  currentQuery = new StandardBooleanQueryNode(Collections.<QueryNode>emptyList(), true);
+                  ((BooleanQueryNode)currentQuery).add(t);
+                }
+                ((BooleanQueryNode)currentQuery).add(new FieldQueryNode(field, term, -1, -1));
+              } else {
+                if (currentQuery != null) {
+                  if (this.defaultOperator == Operator.OR) {
+                    q.add(currentQuery);
+                  } else {
+                    q.add(new ModifierQueryNode(currentQuery, Modifier.MOD_REQ));
+                  }
+                }
+                currentQuery = new FieldQueryNode(field, term, -1, -1);
+              }
+            }
+            if (this.defaultOperator == Operator.OR) {
+              q.add(currentQuery);
+            } else {
+              q.add(new ModifierQueryNode(currentQuery, Modifier.MOD_REQ));
+            }
+            
+            if (q instanceof BooleanQueryNode) {
+              q = new GroupQueryNode(q);
+            }
+            return q;
           }
-          return new GroupQueryNode(
-            new StandardBooleanQueryNode(children, positionCount==1));
         } else {
           // phrase query:
           MultiPhraseQueryNode mpq = new MultiPhraseQueryNode();

Modified: lucene/dev/branches/lucene_solr_4_6/lucene/queryparser/src/test/org/apache/lucene/queryparser/util/QueryParserTestBase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene_solr_4_6/lucene/queryparser/src/test/org/apache/lucene/queryparser/util/QueryParserTestBase.java?rev=1559318&r1=1559317&r2=1559318&view=diff
==============================================================================
--- lucene/dev/branches/lucene_solr_4_6/lucene/queryparser/src/test/org/apache/lucene/queryparser/util/QueryParserTestBase.java (original)
+++ lucene/dev/branches/lucene_solr_4_6/lucene/queryparser/src/test/org/apache/lucene/queryparser/util/QueryParserTestBase.java Sat Jan 18 05:36:28 2014
@@ -558,6 +558,13 @@ public abstract class QueryParserTestBas
     assertQueryEquals("((stop))", qpAnalyzer, "");
     assertTrue(getQuery("term term term", qpAnalyzer) instanceof BooleanQuery);
     assertTrue(getQuery("term +stop", qpAnalyzer) instanceof TermQuery);
+    
+    CommonQueryParserConfiguration cqpc = getParserConfig(qpAnalyzer);
+    setDefaultOperatorAND(cqpc);
+    assertQueryEquals(cqpc, "field", "term phrase term",
+        "+term +(+phrase1 +phrase2) +term");
+    assertQueryEquals(cqpc, "field", "phrase",
+        "+phrase1 +phrase2");
   }
 
   public void testRange() throws Exception {



Re: svn commit: r1559318 - in /lucene/dev/branches/lucene_solr_4_6/lucene: ./ queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/processors/ queryparser/src/test/org/apache/lucene/queryparser/util/

Posted by Adriano Crestani <ad...@gmail.com>.
Nice. Thanks Mark ;)


On Sun, Jan 19, 2014 at 3:08 PM, Mark Miller <ma...@gmail.com> wrote:

>
> On Jan 18, 2014, at 10:03 PM, Adriano Crestani <ad...@gmail.com>
> wrote:
>
> Thanks for letting me know. So it will be pushed automatically to 4.6.2,
> right?!
>
>
> Right.
>
> All we will need to do is change the CHANGES.txt.
>
>
> Right.
>
> Though it looks like we should re spin 4.6.1, in which case it will be
> included.
>
> - Mark
>
>
>
> On Sat, Jan 18, 2014 at 12:35 PM, Robert Muir <rc...@gmail.com> wrote:
>
>> FYI, I am not sure this commit made it in time for the current 4.6.1
>> release candidate being voted on.
>>
>>
>> On Fri, Jan 17, 2014 at 9:36 PM, <ad...@apache.org> wrote:
>>
>>> Author: adrianocrestani
>>> Date: Sat Jan 18 05:36:28 2014
>>> New Revision: 1559318
>>>
>>> URL: http://svn.apache.org/r1559318
>>> Log:
>>> LUCENE-Flexible StandardQueryParser behaves differently than
>>> ClassicQueryParser (4.6 branch)
>>>
>>> Modified:
>>>     lucene/dev/branches/lucene_solr_4_6/lucene/CHANGES.txt
>>>
>>> lucene/dev/branches/lucene_solr_4_6/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/processors/AnalyzerQueryNodeProcessor.java
>>>
>>> lucene/dev/branches/lucene_solr_4_6/lucene/queryparser/src/test/org/apache/lucene/queryparser/util/QueryParserTestBase.java
>>>
>>> Modified: lucene/dev/branches/lucene_solr_4_6/lucene/CHANGES.txt
>>> URL:
>>> http://svn.apache.org/viewvc/lucene/dev/branches/lucene_solr_4_6/lucene/CHANGES.txt?rev=1559318&r1=1559317&r2=1559318&view=diff
>>>
>>> ==============================================================================
>>> --- lucene/dev/branches/lucene_solr_4_6/lucene/CHANGES.txt (original)
>>> +++ lucene/dev/branches/lucene_solr_4_6/lucene/CHANGES.txt Sat Jan 18
>>> 05:36:28 2014
>>> @@ -22,6 +22,9 @@ Bug fixes
>>>
>>>  * LUCENE-5394: Fix TokenSources.getTokenStream to return payloads if
>>>    they were indexed with the term vectors. (Mike McCandless)
>>> +
>>> +* LUCENE-5344: Flexible StandardQueryParser behaves differently than
>>> +  ClassicQueryParser. (Adriano Crestani)
>>>
>>>  * LUCENE-5375: ToChildBlockJoinQuery works harder to detect mis-use,
>>>    when the parent query incorrectly returns child documents, and throw
>>> @@ -139,7 +142,7 @@ Bug Fixes
>>>  * LUCENE-5342: Fixed bulk-merge issue in CompressingStoredFieldsFormat
>>> which
>>>    created corrupted segments when mixing chunk sizes.
>>>    Lucene41StoredFieldsFormat is not impacted. (Adrien Grand, Robert
>>> Muir)
>>> -
>>> +
>>>  API Changes
>>>
>>>  * LUCENE-5222: Add SortField.needsScores(). Previously it was not
>>> possible
>>>
>>> Modified:
>>> lucene/dev/branches/lucene_solr_4_6/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/processors/AnalyzerQueryNodeProcessor.java
>>> URL:
>>> http://svn.apache.org/viewvc/lucene/dev/branches/lucene_solr_4_6/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/processors/AnalyzerQueryNodeProcessor.java?rev=1559318&r1=1559317&r2=1559318&view=diff
>>>
>>> ==============================================================================
>>> ---
>>> lucene/dev/branches/lucene_solr_4_6/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/processors/AnalyzerQueryNodeProcessor.java
>>> (original)
>>> +++
>>> lucene/dev/branches/lucene_solr_4_6/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/processors/AnalyzerQueryNodeProcessor.java
>>> Sat Jan 18 05:36:28 2014
>>> @@ -19,6 +19,7 @@ package org.apache.lucene.queryparser.fl
>>>
>>>  import java.io.IOException;
>>>  import java.util.ArrayList;
>>> +import java.util.Collections;
>>>  import java.util.LinkedList;
>>>  import java.util.List;
>>>
>>> @@ -27,23 +28,32 @@ import org.apache.lucene.analysis.Cachin
>>>  import org.apache.lucene.analysis.TokenStream;
>>>  import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
>>>  import
>>> org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
>>> +import org.apache.lucene.index.Term;
>>>  import org.apache.lucene.queryparser.flexible.core.QueryNodeException;
>>>  import
>>> org.apache.lucene.queryparser.flexible.core.config.QueryConfigHandler;
>>> +import
>>> org.apache.lucene.queryparser.flexible.core.nodes.BooleanQueryNode;
>>>  import org.apache.lucene.queryparser.flexible.core.nodes.FieldQueryNode;
>>>  import org.apache.lucene.queryparser.flexible.core.nodes.FuzzyQueryNode;
>>>  import org.apache.lucene.queryparser.flexible.core.nodes.GroupQueryNode;
>>> +import
>>> org.apache.lucene.queryparser.flexible.core.nodes.ModifierQueryNode;
>>>  import
>>> org.apache.lucene.queryparser.flexible.core.nodes.NoTokenFoundQueryNode;
>>>  import org.apache.lucene.queryparser.flexible.core.nodes.QueryNode;
>>>  import
>>> org.apache.lucene.queryparser.flexible.core.nodes.QuotedFieldQueryNode;
>>>  import org.apache.lucene.queryparser.flexible.core.nodes.RangeQueryNode;
>>>  import
>>> org.apache.lucene.queryparser.flexible.core.nodes.TextableQueryNode;
>>>  import
>>> org.apache.lucene.queryparser.flexible.core.nodes.TokenizedPhraseQueryNode;
>>> +import
>>> org.apache.lucene.queryparser.flexible.core.nodes.ModifierQueryNode.Modifier;
>>>  import
>>> org.apache.lucene.queryparser.flexible.core.processors.QueryNodeProcessorImpl;
>>>  import
>>> org.apache.lucene.queryparser.flexible.standard.config.StandardQueryConfigHandler.ConfigurationKeys;
>>> +import
>>> org.apache.lucene.queryparser.flexible.standard.config.StandardQueryConfigHandler.Operator;
>>>  import
>>> org.apache.lucene.queryparser.flexible.standard.nodes.MultiPhraseQueryNode;
>>>  import
>>> org.apache.lucene.queryparser.flexible.standard.nodes.RegexpQueryNode;
>>>  import
>>> org.apache.lucene.queryparser.flexible.standard.nodes.StandardBooleanQueryNode;
>>>  import
>>> org.apache.lucene.queryparser.flexible.standard.nodes.WildcardQueryNode;
>>> +import org.apache.lucene.search.BooleanClause;
>>> +import org.apache.lucene.search.BooleanQuery;
>>> +import org.apache.lucene.search.Query;
>>> +import org.apache.lucene.util.BytesRef;
>>>  import org.apache.lucene.util.IOUtils;
>>>
>>>  /**
>>> @@ -73,6 +83,8 @@ public class AnalyzerQueryNodeProcessor
>>>    private Analyzer analyzer;
>>>
>>>    private boolean positionIncrementsEnabled;
>>> +
>>> +  private Operator defaultOperator;
>>>
>>>    public AnalyzerQueryNodeProcessor() {
>>>      // empty constructor
>>> @@ -86,6 +98,8 @@ public class AnalyzerQueryNodeProcessor
>>>        this.analyzer = analyzer;
>>>        this.positionIncrementsEnabled = false;
>>>        Boolean positionIncrementsEnabled =
>>> getQueryConfigHandler().get(ConfigurationKeys.ENABLE_POSITION_INCREMENTS);
>>> +      Operator defaultOperator =
>>> getQueryConfigHandler().get(ConfigurationKeys.DEFAULT_OPERATOR);
>>> +      this.defaultOperator = defaultOperator != null ? defaultOperator
>>> : Operator.OR;
>>>
>>>        if (positionIncrementsEnabled != null) {
>>>            this.positionIncrementsEnabled = positionIncrementsEnabled;
>>> @@ -185,24 +199,70 @@ public class AnalyzerQueryNodeProcessor
>>>        } else if (severalTokensAtSamePosition || !(node instanceof
>>> QuotedFieldQueryNode)) {
>>>          if (positionCount == 1 || !(node instanceof
>>> QuotedFieldQueryNode)) {
>>>            // no phrase query:
>>> -          LinkedList<QueryNode> children = new LinkedList<QueryNode>();
>>> -
>>> -          for (int i = 0; i < numTokens; i++) {
>>> -            String term = null;
>>> -            try {
>>> -              boolean hasNext = buffer.incrementToken();
>>> -              assert hasNext == true;
>>> -              term = termAtt.toString();
>>> -
>>> -            } catch (IOException e) {
>>> -              // safe to ignore, because we know the number of tokens
>>> +
>>> +          if (positionCount == 1) {
>>> +            // simple case: only one position, with synonyms
>>> +            LinkedList<QueryNode> children = new
>>> LinkedList<QueryNode>();
>>> +
>>> +            for (int i = 0; i < numTokens; i++) {
>>> +              String term = null;
>>> +              try {
>>> +                boolean hasNext = buffer.incrementToken();
>>> +                assert hasNext == true;
>>> +                term = termAtt.toString();
>>> +
>>> +              } catch (IOException e) {
>>> +                // safe to ignore, because we know the number of tokens
>>> +              }
>>> +
>>> +              children.add(new FieldQueryNode(field, term, -1, -1));
>>> +
>>>              }
>>> -
>>> -            children.add(new FieldQueryNode(field, term, -1, -1));
>>> -
>>> +            return new GroupQueryNode(
>>> +                new StandardBooleanQueryNode(children,
>>> positionCount==1));
>>> +
>>> +          } else {
>>> +            // multiple positions
>>> +            QueryNode q = new
>>> StandardBooleanQueryNode(Collections.<QueryNode>emptyList(),false);
>>> +            QueryNode currentQuery = null;
>>> +            for (int i = 0; i < numTokens; i++) {
>>> +              String term = null;
>>> +              try {
>>> +                boolean hasNext = buffer.incrementToken();
>>> +                assert hasNext == true;
>>> +                term = termAtt.toString();
>>> +              } catch (IOException e) {
>>> +                // safe to ignore, because we know the number of tokens
>>> +              }
>>> +              if (posIncrAtt != null &&
>>> posIncrAtt.getPositionIncrement() == 0) {
>>> +                if (!(currentQuery instanceof BooleanQueryNode)) {
>>> +                  QueryNode t = currentQuery;
>>> +                  currentQuery = new
>>> StandardBooleanQueryNode(Collections.<QueryNode>emptyList(), true);
>>> +                  ((BooleanQueryNode)currentQuery).add(t);
>>> +                }
>>> +                ((BooleanQueryNode)currentQuery).add(new
>>> FieldQueryNode(field, term, -1, -1));
>>> +              } else {
>>> +                if (currentQuery != null) {
>>> +                  if (this.defaultOperator == Operator.OR) {
>>> +                    q.add(currentQuery);
>>> +                  } else {
>>> +                    q.add(new ModifierQueryNode(currentQuery,
>>> Modifier.MOD_REQ));
>>> +                  }
>>> +                }
>>> +                currentQuery = new FieldQueryNode(field, term, -1, -1);
>>> +              }
>>> +            }
>>> +            if (this.defaultOperator == Operator.OR) {
>>> +              q.add(currentQuery);
>>> +            } else {
>>> +              q.add(new ModifierQueryNode(currentQuery,
>>> Modifier.MOD_REQ));
>>> +            }
>>> +
>>> +            if (q instanceof BooleanQueryNode) {
>>> +              q = new GroupQueryNode(q);
>>> +            }
>>> +            return q;
>>>            }
>>> -          return new GroupQueryNode(
>>> -            new StandardBooleanQueryNode(children, positionCount==1));
>>>          } else {
>>>            // phrase query:
>>>            MultiPhraseQueryNode mpq = new MultiPhraseQueryNode();
>>>
>>> Modified:
>>> lucene/dev/branches/lucene_solr_4_6/lucene/queryparser/src/test/org/apache/lucene/queryparser/util/QueryParserTestBase.java
>>> URL:
>>> http://svn.apache.org/viewvc/lucene/dev/branches/lucene_solr_4_6/lucene/queryparser/src/test/org/apache/lucene/queryparser/util/QueryParserTestBase.java?rev=1559318&r1=1559317&r2=1559318&view=diff
>>>
>>> ==============================================================================
>>> ---
>>> lucene/dev/branches/lucene_solr_4_6/lucene/queryparser/src/test/org/apache/lucene/queryparser/util/QueryParserTestBase.java
>>> (original)
>>> +++
>>> lucene/dev/branches/lucene_solr_4_6/lucene/queryparser/src/test/org/apache/lucene/queryparser/util/QueryParserTestBase.java
>>> Sat Jan 18 05:36:28 2014
>>> @@ -558,6 +558,13 @@ public abstract class QueryParserTestBas
>>>      assertQueryEquals("((stop))", qpAnalyzer, "");
>>>      assertTrue(getQuery("term term term", qpAnalyzer) instanceof
>>> BooleanQuery);
>>>      assertTrue(getQuery("term +stop", qpAnalyzer) instanceof TermQuery);
>>> +
>>> +    CommonQueryParserConfiguration cqpc = getParserConfig(qpAnalyzer);
>>> +    setDefaultOperatorAND(cqpc);
>>> +    assertQueryEquals(cqpc, "field", "term phrase term",
>>> +        "+term +(+phrase1 +phrase2) +term");
>>> +    assertQueryEquals(cqpc, "field", "phrase",
>>> +        "+phrase1 +phrase2");
>>>    }
>>>
>>>    public void testRange() throws Exception {
>>>
>>>
>>>
>>
>
>

Re: svn commit: r1559318 - in /lucene/dev/branches/lucene_solr_4_6/lucene: ./ queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/processors/ queryparser/src/test/org/apache/lucene/queryparser/util/

Posted by Mark Miller <ma...@gmail.com>.
On Jan 18, 2014, at 10:03 PM, Adriano Crestani <ad...@gmail.com> wrote:

> Thanks for letting me know. So it will be pushed automatically to 4.6.2, right?!

Right.

> All we will need to do is change the CHANGES.txt.

Right.

Though it looks like we should re spin 4.6.1, in which case it will be included.

- Mark

> 
> 
> On Sat, Jan 18, 2014 at 12:35 PM, Robert Muir <rc...@gmail.com> wrote:
> FYI, I am not sure this commit made it in time for the current 4.6.1 release candidate being voted on.
> 
> 
> On Fri, Jan 17, 2014 at 9:36 PM, <ad...@apache.org> wrote:
> Author: adrianocrestani
> Date: Sat Jan 18 05:36:28 2014
> New Revision: 1559318
> 
> URL: http://svn.apache.org/r1559318
> Log:
> LUCENE-Flexible StandardQueryParser behaves differently than ClassicQueryParser (4.6 branch)
> 
> Modified:
>     lucene/dev/branches/lucene_solr_4_6/lucene/CHANGES.txt
>     lucene/dev/branches/lucene_solr_4_6/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/processors/AnalyzerQueryNodeProcessor.java
>     lucene/dev/branches/lucene_solr_4_6/lucene/queryparser/src/test/org/apache/lucene/queryparser/util/QueryParserTestBase.java
> 
> Modified: lucene/dev/branches/lucene_solr_4_6/lucene/CHANGES.txt
> URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene_solr_4_6/lucene/CHANGES.txt?rev=1559318&r1=1559317&r2=1559318&view=diff
> ==============================================================================
> --- lucene/dev/branches/lucene_solr_4_6/lucene/CHANGES.txt (original)
> +++ lucene/dev/branches/lucene_solr_4_6/lucene/CHANGES.txt Sat Jan 18 05:36:28 2014
> @@ -22,6 +22,9 @@ Bug fixes
> 
>  * LUCENE-5394: Fix TokenSources.getTokenStream to return payloads if
>    they were indexed with the term vectors. (Mike McCandless)
> +
> +* LUCENE-5344: Flexible StandardQueryParser behaves differently than
> +  ClassicQueryParser. (Adriano Crestani)
> 
>  * LUCENE-5375: ToChildBlockJoinQuery works harder to detect mis-use,
>    when the parent query incorrectly returns child documents, and throw
> @@ -139,7 +142,7 @@ Bug Fixes
>  * LUCENE-5342: Fixed bulk-merge issue in CompressingStoredFieldsFormat which
>    created corrupted segments when mixing chunk sizes.
>    Lucene41StoredFieldsFormat is not impacted. (Adrien Grand, Robert Muir)
> -
> +
>  API Changes
> 
>  * LUCENE-5222: Add SortField.needsScores(). Previously it was not possible
> 
> Modified: lucene/dev/branches/lucene_solr_4_6/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/processors/AnalyzerQueryNodeProcessor.java
> URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene_solr_4_6/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/processors/AnalyzerQueryNodeProcessor.java?rev=1559318&r1=1559317&r2=1559318&view=diff
> ==============================================================================
> --- lucene/dev/branches/lucene_solr_4_6/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/processors/AnalyzerQueryNodeProcessor.java (original)
> +++ lucene/dev/branches/lucene_solr_4_6/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/processors/AnalyzerQueryNodeProcessor.java Sat Jan 18 05:36:28 2014
> @@ -19,6 +19,7 @@ package org.apache.lucene.queryparser.fl
> 
>  import java.io.IOException;
>  import java.util.ArrayList;
> +import java.util.Collections;
>  import java.util.LinkedList;
>  import java.util.List;
> 
> @@ -27,23 +28,32 @@ import org.apache.lucene.analysis.Cachin
>  import org.apache.lucene.analysis.TokenStream;
>  import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
>  import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
> +import org.apache.lucene.index.Term;
>  import org.apache.lucene.queryparser.flexible.core.QueryNodeException;
>  import org.apache.lucene.queryparser.flexible.core.config.QueryConfigHandler;
> +import org.apache.lucene.queryparser.flexible.core.nodes.BooleanQueryNode;
>  import org.apache.lucene.queryparser.flexible.core.nodes.FieldQueryNode;
>  import org.apache.lucene.queryparser.flexible.core.nodes.FuzzyQueryNode;
>  import org.apache.lucene.queryparser.flexible.core.nodes.GroupQueryNode;
> +import org.apache.lucene.queryparser.flexible.core.nodes.ModifierQueryNode;
>  import org.apache.lucene.queryparser.flexible.core.nodes.NoTokenFoundQueryNode;
>  import org.apache.lucene.queryparser.flexible.core.nodes.QueryNode;
>  import org.apache.lucene.queryparser.flexible.core.nodes.QuotedFieldQueryNode;
>  import org.apache.lucene.queryparser.flexible.core.nodes.RangeQueryNode;
>  import org.apache.lucene.queryparser.flexible.core.nodes.TextableQueryNode;
>  import org.apache.lucene.queryparser.flexible.core.nodes.TokenizedPhraseQueryNode;
> +import org.apache.lucene.queryparser.flexible.core.nodes.ModifierQueryNode.Modifier;
>  import org.apache.lucene.queryparser.flexible.core.processors.QueryNodeProcessorImpl;
>  import org.apache.lucene.queryparser.flexible.standard.config.StandardQueryConfigHandler.ConfigurationKeys;
> +import org.apache.lucene.queryparser.flexible.standard.config.StandardQueryConfigHandler.Operator;
>  import org.apache.lucene.queryparser.flexible.standard.nodes.MultiPhraseQueryNode;
>  import org.apache.lucene.queryparser.flexible.standard.nodes.RegexpQueryNode;
>  import org.apache.lucene.queryparser.flexible.standard.nodes.StandardBooleanQueryNode;
>  import org.apache.lucene.queryparser.flexible.standard.nodes.WildcardQueryNode;
> +import org.apache.lucene.search.BooleanClause;
> +import org.apache.lucene.search.BooleanQuery;
> +import org.apache.lucene.search.Query;
> +import org.apache.lucene.util.BytesRef;
>  import org.apache.lucene.util.IOUtils;
> 
>  /**
> @@ -73,6 +83,8 @@ public class AnalyzerQueryNodeProcessor
>    private Analyzer analyzer;
> 
>    private boolean positionIncrementsEnabled;
> +
> +  private Operator defaultOperator;
> 
>    public AnalyzerQueryNodeProcessor() {
>      // empty constructor
> @@ -86,6 +98,8 @@ public class AnalyzerQueryNodeProcessor
>        this.analyzer = analyzer;
>        this.positionIncrementsEnabled = false;
>        Boolean positionIncrementsEnabled = getQueryConfigHandler().get(ConfigurationKeys.ENABLE_POSITION_INCREMENTS);
> +      Operator defaultOperator = getQueryConfigHandler().get(ConfigurationKeys.DEFAULT_OPERATOR);
> +      this.defaultOperator = defaultOperator != null ? defaultOperator : Operator.OR;
> 
>        if (positionIncrementsEnabled != null) {
>            this.positionIncrementsEnabled = positionIncrementsEnabled;
> @@ -185,24 +199,70 @@ public class AnalyzerQueryNodeProcessor
>        } else if (severalTokensAtSamePosition || !(node instanceof QuotedFieldQueryNode)) {
>          if (positionCount == 1 || !(node instanceof QuotedFieldQueryNode)) {
>            // no phrase query:
> -          LinkedList<QueryNode> children = new LinkedList<QueryNode>();
> -
> -          for (int i = 0; i < numTokens; i++) {
> -            String term = null;
> -            try {
> -              boolean hasNext = buffer.incrementToken();
> -              assert hasNext == true;
> -              term = termAtt.toString();
> -
> -            } catch (IOException e) {
> -              // safe to ignore, because we know the number of tokens
> +
> +          if (positionCount == 1) {
> +            // simple case: only one position, with synonyms
> +            LinkedList<QueryNode> children = new LinkedList<QueryNode>();
> +
> +            for (int i = 0; i < numTokens; i++) {
> +              String term = null;
> +              try {
> +                boolean hasNext = buffer.incrementToken();
> +                assert hasNext == true;
> +                term = termAtt.toString();
> +
> +              } catch (IOException e) {
> +                // safe to ignore, because we know the number of tokens
> +              }
> +
> +              children.add(new FieldQueryNode(field, term, -1, -1));
> +
>              }
> -
> -            children.add(new FieldQueryNode(field, term, -1, -1));
> -
> +            return new GroupQueryNode(
> +                new StandardBooleanQueryNode(children, positionCount==1));
> +
> +          } else {
> +            // multiple positions
> +            QueryNode q = new StandardBooleanQueryNode(Collections.<QueryNode>emptyList(),false);
> +            QueryNode currentQuery = null;
> +            for (int i = 0; i < numTokens; i++) {
> +              String term = null;
> +              try {
> +                boolean hasNext = buffer.incrementToken();
> +                assert hasNext == true;
> +                term = termAtt.toString();
> +              } catch (IOException e) {
> +                // safe to ignore, because we know the number of tokens
> +              }
> +              if (posIncrAtt != null && posIncrAtt.getPositionIncrement() == 0) {
> +                if (!(currentQuery instanceof BooleanQueryNode)) {
> +                  QueryNode t = currentQuery;
> +                  currentQuery = new StandardBooleanQueryNode(Collections.<QueryNode>emptyList(), true);
> +                  ((BooleanQueryNode)currentQuery).add(t);
> +                }
> +                ((BooleanQueryNode)currentQuery).add(new FieldQueryNode(field, term, -1, -1));
> +              } else {
> +                if (currentQuery != null) {
> +                  if (this.defaultOperator == Operator.OR) {
> +                    q.add(currentQuery);
> +                  } else {
> +                    q.add(new ModifierQueryNode(currentQuery, Modifier.MOD_REQ));
> +                  }
> +                }
> +                currentQuery = new FieldQueryNode(field, term, -1, -1);
> +              }
> +            }
> +            if (this.defaultOperator == Operator.OR) {
> +              q.add(currentQuery);
> +            } else {
> +              q.add(new ModifierQueryNode(currentQuery, Modifier.MOD_REQ));
> +            }
> +
> +            if (q instanceof BooleanQueryNode) {
> +              q = new GroupQueryNode(q);
> +            }
> +            return q;
>            }
> -          return new GroupQueryNode(
> -            new StandardBooleanQueryNode(children, positionCount==1));
>          } else {
>            // phrase query:
>            MultiPhraseQueryNode mpq = new MultiPhraseQueryNode();
> 
> Modified: lucene/dev/branches/lucene_solr_4_6/lucene/queryparser/src/test/org/apache/lucene/queryparser/util/QueryParserTestBase.java
> URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene_solr_4_6/lucene/queryparser/src/test/org/apache/lucene/queryparser/util/QueryParserTestBase.java?rev=1559318&r1=1559317&r2=1559318&view=diff
> ==============================================================================
> --- lucene/dev/branches/lucene_solr_4_6/lucene/queryparser/src/test/org/apache/lucene/queryparser/util/QueryParserTestBase.java (original)
> +++ lucene/dev/branches/lucene_solr_4_6/lucene/queryparser/src/test/org/apache/lucene/queryparser/util/QueryParserTestBase.java Sat Jan 18 05:36:28 2014
> @@ -558,6 +558,13 @@ public abstract class QueryParserTestBas
>      assertQueryEquals("((stop))", qpAnalyzer, "");
>      assertTrue(getQuery("term term term", qpAnalyzer) instanceof BooleanQuery);
>      assertTrue(getQuery("term +stop", qpAnalyzer) instanceof TermQuery);
> +
> +    CommonQueryParserConfiguration cqpc = getParserConfig(qpAnalyzer);
> +    setDefaultOperatorAND(cqpc);
> +    assertQueryEquals(cqpc, "field", "term phrase term",
> +        "+term +(+phrase1 +phrase2) +term");
> +    assertQueryEquals(cqpc, "field", "phrase",
> +        "+phrase1 +phrase2");
>    }
> 
>    public void testRange() throws Exception {
> 
> 
> 
> 


Re: svn commit: r1559318 - in /lucene/dev/branches/lucene_solr_4_6/lucene: ./ queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/processors/ queryparser/src/test/org/apache/lucene/queryparser/util/

Posted by Adriano Crestani <ad...@gmail.com>.
Thanks for letting me know. So it will be pushed automatically to 4.6.2,
right?! All we will need to do is change the CHANGES.txt.


On Sat, Jan 18, 2014 at 12:35 PM, Robert Muir <rc...@gmail.com> wrote:

> FYI, I am not sure this commit made it in time for the current 4.6.1
> release candidate being voted on.
>
>
> On Fri, Jan 17, 2014 at 9:36 PM, <ad...@apache.org> wrote:
>
>> Author: adrianocrestani
>> Date: Sat Jan 18 05:36:28 2014
>> New Revision: 1559318
>>
>> URL: http://svn.apache.org/r1559318
>> Log:
>> LUCENE-Flexible StandardQueryParser behaves differently than
>> ClassicQueryParser (4.6 branch)
>>
>> Modified:
>>     lucene/dev/branches/lucene_solr_4_6/lucene/CHANGES.txt
>>
>> lucene/dev/branches/lucene_solr_4_6/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/processors/AnalyzerQueryNodeProcessor.java
>>
>> lucene/dev/branches/lucene_solr_4_6/lucene/queryparser/src/test/org/apache/lucene/queryparser/util/QueryParserTestBase.java
>>
>> Modified: lucene/dev/branches/lucene_solr_4_6/lucene/CHANGES.txt
>> URL:
>> http://svn.apache.org/viewvc/lucene/dev/branches/lucene_solr_4_6/lucene/CHANGES.txt?rev=1559318&r1=1559317&r2=1559318&view=diff
>>
>> ==============================================================================
>> --- lucene/dev/branches/lucene_solr_4_6/lucene/CHANGES.txt (original)
>> +++ lucene/dev/branches/lucene_solr_4_6/lucene/CHANGES.txt Sat Jan 18
>> 05:36:28 2014
>> @@ -22,6 +22,9 @@ Bug fixes
>>
>>  * LUCENE-5394: Fix TokenSources.getTokenStream to return payloads if
>>    they were indexed with the term vectors. (Mike McCandless)
>> +
>> +* LUCENE-5344: Flexible StandardQueryParser behaves differently than
>> +  ClassicQueryParser. (Adriano Crestani)
>>
>>  * LUCENE-5375: ToChildBlockJoinQuery works harder to detect mis-use,
>>    when the parent query incorrectly returns child documents, and throw
>> @@ -139,7 +142,7 @@ Bug Fixes
>>  * LUCENE-5342: Fixed bulk-merge issue in CompressingStoredFieldsFormat
>> which
>>    created corrupted segments when mixing chunk sizes.
>>    Lucene41StoredFieldsFormat is not impacted. (Adrien Grand, Robert Muir)
>> -
>> +
>>  API Changes
>>
>>  * LUCENE-5222: Add SortField.needsScores(). Previously it was not
>> possible
>>
>> Modified:
>> lucene/dev/branches/lucene_solr_4_6/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/processors/AnalyzerQueryNodeProcessor.java
>> URL:
>> http://svn.apache.org/viewvc/lucene/dev/branches/lucene_solr_4_6/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/processors/AnalyzerQueryNodeProcessor.java?rev=1559318&r1=1559317&r2=1559318&view=diff
>>
>> ==============================================================================
>> ---
>> lucene/dev/branches/lucene_solr_4_6/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/processors/AnalyzerQueryNodeProcessor.java
>> (original)
>> +++
>> lucene/dev/branches/lucene_solr_4_6/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/processors/AnalyzerQueryNodeProcessor.java
>> Sat Jan 18 05:36:28 2014
>> @@ -19,6 +19,7 @@ package org.apache.lucene.queryparser.fl
>>
>>  import java.io.IOException;
>>  import java.util.ArrayList;
>> +import java.util.Collections;
>>  import java.util.LinkedList;
>>  import java.util.List;
>>
>> @@ -27,23 +28,32 @@ import org.apache.lucene.analysis.Cachin
>>  import org.apache.lucene.analysis.TokenStream;
>>  import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
>>  import
>> org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
>> +import org.apache.lucene.index.Term;
>>  import org.apache.lucene.queryparser.flexible.core.QueryNodeException;
>>  import
>> org.apache.lucene.queryparser.flexible.core.config.QueryConfigHandler;
>> +import
>> org.apache.lucene.queryparser.flexible.core.nodes.BooleanQueryNode;
>>  import org.apache.lucene.queryparser.flexible.core.nodes.FieldQueryNode;
>>  import org.apache.lucene.queryparser.flexible.core.nodes.FuzzyQueryNode;
>>  import org.apache.lucene.queryparser.flexible.core.nodes.GroupQueryNode;
>> +import
>> org.apache.lucene.queryparser.flexible.core.nodes.ModifierQueryNode;
>>  import
>> org.apache.lucene.queryparser.flexible.core.nodes.NoTokenFoundQueryNode;
>>  import org.apache.lucene.queryparser.flexible.core.nodes.QueryNode;
>>  import
>> org.apache.lucene.queryparser.flexible.core.nodes.QuotedFieldQueryNode;
>>  import org.apache.lucene.queryparser.flexible.core.nodes.RangeQueryNode;
>>  import
>> org.apache.lucene.queryparser.flexible.core.nodes.TextableQueryNode;
>>  import
>> org.apache.lucene.queryparser.flexible.core.nodes.TokenizedPhraseQueryNode;
>> +import
>> org.apache.lucene.queryparser.flexible.core.nodes.ModifierQueryNode.Modifier;
>>  import
>> org.apache.lucene.queryparser.flexible.core.processors.QueryNodeProcessorImpl;
>>  import
>> org.apache.lucene.queryparser.flexible.standard.config.StandardQueryConfigHandler.ConfigurationKeys;
>> +import
>> org.apache.lucene.queryparser.flexible.standard.config.StandardQueryConfigHandler.Operator;
>>  import
>> org.apache.lucene.queryparser.flexible.standard.nodes.MultiPhraseQueryNode;
>>  import
>> org.apache.lucene.queryparser.flexible.standard.nodes.RegexpQueryNode;
>>  import
>> org.apache.lucene.queryparser.flexible.standard.nodes.StandardBooleanQueryNode;
>>  import
>> org.apache.lucene.queryparser.flexible.standard.nodes.WildcardQueryNode;
>> +import org.apache.lucene.search.BooleanClause;
>> +import org.apache.lucene.search.BooleanQuery;
>> +import org.apache.lucene.search.Query;
>> +import org.apache.lucene.util.BytesRef;
>>  import org.apache.lucene.util.IOUtils;
>>
>>  /**
>> @@ -73,6 +83,8 @@ public class AnalyzerQueryNodeProcessor
>>    private Analyzer analyzer;
>>
>>    private boolean positionIncrementsEnabled;
>> +
>> +  private Operator defaultOperator;
>>
>>    public AnalyzerQueryNodeProcessor() {
>>      // empty constructor
>> @@ -86,6 +98,8 @@ public class AnalyzerQueryNodeProcessor
>>        this.analyzer = analyzer;
>>        this.positionIncrementsEnabled = false;
>>        Boolean positionIncrementsEnabled =
>> getQueryConfigHandler().get(ConfigurationKeys.ENABLE_POSITION_INCREMENTS);
>> +      Operator defaultOperator =
>> getQueryConfigHandler().get(ConfigurationKeys.DEFAULT_OPERATOR);
>> +      this.defaultOperator = defaultOperator != null ? defaultOperator :
>> Operator.OR;
>>
>>        if (positionIncrementsEnabled != null) {
>>            this.positionIncrementsEnabled = positionIncrementsEnabled;
>> @@ -185,24 +199,70 @@ public class AnalyzerQueryNodeProcessor
>>        } else if (severalTokensAtSamePosition || !(node instanceof
>> QuotedFieldQueryNode)) {
>>          if (positionCount == 1 || !(node instanceof
>> QuotedFieldQueryNode)) {
>>            // no phrase query:
>> -          LinkedList<QueryNode> children = new LinkedList<QueryNode>();
>> -
>> -          for (int i = 0; i < numTokens; i++) {
>> -            String term = null;
>> -            try {
>> -              boolean hasNext = buffer.incrementToken();
>> -              assert hasNext == true;
>> -              term = termAtt.toString();
>> -
>> -            } catch (IOException e) {
>> -              // safe to ignore, because we know the number of tokens
>> +
>> +          if (positionCount == 1) {
>> +            // simple case: only one position, with synonyms
>> +            LinkedList<QueryNode> children = new LinkedList<QueryNode>();
>> +
>> +            for (int i = 0; i < numTokens; i++) {
>> +              String term = null;
>> +              try {
>> +                boolean hasNext = buffer.incrementToken();
>> +                assert hasNext == true;
>> +                term = termAtt.toString();
>> +
>> +              } catch (IOException e) {
>> +                // safe to ignore, because we know the number of tokens
>> +              }
>> +
>> +              children.add(new FieldQueryNode(field, term, -1, -1));
>> +
>>              }
>> -
>> -            children.add(new FieldQueryNode(field, term, -1, -1));
>> -
>> +            return new GroupQueryNode(
>> +                new StandardBooleanQueryNode(children,
>> positionCount==1));
>> +
>> +          } else {
>> +            // multiple positions
>> +            QueryNode q = new
>> StandardBooleanQueryNode(Collections.<QueryNode>emptyList(),false);
>> +            QueryNode currentQuery = null;
>> +            for (int i = 0; i < numTokens; i++) {
>> +              String term = null;
>> +              try {
>> +                boolean hasNext = buffer.incrementToken();
>> +                assert hasNext == true;
>> +                term = termAtt.toString();
>> +              } catch (IOException e) {
>> +                // safe to ignore, because we know the number of tokens
>> +              }
>> +              if (posIncrAtt != null &&
>> posIncrAtt.getPositionIncrement() == 0) {
>> +                if (!(currentQuery instanceof BooleanQueryNode)) {
>> +                  QueryNode t = currentQuery;
>> +                  currentQuery = new
>> StandardBooleanQueryNode(Collections.<QueryNode>emptyList(), true);
>> +                  ((BooleanQueryNode)currentQuery).add(t);
>> +                }
>> +                ((BooleanQueryNode)currentQuery).add(new
>> FieldQueryNode(field, term, -1, -1));
>> +              } else {
>> +                if (currentQuery != null) {
>> +                  if (this.defaultOperator == Operator.OR) {
>> +                    q.add(currentQuery);
>> +                  } else {
>> +                    q.add(new ModifierQueryNode(currentQuery,
>> Modifier.MOD_REQ));
>> +                  }
>> +                }
>> +                currentQuery = new FieldQueryNode(field, term, -1, -1);
>> +              }
>> +            }
>> +            if (this.defaultOperator == Operator.OR) {
>> +              q.add(currentQuery);
>> +            } else {
>> +              q.add(new ModifierQueryNode(currentQuery,
>> Modifier.MOD_REQ));
>> +            }
>> +
>> +            if (q instanceof BooleanQueryNode) {
>> +              q = new GroupQueryNode(q);
>> +            }
>> +            return q;
>>            }
>> -          return new GroupQueryNode(
>> -            new StandardBooleanQueryNode(children, positionCount==1));
>>          } else {
>>            // phrase query:
>>            MultiPhraseQueryNode mpq = new MultiPhraseQueryNode();
>>
>> Modified:
>> lucene/dev/branches/lucene_solr_4_6/lucene/queryparser/src/test/org/apache/lucene/queryparser/util/QueryParserTestBase.java
>> URL:
>> http://svn.apache.org/viewvc/lucene/dev/branches/lucene_solr_4_6/lucene/queryparser/src/test/org/apache/lucene/queryparser/util/QueryParserTestBase.java?rev=1559318&r1=1559317&r2=1559318&view=diff
>>
>> ==============================================================================
>> ---
>> lucene/dev/branches/lucene_solr_4_6/lucene/queryparser/src/test/org/apache/lucene/queryparser/util/QueryParserTestBase.java
>> (original)
>> +++
>> lucene/dev/branches/lucene_solr_4_6/lucene/queryparser/src/test/org/apache/lucene/queryparser/util/QueryParserTestBase.java
>> Sat Jan 18 05:36:28 2014
>> @@ -558,6 +558,13 @@ public abstract class QueryParserTestBas
>>      assertQueryEquals("((stop))", qpAnalyzer, "");
>>      assertTrue(getQuery("term term term", qpAnalyzer) instanceof
>> BooleanQuery);
>>      assertTrue(getQuery("term +stop", qpAnalyzer) instanceof TermQuery);
>> +
>> +    CommonQueryParserConfiguration cqpc = getParserConfig(qpAnalyzer);
>> +    setDefaultOperatorAND(cqpc);
>> +    assertQueryEquals(cqpc, "field", "term phrase term",
>> +        "+term +(+phrase1 +phrase2) +term");
>> +    assertQueryEquals(cqpc, "field", "phrase",
>> +        "+phrase1 +phrase2");
>>    }
>>
>>    public void testRange() throws Exception {
>>
>>
>>
>

Re: svn commit: r1559318 - in /lucene/dev/branches/lucene_solr_4_6/lucene: ./ queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/processors/ queryparser/src/test/org/apache/lucene/queryparser/util/

Posted by Robert Muir <rc...@gmail.com>.
FYI, I am not sure this commit made it in time for the current 4.6.1
release candidate being voted on.


On Fri, Jan 17, 2014 at 9:36 PM, <ad...@apache.org> wrote:

> Author: adrianocrestani
> Date: Sat Jan 18 05:36:28 2014
> New Revision: 1559318
>
> URL: http://svn.apache.org/r1559318
> Log:
> LUCENE-Flexible StandardQueryParser behaves differently than
> ClassicQueryParser (4.6 branch)
>
> Modified:
>     lucene/dev/branches/lucene_solr_4_6/lucene/CHANGES.txt
>
> lucene/dev/branches/lucene_solr_4_6/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/processors/AnalyzerQueryNodeProcessor.java
>
> lucene/dev/branches/lucene_solr_4_6/lucene/queryparser/src/test/org/apache/lucene/queryparser/util/QueryParserTestBase.java
>
> Modified: lucene/dev/branches/lucene_solr_4_6/lucene/CHANGES.txt
> URL:
> http://svn.apache.org/viewvc/lucene/dev/branches/lucene_solr_4_6/lucene/CHANGES.txt?rev=1559318&r1=1559317&r2=1559318&view=diff
>
> ==============================================================================
> --- lucene/dev/branches/lucene_solr_4_6/lucene/CHANGES.txt (original)
> +++ lucene/dev/branches/lucene_solr_4_6/lucene/CHANGES.txt Sat Jan 18
> 05:36:28 2014
> @@ -22,6 +22,9 @@ Bug fixes
>
>  * LUCENE-5394: Fix TokenSources.getTokenStream to return payloads if
>    they were indexed with the term vectors. (Mike McCandless)
> +
> +* LUCENE-5344: Flexible StandardQueryParser behaves differently than
> +  ClassicQueryParser. (Adriano Crestani)
>
>  * LUCENE-5375: ToChildBlockJoinQuery works harder to detect mis-use,
>    when the parent query incorrectly returns child documents, and throw
> @@ -139,7 +142,7 @@ Bug Fixes
>  * LUCENE-5342: Fixed bulk-merge issue in CompressingStoredFieldsFormat
> which
>    created corrupted segments when mixing chunk sizes.
>    Lucene41StoredFieldsFormat is not impacted. (Adrien Grand, Robert Muir)
> -
> +
>  API Changes
>
>  * LUCENE-5222: Add SortField.needsScores(). Previously it was not possible
>
> Modified:
> lucene/dev/branches/lucene_solr_4_6/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/processors/AnalyzerQueryNodeProcessor.java
> URL:
> http://svn.apache.org/viewvc/lucene/dev/branches/lucene_solr_4_6/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/processors/AnalyzerQueryNodeProcessor.java?rev=1559318&r1=1559317&r2=1559318&view=diff
>
> ==============================================================================
> ---
> lucene/dev/branches/lucene_solr_4_6/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/processors/AnalyzerQueryNodeProcessor.java
> (original)
> +++
> lucene/dev/branches/lucene_solr_4_6/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/processors/AnalyzerQueryNodeProcessor.java
> Sat Jan 18 05:36:28 2014
> @@ -19,6 +19,7 @@ package org.apache.lucene.queryparser.fl
>
>  import java.io.IOException;
>  import java.util.ArrayList;
> +import java.util.Collections;
>  import java.util.LinkedList;
>  import java.util.List;
>
> @@ -27,23 +28,32 @@ import org.apache.lucene.analysis.Cachin
>  import org.apache.lucene.analysis.TokenStream;
>  import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
>  import
> org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
> +import org.apache.lucene.index.Term;
>  import org.apache.lucene.queryparser.flexible.core.QueryNodeException;
>  import
> org.apache.lucene.queryparser.flexible.core.config.QueryConfigHandler;
> +import org.apache.lucene.queryparser.flexible.core.nodes.BooleanQueryNode;
>  import org.apache.lucene.queryparser.flexible.core.nodes.FieldQueryNode;
>  import org.apache.lucene.queryparser.flexible.core.nodes.FuzzyQueryNode;
>  import org.apache.lucene.queryparser.flexible.core.nodes.GroupQueryNode;
> +import
> org.apache.lucene.queryparser.flexible.core.nodes.ModifierQueryNode;
>  import
> org.apache.lucene.queryparser.flexible.core.nodes.NoTokenFoundQueryNode;
>  import org.apache.lucene.queryparser.flexible.core.nodes.QueryNode;
>  import
> org.apache.lucene.queryparser.flexible.core.nodes.QuotedFieldQueryNode;
>  import org.apache.lucene.queryparser.flexible.core.nodes.RangeQueryNode;
>  import
> org.apache.lucene.queryparser.flexible.core.nodes.TextableQueryNode;
>  import
> org.apache.lucene.queryparser.flexible.core.nodes.TokenizedPhraseQueryNode;
> +import
> org.apache.lucene.queryparser.flexible.core.nodes.ModifierQueryNode.Modifier;
>  import
> org.apache.lucene.queryparser.flexible.core.processors.QueryNodeProcessorImpl;
>  import
> org.apache.lucene.queryparser.flexible.standard.config.StandardQueryConfigHandler.ConfigurationKeys;
> +import
> org.apache.lucene.queryparser.flexible.standard.config.StandardQueryConfigHandler.Operator;
>  import
> org.apache.lucene.queryparser.flexible.standard.nodes.MultiPhraseQueryNode;
>  import
> org.apache.lucene.queryparser.flexible.standard.nodes.RegexpQueryNode;
>  import
> org.apache.lucene.queryparser.flexible.standard.nodes.StandardBooleanQueryNode;
>  import
> org.apache.lucene.queryparser.flexible.standard.nodes.WildcardQueryNode;
> +import org.apache.lucene.search.BooleanClause;
> +import org.apache.lucene.search.BooleanQuery;
> +import org.apache.lucene.search.Query;
> +import org.apache.lucene.util.BytesRef;
>  import org.apache.lucene.util.IOUtils;
>
>  /**
> @@ -73,6 +83,8 @@ public class AnalyzerQueryNodeProcessor
>    private Analyzer analyzer;
>
>    private boolean positionIncrementsEnabled;
> +
> +  private Operator defaultOperator;
>
>    public AnalyzerQueryNodeProcessor() {
>      // empty constructor
> @@ -86,6 +98,8 @@ public class AnalyzerQueryNodeProcessor
>        this.analyzer = analyzer;
>        this.positionIncrementsEnabled = false;
>        Boolean positionIncrementsEnabled =
> getQueryConfigHandler().get(ConfigurationKeys.ENABLE_POSITION_INCREMENTS);
> +      Operator defaultOperator =
> getQueryConfigHandler().get(ConfigurationKeys.DEFAULT_OPERATOR);
> +      this.defaultOperator = defaultOperator != null ? defaultOperator :
> Operator.OR;
>
>        if (positionIncrementsEnabled != null) {
>            this.positionIncrementsEnabled = positionIncrementsEnabled;
> @@ -185,24 +199,70 @@ public class AnalyzerQueryNodeProcessor
>        } else if (severalTokensAtSamePosition || !(node instanceof
> QuotedFieldQueryNode)) {
>          if (positionCount == 1 || !(node instanceof
> QuotedFieldQueryNode)) {
>            // no phrase query:
> -          LinkedList<QueryNode> children = new LinkedList<QueryNode>();
> -
> -          for (int i = 0; i < numTokens; i++) {
> -            String term = null;
> -            try {
> -              boolean hasNext = buffer.incrementToken();
> -              assert hasNext == true;
> -              term = termAtt.toString();
> -
> -            } catch (IOException e) {
> -              // safe to ignore, because we know the number of tokens
> +
> +          if (positionCount == 1) {
> +            // simple case: only one position, with synonyms
> +            LinkedList<QueryNode> children = new LinkedList<QueryNode>();
> +
> +            for (int i = 0; i < numTokens; i++) {
> +              String term = null;
> +              try {
> +                boolean hasNext = buffer.incrementToken();
> +                assert hasNext == true;
> +                term = termAtt.toString();
> +
> +              } catch (IOException e) {
> +                // safe to ignore, because we know the number of tokens
> +              }
> +
> +              children.add(new FieldQueryNode(field, term, -1, -1));
> +
>              }
> -
> -            children.add(new FieldQueryNode(field, term, -1, -1));
> -
> +            return new GroupQueryNode(
> +                new StandardBooleanQueryNode(children, positionCount==1));
> +
> +          } else {
> +            // multiple positions
> +            QueryNode q = new
> StandardBooleanQueryNode(Collections.<QueryNode>emptyList(),false);
> +            QueryNode currentQuery = null;
> +            for (int i = 0; i < numTokens; i++) {
> +              String term = null;
> +              try {
> +                boolean hasNext = buffer.incrementToken();
> +                assert hasNext == true;
> +                term = termAtt.toString();
> +              } catch (IOException e) {
> +                // safe to ignore, because we know the number of tokens
> +              }
> +              if (posIncrAtt != null && posIncrAtt.getPositionIncrement()
> == 0) {
> +                if (!(currentQuery instanceof BooleanQueryNode)) {
> +                  QueryNode t = currentQuery;
> +                  currentQuery = new
> StandardBooleanQueryNode(Collections.<QueryNode>emptyList(), true);
> +                  ((BooleanQueryNode)currentQuery).add(t);
> +                }
> +                ((BooleanQueryNode)currentQuery).add(new
> FieldQueryNode(field, term, -1, -1));
> +              } else {
> +                if (currentQuery != null) {
> +                  if (this.defaultOperator == Operator.OR) {
> +                    q.add(currentQuery);
> +                  } else {
> +                    q.add(new ModifierQueryNode(currentQuery,
> Modifier.MOD_REQ));
> +                  }
> +                }
> +                currentQuery = new FieldQueryNode(field, term, -1, -1);
> +              }
> +            }
> +            if (this.defaultOperator == Operator.OR) {
> +              q.add(currentQuery);
> +            } else {
> +              q.add(new ModifierQueryNode(currentQuery,
> Modifier.MOD_REQ));
> +            }
> +
> +            if (q instanceof BooleanQueryNode) {
> +              q = new GroupQueryNode(q);
> +            }
> +            return q;
>            }
> -          return new GroupQueryNode(
> -            new StandardBooleanQueryNode(children, positionCount==1));
>          } else {
>            // phrase query:
>            MultiPhraseQueryNode mpq = new MultiPhraseQueryNode();
>
> Modified:
> lucene/dev/branches/lucene_solr_4_6/lucene/queryparser/src/test/org/apache/lucene/queryparser/util/QueryParserTestBase.java
> URL:
> http://svn.apache.org/viewvc/lucene/dev/branches/lucene_solr_4_6/lucene/queryparser/src/test/org/apache/lucene/queryparser/util/QueryParserTestBase.java?rev=1559318&r1=1559317&r2=1559318&view=diff
>
> ==============================================================================
> ---
> lucene/dev/branches/lucene_solr_4_6/lucene/queryparser/src/test/org/apache/lucene/queryparser/util/QueryParserTestBase.java
> (original)
> +++
> lucene/dev/branches/lucene_solr_4_6/lucene/queryparser/src/test/org/apache/lucene/queryparser/util/QueryParserTestBase.java
> Sat Jan 18 05:36:28 2014
> @@ -558,6 +558,13 @@ public abstract class QueryParserTestBas
>      assertQueryEquals("((stop))", qpAnalyzer, "");
>      assertTrue(getQuery("term term term", qpAnalyzer) instanceof
> BooleanQuery);
>      assertTrue(getQuery("term +stop", qpAnalyzer) instanceof TermQuery);
> +
> +    CommonQueryParserConfiguration cqpc = getParserConfig(qpAnalyzer);
> +    setDefaultOperatorAND(cqpc);
> +    assertQueryEquals(cqpc, "field", "term phrase term",
> +        "+term +(+phrase1 +phrase2) +term");
> +    assertQueryEquals(cqpc, "field", "phrase",
> +        "+phrase1 +phrase2");
>    }
>
>    public void testRange() throws Exception {
>
>
>