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/17 13:29:53 UTC

[2/7] lucene-solr:master: LUCENE-7413: move legacy numeric support to backwards module

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/105c7eae/lucene/core/src/test/org/apache/lucene/util/TestLegacyNumericUtils.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/util/TestLegacyNumericUtils.java b/lucene/core/src/test/org/apache/lucene/util/TestLegacyNumericUtils.java
deleted file mode 100644
index 2fb20d1..0000000
--- a/lucene/core/src/test/org/apache/lucene/util/TestLegacyNumericUtils.java
+++ /dev/null
@@ -1,564 +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.util;
-
-
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.Iterator;
-import java.util.Random;
-
-public class TestLegacyNumericUtils extends LuceneTestCase {
-
-  public void testLongConversionAndOrdering() throws Exception {
-    // generate a series of encoded longs, each numerical one bigger than the one before
-    BytesRefBuilder last = new BytesRefBuilder();
-    BytesRefBuilder act = new BytesRefBuilder();
-    for (long l=-100000L; l<100000L; l++) {
-      LegacyNumericUtils.longToPrefixCoded(l, 0, act);
-      if (last!=null) {
-        // test if smaller
-        assertTrue("actual bigger than last (BytesRef)", last.get().compareTo(act.get()) < 0 );
-        assertTrue("actual bigger than last (as String)", last.get().utf8ToString().compareTo(act.get().utf8ToString()) < 0 );
-      }
-      // test is back and forward conversion works
-      assertEquals("forward and back conversion should generate same long", l, LegacyNumericUtils.prefixCodedToLong(act.get()));
-      // next step
-      last.copyBytes(act);
-    }
-  }
-
-  public void testIntConversionAndOrdering() throws Exception {
-    // generate a series of encoded ints, each numerical one bigger than the one before
-    BytesRefBuilder act = new BytesRefBuilder();
-    BytesRefBuilder last = new BytesRefBuilder();
-    for (int i=-100000; i<100000; i++) {
-      LegacyNumericUtils.intToPrefixCoded(i, 0, act);
-      if (last!=null) {
-        // test if smaller
-        assertTrue("actual bigger than last (BytesRef)", last.get().compareTo(act.get()) < 0 );
-        assertTrue("actual bigger than last (as String)", last.get().utf8ToString().compareTo(act.get().utf8ToString()) < 0 );
-      }
-      // test is back and forward conversion works
-      assertEquals("forward and back conversion should generate same int", i, LegacyNumericUtils.prefixCodedToInt(act.get()));
-      // next step
-      last.copyBytes(act.get());
-    }
-  }
-
-  public void testLongSpecialValues() throws Exception {
-    long[] vals=new long[]{
-      Long.MIN_VALUE, Long.MIN_VALUE+1, Long.MIN_VALUE+2, -5003400000000L,
-      -4000L, -3000L, -2000L, -1000L, -1L, 0L, 1L, 10L, 300L, 50006789999999999L, Long.MAX_VALUE-2, Long.MAX_VALUE-1, Long.MAX_VALUE
-    };
-    BytesRefBuilder[] prefixVals = new BytesRefBuilder[vals.length];
-    
-    for (int i=0; i<vals.length; i++) {
-      prefixVals[i] = new BytesRefBuilder();
-      LegacyNumericUtils.longToPrefixCoded(vals[i], 0, prefixVals[i]);
-      
-      // check forward and back conversion
-      assertEquals( "forward and back conversion should generate same long", vals[i], LegacyNumericUtils.prefixCodedToLong(prefixVals[i].get()) );
-
-      // test if decoding values as int fails correctly
-      final int index = i;
-      expectThrows(NumberFormatException.class, () -> {
-        LegacyNumericUtils.prefixCodedToInt(prefixVals[index].get());
-      });
-    }
-    
-    // check sort order (prefixVals should be ascending)
-    for (int i=1; i<prefixVals.length; i++) {
-      assertTrue( "check sort order", prefixVals[i-1].get().compareTo(prefixVals[i].get()) < 0 );
-    }
-        
-    // check the prefix encoding, lower precision should have the difference to original value equal to the lower removed bits
-    final BytesRefBuilder ref = new BytesRefBuilder();
-    for (int i=0; i<vals.length; i++) {
-      for (int j=0; j<64; j++) {
-        LegacyNumericUtils.longToPrefixCoded(vals[i], j, ref);
-        long prefixVal= LegacyNumericUtils.prefixCodedToLong(ref.get());
-        long mask=(1L << j) - 1L;
-        assertEquals( "difference between prefix val and original value for "+vals[i]+" with shift="+j, vals[i] & mask, vals[i]-prefixVal );
-      }
-    }
-  }
-
-  public void testIntSpecialValues() throws Exception {
-    int[] vals=new int[]{
-      Integer.MIN_VALUE, Integer.MIN_VALUE+1, Integer.MIN_VALUE+2, -64765767,
-      -4000, -3000, -2000, -1000, -1, 0, 1, 10, 300, 765878989, Integer.MAX_VALUE-2, Integer.MAX_VALUE-1, Integer.MAX_VALUE
-    };
-    BytesRefBuilder[] prefixVals=new BytesRefBuilder[vals.length];
-    
-    for (int i=0; i<vals.length; i++) {
-      prefixVals[i] = new BytesRefBuilder();
-      LegacyNumericUtils.intToPrefixCoded(vals[i], 0, prefixVals[i]);
-      
-      // check forward and back conversion
-      assertEquals( "forward and back conversion should generate same int", vals[i], LegacyNumericUtils.prefixCodedToInt(prefixVals[i].get()) );
-      
-      // test if decoding values as long fails correctly
-      final int index = i;
-      expectThrows(NumberFormatException.class, () -> {
-        LegacyNumericUtils.prefixCodedToLong(prefixVals[index].get());
-      });
-    }
-    
-    // check sort order (prefixVals should be ascending)
-    for (int i=1; i<prefixVals.length; i++) {
-      assertTrue( "check sort order", prefixVals[i-1].get().compareTo(prefixVals[i].get()) < 0 );
-    }
-    
-    // check the prefix encoding, lower precision should have the difference to original value equal to the lower removed bits
-    final BytesRefBuilder ref = new BytesRefBuilder();
-    for (int i=0; i<vals.length; i++) {
-      for (int j=0; j<32; j++) {
-        LegacyNumericUtils.intToPrefixCoded(vals[i], j, ref);
-        int prefixVal= LegacyNumericUtils.prefixCodedToInt(ref.get());
-        int mask=(1 << j) - 1;
-        assertEquals( "difference between prefix val and original value for "+vals[i]+" with shift="+j, vals[i] & mask, vals[i]-prefixVal );
-      }
-    }
-  }
-
-  public void testDoubles() throws Exception {
-    double[] vals=new double[]{
-      Double.NEGATIVE_INFINITY, -2.3E25, -1.0E15, -1.0, -1.0E-1, -1.0E-2, -0.0, 
-      +0.0, 1.0E-2, 1.0E-1, 1.0, 1.0E15, 2.3E25, Double.POSITIVE_INFINITY, Double.NaN
-    };
-    long[] longVals=new long[vals.length];
-    
-    // check forward and back conversion
-    for (int i=0; i<vals.length; i++) {
-      longVals[i]= NumericUtils.doubleToSortableLong(vals[i]);
-      assertTrue( "forward and back conversion should generate same double", Double.compare(vals[i], NumericUtils.sortableLongToDouble(longVals[i]))==0 );
-    }
-    
-    // check sort order (prefixVals should be ascending)
-    for (int i=1; i<longVals.length; i++) {
-      assertTrue( "check sort order", longVals[i-1] < longVals[i] );
-    }
-  }
-
-  public static final double[] DOUBLE_NANs = {
-    Double.NaN,
-    Double.longBitsToDouble(0x7ff0000000000001L),
-    Double.longBitsToDouble(0x7fffffffffffffffL),
-    Double.longBitsToDouble(0xfff0000000000001L),
-    Double.longBitsToDouble(0xffffffffffffffffL)
-  };
-
-  public void testSortableDoubleNaN() {
-    final long plusInf = NumericUtils.doubleToSortableLong(Double.POSITIVE_INFINITY);
-    for (double nan : DOUBLE_NANs) {
-      assertTrue(Double.isNaN(nan));
-      final long sortable = NumericUtils.doubleToSortableLong(nan);
-      assertTrue("Double not sorted correctly: " + nan + ", long repr: " 
-          + sortable + ", positive inf.: " + plusInf, sortable > plusInf);
-    }
-  }
-  
-  public void testFloats() throws Exception {
-    float[] vals=new float[]{
-      Float.NEGATIVE_INFINITY, -2.3E25f, -1.0E15f, -1.0f, -1.0E-1f, -1.0E-2f, -0.0f, 
-      +0.0f, 1.0E-2f, 1.0E-1f, 1.0f, 1.0E15f, 2.3E25f, Float.POSITIVE_INFINITY, Float.NaN
-    };
-    int[] intVals=new int[vals.length];
-    
-    // check forward and back conversion
-    for (int i=0; i<vals.length; i++) {
-      intVals[i]= NumericUtils.floatToSortableInt(vals[i]);
-      assertTrue( "forward and back conversion should generate same double", Float.compare(vals[i], NumericUtils.sortableIntToFloat(intVals[i]))==0 );
-    }
-    
-    // check sort order (prefixVals should be ascending)
-    for (int i=1; i<intVals.length; i++) {
-      assertTrue( "check sort order", intVals[i-1] < intVals[i] );
-    }
-  }
-
-  public static final float[] FLOAT_NANs = {
-    Float.NaN,
-    Float.intBitsToFloat(0x7f800001),
-    Float.intBitsToFloat(0x7fffffff),
-    Float.intBitsToFloat(0xff800001),
-    Float.intBitsToFloat(0xffffffff)
-  };
-
-  public void testSortableFloatNaN() {
-    final int plusInf = NumericUtils.floatToSortableInt(Float.POSITIVE_INFINITY);
-    for (float nan : FLOAT_NANs) {
-      assertTrue(Float.isNaN(nan));
-      final int sortable = NumericUtils.floatToSortableInt(nan);
-      assertTrue("Float not sorted correctly: " + nan + ", int repr: " 
-          + sortable + ", positive inf.: " + plusInf, sortable > plusInf);
-    }
-  }
-
-  // INFO: Tests for trieCodeLong()/trieCodeInt() not needed because implicitely tested by range filter tests
-  
-  /** Note: The neededBounds Iterable must be unsigned (easier understanding what's happening) */
-  private void assertLongRangeSplit(final long lower, final long upper, int precisionStep,
-    final boolean useBitSet, final Iterable<Long> expectedBounds, final Iterable<Integer> expectedShifts
-  ) {
-    // Cannot use FixedBitSet since the range could be long:
-    final LongBitSet bits=useBitSet ? new LongBitSet(upper-lower+1) : null;
-    final Iterator<Long> neededBounds = (expectedBounds == null) ? null : expectedBounds.iterator();
-    final Iterator<Integer> neededShifts = (expectedShifts == null) ? null : expectedShifts.iterator();
-
-    LegacyNumericUtils.splitLongRange(new LegacyNumericUtils.LongRangeBuilder() {
-      @Override
-      public void addRange(long min, long max, int shift) {
-        assertTrue("min, max should be inside bounds", min >= lower && min <= upper && max >= lower && max <= upper);
-        if (useBitSet) for (long l = min; l <= max; l++) {
-          assertFalse("ranges should not overlap", bits.getAndSet(l - lower));
-          // extra exit condition to prevent overflow on MAX_VALUE
-          if (l == max) break;
-        }
-        if (neededBounds == null || neededShifts == null)
-          return;
-        // make unsigned longs for easier display and understanding
-        min ^= 0x8000000000000000L;
-        max ^= 0x8000000000000000L;
-        //System.out.println("0x"+Long.toHexString(min>>>shift)+"L,0x"+Long.toHexString(max>>>shift)+"L)/*shift="+shift+"*/,");
-        assertEquals("shift", neededShifts.next().intValue(), shift);
-        assertEquals("inner min bound", neededBounds.next().longValue(), min >>> shift);
-        assertEquals("inner max bound", neededBounds.next().longValue(), max >>> shift);
-      }
-    }, precisionStep, lower, upper);
-    
-    if (useBitSet) {
-      // after flipping all bits in the range, the cardinality should be zero
-      bits.flip(0,upper-lower+1);
-      assertEquals("The sub-range concenated should match the whole range", 0, bits.cardinality());
-    }
-  }
-  
-  /** LUCENE-2541: LegacyNumericRangeQuery errors with endpoints near long min and max values */
-  public void testLongExtremeValues() throws Exception {
-    // upper end extremes
-    assertLongRangeSplit(Long.MAX_VALUE, Long.MAX_VALUE, 1, true, Arrays.asList(
-      0xffffffffffffffffL,0xffffffffffffffffL
-    ), Arrays.asList(
-      0
-    ));
-    assertLongRangeSplit(Long.MAX_VALUE, Long.MAX_VALUE, 2, true, Arrays.asList(
-      0xffffffffffffffffL,0xffffffffffffffffL
-    ), Arrays.asList(
-      0
-    ));
-    assertLongRangeSplit(Long.MAX_VALUE, Long.MAX_VALUE, 4, true, Arrays.asList(
-      0xffffffffffffffffL,0xffffffffffffffffL
-    ), Arrays.asList(
-      0
-    ));
-    assertLongRangeSplit(Long.MAX_VALUE, Long.MAX_VALUE, 6, true, Arrays.asList(
-      0xffffffffffffffffL,0xffffffffffffffffL
-    ), Arrays.asList(
-      0
-    ));
-    assertLongRangeSplit(Long.MAX_VALUE, Long.MAX_VALUE, 8, true, Arrays.asList(
-      0xffffffffffffffffL,0xffffffffffffffffL
-    ), Arrays.asList(
-      0
-    ));
-    assertLongRangeSplit(Long.MAX_VALUE, Long.MAX_VALUE, 64, true, Arrays.asList(
-      0xffffffffffffffffL,0xffffffffffffffffL
-    ), Arrays.asList(
-      0
-    ));
-
-    assertLongRangeSplit(Long.MAX_VALUE-0xfL, Long.MAX_VALUE, 4, true, Arrays.asList(
-      0xfffffffffffffffL,0xfffffffffffffffL
-    ), Arrays.asList(
-      4
-    ));
-    assertLongRangeSplit(Long.MAX_VALUE-0x10L, Long.MAX_VALUE, 4, true, Arrays.asList(
-      0xffffffffffffffefL,0xffffffffffffffefL,
-      0xfffffffffffffffL,0xfffffffffffffffL
-    ), Arrays.asList(
-      0, 4
-    ));
-
-    // lower end extremes
-    assertLongRangeSplit(Long.MIN_VALUE, Long.MIN_VALUE, 1, true, Arrays.asList(
-      0x0000000000000000L,0x0000000000000000L
-    ), Arrays.asList(
-      0
-    ));
-    assertLongRangeSplit(Long.MIN_VALUE, Long.MIN_VALUE, 2, true, Arrays.asList(
-      0x0000000000000000L,0x0000000000000000L
-    ), Arrays.asList(
-      0
-    ));
-    assertLongRangeSplit(Long.MIN_VALUE, Long.MIN_VALUE, 4, true, Arrays.asList(
-      0x0000000000000000L,0x0000000000000000L
-    ), Arrays.asList(
-      0
-    ));
-    assertLongRangeSplit(Long.MIN_VALUE, Long.MIN_VALUE, 6, true, Arrays.asList(
-      0x0000000000000000L,0x0000000000000000L
-    ), Arrays.asList(
-      0
-    ));
-    assertLongRangeSplit(Long.MIN_VALUE, Long.MIN_VALUE, 8, true, Arrays.asList(
-      0x0000000000000000L,0x0000000000000000L
-    ), Arrays.asList(
-      0
-    ));
-    assertLongRangeSplit(Long.MIN_VALUE, Long.MIN_VALUE, 64, true, Arrays.asList(
-      0x0000000000000000L,0x0000000000000000L
-    ), Arrays.asList(
-      0
-    ));
-
-    assertLongRangeSplit(Long.MIN_VALUE, Long.MIN_VALUE+0xfL, 4, true, Arrays.asList(
-      0x000000000000000L,0x000000000000000L
-    ), Arrays.asList(
-      4
-    ));
-    assertLongRangeSplit(Long.MIN_VALUE, Long.MIN_VALUE+0x10L, 4, true, Arrays.asList(
-      0x0000000000000010L,0x0000000000000010L,
-      0x000000000000000L,0x000000000000000L
-    ), Arrays.asList(
-      0, 4
-    ));
-  }
-  
-  public void testRandomSplit() throws Exception {
-    long num = (long) atLeast(10);
-    for (long i=0; i < num; i++) {
-      executeOneRandomSplit(random());
-    }
-  }
-  
-  private void executeOneRandomSplit(final Random random) throws Exception {
-    long lower = randomLong(random);
-    long len = random.nextInt(16384*1024); // not too large bitsets, else OOME!
-    while (lower + len < lower) { // overflow
-      lower >>= 1;
-    }
-    assertLongRangeSplit(lower, lower + len, random.nextInt(64) + 1, true, null, null);
-  }
-  
-  private long randomLong(final Random random) {
-    long val;
-    switch(random.nextInt(4)) {
-      case 0:
-        val = 1L << (random.nextInt(63)); //  patterns like 0x000000100000 (-1 yields patterns like 0x0000fff)
-        break;
-      case 1:
-        val = -1L << (random.nextInt(63)); // patterns like 0xfffff00000
-        break;
-      default:
-        val = random.nextLong();
-    }
-
-    val += random.nextInt(5)-2;
-
-    if (random.nextBoolean()) {
-      if (random.nextBoolean()) val += random.nextInt(100)-50;
-      if (random.nextBoolean()) val = ~val;
-      if (random.nextBoolean()) val = val<<1;
-      if (random.nextBoolean()) val = val>>>1;
-    }
-
-    return val;
-  }
-  
-  public void testSplitLongRange() throws Exception {
-    // a hard-coded "standard" range
-    assertLongRangeSplit(-5000L, 9500L, 4, true, Arrays.asList(
-      0x7fffffffffffec78L,0x7fffffffffffec7fL,
-      0x8000000000002510L,0x800000000000251cL,
-      0x7fffffffffffec8L, 0x7fffffffffffecfL,
-      0x800000000000250L, 0x800000000000250L,
-      0x7fffffffffffedL,  0x7fffffffffffefL,
-      0x80000000000020L,  0x80000000000024L,
-      0x7ffffffffffffL,   0x8000000000001L
-    ), Arrays.asList(
-      0, 0,
-      4, 4,
-      8, 8,
-      12
-    ));
-    
-    // the same with no range splitting
-    assertLongRangeSplit(-5000L, 9500L, 64, true, Arrays.asList(
-      0x7fffffffffffec78L,0x800000000000251cL
-    ), Arrays.asList(
-      0
-    ));
-    
-    // this tests optimized range splitting, if one of the inner bounds
-    // is also the bound of the next lower precision, it should be used completely
-    assertLongRangeSplit(0L, 1024L+63L, 4, true, Arrays.asList(
-      0x800000000000040L, 0x800000000000043L,
-      0x80000000000000L,  0x80000000000003L
-    ), Arrays.asList(
-      4, 8
-    ));
-    
-    // the full long range should only consist of a lowest precision range; no bitset testing here, as too much memory needed :-)
-    assertLongRangeSplit(Long.MIN_VALUE, Long.MAX_VALUE, 8, false, Arrays.asList(
-      0x00L,0xffL
-    ), Arrays.asList(
-      56
-    ));
-
-    // the same with precisionStep=4
-    assertLongRangeSplit(Long.MIN_VALUE, Long.MAX_VALUE, 4, false, Arrays.asList(
-      0x0L,0xfL
-    ), Arrays.asList(
-      60
-    ));
-
-    // the same with precisionStep=2
-    assertLongRangeSplit(Long.MIN_VALUE, Long.MAX_VALUE, 2, false, Arrays.asList(
-      0x0L,0x3L
-    ), Arrays.asList(
-      62
-    ));
-
-    // the same with precisionStep=1
-    assertLongRangeSplit(Long.MIN_VALUE, Long.MAX_VALUE, 1, false, Arrays.asList(
-      0x0L,0x1L
-    ), Arrays.asList(
-      63
-    ));
-
-    // a inverse range should produce no sub-ranges
-    assertLongRangeSplit(9500L, -5000L, 4, false, Collections.<Long>emptyList(), Collections.<Integer>emptyList());    
-
-    // a 0-length range should reproduce the range itself
-    assertLongRangeSplit(9500L, 9500L, 4, false, Arrays.asList(
-      0x800000000000251cL,0x800000000000251cL
-    ), Arrays.asList(
-      0
-    ));
-  }
-
-  /** Note: The neededBounds Iterable must be unsigned (easier understanding what's happening) */
-  private void assertIntRangeSplit(final int lower, final int upper, int precisionStep,
-    final boolean useBitSet, final Iterable<Integer> expectedBounds, final Iterable<Integer> expectedShifts
-  ) {
-    final FixedBitSet bits=useBitSet ? new FixedBitSet(upper-lower+1) : null;
-    final Iterator<Integer> neededBounds = (expectedBounds == null) ? null : expectedBounds.iterator();
-    final Iterator<Integer> neededShifts = (expectedShifts == null) ? null : expectedShifts.iterator();
-    
-    LegacyNumericUtils.splitIntRange(new LegacyNumericUtils.IntRangeBuilder() {
-      @Override
-      public void addRange(int min, int max, int shift) {
-        assertTrue("min, max should be inside bounds", min >= lower && min <= upper && max >= lower && max <= upper);
-        if (useBitSet) for (int i = min; i <= max; i++) {
-          assertFalse("ranges should not overlap", bits.getAndSet(i - lower));
-          // extra exit condition to prevent overflow on MAX_VALUE
-          if (i == max) break;
-        }
-        if (neededBounds == null)
-          return;
-        // make unsigned ints for easier display and understanding
-        min ^= 0x80000000;
-        max ^= 0x80000000;
-        //System.out.println("0x"+Integer.toHexString(min>>>shift)+",0x"+Integer.toHexString(max>>>shift)+")/*shift="+shift+"*/,");
-        assertEquals("shift", neededShifts.next().intValue(), shift);
-        assertEquals("inner min bound", neededBounds.next().intValue(), min >>> shift);
-        assertEquals("inner max bound", neededBounds.next().intValue(), max >>> shift);
-      }
-    }, precisionStep, lower, upper);
-    
-    if (useBitSet) {
-      // after flipping all bits in the range, the cardinality should be zero
-      bits.flip(0, upper-lower+1);
-      assertEquals("The sub-range concenated should match the whole range", 0, bits.cardinality());
-    }
-  }
-  
-  public void testSplitIntRange() throws Exception {
-    // a hard-coded "standard" range
-    assertIntRangeSplit(-5000, 9500, 4, true, Arrays.asList(
-      0x7fffec78,0x7fffec7f,
-      0x80002510,0x8000251c,
-      0x7fffec8, 0x7fffecf,
-      0x8000250, 0x8000250,
-      0x7fffed,  0x7fffef,
-      0x800020,  0x800024,
-      0x7ffff,   0x80001
-    ), Arrays.asList(
-      0, 0,
-      4, 4,
-      8, 8,
-      12
-    ));
-    
-    // the same with no range splitting
-    assertIntRangeSplit(-5000, 9500, 32, true, Arrays.asList(
-      0x7fffec78,0x8000251c
-    ), Arrays.asList(
-      0
-    ));
-    
-    // this tests optimized range splitting, if one of the inner bounds
-    // is also the bound of the next lower precision, it should be used completely
-    assertIntRangeSplit(0, 1024+63, 4, true, Arrays.asList(
-      0x8000040, 0x8000043,
-      0x800000,  0x800003
-    ), Arrays.asList(
-      4, 8
-    ));
-    
-    // the full int range should only consist of a lowest precision range; no bitset testing here, as too much memory needed :-)
-    assertIntRangeSplit(Integer.MIN_VALUE, Integer.MAX_VALUE, 8, false, Arrays.asList(
-      0x00,0xff
-    ), Arrays.asList(
-      24
-    ));
-
-    // the same with precisionStep=4
-    assertIntRangeSplit(Integer.MIN_VALUE, Integer.MAX_VALUE, 4, false, Arrays.asList(
-      0x0,0xf
-    ), Arrays.asList(
-      28
-    ));
-
-    // the same with precisionStep=2
-    assertIntRangeSplit(Integer.MIN_VALUE, Integer.MAX_VALUE, 2, false, Arrays.asList(
-      0x0,0x3
-    ), Arrays.asList(
-      30
-    ));
-
-    // the same with precisionStep=1
-    assertIntRangeSplit(Integer.MIN_VALUE, Integer.MAX_VALUE, 1, false, Arrays.asList(
-      0x0,0x1
-    ), Arrays.asList(
-      31
-    ));
-
-    // a inverse range should produce no sub-ranges
-    assertIntRangeSplit(9500, -5000, 4, false, Collections.<Integer>emptyList(), Collections.<Integer>emptyList());    
-
-    // a 0-length range should reproduce the range itself
-    assertIntRangeSplit(9500, 9500, 4, false, Arrays.asList(
-      0x8000251c,0x8000251c
-    ), Arrays.asList(
-      0
-    ));
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/105c7eae/lucene/join/build.xml
----------------------------------------------------------------------
diff --git a/lucene/join/build.xml b/lucene/join/build.xml
index b5360c4..b6878b8 100644
--- a/lucene/join/build.xml
+++ b/lucene/join/build.xml
@@ -26,6 +26,7 @@
 
   <path id="classpath">
     <pathelement path="${grouping.jar}"/>
+    <pathelement path="${backward-codecs.jar}"/>
     <path refid="base.classpath"/>
   </path>
 
@@ -34,13 +35,14 @@
     <pathelement location="${build.dir}/classes/java"/>
   </path>
 
-  <target name="init" depends="module-build.init,jar-grouping"/>
+  <target name="init" depends="module-build.init,jar-grouping,jar-backward-codecs"/>
 
-  <target name="javadocs" depends="javadocs-grouping,compile-core,check-javadocs-uptodate"
+  <target name="javadocs" depends="javadocs-grouping,javadocs-backward-codecs,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/105c7eae/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 a9b11ed..4bb692a 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
@@ -19,8 +19,6 @@ package org.apache.lucene.search.join;
 import java.io.IOException;
 import java.util.function.LongConsumer;
 
-import org.apache.lucene.document.FieldType;
-import org.apache.lucene.document.FieldType.LegacyNumericType;
 import org.apache.lucene.index.BinaryDocValues;
 import org.apache.lucene.index.DocValues;
 import org.apache.lucene.index.LeafReader;
@@ -28,10 +26,11 @@ 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;
-import org.apache.lucene.util.LegacyNumericUtils;
 
 abstract class DocValuesTermsCollector<DV> extends SimpleCollector {
   
@@ -85,13 +84,13 @@ abstract class DocValuesTermsCollector<DV> extends SimpleCollector {
         return (l) -> LegacyNumericUtils.longToPrefixCoded(l, 0, bytes);
       default:
         throw new IllegalArgumentException("Unsupported "+type+
-            ". Only "+ LegacyNumericType.INT+" and "+ FieldType.LegacyNumericType.LONG+" are supported."
+            ". 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, FieldType.LegacyNumericType numTyp) {
+  static Function<SortedSetDocValues> sortedNumericAsSortedSetDocValues(String field, LegacyNumericType numTyp) {
     return (ctx) -> {
       final SortedNumericDocValues numerics = DocValues.getSortedNumeric(ctx, field);
       final BytesRefBuilder bytes = new BytesRefBuilder();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/105c7eae/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 b0133e5..4942394 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,7 @@ import java.util.function.BiConsumer;
 import java.util.function.LongFunction;
 
 import org.apache.lucene.document.DoublePoint;
-import org.apache.lucene.document.FieldType.LegacyNumericType;
+import org.apache.lucene.legacy.LegacyNumericType;
 import org.apache.lucene.document.FloatPoint;
 import org.apache.lucene.document.IntPoint;
 import org.apache.lucene.document.LongPoint;
@@ -123,8 +123,8 @@ public final class JoinUtil {
    * @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.document.LegacyIntField} or {@link org.apache.lucene.document.LegacyLongField}
-   * @param numericType               either {@link org.apache.lucene.document.FieldType.LegacyNumericType#INT} or {@link org.apache.lucene.document.FieldType.LegacyNumericType#LONG}, it should correspond to fromField and toField types
+   * @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

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/105c7eae/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 3b03bd3..a39c25f 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,6 +27,7 @@ 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;
@@ -37,7 +38,6 @@ import org.apache.lucene.util.BitSetIterator;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefHash;
 import org.apache.lucene.util.FixedBitSet;
-import org.apache.lucene.util.LegacyNumericUtils;
 
 class TermsIncludingScoreQuery extends Query {
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/105c7eae/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 b29e9ff..6d9eb2a 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
@@ -37,12 +37,9 @@ import org.apache.lucene.document.Document;
 import org.apache.lucene.document.DoubleDocValuesField;
 import org.apache.lucene.document.DoublePoint;
 import org.apache.lucene.document.Field;
-import org.apache.lucene.document.FieldType.LegacyNumericType;
 import org.apache.lucene.document.FloatDocValuesField;
 import org.apache.lucene.document.FloatPoint;
 import org.apache.lucene.document.IntPoint;
-import org.apache.lucene.document.LegacyIntField;
-import org.apache.lucene.document.LegacyLongField;
 import org.apache.lucene.document.LongPoint;
 import org.apache.lucene.document.NumericDocValuesField;
 import org.apache.lucene.document.SortedDocValuesField;
@@ -59,6 +56,9 @@ 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;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/105c7eae/lucene/memory/src/test/org/apache/lucene/index/memory/TestMemoryIndexAgainstRAMDir.java
----------------------------------------------------------------------
diff --git a/lucene/memory/src/test/org/apache/lucene/index/memory/TestMemoryIndexAgainstRAMDir.java b/lucene/memory/src/test/org/apache/lucene/index/memory/TestMemoryIndexAgainstRAMDir.java
index 45e9551..a785720 100644
--- a/lucene/memory/src/test/org/apache/lucene/index/memory/TestMemoryIndexAgainstRAMDir.java
+++ b/lucene/memory/src/test/org/apache/lucene/index/memory/TestMemoryIndexAgainstRAMDir.java
@@ -45,7 +45,6 @@ import org.apache.lucene.document.Field;
 import org.apache.lucene.document.FieldType;
 import org.apache.lucene.document.FloatPoint;
 import org.apache.lucene.document.IntPoint;
-import org.apache.lucene.document.LegacyLongField;
 import org.apache.lucene.document.LongPoint;
 import org.apache.lucene.document.NumericDocValuesField;
 import org.apache.lucene.document.SortedDocValuesField;
@@ -457,9 +456,6 @@ public class TestMemoryIndexAgainstRAMDir extends BaseTokenStreamTestCase {
     Document doc = new Document();
     long randomLong = random().nextLong();
     doc.add(new NumericDocValuesField("numeric", randomLong));
-    if (random().nextBoolean()) {
-      doc.add(new LegacyLongField("numeric", randomLong, Field.Store.NO));
-    }
     int numValues = atLeast(5);
     for (int i = 0; i < numValues; i++) {
       randomLong = random().nextLong();
@@ -468,9 +464,6 @@ public class TestMemoryIndexAgainstRAMDir extends BaseTokenStreamTestCase {
         // randomly duplicate field/value
         doc.add(new SortedNumericDocValuesField("sorted_numeric", randomLong));
       }
-      if (random().nextBoolean()) {
-        doc.add(new LegacyLongField("numeric", randomLong, Field.Store.NO));
-      }
     }
     BytesRef randomTerm = new BytesRef(randomTerm());
     doc.add(new BinaryDocValuesField("binary", randomTerm));

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/105c7eae/lucene/misc/src/test/org/apache/lucene/search/TestDiversifiedTopDocsCollector.java
----------------------------------------------------------------------
diff --git a/lucene/misc/src/test/org/apache/lucene/search/TestDiversifiedTopDocsCollector.java b/lucene/misc/src/test/org/apache/lucene/search/TestDiversifiedTopDocsCollector.java
index da9fdc5..05a3b23 100644
--- a/lucene/misc/src/test/org/apache/lucene/search/TestDiversifiedTopDocsCollector.java
+++ b/lucene/misc/src/test/org/apache/lucene/search/TestDiversifiedTopDocsCollector.java
@@ -21,11 +21,10 @@ import java.util.HashMap;
 import java.util.Map;
 
 import org.apache.lucene.document.Document;
-import org.apache.lucene.document.Field.Store;
 import org.apache.lucene.document.Field;
 import org.apache.lucene.document.FloatDocValuesField;
-import org.apache.lucene.document.LegacyFloatField;
 import org.apache.lucene.document.SortedDocValuesField;
+import org.apache.lucene.document.StoredField;
 import org.apache.lucene.index.BinaryDocValues;
 import org.apache.lucene.index.DocValues;
 import org.apache.lucene.index.FieldInvertState;
@@ -331,7 +330,7 @@ public class TestDiversifiedTopDocsCollector extends LuceneTestCase {
         new BytesRef(""));
     Field weeksAtNumberOneField = new FloatDocValuesField("weeksAtNumberOne",
         0.0F);
-    Field weeksStoredField = new LegacyFloatField("weeks", 0.0F, Store.YES);
+    Field weeksStoredField = new StoredField("weeks", 0.0F);
     Field idField = newStringField("id", "", Field.Store.YES);
     Field songField = newTextField("song", "", Field.Store.NO);
     Field storedArtistField = newTextField("artistName", "", Field.Store.NO);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/105c7eae/lucene/queryparser/build.xml
----------------------------------------------------------------------
diff --git a/lucene/queryparser/build.xml b/lucene/queryparser/build.xml
index b6e43c2..f1d59a3 100644
--- a/lucene/queryparser/build.xml
+++ b/lucene/queryparser/build.xml
@@ -25,15 +25,17 @@
   <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-queries,jar-sandbox,common.compile-core"/>
+  <target name="compile-core" depends="jar-backward-codecs,jar-queries,jar-sandbox,common.compile-core"/>
 
-  <target name="javadocs" depends="javadocs-queries,javadocs-sandbox,compile-core,check-javadocs-uptodate"
+  <target name="javadocs" depends="javadocs-backward-codecs,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/105c7eae/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
index 8ae7d5e..0781afb 100644
--- 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
@@ -16,7 +16,8 @@
  */
 package org.apache.lucene.queryparser.flexible.standard.builders;
 
-import org.apache.lucene.document.FieldType;
+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;
@@ -25,12 +26,11 @@ import org.apache.lucene.queryparser.flexible.messages.MessageImpl;
 import org.apache.lucene.queryparser.flexible.standard.config.LegacyNumericConfig;
 import org.apache.lucene.queryparser.flexible.standard.nodes.LegacyNumericQueryNode;
 import org.apache.lucene.queryparser.flexible.standard.nodes.LegacyNumericRangeQueryNode;
-import org.apache.lucene.search.LegacyNumericRangeQuery;
 
 /**
- * Builds {@link org.apache.lucene.search.LegacyNumericRangeQuery}s out of {@link LegacyNumericRangeQueryNode}s.
+ * Builds {@link org.apache.lucene.legacy.LegacyNumericRangeQuery}s out of {@link LegacyNumericRangeQueryNode}s.
  *
- * @see org.apache.lucene.search.LegacyNumericRangeQuery
+ * @see org.apache.lucene.legacy.LegacyNumericRangeQuery
  * @see LegacyNumericRangeQueryNode
  * @deprecated Index with points and use {@link PointRangeQueryNodeBuilder} instead.
  */
@@ -56,7 +56,7 @@ public class LegacyNumericRangeQueryNodeBuilder implements StandardQueryBuilder
     Number upperNumber = upperNumericNode.getValue();
     
     LegacyNumericConfig numericConfig = numericRangeNode.getNumericConfig();
-    FieldType.LegacyNumericType numberType = numericConfig.getType();
+    LegacyNumericType numberType = numericConfig.getType();
     String field = StringUtils.toString(numericRangeNode.getField());
     boolean minInclusive = numericRangeNode.isLowerInclusive();
     boolean maxInclusive = numericRangeNode.isUpperInclusive();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/105c7eae/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
index 6cd3c49..038023e 100644
--- 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
@@ -19,14 +19,13 @@ package org.apache.lucene.queryparser.flexible.standard.config;
 import java.text.NumberFormat;
 import java.util.Objects;
 
-import org.apache.lucene.document.FieldType;
-import org.apache.lucene.document.FieldType.LegacyNumericType;
+import org.apache.lucene.legacy.LegacyNumericType;
 
 /**
  * This class holds the configuration used to parse numeric queries and create
- * {@link org.apache.lucene.search.LegacyNumericRangeQuery}s.
+ * {@link org.apache.lucene.legacy.LegacyNumericRangeQuery}s.
  * 
- * @see org.apache.lucene.search.LegacyNumericRangeQuery
+ * @see org.apache.lucene.legacy.LegacyNumericRangeQuery
  * @see NumberFormat
  * @deprecated Index with Points instead and use {@link PointsConfig}
  */
@@ -37,7 +36,7 @@ public class LegacyNumericConfig {
   
   private NumberFormat format;
   
-  private FieldType.LegacyNumericType type;
+  private LegacyNumericType type;
   
   /**
    * Constructs a {@link LegacyNumericConfig} object.
@@ -52,7 +51,7 @@ public class LegacyNumericConfig {
    * 
    * @see LegacyNumericConfig#setPrecisionStep(int)
    * @see LegacyNumericConfig#setNumberFormat(NumberFormat)
-   * @see #setType(org.apache.lucene.document.FieldType.LegacyNumericType)
+   * @see #setType(LegacyNumericType)
    */
   public LegacyNumericConfig(int precisionStep, NumberFormat format,
       LegacyNumericType type) {
@@ -67,7 +66,7 @@ public class LegacyNumericConfig {
    * 
    * @return the precision used to index the numeric values
    * 
-   * @see org.apache.lucene.search.LegacyNumericRangeQuery#getPrecisionStep()
+   * @see org.apache.lucene.legacy.LegacyNumericRangeQuery#getPrecisionStep()
    */
   public int getPrecisionStep() {
     return precisionStep;
@@ -79,7 +78,7 @@ public class LegacyNumericConfig {
    * @param precisionStep
    *          the precision used to index the numeric values
    * 
-   * @see org.apache.lucene.search.LegacyNumericRangeQuery#getPrecisionStep()
+   * @see org.apache.lucene.legacy.LegacyNumericRangeQuery#getPrecisionStep()
    */
   public void setPrecisionStep(int precisionStep) {
     this.precisionStep = precisionStep;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/105c7eae/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
index 27c285e..20cde35 100644
--- 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
@@ -16,8 +16,7 @@
  */
 package org.apache.lucene.queryparser.flexible.standard.nodes;
 
-import org.apache.lucene.document.FieldType;
-import org.apache.lucene.document.FieldType.LegacyNumericType;
+import org.apache.lucene.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;
@@ -57,13 +56,13 @@ public class LegacyNumericRangeQueryNode extends
   private static LegacyNumericType getNumericDataType(Number number) throws QueryNodeException {
     
     if (number instanceof Long) {
-      return FieldType.LegacyNumericType.LONG;
+      return LegacyNumericType.LONG;
     } else if (number instanceof Integer) {
-      return FieldType.LegacyNumericType.INT;
+      return LegacyNumericType.INT;
     } else if (number instanceof Double) {
       return LegacyNumericType.DOUBLE;
     } else if (number instanceof Float) {
-      return FieldType.LegacyNumericType.FLOAT;
+      return LegacyNumericType.FLOAT;
     } else {
       throw new QueryNodeException(
           new MessageImpl(

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/105c7eae/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
index f7aef3f..9f4505f 100644
--- 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
@@ -16,19 +16,19 @@
  */
 package org.apache.lucene.queryparser.xml.builders;
 
-import org.apache.lucene.search.LegacyNumericRangeQuery;
 import org.apache.lucene.search.Query;
-import org.apache.lucene.util.LegacyNumericUtils;
+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.search.LegacyNumericRangeQuery}. The table below specifies the required
+ * 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.search.LegacyNumericRangeQuery}:
+ * for {@link org.apache.lucene.legacy.LegacyNumericRangeQuery}:
  * <table summary="supported attributes">
  * <tr>
  * <th>Attribute name</th>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/105c7eae/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
index c6ab7f5..398923e 100644
--- 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
@@ -32,15 +32,15 @@ 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.legacy.LegacyDoubleField;
+import org.apache.lucene.legacy.LegacyFieldType;
+import org.apache.lucene.legacy.LegacyFloatField;
+import org.apache.lucene.legacy.LegacyIntField;
+import org.apache.lucene.legacy.LegacyLongField;
+import org.apache.lucene.legacy.LegacyNumericType;
 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;
@@ -179,7 +179,7 @@ public class TestLegacyNumericQueryParser extends LuceneTestCase {
       ;
     
     randomNumberMap.put(LegacyNumericType.LONG.name(), randomLong);
-    randomNumberMap.put(FieldType.LegacyNumericType.INT.name(), randomInt);
+    randomNumberMap.put(LegacyNumericType.INT.name(), randomInt);
     randomNumberMap.put(LegacyNumericType.FLOAT.name(), randomFloat);
     randomNumberMap.put(LegacyNumericType.DOUBLE.name(), randomDouble);
     randomNumberMap.put(DATE_FIELD_NAME, randomDate);
@@ -201,7 +201,7 @@ public class TestLegacyNumericQueryParser extends LuceneTestCase {
       numericConfigMap.put(type.name(), new LegacyNumericConfig(PRECISION_STEP,
           NUMBER_FORMAT, type));
 
-      FieldType ft = new FieldType(LegacyIntField.TYPE_NOT_STORED);
+      LegacyFieldType ft = new LegacyFieldType(LegacyIntField.TYPE_NOT_STORED);
       ft.setNumericType(type);
       ft.setStored(true);
       ft.setNumericPrecisionStep(PRECISION_STEP);
@@ -231,7 +231,7 @@ public class TestLegacyNumericQueryParser extends LuceneTestCase {
     
     numericConfigMap.put(DATE_FIELD_NAME, new LegacyNumericConfig(PRECISION_STEP,
         DATE_FORMAT, LegacyNumericType.LONG));
-    FieldType ft = new FieldType(LegacyLongField.TYPE_NOT_STORED);
+    LegacyFieldType ft = new LegacyFieldType(LegacyLongField.TYPE_NOT_STORED);
     ft.setStored(true);
     ft.setNumericPrecisionStep(PRECISION_STEP);
     LegacyLongField dateField = new LegacyLongField(DATE_FIELD_NAME, 0l, ft);
@@ -268,10 +268,10 @@ public class TestLegacyNumericQueryParser extends LuceneTestCase {
             || DATE_FIELD_NAME.equals(fieldName)) {
           number = -number.longValue();
           
-        } else if (FieldType.LegacyNumericType.DOUBLE.name().equals(fieldName)) {
+        } else if (LegacyNumericType.DOUBLE.name().equals(fieldName)) {
           number = -number.doubleValue();
           
-        } else if (FieldType.LegacyNumericType.FLOAT.name().equals(fieldName)) {
+        } else if (LegacyNumericType.FLOAT.name().equals(fieldName)) {
           number = -number.floatValue();
           
         } else if (LegacyNumericType.INT.name().equals(fieldName)) {
@@ -299,16 +299,16 @@ public class TestLegacyNumericQueryParser extends LuceneTestCase {
     numericFieldMap.get(LegacyNumericType.DOUBLE.name()).setDoubleValue(
         number.doubleValue());
     
-    number = getNumberType(numberType, FieldType.LegacyNumericType.INT.name());
-    numericFieldMap.get(FieldType.LegacyNumericType.INT.name()).setIntValue(
+    number = getNumberType(numberType, LegacyNumericType.INT.name());
+    numericFieldMap.get(LegacyNumericType.INT.name()).setIntValue(
         number.intValue());
     
     number = getNumberType(numberType, LegacyNumericType.LONG.name());
-    numericFieldMap.get(FieldType.LegacyNumericType.LONG.name()).setLongValue(
+    numericFieldMap.get(LegacyNumericType.LONG.name()).setLongValue(
         number.longValue());
     
-    number = getNumberType(numberType, FieldType.LegacyNumericType.FLOAT.name());
-    numericFieldMap.get(FieldType.LegacyNumericType.FLOAT.name()).setFloatValue(
+    number = getNumberType(numberType, LegacyNumericType.FLOAT.name());
+    numericFieldMap.get(LegacyNumericType.FLOAT.name()).setFloatValue(
         number.floatValue());
     
     number = getNumberType(numberType, DATE_FIELD_NAME);
@@ -456,7 +456,7 @@ public class TestLegacyNumericQueryParser extends LuceneTestCase {
 
     StringBuilder sb = new StringBuilder();
     
-    for (LegacyNumericType type : FieldType.LegacyNumericType.values()) {
+    for (LegacyNumericType type : LegacyNumericType.values()) {
       String boundStr = numberToString(getNumberType(boundType, type.name()));
       
       sb.append("+").append(type.name()).append(operator).append('"').append(boundStr).append('"').append(' ');

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/105c7eae/lucene/queryparser/src/test/org/apache/lucene/queryparser/xml/CoreParserTestIndexData.java
----------------------------------------------------------------------
diff --git a/lucene/queryparser/src/test/org/apache/lucene/queryparser/xml/CoreParserTestIndexData.java b/lucene/queryparser/src/test/org/apache/lucene/queryparser/xml/CoreParserTestIndexData.java
index 71b627e..4763005 100644
--- a/lucene/queryparser/src/test/org/apache/lucene/queryparser/xml/CoreParserTestIndexData.java
+++ b/lucene/queryparser/src/test/org/apache/lucene/queryparser/xml/CoreParserTestIndexData.java
@@ -20,10 +20,10 @@ import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
 import org.apache.lucene.document.IntPoint;
-import org.apache.lucene.document.LegacyIntField;
 import org.apache.lucene.index.DirectoryReader;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.IndexWriter;
+import org.apache.lucene.legacy.LegacyIntField;
 import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.LuceneTestCase;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/105c7eae/lucene/queryparser/src/test/org/apache/lucene/queryparser/xml/builders/TestNumericRangeQueryBuilder.java
----------------------------------------------------------------------
diff --git a/lucene/queryparser/src/test/org/apache/lucene/queryparser/xml/builders/TestNumericRangeQueryBuilder.java b/lucene/queryparser/src/test/org/apache/lucene/queryparser/xml/builders/TestNumericRangeQueryBuilder.java
index 8fc0641..0bc0195 100644
--- a/lucene/queryparser/src/test/org/apache/lucene/queryparser/xml/builders/TestNumericRangeQueryBuilder.java
+++ b/lucene/queryparser/src/test/org/apache/lucene/queryparser/xml/builders/TestNumericRangeQueryBuilder.java
@@ -16,9 +16,9 @@
  */
 package org.apache.lucene.queryparser.xml.builders;
 
-import org.apache.lucene.search.LegacyNumericRangeQuery;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.legacy.LegacyNumericRangeQuery;
 import org.apache.lucene.queryparser.xml.ParserException;
 import org.w3c.dom.Document;
 import org.xml.sax.SAXException;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/105c7eae/lucene/spatial-extras/build.xml
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/build.xml b/lucene/spatial-extras/build.xml
index a77f9ea..2e425fd 100644
--- a/lucene/spatial-extras/build.xml
+++ b/lucene/spatial-extras/build.xml
@@ -31,6 +31,7 @@
   <path id="classpath">
     <path refid="base.classpath"/>
     <path refid="spatialjar"/>
+    <pathelement path="${backward-codecs.jar}" />
     <pathelement path="${queries.jar}" />
     <pathelement path="${misc.jar}" />
     <pathelement path="${spatial3d.jar}" />
@@ -42,16 +43,17 @@
     <pathelement path="src/test-files" />
   </path>
 
-  <target name="compile-core" depends="jar-queries,jar-misc,jar-spatial3d,common.compile-core" />
+  <target name="compile-core" depends="jar-backward-codecs,jar-queries,jar-misc,jar-spatial3d,common.compile-core" />
 
-  <target name="javadocs" depends="javadocs-queries,javadocs-misc,javadocs-spatial3d,compile-core,check-javadocs-uptodate"
+  <target name="javadocs" depends="javadocs-backward-codecs,javadocs-queries,javadocs-misc,javadocs-spatial3d,compile-core,check-javadocs-uptodate"
           unless="javadocs-uptodate-${name}">
     <invoke-module-javadoc>
       <links>
+        <link href="../backward-codecs"/>
         <link href="../queries"/>
         <link href="../misc"/>
         <link href="../spatial3d"/>
       </links>
     </invoke-module-javadoc>
   </target>
-</project>
\ No newline at end of file
+</project>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/105c7eae/lucene/spatial-extras/src/java/org/apache/lucene/spatial/bbox/BBoxStrategy.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/bbox/BBoxStrategy.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/bbox/BBoxStrategy.java
index 63a1138..90e36d8 100644
--- a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/bbox/BBoxStrategy.java
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/bbox/BBoxStrategy.java
@@ -20,17 +20,20 @@ import org.apache.lucene.document.DoubleDocValuesField;
 import org.apache.lucene.document.DoublePoint;
 import org.apache.lucene.document.Field;
 import org.apache.lucene.document.FieldType;
-import org.apache.lucene.document.LegacyDoubleField;
 import org.apache.lucene.document.StoredField;
 import org.apache.lucene.document.StringField;
 import org.apache.lucene.index.DocValuesType;
 import org.apache.lucene.index.IndexOptions;
 import org.apache.lucene.index.Term;
+import org.apache.lucene.legacy.LegacyDoubleField;
+import org.apache.lucene.legacy.LegacyFieldType;
+import org.apache.lucene.legacy.LegacyNumericRangeQuery;
+import org.apache.lucene.legacy.LegacyNumericType;
+import org.apache.lucene.legacy.LegacyNumericUtils;
 import org.apache.lucene.queries.function.ValueSource;
 import org.apache.lucene.search.BooleanClause;
 import org.apache.lucene.search.BooleanQuery;
 import org.apache.lucene.search.ConstantScoreQuery;
-import org.apache.lucene.search.LegacyNumericRangeQuery;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.TermQuery;
 import org.apache.lucene.spatial.SpatialStrategy;
@@ -39,7 +42,6 @@ import org.apache.lucene.spatial.query.SpatialOperation;
 import org.apache.lucene.spatial.query.UnsupportedSpatialOperation;
 import org.apache.lucene.spatial.util.DistanceToShapeValueSource;
 import org.apache.lucene.util.BytesRefBuilder;
-import org.apache.lucene.util.LegacyNumericUtils;
 import org.apache.lucene.util.NumericUtils;
 import org.locationtech.spatial4j.context.SpatialContext;
 import org.locationtech.spatial4j.shape.Point;
@@ -87,7 +89,7 @@ public class BBoxStrategy extends SpatialStrategy {
   public static FieldType DEFAULT_FIELDTYPE;
 
   @Deprecated
-  public static FieldType LEGACY_FIELDTYPE;
+  public static LegacyFieldType LEGACY_FIELDTYPE;
   static {
     // Default: pointValues + docValues
     FieldType type = new FieldType();
@@ -97,14 +99,14 @@ public class BBoxStrategy extends SpatialStrategy {
     type.freeze();
     DEFAULT_FIELDTYPE = type;
     // Legacy default: legacyNumerics + docValues
-    type = new FieldType();
-    type.setIndexOptions(IndexOptions.DOCS);
-    type.setNumericType(FieldType.LegacyNumericType.DOUBLE);
-    type.setNumericPrecisionStep(8);// same as solr default
-    type.setDocValuesType(DocValuesType.NUMERIC);//docValues
-    type.setStored(false);
-    type.freeze();
-    LEGACY_FIELDTYPE = type;
+    LegacyFieldType legacyType = new LegacyFieldType();
+    legacyType.setIndexOptions(IndexOptions.DOCS);
+    legacyType.setNumericType(LegacyNumericType.DOUBLE);
+    legacyType.setNumericPrecisionStep(8);// same as solr default
+    legacyType.setDocValuesType(DocValuesType.NUMERIC);//docValues
+    legacyType.setStored(false);
+    legacyType.freeze();
+    LEGACY_FIELDTYPE = legacyType;
   }
 
   public static final String SUFFIX_MINX = "__minX";
@@ -130,7 +132,7 @@ public class BBoxStrategy extends SpatialStrategy {
   private final boolean hasDocVals;
   private final boolean hasPointVals;
   // equiv to "hasLegacyNumerics":
-  private final FieldType legacyNumericFieldType; // not stored; holds precision step.
+  private final LegacyFieldType legacyNumericFieldType; // not stored; holds precision step.
   private final FieldType xdlFieldType;
 
   /**
@@ -177,16 +179,17 @@ public class BBoxStrategy extends SpatialStrategy {
     if ((this.hasPointVals = fieldType.pointDimensionCount() > 0)) {
       numQuads++;
     }
-    if (fieldType.indexOptions() != IndexOptions.NONE && fieldType.numericType() != null) {
+    if (fieldType.indexOptions() != IndexOptions.NONE && fieldType instanceof LegacyFieldType && ((LegacyFieldType)fieldType).numericType() != null) {
       if (hasPointVals) {
         throw new IllegalArgumentException("pointValues and LegacyNumericType are mutually exclusive");
       }
-      if (fieldType.numericType() != FieldType.LegacyNumericType.DOUBLE) {
-        throw new IllegalArgumentException(getClass() + " does not support " + fieldType.numericType());
+      final LegacyFieldType legacyType = (LegacyFieldType) fieldType;
+      if (legacyType.numericType() != LegacyNumericType.DOUBLE) {
+        throw new IllegalArgumentException(getClass() + " does not support " + legacyType.numericType());
       }
       numQuads++;
-      legacyNumericFieldType = new FieldType(LegacyDoubleField.TYPE_NOT_STORED);
-      legacyNumericFieldType.setNumericPrecisionStep(fieldType.numericPrecisionStep());
+      legacyNumericFieldType = new LegacyFieldType(LegacyDoubleField.TYPE_NOT_STORED);
+      legacyNumericFieldType.setNumericPrecisionStep(legacyType.numericPrecisionStep());
       legacyNumericFieldType.freeze();
     } else {
       legacyNumericFieldType = null;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/105c7eae/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/BytesRefIteratorTokenStream.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/BytesRefIteratorTokenStream.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/BytesRefIteratorTokenStream.java
index e724ab0..757e2bd 100644
--- a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/BytesRefIteratorTokenStream.java
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/BytesRefIteratorTokenStream.java
@@ -26,7 +26,7 @@ import org.apache.lucene.util.BytesRefIterator;
 /**
  * A TokenStream used internally by {@link org.apache.lucene.spatial.prefix.PrefixTreeStrategy}.
  *
- * This is modelled after {@link org.apache.lucene.analysis.LegacyNumericTokenStream}.
+ * This is modelled after {@link org.apache.lucene.legacy.LegacyNumericTokenStream}.
  *
  * @lucene.internal
  */

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/105c7eae/lucene/spatial-extras/src/java/org/apache/lucene/spatial/vector/PointVectorStrategy.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/vector/PointVectorStrategy.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/vector/PointVectorStrategy.java
index 197547c..59aff49 100644
--- a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/vector/PointVectorStrategy.java
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/vector/PointVectorStrategy.java
@@ -20,16 +20,18 @@ import org.apache.lucene.document.DoubleDocValuesField;
 import org.apache.lucene.document.DoublePoint;
 import org.apache.lucene.document.Field;
 import org.apache.lucene.document.FieldType;
-import org.apache.lucene.document.LegacyDoubleField;
 import org.apache.lucene.document.StoredField;
 import org.apache.lucene.index.DocValuesType;
 import org.apache.lucene.index.IndexOptions;
+import org.apache.lucene.legacy.LegacyDoubleField;
+import org.apache.lucene.legacy.LegacyFieldType;
+import org.apache.lucene.legacy.LegacyNumericRangeQuery;
+import org.apache.lucene.legacy.LegacyNumericType;
 import org.apache.lucene.queries.function.FunctionRangeQuery;
 import org.apache.lucene.queries.function.ValueSource;
 import org.apache.lucene.search.BooleanClause;
 import org.apache.lucene.search.BooleanQuery;
 import org.apache.lucene.search.ConstantScoreQuery;
-import org.apache.lucene.search.LegacyNumericRangeQuery;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.spatial.SpatialStrategy;
 import org.apache.lucene.spatial.query.SpatialArgs;
@@ -85,7 +87,7 @@ public class PointVectorStrategy extends SpatialStrategy {
   public static FieldType DEFAULT_FIELDTYPE;
 
   @Deprecated
-  public static FieldType LEGACY_FIELDTYPE;
+  public static LegacyFieldType LEGACY_FIELDTYPE;
   static {
     // Default: pointValues + docValues
     FieldType type = new FieldType();
@@ -95,14 +97,14 @@ public class PointVectorStrategy extends SpatialStrategy {
     type.freeze();
     DEFAULT_FIELDTYPE = type;
     // Legacy default: legacyNumerics
-    type = new FieldType();
-    type.setIndexOptions(IndexOptions.DOCS);
-    type.setNumericType(FieldType.LegacyNumericType.DOUBLE);
-    type.setNumericPrecisionStep(8);// same as solr default
-    type.setDocValuesType(DocValuesType.NONE);//no docValues!
-    type.setStored(false);
-    type.freeze();
-    LEGACY_FIELDTYPE = type;
+    LegacyFieldType legacyType = new LegacyFieldType();
+    legacyType.setIndexOptions(IndexOptions.DOCS);
+    legacyType.setNumericType(LegacyNumericType.DOUBLE);
+    legacyType.setNumericPrecisionStep(8);// same as solr default
+    legacyType.setDocValuesType(DocValuesType.NONE);//no docValues!
+    legacyType.setStored(false);
+    legacyType.freeze();
+    LEGACY_FIELDTYPE = legacyType;
   }
 
   public static final String SUFFIX_X = "__x";
@@ -116,7 +118,7 @@ public class PointVectorStrategy extends SpatialStrategy {
   private final boolean hasDocVals;
   private final boolean hasPointVals;
   // equiv to "hasLegacyNumerics":
-  private final FieldType legacyNumericFieldType; // not stored; holds precision step.
+  private final LegacyFieldType legacyNumericFieldType; // not stored; holds precision step.
 
   /**
    * Create a new {@link PointVectorStrategy} instance that uses {@link DoublePoint} and {@link DoublePoint#newRangeQuery}
@@ -157,16 +159,17 @@ public class PointVectorStrategy extends SpatialStrategy {
     if ((this.hasPointVals = fieldType.pointDimensionCount() > 0)) {
       numPairs++;
     }
-    if (fieldType.indexOptions() != IndexOptions.NONE && fieldType.numericType() != null) {
+    if (fieldType.indexOptions() != IndexOptions.NONE && fieldType instanceof LegacyFieldType && ((LegacyFieldType)fieldType).numericType() != null) {
       if (hasPointVals) {
         throw new IllegalArgumentException("pointValues and LegacyNumericType are mutually exclusive");
       }
-      if (fieldType.numericType() != FieldType.LegacyNumericType.DOUBLE) {
-        throw new IllegalArgumentException(getClass() + " does not support " + fieldType.numericType());
+      final LegacyFieldType legacyType = (LegacyFieldType) fieldType;
+      if (legacyType.numericType() != LegacyNumericType.DOUBLE) {
+        throw new IllegalArgumentException(getClass() + " does not support " + legacyType.numericType());
       }
       numPairs++;
-      legacyNumericFieldType = new FieldType(LegacyDoubleField.TYPE_NOT_STORED);
-      legacyNumericFieldType.setNumericPrecisionStep(fieldType.numericPrecisionStep());
+      legacyNumericFieldType = new LegacyFieldType(LegacyDoubleField.TYPE_NOT_STORED);
+      legacyNumericFieldType.setNumericPrecisionStep(legacyType.numericPrecisionStep());
       legacyNumericFieldType.freeze();
     } else {
       legacyNumericFieldType = null;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/105c7eae/lucene/spatial-extras/src/test/org/apache/lucene/spatial/bbox/TestBBoxStrategy.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/test/org/apache/lucene/spatial/bbox/TestBBoxStrategy.java b/lucene/spatial-extras/src/test/org/apache/lucene/spatial/bbox/TestBBoxStrategy.java
index 01e9259..20df730 100644
--- a/lucene/spatial-extras/src/test/org/apache/lucene/spatial/bbox/TestBBoxStrategy.java
+++ b/lucene/spatial-extras/src/test/org/apache/lucene/spatial/bbox/TestBBoxStrategy.java
@@ -22,6 +22,7 @@ import com.carrotsearch.randomizedtesting.annotations.Repeat;
 import org.apache.lucene.document.FieldType;
 import org.apache.lucene.index.DocValuesType;
 import org.apache.lucene.index.IndexOptions;
+import org.apache.lucene.legacy.LegacyFieldType;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.spatial.SpatialMatchConcern;
 import org.apache.lucene.spatial.prefix.RandomSpatialOpStrategyTestCase;
@@ -100,7 +101,12 @@ public class TestBBoxStrategy extends RandomSpatialOpStrategyTestCase {
     }
     //test we can disable docValues for predicate tests
     if (random().nextBoolean()) {
-      FieldType fieldType = new FieldType(((BBoxStrategy)strategy).getFieldType());
+      FieldType fieldType = ((BBoxStrategy)strategy).getFieldType();
+      if (fieldType instanceof LegacyFieldType) {
+        fieldType = new LegacyFieldType((LegacyFieldType)fieldType);
+      } else {
+        fieldType = new FieldType(fieldType);
+      }
       fieldType.setDocValuesType(DocValuesType.NONE);
       strategy = new BBoxStrategy(ctx, strategy.getFieldName(), fieldType);
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/105c7eae/solr/contrib/analytics/src/java/org/apache/solr/analytics/util/AnalyticsParsers.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/java/org/apache/solr/analytics/util/AnalyticsParsers.java b/solr/contrib/analytics/src/java/org/apache/solr/analytics/util/AnalyticsParsers.java
index 7a7e697..aadb9e2 100644
--- a/solr/contrib/analytics/src/java/org/apache/solr/analytics/util/AnalyticsParsers.java
+++ b/solr/contrib/analytics/src/java/org/apache/solr/analytics/util/AnalyticsParsers.java
@@ -20,8 +20,8 @@ import java.io.IOException;
 import java.time.Instant;
 import java.util.Arrays;
 
+import org.apache.lucene.legacy.LegacyNumericUtils;
 import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.LegacyNumericUtils;
 import org.apache.lucene.util.NumericUtils;
 import org.apache.solr.schema.FieldType;
 import org.apache.solr.schema.TrieDateField;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/105c7eae/solr/contrib/analytics/src/java/org/apache/solr/analytics/util/valuesource/DateFieldSource.java
----------------------------------------------------------------------
diff --git a/solr/contrib/analytics/src/java/org/apache/solr/analytics/util/valuesource/DateFieldSource.java b/solr/contrib/analytics/src/java/org/apache/solr/analytics/util/valuesource/DateFieldSource.java
index 4d66e00..22dde4c 100644
--- a/solr/contrib/analytics/src/java/org/apache/solr/analytics/util/valuesource/DateFieldSource.java
+++ b/solr/contrib/analytics/src/java/org/apache/solr/analytics/util/valuesource/DateFieldSource.java
@@ -24,12 +24,12 @@ import java.util.Map;
 import org.apache.lucene.index.DocValues;
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.NumericDocValues;
+import org.apache.lucene.legacy.LegacyNumericUtils;
 import org.apache.lucene.queries.function.FunctionValues;
 import org.apache.lucene.queries.function.docvalues.LongDocValues;
 import org.apache.lucene.queries.function.valuesource.LongFieldSource;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.LegacyNumericUtils;
 import org.apache.lucene.util.mutable.MutableValue;
 import org.apache.lucene.util.mutable.MutableValueDate;
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/105c7eae/solr/core/src/java/org/apache/solr/handler/component/StatsField.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/component/StatsField.java b/solr/core/src/java/org/apache/solr/handler/component/StatsField.java
index 4c2a2b6..5df1b45 100644
--- a/solr/core/src/java/org/apache/solr/handler/component/StatsField.java
+++ b/solr/core/src/java/org/apache/solr/handler/component/StatsField.java
@@ -29,8 +29,7 @@ import java.util.Map;
 import java.util.Set;
 
 import org.apache.commons.lang.StringUtils;
-import org.apache.lucene.document.FieldType;
-import org.apache.lucene.document.FieldType.LegacyNumericType;
+import org.apache.lucene.legacy.LegacyNumericType;
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.queries.function.FunctionQuery;
 import org.apache.lucene.queries.function.ValueSource;
@@ -637,13 +636,13 @@ public class StatsField {
         return null;
       }
 
-      final FieldType.LegacyNumericType hashableNumType = getHashableNumericType(field);
+      final LegacyNumericType hashableNumType = getHashableNumericType(field);
 
       // some sane defaults
       int log2m = 13;   // roughly equivilent to "cardinality='0.33'"
       int regwidth = 6; // with decent hash, this is plenty for all valid long hashes
 
-      if (LegacyNumericType.FLOAT.equals(hashableNumType) || FieldType.LegacyNumericType.INT.equals(hashableNumType)) {
+      if (LegacyNumericType.FLOAT.equals(hashableNumType) || LegacyNumericType.INT.equals(hashableNumType)) {
         // for 32bit values, we can adjust our default regwidth down a bit
         regwidth--;
 
@@ -707,7 +706,7 @@ public class StatsField {
       if (null == hasher) {
         // if this is a function, or a non Long field, pre-hashed is invalid
         // NOTE: we ignore hashableNumType - it's LONG for non numerics like Strings
-        if (null == field || !FieldType.LegacyNumericType.LONG.equals(field.getType().getNumericType())) {
+        if (null == field || !LegacyNumericType.LONG.equals(field.getType().getNumericType())) {
           throw new SolrException(ErrorCode.BAD_REQUEST, "hllPreHashed is only supported with Long based fields");
         }
       }
@@ -740,16 +739,16 @@ public class StatsField {
   }
 
   /**
-   * Returns the effective {@link org.apache.lucene.document.FieldType.LegacyNumericType} for the field for the purposes of hash values.
+   * Returns the effective {@link LegacyNumericType} for the field for the purposes of hash values.
    * ie: If the field has an explict LegacyNumericType that is returned; If the field has no explicit
-   * LegacyNumericType then {@link org.apache.lucene.document.FieldType.LegacyNumericType#LONG} is returned;  If field is null, then
-   * {@link org.apache.lucene.document.FieldType.LegacyNumericType#FLOAT} is assumed for ValueSource.
+   * LegacyNumericType then {@link LegacyNumericType#LONG} is returned;  If field is null, then
+   * {@link LegacyNumericType#FLOAT} is assumed for ValueSource.
    */
   private static LegacyNumericType getHashableNumericType(SchemaField field) {
     if (null == field) {
       return LegacyNumericType.FLOAT;
     }
     final LegacyNumericType result = field.getType().getNumericType();
-    return null == result ? FieldType.LegacyNumericType.LONG : result;
+    return null == result ? LegacyNumericType.LONG : result;
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/105c7eae/solr/core/src/java/org/apache/solr/request/IntervalFacets.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/request/IntervalFacets.java b/solr/core/src/java/org/apache/solr/request/IntervalFacets.java
index fedc7fe..db26e12 100644
--- a/solr/core/src/java/org/apache/solr/request/IntervalFacets.java
+++ b/solr/core/src/java/org/apache/solr/request/IntervalFacets.java
@@ -25,7 +25,7 @@ import java.util.Iterator;
 import java.util.List;
 import java.util.Locale;
 
-import org.apache.lucene.document.FieldType.LegacyNumericType;
+import org.apache.lucene.legacy.LegacyNumericType;
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.DocValues;
 import org.apache.lucene.index.NumericDocValues;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/105c7eae/solr/core/src/java/org/apache/solr/request/NumericFacets.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/request/NumericFacets.java b/solr/core/src/java/org/apache/solr/request/NumericFacets.java
index 1034947..fd85d3f 100644
--- a/solr/core/src/java/org/apache/solr/request/NumericFacets.java
+++ b/solr/core/src/java/org/apache/solr/request/NumericFacets.java
@@ -33,6 +33,7 @@ import org.apache.lucene.index.NumericDocValues;
 import org.apache.lucene.index.ReaderUtil;
 import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.TermsEnum;
+import org.apache.lucene.legacy.LegacyNumericType;
 import org.apache.lucene.queries.function.FunctionValues;
 import org.apache.lucene.queries.function.ValueSource;
 import org.apache.lucene.util.Bits;
@@ -132,7 +133,7 @@ final class NumericFacets {
     mincount = Math.max(mincount, 1);
     final SchemaField sf = searcher.getSchema().getField(fieldName);
     final FieldType ft = sf.getType();
-    final org.apache.lucene.document.FieldType.LegacyNumericType numericType = ft.getNumericType();
+    final LegacyNumericType numericType = ft.getNumericType();
     if (numericType == null) {
       throw new IllegalStateException();
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/105c7eae/solr/core/src/java/org/apache/solr/schema/BBoxField.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/schema/BBoxField.java b/solr/core/src/java/org/apache/solr/schema/BBoxField.java
index e41d3c6..d7fda7c 100644
--- a/solr/core/src/java/org/apache/solr/schema/BBoxField.java
+++ b/solr/core/src/java/org/apache/solr/schema/BBoxField.java
@@ -23,6 +23,7 @@ import java.util.List;
 import java.util.Map;
 
 import org.apache.lucene.index.DocValuesType;
+import org.apache.lucene.legacy.LegacyFieldType;
 import org.apache.lucene.queries.function.ValueSource;
 import org.apache.lucene.spatial.bbox.BBoxOverlapRatioValueSource;
 import org.apache.lucene.spatial.bbox.BBoxStrategy;
@@ -141,7 +142,11 @@ public class BBoxField extends AbstractSpatialFieldType<BBoxStrategy> implements
     
     //and annoyingly this Field isn't going to have a docValues format because Solr uses a separate Field for that
     if (solrNumField.hasDocValues()) {
-      luceneType = new org.apache.lucene.document.FieldType(luceneType);
+      if (luceneType instanceof LegacyFieldType) {
+        luceneType = new LegacyFieldType((LegacyFieldType)luceneType);
+      } else {
+        luceneType = new org.apache.lucene.document.FieldType(luceneType);
+      }
       luceneType.setDocValuesType(DocValuesType.NUMERIC);
     }
     return new BBoxStrategy(ctx, fieldName, luceneType);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/105c7eae/solr/core/src/java/org/apache/solr/schema/EnumField.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/schema/EnumField.java b/solr/core/src/java/org/apache/solr/schema/EnumField.java
index 27f3a0a..4820e77 100644
--- a/solr/core/src/java/org/apache/solr/schema/EnumField.java
+++ b/solr/core/src/java/org/apache/solr/schema/EnumField.java
@@ -32,24 +32,25 @@ import javax.xml.xpath.XPathConstants;
 import javax.xml.xpath.XPathExpressionException;
 import javax.xml.xpath.XPathFactory;
 
-import org.apache.lucene.document.FieldType;
-import org.apache.lucene.document.LegacyIntField;
 import org.apache.lucene.document.NumericDocValuesField;
 import org.apache.lucene.document.SortedSetDocValuesField;
 import org.apache.lucene.index.IndexOptions;
 import org.apache.lucene.index.IndexableField;
+import org.apache.lucene.legacy.LegacyFieldType;
+import org.apache.lucene.legacy.LegacyIntField;
+import org.apache.lucene.legacy.LegacyNumericRangeQuery;
+import org.apache.lucene.legacy.LegacyNumericType;
+import org.apache.lucene.legacy.LegacyNumericUtils;
 import org.apache.lucene.queries.function.ValueSource;
 import org.apache.lucene.queries.function.valuesource.EnumFieldSource;
 import org.apache.lucene.search.ConstantScoreQuery;
 import org.apache.lucene.search.DocValuesRangeQuery;
-import org.apache.lucene.search.LegacyNumericRangeQuery;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.search.SortField;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefBuilder;
 import org.apache.lucene.util.CharsRef;
 import org.apache.lucene.util.CharsRefBuilder;
-import org.apache.lucene.util.LegacyNumericUtils;
 import org.apache.solr.common.EnumFieldValue;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.response.TextResponseWriter;
@@ -234,8 +235,8 @@ public class EnumField extends PrimitiveFieldType {
    * {@inheritDoc}
    */
   @Override
-  public FieldType.LegacyNumericType getNumericType() {
-    return FieldType.LegacyNumericType.INT;
+  public LegacyNumericType getNumericType() {
+    return LegacyNumericType.INT;
   }
 
   /**
@@ -387,7 +388,7 @@ public class EnumField extends PrimitiveFieldType {
       throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Unknown value for enum field: " + value.toString());
 
     String intAsString =  intValue.toString();
-    final FieldType newType = new FieldType();
+    final LegacyFieldType newType = new LegacyFieldType();
 
     newType.setTokenized(field.isTokenized());
     newType.setStored(field.stored());
@@ -397,7 +398,7 @@ public class EnumField extends PrimitiveFieldType {
     newType.setStoreTermVectorOffsets(field.storeTermOffsets());
     newType.setStoreTermVectorPositions(field.storeTermPositions());
     newType.setStoreTermVectorPayloads(field.storeTermPayloads());
-    newType.setNumericType(FieldType.LegacyNumericType.INT);
+    newType.setNumericType(LegacyNumericType.INT);
     newType.setNumericPrecisionStep(DEFAULT_PRECISION_STEP);
 
     final org.apache.lucene.document.Field f;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/105c7eae/solr/core/src/java/org/apache/solr/schema/FieldType.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/schema/FieldType.java b/solr/core/src/java/org/apache/solr/schema/FieldType.java
index 6556ddb..32a91d9 100644
--- a/solr/core/src/java/org/apache/solr/schema/FieldType.java
+++ b/solr/core/src/java/org/apache/solr/schema/FieldType.java
@@ -38,6 +38,7 @@ import org.apache.lucene.document.Field;
 import org.apache.lucene.index.IndexOptions;
 import org.apache.lucene.index.IndexableField;
 import org.apache.lucene.index.Term;
+import org.apache.lucene.legacy.LegacyNumericType;
 import org.apache.lucene.queries.function.ValueSource;
 import org.apache.lucene.search.DocValuesRangeQuery;
 import org.apache.lucene.search.DocValuesRewriteMethod;
@@ -621,7 +622,7 @@ public abstract class FieldType extends FieldProperties {
 
   /** Return the numeric type of this field, or null if this field is not a
    *  numeric field. */
-  public org.apache.lucene.document.FieldType.LegacyNumericType getNumericType() {
+  public LegacyNumericType getNumericType() {
     return null;
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/105c7eae/solr/core/src/java/org/apache/solr/schema/SpatialPointVectorFieldType.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/schema/SpatialPointVectorFieldType.java b/solr/core/src/java/org/apache/solr/schema/SpatialPointVectorFieldType.java
index 18d80a3..f6bb782 100644
--- a/solr/core/src/java/org/apache/solr/schema/SpatialPointVectorFieldType.java
+++ b/solr/core/src/java/org/apache/solr/schema/SpatialPointVectorFieldType.java
@@ -20,6 +20,8 @@ import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
 
+import org.apache.lucene.legacy.LegacyFieldType;
+import org.apache.lucene.legacy.LegacyNumericType;
 import org.apache.lucene.spatial.vector.PointVectorStrategy;
 
 /**
@@ -78,8 +80,8 @@ public class SpatialPointVectorFieldType extends AbstractSpatialFieldType<PointV
   }
 
   @Override
-  public org.apache.lucene.document.FieldType.LegacyNumericType getNumericType() {
-    return org.apache.lucene.document.FieldType.LegacyNumericType.DOUBLE;
+  public LegacyNumericType getNumericType() {
+    return LegacyNumericType.DOUBLE;
   }
 
   @Override
@@ -88,8 +90,7 @@ public class SpatialPointVectorFieldType extends AbstractSpatialFieldType<PointV
     if (this.getNumericType() != null) {
       // create strategy based on legacy numerics
       // todo remove in 7.0
-      org.apache.lucene.document.FieldType fieldType =
-          new org.apache.lucene.document.FieldType(PointVectorStrategy.LEGACY_FIELDTYPE);
+      LegacyFieldType fieldType = new LegacyFieldType(PointVectorStrategy.LEGACY_FIELDTYPE);
       fieldType.setNumericPrecisionStep(precisionStep);
       return new PointVectorStrategy(ctx, fieldName, fieldType);
     } else {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/105c7eae/solr/core/src/java/org/apache/solr/schema/TrieDoubleField.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/schema/TrieDoubleField.java b/solr/core/src/java/org/apache/solr/schema/TrieDoubleField.java
index edaa0c4..cb409a1 100644
--- a/solr/core/src/java/org/apache/solr/schema/TrieDoubleField.java
+++ b/solr/core/src/java/org/apache/solr/schema/TrieDoubleField.java
@@ -23,13 +23,13 @@ import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.index.DocValues;
 import org.apache.lucene.index.SortedDocValues;
 import org.apache.lucene.index.SortedSetDocValues;
+import org.apache.lucene.legacy.LegacyNumericUtils;
 import org.apache.lucene.queries.function.ValueSource;
 import org.apache.lucene.queries.function.FunctionValues;
 import org.apache.lucene.queries.function.docvalues.DoubleDocValues;
 import org.apache.lucene.queries.function.valuesource.SortedSetFieldSource;
 import org.apache.lucene.search.SortedSetSelector;
 import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.LegacyNumericUtils;
 import org.apache.lucene.util.NumericUtils;
 import org.apache.lucene.util.mutable.MutableValue;
 import org.apache.lucene.util.mutable.MutableValueDouble;