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

[6/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/backward-codecs/src/java/org/apache/lucene/legacy/LegacyNumericTokenStream.java
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/java/org/apache/lucene/legacy/LegacyNumericTokenStream.java b/lucene/backward-codecs/src/java/org/apache/lucene/legacy/LegacyNumericTokenStream.java
new file mode 100644
index 0000000..a2aba19
--- /dev/null
+++ b/lucene/backward-codecs/src/java/org/apache/lucene/legacy/LegacyNumericTokenStream.java
@@ -0,0 +1,357 @@
+/*
+ * 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.legacy;
+
+
+import java.util.Objects;
+
+import org.apache.lucene.analysis.TokenStream;
+import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
+import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
+import org.apache.lucene.analysis.tokenattributes.TermToBytesRefAttribute;
+import org.apache.lucene.analysis.tokenattributes.TypeAttribute;
+import org.apache.lucene.util.Attribute;
+import org.apache.lucene.util.AttributeFactory;
+import org.apache.lucene.util.AttributeImpl;
+import org.apache.lucene.util.AttributeReflector;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefBuilder;
+import org.apache.lucene.util.NumericUtils;
+
+/**
+ * <b>Expert:</b> This class provides a {@link TokenStream}
+ * for indexing numeric values that can be used by {@link
+ * org.apache.lucene.legacy.LegacyNumericRangeQuery}.
+ *
+ * <p>Note that for simple usage, {@link org.apache.lucene.legacy.LegacyIntField}, {@link
+ * org.apache.lucene.legacy.LegacyLongField}, {@link org.apache.lucene.legacy.LegacyFloatField} or {@link org.apache.lucene.legacy.LegacyDoubleField} is
+ * recommended.  These fields disable norms and
+ * term freqs, as they are not usually needed during
+ * searching.  If you need to change these settings, you
+ * should use this class.
+ *
+ * <p>Here's an example usage, for an <code>int</code> field:
+ *
+ * <pre class="prettyprint">
+ *  FieldType fieldType = new FieldType(TextField.TYPE_NOT_STORED);
+ *  fieldType.setOmitNorms(true);
+ *  fieldType.setIndexOptions(IndexOptions.DOCS_ONLY);
+ *  Field field = new Field(name, new LegacyNumericTokenStream(precisionStep).setIntValue(value), fieldType);
+ *  document.add(field);
+ * </pre>
+ *
+ * <p>For optimal performance, re-use the TokenStream and Field instance
+ * for more than one document:
+ *
+ * <pre class="prettyprint">
+ *  LegacyNumericTokenStream stream = new LegacyNumericTokenStream(precisionStep);
+ *  FieldType fieldType = new FieldType(TextField.TYPE_NOT_STORED);
+ *  fieldType.setOmitNorms(true);
+ *  fieldType.setIndexOptions(IndexOptions.DOCS_ONLY);
+ *  Field field = new Field(name, stream, fieldType);
+ *  Document document = new Document();
+ *  document.add(field);
+ *
+ *  for(all documents) {
+ *    stream.setIntValue(value)
+ *    writer.addDocument(document);
+ *  }
+ * </pre>
+ *
+ * <p>This stream is not intended to be used in analyzers;
+ * it's more for iterating the different precisions during
+ * indexing a specific numeric value.</p>
+
+ * <p><b>NOTE</b>: as token streams are only consumed once
+ * the document is added to the index, if you index more
+ * than one numeric field, use a separate <code>LegacyNumericTokenStream</code>
+ * instance for each.</p>
+ *
+ * <p>See {@link org.apache.lucene.legacy.LegacyNumericRangeQuery} for more details on the
+ * <a
+ * href="LegacyNumericRangeQuery.html#precisionStepDesc"><code>precisionStep</code></a>
+ * parameter as well as how numeric fields work under the hood.</p>
+ *
+ * @deprecated Please switch to {@link org.apache.lucene.index.PointValues} instead
+ *
+ * @since 2.9
+ */
+@Deprecated
+public final class LegacyNumericTokenStream extends TokenStream {
+
+  /** The full precision token gets this token type assigned. */
+  public static final String TOKEN_TYPE_FULL_PREC  = "fullPrecNumeric";
+
+  /** The lower precision tokens gets this token type assigned. */
+  public static final String TOKEN_TYPE_LOWER_PREC = "lowerPrecNumeric";
+  
+  /** <b>Expert:</b> Use this attribute to get the details of the currently generated token.
+   * @lucene.experimental
+   * @since 4.0
+   */
+  public interface LegacyNumericTermAttribute extends Attribute {
+    /** Returns current shift value, undefined before first token */
+    int getShift();
+    /** Returns current token's raw value as {@code long} with all {@link #getShift} applied, undefined before first token */
+    long getRawValue();
+    /** Returns value size in bits (32 for {@code float}, {@code int}; 64 for {@code double}, {@code long}) */
+    int getValueSize();
+    
+    /** <em>Don't call this method!</em>
+      * @lucene.internal */
+    void init(long value, int valSize, int precisionStep, int shift);
+
+    /** <em>Don't call this method!</em>
+      * @lucene.internal */
+    void setShift(int shift);
+
+    /** <em>Don't call this method!</em>
+      * @lucene.internal */
+    int incShift();
+  }
+  
+  // just a wrapper to prevent adding CTA
+  private static final class NumericAttributeFactory extends AttributeFactory {
+    private final AttributeFactory delegate;
+
+    NumericAttributeFactory(AttributeFactory delegate) {
+      this.delegate = delegate;
+    }
+  
+    @Override
+    public AttributeImpl createAttributeInstance(Class<? extends Attribute> attClass) {
+      if (CharTermAttribute.class.isAssignableFrom(attClass))
+        throw new IllegalArgumentException("LegacyNumericTokenStream does not support CharTermAttribute.");
+      return delegate.createAttributeInstance(attClass);
+    }
+  }
+
+  /** Implementation of {@link org.apache.lucene.legacy.LegacyNumericTokenStream.LegacyNumericTermAttribute}.
+   * @lucene.internal
+   * @since 4.0
+   */
+  public static final class LegacyNumericTermAttributeImpl extends AttributeImpl implements LegacyNumericTermAttribute,TermToBytesRefAttribute {
+    private long value = 0L;
+    private int valueSize = 0, shift = 0, precisionStep = 0;
+    private BytesRefBuilder bytes = new BytesRefBuilder();
+    
+    /** 
+     * Creates, but does not yet initialize this attribute instance
+     * @see #init(long, int, int, int)
+     */
+    public LegacyNumericTermAttributeImpl() {}
+
+    @Override
+    public BytesRef getBytesRef() {
+      assert valueSize == 64 || valueSize == 32;
+      if (shift >= valueSize) {
+        bytes.clear();
+      } else if (valueSize == 64) {
+        LegacyNumericUtils.longToPrefixCoded(value, shift, bytes);
+      } else {
+        LegacyNumericUtils.intToPrefixCoded((int) value, shift, bytes);
+      }
+      return bytes.get();
+    }
+
+    @Override
+    public int getShift() { return shift; }
+    @Override
+    public void setShift(int shift) { this.shift = shift; }
+    @Override
+    public int incShift() {
+      return (shift += precisionStep);
+    }
+
+    @Override
+    public long getRawValue() { return value  & ~((1L << shift) - 1L); }
+    @Override
+    public int getValueSize() { return valueSize; }
+
+    @Override
+    public void init(long value, int valueSize, int precisionStep, int shift) {
+      this.value = value;
+      this.valueSize = valueSize;
+      this.precisionStep = precisionStep;
+      this.shift = shift;
+    }
+
+    @Override
+    public void clear() {
+      // this attribute has no contents to clear!
+      // we keep it untouched as it's fully controlled by outer class.
+    }
+    
+    @Override
+    public void reflectWith(AttributeReflector reflector) {
+      reflector.reflect(TermToBytesRefAttribute.class, "bytes", getBytesRef());
+      reflector.reflect(LegacyNumericTermAttribute.class, "shift", shift);
+      reflector.reflect(LegacyNumericTermAttribute.class, "rawValue", getRawValue());
+      reflector.reflect(LegacyNumericTermAttribute.class, "valueSize", valueSize);
+    }
+  
+    @Override
+    public void copyTo(AttributeImpl target) {
+      final LegacyNumericTermAttribute a = (LegacyNumericTermAttribute) target;
+      a.init(value, valueSize, precisionStep, shift);
+    }
+    
+    @Override
+    public LegacyNumericTermAttributeImpl clone() {
+      LegacyNumericTermAttributeImpl t = (LegacyNumericTermAttributeImpl)super.clone();
+      // Do a deep clone
+      t.bytes = new BytesRefBuilder();
+      t.bytes.copyBytes(getBytesRef());
+      return t;
+    }
+
+    @Override
+    public int hashCode() {
+      return Objects.hash(precisionStep, shift, value, valueSize);
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+      if (this == obj) return true;
+      if (obj == null) return false;
+      if (getClass() != obj.getClass()) return false;
+      LegacyNumericTermAttributeImpl other = (LegacyNumericTermAttributeImpl) obj;
+      if (precisionStep != other.precisionStep) return false;
+      if (shift != other.shift) return false;
+      if (value != other.value) return false;
+      if (valueSize != other.valueSize) return false;
+      return true;
+    }
+  }
+  
+  /**
+   * Creates a token stream for numeric values using the default <code>precisionStep</code>
+   * {@link org.apache.lucene.legacy.LegacyNumericUtils#PRECISION_STEP_DEFAULT} (16). The stream is not yet initialized,
+   * before using set a value using the various set<em>???</em>Value() methods.
+   */
+  public LegacyNumericTokenStream() {
+    this(AttributeFactory.DEFAULT_ATTRIBUTE_FACTORY, LegacyNumericUtils.PRECISION_STEP_DEFAULT);
+  }
+  
+  /**
+   * Creates a token stream for numeric values with the specified
+   * <code>precisionStep</code>. The stream is not yet initialized,
+   * before using set a value using the various set<em>???</em>Value() methods.
+   */
+  public LegacyNumericTokenStream(final int precisionStep) {
+    this(AttributeFactory.DEFAULT_ATTRIBUTE_FACTORY, precisionStep);
+  }
+
+  /**
+   * Expert: Creates a token stream for numeric values with the specified
+   * <code>precisionStep</code> using the given
+   * {@link org.apache.lucene.util.AttributeFactory}.
+   * The stream is not yet initialized,
+   * before using set a value using the various set<em>???</em>Value() methods.
+   */
+  public LegacyNumericTokenStream(AttributeFactory factory, final int precisionStep) {
+    super(new NumericAttributeFactory(factory));
+    if (precisionStep < 1)
+      throw new IllegalArgumentException("precisionStep must be >=1");
+    this.precisionStep = precisionStep;
+    numericAtt.setShift(-precisionStep);
+  }
+
+  /**
+   * Initializes the token stream with the supplied <code>long</code> value.
+   * @param value the value, for which this TokenStream should enumerate tokens.
+   * @return this instance, because of this you can use it the following way:
+   * <code>new Field(name, new LegacyNumericTokenStream(precisionStep).setLongValue(value))</code>
+   */
+  public LegacyNumericTokenStream setLongValue(final long value) {
+    numericAtt.init(value, valSize = 64, precisionStep, -precisionStep);
+    return this;
+  }
+  
+  /**
+   * Initializes the token stream with the supplied <code>int</code> value.
+   * @param value the value, for which this TokenStream should enumerate tokens.
+   * @return this instance, because of this you can use it the following way:
+   * <code>new Field(name, new LegacyNumericTokenStream(precisionStep).setIntValue(value))</code>
+   */
+  public LegacyNumericTokenStream setIntValue(final int value) {
+    numericAtt.init(value, valSize = 32, precisionStep, -precisionStep);
+    return this;
+  }
+  
+  /**
+   * Initializes the token stream with the supplied <code>double</code> value.
+   * @param value the value, for which this TokenStream should enumerate tokens.
+   * @return this instance, because of this you can use it the following way:
+   * <code>new Field(name, new LegacyNumericTokenStream(precisionStep).setDoubleValue(value))</code>
+   */
+  public LegacyNumericTokenStream setDoubleValue(final double value) {
+    numericAtt.init(NumericUtils.doubleToSortableLong(value), valSize = 64, precisionStep, -precisionStep);
+    return this;
+  }
+  
+  /**
+   * Initializes the token stream with the supplied <code>float</code> value.
+   * @param value the value, for which this TokenStream should enumerate tokens.
+   * @return this instance, because of this you can use it the following way:
+   * <code>new Field(name, new LegacyNumericTokenStream(precisionStep).setFloatValue(value))</code>
+   */
+  public LegacyNumericTokenStream setFloatValue(final float value) {
+    numericAtt.init(NumericUtils.floatToSortableInt(value), valSize = 32, precisionStep, -precisionStep);
+    return this;
+  }
+  
+  @Override
+  public void reset() {
+    if (valSize == 0)
+      throw new IllegalStateException("call set???Value() before usage");
+    numericAtt.setShift(-precisionStep);
+  }
+
+  @Override
+  public boolean incrementToken() {
+    if (valSize == 0)
+      throw new IllegalStateException("call set???Value() before usage");
+    
+    // this will only clear all other attributes in this TokenStream
+    clearAttributes();
+
+    final int shift = numericAtt.incShift();
+    typeAtt.setType((shift == 0) ? TOKEN_TYPE_FULL_PREC : TOKEN_TYPE_LOWER_PREC);
+    posIncrAtt.setPositionIncrement((shift == 0) ? 1 : 0);
+    return (shift < valSize);
+  }
+
+  /** Returns the precision step. */
+  public int getPrecisionStep() {
+    return precisionStep;
+  }
+
+  @Override
+  public String toString() {
+    // We override default because it can throw cryptic "illegal shift value":
+    return getClass().getSimpleName() + "(precisionStep=" + precisionStep + " valueSize=" + numericAtt.getValueSize() + " shift=" + numericAtt.getShift() + ")";
+  }
+  
+  // members
+  private final LegacyNumericTermAttribute numericAtt = addAttribute(LegacyNumericTermAttribute.class);
+  private final TypeAttribute typeAtt = addAttribute(TypeAttribute.class);
+  private final PositionIncrementAttribute posIncrAtt = addAttribute(PositionIncrementAttribute.class);
+  
+  private int valSize = 0; // valSize==0 means not initialized
+  private final int precisionStep;
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/105c7eae/lucene/backward-codecs/src/java/org/apache/lucene/legacy/LegacyNumericType.java
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/java/org/apache/lucene/legacy/LegacyNumericType.java b/lucene/backward-codecs/src/java/org/apache/lucene/legacy/LegacyNumericType.java
new file mode 100644
index 0000000..345b497
--- /dev/null
+++ b/lucene/backward-codecs/src/java/org/apache/lucene/legacy/LegacyNumericType.java
@@ -0,0 +1,34 @@
+/*
+ * 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.legacy;
+
+/** Data type of the numeric value
+ * @since 3.2
+ *
+ * @deprecated Please switch to {@link org.apache.lucene.index.PointValues} instead
+ */
+@Deprecated
+public enum LegacyNumericType {
+  /** 32-bit integer numeric type */
+  INT, 
+  /** 64-bit long numeric type */
+  LONG, 
+  /** 32-bit float numeric type */
+  FLOAT, 
+  /** 64-bit double numeric type */
+  DOUBLE
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/105c7eae/lucene/backward-codecs/src/java/org/apache/lucene/legacy/LegacyNumericUtils.java
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/java/org/apache/lucene/legacy/LegacyNumericUtils.java b/lucene/backward-codecs/src/java/org/apache/lucene/legacy/LegacyNumericUtils.java
new file mode 100644
index 0000000..e6659d7
--- /dev/null
+++ b/lucene/backward-codecs/src/java/org/apache/lucene/legacy/LegacyNumericUtils.java
@@ -0,0 +1,510 @@
+/*
+ * 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.legacy;
+
+
+import java.io.IOException;
+
+import org.apache.lucene.index.FilterLeafReader;
+import org.apache.lucene.index.FilteredTermsEnum;
+import org.apache.lucene.index.Terms;
+import org.apache.lucene.index.TermsEnum;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.BytesRefBuilder;
+
+/**
+ * This is a helper class to generate prefix-encoded representations for numerical values
+ * and supplies converters to represent float/double values as sortable integers/longs.
+ *
+ * <p>To quickly execute range queries in Apache Lucene, a range is divided recursively
+ * into multiple intervals for searching: The center of the range is searched only with
+ * the lowest possible precision in the trie, while the boundaries are matched
+ * more exactly. This reduces the number of terms dramatically.
+ *
+ * <p>This class generates terms to achieve this: First the numerical integer values need to
+ * be converted to bytes. For that integer values (32 bit or 64 bit) are made unsigned
+ * and the bits are converted to ASCII chars with each 7 bit. The resulting byte[] is
+ * sortable like the original integer value (even using UTF-8 sort order). Each value is also
+ * prefixed (in the first char) by the <code>shift</code> value (number of bits removed) used
+ * during encoding.
+ *
+ * <p>For easy usage, the trie algorithm is implemented for indexing inside
+ * {@link org.apache.lucene.legacy.LegacyNumericTokenStream} that can index <code>int</code>, <code>long</code>,
+ * <code>float</code>, and <code>double</code>. For querying,
+ * {@link org.apache.lucene.legacy.LegacyNumericRangeQuery} implements the query part
+ * for the same data types.
+ *
+ * @lucene.internal
+ *
+ * @deprecated Please use {@link org.apache.lucene.index.PointValues} instead.
+ *
+ * @since 2.9, API changed non backwards-compliant in 4.0
+ */
+
+@Deprecated
+public final class LegacyNumericUtils {
+
+  private LegacyNumericUtils() {} // no instance!
+  
+  /**
+   * The default precision step used by {@link org.apache.lucene.legacy.LegacyLongField},
+   * {@link org.apache.lucene.legacy.LegacyDoubleField}, {@link org.apache.lucene.legacy.LegacyNumericTokenStream}, {@link
+   * org.apache.lucene.legacy.LegacyNumericRangeQuery}.
+   */
+  public static final int PRECISION_STEP_DEFAULT = 16;
+  
+  /**
+   * The default precision step used by {@link org.apache.lucene.legacy.LegacyIntField} and
+   * {@link org.apache.lucene.legacy.LegacyFloatField}.
+   */
+  public static final int PRECISION_STEP_DEFAULT_32 = 8;
+  
+  /**
+   * Longs are stored at lower precision by shifting off lower bits. The shift count is
+   * stored as <code>SHIFT_START_LONG+shift</code> in the first byte
+   */
+  public static final byte SHIFT_START_LONG = 0x20;
+
+  /**
+   * The maximum term length (used for <code>byte[]</code> buffer size)
+   * for encoding <code>long</code> values.
+   * @see #longToPrefixCoded
+   */
+  public static final int BUF_SIZE_LONG = 63/7 + 2;
+
+  /**
+   * Integers are stored at lower precision by shifting off lower bits. The shift count is
+   * stored as <code>SHIFT_START_INT+shift</code> in the first byte
+   */
+  public static final byte SHIFT_START_INT  = 0x60;
+
+  /**
+   * The maximum term length (used for <code>byte[]</code> buffer size)
+   * for encoding <code>int</code> values.
+   * @see #intToPrefixCoded
+   */
+  public static final int BUF_SIZE_INT = 31/7 + 2;
+
+  /**
+   * Returns prefix coded bits after reducing the precision by <code>shift</code> bits.
+   * This is method is used by {@link org.apache.lucene.legacy.LegacyNumericTokenStream}.
+   * After encoding, {@code bytes.offset} will always be 0. 
+   * @param val the numeric value
+   * @param shift how many bits to strip from the right
+   * @param bytes will contain the encoded value
+   */
+  public static void longToPrefixCoded(final long val, final int shift, final BytesRefBuilder bytes) {
+    // ensure shift is 0..63
+    if ((shift & ~0x3f) != 0) {
+      throw new IllegalArgumentException("Illegal shift value, must be 0..63; got shift=" + shift);
+    }
+    int nChars = (((63-shift)*37)>>8) + 1;    // i/7 is the same as (i*37)>>8 for i in 0..63
+    bytes.setLength(nChars+1);   // one extra for the byte that contains the shift info
+    bytes.grow(BUF_SIZE_LONG);
+    bytes.setByteAt(0, (byte)(SHIFT_START_LONG + shift));
+    long sortableBits = val ^ 0x8000000000000000L;
+    sortableBits >>>= shift;
+    while (nChars > 0) {
+      // Store 7 bits per byte for compatibility
+      // with UTF-8 encoding of terms
+      bytes.setByteAt(nChars--, (byte)(sortableBits & 0x7f));
+      sortableBits >>>= 7;
+    }
+  }
+
+  /**
+   * Returns prefix coded bits after reducing the precision by <code>shift</code> bits.
+   * This is method is used by {@link org.apache.lucene.legacy.LegacyNumericTokenStream}.
+   * After encoding, {@code bytes.offset} will always be 0.
+   * @param val the numeric value
+   * @param shift how many bits to strip from the right
+   * @param bytes will contain the encoded value
+   */
+  public static void intToPrefixCoded(final int val, final int shift, final BytesRefBuilder bytes) {
+    // ensure shift is 0..31
+    if ((shift & ~0x1f) != 0) {
+      throw new IllegalArgumentException("Illegal shift value, must be 0..31; got shift=" + shift);
+    }
+    int nChars = (((31-shift)*37)>>8) + 1;    // i/7 is the same as (i*37)>>8 for i in 0..63
+    bytes.setLength(nChars+1);   // one extra for the byte that contains the shift info
+    bytes.grow(LegacyNumericUtils.BUF_SIZE_LONG);  // use the max
+    bytes.setByteAt(0, (byte)(SHIFT_START_INT + shift));
+    int sortableBits = val ^ 0x80000000;
+    sortableBits >>>= shift;
+    while (nChars > 0) {
+      // Store 7 bits per byte for compatibility
+      // with UTF-8 encoding of terms
+      bytes.setByteAt(nChars--, (byte)(sortableBits & 0x7f));
+      sortableBits >>>= 7;
+    }
+  }
+
+
+  /**
+   * Returns the shift value from a prefix encoded {@code long}.
+   * @throws NumberFormatException if the supplied {@link BytesRef} is
+   * not correctly prefix encoded.
+   */
+  public static int getPrefixCodedLongShift(final BytesRef val) {
+    final int shift = val.bytes[val.offset] - SHIFT_START_LONG;
+    if (shift > 63 || shift < 0)
+      throw new NumberFormatException("Invalid shift value (" + shift + ") in prefixCoded bytes (is encoded value really an INT?)");
+    return shift;
+  }
+
+  /**
+   * Returns the shift value from a prefix encoded {@code int}.
+   * @throws NumberFormatException if the supplied {@link BytesRef} is
+   * not correctly prefix encoded.
+   */
+  public static int getPrefixCodedIntShift(final BytesRef val) {
+    final int shift = val.bytes[val.offset] - SHIFT_START_INT;
+    if (shift > 31 || shift < 0)
+      throw new NumberFormatException("Invalid shift value in prefixCoded bytes (is encoded value really an INT?)");
+    return shift;
+  }
+
+  /**
+   * Returns a long from prefixCoded bytes.
+   * Rightmost bits will be zero for lower precision codes.
+   * This method can be used to decode a term's value.
+   * @throws NumberFormatException if the supplied {@link BytesRef} is
+   * not correctly prefix encoded.
+   * @see #longToPrefixCoded
+   */
+  public static long prefixCodedToLong(final BytesRef val) {
+    long sortableBits = 0L;
+    for (int i=val.offset+1, limit=val.offset+val.length; i<limit; i++) {
+      sortableBits <<= 7;
+      final byte b = val.bytes[i];
+      if (b < 0) {
+        throw new NumberFormatException(
+          "Invalid prefixCoded numerical value representation (byte "+
+          Integer.toHexString(b&0xff)+" at position "+(i-val.offset)+" is invalid)"
+        );
+      }
+      sortableBits |= b;
+    }
+    return (sortableBits << getPrefixCodedLongShift(val)) ^ 0x8000000000000000L;
+  }
+
+  /**
+   * Returns an int from prefixCoded bytes.
+   * Rightmost bits will be zero for lower precision codes.
+   * This method can be used to decode a term's value.
+   * @throws NumberFormatException if the supplied {@link BytesRef} is
+   * not correctly prefix encoded.
+   * @see #intToPrefixCoded
+   */
+  public static int prefixCodedToInt(final BytesRef val) {
+    int sortableBits = 0;
+    for (int i=val.offset+1, limit=val.offset+val.length; i<limit; i++) {
+      sortableBits <<= 7;
+      final byte b = val.bytes[i];
+      if (b < 0) {
+        throw new NumberFormatException(
+          "Invalid prefixCoded numerical value representation (byte "+
+          Integer.toHexString(b&0xff)+" at position "+(i-val.offset)+" is invalid)"
+        );
+      }
+      sortableBits |= b;
+    }
+    return (sortableBits << getPrefixCodedIntShift(val)) ^ 0x80000000;
+  }
+
+  /**
+   * Splits a long range recursively.
+   * You may implement a builder that adds clauses to a
+   * {@link org.apache.lucene.search.BooleanQuery} for each call to its
+   * {@link LongRangeBuilder#addRange(BytesRef,BytesRef)}
+   * method.
+   * <p>This method is used by {@link org.apache.lucene.legacy.LegacyNumericRangeQuery}.
+   */
+  public static void splitLongRange(final LongRangeBuilder builder,
+    final int precisionStep,  final long minBound, final long maxBound
+  ) {
+    splitRange(builder, 64, precisionStep, minBound, maxBound);
+  }
+  
+  /**
+   * Splits an int range recursively.
+   * You may implement a builder that adds clauses to a
+   * {@link org.apache.lucene.search.BooleanQuery} for each call to its
+   * {@link IntRangeBuilder#addRange(BytesRef,BytesRef)}
+   * method.
+   * <p>This method is used by {@link org.apache.lucene.legacy.LegacyNumericRangeQuery}.
+   */
+  public static void splitIntRange(final IntRangeBuilder builder,
+    final int precisionStep,  final int minBound, final int maxBound
+  ) {
+    splitRange(builder, 32, precisionStep, minBound, maxBound);
+  }
+  
+  /** This helper does the splitting for both 32 and 64 bit. */
+  private static void splitRange(
+    final Object builder, final int valSize,
+    final int precisionStep, long minBound, long maxBound
+  ) {
+    if (precisionStep < 1)
+      throw new IllegalArgumentException("precisionStep must be >=1");
+    if (minBound > maxBound) return;
+    for (int shift=0; ; shift += precisionStep) {
+      // calculate new bounds for inner precision
+      final long diff = 1L << (shift+precisionStep),
+        mask = ((1L<<precisionStep) - 1L) << shift;
+      final boolean
+        hasLower = (minBound & mask) != 0L,
+        hasUpper = (maxBound & mask) != mask;
+      final long
+        nextMinBound = (hasLower ? (minBound + diff) : minBound) & ~mask,
+        nextMaxBound = (hasUpper ? (maxBound - diff) : maxBound) & ~mask;
+      final boolean
+        lowerWrapped = nextMinBound < minBound,
+        upperWrapped = nextMaxBound > maxBound;
+      
+      if (shift+precisionStep>=valSize || nextMinBound>nextMaxBound || lowerWrapped || upperWrapped) {
+        // We are in the lowest precision or the next precision is not available.
+        addRange(builder, valSize, minBound, maxBound, shift);
+        // exit the split recursion loop
+        break;
+      }
+      
+      if (hasLower)
+        addRange(builder, valSize, minBound, minBound | mask, shift);
+      if (hasUpper)
+        addRange(builder, valSize, maxBound & ~mask, maxBound, shift);
+      
+      // recurse to next precision
+      minBound = nextMinBound;
+      maxBound = nextMaxBound;
+    }
+  }
+  
+  /** Helper that delegates to correct range builder */
+  private static void addRange(
+    final Object builder, final int valSize,
+    long minBound, long maxBound,
+    final int shift
+  ) {
+    // for the max bound set all lower bits (that were shifted away):
+    // this is important for testing or other usages of the splitted range
+    // (e.g. to reconstruct the full range). The prefixEncoding will remove
+    // the bits anyway, so they do not hurt!
+    maxBound |= (1L << shift) - 1L;
+    // delegate to correct range builder
+    switch(valSize) {
+      case 64:
+        ((LongRangeBuilder)builder).addRange(minBound, maxBound, shift);
+        break;
+      case 32:
+        ((IntRangeBuilder)builder).addRange((int)minBound, (int)maxBound, shift);
+        break;
+      default:
+        // Should not happen!
+        throw new IllegalArgumentException("valSize must be 32 or 64.");
+    }
+  }
+
+  /**
+   * Callback for {@link #splitLongRange}.
+   * You need to overwrite only one of the methods.
+   * @lucene.internal
+   * @since 2.9, API changed non backwards-compliant in 4.0
+   */
+  public static abstract class LongRangeBuilder {
+    
+    /**
+     * Overwrite this method, if you like to receive the already prefix encoded range bounds.
+     * You can directly build classical (inclusive) range queries from them.
+     */
+    public void addRange(BytesRef minPrefixCoded, BytesRef maxPrefixCoded) {
+      throw new UnsupportedOperationException();
+    }
+    
+    /**
+     * Overwrite this method, if you like to receive the raw long range bounds.
+     * You can use this for e.g. debugging purposes (print out range bounds).
+     */
+    public void addRange(final long min, final long max, final int shift) {
+      final BytesRefBuilder minBytes = new BytesRefBuilder(), maxBytes = new BytesRefBuilder();
+      longToPrefixCoded(min, shift, minBytes);
+      longToPrefixCoded(max, shift, maxBytes);
+      addRange(minBytes.get(), maxBytes.get());
+    }
+  
+  }
+  
+  /**
+   * Callback for {@link #splitIntRange}.
+   * You need to overwrite only one of the methods.
+   * @lucene.internal
+   * @since 2.9, API changed non backwards-compliant in 4.0
+   */
+  public static abstract class IntRangeBuilder {
+    
+    /**
+     * Overwrite this method, if you like to receive the already prefix encoded range bounds.
+     * You can directly build classical range (inclusive) queries from them.
+     */
+    public void addRange(BytesRef minPrefixCoded, BytesRef maxPrefixCoded) {
+      throw new UnsupportedOperationException();
+    }
+    
+    /**
+     * Overwrite this method, if you like to receive the raw int range bounds.
+     * You can use this for e.g. debugging purposes (print out range bounds).
+     */
+    public void addRange(final int min, final int max, final int shift) {
+      final BytesRefBuilder minBytes = new BytesRefBuilder(), maxBytes = new BytesRefBuilder();
+      intToPrefixCoded(min, shift, minBytes);
+      intToPrefixCoded(max, shift, maxBytes);
+      addRange(minBytes.get(), maxBytes.get());
+    }
+  
+  }
+  
+  /**
+   * Filters the given {@link TermsEnum} by accepting only prefix coded 64 bit
+   * terms with a shift value of <tt>0</tt>.
+   * 
+   * @param termsEnum
+   *          the terms enum to filter
+   * @return a filtered {@link TermsEnum} that only returns prefix coded 64 bit
+   *         terms with a shift value of <tt>0</tt>.
+   */
+  public static TermsEnum filterPrefixCodedLongs(TermsEnum termsEnum) {
+    return new SeekingNumericFilteredTermsEnum(termsEnum) {
+
+      @Override
+      protected AcceptStatus accept(BytesRef term) {
+        return LegacyNumericUtils.getPrefixCodedLongShift(term) == 0 ? AcceptStatus.YES : AcceptStatus.END;
+      }
+    };
+  }
+
+  /**
+   * Filters the given {@link TermsEnum} by accepting only prefix coded 32 bit
+   * terms with a shift value of <tt>0</tt>.
+   * 
+   * @param termsEnum
+   *          the terms enum to filter
+   * @return a filtered {@link TermsEnum} that only returns prefix coded 32 bit
+   *         terms with a shift value of <tt>0</tt>.
+   */
+  public static TermsEnum filterPrefixCodedInts(TermsEnum termsEnum) {
+    return new SeekingNumericFilteredTermsEnum(termsEnum) {
+      
+      @Override
+      protected AcceptStatus accept(BytesRef term) {
+        return LegacyNumericUtils.getPrefixCodedIntShift(term) == 0 ? AcceptStatus.YES : AcceptStatus.END;
+      }
+    };
+  }
+
+  /** Just like FilteredTermsEnum, except it adds a limited
+   *  seekCeil implementation that only works with {@link
+   *  #filterPrefixCodedInts} and {@link
+   *  #filterPrefixCodedLongs}. */
+  private static abstract class SeekingNumericFilteredTermsEnum extends FilteredTermsEnum {
+    public SeekingNumericFilteredTermsEnum(final TermsEnum tenum) {
+      super(tenum, false);
+    }
+
+    @Override
+    @SuppressWarnings("fallthrough")
+    public SeekStatus seekCeil(BytesRef term) throws IOException {
+
+      // NOTE: This is not general!!  It only handles YES
+      // and END, because that's all we need for the numeric
+      // case here
+
+      SeekStatus status = tenum.seekCeil(term);
+      if (status == SeekStatus.END) {
+        return SeekStatus.END;
+      }
+
+      actualTerm = tenum.term();
+
+      if (accept(actualTerm) == AcceptStatus.YES) {
+        return status;
+      } else {
+        return SeekStatus.END;
+      }
+    }
+  }
+
+  private static Terms intTerms(Terms terms) {
+    return new FilterLeafReader.FilterTerms(terms) {
+        @Override
+        public TermsEnum iterator() throws IOException {
+          return filterPrefixCodedInts(in.iterator());
+        }
+      };
+  }
+
+  private static Terms longTerms(Terms terms) {
+    return new FilterLeafReader.FilterTerms(terms) {
+        @Override
+        public TermsEnum iterator() throws IOException {
+          return filterPrefixCodedLongs(in.iterator());
+        }
+      };
+  }
+    
+  /**
+   * Returns the minimum int value indexed into this
+   * numeric field or null if no terms exist.
+   */
+  public static Integer getMinInt(Terms terms) throws IOException {
+    // All shift=0 terms are sorted first, so we don't need
+    // to filter the incoming terms; we can just get the
+    // min:
+    BytesRef min = terms.getMin();
+    return (min != null) ? LegacyNumericUtils.prefixCodedToInt(min) : null;
+  }
+
+  /**
+   * Returns the maximum int value indexed into this
+   * numeric field or null if no terms exist.
+   */
+  public static Integer getMaxInt(Terms terms) throws IOException {
+    BytesRef max = intTerms(terms).getMax();
+    return (max != null) ? LegacyNumericUtils.prefixCodedToInt(max) : null;
+  }
+
+  /**
+   * Returns the minimum long value indexed into this
+   * numeric field or null if no terms exist.
+   */
+  public static Long getMinLong(Terms terms) throws IOException {
+    // All shift=0 terms are sorted first, so we don't need
+    // to filter the incoming terms; we can just get the
+    // min:
+    BytesRef min = terms.getMin();
+    return (min != null) ? LegacyNumericUtils.prefixCodedToLong(min) : null;
+  }
+
+  /**
+   * Returns the maximum long value indexed into this
+   * numeric field or null if no terms exist.
+   */
+  public static Long getMaxLong(Terms terms) throws IOException {
+    BytesRef max = longTerms(terms).getMax();
+    return (max != null) ? LegacyNumericUtils.prefixCodedToLong(max) : null;
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/105c7eae/lucene/backward-codecs/src/java/org/apache/lucene/legacy/package-info.java
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/java/org/apache/lucene/legacy/package-info.java b/lucene/backward-codecs/src/java/org/apache/lucene/legacy/package-info.java
new file mode 100644
index 0000000..d0167f8
--- /dev/null
+++ b/lucene/backward-codecs/src/java/org/apache/lucene/legacy/package-info.java
@@ -0,0 +1,21 @@
+/*
+ * 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.
+ */
+ 
+/** 
+ * Deprecated stuff!
+ */
+package org.apache.lucene.legacy;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/105c7eae/lucene/backward-codecs/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java b/lucene/backward-codecs/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java
index 8226022..03480d7 100644
--- a/lucene/backward-codecs/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java
+++ b/lucene/backward-codecs/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java
@@ -47,8 +47,6 @@ import org.apache.lucene.document.FieldType;
 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;
@@ -57,9 +55,12 @@ import org.apache.lucene.document.SortedSetDocValuesField;
 import org.apache.lucene.document.StringField;
 import org.apache.lucene.document.TextField;
 import org.apache.lucene.index.IndexWriterConfig.OpenMode;
+import org.apache.lucene.legacy.LegacyIntField;
+import org.apache.lucene.legacy.LegacyLongField;
+import org.apache.lucene.legacy.LegacyNumericRangeQuery;
+import org.apache.lucene.legacy.LegacyNumericUtils;
 import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.search.IndexSearcher;
-import org.apache.lucene.search.LegacyNumericRangeQuery;
 import org.apache.lucene.search.ScoreDoc;
 import org.apache.lucene.search.TermQuery;
 import org.apache.lucene.store.BaseDirectoryWrapper;
@@ -72,7 +73,6 @@ import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.InfoStream;
-import org.apache.lucene.util.LegacyNumericUtils;
 import org.apache.lucene.util.LineFileDocs;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.TestUtil;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/105c7eae/lucene/backward-codecs/src/test/org/apache/lucene/legacy/TestLegacyField.java
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/legacy/TestLegacyField.java b/lucene/backward-codecs/src/test/org/apache/lucene/legacy/TestLegacyField.java
new file mode 100644
index 0000000..65ff096
--- /dev/null
+++ b/lucene/backward-codecs/src/test/org/apache/lucene/legacy/TestLegacyField.java
@@ -0,0 +1,196 @@
+/*
+ * 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.legacy;
+
+import java.io.StringReader;
+
+import org.apache.lucene.analysis.CannedTokenStream;
+import org.apache.lucene.analysis.Token;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.LuceneTestCase;
+
+public class TestLegacyField extends LuceneTestCase {
+  
+  public void testLegacyDoubleField() throws Exception {
+    Field fields[] = new Field[] {
+        new LegacyDoubleField("foo", 5d, Field.Store.NO),
+        new LegacyDoubleField("foo", 5d, Field.Store.YES)
+    };
+
+    for (Field field : fields) {
+      trySetBoost(field);
+      trySetByteValue(field);
+      trySetBytesValue(field);
+      trySetBytesRefValue(field);
+      field.setDoubleValue(6d); // ok
+      trySetIntValue(field);
+      trySetFloatValue(field);
+      trySetLongValue(field);
+      trySetReaderValue(field);
+      trySetShortValue(field);
+      trySetStringValue(field);
+      trySetTokenStreamValue(field);
+    
+      assertEquals(6d, field.numericValue().doubleValue(), 0.0d);
+    }
+  }
+  
+  public void testLegacyFloatField() throws Exception {
+    Field fields[] = new Field[] {
+        new LegacyFloatField("foo", 5f, Field.Store.NO),
+        new LegacyFloatField("foo", 5f, Field.Store.YES)
+    };
+
+    for (Field field : fields) {
+      trySetBoost(field);
+      trySetByteValue(field);
+      trySetBytesValue(field);
+      trySetBytesRefValue(field);
+      trySetDoubleValue(field);
+      trySetIntValue(field);
+      field.setFloatValue(6f); // ok
+      trySetLongValue(field);
+      trySetReaderValue(field);
+      trySetShortValue(field);
+      trySetStringValue(field);
+      trySetTokenStreamValue(field);
+      
+      assertEquals(6f, field.numericValue().floatValue(), 0.0f);
+    }
+  }
+  
+  public void testLegacyIntField() throws Exception {
+    Field fields[] = new Field[] {
+        new LegacyIntField("foo", 5, Field.Store.NO),
+        new LegacyIntField("foo", 5, Field.Store.YES)
+    };
+
+    for (Field field : fields) {
+      trySetBoost(field);
+      trySetByteValue(field);
+      trySetBytesValue(field);
+      trySetBytesRefValue(field);
+      trySetDoubleValue(field);
+      field.setIntValue(6); // ok
+      trySetFloatValue(field);
+      trySetLongValue(field);
+      trySetReaderValue(field);
+      trySetShortValue(field);
+      trySetStringValue(field);
+      trySetTokenStreamValue(field);
+      
+      assertEquals(6, field.numericValue().intValue());
+    }
+  }
+  
+  public void testLegacyLongField() throws Exception {
+    Field fields[] = new Field[] {
+        new LegacyLongField("foo", 5L, Field.Store.NO),
+        new LegacyLongField("foo", 5L, Field.Store.YES)
+    };
+
+    for (Field field : fields) {
+      trySetBoost(field);
+      trySetByteValue(field);
+      trySetBytesValue(field);
+      trySetBytesRefValue(field);
+      trySetDoubleValue(field);
+      trySetIntValue(field);
+      trySetFloatValue(field);
+      field.setLongValue(6); // ok
+      trySetReaderValue(field);
+      trySetShortValue(field);
+      trySetStringValue(field);
+      trySetTokenStreamValue(field);
+      
+      assertEquals(6L, field.numericValue().longValue());
+    }
+  }
+  
+  private void trySetByteValue(Field f) {
+    expectThrows(IllegalArgumentException.class, () -> {
+      f.setByteValue((byte) 10);
+    });
+  }
+
+  private void trySetBytesValue(Field f) {
+    expectThrows(IllegalArgumentException.class, () -> {
+      f.setBytesValue(new byte[] { 5, 5 });
+    });
+  }
+  
+  private void trySetBytesRefValue(Field f) {
+    expectThrows(IllegalArgumentException.class, () -> {
+      f.setBytesValue(new BytesRef("bogus"));
+    });
+  }
+  
+  private void trySetDoubleValue(Field f) {
+    expectThrows(IllegalArgumentException.class, () -> {
+      f.setDoubleValue(Double.MAX_VALUE);
+    });
+  }
+  
+  private void trySetIntValue(Field f) {
+    expectThrows(IllegalArgumentException.class, () -> {
+      f.setIntValue(Integer.MAX_VALUE);
+    });
+  }
+  
+  private void trySetLongValue(Field f) {
+    expectThrows(IllegalArgumentException.class, () -> {
+      f.setLongValue(Long.MAX_VALUE);
+    });
+  }
+  
+  private void trySetFloatValue(Field f) {
+    expectThrows(IllegalArgumentException.class, () -> {
+      f.setFloatValue(Float.MAX_VALUE);
+    });
+  }
+  
+  private void trySetReaderValue(Field f) {
+    expectThrows(IllegalArgumentException.class, () -> {
+      f.setReaderValue(new StringReader("BOO!"));
+    });
+  }
+  
+  private void trySetShortValue(Field f) {
+    expectThrows(IllegalArgumentException.class, () -> {
+      f.setShortValue(Short.MAX_VALUE);
+    });
+  }
+  
+  private void trySetStringValue(Field f) {
+    expectThrows(IllegalArgumentException.class, () -> {
+      f.setStringValue("BOO!");
+    });
+  }
+  
+  private void trySetTokenStreamValue(Field f) {
+    expectThrows(IllegalArgumentException.class, () -> {
+      f.setTokenStream(new CannedTokenStream(new Token("foo", 0, 3)));
+    });
+  }
+  
+  private void trySetBoost(Field f) {
+    expectThrows(IllegalArgumentException.class, () -> {
+      f.setBoost(5.0f);
+    });
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/105c7eae/lucene/backward-codecs/src/test/org/apache/lucene/legacy/TestLegacyFieldReuse.java
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/legacy/TestLegacyFieldReuse.java b/lucene/backward-codecs/src/test/org/apache/lucene/legacy/TestLegacyFieldReuse.java
new file mode 100644
index 0000000..9335290
--- /dev/null
+++ b/lucene/backward-codecs/src/test/org/apache/lucene/legacy/TestLegacyFieldReuse.java
@@ -0,0 +1,81 @@
+/*
+ * 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.legacy;
+
+
+import java.io.IOException;
+
+import org.apache.lucene.analysis.BaseTokenStreamTestCase;
+import org.apache.lucene.analysis.CannedTokenStream;
+import org.apache.lucene.analysis.Token;
+import org.apache.lucene.analysis.TokenStream;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.legacy.LegacyIntField;
+import org.apache.lucene.legacy.LegacyNumericTokenStream;
+import org.apache.lucene.legacy.LegacyNumericUtils;
+import org.apache.lucene.legacy.LegacyNumericTokenStream.LegacyNumericTermAttribute;
+
+/** test tokenstream reuse by DefaultIndexingChain */
+public class TestLegacyFieldReuse extends BaseTokenStreamTestCase {
+  
+  public void testNumericReuse() throws IOException {
+    LegacyIntField legacyIntField = new LegacyIntField("foo", 5, Field.Store.NO);
+    
+    // passing null
+    TokenStream ts = legacyIntField.tokenStream(null, null);
+    assertTrue(ts instanceof LegacyNumericTokenStream);
+    assertEquals(LegacyNumericUtils.PRECISION_STEP_DEFAULT_32, ((LegacyNumericTokenStream)ts).getPrecisionStep());
+    assertNumericContents(5, ts);
+
+    // now reuse previous stream
+    legacyIntField = new LegacyIntField("foo", 20, Field.Store.NO);
+    TokenStream ts2 = legacyIntField.tokenStream(null, ts);
+    assertSame(ts, ts2);
+    assertNumericContents(20, ts);
+    
+    // pass a bogus stream and ensure it's still ok
+    legacyIntField = new LegacyIntField("foo", 2343, Field.Store.NO);
+    TokenStream bogus = new CannedTokenStream(new Token("bogus", 0, 5));
+    ts = legacyIntField.tokenStream(null, bogus);
+    assertNotSame(bogus, ts);
+    assertNumericContents(2343, ts);
+    
+    // pass another bogus stream (numeric, but different precision step!)
+    legacyIntField = new LegacyIntField("foo", 42, Field.Store.NO);
+    assert 3 != LegacyNumericUtils.PRECISION_STEP_DEFAULT;
+    bogus = new LegacyNumericTokenStream(3);
+    ts = legacyIntField.tokenStream(null, bogus);
+    assertNotSame(bogus, ts);
+    assertNumericContents(42, ts);
+  }
+   
+  private void assertNumericContents(int value, TokenStream ts) throws IOException {
+    assertTrue(ts instanceof LegacyNumericTokenStream);
+    LegacyNumericTermAttribute numericAtt = ts.getAttribute(LegacyNumericTermAttribute.class);
+    ts.reset();
+    boolean seen = false;
+    while (ts.incrementToken()) {
+      if (numericAtt.getShift() == 0) {
+        assertEquals(value, numericAtt.getRawValue());
+        seen = true;
+      }
+    }
+    ts.end();
+    ts.close();
+    assertTrue(seen);
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/105c7eae/lucene/backward-codecs/src/test/org/apache/lucene/legacy/TestLegacyNumericUtils.java
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/legacy/TestLegacyNumericUtils.java b/lucene/backward-codecs/src/test/org/apache/lucene/legacy/TestLegacyNumericUtils.java
new file mode 100644
index 0000000..8607efd
--- /dev/null
+++ b/lucene/backward-codecs/src/test/org/apache/lucene/legacy/TestLegacyNumericUtils.java
@@ -0,0 +1,571 @@
+/*
+ * 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.legacy;
+
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.Random;
+
+import org.apache.lucene.legacy.LegacyNumericUtils;
+import org.apache.lucene.util.BytesRefBuilder;
+import org.apache.lucene.util.FixedBitSet;
+import org.apache.lucene.util.LongBitSet;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.NumericUtils;
+
+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/backward-codecs/src/test/org/apache/lucene/legacy/TestLegacyTerms.java
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/legacy/TestLegacyTerms.java b/lucene/backward-codecs/src/test/org/apache/lucene/legacy/TestLegacyTerms.java
new file mode 100644
index 0000000..27fae15
--- /dev/null
+++ b/lucene/backward-codecs/src/test/org/apache/lucene/legacy/TestLegacyTerms.java
@@ -0,0 +1,164 @@
+/*
+ * 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.legacy;
+
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.MultiFields;
+import org.apache.lucene.index.RandomIndexWriter;
+import org.apache.lucene.index.Terms;
+import org.apache.lucene.index.TermsEnum;
+import org.apache.lucene.legacy.LegacyDoubleField;
+import org.apache.lucene.legacy.LegacyFloatField;
+import org.apache.lucene.legacy.LegacyIntField;
+import org.apache.lucene.legacy.LegacyLongField;
+import org.apache.lucene.legacy.LegacyNumericUtils;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.NumericUtils;
+
+public class TestLegacyTerms extends LuceneTestCase {
+
+  public void testEmptyIntFieldMinMax() throws Exception {
+    assertNull(LegacyNumericUtils.getMinInt(EMPTY_TERMS));
+    assertNull(LegacyNumericUtils.getMaxInt(EMPTY_TERMS));
+  }
+  
+  public void testIntFieldMinMax() throws Exception {
+    Directory dir = newDirectory();
+    RandomIndexWriter w = new RandomIndexWriter(random(), dir);
+    int numDocs = atLeast(100);
+    int minValue = Integer.MAX_VALUE;
+    int maxValue = Integer.MIN_VALUE;
+    for(int i=0;i<numDocs;i++ ){
+      Document doc = new Document();
+      int num = random().nextInt();
+      minValue = Math.min(num, minValue);
+      maxValue = Math.max(num, maxValue);
+      doc.add(new LegacyIntField("field", num, Field.Store.NO));
+      w.addDocument(doc);
+    }
+    
+    IndexReader r = w.getReader();
+    Terms terms = MultiFields.getTerms(r, "field");
+    assertEquals(new Integer(minValue), LegacyNumericUtils.getMinInt(terms));
+    assertEquals(new Integer(maxValue), LegacyNumericUtils.getMaxInt(terms));
+
+    r.close();
+    w.close();
+    dir.close();
+  }
+
+  public void testEmptyLongFieldMinMax() throws Exception {
+    assertNull(LegacyNumericUtils.getMinLong(EMPTY_TERMS));
+    assertNull(LegacyNumericUtils.getMaxLong(EMPTY_TERMS));
+  }
+  
+  public void testLongFieldMinMax() throws Exception {
+    Directory dir = newDirectory();
+    RandomIndexWriter w = new RandomIndexWriter(random(), dir);
+    int numDocs = atLeast(100);
+    long minValue = Long.MAX_VALUE;
+    long maxValue = Long.MIN_VALUE;
+    for(int i=0;i<numDocs;i++ ){
+      Document doc = new Document();
+      long num = random().nextLong();
+      minValue = Math.min(num, minValue);
+      maxValue = Math.max(num, maxValue);
+      doc.add(new LegacyLongField("field", num, Field.Store.NO));
+      w.addDocument(doc);
+    }
+    
+    IndexReader r = w.getReader();
+
+    Terms terms = MultiFields.getTerms(r, "field");
+    assertEquals(new Long(minValue), LegacyNumericUtils.getMinLong(terms));
+    assertEquals(new Long(maxValue), LegacyNumericUtils.getMaxLong(terms));
+
+    r.close();
+    w.close();
+    dir.close();
+  }
+
+  public void testFloatFieldMinMax() throws Exception {
+    Directory dir = newDirectory();
+    RandomIndexWriter w = new RandomIndexWriter(random(), dir);
+    int numDocs = atLeast(100);
+    float minValue = Float.POSITIVE_INFINITY;
+    float maxValue = Float.NEGATIVE_INFINITY;
+    for(int i=0;i<numDocs;i++ ){
+      Document doc = new Document();
+      float num = random().nextFloat();
+      minValue = Math.min(num, minValue);
+      maxValue = Math.max(num, maxValue);
+      doc.add(new LegacyFloatField("field", num, Field.Store.NO));
+      w.addDocument(doc);
+    }
+    
+    IndexReader r = w.getReader();
+    Terms terms = MultiFields.getTerms(r, "field");
+    assertEquals(minValue, NumericUtils.sortableIntToFloat(LegacyNumericUtils.getMinInt(terms)), 0.0f);
+    assertEquals(maxValue, NumericUtils.sortableIntToFloat(LegacyNumericUtils.getMaxInt(terms)), 0.0f);
+
+    r.close();
+    w.close();
+    dir.close();
+  }
+
+  public void testDoubleFieldMinMax() throws Exception {
+    Directory dir = newDirectory();
+    RandomIndexWriter w = new RandomIndexWriter(random(), dir);
+    int numDocs = atLeast(100);
+    double minValue = Double.POSITIVE_INFINITY;
+    double maxValue = Double.NEGATIVE_INFINITY;
+    for(int i=0;i<numDocs;i++ ){
+      Document doc = new Document();
+      double num = random().nextDouble();
+      minValue = Math.min(num, minValue);
+      maxValue = Math.max(num, maxValue);
+      doc.add(new LegacyDoubleField("field", num, Field.Store.NO));
+      w.addDocument(doc);
+    }
+    
+    IndexReader r = w.getReader();
+
+    Terms terms = MultiFields.getTerms(r, "field");
+
+    assertEquals(minValue, NumericUtils.sortableLongToDouble(LegacyNumericUtils.getMinLong(terms)), 0.0);
+    assertEquals(maxValue, NumericUtils.sortableLongToDouble(LegacyNumericUtils.getMaxLong(terms)), 0.0);
+
+    r.close();
+    w.close();
+    dir.close();
+  }
+
+  /**
+   * A complete empty Terms instance that has no terms in it and supports no optional statistics
+   */
+  private static Terms EMPTY_TERMS = new Terms() {
+    public TermsEnum iterator() { return TermsEnum.EMPTY; }
+    public long size() { return -1; }
+    public long getSumTotalTermFreq() { return -1; }
+    public long getSumDocFreq() { return -1; }
+    public int getDocCount() { return -1; }
+    public boolean hasFreqs() { return false; }
+    public boolean hasOffsets() { return false; }
+    public boolean hasPositions() { return false; }
+    public boolean hasPayloads() { return false; }
+  };
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/105c7eae/lucene/backward-codecs/src/test/org/apache/lucene/legacy/TestMultiValuedNumericRangeQuery.java
----------------------------------------------------------------------
diff --git a/lucene/backward-codecs/src/test/org/apache/lucene/legacy/TestMultiValuedNumericRangeQuery.java b/lucene/backward-codecs/src/test/org/apache/lucene/legacy/TestMultiValuedNumericRangeQuery.java
new file mode 100644
index 0000000..386ec17
--- /dev/null
+++ b/lucene/backward-codecs/src/test/org/apache/lucene/legacy/TestMultiValuedNumericRangeQuery.java
@@ -0,0 +1,84 @@
+/*
+ * 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.legacy;
+
+
+import java.util.Locale;
+import java.text.DecimalFormat;
+import java.text.DecimalFormatSymbols;
+
+import org.apache.lucene.analysis.MockAnalyzer;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.RandomIndexWriter;
+import org.apache.lucene.legacy.LegacyIntField;
+import org.apache.lucene.legacy.LegacyNumericRangeQuery;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.TermRangeQuery;
+import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.TestUtil;
+
+public class TestMultiValuedNumericRangeQuery extends LuceneTestCase {
+
+  /** Tests LegacyNumericRangeQuery on a multi-valued field (multiple numeric values per document).
+   * This test ensures, that a classical TermRangeQuery returns exactly the same document numbers as
+   * LegacyNumericRangeQuery (see SOLR-1322 for discussion) and the multiple precision terms per numeric value
+   * do not interfere with multiple numeric values.
+   */
+  public void testMultiValuedNRQ() throws Exception {
+    Directory directory = newDirectory();
+    RandomIndexWriter writer = new RandomIndexWriter(random(), directory,
+        newIndexWriterConfig(new MockAnalyzer(random()))
+        .setMaxBufferedDocs(TestUtil.nextInt(random(), 50, 1000)));
+    
+    DecimalFormat format = new DecimalFormat("00000000000", new DecimalFormatSymbols(Locale.ROOT));
+    
+    int num = atLeast(500);
+    for (int l = 0; l < num; l++) {
+      Document doc = new Document();
+      for (int m=0, c=random().nextInt(10); m<=c; m++) {
+        int value = random().nextInt(Integer.MAX_VALUE);
+        doc.add(newStringField("asc", format.format(value), Field.Store.NO));
+        doc.add(new LegacyIntField("trie", value, Field.Store.NO));
+      }
+      writer.addDocument(doc);
+    }
+    IndexReader reader = writer.getReader();
+    writer.close();
+    
+    IndexSearcher searcher=newSearcher(reader);
+    num = atLeast(50);
+    for (int i = 0; i < num; i++) {
+      int lower=random().nextInt(Integer.MAX_VALUE);
+      int upper=random().nextInt(Integer.MAX_VALUE);
+      if (lower>upper) {
+        int a=lower; lower=upper; upper=a;
+      }
+      TermRangeQuery cq=TermRangeQuery.newStringRange("asc", format.format(lower), format.format(upper), true, true);
+      LegacyNumericRangeQuery<Integer> tq= LegacyNumericRangeQuery.newIntRange("trie", lower, upper, true, true);
+      TopDocs trTopDocs = searcher.search(cq, 1);
+      TopDocs nrTopDocs = searcher.search(tq, 1);
+      assertEquals("Returned count for LegacyNumericRangeQuery and TermRangeQuery must be equal", trTopDocs.totalHits, nrTopDocs.totalHits );
+    }
+    reader.close();
+    directory.close();
+  }
+  
+}