You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ds...@apache.org on 2017/03/12 01:48:07 UTC

lucene-solr:master: SOLR-10039: New LatLonPointSpatialField

Repository: lucene-solr
Updated Branches:
  refs/heads/master 1745b0338 -> 182c20c4e


SOLR-10039: New LatLonPointSpatialField


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

Branch: refs/heads/master
Commit: 182c20c4e55c39362f6d46d388eb2b8e0a9052e6
Parents: 1745b03
Author: David Smiley <ds...@apache.org>
Authored: Sat Mar 11 20:48:01 2017 -0500
Committer: David Smiley <ds...@apache.org>
Committed: Sat Mar 11 20:48:01 2017 -0500

----------------------------------------------------------------------
 solr/CHANGES.txt                                |   3 +
 .../solr/schema/AbstractSpatialFieldType.java   |   2 +-
 .../solr/schema/LatLonPointSpatialField.java    | 272 +++++++++++++++++++
 .../solr/collection1/conf/schema-spatial.xml    |   4 +
 .../apache/solr/search/TestSolr4Spatial.java    |  61 ++++-
 .../basic_configs/conf/managed-schema           |   7 +-
 .../conf/managed-schema                         |   7 +-
 .../conf/managed-schema                         |   7 +-
 8 files changed, 338 insertions(+), 25 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/182c20c4/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index b164405..7bf679f 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -166,6 +166,9 @@ New Features
 
 * SOLR-8045: Deploy V2 API at /v2 instead of /solr/v2 (Cao Manh Dat, Noble Paul)
 
+* SOLR-10039: New LatLonPointSpatialField replacement for LatLonType (and some uses of RPT).  Multi-value capable
+  indexed geo lat-lon points, query by rect or circle.  Efficient distance sorting/boosting too. (David Smiley)
+
 Bug Fixes
 ----------------------
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/182c20c4/solr/core/src/java/org/apache/solr/schema/AbstractSpatialFieldType.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/schema/AbstractSpatialFieldType.java b/solr/core/src/java/org/apache/solr/schema/AbstractSpatialFieldType.java
index 4e2829e..2106205 100644
--- a/solr/core/src/java/org/apache/solr/schema/AbstractSpatialFieldType.java
+++ b/solr/core/src/java/org/apache/solr/schema/AbstractSpatialFieldType.java
@@ -225,7 +225,7 @@ public abstract class AbstractSpatialFieldType<T extends SpatialStrategy> extend
     }
 
     List<IndexableField> result = new ArrayList<>();
-    if (field.indexed()) {
+    if (field.indexed() || field.hasDocValues()) {
       T strategy = getStrategy(field.getName());
       result.addAll(Arrays.asList(strategy.createIndexableFields(shape)));
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/182c20c4/solr/core/src/java/org/apache/solr/schema/LatLonPointSpatialField.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/schema/LatLonPointSpatialField.java b/solr/core/src/java/org/apache/solr/schema/LatLonPointSpatialField.java
new file mode 100644
index 0000000..c09856a
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/schema/LatLonPointSpatialField.java
@@ -0,0 +1,272 @@
+/*
+ * 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.schema;
+
+import java.io.IOException;
+import java.util.Map;
+import java.util.Objects;
+
+import org.apache.lucene.document.Field;
+import org.apache.lucene.document.LatLonDocValuesField;
+import org.apache.lucene.document.LatLonPoint;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.queries.function.FunctionValues;
+import org.apache.lucene.queries.function.ValueSource;
+import org.apache.lucene.queries.function.docvalues.DoubleDocValues;
+import org.apache.lucene.search.FieldComparator;
+import org.apache.lucene.search.IndexOrDocValuesQuery;
+import org.apache.lucene.search.LeafFieldComparator;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.SortField;
+import org.apache.lucene.spatial.SpatialStrategy;
+import org.apache.lucene.spatial.query.SpatialArgs;
+import org.apache.lucene.spatial.query.SpatialOperation;
+import org.apache.lucene.spatial.query.UnsupportedSpatialOperation;
+import org.apache.solr.common.SolrException;
+import org.locationtech.spatial4j.context.SpatialContext;
+import org.locationtech.spatial4j.distance.DistanceUtils;
+import org.locationtech.spatial4j.shape.Circle;
+import org.locationtech.spatial4j.shape.Point;
+import org.locationtech.spatial4j.shape.Rectangle;
+import org.locationtech.spatial4j.shape.Shape;
+
+/**
+ * A spatial implementation based on Lucene's {@code LatLonPoint} and {@code LatLonDocValuesField}. The
+ * first is based on Lucene's "Points" API, which is a BKD Index.  This field type is strictly limited to
+ * coordinates in lat/lon decimal degrees.  The accuracy is about a centimeter.
+ */
+// TODO once LLP & LLDVF are out of Lucene Sandbox, we should be able to javadoc reference them.
+public class LatLonPointSpatialField extends AbstractSpatialFieldType implements SchemaAware {
+  private IndexSchema schema;
+
+  // TODO handle polygons
+
+  @Override
+  public void checkSchemaField(SchemaField field) {
+    // override because if we didn't, FieldType will complain about docValues not being supported (we do support it)
+  }
+
+  @Override
+  public void inform(IndexSchema schema) {
+    this.schema = schema;
+  }
+
+  @Override
+  protected SpatialStrategy newSpatialStrategy(String fieldName) {
+    SchemaField schemaField = schema.getField(fieldName); // TODO change AbstractSpatialFieldType so we get schemaField?
+    return new LatLonPointSpatialStrategy(ctx, fieldName, schemaField.indexed(), schemaField.hasDocValues());
+  }
+
+  // TODO move to Lucene-spatial-extras once LatLonPoint & LatLonDocValuesField moves out of sandbox
+  public static class LatLonPointSpatialStrategy extends SpatialStrategy {
+
+    private final boolean indexed; // for query/filter
+    private final boolean docValues; // for sort. Can be used to query/filter.
+
+    public LatLonPointSpatialStrategy(SpatialContext ctx, String fieldName, boolean indexed, boolean docValues) {
+      super(ctx, fieldName);
+      if (!ctx.isGeo()) {
+        throw new IllegalArgumentException("ctx must be geo=true: " + ctx);
+      }
+      this.indexed = indexed;
+      this.docValues = docValues;
+    }
+
+    @Override
+    public Field[] createIndexableFields(Shape shape) {
+      if (!(shape instanceof Point)) {
+        throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
+            getClass().getSimpleName() + " only supports indexing points; got: " + shape);
+      }
+      Point point = (Point) shape;
+
+      int fieldsLen = (indexed ? 1 : 0) + (docValues ? 1 : 0);
+      Field[] fields = new Field[fieldsLen];
+      int fieldsIdx = 0;
+      if (indexed) {
+        fields[fieldsIdx++] = new LatLonPoint(getFieldName(), point.getY(), point.getX());
+      }
+      if (docValues) {
+        fields[fieldsIdx++] = new LatLonDocValuesField(getFieldName(), point.getY(), point.getX());
+      }
+      return fields;
+    }
+
+    @Override
+    public Query makeQuery(SpatialArgs args) {
+      if (args.getOperation() != SpatialOperation.Intersects) {
+        throw new UnsupportedSpatialOperation(args.getOperation());
+      }
+      Shape shape = args.getShape();
+      if (indexed && docValues) {
+        return new IndexOrDocValuesQuery(makeQueryFromIndex(shape), makeQueryFromDocValues(shape));
+      } else if (indexed) {
+        return makeQueryFromIndex(shape);
+      } else if (docValues) {
+        return makeQueryFromDocValues(shape);
+      } else {
+        throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
+            getFieldName() + " needs indexed (preferred) or docValues to support search");
+      }
+    }
+
+    // Uses LatLonPoint
+    protected Query makeQueryFromIndex(Shape shape) {
+      // note: latitude then longitude order for LLP's methods
+      if (shape instanceof Circle) {
+        Circle circle = (Circle) shape;
+        double radiusMeters = circle.getRadius() * DistanceUtils.DEG_TO_KM * 1000;
+        return LatLonPoint.newDistanceQuery(getFieldName(),
+            circle.getCenter().getY(), circle.getCenter().getX(),
+            radiusMeters);
+      } else if (shape instanceof Rectangle) {
+        Rectangle rect = (Rectangle) shape;
+        return LatLonPoint.newBoxQuery(getFieldName(),
+            rect.getMinY(), rect.getMaxY(), rect.getMinX(), rect.getMaxX());
+      } else if (shape instanceof Point) {
+        Point point = (Point) shape;
+        return LatLonPoint.newDistanceQuery(getFieldName(),
+            point.getY(), point.getX(), 0);
+      } else {
+        throw new UnsupportedOperationException("Shape " + shape.getClass() + " is not supported by " + getClass());
+      }
+//      } else if (shape instanceof LucenePolygonShape) {
+//        // TODO support multi-polygon
+//        Polygon poly = ((LucenePolygonShape)shape).lucenePolygon;
+//        return LatLonPoint.newPolygonQuery(getFieldName(), poly);
+    }
+
+    // Uses DocValuesField  (otherwise identical to above)
+    protected Query makeQueryFromDocValues(Shape shape) {
+      // note: latitude then longitude order for LLP's methods
+      if (shape instanceof Circle) {
+        Circle circle = (Circle) shape;
+        double radiusMeters = circle.getRadius() * DistanceUtils.DEG_TO_KM * 1000;
+        return LatLonDocValuesField.newDistanceQuery(getFieldName(),
+            circle.getCenter().getY(), circle.getCenter().getX(),
+            radiusMeters);
+      } else if (shape instanceof Rectangle) {
+        Rectangle rect = (Rectangle) shape;
+        return LatLonDocValuesField.newBoxQuery(getFieldName(),
+            rect.getMinY(), rect.getMaxY(), rect.getMinX(), rect.getMaxX());
+      } else if (shape instanceof Point) {
+        Point point = (Point) shape;
+        return LatLonDocValuesField.newDistanceQuery(getFieldName(),
+            point.getY(), point.getX(), 0);
+      } else {
+        throw new UnsupportedOperationException("Shape " + shape.getClass() + " is not supported by " + getClass());
+      }
+//      } else if (shape instanceof LucenePolygonShape) {
+//        // TODO support multi-polygon
+//        Polygon poly = ((LucenePolygonShape)shape).lucenePolygon;
+//        return LatLonPoint.newPolygonQuery(getFieldName(), poly);
+    }
+
+    @Override
+    public ValueSource makeDistanceValueSource(Point queryPoint, double multiplier) {
+      if (!docValues) {
+        throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
+            getFieldName() + " must have docValues enabled to support this feature");
+      }
+      // Internally, the distance from LatLonPointSortField/Comparator is in meters. So we must also go from meters to
+      //  degrees, which is what Lucene spatial-extras is oriented around.
+      return new DistanceSortValueSource(getFieldName(), queryPoint,
+          DistanceUtils.KM_TO_DEG / 1000.0 * multiplier);
+    }
+
+    /**
+     * A {@link ValueSource} based around {@code LatLonDocValuesField#newDistanceSort(String, double, double)}.
+     */
+    private static class DistanceSortValueSource extends ValueSource {
+      private final String fieldName;
+      private final Point queryPoint;
+      private final double multiplier;
+
+      DistanceSortValueSource(String fieldName, Point queryPoint, double multiplier) {
+        this.fieldName = fieldName;
+        this.queryPoint = queryPoint;
+        this.multiplier = multiplier;
+      }
+
+      @Override
+      public boolean equals(Object o) {
+        if (this == o) return true;
+        if (o == null || getClass() != o.getClass()) return false;
+        DistanceSortValueSource that = (DistanceSortValueSource) o;
+        return Double.compare(that.multiplier, multiplier) == 0 &&
+            Objects.equals(fieldName, that.fieldName) &&
+            Objects.equals(queryPoint, that.queryPoint);
+      }
+
+      @Override
+      public int hashCode() {
+        return Objects.hash(fieldName, queryPoint, multiplier);
+      }
+
+      @Override
+      public FunctionValues getValues(Map context, LeafReaderContext readerContext) throws IOException {
+        return new DoubleDocValues(this) {
+          @SuppressWarnings("unchecked")
+          final FieldComparator<Double> comparator =
+              (FieldComparator<Double>) getSortField(false).getComparator(1, 1);
+          final LeafFieldComparator leafComparator = comparator.getLeafComparator(readerContext);
+          final double mult = multiplier; // so it's a local field
+
+          // Since this computation is expensive, it's worth caching it just in case.
+          double cacheDoc = -1;
+          double cacheVal = Double.POSITIVE_INFINITY;
+
+          @Override
+          public double doubleVal(int doc) {
+            if (cacheDoc != doc) {
+              try {
+                leafComparator.copy(0, doc);
+                cacheVal = comparator.value(0) * mult;
+                cacheDoc = doc;
+              } catch (IOException e) {
+                throw new RuntimeException(e);
+              }
+            }
+            return cacheVal;
+          }
+
+          @Override
+          public boolean exists(int doc) {
+            return !Double.isInfinite(doubleVal(doc));
+          }
+        };
+      }
+
+      @Override
+      public String description() {
+        return "distSort(" + fieldName + ", " + queryPoint + ", mult:" + multiplier + ")";
+      }
+
+      @Override
+      public SortField getSortField(boolean reverse) {
+        if (reverse) {
+          return super.getSortField(true); // will use an impl that calls getValues
+        }
+        return LatLonDocValuesField.newDistanceSort(fieldName, queryPoint.getY(), queryPoint.getX());
+      }
+
+    }
+
+  }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/182c20c4/solr/core/src/test-files/solr/collection1/conf/schema-spatial.xml
----------------------------------------------------------------------
diff --git a/solr/core/src/test-files/solr/collection1/conf/schema-spatial.xml b/solr/core/src/test-files/solr/collection1/conf/schema-spatial.xml
index 254f58b..9c7a36f 100644
--- a/solr/core/src/test-files/solr/collection1/conf/schema-spatial.xml
+++ b/solr/core/src/test-files/solr/collection1/conf/schema-spatial.xml
@@ -54,6 +54,7 @@
   <fieldType name="bbox" class="solr.BBoxField"
              numberType="tdoubleDV" distanceUnits="degrees" storeSubFields="false"/>
 
+  <fieldType name="llp" class="solr.LatLonPointSpatialField" distanceUnits="degrees" multiValued="true" />
 
   <field name="id" type="string" required="true"/>
 
@@ -64,6 +65,9 @@
   <field name="pointvector" type="pointvector"/>
   <field name="srptgeom" type="srptgeom"/>
   <field name="bbox" type="bbox"/>
+  <field name="llp" type="llp" indexed="true" docValues="true" />
+  <field name="llp_idx" type="llp" indexed="true" docValues="false" />
+  <field name="llp_dv" type="llp" indexed="false" docValues="true" />
 
   <dynamicField name="bboxD_*" type="bbox" indexed="true"/>
   <dynamicField name="str_*" type="string" indexed="true" stored="true"/>

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/182c20c4/solr/core/src/test/org/apache/solr/search/TestSolr4Spatial.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/search/TestSolr4Spatial.java b/solr/core/src/test/org/apache/solr/search/TestSolr4Spatial.java
index 2fe3740..8cd96ae 100644
--- a/solr/core/src/test/org/apache/solr/search/TestSolr4Spatial.java
+++ b/solr/core/src/test/org/apache/solr/search/TestSolr4Spatial.java
@@ -19,7 +19,6 @@ package org.apache.solr.search;
 import java.text.ParseException;
 import java.util.Arrays;
 
-import com.carrotsearch.randomizedtesting.RandomizedTest;
 import com.carrotsearch.randomizedtesting.annotations.ParametersFactory;
 import org.locationtech.spatial4j.context.SpatialContext;
 import org.locationtech.spatial4j.distance.DistanceUtils;
@@ -43,16 +42,18 @@ import org.junit.Test;
  */
 public class TestSolr4Spatial extends SolrTestCaseJ4 {
 
-  private String fieldName;
+  private final String fieldName;
+  private final boolean canCalcDistance;
 
   public TestSolr4Spatial(String fieldName) {
     this.fieldName = fieldName;
+    this.canCalcDistance = !fieldName.equals("llp_idx");
   }
 
   @ParametersFactory
   public static Iterable<Object[]> parameters() {
     return Arrays.asList(new Object[][]{
-        {"srpt_geohash"}, {"srpt_quad"}, {"srpt_packedquad"}, {"stqpt_geohash"}, {"pointvector"}, {"bbox"}
+        {"llp"}, {"llp_idx"}, {"llp_dv"}, {"srpt_geohash"}, {"srpt_quad"}, {"srpt_packedquad"}, {"stqpt_geohash"}, {"pointvector"}, {"bbox"}
     });
   }
 
@@ -105,6 +106,10 @@ public class TestSolr4Spatial extends SolrTestCaseJ4 {
     assertU(adoc("id", "11", fieldName, "89.9,-130"));
     assertU(adoc("id", "12", fieldName, "-89.9,50"));
     assertU(adoc("id", "13", fieldName, "-89.9,-130"));
+    if (random().nextBoolean()) {
+      assertU(commit());
+    }
+    assertU(adoc("id", "99"));//blank
     assertU(commit());
   }
 
@@ -192,7 +197,7 @@ public class TestSolr4Spatial extends SolrTestCaseJ4 {
     //Test using the Lucene spatial syntax
     {
       //never actually need the score but lets test
-      String score = new String[]{null, "none","distance","recipDistance"}[random().nextInt(4)];
+      String score = randomScoreMode();
 
       double distDEG = DistanceUtils.dist2Degrees(distKM, DistanceUtils.EARTH_MEAN_RADIUS_KM);
       Point point = SpatialUtils.parsePoint(ptStr, SpatialContext.GEO);
@@ -225,6 +230,10 @@ public class TestSolr4Spatial extends SolrTestCaseJ4 {
 
   }
 
+  private String randomScoreMode() {
+    return canCalcDistance ? new String[]{null, "none","distance","recipDistance"}[random().nextInt(4)] : "none";
+  }
+
   @Test
   public void testRangeSyntax() {
     setupDocs();
@@ -232,10 +241,10 @@ public class TestSolr4Spatial extends SolrTestCaseJ4 {
     int docId = 1;
     int count = 1;
 
-    String score = random().nextBoolean() ? "none" : "distance";//never actually need the score but lets test
+    String score = randomScoreMode();//never actually need the score but lets test
     assertQ(req(
         "fl", "id", "q","*:*", "rows", "1000",    // testing quotes in range too
-        "fq", "{! score="+score+" df="+fieldName+"}[32,-80 TO \"33 , -79\"]"),//lower-left to upper-right
+        "fq", "{! "+(score==null?"":" score="+score)+" df="+fieldName+"}[32,-80 TO \"33 , -79\"]"),//lower-left to upper-right
 
         "//result/doc/*[@name='id'][.='" + docId + "']",
         "*[count(//doc)=" + count + "]");
@@ -243,13 +252,46 @@ public class TestSolr4Spatial extends SolrTestCaseJ4 {
 
   @Test
   public void testSort() throws Exception {
+    assumeTrue("dist sorting not supported on field " + fieldName, canCalcDistance);
     assertU(adoc("id", "100", fieldName, "1,2"));
     assertU(adoc("id", "101", fieldName, "4,-1"));
-    assertU(adoc("id", "999", fieldName, "70,70"));//far away from these queries
+    if (random().nextBoolean()) {
+      assertU(commit()); // new segment
+    }
+    if (random().nextBoolean()) {
+      assertU(adoc("id", "999", fieldName, "70,70"));//far away from these queries; we filter it out
+    } else {
+      assertU(adoc("id", "999")); // no data
+    }
     assertU(commit());
 
-    //test absence of score=distance means it doesn't score
 
+    // geodist asc
+    assertJQ(req(
+        "q", radiusQuery(3, 4, 9, null, null),
+        "fl","id",
+        "sort","geodist() asc",
+        "sfield", fieldName, "pt", "3,4")
+        , 1e-3
+        , "/response/docs/[0]/id=='100'"
+        , "/response/docs/[1]/id=='101'"
+    );
+    // geodist desc  (simply reverse the assertions)
+    assertJQ(req(
+        "q", radiusQuery(3, 4, 9, null, null),
+        "fl","id",
+        "sort","geodist() desc", // DESC
+        "sfield", fieldName, "pt", "3,4")
+        , 1e-3
+        , "/response/docs/[0]/id=='101'" // FLIPPED
+        , "/response/docs/[1]/id=='100'" // FLIPPED
+    );
+
+    //
+    //  NOTE: the rest work via the score of the spatial query. Generally, you should use geodist() instead.
+    //
+
+    //test absence of score=distance means it doesn't score
     assertJQ(req(
         "q", radiusQuery(3, 4, 9, null, null),
         "fl","id,score")
@@ -345,7 +387,8 @@ public class TestSolr4Spatial extends SolrTestCaseJ4 {
 
   @Test
   public void testSortMultiVal() throws Exception {
-    RandomizedTest.assumeFalse("Multivalue not supported for this field",
+    assumeTrue("dist sorting not supported on field " + fieldName, canCalcDistance);
+    assumeFalse("Multivalue not supported for this field",
         fieldName.equals("pointvector") || fieldName.equals("bbox"));
 
     assertU(adoc("id", "100", fieldName, "1,2"));//1 point

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/182c20c4/solr/server/solr/configsets/basic_configs/conf/managed-schema
----------------------------------------------------------------------
diff --git a/solr/server/solr/configsets/basic_configs/conf/managed-schema b/solr/server/solr/configsets/basic_configs/conf/managed-schema
index 60a0e98..22553d8 100644
--- a/solr/server/solr/configsets/basic_configs/conf/managed-schema
+++ b/solr/server/solr/configsets/basic_configs/conf/managed-schema
@@ -148,9 +148,6 @@
     <dynamicField name="*_d"  type="double" indexed="true"  stored="true"/>
     <dynamicField name="*_ds" type="doubles" indexed="true"  stored="true"/>
 
-    <!-- Type used to index the lat and lon components for the "location" FieldType -->
-    <dynamicField name="*_coordinate"  type="tdouble" indexed="true"  stored="false" useDocValuesAsStored="false" />
-
     <dynamicField name="*_dt"  type="date"    indexed="true"  stored="true"/>
     <dynamicField name="*_dts" type="date"    indexed="true"  stored="true" multiValued="true"/>
     <dynamicField name="*_p"  type="location" indexed="true" stored="true"/>
@@ -551,8 +548,8 @@
     <dynamicField name="*_point" type="point"  indexed="true"  stored="true"/>
     <fieldType name="point" class="solr.PointType" dimension="2" subFieldSuffix="_d"/>
 
-    <!-- A specialized field for geospatial search. If indexed, this fieldType must not be multivalued. -->
-    <fieldType name="location" class="solr.LatLonType" subFieldSuffix="_coordinate"/>
+    <!-- A specialized field for geospatial search filters and distance sorting. -->
+    <fieldType name="location" class="solr.LatLonPointSpatialField" docValues="true"/>
 
     <!-- An alternative geospatial field type new to Solr 4.  It supports multiValued and polygon shapes.
       For more information about this and other Spatial fields new to Solr 4, see:

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/182c20c4/solr/server/solr/configsets/data_driven_schema_configs/conf/managed-schema
----------------------------------------------------------------------
diff --git a/solr/server/solr/configsets/data_driven_schema_configs/conf/managed-schema b/solr/server/solr/configsets/data_driven_schema_configs/conf/managed-schema
index b1373d8..558c05e 100644
--- a/solr/server/solr/configsets/data_driven_schema_configs/conf/managed-schema
+++ b/solr/server/solr/configsets/data_driven_schema_configs/conf/managed-schema
@@ -148,9 +148,6 @@
     <dynamicField name="*_d"  type="double" indexed="true"  stored="true"/>
     <dynamicField name="*_ds" type="doubles" indexed="true"  stored="true"/>
 
-    <!-- Type used to index the lat and lon components for the "location" FieldType -->
-    <dynamicField name="*_coordinate"  type="tdouble" indexed="true"  stored="false" useDocValuesAsStored="false" />
-
     <dynamicField name="*_dt"  type="date"    indexed="true"  stored="true"/>
     <dynamicField name="*_dts" type="date"    indexed="true"  stored="true" multiValued="true"/>
     <dynamicField name="*_p"  type="location" indexed="true" stored="true"/>
@@ -551,8 +548,8 @@
     <dynamicField name="*_point" type="point"  indexed="true"  stored="true"/>
     <fieldType name="point" class="solr.PointType" dimension="2" subFieldSuffix="_d"/>
 
-    <!-- A specialized field for geospatial search. If indexed, this fieldType must not be multivalued. -->
-    <fieldType name="location" class="solr.LatLonType" subFieldSuffix="_coordinate"/>
+    <!-- A specialized field for geospatial search filters and distance sorting. -->
+    <fieldType name="location" class="solr.LatLonPointSpatialField" docValues="true"/>
 
     <!-- An alternative geospatial field type new to Solr 4.  It supports multiValued and polygon shapes.
       For more information about this and other Spatial fields new to Solr 4, see:

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/182c20c4/solr/server/solr/configsets/sample_techproducts_configs/conf/managed-schema
----------------------------------------------------------------------
diff --git a/solr/server/solr/configsets/sample_techproducts_configs/conf/managed-schema b/solr/server/solr/configsets/sample_techproducts_configs/conf/managed-schema
index 4980540..bd292a0 100644
--- a/solr/server/solr/configsets/sample_techproducts_configs/conf/managed-schema
+++ b/solr/server/solr/configsets/sample_techproducts_configs/conf/managed-schema
@@ -212,9 +212,6 @@
    <dynamicField name="*_d"  type="double" indexed="true"  stored="true"/>
    <dynamicField name="*_ds" type="double" indexed="true"  stored="true"  multiValued="true"/>
 
-   <!-- Type used to index the lat and lon components for the "location" FieldType -->
-   <dynamicField name="*_coordinate"  type="tdouble" indexed="true"  stored="false" useDocValuesAsStored="false" />
-
    <dynamicField name="*_dt"  type="date"    indexed="true"  stored="true"/>
    <dynamicField name="*_dts" type="date"    indexed="true"  stored="true" multiValued="true"/>
    <dynamicField name="*_p"  type="location" indexed="true" stored="true"/>
@@ -696,8 +693,8 @@
      -->
     <fieldType name="point" class="solr.PointType" dimension="2" subFieldSuffix="_d"/>
 
-    <!-- A specialized field for geospatial search. If indexed, this fieldType must not be multivalued. -->
-    <fieldType name="location" class="solr.LatLonType" subFieldSuffix="_coordinate"/>
+    <!-- A specialized field for geospatial search filters and distance sorting. -->
+    <fieldType name="location" class="solr.LatLonPointSpatialField" docValues="true"/>
 
     <!-- An alternative geospatial field type new to Solr 4.  It supports multiValued and polygon shapes.
       For more information about this and other Spatial fields new to Solr 4, see: