You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by cp...@apache.org on 2017/05/30 11:28:30 UTC

[21/25] lucene-solr:jira/solr-8668: LUCENE-7850: Move support for legacy numerics to solr/.

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/759fa42b/solr/core/src/java/org/apache/solr/legacy/BBoxStrategy.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/legacy/BBoxStrategy.java b/solr/core/src/java/org/apache/solr/legacy/BBoxStrategy.java
new file mode 100644
index 0000000..c919eb8
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/legacy/BBoxStrategy.java
@@ -0,0 +1,706 @@
+/*
+ * 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.solr.legacy;
+
+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.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.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.Query;
+import org.apache.lucene.search.TermQuery;
+import org.apache.lucene.spatial.SpatialStrategy;
+import org.apache.lucene.spatial.bbox.BBoxOverlapRatioValueSource;
+import org.apache.lucene.spatial.query.SpatialArgs;
+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.NumericUtils;
+import org.locationtech.spatial4j.context.SpatialContext;
+import org.locationtech.spatial4j.shape.Point;
+import org.locationtech.spatial4j.shape.Rectangle;
+import org.locationtech.spatial4j.shape.Shape;
+
+
+/**
+ * A SpatialStrategy for indexing and searching Rectangles by storing its
+ * coordinates in numeric fields. It supports all {@link SpatialOperation}s and
+ * has a custom overlap relevancy. It is based on GeoPortal's <a
+ * href="http://geoportal.svn.sourceforge.net/svnroot/geoportal/Geoportal/trunk/src/com/esri/gpt/catalog/lucene/SpatialClauseAdapter.java">SpatialClauseAdapter</a>.
+ * <p>
+ * <b>Characteristics:</b>
+ * <br>
+ * <ul>
+ * <li>Only indexes Rectangles; just one per field value. Other shapes can be provided
+ * and the bounding box will be used.</li>
+ * <li>Can query only by a Rectangle. Providing other shapes is an error.</li>
+ * <li>Supports most {@link SpatialOperation}s but not Overlaps.</li>
+ * <li>Uses the DocValues API for any sorting / relevancy.</li>
+ * </ul>
+ * <p>
+ * <b>Implementation:</b>
+ * <p>
+ * This uses 4 double fields for minX, maxX, minY, maxY
+ * and a boolean to mark a dateline cross. Depending on the particular {@link
+ * SpatialOperation}s, there are a variety of range queries on {@link DoublePoint}s to be
+ * done.
+ * The {@link #makeOverlapRatioValueSource(org.locationtech.spatial4j.shape.Rectangle, double)}
+ * works by calculating the query bbox overlap percentage against the indexed
+ * shape overlap percentage. The indexed shape's coordinates are retrieved from
+ * {@link org.apache.lucene.index.LeafReader#getNumericDocValues}.
+ *
+ * @lucene.experimental
+ */
+public class BBoxStrategy extends SpatialStrategy {
+
+  // note: we use a FieldType to articulate the options we want on the field.  We don't use it as-is with a Field, we
+  //  create more than one Field.
+
+  /**
+   * pointValues, docValues, and nothing else.
+   */
+  public static FieldType DEFAULT_FIELDTYPE;
+
+  @Deprecated
+  public static LegacyFieldType LEGACY_FIELDTYPE;
+  static {
+    // Default: pointValues + docValues
+    FieldType type = new FieldType();
+    type.setDimensions(1, Double.BYTES);//pointValues (assume Double)
+    type.setDocValuesType(DocValuesType.NUMERIC);//docValues
+    type.setStored(false);
+    type.freeze();
+    DEFAULT_FIELDTYPE = type;
+    // Legacy default: legacyNumerics + docValues
+    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";
+  public static final String SUFFIX_MAXX = "__maxX";
+  public static final String SUFFIX_MINY = "__minY";
+  public static final String SUFFIX_MAXY = "__maxY";
+  public static final String SUFFIX_XDL  = "__xdl";
+
+  /*
+   * The Bounding Box gets stored as four fields for x/y min/max and a flag
+   * that says if the box crosses the dateline (xdl).
+   */
+  final String field_bbox;
+  final String field_minX;
+  final String field_minY;
+  final String field_maxX;
+  final String field_maxY;
+  final String field_xdl; // crosses dateline
+
+  private final FieldType optionsFieldType;//from constructor; aggregate field type used to express all options
+  private final int fieldsLen;
+  private final boolean hasStored;
+  private final boolean hasDocVals;
+  private final boolean hasPointVals;
+  // equiv to "hasLegacyNumerics":
+  private final LegacyFieldType legacyNumericFieldType; // not stored; holds precision step.
+  private final FieldType xdlFieldType;
+
+  /**
+   * Creates a new {@link BBoxStrategy} instance that uses {@link DoublePoint} and {@link DoublePoint#newRangeQuery}
+   */
+  public static BBoxStrategy newInstance(SpatialContext ctx, String fieldNamePrefix) {
+    return new BBoxStrategy(ctx, fieldNamePrefix, DEFAULT_FIELDTYPE);
+  }
+
+  /**
+   * Creates a new {@link BBoxStrategy} instance that uses {@link LegacyDoubleField} for backwards compatibility
+   * @deprecated LegacyNumerics will be removed
+   */
+  @Deprecated
+  public static BBoxStrategy newLegacyInstance(SpatialContext ctx, String fieldNamePrefix) {
+    return new BBoxStrategy(ctx, fieldNamePrefix, LEGACY_FIELDTYPE);
+  }
+
+  /**
+   * Creates this strategy.
+   * {@code fieldType} is used to customize the indexing options of the 4 number fields, and to a lesser degree the XDL
+   * field too. Search requires pointValues (or legacy numerics), and relevancy requires docValues. If these features
+   * aren't needed then disable them.
+   */
+  public BBoxStrategy(SpatialContext ctx, String fieldNamePrefix, FieldType fieldType) {
+    super(ctx, fieldNamePrefix);
+    field_bbox = fieldNamePrefix;
+    field_minX = fieldNamePrefix + SUFFIX_MINX;
+    field_maxX = fieldNamePrefix + SUFFIX_MAXX;
+    field_minY = fieldNamePrefix + SUFFIX_MINY;
+    field_maxY = fieldNamePrefix + SUFFIX_MAXY;
+    field_xdl = fieldNamePrefix + SUFFIX_XDL;
+
+    fieldType.freeze();
+    this.optionsFieldType = fieldType;
+
+    int numQuads = 0;
+    if ((this.hasStored = fieldType.stored())) {
+      numQuads++;
+    }
+    if ((this.hasDocVals = fieldType.docValuesType() != DocValuesType.NONE)) {
+      numQuads++;
+    }
+    if ((this.hasPointVals = fieldType.pointDimensionCount() > 0)) {
+      numQuads++;
+    }
+    if (fieldType.indexOptions() != IndexOptions.NONE && fieldType instanceof LegacyFieldType && ((LegacyFieldType)fieldType).numericType() != null) {
+      if (hasPointVals) {
+        throw new IllegalArgumentException("pointValues and LegacyNumericType are mutually exclusive");
+      }
+      final LegacyFieldType legacyType = (LegacyFieldType) fieldType;
+      if (legacyType.numericType() != LegacyNumericType.DOUBLE) {
+        throw new IllegalArgumentException(getClass() + " does not support " + legacyType.numericType());
+      }
+      numQuads++;
+      legacyNumericFieldType = new LegacyFieldType(LegacyDoubleField.TYPE_NOT_STORED);
+      legacyNumericFieldType.setNumericPrecisionStep(legacyType.numericPrecisionStep());
+      legacyNumericFieldType.freeze();
+    } else {
+      legacyNumericFieldType = null;
+    }
+
+    if (hasPointVals || legacyNumericFieldType != null) { // if we have an index...
+      xdlFieldType = new FieldType(StringField.TYPE_NOT_STORED);
+      xdlFieldType.setIndexOptions(IndexOptions.DOCS);
+      xdlFieldType.freeze();
+    } else {
+      xdlFieldType = null;
+    }
+
+    this.fieldsLen = numQuads * 4 + (xdlFieldType != null ? 1 : 0);
+  }
+
+  /** Returns a field type representing the set of field options. This is identical to what was passed into the
+   * constructor.  It's frozen. */
+  public FieldType getFieldType() {
+    return optionsFieldType;
+  }
+
+  //---------------------------------
+  // Indexing
+  //---------------------------------
+
+  @Override
+  public Field[] createIndexableFields(Shape shape) {
+    return createIndexableFields(shape.getBoundingBox());
+  }
+
+  private Field[] createIndexableFields(Rectangle bbox) {
+    Field[] fields = new Field[fieldsLen];
+    int idx = -1;
+    if (hasStored) {
+      fields[++idx] = new StoredField(field_minX, bbox.getMinX());
+      fields[++idx] = new StoredField(field_minY, bbox.getMinY());
+      fields[++idx] = new StoredField(field_maxX, bbox.getMaxX());
+      fields[++idx] = new StoredField(field_maxY, bbox.getMaxY());
+    }
+    if (hasDocVals) {
+      fields[++idx] = new DoubleDocValuesField(field_minX, bbox.getMinX());
+      fields[++idx] = new DoubleDocValuesField(field_minY, bbox.getMinY());
+      fields[++idx] = new DoubleDocValuesField(field_maxX, bbox.getMaxX());
+      fields[++idx] = new DoubleDocValuesField(field_maxY, bbox.getMaxY());
+    }
+    if (hasPointVals) {
+      fields[++idx] = new DoublePoint(field_minX, bbox.getMinX());
+      fields[++idx] = new DoublePoint(field_minY, bbox.getMinY());
+      fields[++idx] = new DoublePoint(field_maxX, bbox.getMaxX());
+      fields[++idx] = new DoublePoint(field_maxY, bbox.getMaxY());
+    }
+    if (legacyNumericFieldType != null) {
+      fields[++idx] = new LegacyDoubleField(field_minX, bbox.getMinX(), legacyNumericFieldType);
+      fields[++idx] = new LegacyDoubleField(field_minY, bbox.getMinY(), legacyNumericFieldType);
+      fields[++idx] = new LegacyDoubleField(field_maxX, bbox.getMaxX(), legacyNumericFieldType);
+      fields[++idx] = new LegacyDoubleField(field_maxY, bbox.getMaxY(), legacyNumericFieldType);
+    }
+    if (xdlFieldType != null) {
+      fields[++idx] = new Field(field_xdl, bbox.getCrossesDateLine()?"T":"F", xdlFieldType);
+    }
+    assert idx == fields.length - 1;
+    return fields;
+  }
+
+
+  //---------------------------------
+  // Value Source / Relevancy
+  //---------------------------------
+
+  /**
+   * Provides access to each rectangle per document as a ValueSource in which
+   * {@link org.apache.lucene.queries.function.FunctionValues#objectVal(int)} returns a {@link
+   * Shape}.
+   */ //TODO raise to SpatialStrategy
+  public ValueSource makeShapeValueSource() {
+    return new BBoxValueSource(this);
+  }
+
+  @Override
+  public ValueSource makeDistanceValueSource(Point queryPoint, double multiplier) {
+    //TODO if makeShapeValueSource gets lifted to the top; this could become a generic impl.
+    return new DistanceToShapeValueSource(makeShapeValueSource(), queryPoint, multiplier, ctx);
+  }
+
+  /** Returns a similarity based on {@link BBoxOverlapRatioValueSource}. This is just a
+   * convenience method. */
+  public ValueSource makeOverlapRatioValueSource(Rectangle queryBox, double queryTargetProportion) {
+    return new BBoxOverlapRatioValueSource(
+        makeShapeValueSource(), ctx.isGeo(), queryBox, queryTargetProportion, 0.0);
+  }
+
+  //---------------------------------
+  // Query Building
+  //---------------------------------
+
+  //  Utility on SpatialStrategy?
+//  public Query makeQueryWithValueSource(SpatialArgs args, ValueSource valueSource) {
+//    return new CustomScoreQuery(makeQuery(args), new FunctionQuery(valueSource));
+  //or...
+//  return new BooleanQuery.Builder()
+//      .add(new FunctionQuery(valueSource), BooleanClause.Occur.MUST)//matches everything and provides score
+//      .add(filterQuery, BooleanClause.Occur.FILTER)//filters (score isn't used)
+//  .build();
+//  }
+
+  @Override
+  public Query makeQuery(SpatialArgs args) {
+    Shape shape = args.getShape();
+    if (!(shape instanceof Rectangle))
+      throw new UnsupportedOperationException("Can only query by Rectangle, not " + shape);
+
+    Rectangle bbox = (Rectangle) shape;
+    Query spatial;
+
+    // Useful for understanding Relations:
+    // http://edndoc.esri.com/arcsde/9.1/general_topics/understand_spatial_relations.htm
+    SpatialOperation op = args.getOperation();
+         if( op == SpatialOperation.BBoxIntersects ) spatial = makeIntersects(bbox);
+    else if( op == SpatialOperation.BBoxWithin     ) spatial = makeWithin(bbox);
+    else if( op == SpatialOperation.Contains       ) spatial = makeContains(bbox);
+    else if( op == SpatialOperation.Intersects     ) spatial = makeIntersects(bbox);
+    else if( op == SpatialOperation.IsEqualTo      ) spatial = makeEquals(bbox);
+    else if( op == SpatialOperation.IsDisjointTo   ) spatial = makeDisjoint(bbox);
+    else if( op == SpatialOperation.IsWithin       ) spatial = makeWithin(bbox);
+    else { //no Overlaps support yet
+        throw new UnsupportedSpatialOperation(op);
+    }
+    return new ConstantScoreQuery(spatial);
+  }
+
+  /**
+   * Constructs a query to retrieve documents that fully contain the input envelope.
+   *
+   * @return the spatial query
+   */
+  Query makeContains(Rectangle bbox) {
+
+    // general case
+    // docMinX <= queryExtent.getMinX() AND docMinY <= queryExtent.getMinY() AND docMaxX >= queryExtent.getMaxX() AND docMaxY >= queryExtent.getMaxY()
+
+    // Y conditions
+    // docMinY <= queryExtent.getMinY() AND docMaxY >= queryExtent.getMaxY()
+    Query qMinY = this.makeNumericRangeQuery(field_minY, null, bbox.getMinY(), false, true);
+    Query qMaxY = this.makeNumericRangeQuery(field_maxY, bbox.getMaxY(), null, true, false);
+    Query yConditions = this.makeQuery(BooleanClause.Occur.MUST, qMinY, qMaxY);
+
+    // X conditions
+    Query xConditions;
+
+    // queries that do not cross the date line
+    if (!bbox.getCrossesDateLine()) {
+
+      // X Conditions for documents that do not cross the date line,
+      // documents that contain the min X and max X of the query envelope,
+      // docMinX <= queryExtent.getMinX() AND docMaxX >= queryExtent.getMaxX()
+      Query qMinX = this.makeNumericRangeQuery(field_minX, null, bbox.getMinX(), false, true);
+      Query qMaxX = this.makeNumericRangeQuery(field_maxX, bbox.getMaxX(), null, true, false);
+      Query qMinMax = this.makeQuery(BooleanClause.Occur.MUST, qMinX, qMaxX);
+      Query qNonXDL = this.makeXDL(false, qMinMax);
+
+      if (!ctx.isGeo()) {
+        xConditions = qNonXDL;
+      } else {
+        // X Conditions for documents that cross the date line,
+        // the left portion of the document contains the min X of the query
+        // OR the right portion of the document contains the max X of the query,
+        // docMinXLeft <= queryExtent.getMinX() OR docMaxXRight >= queryExtent.getMaxX()
+        Query qXDLLeft = this.makeNumericRangeQuery(field_minX, null, bbox.getMinX(), false, true);
+        Query qXDLRight = this.makeNumericRangeQuery(field_maxX, bbox.getMaxX(), null, true, false);
+        Query qXDLLeftRight = this.makeQuery(BooleanClause.Occur.SHOULD, qXDLLeft, qXDLRight);
+        Query qXDL = this.makeXDL(true, qXDLLeftRight);
+
+        Query qEdgeDL = null;
+        if (bbox.getMinX() == bbox.getMaxX() && Math.abs(bbox.getMinX()) == 180) {
+          double edge = bbox.getMinX() * -1;//opposite dateline edge
+          qEdgeDL = makeQuery(BooleanClause.Occur.SHOULD,
+              makeNumberTermQuery(field_minX, edge), makeNumberTermQuery(field_maxX, edge));
+        }
+
+        // apply the non-XDL and XDL conditions
+        xConditions = this.makeQuery(BooleanClause.Occur.SHOULD, qNonXDL, qXDL, qEdgeDL);
+      }
+    } else {
+      // queries that cross the date line
+
+      // No need to search for documents that do not cross the date line
+
+      // X Conditions for documents that cross the date line,
+      // the left portion of the document contains the min X of the query
+      // AND the right portion of the document contains the max X of the query,
+      // docMinXLeft <= queryExtent.getMinX() AND docMaxXRight >= queryExtent.getMaxX()
+      Query qXDLLeft = this.makeNumericRangeQuery(field_minX, null, bbox.getMinX(), false, true);
+      Query qXDLRight = this.makeNumericRangeQuery(field_maxX, bbox.getMaxX(), null, true, false);
+      Query qXDLLeftRight = this.makeXDL(true, this.makeQuery(BooleanClause.Occur.MUST, qXDLLeft, qXDLRight));
+
+      Query qWorld = makeQuery(BooleanClause.Occur.MUST,
+          makeNumberTermQuery(field_minX, -180), makeNumberTermQuery(field_maxX, 180));
+
+      xConditions = makeQuery(BooleanClause.Occur.SHOULD, qXDLLeftRight, qWorld);
+    }
+
+    // both X and Y conditions must occur
+    return this.makeQuery(BooleanClause.Occur.MUST, xConditions, yConditions);
+  }
+
+  /**
+   * Constructs a query to retrieve documents that are disjoint to the input envelope.
+   *
+   * @return the spatial query
+   */
+  Query makeDisjoint(Rectangle bbox) {
+
+    // general case
+    // docMinX > queryExtent.getMaxX() OR docMaxX < queryExtent.getMinX() OR docMinY > queryExtent.getMaxY() OR docMaxY < queryExtent.getMinY()
+
+    // Y conditions
+    // docMinY > queryExtent.getMaxY() OR docMaxY < queryExtent.getMinY()
+    Query qMinY = this.makeNumericRangeQuery(field_minY, bbox.getMaxY(), null, false, false);
+    Query qMaxY = this.makeNumericRangeQuery(field_maxY, null, bbox.getMinY(), false, false);
+    Query yConditions = this.makeQuery(BooleanClause.Occur.SHOULD, qMinY, qMaxY);
+
+    // X conditions
+    Query xConditions;
+
+    // queries that do not cross the date line
+    if (!bbox.getCrossesDateLine()) {
+
+      // X Conditions for documents that do not cross the date line,
+      // docMinX > queryExtent.getMaxX() OR docMaxX < queryExtent.getMinX()
+      Query qMinX = this.makeNumericRangeQuery(field_minX, bbox.getMaxX(), null, false, false);
+      if (bbox.getMinX() == -180.0 && ctx.isGeo()) {//touches dateline; -180 == 180
+        BooleanQuery.Builder bq = new BooleanQuery.Builder();
+        bq.add(qMinX, BooleanClause.Occur.MUST);
+        bq.add(makeNumberTermQuery(field_maxX, 180.0), BooleanClause.Occur.MUST_NOT);
+        qMinX = bq.build();
+      }
+      Query qMaxX = this.makeNumericRangeQuery(field_maxX, null, bbox.getMinX(), false, false);
+
+      if (bbox.getMaxX() == 180.0 && ctx.isGeo()) {//touches dateline; -180 == 180
+        BooleanQuery.Builder bq = new BooleanQuery.Builder();
+        bq.add(qMaxX, BooleanClause.Occur.MUST);
+        bq.add(makeNumberTermQuery(field_minX, -180.0), BooleanClause.Occur.MUST_NOT);
+        qMaxX = bq.build();
+      }
+      Query qMinMax = this.makeQuery(BooleanClause.Occur.SHOULD, qMinX, qMaxX);
+      Query qNonXDL = this.makeXDL(false, qMinMax);
+
+      if (!ctx.isGeo()) {
+        xConditions = qNonXDL;
+      } else {
+        // X Conditions for documents that cross the date line,
+
+        // both the left and right portions of the document must be disjoint to the query
+        // (docMinXLeft > queryExtent.getMaxX() OR docMaxXLeft < queryExtent.getMinX()) AND
+        // (docMinXRight > queryExtent.getMaxX() OR docMaxXRight < queryExtent.getMinX())
+        // where: docMaxXLeft = 180.0, docMinXRight = -180.0
+        // (docMaxXLeft  < queryExtent.getMinX()) equates to (180.0  < queryExtent.getMinX()) and is ignored
+        // (docMinXRight > queryExtent.getMaxX()) equates to (-180.0 > queryExtent.getMaxX()) and is ignored
+        Query qMinXLeft = this.makeNumericRangeQuery(field_minX, bbox.getMaxX(), null, false, false);
+        Query qMaxXRight = this.makeNumericRangeQuery(field_maxX, null, bbox.getMinX(), false, false);
+        Query qLeftRight = this.makeQuery(BooleanClause.Occur.MUST, qMinXLeft, qMaxXRight);
+        Query qXDL = this.makeXDL(true, qLeftRight);
+
+        // apply the non-XDL and XDL conditions
+        xConditions = this.makeQuery(BooleanClause.Occur.SHOULD, qNonXDL, qXDL);
+      }
+      // queries that cross the date line
+    } else {
+
+      // X Conditions for documents that do not cross the date line,
+      // the document must be disjoint to both the left and right query portions
+      // (docMinX > queryExtent.getMaxX()Left OR docMaxX < queryExtent.getMinX()) AND (docMinX > queryExtent.getMaxX() OR docMaxX < queryExtent.getMinX()Left)
+      // where: queryExtent.getMaxX()Left = 180.0, queryExtent.getMinX()Left = -180.0
+      Query qMinXLeft = this.makeNumericRangeQuery(field_minX, 180.0, null, false, false);
+      Query qMaxXLeft = this.makeNumericRangeQuery(field_maxX, null, bbox.getMinX(), false, false);
+      Query qMinXRight = this.makeNumericRangeQuery(field_minX, bbox.getMaxX(), null, false, false);
+      Query qMaxXRight = this.makeNumericRangeQuery(field_maxX, null, -180.0, false, false);
+      Query qLeft = this.makeQuery(BooleanClause.Occur.SHOULD, qMinXLeft, qMaxXLeft);
+      Query qRight = this.makeQuery(BooleanClause.Occur.SHOULD, qMinXRight, qMaxXRight);
+      Query qLeftRight = this.makeQuery(BooleanClause.Occur.MUST, qLeft, qRight);
+
+      // No need to search for documents that do not cross the date line
+
+      xConditions = this.makeXDL(false, qLeftRight);
+    }
+
+    // either X or Y conditions should occur
+    return this.makeQuery(BooleanClause.Occur.SHOULD, xConditions, yConditions);
+  }
+
+  /**
+   * Constructs a query to retrieve documents that equal the input envelope.
+   *
+   * @return the spatial query
+   */
+  Query makeEquals(Rectangle bbox) {
+
+    // docMinX = queryExtent.getMinX() AND docMinY = queryExtent.getMinY() AND docMaxX = queryExtent.getMaxX() AND docMaxY = queryExtent.getMaxY()
+    Query qMinX = makeNumberTermQuery(field_minX, bbox.getMinX());
+    Query qMinY = makeNumberTermQuery(field_minY, bbox.getMinY());
+    Query qMaxX = makeNumberTermQuery(field_maxX, bbox.getMaxX());
+    Query qMaxY = makeNumberTermQuery(field_maxY, bbox.getMaxY());
+    return makeQuery(BooleanClause.Occur.MUST, qMinX, qMinY, qMaxX, qMaxY);
+  }
+
+  /**
+   * Constructs a query to retrieve documents that intersect the input envelope.
+   *
+   * @return the spatial query
+   */
+  Query makeIntersects(Rectangle bbox) {
+
+    // the original intersects query does not work for envelopes that cross the date line,
+    // switch to a NOT Disjoint query
+
+    // MUST_NOT causes a problem when it's the only clause type within a BooleanQuery,
+    // to get around it we add all documents as a SHOULD
+
+    // there must be an envelope, it must not be disjoint
+    Query qHasEnv;
+    if (ctx.isGeo()) {
+      Query qIsNonXDL = this.makeXDL(false);
+      Query qIsXDL = ctx.isGeo() ? this.makeXDL(true) : null;
+      qHasEnv = this.makeQuery(BooleanClause.Occur.SHOULD, qIsNonXDL, qIsXDL);
+    } else {
+      qHasEnv = this.makeXDL(false);
+    }
+
+    BooleanQuery.Builder qNotDisjoint = new BooleanQuery.Builder();
+    qNotDisjoint.add(qHasEnv, BooleanClause.Occur.MUST);
+    Query qDisjoint = makeDisjoint(bbox);
+    qNotDisjoint.add(qDisjoint, BooleanClause.Occur.MUST_NOT);
+
+    //Query qDisjoint = makeDisjoint();
+    //BooleanQuery qNotDisjoint = new BooleanQuery();
+    //qNotDisjoint.add(new MatchAllDocsQuery(),BooleanClause.Occur.SHOULD);
+    //qNotDisjoint.add(qDisjoint,BooleanClause.Occur.MUST_NOT);
+    return qNotDisjoint.build();
+  }
+
+  /**
+   * Makes a boolean query based upon a collection of queries and a logical operator.
+   *
+   * @param occur the logical operator
+   * @param queries the query collection
+   * @return the query
+   */
+  BooleanQuery makeQuery(BooleanClause.Occur occur, Query... queries) {
+    BooleanQuery.Builder bq = new BooleanQuery.Builder();
+    for (Query query : queries) {
+      if (query != null)
+        bq.add(query, occur);
+    }
+    return bq.build();
+  }
+
+  /**
+   * Constructs a query to retrieve documents are fully within the input envelope.
+   *
+   * @return the spatial query
+   */
+  Query makeWithin(Rectangle bbox) {
+
+    // general case
+    // docMinX >= queryExtent.getMinX() AND docMinY >= queryExtent.getMinY() AND docMaxX <= queryExtent.getMaxX() AND docMaxY <= queryExtent.getMaxY()
+
+    // Y conditions
+    // docMinY >= queryExtent.getMinY() AND docMaxY <= queryExtent.getMaxY()
+    Query qMinY = this.makeNumericRangeQuery(field_minY, bbox.getMinY(), null, true, false);
+    Query qMaxY = this.makeNumericRangeQuery(field_maxY, null, bbox.getMaxY(), false, true);
+    Query yConditions = this.makeQuery(BooleanClause.Occur.MUST, qMinY, qMaxY);
+
+    // X conditions
+    Query xConditions;
+
+    if (ctx.isGeo() && bbox.getMinX() == -180.0 && bbox.getMaxX() == 180.0) {
+      //if query world-wraps, only the y condition matters
+      return yConditions;
+
+    } else if (!bbox.getCrossesDateLine()) {
+      // queries that do not cross the date line
+
+      // docMinX >= queryExtent.getMinX() AND docMaxX <= queryExtent.getMaxX()
+      Query qMinX = this.makeNumericRangeQuery(field_minX, bbox.getMinX(), null, true, false);
+      Query qMaxX = this.makeNumericRangeQuery(field_maxX, null, bbox.getMaxX(), false, true);
+      Query qMinMax = this.makeQuery(BooleanClause.Occur.MUST, qMinX, qMaxX);
+
+      double edge = 0;//none, otherwise opposite dateline of query
+      if (bbox.getMinX() == -180.0)
+        edge = 180;
+      else if (bbox.getMaxX() == 180.0)
+        edge = -180;
+      if (edge != 0 && ctx.isGeo()) {
+        Query edgeQ = makeQuery(BooleanClause.Occur.MUST,
+            makeNumberTermQuery(field_minX, edge), makeNumberTermQuery(field_maxX, edge));
+        qMinMax = makeQuery(BooleanClause.Occur.SHOULD, qMinMax, edgeQ);
+      }
+
+      xConditions = this.makeXDL(false, qMinMax);
+
+      // queries that cross the date line
+    } else {
+
+      // X Conditions for documents that do not cross the date line
+
+      // the document should be within the left portion of the query
+      // docMinX >= queryExtent.getMinX() AND docMaxX <= 180.0
+      Query qMinXLeft = this.makeNumericRangeQuery(field_minX, bbox.getMinX(), null, true, false);
+      Query qMaxXLeft = this.makeNumericRangeQuery(field_maxX, null, 180.0, false, true);
+      Query qLeft = this.makeQuery(BooleanClause.Occur.MUST, qMinXLeft, qMaxXLeft);
+
+      // the document should be within the right portion of the query
+      // docMinX >= -180.0 AND docMaxX <= queryExtent.getMaxX()
+      Query qMinXRight = this.makeNumericRangeQuery(field_minX, -180.0, null, true, false);
+      Query qMaxXRight = this.makeNumericRangeQuery(field_maxX, null, bbox.getMaxX(), false, true);
+      Query qRight = this.makeQuery(BooleanClause.Occur.MUST, qMinXRight, qMaxXRight);
+
+      // either left or right conditions should occur,
+      // apply the left and right conditions to documents that do not cross the date line
+      Query qLeftRight = this.makeQuery(BooleanClause.Occur.SHOULD, qLeft, qRight);
+      Query qNonXDL = this.makeXDL(false, qLeftRight);
+
+      // X Conditions for documents that cross the date line,
+      // the left portion of the document must be within the left portion of the query,
+      // AND the right portion of the document must be within the right portion of the query
+      // docMinXLeft >= queryExtent.getMinX() AND docMaxXLeft <= 180.0
+      // AND docMinXRight >= -180.0 AND docMaxXRight <= queryExtent.getMaxX()
+      Query qXDLLeft = this.makeNumericRangeQuery(field_minX, bbox.getMinX(), null, true, false);
+      Query qXDLRight = this.makeNumericRangeQuery(field_maxX, null, bbox.getMaxX(), false, true);
+      Query qXDLLeftRight = this.makeQuery(BooleanClause.Occur.MUST, qXDLLeft, qXDLRight);
+      Query qXDL = this.makeXDL(true, qXDLLeftRight);
+
+      // apply the non-XDL and XDL conditions
+      xConditions = this.makeQuery(BooleanClause.Occur.SHOULD, qNonXDL, qXDL);
+    }
+
+    // both X and Y conditions must occur
+    return this.makeQuery(BooleanClause.Occur.MUST, xConditions, yConditions);
+  }
+
+  /**
+   * Constructs a query to retrieve documents that do or do not cross the date line.
+   *
+   * @param crossedDateLine <code>true</true> for documents that cross the date line
+   * @return the query
+   */
+  private Query makeXDL(boolean crossedDateLine) {
+    // The 'T' and 'F' values match solr fields
+    return new TermQuery(new Term(field_xdl, crossedDateLine ? "T" : "F"));
+  }
+
+  /**
+   * Constructs a query to retrieve documents that do or do not cross the date line
+   * and match the supplied spatial query.
+   *
+   * @param crossedDateLine <code>true</true> for documents that cross the date line
+   * @param query the spatial query
+   * @return the query
+   */
+  private Query makeXDL(boolean crossedDateLine, Query query) {
+    if (!ctx.isGeo()) {
+      assert !crossedDateLine;
+      return query;
+    }
+    BooleanQuery.Builder bq = new BooleanQuery.Builder();
+    bq.add(this.makeXDL(crossedDateLine), BooleanClause.Occur.MUST);
+    bq.add(query, BooleanClause.Occur.MUST);
+    return bq.build();
+  }
+
+  private Query makeNumberTermQuery(String field, double number) {
+    if (hasPointVals) {
+      return DoublePoint.newExactQuery(field, number);
+    } else if (legacyNumericFieldType != null) {
+      BytesRefBuilder bytes = new BytesRefBuilder();
+      LegacyNumericUtils.longToPrefixCoded(NumericUtils.doubleToSortableLong(number), 0, bytes);
+      return new TermQuery(new Term(field, bytes.get()));
+    }
+    throw new UnsupportedOperationException("An index is required for this operation.");
+  }
+
+  /**
+   * Returns a numeric range query based on FieldType
+   * {@link LegacyNumericRangeQuery} is used for indexes created using {@code FieldType.LegacyNumericType}
+   * {@link DoublePoint#newRangeQuery} is used for indexes created using {@link DoublePoint} fields
+   *
+   * @param fieldname field name. must not be <code>null</code>.
+   * @param min minimum value of the range.
+   * @param max maximum value of the range.
+   * @param minInclusive include the minimum value if <code>true</code>.
+   * @param maxInclusive include the maximum value if <code>true</code>
+   */
+  private Query makeNumericRangeQuery(String fieldname, Double min, Double max, boolean minInclusive, boolean maxInclusive) {
+    if (hasPointVals) {
+      if (min == null) {
+        min = Double.NEGATIVE_INFINITY;
+      }
+
+      if (max == null) {
+        max = Double.POSITIVE_INFINITY;
+      }
+
+      if (minInclusive == false) {
+        min = Math.nextUp(min);
+      }
+
+      if (maxInclusive == false) {
+        max = Math.nextDown(max);
+      }
+
+      return DoublePoint.newRangeQuery(fieldname, min, max);
+    } else if (legacyNumericFieldType != null) {// todo remove legacy numeric support in 7.0
+      return LegacyNumericRangeQuery.newDoubleRange(fieldname, legacyNumericFieldType.numericPrecisionStep(), min, max, minInclusive, maxInclusive);
+    }
+    throw new UnsupportedOperationException("An index is required for this operation.");
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/759fa42b/solr/core/src/java/org/apache/solr/legacy/BBoxValueSource.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/legacy/BBoxValueSource.java b/solr/core/src/java/org/apache/solr/legacy/BBoxValueSource.java
new file mode 100644
index 0000000..cd577c7
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/legacy/BBoxValueSource.java
@@ -0,0 +1,135 @@
+/*
+ * 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.solr.legacy;
+
+import java.io.IOException;
+import java.util.Map;
+
+import org.apache.lucene.index.DocValues;
+import org.apache.lucene.index.LeafReader;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.NumericDocValues;
+import org.apache.lucene.queries.function.FunctionValues;
+import org.apache.lucene.queries.function.ValueSource;
+import org.apache.lucene.search.Explanation;
+import org.locationtech.spatial4j.shape.Rectangle;
+
+/**
+ * A ValueSource in which the indexed Rectangle is returned from
+ * {@link org.apache.lucene.queries.function.FunctionValues#objectVal(int)}.
+ *
+ * @lucene.internal
+ */
+class BBoxValueSource extends ValueSource {
+
+  private final BBoxStrategy strategy;
+
+  public BBoxValueSource(BBoxStrategy strategy) {
+    this.strategy = strategy;
+  }
+
+  @Override
+  public String description() {
+    return "bboxShape(" + strategy.getFieldName() + ")";
+  }
+
+  @Override
+  public FunctionValues getValues(Map context, LeafReaderContext readerContext) throws IOException {
+    LeafReader reader = readerContext.reader();
+    final NumericDocValues minX = DocValues.getNumeric(reader, strategy.field_minX);
+    final NumericDocValues minY = DocValues.getNumeric(reader, strategy.field_minY);
+    final NumericDocValues maxX = DocValues.getNumeric(reader, strategy.field_maxX);
+    final NumericDocValues maxY = DocValues.getNumeric(reader, strategy.field_maxY);
+
+    //reused
+    final Rectangle rect = strategy.getSpatialContext().makeRectangle(0,0,0,0);
+
+    return new FunctionValues() {
+      private int lastDocID = -1;
+
+      private double getDocValue(NumericDocValues values, int doc) throws IOException {
+        int curDocID = values.docID();
+        if (doc > curDocID) {
+          curDocID = values.advance(doc);
+        }
+        if (doc == curDocID) {
+          return Double.longBitsToDouble(values.longValue());
+        } else {
+          return 0.0;
+        }
+      }
+
+      @Override
+      public Object objectVal(int doc) throws IOException {
+        if (doc < lastDocID) {
+          throw new AssertionError("docs were sent out-of-order: lastDocID=" + lastDocID + " vs doc=" + doc);
+        }
+        lastDocID = doc;
+
+        double minXValue = getDocValue(minX, doc);
+        if (minX.docID() != doc) {
+          return null;
+        } else {
+          double minYValue = getDocValue(minY, doc);
+          double maxXValue = getDocValue(maxX, doc);
+          double maxYValue = getDocValue(maxY, doc);
+          rect.reset(minXValue, maxXValue, minYValue, maxYValue);
+          return rect;
+        }
+      }
+
+      @Override
+      public String strVal(int doc) throws IOException {//TODO support WKT output once Spatial4j does
+        Object v = objectVal(doc);
+        return v == null ? null : v.toString();
+      }
+
+      @Override
+      public boolean exists(int doc) throws IOException {
+        getDocValue(minX, doc);
+        return minX.docID() == doc;
+      }
+
+      @Override
+      public Explanation explain(int doc) throws IOException {
+        return Explanation.match(Float.NaN, toString(doc));
+      }
+
+      @Override
+      public String toString(int doc) throws IOException {
+        return description() + '=' + strVal(doc);
+      }
+    };
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) return true;
+    if (o == null || getClass() != o.getClass()) return false;
+
+    BBoxValueSource that = (BBoxValueSource) o;
+
+    if (!strategy.equals(that.strategy)) return false;
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    return strategy.hashCode();
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/759fa42b/solr/core/src/java/org/apache/solr/legacy/DistanceValueSource.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/legacy/DistanceValueSource.java b/solr/core/src/java/org/apache/solr/legacy/DistanceValueSource.java
new file mode 100644
index 0000000..8685d88
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/legacy/DistanceValueSource.java
@@ -0,0 +1,133 @@
+/*
+ * 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.solr.legacy;
+
+import org.apache.lucene.index.NumericDocValues;
+import org.locationtech.spatial4j.distance.DistanceCalculator;
+import org.locationtech.spatial4j.shape.Point;
+import org.apache.lucene.index.LeafReader;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.DocValues;
+import org.apache.lucene.queries.function.FunctionValues;
+import org.apache.lucene.queries.function.ValueSource;
+
+import java.io.IOException;
+import java.util.Map;
+
+/**
+ * An implementation of the Lucene ValueSource model that returns the distance
+ * for a {@link PointVectorStrategy}.
+ *
+ * @lucene.internal
+ */
+public class DistanceValueSource extends ValueSource {
+
+  private PointVectorStrategy strategy;
+  private final Point from;
+  private final double multiplier;
+
+  /**
+   * Constructor.
+   */
+  public DistanceValueSource(PointVectorStrategy strategy, Point from, double multiplier) {
+    this.strategy = strategy;
+    this.from = from;
+    this.multiplier = multiplier;
+  }
+
+  /**
+   * Returns the ValueSource description.
+   */
+  @Override
+  public String description() {
+    return "DistanceValueSource("+strategy+", "+from+")";
+  }
+
+  /**
+   * Returns the FunctionValues used by the function query.
+   */
+  @Override
+  public FunctionValues getValues(Map context, LeafReaderContext readerContext) throws IOException {
+    LeafReader reader = readerContext.reader();
+
+    final NumericDocValues ptX = DocValues.getNumeric(reader, strategy.getFieldNameX());
+    final NumericDocValues ptY = DocValues.getNumeric(reader, strategy.getFieldNameY());
+
+    return new FunctionValues() {
+
+      private int lastDocID = -1;
+
+      private final Point from = DistanceValueSource.this.from;
+      private final DistanceCalculator calculator = strategy.getSpatialContext().getDistCalc();
+      private final double nullValue =
+          (strategy.getSpatialContext().isGeo() ? 180 * multiplier : Double.MAX_VALUE);
+
+      private double getDocValue(NumericDocValues values, int doc) throws IOException {
+        int curDocID = values.docID();
+        if (doc > curDocID) {
+          curDocID = values.advance(doc);
+        }
+        if (doc == curDocID) {
+          return Double.longBitsToDouble(values.longValue());
+        } else {
+          return 0.0;
+        }
+      }
+
+      @Override
+      public float floatVal(int doc) throws IOException {
+        return (float) doubleVal(doc);
+      }
+
+      @Override
+      public double doubleVal(int doc) throws IOException {
+        // make sure it has minX and area
+        double x = getDocValue(ptX, doc);
+        if (ptX.docID() == doc) {
+          double y = getDocValue(ptY, doc);
+          assert ptY.docID() == doc;
+          return calculator.distance(from, x, y) * multiplier;
+        }
+        return nullValue;
+      }
+
+      @Override
+      public String toString(int doc) throws IOException {
+        return description() + "=" + floatVal(doc);
+      }
+    };
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) return true;
+    if (o == null || getClass() != o.getClass()) return false;
+
+    DistanceValueSource that = (DistanceValueSource) o;
+
+    if (!from.equals(that.from)) return false;
+    if (!strategy.equals(that.strategy)) return false;
+    if (multiplier != that.multiplier) return false;
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    return from.hashCode();
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/759fa42b/solr/core/src/java/org/apache/solr/legacy/LegacyDoubleField.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/legacy/LegacyDoubleField.java b/solr/core/src/java/org/apache/solr/legacy/LegacyDoubleField.java
new file mode 100644
index 0000000..b6a2897
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/legacy/LegacyDoubleField.java
@@ -0,0 +1,174 @@
+/*
+ * 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.solr.legacy;
+
+
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.DoublePoint;
+import org.apache.lucene.index.IndexOptions;
+
+
+/**
+ * <p>
+ * Field that indexes <code>double</code> values
+ * for efficient range filtering and sorting. Here's an example usage:
+ * 
+ * <pre class="prettyprint">
+ * document.add(new LegacyDoubleField(name, 6.0, Field.Store.NO));
+ * </pre>
+ * 
+ * For optimal performance, re-use the <code>LegacyDoubleField</code> and
+ * {@link Document} instance for more than one document:
+ * 
+ * <pre class="prettyprint">
+ *  LegacyDoubleField field = new LegacyDoubleField(name, 0.0, Field.Store.NO);
+ *  Document document = new Document();
+ *  document.add(field);
+ * 
+ *  for(all documents) {
+ *    ...
+ *    field.setDoubleValue(value)
+ *    writer.addDocument(document);
+ *    ...
+ *  }
+ * </pre>
+ *
+ * See also {@link LegacyIntField}, {@link LegacyLongField}, {@link
+ * LegacyFloatField}.
+ *
+ * <p>To perform range querying or filtering against a
+ * <code>LegacyDoubleField</code>, use {@link org.apache.solr.legacy.LegacyNumericRangeQuery}.
+ * To sort according to a
+ * <code>LegacyDoubleField</code>, use the normal numeric sort types, eg
+ * {@link org.apache.lucene.search.SortField.Type#DOUBLE}. <code>LegacyDoubleField</code>
+ * values can also be loaded directly from {@link org.apache.lucene.index.LeafReader#getNumericDocValues}.</p>
+ *
+ * <p>You may add the same field name as an <code>LegacyDoubleField</code> to
+ * the same document more than once.  Range querying and
+ * filtering will be the logical OR of all values; so a range query
+ * will hit all documents that have at least one value in
+ * the range. However sort behavior is not defined.  If you need to sort,
+ * you should separately index a single-valued <code>LegacyDoubleField</code>.</p>
+ *
+ * <p>A <code>LegacyDoubleField</code> will consume somewhat more disk space
+ * in the index than an ordinary single-valued field.
+ * However, for a typical index that includes substantial
+ * textual content per document, this increase will likely
+ * be in the noise. </p>
+ *
+ * <p>Within Lucene, each numeric value is indexed as a
+ * <em>trie</em> structure, where each term is logically
+ * assigned to larger and larger pre-defined brackets (which
+ * are simply lower-precision representations of the value).
+ * The step size between each successive bracket is called the
+ * <code>precisionStep</code>, measured in bits.  Smaller
+ * <code>precisionStep</code> values result in larger number
+ * of brackets, which consumes more disk space in the index
+ * but may result in faster range search performance.  The
+ * default value, 16, was selected for a reasonable tradeoff
+ * of disk space consumption versus performance.  You can
+ * create a custom {@link LegacyFieldType} and invoke the {@link
+ * LegacyFieldType#setNumericPrecisionStep} method if you'd
+ * like to change the value.  Note that you must also
+ * specify a congruent value when creating {@link
+ * org.apache.solr.legacy.LegacyNumericRangeQuery}.
+ * For low cardinality fields larger precision steps are good.
+ * If the cardinality is &lt; 100, it is fair
+ * to use {@link Integer#MAX_VALUE}, which produces one
+ * term per value.
+ *
+ * <p>For more information on the internals of numeric trie
+ * indexing, including the <a
+ * href="LegacyNumericRangeQuery.html#precisionStepDesc"><code>precisionStep</code></a>
+ * configuration, see {@link org.apache.solr.legacy.LegacyNumericRangeQuery}. The format of
+ * indexed values is described in {@link org.apache.solr.legacy.LegacyNumericUtils}.
+ *
+ * <p>If you only need to sort by numeric value, and never
+ * run range querying/filtering, you can index using a
+ * <code>precisionStep</code> of {@link Integer#MAX_VALUE}.
+ * This will minimize disk space consumed. </p>
+ *
+ * <p>More advanced users can instead use {@link
+ * org.apache.solr.legacy.LegacyNumericTokenStream} directly, when indexing numbers. This
+ * class is a wrapper around this token stream type for
+ * easier, more intuitive usage.</p>
+ *
+ * @deprecated Please use {@link DoublePoint} instead
+ *
+ * @since 2.9
+ */
+
+@Deprecated
+public final class LegacyDoubleField extends LegacyField {
+  
+  /** 
+   * Type for a LegacyDoubleField that is not stored:
+   * normalization factors, frequencies, and positions are omitted.
+   */
+  public static final LegacyFieldType TYPE_NOT_STORED = new LegacyFieldType();
+  static {
+    TYPE_NOT_STORED.setTokenized(true);
+    TYPE_NOT_STORED.setOmitNorms(true);
+    TYPE_NOT_STORED.setIndexOptions(IndexOptions.DOCS);
+    TYPE_NOT_STORED.setNumericType(LegacyNumericType.DOUBLE);
+    TYPE_NOT_STORED.freeze();
+  }
+
+  /** 
+   * Type for a stored LegacyDoubleField:
+   * normalization factors, frequencies, and positions are omitted.
+   */
+  public static final LegacyFieldType TYPE_STORED = new LegacyFieldType();
+  static {
+    TYPE_STORED.setTokenized(true);
+    TYPE_STORED.setOmitNorms(true);
+    TYPE_STORED.setIndexOptions(IndexOptions.DOCS);
+    TYPE_STORED.setNumericType(LegacyNumericType.DOUBLE);
+    TYPE_STORED.setStored(true);
+    TYPE_STORED.freeze();
+  }
+
+  /** Creates a stored or un-stored LegacyDoubleField with the provided value
+   *  and default <code>precisionStep</code> {@link
+   *  org.apache.solr.legacy.LegacyNumericUtils#PRECISION_STEP_DEFAULT} (16).
+   *  @param name field name
+   *  @param value 64-bit double value
+   *  @param stored Store.YES if the content should also be stored
+   *  @throws IllegalArgumentException if the field name is null. 
+   */
+  public LegacyDoubleField(String name, double value, Store stored) {
+    super(name, stored == Store.YES ? TYPE_STORED : TYPE_NOT_STORED);
+    fieldsData = Double.valueOf(value);
+  }
+  
+  /** Expert: allows you to customize the {@link
+   *  LegacyFieldType}. 
+   *  @param name field name
+   *  @param value 64-bit double value
+   *  @param type customized field type: must have {@link LegacyFieldType#numericType()}
+   *         of {@link LegacyNumericType#DOUBLE}.
+   *  @throws IllegalArgumentException if the field name or type is null, or
+   *          if the field type does not have a DOUBLE numericType()
+   */
+  public LegacyDoubleField(String name, double value, LegacyFieldType type) {
+    super(name, type);
+    if (type.numericType() != LegacyNumericType.DOUBLE) {
+      throw new IllegalArgumentException("type.numericType() must be DOUBLE but got " + type.numericType());
+    }
+    fieldsData = Double.valueOf(value);
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/759fa42b/solr/core/src/java/org/apache/solr/legacy/LegacyField.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/legacy/LegacyField.java b/solr/core/src/java/org/apache/solr/legacy/LegacyField.java
new file mode 100644
index 0000000..7a6bde0
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/legacy/LegacyField.java
@@ -0,0 +1,90 @@
+/*
+ * 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.solr.legacy;
+
+import org.apache.lucene.analysis.Analyzer;
+import org.apache.lucene.analysis.TokenStream;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.index.IndexOptions;
+
+/**
+ * Field extension with support for legacy numerics
+ * @deprecated Please switch to {@link org.apache.lucene.index.PointValues} instead
+ */
+@Deprecated
+public class LegacyField extends Field {
+
+  /**
+   * Expert: creates a field with no initial value.
+   * Intended only for custom LegacyField subclasses.
+   * @param name field name
+   * @param type field type
+   * @throws IllegalArgumentException if either the name or type
+   *         is null.
+   */
+  public LegacyField(String name, LegacyFieldType type) {
+    super(name, type);
+  }
+  
+  @Override
+  public TokenStream tokenStream(Analyzer analyzer, TokenStream reuse) {
+    if (fieldType().indexOptions() == IndexOptions.NONE) {
+      // Not indexed
+      return null;
+    }
+    final LegacyFieldType fieldType = (LegacyFieldType) fieldType();
+    final LegacyNumericType numericType = fieldType.numericType();
+    if (numericType != null) {
+      if (!(reuse instanceof LegacyNumericTokenStream && ((LegacyNumericTokenStream)reuse).getPrecisionStep() == fieldType.numericPrecisionStep())) {
+        // lazy init the TokenStream as it is heavy to instantiate
+        // (attributes,...) if not needed (stored field loading)
+        reuse = new LegacyNumericTokenStream(fieldType.numericPrecisionStep());
+      }
+      final LegacyNumericTokenStream nts = (LegacyNumericTokenStream) reuse;
+      // initialize value in TokenStream
+      final Number val = (Number) fieldsData;
+      switch (numericType) {
+      case INT:
+        nts.setIntValue(val.intValue());
+        break;
+      case LONG:
+        nts.setLongValue(val.longValue());
+        break;
+      case FLOAT:
+        nts.setFloatValue(val.floatValue());
+        break;
+      case DOUBLE:
+        nts.setDoubleValue(val.doubleValue());
+        break;
+      default:
+        throw new AssertionError("Should never get here");
+      }
+      return reuse;
+    }
+    return super.tokenStream(analyzer, reuse);
+  }
+  
+  @Override
+  public void setTokenStream(TokenStream tokenStream) {
+    final LegacyFieldType fieldType = (LegacyFieldType) fieldType();
+    if (fieldType.numericType() != null) {
+      throw new IllegalArgumentException("cannot set private TokenStream on numeric fields");
+    }
+    super.setTokenStream(tokenStream);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/759fa42b/solr/core/src/java/org/apache/solr/legacy/LegacyFieldType.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/legacy/LegacyFieldType.java b/solr/core/src/java/org/apache/solr/legacy/LegacyFieldType.java
new file mode 100644
index 0000000..a18a00a
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/legacy/LegacyFieldType.java
@@ -0,0 +1,149 @@
+/*
+ * 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.solr.legacy;
+
+import org.apache.lucene.document.FieldType;
+import org.apache.lucene.index.IndexOptions;
+
+/**
+ * FieldType extension with support for legacy numerics
+ * @deprecated Please switch to {@link org.apache.lucene.index.PointValues} instead
+ */
+@Deprecated
+public final class LegacyFieldType extends FieldType {
+  private LegacyNumericType numericType;
+  private int numericPrecisionStep = LegacyNumericUtils.PRECISION_STEP_DEFAULT;
+
+  /**
+   * Create a new mutable LegacyFieldType with all of the properties from <code>ref</code>
+   */
+  public LegacyFieldType(LegacyFieldType ref) {
+    super(ref);
+    this.numericType = ref.numericType;
+    this.numericPrecisionStep = ref.numericPrecisionStep;
+  }
+  
+  /**
+   * Create a new FieldType with default properties.
+   */
+  public LegacyFieldType() {
+  }
+  
+  /**
+   * Specifies the field's numeric type.
+   * @param type numeric type, or null if the field has no numeric type.
+   * @throws IllegalStateException if this FieldType is frozen against
+   *         future modifications.
+   * @see #numericType()
+   *
+   * @deprecated Please switch to {@link org.apache.lucene.index.PointValues} instead
+   */
+  @Deprecated
+  public void setNumericType(LegacyNumericType type) {
+    checkIfFrozen();
+    numericType = type;
+  }
+  
+  /** 
+   * LegacyNumericType: if non-null then the field's value will be indexed
+   * numerically so that {@link org.apache.solr.legacy.LegacyNumericRangeQuery} can be used at
+   * search time. 
+   * <p>
+   * The default is <code>null</code> (no numeric type) 
+   * @see #setNumericType(LegacyNumericType)
+   *
+   * @deprecated Please switch to {@link org.apache.lucene.index.PointValues} instead
+   */
+  @Deprecated
+  public LegacyNumericType numericType() {
+    return numericType;
+  }
+  
+  /**
+   * Sets the numeric precision step for the field.
+   * @param precisionStep numeric precision step for the field
+   * @throws IllegalArgumentException if precisionStep is less than 1. 
+   * @throws IllegalStateException if this FieldType is frozen against
+   *         future modifications.
+   * @see #numericPrecisionStep()
+   *
+   * @deprecated Please switch to {@link org.apache.lucene.index.PointValues} instead
+   */
+  @Deprecated
+  public void setNumericPrecisionStep(int precisionStep) {
+    checkIfFrozen();
+    if (precisionStep < 1) {
+      throw new IllegalArgumentException("precisionStep must be >= 1 (got " + precisionStep + ")");
+    }
+    this.numericPrecisionStep = precisionStep;
+  }
+  
+  /** 
+   * Precision step for numeric field. 
+   * <p>
+   * This has no effect if {@link #numericType()} returns null.
+   * <p>
+   * The default is {@link org.apache.solr.legacy.LegacyNumericUtils#PRECISION_STEP_DEFAULT}
+   * @see #setNumericPrecisionStep(int)
+   *
+   * @deprecated Please switch to {@link org.apache.lucene.index.PointValues} instead
+   */
+  @Deprecated
+  public int numericPrecisionStep() {
+    return numericPrecisionStep;
+  }
+
+  @Override
+  public int hashCode() {
+    final int prime = 31;
+    int result = super.hashCode();
+    result = prime * result + numericPrecisionStep;
+    result = prime * result + ((numericType == null) ? 0 : numericType.hashCode());
+    return result;
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (!super.equals(obj)) {
+      return false;
+    }
+    if (getClass() != obj.getClass()) return false;
+    LegacyFieldType other = (LegacyFieldType) obj;
+    if (numericPrecisionStep != other.numericPrecisionStep) return false;
+    if (numericType != other.numericType) return false;
+    return true;
+  }
+
+  /** Prints a Field for human consumption. */
+  @Override
+  public String toString() {
+    StringBuilder result = new StringBuilder();
+    result.append(super.toString());
+    if (indexOptions() != IndexOptions.NONE) {
+      if (result.length() > 0) {
+        result.append(",");
+      }
+      if (numericType != null) {
+        result.append(",numericType=");
+        result.append(numericType);
+        result.append(",numericPrecisionStep=");
+        result.append(numericPrecisionStep);
+      }
+    }
+    return result.toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/759fa42b/solr/core/src/java/org/apache/solr/legacy/LegacyFloatField.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/legacy/LegacyFloatField.java b/solr/core/src/java/org/apache/solr/legacy/LegacyFloatField.java
new file mode 100644
index 0000000..79ec0bd
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/legacy/LegacyFloatField.java
@@ -0,0 +1,174 @@
+/*
+ * 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.solr.legacy;
+
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.FloatPoint;
+import org.apache.lucene.index.IndexOptions;
+
+/**
+ * <p>
+ * Field that indexes <code>float</code> values
+ * for efficient range filtering and sorting. Here's an example usage:
+ * 
+ * <pre class="prettyprint">
+ * document.add(new LegacyFloatField(name, 6.0F, Field.Store.NO));
+ * </pre>
+ * 
+ * For optimal performance, re-use the <code>LegacyFloatField</code> and
+ * {@link Document} instance for more than one document:
+ * 
+ * <pre class="prettyprint">
+ *  LegacyFloatField field = new LegacyFloatField(name, 0.0F, Field.Store.NO);
+ *  Document document = new Document();
+ *  document.add(field);
+ * 
+ *  for(all documents) {
+ *    ...
+ *    field.setFloatValue(value)
+ *    writer.addDocument(document);
+ *    ...
+ *  }
+ * </pre>
+ *
+ * See also {@link LegacyIntField}, {@link LegacyLongField}, {@link
+ * LegacyDoubleField}.
+ *
+ * <p>To perform range querying or filtering against a
+ * <code>LegacyFloatField</code>, use {@link org.apache.solr.legacy.LegacyNumericRangeQuery}.
+ * To sort according to a
+ * <code>LegacyFloatField</code>, use the normal numeric sort types, eg
+ * {@link org.apache.lucene.search.SortField.Type#FLOAT}. <code>LegacyFloatField</code>
+ * values can also be loaded directly from {@link org.apache.lucene.index.LeafReader#getNumericDocValues}.</p>
+ *
+ * <p>You may add the same field name as an <code>LegacyFloatField</code> to
+ * the same document more than once.  Range querying and
+ * filtering will be the logical OR of all values; so a range query
+ * will hit all documents that have at least one value in
+ * the range. However sort behavior is not defined.  If you need to sort,
+ * you should separately index a single-valued <code>LegacyFloatField</code>.</p>
+ *
+ * <p>A <code>LegacyFloatField</code> will consume somewhat more disk space
+ * in the index than an ordinary single-valued field.
+ * However, for a typical index that includes substantial
+ * textual content per document, this increase will likely
+ * be in the noise. </p>
+ *
+ * <p>Within Lucene, each numeric value is indexed as a
+ * <em>trie</em> structure, where each term is logically
+ * assigned to larger and larger pre-defined brackets (which
+ * are simply lower-precision representations of the value).
+ * The step size between each successive bracket is called the
+ * <code>precisionStep</code>, measured in bits.  Smaller
+ * <code>precisionStep</code> values result in larger number
+ * of brackets, which consumes more disk space in the index
+ * but may result in faster range search performance.  The
+ * default value, 8, was selected for a reasonable tradeoff
+ * of disk space consumption versus performance.  You can
+ * create a custom {@link LegacyFieldType} and invoke the {@link
+ * LegacyFieldType#setNumericPrecisionStep} method if you'd
+ * like to change the value.  Note that you must also
+ * specify a congruent value when creating {@link
+ * org.apache.solr.legacy.LegacyNumericRangeQuery}.
+ * For low cardinality fields larger precision steps are good.
+ * If the cardinality is &lt; 100, it is fair
+ * to use {@link Integer#MAX_VALUE}, which produces one
+ * term per value.
+ *
+ * <p>For more information on the internals of numeric trie
+ * indexing, including the <a
+ * href="LegacyNumericRangeQuery.html#precisionStepDesc"><code>precisionStep</code></a>
+ * configuration, see {@link org.apache.solr.legacy.LegacyNumericRangeQuery}. The format of
+ * indexed values is described in {@link org.apache.solr.legacy.LegacyNumericUtils}.
+ *
+ * <p>If you only need to sort by numeric value, and never
+ * run range querying/filtering, you can index using a
+ * <code>precisionStep</code> of {@link Integer#MAX_VALUE}.
+ * This will minimize disk space consumed. </p>
+ *
+ * <p>More advanced users can instead use {@link
+ * org.apache.solr.legacy.LegacyNumericTokenStream} directly, when indexing numbers. This
+ * class is a wrapper around this token stream type for
+ * easier, more intuitive usage.</p>
+ *
+ * @deprecated Please use {@link FloatPoint} instead
+ *
+ * @since 2.9
+ */
+
+@Deprecated
+public final class LegacyFloatField extends LegacyField {
+  
+  /** 
+   * Type for a LegacyFloatField that is not stored:
+   * normalization factors, frequencies, and positions are omitted.
+   */
+  public static final LegacyFieldType TYPE_NOT_STORED = new LegacyFieldType();
+  static {
+    TYPE_NOT_STORED.setTokenized(true);
+    TYPE_NOT_STORED.setOmitNorms(true);
+    TYPE_NOT_STORED.setIndexOptions(IndexOptions.DOCS);
+    TYPE_NOT_STORED.setNumericType(LegacyNumericType.FLOAT);
+    TYPE_NOT_STORED.setNumericPrecisionStep(LegacyNumericUtils.PRECISION_STEP_DEFAULT_32);
+    TYPE_NOT_STORED.freeze();
+  }
+
+  /** 
+   * Type for a stored LegacyFloatField:
+   * normalization factors, frequencies, and positions are omitted.
+   */
+  public static final LegacyFieldType TYPE_STORED = new LegacyFieldType();
+  static {
+    TYPE_STORED.setTokenized(true);
+    TYPE_STORED.setOmitNorms(true);
+    TYPE_STORED.setIndexOptions(IndexOptions.DOCS);
+    TYPE_STORED.setNumericType(LegacyNumericType.FLOAT);
+    TYPE_STORED.setNumericPrecisionStep(LegacyNumericUtils.PRECISION_STEP_DEFAULT_32);
+    TYPE_STORED.setStored(true);
+    TYPE_STORED.freeze();
+  }
+
+  /** Creates a stored or un-stored LegacyFloatField with the provided value
+   *  and default <code>precisionStep</code> {@link
+   *  org.apache.solr.legacy.LegacyNumericUtils#PRECISION_STEP_DEFAULT_32} (8).
+   *  @param name field name
+   *  @param value 32-bit double value
+   *  @param stored Store.YES if the content should also be stored
+   *  @throws IllegalArgumentException if the field name is null.
+   */
+  public LegacyFloatField(String name, float value, Store stored) {
+    super(name, stored == Store.YES ? TYPE_STORED : TYPE_NOT_STORED);
+    fieldsData = Float.valueOf(value);
+  }
+  
+  /** Expert: allows you to customize the {@link
+   *  LegacyFieldType}. 
+   *  @param name field name
+   *  @param value 32-bit float value
+   *  @param type customized field type: must have {@link LegacyFieldType#numericType()}
+   *         of {@link LegacyNumericType#FLOAT}.
+   *  @throws IllegalArgumentException if the field name or type is null, or
+   *          if the field type does not have a FLOAT numericType()
+   */
+  public LegacyFloatField(String name, float value, LegacyFieldType type) {
+    super(name, type);
+    if (type.numericType() != LegacyNumericType.FLOAT) {
+      throw new IllegalArgumentException("type.numericType() must be FLOAT but got " + type.numericType());
+    }
+    fieldsData = Float.valueOf(value);
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/759fa42b/solr/core/src/java/org/apache/solr/legacy/LegacyIntField.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/legacy/LegacyIntField.java b/solr/core/src/java/org/apache/solr/legacy/LegacyIntField.java
new file mode 100644
index 0000000..838ad4e
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/legacy/LegacyIntField.java
@@ -0,0 +1,175 @@
+/*
+ * 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.solr.legacy;
+
+
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.IntPoint;
+import org.apache.lucene.index.IndexOptions;
+
+/**
+ * <p>
+ * Field that indexes <code>int</code> values
+ * for efficient range filtering and sorting. Here's an example usage:
+ * 
+ * <pre class="prettyprint">
+ * document.add(new LegacyIntField(name, 6, Field.Store.NO));
+ * </pre>
+ * 
+ * For optimal performance, re-use the <code>LegacyIntField</code> and
+ * {@link Document} instance for more than one document:
+ * 
+ * <pre class="prettyprint">
+ *  LegacyIntField field = new LegacyIntField(name, 6, Field.Store.NO);
+ *  Document document = new Document();
+ *  document.add(field);
+ * 
+ *  for(all documents) {
+ *    ...
+ *    field.setIntValue(value)
+ *    writer.addDocument(document);
+ *    ...
+ *  }
+ * </pre>
+ *
+ * See also {@link LegacyLongField}, {@link LegacyFloatField}, {@link
+ * LegacyDoubleField}.
+ *
+ * <p>To perform range querying or filtering against a
+ * <code>LegacyIntField</code>, use {@link org.apache.solr.legacy.LegacyNumericRangeQuery}.
+ * To sort according to a
+ * <code>LegacyIntField</code>, use the normal numeric sort types, eg
+ * {@link org.apache.lucene.search.SortField.Type#INT}. <code>LegacyIntField</code>
+ * values can also be loaded directly from {@link org.apache.lucene.index.LeafReader#getNumericDocValues}.</p>
+ *
+ * <p>You may add the same field name as an <code>LegacyIntField</code> to
+ * the same document more than once.  Range querying and
+ * filtering will be the logical OR of all values; so a range query
+ * will hit all documents that have at least one value in
+ * the range. However sort behavior is not defined.  If you need to sort,
+ * you should separately index a single-valued <code>LegacyIntField</code>.</p>
+ *
+ * <p>An <code>LegacyIntField</code> will consume somewhat more disk space
+ * in the index than an ordinary single-valued field.
+ * However, for a typical index that includes substantial
+ * textual content per document, this increase will likely
+ * be in the noise. </p>
+ *
+ * <p>Within Lucene, each numeric value is indexed as a
+ * <em>trie</em> structure, where each term is logically
+ * assigned to larger and larger pre-defined brackets (which
+ * are simply lower-precision representations of the value).
+ * The step size between each successive bracket is called the
+ * <code>precisionStep</code>, measured in bits.  Smaller
+ * <code>precisionStep</code> values result in larger number
+ * of brackets, which consumes more disk space in the index
+ * but may result in faster range search performance.  The
+ * default value, 8, was selected for a reasonable tradeoff
+ * of disk space consumption versus performance.  You can
+ * create a custom {@link LegacyFieldType} and invoke the {@link
+ * LegacyFieldType#setNumericPrecisionStep} method if you'd
+ * like to change the value.  Note that you must also
+ * specify a congruent value when creating {@link
+ * org.apache.solr.legacy.LegacyNumericRangeQuery}.
+ * For low cardinality fields larger precision steps are good.
+ * If the cardinality is &lt; 100, it is fair
+ * to use {@link Integer#MAX_VALUE}, which produces one
+ * term per value.
+ *
+ * <p>For more information on the internals of numeric trie
+ * indexing, including the <a
+ * href="LegacyNumericRangeQuery.html#precisionStepDesc"><code>precisionStep</code></a>
+ * configuration, see {@link org.apache.solr.legacy.LegacyNumericRangeQuery}. The format of
+ * indexed values is described in {@link org.apache.solr.legacy.LegacyNumericUtils}.
+ *
+ * <p>If you only need to sort by numeric value, and never
+ * run range querying/filtering, you can index using a
+ * <code>precisionStep</code> of {@link Integer#MAX_VALUE}.
+ * This will minimize disk space consumed. </p>
+ *
+ * <p>More advanced users can instead use {@link
+ * org.apache.solr.legacy.LegacyNumericTokenStream} directly, when indexing numbers. This
+ * class is a wrapper around this token stream type for
+ * easier, more intuitive usage.</p>
+ *
+ * @deprecated Please use {@link IntPoint} instead
+ *
+ * @since 2.9
+ */
+
+@Deprecated
+public final class LegacyIntField extends LegacyField {
+  
+  /** 
+   * Type for an LegacyIntField that is not stored:
+   * normalization factors, frequencies, and positions are omitted.
+   */
+  public static final LegacyFieldType TYPE_NOT_STORED = new LegacyFieldType();
+  static {
+    TYPE_NOT_STORED.setTokenized(true);
+    TYPE_NOT_STORED.setOmitNorms(true);
+    TYPE_NOT_STORED.setIndexOptions(IndexOptions.DOCS);
+    TYPE_NOT_STORED.setNumericType(LegacyNumericType.INT);
+    TYPE_NOT_STORED.setNumericPrecisionStep(LegacyNumericUtils.PRECISION_STEP_DEFAULT_32);
+    TYPE_NOT_STORED.freeze();
+  }
+
+  /** 
+   * Type for a stored LegacyIntField:
+   * normalization factors, frequencies, and positions are omitted.
+   */
+  public static final LegacyFieldType TYPE_STORED = new LegacyFieldType();
+  static {
+    TYPE_STORED.setTokenized(true);
+    TYPE_STORED.setOmitNorms(true);
+    TYPE_STORED.setIndexOptions(IndexOptions.DOCS);
+    TYPE_STORED.setNumericType(LegacyNumericType.INT);
+    TYPE_STORED.setNumericPrecisionStep(LegacyNumericUtils.PRECISION_STEP_DEFAULT_32);
+    TYPE_STORED.setStored(true);
+    TYPE_STORED.freeze();
+  }
+
+  /** Creates a stored or un-stored LegacyIntField with the provided value
+   *  and default <code>precisionStep</code> {@link
+   *  org.apache.solr.legacy.LegacyNumericUtils#PRECISION_STEP_DEFAULT_32} (8).
+   *  @param name field name
+   *  @param value 32-bit integer value
+   *  @param stored Store.YES if the content should also be stored
+   *  @throws IllegalArgumentException if the field name is null.
+   */
+  public LegacyIntField(String name, int value, Store stored) {
+    super(name, stored == Store.YES ? TYPE_STORED : TYPE_NOT_STORED);
+    fieldsData = Integer.valueOf(value);
+  }
+  
+  /** Expert: allows you to customize the {@link
+   *  LegacyFieldType}. 
+   *  @param name field name
+   *  @param value 32-bit integer value
+   *  @param type customized field type: must have {@link LegacyFieldType#numericType()}
+   *         of {@link LegacyNumericType#INT}.
+   *  @throws IllegalArgumentException if the field name or type is null, or
+   *          if the field type does not have a INT numericType()
+   */
+  public LegacyIntField(String name, int value, LegacyFieldType type) {
+    super(name, type);
+    if (type.numericType() != LegacyNumericType.INT) {
+      throw new IllegalArgumentException("type.numericType() must be INT but got " + type.numericType());
+    }
+    fieldsData = Integer.valueOf(value);
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/759fa42b/solr/core/src/java/org/apache/solr/legacy/LegacyLongField.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/legacy/LegacyLongField.java b/solr/core/src/java/org/apache/solr/legacy/LegacyLongField.java
new file mode 100644
index 0000000..fb48437
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/legacy/LegacyLongField.java
@@ -0,0 +1,184 @@
+/*
+ * 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.solr.legacy;
+
+
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.LongPoint;
+import org.apache.lucene.index.IndexOptions;
+
+
+/**
+ * <p>
+ * Field that indexes <code>long</code> values
+ * for efficient range filtering and sorting. Here's an example usage:
+ * 
+ * <pre class="prettyprint">
+ * document.add(new LegacyLongField(name, 6L, Field.Store.NO));
+ * </pre>
+ * 
+ * For optimal performance, re-use the <code>LegacyLongField</code> and
+ * {@link Document} instance for more than one document:
+ * 
+ * <pre class="prettyprint">
+ *  LegacyLongField field = new LegacyLongField(name, 0L, Field.Store.NO);
+ *  Document document = new Document();
+ *  document.add(field);
+ * 
+ *  for(all documents) {
+ *    ...
+ *    field.setLongValue(value)
+ *    writer.addDocument(document);
+ *    ...
+ *  }
+ * </pre>
+ *
+ * See also {@link LegacyIntField}, {@link LegacyFloatField}, {@link
+ * LegacyDoubleField}.
+ *
+ * Any type that can be converted to long can also be
+ * indexed.  For example, date/time values represented by a
+ * {@link java.util.Date} can be translated into a long
+ * value using the {@link java.util.Date#getTime} method.  If you
+ * don't need millisecond precision, you can quantize the
+ * value, either by dividing the result of
+ * {@link java.util.Date#getTime} or using the separate getters
+ * (for year, month, etc.) to construct an <code>int</code> or
+ * <code>long</code> value.
+ *
+ * <p>To perform range querying or filtering against a
+ * <code>LegacyLongField</code>, use {@link org.apache.solr.legacy.LegacyNumericRangeQuery}.
+ * To sort according to a
+ * <code>LegacyLongField</code>, use the normal numeric sort types, eg
+ * {@link org.apache.lucene.search.SortField.Type#LONG}. <code>LegacyLongField</code>
+ * values can also be loaded directly from {@link org.apache.lucene.index.LeafReader#getNumericDocValues}.
+ *
+ * <p>You may add the same field name as an <code>LegacyLongField</code> to
+ * the same document more than once.  Range querying and
+ * filtering will be the logical OR of all values; so a range query
+ * will hit all documents that have at least one value in
+ * the range. However sort behavior is not defined.  If you need to sort,
+ * you should separately index a single-valued <code>LegacyLongField</code>.
+ *
+ * <p>A <code>LegacyLongField</code> will consume somewhat more disk space
+ * in the index than an ordinary single-valued field.
+ * However, for a typical index that includes substantial
+ * textual content per document, this increase will likely
+ * be in the noise. </p>
+ *
+ * <p>Within Lucene, each numeric value is indexed as a
+ * <em>trie</em> structure, where each term is logically
+ * assigned to larger and larger pre-defined brackets (which
+ * are simply lower-precision representations of the value).
+ * The step size between each successive bracket is called the
+ * <code>precisionStep</code>, measured in bits.  Smaller
+ * <code>precisionStep</code> values result in larger number
+ * of brackets, which consumes more disk space in the index
+ * but may result in faster range search performance.  The
+ * default value, 16, was selected for a reasonable tradeoff
+ * of disk space consumption versus performance.  You can
+ * create a custom {@link LegacyFieldType} and invoke the {@link
+ * LegacyFieldType#setNumericPrecisionStep} method if you'd
+ * like to change the value.  Note that you must also
+ * specify a congruent value when creating {@link
+ * org.apache.solr.legacy.LegacyNumericRangeQuery}.
+ * For low cardinality fields larger precision steps are good.
+ * If the cardinality is &lt; 100, it is fair
+ * to use {@link Integer#MAX_VALUE}, which produces one
+ * term per value.
+ *
+ * <p>For more information on the internals of numeric trie
+ * indexing, including the <a
+ * href="LegacyNumericRangeQuery.html#precisionStepDesc"><code>precisionStep</code></a>
+ * configuration, see {@link org.apache.solr.legacy.LegacyNumericRangeQuery}. The format of
+ * indexed values is described in {@link org.apache.solr.legacy.LegacyNumericUtils}.
+ *
+ * <p>If you only need to sort by numeric value, and never
+ * run range querying/filtering, you can index using a
+ * <code>precisionStep</code> of {@link Integer#MAX_VALUE}.
+ * This will minimize disk space consumed.
+ *
+ * <p>More advanced users can instead use {@link
+ * org.apache.solr.legacy.LegacyNumericTokenStream} directly, when indexing numbers. This
+ * class is a wrapper around this token stream type for
+ * easier, more intuitive usage.</p>
+ *
+ * @deprecated Please use {@link LongPoint} instead
+ *
+ * @since 2.9
+ */
+
+@Deprecated
+public final class LegacyLongField extends LegacyField {
+  
+  /** 
+   * Type for a LegacyLongField that is not stored:
+   * normalization factors, frequencies, and positions are omitted.
+   */
+  public static final LegacyFieldType TYPE_NOT_STORED = new LegacyFieldType();
+  static {
+    TYPE_NOT_STORED.setTokenized(true);
+    TYPE_NOT_STORED.setOmitNorms(true);
+    TYPE_NOT_STORED.setIndexOptions(IndexOptions.DOCS);
+    TYPE_NOT_STORED.setNumericType(LegacyNumericType.LONG);
+    TYPE_NOT_STORED.freeze();
+  }
+
+  /** 
+   * Type for a stored LegacyLongField:
+   * normalization factors, frequencies, and positions are omitted.
+   */
+  public static final LegacyFieldType TYPE_STORED = new LegacyFieldType();
+  static {
+    TYPE_STORED.setTokenized(true);
+    TYPE_STORED.setOmitNorms(true);
+    TYPE_STORED.setIndexOptions(IndexOptions.DOCS);
+    TYPE_STORED.setNumericType(LegacyNumericType.LONG);
+    TYPE_STORED.setStored(true);
+    TYPE_STORED.freeze();
+  }
+
+  /** Creates a stored or un-stored LegacyLongField with the provided value
+   *  and default <code>precisionStep</code> {@link
+   *  org.apache.solr.legacy.LegacyNumericUtils#PRECISION_STEP_DEFAULT} (16).
+   *  @param name field name
+   *  @param value 64-bit long value
+   *  @param stored Store.YES if the content should also be stored
+   *  @throws IllegalArgumentException if the field name is null.
+   */
+  public LegacyLongField(String name, long value, Store stored) {
+    super(name, stored == Store.YES ? TYPE_STORED : TYPE_NOT_STORED);
+    fieldsData = Long.valueOf(value);
+  }
+  
+  /** Expert: allows you to customize the {@link
+   *  LegacyFieldType}. 
+   *  @param name field name
+   *  @param value 64-bit long value
+   *  @param type customized field type: must have {@link LegacyFieldType#numericType()}
+   *         of {@link LegacyNumericType#LONG}.
+   *  @throws IllegalArgumentException if the field name or type is null, or
+   *          if the field type does not have a LONG numericType()
+   */
+  public LegacyLongField(String name, long value, LegacyFieldType type) {
+    super(name, type);
+    if (type.numericType() != LegacyNumericType.LONG) {
+      throw new IllegalArgumentException("type.numericType() must be LONG but got " + type.numericType());
+    }
+    fieldsData = Long.valueOf(value);
+  }
+}