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 2011/08/03 02:57:57 UTC

svn commit: r1153315 [3/3] - in /lucene/dev/branches/branch_3x/lucene/contrib: ./ queryparser/src/java/ queryparser/src/java/org/apache/lucene/queryParser/core/config/ queryparser/src/java/org/apache/lucene/queryParser/precedence/processors/ queryparse...

Modified: lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/standard/processors/ParametricRangeQueryNodeProcessor.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/standard/processors/ParametricRangeQueryNodeProcessor.java?rev=1153315&r1=1153314&r2=1153315&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/standard/processors/ParametricRangeQueryNodeProcessor.java (original)
+++ lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/standard/processors/ParametricRangeQueryNodeProcessor.java Wed Aug  3 00:57:50 2011
@@ -35,9 +35,7 @@ import org.apache.lucene.queryParser.cor
 import org.apache.lucene.queryParser.core.nodes.QueryNode;
 import org.apache.lucene.queryParser.core.nodes.ParametricQueryNode.CompareOperator;
 import org.apache.lucene.queryParser.core.processors.QueryNodeProcessorImpl;
-import org.apache.lucene.queryParser.standard.config.DateResolutionAttribute;
-import org.apache.lucene.queryParser.standard.config.LocaleAttribute;
-import org.apache.lucene.queryParser.standard.config.RangeCollatorAttribute;
+import org.apache.lucene.queryParser.standard.config.StandardQueryConfigHandler.ConfigurationKeys;
 import org.apache.lucene.queryParser.standard.nodes.RangeQueryNode;
 
 /**
@@ -45,24 +43,19 @@ import org.apache.lucene.queryParser.sta
  * {@link RangeQueryNode} objects. It reads the lower and upper bounds value
  * from the {@link ParametricRangeQueryNode} object and try to parse their
  * values using a {@link DateFormat}. If the values cannot be parsed to a date
- * value, it will only create the {@link RangeQueryNode} using the non-parsed
- * values. <br/>
+ * value, it will only create the {@link RangeQueryNode} using the
+ * non-parsed values. <br/>
  * <br/>
- * If a {@link LocaleAttribute} is defined in the {@link QueryConfigHandler} it
+ * If a {@link ConfigurationKeys#LOCALE} is defined in the {@link QueryConfigHandler} it
  * will be used to parse the date, otherwise {@link Locale#getDefault()} will be
  * used. <br/>
  * <br/>
- * If a {@link DateResolutionAttribute} is defined and the {@link Resolution} is
+ * If a {@link ConfigurationKeys#DATE_RESOLUTION} is defined and the {@link Resolution} is
  * not <code>null</code> it will also be used to parse the date value. <br/>
  * <br/>
- * This processor will also try to retrieve a {@link RangeCollatorAttribute}
- * from the {@link QueryConfigHandler}. If a {@link RangeCollatorAttribute} is
- * found and the {@link Collator} is not <code>null</code>, it's set on the
- * {@link RangeQueryNode}. <br/>
  * 
- * @see RangeCollatorAttribute
- * @see DateResolutionAttribute
- * @see LocaleAttribute
+ * @see ConfigurationKeys#DATE_RESOLUTION
+ * @see ConfigurationKeys#LOCALE
  * @see RangeQueryNode
  * @see ParametricRangeQueryNode
  */
@@ -79,25 +72,16 @@ public class ParametricRangeQueryNodePro
       ParametricRangeQueryNode parametricRangeNode = (ParametricRangeQueryNode) node;
       ParametricQueryNode upper = parametricRangeNode.getUpperBound();
       ParametricQueryNode lower = parametricRangeNode.getLowerBound();
-      Locale locale = Locale.getDefault();
-      Collator collator = null;
       DateTools.Resolution dateRes = null;
       boolean inclusive = false;
-
-      if (getQueryConfigHandler().hasAttribute(RangeCollatorAttribute.class)) {
-
-        collator = getQueryConfigHandler().getAttribute(
-            RangeCollatorAttribute.class).getRangeCollator();
-
-      }
-
-      if (getQueryConfigHandler().hasAttribute(LocaleAttribute.class)) {
-
-        locale = getQueryConfigHandler().getAttribute(LocaleAttribute.class)
-            .getLocale();
-
-      }
-
+      
+      Locale locale = getQueryConfigHandler().get(ConfigurationKeys.LOCALE,
+          Locale.getDefault());
+      
+      Collator collator = getQueryConfigHandler().get(
+          ConfigurationKeys.RANGE_COLLATOR);
+      
+      
       CharSequence field = parametricRangeNode.getField();
       String fieldStr = null;
 
@@ -109,14 +93,7 @@ public class ParametricRangeQueryNodePro
           .getFieldConfig(fieldStr);
 
       if (fieldConfig != null) {
-
-        if (fieldConfig.hasAttribute(DateResolutionAttribute.class)) {
-
-          dateRes = fieldConfig.getAttribute(DateResolutionAttribute.class)
-              .getDateResolution();
-
-        }
-
+        dateRes = fieldConfig.get(ConfigurationKeys.DATE_RESOLUTION);
       }
 
       if (upper.getOperator() == CompareOperator.LE) {

Modified: lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/overview.html
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/overview.html?rev=1153315&r1=1153314&r2=1153315&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/overview.html (original)
+++ lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/overview.html Wed Aug  3 00:57:50 2011
@@ -108,8 +108,7 @@ builder that will transform the QueryNod
 </dl>
 
 <p>
-Furthermore, the query parser uses flexible configuration objects, which
-are based on AttributeSource/Attribute. It also uses message classes that
+Furthermore, the query parser uses flexible configuration objects. It also uses message classes that
 allow to attach resource bundles. This makes it possible to translate
 messages, which is an important feature of a query parser.
 </p>

Modified: lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/test/org/apache/lucene/queryParser/precedence/TestPrecedenceQueryParser.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/test/org/apache/lucene/queryParser/precedence/TestPrecedenceQueryParser.java?rev=1153315&r1=1153314&r2=1153315&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/test/org/apache/lucene/queryParser/precedence/TestPrecedenceQueryParser.java (original)
+++ lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/test/org/apache/lucene/queryParser/precedence/TestPrecedenceQueryParser.java Wed Aug  3 00:57:50 2011
@@ -43,7 +43,7 @@ import org.apache.lucene.document.DateTo
 import org.apache.lucene.queryParser.TestQueryParser;
 import org.apache.lucene.queryParser.core.QueryNodeException;
 import org.apache.lucene.queryParser.core.QueryNodeParseException;
-import org.apache.lucene.queryParser.standard.config.DefaultOperatorAttribute.Operator;
+import org.apache.lucene.queryParser.standard.config.StandardQueryConfigHandler;
 import org.apache.lucene.queryParser.standard.parser.ParseException;
 import org.apache.lucene.search.BooleanQuery;
 import org.apache.lucene.search.FuzzyQuery;
@@ -132,7 +132,7 @@ public class TestPrecedenceQueryParser e
       a = new MockAnalyzer(random, MockTokenizer.SIMPLE, true);
     PrecedenceQueryParser qp = new PrecedenceQueryParser();
     qp.setAnalyzer(a);
-    qp.setDefaultOperator(Operator.OR);
+    qp.setDefaultOperator(StandardQueryConfigHandler.Operator.OR);
     return qp;
   }
 
@@ -178,7 +178,7 @@ public class TestPrecedenceQueryParser e
       a = new MockAnalyzer(random, MockTokenizer.SIMPLE, true);
     PrecedenceQueryParser qp = new PrecedenceQueryParser();
     qp.setAnalyzer(a);
-    qp.setDefaultOperator(Operator.AND);
+    qp.setDefaultOperator(StandardQueryConfigHandler.Operator.AND);
     return qp.parse(query, "field");
   }
 
@@ -238,11 +238,11 @@ public class TestPrecedenceQueryParser e
     PrecedenceQueryParser qp = new PrecedenceQueryParser();
     qp.setAnalyzer(new MockAnalyzer(random));
     // make sure OR is the default:
-    assertEquals(Operator.OR, qp.getDefaultOperator());
-    qp.setDefaultOperator(Operator.AND);
-    assertEquals(Operator.AND, qp.getDefaultOperator());
-    qp.setDefaultOperator(Operator.OR);
-    assertEquals(Operator.OR, qp.getDefaultOperator());
+    assertEquals(StandardQueryConfigHandler.Operator.OR, qp.getDefaultOperator());
+    qp.setDefaultOperator(StandardQueryConfigHandler.Operator.AND);
+    assertEquals(StandardQueryConfigHandler.Operator.AND, qp.getDefaultOperator());
+    qp.setDefaultOperator(StandardQueryConfigHandler.Operator.OR);
+    assertEquals(StandardQueryConfigHandler.Operator.OR, qp.getDefaultOperator());
 
     assertQueryEquals("a OR !b", null, "a -b");
     assertQueryEquals("a OR ! b", null, "a -b");
@@ -623,7 +623,7 @@ public class TestPrecedenceQueryParser e
     query2 = parser.parse("A (-B +C)", "field");
     assertEquals(query1, query2);
     
-    parser.setDefaultOperator(Operator.AND);
+    parser.setDefaultOperator(StandardQueryConfigHandler.Operator.AND);
     query1 = parser.parse("A AND B OR C AND D", "field");
     query2 = parser.parse("(A AND B) OR (C AND D)", "field");
     assertEquals(query1, query2);

Modified: lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/test/org/apache/lucene/queryParser/spans/SpansQueryConfigHandler.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/test/org/apache/lucene/queryParser/spans/SpansQueryConfigHandler.java?rev=1153315&r1=1153314&r2=1153315&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/test/org/apache/lucene/queryParser/spans/SpansQueryConfigHandler.java (original)
+++ lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/test/org/apache/lucene/queryParser/spans/SpansQueryConfigHandler.java Wed Aug  3 00:57:50 2011
@@ -17,6 +17,7 @@ package org.apache.lucene.queryParser.sp
  * limitations under the License.
  */
 
+import org.apache.lucene.queryParser.core.config.ConfigurationKey;
 import org.apache.lucene.queryParser.core.config.FieldConfig;
 import org.apache.lucene.queryParser.core.config.QueryConfigHandler;
 
@@ -27,9 +28,11 @@ import org.apache.lucene.queryParser.cor
  * It does not return any configuration for a field in specific.
  */
 public class SpansQueryConfigHandler extends QueryConfigHandler {
-
+  
+  final public static ConfigurationKey<String> UNIQUE_FIELD = ConfigurationKey.newInstance();
+  
   public SpansQueryConfigHandler() {
-    addAttribute(UniqueFieldAttribute.class);
+    // empty constructor
   }
 
   @Override

Modified: lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/test/org/apache/lucene/queryParser/spans/TestSpanQueryParser.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/test/org/apache/lucene/queryParser/spans/TestSpanQueryParser.java?rev=1153315&r1=1153314&r2=1153315&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/test/org/apache/lucene/queryParser/spans/TestSpanQueryParser.java (original)
+++ lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/test/org/apache/lucene/queryParser/spans/TestSpanQueryParser.java Wed Aug  3 00:57:50 2011
@@ -127,11 +127,10 @@ public class TestSpanQueryParser extends
     return getSpanQuery("", query);
   }
 
-  public SpanQuery getSpanQuery(CharSequence uniqueField, CharSequence query)
+  public SpanQuery getSpanQuery(String uniqueField, CharSequence query)
       throws QueryNodeException {
-    UniqueFieldAttribute uniqueFieldAtt = this.spanQueryConfigHandler
-        .getAttribute(UniqueFieldAttribute.class);
-    uniqueFieldAtt.setUniqueField(uniqueField);
+    
+    this.spanQueryConfigHandler.set(SpansQueryConfigHandler.UNIQUE_FIELD, uniqueField);
 
     QueryNode queryTree = this.queryParser.parse(query, "defaultField");
     queryTree = this.spanProcessorPipeline.process(queryTree);

Modified: lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/test/org/apache/lucene/queryParser/spans/TestSpanQueryParserSimpleSample.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/test/org/apache/lucene/queryParser/spans/TestSpanQueryParserSimpleSample.java?rev=1153315&r1=1153314&r2=1153315&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/test/org/apache/lucene/queryParser/spans/TestSpanQueryParserSimpleSample.java (original)
+++ lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/test/org/apache/lucene/queryParser/spans/TestSpanQueryParserSimpleSample.java Wed Aug  3 00:57:50 2011
@@ -104,9 +104,7 @@ public class TestSpanQueryParserSimpleSa
     // create a config handler with a attribute used in
     // UniqueFieldQueryNodeProcessor
     QueryConfigHandler spanQueryConfigHandler = new SpansQueryConfigHandler();
-    UniqueFieldAttribute uniqueFieldAtt = spanQueryConfigHandler
-        .getAttribute(UniqueFieldAttribute.class);
-    uniqueFieldAtt.setUniqueField("index");
+    spanQueryConfigHandler.set(SpansQueryConfigHandler.UNIQUE_FIELD, "index");
 
     // set up the processor pipeline with the ConfigHandler
     // and create the pipeline for this simple demo

Modified: lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/test/org/apache/lucene/queryParser/spans/UniqueFieldQueryNodeProcessor.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/test/org/apache/lucene/queryParser/spans/UniqueFieldQueryNodeProcessor.java?rev=1153315&r1=1153314&r2=1153315&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/test/org/apache/lucene/queryParser/spans/UniqueFieldQueryNodeProcessor.java (original)
+++ lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/test/org/apache/lucene/queryParser/spans/UniqueFieldQueryNodeProcessor.java Wed Aug  3 00:57:50 2011
@@ -57,14 +57,12 @@ public class UniqueFieldQueryNodeProcess
             "A config handler is expected by the processor UniqueFieldQueryNodeProcessor!");
       }
 
-      if (!queryConfig.hasAttribute(UniqueFieldAttribute.class)) {
+      if (!queryConfig.has(SpansQueryConfigHandler.UNIQUE_FIELD)) {
         throw new IllegalArgumentException(
             "UniqueFieldAttribute should be defined in the config handler!");
       }
 
-      CharSequence uniqueField = queryConfig.getAttribute(
-          UniqueFieldAttribute.class).getUniqueField();
-
+      String uniqueField = queryConfig.get(SpansQueryConfigHandler.UNIQUE_FIELD);
       fieldNode.setField(uniqueField);
 
     }

Modified: lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/test/org/apache/lucene/queryParser/standard/TestMultiAnalyzerQPHelper.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/test/org/apache/lucene/queryParser/standard/TestMultiAnalyzerQPHelper.java?rev=1153315&r1=1153314&r2=1153315&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/test/org/apache/lucene/queryParser/standard/TestMultiAnalyzerQPHelper.java (original)
+++ lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/test/org/apache/lucene/queryParser/standard/TestMultiAnalyzerQPHelper.java Wed Aug  3 00:57:50 2011
@@ -29,7 +29,7 @@ import org.apache.lucene.analysis.tokena
 import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
 import org.apache.lucene.analysis.tokenattributes.TypeAttribute;
 import org.apache.lucene.queryParser.core.QueryNodeException;
-import org.apache.lucene.queryParser.standard.config.DefaultOperatorAttribute.Operator;
+import org.apache.lucene.queryParser.standard.config.StandardQueryConfigHandler;
 import org.apache.lucene.util.LuceneTestCase;
 
 /**
@@ -43,6 +43,7 @@ public class TestMultiAnalyzerQPHelper e
 
   private static int multiToken = 0;
 
+  @SuppressWarnings("deprecation")
   public void testMultiAnalyzer() throws QueryNodeException {
 
     StandardQueryParser qp = new StandardQueryParser();
@@ -103,7 +104,7 @@ public class TestMultiAnalyzerQPHelper e
     qp.setDefaultPhraseSlop(0);
 
     // non-default operator:
-    qp.setDefaultOperator(Operator.AND);
+    qp.setDefaultOperator(StandardQueryConfigHandler.Operator.AND);
     assertEquals("+(multi multi2) +foo", qp.parse("multi foo", "").toString());
 
   }

Modified: lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/test/org/apache/lucene/queryParser/standard/TestMultiAnalyzerWrapper.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/test/org/apache/lucene/queryParser/standard/TestMultiAnalyzerWrapper.java?rev=1153315&r1=1153314&r2=1153315&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/test/org/apache/lucene/queryParser/standard/TestMultiAnalyzerWrapper.java (original)
+++ lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/test/org/apache/lucene/queryParser/standard/TestMultiAnalyzerWrapper.java Wed Aug  3 00:57:50 2011
@@ -43,6 +43,7 @@ public class TestMultiAnalyzerWrapper ex
 
   private static int multiToken = 0;
 
+  @SuppressWarnings("deprecation")
   public void testMultiAnalyzer() throws ParseException {
 
     QueryParserWrapper qp = new QueryParserWrapper("", new MultiAnalyzer());
@@ -125,6 +126,7 @@ public class TestMultiAnalyzerWrapper ex
   //    
   // }
 
+  @SuppressWarnings("deprecation")
   public void testPosIncrementAnalyzer() throws ParseException {
     QueryParserWrapper qp = new QueryParserWrapper("",
         new PosIncrementAnalyzer());

Modified: lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/test/org/apache/lucene/queryParser/standard/TestMultiFieldQPHelper.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/test/org/apache/lucene/queryParser/standard/TestMultiFieldQPHelper.java?rev=1153315&r1=1153314&r2=1153315&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/test/org/apache/lucene/queryParser/standard/TestMultiFieldQPHelper.java (original)
+++ lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/test/org/apache/lucene/queryParser/standard/TestMultiFieldQPHelper.java Wed Aug  3 00:57:50 2011
@@ -24,12 +24,11 @@ import java.util.Map;
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.analysis.TokenStream;
-import org.apache.lucene.analysis.standard.StandardAnalyzer;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
 import org.apache.lucene.index.IndexWriter;
 import org.apache.lucene.queryParser.core.QueryNodeException;
-import org.apache.lucene.queryParser.standard.config.DefaultOperatorAttribute.Operator;
+import org.apache.lucene.queryParser.standard.config.StandardQueryConfigHandler;
 import org.apache.lucene.search.BooleanClause;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.Query;
@@ -134,7 +133,7 @@ public class TestMultiFieldQPHelper exte
     assertEquals("(b:one t:one) f:two", q.toString());
 
     // AND mode:
-    mfqp.setDefaultOperator(Operator.AND);
+    mfqp.setDefaultOperator(StandardQueryConfigHandler.Operator.AND);
     q = mfqp.parse("one two", null);
     assertEquals("+(b:one t:one) +(b:two t:two)", q.toString());
     q = mfqp.parse("\"aa bb cc\" \"dd ee\"", null);
@@ -330,7 +329,7 @@ public class TestMultiFieldQPHelper exte
 
     mfqp.setMultiFields(new String[] { "body" });
     mfqp.setAnalyzer(analyzer);
-    mfqp.setDefaultOperator(Operator.AND);
+    mfqp.setDefaultOperator(StandardQueryConfigHandler.Operator.AND);
     Query q = mfqp.parse("the footest", null);
     IndexSearcher is = new IndexSearcher(ramDir, true);
     ScoreDoc[] hits = is.search(q, null, 1000).scoreDocs;

Modified: lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/test/org/apache/lucene/queryParser/standard/TestQPHelper.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/test/org/apache/lucene/queryParser/standard/TestQPHelper.java?rev=1153315&r1=1153314&r2=1153315&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/test/org/apache/lucene/queryParser/standard/TestQPHelper.java (original)
+++ lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/test/org/apache/lucene/queryParser/standard/TestQPHelper.java Wed Aug  3 00:57:50 2011
@@ -59,7 +59,8 @@ import org.apache.lucene.queryParser.cor
 import org.apache.lucene.queryParser.core.nodes.QueryNode;
 import org.apache.lucene.queryParser.core.processors.QueryNodeProcessorImpl;
 import org.apache.lucene.queryParser.core.processors.QueryNodeProcessorPipeline;
-import org.apache.lucene.queryParser.standard.config.DefaultOperatorAttribute.Operator;
+import org.apache.lucene.queryParser.standard.config.StandardQueryConfigHandler;
+import org.apache.lucene.queryParser.standard.config.StandardQueryConfigHandler.Operator;
 import org.apache.lucene.queryParser.standard.nodes.WildcardQueryNode;
 import org.apache.lucene.search.BooleanClause;
 import org.apache.lucene.search.BooleanQuery;
@@ -196,7 +197,7 @@ public class TestQPHelper extends Lucene
     StandardQueryParser qp = new StandardQueryParser();
     qp.setAnalyzer(a);
 
-    qp.setDefaultOperator(Operator.OR);
+    qp.setDefaultOperator(StandardQueryConfigHandler.Operator.OR);
 
     return qp;