diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt index b3aab3df53b..0f12f453a71 100644 --- a/lucene/CHANGES.txt +++ b/lucene/CHANGES.txt @@ -159,7 +159,11 @@ New Features * LUCENE-5437: ASCIIFoldingFilter now has an option to preserve the original token and emit it on the same position as the folded token only if the actual token was - folded. (Simon Willnauer, Nik Everett) + folded. (Simon Willnauer, Nik Everett) + +* LUCENE-5408: Add spatial SerializedDVStrategy that serializes a binary + representations of a shape into BinaryDocValues. It supports exact geometry + relationship calculations. (David Smiley) Build diff --git a/lucene/spatial/src/java/org/apache/lucene/spatial/serialized/SerializedDVStrategy.java b/lucene/spatial/src/java/org/apache/lucene/spatial/serialized/SerializedDVStrategy.java new file mode 100644 index 00000000000..6fe58739ce2 --- /dev/null +++ b/lucene/spatial/src/java/org/apache/lucene/spatial/serialized/SerializedDVStrategy.java @@ -0,0 +1,291 @@ +package org.apache.lucene.spatial.serialized; + +/* + * 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.io.BinaryCodec; +import com.spatial4j.core.shape.Point; +import com.spatial4j.core.shape.Shape; +import org.apache.lucene.document.BinaryDocValuesField; +import org.apache.lucene.document.Field; +import org.apache.lucene.index.AtomicReaderContext; +import org.apache.lucene.index.BinaryDocValues; +import org.apache.lucene.queries.function.FunctionValues; +import org.apache.lucene.queries.function.ValueSource; +import org.apache.lucene.search.DocIdSet; +import org.apache.lucene.search.DocIdSetIterator; +import org.apache.lucene.search.Explanation; +import org.apache.lucene.search.Filter; +import org.apache.lucene.search.Query; +import org.apache.lucene.spatial.SpatialStrategy; +import org.apache.lucene.spatial.query.SpatialArgs; +import org.apache.lucene.spatial.util.DistanceToShapeValueSource; +import org.apache.lucene.spatial.util.ShapePredicateValueSource; +import org.apache.lucene.util.Bits; +import org.apache.lucene.util.BytesRef; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.FilterOutputStream; +import java.io.IOException; +import java.util.Map; + + +/** + * A SpatialStrategy based on serializing a Shape stored into BinaryDocValues. + * This is not at all fast; it's designed to be used in conjuction with another index based + * SpatialStrategy that is approximated (like {@link org.apache.lucene.spatial.prefix.RecursivePrefixTreeStrategy}) + * to add precision or eventually make more specific / advanced calculations on the per-document + * geometry. + * The serialization uses Spatial4j's {@link com.spatial4j.core.io.BinaryCodec}. + * + * @lucene.experimental + */ +public class SerializedDVStrategy extends SpatialStrategy { + + /** + * A cache heuristic for the buf size based on the last shape size. + */ + //TODO do we make this non-volatile since it's merely a heuristic? + private volatile int indexLastBufSize = 8 * 1024;//8KB default on first run + + /** + * Constructs the spatial strategy with its mandatory arguments. + */ + public SerializedDVStrategy(SpatialContext ctx, String fieldName) { + super(ctx, fieldName); + } + + @Override + public Field[] createIndexableFields(Shape shape) { + int bufSize = Math.max(128, (int) (this.indexLastBufSize * 1.5));//50% headroom over last + ByteArrayOutputStream byteStream = new ByteArrayOutputStream(bufSize); + final BytesRef bytesRef = new BytesRef();//receiver of byteStream's bytes + try { + ctx.getBinaryCodec().writeShape(new DataOutputStream(byteStream), shape); + //this is a hack to avoid redundant byte array copying by byteStream.toByteArray() + byteStream.writeTo(new FilterOutputStream(null/*not used*/) { + @Override + public void write(byte[] b, int off, int len) throws IOException { + bytesRef.bytes = b; + bytesRef.offset = off; + bytesRef.length = len; + } + }); + } catch (IOException e) { + throw new RuntimeException(e); + } + this.indexLastBufSize = bytesRef.length;//cache heuristic + return new Field[]{new BinaryDocValuesField(getFieldName(), bytesRef)}; + } + + @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); + } + + @Override + public Query makeQuery(SpatialArgs args) { + throw new UnsupportedOperationException("This strategy can't return a query that operates" + + " efficiently. Instead try a Filter or ValueSource."); + } + + /** + * Returns a Filter that should be used with {@link org.apache.lucene.search.FilteredQuery#QUERY_FIRST_FILTER_STRATEGY}. + * Use in another manner is likely to result in an {@link java.lang.UnsupportedOperationException} + * to prevent misuse because the filter can't efficiently work via iteration. + */ + @Override + public Filter makeFilter(final SpatialArgs args) { + ValueSource shapeValueSource = makeShapeValueSource(); + ShapePredicateValueSource predicateValueSource = new ShapePredicateValueSource( + shapeValueSource, args.getOperation(), args.getShape()); + return new PredicateValueSourceFilter(predicateValueSource); + } + + /** + * Provides access to each shape 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 ShapeDocValueSource(getFieldName(), ctx.getBinaryCodec()); + } + + /** This filter only supports returning a DocSet with a bits(). If you try to grab the + * iterator then you'll get an UnsupportedOperationException. + */ + static class PredicateValueSourceFilter extends Filter { + private final ValueSource predicateValueSource;//we call boolVal(doc) + + public PredicateValueSourceFilter(ValueSource predicateValueSource) { + this.predicateValueSource = predicateValueSource; + } + + @Override + public DocIdSet getDocIdSet(final AtomicReaderContext context, final Bits acceptDocs) throws IOException { + return new DocIdSet() { + @Override + public DocIdSetIterator iterator() throws IOException { + throw new UnsupportedOperationException( + "Iteration is too slow; instead try FilteredQuery.QUERY_FIRST_FILTER_STRATEGY"); + //Note that if you're truly bent on doing this, then see FunctionValues.getRangeScorer + } + + @Override + public Bits bits() throws IOException { + //null Map context -- we simply don't have one. That's ok. + final FunctionValues predFuncValues = predicateValueSource.getValues(null, context); + + return new Bits() { + + @Override + public boolean get(int index) { + if (acceptDocs != null && !acceptDocs.get(index)) + return false; + return predFuncValues.boolVal(index); + } + + @Override + public int length() { + return context.reader().maxDoc(); + } + }; + } + }; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + PredicateValueSourceFilter that = (PredicateValueSourceFilter) o; + + if (!predicateValueSource.equals(that.predicateValueSource)) return false; + + return true; + } + + @Override + public int hashCode() { + return predicateValueSource.hashCode(); + } + }//PredicateValueSourceFilter + + /** + * Implements a ValueSource by deserializing a Shape in from BinaryDocValues using BinaryCodec. + * @see #makeShapeValueSource() + */ + static class ShapeDocValueSource extends ValueSource { + + private final String fieldName; + private final BinaryCodec binaryCodec;//spatial4j + + private ShapeDocValueSource(String fieldName, BinaryCodec binaryCodec) { + this.fieldName = fieldName; + this.binaryCodec = binaryCodec; + } + + @Override + public FunctionValues getValues(Map context, AtomicReaderContext readerContext) throws IOException { + final BinaryDocValues docValues = readerContext.reader().getBinaryDocValues(fieldName); + + return new FunctionValues() { + int bytesRefDoc = -1; + BytesRef bytesRef = new BytesRef();//scratch + + boolean fillBytes(int doc) { + if (bytesRefDoc != doc) { + docValues.get(doc, bytesRef); + bytesRefDoc = doc; + } + return bytesRef.length != 0; + } + + @Override + public boolean exists(int doc) { + return fillBytes(doc); + } + + @Override + public boolean bytesVal(int doc, BytesRef target) { + if (fillBytes(doc)) { + target.bytes = bytesRef.bytes; + target.offset = bytesRef.offset; + target.length = bytesRef.length; + return true; + } else { + target.length = 0; + return false; + } + } + + @Override + public Object objectVal(int docId) { + if (!fillBytes(docId)) + return null; + DataInputStream dataInput = new DataInputStream( + new ByteArrayInputStream(bytesRef.bytes, bytesRef.offset, bytesRef.length)); + try { + return binaryCodec.readShape(dataInput); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + @Override + public Explanation explain(int doc) { + return new Explanation(Float.NaN, toString(doc)); + } + + @Override + public String toString(int doc) { + return description() + "=" + objectVal(doc);//TODO truncate? + } + + }; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + ShapeDocValueSource that = (ShapeDocValueSource) o; + + if (!fieldName.equals(that.fieldName)) return false; + + return true; + } + + @Override + public int hashCode() { + int result = fieldName.hashCode(); + return result; + } + + @Override + public String description() { + return "shapeDocVal(" + fieldName + ")"; + } + }//ShapeDocValueSource +} diff --git a/lucene/spatial/src/java/org/apache/lucene/spatial/serialized/package.html b/lucene/spatial/src/java/org/apache/lucene/spatial/serialized/package.html new file mode 100644 index 00000000000..18199eb8f44 --- /dev/null +++ b/lucene/spatial/src/java/org/apache/lucene/spatial/serialized/package.html @@ -0,0 +1,23 @@ + + + + +
+Strategies that serialize the shape (non-indexed). + + \ No newline at end of file diff --git a/lucene/spatial/src/java/org/apache/lucene/spatial/util/DistanceToShapeValueSource.java b/lucene/spatial/src/java/org/apache/lucene/spatial/util/DistanceToShapeValueSource.java new file mode 100644 index 00000000000..b6ac073aa27 --- /dev/null +++ b/lucene/spatial/src/java/org/apache/lucene/spatial/util/DistanceToShapeValueSource.java @@ -0,0 +1,118 @@ +package org.apache.lucene.spatial.util; + +/* + * 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.distance.DistanceCalculator; +import com.spatial4j.core.shape.Point; +import com.spatial4j.core.shape.Shape; +import org.apache.lucene.index.AtomicReaderContext; +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.Explanation; +import org.apache.lucene.search.IndexSearcher; + +import java.io.IOException; +import java.util.Map; + +/** + * The distance from a provided Point to a Point retrieved from a ValueSource via + * {@link org.apache.lucene.queries.function.FunctionValues#objectVal(int)}. The distance + * is calculated via a {@link com.spatial4j.core.distance.DistanceCalculator}. + * + * @lucene.experimental + */ +public class DistanceToShapeValueSource extends ValueSource { + private final ValueSource shapeValueSource; + private final Point queryPoint; + private final double multiplier; + private final DistanceCalculator distCalc; + + //TODO if FunctionValues returns NaN; will things be ok? + private final double nullValue;//computed + + public DistanceToShapeValueSource(ValueSource shapeValueSource, Point queryPoint, + double multiplier, SpatialContext ctx) { + this.shapeValueSource = shapeValueSource; + this.queryPoint = queryPoint; + this.multiplier = multiplier; + this.distCalc = ctx.getDistCalc(); + this.nullValue = + (ctx.isGeo() ? 180 * multiplier : Double.MAX_VALUE); + } + + @Override + public String description() { + return "distance(" + queryPoint + " to " + shapeValueSource.description() + ")*" + multiplier + ")"; + } + + @Override + public void createWeight(Map context, IndexSearcher searcher) throws IOException { + shapeValueSource.createWeight(context, searcher); + } + + @Override + public FunctionValues getValues(Map context, AtomicReaderContext readerContext) throws IOException { + final FunctionValues shapeValues = shapeValueSource.getValues(context, readerContext); + + return new DoubleDocValues(this) { + @Override + public double doubleVal(int doc) { + Shape shape = (Shape) shapeValues.objectVal(doc); + if (shape == null || shape.isEmpty()) + return nullValue; + Point pt = shape.getCenter(); + return distCalc.distance(queryPoint, pt) * multiplier; + } + + @Override + public Explanation explain(int doc) { + Explanation exp = super.explain(doc); + exp.addDetail(shapeValues.explain(doc)); + return exp; + } + }; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + DistanceToShapeValueSource that = (DistanceToShapeValueSource) o; + + if (!queryPoint.equals(that.queryPoint)) return false; + if (Double.compare(that.multiplier, multiplier) != 0) return false; + if (!shapeValueSource.equals(that.shapeValueSource)) return false; + if (!distCalc.equals(that.distCalc)) return false; + + return true; + } + + @Override + public int hashCode() { + int result; + long temp; + result = shapeValueSource.hashCode(); + result = 31 * result + queryPoint.hashCode(); + temp = Double.doubleToLongBits(multiplier); + result = 31 * result + (int) (temp ^ (temp >>> 32)); + return result; + } +} diff --git a/lucene/spatial/src/java/org/apache/lucene/spatial/util/ShapePredicateValueSource.java b/lucene/spatial/src/java/org/apache/lucene/spatial/util/ShapePredicateValueSource.java new file mode 100644 index 00000000000..e5c7265958f --- /dev/null +++ b/lucene/spatial/src/java/org/apache/lucene/spatial/util/ShapePredicateValueSource.java @@ -0,0 +1,109 @@ +package org.apache.lucene.spatial.util; + +/* + * 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.shape.Shape; +import org.apache.lucene.index.AtomicReaderContext; +import org.apache.lucene.queries.function.FunctionValues; +import org.apache.lucene.queries.function.ValueSource; +import org.apache.lucene.queries.function.docvalues.BoolDocValues; +import org.apache.lucene.search.Explanation; +import org.apache.lucene.search.IndexSearcher; +import org.apache.lucene.spatial.query.SpatialOperation; + +import java.io.IOException; +import java.util.Map; + +/** + * A boolean ValueSource that compares a shape from a provided ValueSource with a given Shape and sees + * if it matches a given {@link SpatialOperation} (the predicate). + * + * @lucene.experimental + */ +public class ShapePredicateValueSource extends ValueSource { + private final ValueSource shapeValuesource;//the left hand side + private final SpatialOperation op; + private final Shape queryShape;//the right hand side (constant) + + /** + * + * @param shapeValuesource Must yield {@link Shape} instances from it's objectVal(doc). If null + * then the result is false. This is the left-hand (indexed) side. + * @param op the predicate + * @param queryShape The shape on the right-hand (query) side. + */ + public ShapePredicateValueSource(ValueSource shapeValuesource, SpatialOperation op, Shape queryShape) { + this.shapeValuesource = shapeValuesource; + this.op = op; + this.queryShape = queryShape; + } + + @Override + public String description() { + return shapeValuesource + " " + op + " " + queryShape; + } + + @Override + public void createWeight(Map context, IndexSearcher searcher) throws IOException { + shapeValuesource.createWeight(context, searcher); + } + + @Override + public FunctionValues getValues(Map context, AtomicReaderContext readerContext) throws IOException { + final FunctionValues shapeValues = shapeValuesource.getValues(context, readerContext); + + return new BoolDocValues(this) { + @Override + public boolean boolVal(int doc) { + Shape indexedShape = (Shape) shapeValues.objectVal(doc); + if (indexedShape == null) + return false; + return op.evaluate(indexedShape, queryShape); + } + + @Override + public Explanation explain(int doc) { + Explanation exp = super.explain(doc); + exp.addDetail(shapeValues.explain(doc)); + return exp; + } + }; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + ShapePredicateValueSource that = (ShapePredicateValueSource) o; + + if (!shapeValuesource.equals(that.shapeValuesource)) return false; + if (!op.equals(that.op)) return false; + if (!queryShape.equals(that.queryShape)) return false; + + return true; + } + + @Override + public int hashCode() { + int result = shapeValuesource.hashCode(); + result = 31 * result + op.hashCode(); + result = 31 * result + queryShape.hashCode(); + return result; + } +} diff --git a/lucene/spatial/src/test/org/apache/lucene/spatial/DistanceStrategyTest.java b/lucene/spatial/src/test/org/apache/lucene/spatial/DistanceStrategyTest.java index d8ce841086f..be214c1e99a 100644 --- a/lucene/spatial/src/test/org/apache/lucene/spatial/DistanceStrategyTest.java +++ b/lucene/spatial/src/test/org/apache/lucene/spatial/DistanceStrategyTest.java @@ -29,6 +29,7 @@ import org.apache.lucene.spatial.prefix.TermQueryPrefixTreeStrategy; import org.apache.lucene.spatial.prefix.tree.GeohashPrefixTree; import org.apache.lucene.spatial.prefix.tree.QuadPrefixTree; import org.apache.lucene.spatial.prefix.tree.SpatialPrefixTree; +import org.apache.lucene.spatial.serialized.SerializedDVStrategy; import org.apache.lucene.spatial.vector.PointVectorStrategy; import org.junit.Test; @@ -61,6 +62,9 @@ public class DistanceStrategyTest extends StrategyTestCase { strategy = new BBoxStrategy(ctx, "bbox"); ctorArgs.add(new Object[]{new Param(strategy)}); + strategy = new SerializedDVStrategy(ctx, "serialized"); + ctorArgs.add(new Object[]{new Param(strategy)}); + return ctorArgs; } @@ -88,9 +92,12 @@ public class DistanceStrategyTest extends StrategyTestCase { @Test public void testDistanceOrder() throws IOException { - adoc("100", ctx.makePoint(2,1)); - adoc("101", ctx.makePoint(-1,4)); + adoc("100", ctx.makePoint(2, 1)); + adoc("101", ctx.makePoint(-1, 4)); adoc("103", (Shape)null);//test score for nothing + adoc("999", ctx.makePoint(2, 1));//test deleted + commit(); + deleteDoc("999"); commit(); //FYI distances are in docid order checkDistValueSource(ctx.makePoint(4, 3), 2.8274937f, 5.0898066f, 180f); @@ -104,6 +111,9 @@ public class DistanceStrategyTest extends StrategyTestCase { Point p101 = ctx.makePoint(-1, 4); adoc("101", p101); adoc("103", (Shape)null);//test score for nothing + adoc("999", ctx.makePoint(2, 1));//test deleted + commit(); + deleteDoc("999"); commit(); double dist = ctx.getDistCalc().distance(p100, p101); diff --git a/lucene/spatial/src/test/org/apache/lucene/spatial/QueryEqualsHashCodeTest.java b/lucene/spatial/src/test/org/apache/lucene/spatial/QueryEqualsHashCodeTest.java index 92bfa5126fd..913e88f60d6 100644 --- a/lucene/spatial/src/test/org/apache/lucene/spatial/QueryEqualsHashCodeTest.java +++ b/lucene/spatial/src/test/org/apache/lucene/spatial/QueryEqualsHashCodeTest.java @@ -22,10 +22,12 @@ import com.spatial4j.core.shape.Shape; import org.apache.lucene.spatial.bbox.BBoxStrategy; import org.apache.lucene.spatial.prefix.RecursivePrefixTreeStrategy; import org.apache.lucene.spatial.prefix.TermQueryPrefixTreeStrategy; +import org.apache.lucene.spatial.prefix.tree.GeohashPrefixTree; import org.apache.lucene.spatial.prefix.tree.QuadPrefixTree; import org.apache.lucene.spatial.prefix.tree.SpatialPrefixTree; import org.apache.lucene.spatial.query.SpatialArgs; import org.apache.lucene.spatial.query.SpatialOperation; +import org.apache.lucene.spatial.serialized.SerializedDVStrategy; import org.apache.lucene.spatial.vector.PointVectorStrategy; import org.apache.lucene.util.LuceneTestCase; import org.junit.Test; @@ -40,43 +42,52 @@ public class QueryEqualsHashCodeTest extends LuceneTestCase { @Test public void testEqualsHashCode() { - final SpatialPrefixTree grid = new QuadPrefixTree(ctx,10); - final SpatialArgs args1 = makeArgs1(); - final SpatialArgs args2 = makeArgs2(); + final SpatialPrefixTree gridQuad = new QuadPrefixTree(ctx,10); + final SpatialPrefixTree gridGeohash = new GeohashPrefixTree(ctx,10); - Collection