You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by rm...@apache.org on 2012/07/29 16:38:38 UTC

svn commit: r1366854 [1/2] - in /lucene/dev/trunk/lucene: ./ queryparser/src/java/org/apache/lucene/queryparser/classic/ queryparser/src/java/org/apache/lucene/queryparser/flexible/precedence/processors/ queryparser/src/java/org/apache/lucene/querypars...

Author: rmuir
Date: Sun Jul 29 14:38:37 2012
New Revision: 1366854

URL: http://svn.apache.org/viewvc?rev=1366854&view=rev
Log:
LUCENE-4109: BooleanQueries are not parsed correctly with the flexible queryparser

Added:
    lucene/dev/trunk/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/CommonQueryParserConfiguration.java   (with props)
    lucene/dev/trunk/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/processors/BooleanQuery2ModifierNodeProcessor.java   (with props)
    lucene/dev/trunk/lucene/queryparser/src/test/org/apache/lucene/queryparser/flexible/standard/TestStandardQP.java   (with props)
Modified:
    lucene/dev/trunk/lucene/CHANGES.txt
    lucene/dev/trunk/lucene/queryparser/src/java/org/apache/lucene/queryparser/classic/QueryParserBase.java
    lucene/dev/trunk/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/precedence/processors/PrecedenceQueryNodeProcessorPipeline.java
    lucene/dev/trunk/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/StandardQueryParser.java
    lucene/dev/trunk/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/processors/GroupQueryNodeProcessor.java
    lucene/dev/trunk/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/processors/MultiFieldQueryNodeProcessor.java
    lucene/dev/trunk/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/processors/StandardQueryNodeProcessorPipeline.java
    lucene/dev/trunk/lucene/queryparser/src/test/org/apache/lucene/queryparser/classic/TestQueryParser.java
    lucene/dev/trunk/lucene/queryparser/src/test/org/apache/lucene/queryparser/ext/TestExtendableQueryParser.java
    lucene/dev/trunk/lucene/queryparser/src/test/org/apache/lucene/queryparser/flexible/standard/TestMultiAnalyzerQPHelper.java
    lucene/dev/trunk/lucene/queryparser/src/test/org/apache/lucene/queryparser/flexible/standard/TestMultiFieldQPHelper.java
    lucene/dev/trunk/lucene/queryparser/src/test/org/apache/lucene/queryparser/flexible/standard/TestQPHelper.java
    lucene/dev/trunk/lucene/queryparser/src/test/org/apache/lucene/queryparser/util/QueryParserTestBase.java

Modified: lucene/dev/trunk/lucene/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/CHANGES.txt?rev=1366854&r1=1366853&r2=1366854&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/CHANGES.txt (original)
+++ lucene/dev/trunk/lucene/CHANGES.txt Sun Jul 29 14:38:37 2012
@@ -82,6 +82,9 @@ Optimizations
 
 Bug Fixes
 
+* LUCENE-4109: BooleanQueries are not parsed correctly with the 
+  flexible query parser. (Karsten Rauch via Robert Muir)
+
 * LUCENE-4176: Fix AnalyzingQueryParser to analyze range endpoints as bytes,
   so that it works correctly with Analyzers that produce binary non-UTF-8 terms
   such as CollationAnalyzer. (Nattapong Sirilappanich via Robert Muir) 
@@ -113,6 +116,11 @@ Bug Fixes
 * LUCENE-4245: Make IndexWriter#close() and MergeScheduler#close()
   non-interruptible.  (Mark Miller, Uwe Schindler)
 
+Changes in Runtime Behavior
+
+* LUCENE-4109: Enable position increments in the flexible queryparser by default.
+  (Karsten Rauch via Robert Muir)
+
 Build
 
 * LUCENE-4094: Support overriding file.encoding on forked test JVMs

Modified: lucene/dev/trunk/lucene/queryparser/src/java/org/apache/lucene/queryparser/classic/QueryParserBase.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/queryparser/src/java/org/apache/lucene/queryparser/classic/QueryParserBase.java?rev=1366854&r1=1366853&r2=1366854&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/queryparser/src/java/org/apache/lucene/queryparser/classic/QueryParserBase.java (original)
+++ lucene/dev/trunk/lucene/queryparser/src/java/org/apache/lucene/queryparser/classic/QueryParserBase.java Sun Jul 29 14:38:37 2012
@@ -30,6 +30,7 @@ import org.apache.lucene.analysis.tokena
 import org.apache.lucene.document.DateTools;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.queryparser.classic.QueryParser.Operator;
+import org.apache.lucene.queryparser.flexible.standard.CommonQueryParserConfiguration;
 import org.apache.lucene.search.*;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.Version;
@@ -37,7 +38,7 @@ import org.apache.lucene.util.Version;
 /** This class is overridden by QueryParser in QueryParser.jj
  * and acts to separate the majority of the Java code from the .jj grammar file. 
  */
-public abstract class QueryParserBase {
+public abstract class QueryParserBase implements CommonQueryParserConfiguration {
 
   /** Do not catch this exception in your code, it means you are using methods that you should no longer use. */
   public static class MethodRemovedUseAnother extends Throwable {}

Modified: lucene/dev/trunk/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/precedence/processors/PrecedenceQueryNodeProcessorPipeline.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/precedence/processors/PrecedenceQueryNodeProcessorPipeline.java?rev=1366854&r1=1366853&r2=1366854&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/precedence/processors/PrecedenceQueryNodeProcessorPipeline.java (original)
+++ lucene/dev/trunk/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/precedence/processors/PrecedenceQueryNodeProcessorPipeline.java Sun Jul 29 14:38:37 2012
@@ -18,6 +18,7 @@ package org.apache.lucene.queryparser.fl
  */
 
 import org.apache.lucene.queryparser.flexible.precedence.PrecedenceQueryParser;
+import org.apache.lucene.queryparser.flexible.standard.processors.BooleanQuery2ModifierNodeProcessor;
 import org.apache.lucene.queryparser.flexible.standard.processors.GroupQueryNodeProcessor;
 import org.apache.lucene.queryparser.flexible.standard.processors.StandardQueryNodeProcessorPipeline;
 import org.apache.lucene.queryparser.flexible.core.config.QueryConfigHandler;
@@ -46,7 +47,7 @@ public class PrecedenceQueryNodeProcesso
     
     for (int i = 0 ; i < size() ; i++) {
       
-      if (get(i).getClass().equals(GroupQueryNodeProcessor.class)) {
+      if (get(i).getClass().equals(BooleanQuery2ModifierNodeProcessor.class)) {
         remove(i--);
       }
       

Added: lucene/dev/trunk/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/CommonQueryParserConfiguration.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/CommonQueryParserConfiguration.java?rev=1366854&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/CommonQueryParserConfiguration.java (added)
+++ lucene/dev/trunk/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/CommonQueryParserConfiguration.java Sun Jul 29 14:38:37 2012
@@ -0,0 +1,161 @@
+package org.apache.lucene.queryparser.flexible.standard;
+
+import java.util.Locale;
+import java.util.TimeZone;
+import java.util.TooManyListenersException;
+
+import org.apache.lucene.analysis.Analyzer;
+import org.apache.lucene.document.DateTools;
+import org.apache.lucene.document.DateTools.Resolution;
+import org.apache.lucene.search.FuzzyQuery;
+import org.apache.lucene.search.MultiTermQuery;
+
+/*
+ * 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.
+ */
+
+public interface CommonQueryParserConfiguration {
+  
+  /**
+   * Set to <code>true</code> to allow leading wildcard characters.
+   * <p>
+   * When set, <code>*</code> or <code>?</code> are allowed as the first
+   * character of a PrefixQuery and WildcardQuery. Note that this can produce
+   * very slow queries on big indexes.
+   * <p>
+   * Default: false.
+   */
+  public void setLowercaseExpandedTerms(boolean lowercaseExpandedTerms);
+  
+  /**
+   * @see #setLowercaseExpandedTerms(boolean)
+   */
+  public boolean getLowercaseExpandedTerms();
+  
+  /**
+   * Set to <code>true</code> to allow leading wildcard characters.
+   * <p>
+   * When set, <code>*</code> or <code>?</code> are allowed as the first
+   * character of a PrefixQuery and WildcardQuery. Note that this can produce
+   * very slow queries on big indexes.
+   * <p>
+   * Default: false.
+   */
+  public void setAllowLeadingWildcard(boolean allowLeadingWildcard);
+  
+  /**
+   * Set to <code>true</code> to enable position increments in result query.
+   * <p>
+   * When set, result phrase and multi-phrase queries will be aware of position
+   * increments. Useful when e.g. a StopFilter increases the position increment
+   * of the token that follows an omitted token.
+   * <p>
+   * Default: false.
+   */
+  public void setEnablePositionIncrements(boolean enabled);
+  
+  /**
+   * @see #setEnablePositionIncrements(boolean)
+   */
+  public boolean getEnablePositionIncrements();
+  
+  /**
+   * By default, it uses
+   * {@link MultiTermQuery#CONSTANT_SCORE_AUTO_REWRITE_DEFAULT} when creating a
+   * prefix, wildcard and range queries. This implementation is generally
+   * preferable because it a) Runs faster b) Does not have the scarcity of terms
+   * unduly influence score c) avoids any {@link TooManyListenersException}
+   * exception. However, if your application really needs to use the
+   * old-fashioned boolean queries expansion rewriting and the above points are
+   * not relevant then use this change the rewrite method.
+   */
+  public void setMultiTermRewriteMethod(MultiTermQuery.RewriteMethod method);
+  
+  /**
+   * @see #setMultiTermRewriteMethod(org.apache.lucene.search.MultiTermQuery.RewriteMethod)
+   */
+  public MultiTermQuery.RewriteMethod getMultiTermRewriteMethod();
+  
+  
+  /**
+   * Set the prefix length for fuzzy queries. Default is 0.
+   * 
+   * @param fuzzyPrefixLength
+   *          The fuzzyPrefixLength to set.
+   */
+  public void setFuzzyPrefixLength(int fuzzyPrefixLength);
+  
+  /**
+   * Set locale used by date range parsing.
+   */
+  public void setLocale(Locale locale);
+  
+  /**
+   * Returns current locale, allowing access by subclasses.
+   */
+  public Locale getLocale();
+  
+  public void setTimeZone(TimeZone timeZone);
+  
+  public TimeZone getTimeZone();
+  
+  /**
+   * Sets the default slop for phrases. If zero, then exact phrase matches are
+   * required. Default value is zero.
+   */
+  public void setPhraseSlop(int defaultPhraseSlop);
+  
+  public Analyzer getAnalyzer();
+  
+  /**
+   * @see #setAllowLeadingWildcard(boolean)
+   */
+  public boolean getAllowLeadingWildcard();
+  
+  /**
+   * Get the minimal similarity for fuzzy queries.
+   */
+  public float getFuzzyMinSim();
+  
+  /**
+   * Get the prefix length for fuzzy queries.
+   * 
+   * @return Returns the fuzzyPrefixLength.
+   */
+  public int getFuzzyPrefixLength();
+  
+  /**
+   * Gets the default slop for phrases.
+   */
+  public int getPhraseSlop();
+  
+  /**
+   * Set the minimum similarity for fuzzy queries. Default is defined on
+   * {@link FuzzyQuery#defaultMinSimilarity}.
+   */
+  public void setFuzzyMinSim(float fuzzyMinSim);
+  
+  /**
+   * Sets the default {@link Resolution} used for certain field when
+   * no {@link Resolution} is defined for this field.
+   * 
+   * @param dateResolution the default {@link Resolution}
+   */
+  public void setDateResolution(DateTools.Resolution dateResolution);
+  
+  
+  
+}
\ No newline at end of file

Modified: lucene/dev/trunk/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/StandardQueryParser.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/StandardQueryParser.java?rev=1366854&r1=1366853&r2=1366854&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/StandardQueryParser.java (original)
+++ lucene/dev/trunk/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/StandardQueryParser.java Sun Jul 29 14:38:37 2012
@@ -110,7 +110,7 @@ import org.apache.lucene.search.Query;
  * @see StandardQueryNodeProcessorPipeline
  * @see StandardQueryTreeBuilder
  */
-public class StandardQueryParser extends QueryParserHelper {
+public class StandardQueryParser extends QueryParserHelper implements CommonQueryParserConfiguration {
   
   /**
    * Constructs a {@link StandardQueryParser} object.
@@ -119,6 +119,7 @@ public class StandardQueryParser extends
     super(new StandardQueryConfigHandler(), new StandardSyntaxParser(),
         new StandardQueryNodeProcessorPipeline(null),
         new StandardQueryTreeBuilder());
+    setEnablePositionIncrements(true);
   }
   
   /**

Added: lucene/dev/trunk/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/processors/BooleanQuery2ModifierNodeProcessor.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/processors/BooleanQuery2ModifierNodeProcessor.java?rev=1366854&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/processors/BooleanQuery2ModifierNodeProcessor.java (added)
+++ lucene/dev/trunk/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/processors/BooleanQuery2ModifierNodeProcessor.java Sun Jul 29 14:38:37 2012
@@ -0,0 +1,202 @@
+package org.apache.lucene.queryparser.flexible.standard.processors;
+
+/**
+ * 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.
+ */
+
+import java.util.ArrayList;
+import java.util.List;
+
+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.AndQueryNode;
+import org.apache.lucene.queryparser.flexible.core.nodes.BooleanQueryNode;
+import org.apache.lucene.queryparser.flexible.core.nodes.ModifierQueryNode;
+import org.apache.lucene.queryparser.flexible.core.nodes.ModifierQueryNode.Modifier;
+import org.apache.lucene.queryparser.flexible.core.nodes.QueryNode;
+import org.apache.lucene.queryparser.flexible.core.processors.QueryNodeProcessor;
+import org.apache.lucene.queryparser.flexible.precedence.processors.BooleanModifiersQueryNodeProcessor;
+import org.apache.lucene.queryparser.flexible.standard.config.StandardQueryConfigHandler;
+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.BooleanModifierNode;
+import org.apache.lucene.queryparser.flexible.standard.parser.StandardSyntaxParser;
+
+/**
+ * <p>
+ * This processor is used to apply the correct {@link ModifierQueryNode} to
+ * {@link BooleanQueryNode}s children. This is a variant of
+ * {@link BooleanModifiersQueryNodeProcessor} which ignores precedence.
+ * </p>
+ * <p>
+ * The {@link StandardSyntaxParser} knows the rules of precedence, but lucene
+ * does not. e.g. <code>(A AND B OR C AND D)</code> ist treated like
+ * <code>(+A +B +C +D)</code>.
+ * </p>
+ * <p>
+ * This processor walks through the query node tree looking for
+ * {@link BooleanQueryNode}s. If an {@link AndQueryNode} is found, every child,
+ * which is not a {@link ModifierQueryNode} or the {@link ModifierQueryNode} is
+ * {@link Modifier#MOD_NONE}, becomes a {@link Modifier#MOD_REQ}. For default
+ * {@link BooleanQueryNode}, it checks the default operator is
+ * {@link Operator#AND}, if it is, the same operation when an
+ * {@link AndQueryNode} is found is applied to it. Each {@link BooleanQueryNode}
+ * which direct parent is also a {@link BooleanQueryNode} is removed (to ignore
+ * the rules of precidence).
+ * </p>
+ * 
+ * @see ConfigurationKeys#DEFAULT_OPERATOR
+ * @see BooleanModifiersQueryNodeProcessor
+ */
+public class BooleanQuery2ModifierNodeProcessor implements QueryNodeProcessor {
+  final static String TAG_REMOVE = "remove";
+  final static String TAG_MODIFIER = "wrapWithModifier";
+  final static String TAG_BOOLEAN_ROOT = "booleanRoot";
+  
+  QueryConfigHandler queryConfigHandler;
+  
+  private final ArrayList<QueryNode> childrenBuffer = new ArrayList<QueryNode>();
+  
+  private Boolean usingAnd = false;
+  
+  public BooleanQuery2ModifierNodeProcessor() {
+    // empty constructor
+  }
+  
+  @Override
+  public QueryNode process(QueryNode queryTree) throws QueryNodeException {
+    Operator op = getQueryConfigHandler().get(
+        ConfigurationKeys.DEFAULT_OPERATOR);
+    
+    if (op == null) {
+      throw new IllegalArgumentException(
+          "StandardQueryConfigHandler.ConfigurationKeys.DEFAULT_OPERATOR should be set on the QueryConfigHandler");
+    }
+    
+    this.usingAnd = StandardQueryConfigHandler.Operator.AND == op;
+    
+    return processIteration(queryTree);
+    
+  }
+  
+  protected void processChildren(QueryNode queryTree) throws QueryNodeException {
+    List<QueryNode> children = queryTree.getChildren();
+    if (children != null && children.size() > 0) {
+      for (QueryNode child : children) {
+        child = processIteration(child);
+      }
+    }
+  }
+  
+  private QueryNode processIteration(QueryNode queryTree)
+      throws QueryNodeException {
+    queryTree = preProcessNode(queryTree);
+    
+    processChildren(queryTree);
+    
+    queryTree = postProcessNode(queryTree);
+    
+    return queryTree;
+    
+  }
+  
+  protected void fillChildrenBufferAndApplyModifiery(QueryNode parent) {
+    for (QueryNode node : parent.getChildren()) {
+      if (node.containsTag(TAG_REMOVE)) {
+        fillChildrenBufferAndApplyModifiery(node);
+      } else if (node.containsTag(TAG_MODIFIER)) {
+        childrenBuffer.add(applyModifier(node,
+            (Modifier) node.getTag(TAG_MODIFIER)));
+      } else {
+        childrenBuffer.add(node);
+      }
+    }
+  }
+  
+  protected QueryNode postProcessNode(QueryNode node) throws QueryNodeException {
+    if (node.containsTag(TAG_BOOLEAN_ROOT)) {
+      this.childrenBuffer.clear();
+      fillChildrenBufferAndApplyModifiery(node);
+      node.set(childrenBuffer);
+    }
+    return node;
+    
+  }
+  
+  protected QueryNode preProcessNode(QueryNode node) throws QueryNodeException {
+    QueryNode parent = node.getParent();
+    if (node instanceof BooleanQueryNode) {
+      if (parent instanceof BooleanQueryNode) {
+        node.setTag(TAG_REMOVE, Boolean.TRUE); // no precedence
+      } else {
+        node.setTag(TAG_BOOLEAN_ROOT, Boolean.TRUE);
+      }
+    } else if (parent instanceof BooleanQueryNode) {
+      if ((parent instanceof AndQueryNode)
+          || (usingAnd && isDefaultBooleanQueryNode(parent))) {
+        tagModifierButDoNotOverride(node, ModifierQueryNode.Modifier.MOD_REQ);
+      }
+    }
+    return node;
+  }
+  
+  protected boolean isDefaultBooleanQueryNode(QueryNode toTest) {
+    return toTest != null && BooleanQueryNode.class.equals(toTest.getClass());
+  }
+  
+  private QueryNode applyModifier(QueryNode node, Modifier mod) {
+    
+    // check if modifier is not already defined and is default
+    if (!(node instanceof ModifierQueryNode)) {
+      return new BooleanModifierNode(node, mod);
+      
+    } else {
+      ModifierQueryNode modNode = (ModifierQueryNode) node;
+      
+      if (modNode.getModifier() == Modifier.MOD_NONE) {
+        return new ModifierQueryNode(modNode.getChild(), mod);
+      }
+      
+    }
+    
+    return node;
+    
+  }
+  
+  protected void tagModifierButDoNotOverride(QueryNode node, Modifier mod) {
+    if (node instanceof ModifierQueryNode) {
+      ModifierQueryNode modNode = (ModifierQueryNode) node;
+      if (modNode.getModifier() == Modifier.MOD_NONE) {
+        node.setTag(TAG_MODIFIER, mod);
+      }
+    } else {
+      node.setTag(TAG_MODIFIER, ModifierQueryNode.Modifier.MOD_REQ);
+    }
+  }
+  
+  @Override
+  public void setQueryConfigHandler(QueryConfigHandler queryConfigHandler) {
+    this.queryConfigHandler = queryConfigHandler;
+    
+  }
+  
+  @Override
+  public QueryConfigHandler getQueryConfigHandler() {
+    return queryConfigHandler;
+  }
+  
+}
+

Modified: lucene/dev/trunk/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/processors/GroupQueryNodeProcessor.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/processors/GroupQueryNodeProcessor.java?rev=1366854&r1=1366853&r2=1366854&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/processors/GroupQueryNodeProcessor.java (original)
+++ lucene/dev/trunk/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/processors/GroupQueryNodeProcessor.java Sun Jul 29 14:38:37 2012
@@ -49,6 +49,7 @@ import org.apache.lucene.queryparser.fle
  * Example: TODO: describe a good example to show how this processor works
  * 
  * @see org.apache.lucene.queryparser.flexible.standard.config.StandardQueryConfigHandler
+ * @deprecated use {@link BooleanQuery2ModifierNodeProcessor} instead
  */
 public class GroupQueryNodeProcessor implements QueryNodeProcessor {
 

Modified: lucene/dev/trunk/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/processors/MultiFieldQueryNodeProcessor.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/processors/MultiFieldQueryNodeProcessor.java?rev=1366854&r1=1366853&r2=1366854&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/processors/MultiFieldQueryNodeProcessor.java (original)
+++ lucene/dev/trunk/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/processors/MultiFieldQueryNodeProcessor.java Sun Jul 29 14:38:37 2012
@@ -25,6 +25,7 @@ import org.apache.lucene.queryparser.fle
 import org.apache.lucene.queryparser.flexible.core.nodes.BooleanQueryNode;
 import org.apache.lucene.queryparser.flexible.core.nodes.FieldableNode;
 import org.apache.lucene.queryparser.flexible.core.nodes.GroupQueryNode;
+import org.apache.lucene.queryparser.flexible.core.nodes.OrQueryNode;
 import org.apache.lucene.queryparser.flexible.core.nodes.QueryNode;
 import org.apache.lucene.queryparser.flexible.core.processors.QueryNodeProcessorImpl;
 import org.apache.lucene.queryparser.flexible.standard.config.StandardQueryConfigHandler.ConfigurationKeys;
@@ -108,7 +109,7 @@ public class MultiFieldQueryNodeProcesso
 
             }
 
-            return new GroupQueryNode(new BooleanQueryNode(children));
+            return new GroupQueryNode(new OrQueryNode(children));
 
           }
 

Modified: lucene/dev/trunk/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/processors/StandardQueryNodeProcessorPipeline.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/processors/StandardQueryNodeProcessorPipeline.java?rev=1366854&r1=1366853&r2=1366854&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/processors/StandardQueryNodeProcessorPipeline.java (original)
+++ lucene/dev/trunk/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/processors/StandardQueryNodeProcessorPipeline.java Sun Jul 29 14:38:37 2012
@@ -63,7 +63,8 @@ public class StandardQueryNodeProcessorP
     add(new AllowLeadingWildcardProcessor());    
     add(new AnalyzerQueryNodeProcessor());
     add(new PhraseSlopQueryNodeProcessor());
-    add(new GroupQueryNodeProcessor());
+    //add(new GroupQueryNodeProcessor());
+    add(new BooleanQuery2ModifierNodeProcessor());
     add(new NoChildOptimizationQueryNodeProcessor());
     add(new RemoveDeletedQueryNodesProcessor());
     add(new RemoveEmptyNonLeafQueryNodeProcessor());

Modified: lucene/dev/trunk/lucene/queryparser/src/test/org/apache/lucene/queryparser/classic/TestQueryParser.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/queryparser/src/test/org/apache/lucene/queryparser/classic/TestQueryParser.java?rev=1366854&r1=1366853&r2=1366854&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/queryparser/src/test/org/apache/lucene/queryparser/classic/TestQueryParser.java (original)
+++ lucene/dev/trunk/lucene/queryparser/src/test/org/apache/lucene/queryparser/classic/TestQueryParser.java Sun Jul 29 14:38:37 2012
@@ -20,19 +20,269 @@ package org.apache.lucene.queryparser.cl
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.analysis.MockTokenizer;
+import org.apache.lucene.document.DateTools.Resolution;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.queryparser.classic.QueryParser.Operator;
+import org.apache.lucene.queryparser.flexible.standard.CommonQueryParserConfiguration;
 import org.apache.lucene.queryparser.util.QueryParserTestBase;
+import org.apache.lucene.search.BooleanClause;
+import org.apache.lucene.search.BooleanQuery;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.TermQuery;
 
 /**
  * Tests QueryParser.
  */
 public class TestQueryParser extends QueryParserTestBase {
   
-  @Override
+  public static class QPTestParser extends QueryParser {
+    public QPTestParser(String f, Analyzer a) {
+      super(TEST_VERSION_CURRENT, f, a);
+    }
+    
+    @Override
+    protected Query getFuzzyQuery(String field, String termStr,
+        float minSimilarity) throws ParseException {
+      throw new ParseException("Fuzzy queries not allowed");
+    }
+    
+    @Override
+    protected Query getWildcardQuery(String field, String termStr)
+        throws ParseException {
+      throw new ParseException("Wildcard queries not allowed");
+    }
+  }
+  
   public QueryParser getParser(Analyzer a) throws Exception {
-    if (a == null)
-      a = new MockAnalyzer(random(), MockTokenizer.SIMPLE, true);
-    QueryParser qp = new QueryParser(TEST_VERSION_CURRENT, "field", a);
+    if (a == null) a = new MockAnalyzer(random(), MockTokenizer.SIMPLE, true);
+    QueryParser qp = new QueryParser(TEST_VERSION_CURRENT, getDefaultField(), a);
     qp.setDefaultOperator(QueryParserBase.OR_OPERATOR);
     return qp;
   }
+  
+  @Override
+  public CommonQueryParserConfiguration getParserConfig(Analyzer a)
+      throws Exception {
+    return getParser(a);
+  }
+  
+  @Override
+  public Query getQuery(String query, CommonQueryParserConfiguration cqpC)
+      throws Exception {
+    assert cqpC != null : "Parameter must not be null";
+    assert (cqpC instanceof QueryParser) : "Parameter must be instance of QueryParser";
+    QueryParser qp = (QueryParser) cqpC;
+    return qp.parse(query);
+  }
+  
+  @Override
+  public Query getQuery(String query, Analyzer a) throws Exception {
+    return getParser(a).parse(query);
+  }
+  
+  @Override
+  public boolean isQueryParserException(Exception exception) {
+    return exception instanceof ParseException;
+  }
+  
+  @Override
+  public void setDefaultOperatorOR(CommonQueryParserConfiguration cqpC) {
+    assert (cqpC instanceof QueryParser);
+    QueryParser qp = (QueryParser) cqpC;
+    qp.setDefaultOperator(Operator.OR);
+  }
+  
+  @Override
+  public void setDefaultOperatorAND(CommonQueryParserConfiguration cqpC) {
+    assert (cqpC instanceof QueryParser);
+    QueryParser qp = (QueryParser) cqpC;
+    qp.setDefaultOperator(Operator.AND);
+  }
+  
+  @Override
+  public void setAnalyzeRangeTerms(CommonQueryParserConfiguration cqpC,
+      boolean value) {
+    assert (cqpC instanceof QueryParser);
+    QueryParser qp = (QueryParser) cqpC;
+    qp.setAnalyzeRangeTerms(value);
+  }
+  
+  @Override
+  public void setAutoGeneratePhraseQueries(CommonQueryParserConfiguration cqpC,
+      boolean value) {
+    assert (cqpC instanceof QueryParser);
+    QueryParser qp = (QueryParser) cqpC;
+    qp.setAutoGeneratePhraseQueries(value);
+  }
+  
+  @Override
+  public void setDateResolution(CommonQueryParserConfiguration cqpC,
+      CharSequence field, Resolution value) {
+    assert (cqpC instanceof QueryParser);
+    QueryParser qp = (QueryParser) cqpC;
+    qp.setDateResolution(field.toString(), value);
+  }
+  
+  @Override
+  public void testDefaultOperator() throws Exception {
+    QueryParser qp = getParser(new MockAnalyzer(random()));
+    // make sure OR is the default:
+    assertEquals(QueryParserBase.OR_OPERATOR, qp.getDefaultOperator());
+    setDefaultOperatorAND(qp);
+    assertEquals(QueryParserBase.AND_OPERATOR, qp.getDefaultOperator());
+    setDefaultOperatorOR(qp);
+    assertEquals(QueryParserBase.OR_OPERATOR, qp.getDefaultOperator());
+  }
+  
+  // LUCENE-2002: when we run javacc to regen QueryParser,
+  // we also run a replaceregexp step to fix 2 of the public
+  // ctors (change them to protected):
+  //
+  // protected QueryParser(CharStream stream)
+  //
+  // protected QueryParser(QueryParserTokenManager tm)
+  //
+  // This test is here as a safety, in case that ant step
+  // doesn't work for some reason.
+  public void testProtectedCtors() throws Exception {
+    try {
+      QueryParser.class.getConstructor(new Class[] {CharStream.class});
+      fail("please switch public QueryParser(CharStream) to be protected");
+    } catch (NoSuchMethodException nsme) {
+      // expected
+    }
+    try {
+      QueryParser.class
+          .getConstructor(new Class[] {QueryParserTokenManager.class});
+      fail("please switch public QueryParser(QueryParserTokenManager) to be protected");
+    } catch (NoSuchMethodException nsme) {
+      // expected
+    }
+  }
+  
+  @Override
+  public void testStarParsing() throws Exception {
+    final int[] type = new int[1];
+    QueryParser qp = new QueryParser(TEST_VERSION_CURRENT, "field",
+        new MockAnalyzer(random(), MockTokenizer.WHITESPACE, false)) {
+      @Override
+      protected Query getWildcardQuery(String field, String termStr) {
+        // override error checking of superclass
+        type[0] = 1;
+        return new TermQuery(new Term(field, termStr));
+      }
+      
+      @Override
+      protected Query getPrefixQuery(String field, String termStr) {
+        // override error checking of superclass
+        type[0] = 2;
+        return new TermQuery(new Term(field, termStr));
+      }
+      
+      @Override
+      protected Query getFieldQuery(String field, String queryText,
+          boolean quoted) throws ParseException {
+        type[0] = 3;
+        return super.getFieldQuery(field, queryText, quoted);
+      }
+    };
+    
+    TermQuery tq;
+    
+    tq = (TermQuery) qp.parse("foo:zoo*");
+    assertEquals("zoo", tq.getTerm().text());
+    assertEquals(2, type[0]);
+    
+    tq = (TermQuery) qp.parse("foo:zoo*^2");
+    assertEquals("zoo", tq.getTerm().text());
+    assertEquals(2, type[0]);
+    assertEquals(tq.getBoost(), 2, 0);
+    
+    tq = (TermQuery) qp.parse("foo:*");
+    assertEquals("*", tq.getTerm().text());
+    assertEquals(1, type[0]); // could be a valid prefix query in the future too
+    
+    tq = (TermQuery) qp.parse("foo:*^2");
+    assertEquals("*", tq.getTerm().text());
+    assertEquals(1, type[0]);
+    assertEquals(tq.getBoost(), 2, 0);
+    
+    tq = (TermQuery) qp.parse("*:foo");
+    assertEquals("*", tq.getTerm().field());
+    assertEquals("foo", tq.getTerm().text());
+    assertEquals(3, type[0]);
+    
+    tq = (TermQuery) qp.parse("*:*");
+    assertEquals("*", tq.getTerm().field());
+    assertEquals("*", tq.getTerm().text());
+    assertEquals(1, type[0]); // could be handled as a prefix query in the
+                              // future
+    
+    tq = (TermQuery) qp.parse("(*:*)");
+    assertEquals("*", tq.getTerm().field());
+    assertEquals("*", tq.getTerm().text());
+    assertEquals(1, type[0]);
+    
+  }
+  
+  public void testCustomQueryParserWildcard() {
+    try {
+      new QPTestParser("contents", new MockAnalyzer(random(),
+          MockTokenizer.WHITESPACE, false)).parse("a?t");
+      fail("Wildcard queries should not be allowed");
+    } catch (ParseException expected) {
+      // expected exception
+    }
+  }
+  
+  public void testCustomQueryParserFuzzy() throws Exception {
+    try {
+      new QPTestParser("contents", new MockAnalyzer(random(),
+          MockTokenizer.WHITESPACE, false)).parse("xunit~");
+      fail("Fuzzy queries should not be allowed");
+    } catch (ParseException expected) {
+      // expected exception
+    }
+  }
+  
+  /** query parser that doesn't expand synonyms when users use double quotes */
+  private class SmartQueryParser extends QueryParser {
+    Analyzer morePrecise = new Analyzer2();
+    
+    public SmartQueryParser() {
+      super(TEST_VERSION_CURRENT, "field", new Analyzer1());
+    }
+    
+    @Override
+    protected Query getFieldQuery(String field, String queryText, boolean quoted)
+        throws ParseException {
+      if (quoted) return newFieldQuery(morePrecise, field, queryText, quoted);
+      else return super.getFieldQuery(field, queryText, quoted);
+    }
+  }
+  
+  @Override
+  public void testNewFieldQuery() throws Exception {
+    /** ordinary behavior, synonyms form uncoordinated boolean query */
+    QueryParser dumb = new QueryParser(TEST_VERSION_CURRENT, "field",
+        new Analyzer1());
+    BooleanQuery expanded = new BooleanQuery(true);
+    expanded.add(new TermQuery(new Term("field", "dogs")),
+        BooleanClause.Occur.SHOULD);
+    expanded.add(new TermQuery(new Term("field", "dog")),
+        BooleanClause.Occur.SHOULD);
+    assertEquals(expanded, dumb.parse("\"dogs\""));
+    /** even with the phrase operator the behavior is the same */
+    assertEquals(expanded, dumb.parse("dogs"));
+    
+    /**
+     * custom behavior, the synonyms are expanded, unless you use quote operator
+     */
+    QueryParser smart = new SmartQueryParser();
+    assertEquals(expanded, smart.parse("dogs"));
+    
+    Query unexpanded = new TermQuery(new Term("field", "dogs"));
+    assertEquals(unexpanded, smart.parse("\"dogs\""));
+  }
+  
 }

Modified: lucene/dev/trunk/lucene/queryparser/src/test/org/apache/lucene/queryparser/ext/TestExtendableQueryParser.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/queryparser/src/test/org/apache/lucene/queryparser/ext/TestExtendableQueryParser.java?rev=1366854&r1=1366853&r2=1366854&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/queryparser/src/test/org/apache/lucene/queryparser/ext/TestExtendableQueryParser.java (original)
+++ lucene/dev/trunk/lucene/queryparser/src/test/org/apache/lucene/queryparser/ext/TestExtendableQueryParser.java Sun Jul 29 14:38:37 2012
@@ -25,7 +25,7 @@ import org.apache.lucene.analysis.MockTo
 import org.apache.lucene.queryparser.classic.ParseException;
 import org.apache.lucene.queryparser.classic.QueryParser;
 import org.apache.lucene.queryparser.classic.QueryParserBase;
-import org.apache.lucene.queryparser.util.QueryParserTestBase;
+import org.apache.lucene.queryparser.classic.TestQueryParser;
 import org.apache.lucene.search.BooleanClause;
 import org.apache.lucene.search.BooleanQuery;
 import org.apache.lucene.search.Query;
@@ -34,7 +34,7 @@ import org.apache.lucene.search.TermQuer
 /**
  * Testcase for the class {@link ExtendableQueryParser}
  */
-public class TestExtendableQueryParser extends QueryParserTestBase {
+public class TestExtendableQueryParser extends TestQueryParser {
   private static char[] DELIMITERS = new char[] {
       Extensions.DEFAULT_EXTENSION_FIELD_DELIMITER, '-', '|' };
 
@@ -48,8 +48,8 @@ public class TestExtendableQueryParser e
     if (a == null)
       a = new MockAnalyzer(random(), MockTokenizer.SIMPLE, true);
     QueryParser qp = extensions == null ? new ExtendableQueryParser(
-        TEST_VERSION_CURRENT, "field", a) : new ExtendableQueryParser(
-        TEST_VERSION_CURRENT, "field", a, extensions);
+        TEST_VERSION_CURRENT, getDefaultField(), a) : new ExtendableQueryParser(
+        TEST_VERSION_CURRENT, getDefaultField(), a, extensions);
     qp.setDefaultOperator(QueryParserBase.OR_OPERATOR);
     return qp;
   }
@@ -92,7 +92,7 @@ public class TestExtendableQueryParser e
       assertTrue("expected instance of TermQuery but was " + query.getClass(),
           query instanceof TermQuery);
       tquery = (TermQuery) query;
-      assertEquals("field", tquery.getTerm().field());
+      assertEquals(getDefaultField(), tquery.getTerm().field());
       assertEquals("bar", tquery.getTerm().text());
     }
   }
@@ -108,7 +108,7 @@ public class TestExtendableQueryParser e
       assertTrue("expected instance of TermQuery but was " + parse.getClass(),
           parse instanceof TermQuery);
       TermQuery tquery = (TermQuery) parse;
-      assertEquals("field", tquery.getTerm().field());
+      assertEquals(getDefaultField(), tquery.getTerm().field());
       assertEquals("foo & bar", tquery.getTerm().text());
     }
   }

Modified: lucene/dev/trunk/lucene/queryparser/src/test/org/apache/lucene/queryparser/flexible/standard/TestMultiAnalyzerQPHelper.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/queryparser/src/test/org/apache/lucene/queryparser/flexible/standard/TestMultiAnalyzerQPHelper.java?rev=1366854&r1=1366853&r2=1366854&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/queryparser/src/test/org/apache/lucene/queryparser/flexible/standard/TestMultiAnalyzerQPHelper.java (original)
+++ lucene/dev/trunk/lucene/queryparser/src/test/org/apache/lucene/queryparser/flexible/standard/TestMultiAnalyzerQPHelper.java Sun Jul 29 14:38:37 2012
@@ -131,11 +131,11 @@ public class TestMultiAnalyzerQPHelper e
     qp.setAnalyzer(new PosIncrementAnalyzer());
 
     assertEquals("quick brown", qp.parse("the quick brown", "").toString());
-    assertEquals("\"quick brown\"", qp.parse("\"the quick brown\"", "")
+    assertEquals("\"? quick brown\"", qp.parse("\"the quick brown\"", "")
         .toString());
     assertEquals("quick brown fox", qp.parse("the quick brown fox", "")
         .toString());
-    assertEquals("\"quick brown fox\"", qp.parse("\"the quick brown fox\"", "")
+    assertEquals("\"? quick brown fox\"", qp.parse("\"the quick brown fox\"", "")
         .toString());
   }
 

Modified: lucene/dev/trunk/lucene/queryparser/src/test/org/apache/lucene/queryparser/flexible/standard/TestMultiFieldQPHelper.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/queryparser/src/test/org/apache/lucene/queryparser/flexible/standard/TestMultiFieldQPHelper.java?rev=1366854&r1=1366853&r2=1366854&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/queryparser/src/test/org/apache/lucene/queryparser/flexible/standard/TestMultiFieldQPHelper.java (original)
+++ lucene/dev/trunk/lucene/queryparser/src/test/org/apache/lucene/queryparser/flexible/standard/TestMultiFieldQPHelper.java Sun Jul 29 14:38:37 2012
@@ -183,15 +183,15 @@ public class TestMultiFieldQPHelper exte
 
     String[] queries2 = { "+one", "+two" };
     q = QueryParserUtil.parse(queries2, fields, new MockAnalyzer(random()));
-    assertEquals("(+b:one) (+t:two)", q.toString());
+    assertEquals("b:one t:two", q.toString());
 
     String[] queries3 = { "one", "+two" };
     q = QueryParserUtil.parse(queries3, fields, new MockAnalyzer(random()));
-    assertEquals("b:one (+t:two)", q.toString());
+    assertEquals("b:one t:two", q.toString());
 
     String[] queries4 = { "one +more", "+two" };
     q = QueryParserUtil.parse(queries4, fields, new MockAnalyzer(random()));
-    assertEquals("(b:one +b:more) (+t:two)", q.toString());
+    assertEquals("(b:one +b:more) t:two", q.toString());
 
     String[] queries5 = { "blah" };
     try {

Modified: lucene/dev/trunk/lucene/queryparser/src/test/org/apache/lucene/queryparser/flexible/standard/TestQPHelper.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/queryparser/src/test/org/apache/lucene/queryparser/flexible/standard/TestQPHelper.java?rev=1366854&r1=1366853&r2=1366854&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/queryparser/src/test/org/apache/lucene/queryparser/flexible/standard/TestQPHelper.java (original)
+++ lucene/dev/trunk/lucene/queryparser/src/test/org/apache/lucene/queryparser/flexible/standard/TestQPHelper.java Sun Jul 29 14:38:37 2012
@@ -491,6 +491,15 @@ public class TestQPHelper extends Lucene
     assertQueryEquals(".NET", a, ".NET");
   }
 
+  public void testGroup() throws Exception {
+    assertQueryEquals("!(a AND b) OR c", null, "-(+a +b) c");
+    assertQueryEquals("!(a AND b) AND c", null, "-(+a +b) +c");
+    assertQueryEquals("((a AND b) AND c)", null, "+(+a +b) +c");
+    assertQueryEquals("(a AND b) AND c", null, "+(+a +b) +c");
+    assertQueryEquals("b !(a AND b)", null, "b -(+a +b)");
+    assertQueryEquals("(a AND b)^4 OR c", null, "((+a +b)^4.0) c");
+  }
+
   public void testSlop() throws Exception {
 
     assertQueryEquals("\"term germ\"~2", null, "\"term germ\"~2");
@@ -1313,8 +1322,8 @@ public class TestQPHelper extends Lucene
     parser.setAnalyzer(new MockAnalyzer(random()));
 
     BooleanQuery exp = new BooleanQuery();
-    exp.add(new BooleanClause(new RegexpQuery(new Term("b", "ab.+")), BooleanClause.Occur.MUST));
-    exp.add(new BooleanClause(new RegexpQuery(new Term("t", "ab.+")), BooleanClause.Occur.MUST));
+    exp.add(new BooleanClause(new RegexpQuery(new Term("b", "ab.+")), BooleanClause.Occur.SHOULD));//TODO spezification? was "MUST"
+    exp.add(new BooleanClause(new RegexpQuery(new Term("t", "ab.+")), BooleanClause.Occur.SHOULD));//TODO spezification? was "MUST"
 
     assertEquals(exp, parser.parse("/ab.+/", null));
 

Added: lucene/dev/trunk/lucene/queryparser/src/test/org/apache/lucene/queryparser/flexible/standard/TestStandardQP.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/queryparser/src/test/org/apache/lucene/queryparser/flexible/standard/TestStandardQP.java?rev=1366854&view=auto
==============================================================================
--- lucene/dev/trunk/lucene/queryparser/src/test/org/apache/lucene/queryparser/flexible/standard/TestStandardQP.java (added)
+++ lucene/dev/trunk/lucene/queryparser/src/test/org/apache/lucene/queryparser/flexible/standard/TestStandardQP.java Sun Jul 29 14:38:37 2012
@@ -0,0 +1,214 @@
+package org.apache.lucene.queryparser.flexible.standard;
+
+/*
+ * 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.
+ */
+
+import java.io.Reader;
+
+import org.apache.lucene.analysis.Analyzer;
+import org.apache.lucene.analysis.MockAnalyzer;
+import org.apache.lucene.analysis.MockTokenizer;
+import org.apache.lucene.document.DateTools.Resolution;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.queryparser.flexible.core.QueryNodeException;
+import org.apache.lucene.queryparser.flexible.standard.config.StandardQueryConfigHandler;
+import org.apache.lucene.queryparser.flexible.standard.config.StandardQueryConfigHandler.Operator;
+import org.apache.lucene.queryparser.util.QueryParserTestBase;
+import org.apache.lucene.search.BooleanClause;
+import org.apache.lucene.search.BooleanQuery;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.TermQuery;
+import org.apache.lucene.search.WildcardQuery;
+
+/**
+ * Tests QueryParser.
+ */
+public class TestStandardQP extends QueryParserTestBase {
+  
+  public StandardQueryParser getParser(Analyzer a) throws Exception {
+    if (a == null) a = new MockAnalyzer(random(), MockTokenizer.SIMPLE, true);
+    StandardQueryParser qp = new StandardQueryParser(a);
+    qp.setDefaultOperator(Operator.OR);
+
+    return qp;
+  }
+  
+  public Query parse(String query, StandardQueryParser qp) throws Exception {
+    return qp.parse(query, getDefaultField());
+  }
+  
+  @Override
+  public CommonQueryParserConfiguration getParserConfig(Analyzer a)
+      throws Exception {
+    return getParser(a);
+  }
+  
+  @Override
+  public Query getQuery(String query, CommonQueryParserConfiguration cqpC)
+      throws Exception {
+    assert cqpC != null : "Parameter must not be null";
+    assert (cqpC instanceof StandardQueryParser) : "Parameter must be instance of StandardQueryParser";
+    StandardQueryParser qp = (StandardQueryParser) cqpC;
+    return parse(query, qp);
+  }
+  
+  @Override
+  public Query getQuery(String query, Analyzer a) throws Exception {
+    return parse(query, getParser(a));
+  }
+  
+  @Override
+  public boolean isQueryParserException(Exception exception) {
+    return exception instanceof QueryNodeException;
+  }
+
+  @Override
+  public void setDefaultOperatorOR(CommonQueryParserConfiguration cqpC) {
+    assert (cqpC instanceof StandardQueryParser);
+    StandardQueryParser qp = (StandardQueryParser) cqpC;
+    qp.setDefaultOperator(Operator.OR);
+  }
+  
+  @Override
+  public void setDefaultOperatorAND(CommonQueryParserConfiguration cqpC) {
+    assert (cqpC instanceof StandardQueryParser);
+    StandardQueryParser qp = (StandardQueryParser) cqpC;
+    qp.setDefaultOperator(Operator.AND);
+  }
+  
+  @Override
+  public void setAnalyzeRangeTerms(CommonQueryParserConfiguration cqpC,
+      boolean value) {
+    throw new UnsupportedOperationException();
+  }
+  
+  @Override
+  public void setAutoGeneratePhraseQueries(CommonQueryParserConfiguration cqpC,
+      boolean value) {
+    throw new UnsupportedOperationException();
+  }
+  
+  @Override
+  public void setDateResolution(CommonQueryParserConfiguration cqpC,
+      CharSequence field, Resolution value) {
+    assert (cqpC instanceof StandardQueryParser);
+    StandardQueryParser qp = (StandardQueryParser) cqpC;
+    qp.getDateResolutionMap().put(field, value);
+  }
+  
+
+  
+  @Override
+  public void testOperatorVsWhitespace() throws Exception {
+    // LUCENE-2566 is not implemented for StandardQueryParser
+    // TODO implement LUCENE-2566 and remove this (override)method
+    Analyzer a = new Analyzer() {
+      @Override
+      public TokenStreamComponents createComponents(String fieldName,
+          Reader reader) {
+        return new TokenStreamComponents(new MockTokenizer(reader,
+            MockTokenizer.WHITESPACE, false));
+      }
+    };
+    assertQueryEquals("a - b", a, "a -b");
+    assertQueryEquals("a + b", a, "a +b");
+    assertQueryEquals("a ! b", a, "a -b");
+  }
+  
+  @Override
+  public void testRangeWithPhrase() throws Exception {
+    // StandardSyntaxParser does not differentiate between a term and a
+    // one-term-phrase in a range query.
+    // Is this an issue? Should StandardSyntaxParser mark the text as
+    // wasEscaped=true ?
+    assertQueryEquals("[\\* TO \"*\"]", null, "[\\* TO *]");
+  }
+  
+  @Override
+  public void testEscapedVsQuestionMarkAsWildcard() throws Exception {
+    Analyzer a = new MockAnalyzer(random(), MockTokenizer.WHITESPACE, false);
+    assertQueryEquals("a:b\\-?c", a, "a:b-?c");
+    assertQueryEquals("a:b\\+?c", a, "a:b+?c");
+    assertQueryEquals("a:b\\:?c", a, "a:b:?c");
+    
+    assertQueryEquals("a:b\\\\?c", a, "a:b\\?c");
+  }
+  
+  @Override
+  public void testEscapedWildcard() throws Exception {
+    CommonQueryParserConfiguration qp = getParserConfig( new MockAnalyzer(random(), MockTokenizer.WHITESPACE, false));
+    WildcardQuery q = new WildcardQuery(new Term("field", "foo?ba?r"));//TODO not correct!!
+    assertEquals(q, getQuery("foo\\?ba?r", qp));
+  }
+
+  
+  @Override
+  public void testCollatedRange() throws Exception {
+    try {
+      setAnalyzeRangeTerms(getParser(null), true);
+      super.testCollatedRange();
+    } catch (UnsupportedOperationException e) {
+      // expected
+    }
+  }
+  
+  @Override
+  public void testAutoGeneratePhraseQueriesOn() throws Exception {
+    try {
+      setAutoGeneratePhraseQueries(getParser(null), true);
+      super.testAutoGeneratePhraseQueriesOn();
+    } catch (UnsupportedOperationException e) {
+      // expected
+    }
+  }
+  
+  @Override
+  public void testStarParsing() throws Exception {
+  }
+    
+  @Override
+  public void testDefaultOperator() throws Exception {
+    StandardQueryParser qp = getParser(new MockAnalyzer(random()));
+    // make sure OR is the default:
+    assertEquals(StandardQueryConfigHandler.Operator.OR, qp.getDefaultOperator());
+    setDefaultOperatorAND(qp);
+    assertEquals(StandardQueryConfigHandler.Operator.AND, qp.getDefaultOperator());
+    setDefaultOperatorOR(qp);
+    assertEquals(StandardQueryConfigHandler.Operator.OR, qp.getDefaultOperator());
+  }
+  
+ 
+  @Override
+  public void testNewFieldQuery() throws Exception {
+    /** ordinary behavior, synonyms form uncoordinated boolean query */
+    StandardQueryParser dumb = getParser(new Analyzer1());
+    BooleanQuery expanded = new BooleanQuery(true);
+    expanded.add(new TermQuery(new Term("field", "dogs")),
+        BooleanClause.Occur.SHOULD);
+    expanded.add(new TermQuery(new Term("field", "dog")),
+        BooleanClause.Occur.SHOULD);
+    assertEquals(expanded, dumb.parse("\"dogs\"","field"));
+    /** even with the phrase operator the behavior is the same */
+    assertEquals(expanded, dumb.parse("dogs","field"));
+    
+    /**
+     * custom behavior, the synonyms are expanded, unless you use quote operator
+     */
+    //TODO test something like "SmartQueryParser()"
+  }
+
+}