You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ho...@apache.org on 2016/03/12 01:27:04 UTC

[23/50] [abbrv] lucene-solr git commit: LUCENE-7089, LUCENE-7075: add points to flexible queryparser to replace legacy numerics support

LUCENE-7089, LUCENE-7075: add points to flexible queryparser to replace legacy numerics support


Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/89cc676f
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/89cc676f
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/89cc676f

Branch: refs/heads/jira/SOLR-445
Commit: 89cc676f2bf560091bd24db544faa16946654164
Parents: f24810b
Author: Robert Muir <rm...@apache.org>
Authored: Wed Mar 9 21:05:26 2016 -0500
Committer: Robert Muir <rm...@apache.org>
Committed: Wed Mar 9 21:06:41 2016 -0500

----------------------------------------------------------------------
 .../flexible/core/nodes/package-info.java       |   4 +-
 .../flexible/standard/StandardQueryParser.java  |  29 +-
 .../LegacyNumericRangeQueryNodeBuilder.java     |  93 ++++
 .../builders/NumericRangeQueryNodeBuilder.java  |  91 ----
 .../builders/PointRangeQueryNodeBuilder.java    | 137 +++++
 .../builders/StandardQueryTreeBuilder.java      |  12 +-
 .../standard/config/LegacyNumericConfig.java    | 166 ++++++
 .../LegacyNumericFieldConfigListener.java       |  75 +++
 .../flexible/standard/config/NumericConfig.java | 164 ------
 .../config/NumericFieldConfigListener.java      |  73 ---
 .../flexible/standard/config/PointsConfig.java  | 124 +++++
 .../standard/config/PointsConfigListener.java   |  65 +++
 .../config/StandardQueryConfigHandler.java      |  39 +-
 .../standard/nodes/LegacyNumericQueryNode.java  | 153 ++++++
 .../nodes/LegacyNumericRangeQueryNode.java      | 153 ++++++
 .../standard/nodes/NumericQueryNode.java        | 151 ------
 .../standard/nodes/NumericRangeQueryNode.java   | 151 ------
 .../flexible/standard/nodes/PointQueryNode.java | 151 ++++++
 .../standard/nodes/PointRangeQueryNode.java     | 124 +++++
 .../LegacyNumericQueryNodeProcessor.java        | 154 ++++++
 .../LegacyNumericRangeQueryNodeProcessor.java   | 170 ++++++
 .../processors/NumericQueryNodeProcessor.java   | 152 ------
 .../NumericRangeQueryNodeProcessor.java         | 168 ------
 .../processors/PointQueryNodeProcessor.java     | 136 +++++
 .../PointRangeQueryNodeProcessor.java           | 148 +++++
 .../StandardQueryNodeProcessorPipeline.java     |   6 +-
 .../standard/TestLegacyNumericQueryParser.java  | 535 +++++++++++++++++++
 .../standard/TestNumericQueryParser.java        | 535 -------------------
 .../flexible/standard/TestPointQueryParser.java |  82 +++
 lucene/tools/junit4/cached-timehints.txt        |   2 +-
 30 files changed, 2535 insertions(+), 1508 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/89cc676f/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/core/nodes/package-info.java
----------------------------------------------------------------------
diff --git a/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/core/nodes/package-info.java b/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/core/nodes/package-info.java
index c9d55ea..23c72a1 100644
--- a/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/core/nodes/package-info.java
+++ b/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/core/nodes/package-info.java
@@ -52,14 +52,14 @@
  * <li>FuzzyQueryNode - fuzzy node</li>
  * <li>TermRangeQueryNode - 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>LegacyNumericRangeQueryNode - used for numeric range search</li>
  * <li>TokenizedPhraseQueryNode - used by tokenizers/lemmatizers/analyzers for phrases/autophrases</li>
  * </ul>
  * <p>
  * Leaf Nodes:
  * <ul>
  * <li>FieldQueryNode - field/value node</li>
- * <li>NumericQueryNode - used for numeric search</li>
+ * <li>LegacyNumericQueryNode - used for numeric search</li>
  * <li>PathQueryNode - {@link org.apache.lucene.queryparser.flexible.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>PrefixWildcardQueryNode - non-phrase wildcard query</li>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/89cc676f/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/StandardQueryParser.java
----------------------------------------------------------------------
diff --git a/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/StandardQueryParser.java b/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/StandardQueryParser.java
index ada65a4..2774cf0 100644
--- a/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/StandardQueryParser.java
+++ b/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/StandardQueryParser.java
@@ -29,7 +29,8 @@ import org.apache.lucene.queryparser.flexible.core.QueryParserHelper;
 import org.apache.lucene.queryparser.flexible.core.config.QueryConfigHandler;
 import org.apache.lucene.queryparser.flexible.standard.builders.StandardQueryTreeBuilder;
 import org.apache.lucene.queryparser.flexible.standard.config.FuzzyConfig;
-import org.apache.lucene.queryparser.flexible.standard.config.NumericConfig;
+import org.apache.lucene.queryparser.flexible.standard.config.LegacyNumericConfig;
+import org.apache.lucene.queryparser.flexible.standard.config.PointsConfig;
 import org.apache.lucene.queryparser.flexible.standard.config.StandardQueryConfigHandler;
 import org.apache.lucene.queryparser.flexible.standard.config.StandardQueryConfigHandler.ConfigurationKeys;
 import org.apache.lucene.queryparser.flexible.standard.config.StandardQueryConfigHandler.Operator;
@@ -322,12 +323,30 @@ public class StandardQueryParser extends QueryParserHelper implements CommonQuer
     
   }
   
-  public void setNumericConfigMap(Map<String,NumericConfig> numericConfigMap) {
-    getQueryConfigHandler().set(ConfigurationKeys.NUMERIC_CONFIG_MAP, numericConfigMap);
+  /**
+   * Sets field configuration for legacy numeric fields
+   * @deprecated Index with points instead and use {@link #setPointsConfigMap(Map)}
+   */
+  @Deprecated
+  public void setLegacyNumericConfigMap(Map<String,LegacyNumericConfig> legacyNumericConfigMap) {
+    getQueryConfigHandler().set(ConfigurationKeys.LEGACY_NUMERIC_CONFIG_MAP, legacyNumericConfigMap);
+  }
+  
+  /**
+   * Gets field configuration for legacy numeric fields
+   * @deprecated Index with points instead and use {@link #getPointsConfigMap()}
+   */
+  @Deprecated
+  public Map<String,LegacyNumericConfig> getLegacyNumericConfigMap() {
+    return getQueryConfigHandler().get(ConfigurationKeys.LEGACY_NUMERIC_CONFIG_MAP);
+  }
+  
+  public void setPointsConfigMap(Map<String,PointsConfig> pointsConfigMap) {
+    getQueryConfigHandler().set(ConfigurationKeys.POINTS_CONFIG_MAP, pointsConfigMap);
   }
   
-  public Map<String,NumericConfig> getNumericConfigMap() {
-    return getQueryConfigHandler().get(ConfigurationKeys.NUMERIC_CONFIG_MAP);
+  public Map<String,PointsConfig> getPointsConfigMap() {
+    return getQueryConfigHandler().get(ConfigurationKeys.POINTS_CONFIG_MAP);
   }
   
   /**

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/89cc676f/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/builders/LegacyNumericRangeQueryNodeBuilder.java
----------------------------------------------------------------------
diff --git a/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/builders/LegacyNumericRangeQueryNodeBuilder.java b/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/builders/LegacyNumericRangeQueryNodeBuilder.java
new file mode 100644
index 0000000..8ae7d5e
--- /dev/null
+++ b/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/builders/LegacyNumericRangeQueryNodeBuilder.java
@@ -0,0 +1,93 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.queryparser.flexible.standard.builders;
+
+import org.apache.lucene.document.FieldType;
+import org.apache.lucene.queryparser.flexible.core.QueryNodeException;
+import org.apache.lucene.queryparser.flexible.core.messages.QueryParserMessages;
+import org.apache.lucene.queryparser.flexible.core.nodes.QueryNode;
+import org.apache.lucene.queryparser.flexible.core.util.StringUtils;
+import org.apache.lucene.queryparser.flexible.messages.MessageImpl;
+import org.apache.lucene.queryparser.flexible.standard.config.LegacyNumericConfig;
+import org.apache.lucene.queryparser.flexible.standard.nodes.LegacyNumericQueryNode;
+import org.apache.lucene.queryparser.flexible.standard.nodes.LegacyNumericRangeQueryNode;
+import org.apache.lucene.search.LegacyNumericRangeQuery;
+
+/**
+ * Builds {@link org.apache.lucene.search.LegacyNumericRangeQuery}s out of {@link LegacyNumericRangeQueryNode}s.
+ *
+ * @see org.apache.lucene.search.LegacyNumericRangeQuery
+ * @see LegacyNumericRangeQueryNode
+ * @deprecated Index with points and use {@link PointRangeQueryNodeBuilder} instead.
+ */
+@Deprecated
+public class LegacyNumericRangeQueryNodeBuilder implements StandardQueryBuilder {
+  
+  /**
+   * Constructs a {@link LegacyNumericRangeQueryNodeBuilder} object.
+   */
+  public LegacyNumericRangeQueryNodeBuilder() {
+  // empty constructor
+  }
+  
+  @Override
+  public LegacyNumericRangeQuery<? extends Number> build(QueryNode queryNode)
+      throws QueryNodeException {
+    LegacyNumericRangeQueryNode numericRangeNode = (LegacyNumericRangeQueryNode) queryNode;
+    
+    LegacyNumericQueryNode lowerNumericNode = numericRangeNode.getLowerBound();
+    LegacyNumericQueryNode upperNumericNode = numericRangeNode.getUpperBound();
+    
+    Number lowerNumber = lowerNumericNode.getValue();
+    Number upperNumber = upperNumericNode.getValue();
+    
+    LegacyNumericConfig numericConfig = numericRangeNode.getNumericConfig();
+    FieldType.LegacyNumericType 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 LegacyNumericRangeQuery.newLongRange(field, precisionStep,
+            (Long) lowerNumber, (Long) upperNumber, minInclusive, maxInclusive);
+      
+      case INT:
+        return LegacyNumericRangeQuery.newIntRange(field, precisionStep,
+            (Integer) lowerNumber, (Integer) upperNumber, minInclusive,
+            maxInclusive);
+      
+      case FLOAT:
+        return LegacyNumericRangeQuery.newFloatRange(field, precisionStep,
+            (Float) lowerNumber, (Float) upperNumber, minInclusive,
+            maxInclusive);
+      
+      case DOUBLE:
+        return LegacyNumericRangeQuery.newDoubleRange(field, precisionStep,
+            (Double) lowerNumber, (Double) upperNumber, minInclusive,
+            maxInclusive);
+        
+        default :
+          throw new QueryNodeException(new MessageImpl(
+            QueryParserMessages.UNSUPPORTED_NUMERIC_DATA_TYPE, numberType));
+        
+    }
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/89cc676f/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/builders/NumericRangeQueryNodeBuilder.java
----------------------------------------------------------------------
diff --git a/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/builders/NumericRangeQueryNodeBuilder.java b/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/builders/NumericRangeQueryNodeBuilder.java
deleted file mode 100644
index 6c8790f..0000000
--- a/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/builders/NumericRangeQueryNodeBuilder.java
+++ /dev/null
@@ -1,91 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.lucene.queryparser.flexible.standard.builders;
-
-import org.apache.lucene.document.FieldType;
-import org.apache.lucene.queryparser.flexible.core.QueryNodeException;
-import org.apache.lucene.queryparser.flexible.core.messages.QueryParserMessages;
-import org.apache.lucene.queryparser.flexible.core.nodes.QueryNode;
-import org.apache.lucene.queryparser.flexible.core.util.StringUtils;
-import org.apache.lucene.queryparser.flexible.messages.MessageImpl;
-import org.apache.lucene.queryparser.flexible.standard.config.NumericConfig;
-import org.apache.lucene.queryparser.flexible.standard.nodes.NumericQueryNode;
-import org.apache.lucene.queryparser.flexible.standard.nodes.NumericRangeQueryNode;
-import org.apache.lucene.search.LegacyNumericRangeQuery;
-
-/**
- * Builds {@link org.apache.lucene.search.LegacyNumericRangeQuery}s out of {@link NumericRangeQueryNode}s.
- *
- * @see org.apache.lucene.search.LegacyNumericRangeQuery
- * @see NumericRangeQueryNode
- */
-public class NumericRangeQueryNodeBuilder implements StandardQueryBuilder {
-  
-  /**
-   * Constructs a {@link NumericRangeQueryNodeBuilder} object.
-   */
-  public NumericRangeQueryNodeBuilder() {
-  // empty constructor
-  }
-  
-  @Override
-  public LegacyNumericRangeQuery<? extends Number> build(QueryNode queryNode)
-      throws QueryNodeException {
-    NumericRangeQueryNode numericRangeNode = (NumericRangeQueryNode) queryNode;
-    
-    NumericQueryNode lowerNumericNode = numericRangeNode.getLowerBound();
-    NumericQueryNode upperNumericNode = numericRangeNode.getUpperBound();
-    
-    Number lowerNumber = lowerNumericNode.getValue();
-    Number upperNumber = upperNumericNode.getValue();
-    
-    NumericConfig numericConfig = numericRangeNode.getNumericConfig();
-    FieldType.LegacyNumericType 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 LegacyNumericRangeQuery.newLongRange(field, precisionStep,
-            (Long) lowerNumber, (Long) upperNumber, minInclusive, maxInclusive);
-      
-      case INT:
-        return LegacyNumericRangeQuery.newIntRange(field, precisionStep,
-            (Integer) lowerNumber, (Integer) upperNumber, minInclusive,
-            maxInclusive);
-      
-      case FLOAT:
-        return LegacyNumericRangeQuery.newFloatRange(field, precisionStep,
-            (Float) lowerNumber, (Float) upperNumber, minInclusive,
-            maxInclusive);
-      
-      case DOUBLE:
-        return LegacyNumericRangeQuery.newDoubleRange(field, precisionStep,
-            (Double) lowerNumber, (Double) upperNumber, minInclusive,
-            maxInclusive);
-        
-        default :
-          throw new QueryNodeException(new MessageImpl(
-            QueryParserMessages.UNSUPPORTED_NUMERIC_DATA_TYPE, numberType));
-        
-    }
-  }
-  
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/89cc676f/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/builders/PointRangeQueryNodeBuilder.java
----------------------------------------------------------------------
diff --git a/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/builders/PointRangeQueryNodeBuilder.java b/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/builders/PointRangeQueryNodeBuilder.java
new file mode 100644
index 0000000..0cce4bf
--- /dev/null
+++ b/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/builders/PointRangeQueryNodeBuilder.java
@@ -0,0 +1,137 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.queryparser.flexible.standard.builders;
+
+import org.apache.lucene.document.DoublePoint;
+import org.apache.lucene.document.FloatPoint;
+import org.apache.lucene.document.IntPoint;
+import org.apache.lucene.document.LongPoint;
+import org.apache.lucene.index.PointValues;
+import org.apache.lucene.queryparser.flexible.core.QueryNodeException;
+import org.apache.lucene.queryparser.flexible.core.messages.QueryParserMessages;
+import org.apache.lucene.queryparser.flexible.core.nodes.QueryNode;
+import org.apache.lucene.queryparser.flexible.core.util.StringUtils;
+import org.apache.lucene.queryparser.flexible.messages.MessageImpl;
+import org.apache.lucene.queryparser.flexible.standard.config.PointsConfig;
+import org.apache.lucene.queryparser.flexible.standard.nodes.PointQueryNode;
+import org.apache.lucene.queryparser.flexible.standard.nodes.PointRangeQueryNode;
+import org.apache.lucene.search.Query;
+
+/**
+ * Builds {@link PointValues} range queries out of {@link PointRangeQueryNode}s.
+ *
+ * @see PointRangeQueryNode
+ */
+public class PointRangeQueryNodeBuilder implements StandardQueryBuilder {
+  
+  /**
+   * Constructs a {@link PointRangeQueryNodeBuilder} object.
+   */
+  public PointRangeQueryNodeBuilder() {
+  // empty constructor
+  }
+  
+  @Override
+  public Query build(QueryNode queryNode) throws QueryNodeException {
+    PointRangeQueryNode numericRangeNode = (PointRangeQueryNode) queryNode;
+    
+    PointQueryNode lowerNumericNode = numericRangeNode.getLowerBound();
+    PointQueryNode upperNumericNode = numericRangeNode.getUpperBound();
+    
+    Number lowerNumber = lowerNumericNode.getValue();
+    Number upperNumber = upperNumericNode.getValue();
+    
+    PointsConfig pointsConfig = numericRangeNode.getPointsConfig();
+    Class<? extends Number> numberType = pointsConfig.getType();
+    String field = StringUtils.toString(numericRangeNode.getField());
+    boolean minInclusive = numericRangeNode.isLowerInclusive();
+    boolean maxInclusive = numericRangeNode.isUpperInclusive();
+    
+    // TODO: push down cleaning up of crazy nulls and inclusive/exclusive elsewhere
+    if (Integer.class.equals(numberType)) {
+      Integer lower = (Integer) lowerNumber;
+      if (lower == null) {
+        lower = Integer.MIN_VALUE;
+      }
+      if (minInclusive == false) {
+        lower = lower + 1;
+      }
+      
+      Integer upper = (Integer) upperNumber;
+      if (upper == null) {
+        upper = Integer.MAX_VALUE;
+      }
+      if (maxInclusive == false) {
+        upper = upper - 1;
+      }
+      return IntPoint.newRangeQuery(field, lower, upper);
+    } else if (Long.class.equals(numberType)) {
+      Long lower = (Long) lowerNumber;
+      if (lower == null) {
+        lower = Long.MIN_VALUE;
+      }
+      if (minInclusive == false) {
+        lower = lower + 1;
+      }
+      
+      Long upper = (Long) upperNumber;
+      if (upper == null) {
+        upper = Long.MAX_VALUE;
+      }
+      if (maxInclusive == false) {
+        upper = upper - 1;
+      }
+      return LongPoint.newRangeQuery(field, lower, upper);
+    } else if (Float.class.equals(numberType)) {
+      Float lower = (Float) lowerNumber;
+      if (lower == null) {
+        lower = Float.NEGATIVE_INFINITY;
+      }
+      if (minInclusive == false) {
+        lower = Math.nextUp(lower);
+      }
+      
+      Float upper = (Float) upperNumber;
+      if (upper == null) {
+        upper = Float.POSITIVE_INFINITY;
+      }
+      if (maxInclusive == false) {
+        upper = Math.nextDown(upper);
+      }
+      return FloatPoint.newRangeQuery(field, lower, upper);
+    } else if (Double.class.equals(numberType)) {
+      Double lower = (Double) lowerNumber;
+      if (lower == null) {
+        lower = Double.NEGATIVE_INFINITY;
+      }
+      if (minInclusive == false) {
+        lower = Math.nextUp(lower);
+      }
+      
+      Double upper = (Double) upperNumber;
+      if (upper == null) {
+        upper = Double.POSITIVE_INFINITY;
+      }
+      if (maxInclusive == false) {
+        upper = Math.nextDown(upper);
+      }
+      return DoublePoint.newRangeQuery(field, lower, upper);
+    } else {
+      throw new QueryNodeException(new MessageImpl(QueryParserMessages.UNSUPPORTED_NUMERIC_DATA_TYPE, numberType));
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/89cc676f/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/builders/StandardQueryTreeBuilder.java
----------------------------------------------------------------------
diff --git a/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/builders/StandardQueryTreeBuilder.java b/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/builders/StandardQueryTreeBuilder.java
index 2d7c643..360f6a7 100644
--- a/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/builders/StandardQueryTreeBuilder.java
+++ b/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/builders/StandardQueryTreeBuilder.java
@@ -30,8 +30,10 @@ import org.apache.lucene.queryparser.flexible.core.nodes.QueryNode;
 import org.apache.lucene.queryparser.flexible.core.nodes.SlopQueryNode;
 import org.apache.lucene.queryparser.flexible.core.nodes.TokenizedPhraseQueryNode;
 import org.apache.lucene.queryparser.flexible.standard.nodes.MultiPhraseQueryNode;
-import org.apache.lucene.queryparser.flexible.standard.nodes.NumericQueryNode;
-import org.apache.lucene.queryparser.flexible.standard.nodes.NumericRangeQueryNode;
+import org.apache.lucene.queryparser.flexible.standard.nodes.PointQueryNode;
+import org.apache.lucene.queryparser.flexible.standard.nodes.PointRangeQueryNode;
+import org.apache.lucene.queryparser.flexible.standard.nodes.LegacyNumericQueryNode;
+import org.apache.lucene.queryparser.flexible.standard.nodes.LegacyNumericRangeQueryNode;
 import org.apache.lucene.queryparser.flexible.standard.nodes.PrefixWildcardQueryNode;
 import org.apache.lucene.queryparser.flexible.standard.nodes.TermRangeQueryNode;
 import org.apache.lucene.queryparser.flexible.standard.nodes.RegexpQueryNode;
@@ -57,8 +59,10 @@ public class StandardQueryTreeBuilder extends QueryTreeBuilder implements
     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(LegacyNumericQueryNode.class, new DummyQueryNodeBuilder());
+    setBuilder(LegacyNumericRangeQueryNode.class, new LegacyNumericRangeQueryNodeBuilder());
+    setBuilder(PointQueryNode.class, new DummyQueryNodeBuilder());
+    setBuilder(PointRangeQueryNode.class, new PointRangeQueryNodeBuilder());
     setBuilder(BoostQueryNode.class, new BoostQueryNodeBuilder());
     setBuilder(ModifierQueryNode.class, new ModifierQueryNodeBuilder());
     setBuilder(WildcardQueryNode.class, new WildcardQueryNodeBuilder());

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/89cc676f/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/config/LegacyNumericConfig.java
----------------------------------------------------------------------
diff --git a/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/config/LegacyNumericConfig.java b/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/config/LegacyNumericConfig.java
new file mode 100644
index 0000000..985f55a
--- /dev/null
+++ b/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/config/LegacyNumericConfig.java
@@ -0,0 +1,166 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.queryparser.flexible.standard.config;
+
+import java.text.NumberFormat;
+import java.util.Objects;
+
+import org.apache.lucene.document.FieldType;
+import org.apache.lucene.document.FieldType.LegacyNumericType;
+
+/**
+ * This class holds the configuration used to parse numeric queries and create
+ * {@link org.apache.lucene.search.LegacyNumericRangeQuery}s.
+ * 
+ * @see org.apache.lucene.search.LegacyNumericRangeQuery
+ * @see NumberFormat
+ * @deprecated Index with Points instead and use {@link PointsConfig}
+ */
+@Deprecated
+public class LegacyNumericConfig {
+  
+  private int precisionStep;
+  
+  private NumberFormat format;
+  
+  private FieldType.LegacyNumericType type;
+  
+  /**
+   * Constructs a {@link LegacyNumericConfig} 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 LegacyNumericConfig#setPrecisionStep(int)
+   * @see LegacyNumericConfig#setNumberFormat(NumberFormat)
+   * @see #setType(org.apache.lucene.document.FieldType.LegacyNumericType)
+   */
+  public LegacyNumericConfig(int precisionStep, NumberFormat format,
+      LegacyNumericType 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 org.apache.lucene.search.LegacyNumericRangeQuery#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 org.apache.lucene.search.LegacyNumericRangeQuery#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 LegacyNumericType 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(LegacyNumericType 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 LegacyNumericConfig) {
+      LegacyNumericConfig other = (LegacyNumericConfig) obj;
+      
+      if (this.precisionStep == other.precisionStep
+          && this.type == other.type
+          && (this.format == other.format || (this.format.equals(other.format)))) {
+        return true;
+      }
+      
+    }
+    
+    return false;
+    
+  }
+  
+  @Override
+  public int hashCode() {
+    return Objects.hash(precisionStep, type, format);
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/89cc676f/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/config/LegacyNumericFieldConfigListener.java
----------------------------------------------------------------------
diff --git a/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/config/LegacyNumericFieldConfigListener.java b/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/config/LegacyNumericFieldConfigListener.java
new file mode 100644
index 0000000..f2d3124
--- /dev/null
+++ b/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/config/LegacyNumericFieldConfigListener.java
@@ -0,0 +1,75 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.queryparser.flexible.standard.config;
+
+import java.util.Map;
+
+import org.apache.lucene.queryparser.flexible.core.config.FieldConfig;
+import org.apache.lucene.queryparser.flexible.core.config.FieldConfigListener;
+import org.apache.lucene.queryparser.flexible.core.config.QueryConfigHandler;
+import org.apache.lucene.queryparser.flexible.standard.config.StandardQueryConfigHandler.ConfigurationKeys;
+
+/**
+ * This listener is used to listen to {@link FieldConfig} requests in
+ * {@link QueryConfigHandler} and add {@link ConfigurationKeys#LEGACY_NUMERIC_CONFIG}
+ * based on the {@link ConfigurationKeys#LEGACY_NUMERIC_CONFIG_MAP} set in the
+ * {@link QueryConfigHandler}.
+ * 
+ * @see LegacyNumericConfig
+ * @see QueryConfigHandler
+ * @see ConfigurationKeys#LEGACY_NUMERIC_CONFIG
+ * @see ConfigurationKeys#LEGACY_NUMERIC_CONFIG_MAP
+ * @deprecated Index with Points instead and use {@link PointsConfigListener}
+ */
+@Deprecated
+public class LegacyNumericFieldConfigListener implements FieldConfigListener {
+  
+  final private QueryConfigHandler config;
+  
+  /**
+   * Constructs a {@link LegacyNumericFieldConfigListener} object using the given {@link QueryConfigHandler}.
+   * 
+   * @param config the {@link QueryConfigHandler} it will listen too
+   */
+  public LegacyNumericFieldConfigListener(QueryConfigHandler config) {
+    
+    if (config == null) {
+      throw new IllegalArgumentException("config cannot be null!");
+    }
+    
+    this.config = config;
+    
+  }
+  
+  @Override
+  public void buildFieldConfig(FieldConfig fieldConfig) {
+    Map<String,LegacyNumericConfig> numericConfigMap = config
+        .get(ConfigurationKeys.LEGACY_NUMERIC_CONFIG_MAP);
+    
+    if (numericConfigMap != null) {
+      LegacyNumericConfig numericConfig = numericConfigMap
+          .get(fieldConfig.getField());
+      
+      if (numericConfig != null) {
+        fieldConfig.set(ConfigurationKeys.LEGACY_NUMERIC_CONFIG, numericConfig);
+      }
+      
+    }
+    
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/89cc676f/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/config/NumericConfig.java
----------------------------------------------------------------------
diff --git a/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/config/NumericConfig.java b/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/config/NumericConfig.java
deleted file mode 100644
index c457a4e..0000000
--- a/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/config/NumericConfig.java
+++ /dev/null
@@ -1,164 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.lucene.queryparser.flexible.standard.config;
-
-import java.text.NumberFormat;
-import java.util.Objects;
-
-import org.apache.lucene.document.FieldType;
-import org.apache.lucene.document.FieldType.LegacyNumericType;
-
-/**
- * This class holds the configuration used to parse numeric queries and create
- * {@link org.apache.lucene.search.LegacyNumericRangeQuery}s.
- * 
- * @see org.apache.lucene.search.LegacyNumericRangeQuery
- * @see NumberFormat
- */
-public class NumericConfig {
-  
-  private int precisionStep;
-  
-  private NumberFormat format;
-  
-  private FieldType.LegacyNumericType 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.FieldType.LegacyNumericType)
-   */
-  public NumericConfig(int precisionStep, NumberFormat format,
-      LegacyNumericType 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 org.apache.lucene.search.LegacyNumericRangeQuery#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 org.apache.lucene.search.LegacyNumericRangeQuery#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 LegacyNumericType 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(LegacyNumericType 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;
-    
-  }
-  
-  @Override
-  public int hashCode() {
-    return Objects.hash(precisionStep, type, format);
-  }
-  
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/89cc676f/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/config/NumericFieldConfigListener.java
----------------------------------------------------------------------
diff --git a/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/config/NumericFieldConfigListener.java b/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/config/NumericFieldConfigListener.java
deleted file mode 100644
index c28cf2c..0000000
--- a/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/config/NumericFieldConfigListener.java
+++ /dev/null
@@ -1,73 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.lucene.queryparser.flexible.standard.config;
-
-import java.util.Map;
-
-import org.apache.lucene.queryparser.flexible.core.config.FieldConfig;
-import org.apache.lucene.queryparser.flexible.core.config.FieldConfigListener;
-import org.apache.lucene.queryparser.flexible.core.config.QueryConfigHandler;
-import org.apache.lucene.queryparser.flexible.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;
-    
-  }
-  
-  @Override
-  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);
-      }
-      
-    }
-    
-  }
-  
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/89cc676f/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/config/PointsConfig.java
----------------------------------------------------------------------
diff --git a/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/config/PointsConfig.java b/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/config/PointsConfig.java
new file mode 100644
index 0000000..db59b48
--- /dev/null
+++ b/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/config/PointsConfig.java
@@ -0,0 +1,124 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.queryparser.flexible.standard.config;
+
+import java.text.NumberFormat;
+
+import org.apache.lucene.index.PointValues;
+
+/**
+ * This class holds the configuration used to parse numeric queries and create
+ * {@link PointValues} queries.
+ * 
+ * @see PointValues
+ * @see NumberFormat
+ */
+public class PointsConfig {
+    
+  private NumberFormat format;
+  
+  private Class<? extends Number> type;
+  
+  /**
+   * Constructs a {@link PointsConfig} object.
+   * 
+   * @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 PointsConfig#setNumberFormat(NumberFormat)
+   */
+  public PointsConfig(NumberFormat format, Class<? extends Number> type) {
+    setNumberFormat(format);
+    setType(type);    
+  }
+  
+  /**
+   * 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 Class<? extends Number> 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(Class<? extends Number> type) {
+    if (type == null) {
+      throw new IllegalArgumentException("type cannot be null!");
+    }
+    if (Integer.class.equals(type) == false &&
+        Long.class.equals(type) == false &&
+        Float.class.equals(type) == false &&
+        Double.class.equals(type) == false) {
+      throw new IllegalArgumentException("unsupported numeric type: " + type);
+    }
+    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 int hashCode() {
+    final int prime = 31;
+    int result = 1;
+    result = prime * result + format.hashCode();
+    result = prime * result + type.hashCode();
+    return result;
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (this == obj) return true;
+    if (obj == null) return false;
+    if (getClass() != obj.getClass()) return false;
+    PointsConfig other = (PointsConfig) obj;
+    if (!format.equals(other.format)) return false;
+    if (!type.equals(other.type)) return false;
+    return true;
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/89cc676f/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/config/PointsConfigListener.java
----------------------------------------------------------------------
diff --git a/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/config/PointsConfigListener.java b/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/config/PointsConfigListener.java
new file mode 100644
index 0000000..9efbbb7
--- /dev/null
+++ b/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/config/PointsConfigListener.java
@@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.queryparser.flexible.standard.config;
+
+import java.util.Map;
+
+import org.apache.lucene.queryparser.flexible.core.config.FieldConfig;
+import org.apache.lucene.queryparser.flexible.core.config.FieldConfigListener;
+import org.apache.lucene.queryparser.flexible.core.config.QueryConfigHandler;
+import org.apache.lucene.queryparser.flexible.standard.config.StandardQueryConfigHandler.ConfigurationKeys;
+
+/**
+ * This listener is used to listen to {@link FieldConfig} requests in
+ * {@link QueryConfigHandler} and add {@link ConfigurationKeys#POINTS_CONFIG}
+ * based on the {@link ConfigurationKeys#POINTS_CONFIG_MAP} set in the
+ * {@link QueryConfigHandler}.
+ * 
+ * @see PointsConfig
+ * @see QueryConfigHandler
+ * @see ConfigurationKeys#POINTS_CONFIG
+ * @see ConfigurationKeys#POINTS_CONFIG_MAP
+ */
+public class PointsConfigListener implements FieldConfigListener {
+  
+  final private QueryConfigHandler config;
+  
+  /**
+   * Constructs a {@link PointsConfigListener} object using the given {@link QueryConfigHandler}.
+   * 
+   * @param config the {@link QueryConfigHandler} it will listen too
+   */
+  public PointsConfigListener(QueryConfigHandler config) { 
+    if (config == null) {
+      throw new IllegalArgumentException("config cannot be null!");
+    }
+    this.config = config;
+  }
+  
+  @Override
+  public void buildFieldConfig(FieldConfig fieldConfig) {
+    Map<String,PointsConfig> pointsConfigMap = config.get(ConfigurationKeys.POINTS_CONFIG_MAP);
+    
+    if (pointsConfigMap != null) {
+      PointsConfig pointsConfig = pointsConfigMap.get(fieldConfig.getField());
+      
+      if (pointsConfig != null) {
+        fieldConfig.set(ConfigurationKeys.POINTS_CONFIG, pointsConfig);
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/89cc676f/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/config/StandardQueryConfigHandler.java
----------------------------------------------------------------------
diff --git a/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/config/StandardQueryConfigHandler.java b/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/config/StandardQueryConfigHandler.java
index 77bd7bb..bba95ee 100644
--- a/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/config/StandardQueryConfigHandler.java
+++ b/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/config/StandardQueryConfigHandler.java
@@ -167,21 +167,41 @@ public class StandardQueryConfigHandler extends QueryConfigHandler {
     final public static ConfigurationKey<Float> BOOST = ConfigurationKey.newInstance();
     
     /**
-     * Key used to set a field to its {@link NumericConfig}.
+     * Key used to set a field to its {@link LegacyNumericConfig}.
      * 
-     * @see StandardQueryParser#setNumericConfigMap(Map)
-     * @see StandardQueryParser#getNumericConfigMap()
+     * @see StandardQueryParser#setLegacyNumericConfigMap(Map)
+     * @see StandardQueryParser#getLegacyNumericConfigMap()
+     * @deprecated Index with Points instead and use {@link #POINTS_CONFIG}
      */
-    final public static ConfigurationKey<NumericConfig> NUMERIC_CONFIG = ConfigurationKey.newInstance();
+    @Deprecated
+    final public static ConfigurationKey<LegacyNumericConfig> LEGACY_NUMERIC_CONFIG = ConfigurationKey.newInstance();
     
     /**
-     * Key used to set the {@link NumericConfig} in {@link FieldConfig} for numeric fields.
+     * Key used to set the {@link LegacyNumericConfig} in {@link FieldConfig} for numeric fields.
      * 
-     * @see StandardQueryParser#setNumericConfigMap(Map)
-     * @see StandardQueryParser#getNumericConfigMap()
+     * @see StandardQueryParser#setLegacyNumericConfigMap(Map)
+     * @see StandardQueryParser#getLegacyNumericConfigMap()
+     * @deprecated Index with Points instead and use {@link #POINTS_CONFIG_MAP}
      */
-    final public static ConfigurationKey<Map<String,NumericConfig>> NUMERIC_CONFIG_MAP = ConfigurationKey.newInstance();
+    @Deprecated
+    final public static ConfigurationKey<Map<String,LegacyNumericConfig>> LEGACY_NUMERIC_CONFIG_MAP = ConfigurationKey.newInstance();
     
+    /**
+     * Key used to set a field to its {@link PointsConfig}.
+     * 
+     * @see StandardQueryParser#setLegacyNumericConfigMap(Map)
+     * @see StandardQueryParser#getLegacyNumericConfigMap()
+     */
+    final public static ConfigurationKey<PointsConfig> POINTS_CONFIG = ConfigurationKey.newInstance();
+
+    /**
+     * Key used to set the {@link PointsConfig} in {@link FieldConfig} for point fields.
+     * 
+     * @see StandardQueryParser#setLegacyNumericConfigMap(Map)
+     * @see StandardQueryParser#getLegacyNumericConfigMap()
+     */
+    final public static ConfigurationKey<Map<String,PointsConfig>> POINTS_CONFIG_MAP = ConfigurationKey.newInstance();
+
   }
   
   /**
@@ -195,7 +215,8 @@ public class StandardQueryConfigHandler extends QueryConfigHandler {
     // Add listener that will build the FieldConfig.
     addFieldConfigListener(new FieldBoostMapFCListener(this));
     addFieldConfigListener(new FieldDateResolutionFCListener(this));
-    addFieldConfigListener(new NumericFieldConfigListener(this));
+    addFieldConfigListener(new LegacyNumericFieldConfigListener(this));
+    addFieldConfigListener(new PointsConfigListener(this));
     
     // Default Values
     set(ConfigurationKeys.ALLOW_LEADING_WILDCARD, false); // default in 2.9

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/89cc676f/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/nodes/LegacyNumericQueryNode.java
----------------------------------------------------------------------
diff --git a/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/nodes/LegacyNumericQueryNode.java b/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/nodes/LegacyNumericQueryNode.java
new file mode 100644
index 0000000..b644d8a
--- /dev/null
+++ b/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/nodes/LegacyNumericQueryNode.java
@@ -0,0 +1,153 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.queryparser.flexible.standard.nodes;
+
+import java.text.NumberFormat;
+import java.util.Locale;
+
+import org.apache.lucene.queryparser.flexible.core.nodes.FieldQueryNode;
+import org.apache.lucene.queryparser.flexible.core.nodes.FieldValuePairQueryNode;
+import org.apache.lucene.queryparser.flexible.core.nodes.QueryNodeImpl;
+import org.apache.lucene.queryparser.flexible.core.parser.EscapeQuerySyntax;
+import org.apache.lucene.queryparser.flexible.core.parser.EscapeQuerySyntax.Type;
+import org.apache.lucene.queryparser.flexible.standard.config.LegacyNumericConfig;
+
+/**
+ * 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 LegacyNumericConfig
+ * @deprecated Index with Points instead and use {@link PointQueryNode} instead.
+ */
+@Deprecated
+public class LegacyNumericQueryNode extends QueryNodeImpl implements
+    FieldValuePairQueryNode<Number> {
+  
+  private NumberFormat numberFormat;
+  
+  private CharSequence field;
+  
+  private Number value;
+  
+  /**
+   * Creates a {@link LegacyNumericQueryNode} 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 LegacyNumericQueryNode(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
+   */
+  @Override
+  public CharSequence getField() {
+    return this.field;
+  }
+  
+  /**
+   * Sets the field associated with this node.
+   * 
+   * @param fieldName the field associated with this node
+   */
+  @Override
+  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.format(this.value),
+        Locale.ROOT, Type.NORMAL);
+  }
+  
+  @Override
+  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
+   */
+  @Override
+  public Number getValue() {
+    return value;
+  }
+  
+  /**
+   * Sets the numeric value.
+   * 
+   * @param value the numeric value
+   */
+  @Override
+  public void setValue(Number value) {
+    this.value = value;
+  }
+  
+  @Override
+  public String toString() {
+    return "<numeric field='" + this.field + "' number='"
+        + numberFormat.format(value) + "'/>";
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/89cc676f/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/nodes/LegacyNumericRangeQueryNode.java
----------------------------------------------------------------------
diff --git a/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/nodes/LegacyNumericRangeQueryNode.java b/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/nodes/LegacyNumericRangeQueryNode.java
new file mode 100644
index 0000000..088ab98
--- /dev/null
+++ b/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/nodes/LegacyNumericRangeQueryNode.java
@@ -0,0 +1,153 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.queryparser.flexible.standard.nodes;
+
+import org.apache.lucene.document.FieldType;
+import org.apache.lucene.document.FieldType.LegacyNumericType;
+import org.apache.lucene.queryparser.flexible.core.QueryNodeException;
+import org.apache.lucene.queryparser.flexible.core.messages.QueryParserMessages;
+import org.apache.lucene.queryparser.flexible.messages.MessageImpl;
+import org.apache.lucene.queryparser.flexible.standard.config.LegacyNumericConfig;
+
+/**
+ * This query node represents a range query composed by {@link LegacyNumericQueryNode}
+ * bounds, which means the bound values are {@link Number}s.
+ * 
+ * @see LegacyNumericQueryNode
+ * @see AbstractRangeQueryNode
+ * @deprecated Index with Points instead and use {@link PointRangeQueryNode} instead.
+ */
+@Deprecated
+public class LegacyNumericRangeQueryNode extends
+    AbstractRangeQueryNode<LegacyNumericQueryNode> {
+  
+  public LegacyNumericConfig numericConfig; 
+  
+  /**
+   * Constructs a {@link LegacyNumericRangeQueryNode} object using the given
+   * {@link LegacyNumericQueryNode} as its bounds and {@link LegacyNumericConfig}.
+   * 
+   * @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 LegacyNumericConfig} that represents associated with the upper and lower bounds
+   * 
+   * @see #setBounds(LegacyNumericQueryNode, LegacyNumericQueryNode, boolean, boolean, LegacyNumericConfig)
+   */
+  public LegacyNumericRangeQueryNode(LegacyNumericQueryNode lower, LegacyNumericQueryNode upper,
+      boolean lowerInclusive, boolean upperInclusive, LegacyNumericConfig numericConfig) throws QueryNodeException {
+    setBounds(lower, upper, lowerInclusive, upperInclusive, numericConfig);
+  }
+  
+  private static LegacyNumericType getNumericDataType(Number number) throws QueryNodeException {
+    
+    if (number instanceof Long) {
+      return FieldType.LegacyNumericType.LONG;
+    } else if (number instanceof Integer) {
+      return FieldType.LegacyNumericType.INT;
+    } else if (number instanceof Double) {
+      return LegacyNumericType.DOUBLE;
+    } else if (number instanceof Float) {
+      return FieldType.LegacyNumericType.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 LegacyNumericConfig} 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 LegacyNumericConfig} that represents associated with the upper and lower bounds
+   * 
+   */
+  public void setBounds(LegacyNumericQueryNode lower, LegacyNumericQueryNode upper,
+      boolean lowerInclusive, boolean upperInclusive, LegacyNumericConfig numericConfig) throws QueryNodeException {
+    
+    if (numericConfig == null) {
+      throw new IllegalArgumentException("numericConfig cannot be null!");
+    }
+    
+    LegacyNumericType 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 LegacyNumericConfig} associated with the lower and upper bounds.
+   * 
+   * @return the {@link LegacyNumericConfig} associated with the lower and upper bounds
+   */
+  public LegacyNumericConfig 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();
+    
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/89cc676f/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/nodes/NumericQueryNode.java
----------------------------------------------------------------------
diff --git a/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/nodes/NumericQueryNode.java b/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/nodes/NumericQueryNode.java
deleted file mode 100644
index 7509a39..0000000
--- a/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/nodes/NumericQueryNode.java
+++ /dev/null
@@ -1,151 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.lucene.queryparser.flexible.standard.nodes;
-
-import java.text.NumberFormat;
-import java.util.Locale;
-
-import org.apache.lucene.queryparser.flexible.core.nodes.FieldQueryNode;
-import org.apache.lucene.queryparser.flexible.core.nodes.FieldValuePairQueryNode;
-import org.apache.lucene.queryparser.flexible.core.nodes.QueryNodeImpl;
-import org.apache.lucene.queryparser.flexible.core.parser.EscapeQuerySyntax;
-import org.apache.lucene.queryparser.flexible.core.parser.EscapeQuerySyntax.Type;
-import org.apache.lucene.queryparser.flexible.standard.config.NumericConfig;
-
-/**
- * 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 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
-   */
-  @Override
-  public CharSequence getField() {
-    return this.field;
-  }
-  
-  /**
-   * Sets the field associated with this node.
-   * 
-   * @param fieldName the field associated with this node
-   */
-  @Override
-  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.format(this.value),
-        Locale.ROOT, Type.NORMAL);
-  }
-  
-  @Override
-  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
-   */
-  @Override
-  public Number getValue() {
-    return value;
-  }
-  
-  /**
-   * Sets the numeric value.
-   * 
-   * @param value the numeric value
-   */
-  @Override
-  public void setValue(Number value) {
-    this.value = value;
-  }
-  
-  @Override
-  public String toString() {
-    return "<numeric field='" + this.field + "' number='"
-        + numberFormat.format(value) + "'/>";
-  }
-  
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/89cc676f/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/nodes/NumericRangeQueryNode.java
----------------------------------------------------------------------
diff --git a/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/nodes/NumericRangeQueryNode.java b/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/nodes/NumericRangeQueryNode.java
deleted file mode 100644
index c132aa1..0000000
--- a/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/nodes/NumericRangeQueryNode.java
+++ /dev/null
@@ -1,151 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.lucene.queryparser.flexible.standard.nodes;
-
-import org.apache.lucene.document.FieldType;
-import org.apache.lucene.document.FieldType.LegacyNumericType;
-import org.apache.lucene.queryparser.flexible.core.QueryNodeException;
-import org.apache.lucene.queryparser.flexible.core.messages.QueryParserMessages;
-import org.apache.lucene.queryparser.flexible.messages.MessageImpl;
-import org.apache.lucene.queryparser.flexible.standard.config.NumericConfig;
-
-/**
- * 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 LegacyNumericType getNumericDataType(Number number) throws QueryNodeException {
-    
-    if (number instanceof Long) {
-      return FieldType.LegacyNumericType.LONG;
-    } else if (number instanceof Integer) {
-      return FieldType.LegacyNumericType.INT;
-    } else if (number instanceof Double) {
-      return LegacyNumericType.DOUBLE;
-    } else if (number instanceof Float) {
-      return FieldType.LegacyNumericType.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!");
-    }
-    
-    LegacyNumericType 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();
-    
-  }
-  
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/89cc676f/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/nodes/PointQueryNode.java
----------------------------------------------------------------------
diff --git a/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/nodes/PointQueryNode.java b/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/nodes/PointQueryNode.java
new file mode 100644
index 0000000..6d4cba7
--- /dev/null
+++ b/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/nodes/PointQueryNode.java
@@ -0,0 +1,151 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.queryparser.flexible.standard.nodes;
+
+import java.text.NumberFormat;
+import java.util.Locale;
+
+import org.apache.lucene.queryparser.flexible.core.nodes.FieldQueryNode;
+import org.apache.lucene.queryparser.flexible.core.nodes.FieldValuePairQueryNode;
+import org.apache.lucene.queryparser.flexible.core.nodes.QueryNodeImpl;
+import org.apache.lucene.queryparser.flexible.core.parser.EscapeQuerySyntax;
+import org.apache.lucene.queryparser.flexible.core.parser.EscapeQuerySyntax.Type;
+import org.apache.lucene.queryparser.flexible.standard.config.PointsConfig;
+
+/**
+ * This query node represents a field query that holds a point value. It is
+ * similar to {@link FieldQueryNode}, however the {@link #getValue()} returns a
+ * {@link Number}.
+ * 
+ * @see PointsConfig
+ */
+public class PointQueryNode extends QueryNodeImpl implements
+    FieldValuePairQueryNode<Number> {
+  
+  private NumberFormat numberFormat;
+  
+  private CharSequence field;
+  
+  private Number value;
+  
+  /**
+   * Creates a {@link PointQueryNode} 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 PointQueryNode(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
+   */
+  @Override
+  public CharSequence getField() {
+    return this.field;
+  }
+  
+  /**
+   * Sets the field associated with this node.
+   * 
+   * @param fieldName the field associated with this node
+   */
+  @Override
+  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.format(this.value),
+        Locale.ROOT, Type.NORMAL);
+  }
+  
+  @Override
+  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
+   */
+  @Override
+  public Number getValue() {
+    return value;
+  }
+  
+  /**
+   * Sets the numeric value.
+   * 
+   * @param value the numeric value
+   */
+  @Override
+  public void setValue(Number value) {
+    this.value = value;
+  }
+  
+  @Override
+  public String toString() {
+    return "<numeric field='" + this.field + "' number='"
+        + numberFormat.format(value) + "'/>";
+  }
+  
+}