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/07/05 05:23:11 UTC

svn commit: r1357452 - in /lucene/dev/branches/branch_4x: ./ lucene/ lucene/spatial/ lucene/spatial/src/java/org/apache/lucene/spatial/ lucene/spatial/src/java/org/apache/lucene/spatial/prefix/ lucene/spatial/src/java/org/apache/lucene/spatial/vector/ ...

Author: dsmiley
Date: Thu Jul  5 03:23:10 2012
New Revision: 1357452

URL: http://svn.apache.org/viewvc?rev=1357452&view=rev
Log:
LUCENE-4099 remove SpatialFieldInfo (and corresponding generics) and put fieldName into Strategy instead of methods. Thus an instance of SpatialStrategy is now a per-field object, not per-fieldType.

Removed:
    lucene/dev/branches/branch_4x/lucene/spatial/src/java/org/apache/lucene/spatial/SimpleSpatialFieldInfo.java
    lucene/dev/branches/branch_4x/lucene/spatial/src/java/org/apache/lucene/spatial/SpatialFieldInfo.java
    lucene/dev/branches/branch_4x/lucene/spatial/src/java/org/apache/lucene/spatial/vector/TwoDoublesFieldInfo.java
Modified:
    lucene/dev/branches/branch_4x/   (props changed)
    lucene/dev/branches/branch_4x/lucene/   (props changed)
    lucene/dev/branches/branch_4x/lucene/spatial/   (props changed)
    lucene/dev/branches/branch_4x/lucene/spatial/src/java/org/apache/lucene/spatial/SpatialStrategy.java
    lucene/dev/branches/branch_4x/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/PrefixTreeStrategy.java
    lucene/dev/branches/branch_4x/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/RecursivePrefixTreeStrategy.java
    lucene/dev/branches/branch_4x/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/TermQueryPrefixTreeStrategy.java
    lucene/dev/branches/branch_4x/lucene/spatial/src/java/org/apache/lucene/spatial/vector/DistanceValueSource.java
    lucene/dev/branches/branch_4x/lucene/spatial/src/java/org/apache/lucene/spatial/vector/TwoDoublesStrategy.java
    lucene/dev/branches/branch_4x/lucene/spatial/src/test/org/apache/lucene/spatial/PortedSolr3Test.java
    lucene/dev/branches/branch_4x/lucene/spatial/src/test/org/apache/lucene/spatial/StrategyTestCase.java
    lucene/dev/branches/branch_4x/lucene/spatial/src/test/org/apache/lucene/spatial/prefix/TestRecursivePrefixTreeStrategy.java
    lucene/dev/branches/branch_4x/lucene/spatial/src/test/org/apache/lucene/spatial/prefix/TestTermQueryPrefixGridStrategy.java
    lucene/dev/branches/branch_4x/lucene/spatial/src/test/org/apache/lucene/spatial/vector/TestTwoDoublesStrategy.java

Modified: lucene/dev/branches/branch_4x/lucene/spatial/src/java/org/apache/lucene/spatial/SpatialStrategy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/spatial/src/java/org/apache/lucene/spatial/SpatialStrategy.java?rev=1357452&r1=1357451&r2=1357452&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/spatial/src/java/org/apache/lucene/spatial/SpatialStrategy.java (original)
+++ lucene/dev/branches/branch_4x/lucene/spatial/src/java/org/apache/lucene/spatial/SpatialStrategy.java Thu Jul  5 03:23:10 2012
@@ -32,13 +32,22 @@ import org.apache.lucene.spatial.query.S
  *
  * @lucene.experimental
  */
-public abstract class SpatialStrategy<T extends SpatialFieldInfo> {
+public abstract class SpatialStrategy {
 
   protected boolean ignoreIncompatibleGeometry = false;
   protected final SpatialContext ctx;
+  private final String fieldName;
 
-  public SpatialStrategy(SpatialContext ctx) {
+  /**
+   * Constructs the spatial strategy with its mandatory arguments.
+   */
+  public SpatialStrategy(SpatialContext ctx, String fieldName) {
+    if (ctx == null)
+      throw new IllegalArgumentException("ctx is required");
     this.ctx = ctx;
+    if (fieldName == null || fieldName.length() == 0)
+      throw new IllegalArgumentException("fieldName is required");
+    this.fieldName = fieldName;
   }
 
   public SpatialContext getSpatialContext() {
@@ -51,39 +60,48 @@ public abstract class SpatialStrategy<T 
   }
 
   /**
+   * The name of the field or the prefix of them if there are multiple
+   * fields needed internally.
+   * @return Not null.
+   */
+  public String getFieldName() {
+    return fieldName;
+  }
+
+  /**
    * Corresponds with Solr's FieldType.createField().
    *
    * This may return a null field if it does not want to make anything.
    * This is reasonable behavior if 'ignoreIncompatibleGeometry=true' and the
    * geometry is incompatible
    */
-  public abstract IndexableField createField(T fieldInfo, Shape shape, boolean index, boolean store);
+  public abstract IndexableField createField(Shape shape, boolean index, boolean store);
 
   /** Corresponds with Solr's FieldType.createFields(). */
-  public IndexableField[] createFields(T fieldInfo, Shape shape, boolean index, boolean store) {
-    return new IndexableField[] { createField(fieldInfo, shape, index, store) };
+  public IndexableField[] createFields(Shape shape, boolean index, boolean store) {
+    return new IndexableField[] { createField(shape, index, store) };
   }
 
   /**
    * The value source yields a number that is proportional to the distance between the query shape and indexed data.
    */
-  public abstract ValueSource makeValueSource(SpatialArgs args, T fieldInfo);
+  public abstract ValueSource makeValueSource(SpatialArgs args);
 
   /**
    * Make a query which has a score based on the distance from the data to the query shape.
    * The default implementation constructs a {@link FilteredQuery} based on
-   * {@link #makeFilter(org.apache.lucene.spatial.query.SpatialArgs, SpatialFieldInfo)} and
-   * {@link #makeValueSource(org.apache.lucene.spatial.query.SpatialArgs, SpatialFieldInfo)}.
+   * {@link #makeFilter(org.apache.lucene.spatial.query.SpatialArgs)} and
+   * {@link #makeValueSource(org.apache.lucene.spatial.query.SpatialArgs)}.
    */
-  public Query makeQuery(SpatialArgs args, T fieldInfo) {
-    Filter filter = makeFilter(args, fieldInfo);
-    ValueSource vs = makeValueSource(args, fieldInfo);
+  public Query makeQuery(SpatialArgs args) {
+    Filter filter = makeFilter(args);
+    ValueSource vs = makeValueSource(args);
     return new FilteredQuery(new FunctionQuery(vs), filter);
   }
   /**
    * Make a Filter
    */
-  public abstract Filter makeFilter(SpatialArgs args, T fieldInfo);
+  public abstract Filter makeFilter(SpatialArgs args);
 
   public boolean isIgnoreIncompatibleGeometry() {
     return ignoreIncompatibleGeometry;
@@ -92,4 +110,9 @@ public abstract class SpatialStrategy<T 
   public void setIgnoreIncompatibleGeometry(boolean ignoreIncompatibleGeometry) {
     this.ignoreIncompatibleGeometry = ignoreIncompatibleGeometry;
   }
+
+  @Override
+  public String toString() {
+    return getClass().getSimpleName()+" field:"+fieldName+" ctx="+ctx;
+  }
 }

Modified: lucene/dev/branches/branch_4x/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/PrefixTreeStrategy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/PrefixTreeStrategy.java?rev=1357452&r1=1357451&r2=1357452&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/PrefixTreeStrategy.java (original)
+++ lucene/dev/branches/branch_4x/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/PrefixTreeStrategy.java Thu Jul  5 03:23:10 2012
@@ -27,7 +27,6 @@ import org.apache.lucene.document.FieldT
 import org.apache.lucene.document.StoredField;
 import org.apache.lucene.index.IndexableField;
 import org.apache.lucene.queries.function.ValueSource;
-import org.apache.lucene.spatial.SimpleSpatialFieldInfo;
 import org.apache.lucene.spatial.SpatialStrategy;
 import org.apache.lucene.spatial.prefix.tree.Node;
 import org.apache.lucene.spatial.prefix.tree.SpatialPrefixTree;
@@ -42,14 +41,14 @@ import java.util.concurrent.ConcurrentHa
 /**
  * @lucene.internal
  */
-public abstract class PrefixTreeStrategy extends SpatialStrategy<SimpleSpatialFieldInfo> {
+public abstract class PrefixTreeStrategy extends SpatialStrategy {
   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());
+  public PrefixTreeStrategy(SpatialPrefixTree grid, String fieldName) {
+    super(grid.getSpatialContext(), fieldName);
     this.grid = grid;
   }
 
@@ -64,7 +63,7 @@ public abstract class PrefixTreeStrategy
   }
 
   @Override
-  public IndexableField createField(SimpleSpatialFieldInfo fieldInfo, Shape shape, boolean index, boolean store) {
+  public IndexableField createField(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
@@ -79,7 +78,7 @@ public abstract class PrefixTreeStrategy
     //TODO is CellTokenStream supposed to be re-used somehow? see Uwe's comments:
     //  http://code.google.com/p/lucene-spatial-playground/issues/detail?id=4
 
-    String fname = fieldInfo.getFieldName();
+    String fname = getFieldName();
     if( store ) {
       //TODO figure out how to re-use original string instead of reconstituting it.
       String wkt = grid.getSpatialContext().toString(shape);
@@ -153,19 +152,19 @@ public abstract class PrefixTreeStrategy
   }
 
   @Override
-  public ValueSource makeValueSource(SpatialArgs args, SimpleSpatialFieldInfo fieldInfo) {
+  public ValueSource makeValueSource(SpatialArgs args) {
     DistanceCalculator calc = grid.getSpatialContext().getDistCalc();
-    return makeValueSource(args, fieldInfo, calc);
+    return makeValueSource(args, calc);
   }
   
-  public ValueSource makeValueSource(SpatialArgs args, SimpleSpatialFieldInfo fieldInfo, DistanceCalculator calc) {
-    PointPrefixTreeFieldCacheProvider p = provider.get( fieldInfo.getFieldName() );
+  public ValueSource makeValueSource(SpatialArgs args, DistanceCalculator calc) {
+    PointPrefixTreeFieldCacheProvider p = provider.get( getFieldName() );
     if( p == null ) {
       synchronized (this) {//double checked locking idiom is okay since provider is threadsafe
-        p = provider.get( fieldInfo.getFieldName() );
+        p = provider.get( getFieldName() );
         if (p == null) {
-          p = new PointPrefixTreeFieldCacheProvider(grid, fieldInfo.getFieldName(), defaultFieldValuesArrayLen);
-          provider.put(fieldInfo.getFieldName(),p);
+          p = new PointPrefixTreeFieldCacheProvider(grid, getFieldName(), defaultFieldValuesArrayLen);
+          provider.put(getFieldName(),p);
         }
       }
     }

Modified: lucene/dev/branches/branch_4x/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/RecursivePrefixTreeStrategy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/RecursivePrefixTreeStrategy.java?rev=1357452&r1=1357451&r2=1357452&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/RecursivePrefixTreeStrategy.java (original)
+++ lucene/dev/branches/branch_4x/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/RecursivePrefixTreeStrategy.java Thu Jul  5 03:23:10 2012
@@ -19,7 +19,6 @@ package org.apache.lucene.spatial.prefix
 
 import com.spatial4j.core.shape.Shape;
 import org.apache.lucene.search.Filter;
-import org.apache.lucene.spatial.SimpleSpatialFieldInfo;
 import org.apache.lucene.spatial.prefix.tree.SpatialPrefixTree;
 import org.apache.lucene.spatial.query.SpatialArgs;
 import org.apache.lucene.spatial.query.SpatialOperation;
@@ -32,14 +31,15 @@ import org.apache.lucene.spatial.query.U
  */
 public class RecursivePrefixTreeStrategy extends PrefixTreeStrategy {
 
-  private int prefixGridScanLevel;//TODO how is this customized?
+  private int prefixGridScanLevel;
 
-  public RecursivePrefixTreeStrategy(SpatialPrefixTree grid) {
-    super(grid);
+  public RecursivePrefixTreeStrategy(SpatialPrefixTree grid, String fieldName) {
+    super(grid, fieldName);
     prefixGridScanLevel = grid.getMaxLevels() - 4;//TODO this default constant is dependent on the prefix grid size
   }
 
   public void setPrefixGridScanLevel(int prefixGridScanLevel) {
+    //TODO if negative then subtract from maxlevels
     this.prefixGridScanLevel = prefixGridScanLevel;
   }
 
@@ -49,7 +49,7 @@ public class RecursivePrefixTreeStrategy
   }
 
   @Override
-  public Filter makeFilter(SpatialArgs args, SimpleSpatialFieldInfo fieldInfo) {
+  public Filter makeFilter(SpatialArgs args) {
     final SpatialOperation op = args.getOperation();
     if (! SpatialOperation.is(op, SpatialOperation.IsWithin, SpatialOperation.Intersects, SpatialOperation.BBoxWithin, SpatialOperation.BBoxIntersects))
       throw new UnsupportedSpatialOperation(op);
@@ -59,7 +59,7 @@ public class RecursivePrefixTreeStrategy
     int detailLevel = grid.getMaxLevelForPrecision(shape,args.getDistPrecision());
 
     return new RecursivePrefixTreeFilter(
-        fieldInfo.getFieldName(), grid,shape, prefixGridScanLevel, detailLevel);
+        getFieldName(), grid,shape, prefixGridScanLevel, detailLevel);
   }
 }
 

Modified: lucene/dev/branches/branch_4x/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/TermQueryPrefixTreeStrategy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/TermQueryPrefixTreeStrategy.java?rev=1357452&r1=1357451&r2=1357452&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/TermQueryPrefixTreeStrategy.java (original)
+++ lucene/dev/branches/branch_4x/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/TermQueryPrefixTreeStrategy.java Thu Jul  5 03:23:10 2012
@@ -21,7 +21,6 @@ import com.spatial4j.core.shape.Shape;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.queries.TermsFilter;
 import org.apache.lucene.search.Filter;
-import org.apache.lucene.spatial.SimpleSpatialFieldInfo;
 import org.apache.lucene.spatial.prefix.tree.Node;
 import org.apache.lucene.spatial.prefix.tree.SpatialPrefixTree;
 import org.apache.lucene.spatial.query.SpatialArgs;
@@ -38,12 +37,12 @@ import java.util.List;
  */
 public class TermQueryPrefixTreeStrategy extends PrefixTreeStrategy {
 
-  public TermQueryPrefixTreeStrategy(SpatialPrefixTree grid) {
-    super(grid);
+  public TermQueryPrefixTreeStrategy(SpatialPrefixTree grid, String fieldName) {
+    super(grid, fieldName);
   }
 
   @Override
-  public Filter makeFilter(SpatialArgs args, SimpleSpatialFieldInfo fieldInfo) {
+  public Filter makeFilter(SpatialArgs args) {
     final SpatialOperation op = args.getOperation();
     if (! SpatialOperation.is(op, SpatialOperation.IsWithin, SpatialOperation.Intersects, SpatialOperation.BBoxWithin, SpatialOperation.BBoxIntersects))
       throw new UnsupportedSpatialOperation(op);
@@ -53,7 +52,7 @@ public class TermQueryPrefixTreeStrategy
     List<Node> cells = grid.getNodes(shape, detailLevel, false);
     TermsFilter filter = new TermsFilter();
     for (Node cell : cells) {
-      filter.addTerm(new Term(fieldInfo.getFieldName(), cell.getTokenString()));
+      filter.addTerm(new Term(getFieldName(), cell.getTokenString()));
     }
     return filter;
   }

Modified: lucene/dev/branches/branch_4x/lucene/spatial/src/java/org/apache/lucene/spatial/vector/DistanceValueSource.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/spatial/src/java/org/apache/lucene/spatial/vector/DistanceValueSource.java?rev=1357452&r1=1357451&r2=1357452&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/spatial/src/java/org/apache/lucene/spatial/vector/DistanceValueSource.java (original)
+++ lucene/dev/branches/branch_4x/lucene/spatial/src/java/org/apache/lucene/spatial/vector/DistanceValueSource.java Thu Jul  5 03:23:10 2012
@@ -24,7 +24,6 @@ import org.apache.lucene.index.AtomicRea
 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.util.Bits;
 
 import java.io.IOException;
@@ -37,19 +36,17 @@ import java.util.Map;
  */
 public class DistanceValueSource extends ValueSource {
 
-  private final TwoDoublesFieldInfo fields;
-  private final DistanceCalculator calculator;
+  private TwoDoublesStrategy strategy;
   private final Point from;
-  private final DoubleParser parser;
+  private final DistanceCalculator calculator;
 
   /**
    * Constructor.
    */
-  public DistanceValueSource(Point from, DistanceCalculator calc, TwoDoublesFieldInfo fields, DoubleParser parser) {
+  public DistanceValueSource(TwoDoublesStrategy strategy, Point from, DistanceCalculator calc) {
+    this.strategy = strategy;
     this.from = from;
-    this.fields = fields;
     this.calculator = calc;
-    this.parser = parser;
   }
 
   /**
@@ -68,10 +65,10 @@ public class DistanceValueSource extends
   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());
+    final double[] ptX = FieldCache.DEFAULT.getDoubles(reader, strategy.getFieldNameX(), true);
+    final double[] ptY = FieldCache.DEFAULT.getDoubles(reader, strategy.getFieldNameY(), true);
+    final Bits validX =  FieldCache.DEFAULT.getDocsWithField(reader, strategy.getFieldNameX());
+    final Bits validY =  FieldCache.DEFAULT.getDocsWithField(reader, strategy.getFieldNameY());
 
     return new FunctionValues() {
       @Override
@@ -108,7 +105,7 @@ public class DistanceValueSource extends
     DistanceValueSource that = (DistanceValueSource) o;
 
     if (calculator != null ? !calculator.equals(that.calculator) : that.calculator != null) return false;
-    if (fields != null ? !fields.equals(that.fields) : that.fields != null) return false;
+    if (strategy != null ? !strategy.equals(that.strategy) : that.strategy != null) return false;
     if (from != null ? !from.equals(that.from) : that.from != null) return false;
 
     return true;
@@ -116,7 +113,7 @@ public class DistanceValueSource extends
 
   @Override
   public int hashCode() {
-    int result = fields != null ? fields.hashCode() : 0;
+    int result = strategy != null ? strategy.hashCode() : 0;
     result = 31 * result + (calculator != null ? calculator.hashCode() : 0);
     result = 31 * result + (from != null ? from.hashCode() : 0);
     return result;

Modified: lucene/dev/branches/branch_4x/lucene/spatial/src/java/org/apache/lucene/spatial/vector/TwoDoublesStrategy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/spatial/src/java/org/apache/lucene/spatial/vector/TwoDoublesStrategy.java?rev=1357452&r1=1357451&r2=1357452&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/spatial/src/java/org/apache/lucene/spatial/vector/TwoDoublesStrategy.java (original)
+++ lucene/dev/branches/branch_4x/lucene/spatial/src/java/org/apache/lucene/spatial/vector/TwoDoublesStrategy.java Thu Jul  5 03:23:10 2012
@@ -23,33 +23,58 @@ import com.spatial4j.core.shape.Circle;
 import com.spatial4j.core.shape.Point;
 import com.spatial4j.core.shape.Rectangle;
 import com.spatial4j.core.shape.Shape;
+import org.apache.lucene.document.DoubleField;
 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.search.BooleanClause;
+import org.apache.lucene.search.BooleanQuery;
+import org.apache.lucene.search.Filter;
+import org.apache.lucene.search.FilteredQuery;
+import org.apache.lucene.search.MatchAllDocsQuery;
+import org.apache.lucene.search.NumericRangeQuery;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.QueryWrapperFilter;
 import org.apache.lucene.spatial.SpatialStrategy;
 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.util.CachingDoubleValueSource;
-import org.apache.lucene.spatial.util.NumericFieldInfo;
 import org.apache.lucene.spatial.util.ValueSourceFilter;
 
 /**
  * @lucene.experimental
  */
-public class TwoDoublesStrategy extends SpatialStrategy<TwoDoublesFieldInfo> {
+public class TwoDoublesStrategy extends SpatialStrategy {
 
-  private final NumericFieldInfo finfo;
-  private final DoubleParser parser;
+  public static final String SUFFIX_X = "__x";
+  public static final String SUFFIX_Y = "__y";
 
-  public TwoDoublesStrategy(SpatialContext ctx, NumericFieldInfo finfo, DoubleParser parser) {
-    super(ctx);
-    this.finfo = finfo;
-    this.parser = parser;
+  private final String fieldNameX;
+  private final String fieldNameY;
+
+  public int precisionStep = 8; // same as solr default
+
+  public TwoDoublesStrategy(SpatialContext ctx, String fieldNamePrefix) {
+    super(ctx, fieldNamePrefix);
+    this.fieldNameX = fieldNamePrefix+SUFFIX_X;
+    this.fieldNameY = fieldNamePrefix+SUFFIX_Y;
+  }
+
+  public void setPrecisionStep( int p ) {
+    precisionStep = p;
+    if (precisionStep<=0 || precisionStep>=64)
+      precisionStep=Integer.MAX_VALUE;
+  }
+
+  String getFieldNameX() {
+    return fieldNameX;
+  }
+
+  String getFieldNameY() {
+    return fieldNameY;
   }
 
   @Override
@@ -58,20 +83,19 @@ public class TwoDoublesStrategy extends 
   }
 
   @Override
-  public IndexableField[] createFields(TwoDoublesFieldInfo fieldInfo,
-      Shape shape, boolean index, boolean store) {
+  public IndexableField[] createFields(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() );
+        f[0] = createDouble(fieldNameX, point.getX(), index, store);
+        f[1] = createDouble(fieldNameY, point.getY(), index, store);
       }
       if(store) {
         FieldType customType = new FieldType();
         customType.setStored(true);
-        f[f.length-1] = new Field( fieldInfo.getFieldName(), ctx.toString( shape ), customType );
+        f[f.length-1] = new Field( getFieldName(), ctx.toString( shape ), customType );
       }
       return f;
     }
@@ -81,39 +105,50 @@ public class TwoDoublesStrategy extends 
     return new IndexableField[0]; // nothing (solr does not support null)
   }
 
+  private IndexableField createDouble(String name, double v, boolean index, boolean store) {
+    if (!store && !index)
+      throw new IllegalArgumentException("field must be indexed or stored");
+
+    FieldType fieldType = new FieldType(DoubleField.TYPE_NOT_STORED);
+    fieldType.setStored(store);
+    fieldType.setIndexed(index);
+    fieldType.setNumericPrecisionStep(precisionStep);
+    return new DoubleField(name,v,fieldType);
+  }
+
   @Override
-  public IndexableField createField(TwoDoublesFieldInfo indexInfo, Shape shape,
-      boolean index, boolean store) {
+  public IndexableField createField(Shape shape,
+                                    boolean index, boolean store) {
     throw new UnsupportedOperationException("Point is poly field");
   }
 
   @Override
-  public ValueSource makeValueSource(SpatialArgs args, TwoDoublesFieldInfo fieldInfo) {
+  public ValueSource makeValueSource(SpatialArgs args) {
     Point p = args.getShape().getCenter();
-    return new DistanceValueSource(p, ctx.getDistCalc(), fieldInfo, parser);
+    return new DistanceValueSource(this, p, ctx.getDistCalc());
   }
 
   @Override
-  public Filter makeFilter(SpatialArgs args, TwoDoublesFieldInfo fieldInfo) {
+  public Filter makeFilter(SpatialArgs args) {
     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);
+        Query bbox = makeWithin(circle.getBoundingBox());
 
         // Make the ValueSource
-        ValueSource valueSource = makeValueSource(args, fieldInfo);
+        ValueSource valueSource = makeValueSource(args);
 
         return new ValueSourceFilter(
             new QueryWrapperFilter( bbox ), valueSource, 0, circle.getDistance() );
       }
     }
-    return new QueryWrapperFilter( makeQuery(args, fieldInfo) );
+    return new QueryWrapperFilter( makeQuery(args) );
   }
 
   @Override
-  public Query makeQuery(SpatialArgs args, TwoDoublesFieldInfo fieldInfo) {
+  public Query makeQuery(SpatialArgs args) {
     // For starters, just limit the bbox
     Shape shape = args.getShape();
     if (!(shape instanceof Rectangle || shape instanceof Circle)) {
@@ -135,17 +170,17 @@ public class TwoDoublesStrategy extends 
     if( SpatialOperation.is( op,
         SpatialOperation.BBoxWithin,
         SpatialOperation.BBoxIntersects ) ) {
-        spatial = makeWithin(bbox, fieldInfo);
+        spatial = makeWithin(bbox);
     }
     else if( SpatialOperation.is( op,
       SpatialOperation.Intersects,
       SpatialOperation.IsWithin ) ) {
-      spatial = makeWithin(bbox, fieldInfo);
+      spatial = makeWithin(bbox);
       if( args.getShape() instanceof Circle) {
         Circle circle = (Circle)args.getShape();
 
         // Make the ValueSource
-        valueSource = makeValueSource(args, fieldInfo);
+        valueSource = makeValueSource(args);
 
         ValueSourceFilter vsf = new ValueSourceFilter(
             new QueryWrapperFilter( spatial ), valueSource, 0, circle.getDistance() );
@@ -154,7 +189,7 @@ public class TwoDoublesStrategy extends 
       }
     }
     else if( op == SpatialOperation.IsDisjointTo ) {
-      spatial =  makeDisjoint(bbox, fieldInfo);
+      spatial =  makeDisjoint(bbox);
     }
 
     if( spatial == null ) {
@@ -165,7 +200,7 @@ public class TwoDoublesStrategy extends 
       valueSource = new CachingDoubleValueSource(valueSource);
     }
     else {
-      valueSource = makeValueSource(args, fieldInfo);
+      valueSource = makeValueSource(args);
     }
     Query spatialRankingQuery = new FunctionQuery(valueSource);
     BooleanQuery bq = new BooleanQuery();
@@ -178,17 +213,17 @@ public class TwoDoublesStrategy extends 
    * Constructs a query to retrieve documents that fully contain the input envelope.
    * @return the spatial query
    */
-  private Query makeWithin(Rectangle bbox, TwoDoublesFieldInfo fieldInfo) {
+  private Query makeWithin(Rectangle bbox) {
     Query qX = NumericRangeQuery.newDoubleRange(
-      fieldInfo.getFieldNameX(),
-      finfo.precisionStep,
+      fieldNameX,
+      precisionStep,
       bbox.getMinX(),
       bbox.getMaxX(),
       true,
       true);
     Query qY = NumericRangeQuery.newDoubleRange(
-      fieldInfo.getFieldNameY(),
-      finfo.precisionStep,
+      fieldNameY,
+      precisionStep,
       bbox.getMinY(),
       bbox.getMaxY(),
       true,
@@ -204,17 +239,17 @@ public class TwoDoublesStrategy extends 
    * Constructs a query to retrieve documents that fully contain the input envelope.
    * @return the spatial query
    */
-  Query makeDisjoint(Rectangle bbox, TwoDoublesFieldInfo fieldInfo) {
+  Query makeDisjoint(Rectangle bbox) {
     Query qX = NumericRangeQuery.newDoubleRange(
-      fieldInfo.getFieldNameX(),
-      finfo.precisionStep,
+      fieldNameX,
+      precisionStep,
       bbox.getMinX(),
       bbox.getMaxX(),
       true,
       true);
     Query qY = NumericRangeQuery.newDoubleRange(
-      fieldInfo.getFieldNameY(),
-      finfo.precisionStep,
+      fieldNameY,
+      precisionStep,
       bbox.getMinY(),
       bbox.getMaxY(),
       true,
@@ -225,6 +260,7 @@ public class TwoDoublesStrategy extends 
     bq.add(qY,BooleanClause.Occur.MUST_NOT);
     return bq;
   }
+
 }
 
 

Modified: lucene/dev/branches/branch_4x/lucene/spatial/src/test/org/apache/lucene/spatial/PortedSolr3Test.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/spatial/src/test/org/apache/lucene/spatial/PortedSolr3Test.java?rev=1357452&r1=1357451&r2=1357452&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/spatial/src/test/org/apache/lucene/spatial/PortedSolr3Test.java (original)
+++ lucene/dev/branches/branch_4x/lucene/spatial/src/test/org/apache/lucene/spatial/PortedSolr3Test.java Thu Jul  5 03:23:10 2012
@@ -58,27 +58,25 @@ public class PortedSolr3Test extends Str
     SpatialStrategy strategy;
 
     grid = new GeohashPrefixTree(ctx,12);
-    strategy = new RecursivePrefixTreeStrategy(grid);
-    ctorArgs.add(new Object[]{"recursive_geohash",strategy});
+    strategy = new RecursivePrefixTreeStrategy(grid, "recursive_geohash");
+    ctorArgs.add(new Object[]{strategy});
 
     grid = new QuadPrefixTree(ctx,25);
-    strategy = new RecursivePrefixTreeStrategy(grid);
-    ctorArgs.add(new Object[]{"recursive_quad",strategy});
+    strategy = new RecursivePrefixTreeStrategy(grid, "recursive_quad");
+    ctorArgs.add(new Object[]{strategy});
 
     grid = new GeohashPrefixTree(ctx,12);
-    strategy = new TermQueryPrefixTreeStrategy(grid);
-    ctorArgs.add(new Object[]{"termquery_geohash",strategy});
+    strategy = new TermQueryPrefixTreeStrategy(grid, "termquery_geohash");
+    ctorArgs.add(new Object[]{strategy});
 
     return ctorArgs;
   }
 
 //  private String fieldName;
 
-  public PortedSolr3Test(String fieldName, SpatialStrategy strategy) {
-    ctx = strategy.getSpatialContext();
+  public PortedSolr3Test(SpatialStrategy strategy) {
+    this.ctx = strategy.getSpatialContext();
     this.strategy = strategy;
-//    this.fieldName = fieldName;
-    fieldInfo = new SimpleSpatialFieldInfo( fieldName );
   }
 
   private void setupDocs() throws IOException {
@@ -156,7 +154,7 @@ public class PortedSolr3Test extends Str
 
   private void checkHitsOrdered(String spatialQ, String... ids) {
     SpatialArgs args = this.argsParser.parse(spatialQ,ctx);
-    Query query = strategy.makeQuery(args, fieldInfo);
+    Query query = strategy.makeQuery(args);
     SearchResults results = executeQuery(query, 100);
     String[] resultIds = new String[results.numFound];
     int i = 0;
@@ -177,7 +175,7 @@ public class PortedSolr3Test extends Str
   private Document newDoc(String id, Shape shape) {
     Document doc = new Document();
     doc.add(new StringField("id", id, Field.Store.YES));
-    for (IndexableField f : strategy.createFields(fieldInfo, shape, true, storeShape)) {
+    for (IndexableField f : strategy.createFields(shape, true, storeShape)) {
       doc.add(f);
     }
     return doc;
@@ -199,9 +197,9 @@ public class PortedSolr3Test extends Str
     //args.setDistPrecision(0.025);
     Query query;
     if (random().nextBoolean()) {
-      query = strategy.makeQuery(args, fieldInfo);
+      query = strategy.makeQuery(args);
     } else {
-      query = new FilteredQuery(new MatchAllDocsQuery(),strategy.makeFilter(args, fieldInfo));
+      query = new FilteredQuery(new MatchAllDocsQuery(),strategy.makeFilter(args));
     }
     SearchResults results = executeQuery(query, 100);
     assertEquals(""+shape,assertNumFound,results.numFound);

Modified: lucene/dev/branches/branch_4x/lucene/spatial/src/test/org/apache/lucene/spatial/StrategyTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/spatial/src/test/org/apache/lucene/spatial/StrategyTestCase.java?rev=1357452&r1=1357451&r2=1357452&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/spatial/src/test/org/apache/lucene/spatial/StrategyTestCase.java (original)
+++ lucene/dev/branches/branch_4x/lucene/spatial/src/test/org/apache/lucene/spatial/StrategyTestCase.java Thu Jul  5 03:23:10 2012
@@ -31,10 +31,15 @@ import org.junit.Assert;
 
 import java.io.IOException;
 import java.io.InputStream;
-import java.util.*;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
 import java.util.logging.Logger;
 
-public abstract class StrategyTestCase<T extends SpatialFieldInfo> extends SpatialTestCase {
+public abstract class StrategyTestCase extends SpatialTestCase {
 
   public static final String DATA_STATES_POLY = "states-poly.txt";
   public static final String DATA_STATES_BBOX = "states-bbox.txt";
@@ -51,9 +56,8 @@ public abstract class StrategyTestCase<T
 
   protected final SpatialArgsParser argsParser = new SpatialArgsParser();
 
-  protected SpatialStrategy<T> strategy;
+  protected SpatialStrategy strategy;
   protected SpatialContext ctx;
-  protected T fieldInfo;
   protected boolean storeShape = true;
 
   protected void executeQueries(SpatialMatchConcern concern, String... testQueryFile) throws IOException {
@@ -79,7 +83,7 @@ public abstract class StrategyTestCase<T
       document.add(new StringField("id", data.id, Field.Store.YES));
       document.add(new StringField("name", data.name, Field.Store.YES));
       Shape shape = ctx.readShape(data.shape);
-      for (IndexableField f : strategy.createFields(fieldInfo, shape, true, storeShape)) {
+      for (IndexableField f : strategy.createFields(shape, true, storeShape)) {
         if( f != null ) { // null if incompatibleGeometry && ignore
           document.add(f);
         }
@@ -107,7 +111,7 @@ public abstract class StrategyTestCase<T
       SpatialTestQuery q = queries.next();
 
       String msg = q.line; //"Query: " + q.args.toString(ctx);
-      SearchResults got = executeQuery(strategy.makeQuery(q.args, fieldInfo), 100);
+      SearchResults got = executeQuery(strategy.makeQuery(q.args), 100);
       if (concern.orderIsImportant) {
         Iterator<String> ids = q.ids.iterator();
         for (SearchResult r : got.results) {

Modified: lucene/dev/branches/branch_4x/lucene/spatial/src/test/org/apache/lucene/spatial/prefix/TestRecursivePrefixTreeStrategy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/spatial/src/test/org/apache/lucene/spatial/prefix/TestRecursivePrefixTreeStrategy.java?rev=1357452&r1=1357451&r2=1357452&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/spatial/src/test/org/apache/lucene/spatial/prefix/TestRecursivePrefixTreeStrategy.java (original)
+++ lucene/dev/branches/branch_4x/lucene/spatial/src/test/org/apache/lucene/spatial/prefix/TestRecursivePrefixTreeStrategy.java Thu Jul  5 03:23:10 2012
@@ -28,7 +28,6 @@ import org.apache.lucene.document.Docume
 import org.apache.lucene.document.Field;
 import org.apache.lucene.document.StringField;
 import org.apache.lucene.index.IndexableField;
-import org.apache.lucene.spatial.SimpleSpatialFieldInfo;
 import org.apache.lucene.spatial.SpatialMatchConcern;
 import org.apache.lucene.spatial.StrategyTestCase;
 import org.apache.lucene.spatial.prefix.tree.GeohashPrefixTree;
@@ -37,11 +36,15 @@ import org.apache.lucene.spatial.query.S
 import org.junit.Test;
 
 import java.io.IOException;
-import java.util.*;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
 
 import static java.lang.Math.toRadians;
 
-public class TestRecursivePrefixTreeStrategy extends StrategyTestCase<SimpleSpatialFieldInfo> {
+public class TestRecursivePrefixTreeStrategy extends StrategyTestCase {
 
   private int maxLength;
 
@@ -49,9 +52,8 @@ public class TestRecursivePrefixTreeStra
   private void init(int maxLength) {
     this.maxLength = maxLength;
     this.ctx = SimpleSpatialContext.GEO_KM;
-    this.strategy = new RecursivePrefixTreeStrategy(new GeohashPrefixTree(
-        ctx, maxLength ));
-    this.fieldInfo = new SimpleSpatialFieldInfo( getClass().getSimpleName() );
+    GeohashPrefixTree grid = new GeohashPrefixTree(ctx, maxLength);
+    this.strategy = new RecursivePrefixTreeStrategy(grid, getClass().getSimpleName());
   }
 
   @Test
@@ -134,7 +136,7 @@ public class TestRecursivePrefixTreeStra
     Shape shape = ctx.makeCircle(pt,dist);
     SpatialArgs args = new SpatialArgs(SpatialOperation.Intersects,shape);
     args.setDistPrecision(0.0);
-    SearchResults got = executeQuery(strategy.makeQuery(args, fieldInfo), 100);
+    SearchResults got = executeQuery(strategy.makeQuery(args), 100);
     assertEquals(""+shape,assertNumFound,got.numFound);
     if (assertIds != null) {
       Set<Integer> gotIds = new HashSet<Integer>();
@@ -151,7 +153,7 @@ public class TestRecursivePrefixTreeStra
   private Document newDoc(String id, Shape shape) {
     Document doc = new Document();
     doc.add(new StringField("id", id, Field.Store.YES));
-    for (IndexableField f : strategy.createFields(fieldInfo, shape, true, storeShape)) {
+    for (IndexableField f : strategy.createFields(shape, true, storeShape)) {
       doc.add(f);
     }
     return doc;

Modified: lucene/dev/branches/branch_4x/lucene/spatial/src/test/org/apache/lucene/spatial/prefix/TestTermQueryPrefixGridStrategy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/spatial/src/test/org/apache/lucene/spatial/prefix/TestTermQueryPrefixGridStrategy.java?rev=1357452&r1=1357451&r2=1357452&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/spatial/src/test/org/apache/lucene/spatial/prefix/TestTermQueryPrefixGridStrategy.java (original)
+++ lucene/dev/branches/branch_4x/lucene/spatial/src/test/org/apache/lucene/spatial/prefix/TestTermQueryPrefixGridStrategy.java Thu Jul  5 03:23:10 2012
@@ -24,7 +24,6 @@ import com.spatial4j.core.shape.simple.P
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
 import org.apache.lucene.document.StringField;
-import org.apache.lucene.spatial.SimpleSpatialFieldInfo;
 import org.apache.lucene.spatial.SpatialTestCase;
 import org.apache.lucene.spatial.prefix.tree.QuadPrefixTree;
 import org.apache.lucene.spatial.query.SpatialArgsParser;
@@ -38,15 +37,14 @@ public class TestTermQueryPrefixGridStra
 
   @Test
   public void testNGramPrefixGridLosAngeles() throws IOException {
-    SimpleSpatialFieldInfo fieldInfo = new SimpleSpatialFieldInfo("geo");
     SpatialContext ctx = SimpleSpatialContext.GEO_KM;
-    TermQueryPrefixTreeStrategy prefixGridStrategy = new TermQueryPrefixTreeStrategy(new QuadPrefixTree(ctx));
+    TermQueryPrefixTreeStrategy prefixGridStrategy = new TermQueryPrefixTreeStrategy(new QuadPrefixTree(ctx), "geo");
 
     Shape point = new PointImpl(-118.243680, 34.052230);
 
     Document losAngeles = new Document();
     losAngeles.add(new StringField("name", "Los Angeles", Field.Store.YES));
-    losAngeles.add(prefixGridStrategy.createField(fieldInfo, point, true, true));
+    losAngeles.add(prefixGridStrategy.createField(point, true, true));
 
     addDocumentsAndCommit(Arrays.asList(losAngeles));
 

Modified: lucene/dev/branches/branch_4x/lucene/spatial/src/test/org/apache/lucene/spatial/vector/TestTwoDoublesStrategy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/lucene/spatial/src/test/org/apache/lucene/spatial/vector/TestTwoDoublesStrategy.java?rev=1357452&r1=1357451&r2=1357452&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/lucene/spatial/src/test/org/apache/lucene/spatial/vector/TestTwoDoublesStrategy.java (original)
+++ lucene/dev/branches/branch_4x/lucene/spatial/src/test/org/apache/lucene/spatial/vector/TestTwoDoublesStrategy.java Thu Jul  5 03:23:10 2012
@@ -23,35 +23,31 @@ import com.spatial4j.core.shape.Circle;
 import com.spatial4j.core.shape.Point;
 import com.spatial4j.core.shape.simple.CircleImpl;
 import com.spatial4j.core.shape.simple.PointImpl;
-import org.apache.lucene.search.FieldCache;
 import org.apache.lucene.search.Query;
 import org.apache.lucene.spatial.SpatialMatchConcern;
 import org.apache.lucene.spatial.StrategyTestCase;
 import org.apache.lucene.spatial.query.SpatialArgs;
 import org.apache.lucene.spatial.query.SpatialOperation;
-import org.apache.lucene.spatial.util.NumericFieldInfo;
 import org.junit.Before;
 import org.junit.Test;
 
 import java.io.IOException;
 
-public class TestTwoDoublesStrategy extends StrategyTestCase<TwoDoublesFieldInfo> {
+public class TestTwoDoublesStrategy extends StrategyTestCase {
 
   @Before
   @Override
   public void setUp() throws Exception {
     super.setUp();
     this.ctx = SimpleSpatialContext.GEO_KM;
-    this.strategy = new TwoDoublesStrategy(ctx,
-        new NumericFieldInfo(), FieldCache.NUMERIC_UTILS_DOUBLE_PARSER);
-    this.fieldInfo = new TwoDoublesFieldInfo(getClass().getSimpleName());
+    this.strategy = new TwoDoublesStrategy(ctx, getClass().getSimpleName());
   }
 
   @Test
   public void testCircleShapeSupport() {
     Circle circle = new CircleImpl(new PointImpl(0, 0), 10, this.ctx);
     SpatialArgs args = new SpatialArgs(SpatialOperation.Intersects, circle);
-    Query query = this.strategy.makeQuery(args, this.fieldInfo);
+    Query query = this.strategy.makeQuery(args);
 
     assertNotNull(query);
   }
@@ -60,7 +56,7 @@ public class TestTwoDoublesStrategy exte
   public void testInvalidQueryShape() {
     Point point = new PointImpl(0, 0);
     SpatialArgs args = new SpatialArgs(SpatialOperation.Intersects, point);
-    this.strategy.makeQuery(args, this.fieldInfo);
+    this.strategy.makeQuery(args);
   }
 
   @Test