You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by us...@apache.org on 2011/08/18 22:44:42 UTC

svn commit: r1159411 [1/2] - in /lucene/dev/branches/branch_3x: ./ lucene/ lucene/backwards/ lucene/backwards/src/test-framework/ lucene/backwards/src/test/ lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/messages/ lucene/contrib...

Author: uschindler
Date: Thu Aug 18 20:44:40 2011
New Revision: 1159411

URL: http://svn.apache.org/viewvc?rev=1159411&view=rev
Log:
LUCENE-1768: NumericRange support for flexible query parser (3.x port)

Added:
    lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/FieldValuePairQueryNode.java   (with props)
    lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/RangeQueryNode.java   (with props)
    lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/ValueQueryNode.java   (with props)
    lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/standard/builders/DummyQueryNodeBuilder.java   (with props)
    lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/standard/builders/NumericRangeQueryNodeBuilder.java   (with props)
    lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/standard/builders/TermRangeQueryNodeBuilder.java   (with props)
    lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/standard/config/NumberDateFormat.java   (with props)
    lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/standard/config/NumericConfig.java   (with props)
    lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/standard/config/NumericFieldConfigListener.java   (with props)
    lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/standard/nodes/AbstractRangeQueryNode.java   (with props)
    lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/standard/nodes/NumericQueryNode.java   (with props)
    lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/standard/nodes/NumericRangeQueryNode.java   (with props)
    lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/standard/nodes/TermRangeQueryNode.java   (with props)
    lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/standard/processors/NumericQueryNodeProcessor.java   (with props)
    lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/standard/processors/NumericRangeQueryNodeProcessor.java   (with props)
    lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/test/org/apache/lucene/queryParser/standard/TestNumericQueryParser.java   (with props)
Modified:
    lucene/dev/branches/branch_3x/   (props changed)
    lucene/dev/branches/branch_3x/lucene/   (props changed)
    lucene/dev/branches/branch_3x/lucene/backwards/   (props changed)
    lucene/dev/branches/branch_3x/lucene/backwards/src/test/   (props changed)
    lucene/dev/branches/branch_3x/lucene/backwards/src/test-framework/   (props changed)
    lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/messages/QueryParserMessages.java
    lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/FieldQueryNode.java
    lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/ParametricQueryNode.java
    lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/ParametricRangeQueryNode.java
    lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/package.html
    lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/standard/StandardQueryParser.java
    lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/standard/builders/RangeQueryNodeBuilder.java
    lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/standard/builders/StandardQueryTreeBuilder.java
    lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/standard/config/StandardQueryConfigHandler.java
    lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/standard/nodes/RangeQueryNode.java
    lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/standard/processors/MultiTermRewriteMethodProcessor.java
    lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/standard/processors/ParametricRangeQueryNodeProcessor.java
    lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/standard/processors/StandardQueryNodeProcessorPipeline.java
    lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/standard/processors/package.html
    lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/resources/org/apache/lucene/queryParser/core/messages/QueryParserMessages.properties
    lucene/dev/branches/branch_3x/solr/   (props changed)

Modified: lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/messages/QueryParserMessages.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/messages/QueryParserMessages.java?rev=1159411&r1=1159410&r2=1159411&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/messages/QueryParserMessages.java (original)
+++ lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/messages/QueryParserMessages.java Thu Aug 18 20:44:40 2011
@@ -50,5 +50,8 @@ public class QueryParserMessages extends
   public static String WILDCARD_NOT_SUPPORTED;
   public static String TOO_MANY_BOOLEAN_CLAUSES;
   public static String LEADING_WILDCARD_NOT_ALLOWED;
+  public static String COULD_NOT_PARSE_NUMBER;
+  public static String NUMBER_CLASS_NOT_SUPPORTED_BY_NUMERIC_RANGE_QUERY;
+  public static String UNSUPPORTED_NUMERIC_DATA_TYPE;
 
 }

Modified: lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/FieldQueryNode.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/FieldQueryNode.java?rev=1159411&r1=1159410&r2=1159411&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/FieldQueryNode.java (original)
+++ lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/FieldQueryNode.java Thu Aug 18 20:44:40 2011
@@ -25,8 +25,7 @@ import org.apache.lucene.queryParser.cor
 /**
  * A {@link FieldQueryNode} represents a element that contains field/text tuple
  */
-public class FieldQueryNode extends QueryNodeImpl implements TextableQueryNode,
-    FieldableNode {
+public class FieldQueryNode extends QueryNodeImpl implements FieldValuePairQueryNode<CharSequence>, TextableQueryNode {
 
   private static final long serialVersionUID = 3634521145130758265L;
 
@@ -182,4 +181,12 @@ public class FieldQueryNode extends Quer
 
   }
 
+	public CharSequence getValue() {
+		return getText();
+	}
+
+	public void setValue(CharSequence value) {
+		setText(value);
+	}
+
 }

Added: lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/FieldValuePairQueryNode.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/FieldValuePairQueryNode.java?rev=1159411&view=auto
==============================================================================
--- lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/FieldValuePairQueryNode.java (added)
+++ lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/FieldValuePairQueryNode.java Thu Aug 18 20:44:40 2011
@@ -0,0 +1,30 @@
+package org.apache.lucene.queryParser.core.nodes;
+
+/**
+ * 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.
+ */
+
+/**
+ * This interface should be implemented by {@link QueryNode} that holds a field
+ * and an arbitrary value.
+ * 
+ * @see FieldableNode
+ * @see ValueQueryNode
+ */
+public interface FieldValuePairQueryNode<T extends Object> extends
+    FieldableNode, ValueQueryNode<T> {
+
+}

Modified: lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/ParametricQueryNode.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/ParametricQueryNode.java?rev=1159411&r1=1159410&r2=1159411&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/ParametricQueryNode.java (original)
+++ lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/ParametricQueryNode.java Thu Aug 18 20:44:40 2011
@@ -22,7 +22,11 @@ import org.apache.lucene.queryParser.cor
 /**
  * A {@link ParametricQueryNode} represents LE, LT, GE, GT, EQ, NE query.
  * Example: date >= "2009-10-10" OR price = 200
+ * 
+ * @deprecated this class will be removed in future. {@link FieldQueryNode} 
+ * should be used instead.
  */
+@Deprecated
 public class ParametricQueryNode extends FieldQueryNode {
 
   private static final long serialVersionUID = -5770038129741218116L;

Modified: lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/ParametricRangeQueryNode.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/ParametricRangeQueryNode.java?rev=1159411&r1=1159410&r2=1159411&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/ParametricRangeQueryNode.java (original)
+++ lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/ParametricRangeQueryNode.java Thu Aug 18 20:44:40 2011
@@ -27,7 +27,7 @@ import org.apache.lucene.queryParser.cor
  * Example: date >= "2009-10-10" OR price = 200
  */
 public class ParametricRangeQueryNode extends QueryNodeImpl implements
-    FieldableNode {
+    RangeQueryNode<ParametricQueryNode> {
 
   private static final long serialVersionUID = 7120958816535573935L;
 
@@ -118,4 +118,12 @@ public class ParametricRangeQueryNode ex
 
   }
 
+  public boolean isLowerInclusive() {
+    return getUpperBound().getOperator() == CompareOperator.GE;
+  }
+  
+  public boolean isUpperInclusive() {
+    return getLowerBound().getOperator() == CompareOperator.LE;
+  }
+  
 }

Added: lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/RangeQueryNode.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/RangeQueryNode.java?rev=1159411&view=auto
==============================================================================
--- lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/RangeQueryNode.java (added)
+++ lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/RangeQueryNode.java Thu Aug 18 20:44:40 2011
@@ -0,0 +1,36 @@
+package org.apache.lucene.queryParser.core.nodes;
+
+/**
+ * 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.
+ */
+ 
+ /**
+ * This interface should be implemented by a {@link QueryNode} that represents
+ * some kind of range query.
+ *
+ */
+public interface RangeQueryNode<T extends FieldValuePairQueryNode<?>> extends
+    FieldableNode {
+
+  T getLowerBound();
+  
+  T getUpperBound();
+  
+  boolean isLowerInclusive();
+  
+  boolean isUpperInclusive();
+  
+}

Added: lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/ValueQueryNode.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/ValueQueryNode.java?rev=1159411&view=auto
==============================================================================
--- lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/ValueQueryNode.java (added)
+++ lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/ValueQueryNode.java Thu Aug 18 20:44:40 2011
@@ -0,0 +1,30 @@
+package org.apache.lucene.queryParser.core.nodes;
+
+/**
+ * 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.
+ */
+
+/**
+ * This interface should be implemented by a {@link QueryNode} that holds an
+ * arbitrary value.
+ */
+public interface ValueQueryNode<T extends Object> extends QueryNode {
+  
+  public void setValue(T value);
+  
+  public T getValue();
+  
+}

Modified: lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/package.html
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/package.html?rev=1159411&r1=1159410&r2=1159411&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/package.html (original)
+++ lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/core/nodes/package.html Thu Aug 18 20:44:40 2011
@@ -61,6 +61,7 @@ Grouping nodes:
 <li>FuzzyQueryNode - fuzzy node</li>
 <li>ParametricRangeQueryNode - used for parametric field:[low_value TO high_value]</li>
 <li>ProximityQueryNode - used for proximity search</li>
+<li>NumericRangeQueryNode - used for numeric range search</li>
 <li>TokenizedPhraseQueryNode - used by tokenizers/lemmatizers/analyzers for phrases/autophrases</li>
 </ul>
 </p>
@@ -68,6 +69,7 @@ Grouping nodes:
 Leaf Nodes:
 <ul>
 <li>FieldQueryNode - field/value node</li>
+<li>NumericQueryNode - used for numeric search</li>
 <li>PathQueryNode - {@link org.apache.lucene.queryParser.core.nodes.QueryNode} object used with path-like queries</li>
 <li>OpaqueQueryNode - Used as for part of the query that can be parsed by other parsers. schema/value</li> 
 <li>ParametricQueryNode - used for parametric field [>=|<=|=|<|>] value</li>

Modified: lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/standard/StandardQueryParser.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/standard/StandardQueryParser.java?rev=1159411&r1=1159410&r2=1159411&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/standard/StandardQueryParser.java (original)
+++ lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/standard/StandardQueryParser.java Thu Aug 18 20:44:40 2011
@@ -31,6 +31,7 @@ import org.apache.lucene.queryParser.cor
 import org.apache.lucene.queryParser.standard.builders.StandardQueryTreeBuilder;
 import org.apache.lucene.queryParser.standard.config.DefaultOperatorAttribute;
 import org.apache.lucene.queryParser.standard.config.FuzzyConfig;
+import org.apache.lucene.queryParser.standard.config.NumericConfig;
 import org.apache.lucene.queryParser.standard.config.StandardQueryConfigHandler;
 import org.apache.lucene.queryParser.standard.config.StandardQueryConfigHandler.Operator;
 import org.apache.lucene.queryParser.standard.config.StandardQueryConfigHandler.ConfigurationKeys;
@@ -112,7 +113,7 @@ import org.apache.lucene.search.Query;
  * @see StandardQueryTreeBuilder
  */
 public class StandardQueryParser extends QueryParserHelper {
-
+  
   /**
    * Constructs a {@link StandardQueryParser} object.
    */
@@ -121,7 +122,7 @@ public class StandardQueryParser extends
         new StandardQueryNodeProcessorPipeline(null),
         new StandardQueryTreeBuilder());
   }
-
+  
   /**
    * Constructs a {@link StandardQueryParser} object and sets an
    * {@link Analyzer} to it. The same as:
@@ -136,15 +137,16 @@ public class StandardQueryParser extends
    */
   public StandardQueryParser(Analyzer analyzer) {
     this();
-
+    
     this.setAnalyzer(analyzer);
   }
   
   @Override
-  public String toString(){
-    return "<StandardQueryParser config=\"" + this.getQueryConfigHandler() + "\"/>";
+  public String toString() {
+    return "<StandardQueryParser config=\"" + this.getQueryConfigHandler()
+        + "\"/>";
   }
-
+  
   /**
    * Overrides {@link QueryParserHelper#parse(String, String)} so it casts the
    * return object to {@link Query}. For more reference about this method, check
@@ -163,11 +165,11 @@ public class StandardQueryParser extends
   @Override
   public Query parse(String query, String defaultField)
       throws QueryNodeException {
-
+    
     return (Query) super.parse(query, defaultField);
-
+    
   }
-
+  
   /**
    * Gets implicit operator setting, which will be either {@link Operator#AND}
    * or {@link Operator#OR}.
@@ -175,7 +177,7 @@ public class StandardQueryParser extends
   public StandardQueryConfigHandler.Operator getDefaultOperator() {
     return getQueryConfigHandler().get(ConfigurationKeys.DEFAULT_OPERATOR);
   }
-
+  
   /**
    * Sets the collator used to determine index term inclusion in ranges for
    * RangeQuerys.
@@ -235,7 +237,7 @@ public class StandardQueryParser extends
   public void setDefaultOperator(org.apache.lucene.queryParser.standard.config.StandardQueryConfigHandler.Operator operator) {
     getQueryConfigHandler().set(ConfigurationKeys.DEFAULT_OPERATOR, operator);
   }
-
+  
   /**
    * Set to <code>true</code> to allow leading wildcard characters.
    * <p>
@@ -248,7 +250,7 @@ public class StandardQueryParser extends
   public void setLowercaseExpandedTerms(boolean lowercaseExpandedTerms) {
     getQueryConfigHandler().set(ConfigurationKeys.LOWERCASE_EXPANDED_TERMS, lowercaseExpandedTerms);
   }
-
+  
   /**
    * @see #setLowercaseExpandedTerms(boolean)
    */
@@ -263,7 +265,7 @@ public class StandardQueryParser extends
     }
     
   }
-
+  
   /**
    * Set to <code>true</code> to allow leading wildcard characters.
    * <p>
@@ -276,7 +278,7 @@ public class StandardQueryParser extends
   public void setAllowLeadingWildcard(boolean allowLeadingWildcard) {
     getQueryConfigHandler().set(ConfigurationKeys.ALLOW_LEADING_WILDCARD, allowLeadingWildcard);
   }
-
+  
   /**
    * Set to <code>true</code> to enable position increments in result query.
    * <p>
@@ -289,7 +291,7 @@ public class StandardQueryParser extends
   public void setEnablePositionIncrements(boolean enabled) {
     getQueryConfigHandler().set(ConfigurationKeys.ENABLE_POSITION_INCREMENTS, enabled);
   }
-
+  
   /**
    * @see #setEnablePositionIncrements(boolean)
    */
@@ -304,7 +306,7 @@ public class StandardQueryParser extends
     }
     
   }
-
+  
   /**
    * By default, it uses
    * {@link MultiTermQuery#CONSTANT_SCORE_AUTO_REWRITE_DEFAULT} when creating a
@@ -318,14 +320,14 @@ public class StandardQueryParser extends
   public void setMultiTermRewriteMethod(MultiTermQuery.RewriteMethod method) {
     getQueryConfigHandler().set(ConfigurationKeys.MULTI_TERM_REWRITE_METHOD, method);
   }
-
+  
   /**
    * @see #setMultiTermRewriteMethod(org.apache.lucene.search.MultiTermQuery.RewriteMethod)
    */
   public MultiTermQuery.RewriteMethod getMultiTermRewriteMethod() {
     return getQueryConfigHandler().get(ConfigurationKeys.MULTI_TERM_REWRITE_METHOD);
   }
-
+  
   /**
    * Set the fields a query should be expanded to when the field is
    * <code>null</code>
@@ -333,15 +335,15 @@ public class StandardQueryParser extends
    * @param fields the fields used to expand the query
    */
   public void setMultiFields(CharSequence[] fields) {
-
+    
     if (fields == null) {
       fields = new CharSequence[0];
     }
 
     getQueryConfigHandler().set(ConfigurationKeys.MULTI_FIELDS, fields);
-
+    
   }
-
+  
   /**
    * Returns the fields used to expand the query when the field for a
    * certain query is <code>null</code>
@@ -370,21 +372,29 @@ public class StandardQueryParser extends
     fuzzyConfig.setPrefixLength(fuzzyPrefixLength);
     
   }
-
+  
+  public void setNumericConfigMap(Map<String,NumericConfig> numericConfigMap) {
+    getQueryConfigHandler().set(ConfigurationKeys.NUMERIC_CONFIG_MAP, numericConfigMap);
+  }
+  
+  public Map<String,NumericConfig> getNumericConfigMap() {
+    return getQueryConfigHandler().get(ConfigurationKeys.NUMERIC_CONFIG_MAP);
+  }
+  
   /**
    * Set locale used by date range parsing.
    */
   public void setLocale(Locale locale) {
     getQueryConfigHandler().set(ConfigurationKeys.LOCALE, locale);
   }
-
+  
   /**
    * Returns current locale, allowing access by subclasses.
    */
   public Locale getLocale() {
     return getQueryConfigHandler().get(ConfigurationKeys.LOCALE);
   }
-
+  
   /**
    * Sets the default slop for phrases. If zero, then exact phrase matches are
    * required. Default value is zero.
@@ -408,10 +418,10 @@ public class StandardQueryParser extends
     getQueryConfigHandler().set(ConfigurationKeys.ANALYZER, analyzer);
   }
   
-  public Analyzer getAnalyzer() {    
+  public Analyzer getAnalyzer() {
     return getQueryConfigHandler().get(ConfigurationKeys.ANALYZER);       
   }
-
+  
   /**
    * @see #setAllowLeadingWildcard(boolean)
    */
@@ -425,7 +435,7 @@ public class StandardQueryParser extends
       return allowLeadingWildcard;
     }
   }
-
+  
   /**
    * Get the minimal similarity for fuzzy queries.
    */
@@ -438,7 +448,7 @@ public class StandardQueryParser extends
       return fuzzyConfig.getMinSimilarity();
     }
   }
-
+  
   /**
    * Get the prefix length for fuzzy queries.
    * 
@@ -453,7 +463,7 @@ public class StandardQueryParser extends
       return fuzzyConfig.getPrefixLength();
     }
   }
-
+  
   /**
    * Gets the default slop for phrases.
    */
@@ -467,7 +477,7 @@ public class StandardQueryParser extends
       return phraseSlop;
     }
   }
-
+  
   /**
    * Set the minimum similarity for fuzzy queries. Default is defined on
    * {@link FuzzyQuery#defaultMinSimilarity}.

Added: lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/standard/builders/DummyQueryNodeBuilder.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/standard/builders/DummyQueryNodeBuilder.java?rev=1159411&view=auto
==============================================================================
--- lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/standard/builders/DummyQueryNodeBuilder.java (added)
+++ lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/standard/builders/DummyQueryNodeBuilder.java Thu Aug 18 20:44:40 2011
@@ -0,0 +1,46 @@
+package org.apache.lucene.queryParser.standard.builders;
+
+/**
+ * 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 org.apache.lucene.queryParser.core.QueryNodeException;
+import org.apache.lucene.queryParser.core.nodes.QueryNode;
+import org.apache.lucene.search.TermQuery;
+
+/**
+ * This builder does nothing. Commonly used for {@link QueryNode} objects that
+ * are built by its parent's builder.
+ */
+public class DummyQueryNodeBuilder implements StandardQueryBuilder {
+  
+  /**
+   * Constructs a {@link DummyQueryNodeBuilder} object.
+   */
+  public DummyQueryNodeBuilder() {
+  // empty constructor
+  }
+  
+  /**
+   * Always return <code>null</code>.
+   * 
+   * return <code>null</code>
+   */
+  public TermQuery build(QueryNode queryNode) throws QueryNodeException {
+    return null;
+  }
+
+}

Added: lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/standard/builders/NumericRangeQueryNodeBuilder.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/standard/builders/NumericRangeQueryNodeBuilder.java?rev=1159411&view=auto
==============================================================================
--- lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/standard/builders/NumericRangeQueryNodeBuilder.java (added)
+++ lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/standard/builders/NumericRangeQueryNodeBuilder.java Thu Aug 18 20:44:40 2011
@@ -0,0 +1,102 @@
+package org.apache.lucene.queryParser.standard.builders;
+
+/**
+ * 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 org.apache.lucene.document.NumericField;
+import org.apache.lucene.messages.MessageImpl;
+import org.apache.lucene.queryParser.core.QueryNodeException;
+import org.apache.lucene.queryParser.core.messages.QueryParserMessages;
+import org.apache.lucene.queryParser.core.nodes.QueryNode;
+import org.apache.lucene.queryParser.core.util.StringUtils;
+import org.apache.lucene.queryParser.standard.config.NumericConfig;
+import org.apache.lucene.queryParser.standard.nodes.NumericQueryNode;
+import org.apache.lucene.queryParser.standard.nodes.NumericRangeQueryNode;
+import org.apache.lucene.search.NumericRangeQuery;
+
+/**
+ * Builds {@link NumericRangeQuery}s out of {@link NumericRangeQueryNode}s.
+ *
+ * @see NumericRangeQuery
+ * @see NumericRangeQueryNode
+ */
+public class NumericRangeQueryNodeBuilder implements StandardQueryBuilder {
+  
+  /**
+   * Constructs a {@link NumericRangeQueryNodeBuilder} object.
+   */
+  public NumericRangeQueryNodeBuilder() {
+  // empty constructor
+  }
+  
+  public NumericRangeQuery<? extends Number> build(QueryNode queryNode)
+      throws QueryNodeException {
+    NumericRangeQueryNode numericRangeNode = (NumericRangeQueryNode) queryNode;
+    
+    NumericQueryNode lowerNumericNode = numericRangeNode.getLowerBound();
+    NumericQueryNode upperNumericNode = numericRangeNode.getUpperBound();
+    
+    Number lowerNumber, upperNumber;
+    
+    if (lowerNumericNode != null) {
+      lowerNumber = lowerNumericNode.getValue();
+    } else {
+      lowerNumber = null;
+    }
+    
+    if (upperNumericNode != null) {
+      upperNumber = upperNumericNode.getValue();
+    } else {
+      upperNumber = null;
+    }
+    
+    NumericConfig numericConfig = numericRangeNode.getNumericConfig();
+    NumericField.DataType numberType = numericConfig.getType();
+    String field = StringUtils.toString(numericRangeNode.getField());
+    boolean minInclusive = numericRangeNode.isLowerInclusive();
+    boolean maxInclusive = numericRangeNode.isUpperInclusive();
+    int precisionStep = numericConfig.getPrecisionStep();
+    
+    switch (numberType) {
+      
+      case LONG:
+        return NumericRangeQuery.newLongRange(field, precisionStep,
+            (Long) lowerNumber, (Long) upperNumber, minInclusive, maxInclusive);
+      
+      case INT:
+        return NumericRangeQuery.newIntRange(field, precisionStep,
+            (Integer) lowerNumber, (Integer) upperNumber, minInclusive,
+            maxInclusive);
+      
+      case FLOAT:
+        return NumericRangeQuery.newFloatRange(field, precisionStep,
+            (Float) lowerNumber, (Float) upperNumber, minInclusive,
+            maxInclusive);
+      
+      case DOUBLE:
+        return NumericRangeQuery.newDoubleRange(field, precisionStep,
+            (Double) lowerNumber, (Double) upperNumber, minInclusive,
+            maxInclusive);
+        
+        default :
+          throw new QueryNodeException(new MessageImpl(
+            QueryParserMessages.UNSUPPORTED_NUMERIC_DATA_TYPE, numberType));
+        
+    }
+  }
+  
+}

Modified: lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/standard/builders/RangeQueryNodeBuilder.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/standard/builders/RangeQueryNodeBuilder.java?rev=1159411&r1=1159410&r2=1159411&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/standard/builders/RangeQueryNodeBuilder.java (original)
+++ lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/standard/builders/RangeQueryNodeBuilder.java Thu Aug 18 20:44:40 2011
@@ -67,5 +67,7 @@ public class RangeQueryNodeBuilder imple
     return rangeQuery;
 
   }
+  
+  
 
 }

Modified: lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/standard/builders/StandardQueryTreeBuilder.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/standard/builders/StandardQueryTreeBuilder.java?rev=1159411&r1=1159410&r2=1159411&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/standard/builders/StandardQueryTreeBuilder.java (original)
+++ lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/standard/builders/StandardQueryTreeBuilder.java Thu Aug 18 20:44:40 2011
@@ -31,9 +31,12 @@ import org.apache.lucene.queryParser.cor
 import org.apache.lucene.queryParser.core.nodes.SlopQueryNode;
 import org.apache.lucene.queryParser.core.nodes.TokenizedPhraseQueryNode;
 import org.apache.lucene.queryParser.standard.nodes.MultiPhraseQueryNode;
+import org.apache.lucene.queryParser.standard.nodes.NumericQueryNode;
+import org.apache.lucene.queryParser.standard.nodes.NumericRangeQueryNode;
 import org.apache.lucene.queryParser.standard.nodes.PrefixWildcardQueryNode;
 import org.apache.lucene.queryParser.standard.nodes.RangeQueryNode;
 import org.apache.lucene.queryParser.standard.nodes.StandardBooleanQueryNode;
+import org.apache.lucene.queryParser.standard.nodes.TermRangeQueryNode;
 import org.apache.lucene.queryParser.standard.nodes.WildcardQueryNode;
 import org.apache.lucene.queryParser.standard.processors.StandardQueryNodeProcessorPipeline;
 import org.apache.lucene.search.Query;
@@ -49,12 +52,14 @@ import org.apache.lucene.search.Query;
  */
 public class StandardQueryTreeBuilder extends QueryTreeBuilder implements
     StandardQueryBuilder {
-
+  
   public StandardQueryTreeBuilder() {
     setBuilder(GroupQueryNode.class, new GroupQueryNodeBuilder());
     setBuilder(FieldQueryNode.class, new FieldQueryNodeBuilder());
     setBuilder(BooleanQueryNode.class, new BooleanQueryNodeBuilder());
     setBuilder(FuzzyQueryNode.class, new FuzzyQueryNodeBuilder());
+    setBuilder(NumericQueryNode.class, new DummyQueryNodeBuilder());
+    setBuilder(NumericRangeQueryNode.class, new NumericRangeQueryNodeBuilder());
     setBuilder(BoostQueryNode.class, new BoostQueryNodeBuilder());
     setBuilder(ModifierQueryNode.class, new ModifierQueryNodeBuilder());
     setBuilder(WildcardQueryNode.class, new WildcardQueryNodeBuilder());
@@ -63,6 +68,7 @@ public class StandardQueryTreeBuilder ex
     setBuilder(PrefixWildcardQueryNode.class,
         new PrefixWildcardQueryNodeBuilder());
     setBuilder(RangeQueryNode.class, new RangeQueryNodeBuilder());
+    setBuilder(TermRangeQueryNode.class, new TermRangeQueryNodeBuilder());
     setBuilder(SlopQueryNode.class, new SlopQueryNodeBuilder());
     setBuilder(StandardBooleanQueryNode.class,
         new StandardBooleanQueryNodeBuilder());

Added: lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/standard/builders/TermRangeQueryNodeBuilder.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/standard/builders/TermRangeQueryNodeBuilder.java?rev=1159411&view=auto
==============================================================================
--- lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/standard/builders/TermRangeQueryNodeBuilder.java (added)
+++ lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/standard/builders/TermRangeQueryNodeBuilder.java Thu Aug 18 20:44:40 2011
@@ -0,0 +1,60 @@
+package org.apache.lucene.queryParser.standard.builders;
+
+/**
+ * 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 org.apache.lucene.queryParser.core.QueryNodeException;
+import org.apache.lucene.queryParser.core.nodes.FieldQueryNode;
+import org.apache.lucene.queryParser.core.nodes.QueryNode;
+import org.apache.lucene.queryParser.core.util.StringUtils;
+import org.apache.lucene.queryParser.standard.nodes.TermRangeQueryNode;
+import org.apache.lucene.queryParser.standard.processors.MultiTermRewriteMethodProcessor;
+import org.apache.lucene.search.MultiTermQuery;
+import org.apache.lucene.search.TermRangeQuery;
+
+/**
+ * Builds a {@link TermRangeQuery} object from a {@link TermRangeQueryNode}
+ * object.
+ */
+public class TermRangeQueryNodeBuilder implements StandardQueryBuilder {
+  
+  public TermRangeQueryNodeBuilder() {
+  // empty constructor
+  }
+  
+  public TermRangeQuery build(QueryNode queryNode) throws QueryNodeException {
+    TermRangeQueryNode rangeNode = (TermRangeQueryNode) queryNode;
+    FieldQueryNode upper = rangeNode.getUpperBound();
+    FieldQueryNode lower = rangeNode.getLowerBound();
+    
+    String field = StringUtils.toString(rangeNode.getField());
+    
+    TermRangeQuery rangeQuery = new TermRangeQuery(field, lower
+        .getTextAsString(), upper.getTextAsString(), rangeNode
+        .isLowerInclusive(), rangeNode.isUpperInclusive());
+    
+    MultiTermQuery.RewriteMethod method = (MultiTermQuery.RewriteMethod) queryNode
+        .getTag(MultiTermRewriteMethodProcessor.TAG_ID);
+    if (method != null) {
+      rangeQuery.setRewriteMethod(method);
+    }
+    
+    return rangeQuery;
+    
+  }
+  
+}

Added: lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/standard/config/NumberDateFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/standard/config/NumberDateFormat.java?rev=1159411&view=auto
==============================================================================
--- lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/standard/config/NumberDateFormat.java (added)
+++ lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/standard/config/NumberDateFormat.java Thu Aug 18 20:44:40 2011
@@ -0,0 +1,71 @@
+package org.apache.lucene.queryParser.standard.config;
+
+/**
+ * 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.text.DateFormat;
+import java.text.FieldPosition;
+import java.text.Format;
+import java.text.NumberFormat;
+import java.text.ParsePosition;
+import java.util.Date;
+
+/**
+ * This {@link Format} parses {@link Long} into date strings and vice-versa. It
+ * uses the given {@link DateFormat} to parse and format dates, but before, it
+ * converts {@link Long} to {@link Date} objects or vice-versa.
+ */
+public class NumberDateFormat extends NumberFormat {
+  
+  private static final long serialVersionUID = 964823936071308283L;
+  
+  final private DateFormat dateFormat;
+  
+  /**
+   * Constructs a {@link NumberDateFormat} object using the given {@link DateFormat}.
+   * 
+   * @param dateFormat {@link DateFormat} used to parse and format dates
+   */
+  public NumberDateFormat(DateFormat dateFormat) {
+    this.dateFormat = dateFormat;
+  }
+  
+  @Override
+  public StringBuffer format(double number, StringBuffer toAppendTo,
+      FieldPosition pos) {
+    return dateFormat.format(new Date((long) number), toAppendTo, pos);
+  }
+  
+  @Override
+  public StringBuffer format(long number, StringBuffer toAppendTo,
+      FieldPosition pos) {
+    return dateFormat.format(new Date(number), toAppendTo, pos);
+  }
+  
+  @Override
+  public Number parse(String source, ParsePosition parsePosition) {
+    final Date date = dateFormat.parse(source, parsePosition);
+    return (date == null) ? null : date.getTime();
+  }
+  
+  @Override
+  public StringBuffer format(Object number, StringBuffer toAppendTo,
+      FieldPosition pos) {
+    return dateFormat.format(number, toAppendTo, pos);
+  }
+  
+}

Added: lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/standard/config/NumericConfig.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/standard/config/NumericConfig.java?rev=1159411&view=auto
==============================================================================
--- lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/standard/config/NumericConfig.java (added)
+++ lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/standard/config/NumericConfig.java Thu Aug 18 20:44:40 2011
@@ -0,0 +1,159 @@
+package org.apache.lucene.queryParser.standard.config;
+
+/**
+ * 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.text.NumberFormat;
+
+import org.apache.lucene.document.NumericField;
+import org.apache.lucene.search.NumericRangeQuery;
+
+/**
+ * This class holds the configuration used to parse numeric queries and create
+ * {@link NumericRangeQuery}s.
+ * 
+ * @see NumericRangeQuery
+ * @see NumberFormat
+ */
+public class NumericConfig {
+  
+  private int precisionStep;
+  
+  private NumberFormat format;
+  
+  private NumericField.DataType type;
+  
+  /**
+   * Constructs a {@link NumericConfig} object.
+   * 
+   * @param precisionStep
+   *          the precision used to index the numeric values
+   * @param format
+   *          the {@link NumberFormat} used to parse a {@link String} to
+   *          {@link Number}
+   * @param type
+   *          the numeric type used to index the numeric values
+   * 
+   * @see NumericConfig#setPrecisionStep(int)
+   * @see NumericConfig#setNumberFormat(NumberFormat)
+   * @see #setType(org.apache.lucene.document.NumericField.DataType)
+   */
+  public NumericConfig(int precisionStep, NumberFormat format,
+      NumericField.DataType type) {
+    setPrecisionStep(precisionStep);
+    setNumberFormat(format);
+    setType(type);
+    
+  }
+  
+  /**
+   * Returns the precision used to index the numeric values
+   * 
+   * @return the precision used to index the numeric values
+   * 
+   * @see NumericRangeQuery#getPrecisionStep()
+   */
+  public int getPrecisionStep() {
+    return precisionStep;
+  }
+  
+  /**
+   * Sets the precision used to index the numeric values
+   * 
+   * @param precisionStep
+   *          the precision used to index the numeric values
+   * 
+   * @see NumericRangeQuery#getPrecisionStep()
+   */
+  public void setPrecisionStep(int precisionStep) {
+    this.precisionStep = precisionStep;
+  }
+  
+  /**
+   * Returns the {@link NumberFormat} used to parse a {@link String} to
+   * {@link Number}
+   * 
+   * @return the {@link NumberFormat} used to parse a {@link String} to
+   *         {@link Number}
+   */
+  public NumberFormat getNumberFormat() {
+    return format;
+  }
+  
+  /**
+   * Returns the numeric type used to index the numeric values
+   * 
+   * @return the numeric type used to index the numeric values
+   */
+  public NumericField.DataType getType() {
+    return type;
+  }
+  
+  /**
+   * Sets the numeric type used to index the numeric values
+   * 
+   * @param type the numeric type used to index the numeric values
+   */
+  public void setType(NumericField.DataType type) {
+    
+    if (type == null) {
+      throw new IllegalArgumentException("type cannot be null!");
+    }
+    
+    this.type = type;
+    
+  }
+  
+  /**
+   * Sets the {@link NumberFormat} used to parse a {@link String} to
+   * {@link Number}
+   * 
+   * @param format
+   *          the {@link NumberFormat} used to parse a {@link String} to
+   *          {@link Number}, cannot be <code>null</code>
+   */
+  public void setNumberFormat(NumberFormat format) {
+    
+    if (format == null) {
+      throw new IllegalArgumentException("format cannot be null!");
+    }
+    
+    this.format = format;
+    
+  }
+  
+  @Override
+  public boolean equals(Object obj) {
+    
+    if (obj == this) return true;
+    
+    if (obj instanceof NumericConfig) {
+      NumericConfig other = (NumericConfig) obj;
+      
+      if (this.precisionStep == other.precisionStep
+          && this.type == other.type
+          && (this.format == other.format || (this.format.equals(other.format)))) {
+        return true;
+      }
+      
+    }
+    
+    return false;
+    
+  }
+  
+}

Added: lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/standard/config/NumericFieldConfigListener.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/standard/config/NumericFieldConfigListener.java?rev=1159411&view=auto
==============================================================================
--- lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/standard/config/NumericFieldConfigListener.java (added)
+++ lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/standard/config/NumericFieldConfigListener.java Thu Aug 18 20:44:40 2011
@@ -0,0 +1,73 @@
+package org.apache.lucene.queryParser.standard.config;
+
+/**
+ * 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.Map;
+
+import org.apache.lucene.queryParser.core.config.FieldConfig;
+import org.apache.lucene.queryParser.core.config.FieldConfigListener;
+import org.apache.lucene.queryParser.core.config.QueryConfigHandler;
+import org.apache.lucene.queryParser.standard.config.StandardQueryConfigHandler.ConfigurationKeys;
+
+/**
+ * This listener is used to listen to {@link FieldConfig} requests in
+ * {@link QueryConfigHandler} and add {@link ConfigurationKeys#NUMERIC_CONFIG}
+ * based on the {@link ConfigurationKeys#NUMERIC_CONFIG_MAP} set in the
+ * {@link QueryConfigHandler}.
+ * 
+ * @see NumericConfig
+ * @see QueryConfigHandler
+ * @see ConfigurationKeys#NUMERIC_CONFIG
+ * @see ConfigurationKeys#NUMERIC_CONFIG_MAP
+ */
+public class NumericFieldConfigListener implements FieldConfigListener {
+  
+  final private QueryConfigHandler config;
+  
+  /**
+   * Construcs a {@link NumericFieldConfigListener} object using the given {@link QueryConfigHandler}.
+   * 
+   * @param config the {@link QueryConfigHandler} it will listen too
+   */
+  public NumericFieldConfigListener(QueryConfigHandler config) {
+    
+    if (config == null) {
+      throw new IllegalArgumentException("config cannot be null!");
+    }
+    
+    this.config = config;
+    
+  }
+  
+  public void buildFieldConfig(FieldConfig fieldConfig) {
+    Map<String,NumericConfig> numericConfigMap = config
+        .get(ConfigurationKeys.NUMERIC_CONFIG_MAP);
+    
+    if (numericConfigMap != null) {
+      NumericConfig numericConfig = numericConfigMap
+          .get(fieldConfig.getField());
+      
+      if (numericConfig != null) {
+        fieldConfig.set(ConfigurationKeys.NUMERIC_CONFIG, numericConfig);
+      }
+      
+    }
+    
+  }
+  
+}

Modified: lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/standard/config/StandardQueryConfigHandler.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/standard/config/StandardQueryConfigHandler.java?rev=1159411&r1=1159410&r2=1159411&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/standard/config/StandardQueryConfigHandler.java (original)
+++ lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/standard/config/StandardQueryConfigHandler.java Thu Aug 18 20:44:40 2011
@@ -177,6 +177,26 @@ public class StandardQueryConfigHandler 
      */
     final public static ConfigurationKey<Float> BOOST = ConfigurationKey
         .newInstance();
+    
+    /**
+     * Key used to set a field to its {@link NumericConfig}.
+     * 
+     * @see StandardQueryParser#setNumericConfigMap(Map)
+     * @see StandardQueryParser#getNumericConfigMap()
+     */
+    final public static ConfigurationKey<NumericConfig> NUMERIC_CONFIG = ConfigurationKey
+        .newInstance();
+    
+    /**
+     * Key used to set the {@link NumericConfig} in {@link FieldConfig} for
+     * numeric fields.
+     * 
+     * @see StandardQueryParser#setNumericConfigMap(Map)
+     * @see StandardQueryParser#getNumericConfigMap()
+     */
+    final public static ConfigurationKey<Map<String,NumericConfig>> NUMERIC_CONFIG_MAP = ConfigurationKey
+        .newInstance();
+        
 
     /**
      * Key used to set the {@link Collator} used when creating {@link TermRangeQuery}s.
@@ -198,6 +218,7 @@ public class StandardQueryConfigHandler 
     // Add listener that will build the FieldConfig.
     addFieldConfigListener(new FieldBoostMapFCListener(this));
     addFieldConfigListener(new FieldDateResolutionFCListener(this));
+    addFieldConfigListener(new NumericFieldConfigListener(this));
 
     // Default Values
     set(ConfigurationKeys.ALLOW_LEADING_WILDCARD, false); // default in 2.9

Added: lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/standard/nodes/AbstractRangeQueryNode.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/standard/nodes/AbstractRangeQueryNode.java?rev=1159411&view=auto
==============================================================================
--- lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/standard/nodes/AbstractRangeQueryNode.java (added)
+++ lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/standard/nodes/AbstractRangeQueryNode.java Thu Aug 18 20:44:40 2011
@@ -0,0 +1,210 @@
+package org.apache.lucene.queryParser.standard.nodes;
+
+/**
+ * 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 org.apache.lucene.queryParser.core.nodes.FieldValuePairQueryNode;
+import org.apache.lucene.queryParser.core.nodes.QueryNode;
+import org.apache.lucene.queryParser.core.nodes.QueryNodeImpl;
+import org.apache.lucene.queryParser.core.nodes.RangeQueryNode;
+import org.apache.lucene.queryParser.core.parser.EscapeQuerySyntax;
+import org.apache.lucene.queryParser.core.util.StringUtils;
+
+/**
+ * This class should be extended by nodes intending to represent range queries.
+ * 
+ * @param <T>
+ *          the type of the range query bounds (lower and upper)
+ */
+public abstract class AbstractRangeQueryNode<T extends FieldValuePairQueryNode<?>>
+    extends QueryNodeImpl implements RangeQueryNode<FieldValuePairQueryNode<?>> {
+  
+  private static final long serialVersionUID = 4475492120315147792L;
+  
+  private boolean lowerInclusive, upperInclusive;
+  
+  /**
+   * Constructs an {@link AbstractRangeQueryNode}, it should be invoked only by
+   * its extenders.
+   */
+  protected AbstractRangeQueryNode() {
+    setLeaf(false);
+    allocate();
+  }
+  
+  /**
+   * Returns the field associated with this node.
+   * 
+   * @return the field associated with this node
+   * 
+   * @see FieldableNode
+   */
+  public CharSequence getField() {
+    CharSequence field = null;
+    T lower = getLowerBound();
+    T upper = getUpperBound();
+    
+    if (lower != null) {
+      field = lower.getField();
+      
+    } else if (upper != null) {
+      field = upper.getField();
+    }
+    
+    return field;
+    
+  }
+  
+  /**
+   * Sets the field associated with this node.
+   * 
+   * @param fieldName the field associated with this node
+   */
+  public void setField(CharSequence fieldName) {
+    T lower = getLowerBound();
+    T upper = getUpperBound();
+    
+    if (lower != null) {
+      lower.setField(fieldName);
+    }
+    
+    if (upper != null) {
+      upper.setField(fieldName);
+    }
+    
+  }
+  
+  /**
+   * Returns the lower bound node.
+   * 
+   * @return the lower bound node.
+   */
+  @SuppressWarnings("unchecked")
+  public T getLowerBound() {
+    return (T) getChildren().get(0);
+  }
+  
+  /**
+   * Returns the upper bound node.
+   * 
+   * @return the upper bound node.
+   */
+  @SuppressWarnings("unchecked")
+  public T getUpperBound() {
+    return (T) getChildren().get(1);
+  }
+  
+  /**
+   * Returns whether the lower bound is inclusive or exclusive.
+   * 
+   * @return <code>true</code> if the lower bound is inclusive, otherwise, <code>false</code>
+   */
+  public boolean isLowerInclusive() {
+    return lowerInclusive;
+  }
+  
+  /**
+   * Returns whether the upper bound is inclusive or exclusive.
+   * 
+   * @return <code>true</code> if the upper bound is inclusive, otherwise, <code>false</code>
+   */
+  public boolean isUpperInclusive() {
+    return upperInclusive;
+  }
+  
+  /**
+   * Sets the lower and upper bounds.
+   * 
+   * @param lower the lower bound, <code>null</code> if lower bound is open
+   * @param upper the upper bound, <code>null</code> if upper bound is open
+   * @param lowerInclusive <code>true</code> if the lower bound is inclusive, otherwise, <code>false</code>
+   * @param upperInclusive <code>true</code> if the upper bound is inclusive, otherwise, <code>false</code>
+   * 
+   * @see #getLowerBound()
+   * @see #getUpperBound()
+   * @see #isLowerInclusive()
+   * @see #isUpperInclusive()
+   */
+  public void setBounds(T lower, T upper, boolean lowerInclusive,
+      boolean upperInclusive) {
+    
+    if (lower != null && upper != null) {
+      String lowerField = StringUtils.toString(lower.getField());
+      String upperField = StringUtils.toString(upper.getField());
+      
+      if ((upperField == null && lowerField == null)
+          || (upperField != null && !upperField.equals(lowerField))) {
+        throw new IllegalArgumentException(
+            "lower and upper bounds should have the same field name!");
+      }
+      
+      this.lowerInclusive = lowerInclusive;
+      this.upperInclusive = upperInclusive;
+      
+      ArrayList<QueryNode> children = new ArrayList<QueryNode>(2);
+      children.add(lower);
+      children.add(upper);
+      
+      set(children);
+      
+    }
+    
+  }
+  
+  public CharSequence toQueryString(EscapeQuerySyntax escapeSyntaxParser) {
+    StringBuilder sb = new StringBuilder();
+    
+    T lower = getLowerBound();
+    T upper = getUpperBound();
+    
+    if (lowerInclusive) {
+      sb.append('[');
+      
+    } else {
+      sb.append('{');
+    }
+    
+    if (lower != null) {
+      sb.append(lower.toQueryString(escapeSyntaxParser));
+      
+    } else {
+      sb.append("...");
+    }
+    
+    sb.append(' ');
+    
+    if (upper != null) {
+      sb.append(upper.toQueryString(escapeSyntaxParser));
+      
+    } else {
+      sb.append("...");
+    }
+    
+    if (upperInclusive) {
+      sb.append(']');
+      
+    } else {
+      sb.append('}');
+    }
+    
+    return sb.toString();
+    
+  }
+  
+}

Added: lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/standard/nodes/NumericQueryNode.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/standard/nodes/NumericQueryNode.java?rev=1159411&view=auto
==============================================================================
--- lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/standard/nodes/NumericQueryNode.java (added)
+++ lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/standard/nodes/NumericQueryNode.java Thu Aug 18 20:44:40 2011
@@ -0,0 +1,147 @@
+package org.apache.lucene.queryParser.standard.nodes;
+
+/**
+ * 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.text.NumberFormat;
+import java.util.Locale;
+
+import org.apache.lucene.queryParser.core.nodes.FieldValuePairQueryNode;
+import org.apache.lucene.queryParser.core.nodes.QueryNodeImpl;
+import org.apache.lucene.queryParser.core.parser.EscapeQuerySyntax;
+import org.apache.lucene.queryParser.core.parser.EscapeQuerySyntax.Type;
+
+/**
+ * This query node represents a field query that holds a numeric value. It is
+ * similar to {@link FieldQueryNode}, however the {@link #getValue()} returns a
+ * {@link Number}.
+ * 
+ * @see NumericConfig
+ */
+public class NumericQueryNode extends QueryNodeImpl implements
+    FieldValuePairQueryNode<Number> {
+  
+  private static final long serialVersionUID = -1969102979874574778L;
+
+  private NumberFormat numberFormat;
+  
+  private CharSequence field;
+  
+  private Number value;
+  
+  /**
+   * Creates a {@link NumericQueryNode} object using the given field,
+   * {@link Number} value and {@link NumberFormat} used to convert the value to
+   * {@link String}.
+   * 
+   * @param field the field associated with this query node
+   * @param value the value hold by this node
+   * @param numberFormat the {@link NumberFormat} used to convert the value to {@link String}
+   */
+  public NumericQueryNode(CharSequence field, Number value,
+      NumberFormat numberFormat) {
+    
+    super();
+    
+    setNumberFormat(numberFormat);
+    setField(field);
+    setValue(value);
+    
+  }
+  
+  /**
+   * Returns the field associated with this node.
+   * 
+   * @return the field associated with this node
+   */
+  public CharSequence getField() {
+    return this.field;
+  }
+  
+  /**
+   * Sets the field associated with this node.
+   * 
+   * @param fieldName the field associated with this node
+   */
+  public void setField(CharSequence fieldName) {
+    this.field = fieldName;
+  }
+  
+  /**
+   * This method is used to get the value converted to {@link String} and
+   * escaped using the given {@link EscapeQuerySyntax}.
+   * 
+   * @param escaper the {@link EscapeQuerySyntax} used to escape the value {@link String}
+   * 
+   * @return the value converte to {@link String} and escaped
+   */
+  protected CharSequence getTermEscaped(EscapeQuerySyntax escaper) {
+    return escaper.escape(NumberFormat.getNumberInstance().format(this.value),
+        Locale.ENGLISH, Type.NORMAL);
+  }
+  
+  public CharSequence toQueryString(EscapeQuerySyntax escapeSyntaxParser) {
+    if (isDefaultField(this.field)) {
+      return getTermEscaped(escapeSyntaxParser);
+    } else {
+      return this.field + ":" + getTermEscaped(escapeSyntaxParser);
+    }
+  }
+  
+  /**
+   * Sets the {@link NumberFormat} used to convert the value to {@link String}.
+   * 
+   * @param format the {@link NumberFormat} used to convert the value to {@link String}
+   */
+  public void setNumberFormat(NumberFormat format) {
+    this.numberFormat = format;
+  }
+  
+  /**
+   * Returns the {@link NumberFormat} used to convert the value to {@link String}.
+   * 
+   * @return the {@link NumberFormat} used to convert the value to {@link String}
+   */
+  public NumberFormat getNumberFormat() {
+    return this.numberFormat;
+  }
+  
+  /**
+   * Returns the numeric value as {@link Number}.
+   * 
+   * @return the numeric value
+   */
+  public Number getValue() {
+    return value;
+  }
+  
+  /**
+   * Sets the numeric value.
+   * 
+   * @param value the numeric value
+   */
+  public void setValue(Number value) {
+    this.value = value;
+  }
+  
+  @Override
+  public String toString() {
+    return "<numeric field='" + this.field + "' number='"
+        + numberFormat.format(value) + "'/>";
+  }
+  
+}

Added: lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/standard/nodes/NumericRangeQueryNode.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/standard/nodes/NumericRangeQueryNode.java?rev=1159411&view=auto
==============================================================================
--- lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/standard/nodes/NumericRangeQueryNode.java (added)
+++ lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/standard/nodes/NumericRangeQueryNode.java Thu Aug 18 20:44:40 2011
@@ -0,0 +1,151 @@
+package org.apache.lucene.queryParser.standard.nodes;
+
+import org.apache.lucene.document.NumericField;
+import org.apache.lucene.messages.MessageImpl;
+import org.apache.lucene.queryParser.core.QueryNodeException;
+import org.apache.lucene.queryParser.core.messages.QueryParserMessages;
+import org.apache.lucene.queryParser.standard.config.NumericConfig;
+
+/**
+ * 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.
+ */
+
+/**
+ * This query node represents a range query composed by {@link NumericQueryNode}
+ * bounds, which means the bound values are {@link Number}s.
+ * 
+ * @see NumericQueryNode
+ * @see AbstractRangeQueryNode
+ */
+public class NumericRangeQueryNode extends
+    AbstractRangeQueryNode<NumericQueryNode> {
+  
+  public NumericConfig numericConfig; 
+  
+  /**
+   * Constructs a {@link NumericRangeQueryNode} object using the given
+   * {@link NumericQueryNode} as its bounds and {@link NumericConfig}.
+   * 
+   * @param lower the lower bound
+   * @param upper the upper bound
+   * @param lowerInclusive <code>true</code> if the lower bound is inclusive, otherwise, <code>false</code>
+   * @param upperInclusive <code>true</code> if the upper bound is inclusive, otherwise, <code>false</code>
+   * @param numericConfig the {@link NumericConfig} that represents associated with the upper and lower bounds
+   * 
+   * @see #setBounds(NumericQueryNode, NumericQueryNode, boolean, boolean, NumericConfig)
+   */
+  public NumericRangeQueryNode(NumericQueryNode lower, NumericQueryNode upper,
+      boolean lowerInclusive, boolean upperInclusive, NumericConfig numericConfig) throws QueryNodeException {
+    setBounds(lower, upper, lowerInclusive, upperInclusive, numericConfig);
+  }
+  
+  private static NumericField.DataType getNumericDataType(Number number) throws QueryNodeException {
+    
+    if (number instanceof Long) {
+      return NumericField.DataType.LONG;
+    } else if (number instanceof Integer) {
+      return NumericField.DataType.INT;
+    } else if (number instanceof Double) {
+      return NumericField.DataType.DOUBLE;
+    } else if (number instanceof Float) {
+      return NumericField.DataType.FLOAT;
+    } else {
+      throw new QueryNodeException(
+          new MessageImpl(
+              QueryParserMessages.NUMBER_CLASS_NOT_SUPPORTED_BY_NUMERIC_RANGE_QUERY,
+              number.getClass()));
+    }
+    
+  }
+  
+  /**
+   * Sets the upper and lower bounds of this range query node and the
+   * {@link NumericConfig} associated with these bounds.
+   * 
+   * @param lower the lower bound
+   * @param upper the upper bound
+   * @param lowerInclusive <code>true</code> if the lower bound is inclusive, otherwise, <code>false</code>
+   * @param upperInclusive <code>true</code> if the upper bound is inclusive, otherwise, <code>false</code>
+   * @param numericConfig the {@link NumericConfig} that represents associated with the upper and lower bounds
+   * 
+   */
+  public void setBounds(NumericQueryNode lower, NumericQueryNode upper,
+      boolean lowerInclusive, boolean upperInclusive, NumericConfig numericConfig) throws QueryNodeException {
+    
+    if (numericConfig == null) {
+      throw new IllegalArgumentException("numericConfig cannot be null!");
+    }
+    
+    NumericField.DataType lowerNumberType, upperNumberType;
+    
+    if (lower != null && lower.getValue() != null) {
+      lowerNumberType = getNumericDataType(lower.getValue());
+    } else {
+      lowerNumberType = null;
+    }
+    
+    if (upper != null && upper.getValue() != null) {
+      upperNumberType = getNumericDataType(upper.getValue());
+    } else {
+      upperNumberType = null;
+    }
+    
+    if (lowerNumberType != null
+        && !lowerNumberType.equals(numericConfig.getType())) {
+      throw new IllegalArgumentException(
+          "lower value's type should be the same as numericConfig type: "
+              + lowerNumberType + " != " + numericConfig.getType());
+    }
+    
+    if (upperNumberType != null
+        && !upperNumberType.equals(numericConfig.getType())) {
+      throw new IllegalArgumentException(
+          "upper value's type should be the same as numericConfig type: "
+              + upperNumberType + " != " + numericConfig.getType());
+    }
+    
+    super.setBounds(lower, upper, lowerInclusive, upperInclusive);
+    this.numericConfig = numericConfig;
+    
+  }
+  
+  /**
+   * Returns the {@link NumericConfig} associated with the lower and upper bounds.
+   * 
+   * @return the {@link NumericConfig} associated with the lower and upper bounds
+   */
+  public NumericConfig getNumericConfig() {
+    return this.numericConfig;
+  }
+  
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("<numericRange lowerInclusive='");
+    
+    sb.append(isLowerInclusive()).append("' upperInclusive='").append(
+        isUpperInclusive()).append(
+        "' precisionStep='" + numericConfig.getPrecisionStep()).append(
+        "' type='" + numericConfig.getType()).append("'>\n");
+    
+    sb.append(getLowerBound()).append('\n');
+    sb.append(getUpperBound()).append('\n');
+    sb.append("</numericRange>");
+    
+    return sb.toString();
+    
+  }
+  
+}

Modified: lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/standard/nodes/RangeQueryNode.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/standard/nodes/RangeQueryNode.java?rev=1159411&r1=1159410&r2=1159411&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/standard/nodes/RangeQueryNode.java (original)
+++ lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/standard/nodes/RangeQueryNode.java Thu Aug 18 20:44:40 2011
@@ -19,8 +19,9 @@ package org.apache.lucene.queryParser.st
 
 import java.text.Collator;
 
+import org.apache.lucene.queryParser.core.nodes.FieldQueryNode;
 import org.apache.lucene.queryParser.core.nodes.ParametricQueryNode;
-import org.apache.lucene.queryParser.core.nodes.ParametricRangeQueryNode;
+import org.apache.lucene.queryParser.core.nodes.ParametricQueryNode.CompareOperator;
 import org.apache.lucene.queryParser.standard.config.StandardQueryConfigHandler.ConfigurationKeys;
 import org.apache.lucene.queryParser.standard.processors.ParametricRangeQueryNodeProcessor;
 
@@ -31,8 +32,12 @@ import org.apache.lucene.queryParser.sta
  * @see ParametricRangeQueryNodeProcessor
  * @see ConfigurationKeys#RANGE_COLLATOR
  * @see org.apache.lucene.search.TermRangeQuery
+ * 
+ * @deprecated this class will be removed in future, {@link TermRangeQueryNode} should
+ * be used instead
  */
-public class RangeQueryNode extends ParametricRangeQueryNode {
+@Deprecated
+public class RangeQueryNode extends TermRangeQueryNode {
 
   private static final long serialVersionUID = 7400866652044314657L;
 
@@ -42,13 +47,61 @@ public class RangeQueryNode extends Para
    * @param lower
    * @param upper
    */
-  public RangeQueryNode(ParametricQueryNode lower, ParametricQueryNode upper, Collator collator) {
-    super(lower, upper);
-
+  public RangeQueryNode(ParametricQueryNode lower, ParametricQueryNode upper,
+      Collator collator) {
+    
+    super(lower, upper, lower.getOperator() == CompareOperator.LE, upper
+        .getOperator() == CompareOperator.GE);
+    
     this.collator = collator;
-
+    
   }
-
+  
+  @Override
+  public ParametricQueryNode getLowerBound() {
+    return (ParametricQueryNode) super.getLowerBound();
+  }
+  
+  @Override
+  public ParametricQueryNode getUpperBound() {
+    return (ParametricQueryNode) super.getUpperBound();
+  }
+  
+  /**
+   * Sets lower and upper bounds. The method signature expects
+   * {@link FieldQueryNode} objects as lower and upper, however,
+   * an {@link IllegalArgumentException} will be thrown at runtime
+   * if a non {@link ParametricQueryNode} is passed as lower and upper.
+   * 
+   * @param lower a {@link ParametricQueryNode} object
+   * @param upper a {@link ParametricQueryNode} object
+   * @param lowerInclusive <code>true</code> if lower bound is inclusive, otherwise, <code>false</code>
+   * @param upperInclusive <code>true</code> if upper bound is inclusive, otherwise, <code>false</code>
+   * 
+   * @throws IllegalArgumentException if lower or upper are not instance of {@link ParametricQueryNode}
+   * 
+   * @see TermRangeQueryNode#setBounds(FieldQueryNode, FieldQueryNode, boolean, boolean)
+   */
+  @Override
+  public void setBounds(FieldQueryNode lower, FieldQueryNode upper,
+      boolean lowerInclusive, boolean upperInclusive) {
+    
+    if (lower != null && !(lower instanceof ParametricQueryNode)) {
+      throw new IllegalArgumentException("lower should be an instance of "
+          + ParametricQueryNode.class.getCanonicalName() + ", but found "
+          + lower.getClass().getCanonicalName());
+    }
+    
+    if (upper != null && !(upper instanceof ParametricQueryNode)) {
+      throw new IllegalArgumentException("upper should be an instance of "
+          + ParametricQueryNode.class.getCanonicalName() + ", but found "
+          + lower.getClass().getCanonicalName());
+    }
+    
+    super.setBounds(lower, upper, lowerInclusive, upperInclusive);
+    
+  }
+  
   @Override
   public String toString() {
     StringBuilder sb = new StringBuilder("<range>\n\t");
@@ -66,5 +119,5 @@ public class RangeQueryNode extends Para
   public Collator getCollator() {
     return this.collator;
   }
-
+  
 }

Added: lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/standard/nodes/TermRangeQueryNode.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/standard/nodes/TermRangeQueryNode.java?rev=1159411&view=auto
==============================================================================
--- lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/standard/nodes/TermRangeQueryNode.java (added)
+++ lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/standard/nodes/TermRangeQueryNode.java Thu Aug 18 20:44:40 2011
@@ -0,0 +1,53 @@
+package org.apache.lucene.queryParser.standard.nodes;
+
+/**
+ * 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 org.apache.lucene.queryParser.core.nodes.FieldQueryNode;
+
+/**
+ * This query node represents a range query composed by {@link FieldQueryNode}
+* bounds, which means the bound values are strings.
+ * 
+ * @see FieldQueryNode
+ * @see AbstractRangeQueryNode
+ */
+public class TermRangeQueryNode extends AbstractRangeQueryNode<FieldQueryNode> {
+  
+  private static final long serialVersionUID = 5470829448078569869L;
+  
+  /**
+   * Constructs a {@link TermRangeQueryNode} object using the given
+   * {@link FieldQueryNode} as its bounds.
+   * 
+   * @param lower
+   *          the lower bound
+   * @param upper
+   *          the upper bound
+   * @param lowerInclusive
+   *          <code>true</code> if the lower bound is inclusive, otherwise,
+   *          <code>false</code>
+   * @param upperInclusive
+   *          <code>true</code> if the upper bound is inclusive, otherwise,
+   *          <code>false</code>
+   */
+  public TermRangeQueryNode(FieldQueryNode lower,
+      FieldQueryNode upper, boolean lowerInclusive, boolean upperInclusive) {
+    setBounds(lower, upper, lowerInclusive, upperInclusive);
+  }
+  
+}

Modified: lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/standard/processors/MultiTermRewriteMethodProcessor.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/standard/processors/MultiTermRewriteMethodProcessor.java?rev=1159411&r1=1159410&r2=1159411&view=diff
==============================================================================
--- lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/standard/processors/MultiTermRewriteMethodProcessor.java (original)
+++ lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/standard/processors/MultiTermRewriteMethodProcessor.java Thu Aug 18 20:44:40 2011
@@ -23,6 +23,7 @@ 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.standard.config.StandardQueryConfigHandler.ConfigurationKeys;
+import org.apache.lucene.queryParser.standard.nodes.AbstractRangeQueryNode;
 import org.apache.lucene.queryParser.standard.nodes.WildcardQueryNode;
 import org.apache.lucene.search.MultiTermQuery;
 
@@ -42,7 +43,7 @@ public class MultiTermRewriteMethodProce
     // set setMultiTermRewriteMethod for WildcardQueryNode and
     // PrefixWildcardQueryNode
     if (node instanceof WildcardQueryNode
-        || node instanceof ParametricRangeQueryNode) {
+        || node instanceof AbstractRangeQueryNode) {
 
 
       // read the attribute value and use a TAG to take the value to the Builder

Added: lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/standard/processors/NumericQueryNodeProcessor.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/standard/processors/NumericQueryNodeProcessor.java?rev=1159411&view=auto
==============================================================================
--- lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/standard/processors/NumericQueryNodeProcessor.java (added)
+++ lucene/dev/branches/branch_3x/lucene/contrib/queryparser/src/java/org/apache/lucene/queryParser/standard/processors/NumericQueryNodeProcessor.java Thu Aug 18 20:44:40 2011
@@ -0,0 +1,145 @@
+package org.apache.lucene.queryParser.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.text.NumberFormat;
+import java.text.ParseException;
+import java.util.List;
+
+import org.apache.lucene.messages.MessageImpl;
+import org.apache.lucene.queryParser.core.QueryNodeException;
+import org.apache.lucene.queryParser.core.QueryNodeParseException;
+import org.apache.lucene.queryParser.core.config.FieldConfig;
+import org.apache.lucene.queryParser.core.config.QueryConfigHandler;
+import org.apache.lucene.queryParser.core.messages.QueryParserMessages;
+import org.apache.lucene.queryParser.core.nodes.FieldQueryNode;
+import org.apache.lucene.queryParser.core.nodes.ParametricQueryNode;
+import org.apache.lucene.queryParser.core.nodes.QueryNode;
+import org.apache.lucene.queryParser.core.processors.QueryNodeProcessorImpl;
+import org.apache.lucene.queryParser.standard.config.NumericConfig;
+import org.apache.lucene.queryParser.standard.config.StandardQueryConfigHandler.ConfigurationKeys;
+import org.apache.lucene.queryParser.standard.nodes.NumericQueryNode;
+import org.apache.lucene.queryParser.standard.nodes.NumericRangeQueryNode;
+
+/**
+ * This processor is used to convert {@link FieldQueryNode}s to
+ * {@link NumericRangeQueryNode}s. It looks for
+ * {@link ConfigurationKeys#NUMERIC_CONFIG} set in the {@link FieldConfig} of
+ * every {@link FieldQueryNode} found. If
+ * {@link ConfigurationKeys#NUMERIC_CONFIG} is found, it considers that
+ * {@link FieldQueryNode} to be a numeric query and convert it to
+ * {@link NumericRangeQueryNode} with upper and lower inclusive and lower and
+ * upper equals to the value represented by the {@link FieldQueryNode} converted
+ * to {@link Number}. It means that <b>field:1</b> is converted to <b>field:[1 TO
+ * 1]</b>. <br/>
+ * <br/>
+ * Note that {@link ParametricQueryNode}s are ignored, even being a
+ * {@link FieldQueryNode}.
+ * 
+ * @see ConfigurationKeys#NUMERIC_CONFIG
+ * @see FieldQueryNode
+ * @see NumericConfig
+ * @see NumericQueryNode
+ */
+public class NumericQueryNodeProcessor extends QueryNodeProcessorImpl {
+  
+  /**
+   * Constructs a {@link NumericQueryNodeProcessor} object.
+   */
+  public NumericQueryNodeProcessor() {
+  // empty constructor
+  }
+  
+  @Override
+  protected QueryNode postProcessNode(QueryNode node) throws QueryNodeException {
+    
+    if (node instanceof FieldQueryNode
+        && !(node instanceof ParametricQueryNode)) {
+      
+      QueryConfigHandler config = getQueryConfigHandler();
+      
+      if (config != null) {
+        FieldQueryNode fieldNode = (FieldQueryNode) node;
+        FieldConfig fieldConfig = config.getFieldConfig(fieldNode
+            .getFieldAsString());
+        
+        if (fieldConfig != null) {
+          NumericConfig numericConfig = fieldConfig
+              .get(ConfigurationKeys.NUMERIC_CONFIG);
+          
+          if (numericConfig != null) {
+            
+            NumberFormat numberFormat = numericConfig.getNumberFormat();
+            Number number;
+            
+            try {
+              number = numberFormat.parse(fieldNode.getTextAsString());
+              
+            } catch (ParseException e) {
+              throw new QueryNodeParseException(new MessageImpl(
+                  QueryParserMessages.COULD_NOT_PARSE_NUMBER, fieldNode
+                      .getTextAsString(), numberFormat.getClass()
+                      .getCanonicalName()), e);
+            }
+            
+            switch (numericConfig.getType()) {
+              case LONG:
+                number = number.longValue();
+                break;
+              case INT:
+                number = number.intValue();
+                break;
+              case DOUBLE:
+                number = number.doubleValue();
+                break;
+              case FLOAT:
+                number = number.floatValue();
+            }
+            
+            NumericQueryNode lowerNode = new NumericQueryNode(fieldNode
+                .getField(), number, numberFormat);
+            NumericQueryNode upperNode = new NumericQueryNode(fieldNode
+                .getField(), number, numberFormat);
+            
+            return new NumericRangeQueryNode(lowerNode, upperNode, true, true,
+                numericConfig);
+            
+          }
+          
+        }
+        
+      }
+      
+    }
+    
+    return node;
+    
+  }
+  
+  @Override
+  protected QueryNode preProcessNode(QueryNode node) throws QueryNodeException {
+    return node;
+  }
+  
+  @Override
+  protected List<QueryNode> setChildrenOrder(List<QueryNode> children)
+      throws QueryNodeException {
+    return children;
+  }
+  
+}