Skip to content

Commit

Permalink
apply SOLR-3304 patch, w/ fix for LUCENE-4192
Browse files Browse the repository at this point in the history
andyfowler committed Aug 22, 2012
1 parent 286e2ea commit dfae32c
Showing 11 changed files with 1,944 additions and 5 deletions.
Original file line number Diff line number Diff line change
@@ -0,0 +1,192 @@
package org.apache.solr.schema;

/*
* 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.
*/

import com.spatial4j.core.context.SpatialContext;
import com.spatial4j.core.context.SpatialContextFactory;
import com.spatial4j.core.shape.Point;
import com.spatial4j.core.shape.Rectangle;
import com.spatial4j.core.shape.Shape;
import org.apache.lucene.document.StoredField;
import org.apache.lucene.index.IndexableField;
import org.apache.lucene.queries.function.ValueSource;
import org.apache.lucene.search.ConstantScoreQuery;
import org.apache.lucene.search.Filter;
import org.apache.lucene.search.Query;
import org.apache.lucene.search.QueryWrapperFilter;
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.SpatialArgsParser;
import org.apache.lucene.spatial.query.SpatialOperation;
import org.apache.solr.common.SolrException;
import org.apache.solr.common.params.SolrParams;
import org.apache.solr.response.TextResponseWriter;
import org.apache.solr.search.QParser;
import org.apache.solr.util.MapListener;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import java.io.IOException;
import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;

public abstract class AbstractSpatialFieldType<T extends SpatialStrategy> extends FieldType
{
protected final Logger log = LoggerFactory.getLogger( getClass() );

protected SpatialContext ctx;
protected SpatialArgsParser argsParser;

protected boolean ignoreIncompatibleGeometry = false;

private final ConcurrentHashMap<String, T> fieldStrategyMap = new ConcurrentHashMap<String,T>();

@Override
protected void init(IndexSchema schema, Map<String, String> args) {
super.init(schema, args);
String v = args.remove( "ignoreIncompatibleGeometry" );
if( v != null ) {
ignoreIncompatibleGeometry = Boolean.valueOf( v );
}

//Solr expects us to remove the parameters we've used.
MapListener<String, String> argsWrap = new MapListener<String, String>(args);
ctx = SpatialContextFactory.makeSpatialContext(argsWrap, schema.getResourceLoader().getClassLoader());
args.keySet().removeAll(argsWrap.getSeenKeys());

argsParser = new SpatialArgsParser();//might make pluggable some day?
}

//--------------------------------------------------------------
// Indexing
//--------------------------------------------------------------

@Override
public final IndexableField createField(SchemaField field, Object val, float boost) {
throw new IllegalStateException("should be calling createFields because isPolyField() is true");
}

@Override
public final IndexableField[] createFields(SchemaField field, Object val, float boost) {
Shape shape = (val instanceof Shape)?((Shape)val): ctx.readShape( val.toString() );
if( shape == null ) {
log.warn( "Field {}: null shape for input: {}", field, val );
return null;
}

IndexableField[] indexableFields = null;
if (field.indexed()) {
T strategy = fieldStrategyMap.get(field.getName());
//double-checked locking idiom
if (strategy == null) {
synchronized (fieldStrategyMap) {
strategy = fieldStrategyMap.get(field.getName());
if (strategy == null) {
strategy = newSpatialStrategy(field.getName());
fieldStrategyMap.put(field.getName(),strategy);
}
}
}
indexableFields = strategy.createIndexableFields(shape);
}

StoredField storedField = null;
if (field.stored()) {
storedField = new StoredField(field.getName(),ctx.toString(shape));//normalizes the shape
}

if (indexableFields == null) {
if (storedField == null)
return null;
return new IndexableField[]{storedField};
} else {
if (storedField == null)
return indexableFields;
IndexableField[] result = new IndexableField[indexableFields.length+1];
System.arraycopy(indexableFields,0,result,0,indexableFields.length);
result[result.length-1] = storedField;
return result;
}
}

/** Called from {@link #createFields(SchemaField, Object, float)} upon first use by fieldName. } */
protected abstract T newSpatialStrategy(String fieldName);

@Override
public final boolean isPolyField() {
return true;
}

//--------------------------------------------------------------
// Query Support
//--------------------------------------------------------------

@Override
public Query getRangeQuery(QParser parser, SchemaField field, String part1, String part2, boolean minInclusive, boolean maxInclusive) {
if (!minInclusive || !maxInclusive)
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Both sides of range query must be inclusive: " + field.getName());
Shape shape1 = ctx.readShape(part1);
Shape shape2 = ctx.readShape(part2);
if (!(shape1 instanceof Point) || !(shape2 instanceof Point))
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Both sides of range query must be points: " + field.getName());
Point p1 = (Point) shape1;
Point p2 = (Point) shape2;
Rectangle bbox = ctx.makeRect(p1.getX(),p2.getX(),p1.getY(),p2.getY());
SpatialArgs spatialArgs = new SpatialArgs(SpatialOperation.Intersects,bbox);
return getQueryFromSpatialArgs(parser, field, spatialArgs);
}

@Override
public ValueSource getValueSource(SchemaField field, QParser parser) {
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "ValueSource not supported on SpatialField: " + field.getName());
}

@Override
public Query getFieldQuery(QParser parser, SchemaField field, String externalVal) {
return getQueryFromSpatialArgs(parser, field, argsParser.parse(externalVal, ctx));
}

private Query getQueryFromSpatialArgs(QParser parser, SchemaField field, SpatialArgs spatialArgs) {
SpatialStrategy spatialStrategy = fieldStrategyMap.get(field.getName());
//see SOLR-2883 needScore
SolrParams localParams = parser.getLocalParams();
if (localParams == null || localParams.getBool("needScore", true)) {
return spatialStrategy.makeQuery(spatialArgs);
} else {
Filter filter = spatialStrategy.makeFilter(spatialArgs);
if (filter instanceof QueryWrapperFilter) {
QueryWrapperFilter queryWrapperFilter = (QueryWrapperFilter) filter;
return queryWrapperFilter.getQuery();
}
return new ConstantScoreQuery(filter);
}
}

@Override
public void write(TextResponseWriter writer, String name, IndexableField f) throws IOException {
writer.writeStr(name, f.stringValue(), true);
}

@Override
public SortField getSortField(SchemaField field, boolean top) {
throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Sorting not supported on SpatialField: " + field.getName());
}
}


Original file line number Diff line number Diff line change
@@ -0,0 +1,68 @@
package org.apache.solr.schema;

/*
* 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.
*/

import org.apache.lucene.spatial.prefix.PrefixTreeStrategy;
import org.apache.lucene.spatial.prefix.tree.SpatialPrefixTree;
import org.apache.lucene.spatial.prefix.tree.SpatialPrefixTreeFactory;
import org.apache.solr.util.MapListener;

import java.util.Map;

public abstract class AbstractSpatialPrefixTreeFieldType<T extends PrefixTreeStrategy> extends AbstractSpatialFieldType<T> {

protected SpatialPrefixTree grid;
private Double distErrPct;
private Integer defaultFieldValuesArrayLen;

@Override
protected void init(IndexSchema schema, Map<String, String> args) {
super.init(schema, args);

//Solr expects us to remove the parameters we've used.
MapListener<String, String> argsWrap = new MapListener<String, String>(args);
grid = SpatialPrefixTreeFactory.makeSPT(argsWrap, schema.getResourceLoader().getClassLoader(), ctx);
args.keySet().removeAll(argsWrap.getSeenKeys());

String v = args.remove("distErrPct");
if (v != null)
distErrPct = Double.valueOf(v);

v = args.remove("defaultFieldValuesArrayLen");
if (v != null)
defaultFieldValuesArrayLen = Integer.valueOf(v);
}


@Override
protected T newSpatialStrategy(String fieldName) {
@SuppressWarnings("unchecked") T strat = (T) newPrefixTreeStrategy(fieldName);

strat.setIgnoreIncompatibleGeometry( ignoreIncompatibleGeometry );
if (distErrPct != null)
strat.setDistErrPct(distErrPct);
if (defaultFieldValuesArrayLen != null)
strat.setDefaultFieldValuesArrayLen(defaultFieldValuesArrayLen);

log.info(this.toString()+" strat: "+strat+" maxLevels: "+ grid.getMaxLevels());//TODO output maxDetailKm
return strat;
}

protected abstract PrefixTreeStrategy newPrefixTreeStrategy(String fieldName);

}
Original file line number Diff line number Diff line change
@@ -0,0 +1,30 @@
package org.apache.solr.schema;

/*
* 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.
*/

import org.apache.lucene.spatial.prefix.PrefixTreeStrategy;
import org.apache.lucene.spatial.prefix.RecursivePrefixTreeStrategy;

public class SpatialRecursivePrefixTreeFieldType extends AbstractSpatialPrefixTreeFieldType<RecursivePrefixTreeStrategy> {

@Override
protected PrefixTreeStrategy newPrefixTreeStrategy(String fieldName) {
return new RecursivePrefixTreeStrategy(grid, fieldName);
}
}

Original file line number Diff line number Diff line change
@@ -0,0 +1,30 @@
/*
* 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 org.apache.lucene.spatial.prefix.PrefixTreeStrategy;
import org.apache.lucene.spatial.prefix.TermQueryPrefixTreeStrategy;

public class SpatialTermQueryPrefixTreeFieldType extends AbstractSpatialPrefixTreeFieldType<TermQueryPrefixTreeStrategy> {

@Override
protected PrefixTreeStrategy newPrefixTreeStrategy(String fieldName) {
return new TermQueryPrefixTreeStrategy(grid,fieldName);
}
}

Original file line number Diff line number Diff line change
@@ -0,0 +1,48 @@
package org.apache.solr.schema;

/*
* 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.
*/

import org.apache.lucene.spatial.vector.TwoDoublesStrategy;

import java.util.Map;


public class SpatialTwoDoublesFieldType extends AbstractSpatialFieldType<TwoDoublesStrategy> {

private Integer precisionStep;

@Override
protected void init(IndexSchema schema, Map<String, String> args) {
super.init(schema, args);

String v = args.remove("precisionStep");
if (v != null) {
precisionStep = Integer.valueOf(v);
}
}

@Override
protected TwoDoublesStrategy newSpatialStrategy(String fieldName) {
TwoDoublesStrategy strat = new TwoDoublesStrategy(ctx, fieldName);
if (precisionStep != null)
strat.setPrecisionStep(precisionStep);
return strat;
}

}

59 changes: 59 additions & 0 deletions solr/core/src/java/org/apache/solr/util/MapListener.java
Original file line number Diff line number Diff line change
@@ -0,0 +1,59 @@
/*
* 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.util;

import com.google.common.collect.ForwardingMap;

import java.util.HashSet;
import java.util.Map;
import java.util.Set;

/**
* Wraps another map, keeping track of each key that was seen via {@link #get(Object)} or {@link #remove(Object)}.
*/
@SuppressWarnings("unchecked")
public class MapListener<K, V> extends ForwardingMap<K, V> {
private final Map<K, V> target;
private final Set<K> seenKeys;

public MapListener(Map<K, V> target) {
this.target = target;
seenKeys = new HashSet<K>(target.size());
}

public Set<K> getSeenKeys() {
return seenKeys;
}

@Override
public V get(Object key) {
seenKeys.add((K) key);
return super.get(key);
}

@Override
public V remove(Object key) {
seenKeys.add((K) key);
return super.remove(key);
}

@Override
protected Map<K, V> delegate() {
return target;
}
}
59 changes: 59 additions & 0 deletions solr/core/src/test-files/solr/collection1/conf/schema-spatial.xml
Original file line number Diff line number Diff line change
@@ -0,0 +1,59 @@
<?xml version="1.0" ?>
<!--
Licensed to the Apache Software Foundation (ASF) under one or more
contributor license agreements. See the NOTICE file distributed with
this work for additional information regarding copyright ownership.
The ASF licenses this file to You under the Apache License, Version 2.0
(the "License"); you may not use this file except in compliance with
the License. You may obtain a copy of the License at
http://www.apache.org/licenses/LICENSE-2.0
Unless required by applicable law or agreed to in writing, software
distributed under the License is distributed on an "AS IS" BASIS,
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
See the License for the specific language governing permissions and
limitations under the License.
-->

<schema name="test" version="1.4">
<types>

<fieldType name="tint" class="solr.TrieIntField" precisionStep="8" omitNorms="true" positionIncrementGap="0"/>
<fieldType name="tfloat" class="solr.TrieFloatField" precisionStep="8" omitNorms="true" positionIncrementGap="0"/>
<fieldType name="tlong" class="solr.TrieLongField" precisionStep="8" omitNorms="true" positionIncrementGap="0"/>
<fieldType name="tdouble" class="solr.TrieDoubleField" precisionStep="8" omitNorms="true" positionIncrementGap="0"/>

<fieldType name="string" class="solr.StrField" sortMissingLast="true"/>

<fieldType name="srpt_geohash" class="solr.SpatialRecursivePrefixTreeFieldType"
prefixTree="geohash"
/>
<fieldType name="srpt_quad" class="solr.SpatialRecursivePrefixTreeFieldType"
prefixTree="quad"
/>
<fieldType name="srpt_100km" class="solr.SpatialRecursivePrefixTreeFieldType"
maxDetailDist="100"
/>
<fieldType name="stqpt_geohash" class="solr.SpatialTermQueryPrefixTreeFieldType"
prefixTree="geohash" />

<fieldType name="stqpt_u" class="solr.SpatialTermQueryPrefixTreeFieldType"
units="u" distCalculator="cartesian^2" worldBounds="0 0 1000 1000"/>

</types>


<fields>

<field name="id" type="string" required="true"/>

<field name="srpt_geohash" type="srpt_geohash" multiValued="true" />
<field name="srpt_quad" type="srpt_quad" multiValued="true" />
<field name="stqpt_geohash" type="stqpt_geohash" multiValued="true" />

</fields>

<uniqueKey>id</uniqueKey>

</schema>
114 changes: 114 additions & 0 deletions solr/core/src/test/org/apache/solr/search/TestNewSpatialSort.java
Original file line number Diff line number Diff line change
@@ -0,0 +1,114 @@
package org.apache.solr.search;

/*
* 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.
*/

import com.carrotsearch.randomizedtesting.annotations.ParametersFactory;
import org.apache.solr.SolrTestCaseJ4;
import org.junit.BeforeClass;
import org.junit.Test;

import java.util.Arrays;

public class TestNewSpatialSort extends SolrTestCaseJ4
{

private String fieldName;

public TestNewSpatialSort(String fieldName) {
this.fieldName = fieldName;
}

@ParametersFactory
public static Iterable<Object[]> parameters() {
return Arrays.asList(new Object[][]{
{"srpt_geohash"}, {"srpt_quad"}, {"stqpt_geohash"}
});
}

@BeforeClass
public static void beforeClass() throws Exception {
initCore("solrconfig-basic.xml", "schema-spatial.xml");
}

/** Test that queries against a spatial field return the distance as the score. */
@Test
public void directQuery() throws Exception {
assertU(adoc("id", "100", fieldName, "1,2"));
assertU(adoc("id", "101", fieldName, "4,-1"));
assertU(commit());

assertJQ(req(
"q", fieldName +":\"Intersects(Circle(3,4 d=1000))\"",
"fl","id,score",
"sort","score asc")//want ascending due to increasing distance
, 1e-3
, "/response/docs/[0]/id=='100'"
, "/response/docs/[0]/score==314.4033"
, "/response/docs/[1]/id=='101'"
, "/response/docs/[1]/score==565.9615"
);
//query again with the query point closer to #101, and check the new ordering
assertJQ(req(
"q", fieldName +":\"Intersects(Circle(4,0 d=1000))\"",
"fl","id,score",
"sort","score asc")//want ascending due to increasing distance
, 1e-4
, "/response/docs/[0]/id=='101'"
, "/response/docs/[1]/id=='100'"
);

//use sort=query(...)
assertJQ(req(
"q","*:*",
"fl","id,score",
"sort","query($sortQuery) asc", //want ascending due to increasing distance
"sortQuery", fieldName +":\"Intersects(Circle(3,4 d=1000))\"" )
, 1e-4
, "/response/docs/[0]/id=='100'"
, "/response/docs/[1]/id=='101'" );

//check reversed direction with query point closer to #101
assertJQ(req(
"q","*:*",
"fl","id,score",
"sort","query($sortQuery) asc", //want ascending due to increasing distance
"sortQuery", fieldName +":\"Intersects(Circle(4,0 d=1000))\"" )
, 1e-4
, "/response/docs/[0]/id=='101'"
, "/response/docs/[1]/id=='100'" );
}

@Test
public void multiVal() throws Exception {
//RandomizedTest.assumeFalse("Multivalue not supported for this field",fieldName.startsWith("stqpt"));

assertU(adoc("id", "100", fieldName, "1,2"));//1 point
assertU(adoc("id", "101", fieldName, "4,-1", fieldName, "3,5"));//2 points, 2nd is pretty close to query point
assertU(commit());

assertJQ(req(
"q", fieldName +":\"Intersects(Circle(3,4 d=1000))\"",
"fl","id,score",
"sort","score asc")//want ascending due to increasing distance
, 1e-4
, "/response/docs/[0]/id=='101'"
, "/response/docs/[0]/score==111.042725"//dist to 3,5
);
}

}
239 changes: 239 additions & 0 deletions solr/core/src/test/org/apache/solr/search/TestSolr4Spatial.java
Original file line number Diff line number Diff line change
@@ -0,0 +1,239 @@
package org.apache.solr.search;

/*
* 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.
*/

import com.carrotsearch.randomizedtesting.annotations.ParametersFactory;
import org.apache.solr.SolrTestCaseJ4;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Ignore;
import org.junit.Test;

import java.util.Arrays;

/**
* Test Solr 4's new spatial capabilities from the new Lucene spatial module. Don't thoroughly test it here because
* Lucene spatial has its own tests. Some of these tests were ported from Solr 3 spatial tests.
*/
public class TestSolr4Spatial extends SolrTestCaseJ4 {

private String fieldName;

public TestSolr4Spatial(String fieldName) {
this.fieldName = fieldName;
}

@ParametersFactory
public static Iterable<Object[]> parameters() {
return Arrays.asList(new Object[][]{
{"srpt_geohash"}, {"srpt_quad"}, {"stqpt_geohash"}
} );
}

@BeforeClass
public static void beforeClass() throws Exception {
initCore("solrconfig-basic.xml", "schema-spatial.xml");
}

@Before
public void setUp() throws Exception {
super.setUp();
clearIndex();
}

private void setupDocs() {
assertU(adoc("id", "1", fieldName, "32.7693246, -79.9289094"));
assertU(adoc("id", "2", fieldName, "33.7693246, -80.9289094"));
assertU(adoc("id", "3", fieldName, "-32.7693246, 50.9289094"));
assertU(adoc("id", "4", fieldName, "-50.7693246, 60.9289094"));
assertU(adoc("id", "5", fieldName, "0,0"));
assertU(adoc("id", "6", fieldName, "0.1,0.1"));
assertU(adoc("id", "7", fieldName, "-0.1,-0.1"));
assertU(adoc("id", "8", fieldName, "0,179.9"));
assertU(adoc("id", "9", fieldName, "0,-179.9"));
assertU(adoc("id", "10", fieldName, "89.9,50"));
assertU(adoc("id", "11", fieldName, "89.9,-130"));
assertU(adoc("id", "12", fieldName, "-89.9,50"));
assertU(adoc("id", "13", fieldName, "-89.9,-130"));
assertU(commit());
}

@Test
public void testIntersectFilter() throws Exception {
setupDocs();
//Try some edge cases
checkHits(fieldName, "1,1", 175, 3, 5, 6, 7);
checkHits(fieldName, "0,179.8", 200, 2, 8, 9);
checkHits(fieldName, "89.8, 50", 200, 2, 10, 11);//this goes over the north pole
checkHits(fieldName, "-89.8, 50", 200, 2, 12, 13);//this goes over the south pole
//try some normal cases
checkHits(fieldName, "33.0,-80.0", 300, 2);
//large distance
checkHits(fieldName, "1,1", 5000, 3, 5, 6, 7);
//Because we are generating a box based on the west/east longitudes and the south/north latitudes, which then
//translates to a range query, which is slightly more inclusive. Thus, even though 0.0 is 15.725 kms away,
//it will be included, b/c of the box calculation.
checkHits(fieldName, false, "0.1,0.1", 15, 2, 5, 6);

//try some more
clearIndex();
assertU(adoc("id", "14", fieldName, "0,5"));
assertU(adoc("id", "15", fieldName, "0,15"));
//3000KM from 0,0, see http://www.movable-type.co.uk/scripts/latlong.html
assertU(adoc("id", "16", fieldName, "18.71111,19.79750"));
assertU(adoc("id", "17", fieldName, "44.043900,-95.436643"));
assertU(commit());

checkHits(fieldName, "0,0", 1000, 1, 14);
checkHits(fieldName, "0,0", 2000, 2, 14, 15);
checkHits(fieldName, false, "0,0", 3000, 3, 14, 15, 16);
checkHits(fieldName, "0,0", 3001, 3, 14, 15, 16);
checkHits(fieldName, "0,0", 3000.1, 3, 14, 15, 16);

//really fine grained distance and reflects some of the vagaries of how we are calculating the box
checkHits(fieldName, "43.517030,-96.789603", 109, 0);

//falls outside of the real distance, but inside the bounding box
checkHits(fieldName, true, "43.517030,-96.789603", 110, 0);
checkHits(fieldName, false, "43.517030,-96.789603", 110, 1, 17);
}

@Test @Ignore /** nocommit fix in Lucene spatial or spatial4j */
public void checkResultFormat() throws Exception {
//Check lat,lon format in response
String IN = "89.9,-130";//lat,lon
String OUT = "89.900000,-130.000000";//lat,lon padded to 6 fractional digits

assertU(adoc("id", "11", fieldName, IN));
assertU(commit());

assertQ(req(
"fl", "id," + fieldName, "q", "*:*", "rows", "1000",
"fq", "{!field needScore=false f="+fieldName+"}Intersects(Circle(89.9,-130 d=100))"),
"//result/doc/*[@name='" + fieldName + "']//text()='" + OUT + "'");
}

private void checkHits(String fieldName, String pt, double distance, int count, int ... docIds) {
checkHits(fieldName, true, pt, distance, count, docIds);
}

private void checkHits(String fieldName, boolean exact, String ptStr, double distance, int count, int ... docIds) {
String [] tests = new String[docIds != null && docIds.length > 0 ? docIds.length + 1 : 1];
//test for presence of required ids first
int i = 0;
if (docIds != null && docIds.length > 0) {
for (int docId : docIds) {
tests[i++] = "//result/doc/*[@name='id'][.='" + docId + "']";
}
}
//check total length last; maybe response includes ids it shouldn't. Nicer to check this last instead of first so
// that there may be a more specific detailed id to investigate.
tests[i++] = "*[count(//doc)=" + count + "]";

String method = exact ? "IsWithin" : "BBoxWithin";
boolean needScore = random().nextBoolean();//never actually need the score but lets test

//default distPrec=0.025 works with these tests
assertQ(req(
"fl", "id", "q","*:*", "rows", "1000",
"fq", "{!field needScore="+needScore+" f="+fieldName+"}"+method+"(Circle("+ptStr.replaceAll(" ","")+" d="+distance+"))"),
tests);
}

@Test
public void testRangeSyntax() {
setupDocs();
//match docId 1
int docId = 1;
int count = 1;
boolean needScore = random().nextBoolean();//never actually need the score but lets test
assertQ(req(
"fl", "id", "q","*:*", "rows", "1000",
"fq", "{! needScore="+needScore+" df="+fieldName+"}[32,-80 TO 33,-79]"),//lower-left to upper-right

"//result/doc/*[@name='id'][.='" + docId + "']",
"*[count(//doc)=" + count + "]");
}

/** Test that queries against a spatial field return the distance as the score. */
@Test
public void testSort() throws Exception {
assertU(adoc("id", "100", fieldName, "1,2"));
assertU(adoc("id", "101", fieldName, "4,-1"));
assertU(commit());

assertJQ(req(
"q", fieldName +":\"Intersects(Circle(3,4 d=1000))\"",
"fl","id,score",
"sort","score asc")//want ascending due to increasing distance
, 1e-3
, "/response/docs/[0]/id=='100'"
, "/response/docs/[0]/score==314.4033"
, "/response/docs/[1]/id=='101'"
, "/response/docs/[1]/score==565.9615"
);
//query again with the query point closer to #101, and check the new ordering
assertJQ(req(
"q", fieldName +":\"Intersects(Circle(4,0 d=1000))\"",
"fl","id,score",
"sort","score asc")//want ascending due to increasing distance
, 1e-4
, "/response/docs/[0]/id=='101'"
, "/response/docs/[1]/id=='100'"
);

//use sort=query(...)
assertJQ(req(
"q","*:*",
"fl","id,score",
"sort","query($sortQuery) asc", //want ascending due to increasing distance
"sortQuery", fieldName +":\"Intersects(Circle(3,4 d=1000))\"" )
, 1e-4
, "/response/docs/[0]/id=='100'"
, "/response/docs/[1]/id=='101'" );

//check reversed direction with query point closer to #101
assertJQ(req(
"q","*:*",
"fl","id,score",
"sort","query($sortQuery) asc", //want ascending due to increasing distance
"sortQuery", fieldName +":\"Intersects(Circle(4,0 d=1000))\"" )
, 1e-4
, "/response/docs/[0]/id=='101'"
, "/response/docs/[1]/id=='100'" );
}

@Test
public void testSortMultiVal() throws Exception {
//RandomizedTest.assumeFalse("Multivalue not supported for this field",fieldName.startsWith("stqpt"));

assertU(adoc("id", "100", fieldName, "1,2"));//1 point
assertU(adoc("id", "101", fieldName, "4,-1", fieldName, "3,5"));//2 points, 2nd is pretty close to query point
assertU(commit());

assertJQ(req(
"q", fieldName +":\"Intersects(Circle(3,4 d=1000))\"",
"fl","id,score",
"sort","score asc")//want ascending due to increasing distance
, 1e-4
, "/response/docs/[0]/id=='101'"
, "/response/docs/[0]/score==111.042725"//dist to 3,5
);
}

}
11 changes: 6 additions & 5 deletions solr/example/solr/collection1/conf/schema.xml
Original file line number Diff line number Diff line change
@@ -675,11 +675,12 @@
<!-- A specialized field for geospatial search. If indexed, this fieldType must not be multivalued. -->
<fieldType name="location" class="solr.LatLonType" subFieldSuffix="_coordinate"/>

<!--
A Geohash is a compact representation of a latitude longitude pair in a single field.
See http://wiki.apache.org/solr/SpatialSearch
-->
<fieldtype name="geohash" class="solr.GeoHashField"/>
<!-- 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://wiki.apache.org/solr/SolrAdaptersForLuceneSpatial4
-->
<fieldType name="location_2d_trie" class="solr.SpatialRecursivePrefixTreeFieldType"
distErrPct="0.025" maxDetailDist="0.001" />

<!-- Money/currency field type. See http://wiki.apache.org/solr/MoneyFieldType
Parameters:
1,099 changes: 1,099 additions & 0 deletions solr/example/solr/collection1/conf/schema.xml.orig

Large diffs are not rendered by default.

0 comments on commit dfae32c

Please sign in to comment.