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:03 UTC

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

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/89cc676f/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/nodes/PointRangeQueryNode.java
----------------------------------------------------------------------
diff --git a/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/nodes/PointRangeQueryNode.java b/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/nodes/PointRangeQueryNode.java
new file mode 100644
index 0000000..cb838fc
--- /dev/null
+++ b/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/nodes/PointRangeQueryNode.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.nodes;
+
+import org.apache.lucene.queryparser.flexible.core.QueryNodeException;
+import org.apache.lucene.queryparser.flexible.standard.config.PointsConfig;
+
+/**
+ * This query node represents a range query composed by {@link PointQueryNode}
+ * bounds, which means the bound values are {@link Number}s.
+ * 
+ * @see PointQueryNode
+ * @see AbstractRangeQueryNode
+ */
+public class PointRangeQueryNode extends AbstractRangeQueryNode<PointQueryNode> {
+  
+  public PointsConfig numericConfig; 
+  
+  /**
+   * Constructs a {@link PointRangeQueryNode} object using the given
+   * {@link PointQueryNode} as its bounds and {@link PointsConfig}.
+   * 
+   * @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 PointsConfig} that represents associated with the upper and lower bounds
+   * 
+   * @see #setBounds(PointQueryNode, PointQueryNode, boolean, boolean, PointsConfig)
+   */
+  public PointRangeQueryNode(PointQueryNode lower, PointQueryNode upper,
+      boolean lowerInclusive, boolean upperInclusive, PointsConfig numericConfig) throws QueryNodeException {
+    setBounds(lower, upper, lowerInclusive, upperInclusive, numericConfig);
+  }
+  
+  /**
+   * Sets the upper and lower bounds of this range query node and the
+   * {@link PointsConfig} 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 pointsConfig the {@link PointsConfig} that represents associated with the upper and lower bounds
+   * 
+   */
+  public void setBounds(PointQueryNode lower, PointQueryNode upper,
+      boolean lowerInclusive, boolean upperInclusive, PointsConfig pointsConfig) throws QueryNodeException {
+    
+    if (pointsConfig == null) {
+      throw new IllegalArgumentException("pointsConfig cannot be null!");
+    }
+    
+    Class<? extends Number> lowerNumberType, upperNumberType;
+    
+    if (lower != null && lower.getValue() != null) {
+      lowerNumberType = lower.getValue().getClass();
+    } else {
+      lowerNumberType = null;
+    }
+    
+    if (upper != null && upper.getValue() != null) {
+      upperNumberType = upper.getValue().getClass();
+    } else {
+      upperNumberType = null;
+    }
+    
+    if (lowerNumberType != null
+        && !lowerNumberType.equals(pointsConfig.getType())) {
+      throw new IllegalArgumentException(
+          "lower value's type should be the same as numericConfig type: "
+              + lowerNumberType + " != " + pointsConfig.getType());
+    }
+    
+    if (upperNumberType != null
+        && !upperNumberType.equals(pointsConfig.getType())) {
+      throw new IllegalArgumentException(
+          "upper value's type should be the same as numericConfig type: "
+              + upperNumberType + " != " + pointsConfig.getType());
+    }
+    
+    super.setBounds(lower, upper, lowerInclusive, upperInclusive);
+    this.numericConfig = pointsConfig;
+  }
+  
+  /**
+   * Returns the {@link PointsConfig} associated with the lower and upper bounds.
+   * 
+   * @return the {@link PointsConfig} associated with the lower and upper bounds
+   */
+  public PointsConfig getPointsConfig() {
+    return this.numericConfig;
+  }
+  
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder();
+    sb.append("<pointRange lowerInclusive='");
+    sb.append(isLowerInclusive());
+    sb.append("' upperInclusive='");
+    sb.append(isUpperInclusive());
+    sb.append("' type='");
+    sb.append(numericConfig.getType().getSimpleName());
+    sb.append("'>\n");
+    sb.append(getLowerBound()).append('\n');
+    sb.append(getUpperBound()).append('\n');
+    sb.append("</pointRange>");
+    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/processors/LegacyNumericQueryNodeProcessor.java
----------------------------------------------------------------------
diff --git a/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/processors/LegacyNumericQueryNodeProcessor.java b/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/processors/LegacyNumericQueryNodeProcessor.java
new file mode 100644
index 0000000..8b71824
--- /dev/null
+++ b/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/processors/LegacyNumericQueryNodeProcessor.java
@@ -0,0 +1,154 @@
+/*
+ * 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.processors;
+
+import java.text.NumberFormat;
+import java.text.ParseException;
+import java.util.List;
+
+import org.apache.lucene.queryparser.flexible.messages.MessageImpl;
+import org.apache.lucene.queryparser.flexible.core.QueryNodeException;
+import org.apache.lucene.queryparser.flexible.core.QueryNodeParseException;
+import org.apache.lucene.queryparser.flexible.core.config.FieldConfig;
+import org.apache.lucene.queryparser.flexible.core.config.QueryConfigHandler;
+import org.apache.lucene.queryparser.flexible.core.messages.QueryParserMessages;
+import org.apache.lucene.queryparser.flexible.core.nodes.FieldQueryNode;
+import org.apache.lucene.queryparser.flexible.core.nodes.QueryNode;
+import org.apache.lucene.queryparser.flexible.core.nodes.RangeQueryNode;
+import org.apache.lucene.queryparser.flexible.core.processors.QueryNodeProcessorImpl;
+import org.apache.lucene.queryparser.flexible.standard.config.LegacyNumericConfig;
+import org.apache.lucene.queryparser.flexible.standard.config.StandardQueryConfigHandler.ConfigurationKeys;
+import org.apache.lucene.queryparser.flexible.standard.nodes.LegacyNumericQueryNode;
+import org.apache.lucene.queryparser.flexible.standard.nodes.LegacyNumericRangeQueryNode;
+
+/**
+ * This processor is used to convert {@link FieldQueryNode}s to
+ * {@link LegacyNumericRangeQueryNode}s. It looks for
+ * {@link ConfigurationKeys#LEGACY_NUMERIC_CONFIG} set in the {@link FieldConfig} of
+ * every {@link FieldQueryNode} found. If
+ * {@link ConfigurationKeys#LEGACY_NUMERIC_CONFIG} is found, it considers that
+ * {@link FieldQueryNode} to be a numeric query and convert it to
+ * {@link LegacyNumericRangeQueryNode} 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 FieldQueryNode}s children of a
+ * {@link RangeQueryNode} are ignored.
+ * 
+ * @see ConfigurationKeys#LEGACY_NUMERIC_CONFIG
+ * @see FieldQueryNode
+ * @see LegacyNumericConfig
+ * @see LegacyNumericQueryNode
+ * @deprecated Index with points and use {@link PointQueryNodeProcessor} instead.
+ */
+@Deprecated
+public class LegacyNumericQueryNodeProcessor extends QueryNodeProcessorImpl {
+  
+  /**
+   * Constructs a {@link LegacyNumericQueryNodeProcessor} object.
+   */
+  public LegacyNumericQueryNodeProcessor() {
+  // empty constructor
+  }
+  
+  @Override
+  protected QueryNode postProcessNode(QueryNode node) throws QueryNodeException {
+    
+    if (node instanceof FieldQueryNode
+        && !(node.getParent() instanceof RangeQueryNode)) {
+      
+      QueryConfigHandler config = getQueryConfigHandler();
+      
+      if (config != null) {
+        FieldQueryNode fieldNode = (FieldQueryNode) node;
+        FieldConfig fieldConfig = config.getFieldConfig(fieldNode
+            .getFieldAsString());
+        
+        if (fieldConfig != null) {
+          LegacyNumericConfig numericConfig = fieldConfig
+              .get(ConfigurationKeys.LEGACY_NUMERIC_CONFIG);
+          
+          if (numericConfig != null) {
+            
+            NumberFormat numberFormat = numericConfig.getNumberFormat();
+            String text = fieldNode.getTextAsString();
+            Number number = null;
+            
+            if (text.length() > 0) {
+              
+              try {
+                number = numberFormat.parse(text);
+                
+              } 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();
+              }
+              
+            } else {
+              throw new QueryNodeParseException(new MessageImpl(
+                  QueryParserMessages.NUMERIC_CANNOT_BE_EMPTY, fieldNode.getFieldAsString()));
+            }
+            
+            LegacyNumericQueryNode lowerNode = new LegacyNumericQueryNode(fieldNode
+                .getField(), number, numberFormat);
+            LegacyNumericQueryNode upperNode = new LegacyNumericQueryNode(fieldNode
+                .getField(), number, numberFormat);
+            
+            return new LegacyNumericRangeQueryNode(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;
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/89cc676f/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/processors/LegacyNumericRangeQueryNodeProcessor.java
----------------------------------------------------------------------
diff --git a/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/processors/LegacyNumericRangeQueryNodeProcessor.java b/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/processors/LegacyNumericRangeQueryNodeProcessor.java
new file mode 100644
index 0000000..5a54b7b
--- /dev/null
+++ b/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/processors/LegacyNumericRangeQueryNodeProcessor.java
@@ -0,0 +1,170 @@
+/*
+ * 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.processors;
+
+import java.text.NumberFormat;
+import java.text.ParseException;
+import java.util.List;
+
+import org.apache.lucene.queryparser.flexible.messages.MessageImpl;
+import org.apache.lucene.queryparser.flexible.core.QueryNodeException;
+import org.apache.lucene.queryparser.flexible.core.QueryNodeParseException;
+import org.apache.lucene.queryparser.flexible.core.config.FieldConfig;
+import org.apache.lucene.queryparser.flexible.core.config.QueryConfigHandler;
+import org.apache.lucene.queryparser.flexible.core.messages.QueryParserMessages;
+import org.apache.lucene.queryparser.flexible.core.nodes.FieldQueryNode;
+import org.apache.lucene.queryparser.flexible.core.nodes.QueryNode;
+import org.apache.lucene.queryparser.flexible.core.processors.QueryNodeProcessorImpl;
+import org.apache.lucene.queryparser.flexible.core.util.StringUtils;
+import org.apache.lucene.queryparser.flexible.standard.config.LegacyNumericConfig;
+import org.apache.lucene.queryparser.flexible.standard.config.StandardQueryConfigHandler.ConfigurationKeys;
+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.TermRangeQueryNode;
+
+/**
+ * This processor is used to convert {@link TermRangeQueryNode}s to
+ * {@link LegacyNumericRangeQueryNode}s. It looks for
+ * {@link ConfigurationKeys#LEGACY_NUMERIC_CONFIG} set in the {@link FieldConfig} of
+ * every {@link TermRangeQueryNode} found. If
+ * {@link ConfigurationKeys#LEGACY_NUMERIC_CONFIG} is found, it considers that
+ * {@link TermRangeQueryNode} to be a numeric range query and convert it to
+ * {@link LegacyNumericRangeQueryNode}.
+ * 
+ * @see ConfigurationKeys#LEGACY_NUMERIC_CONFIG
+ * @see TermRangeQueryNode
+ * @see LegacyNumericConfig
+ * @see LegacyNumericRangeQueryNode
+ * @deprecated Index with points and use {@link PointRangeQueryNodeProcessor} instead.
+ */
+@Deprecated
+public class LegacyNumericRangeQueryNodeProcessor extends QueryNodeProcessorImpl {
+  
+  /**
+   * Constructs an empty {@link LegacyNumericRangeQueryNode} object.
+   */
+  public LegacyNumericRangeQueryNodeProcessor() {
+  // empty constructor
+  }
+  
+  @Override
+  protected QueryNode postProcessNode(QueryNode node) throws QueryNodeException {
+    
+    if (node instanceof TermRangeQueryNode) {
+      QueryConfigHandler config = getQueryConfigHandler();
+      
+      if (config != null) {
+        TermRangeQueryNode termRangeNode = (TermRangeQueryNode) node;
+        FieldConfig fieldConfig = config.getFieldConfig(StringUtils
+            .toString(termRangeNode.getField()));
+        
+        if (fieldConfig != null) {
+          
+          LegacyNumericConfig numericConfig = fieldConfig
+              .get(ConfigurationKeys.LEGACY_NUMERIC_CONFIG);
+          
+          if (numericConfig != null) {
+            
+            FieldQueryNode lower = termRangeNode.getLowerBound();
+            FieldQueryNode upper = termRangeNode.getUpperBound();
+            
+            String lowerText = lower.getTextAsString();
+            String upperText = upper.getTextAsString();
+            NumberFormat numberFormat = numericConfig.getNumberFormat();
+            Number lowerNumber = null, upperNumber = null;
+            
+             if (lowerText.length() > 0) {
+              
+              try {
+                lowerNumber = numberFormat.parse(lowerText);
+                
+              } catch (ParseException e) {
+                throw new QueryNodeParseException(new MessageImpl(
+                    QueryParserMessages.COULD_NOT_PARSE_NUMBER, lower
+                        .getTextAsString(), numberFormat.getClass()
+                        .getCanonicalName()), e);
+              }
+              
+            }
+            
+             if (upperText.length() > 0) {
+            
+              try {
+                upperNumber = numberFormat.parse(upperText);
+                
+              } catch (ParseException e) {
+                throw new QueryNodeParseException(new MessageImpl(
+                    QueryParserMessages.COULD_NOT_PARSE_NUMBER, upper
+                        .getTextAsString(), numberFormat.getClass()
+                        .getCanonicalName()), e);
+              }
+            
+            }
+            
+            switch (numericConfig.getType()) {
+              case LONG:
+                if (upperNumber != null) upperNumber = upperNumber.longValue();
+                if (lowerNumber != null) lowerNumber = lowerNumber.longValue();
+                break;
+              case INT:
+                if (upperNumber != null) upperNumber = upperNumber.intValue();
+                if (lowerNumber != null) lowerNumber = lowerNumber.intValue();
+                break;
+              case DOUBLE:
+                if (upperNumber != null) upperNumber = upperNumber.doubleValue();
+                if (lowerNumber != null) lowerNumber = lowerNumber.doubleValue();
+                break;
+              case FLOAT:
+                if (upperNumber != null) upperNumber = upperNumber.floatValue();
+                if (lowerNumber != null) lowerNumber = lowerNumber.floatValue();
+            }
+            
+            LegacyNumericQueryNode lowerNode = new LegacyNumericQueryNode(
+                termRangeNode.getField(), lowerNumber, numberFormat);
+            LegacyNumericQueryNode upperNode = new LegacyNumericQueryNode(
+                termRangeNode.getField(), upperNumber, numberFormat);
+            
+            boolean lowerInclusive = termRangeNode.isLowerInclusive();
+            boolean upperInclusive = termRangeNode.isUpperInclusive();
+            
+            return new LegacyNumericRangeQueryNode(lowerNode, upperNode,
+                lowerInclusive, upperInclusive, 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;
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/89cc676f/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/processors/NumericQueryNodeProcessor.java
----------------------------------------------------------------------
diff --git a/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/processors/NumericQueryNodeProcessor.java b/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/processors/NumericQueryNodeProcessor.java
deleted file mode 100644
index 10bd6ba..0000000
--- a/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/processors/NumericQueryNodeProcessor.java
+++ /dev/null
@@ -1,152 +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.processors;
-
-import java.text.NumberFormat;
-import java.text.ParseException;
-import java.util.List;
-
-import org.apache.lucene.queryparser.flexible.messages.MessageImpl;
-import org.apache.lucene.queryparser.flexible.core.QueryNodeException;
-import org.apache.lucene.queryparser.flexible.core.QueryNodeParseException;
-import org.apache.lucene.queryparser.flexible.core.config.FieldConfig;
-import org.apache.lucene.queryparser.flexible.core.config.QueryConfigHandler;
-import org.apache.lucene.queryparser.flexible.core.messages.QueryParserMessages;
-import org.apache.lucene.queryparser.flexible.core.nodes.FieldQueryNode;
-import org.apache.lucene.queryparser.flexible.core.nodes.QueryNode;
-import org.apache.lucene.queryparser.flexible.core.nodes.RangeQueryNode;
-import org.apache.lucene.queryparser.flexible.core.processors.QueryNodeProcessorImpl;
-import org.apache.lucene.queryparser.flexible.standard.config.NumericConfig;
-import org.apache.lucene.queryparser.flexible.standard.config.StandardQueryConfigHandler.ConfigurationKeys;
-import org.apache.lucene.queryparser.flexible.standard.nodes.NumericQueryNode;
-import org.apache.lucene.queryparser.flexible.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 FieldQueryNode}s children of a
- * {@link RangeQueryNode} are ignored.
- * 
- * @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.getParent() instanceof RangeQueryNode)) {
-      
-      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();
-            String text = fieldNode.getTextAsString();
-            Number number = null;
-            
-            if (text.length() > 0) {
-              
-              try {
-                number = numberFormat.parse(text);
-                
-              } 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();
-              }
-              
-            } else {
-              throw new QueryNodeParseException(new MessageImpl(
-                  QueryParserMessages.NUMERIC_CANNOT_BE_EMPTY, fieldNode.getFieldAsString()));
-            }
-            
-            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;
-  }
-  
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/89cc676f/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/processors/NumericRangeQueryNodeProcessor.java
----------------------------------------------------------------------
diff --git a/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/processors/NumericRangeQueryNodeProcessor.java b/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/processors/NumericRangeQueryNodeProcessor.java
deleted file mode 100644
index bbe5284..0000000
--- a/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/processors/NumericRangeQueryNodeProcessor.java
+++ /dev/null
@@ -1,168 +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.processors;
-
-import java.text.NumberFormat;
-import java.text.ParseException;
-import java.util.List;
-
-import org.apache.lucene.queryparser.flexible.messages.MessageImpl;
-import org.apache.lucene.queryparser.flexible.core.QueryNodeException;
-import org.apache.lucene.queryparser.flexible.core.QueryNodeParseException;
-import org.apache.lucene.queryparser.flexible.core.config.FieldConfig;
-import org.apache.lucene.queryparser.flexible.core.config.QueryConfigHandler;
-import org.apache.lucene.queryparser.flexible.core.messages.QueryParserMessages;
-import org.apache.lucene.queryparser.flexible.core.nodes.FieldQueryNode;
-import org.apache.lucene.queryparser.flexible.core.nodes.QueryNode;
-import org.apache.lucene.queryparser.flexible.core.processors.QueryNodeProcessorImpl;
-import org.apache.lucene.queryparser.flexible.core.util.StringUtils;
-import org.apache.lucene.queryparser.flexible.standard.config.NumericConfig;
-import org.apache.lucene.queryparser.flexible.standard.config.StandardQueryConfigHandler.ConfigurationKeys;
-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.TermRangeQueryNode;
-
-/**
- * This processor is used to convert {@link TermRangeQueryNode}s to
- * {@link NumericRangeQueryNode}s. It looks for
- * {@link ConfigurationKeys#NUMERIC_CONFIG} set in the {@link FieldConfig} of
- * every {@link TermRangeQueryNode} found. If
- * {@link ConfigurationKeys#NUMERIC_CONFIG} is found, it considers that
- * {@link TermRangeQueryNode} to be a numeric range query and convert it to
- * {@link NumericRangeQueryNode}.
- * 
- * @see ConfigurationKeys#NUMERIC_CONFIG
- * @see TermRangeQueryNode
- * @see NumericConfig
- * @see NumericRangeQueryNode
- */
-public class NumericRangeQueryNodeProcessor extends QueryNodeProcessorImpl {
-  
-  /**
-   * Constructs an empty {@link NumericRangeQueryNode} object.
-   */
-  public NumericRangeQueryNodeProcessor() {
-  // empty constructor
-  }
-  
-  @Override
-  protected QueryNode postProcessNode(QueryNode node) throws QueryNodeException {
-    
-    if (node instanceof TermRangeQueryNode) {
-      QueryConfigHandler config = getQueryConfigHandler();
-      
-      if (config != null) {
-        TermRangeQueryNode termRangeNode = (TermRangeQueryNode) node;
-        FieldConfig fieldConfig = config.getFieldConfig(StringUtils
-            .toString(termRangeNode.getField()));
-        
-        if (fieldConfig != null) {
-          
-          NumericConfig numericConfig = fieldConfig
-              .get(ConfigurationKeys.NUMERIC_CONFIG);
-          
-          if (numericConfig != null) {
-            
-            FieldQueryNode lower = termRangeNode.getLowerBound();
-            FieldQueryNode upper = termRangeNode.getUpperBound();
-            
-            String lowerText = lower.getTextAsString();
-            String upperText = upper.getTextAsString();
-            NumberFormat numberFormat = numericConfig.getNumberFormat();
-            Number lowerNumber = null, upperNumber = null;
-            
-             if (lowerText.length() > 0) {
-              
-              try {
-                lowerNumber = numberFormat.parse(lowerText);
-                
-              } catch (ParseException e) {
-                throw new QueryNodeParseException(new MessageImpl(
-                    QueryParserMessages.COULD_NOT_PARSE_NUMBER, lower
-                        .getTextAsString(), numberFormat.getClass()
-                        .getCanonicalName()), e);
-              }
-              
-            }
-            
-             if (upperText.length() > 0) {
-            
-              try {
-                upperNumber = numberFormat.parse(upperText);
-                
-              } catch (ParseException e) {
-                throw new QueryNodeParseException(new MessageImpl(
-                    QueryParserMessages.COULD_NOT_PARSE_NUMBER, upper
-                        .getTextAsString(), numberFormat.getClass()
-                        .getCanonicalName()), e);
-              }
-            
-            }
-            
-            switch (numericConfig.getType()) {
-              case LONG:
-                if (upperNumber != null) upperNumber = upperNumber.longValue();
-                if (lowerNumber != null) lowerNumber = lowerNumber.longValue();
-                break;
-              case INT:
-                if (upperNumber != null) upperNumber = upperNumber.intValue();
-                if (lowerNumber != null) lowerNumber = lowerNumber.intValue();
-                break;
-              case DOUBLE:
-                if (upperNumber != null) upperNumber = upperNumber.doubleValue();
-                if (lowerNumber != null) lowerNumber = lowerNumber.doubleValue();
-                break;
-              case FLOAT:
-                if (upperNumber != null) upperNumber = upperNumber.floatValue();
-                if (lowerNumber != null) lowerNumber = lowerNumber.floatValue();
-            }
-            
-            NumericQueryNode lowerNode = new NumericQueryNode(
-                termRangeNode.getField(), lowerNumber, numberFormat);
-            NumericQueryNode upperNode = new NumericQueryNode(
-                termRangeNode.getField(), upperNumber, numberFormat);
-            
-            boolean lowerInclusive = termRangeNode.isLowerInclusive();
-            boolean upperInclusive = termRangeNode.isUpperInclusive();
-            
-            return new NumericRangeQueryNode(lowerNode, upperNode,
-                lowerInclusive, upperInclusive, 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;
-  }
-  
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/89cc676f/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/processors/PointQueryNodeProcessor.java
----------------------------------------------------------------------
diff --git a/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/processors/PointQueryNodeProcessor.java b/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/processors/PointQueryNodeProcessor.java
new file mode 100644
index 0000000..81a8449
--- /dev/null
+++ b/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/processors/PointQueryNodeProcessor.java
@@ -0,0 +1,136 @@
+/*
+ * 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.processors;
+
+import java.text.NumberFormat;
+import java.text.ParseException;
+import java.util.List;
+
+import org.apache.lucene.queryparser.flexible.messages.MessageImpl;
+import org.apache.lucene.queryparser.flexible.core.QueryNodeException;
+import org.apache.lucene.queryparser.flexible.core.QueryNodeParseException;
+import org.apache.lucene.queryparser.flexible.core.config.FieldConfig;
+import org.apache.lucene.queryparser.flexible.core.config.QueryConfigHandler;
+import org.apache.lucene.queryparser.flexible.core.messages.QueryParserMessages;
+import org.apache.lucene.queryparser.flexible.core.nodes.FieldQueryNode;
+import org.apache.lucene.queryparser.flexible.core.nodes.QueryNode;
+import org.apache.lucene.queryparser.flexible.core.nodes.RangeQueryNode;
+import org.apache.lucene.queryparser.flexible.core.processors.QueryNodeProcessorImpl;
+import org.apache.lucene.queryparser.flexible.standard.config.PointsConfig;
+import org.apache.lucene.queryparser.flexible.standard.config.StandardQueryConfigHandler.ConfigurationKeys;
+import org.apache.lucene.queryparser.flexible.standard.nodes.PointQueryNode;
+import org.apache.lucene.queryparser.flexible.standard.nodes.PointRangeQueryNode;
+
+/**
+ * This processor is used to convert {@link FieldQueryNode}s to
+ * {@link PointRangeQueryNode}s. It looks for
+ * {@link ConfigurationKeys#POINTS_CONFIG} set in the {@link FieldConfig} of
+ * every {@link FieldQueryNode} found. If
+ * {@link ConfigurationKeys#POINTS_CONFIG} is found, it considers that
+ * {@link FieldQueryNode} to be a numeric query and convert it to
+ * {@link PointRangeQueryNode} 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 FieldQueryNode}s children of a
+ * {@link RangeQueryNode} are ignored.
+ * 
+ * @see ConfigurationKeys#POINTS_CONFIG
+ * @see FieldQueryNode
+ * @see PointsConfig
+ * @see PointQueryNode
+ */
+public class PointQueryNodeProcessor extends QueryNodeProcessorImpl {
+  
+  /**
+   * Constructs a {@link PointQueryNodeProcessor} object.
+   */
+  public PointQueryNodeProcessor() {
+  // empty constructor
+  }
+  
+  @Override
+  protected QueryNode postProcessNode(QueryNode node) throws QueryNodeException {
+    
+    if (node instanceof FieldQueryNode
+        && !(node.getParent() instanceof RangeQueryNode)) {
+      
+      QueryConfigHandler config = getQueryConfigHandler();
+      
+      if (config != null) {
+        FieldQueryNode fieldNode = (FieldQueryNode) node;
+        FieldConfig fieldConfig = config.getFieldConfig(fieldNode
+            .getFieldAsString());
+        
+        if (fieldConfig != null) {
+          PointsConfig numericConfig = fieldConfig.get(ConfigurationKeys.POINTS_CONFIG);
+          
+          if (numericConfig != null) {
+            
+            NumberFormat numberFormat = numericConfig.getNumberFormat();
+            String text = fieldNode.getTextAsString();
+            Number number = null;
+            
+            if (text.length() > 0) {
+              
+              try {
+                number = numberFormat.parse(text);
+                
+              } catch (ParseException e) {
+                throw new QueryNodeParseException(new MessageImpl(
+                    QueryParserMessages.COULD_NOT_PARSE_NUMBER, fieldNode
+                        .getTextAsString(), numberFormat.getClass()
+                        .getCanonicalName()), e);
+              }
+              
+              if (Integer.class.equals(numericConfig.getType())) {
+                number = number.intValue();
+              } else if (Long.class.equals(numericConfig.getType())) {
+                number = number.longValue();
+              } else if (Double.class.equals(numericConfig.getType())) {
+                number = number.doubleValue();
+              } else if (Float.class.equals(numericConfig.getType())) {
+                number = number.floatValue();
+              }
+              
+            } else {
+              throw new QueryNodeParseException(new MessageImpl(
+                  QueryParserMessages.NUMERIC_CANNOT_BE_EMPTY, fieldNode.getFieldAsString()));
+            }
+            
+            PointQueryNode lowerNode = new PointQueryNode(fieldNode.getField(), number, numberFormat);
+            PointQueryNode upperNode = new PointQueryNode(fieldNode.getField(), number, numberFormat);
+            
+            return new PointRangeQueryNode(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;
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/89cc676f/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/processors/PointRangeQueryNodeProcessor.java
----------------------------------------------------------------------
diff --git a/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/processors/PointRangeQueryNodeProcessor.java b/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/processors/PointRangeQueryNodeProcessor.java
new file mode 100644
index 0000000..2ffc437
--- /dev/null
+++ b/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/processors/PointRangeQueryNodeProcessor.java
@@ -0,0 +1,148 @@
+/*
+ * 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.processors;
+
+import java.text.NumberFormat;
+import java.text.ParseException;
+import java.util.List;
+
+import org.apache.lucene.queryparser.flexible.messages.MessageImpl;
+import org.apache.lucene.queryparser.flexible.core.QueryNodeException;
+import org.apache.lucene.queryparser.flexible.core.QueryNodeParseException;
+import org.apache.lucene.queryparser.flexible.core.config.FieldConfig;
+import org.apache.lucene.queryparser.flexible.core.config.QueryConfigHandler;
+import org.apache.lucene.queryparser.flexible.core.messages.QueryParserMessages;
+import org.apache.lucene.queryparser.flexible.core.nodes.FieldQueryNode;
+import org.apache.lucene.queryparser.flexible.core.nodes.QueryNode;
+import org.apache.lucene.queryparser.flexible.core.processors.QueryNodeProcessorImpl;
+import org.apache.lucene.queryparser.flexible.core.util.StringUtils;
+import org.apache.lucene.queryparser.flexible.standard.config.PointsConfig;
+import org.apache.lucene.queryparser.flexible.standard.config.StandardQueryConfigHandler.ConfigurationKeys;
+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.TermRangeQueryNode;
+
+/**
+ * This processor is used to convert {@link TermRangeQueryNode}s to
+ * {@link PointRangeQueryNode}s. It looks for
+ * {@link ConfigurationKeys#POINTS_CONFIG} set in the {@link FieldConfig} of
+ * every {@link TermRangeQueryNode} found. If
+ * {@link ConfigurationKeys#POINTS_CONFIG} is found, it considers that
+ * {@link TermRangeQueryNode} to be a numeric range query and convert it to
+ * {@link PointRangeQueryNode}.
+ * 
+ * @see ConfigurationKeys#POINTS_CONFIG
+ * @see TermRangeQueryNode
+ * @see PointsConfig
+ * @see PointRangeQueryNode
+ */
+public class PointRangeQueryNodeProcessor extends QueryNodeProcessorImpl {
+
+  /**
+   * Constructs an empty {@link PointRangeQueryNodeProcessor} object.
+   */
+  public PointRangeQueryNodeProcessor() {
+    // empty constructor
+  }
+
+  @Override
+  protected QueryNode postProcessNode(QueryNode node) throws QueryNodeException {
+
+    if (node instanceof TermRangeQueryNode) {
+      QueryConfigHandler config = getQueryConfigHandler();
+
+      if (config != null) {
+        TermRangeQueryNode termRangeNode = (TermRangeQueryNode) node;
+        FieldConfig fieldConfig = config.getFieldConfig(StringUtils.toString(termRangeNode.getField()));
+
+        if (fieldConfig != null) {
+          PointsConfig numericConfig = fieldConfig.get(ConfigurationKeys.POINTS_CONFIG);
+
+          if (numericConfig != null) {
+            FieldQueryNode lower = termRangeNode.getLowerBound();
+            FieldQueryNode upper = termRangeNode.getUpperBound();
+
+            String lowerText = lower.getTextAsString();
+            String upperText = upper.getTextAsString();
+            NumberFormat numberFormat = numericConfig.getNumberFormat();
+            Number lowerNumber = null, upperNumber = null;
+
+            if (lowerText.length() > 0) {
+
+              try {
+                lowerNumber = numberFormat.parse(lowerText);
+
+              } catch (ParseException e) {
+                throw new QueryNodeParseException(new MessageImpl(
+                    QueryParserMessages.COULD_NOT_PARSE_NUMBER, lower
+                    .getTextAsString(), numberFormat.getClass()
+                    .getCanonicalName()), e);
+              }
+
+            }
+
+            if (upperText.length() > 0) {
+
+              try {
+                upperNumber = numberFormat.parse(upperText);
+
+              } catch (ParseException e) {
+                throw new QueryNodeParseException(new MessageImpl(
+                    QueryParserMessages.COULD_NOT_PARSE_NUMBER, upper
+                    .getTextAsString(), numberFormat.getClass()
+                    .getCanonicalName()), e);
+              }
+            }
+
+            if (Integer.class.equals(numericConfig.getType())) {
+              if (upperNumber != null) upperNumber = upperNumber.intValue();
+              if (lowerNumber != null) lowerNumber = lowerNumber.intValue();
+            } else if (Long.class.equals(numericConfig.getType())) {
+              if (upperNumber != null) upperNumber = upperNumber.longValue();
+              if (lowerNumber != null) lowerNumber = lowerNumber.longValue();
+            } else if (Double.class.equals(numericConfig.getType())) {
+              if (upperNumber != null) upperNumber = upperNumber.doubleValue();
+              if (lowerNumber != null) lowerNumber = lowerNumber.doubleValue();
+            } else if (Float.class.equals(numericConfig.getType())) {
+              if (upperNumber != null) upperNumber = upperNumber.floatValue();
+              if (lowerNumber != null) lowerNumber = lowerNumber.floatValue();
+            }
+
+            PointQueryNode lowerNode = new PointQueryNode(termRangeNode.getField(), lowerNumber, numberFormat);
+            PointQueryNode upperNode = new PointQueryNode(termRangeNode.getField(), upperNumber, numberFormat);
+
+            boolean lowerInclusive = termRangeNode.isLowerInclusive();
+            boolean upperInclusive = termRangeNode.isUpperInclusive();
+
+            return new PointRangeQueryNode(lowerNode, upperNode, lowerInclusive, upperInclusive, 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;
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/89cc676f/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/processors/StandardQueryNodeProcessorPipeline.java
----------------------------------------------------------------------
diff --git a/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/processors/StandardQueryNodeProcessorPipeline.java b/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/processors/StandardQueryNodeProcessorPipeline.java
index 06f38c2..6e4a394 100644
--- a/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/processors/StandardQueryNodeProcessorPipeline.java
+++ b/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/processors/StandardQueryNodeProcessorPipeline.java
@@ -55,8 +55,10 @@ public class StandardQueryNodeProcessorPipeline extends
     add(new FuzzyQueryNodeProcessor());
     add(new MatchAllDocsQueryNodeProcessor());
     add(new OpenRangeQueryNodeProcessor());
-    add(new NumericQueryNodeProcessor());
-    add(new NumericRangeQueryNodeProcessor());
+    add(new LegacyNumericQueryNodeProcessor());
+    add(new LegacyNumericRangeQueryNodeProcessor());
+    add(new PointQueryNodeProcessor());
+    add(new PointRangeQueryNodeProcessor());
     add(new LowercaseExpandedTermsQueryNodeProcessor());
     add(new TermRangeQueryNodeProcessor());
     add(new AllowLeadingWildcardProcessor());    

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/89cc676f/lucene/queryparser/src/test/org/apache/lucene/queryparser/flexible/standard/TestLegacyNumericQueryParser.java
----------------------------------------------------------------------
diff --git a/lucene/queryparser/src/test/org/apache/lucene/queryparser/flexible/standard/TestLegacyNumericQueryParser.java b/lucene/queryparser/src/test/org/apache/lucene/queryparser/flexible/standard/TestLegacyNumericQueryParser.java
new file mode 100644
index 0000000..c6ab7f5
--- /dev/null
+++ b/lucene/queryparser/src/test/org/apache/lucene/queryparser/flexible/standard/TestLegacyNumericQueryParser.java
@@ -0,0 +1,535 @@
+/*
+ * 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;
+
+import java.io.IOException;
+import java.text.DateFormat;
+import java.text.NumberFormat;
+import java.text.ParseException;
+import java.text.SimpleDateFormat;
+import java.util.Collections;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Random;
+import java.util.TimeZone;
+
+import org.apache.lucene.analysis.Analyzer;
+import org.apache.lucene.analysis.MockAnalyzer;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.LegacyDoubleField;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.document.FieldType.LegacyNumericType;
+import org.apache.lucene.document.FieldType;
+import org.apache.lucene.document.LegacyFloatField;
+import org.apache.lucene.document.LegacyIntField;
+import org.apache.lucene.document.LegacyLongField;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.RandomIndexWriter;
+import org.apache.lucene.queryparser.flexible.core.QueryNodeException;
+import org.apache.lucene.queryparser.flexible.core.parser.EscapeQuerySyntax;
+import org.apache.lucene.queryparser.flexible.standard.config.NumberDateFormat;
+import org.apache.lucene.queryparser.flexible.standard.config.LegacyNumericConfig;
+import org.apache.lucene.queryparser.flexible.standard.parser.EscapeQuerySyntaxImpl;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.TestUtil;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class TestLegacyNumericQueryParser extends LuceneTestCase {
+  
+  private static enum NumberType {
+    NEGATIVE, ZERO, POSITIVE;
+  }
+  
+  final private static int[] DATE_STYLES = {DateFormat.FULL, DateFormat.LONG,
+      DateFormat.MEDIUM, DateFormat.SHORT};
+  
+  final private static int PRECISION_STEP = 8;
+  final private static String FIELD_NAME = "field";
+  private static Locale LOCALE;
+  private static TimeZone TIMEZONE;
+  private static Map<String,Number> RANDOM_NUMBER_MAP;
+  private static EscapeQuerySyntax ESCAPER = new EscapeQuerySyntaxImpl();
+  final private static String DATE_FIELD_NAME = "date";
+  private static int DATE_STYLE;
+  private static int TIME_STYLE;
+  
+  private static Analyzer ANALYZER;
+  
+  private static NumberFormat NUMBER_FORMAT;
+  
+  private static StandardQueryParser qp;
+  
+  private static NumberDateFormat DATE_FORMAT;
+  
+  private static Directory directory = null;
+  private static IndexReader reader = null;
+  private static IndexSearcher searcher = null;
+  
+  private static boolean checkDateFormatSanity(DateFormat dateFormat, long date) {
+    try {
+      return date == dateFormat.parse(dateFormat.format(new Date(date)))
+        .getTime();
+    } catch (ParseException e) {
+      return false;
+    }
+  }
+  
+  @BeforeClass
+  public static void beforeClass() throws Exception {
+    ANALYZER = new MockAnalyzer(random());
+    
+    qp = new StandardQueryParser(ANALYZER);
+    
+    final HashMap<String,Number> randomNumberMap = new HashMap<>();
+    
+    SimpleDateFormat dateFormat;
+    long randomDate;
+    boolean dateFormatSanityCheckPass;
+    int count = 0;
+    do {
+      if (count > 100) {
+        fail("This test has problems to find a sane random DateFormat/NumberFormat. Stopped trying after 100 iterations.");
+      }
+      
+      dateFormatSanityCheckPass = true;
+      LOCALE = randomLocale(random());
+      TIMEZONE = randomTimeZone(random());
+      DATE_STYLE = randomDateStyle(random());
+      TIME_STYLE = randomDateStyle(random());
+      
+      // assumes localized date pattern will have at least year, month, day,
+      // hour, minute
+      dateFormat = (SimpleDateFormat) DateFormat.getDateTimeInstance(
+          DATE_STYLE, TIME_STYLE, LOCALE);
+      
+      // not all date patterns includes era, full year, timezone and second,
+      // so we add them here
+      dateFormat.applyPattern(dateFormat.toPattern() + " G s Z yyyy");
+      dateFormat.setTimeZone(TIMEZONE);
+      
+      DATE_FORMAT = new NumberDateFormat(dateFormat);
+      
+      do {
+        randomDate = random().nextLong();
+        
+        // prune date value so it doesn't pass in insane values to some
+        // calendars.
+        randomDate = randomDate % 3400000000000l;
+        
+        // truncate to second
+        randomDate = (randomDate / 1000L) * 1000L;
+        
+        // only positive values
+        randomDate = Math.abs(randomDate);
+      } while (randomDate == 0L);
+      
+      dateFormatSanityCheckPass &= checkDateFormatSanity(dateFormat, randomDate);
+      
+      dateFormatSanityCheckPass &= checkDateFormatSanity(dateFormat, 0);
+      
+      dateFormatSanityCheckPass &= checkDateFormatSanity(dateFormat,
+          -randomDate);
+      
+      count++;
+    } while (!dateFormatSanityCheckPass);
+    
+    NUMBER_FORMAT = NumberFormat.getNumberInstance(LOCALE);
+    NUMBER_FORMAT.setMaximumFractionDigits((random().nextInt() & 20) + 1);
+    NUMBER_FORMAT.setMinimumFractionDigits((random().nextInt() & 20) + 1);
+    NUMBER_FORMAT.setMaximumIntegerDigits((random().nextInt() & 20) + 1);
+    NUMBER_FORMAT.setMinimumIntegerDigits((random().nextInt() & 20) + 1);
+    
+    double randomDouble;
+    long randomLong;
+    int randomInt;
+    float randomFloat;
+    
+    while ((randomLong = normalizeNumber(Math.abs(random().nextLong()))
+        .longValue()) == 0L)
+      ;
+    while ((randomDouble = normalizeNumber(Math.abs(random().nextDouble()))
+        .doubleValue()) == 0.0)
+      ;
+    while ((randomFloat = normalizeNumber(Math.abs(random().nextFloat()))
+        .floatValue()) == 0.0f)
+      ;
+    while ((randomInt = normalizeNumber(Math.abs(random().nextInt())).intValue()) == 0)
+      ;
+    
+    randomNumberMap.put(LegacyNumericType.LONG.name(), randomLong);
+    randomNumberMap.put(FieldType.LegacyNumericType.INT.name(), randomInt);
+    randomNumberMap.put(LegacyNumericType.FLOAT.name(), randomFloat);
+    randomNumberMap.put(LegacyNumericType.DOUBLE.name(), randomDouble);
+    randomNumberMap.put(DATE_FIELD_NAME, randomDate);
+    
+    RANDOM_NUMBER_MAP = Collections.unmodifiableMap(randomNumberMap);
+    
+    directory = newDirectory();
+    RandomIndexWriter writer = new RandomIndexWriter(random(), directory,
+        newIndexWriterConfig(new MockAnalyzer(random()))
+            .setMaxBufferedDocs(TestUtil.nextInt(random(), 50, 1000))
+            .setMergePolicy(newLogMergePolicy()));
+    
+    Document doc = new Document();
+    HashMap<String,LegacyNumericConfig> numericConfigMap = new HashMap<>();
+    HashMap<String,Field> numericFieldMap = new HashMap<>();
+    qp.setLegacyNumericConfigMap(numericConfigMap);
+    
+    for (LegacyNumericType type : LegacyNumericType.values()) {
+      numericConfigMap.put(type.name(), new LegacyNumericConfig(PRECISION_STEP,
+          NUMBER_FORMAT, type));
+
+      FieldType ft = new FieldType(LegacyIntField.TYPE_NOT_STORED);
+      ft.setNumericType(type);
+      ft.setStored(true);
+      ft.setNumericPrecisionStep(PRECISION_STEP);
+      ft.freeze();
+      final Field field;
+
+      switch(type) {
+      case INT:
+        field = new LegacyIntField(type.name(), 0, ft);
+        break;
+      case FLOAT:
+        field = new LegacyFloatField(type.name(), 0.0f, ft);
+        break;
+      case LONG:
+        field = new LegacyLongField(type.name(), 0l, ft);
+        break;
+      case DOUBLE:
+        field = new LegacyDoubleField(type.name(), 0.0, ft);
+        break;
+      default:
+        fail();
+        field = null;
+      }
+      numericFieldMap.put(type.name(), field);
+      doc.add(field);
+    }
+    
+    numericConfigMap.put(DATE_FIELD_NAME, new LegacyNumericConfig(PRECISION_STEP,
+        DATE_FORMAT, LegacyNumericType.LONG));
+    FieldType ft = new FieldType(LegacyLongField.TYPE_NOT_STORED);
+    ft.setStored(true);
+    ft.setNumericPrecisionStep(PRECISION_STEP);
+    LegacyLongField dateField = new LegacyLongField(DATE_FIELD_NAME, 0l, ft);
+    numericFieldMap.put(DATE_FIELD_NAME, dateField);
+    doc.add(dateField);
+    
+    for (NumberType numberType : NumberType.values()) {
+      setFieldValues(numberType, numericFieldMap);
+      if (VERBOSE) System.out.println("Indexing document: " + doc);
+      writer.addDocument(doc);
+    }
+    
+    reader = writer.getReader();
+    searcher = newSearcher(reader);
+    writer.close();
+    
+  }
+  
+  private static Number getNumberType(NumberType numberType, String fieldName) {
+    
+    if (numberType == null) {
+      return null;
+    }
+    
+    switch (numberType) {
+      
+      case POSITIVE:
+        return RANDOM_NUMBER_MAP.get(fieldName);
+        
+      case NEGATIVE:
+        Number number = RANDOM_NUMBER_MAP.get(fieldName);
+        
+        if (LegacyNumericType.LONG.name().equals(fieldName)
+            || DATE_FIELD_NAME.equals(fieldName)) {
+          number = -number.longValue();
+          
+        } else if (FieldType.LegacyNumericType.DOUBLE.name().equals(fieldName)) {
+          number = -number.doubleValue();
+          
+        } else if (FieldType.LegacyNumericType.FLOAT.name().equals(fieldName)) {
+          number = -number.floatValue();
+          
+        } else if (LegacyNumericType.INT.name().equals(fieldName)) {
+          number = -number.intValue();
+          
+        } else {
+          throw new IllegalArgumentException("field name not found: "
+              + fieldName);
+        }
+        
+        return number;
+        
+      default:
+        return 0;
+        
+    }
+    
+  }
+  
+  private static void setFieldValues(NumberType numberType,
+      HashMap<String,Field> numericFieldMap) {
+    
+    Number number = getNumberType(numberType, LegacyNumericType.DOUBLE
+        .name());
+    numericFieldMap.get(LegacyNumericType.DOUBLE.name()).setDoubleValue(
+        number.doubleValue());
+    
+    number = getNumberType(numberType, FieldType.LegacyNumericType.INT.name());
+    numericFieldMap.get(FieldType.LegacyNumericType.INT.name()).setIntValue(
+        number.intValue());
+    
+    number = getNumberType(numberType, LegacyNumericType.LONG.name());
+    numericFieldMap.get(FieldType.LegacyNumericType.LONG.name()).setLongValue(
+        number.longValue());
+    
+    number = getNumberType(numberType, FieldType.LegacyNumericType.FLOAT.name());
+    numericFieldMap.get(FieldType.LegacyNumericType.FLOAT.name()).setFloatValue(
+        number.floatValue());
+    
+    number = getNumberType(numberType, DATE_FIELD_NAME);
+    numericFieldMap.get(DATE_FIELD_NAME).setLongValue(number.longValue());
+  }
+  
+  private static int randomDateStyle(Random random) {
+    return DATE_STYLES[random.nextInt(DATE_STYLES.length)];
+  }
+  
+  @Test
+  public void testInclusiveNumericRange() throws Exception {
+    assertRangeQuery(NumberType.ZERO, NumberType.ZERO, true, true, 1);
+    assertRangeQuery(NumberType.ZERO, NumberType.POSITIVE, true, true, 2);
+    assertRangeQuery(NumberType.NEGATIVE, NumberType.ZERO, true, true, 2);
+    assertRangeQuery(NumberType.NEGATIVE, NumberType.POSITIVE, true, true, 3);
+    assertRangeQuery(NumberType.NEGATIVE, NumberType.NEGATIVE, true, true, 1);
+  }
+  
+   @Test
+  // test disabled since standard syntax parser does not work with inclusive and
+  // exclusive at the same time
+  public void testInclusiveLowerNumericRange() throws Exception {
+    assertRangeQuery(NumberType.NEGATIVE, NumberType.ZERO, false, true, 1);
+    assertRangeQuery(NumberType.ZERO, NumberType.POSITIVE, false, true, 1);
+    assertRangeQuery(NumberType.NEGATIVE, NumberType.POSITIVE, false, true, 2);
+    assertRangeQuery(NumberType.NEGATIVE, NumberType.NEGATIVE, false, true, 0);
+   }
+  
+  @Test
+  // test disabled since standard syntax parser does not work with inclusive and
+  // exclusive at the same time
+  public void testInclusiveUpperNumericRange() throws Exception {
+    assertRangeQuery(NumberType.NEGATIVE, NumberType.ZERO, true, false, 1);
+    assertRangeQuery(NumberType.ZERO, NumberType.POSITIVE, true, false, 1);
+    assertRangeQuery(NumberType.NEGATIVE, NumberType.POSITIVE, true, false, 2);
+    assertRangeQuery(NumberType.NEGATIVE, NumberType.NEGATIVE, true, false, 0);
+  }
+  
+  @Test
+  public void testExclusiveNumericRange() throws Exception {
+    assertRangeQuery(NumberType.ZERO, NumberType.ZERO, false, false, 0);
+    assertRangeQuery(NumberType.ZERO, NumberType.POSITIVE, false, false, 0);
+    assertRangeQuery(NumberType.NEGATIVE, NumberType.ZERO, false, false, 0);
+    assertRangeQuery(NumberType.NEGATIVE, NumberType.POSITIVE, false, false, 1);
+    assertRangeQuery(NumberType.NEGATIVE, NumberType.NEGATIVE, false, false, 0);
+  }
+  
+  @Test
+  public void testOpenRangeNumericQuery() throws Exception {
+    assertOpenRangeQuery(NumberType.ZERO, "<", 1);
+    assertOpenRangeQuery(NumberType.POSITIVE, "<", 2);
+    assertOpenRangeQuery(NumberType.NEGATIVE, "<", 0);
+    
+    assertOpenRangeQuery(NumberType.ZERO, "<=", 2);
+    assertOpenRangeQuery(NumberType.POSITIVE, "<=", 3);
+    assertOpenRangeQuery(NumberType.NEGATIVE, "<=", 1);
+    
+    assertOpenRangeQuery(NumberType.ZERO, ">", 1);
+    assertOpenRangeQuery(NumberType.POSITIVE, ">", 0);
+    assertOpenRangeQuery(NumberType.NEGATIVE, ">", 2);
+    
+    assertOpenRangeQuery(NumberType.ZERO, ">=", 2);
+    assertOpenRangeQuery(NumberType.POSITIVE, ">=", 1);
+    assertOpenRangeQuery(NumberType.NEGATIVE, ">=", 3);
+    
+    assertOpenRangeQuery(NumberType.NEGATIVE, "=", 1);
+    assertOpenRangeQuery(NumberType.ZERO, "=", 1);
+    assertOpenRangeQuery(NumberType.POSITIVE, "=", 1);
+    
+    assertRangeQuery(NumberType.NEGATIVE, null, true, true, 3);
+    assertRangeQuery(NumberType.NEGATIVE, null, false, true, 2);
+    assertRangeQuery(NumberType.POSITIVE, null, true, false, 1);
+    assertRangeQuery(NumberType.ZERO, null, false, false, 1);
+
+    assertRangeQuery(null, NumberType.POSITIVE, true, true, 3);
+    assertRangeQuery(null, NumberType.POSITIVE, true, false, 2);
+    assertRangeQuery(null, NumberType.NEGATIVE, false, true, 1);
+    assertRangeQuery(null, NumberType.ZERO, false, false, 1);
+    
+    assertRangeQuery(null, null, false, false, 3);
+    assertRangeQuery(null, null, true, true, 3);
+    
+  }
+  
+  @Test
+  public void testSimpleNumericQuery() throws Exception {
+    assertSimpleQuery(NumberType.ZERO, 1);
+    assertSimpleQuery(NumberType.POSITIVE, 1);
+    assertSimpleQuery(NumberType.NEGATIVE, 1);
+  }
+  
+  public void assertRangeQuery(NumberType lowerType, NumberType upperType,
+      boolean lowerInclusive, boolean upperInclusive, int expectedDocCount)
+      throws QueryNodeException, IOException {
+    
+    StringBuilder sb = new StringBuilder();
+    
+    String lowerInclusiveStr = (lowerInclusive ? "[" : "{");
+    String upperInclusiveStr = (upperInclusive ? "]" : "}");
+    
+    for (LegacyNumericType type : LegacyNumericType.values()) {
+      String lowerStr = numberToString(getNumberType(lowerType, type.name()));
+      String upperStr = numberToString(getNumberType(upperType, type.name()));
+      
+      sb.append("+").append(type.name()).append(':').append(lowerInclusiveStr)
+          .append('"').append(lowerStr).append("\" TO \"").append(upperStr)
+          .append('"').append(upperInclusiveStr).append(' ');
+    }
+    
+    Number lowerDateNumber = getNumberType(lowerType, DATE_FIELD_NAME);
+    Number upperDateNumber = getNumberType(upperType, DATE_FIELD_NAME);
+    String lowerDateStr;
+    String upperDateStr;
+    
+    if (lowerDateNumber != null) {
+      lowerDateStr = ESCAPER.escape(
+          DATE_FORMAT.format(new Date(lowerDateNumber.longValue())), LOCALE,
+          EscapeQuerySyntax.Type.STRING).toString();
+      
+    } else {
+      lowerDateStr = "*";
+    }
+    
+    if (upperDateNumber != null) {
+    upperDateStr = ESCAPER.escape(
+          DATE_FORMAT.format(new Date(upperDateNumber.longValue())), LOCALE,
+          EscapeQuerySyntax.Type.STRING).toString();
+    
+    } else {
+      upperDateStr = "*";
+    }
+    
+    sb.append("+").append(DATE_FIELD_NAME).append(':')
+        .append(lowerInclusiveStr).append('"').append(lowerDateStr).append(
+            "\" TO \"").append(upperDateStr).append('"').append(
+            upperInclusiveStr);
+    
+    testQuery(sb.toString(), expectedDocCount);
+    
+  }
+  
+  public void assertOpenRangeQuery(NumberType boundType, String operator, int expectedDocCount)
+      throws QueryNodeException, IOException {
+
+    StringBuilder sb = new StringBuilder();
+    
+    for (LegacyNumericType type : FieldType.LegacyNumericType.values()) {
+      String boundStr = numberToString(getNumberType(boundType, type.name()));
+      
+      sb.append("+").append(type.name()).append(operator).append('"').append(boundStr).append('"').append(' ');
+    }
+    
+    String boundDateStr = ESCAPER.escape(
+        DATE_FORMAT.format(new Date(getNumberType(boundType, DATE_FIELD_NAME)
+            .longValue())), LOCALE, EscapeQuerySyntax.Type.STRING).toString();
+    
+    sb.append("+").append(DATE_FIELD_NAME).append(operator).append('"').append(boundDateStr).append('"');
+    
+    testQuery(sb.toString(), expectedDocCount);
+  }
+  
+  public void assertSimpleQuery(NumberType numberType, int expectedDocCount)
+      throws QueryNodeException, IOException {
+    StringBuilder sb = new StringBuilder();
+    
+    for (LegacyNumericType type : LegacyNumericType.values()) {
+      String numberStr = numberToString(getNumberType(numberType, type.name()));
+      sb.append('+').append(type.name()).append(":\"").append(numberStr)
+          .append("\" ");
+    }
+    
+    String dateStr = ESCAPER.escape(
+        DATE_FORMAT.format(new Date(getNumberType(numberType, DATE_FIELD_NAME)
+            .longValue())), LOCALE, EscapeQuerySyntax.Type.STRING).toString();
+    
+    sb.append('+').append(DATE_FIELD_NAME).append(":\"").append(dateStr)
+        .append('"');
+    
+    testQuery(sb.toString(), expectedDocCount);
+    
+  }
+  
+  private void testQuery(String queryStr, int expectedDocCount)
+      throws QueryNodeException, IOException {
+    if (VERBOSE) System.out.println("Parsing: " + queryStr);
+    
+    Query query = qp.parse(queryStr, FIELD_NAME);
+    if (VERBOSE) System.out.println("Querying: " + query);
+    TopDocs topDocs = searcher.search(query, 1000);
+    
+    String msg = "Query <" + queryStr + "> retrieved " + topDocs.totalHits
+        + " document(s), " + expectedDocCount + " document(s) expected.";
+    
+    if (VERBOSE) System.out.println(msg);
+    
+    assertEquals(msg, expectedDocCount, topDocs.totalHits);
+  }
+  
+  private static String numberToString(Number number) {
+    return number == null ? "*" : ESCAPER.escape(NUMBER_FORMAT.format(number),
+        LOCALE, EscapeQuerySyntax.Type.STRING).toString();
+  }
+  
+  private static Number normalizeNumber(Number number) throws ParseException {
+    return NUMBER_FORMAT.parse(NUMBER_FORMAT.format(number));
+  }
+  
+  @AfterClass
+  public static void afterClass() throws Exception {
+    searcher = null;
+    reader.close();
+    reader = null;
+    directory.close();
+    directory = null;
+    qp = null;
+    LOCALE = null;
+    TIMEZONE = null;
+    NUMBER_FORMAT = null;
+    DATE_FORMAT = null;
+    ESCAPER = null;
+  }
+  
+}