You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by rm...@apache.org on 2016/08/22 12:40:24 UTC

[2/2] lucene-solr:master: LUCENE-7418: remove deprecated legacy numerics from join/ and queryparser/

LUCENE-7418: remove deprecated legacy numerics from join/ and queryparser/


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

Branch: refs/heads/master
Commit: 5347cc8ea7ec390c33584fce37c85ce118866e98
Parents: c9c2d55
Author: Robert Muir <rm...@apache.org>
Authored: Mon Aug 22 08:39:13 2016 -0400
Committer: Robert Muir <rm...@apache.org>
Committed: Mon Aug 22 08:39:13 2016 -0400

----------------------------------------------------------------------
 dev-tools/idea/lucene/join/join.iml             |   1 -
 .../idea/lucene/queryparser/queryparser.iml     |   1 -
 lucene/join/build.xml                           |   6 +-
 .../search/join/DocValuesTermsCollector.java    |  82 ---
 .../org/apache/lucene/search/join/JoinUtil.java |  45 --
 .../search/join/TermsIncludingScoreQuery.java   |   9 -
 .../apache/lucene/search/join/TestJoinUtil.java |  23 +-
 lucene/queryparser/build.xml                    |   6 +-
 .../flexible/standard/StandardQueryParser.java  |  19 -
 .../LegacyNumericRangeQueryNodeBuilder.java     |  93 ----
 .../builders/StandardQueryTreeBuilder.java      |   4 -
 .../standard/config/LegacyNumericConfig.java    | 165 ------
 .../LegacyNumericFieldConfigListener.java       |  75 ---
 .../config/StandardQueryConfigHandler.java      |  29 +-
 .../standard/nodes/LegacyNumericQueryNode.java  | 153 ------
 .../nodes/LegacyNumericRangeQueryNode.java      | 152 ------
 .../LegacyNumericQueryNodeProcessor.java        | 154 ------
 .../LegacyNumericRangeQueryNodeProcessor.java   | 170 ------
 .../StandardQueryNodeProcessorPipeline.java     |   2 -
 .../lucene/queryparser/xml/CoreParser.java      |   1 -
 .../LegacyNumericRangeQueryBuilder.java         | 135 -----
 .../standard/TestLegacyNumericQueryParser.java  | 535 -------------------
 .../xml/CoreParserTestIndexData.java            |   2 -
 .../queryparser/xml/LegacyNumericRangeQuery.xml |  31 --
 .../LegacyNumericRangeQueryWithoutLowerTerm.xml |  31 --
 .../xml/LegacyNumericRangeQueryWithoutRange.xml |  31 --
 .../LegacyNumericRangeQueryWithoutUpperTerm.xml |  31 --
 .../lucene/queryparser/xml/TestCoreParser.java  |  20 -
 .../builders/TestNumericRangeQueryBuilder.java  | 179 -------
 .../search/LegacyNumericRangeQueryBuilder.java  | 136 +++++
 .../org/apache/solr/search/SolrCoreParser.java  |   2 +-
 .../TestLegacyNumericRangeQueryBuilder.java     | 179 +++++++
 32 files changed, 331 insertions(+), 2171 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/5347cc8e/dev-tools/idea/lucene/join/join.iml
----------------------------------------------------------------------
diff --git a/dev-tools/idea/lucene/join/join.iml b/dev-tools/idea/lucene/join/join.iml
index 6de5e90..1f9e80b 100644
--- a/dev-tools/idea/lucene/join/join.iml
+++ b/dev-tools/idea/lucene/join/join.iml
@@ -14,7 +14,6 @@
     <orderEntry type="library" scope="TEST" name="JUnit" level="project" />
     <orderEntry type="module" scope="TEST" module-name="lucene-test-framework" />
     <orderEntry type="module" module-name="grouping" />
-    <orderEntry type="module" module-name="backward-codecs" />
     <orderEntry type="module" module-name="lucene-core" />
   </component>
 </module>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/5347cc8e/dev-tools/idea/lucene/queryparser/queryparser.iml
----------------------------------------------------------------------
diff --git a/dev-tools/idea/lucene/queryparser/queryparser.iml b/dev-tools/idea/lucene/queryparser/queryparser.iml
index 86a50a5..cd2915f 100644
--- a/dev-tools/idea/lucene/queryparser/queryparser.iml
+++ b/dev-tools/idea/lucene/queryparser/queryparser.iml
@@ -17,6 +17,5 @@
     <orderEntry type="module" module-name="lucene-core" />
     <orderEntry type="module" module-name="queries" />
     <orderEntry type="module" module-name="sandbox" />
-    <orderEntry type="module" module-name="backward-codecs" />
   </component>
 </module>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/5347cc8e/lucene/join/build.xml
----------------------------------------------------------------------
diff --git a/lucene/join/build.xml b/lucene/join/build.xml
index b6878b8..b5360c4 100644
--- a/lucene/join/build.xml
+++ b/lucene/join/build.xml
@@ -26,7 +26,6 @@
 
   <path id="classpath">
     <pathelement path="${grouping.jar}"/>
-    <pathelement path="${backward-codecs.jar}"/>
     <path refid="base.classpath"/>
   </path>
 
@@ -35,14 +34,13 @@
     <pathelement location="${build.dir}/classes/java"/>
   </path>
 
-  <target name="init" depends="module-build.init,jar-grouping,jar-backward-codecs"/>
+  <target name="init" depends="module-build.init,jar-grouping"/>
 
-  <target name="javadocs" depends="javadocs-grouping,javadocs-backward-codecs,compile-core,check-javadocs-uptodate"
+  <target name="javadocs" depends="javadocs-grouping,compile-core,check-javadocs-uptodate"
           unless="javadocs-uptodate-${name}">
     <invoke-module-javadoc>
       <links>
         <link href="../grouping"/>
-        <link href="../backward-codecs"/>
       </links>
     </invoke-module-javadoc>
   </target>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/5347cc8e/lucene/join/src/java/org/apache/lucene/search/join/DocValuesTermsCollector.java
----------------------------------------------------------------------
diff --git a/lucene/join/src/java/org/apache/lucene/search/join/DocValuesTermsCollector.java b/lucene/join/src/java/org/apache/lucene/search/join/DocValuesTermsCollector.java
index 4bb692a..ca1337d 100644
--- a/lucene/join/src/java/org/apache/lucene/search/join/DocValuesTermsCollector.java
+++ b/lucene/join/src/java/org/apache/lucene/search/join/DocValuesTermsCollector.java
@@ -17,20 +17,13 @@
 package org.apache.lucene.search.join;
 
 import java.io.IOException;
-import java.util.function.LongConsumer;
 
 import org.apache.lucene.index.BinaryDocValues;
 import org.apache.lucene.index.DocValues;
 import org.apache.lucene.index.LeafReader;
 import org.apache.lucene.index.LeafReaderContext;
-import org.apache.lucene.index.NumericDocValues;
-import org.apache.lucene.index.SortedNumericDocValues;
 import org.apache.lucene.index.SortedSetDocValues;
-import org.apache.lucene.legacy.LegacyNumericType;
-import org.apache.lucene.legacy.LegacyNumericUtils;
 import org.apache.lucene.search.SimpleCollector;
-import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.BytesRefBuilder;
 
 abstract class DocValuesTermsCollector<DV> extends SimpleCollector {
   
@@ -57,79 +50,4 @@ abstract class DocValuesTermsCollector<DV> extends SimpleCollector {
   static Function<SortedSetDocValues> sortedSetDocValues(String field) {
     return (ctx) -> DocValues.getSortedSet(ctx, field);
   }
-  
-  static Function<BinaryDocValues> numericAsBinaryDocValues(String field, LegacyNumericType numTyp) {
-    return (ctx) -> {
-      final NumericDocValues numeric = DocValues.getNumeric(ctx, field);
-      final BytesRefBuilder bytes = new BytesRefBuilder();
-      
-      final LongConsumer coder = coder(bytes, numTyp, field);
-      
-      return new BinaryDocValues() {
-        @Override
-        public BytesRef get(int docID) {
-          final long lVal = numeric.get(docID);
-          coder.accept(lVal);
-          return bytes.get();
-        }
-      };
-    };
-  }
-  
-  static LongConsumer coder(BytesRefBuilder bytes, LegacyNumericType type, String fieldName){
-    switch(type){
-      case INT: 
-        return (l) -> LegacyNumericUtils.intToPrefixCoded((int) l, 0, bytes);
-      case LONG: 
-        return (l) -> LegacyNumericUtils.longToPrefixCoded(l, 0, bytes);
-      default:
-        throw new IllegalArgumentException("Unsupported "+type+
-            ". Only "+ LegacyNumericType.INT+" and "+ LegacyNumericType.LONG+" are supported."
-            + "Field "+fieldName );
-    }
-  }
-  
-  /** this adapter is quite weird. ords are per doc index, don't use ords across different docs*/
-  static Function<SortedSetDocValues> sortedNumericAsSortedSetDocValues(String field, LegacyNumericType numTyp) {
-    return (ctx) -> {
-      final SortedNumericDocValues numerics = DocValues.getSortedNumeric(ctx, field);
-      final BytesRefBuilder bytes = new BytesRefBuilder();
-      
-      final LongConsumer coder = coder(bytes, numTyp, field);
-      
-      return new SortedSetDocValues() {
-
-        private int index = Integer.MIN_VALUE;
-
-        @Override
-        public long nextOrd() {
-          return index < numerics.count()-1 ? ++index : NO_MORE_ORDS;
-        }
-
-        @Override
-        public void setDocument(int docID) {
-          numerics.setDocument(docID);
-          index=-1;
-        }
-
-        @Override
-        public BytesRef lookupOrd(long ord) {
-          assert ord>=0 && ord<numerics.count();
-          final long value = numerics.valueAt((int)ord);
-          coder.accept(value);
-          return bytes.get();
-        }
-
-        @Override
-        public long getValueCount() {
-          throw new UnsupportedOperationException("it's just number encoding wrapper");
-        }
-        
-        @Override
-        public long lookupTerm(BytesRef key) {
-          throw new UnsupportedOperationException("it's just number encoding wrapper");
-        }
-      };
-    };
-  }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/5347cc8e/lucene/join/src/java/org/apache/lucene/search/join/JoinUtil.java
----------------------------------------------------------------------
diff --git a/lucene/join/src/java/org/apache/lucene/search/join/JoinUtil.java b/lucene/join/src/java/org/apache/lucene/search/join/JoinUtil.java
index 4942394..b7c2a5b 100644
--- a/lucene/join/src/java/org/apache/lucene/search/join/JoinUtil.java
+++ b/lucene/join/src/java/org/apache/lucene/search/join/JoinUtil.java
@@ -26,7 +26,6 @@ import java.util.function.BiConsumer;
 import java.util.function.LongFunction;
 
 import org.apache.lucene.document.DoublePoint;
-import org.apache.lucene.legacy.LegacyNumericType;
 import org.apache.lucene.document.FloatPoint;
 import org.apache.lucene.document.IntPoint;
 import org.apache.lucene.document.LongPoint;
@@ -112,50 +111,6 @@ public final class JoinUtil {
   }
   
   /**
-   * @deprecated Because {@link LegacyNumericType} is deprecated, instead use {@link #createJoinQuery(String, boolean, String, Class, Query, IndexSearcher, ScoreMode)}
-   *
-   * Method for query time joining for numeric fields. It supports multi- and single- values longs and ints. 
-   * All considerations from {@link JoinUtil#createJoinQuery(String, boolean, String, Query, IndexSearcher, ScoreMode)} are applicable here too,
-   * though memory consumption might be higher.
-   * <p>
-   *
-   * @param fromField                 The from field to join from
-   * @param multipleValuesPerDocument Whether the from field has multiple terms per document
-   *                                  when true fromField might be {@link DocValuesType#SORTED_NUMERIC},
-   *                                  otherwise fromField should be {@link DocValuesType#NUMERIC}
-   * @param toField                   The to field to join to, should be {@link org.apache.lucene.legacy.LegacyIntField} or {@link org.apache.lucene.legacy.LegacyLongField}
-   * @param numericType               either {@link LegacyNumericType#INT} or {@link LegacyNumericType#LONG}, it should correspond to fromField and toField types
-   * @param fromQuery                 The query to match documents on the from side
-   * @param fromSearcher              The searcher that executed the specified fromQuery
-   * @param scoreMode                 Instructs how scores from the fromQuery are mapped to the returned query
-   * @return a {@link Query} instance that can be used to join documents based on the
-   *         terms in the from and to field
-   * @throws IOException If I/O related errors occur
-   */
-  @Deprecated
-  public static Query createJoinQuery(String fromField,
-      boolean multipleValuesPerDocument,
-      String toField, LegacyNumericType numericType,
-      Query fromQuery,
-      IndexSearcher fromSearcher,
-      ScoreMode scoreMode) throws IOException {
-    
-    final GenericTermsCollector termsCollector;
-     
-    if (multipleValuesPerDocument) {
-      Function<SortedSetDocValues> mvFunction = DocValuesTermsCollector.sortedNumericAsSortedSetDocValues(fromField,numericType);
-      termsCollector = GenericTermsCollector.createCollectorMV(mvFunction, scoreMode);
-    } else {
-      Function<BinaryDocValues> svFunction = DocValuesTermsCollector.numericAsBinaryDocValues(fromField,numericType);
-      termsCollector =  GenericTermsCollector.createCollectorSV(svFunction, scoreMode);
-    }
-    
-    return createJoinQuery(multipleValuesPerDocument, toField, fromQuery, fromSearcher, scoreMode,
-        termsCollector);
-    
-  }
-
-  /**
    * Method for query time joining for numeric fields. It supports multi- and single- values longs, ints, floats and longs.
    * All considerations from {@link JoinUtil#createJoinQuery(String, boolean, String, Query, IndexSearcher, ScoreMode)} are applicable here too,
    * though memory consumption might be higher.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/5347cc8e/lucene/join/src/java/org/apache/lucene/search/join/TermsIncludingScoreQuery.java
----------------------------------------------------------------------
diff --git a/lucene/join/src/java/org/apache/lucene/search/join/TermsIncludingScoreQuery.java b/lucene/join/src/java/org/apache/lucene/search/join/TermsIncludingScoreQuery.java
index a39c25f..28d3044 100644
--- a/lucene/join/src/java/org/apache/lucene/search/join/TermsIncludingScoreQuery.java
+++ b/lucene/join/src/java/org/apache/lucene/search/join/TermsIncludingScoreQuery.java
@@ -27,7 +27,6 @@ import org.apache.lucene.index.PostingsEnum;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.TermsEnum;
-import org.apache.lucene.legacy.LegacyNumericUtils;
 import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.search.Explanation;
 import org.apache.lucene.search.IndexSearcher;
@@ -245,14 +244,6 @@ class TermsIncludingScoreQuery extends Query {
     for (int i = 0; i < terms.size(); i++) {
       terms.get(ords[i], ref);
       out.print(ref+" "+ref.utf8ToString()+" ");
-      try {
-        out.print(Long.toHexString(LegacyNumericUtils.prefixCodedToLong(ref))+"L");
-      } catch (Exception e) {
-        try {
-          out.print(Integer.toHexString(LegacyNumericUtils.prefixCodedToInt(ref))+"i");
-        } catch (Exception ee) {
-        }
-      }
       out.println(" score="+scores[ords[i]]);
       out.println("");
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/5347cc8e/lucene/join/src/test/org/apache/lucene/search/join/TestJoinUtil.java
----------------------------------------------------------------------
diff --git a/lucene/join/src/test/org/apache/lucene/search/join/TestJoinUtil.java b/lucene/join/src/test/org/apache/lucene/search/join/TestJoinUtil.java
index 6d9eb2a..17cd4a2 100644
--- a/lucene/join/src/test/org/apache/lucene/search/join/TestJoinUtil.java
+++ b/lucene/join/src/test/org/apache/lucene/search/join/TestJoinUtil.java
@@ -56,9 +56,6 @@ import org.apache.lucene.index.LeafReader;
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.MultiDocValues;
 import org.apache.lucene.index.MultiDocValues.OrdinalMap;
-import org.apache.lucene.legacy.LegacyIntField;
-import org.apache.lucene.legacy.LegacyLongField;
-import org.apache.lucene.legacy.LegacyNumericType;
 import org.apache.lucene.index.MultiFields;
 import org.apache.lucene.index.NoMergePolicy;
 import org.apache.lucene.index.NumericDocValues;
@@ -971,7 +968,7 @@ public class TestJoinUtil extends LuceneTestCase {
           final String fromField = from ? "from":"to";
           final String toField = from ? "to":"from";
 
-          int surpriseMe = random().nextInt(3);
+          int surpriseMe = random().nextInt(2);
           switch (surpriseMe) {
             case 0:
               Class<? extends Number> numType;
@@ -992,10 +989,6 @@ public class TestJoinUtil extends LuceneTestCase {
               joinQuery = JoinUtil.createJoinQuery(fromField + suffix, muliValsQuery, toField + suffix, numType, actualQuery, indexSearcher, scoreMode);
               break;
             case 1:
-              final LegacyNumericType legacyNumType = random().nextBoolean() ? LegacyNumericType.INT: LegacyNumericType.LONG ;
-              joinQuery = JoinUtil.createJoinQuery(fromField+legacyNumType, muliValsQuery, toField+legacyNumType, legacyNumType, actualQuery, indexSearcher, scoreMode);
-              break;
-            case 2:
               joinQuery = JoinUtil.createJoinQuery(fromField, muliValsQuery, toField, actualQuery, indexSearcher, scoreMode);
               break;
             default:
@@ -1336,26 +1329,24 @@ public class TestJoinUtil extends LuceneTestCase {
     document.add(newTextField(random, fieldName, linkValue, Field.Store.NO));
 
     final int linkInt = Integer.parseUnsignedInt(linkValue,16);
-    document.add(new LegacyIntField(fieldName + LegacyNumericType.INT, linkInt, Field.Store.NO));
-    document.add(new IntPoint(fieldName + LegacyNumericType.INT, linkInt));
+    document.add(new IntPoint(fieldName + "INT", linkInt));
     document.add(new FloatPoint(fieldName + "FLOAT", linkInt));
 
     final long linkLong = linkInt<<32 | linkInt;
-    document.add(new LegacyLongField(fieldName +  LegacyNumericType.LONG, linkLong, Field.Store.NO));
-    document.add(new LongPoint(fieldName + LegacyNumericType.LONG, linkLong));
+    document.add(new LongPoint(fieldName + "LONG", linkLong));
     document.add(new DoublePoint(fieldName + "DOUBLE", linkLong));
 
     if (multipleValuesPerDocument) {
       document.add(new SortedSetDocValuesField(fieldName, new BytesRef(linkValue)));
-      document.add(new SortedNumericDocValuesField(fieldName+ LegacyNumericType.INT, linkInt));
+      document.add(new SortedNumericDocValuesField(fieldName+ "INT", linkInt));
       document.add(new SortedNumericDocValuesField(fieldName+ "FLOAT", Float.floatToRawIntBits(linkInt)));
-      document.add(new SortedNumericDocValuesField(fieldName+ LegacyNumericType.LONG, linkLong));
+      document.add(new SortedNumericDocValuesField(fieldName+ "LONG", linkLong));
       document.add(new SortedNumericDocValuesField(fieldName+ "DOUBLE", Double.doubleToRawLongBits(linkLong)));
     } else {
       document.add(new SortedDocValuesField(fieldName, new BytesRef(linkValue)));
-      document.add(new NumericDocValuesField(fieldName+ LegacyNumericType.INT, linkInt));
+      document.add(new NumericDocValuesField(fieldName+ "INT", linkInt));
       document.add(new FloatDocValuesField(fieldName+ "FLOAT", linkInt));
-      document.add(new NumericDocValuesField(fieldName+ LegacyNumericType.LONG, linkLong));
+      document.add(new NumericDocValuesField(fieldName+ "LONG", linkLong));
       document.add(new DoubleDocValuesField(fieldName+ "DOUBLE", linkLong));
     }
     if (globalOrdinalJoin) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/5347cc8e/lucene/queryparser/build.xml
----------------------------------------------------------------------
diff --git a/lucene/queryparser/build.xml b/lucene/queryparser/build.xml
index f1d59a3..b6e43c2 100644
--- a/lucene/queryparser/build.xml
+++ b/lucene/queryparser/build.xml
@@ -25,17 +25,15 @@
   <path id="classpath">
     <pathelement path="${queries.jar}"/>
     <pathelement path="${sandbox.jar}"/>
-    <pathelement path="${backward-codecs.jar}"/>
     <path refid="base.classpath"/>
   </path>
 
-  <target name="compile-core" depends="jar-backward-codecs,jar-queries,jar-sandbox,common.compile-core"/>
+  <target name="compile-core" depends="jar-queries,jar-sandbox,common.compile-core"/>
 
-  <target name="javadocs" depends="javadocs-backward-codecs,javadocs-queries,javadocs-sandbox,compile-core,check-javadocs-uptodate"
+  <target name="javadocs" depends="javadocs-queries,javadocs-sandbox,compile-core,check-javadocs-uptodate"
           unless="javadocs-uptodate-${name}">
     <invoke-module-javadoc>
       <links>
-        <link href="../backward-codecs"/>
         <link href="../queries"/>
         <link href="../sandbox"/>
       </links>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/5347cc8e/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 32cbd02..c3dd6fa 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,6 @@ 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.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;
@@ -293,24 +292,6 @@ public class StandardQueryParser extends QueryParserHelper implements CommonQuer
     
   }
   
-  /**
-   * 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);
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/5347cc8e/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
deleted file mode 100644
index 0781afb..0000000
--- a/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/builders/LegacyNumericRangeQueryNodeBuilder.java
+++ /dev/null
@@ -1,93 +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.legacy.LegacyNumericRangeQuery;
-import org.apache.lucene.legacy.LegacyNumericType;
-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;
-
-/**
- * Builds {@link org.apache.lucene.legacy.LegacyNumericRangeQuery}s out of {@link LegacyNumericRangeQueryNode}s.
- *
- * @see org.apache.lucene.legacy.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();
-    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/5347cc8e/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 41d4046..7db4205 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
@@ -32,8 +32,6 @@ import org.apache.lucene.queryparser.flexible.core.nodes.TokenizedPhraseQueryNod
 import org.apache.lucene.queryparser.flexible.standard.nodes.MultiPhraseQueryNode;
 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;
@@ -59,8 +57,6 @@ public class StandardQueryTreeBuilder extends QueryTreeBuilder implements
     setBuilder(FieldQueryNode.class, new FieldQueryNodeBuilder());
     setBuilder(BooleanQueryNode.class, new BooleanQueryNodeBuilder());
     setBuilder(FuzzyQueryNode.class, new FuzzyQueryNodeBuilder());
-    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());

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/5347cc8e/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
deleted file mode 100644
index 038023e..0000000
--- a/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/config/LegacyNumericConfig.java
+++ /dev/null
@@ -1,165 +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.legacy.LegacyNumericType;
-
-/**
- * This class holds the configuration used to parse numeric queries and create
- * {@link org.apache.lucene.legacy.LegacyNumericRangeQuery}s.
- * 
- * @see org.apache.lucene.legacy.LegacyNumericRangeQuery
- * @see NumberFormat
- * @deprecated Index with Points instead and use {@link PointsConfig}
- */
-@Deprecated
-public class LegacyNumericConfig {
-  
-  private int precisionStep;
-  
-  private NumberFormat format;
-  
-  private 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(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.legacy.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.legacy.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 must not 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}, must not be <code>null</code>
-   */
-  public void setNumberFormat(NumberFormat format) {
-    
-    if (format == null) {
-      throw new IllegalArgumentException("format must not 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/5347cc8e/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
deleted file mode 100644
index e5ba9da..0000000
--- a/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/config/LegacyNumericFieldConfigListener.java
+++ /dev/null
@@ -1,75 +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#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 must not 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/5347cc8e/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 5c53d02..f401226 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
@@ -159,38 +159,18 @@ public class StandardQueryConfigHandler extends QueryConfigHandler {
     final public static ConfigurationKey<Float> BOOST = ConfigurationKey.newInstance();
     
     /**
-     * Key used to set a field to its {@link LegacyNumericConfig}.
-     * 
-     * @see StandardQueryParser#setLegacyNumericConfigMap(Map)
-     * @see StandardQueryParser#getLegacyNumericConfigMap()
-     * @deprecated Index with Points instead and use {@link #POINTS_CONFIG}
-     */
-    @Deprecated
-    final public static ConfigurationKey<LegacyNumericConfig> LEGACY_NUMERIC_CONFIG = ConfigurationKey.newInstance();
-    
-    /**
-     * Key used to set the {@link LegacyNumericConfig} in {@link FieldConfig} for numeric fields.
-     * 
-     * @see StandardQueryParser#setLegacyNumericConfigMap(Map)
-     * @see StandardQueryParser#getLegacyNumericConfigMap()
-     * @deprecated Index with Points instead and use {@link #POINTS_CONFIG_MAP}
-     */
-    @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()
+     * @see StandardQueryParser#setPointsConfigMap(Map)
+     * @see StandardQueryParser#getPointsConfigMap()
      */
     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()
+     * @see StandardQueryParser#setPointsConfigMap(Map)
+     * @see StandardQueryParser#getPointsConfigMap()
      */
     final public static ConfigurationKey<Map<String,PointsConfig>> POINTS_CONFIG_MAP = ConfigurationKey.newInstance();
 
@@ -207,7 +187,6 @@ public class StandardQueryConfigHandler extends QueryConfigHandler {
     // Add listener that will build the FieldConfig.
     addFieldConfigListener(new FieldBoostMapFCListener(this));
     addFieldConfigListener(new FieldDateResolutionFCListener(this));
-    addFieldConfigListener(new LegacyNumericFieldConfigListener(this));
     addFieldConfigListener(new PointsConfigListener(this));
     
     // Default Values

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/5347cc8e/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
deleted file mode 100644
index b644d8a..0000000
--- a/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/nodes/LegacyNumericQueryNode.java
+++ /dev/null
@@ -1,153 +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.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/5347cc8e/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
deleted file mode 100644
index 20cde35..0000000
--- a/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/nodes/LegacyNumericRangeQueryNode.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.nodes;
-
-import org.apache.lucene.legacy.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 LegacyNumericType.LONG;
-    } else if (number instanceof Integer) {
-      return LegacyNumericType.INT;
-    } else if (number instanceof Double) {
-      return LegacyNumericType.DOUBLE;
-    } else if (number instanceof Float) {
-      return 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 must not 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/5347cc8e/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
deleted file mode 100644
index 8b71824..0000000
--- a/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/processors/LegacyNumericQueryNodeProcessor.java
+++ /dev/null
@@ -1,154 +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.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/5347cc8e/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
deleted file mode 100644
index 5a54b7b..0000000
--- a/lucene/queryparser/src/java/org/apache/lucene/queryparser/flexible/standard/processors/LegacyNumericRangeQueryNodeProcessor.java
+++ /dev/null
@@ -1,170 +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.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/5347cc8e/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 5b681b4..15a44df 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
@@ -53,8 +53,6 @@ public class StandardQueryNodeProcessorPipeline extends
     add(new RegexpQueryNodeProcessor());
     add(new MatchAllDocsQueryNodeProcessor());
     add(new OpenRangeQueryNodeProcessor());
-    add(new LegacyNumericQueryNodeProcessor());
-    add(new LegacyNumericRangeQueryNodeProcessor());
     add(new PointQueryNodeProcessor());
     add(new PointRangeQueryNodeProcessor());
     add(new TermRangeQueryNodeProcessor());

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/5347cc8e/lucene/queryparser/src/java/org/apache/lucene/queryparser/xml/CoreParser.java
----------------------------------------------------------------------
diff --git a/lucene/queryparser/src/java/org/apache/lucene/queryparser/xml/CoreParser.java b/lucene/queryparser/src/java/org/apache/lucene/queryparser/xml/CoreParser.java
index 98a878f..1bf82ac 100644
--- a/lucene/queryparser/src/java/org/apache/lucene/queryparser/xml/CoreParser.java
+++ b/lucene/queryparser/src/java/org/apache/lucene/queryparser/xml/CoreParser.java
@@ -71,7 +71,6 @@ public class CoreParser implements QueryBuilder {
     queryFactory.addBuilder("TermsQuery", new TermsQueryBuilder(analyzer));
     queryFactory.addBuilder("MatchAllDocsQuery", new MatchAllDocsQueryBuilder());
     queryFactory.addBuilder("BooleanQuery", new BooleanQueryBuilder(queryFactory));
-    queryFactory.addBuilder("LegacyNumericRangeQuery", new LegacyNumericRangeQueryBuilder());
     queryFactory.addBuilder("PointRangeQuery", new PointRangeQueryBuilder());
     queryFactory.addBuilder("RangeQuery", new RangeQueryBuilder());
     queryFactory.addBuilder("DisjunctionMaxQuery", new DisjunctionMaxQueryBuilder(queryFactory));

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/5347cc8e/lucene/queryparser/src/java/org/apache/lucene/queryparser/xml/builders/LegacyNumericRangeQueryBuilder.java
----------------------------------------------------------------------
diff --git a/lucene/queryparser/src/java/org/apache/lucene/queryparser/xml/builders/LegacyNumericRangeQueryBuilder.java b/lucene/queryparser/src/java/org/apache/lucene/queryparser/xml/builders/LegacyNumericRangeQueryBuilder.java
deleted file mode 100644
index 9f4505f..0000000
--- a/lucene/queryparser/src/java/org/apache/lucene/queryparser/xml/builders/LegacyNumericRangeQueryBuilder.java
+++ /dev/null
@@ -1,135 +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.xml.builders;
-
-import org.apache.lucene.search.Query;
-import org.apache.lucene.legacy.LegacyNumericRangeQuery;
-import org.apache.lucene.legacy.LegacyNumericUtils;
-import org.apache.lucene.queryparser.xml.DOMUtils;
-import org.apache.lucene.queryparser.xml.ParserException;
-import org.apache.lucene.queryparser.xml.QueryBuilder;
-import org.w3c.dom.Element;
-
-/**
- * Creates a {@link org.apache.lucene.legacy.LegacyNumericRangeQuery}. The table below specifies the required
- * attributes and the defaults if optional attributes are omitted. For more
- * detail on what each of the attributes actually do, consult the documentation
- * for {@link org.apache.lucene.legacy.LegacyNumericRangeQuery}:
- * <table summary="supported attributes">
- * <tr>
- * <th>Attribute name</th>
- * <th>Values</th>
- * <th>Required</th>
- * <th>Default</th>
- * </tr>
- * <tr>
- * <td>fieldName</td>
- * <td>String</td>
- * <td>Yes</td>
- * <td>N/A</td>
- * </tr>
- * <tr>
- * <td>lowerTerm</td>
- * <td>Specified by <tt>type</tt></td>
- * <td>No</td>
- * <td>Null</td>
- * </tr>
- * <tr>
- * <td>upperTerm</td>
- * <td>Specified by <tt>type</tt></td>
- * <td>No</td>
- * <td>Null</td>
- * </tr>
- * <tr>
- * <td>type</td>
- * <td>int, long, float, double</td>
- * <td>No</td>
- * <td>int</td>
- * </tr>
- * <tr>
- * <td>includeLower</td>
- * <td>true, false</td>
- * <td>No</td>
- * <td>true</td>
- * </tr>
- * <tr>
- * <td>includeUpper</td>
- * <td>true, false</td>
- * <td>No</td>
- * <td>true</td>
- * </tr>
- * <tr>
- * <td>precisionStep</td>
- * <td>Integer</td>
- * <td>No</td>
- * <td>4</td>
- * </tr>
- * </table>
- * <p>
- * A {@link ParserException} will be thrown if an error occurs parsing the
- * supplied <tt>lowerTerm</tt> or <tt>upperTerm</tt> into the numeric type
- * specified by <tt>type</tt>.
- * @deprecated Index with points and use {@link PointRangeQueryBuilder} instead 
- */
-@Deprecated
-public class LegacyNumericRangeQueryBuilder implements QueryBuilder {
-
-  @Override
-  public Query getQuery(Element e) throws ParserException {
-    String field = DOMUtils.getAttributeWithInheritanceOrFail(e, "fieldName");
-    final String lowerTerm = DOMUtils.getAttribute(e, "lowerTerm", null);
-    final String upperTerm = DOMUtils.getAttribute(e, "upperTerm", null);
-    boolean lowerInclusive = DOMUtils.getAttribute(e, "includeLower", true);
-    boolean upperInclusive = DOMUtils.getAttribute(e, "includeUpper", true);
-    int precisionStep = DOMUtils.getAttribute(e, "precisionStep", LegacyNumericUtils.PRECISION_STEP_DEFAULT);
-
-    String type = DOMUtils.getAttribute(e, "type", "int");
-    try {
-      Query filter;
-      if (type.equalsIgnoreCase("int")) {
-        filter = LegacyNumericRangeQuery.newIntRange(field, precisionStep,
-            (lowerTerm == null ? null : Integer.valueOf(lowerTerm)),
-            (upperTerm == null ? null : Integer.valueOf(upperTerm)),
-            lowerInclusive,
-            upperInclusive);
-      } else if (type.equalsIgnoreCase("long")) {
-        filter = LegacyNumericRangeQuery.newLongRange(field, precisionStep,
-            (lowerTerm == null ? null : Long.valueOf(lowerTerm)),
-            (upperTerm == null ? null : Long.valueOf(upperTerm)),
-            lowerInclusive,
-            upperInclusive);
-      } else if (type.equalsIgnoreCase("double")) {
-        filter = LegacyNumericRangeQuery.newDoubleRange(field, precisionStep,
-            (lowerTerm == null ? null : Double.valueOf(lowerTerm)),
-            (upperTerm == null ? null : Double.valueOf(upperTerm)),
-            lowerInclusive,
-            upperInclusive);
-      } else if (type.equalsIgnoreCase("float")) {
-        filter = LegacyNumericRangeQuery.newFloatRange(field, precisionStep,
-            (lowerTerm == null ? null : Float.valueOf(lowerTerm)),
-            (upperTerm == null ? null : Float.valueOf(upperTerm)),
-            lowerInclusive,
-            upperInclusive);
-      } else {
-        throw new ParserException("type attribute must be one of: [long, int, double, float]");
-      }
-      return filter;
-    } catch (NumberFormatException nfe) {
-      throw new ParserException("Could not parse lowerTerm or upperTerm into a number", nfe);
-    }
-  }
-}