LUCENE-4942: Optimize spatial PrefixTreeStrategy indexes for non-point data

* Redundant prefix terms on leaf cells are no longer emitted;
* Cell.isLeaf should always return true at maxLevels

git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1665656 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
David Wayne Smiley 2015-03-10 19:25:24 +00:00
parent 37ba6f6a27
commit f3013524b2
24 changed files with 595 additions and 442 deletions

View File

@ -138,6 +138,10 @@ Optimizations
* LUCENE-6320: Speed up CheckIndex. (Robert Muir)
* LUCENE-4942: Optimized the encoding of PrefixTreeStrategy indexes for
non-point data: 33% smaller index, 68% faster indexing, and 44% faster
searching. YMMV (David Smiley)
API Changes
* LUCENE-6204, LUCENE-6208: Simplify CompoundFormat: remove files()

View File

@ -21,6 +21,7 @@ import java.io.IOException;
import java.util.Iterator;
import com.spatial4j.core.shape.Shape;
import com.spatial4j.core.shape.SpatialRelation;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.search.DocIdSet;
@ -53,13 +54,11 @@ public abstract class AbstractVisitingPrefixTreeFilter extends AbstractPrefixTre
// least it would just make things more complicated.
protected final int prefixGridScanLevel;//at least one less than grid.getMaxLevels()
protected final boolean hasIndexedLeaves;
public AbstractVisitingPrefixTreeFilter(Shape queryShape, String fieldName, SpatialPrefixTree grid,
int detailLevel, int prefixGridScanLevel, boolean hasIndexedLeaves) {
int detailLevel, int prefixGridScanLevel) {
super(queryShape, fieldName, grid, detailLevel);
this.prefixGridScanLevel = Math.max(0, Math.min(prefixGridScanLevel, grid.getMaxLevels() - 1));
this.hasIndexedLeaves = hasIndexedLeaves;
assert detailLevel <= grid.getMaxLevels();
}
@ -90,7 +89,7 @@ public abstract class AbstractVisitingPrefixTreeFilter extends AbstractPrefixTre
* which by default finds the top cells that intersect {@code queryShape}. If
* there isn't an indexed cell for a corresponding cell returned for this
* method then it's short-circuited until it finds one, at which point
* {@link #visit(org.apache.lucene.spatial.prefix.tree.Cell)} is called. At
* {@link #visitPrefix(org.apache.lucene.spatial.prefix.tree.Cell)} is called. At
* some depths, of the tree, the algorithm switches to a scanning mode that
* calls {@link #visitScanned(org.apache.lucene.spatial.prefix.tree.Cell)}
* for each leaf cell found.
@ -114,10 +113,6 @@ public abstract class AbstractVisitingPrefixTreeFilter extends AbstractPrefixTre
does act as a short-circuit. So maybe do some percent of the time or when the level
is above some threshold.
* Once we don't have redundant non-leaves indexed with leaf cells (LUCENE-4942), we can
sometimes know to call next() instead of seek() if we're processing a leaf cell that
didn't have a corresponding non-leaf.
*/
//
@ -125,10 +120,10 @@ public abstract class AbstractVisitingPrefixTreeFilter extends AbstractPrefixTre
//
private VNode curVNode;//current pointer, derived from query shape
private BytesRef curVNodeTerm = new BytesRef();//curVNode.cell's term, without leaf
private Cell scanCell;
private BytesRef curVNodeTerm = new BytesRef();//curVNode.cell's term, without leaf. in main loop only
private BytesRef thisTerm;//the result of termsEnum.term()
private Cell indexedCell;//Cell wrapper of thisTerm. Always updated when thisTerm is.
public VisitorTemplate(LeafReaderContext context, Bits acceptDocs) throws IOException {
super(context, acceptDocs);
@ -138,9 +133,9 @@ public abstract class AbstractVisitingPrefixTreeFilter extends AbstractPrefixTre
assert curVNode == null : "Called more than once?";
if (termsEnum == null)
return null;
//advance
if ((thisTerm = termsEnum.next()) == null)
return null; // all done
if (!nextTerm()) {//advances
return null;
}
curVNode = new VNode(null);
curVNode.reset(grid.getWorldCell());
@ -177,30 +172,46 @@ public abstract class AbstractVisitingPrefixTreeFilter extends AbstractPrefixTre
}
//Seek to curVNode's cell (or skip if termsEnum has moved beyond)
curVNode.cell.getTokenBytesNoLeaf(curVNodeTerm);
int compare = thisTerm.compareTo(curVNodeTerm);
final int compare = indexedCell.compareToNoLeaf(curVNode.cell);
if (compare > 0) {
// leap frog (termsEnum is beyond where we would otherwise seek)
assert ! context.reader().terms(fieldName).iterator(null).seekExact(curVNodeTerm) : "should be absent";
} else {
if (compare < 0) {
// Seek !
TermsEnum.SeekStatus seekStatus = termsEnum.seekCeil(curVNodeTerm);
if (seekStatus == TermsEnum.SeekStatus.END)
break; // all done
thisTerm = termsEnum.term();
if (seekStatus == TermsEnum.SeekStatus.NOT_FOUND) {
continue; // leap frog
}
}
// Visit!
boolean descend = visit(curVNode.cell);
//advance
if ((thisTerm = termsEnum.next()) == null)
// The indexed cell is after; continue loop to next query cell
continue;
}
if (compare < 0) {
// The indexed cell is before; seek ahead to query cell:
// Seek !
curVNode.cell.getTokenBytesNoLeaf(curVNodeTerm);
TermsEnum.SeekStatus seekStatus = termsEnum.seekCeil(curVNodeTerm);
if (seekStatus == TermsEnum.SeekStatus.END)
break; // all done
if (descend)
addIntersectingChildren();
thisTerm = termsEnum.term();
indexedCell = grid.readCell(thisTerm, indexedCell);
if (seekStatus == TermsEnum.SeekStatus.NOT_FOUND) {
// Did we find a leaf of the cell we were looking for or something after?
if (!indexedCell.isLeaf() || indexedCell.compareToNoLeaf(curVNode.cell) != 0)
continue; // The indexed cell is after; continue loop to next query cell
}
}
// indexedCell == queryCell (disregarding leaf).
// If indexedCell is a leaf then there's no prefix (prefix sorts before) -- just visit and continue
if (indexedCell.isLeaf()) {
visitLeaf(indexedCell);//TODO or query cell? Though shouldn't matter.
if (!nextTerm()) break;
continue;
}
// If a prefix (non-leaf) then visit; see if we descend.
final boolean descend = visitPrefix(curVNode.cell);//need to use curVNode.cell not indexedCell
if (!nextTerm()) break;
// Check for adjacent leaf with the same prefix
if (indexedCell.isLeaf() && indexedCell.getLevel() == curVNode.cell.getLevel()) {
visitLeaf(indexedCell);//TODO or query cell? Though shouldn't matter.
if (!nextTerm()) break;
}
if (descend) {
addIntersectingChildren();
}
}//main loop
@ -208,7 +219,7 @@ public abstract class AbstractVisitingPrefixTreeFilter extends AbstractPrefixTre
return finish();
}
/** Called initially, and whenever {@link #visit(org.apache.lucene.spatial.prefix.tree.Cell)}
/** Called initially, and whenever {@link #visitPrefix(org.apache.lucene.spatial.prefix.tree.Cell)}
* returns true. */
private void addIntersectingChildren() throws IOException {
assert thisTerm != null;
@ -216,20 +227,6 @@ public abstract class AbstractVisitingPrefixTreeFilter extends AbstractPrefixTre
if (cell.getLevel() >= detailLevel)
throw new IllegalStateException("Spatial logic error");
//Check for adjacent leaf (happens for indexed non-point shapes)
if (hasIndexedLeaves && cell.getLevel() != 0) {
//If the next indexed term just adds a leaf marker to cell,
// then add all of those docs
scanCell = grid.readCell(thisTerm, scanCell);
assert curVNode.cell.isPrefixOf(scanCell) : "missing leaf or descendants";
if (scanCell.getLevel() == cell.getLevel() && scanCell.isLeaf()) {
visitLeaf(scanCell);
//advance
if ((thisTerm = termsEnum.next()) == null)
return; // all done
}
}
//Decide whether to continue to divide & conquer, or whether it's time to
// scan through terms beneath this cell.
// Scanning is a performance optimization trade-off.
@ -269,22 +266,22 @@ public abstract class AbstractVisitingPrefixTreeFilter extends AbstractPrefixTre
* #visitScanned(org.apache.lucene.spatial.prefix.tree.Cell)}.
*/
protected void scan(int scanDetailLevel) throws IOException {
for ( ;
thisTerm != null;
thisTerm = termsEnum.next()) {
scanCell = grid.readCell(thisTerm, scanCell);
if (!curVNode.cell.isPrefixOf(scanCell))
break;
int termLevel = scanCell.getLevel();
if (termLevel < scanDetailLevel) {
if (scanCell.isLeaf())
visitScanned(scanCell);
} else if (termLevel == scanDetailLevel) {
if (!scanCell.isLeaf())//LUCENE-5529
visitScanned(scanCell);
//note: this can be a do-while instead in 6x; 5x has a back-compat with redundant leaves -- LUCENE-4942
while (curVNode.cell.isPrefixOf(indexedCell)) {
if (indexedCell.getLevel() == scanDetailLevel
|| (indexedCell.getLevel() < scanDetailLevel && indexedCell.isLeaf())) {
visitScanned(indexedCell);
}
}//term loop
//advance
if (!nextTerm()) break;
}
}
private boolean nextTerm() throws IOException {
if ((thisTerm = termsEnum.next()) == null)
return false;
indexedCell = grid.readCell(thisTerm, indexedCell);
return true;
}
/** Used for {@link VNode#children}. */
@ -322,23 +319,21 @@ public abstract class AbstractVisitingPrefixTreeFilter extends AbstractPrefixTre
protected abstract DocIdSet finish() throws IOException;
/**
* Visit an indexed non-leaf cell returned from
* {@link #findSubCellsToVisit(org.apache.lucene.spatial.prefix.tree.Cell)}
* that is also found in the index.
* It will also be called by the default implementation of
* {@link #visitScanned(org.apache.lucene.spatial.prefix.tree.Cell)} for
* cells at the bottom detail level.
* Visit an indexed non-leaf cell. The presence of a prefix cell implies
* there are leaf cells at further levels. The cell passed should have it's
* {@link org.apache.lucene.spatial.prefix.tree.Cell#getShapeRel()} set
* relative to the filtered shape.
*
* @param cell An intersecting cell; not a leaf.
* @return true to descend to more levels. It is an error to return true
* if cell.level == detailLevel
* @return true to descend to more levels.
*/
protected abstract boolean visit(Cell cell) throws IOException;
protected abstract boolean visitPrefix(Cell cell) throws IOException;
/**
* Called when an indexed leaf cell is found. An
* indexed leaf cell means associated documents generally won't be found at
* further detail levels.
* indexed leaf cell usually means associated documents won't be found at
* further detail levels. However, if a document has
* multiple overlapping shapes at different resolutions, then this isn't true.
*/
protected abstract void visitLeaf(Cell cell) throws IOException;
@ -347,14 +342,16 @@ public abstract class AbstractVisitingPrefixTreeFilter extends AbstractPrefixTre
* might not even intersect the query shape, so be sure to check for that.
* The default implementation will check that and if passes then call
* {@link #visitLeaf(org.apache.lucene.spatial.prefix.tree.Cell)} or
* {@link #visit(org.apache.lucene.spatial.prefix.tree.Cell)}.
* {@link #visitPrefix(org.apache.lucene.spatial.prefix.tree.Cell)}.
*/
protected void visitScanned(Cell cell) throws IOException {
if (queryShape.relate(cell.getShape()).intersects()) {
final SpatialRelation relate = cell.getShape().relate(queryShape);
if (relate.intersects()) {
cell.setShapeRel(relate);//just being pedantic
if (cell.isLeaf()) {
visitLeaf(cell);
} else {
visit(cell);
visitPrefix(cell);
}
}
}

View File

@ -0,0 +1,157 @@
package org.apache.lucene.spatial.prefix;
/*
* 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 java.io.IOException;
import org.apache.lucene.analysis.TokenStream;
import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
import org.apache.lucene.analysis.tokenattributes.TermToBytesRefAttribute;
import org.apache.lucene.util.Attribute;
import org.apache.lucene.util.AttributeFactory;
import org.apache.lucene.util.AttributeImpl;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefIterator;
/**
* A TokenStream used internally by {@link org.apache.lucene.spatial.prefix.PrefixTreeStrategy}.
*
* This is modelled after {@link org.apache.lucene.analysis.NumericTokenStream}.
*
* @lucene.internal
*/
class BytesRefIteratorTokenStream extends TokenStream {
// just a wrapper to prevent adding CharTermAttribute
private static final class BRAttributeFactory extends AttributeFactory {
private final AttributeFactory delegate;
BRAttributeFactory(AttributeFactory delegate) {
this.delegate = delegate;
}
@Override
public AttributeImpl createAttributeInstance(Class<? extends Attribute> attClass) {
if (CharTermAttribute.class.isAssignableFrom(attClass))
throw new IllegalArgumentException(getClass() + " does not support CharTermAttribute.");
return delegate.createAttributeInstance(attClass);
}
}
private static final class BRTermToBytesRefAttributeImpl extends AttributeImpl
implements TermToBytesRefAttribute {
private final BytesRef bytes = new BytesRef();
void setBytesRef(BytesRef inputBytes) {
// shallow clone. this.bytesRef is final
bytes.bytes = inputBytes.bytes;
bytes.offset = inputBytes.offset;
bytes.length = inputBytes.length;
}
@Override
public void clear() {
// we keep it untouched as it's fully controlled by the outer class.
}
@Override
public void copyTo(AttributeImpl target) {
final BRTermToBytesRefAttributeImpl a = (BRTermToBytesRefAttributeImpl) target;
a.setBytesRef(BytesRef.deepCopyOf(bytes));
}
@Override
public void fillBytesRef() {
//nothing to do; it's populated by incrementToken
}
@Override
public BytesRef getBytesRef() {
return bytes;
}
@Override
public BRTermToBytesRefAttributeImpl clone() {
final BRTermToBytesRefAttributeImpl clone = (BRTermToBytesRefAttributeImpl) super.clone();
clone.setBytesRef(BytesRef.deepCopyOf(bytes));
return clone;
}
@Override
public int hashCode() {
return bytes.hashCode();
}
@Override
public boolean equals(Object obj) {
if (this == obj) return true;
if (obj == null) return false;
if (getClass() != obj.getClass()) return false;
BRTermToBytesRefAttributeImpl other = (BRTermToBytesRefAttributeImpl) obj;
if (!bytes.equals(other.bytes)) return false;
return true;
}
}
public BytesRefIteratorTokenStream() {
super(new BRAttributeFactory(AttributeFactory.DEFAULT_ATTRIBUTE_FACTORY));
addAttributeImpl(new BRTermToBytesRefAttributeImpl());//because non-public constructor
bytesAtt = (BRTermToBytesRefAttributeImpl) addAttribute(TermToBytesRefAttribute.class);
}
public BytesRefIterator getBytesRefIterator() {
return bytesIter;
}
public BytesRefIteratorTokenStream setBytesRefIterator(BytesRefIterator iter) {
this.bytesIter = iter;
return this;
}
@Override
public void reset() throws IOException {
if (bytesIter == null)
throw new IllegalStateException("call setBytesRefIterator() before usage");
bytesAtt.getBytesRef().length = 0;
}
@Override
public final boolean incrementToken() throws IOException {
if (bytesIter == null)
throw new IllegalStateException("call setBytesRefIterator() before usage");
// this will only clear all other attributes in this TokenStream
clearAttributes();//TODO but there should be no "other" attributes
// get next
BytesRef bytes = bytesIter.next();
if (bytes == null) {
return false;
} else {
bytesAtt.setBytesRef(bytes);
//note: we don't bother setting posInc or type attributes. There's no point to it.
return true;
}
}
//members
private final BRTermToBytesRefAttributeImpl bytesAtt;
private BytesRefIterator bytesIter = null; // null means not initialized
}

View File

@ -0,0 +1,50 @@
package org.apache.lucene.spatial.prefix;
/*
* 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 java.util.Iterator;
import org.apache.lucene.spatial.prefix.tree.Cell;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefIterator;
/**
* A reset'able {@link org.apache.lucene.util.BytesRefIterator} wrapper around
* an {@link java.util.Iterator} of {@link org.apache.lucene.spatial.prefix.tree.Cell}s.
*
* @see PrefixTreeStrategy#newCellToBytesRefIterator()
*
* @lucene.internal
*/
public class CellToBytesRefIterator implements BytesRefIterator {
protected Iterator<Cell> cellIter;
protected BytesRef bytesRef = new BytesRef();
public void reset(Iterator<Cell> cellIter) {
this.cellIter = cellIter;
}
@Override
public BytesRef next() {
if (!cellIter.hasNext()) {
return null;
}
return cellIter.next().getTokenBytesWithLeaf(bytesRef);
}
}

View File

@ -1,208 +0,0 @@
package org.apache.lucene.spatial.prefix;
/*
* 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.analysis.TokenStream;
import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
import org.apache.lucene.analysis.tokenattributes.TermToBytesRefAttribute;
import org.apache.lucene.spatial.prefix.tree.Cell;
import org.apache.lucene.util.Attribute;
import org.apache.lucene.util.AttributeFactory;
import org.apache.lucene.util.AttributeImpl;
import org.apache.lucene.util.AttributeReflector;
import org.apache.lucene.util.BytesRef;
import java.io.IOException;
import java.util.Iterator;
import java.util.Objects;
/**
* A TokenStream used internally by {@link org.apache.lucene.spatial.prefix.PrefixTreeStrategy}.
*
* This is highly modelled after {@link org.apache.lucene.analysis.NumericTokenStream}.
*
* If there is demand for it to be public; it could be made to be.
*
* @lucene.internal
*/
class CellTokenStream extends TokenStream {
private interface CellTermAttribute extends Attribute {
Cell getCell();
void setCell(Cell cell);
//TODO one day deprecate this once we have better encodings
boolean getOmitLeafByte();
void setOmitLeafByte(boolean b);
}
// just a wrapper to prevent adding CTA
private static final class CellAttributeFactory extends AttributeFactory {
private final AttributeFactory delegate;
CellAttributeFactory(AttributeFactory delegate) {
this.delegate = delegate;
}
@Override
public AttributeImpl createAttributeInstance(Class<? extends Attribute> attClass) {
if (CharTermAttribute.class.isAssignableFrom(attClass))
throw new IllegalArgumentException("CellTokenStream does not support CharTermAttribute.");
return delegate.createAttributeInstance(attClass);
}
}
private static final class CellTermAttributeImpl extends AttributeImpl
implements CellTermAttribute, TermToBytesRefAttribute {
private BytesRef bytes = new BytesRef();
private Cell cell;
private boolean omitLeafByte;//false by default (whether there's a leaf byte or not)
@Override
public Cell getCell() {
return cell;
}
@Override
public boolean getOmitLeafByte() {
return omitLeafByte;
}
@Override
public void setCell(Cell cell) {
this.cell = cell;
omitLeafByte = false;//reset
}
@Override
public void setOmitLeafByte(boolean b) {
omitLeafByte = b;
}
@Override
public void clear() {
// this attribute has no contents to clear!
// we keep it untouched as it's fully controlled by outer class.
}
@Override
public void copyTo(AttributeImpl target) {
final CellTermAttribute a = (CellTermAttribute) target;
a.setCell(cell);
a.setOmitLeafByte(omitLeafByte);
}
@Override
public void fillBytesRef() {
if (omitLeafByte)
cell.getTokenBytesNoLeaf(bytes);
else
cell.getTokenBytesWithLeaf(bytes);
}
@Override
public BytesRef getBytesRef() {
return bytes;
}
@Override
public void reflectWith(AttributeReflector reflector) {
fillBytesRef();
reflector.reflect(TermToBytesRefAttribute.class, "bytes", BytesRef.deepCopyOf(bytes));
}
@Override
public CellTermAttributeImpl clone() {
final CellTermAttributeImpl clone = (CellTermAttributeImpl) super.clone();
clone.bytes = BytesRef.deepCopyOf(bytes);
return clone;
}
@Override
public int hashCode() {
return Objects.hash(cell, omitLeafByte);
}
@Override
public boolean equals(Object obj) {
if (this == obj) return true;
if (obj == null) return false;
if (getClass() != obj.getClass()) return false;
CellTermAttributeImpl other = (CellTermAttributeImpl) obj;
if (cell == null) {
if (other.cell != null) return false;
} else if (!cell.equals(other.cell)) return false;
if (omitLeafByte != other.omitLeafByte) return false;
return true;
}
}
public CellTokenStream() {
this(AttributeFactory.DEFAULT_ATTRIBUTE_FACTORY);
}
public CellTokenStream(AttributeFactory factory) {
super(new CellAttributeFactory(factory));
}
public CellTokenStream setCells(Iterator<Cell> iter) {
this.iter = iter;
return this;
}
@Override
public void reset() throws IOException {
if (iter == null)
throw new IllegalStateException("call setCells() before usage");
cellAtt.setCell(null);
cellAtt.setOmitLeafByte(false);
}
/** Outputs the token of a cell, and if it's a leaf, outputs it again with the leaf byte. */
@Override
public final boolean incrementToken() {
if (iter == null)
throw new IllegalStateException("call setCells() before usage");
// this will only clear all other attributes in this TokenStream
clearAttributes();
if (cellAtt.getOmitLeafByte()) {
cellAtt.setOmitLeafByte(false);
return true;
}
//get next
if (!iter.hasNext())
return false;
cellAtt.setCell(iter.next());
if (cellAtt.getCell().isLeaf())
cellAtt.setOmitLeafByte(true);
return true;
}
{
addAttributeImpl(new CellTermAttributeImpl());//because non-public constructor
}
//members
private final CellTermAttribute cellAtt = addAttribute(CellTermAttribute.class);
//TODO support position increment, and TypeAttribute
private Iterator<Cell> iter = null; // null means not initialized
}

View File

@ -17,11 +17,14 @@ package org.apache.lucene.spatial.prefix;
* limitations under the License.
*/
import java.io.IOException;
import java.util.Arrays;
import com.spatial4j.core.shape.Shape;
import com.spatial4j.core.shape.SpatialRelation;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.index.PostingsEnum;
import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.search.DocIdSet;
import org.apache.lucene.search.DocIdSetIterator;
import org.apache.lucene.spatial.prefix.tree.Cell;
@ -32,9 +35,6 @@ import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.RamUsageEstimator;
import org.apache.lucene.util.SentinelIntSet;
import java.io.IOException;
import java.util.Arrays;
/**
* Finds docs where its indexed shape {@link org.apache.lucene.spatial.query.SpatialOperation#Contains
* CONTAINS} the query shape. For use on {@link RecursivePrefixTreeStrategy}.
@ -43,11 +43,6 @@ import java.util.Arrays;
*/
public class ContainsPrefixTreeFilter extends AbstractPrefixTreeFilter {
/*
Future optimizations:
Instead of seekExact, use seekCeil with some leap-frogging, like Intersects does.
*/
/**
* If the spatial data for a document is comprised of multiple overlapping or adjacent parts,
* it might fail to match a query shape when doing the CONTAINS predicate when the sum of
@ -77,8 +72,8 @@ public class ContainsPrefixTreeFilter extends AbstractPrefixTreeFilter {
@Override
public String toString(String field) {
return "ContainsPrefixTreeFilter(" +
// TODO: print something about the shape?
"fieldName=" + fieldName + "," +
"queryShape=" + queryShape + "," +
"detailLevel=" + detailLevel + "," +
"multiOverlappingIndexedShapes=" + multiOverlappingIndexedShapes +
")";
@ -93,21 +88,21 @@ public class ContainsPrefixTreeFilter extends AbstractPrefixTreeFilter {
public ContainsVisitor(LeafReaderContext context, Bits acceptDocs) throws IOException {
super(context, acceptDocs);
if (termsEnum != null) {
nextTerm();//advance to first
}
}
//The reused value of cell.getTokenBytesNoLeaf which is always then seek()'ed to. It's used in assertions too.
BytesRef termBytes = new BytesRef();//no leaf
Cell nextCell;//see getLeafDocs
BytesRef seekTerm = new BytesRef();//temp; see seek()
BytesRef thisTerm;//current term in termsEnum
Cell indexedCell;//the cell wrapper around thisTerm
/** This is the primary algorithm; recursive. Returns null if finds none. */
private SmallDocSet visit(Cell cell, Bits acceptContains) throws IOException {
if (termsEnum == null)//signals all done
if (thisTerm == null)//signals all done
return null;
// Leaf docs match all query shape
SmallDocSet leafDocs = getLeafDocs(cell, acceptContains);
// Get the AND of all child results (into combinedSubResults)
SmallDocSet combinedSubResults = null;
// Optimization: use null subCellsFilter when we know cell is within the query shape.
@ -119,68 +114,105 @@ public class ContainsPrefixTreeFilter extends AbstractPrefixTreeFilter {
CellIterator subCells = cell.getNextLevelCells(subCellsFilter);
while (subCells.hasNext()) {
Cell subCell = subCells.next();
if (!seekExact(subCell))
if (!seek(subCell)) {
combinedSubResults = null;
else if (subCell.getLevel() == detailLevel)
} else if (subCell.getLevel() == detailLevel) {
combinedSubResults = getDocs(subCell, acceptContains);
else if (!multiOverlappingIndexedShapes &&
subCell.getShapeRel() == SpatialRelation.WITHIN)
} else if (!multiOverlappingIndexedShapes &&
subCell.getShapeRel() == SpatialRelation.WITHIN) {
combinedSubResults = getLeafDocs(subCell, acceptContains);
else
combinedSubResults = visit(subCell, acceptContains); //recursion
} else {
//OR the leaf docs with all child results
SmallDocSet leafDocs = getLeafDocs(subCell, acceptContains);
SmallDocSet subDocs = visit(subCell, acceptContains); //recursion
combinedSubResults = union(leafDocs, subDocs);
}
if (combinedSubResults == null)
break;
acceptContains = combinedSubResults;//has the 'AND' effect on next iteration
}
// Result: OR the leaf docs with AND of all child results
if (combinedSubResults != null) {
if (leafDocs == null)
return combinedSubResults;
return leafDocs.union(combinedSubResults);//union is 'or'
}
return leafDocs;
return combinedSubResults;
}
private boolean seekExact(Cell cell) throws IOException {
assert cell.getTokenBytesNoLeaf(null).compareTo(termBytes) > 0;
if (termsEnum == null)
private boolean seek(Cell cell) throws IOException {
if (thisTerm == null)
return false;
termBytes = cell.getTokenBytesNoLeaf(termBytes);
assert assertCloneTermBytes(); //assertions look at termBytes later on
return termsEnum.seekExact(termBytes);
}
private boolean assertCloneTermBytes() {
termBytes = BytesRef.deepCopyOf(termBytes);
return true;
final int compare = indexedCell.compareToNoLeaf(cell);
if (compare > 0) {
return false;//leap-frog effect
} else if (compare == 0) {
return true; // already there!
} else {//compare > 0
//seek!
seekTerm = cell.getTokenBytesNoLeaf(seekTerm);
final TermsEnum.SeekStatus seekStatus = termsEnum.seekCeil(seekTerm);
if (seekStatus == TermsEnum.SeekStatus.END) {
thisTerm = null;//all done
return false;
}
thisTerm = termsEnum.term();
indexedCell = grid.readCell(thisTerm, indexedCell);
if (seekStatus == TermsEnum.SeekStatus.FOUND) {
return true;
}
return indexedCell.isLeaf() && indexedCell.compareToNoLeaf(cell) == 0;
}
}
/** Get prefix & leaf docs at this cell. */
private SmallDocSet getDocs(Cell cell, Bits acceptContains) throws IOException {
assert cell.getTokenBytesNoLeaf(null).equals(termBytes);
return collectDocs(acceptContains);
assert indexedCell.compareToNoLeaf(cell) == 0;
//called when we've reached detailLevel.
if (indexedCell.isLeaf()) {//only a leaf
SmallDocSet result = collectDocs(acceptContains);
nextTerm();
return result;
} else {
SmallDocSet docsAtPrefix = collectDocs(acceptContains);
if (!nextTerm()) {
return docsAtPrefix;
}
//collect leaf too
if (indexedCell.isLeaf() && indexedCell.compareToNoLeaf(cell) == 0) {
SmallDocSet docsAtLeaf = collectDocs(acceptContains);
nextTerm();
return union(docsAtPrefix, docsAtLeaf);
} else {
return docsAtPrefix;
}
}
}
/** Gets docs on the leaf of the given cell, _if_ there is a leaf cell, otherwise null. */
private SmallDocSet getLeafDocs(Cell cell, Bits acceptContains) throws IOException {
assert cell.getTokenBytesNoLeaf(null).equals(termBytes);
assert indexedCell.compareToNoLeaf(cell) == 0;
//Advance past prefix if we're at a prefix; return null if no leaf
if (!indexedCell.isLeaf()) {
if (!nextTerm() || !indexedCell.isLeaf() || indexedCell.getLevel() != cell.getLevel()) {
return null;
}
}
SmallDocSet result = collectDocs(acceptContains);
nextTerm();
return result;
}
if (termsEnum == null)
return null;
BytesRef nextTerm = termsEnum.next();
if (nextTerm == null) {
termsEnum = null;//signals all done
return null;
}
nextCell = grid.readCell(nextTerm, nextCell);
assert cell.isPrefixOf(nextCell);
if (nextCell.getLevel() == cell.getLevel() && nextCell.isLeaf()) {
return collectDocs(acceptContains);
} else {
return null;
private boolean nextTerm() throws IOException {
if ((thisTerm = termsEnum.next()) == null)
return false;
indexedCell = grid.readCell(thisTerm, indexedCell);
return true;
}
private SmallDocSet union(SmallDocSet aSet, SmallDocSet bSet) {
if (bSet != null) {
if (aSet == null)
return bSet;
return aSet.union(bSet);//union is 'or'
}
return aSet;
}
private SmallDocSet collectDocs(Bits acceptContains) throws IOException {

View File

@ -39,8 +39,8 @@ public class IntersectsPrefixTreeFilter extends AbstractVisitingPrefixTreeFilter
public IntersectsPrefixTreeFilter(Shape queryShape, String fieldName,
SpatialPrefixTree grid, int detailLevel,
int prefixGridScanLevel, boolean hasIndexedLeaves) {
super(queryShape, fieldName, grid, detailLevel, prefixGridScanLevel, hasIndexedLeaves);
int prefixGridScanLevel) {
super(queryShape, fieldName, grid, detailLevel, prefixGridScanLevel);
}
@Override
@ -68,7 +68,7 @@ public class IntersectsPrefixTreeFilter extends AbstractVisitingPrefixTreeFilter
}
@Override
protected boolean visit(Cell cell) throws IOException {
protected boolean visitPrefix(Cell cell) throws IOException {
if (cell.getShapeRel() == SpatialRelation.WITHIN || cell.getLevel() == detailLevel) {
collectDocs(results);
return false;
@ -87,11 +87,10 @@ public class IntersectsPrefixTreeFilter extends AbstractVisitingPrefixTreeFilter
@Override
public String toString(String field) {
return "IntersectsPrefixTreeFilter(" +
// TODO: print something about the shape?
"fieldName=" + fieldName + "," +
"queryShape=" + queryShape + "," +
"detailLevel=" + detailLevel + "," +
"prefixGridScanLevel=" + prefixGridScanLevel + "," +
"hasIndexedLeaves=" + hasIndexedLeaves +
"prefixGridScanLevel=" + prefixGridScanLevel +
")";
}

View File

@ -19,14 +19,13 @@ package org.apache.lucene.spatial.prefix;
import java.io.IOException;
import java.util.Arrays;
import java.util.Iterator;
import java.util.Map;
import java.util.SortedMap;
import java.util.TreeMap;
import com.spatial4j.core.shape.Point;
import com.spatial4j.core.shape.Shape;
import org.apache.lucene.analysis.TokenStream;
import org.apache.lucene.document.Field;
import org.apache.lucene.index.IndexReaderContext;
import org.apache.lucene.queries.function.ValueSource;
import org.apache.lucene.search.Filter;
@ -59,11 +58,9 @@ public class NumberRangePrefixTreeStrategy extends RecursivePrefixTreeStrategy {
}
@Override
public Field[] createIndexableFields(Shape shape) {
protected Iterator<Cell> createCellIteratorToIndex(Shape shape, int detailLevel, Iterator<Cell> reuse) {
//levels doesn't actually matter; NumberRange based Shapes have their own "level".
TokenStream tokenStream = createTokenStream(shape, grid.getMaxLevels());
Field field = new Field(getFieldName(), tokenStream, FIELD_TYPE);
return new Field[]{field};
return super.createCellIteratorToIndex(shape, grid.getMaxLevels(), reuse);
}
/** Unsupported. */

View File

@ -24,7 +24,8 @@ import org.apache.lucene.spatial.util.ShapeFieldCacheProvider;
import org.apache.lucene.util.BytesRef;
/**
* Implementation of {@link ShapeFieldCacheProvider} designed for {@link PrefixTreeStrategy}s that index points.
* Implementation of {@link ShapeFieldCacheProvider} designed for {@link PrefixTreeStrategy}s that index points
* (AND ONLY POINTS!).
*
* @lucene.internal
*/
@ -40,8 +41,8 @@ public class PointPrefixTreeFieldCacheProvider extends ShapeFieldCacheProvider<P
@Override
protected Point readShape(BytesRef term) {
scanCell = grid.readCell(term, scanCell);;
if (scanCell.getLevel() == grid.getMaxLevels() && !scanCell.isLeaf())//points are never flagged as leaf
scanCell = grid.readCell(term, scanCell);
if (scanCell.getLevel() == grid.getMaxLevels())
return scanCell.getShape().getCenter();
return null;
}

View File

@ -20,9 +20,9 @@ package org.apache.lucene.spatial.prefix;
import java.io.IOException;
import com.spatial4j.core.shape.Shape;
import org.apache.lucene.index.PostingsEnum;
import org.apache.lucene.index.IndexReaderContext;
import org.apache.lucene.index.LeafReaderContext;
import org.apache.lucene.index.PostingsEnum;
import org.apache.lucene.search.DocIdSet;
import org.apache.lucene.search.DocIdSetIterator;
import org.apache.lucene.search.Filter;
@ -33,6 +33,17 @@ import org.apache.lucene.util.SparseFixedBitSet;
/**
* Computes facets on cells for {@link org.apache.lucene.spatial.prefix.PrefixTreeStrategy}.
* <p>
* <em>NOTE:</em> If for a given document and a given field using
* {@link org.apache.lucene.spatial.prefix.RecursivePrefixTreeStrategy}
* multiple values are indexed (i.e. multi-valued) and at least one of them is a non-point, then there is a possibility
* of double-counting the document in the facet results. Since each shape is independently turned into grid cells at
* a resolution chosen by the shape's size, it's possible they will be indexed at different resolutions. This means
* the document could be present in BOTH the postings for a cell in both its prefix and leaf variants. To avoid this,
* use a single valued field with a {@link com.spatial4j.core.shape.ShapeCollection} (or WKT equivalent). Or
* calculate a suitable level/distErr to index both and call
* {@link org.apache.lucene.spatial.prefix.PrefixTreeStrategy#createIndexableFields(com.spatial4j.core.shape.Shape, int)}
* with the same value for all shapes for a given document/field.
*
* @lucene.experimental
*/
@ -114,8 +125,7 @@ public class PrefixTreeFacetCounter {
//AbstractVisitingPrefixTreeFilter is a Lucene Filter. We don't need a filter; we use it for its great prefix-tree
// traversal code. TODO consider refactoring if/when it makes sense (more use cases than this)
new AbstractVisitingPrefixTreeFilter(queryShape, strategy.getFieldName(), tree, facetLevel, scanLevel,
!strategy.isPointsOnly()) {
new AbstractVisitingPrefixTreeFilter(queryShape, strategy.getFieldName(), tree, facetLevel, scanLevel) {
@Override
public String toString(String field) {
@ -139,7 +149,7 @@ public class PrefixTreeFacetCounter {
}
@Override
protected boolean visit(Cell cell) throws IOException {
protected boolean visitPrefix(Cell cell) throws IOException {
// At facetLevel...
if (cell.getLevel() == facetLevel) {
// Count docs

View File

@ -24,7 +24,6 @@ import java.util.concurrent.ConcurrentHashMap;
import com.spatial4j.core.shape.Point;
import com.spatial4j.core.shape.Shape;
import org.apache.lucene.analysis.TokenStream;
import org.apache.lucene.document.Field;
import org.apache.lucene.document.FieldType;
import org.apache.lucene.index.IndexOptions;
@ -127,8 +126,8 @@ public abstract class PrefixTreeStrategy extends SpatialStrategy {
return pointsOnly;
}
/** True if only indexed points shall be supported. There are no "leafs" in such a case. See
* {@link org.apache.lucene.spatial.prefix.IntersectsPrefixTreeFilter#hasIndexedLeaves}. */
/** True if only indexed points shall be supported. There are no "leafs" in such a case, except those
* at maximum precision. */
public void setPointsOnly(boolean pointsOnly) {
this.pointsOnly = pointsOnly;
}
@ -142,23 +141,33 @@ public abstract class PrefixTreeStrategy extends SpatialStrategy {
/**
* Turns {@link SpatialPrefixTree#getTreeCellIterator(Shape, int)} into a
* {@link org.apache.lucene.analysis.TokenStream}.
* {@code simplifyIndexedCells} is an optional hint affecting non-point shapes: it will
* simply/aggregate sets of complete leaves in a cell to its parent, resulting in ~20-25%
* fewer cells. It will likely be removed in the future.
*/
public Field[] createIndexableFields(Shape shape, double distErr) {
int detailLevel = grid.getLevelForDistance(distErr);
TokenStream tokenStream = createTokenStream(shape, detailLevel);
return createIndexableFields(shape, detailLevel);
}
public Field[] createIndexableFields(Shape shape, int detailLevel) {
//TODO re-use TokenStream LUCENE-5776: Subclass Field, put cell iterator there, override tokenStream()
Iterator<Cell> cells = createCellIteratorToIndex(shape, detailLevel, null);
CellToBytesRefIterator cellToBytesRefIterator = newCellToBytesRefIterator();
cellToBytesRefIterator.reset(cells);
BytesRefIteratorTokenStream tokenStream = new BytesRefIteratorTokenStream();
tokenStream.setBytesRefIterator(cellToBytesRefIterator);
Field field = new Field(getFieldName(), tokenStream, FIELD_TYPE);
return new Field[]{field};
}
protected TokenStream createTokenStream(Shape shape, int detailLevel) {
protected CellToBytesRefIterator newCellToBytesRefIterator() {
//subclasses could return one that never emits leaves, or does both, or who knows.
return new CellToBytesRefIterator();
}
protected Iterator<Cell> createCellIteratorToIndex(Shape shape, int detailLevel, Iterator<Cell> reuse) {
if (pointsOnly && !(shape instanceof Point)) {
throw new IllegalArgumentException("pointsOnly is true yet a " + shape.getClass() + " is given for indexing");
}
Iterator<Cell> cells = grid.getTreeCellIterator(shape, detailLevel);
return new CellTokenStream().setCells(cells);
return grid.getTreeCellIterator(shape, detailLevel);//TODO should take a re-use iterator
}
/* Indexed, tokenized, not stored. */

View File

@ -18,11 +18,11 @@ package org.apache.lucene.spatial.prefix;
*/
import java.util.ArrayList;
import java.util.Iterator;
import java.util.List;
import com.spatial4j.core.shape.Point;
import com.spatial4j.core.shape.Shape;
import org.apache.lucene.analysis.TokenStream;
import org.apache.lucene.search.Filter;
import org.apache.lucene.spatial.prefix.tree.Cell;
import org.apache.lucene.spatial.prefix.tree.CellIterator;
@ -116,13 +116,13 @@ public class RecursivePrefixTreeStrategy extends PrefixTreeStrategy {
}
@Override
protected TokenStream createTokenStream(Shape shape, int detailLevel) {
protected Iterator<Cell> createCellIteratorToIndex(Shape shape, int detailLevel, Iterator<Cell> reuse) {
if (shape instanceof Point || !pruneLeafyBranches)
return super.createTokenStream(shape, detailLevel);
return super.createCellIteratorToIndex(shape, detailLevel, reuse);
List<Cell> cells = new ArrayList<>(4096);
recursiveTraverseAndPrune(grid.getWorldCell(), shape, detailLevel, cells);
return new CellTokenStream().setCells(cells.iterator());
return cells.iterator();
}
/** Returns true if cell was added as a leaf. If it wasn't it recursively descends. */
@ -174,10 +174,10 @@ public class RecursivePrefixTreeStrategy extends PrefixTreeStrategy {
if (op == SpatialOperation.Intersects) {
return new IntersectsPrefixTreeFilter(
shape, getFieldName(), grid, detailLevel, prefixGridScanLevel, !pointsOnly);
shape, getFieldName(), grid, detailLevel, prefixGridScanLevel);
} else if (op == SpatialOperation.IsWithin) {
return new WithinPrefixTreeFilter(
shape, getFieldName(), grid, detailLevel, prefixGridScanLevel, !pointsOnly,
shape, getFieldName(), grid, detailLevel, prefixGridScanLevel,
-1);//-1 flag is slower but ensures correct results
} else if (op == SpatialOperation.Contains) {
return new ContainsPrefixTreeFilter(shape, getFieldName(), grid, detailLevel,

View File

@ -20,6 +20,8 @@ package org.apache.lucene.spatial.prefix;
import java.util.ArrayList;
import java.util.List;
import com.spatial4j.core.shape.Point;
import com.spatial4j.core.shape.Shape;
import org.apache.lucene.queries.TermsQuery;
import org.apache.lucene.search.Filter;
import org.apache.lucene.search.QueryWrapperFilter;
@ -32,9 +34,6 @@ import org.apache.lucene.spatial.query.UnsupportedSpatialOperation;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.BytesRefBuilder;
import com.spatial4j.core.shape.Point;
import com.spatial4j.core.shape.Shape;
/**
* A basic implementation of {@link PrefixTreeStrategy} using a large
* {@link TermsQuery} of all the cells from
@ -55,6 +54,20 @@ public class TermQueryPrefixTreeStrategy extends PrefixTreeStrategy {
super(grid, fieldName);
}
@Override
protected CellToBytesRefIterator newCellToBytesRefIterator() {
//Ensure we don't have leaves, as this strategy doesn't handle them.
return new CellToBytesRefIterator() {
@Override
public BytesRef next() {
if (!cellIter.hasNext()) {
return null;
}
return cellIter.next().getTokenBytesNoLeaf(bytesRef);
}
};
}
@Override
public Filter makeFilter(SpatialArgs args) {
final SpatialOperation op = args.getOperation();
@ -92,7 +105,8 @@ public class TermQueryPrefixTreeStrategy extends PrefixTreeStrategy {
for (BytesRef byteRef : terms) {
byteRef.bytes = masterBytes.bytes();
}
//unfortunately TermsFilter will needlessly sort & dedupe
//unfortunately TermsQuery will needlessly sort & dedupe
//TODO an automatonQuery might be faster?
return new QueryWrapperFilter(new TermsQuery(getFieldName(), terms));
}

View File

@ -60,15 +60,15 @@ public class WithinPrefixTreeFilter extends AbstractVisitingPrefixTreeFilter {
private final Shape bufferedQueryShape;//if null then the whole world
/**
* See {@link AbstractVisitingPrefixTreeFilter#AbstractVisitingPrefixTreeFilter(com.spatial4j.core.shape.Shape, String, org.apache.lucene.spatial.prefix.tree.SpatialPrefixTree, int, int, boolean)}.
* See {@link AbstractVisitingPrefixTreeFilter#AbstractVisitingPrefixTreeFilter(com.spatial4j.core.shape.Shape, String, org.apache.lucene.spatial.prefix.tree.SpatialPrefixTree, int, int)}.
* {@code queryBuffer} is the (minimum) distance beyond the query shape edge
* where non-matching documents are looked for so they can be excluded. If
* -1 is used then the whole world is examined (a good default for correctness).
*/
public WithinPrefixTreeFilter(Shape queryShape, String fieldName, SpatialPrefixTree grid,
int detailLevel, int prefixGridScanLevel, boolean hasIndexedLeaves,
int detailLevel, int prefixGridScanLevel,
double queryBuffer) {
super(queryShape, fieldName, grid, detailLevel, prefixGridScanLevel, hasIndexedLeaves);
super(queryShape, fieldName, grid, detailLevel, prefixGridScanLevel);
this.bufferedQueryShape = queryBuffer == -1 ? null : bufferShape(queryShape, queryBuffer);
}
@ -94,11 +94,10 @@ public class WithinPrefixTreeFilter extends AbstractVisitingPrefixTreeFilter {
@Override
public String toString(String field) {
return "WithinPrefixTreeFilter(" +
// TODO: print something about the shape?
"fieldName=" + fieldName + "," +
"queryShape=" + queryShape + "," +
"detailLevel=" + detailLevel + "," +
"prefixGridScanLevel=" + prefixGridScanLevel + "," +
"hasIndexedLeaves=" + hasIndexedLeaves +
"prefixGridScanLevel=" + prefixGridScanLevel +
")";
}
@ -152,7 +151,6 @@ public class WithinPrefixTreeFilter extends AbstractVisitingPrefixTreeFilter {
return new VisitorTemplate(context, acceptDocs) {
private FixedBitSet inside;
private FixedBitSet outside;
private SpatialRelation visitRelation;
@Override
protected void start() {
@ -173,32 +171,26 @@ public class WithinPrefixTreeFilter extends AbstractVisitingPrefixTreeFilter {
}
@Override
protected boolean visit(Cell cell) throws IOException {
protected boolean visitPrefix(Cell cell) throws IOException {
//cell.relate is based on the bufferedQueryShape; we need to examine what
// the relation is against the queryShape
visitRelation = cell.getShape().relate(queryShape);
if (visitRelation == SpatialRelation.WITHIN) {
SpatialRelation visitRelation = cell.getShape().relate(queryShape);
if (cell.getLevel() == detailLevel) {
collectDocs(visitRelation.intersects() ? inside : outside);
return false;
} else if (visitRelation == SpatialRelation.WITHIN) {
collectDocs(inside);
return false;
} else if (visitRelation == SpatialRelation.DISJOINT) {
collectDocs(outside);
return false;
} else if (cell.getLevel() == detailLevel) {
collectDocs(inside);
return false;
}
return true;
}
@Override
protected void visitLeaf(Cell cell) throws IOException {
//visitRelation is declared as a field, populated by visit() so we don't recompute it.
// We have a specialized visitScanned() which doesn't call this. If we didn't, we would
// not be able to assume visitRelation is from a prior visit() call since in scanning,
// parent cells aren't visited.
assert detailLevel != cell.getLevel();
assert visitRelation == cell.getShape().relate(queryShape);
if (allCellsIntersectQuery(cell, visitRelation))
if (allCellsIntersectQuery(cell))
collectDocs(inside);
else
collectDocs(outside);
@ -207,9 +199,8 @@ public class WithinPrefixTreeFilter extends AbstractVisitingPrefixTreeFilter {
/** Returns true if the provided cell, and all its sub-cells down to
* detailLevel all intersect the queryShape.
*/
private boolean allCellsIntersectQuery(Cell cell, SpatialRelation relate/*cell to query*/) {
if (relate == null)
relate = cell.getShape().relate(queryShape);
private boolean allCellsIntersectQuery(Cell cell) {
SpatialRelation relate = cell.getShape().relate(queryShape);
if (cell.getLevel() == detailLevel)
return relate.intersects();
if (relate == SpatialRelation.WITHIN)
@ -221,7 +212,7 @@ public class WithinPrefixTreeFilter extends AbstractVisitingPrefixTreeFilter {
CellIterator subCells = cell.getNextLevelCells(null);
while (subCells.hasNext()) {
Cell subCell = subCells.next();
if (!allCellsIntersectQuery(subCell, null))//recursion
if (!allCellsIntersectQuery(subCell))//recursion
return false;
}
return true;
@ -229,12 +220,12 @@ public class WithinPrefixTreeFilter extends AbstractVisitingPrefixTreeFilter {
@Override
protected void visitScanned(Cell cell) throws IOException {
//slightly optimize over default impl; required for our 'visitRelation' field re-use above
if (allCellsIntersectQuery(cell, null)) {
collectDocs(inside);
} else {
collectDocs(outside);
}
visitLeaf(cell);//collects as we want, even if not a leaf
// if (cell.isLeaf()) {
// visitLeaf(cell);
// } else {
// visitPrefix(cell);
// }
}
}.getDocIdSet();

View File

@ -51,7 +51,6 @@ public interface Cell {
* Some cells are flagged as leaves, which are indexed as such. A leaf cell is either within some
* shape or it both intersects and the cell is at an accuracy threshold such that no smaller cells
* for the shape will be represented.
* further cells with this prefix for the shape (always true at maxLevels).
*/
boolean isLeaf();
@ -62,14 +61,14 @@ public interface Cell {
void setLeaf();
/**
* Returns the bytes for this cell, with a leaf byte if this is a leaf cell.
* Returns the bytes for this cell, with a leaf byte <em>if this is a leaf cell</em>.
* The result param is used to save object allocation, though its bytes aren't used.
* @param result where the result goes, or null to create new
*/
BytesRef getTokenBytesWithLeaf(BytesRef result);
/**
* Returns the bytes for this cell, without leaf set. The bytes should sort before
* Returns the bytes for this cell, without a leaf set. The bytes should sort before
* {@link #getTokenBytesWithLeaf(org.apache.lucene.util.BytesRef)}.
* The result param is used to save object allocation, though its bytes aren't used.
* @param result where the result goes, or null to create new

View File

@ -17,6 +17,10 @@ package org.apache.lucene.spatial.prefix.tree;
* limitations under the License.
*/
import java.util.ArrayList;
import java.util.Collection;
import java.util.List;
import com.spatial4j.core.context.SpatialContext;
import com.spatial4j.core.io.GeohashUtils;
import com.spatial4j.core.shape.Point;
@ -24,10 +28,6 @@ import com.spatial4j.core.shape.Rectangle;
import com.spatial4j.core.shape.Shape;
import org.apache.lucene.util.BytesRef;
import java.util.ArrayList;
import java.util.Collection;
import java.util.List;
/**
* A {@link SpatialPrefixTree} based on
* <a href="http://en.wikipedia.org/wiki/Geohash">Geohashes</a>.
@ -104,7 +104,7 @@ public class GeohashPrefixTree extends LegacyPrefixTree {
GhCell(String geohash) {
super(stringToBytesPlus1(geohash), 0, geohash.length());
this.geohash = geohash;
if (isLeaf())
if (isLeaf() && getLevel() < getMaxLevels())//we don't have a leaf byte at max levels (an opt)
this.geohash = geohash.substring(0, geohash.length() - 1);
}
@ -115,6 +115,9 @@ public class GeohashPrefixTree extends LegacyPrefixTree {
@Override
protected GeohashPrefixTree getGrid() { return GeohashPrefixTree.this; }
@Override
protected int getMaxLevels() { return maxLevels; }
@Override
protected void readCell(BytesRef bytesRef) {
super.readCell(bytesRef);

View File

@ -17,14 +17,14 @@ package org.apache.lucene.spatial.prefix.tree;
* limitations under the License.
*/
import java.util.Collection;
import com.spatial4j.core.shape.Point;
import com.spatial4j.core.shape.Shape;
import com.spatial4j.core.shape.SpatialRelation;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.StringHelper;
import java.util.Collection;
/** The base for the original two SPT's: Geohash and Quad. Don't subclass this for new SPTs.
* @lucene.internal */
//public for RPT pruneLeafyBranches code
@ -64,7 +64,7 @@ public abstract class LegacyCell implements Cell {
shape = null;
this.bytes = bytes.bytes;
this.b_off = bytes.offset;
this.b_len = bytes.length;
this.b_len = (short) bytes.length;
readLeafAdjust();
}
@ -72,6 +72,8 @@ public abstract class LegacyCell implements Cell {
isLeaf = (b_len > 0 && bytes[b_off + b_len - 1] == LEAF_BYTE);
if (isLeaf)
b_len--;
if (getLevel() == getMaxLevels())
isLeaf = true;
}
// @Override
@ -88,6 +90,8 @@ public abstract class LegacyCell implements Cell {
protected abstract SpatialPrefixTree getGrid();
protected abstract int getMaxLevels();
@Override
public SpatialRelation getShapeRel() {
return shapeRel;
@ -111,7 +115,7 @@ public abstract class LegacyCell implements Cell {
@Override
public BytesRef getTokenBytesWithLeaf(BytesRef result) {
result = getTokenBytesNoLeaf(result);
if (!isLeaf)
if (!isLeaf || getLevel() == getMaxLevels())
return result;
if (result.bytes.length < result.offset + result.length + 1) {
assert false : "Not supposed to happen; performance bug";

View File

@ -17,14 +17,14 @@ package org.apache.lucene.spatial.prefix.tree;
* limitations under the License.
*/
import java.util.Arrays;
import com.spatial4j.core.context.SpatialContext;
import com.spatial4j.core.shape.Point;
import com.spatial4j.core.shape.Rectangle;
import com.spatial4j.core.shape.Shape;
import org.apache.lucene.util.BytesRef;
import java.util.Arrays;
/** The base for the original two SPT's: Geohash and Quad. Don't subclass this for new SPTs.
* @lucene.internal */
abstract class LegacyPrefixTree extends SpatialPrefixTree {
@ -69,7 +69,7 @@ abstract class LegacyPrefixTree extends SpatialPrefixTree {
// subcells from the bytesRef in a loop. This avoids an O(N^2) encode, and we have O(N) instead.
Cell cell = getCell((Point) shape, detailLevel);
assert !cell.isLeaf() && cell instanceof LegacyCell;
assert cell instanceof LegacyCell;
BytesRef fullBytes = cell.getTokenBytesNoLeaf(null);
//fill in reverse order to be sorted
Cell[] cells = new Cell[detailLevel];

View File

@ -17,14 +17,6 @@ package org.apache.lucene.spatial.prefix.tree;
* limitations under the License.
*/
import com.spatial4j.core.context.SpatialContext;
import com.spatial4j.core.shape.Point;
import com.spatial4j.core.shape.Rectangle;
import com.spatial4j.core.shape.Shape;
import com.spatial4j.core.shape.SpatialRelation;
import org.apache.lucene.util.BytesRef;
import java.io.PrintStream;
import java.text.NumberFormat;
import java.util.ArrayList;
@ -33,6 +25,13 @@ import java.util.Collection;
import java.util.List;
import java.util.Locale;
import com.spatial4j.core.context.SpatialContext;
import com.spatial4j.core.shape.Point;
import com.spatial4j.core.shape.Rectangle;
import com.spatial4j.core.shape.Shape;
import com.spatial4j.core.shape.SpatialRelation;
import org.apache.lucene.util.BytesRef;
/**
* A {@link SpatialPrefixTree} which uses a
* <a href="http://en.wikipedia.org/wiki/Quadtree">quad tree</a> in which an
@ -230,6 +229,9 @@ public class QuadPrefixTree extends LegacyPrefixTree {
@Override
protected QuadPrefixTree getGrid() { return QuadPrefixTree.this; }
@Override
protected int getMaxLevels() { return maxLevels; }
@Override
protected Collection<Cell> getSubCells() {
BytesRef source = getTokenBytesNoLeaf(null);

View File

@ -0,0 +1,45 @@
package org.apache.lucene.spatial.prefix;
/*
* 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.tree.Cell;
import org.apache.lucene.util.BytesRef;
/** For testing Lucene <= 5.0. Index redundant prefixes for leaf cells. Fixed in LUCENE-4942. */
class CellToBytesRefIterator50 extends CellToBytesRefIterator {
Cell repeatCell;
@Override
public BytesRef next() {
if (repeatCell != null) {
bytesRef = repeatCell.getTokenBytesWithLeaf(bytesRef);
repeatCell = null;
return bytesRef;
}
if (!cellIter.hasNext()) {
return null;
}
Cell cell = cellIter.next();
bytesRef = cell.getTokenBytesNoLeaf(bytesRef);
if (cell.isLeaf()) {
repeatCell = cell;
}
return bytesRef;
}
}

View File

@ -28,6 +28,7 @@ import org.apache.lucene.spatial.query.SpatialOperation;
import org.junit.Before;
import org.junit.Test;
import static com.carrotsearch.randomizedtesting.RandomizedTest.randomBoolean;
import static com.carrotsearch.randomizedtesting.RandomizedTest.randomIntBetween;
public class DateNRStrategyTest extends RandomSpatialOpStrategyTestCase {
@ -42,7 +43,17 @@ public class DateNRStrategyTest extends RandomSpatialOpStrategyTestCase {
public void setUp() throws Exception {
super.setUp();
tree = DateRangePrefixTree.INSTANCE;
strategy = new NumberRangePrefixTreeStrategy(tree, "dateRange");
if (randomBoolean()) {
strategy = new NumberRangePrefixTreeStrategy(tree, "dateRange");
} else {
//Test the format that existed <= Lucene 5.0
strategy = new NumberRangePrefixTreeStrategy(tree, "dateRange") {
@Override
protected CellToBytesRefIterator newCellToBytesRefIterator() {
return new CellToBytesRefIterator50();
}
};
}
Calendar tmpCal = tree.newCal();
int randomCalWindowField = randomIntBetween(1, Calendar.ZONE_OFFSET - 1);//we're not allowed to add zone offset
tmpCal.add(randomCalWindowField, 2_000);

View File

@ -31,7 +31,6 @@ import com.spatial4j.core.shape.Rectangle;
import com.spatial4j.core.shape.Shape;
import com.spatial4j.core.shape.SpatialRelation;
import com.spatial4j.core.shape.impl.RectangleImpl;
import org.apache.lucene.search.Filter;
import org.apache.lucene.search.FilteredQuery;
import org.apache.lucene.search.MatchAllDocsQuery;
@ -235,7 +234,7 @@ public class HeatmapFacetCounterTest extends StrategyTestCase {
// we use IntersectsPrefixTreeFilter directly so that we can specify the level to go to exactly.
RecursivePrefixTreeStrategy strategy = (RecursivePrefixTreeStrategy) this.strategy;
Filter filter = new IntersectsPrefixTreeFilter(
pt, strategy.getFieldName(), grid, facetLevel, grid.getMaxLevels(), !strategy.isPointsOnly());
pt, strategy.getFieldName(), grid, facetLevel, grid.getMaxLevels());
final TotalHitCountCollector collector = new TotalHitCountCollector();
indexSearcher.search(new FilteredQuery(new MatchAllDocsQuery(), filter), collector);
cellsValidated++;

View File

@ -0,0 +1,32 @@
package org.apache.lucene.spatial.prefix;
/*
* 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.
*/
/** Test RandomSpatialOpFuzzyPrefixTreeTest using the PrefixTree index format found in 5.0 and prior. */
public class RandomSpatialOpFuzzyPrefixTree50Test extends RandomSpatialOpFuzzyPrefixTreeTest {
protected RecursivePrefixTreeStrategy newRPT() {
return new RecursivePrefixTreeStrategy(this.grid, getClass().getSimpleName()) {
@Override
protected CellToBytesRefIterator newCellToBytesRefIterator() {
return new CellToBytesRefIterator50();
}
};
}
}

View File

@ -67,7 +67,7 @@ public class RandomSpatialOpFuzzyPrefixTreeTest extends StrategyTestCase {
static final int ITERATIONS = 10;
private SpatialPrefixTree grid;
protected SpatialPrefixTree grid;
private SpatialContext ctx2D;
public void setupGrid(int maxLevels) throws IOException {
@ -108,7 +108,7 @@ public class RandomSpatialOpFuzzyPrefixTreeTest extends StrategyTestCase {
if (maxLevels == -1)
maxLevels = randomIntBetween(1, 8);//max 64k cells (4^8), also 256*256
this.grid = new QuadPrefixTree(ctx, maxLevels);
this.strategy = new RecursivePrefixTreeStrategy(grid, getClass().getSimpleName());
this.strategy = newRPT();
}
public void setupGeohashGrid(int maxLevels) {
@ -117,7 +117,11 @@ public class RandomSpatialOpFuzzyPrefixTreeTest extends StrategyTestCase {
if (maxLevels == -1)
maxLevels = randomIntBetween(1, 3);//max 16k cells (32^3)
this.grid = new GeohashPrefixTree(ctx, maxLevels);
this.strategy = new RecursivePrefixTreeStrategy(grid, getClass().getSimpleName());
this.strategy = newRPT();
}
protected RecursivePrefixTreeStrategy newRPT() {
return new RecursivePrefixTreeStrategy(this.grid, getClass().getSimpleName());
}
@Test
@ -226,6 +230,7 @@ public class RandomSpatialOpFuzzyPrefixTreeTest extends StrategyTestCase {
return doc;
}
@SuppressWarnings("fallthrough")
private void doTest(final SpatialOperation operation) throws IOException {
//first show that when there's no data, a query will result in no results
{