You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by nk...@apache.org on 2016/02/05 18:03:27 UTC

[83/87] [abbrv] lucene-solr git commit: LUCENE-6997: refactors lucene-spatial module to a new lucene-spatial-extras module, and refactors sandbox GeoPointField and queries to lucene-spatial module

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/50a2f754/lucene/spatial-extras/src/java/org/apache/lucene/spatial/composite/CompositeSpatialStrategy.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/composite/CompositeSpatialStrategy.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/composite/CompositeSpatialStrategy.java
new file mode 100644
index 0000000..7dc2dfa
--- /dev/null
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/composite/CompositeSpatialStrategy.java
@@ -0,0 +1,144 @@
+/*
+ * 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.lucene.spatial.composite;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+import com.spatial4j.core.shape.Point;
+import com.spatial4j.core.shape.Shape;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.queries.function.ValueSource;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.spatial.SpatialStrategy;
+import org.apache.lucene.spatial.prefix.RecursivePrefixTreeStrategy;
+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.query.UnsupportedSpatialOperation;
+import org.apache.lucene.spatial.serialized.SerializedDVStrategy;
+import org.apache.lucene.spatial.util.ShapePredicateValueSource;
+
+/**
+ * A composite {@link SpatialStrategy} based on {@link RecursivePrefixTreeStrategy} (RPT) and
+ * {@link SerializedDVStrategy} (SDV).
+ * RPT acts as an index to the precision available in SDV, and in some circumstances can avoid geometry lookups based
+ * on where a cell is in relation to the query shape.  Currently the only predicate optimized like this is Intersects.
+ * All predicates are supported except for the BBox* ones, and Disjoint.
+ *
+ * @lucene.experimental
+ */
+public class CompositeSpatialStrategy extends SpatialStrategy {
+
+  //TODO support others? (BBox)
+  private final RecursivePrefixTreeStrategy indexStrategy;
+
+  /** Has the geometry. */ // TODO support others?
+  private final SerializedDVStrategy geometryStrategy;
+  private boolean optimizePredicates = true;
+
+  public CompositeSpatialStrategy(String fieldName,
+                                  RecursivePrefixTreeStrategy indexStrategy, SerializedDVStrategy geometryStrategy) {
+    super(indexStrategy.getSpatialContext(), fieldName);//field name; unused
+    this.indexStrategy = indexStrategy;
+    this.geometryStrategy = geometryStrategy;
+  }
+
+  public RecursivePrefixTreeStrategy getIndexStrategy() {
+    return indexStrategy;
+  }
+
+  public SerializedDVStrategy getGeometryStrategy() {
+    return geometryStrategy;
+  }
+
+  public boolean isOptimizePredicates() {
+    return optimizePredicates;
+  }
+
+  /** Set to false to NOT use optimized search predicates that avoid checking the geometry sometimes. Only useful for
+   * benchmarking. */
+  public void setOptimizePredicates(boolean optimizePredicates) {
+    this.optimizePredicates = optimizePredicates;
+  }
+
+  @Override
+  public Field[] createIndexableFields(Shape shape) {
+    List<Field> fields = new ArrayList<>();
+    Collections.addAll(fields, indexStrategy.createIndexableFields(shape));
+    Collections.addAll(fields, geometryStrategy.createIndexableFields(shape));
+    return fields.toArray(new Field[fields.size()]);
+  }
+
+  @Override
+  public ValueSource makeDistanceValueSource(Point queryPoint, double multiplier) {
+    //TODO consider indexing center-point in DV?  Guarantee contained by the shape, which could then be used for
+    // other purposes like faster WITHIN predicate?
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public Query makeQuery(SpatialArgs args) {
+    final SpatialOperation pred = args.getOperation();
+
+    if (pred == SpatialOperation.BBoxIntersects || pred == SpatialOperation.BBoxWithin) {
+      throw new UnsupportedSpatialOperation(pred);
+    }
+
+    if (pred == SpatialOperation.IsDisjointTo) {
+//      final Query intersectQuery = makeQuery(new SpatialArgs(SpatialOperation.Intersects, args.getShape()));
+//      DocValues.getDocsWithField(reader, geometryStrategy.getFieldName());
+      //TODO resurrect Disjoint spatial query utility accepting a field name known to have DocValues.
+      // update class docs when it's added.
+      throw new UnsupportedSpatialOperation(pred);
+    }
+
+    final ShapePredicateValueSource predicateValueSource =
+        new ShapePredicateValueSource(geometryStrategy.makeShapeValueSource(), pred, args.getShape());
+    //System.out.println("PredOpt: " + optimizePredicates);
+    if (pred == SpatialOperation.Intersects && optimizePredicates) {
+      // We have a smart Intersects impl
+
+      final SpatialPrefixTree grid = indexStrategy.getGrid();
+      final int detailLevel = grid.getLevelForDistance(args.resolveDistErr(ctx, 0.0));//default to max precision
+      return new IntersectsRPTVerifyQuery(args.getShape(), indexStrategy.getFieldName(), grid,
+          detailLevel, indexStrategy.getPrefixGridScanLevel(), predicateValueSource);
+    } else {
+      //The general path; all index matches get verified
+
+      SpatialArgs indexArgs;
+      if (pred == SpatialOperation.Contains) {
+        // note: we could map IsWithin as well but it's pretty darned slow since it touches all world grids
+        indexArgs = args;
+      } else {
+        //TODO add args.clone method with new predicate? Or simply make non-final?
+        indexArgs = new SpatialArgs(SpatialOperation.Intersects, args.getShape());
+        indexArgs.setDistErr(args.getDistErr());
+        indexArgs.setDistErrPct(args.getDistErrPct());
+      }
+
+      if (indexArgs.getDistErr() == null && indexArgs.getDistErrPct() == null) {
+        indexArgs.setDistErrPct(0.10);
+      }
+
+      final Query indexQuery = indexStrategy.makeQuery(indexArgs);
+      return new CompositeVerifyQuery(indexQuery, predicateValueSource);
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/50a2f754/lucene/spatial-extras/src/java/org/apache/lucene/spatial/composite/CompositeVerifyQuery.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/composite/CompositeVerifyQuery.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/composite/CompositeVerifyQuery.java
new file mode 100644
index 0000000..e03d959
--- /dev/null
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/composite/CompositeVerifyQuery.java
@@ -0,0 +1,121 @@
+/*
+ * 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.lucene.spatial.composite;
+
+import java.io.IOException;
+import java.util.Map;
+
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.queries.function.FunctionValues;
+import org.apache.lucene.queries.function.ValueSource;
+import org.apache.lucene.search.ConstantScoreScorer;
+import org.apache.lucene.search.ConstantScoreWeight;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.Scorer;
+import org.apache.lucene.search.TwoPhaseIterator;
+import org.apache.lucene.search.Weight;
+import org.apache.lucene.util.Bits;
+
+/**
+ * A Query that considers an "indexQuery" to have approximate results, and a follow-on
+ * {@link ValueSource}/{@link FunctionValues#boolVal(int)} is called to verify each hit
+ * from {@link TwoPhaseIterator#matches()}.
+ *
+ * @lucene.experimental
+ */
+public class CompositeVerifyQuery extends Query {
+  final Query indexQuery;//approximation (matches more than needed)
+  final ValueSource predicateValueSource;//we call boolVal(doc)
+
+  public CompositeVerifyQuery(Query indexQuery, ValueSource predicateValueSource) {
+    this.indexQuery = indexQuery;
+    this.predicateValueSource = predicateValueSource;
+  }
+
+  @Override
+  public Query rewrite(IndexReader reader) throws IOException {
+    final Query rewritten = indexQuery.rewrite(reader);
+    if (rewritten != indexQuery) {
+      return new CompositeVerifyQuery(rewritten, predicateValueSource);
+    }
+    return super.rewrite(reader);
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) return true;
+    if (o == null || getClass() != o.getClass()) return false;
+    if (!super.equals(o)) return false;
+
+    CompositeVerifyQuery that = (CompositeVerifyQuery) o;
+
+    if (!indexQuery.equals(that.indexQuery)) return false;
+    if (!predicateValueSource.equals(that.predicateValueSource)) return false;
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    int result = super.hashCode();
+    result = 31 * result + indexQuery.hashCode();
+    result = 31 * result + predicateValueSource.hashCode();
+    return result;
+  }
+
+  @Override
+  public String toString(String field) {
+    //TODO verify this looks good
+    return getClass().getSimpleName() + "(" + indexQuery.toString(field) + ", " + predicateValueSource + ")";
+  }
+
+  @Override
+  public Weight createWeight(IndexSearcher searcher, boolean needsScores) throws IOException {
+    final Weight indexQueryWeight = indexQuery.createWeight(searcher, false);//scores aren't unsupported
+    final Map valueSourceContext = ValueSource.newContext(searcher);
+
+    return new ConstantScoreWeight(this) {
+
+      @Override
+      public Scorer scorer(LeafReaderContext context) throws IOException {
+
+        final Scorer indexQueryScorer = indexQueryWeight.scorer(context);
+        if (indexQueryScorer == null) {
+          return null;
+        }
+
+        final FunctionValues predFuncValues = predicateValueSource.getValues(valueSourceContext, context);
+
+        final TwoPhaseIterator twoPhaseIterator = new TwoPhaseIterator(indexQueryScorer.iterator()) {
+          @Override
+          public boolean matches() throws IOException {
+            return predFuncValues.boolVal(indexQueryScorer.docID());
+          }
+
+          @Override
+          public float matchCost() {
+            return 100; // TODO: use cost of predFuncValues.boolVal()
+          }
+        };
+
+        return new ConstantScoreScorer(this, score(), twoPhaseIterator);
+      }
+    };
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/50a2f754/lucene/spatial-extras/src/java/org/apache/lucene/spatial/composite/IntersectsRPTVerifyQuery.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/composite/IntersectsRPTVerifyQuery.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/composite/IntersectsRPTVerifyQuery.java
new file mode 100644
index 0000000..a963b6e
--- /dev/null
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/composite/IntersectsRPTVerifyQuery.java
@@ -0,0 +1,235 @@
+/*
+ * 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.lucene.spatial.composite;
+
+import java.io.IOException;
+import java.util.Map;
+
+import com.spatial4j.core.shape.Shape;
+import com.spatial4j.core.shape.SpatialRelation;
+
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.queries.function.FunctionValues;
+import org.apache.lucene.queries.function.ValueSource;
+import org.apache.lucene.search.ConstantScoreScorer;
+import org.apache.lucene.search.ConstantScoreWeight;
+import org.apache.lucene.search.DocIdSet;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.Scorer;
+import org.apache.lucene.search.TwoPhaseIterator;
+import org.apache.lucene.search.Weight;
+import org.apache.lucene.spatial.prefix.AbstractVisitingPrefixTreeQuery;
+import org.apache.lucene.spatial.prefix.tree.Cell;
+import org.apache.lucene.spatial.prefix.tree.SpatialPrefixTree;
+import org.apache.lucene.util.DocIdSetBuilder;
+
+/**
+ * A spatial Intersects predicate that distinguishes an approximated match from an exact match based on which cells
+ * are within the query shape. It exposes a {@link TwoPhaseIterator} that will verify a match with a provided
+ * predicate in the form of a {@link ValueSource} by calling {@link FunctionValues#boolVal(int)}.
+ *
+ * @lucene.internal
+ */
+public class IntersectsRPTVerifyQuery extends Query {
+
+  private final IntersectsDifferentiatingQuery intersectsDiffQuery;
+  private final ValueSource predicateValueSource; // we call FunctionValues.boolVal(doc)
+
+  public IntersectsRPTVerifyQuery(Shape queryShape, String fieldName, SpatialPrefixTree grid, int detailLevel,
+                                  int prefixGridScanLevel, ValueSource predicateValueSource) {
+    this.predicateValueSource = predicateValueSource;
+    this.intersectsDiffQuery = new IntersectsDifferentiatingQuery(queryShape, fieldName, grid, detailLevel,
+        prefixGridScanLevel);
+  }
+
+  @Override
+  public String toString(String field) {
+    return "IntersectsVerified(fieldName=" + field + ")";
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) return true;
+    if (!super.equals(o)) return false;
+
+    IntersectsRPTVerifyQuery that = (IntersectsRPTVerifyQuery) o;
+
+    if (!intersectsDiffQuery.equals(that.intersectsDiffQuery)) return false;
+    return predicateValueSource.equals(that.predicateValueSource);
+
+  }
+
+  @Override
+  public int hashCode() {
+    int result = super.hashCode();
+    result = 31 * result + intersectsDiffQuery.hashCode();
+    result = 31 * result + predicateValueSource.hashCode();
+    return result;
+  }
+
+  @Override
+  public Weight createWeight(IndexSearcher searcher, boolean needsScores) throws IOException {
+    final Map valueSourceContext = ValueSource.newContext(searcher);
+
+    return new ConstantScoreWeight(this) {
+      @Override
+      public Scorer scorer(LeafReaderContext context) throws IOException {
+        // Compute approx & exact
+        final IntersectsDifferentiatingQuery.IntersectsDifferentiatingVisitor result =
+            intersectsDiffQuery.compute(context);
+        if (result.approxDocIdSet == null) {
+          return null;
+        }
+        final DocIdSetIterator approxDISI = result.approxDocIdSet.iterator();
+        if (approxDISI == null) {
+          return null;
+        }
+        final DocIdSetIterator exactIterator;
+        if (result.exactDocIdSet != null) {
+          // If both sets are the same, there's nothing to verify; we needn't return a TwoPhaseIterator
+          if (result.approxDocIdSet == result.exactDocIdSet) {
+            return new ConstantScoreScorer(this, score(), approxDISI);
+          }
+          exactIterator = result.exactDocIdSet.iterator();
+          assert exactIterator != null;
+        } else {
+          exactIterator = null;
+        }
+
+        final FunctionValues predFuncValues = predicateValueSource.getValues(valueSourceContext, context);
+
+        final TwoPhaseIterator twoPhaseIterator = new TwoPhaseIterator(approxDISI) {
+          @Override
+          public boolean matches() throws IOException {
+            final int doc = approxDISI.docID();
+            if (exactIterator != null) {
+              if (exactIterator.docID() < doc) {
+                exactIterator.advance(doc);
+              }
+              if (exactIterator.docID() == doc) {
+                return true;
+              }
+            }
+
+            return predFuncValues.boolVal(doc);
+          }
+
+          @Override
+          public float matchCost() {
+            return 100; // TODO: use cost of exactIterator.advance() and predFuncValues.boolVal()
+          }
+        };
+
+        return new ConstantScoreScorer(this, score(), twoPhaseIterator);
+      }
+    };
+  }
+
+  //This may be a "Query" but we don't use it as-such; the caller calls the constructor and then compute() and examines
+  // the results which consists of two parts -- the approximated results, and a subset of exact matches. The
+  // difference needs to be verified.
+  // TODO refactor AVPTQ to not be a Query?
+  private static class IntersectsDifferentiatingQuery extends AbstractVisitingPrefixTreeQuery {
+
+    public IntersectsDifferentiatingQuery(Shape queryShape, String fieldName, SpatialPrefixTree grid,
+                                          int detailLevel, int prefixGridScanLevel) {
+      super(queryShape, fieldName, grid, detailLevel, prefixGridScanLevel);
+    }
+
+    IntersectsDifferentiatingQuery.IntersectsDifferentiatingVisitor compute(LeafReaderContext context)
+        throws IOException {
+      final IntersectsDifferentiatingQuery.IntersectsDifferentiatingVisitor result =
+          new IntersectsDifferentiatingQuery.IntersectsDifferentiatingVisitor(context);
+      result.getDocIdSet();//computes
+      return result;
+    }
+
+    // TODO consider if IntersectsPrefixTreeQuery should simply do this and provide both sets
+
+    class IntersectsDifferentiatingVisitor extends VisitorTemplate {
+      DocIdSetBuilder approxBuilder = new DocIdSetBuilder(maxDoc);
+      DocIdSetBuilder exactBuilder = new DocIdSetBuilder(maxDoc);
+      boolean approxIsEmpty = true;
+      boolean exactIsEmpty = true;
+      DocIdSet exactDocIdSet;
+      DocIdSet approxDocIdSet;
+
+      public IntersectsDifferentiatingVisitor(LeafReaderContext context) throws IOException {
+        super(context);
+      }
+
+      @Override
+      protected void start() throws IOException {
+      }
+
+      @Override
+      protected DocIdSet finish() throws IOException {
+        if (exactIsEmpty) {
+          exactDocIdSet = null;
+        } else {
+          exactDocIdSet = exactBuilder.build();
+        }
+        if (approxIsEmpty) {
+          approxDocIdSet = exactDocIdSet;//optimization
+        } else {
+          if (exactDocIdSet != null) {
+            approxBuilder.add(exactDocIdSet.iterator());
+          }
+          approxDocIdSet = approxBuilder.build();
+        }
+        return null;//unused in this weird re-use of AVPTQ
+      }
+
+      @Override
+      protected boolean visitPrefix(Cell cell) throws IOException {
+        if (cell.getShapeRel() == SpatialRelation.WITHIN) {
+          exactIsEmpty = false;
+          collectDocs(exactBuilder);//note: we'll add exact to approx on finish()
+          return false;
+        } else if (cell.getLevel() == detailLevel) {
+          approxIsEmpty = false;
+          collectDocs(approxBuilder);
+          return false;
+        }
+        return true;
+      }
+
+      @Override
+      protected void visitLeaf(Cell cell) throws IOException {
+        if (cell.getShapeRel() == SpatialRelation.WITHIN) {
+          exactIsEmpty = false;
+          collectDocs(exactBuilder);//note: we'll add exact to approx on finish()
+        } else {
+          approxIsEmpty = false;
+          collectDocs(approxBuilder);
+        }
+      }
+    }
+
+    @Override
+    public DocIdSet getDocIdSet(LeafReaderContext context) throws IOException {
+      throw new IllegalStateException();
+    }
+
+    @Override
+    public String toString(String field) {
+      throw new IllegalStateException();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/50a2f754/lucene/spatial-extras/src/java/org/apache/lucene/spatial/composite/package-info.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/composite/package-info.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/composite/package-info.java
new file mode 100644
index 0000000..04cb9fc
--- /dev/null
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/composite/package-info.java
@@ -0,0 +1,18 @@
+/*
+ * 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.
+ */
+/** Composite strategies. */
+package org.apache.lucene.spatial.composite;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/50a2f754/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/AbstractPrefixTreeQuery.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/AbstractPrefixTreeQuery.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/AbstractPrefixTreeQuery.java
new file mode 100644
index 0000000..127e689
--- /dev/null
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/AbstractPrefixTreeQuery.java
@@ -0,0 +1,133 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.lucene.spatial.prefix;
+
+import java.io.IOException;
+
+import com.spatial4j.core.shape.Shape;
+import org.apache.lucene.index.LeafReader;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.index.PostingsEnum;
+import org.apache.lucene.index.Terms;
+import org.apache.lucene.index.TermsEnum;
+import org.apache.lucene.search.ConstantScoreScorer;
+import org.apache.lucene.search.ConstantScoreWeight;
+import org.apache.lucene.search.DocIdSet;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.Scorer;
+import org.apache.lucene.search.Weight;
+import org.apache.lucene.spatial.prefix.tree.SpatialPrefixTree;
+import org.apache.lucene.util.BitSet;
+import org.apache.lucene.util.DocIdSetBuilder;
+
+/**
+ * Base class for Lucene Queries on SpatialPrefixTree fields.
+ * @lucene.internal
+ */
+public abstract class AbstractPrefixTreeQuery extends Query {
+
+  protected final Shape queryShape;
+  protected final String fieldName;
+  protected final SpatialPrefixTree grid;//not in equals/hashCode since it's implied for a specific field
+  protected final int detailLevel;
+
+  public AbstractPrefixTreeQuery(Shape queryShape, String fieldName, SpatialPrefixTree grid, int detailLevel) {
+    this.queryShape = queryShape;
+    this.fieldName = fieldName;
+    this.grid = grid;
+    this.detailLevel = detailLevel;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) return true;
+    if (super.equals(o) == false) return false;
+
+    AbstractPrefixTreeQuery that = (AbstractPrefixTreeQuery) o;
+
+    if (detailLevel != that.detailLevel) return false;
+    if (!fieldName.equals(that.fieldName)) return false;
+    if (!queryShape.equals(that.queryShape)) return false;
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    int result = super.hashCode();
+    result = 31 * result + queryShape.hashCode();
+    result = 31 * result + fieldName.hashCode();
+    result = 31 * result + detailLevel;
+    return result;
+  }
+
+  @Override
+  public Weight createWeight(IndexSearcher searcher, boolean needsScores) throws IOException {
+    return new ConstantScoreWeight(this) {
+      @Override
+      public Scorer scorer(LeafReaderContext context) throws IOException {
+        DocIdSet docSet = getDocIdSet(context);
+        if (docSet == null) {
+          return null;
+        }
+        DocIdSetIterator disi = docSet.iterator();
+        if (disi == null) {
+          return null;
+        }
+        return new ConstantScoreScorer(this, score(), disi);
+      }
+    };
+  }
+
+  protected abstract DocIdSet getDocIdSet(LeafReaderContext context) throws IOException;
+
+  /** Holds transient state and docid collecting utility methods as part of
+   * traversing a {@link TermsEnum} for a {@link org.apache.lucene.index.LeafReaderContext}. */
+  public abstract class BaseTermsEnumTraverser {//TODO rename to LeafTermsEnumTraverser ?
+    //note: only 'fieldName' (accessed in constructor) keeps this from being a static inner class
+
+    protected final LeafReaderContext context;
+    protected final int maxDoc;
+
+    protected TermsEnum termsEnum;//remember to check for null!
+    protected PostingsEnum postingsEnum;
+
+    public BaseTermsEnumTraverser(LeafReaderContext context) throws IOException {
+      this.context = context;
+      LeafReader reader = context.reader();
+      this.maxDoc = reader.maxDoc();
+      Terms terms = reader.terms(fieldName);
+      if (terms != null)
+        this.termsEnum = terms.iterator();
+    }
+
+    protected void collectDocs(BitSet bitSet) throws IOException {
+      assert termsEnum != null;
+      postingsEnum = termsEnum.postings(postingsEnum, PostingsEnum.NONE);
+      bitSet.or(postingsEnum);
+    }
+
+    protected void collectDocs(DocIdSetBuilder docSetBuilder) throws IOException {
+      assert termsEnum != null;
+      postingsEnum = termsEnum.postings(postingsEnum, PostingsEnum.NONE);
+      docSetBuilder.add(postingsEnum);
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/50a2f754/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/AbstractVisitingPrefixTreeQuery.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/AbstractVisitingPrefixTreeQuery.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/AbstractVisitingPrefixTreeQuery.java
new file mode 100644
index 0000000..2237ca9
--- /dev/null
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/AbstractVisitingPrefixTreeQuery.java
@@ -0,0 +1,380 @@
+/*
+ * 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.lucene.spatial.prefix;
+
+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;
+import org.apache.lucene.spatial.prefix.tree.Cell;
+import org.apache.lucene.spatial.prefix.tree.CellIterator;
+import org.apache.lucene.spatial.prefix.tree.SpatialPrefixTree;
+import org.apache.lucene.util.BytesRef;
+
+/**
+ * Traverses a {@link SpatialPrefixTree} indexed field, using the template and
+ * visitor design patterns for subclasses to guide the traversal and collect
+ * matching documents.
+ * <p>
+ * Subclasses implement {@link #getDocIdSet(org.apache.lucene.index.LeafReaderContext)}
+ * by instantiating a custom {@link VisitorTemplate} subclass (i.e. an anonymous inner class)
+ * and implement the required methods.
+ *
+ * @lucene.internal
+ */
+public abstract class AbstractVisitingPrefixTreeQuery extends AbstractPrefixTreeQuery {
+
+  //Historical note: this code resulted from a refactoring of RecursivePrefixTreeQuery,
+  // which in turn came out of SOLR-2155
+
+  //This class perhaps could have been implemented in terms of FilteredTermsEnum & MultiTermQuery.
+  //  Maybe so for simple Intersects predicate but not for when we want to collect terms
+  //  differently depending on cell state like IsWithin and for fuzzy/accurate collection planned improvements.  At
+  //  least it would just make things more complicated.
+
+  protected final int prefixGridScanLevel;//at least one less than grid.getMaxLevels()
+
+  public AbstractVisitingPrefixTreeQuery(Shape queryShape, String fieldName, SpatialPrefixTree grid,
+                                         int detailLevel, int prefixGridScanLevel) {
+    super(queryShape, fieldName, grid, detailLevel);
+    this.prefixGridScanLevel = Math.max(0, Math.min(prefixGridScanLevel, grid.getMaxLevels() - 1));
+    assert detailLevel <= grid.getMaxLevels();
+  }
+
+  /**
+   * An abstract class designed to make it easy to implement predicates or
+   * other operations on a {@link SpatialPrefixTree} indexed field. An instance
+   * of this class is not designed to be re-used across LeafReaderContext
+   * instances so simply create a new one per-leaf.
+   * The {@link #getDocIdSet()} method here starts the work. It first checks
+   * that there are indexed terms; if not it quickly returns null. Then it calls
+   * {@link #start()} so a subclass can set up a return value, like an
+   * {@link org.apache.lucene.util.FixedBitSet}. Then it starts the traversal
+   * process, calling {@link #findSubCellsToVisit(org.apache.lucene.spatial.prefix.tree.Cell)}
+   * 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 #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.
+   *
+   * @lucene.internal
+   */
+  public abstract class VisitorTemplate extends BaseTermsEnumTraverser {
+
+  /* Future potential optimizations:
+
+  * Can a polygon query shape be optimized / made-simpler at recursive depths
+    (e.g. intersection of shape + cell box)
+
+  * RE "scan" vs divide & conquer performance decision:
+    We should use termsEnum.docFreq() as an estimate on the number of places at
+    this depth.  It would be nice if termsEnum knew how many terms
+    start with the current term without having to repeatedly next() & test to find out.
+
+  * Perhaps don't do intermediate seek()'s to cells above detailLevel that have Intersects
+    relation because we won't be collecting those docs any way.  However seeking
+    does act as a short-circuit.  So maybe do some percent of the time or when the level
+    is above some threshold.
+
+  */
+
+    //
+    //  TODO MAJOR REFACTOR SIMPLIFICATION BASED ON TreeCellIterator  TODO
+    //
+
+    private VNode curVNode;//current pointer, derived from query shape
+    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) throws IOException {
+      super(context);
+    }
+
+    public DocIdSet getDocIdSet() throws IOException {
+      assert curVNode == null : "Called more than once?";
+      if (termsEnum == null)
+        return null;
+      if (!nextTerm()) {//advances
+        return null;
+      }
+
+      curVNode = new VNode(null);
+      curVNode.reset(grid.getWorldCell());
+
+      start();
+
+      addIntersectingChildren();
+
+      main: while (thisTerm != null) {//terminates for other reasons too!
+
+        //Advance curVNode pointer
+        if (curVNode.children != null) {
+          //-- HAVE CHILDREN: DESCEND
+          assert curVNode.children.hasNext();//if we put it there then it has something
+          preSiblings(curVNode);
+          curVNode = curVNode.children.next();
+        } else {
+          //-- NO CHILDREN: ADVANCE TO NEXT SIBLING
+          VNode parentVNode = curVNode.parent;
+          while (true) {
+            if (parentVNode == null)
+              break main; // all done
+            if (parentVNode.children.hasNext()) {
+              //advance next sibling
+              curVNode = parentVNode.children.next();
+              break;
+            } else {
+              //reached end of siblings; pop up
+              postSiblings(parentVNode);
+              parentVNode.children = null;//GC
+              parentVNode = parentVNode.parent;
+            }
+          }
+        }
+
+        //Seek to curVNode's cell (or skip if termsEnum has moved beyond)
+        final int compare = indexedCell.compareToNoLeaf(curVNode.cell);
+        if (compare > 0) {
+          // 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
+          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
+
+      return finish();
+    }
+
+    /** Called initially, and whenever {@link #visitPrefix(org.apache.lucene.spatial.prefix.tree.Cell)}
+     * returns true. */
+    private void addIntersectingChildren() throws IOException {
+      assert thisTerm != null;
+      Cell cell = curVNode.cell;
+      if (cell.getLevel() >= detailLevel)
+        throw new IllegalStateException("Spatial logic error");
+
+      //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.
+
+      //TODO use termsEnum.docFreq() as heuristic
+      boolean scan = cell.getLevel() >= prefixGridScanLevel;//simple heuristic
+
+      if (!scan) {
+        //Divide & conquer (ultimately termsEnum.seek())
+
+        Iterator<Cell> subCellsIter = findSubCellsToVisit(cell);
+        if (!subCellsIter.hasNext())//not expected
+          return;
+        curVNode.children = new VNodeCellIterator(subCellsIter, new VNode(curVNode));
+
+      } else {
+        //Scan (loop of termsEnum.next())
+
+        scan(detailLevel);
+      }
+    }
+
+    /**
+     * Called when doing a divide and conquer to find the next intersecting cells
+     * of the query shape that are beneath {@code cell}. {@code cell} is
+     * guaranteed to have an intersection and thus this must return some number
+     * of nodes.
+     */
+    protected CellIterator findSubCellsToVisit(Cell cell) {
+      return cell.getNextLevelCells(queryShape);
+    }
+
+    /**
+     * Scans ({@code termsEnum.next()}) terms until a term is found that does
+     * not start with curVNode's cell. If it finds a leaf cell or a cell at
+     * level {@code scanDetailLevel} then it calls {@link
+     * #visitScanned(org.apache.lucene.spatial.prefix.tree.Cell)}.
+     */
+    protected void scan(int scanDetailLevel) throws IOException {
+      //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);
+        }
+        //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}. */
+    private class VNodeCellIterator implements Iterator<VNode> {
+
+      final Iterator<Cell> cellIter;
+      private final VNode vNode;
+
+      VNodeCellIterator(Iterator<Cell> cellIter, VNode vNode) {
+        this.cellIter = cellIter;
+        this.vNode = vNode;
+      }
+
+      @Override
+      public boolean hasNext() {
+        return cellIter.hasNext();
+      }
+
+      @Override
+      public VNode next() {
+        assert hasNext();
+        vNode.reset(cellIter.next());
+        return vNode;
+      }
+
+      @Override
+      public void remove() {//it always removes
+      }
+    }
+
+    /** Called first to setup things. */
+    protected abstract void start() throws IOException;
+
+    /** Called last to return the result. */
+    protected abstract DocIdSet finish() throws IOException;
+
+    /**
+     * 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.
+     */
+    protected abstract boolean visitPrefix(Cell cell) throws IOException;
+
+    /**
+     * Called when an indexed leaf cell is found. An
+     * 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;
+
+    /**
+     * The cell is either indexed as a leaf or is the last level of detail. It
+     * 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 #visitPrefix(org.apache.lucene.spatial.prefix.tree.Cell)}.
+     */
+    protected void visitScanned(Cell cell) throws IOException {
+      final SpatialRelation relate = cell.getShape().relate(queryShape);
+      if (relate.intersects()) {
+        cell.setShapeRel(relate);//just being pedantic
+        if (cell.isLeaf()) {
+          visitLeaf(cell);
+        } else {
+          visitPrefix(cell);
+        }
+      }
+    }
+
+    protected void preSiblings(VNode vNode) throws IOException {
+    }
+
+    protected void postSiblings(VNode vNode) throws IOException {
+    }
+  }//class VisitorTemplate
+
+  /**
+   * A visitor node/cell found via the query shape for {@link VisitorTemplate}.
+   * Sometimes these are reset(cell). It's like a LinkedList node but forms a
+   * tree.
+   *
+   * @lucene.internal
+   */
+  protected static class VNode {
+    //Note: The VNode tree adds more code to debug/maintain v.s. a flattened
+    // LinkedList that we used to have. There is more opportunity here for
+    // custom behavior (see preSiblings & postSiblings) but that's not
+    // leveraged yet. Maybe this is slightly more GC friendly.
+
+    final VNode parent;//only null at the root
+    Iterator<VNode> children;//null, then sometimes set, then null
+    Cell cell;//not null (except initially before reset())
+
+    /**
+     * call reset(cell) after to set the cell.
+     */
+    VNode(VNode parent) { // remember to call reset(cell) after
+      this.parent = parent;
+    }
+
+    void reset(Cell cell) {
+      assert cell != null;
+      this.cell = cell;
+      assert children == null;
+    }
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/50a2f754/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/BytesRefIteratorTokenStream.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/BytesRefIteratorTokenStream.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/BytesRefIteratorTokenStream.java
new file mode 100644
index 0000000..e724ab0
--- /dev/null
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/BytesRefIteratorTokenStream.java
@@ -0,0 +1,72 @@
+/*
+ * 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.lucene.spatial.prefix;
+
+import java.io.IOException;
+
+import org.apache.lucene.analysis.TokenStream;
+import org.apache.lucene.analysis.tokenattributes.BytesTermAttribute;
+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.LegacyNumericTokenStream}.
+ *
+ * @lucene.internal
+ */
+class BytesRefIteratorTokenStream extends TokenStream {
+
+  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");
+  }
+
+  @Override
+  public final boolean incrementToken() throws IOException {
+    if (bytesIter == null)
+      throw new IllegalStateException("call setBytesRefIterator() before usage");
+
+    // get next
+    BytesRef bytes = bytesIter.next();
+    if (bytes == null) {
+      return false;
+    } else {
+      clearAttributes();
+      bytesAtt.setBytesRef(bytes);
+      //note: we don't bother setting posInc or type attributes.  There's no point to it.
+      return true;
+    }
+  }
+
+  //members
+  private final BytesTermAttribute bytesAtt = addAttribute(BytesTermAttribute.class);
+
+  private BytesRefIterator bytesIter = null; // null means not initialized
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/50a2f754/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/CellToBytesRefIterator.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/CellToBytesRefIterator.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/CellToBytesRefIterator.java
new file mode 100644
index 0000000..0b81b26
--- /dev/null
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/CellToBytesRefIterator.java
@@ -0,0 +1,49 @@
+/*
+ * 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.lucene.spatial.prefix;
+
+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);
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/50a2f754/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/ContainsPrefixTreeQuery.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/ContainsPrefixTreeQuery.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/ContainsPrefixTreeQuery.java
new file mode 100644
index 0000000..7eadb64
--- /dev/null
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/ContainsPrefixTreeQuery.java
@@ -0,0 +1,362 @@
+/*
+ * 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.lucene.spatial.prefix;
+
+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;
+import org.apache.lucene.spatial.prefix.tree.CellIterator;
+import org.apache.lucene.spatial.prefix.tree.SpatialPrefixTree;
+import org.apache.lucene.util.Bits;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.RamUsageEstimator;
+import org.apache.lucene.util.SentinelIntSet;
+
+/**
+ * Finds docs where its indexed shape {@link org.apache.lucene.spatial.query.SpatialOperation#Contains
+ * CONTAINS} the query shape. For use on {@link RecursivePrefixTreeStrategy}.
+ *
+ * @lucene.experimental
+ */
+public class ContainsPrefixTreeQuery extends AbstractPrefixTreeQuery {
+
+  /**
+   * 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
+   * those shapes contain the query shape but none do individually.  Set this to false to
+   * increase performance if you don't care about that circumstance (such as if your indexed
+   * data doesn't even have such conditions).  See LUCENE-5062.
+   */
+  protected final boolean multiOverlappingIndexedShapes;
+
+  public ContainsPrefixTreeQuery(Shape queryShape, String fieldName, SpatialPrefixTree grid, int detailLevel, boolean multiOverlappingIndexedShapes) {
+    super(queryShape, fieldName, grid, detailLevel);
+    this.multiOverlappingIndexedShapes = multiOverlappingIndexedShapes;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (!super.equals(o))
+      return false;
+    return multiOverlappingIndexedShapes == ((ContainsPrefixTreeQuery)o).multiOverlappingIndexedShapes;
+  }
+
+  @Override
+  public int hashCode() {
+    return super.hashCode() + (multiOverlappingIndexedShapes ? 1 : 0);
+  }
+
+  @Override
+  public String toString(String field) {
+    return getClass().getSimpleName() + "(" +
+        "fieldName=" + fieldName + "," +
+        "queryShape=" + queryShape + "," +
+        "detailLevel=" + detailLevel + "," +
+        "multiOverlappingIndexedShapes=" + multiOverlappingIndexedShapes +
+        ")";
+  }
+
+  @Override
+  protected DocIdSet getDocIdSet(LeafReaderContext context) throws IOException {
+    return new ContainsVisitor(context).visit(grid.getWorldCell(), null);
+  }
+
+  private class ContainsVisitor extends BaseTermsEnumTraverser {
+
+    public ContainsVisitor(LeafReaderContext context) throws IOException {
+      super(context);
+      if (termsEnum != null) {
+        nextTerm();//advance to first
+      }
+    }
+
+    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 (thisTerm == null)//signals all done
+        return null;
+
+      // 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.
+      Shape subCellsFilter = queryShape;
+      if (cell.getLevel() != 0 && ((cell.getShapeRel() == null || cell.getShapeRel() == SpatialRelation.WITHIN))) {
+        subCellsFilter = null;
+        assert cell.getShape().relate(queryShape) == SpatialRelation.WITHIN;
+      }
+      CellIterator subCells = cell.getNextLevelCells(subCellsFilter);
+      while (subCells.hasNext()) {
+        Cell subCell = subCells.next();
+        if (!seek(subCell)) {
+          combinedSubResults = null;
+        } else if (subCell.getLevel() == detailLevel) {
+          combinedSubResults = getDocs(subCell, acceptContains);
+        } else if (!multiOverlappingIndexedShapes &&
+            subCell.getShapeRel() == SpatialRelation.WITHIN) {
+          combinedSubResults = getLeafDocs(subCell, acceptContains);
+        } 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
+      }
+
+      return combinedSubResults;
+    }
+
+    private boolean seek(Cell cell) throws IOException {
+      if (thisTerm == null)
+        return false;
+      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 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 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;
+    }
+
+    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 {
+      SmallDocSet set = null;
+
+      postingsEnum = termsEnum.postings(postingsEnum, PostingsEnum.NONE);
+      int docid;
+      while ((docid = postingsEnum.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
+        if (acceptContains != null && acceptContains.get(docid) == false) {
+          continue;
+        }
+        if (set == null) {
+          int size = termsEnum.docFreq();
+          if (size <= 0)
+            size = 16;
+          set = new SmallDocSet(size);
+        }
+        set.set(docid);
+      }
+      return set;
+    }
+
+  }//class ContainsVisitor
+
+  /** A hash based mutable set of docIds. If this were Solr code then we might
+   * use a combination of HashDocSet and SortedIntDocSet instead. */
+  // TODO use DocIdSetBuilder?
+  private static class SmallDocSet extends DocIdSet implements Bits {
+
+    private final SentinelIntSet intSet;
+    private int maxInt = 0;
+
+    public SmallDocSet(int size) {
+      intSet = new SentinelIntSet(size, -1);
+    }
+
+    @Override
+    public boolean get(int index) {
+      return intSet.exists(index);
+    }
+
+    public void set(int index) {
+      intSet.put(index);
+      if (index > maxInt)
+        maxInt = index;
+    }
+
+    /** Largest docid. */
+    @Override
+    public int length() {
+      return maxInt;
+    }
+
+    /** Number of docids. */
+    public int size() {
+      return intSet.size();
+    }
+
+    /** NOTE: modifies and returns either "this" or "other" */
+    public SmallDocSet union(SmallDocSet other) {
+      SmallDocSet bigger;
+      SmallDocSet smaller;
+      if (other.intSet.size() > this.intSet.size()) {
+        bigger = other;
+        smaller = this;
+      } else {
+        bigger = this;
+        smaller = other;
+      }
+      //modify bigger
+      for (int v : smaller.intSet.keys) {
+        if (v == smaller.intSet.emptyVal)
+          continue;
+        bigger.set(v);
+      }
+      return bigger;
+    }
+
+    @Override
+    public Bits bits() throws IOException {
+      //if the # of docids is super small, return null since iteration is going
+      // to be faster
+      return size() > 4 ? this : null;
+    }
+
+    @Override
+    public DocIdSetIterator iterator() throws IOException {
+      if (size() == 0)
+        return null;
+      //copy the unsorted values to a new array then sort them
+      int d = 0;
+      final int[] docs = new int[intSet.size()];
+      for (int v : intSet.keys) {
+        if (v == intSet.emptyVal)
+          continue;
+        docs[d++] = v;
+      }
+      assert d == intSet.size();
+      final int size = d;
+
+      //sort them
+      Arrays.sort(docs, 0, size);
+
+      return new DocIdSetIterator() {
+        int idx = -1;
+        @Override
+        public int docID() {
+          if (idx < 0) {
+            return -1;
+          } else if (idx < size) {
+            return docs[idx];
+          } else {
+            return NO_MORE_DOCS;
+          }
+        }
+
+        @Override
+        public int nextDoc() throws IOException {
+          if (++idx < size)
+            return docs[idx];
+          return NO_MORE_DOCS;
+        }
+
+        @Override
+        public int advance(int target) throws IOException {
+          //for this small set this is likely faster vs. a binary search
+          // into the sorted array
+          return slowAdvance(target);
+        }
+
+        @Override
+        public long cost() {
+          return size;
+        }
+      };
+    }
+
+    @Override
+    public long ramBytesUsed() {
+      return RamUsageEstimator.alignObjectSize(
+            RamUsageEstimator.NUM_BYTES_OBJECT_REF
+          + RamUsageEstimator.NUM_BYTES_INT)
+          + intSet.ramBytesUsed();
+    }
+
+  }//class SmallDocSet
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/50a2f754/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/HeatmapFacetCounter.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/HeatmapFacetCounter.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/HeatmapFacetCounter.java
new file mode 100644
index 0000000..c6700cd
--- /dev/null
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/HeatmapFacetCounter.java
@@ -0,0 +1,310 @@
+/*
+ * 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.lucene.spatial.prefix;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+
+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.index.IndexReaderContext;
+import org.apache.lucene.spatial.prefix.tree.Cell;
+import org.apache.lucene.spatial.prefix.tree.CellIterator;
+import org.apache.lucene.spatial.prefix.tree.SpatialPrefixTree;
+import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.Bits;
+
+/**
+ * Computes spatial facets in two dimensions as a grid of numbers.  The data is often visualized as a so-called
+ * "heatmap", hence the name.
+ *
+ * @lucene.experimental
+ */
+public class HeatmapFacetCounter {
+  //TODO where should this code live? It could go to PrefixTreeFacetCounter, or maybe here in its own class is fine.
+
+  /** Maximum number of supported rows (or columns). */
+  public static final int MAX_ROWS_OR_COLUMNS = (int) Math.sqrt(ArrayUtil.MAX_ARRAY_LENGTH);
+  static {
+    Math.multiplyExact(MAX_ROWS_OR_COLUMNS, MAX_ROWS_OR_COLUMNS);//will throw if doesn't stay within integer
+  }
+
+  /** Response structure */
+  public static class Heatmap {
+    public final int columns;
+    public final int rows;
+    public final int[] counts;//in order of 1st column (all rows) then 2nd column (all rows) etc.
+    public final Rectangle region;
+
+    public Heatmap(int columns, int rows, Rectangle region) {
+      this.columns = columns;
+      this.rows = rows;
+      this.counts = new int[columns * rows];
+      this.region = region;
+    }
+
+    public int getCount(int x, int y) {
+      return counts[x * rows + y];
+    }
+
+    @Override
+    public String toString() {
+      return "Heatmap{" + columns + "x" + rows + " " + region + '}';
+    }
+  }
+
+  /**
+   * Calculates spatial 2D facets (aggregated counts) in a grid, sometimes called a heatmap.
+   * Facet computation is implemented by navigating the underlying indexed terms efficiently. If you don't know exactly
+   * what facetLevel to go to for a given input box but you have some sense of how many cells there should be relative
+   * to the size of the shape, then consider using the logic that {@link org.apache.lucene.spatial.prefix.PrefixTreeStrategy}
+   * uses when approximating what level to go to when indexing a shape given a distErrPct.
+   *
+   * @param context the IndexReader's context
+   * @param topAcceptDocs a Bits to limit counted docs.  If null, live docs are counted.
+   * @param inputShape the shape to gather grid squares for; typically a {@link Rectangle}.
+   *                   The <em>actual</em> heatmap area will usually be larger since the cells on the edge that overlap
+   *                   are returned. We always return a rectangle of integers even if the inputShape isn't a rectangle
+   *                   -- the non-intersecting cells will all be 0.
+   *                   If null is given, the entire world is assumed.
+   * @param facetLevel the target depth (detail) of cells.
+   * @param maxCells the maximum number of cells to return. If the cells exceed this count, an
+   */
+  public static Heatmap calcFacets(PrefixTreeStrategy strategy, IndexReaderContext context, Bits topAcceptDocs,
+                                   Shape inputShape, final int facetLevel, int maxCells) throws IOException {
+    if (maxCells > (MAX_ROWS_OR_COLUMNS * MAX_ROWS_OR_COLUMNS)) {
+      throw new IllegalArgumentException("maxCells (" + maxCells + ") should be <= " + MAX_ROWS_OR_COLUMNS);
+    }
+    if (inputShape == null) {
+      inputShape = strategy.getSpatialContext().getWorldBounds();
+    }
+    final Rectangle inputRect = inputShape.getBoundingBox();
+    //First get the rect of the cell at the bottom-left at depth facetLevel
+    final SpatialPrefixTree grid = strategy.getGrid();
+    final SpatialContext ctx = grid.getSpatialContext();
+    final Point cornerPt = ctx.makePoint(inputRect.getMinX(), inputRect.getMinY());
+    final CellIterator cellIterator = grid.getTreeCellIterator(cornerPt, facetLevel);
+    Cell cornerCell = null;
+    while (cellIterator.hasNext()) {
+      cornerCell = cellIterator.next();
+    }
+    assert cornerCell != null && cornerCell.getLevel() == facetLevel : "Cell not at target level: " + cornerCell;
+    final Rectangle cornerRect = (Rectangle) cornerCell.getShape();
+    assert cornerRect.hasArea();
+    //Now calculate the number of columns and rows necessary to cover the inputRect
+    double heatMinX = cornerRect.getMinX();//note: we might change this below...
+    final double cellWidth = cornerRect.getWidth();
+    final Rectangle worldRect = ctx.getWorldBounds();
+    final int columns = calcRowsOrCols(cellWidth, heatMinX, inputRect.getWidth(), inputRect.getMinX(), worldRect.getWidth());
+    final double heatMinY = cornerRect.getMinY();
+    final double cellHeight = cornerRect.getHeight();
+    final int rows = calcRowsOrCols(cellHeight, heatMinY, inputRect.getHeight(), inputRect.getMinY(), worldRect.getHeight());
+    assert rows > 0 && columns > 0;
+    if (columns > MAX_ROWS_OR_COLUMNS || rows > MAX_ROWS_OR_COLUMNS || columns * rows > maxCells) {
+      throw new IllegalArgumentException(
+          "Too many cells (" + columns + " x " + rows + ") for level " + facetLevel + " shape " + inputRect);
+    }
+
+    //Create resulting heatmap bounding rectangle & Heatmap object.
+    final double halfCellWidth = cellWidth / 2.0;
+    // if X world-wraps, use world bounds' range
+    if (columns * cellWidth + halfCellWidth > worldRect.getWidth()) {
+      heatMinX = worldRect.getMinX();
+    }
+    double heatMaxX = heatMinX + columns * cellWidth;
+    if (Math.abs(heatMaxX - worldRect.getMaxX()) < halfCellWidth) {//numeric conditioning issue
+      heatMaxX = worldRect.getMaxX();
+    } else if (heatMaxX > worldRect.getMaxX()) {//wraps dateline (won't happen if !geo)
+      heatMaxX = heatMaxX - worldRect.getMaxX() +  worldRect.getMinX();
+    }
+    final double halfCellHeight = cellHeight / 2.0;
+    double heatMaxY = heatMinY + rows * cellHeight;
+    if (Math.abs(heatMaxY - worldRect.getMaxY()) < halfCellHeight) {//numeric conditioning issue
+      heatMaxY = worldRect.getMaxY();
+    }
+
+    final Heatmap heatmap = new Heatmap(columns, rows, ctx.makeRectangle(heatMinX, heatMaxX, heatMinY, heatMaxY));
+
+    //All ancestor cell counts (of facetLevel) will be captured during facet visiting and applied later. If the data is
+    // just points then there won't be any ancestors.
+    //Facet count of ancestors covering all of the heatmap:
+    int[] allCellsAncestorCount = new int[1]; // single-element array so it can be accumulated in the inner class
+    //All other ancestors:
+    Map<Rectangle,Integer> ancestors = new HashMap<>();
+
+    //Now lets count some facets!
+    PrefixTreeFacetCounter.compute(strategy, context, topAcceptDocs, inputShape, facetLevel,
+        new PrefixTreeFacetCounter.FacetVisitor() {
+      @Override
+      public void visit(Cell cell, int count) {
+        final double heatMinX = heatmap.region.getMinX();
+        final Rectangle rect = (Rectangle) cell.getShape();
+        if (cell.getLevel() == facetLevel) {//heatmap level; count it directly
+          //convert to col & row
+          int column;
+          if (rect.getMinX() >= heatMinX) {
+            column = (int) Math.round((rect.getMinX() - heatMinX) / cellWidth);
+          } else { // due to dateline wrap
+            column = (int) Math.round((rect.getMinX() + 360 - heatMinX) / cellWidth);
+          }
+          int row = (int) Math.round((rect.getMinY() - heatMinY) / cellHeight);
+          //note: unfortunately, it's possible for us to visit adjacent cells to the heatmap (if the SpatialPrefixTree
+          // allows adjacent cells to overlap on the seam), so we need to skip them
+          if (column < 0 || column >= heatmap.columns || row < 0 || row >= heatmap.rows) {
+            return;
+          }
+          // increment
+          heatmap.counts[column * heatmap.rows + row] += count;
+
+        } else if (rect.relate(heatmap.region) == SpatialRelation.CONTAINS) {//containing ancestor
+          allCellsAncestorCount[0] += count;
+
+        } else { // ancestor
+          // note: not particularly efficient (possible put twice, and Integer wrapper); oh well
+          Integer existingCount = ancestors.put(rect, count);
+          if (existingCount != null) {
+            ancestors.put(rect, count + existingCount);
+          }
+        }
+      }
+    });
+
+    //Update the heatmap counts with ancestor counts
+
+    // Apply allCellsAncestorCount
+    if (allCellsAncestorCount[0] > 0) {
+      for (int i = 0; i < heatmap.counts.length; i++) {
+        heatmap.counts[i] += allCellsAncestorCount[0];
+      }
+    }
+
+    // Apply ancestors
+    //  note: This approach isn't optimized for a ton of ancestor cells. We'll potentially increment the same cells
+    //    multiple times in separate passes if any ancestors overlap. IF this poses a problem, we could optimize it
+    //    with additional complication by keeping track of intervals in a sorted tree structure (possible TreeMap/Set)
+    //    and iterate them cleverly such that we just make one pass at this stage.
+
+    int[] pair = new int[2];//output of intersectInterval
+    for (Map.Entry<Rectangle, Integer> entry : ancestors.entrySet()) {
+      Rectangle rect = entry.getKey();
+      final int count = entry.getValue();
+      //note: we approach this in a way that eliminates int overflow/underflow (think huge cell, tiny heatmap)
+      intersectInterval(heatMinY, heatMaxY, cellHeight, rows, rect.getMinY(), rect.getMaxY(), pair);
+      final int startRow = pair[0];
+      final int endRow = pair[1];
+
+      if (!heatmap.region.getCrossesDateLine()) {
+        intersectInterval(heatMinX, heatMaxX, cellWidth, columns, rect.getMinX(), rect.getMaxX(), pair);
+        final int startCol = pair[0];
+        final int endCol = pair[1];
+        incrementRange(heatmap, startCol, endCol, startRow, endRow, count);
+
+      } else {
+        //left half of dateline:
+        if (rect.getMaxX() >= heatMinX) {
+          final int leftColumns = (int) Math.round((180 - heatMinX) / cellWidth) + 1;
+          intersectInterval(heatMinX, 180, cellWidth, leftColumns, rect.getMinX(), rect.getMaxX(), pair);
+          final int startCol = pair[0];
+          final int endCol = pair[1];
+          incrementRange(heatmap, startCol, endCol, startRow, endRow, count);
+        }
+        //right half of dateline
+        if (rect.getMinY() <= heatMaxX) {
+          final int rightColumns = (int) Math.round(heatMaxX / cellWidth) + 1;
+          intersectInterval(0, heatMaxX, cellWidth, rightColumns, rect.getMinX(), rect.getMaxX(), pair);
+          final int startCol = pair[0];
+          final int endCol = pair[1];
+          incrementRange(heatmap, startCol, endCol, startRow, endRow, count);
+        }
+      }
+
+    }
+
+    return heatmap;
+  }
+
+  private static void intersectInterval(double heatMin, double heatMax, double heatCellLen, int heatLen,
+                                        double cellMin, double cellMax,
+                                        int[] out) {
+    //precondition: we know there's an intersection
+    if (heatMin >= cellMin) {
+      out[0] = 0;
+    } else {
+      out[0] = (int) Math.round((cellMin - heatMin) / heatCellLen);
+    }
+    if (heatMax <= cellMax) {
+      out[1] = heatLen - 1;
+    } else {
+      out[1] = (int) Math.round((cellMax - heatMin) / heatCellLen) - 1;
+    }
+  }
+
+  private static void incrementRange(Heatmap heatmap, int startColumn, int endColumn, int startRow, int endRow,
+                                     int count) {
+    //startColumn & startRow are not necessarily within the heatmap range; likewise numRows/columns may overlap.
+    if (startColumn < 0) {
+      endColumn += startColumn;
+      startColumn = 0;
+    }
+    endColumn = Math.min(heatmap.columns-1, endColumn);
+
+    if (startRow < 0) {
+      endRow += startRow;
+      startRow = 0;
+    }
+    endRow = Math.min(heatmap.rows-1, endRow);
+
+    if (startRow > endRow) {
+      return;//short-circuit
+    }
+    for (int c = startColumn; c <= endColumn; c++) {
+      int cBase = c * heatmap.rows;
+      for (int r = startRow; r <= endRow; r++) {
+        heatmap.counts[cBase + r] += count;
+      }
+    }
+  }
+
+  /** Computes the number of intervals (rows or columns) to cover a range given the sizes. */
+  private static int calcRowsOrCols(double cellRange, double cellMin, double requestRange, double requestMin,
+                                    double worldRange) {
+    assert requestMin >= cellMin;
+    //Idealistically this wouldn't be so complicated but we concern ourselves with overflow and edge cases
+    double range = (requestRange + (requestMin - cellMin));
+    if (range == 0) {
+      return 1;
+    }
+    final double intervals = Math.ceil(range / cellRange);
+    if (intervals > Integer.MAX_VALUE) {
+      return Integer.MAX_VALUE;//should result in an error soon (exceed thresholds)
+    }
+    // ensures we don't have more intervals than world bounds (possibly due to rounding/edge issue)
+    final long intervalsMax = Math.round(worldRange / cellRange);
+    if (intervalsMax > Integer.MAX_VALUE) {
+      //just return intervals
+      return (int) intervals;
+    }
+    return Math.min((int)intervalsMax, (int)intervals);
+  }
+
+  private HeatmapFacetCounter() {
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/50a2f754/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/IntersectsPrefixTreeQuery.java
----------------------------------------------------------------------
diff --git a/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/IntersectsPrefixTreeQuery.java b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/IntersectsPrefixTreeQuery.java
new file mode 100644
index 0000000..ccb0f89
--- /dev/null
+++ b/lucene/spatial-extras/src/java/org/apache/lucene/spatial/prefix/IntersectsPrefixTreeQuery.java
@@ -0,0 +1,95 @@
+/*
+ * 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.lucene.spatial.prefix;
+
+import java.io.IOException;
+
+import com.spatial4j.core.shape.Shape;
+import com.spatial4j.core.shape.SpatialRelation;
+import org.apache.lucene.index.LeafReaderContext;
+import org.apache.lucene.search.DocIdSet;
+import org.apache.lucene.spatial.prefix.tree.Cell;
+import org.apache.lucene.spatial.prefix.tree.SpatialPrefixTree;
+import org.apache.lucene.util.BitDocIdSet;
+import org.apache.lucene.util.FixedBitSet;
+
+/**
+ * A Query matching documents that have an {@link SpatialRelation#INTERSECTS}
+ * (i.e. not DISTINCT) relationship with a provided query shape.
+ *
+ * @lucene.internal
+ */
+public class IntersectsPrefixTreeQuery extends AbstractVisitingPrefixTreeQuery {
+
+  public IntersectsPrefixTreeQuery(Shape queryShape, String fieldName,
+                                   SpatialPrefixTree grid, int detailLevel,
+                                   int prefixGridScanLevel) {
+    super(queryShape, fieldName, grid, detailLevel, prefixGridScanLevel);
+  }
+
+  @Override
+  protected DocIdSet getDocIdSet(LeafReaderContext context) throws IOException {
+    /* Possible optimizations (in IN ADDITION TO THOSE LISTED IN VISITORTEMPLATE):
+
+    * If docFreq is 1 (or < than some small threshold), then check to see if we've already
+      collected it; if so short-circuit. Don't do this just for point data, as there is
+      no benefit, or only marginal benefit when multi-valued.
+
+    * Point query shape optimization when the only indexed data is a point (no leaves).  Result is a term query.
+
+     */
+    return new VisitorTemplate(context) {
+      private FixedBitSet results;
+
+      @Override
+      protected void start() {
+        results = new FixedBitSet(maxDoc);
+      }
+
+      @Override
+      protected DocIdSet finish() {
+        return new BitDocIdSet(results);
+      }
+
+      @Override
+      protected boolean visitPrefix(Cell cell) throws IOException {
+        if (cell.getShapeRel() == SpatialRelation.WITHIN || cell.getLevel() == detailLevel) {
+          collectDocs(results);
+          return false;
+        }
+        return true;
+      }
+
+      @Override
+      protected void visitLeaf(Cell cell) throws IOException {
+        collectDocs(results);
+      }
+
+    }.getDocIdSet();
+  }
+
+  @Override
+  public String toString(String field) {
+    return getClass().getSimpleName() + "(" +
+        "fieldName=" + fieldName + "," +
+        "queryShape=" + queryShape + "," +
+        "detailLevel=" + detailLevel + "," +
+        "prefixGridScanLevel=" + prefixGridScanLevel +
+        ")";
+  }
+
+}