You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ds...@apache.org on 2012/02/20 23:45:37 UTC

svn commit: r1291499 [4/12] - in /lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene: ./ src/ src/main/ src/main/java/ src/main/java/org/ src/main/java/org/apache/ src/main/java/org/apache/lucene/ src/main/java/org/apache/lucene/s...

Added: lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/strategy/package-info.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/strategy/package-info.java?rev=1291499&view=auto
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/strategy/package-info.java (added)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/strategy/package-info.java Mon Feb 20 22:45:32 2012
@@ -0,0 +1,26 @@
+/*
+ * 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.
+ */
+
+
+/**
+ * Lucene spatial search
+ *
+ * Check:
+ * http://blog.notdot.net/2009/11/Damn-Cool-Algorithms-Spatial-indexing-with-Quadtrees-and-Hilbert-Curves
+ */
+package org.apache.lucene.spatial.strategy;
+

Added: lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/strategy/prefix/PointPrefixTreeFieldCacheProvider.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/strategy/prefix/PointPrefixTreeFieldCacheProvider.java?rev=1291499&view=auto
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/strategy/prefix/PointPrefixTreeFieldCacheProvider.java (added)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/strategy/prefix/PointPrefixTreeFieldCacheProvider.java Mon Feb 20 22:45:32 2012
@@ -0,0 +1,43 @@
+/*
+ * 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.strategy.prefix;
+
+import org.apache.lucene.spatial.base.prefix.Node;
+import org.apache.lucene.spatial.base.prefix.SpatialPrefixTree;
+import org.apache.lucene.spatial.base.shape.Point;
+import org.apache.lucene.spatial.strategy.util.ShapeFieldCacheProvider;
+import org.apache.lucene.util.BytesRef;
+
+public class PointPrefixTreeFieldCacheProvider extends ShapeFieldCacheProvider<Point> {
+
+  final SpatialPrefixTree grid; //
+
+  public PointPrefixTreeFieldCacheProvider(SpatialPrefixTree grid, String shapeField, int defaultSize) {
+    super( shapeField, defaultSize );
+    this.grid = grid;
+  }
+
+  //A kluge that this is a field
+  private Node scanCell = null;
+
+  @Override
+  protected Point readShape(BytesRef term) {
+    scanCell = grid.getNode(term.bytes, term.offset, term.length, scanCell);
+    return scanCell.isLeaf() ? scanCell.getShape().getCenter() : null;
+  }
+}

Added: lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/strategy/prefix/PrefixCellsTokenizer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/strategy/prefix/PrefixCellsTokenizer.java?rev=1291499&view=auto
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/strategy/prefix/PrefixCellsTokenizer.java (added)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/strategy/prefix/PrefixCellsTokenizer.java Mon Feb 20 22:45:32 2012
@@ -0,0 +1,89 @@
+/*
+ * 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.strategy.prefix;
+
+import java.io.IOException;
+import java.io.Reader;
+
+import org.apache.lucene.analysis.Tokenizer;
+import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
+
+
+/**
+ *
+ */
+class PrefixCellsTokenizer extends Tokenizer {
+  public PrefixCellsTokenizer(Reader input) {
+    super(input);
+  }
+
+  private final CharTermAttribute termAtt = addAttribute(CharTermAttribute.class);
+
+  @Override
+  public final boolean incrementToken() throws IOException {
+    clearAttributes();
+    int length = 0;
+    char[] buffer = termAtt.buffer();
+    while (true) {
+      char c = (char) input.read();
+      if (c < 0) break;
+      if (c == 'a' || c == 'A') {
+        buffer[length++] = 'A';
+        continue;
+      }
+      if (c == 'b' || c == 'B') {
+        buffer[length++] = 'B';
+        continue;
+      }
+      if (c == 'c' || c == 'C') {
+        buffer[length++] = 'C';
+        continue;
+      }
+      if (c == 'd' || c == 'D') {
+        buffer[length++] = 'D';
+        continue;
+      }
+      if (c == '*') {
+        buffer[length++] = '*';
+        continue;
+      }
+      if (c == '+') {
+        buffer[length++] = '+';
+        continue;
+      }
+
+      if (length > 0) {
+        // Skip any other character
+        break;
+      }
+    }
+
+    termAtt.setLength(length);
+    return length > 0; // should only happen at the end
+  }
+
+  @Override
+  public final void end() {
+
+  }
+
+  @Override
+  public void reset(Reader input) throws IOException {
+    super.reset(input);
+  }
+}
\ No newline at end of file

Added: lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/strategy/prefix/PrefixTreeStrategy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/strategy/prefix/PrefixTreeStrategy.java?rev=1291499&view=auto
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/strategy/prefix/PrefixTreeStrategy.java (added)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/strategy/prefix/PrefixTreeStrategy.java Mon Feb 20 22:45:32 2012
@@ -0,0 +1,174 @@
+/*
+ * 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.strategy.prefix;
+
+import org.apache.lucene.analysis.TokenStream;
+import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.document.FieldType;
+import org.apache.lucene.document.StoredField;
+import org.apache.lucene.index.IndexableField;
+import org.apache.lucene.queries.function.ValueSource;
+import org.apache.lucene.spatial.base.distance.DistanceCalculator;
+import org.apache.lucene.spatial.base.prefix.Node;
+import org.apache.lucene.spatial.base.prefix.SpatialPrefixTree;
+import org.apache.lucene.spatial.base.query.SpatialArgs;
+import org.apache.lucene.spatial.base.shape.Point;
+import org.apache.lucene.spatial.base.shape.Shape;
+import org.apache.lucene.spatial.strategy.SimpleSpatialFieldInfo;
+import org.apache.lucene.spatial.strategy.SpatialStrategy;
+import org.apache.lucene.spatial.strategy.util.CachedDistanceValueSource;
+
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+
+public abstract class PrefixTreeStrategy extends SpatialStrategy<SimpleSpatialFieldInfo> {
+  protected final SpatialPrefixTree grid;
+  private final Map<String, PointPrefixTreeFieldCacheProvider> provider = new ConcurrentHashMap<String, PointPrefixTreeFieldCacheProvider>();
+  protected int defaultFieldValuesArrayLen = 2;
+  protected double distErrPct = SpatialArgs.DEFAULT_DIST_PRECISION;
+
+  public PrefixTreeStrategy(SpatialPrefixTree grid) {
+    super(grid.getSpatialContext());
+    this.grid = grid;
+  }
+
+  /** Used in the in-memory ValueSource as a default ArrayList length for this field's array of values, per doc. */
+  public void setDefaultFieldValuesArrayLen(int defaultFieldValuesArrayLen) {
+    this.defaultFieldValuesArrayLen = defaultFieldValuesArrayLen;
+  }
+
+  /** See {@link SpatialPrefixTree#getMaxLevelForPrecision(org.apache.lucene.spatial.base.shape.Shape, double)}. */
+  public void setDistErrPct(double distErrPct) {
+    this.distErrPct = distErrPct;
+  }
+
+  @Override
+  public IndexableField createField(SimpleSpatialFieldInfo fieldInfo, Shape shape, boolean index, boolean store) {
+    int detailLevel = grid.getMaxLevelForPrecision(shape,distErrPct);
+    List<Node> cells = grid.getNodes(shape, detailLevel, true);//true=intermediates cells
+    //If shape isn't a point, add a full-resolution center-point so that
+    // PrefixFieldCacheProvider has the center-points.
+    // TODO index each center of a multi-point? Yes/no?
+    if (!(shape instanceof Point)) {
+      Point ctr = shape.getCenter();
+      //TODO should be smarter; don't index 2 tokens for this in CellTokenizer. Harmless though.
+      cells.add(grid.getNodes(ctr,grid.getMaxLevels(),false).get(0));
+    }
+
+    String fname = fieldInfo.getFieldName();
+    if( store ) {
+      //TODO figure out how to re-use original string instead of reconstituting it.
+      String wkt = grid.getSpatialContext().toString(shape);
+      if( index ) {
+        Field f = new Field(fname,wkt,TYPE_STORED);
+        f.setTokenStream(new CellTokenStream(cells.iterator()));
+        return f;
+      }
+      return new StoredField(fname,wkt);
+    }
+    
+    if( index ) {
+      return new Field(fname,new CellTokenStream(cells.iterator()),TYPE_UNSTORED);
+    }
+    
+    throw new UnsupportedOperationException("Fields need to be indexed or store ["+fname+"]");
+  }
+
+  /* Indexed, tokenized, not stored. */
+  public static final FieldType TYPE_UNSTORED = new FieldType();
+
+  /* Indexed, tokenized, stored. */
+  public static final FieldType TYPE_STORED = new FieldType();
+
+  static {
+    TYPE_UNSTORED.setIndexed(true);
+    TYPE_UNSTORED.setTokenized(true);
+    TYPE_UNSTORED.setOmitNorms(true);
+    TYPE_UNSTORED.freeze();
+
+    TYPE_STORED.setStored(true);
+    TYPE_STORED.setIndexed(true);
+    TYPE_STORED.setTokenized(true);
+    TYPE_STORED.setOmitNorms(true);
+    TYPE_STORED.freeze();
+  }
+
+  /** Outputs the tokenString of a cell, and if its a leaf, outputs it again with the leaf byte. */
+  final static class CellTokenStream extends TokenStream {
+
+    private final CharTermAttribute termAtt = addAttribute(CharTermAttribute.class);
+
+    private Iterator<Node> iter = null;
+
+    public CellTokenStream(Iterator<Node> tokens) {
+      this.iter = tokens;
+    }
+
+    CharSequence nextTokenStringNeedingLeaf = null;
+
+    @Override
+    public boolean incrementToken() throws IOException {
+      clearAttributes();
+      if (nextTokenStringNeedingLeaf != null) {
+        termAtt.append(nextTokenStringNeedingLeaf);
+        termAtt.append((char) Node.LEAF_BYTE);
+        nextTokenStringNeedingLeaf = null;
+        return true;
+      }
+      if (iter.hasNext()) {
+        Node cell = iter.next();
+        CharSequence token = cell.getTokenString();
+        termAtt.append(token);
+        if (cell.isLeaf())
+          nextTokenStringNeedingLeaf = token;
+        return true;
+      }
+      return false;
+    }
+
+  }
+
+  @Override
+  public ValueSource makeValueSource(SpatialArgs args, SimpleSpatialFieldInfo fieldInfo) {
+    DistanceCalculator calc = grid.getSpatialContext().getDistCalc();
+    return makeValueSource(args, fieldInfo, calc);
+  }
+  
+  public ValueSource makeValueSource(SpatialArgs args, SimpleSpatialFieldInfo fieldInfo, DistanceCalculator calc) {
+    PointPrefixTreeFieldCacheProvider p = provider.get( fieldInfo.getFieldName() );
+    if( p == null ) {
+      synchronized (this) {//double checked locking idiom is okay since provider is threadsafe
+        p = provider.get( fieldInfo.getFieldName() );
+        if (p == null) {
+          p = new PointPrefixTreeFieldCacheProvider(grid, fieldInfo.getFieldName(), defaultFieldValuesArrayLen);
+          provider.put(fieldInfo.getFieldName(),p);
+        }
+      }
+    }
+    Point point = args.getShape().getCenter();
+    return new CachedDistanceValueSource(point, calc, p);
+  }
+
+  public SpatialPrefixTree getGrid() {
+    return grid;
+  }
+}

Added: lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/strategy/prefix/RecursivePrefixTreeFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/strategy/prefix/RecursivePrefixTreeFilter.java?rev=1291499&view=auto
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/strategy/prefix/RecursivePrefixTreeFilter.java (added)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/strategy/prefix/RecursivePrefixTreeFilter.java Mon Feb 20 22:45:32 2012
@@ -0,0 +1,192 @@
+/*
+ * 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.strategy.prefix;
+
+import org.apache.lucene.index.*;
+import org.apache.lucene.search.DocIdSet;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.search.Filter;
+import org.apache.lucene.spatial.base.shape.SpatialRelation;
+import org.apache.lucene.spatial.base.prefix.Node;
+import org.apache.lucene.spatial.base.prefix.SpatialPrefixTree;
+import org.apache.lucene.spatial.base.shape.Shape;
+import org.apache.lucene.util.Bits;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.OpenBitSet;
+import org.apache.lucene.util.StringHelper;
+
+import java.io.IOException;
+import java.util.LinkedList;
+
+/**
+ * Performs a spatial intersection filter against a field indexed with {@link SpatialPrefixTree}, a Trie.
+ * SPT yields terms (grids) at length 1 and at greater lengths corresponding to greater precisions.
+ * This filter recursively traverses each grid length and uses methods on {@link Shape} to efficiently know
+ * that all points at a prefix fit in the shape or not to either short-circuit unnecessary traversals or to efficiently
+ * load all enclosed points.
+ */
+public class RecursivePrefixTreeFilter extends Filter {
+
+  /* TODOs for future:
+
+Can a polygon query shape be optimized / made-simpler at recursive depths (e.g. intersection of shape + cell box)
+
+RE "scan" threshold:
+  // IF configured to do so, we could use term.freq() as an estimate on the number of places at this depth.  OR, perhaps
+  //  make estimates based on the total known term count at this level?
+  if (!scan) {
+    //Make some estimations on how many points there are at this level and how few there would need to be to set
+    // !scan to false.
+    long termsThreshold = (long) estimateNumberIndexedTerms(cell.length(),queryShape.getDocFreqExpenseThreshold(cell));
+    long thisOrd = termsEnum.ord();
+    scan = (termsEnum.seek(thisOrd+termsThreshold+1) == TermsEnum.SeekStatus.END
+            || !cell.contains(termsEnum.term()));
+    termsEnum.seek(thisOrd);//return to last position
+  }
+
+  */
+
+  private final String fieldName;
+  private final SpatialPrefixTree grid;
+  private final Shape queryShape;
+  private final int prefixGridScanLevel;//at least one less than grid.getMaxLevels()
+  private final int detailLevel;
+
+  public RecursivePrefixTreeFilter(String fieldName, SpatialPrefixTree grid, Shape queryShape, int prefixGridScanLevel,
+                             int detailLevel) {
+    this.fieldName = fieldName;
+    this.grid = grid;
+    this.queryShape = queryShape;
+    this.prefixGridScanLevel = Math.max(1,Math.min(prefixGridScanLevel,grid.getMaxLevels()-1));
+    this.detailLevel = detailLevel;
+    assert detailLevel <= grid.getMaxLevels();
+  }
+
+  @Override
+  public DocIdSet getDocIdSet(AtomicReaderContext ctx, Bits acceptDocs) throws IOException {
+    AtomicReader reader = ctx.reader();
+    OpenBitSet bits = new OpenBitSet(reader.maxDoc());
+    Terms terms = reader.terms(fieldName);
+    if (terms == null)
+      return null;
+    TermsEnum termsEnum = terms.iterator(null);
+    DocsEnum docsEnum = null;//cached for termsEnum.docs() calls
+    Node scanCell = null;
+
+    //cells is treated like a stack. LinkedList conveniently has bulk add to beginning. It's in sorted order so that we
+    //  always advance forward through the termsEnum index.
+    LinkedList<Node> cells = new LinkedList<Node>(
+        grid.getWorldNode().getSubCells(queryShape) );
+
+    //This is a recursive algorithm that starts with one or more "big" cells, and then recursively dives down into the
+    // first such cell that intersects with the query shape.  It's a depth first traversal because we don't move onto
+    // the next big cell (breadth) until we're completely done considering all smaller cells beneath it. For a given
+    // cell, if it's *within* the query shape then we can conveniently short-circuit the depth traversal and
+    // grab all documents assigned to this cell/term.  For an intersection of the cell and query shape, we either
+    // recursively step down another grid level or we decide heuristically (via prefixGridScanLevel) that there aren't
+    // that many points, and so we scan through all terms within this cell (i.e. the term starts with the cell's term),
+    // seeing which ones are within the query shape.
+    while(!cells.isEmpty()) {
+      final Node cell = cells.removeFirst();
+      final BytesRef cellTerm = new BytesRef(cell.getTokenBytes());
+      TermsEnum.SeekStatus seekStat = termsEnum.seekCeil(cellTerm);
+      if (seekStat == TermsEnum.SeekStatus.END)
+        break;
+      if (seekStat == TermsEnum.SeekStatus.NOT_FOUND)
+        continue;
+      if (cell.getLevel() == detailLevel || cell.isLeaf()) {
+        docsEnum = termsEnum.docs(acceptDocs, docsEnum, false);
+        addDocs(docsEnum,bits);
+      } else {//any other intersection
+        //If the next indexed term is the leaf marker, then add all of them
+        BytesRef nextCellTerm = termsEnum.next();
+        assert StringHelper.startsWith(nextCellTerm, cellTerm);
+        scanCell = grid.getNode(nextCellTerm.bytes, nextCellTerm.offset, nextCellTerm.length, scanCell);
+        if (scanCell.isLeaf()) {
+          docsEnum = termsEnum.docs(acceptDocs, docsEnum, false);
+          addDocs(docsEnum,bits);
+          termsEnum.next();//move pointer to avoid potential redundant addDocs() below
+        }
+
+        //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.
+        boolean scan = cell.getLevel() >= prefixGridScanLevel;//simple heuristic
+
+        if (!scan) {
+          //Divide & conquer
+          cells.addAll(0, cell.getSubCells(queryShape));//add to beginning
+        } else {
+          //Scan through all terms within this cell to see if they are within the queryShape. No seek()s.
+          for(BytesRef term = termsEnum.term(); term != null && StringHelper.startsWith(term,cellTerm); term = termsEnum.next()) {
+            scanCell = grid.getNode(term.bytes, term.offset, term.length, scanCell);
+            int termLevel = scanCell.getLevel();
+            if (termLevel > detailLevel)
+              continue;
+            if (termLevel == detailLevel || scanCell.isLeaf()) {
+              //TODO should put more thought into implications of box vs point
+              Shape cShape = termLevel == grid.getMaxLevels() ? scanCell.getCenter() : scanCell.getShape();
+              if(queryShape.relate(cShape, grid.getSpatialContext()) == SpatialRelation.DISJOINT)
+                continue;
+
+              docsEnum = termsEnum.docs(acceptDocs, docsEnum, false);
+              addDocs(docsEnum,bits);
+            }
+          }//term loop
+        }
+      }
+    }//cell loop
+
+    return bits;
+  }
+
+  private void addDocs(DocsEnum docsEnum, OpenBitSet bits) throws IOException {
+    int docid;
+    while ((docid = docsEnum.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
+      bits.fastSet(docid);
+    }
+  }
+
+  @Override
+  public String toString() {
+    return "GeoFilter{fieldName='" + fieldName + '\'' + ", shape=" + queryShape + '}';
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) return true;
+    if (o == null || getClass() != o.getClass()) return false;
+
+    RecursivePrefixTreeFilter that = (RecursivePrefixTreeFilter) o;
+
+    if (!fieldName.equals(that.fieldName)) return false;
+    //note that we don't need to look at grid since for the same field it should be the same
+    if (prefixGridScanLevel != that.prefixGridScanLevel) return false;
+    if (detailLevel != that.detailLevel) return false;
+    if (!queryShape.equals(that.queryShape)) return false;
+
+    return true;
+  }
+
+  @Override
+  public int hashCode() {
+    int result = fieldName.hashCode();
+    result = 31 * result + queryShape.hashCode();
+    result = 31 * result + detailLevel;
+    return result;
+  }
+}

Added: lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/strategy/prefix/RecursivePrefixTreeStrategy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/strategy/prefix/RecursivePrefixTreeStrategy.java?rev=1291499&view=auto
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/strategy/prefix/RecursivePrefixTreeStrategy.java (added)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/strategy/prefix/RecursivePrefixTreeStrategy.java Mon Feb 20 22:45:32 2012
@@ -0,0 +1,76 @@
+/*
+ * 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.strategy.prefix;
+
+import org.apache.lucene.search.Filter;
+import org.apache.lucene.search.FilteredQuery;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.queries.function.ValueSource;
+import org.apache.lucene.queries.function.FunctionQuery;
+import org.apache.lucene.spatial.base.exception.UnsupportedSpatialOperation;
+import org.apache.lucene.spatial.base.prefix.SpatialPrefixTree;
+import org.apache.lucene.spatial.base.query.SpatialArgs;
+import org.apache.lucene.spatial.base.query.SpatialOperation;
+import org.apache.lucene.spatial.base.shape.Shape;
+import org.apache.lucene.spatial.strategy.SimpleSpatialFieldInfo;
+
+
+public class RecursivePrefixTreeStrategy extends PrefixTreeStrategy {
+
+  private int prefixGridScanLevel;//TODO how is this customized?
+
+  public RecursivePrefixTreeStrategy(SpatialPrefixTree grid) {
+    super(grid);
+    prefixGridScanLevel = grid.getMaxLevels() - 4;//TODO this default constant is dependent on the prefix grid size
+  }
+
+  public void setPrefixGridScanLevel(int prefixGridScanLevel) {
+    this.prefixGridScanLevel = prefixGridScanLevel;
+  }
+
+  @Override
+  public String toString() {
+    return getClass().getSimpleName()+"(prefixGridScanLevel:"+prefixGridScanLevel+",SPG:("+ grid +"))";
+  }
+
+  @Override
+  public Query makeQuery(SpatialArgs args, SimpleSpatialFieldInfo fieldInfo) {
+    Filter f = makeFilter(args, fieldInfo);
+
+    ValueSource vs = makeValueSource(args, fieldInfo);
+    return new FilteredQuery( new FunctionQuery(vs), f );
+  }
+
+  @Override
+  public Filter makeFilter(SpatialArgs args, SimpleSpatialFieldInfo fieldInfo) {
+    final SpatialOperation op = args.getOperation();
+    if (! SpatialOperation.is(op, SpatialOperation.IsWithin, SpatialOperation.Intersects, SpatialOperation.BBoxWithin))
+      throw new UnsupportedSpatialOperation(op);
+
+    Shape qshape = args.getShape();
+
+    int detailLevel = grid.getMaxLevelForPrecision(qshape,args.getDistPrecision());
+
+    return new RecursivePrefixTreeFilter(
+        fieldInfo.getFieldName(), grid,qshape, prefixGridScanLevel, detailLevel);
+  }
+}
+
+
+
+

Added: lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/strategy/prefix/TermQueryPrefixTreeStrategy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/strategy/prefix/TermQueryPrefixTreeStrategy.java?rev=1291499&view=auto
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/strategy/prefix/TermQueryPrefixTreeStrategy.java (added)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/strategy/prefix/TermQueryPrefixTreeStrategy.java Mon Feb 20 22:45:32 2012
@@ -0,0 +1,62 @@
+/*
+ * 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.strategy.prefix;
+
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.*;
+import org.apache.lucene.spatial.base.exception.UnsupportedSpatialOperation;
+import org.apache.lucene.spatial.base.prefix.Node;
+import org.apache.lucene.spatial.base.prefix.SpatialPrefixTree;
+import org.apache.lucene.spatial.base.query.SpatialArgs;
+import org.apache.lucene.spatial.base.query.SpatialOperation;
+import org.apache.lucene.spatial.base.shape.Shape;
+import org.apache.lucene.spatial.strategy.SimpleSpatialFieldInfo;
+
+import java.util.List;
+
+public class TermQueryPrefixTreeStrategy extends PrefixTreeStrategy {
+
+  public TermQueryPrefixTreeStrategy(SpatialPrefixTree grid) {
+    super(grid);
+  }
+
+  @Override
+  public Filter makeFilter(SpatialArgs args, SimpleSpatialFieldInfo fieldInfo) {
+    return new QueryWrapperFilter( makeQuery(args, fieldInfo) );
+  }
+
+  @Override
+  public Query makeQuery(SpatialArgs args, SimpleSpatialFieldInfo fieldInfo) {
+    if (args.getOperation() != SpatialOperation.Intersects &&
+        args.getOperation() != SpatialOperation.IsWithin &&
+        args.getOperation() != SpatialOperation.Overlaps ){
+      // TODO -- can translate these other query types
+      throw new UnsupportedSpatialOperation(args.getOperation());
+    }
+    Shape qshape = args.getShape();
+    int detailLevel = grid.getMaxLevelForPrecision(qshape, args.getDistPrecision());
+    List<Node> cells = grid.getNodes(qshape, detailLevel, false);
+
+    BooleanQuery booleanQuery = new BooleanQuery();
+    for (Node cell : cells) {
+      booleanQuery.add(new TermQuery(new Term(fieldInfo.getFieldName(), cell.getTokenString())), BooleanClause.Occur.SHOULD);
+    }
+    return booleanQuery;
+  }
+
+}

Added: lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/strategy/prefix/package-info.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/strategy/prefix/package-info.java?rev=1291499&view=auto
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/strategy/prefix/package-info.java (added)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/strategy/prefix/package-info.java Mon Feb 20 22:45:32 2012
@@ -0,0 +1,22 @@
+/*
+ * 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.
+ */
+
+/**
+ * Prefix Tree Strategy
+ */
+package org.apache.lucene.spatial.strategy.prefix;
+

Added: lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/strategy/util/CachedDistanceValueSource.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/strategy/util/CachedDistanceValueSource.java?rev=1291499&view=auto
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/strategy/util/CachedDistanceValueSource.java (added)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/strategy/util/CachedDistanceValueSource.java Mon Feb 20 22:45:32 2012
@@ -0,0 +1,106 @@
+/*
+ * 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.strategy.util;
+
+import org.apache.commons.lang.builder.EqualsBuilder;
+import org.apache.commons.lang.builder.HashCodeBuilder;
+import org.apache.lucene.index.AtomicReaderContext;
+import org.apache.lucene.queries.function.FunctionValues;
+import org.apache.lucene.queries.function.ValueSource;
+import org.apache.lucene.spatial.base.distance.DistanceCalculator;
+import org.apache.lucene.spatial.base.shape.Point;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+
+/**
+ *
+ * An implementation of the Lucene ValueSource model to support spatial relevance ranking.
+ *
+ */
+public class CachedDistanceValueSource extends ValueSource {
+
+  private final ShapeFieldCacheProvider<Point> provider;
+  private final DistanceCalculator calculator;
+  private final Point from;
+
+  public CachedDistanceValueSource(Point from, DistanceCalculator calc, ShapeFieldCacheProvider<Point> provider) {
+    this.from = from;
+    this.provider = provider;
+    this.calculator = calc;
+  }
+
+  @Override
+  public String description() {
+    return "DistanceValueSource("+calculator+")";
+  }
+
+  @Override
+  public FunctionValues getValues(Map context, AtomicReaderContext readerContext) throws IOException {
+    final ShapeFieldCache<Point> cache =
+      provider.getCache(readerContext.reader());
+
+    return new FunctionValues() {
+      @Override
+      public float floatVal(int doc) {
+        return (float) doubleVal(doc);
+      }
+
+      @Override
+      public double doubleVal(int doc) {
+        List<Point> vals = cache.getShapes( doc );
+        if( vals != null ) {
+          double v = calculator.distance(from, vals.get(0));
+          for( int i=1; i<vals.size(); i++ ) {
+            v = Math.min(v, calculator.distance(from, vals.get(i)));
+          }
+          return v;
+        }
+        return Double.NaN; // ?? maybe max?
+      }
+
+      @Override
+      public String toString(int doc) {
+        return description() + "=" + floatVal(doc);
+      }
+    };
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (obj == null) { return false; }
+    if (obj == this) { return true; }
+    if (obj.getClass() != getClass()) {
+      return false;
+    }
+    CachedDistanceValueSource rhs = (CachedDistanceValueSource) obj;
+    return new EqualsBuilder()
+                  .append(calculator, rhs.calculator)
+                  .append(from, rhs.from)
+                  .isEquals();
+  }
+
+  @Override
+  public int hashCode() {
+    return new HashCodeBuilder(31, 97).
+        append(calculator).
+        append(from).
+        toHashCode();
+  }
+}

Added: lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/strategy/util/CachingDoubleValueSource.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/strategy/util/CachingDoubleValueSource.java?rev=1291499&view=auto
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/strategy/util/CachingDoubleValueSource.java (added)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/strategy/util/CachingDoubleValueSource.java Mon Feb 20 22:45:32 2012
@@ -0,0 +1,91 @@
+/*
+ * 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.strategy.util;
+
+import org.apache.commons.lang.builder.HashCodeBuilder;
+import org.apache.lucene.index.AtomicReaderContext;
+import org.apache.lucene.queries.function.FunctionValues;
+import org.apache.lucene.queries.function.ValueSource;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+
+public class CachingDoubleValueSource extends ValueSource {
+
+  final ValueSource source;
+  final Map<Integer, Double> cache;
+
+  public CachingDoubleValueSource( ValueSource source )
+  {
+    this.source = source;
+    cache = new HashMap<Integer, Double>();
+  }
+
+  @Override
+  public String description() {
+    return "Cached["+source.description()+"]";
+  }
+
+  @Override
+  public FunctionValues getValues(Map context, AtomicReaderContext readerContext) throws IOException {
+    final int base = readerContext.docBase;
+    final FunctionValues vals = source.getValues(context,readerContext);
+    return new FunctionValues() {
+
+      @Override
+      public double doubleVal(int doc) {
+        Integer key = Integer.valueOf( base+doc );
+        Double v = cache.get( key );
+        if( v == null ) {
+          v = Double.valueOf( vals.doubleVal(doc) );
+          cache.put( key, v );
+        }
+        return v.doubleValue();
+      }
+
+      @Override
+      public float floatVal(int doc) {
+        return (float)doubleVal(doc);
+      }
+
+      @Override
+      public String toString(int doc) {
+        return doubleVal(doc)+"";
+      }
+    };
+  }
+
+  @Override
+  public boolean equals(Object obj) {
+    if (obj == null) { return false; }
+    if (obj == this) { return true; }
+    if (obj.getClass() != getClass()) {
+      return false;
+    }
+    CachingDoubleValueSource rhs = (CachingDoubleValueSource) obj;
+    return source.equals( rhs.source );
+  }
+
+  @Override
+  public int hashCode() {
+    return new HashCodeBuilder(61, 23).
+      append(source).
+      toHashCode();
+  }
+}

Added: lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/strategy/util/NumericFieldInfo.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/strategy/util/NumericFieldInfo.java?rev=1291499&view=auto
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/strategy/util/NumericFieldInfo.java (added)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/strategy/util/NumericFieldInfo.java Mon Feb 20 22:45:32 2012
@@ -0,0 +1,51 @@
+/*
+ * 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.strategy.util;
+
+import org.apache.lucene.analysis.NumericTokenStream;
+import org.apache.lucene.document.DoubleField;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.document.FieldType;
+import org.apache.lucene.index.IndexableField;
+
+/**
+ * Hold some of the parameters used by solr...
+ */
+public class NumericFieldInfo {
+  public int precisionStep = 8; // same as solr default
+  public boolean store = true;
+  public boolean index = true;
+
+  public void setPrecisionStep( int p ) {
+    precisionStep = p;
+    if (precisionStep<=0 || precisionStep>=64)
+      precisionStep=Integer.MAX_VALUE;
+  }
+
+  public IndexableField createDouble( String name, double v ) {
+    if (!store && !index)
+      throw new IllegalArgumentException("field must be indexed or stored");
+
+    FieldType fieldType = new FieldType(DoubleField.TYPE);
+    fieldType.setStored(store);
+    fieldType.setIndexed(index);
+    fieldType.setNumericPrecisionStep(precisionStep);
+    return new DoubleField(name,v,fieldType);
+
+  }
+}

Added: lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/strategy/util/ShapeFieldCache.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/strategy/util/ShapeFieldCache.java?rev=1291499&view=auto
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/strategy/util/ShapeFieldCache.java (added)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/strategy/util/ShapeFieldCache.java Mon Feb 20 22:45:32 2012
@@ -0,0 +1,46 @@
+/*
+ * 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.strategy.util;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.lucene.spatial.base.shape.Shape;
+
+public class ShapeFieldCache<T extends Shape> {
+  private List<T>[] cache;
+  public int defaultLength;
+
+  @SuppressWarnings({"unchecked"})
+  public ShapeFieldCache( int length, int defaultLength ) {
+    cache = new List[length];
+    this.defaultLength= defaultLength;
+  }
+
+  public void add( int docid, T s ) {
+    List<T> list = cache[docid];
+    if( list == null ) {
+      list = cache[docid] = new ArrayList<T>(defaultLength);
+    }
+    list.add( s );
+  }
+
+  public List<T> getShapes( int docid ) {
+    return cache[docid];
+  }
+}

Added: lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/strategy/util/ShapeFieldCacheProvider.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/strategy/util/ShapeFieldCacheProvider.java?rev=1291499&view=auto
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/strategy/util/ShapeFieldCacheProvider.java (added)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/strategy/util/ShapeFieldCacheProvider.java Mon Feb 20 22:45:32 2012
@@ -0,0 +1,82 @@
+/*
+ * 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.strategy.util;
+
+import org.apache.lucene.index.*;
+import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.spatial.base.shape.Shape;
+import org.apache.lucene.util.BytesRef;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.WeakHashMap;
+
+
+public abstract class ShapeFieldCacheProvider<T extends Shape> {
+  static final Logger log = LoggerFactory.getLogger(ShapeFieldCacheProvider.class);
+
+  // it may be a List<T> or T
+  WeakHashMap<IndexReader, ShapeFieldCache<T>> sidx = new WeakHashMap<IndexReader, ShapeFieldCache<T>>();
+
+  protected final int defaultSize;
+  protected final String shapeField;
+
+  public ShapeFieldCacheProvider(String shapeField, int defaultSize) {
+    this.shapeField = shapeField;
+    this.defaultSize = defaultSize;
+  }
+
+  protected abstract T readShape( BytesRef term );
+
+  public synchronized ShapeFieldCache<T> getCache(AtomicReader reader) throws IOException {
+    ShapeFieldCache<T> idx = sidx.get(reader);
+    if (idx != null) {
+      return idx;
+    }
+    long startTime = System.currentTimeMillis();
+
+    log.info("Building Cache [" + reader.maxDoc() + "]");
+    idx = new ShapeFieldCache<T>(reader.maxDoc(),defaultSize);
+    int count = 0;
+    DocsEnum docs = null;
+    Terms terms = reader.terms(shapeField);
+    TermsEnum te = null;
+    if (terms != null) {
+      te = terms.iterator(te);
+      BytesRef term = te.next();
+      while (term != null) {
+        T shape = readShape(term);
+        if( shape != null ) {
+          docs = te.docs(null, docs, false);
+          Integer docid = docs.nextDoc();
+          while (docid != DocIdSetIterator.NO_MORE_DOCS) {
+            idx.add( docid, shape );
+            docid = docs.nextDoc();
+            count++;
+          }
+        }
+        term = te.next();
+      }
+    }
+    sidx.put(reader, idx);
+    long elapsed = System.currentTimeMillis() - startTime;
+    log.info("Cached: [" + count + " in " + elapsed + "ms] " + idx);
+    return idx;
+  }
+}

Added: lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/strategy/util/StringListTokenizer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/strategy/util/StringListTokenizer.java?rev=1291499&view=auto
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/strategy/util/StringListTokenizer.java (added)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/strategy/util/StringListTokenizer.java Mon Feb 20 22:45:32 2012
@@ -0,0 +1,58 @@
+/*
+ * 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.strategy.util;
+
+import java.io.IOException;
+import java.io.Reader;
+import java.util.Iterator;
+
+import org.apache.lucene.analysis.TokenStream;
+import org.apache.lucene.analysis.Tokenizer;
+import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
+
+/**
+ * Put a list of strings directly into the token stream
+ */
+public final class StringListTokenizer extends TokenStream {
+
+  private final CharTermAttribute termAtt = addAttribute(CharTermAttribute.class);
+
+  private final Iterable<String> tokens;
+  private Iterator<String> iter = null;
+
+  public StringListTokenizer(Iterable<String> tokens) {
+    this.tokens = tokens;
+  }
+
+  @Override
+  public boolean incrementToken() {
+    if (iter.hasNext()) {
+      clearAttributes();
+      String t = iter.next();
+      termAtt.append(t);
+      return true;
+    }
+    return false;
+  }
+
+  @Override
+  public void reset() throws IOException {
+    super.reset();
+    iter = tokens.iterator();
+  }
+}
\ No newline at end of file

Added: lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/strategy/util/TruncateFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/strategy/util/TruncateFilter.java?rev=1291499&view=auto
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/strategy/util/TruncateFilter.java (added)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/strategy/util/TruncateFilter.java Mon Feb 20 22:45:32 2012
@@ -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.strategy.util;
+
+
+import java.io.IOException;
+
+import org.apache.lucene.analysis.TokenFilter;
+import org.apache.lucene.analysis.TokenStream;
+import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
+
+public class TruncateFilter extends TokenFilter {
+
+  private final int maxTokenLength;
+
+  private final CharTermAttribute termAttr = addAttribute(CharTermAttribute.class);
+
+  public TruncateFilter(TokenStream in, int maxTokenLength) {
+    super(in);
+    this.maxTokenLength = maxTokenLength;
+  }
+
+  @Override
+  public final boolean incrementToken() throws IOException {
+    if (!input.incrementToken()) {
+      return false;
+    }
+
+    if (termAttr.length() > maxTokenLength) {
+      termAttr.setLength(maxTokenLength);
+    }
+    return true;
+  }
+}

Added: lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/strategy/util/ValueSourceFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/strategy/util/ValueSourceFilter.java?rev=1291499&view=auto
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/strategy/util/ValueSourceFilter.java (added)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/strategy/util/ValueSourceFilter.java Mon Feb 20 22:45:32 2012
@@ -0,0 +1,59 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.lucene.spatial.strategy.util;
+
+import org.apache.lucene.index.AtomicReaderContext;
+import org.apache.lucene.queries.function.FunctionValues;
+import org.apache.lucene.queries.function.ValueSource;
+import org.apache.lucene.search.DocIdSet;
+import org.apache.lucene.search.Filter;
+import org.apache.lucene.search.FilteredDocIdSet;
+import org.apache.lucene.util.Bits;
+
+import java.io.IOException;
+
+public class ValueSourceFilter extends Filter {
+
+  final Filter startingFilter;
+  final ValueSource source;
+  final double min;
+  final double max;
+
+  public ValueSourceFilter( Filter startingFilter, ValueSource source, double min, double max )
+  {
+    if (startingFilter == null) {
+      throw new IllegalArgumentException("please provide a non-null startingFilter; you can use QueryWrapperFilter(MatchAllDocsQuery) as a no-op filter");
+    }
+    this.startingFilter = startingFilter;
+    this.source = source;
+    this.min = min;
+    this.max = max;
+  }
+
+  @Override
+  public DocIdSet getDocIdSet(AtomicReaderContext context, Bits acceptDocs) throws IOException {
+    final FunctionValues values = source.getValues( null, context );
+    return new FilteredDocIdSet(startingFilter.getDocIdSet(context, acceptDocs)) {
+      @Override
+      public boolean match(int doc) {
+        double val = values.doubleVal( doc );
+        return val > min && val < max;
+      }
+    };
+  }
+}

Added: lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/strategy/vector/DistanceValueSource.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/strategy/vector/DistanceValueSource.java?rev=1291499&view=auto
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/strategy/vector/DistanceValueSource.java (added)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/strategy/vector/DistanceValueSource.java Mon Feb 20 22:45:32 2012
@@ -0,0 +1,130 @@
+/*
+ * 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.strategy.vector;
+
+import org.apache.commons.lang.builder.EqualsBuilder;
+import org.apache.commons.lang.builder.HashCodeBuilder;
+import org.apache.lucene.index.AtomicReader;
+import org.apache.lucene.index.AtomicReaderContext;
+import org.apache.lucene.queries.function.FunctionValues;
+import org.apache.lucene.queries.function.ValueSource;
+import org.apache.lucene.search.FieldCache;
+import org.apache.lucene.search.FieldCache.DoubleParser;
+import org.apache.lucene.spatial.base.distance.DistanceCalculator;
+import org.apache.lucene.spatial.base.shape.Point;
+import org.apache.lucene.spatial.base.shape.simple.PointImpl;
+import org.apache.lucene.util.Bits;
+
+import java.io.IOException;
+import java.util.Map;
+
+/**
+ *
+ * An implementation of the Lucene ValueSource model to support spatial relevance ranking.
+ *
+ */
+public class DistanceValueSource extends ValueSource {
+
+  private final TwoDoublesFieldInfo fields;
+  private final DistanceCalculator calculator;
+  private final Point from;
+  private final DoubleParser parser;
+
+  /**
+   * Constructor.
+   */
+  public DistanceValueSource(Point from, DistanceCalculator calc, TwoDoublesFieldInfo fields, DoubleParser parser) {
+    this.from = from;
+    this.fields = fields;
+    this.calculator = calc;
+    this.parser = parser;
+  }
+
+  /**
+   * Returns the ValueSource description.
+   */
+  @Override
+  public String description() {
+    return "DistanceValueSource("+calculator+")";
+  }
+
+
+  /**
+   * Returns the FunctionValues used by the function query.
+   */
+  @Override
+  public FunctionValues getValues(Map context, AtomicReaderContext readerContext) throws IOException {
+    AtomicReader reader = readerContext.reader();
+
+    final double[] ptX = FieldCache.DEFAULT.getDoubles(reader, fields.getFieldNameX(), true);
+    final double[] ptY = FieldCache.DEFAULT.getDoubles(reader, fields.getFieldNameY(), true);
+    final Bits validX =  FieldCache.DEFAULT.getDocsWithField(reader, fields.getFieldNameX());
+    final Bits validY =  FieldCache.DEFAULT.getDocsWithField(reader, fields.getFieldNameY());
+
+    return new FunctionValues() {
+      @Override
+      public float floatVal(int doc) {
+        return (float) doubleVal(doc);
+      }
+
+      @Override
+      public double doubleVal(int doc) {
+        // make sure it has minX and area
+        if (validX.get(doc) && validY.get(doc)) {
+          PointImpl pt = new PointImpl( ptX[doc],  ptY[doc] );
+          return calculator.distance(from, pt);
+        }
+        return 0;
+      }
+
+      @Override
+      public String toString(int doc) {
+        return description() + "=" + floatVal(doc);
+      }
+    };
+  }
+
+  /**
+   * Determines if this ValueSource is equal to another.
+   * @param obj the ValueSource to compare
+   * @return <code>true</code> if the two objects are based upon the same query envelope
+   */
+  @Override
+  public boolean equals(Object obj) {
+    if (obj == null) { return false; }
+    if (obj == this) { return true; }
+    if (obj.getClass() != getClass()) {
+      return false;
+    }
+    DistanceValueSource rhs = (DistanceValueSource) obj;
+    return new EqualsBuilder()
+                  .append(calculator, rhs.calculator)
+                  .append(from, rhs.from)
+                  .append(fields, rhs.fields)
+                  .isEquals();
+  }
+
+  @Override
+  public int hashCode() {
+    return new HashCodeBuilder(59, 7).
+        append(calculator).
+        append(from).
+        append(fields).
+        toHashCode();
+  }
+}

Added: lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/strategy/vector/TwoDoublesFieldInfo.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/strategy/vector/TwoDoublesFieldInfo.java?rev=1291499&view=auto
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/strategy/vector/TwoDoublesFieldInfo.java (added)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/strategy/vector/TwoDoublesFieldInfo.java Mon Feb 20 22:45:32 2012
@@ -0,0 +1,48 @@
+/*
+ * 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.strategy.vector;
+
+import org.apache.lucene.spatial.strategy.SpatialFieldInfo;
+
+public class TwoDoublesFieldInfo implements SpatialFieldInfo {
+
+  public static final String SUFFIX_X = "__x";
+  public static final String SUFFIX_Y = "__y";
+
+  private final String fieldName;
+  private final String fieldNameX;
+  private final String fieldNameY;
+
+  public TwoDoublesFieldInfo(String fieldNamePrefix) {
+    fieldName = fieldNamePrefix;
+    fieldNameX = fieldNamePrefix + SUFFIX_X;
+    fieldNameY = fieldNamePrefix + SUFFIX_Y;
+  }
+
+  public String getFieldName() {
+    return fieldName;
+  }
+
+  public String getFieldNameX() {
+    return fieldNameX;
+  }
+
+  public String getFieldNameY() {
+    return fieldNameY;
+  }
+}

Added: lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/strategy/vector/TwoDoublesStrategy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/strategy/vector/TwoDoublesStrategy.java?rev=1291499&view=auto
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/strategy/vector/TwoDoublesStrategy.java (added)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/main/java/org/apache/lucene/spatial/strategy/vector/TwoDoublesStrategy.java Mon Feb 20 22:45:32 2012
@@ -0,0 +1,236 @@
+/*
+ * 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.strategy.vector;
+
+import org.apache.lucene.document.Field;
+import org.apache.lucene.document.FieldType;
+import org.apache.lucene.index.IndexableField;
+import org.apache.lucene.queries.function.FunctionQuery;
+import org.apache.lucene.queries.function.ValueSource;
+import org.apache.lucene.search.*;
+import org.apache.lucene.search.FieldCache.DoubleParser;
+import org.apache.lucene.spatial.base.context.SpatialContext;
+import org.apache.lucene.spatial.base.exception.InvalidShapeException;
+import org.apache.lucene.spatial.base.exception.UnsupportedSpatialOperation;
+import org.apache.lucene.spatial.base.query.SpatialArgs;
+import org.apache.lucene.spatial.base.query.SpatialOperation;
+import org.apache.lucene.spatial.base.shape.Circle;
+import org.apache.lucene.spatial.base.shape.Point;
+import org.apache.lucene.spatial.base.shape.Rectangle;
+import org.apache.lucene.spatial.base.shape.Shape;
+import org.apache.lucene.spatial.strategy.SpatialStrategy;
+import org.apache.lucene.spatial.strategy.util.CachingDoubleValueSource;
+import org.apache.lucene.spatial.strategy.util.NumericFieldInfo;
+import org.apache.lucene.spatial.strategy.util.ValueSourceFilter;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+public class TwoDoublesStrategy extends SpatialStrategy<TwoDoublesFieldInfo> {
+
+  static final Logger log = LoggerFactory.getLogger(TwoDoublesStrategy.class);
+
+  private final NumericFieldInfo finfo;
+  private final DoubleParser parser;
+
+  public TwoDoublesStrategy(SpatialContext ctx, NumericFieldInfo finfo, DoubleParser parser) {
+    super(ctx);
+    this.finfo = finfo;
+    this.parser = parser;
+  }
+
+  @Override
+  public boolean isPolyField() {
+    return true;
+  }
+
+  @Override
+  public IndexableField[] createFields(TwoDoublesFieldInfo fieldInfo,
+      Shape shape, boolean index, boolean store) {
+    if( shape instanceof Point ) {
+      Point point = (Point)shape;
+
+      IndexableField[] f = new IndexableField[(index ? 2 : 0) + (store ? 1 : 0)];
+      if (index) {
+        f[0] = finfo.createDouble( fieldInfo.getFieldNameX(), point.getX() );
+        f[1] = finfo.createDouble( fieldInfo.getFieldNameY(), point.getY() );
+      }
+      if(store) {
+        FieldType customType = new FieldType();
+        customType.setStored(true);
+        f[f.length-1] = new Field( fieldInfo.getFieldName(), ctx.toString( shape ), customType );
+      }
+      return f;
+    }
+    if( !ignoreIncompatibleGeometry ) {
+      throw new IllegalArgumentException( "TwoDoublesStrategy can not index: "+shape );
+    }
+    return null;
+  }
+
+  @Override
+  public IndexableField createField(TwoDoublesFieldInfo indexInfo, Shape shape,
+      boolean index, boolean store) {
+    throw new UnsupportedOperationException("Point is poly field");
+  }
+
+  @Override
+  public ValueSource makeValueSource(SpatialArgs args, TwoDoublesFieldInfo fieldInfo) {
+    Point p = args.getShape().getCenter();
+    return new DistanceValueSource(p, ctx.getDistCalc(), fieldInfo, parser);
+  }
+
+  @Override
+  public Filter makeFilter(SpatialArgs args, TwoDoublesFieldInfo fieldInfo) {
+    if( args.getShape() instanceof Circle) {
+      if( SpatialOperation.is( args.getOperation(),
+          SpatialOperation.Intersects,
+          SpatialOperation.IsWithin )) {
+        Circle circle = (Circle)args.getShape();
+        Query bbox = makeWithin(circle.getBoundingBox(), fieldInfo);
+
+        // Make the ValueSource
+        ValueSource valueSource = makeValueSource(args, fieldInfo);
+
+        return new ValueSourceFilter(
+            new QueryWrapperFilter( bbox ), valueSource, 0, circle.getDistance() );
+      }
+    }
+    return new QueryWrapperFilter( makeQuery(args, fieldInfo) );
+  }
+
+  @Override
+  public Query makeQuery(SpatialArgs args, TwoDoublesFieldInfo fieldInfo) {
+    // For starters, just limit the bbox
+    Shape shape = args.getShape();
+    if (!(shape instanceof Rectangle)) {
+      throw new InvalidShapeException("A rectangle is the only supported at this time, not "+shape.getClass());//TODO
+    }
+    Rectangle bbox = (Rectangle) shape;
+    if (bbox.getCrossesDateLine()) {
+      throw new UnsupportedOperationException( "Crossing dateline not yet supported" );
+    }
+
+    ValueSource valueSource = null;
+
+    Query spatial = null;
+    SpatialOperation op = args.getOperation();
+
+    if( SpatialOperation.is( op,
+        SpatialOperation.BBoxWithin,
+        SpatialOperation.BBoxIntersects ) ) {
+        spatial = makeWithin(bbox, fieldInfo);
+    }
+    else if( SpatialOperation.is( op,
+      SpatialOperation.Intersects,
+      SpatialOperation.IsWithin ) ) {
+      spatial = makeWithin(bbox, fieldInfo);
+      if( args.getShape() instanceof Circle) {
+        Circle circle = (Circle)args.getShape();
+
+        // Make the ValueSource
+        valueSource = makeValueSource(args, fieldInfo);
+
+        ValueSourceFilter vsf = new ValueSourceFilter(
+            new QueryWrapperFilter( spatial ), valueSource, 0, circle.getDistance() );
+
+        spatial = new FilteredQuery( new MatchAllDocsQuery(), vsf );
+      }
+    }
+    else if( op == SpatialOperation.IsDisjointTo ) {
+      spatial =  makeDisjoint(bbox, fieldInfo);
+    }
+
+    if( spatial == null ) {
+      throw new UnsupportedSpatialOperation(args.getOperation());
+    }
+
+    try {
+      if( valueSource != null ) {
+        valueSource = new CachingDoubleValueSource(valueSource);
+      }
+      else {
+        valueSource = makeValueSource(args, fieldInfo);
+      }
+      Query spatialRankingQuery = new FunctionQuery(valueSource);
+      BooleanQuery bq = new BooleanQuery();
+      bq.add(spatial,BooleanClause.Occur.MUST);
+      bq.add(spatialRankingQuery,BooleanClause.Occur.MUST);
+      return bq;
+    } catch(Exception ex) {
+      log.warn("error making score", ex);
+    }
+    return spatial;
+  }
+
+  /**
+   * Constructs a query to retrieve documents that fully contain the input envelope.
+   * @return the spatial query
+   */
+  private Query makeWithin(Rectangle bbox, TwoDoublesFieldInfo fieldInfo) {
+    Query qX = NumericRangeQuery.newDoubleRange(
+      fieldInfo.getFieldNameX(),
+      finfo.precisionStep,
+      bbox.getMinX(),
+      bbox.getMaxX(),
+      true,
+      true);
+    Query qY = NumericRangeQuery.newDoubleRange(
+      fieldInfo.getFieldNameY(),
+      finfo.precisionStep,
+      bbox.getMinY(),
+      bbox.getMaxY(),
+      true,
+      true);
+
+    BooleanQuery bq = new BooleanQuery();
+    bq.add(qX,BooleanClause.Occur.MUST);
+    bq.add(qY,BooleanClause.Occur.MUST);
+    return bq;
+  }
+
+  /**
+   * Constructs a query to retrieve documents that fully contain the input envelope.
+   * @return the spatial query
+   */
+  Query makeDisjoint(Rectangle bbox, TwoDoublesFieldInfo fieldInfo) {
+    Query qX = NumericRangeQuery.newDoubleRange(
+      fieldInfo.getFieldNameX(),
+      finfo.precisionStep,
+      bbox.getMinX(),
+      bbox.getMaxX(),
+      true,
+      true);
+    Query qY = NumericRangeQuery.newDoubleRange(
+      fieldInfo.getFieldNameY(),
+      finfo.precisionStep,
+      bbox.getMinY(),
+      bbox.getMaxY(),
+      true,
+      true);
+
+    BooleanQuery bq = new BooleanQuery();
+    bq.add(qX,BooleanClause.Occur.MUST_NOT);
+    bq.add(qY,BooleanClause.Occur.MUST_NOT);
+    return bq;
+  }
+}
+
+
+
+

Added: lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/test/java/org/apache/lucene/spatial/RandomSeed.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/test/java/org/apache/lucene/spatial/RandomSeed.java?rev=1291499&view=auto
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/test/java/org/apache/lucene/spatial/RandomSeed.java (added)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/test/java/org/apache/lucene/spatial/RandomSeed.java Mon Feb 20 22:45:32 2012
@@ -0,0 +1,34 @@
+/*
+ * 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;
+
+/**
+ * Reads "tests.seed" system property to initialized a global final constant.
+ * @author David Smiley - dsmiley@mitre.org
+ */
+public class RandomSeed {
+  private static final long _seed;
+  static {
+    _seed = Long.parseLong(System.getProperty("tests.seed", "" + System.currentTimeMillis()));
+    System.out.println("tests.seed="+_seed);
+  }
+  public static long seed() {
+    return _seed;
+  }
+  private RandomSeed() {}
+}

Added: lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/test/java/org/apache/lucene/spatial/SpatialMatchConcern.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/test/java/org/apache/lucene/spatial/SpatialMatchConcern.java?rev=1291499&view=auto
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/test/java/org/apache/lucene/spatial/SpatialMatchConcern.java (added)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/test/java/org/apache/lucene/spatial/SpatialMatchConcern.java Mon Feb 20 22:45:32 2012
@@ -0,0 +1,32 @@
+/*
+ * 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;
+
+public class SpatialMatchConcern {
+  public final boolean orderIsImportant;
+  public final boolean resultsAreSuperset; // if the strategy can not give exact answers, but used to limit results
+
+  private SpatialMatchConcern( boolean order, boolean superset ) {
+    this.orderIsImportant = order;
+    this.resultsAreSuperset = superset;
+  }
+
+  public static final SpatialMatchConcern EXACT = new SpatialMatchConcern( true, false );
+  public static final SpatialMatchConcern FILTER = new SpatialMatchConcern( false, false );
+  public static final SpatialMatchConcern SUPERSET = new SpatialMatchConcern( false, true );
+}

Added: lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/test/java/org/apache/lucene/spatial/SpatialTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/test/java/org/apache/lucene/spatial/SpatialTestCase.java?rev=1291499&view=auto
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/test/java/org/apache/lucene/spatial/SpatialTestCase.java (added)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/test/java/org/apache/lucene/spatial/SpatialTestCase.java Mon Feb 20 22:45:32 2012
@@ -0,0 +1,140 @@
+/*
+ * 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;
+
+import org.apache.lucene.analysis.core.WhitespaceAnalyzer;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.index.DirectoryReader;
+import org.apache.lucene.index.IndexWriter;
+import org.apache.lucene.index.IndexWriterConfig;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.ScoreDoc;
+import org.apache.lucene.search.TopDocs;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.LuceneTestCase;
+import org.junit.After;
+import org.junit.Before;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+public abstract class SpatialTestCase extends LuceneTestCase {
+
+  private DirectoryReader indexReader;
+  private IndexWriter indexWriter;
+  private Directory directory;
+  private IndexSearcher indexSearcher;
+
+  @Override
+  @Before
+  public void setUp() throws Exception {
+    super.setUp();
+
+    directory = newDirectory();
+
+    IndexWriterConfig writerConfig = newIndexWriterConfig(random, TEST_VERSION_CURRENT, new WhitespaceAnalyzer(TEST_VERSION_CURRENT));
+    indexWriter = new IndexWriter(directory, writerConfig);
+  }
+
+  @Override
+  @After
+  public void tearDown() throws Exception {
+    if (indexWriter != null) {
+      indexWriter.close();
+    }
+    if (indexReader != null) {
+      indexReader.close();
+    }
+    if (directory != null) {
+      directory.close();
+    }
+    super.tearDown();
+  }
+
+  // ================================================= Helper Methods ================================================
+
+  protected void addDocument(Document doc) throws IOException {
+    indexWriter.addDocument(doc);
+  }
+
+  protected void addDocumentsAndCommit(List<Document> documents) throws IOException {
+    for (Document document : documents) {
+      indexWriter.addDocument(document);
+    }
+    commit();
+  }
+
+  protected void deleteAll() throws IOException {
+    indexWriter.deleteAll();
+  }
+
+  protected void commit() throws IOException {
+    indexWriter.commit();
+    if (indexReader == null) {
+      indexReader = DirectoryReader.open(directory);
+    } else {
+      indexReader = DirectoryReader.openIfChanged(indexReader);
+    }
+    indexSearcher = newSearcher(indexReader);
+  }
+
+  protected void verifyDocumentsIndexed(int numDocs) {
+    assertEquals(numDocs, indexReader.numDocs());
+  }
+
+  protected SearchResults executeQuery(Query query, int numDocs) {
+    try {
+      TopDocs topDocs = indexSearcher.search(query, numDocs);
+
+      List<SearchResult> results = new ArrayList<SearchResult>();
+      for (ScoreDoc scoreDoc : topDocs.scoreDocs) {
+        results.add(new SearchResult(scoreDoc.score, indexSearcher.doc(scoreDoc.doc)));
+      }
+      return new SearchResults(topDocs.totalHits, results);
+    } catch (IOException ioe) {
+      throw new RuntimeException("IOException thrown while executing query", ioe);
+    }
+  }
+
+  // ================================================= Inner Classes =================================================
+
+  protected static class SearchResults {
+
+    public int numFound;
+    public List<SearchResult> results;
+
+    public SearchResults(int numFound, List<SearchResult> results) {
+      this.numFound = numFound;
+      this.results = results;
+    }
+  }
+
+  protected static class SearchResult {
+
+    public float score;
+    public Document document;
+
+    public SearchResult(float score, Document document) {
+      this.score = score;
+      this.document = document;
+    }
+  }
+}
+

Added: lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/test/java/org/apache/lucene/spatial/SpatialTestQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/test/java/org/apache/lucene/spatial/SpatialTestQuery.java?rev=1291499&view=auto
==============================================================================
--- lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/test/java/org/apache/lucene/spatial/SpatialTestQuery.java (added)
+++ lucene/dev/branches/lucene3795_lsp_spatial_module/modules/spatial-lucene/src/test/java/org/apache/lucene/spatial/SpatialTestQuery.java Mon Feb 20 22:45:32 2012
@@ -0,0 +1,87 @@
+/*
+ * 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;
+
+import org.apache.lucene.spatial.base.context.SpatialContext;
+import org.apache.lucene.spatial.base.io.LineReader;
+import org.apache.lucene.spatial.base.query.SpatialArgs;
+import org.apache.lucene.spatial.base.query.SpatialArgsParser;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.StringTokenizer;
+
+/**
+ * Helper class to execute queries
+ */
+public class SpatialTestQuery {
+  public String testname;
+  public String line;
+  public int lineNumber = -1;
+  public SpatialArgs args;
+  public List<String> ids = new ArrayList<String>();
+
+  /**
+   * Get Test Queries
+   */
+  public static Iterator<SpatialTestQuery> getTestQueries(
+      final SpatialArgsParser parser,
+      final SpatialContext ctx,
+      final String name,
+      final InputStream in ) throws IOException {
+    return new LineReader<SpatialTestQuery>(new InputStreamReader(in,"UTF-8")) {
+
+      @Override
+      public SpatialTestQuery parseLine(String line) {
+        SpatialTestQuery test = new SpatialTestQuery();
+        test.line = line;
+        test.lineNumber = getLineNumber();
+
+        try {
+          // skip a comment
+          if( line.startsWith( "[" ) ) {
+            int idx = line.indexOf( ']' );
+            if( idx > 0 ) {
+              line = line.substring( idx+1 );
+            }
+          }
+
+          int idx = line.indexOf('@');
+          StringTokenizer st = new StringTokenizer(line.substring(0, idx));
+          while (st.hasMoreTokens()) {
+            test.ids.add(st.nextToken().trim());
+          }
+          test.args = parser.parse(line.substring(idx + 1).trim(), ctx);
+          return test;
+        }
+        catch( Exception ex ) {
+          throw new RuntimeException( "invalid query line: "+test.line, ex );
+        }
+      }
+    };
+  }
+
+  @Override
+  public String toString() {
+    return line;
+  }
+}