You are viewing a plain text version of this content. The canonical link for it is here.
Posted to java-commits@lucene.apache.org by ry...@apache.org on 2008/12/30 08:37:18 UTC

svn commit: r730067 [2/2] - in /lucene/java/trunk/contrib/spatial: ./ src/ src/java/ src/java/org/ src/java/org/apache/ src/java/org/apache/lucene/ src/java/org/apache/lucene/spatial/ src/java/org/apache/lucene/spatial/geometry/ src/java/org/apache/luc...

Added: lucene/java/trunk/contrib/spatial/src/java/org/apache/lucene/spatial/tier/DistanceFilter.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/spatial/src/java/org/apache/lucene/spatial/tier/DistanceFilter.java?rev=730067&view=auto
==============================================================================
--- lucene/java/trunk/contrib/spatial/src/java/org/apache/lucene/spatial/tier/DistanceFilter.java (added)
+++ lucene/java/trunk/contrib/spatial/src/java/org/apache/lucene/spatial/tier/DistanceFilter.java Mon Dec 29 23:37:17 2008
@@ -0,0 +1,248 @@
+/**
+ * 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.tier;
+
+import java.io.IOException;
+import java.util.BitSet;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.WeakHashMap;
+import java.util.logging.Logger;
+
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.search.FieldCache;
+import org.apache.lucene.spatial.ISerialChainFilter;
+import org.apache.lucene.spatial.tier.DistanceHandler.Precision;
+
+import org.apache.lucene.spatial.NumberUtils;
+
+
+
+public class DistanceFilter extends ISerialChainFilter {
+
+  /**
+   * 
+   */
+  private static final long serialVersionUID = 1L;
+  
+  private double distance;
+  private double lat;
+  private double lng;
+  private String latField;
+  private String lngField;
+  private Logger log = Logger.getLogger(getClass().getName());
+  
+  private Map<Integer,Double> distances = null;
+  private Precision precise = null;
+  
+  /**
+   * Provide a distance filter based from a center point with a radius
+   * in miles
+   * @param lat
+   * @param lng
+   * @param miles
+   * @param latField
+   * @param lngField
+   */
+  public DistanceFilter(double lat, double lng, double miles, String latField, String lngField){
+    distance = miles;
+    this.lat = lat;
+    this.lng = lng;
+    this.latField = latField;
+    this.lngField = lngField;
+  }
+  
+  
+  public Map<Integer,Double> getDistances(){
+    return distances;
+  }
+  
+  public Double getDistance(int docid){
+    return distances.get(docid);
+  }
+  
+  @Override
+  public BitSet bits(IndexReader reader) throws IOException {
+
+    /* Create a BitSet to store the result */
+    int maxdocs = reader.numDocs();
+    BitSet bits = new BitSet(maxdocs);
+    
+    setPrecision(maxdocs);
+    // create an intermediate cache to avoid recomputing
+    //   distances for the same point 
+    //   TODO: Why is this a WeakHashMap? 
+    WeakHashMap<String,Double> cdistance = new WeakHashMap<String,Double>(maxdocs);
+    
+    String[] latIndex = FieldCache.DEFAULT.getStrings(reader, latField);
+    String[] lngIndex = FieldCache.DEFAULT.getStrings(reader, lngField);
+
+    /* store calculated distances for reuse by other components */
+    distances = new HashMap<Integer,Double>(maxdocs);
+    for (int i = 0 ; i < maxdocs; i++) {
+      
+      String sx = latIndex[i];
+      String sy = lngIndex[i];
+  
+      double x = NumberUtils.SortableStr2double(sx);
+      double y = NumberUtils.SortableStr2double(sy);
+      
+      // round off lat / longs if necessary
+//      x = DistanceHandler.getPrecision(x, precise);
+//      y = DistanceHandler.getPrecision(y, precise);
+      
+      String ck = new Double(x).toString()+","+new Double(y).toString();
+      Double cachedDistance = cdistance.get(ck);
+      
+      
+      double d;
+      
+      if(cachedDistance != null){
+        d = cachedDistance.doubleValue();
+      } else {
+        d = DistanceUtils.getInstance().getDistanceMi(lat, lng, x, y);
+        cdistance.put(ck, d);
+      }
+      distances.put(i, d);
+      
+      if (d < distance){
+        bits.set(i);
+      }
+      
+    }
+    
+    return bits;
+  }
+
+  
+  @Override
+  public BitSet bits(IndexReader reader, BitSet bits) throws Exception {
+
+  
+    /* Create a BitSet to store the result */
+    int size = bits.cardinality();
+    BitSet result = new BitSet(size);
+    
+
+    /* create an intermediate cache to avoid recomputing
+         distances for the same point  */
+    HashMap<String,Double> cdistance = new HashMap<String,Double>(size);
+    
+
+    /* store calculated distances for reuse by other components */
+    distances = new HashMap<Integer,Double>(size);
+    
+    long start = System.currentTimeMillis();
+    String[] latIndex = FieldCache.DEFAULT.getStrings(reader, latField);
+    String[] lngIndex = FieldCache.DEFAULT.getStrings(reader, lngField);
+    
+    /* loop over all set bits (hits from the boundary box filters) */
+    int i = bits.nextSetBit(0);
+    while (i >= 0){
+      double x,y;
+      
+      // if we have a completed
+      // filter chain, lat / lngs can be retrived from 
+      // memory rather than document base.
+
+      String sx = latIndex[i];
+      String sy = lngIndex[i];
+      x = NumberUtils.SortableStr2double(sx);
+      y = NumberUtils.SortableStr2double(sy);
+      
+      // round off lat / longs if necessary
+//      x = DistanceHandler.getPrecision(x, precise);
+//      y = DistanceHandler.getPrecision(y, precise);
+
+      String ck = new Double(x).toString()+","+new Double(y).toString();
+      Double cachedDistance = cdistance.get(ck);
+      double d;
+      
+      if(cachedDistance != null){
+        d = cachedDistance.doubleValue();
+        
+      } else {
+        d = DistanceUtils.getInstance().getDistanceMi(lat, lng, x, y);
+        //d = DistanceUtils.getLLMDistance(lat, lng, x, y);
+        cdistance.put(ck, d);
+      }
+      
+      distances.put(i, d);
+        
+      if (d < distance){
+        result.set(i);
+      }
+      i = bits.nextSetBit(i+1);
+    }
+    
+    long end = System.currentTimeMillis();
+    log.fine("Time taken : "+ (end - start) + 
+        ", results : "+ distances.size() + 
+        ", cached : "+ cdistance.size() +
+        ", incoming size: "+ size);
+  
+
+    cdistance = null;
+    
+    return result;
+  }
+
+  /** Returns true if <code>o</code> is equal to this. */
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) return true;
+    if (!(o instanceof DistanceFilter)) return false;
+    DistanceFilter other = (DistanceFilter) o;
+
+    if (this.distance != other.distance ||
+        this.lat != other.lat ||
+        this.lng != other.lng ||
+        !this.latField.equals(other.latField) ||
+        !this.lngField.equals(other.lngField)) {
+      return false;
+    }
+    return true;
+  }
+
+  /** Returns a hash code value for this object.*/
+  @Override
+  public int hashCode() {
+    int h = new Double(distance).hashCode();
+    h ^= new Double(lat).hashCode();
+    h ^= new Double(lng).hashCode();
+    h ^= latField.hashCode();
+    h ^= lngField.hashCode();
+    return h;
+  }
+  
+  private void setPrecision(int maxDocs) {
+    precise = Precision.EXACT;
+    
+    if (maxDocs > 1000 && distance > 10) {
+      precise = Precision.TWENTYFEET;
+    }
+    
+    if (maxDocs > 10000 && distance > 10){
+      precise = Precision.TWOHUNDREDFEET;
+    }
+  }
+
+  public void setDistances(Map<Integer, Double> distances) {
+    this.distances = distances;
+  }
+}

Added: lucene/java/trunk/contrib/spatial/src/java/org/apache/lucene/spatial/tier/DistanceHandler.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/spatial/src/java/org/apache/lucene/spatial/tier/DistanceHandler.java?rev=730067&view=auto
==============================================================================
--- lucene/java/trunk/contrib/spatial/src/java/org/apache/lucene/spatial/tier/DistanceHandler.java (added)
+++ lucene/java/trunk/contrib/spatial/src/java/org/apache/lucene/spatial/tier/DistanceHandler.java Mon Dec 29 23:37:17 2008
@@ -0,0 +1,100 @@
+/**
+ * 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.tier;
+
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * Provide a high level access point to distances
+ * Used by DistanceSortSource and DistanceQuery
+ *  
+ * @author pjaol
+ *
+ */
+public class DistanceHandler {
+  
+  private Map<Integer,Double> distances;
+  public enum Precision {EXACT, TWOFEET, TWENTYFEET, TWOHUNDREDFEET};
+  private Precision precise;
+  
+  public DistanceHandler (Map<Integer,Double> distances, Precision precise){
+    this.distances = distances;
+    this.precise = precise; 
+    
+  }
+  
+  
+  public static double getPrecision(double x, Precision thisPrecise){
+    
+    if(thisPrecise != null){
+      double dif = 0;
+      switch(thisPrecise) {
+        case EXACT: return x;
+        case TWOFEET:        dif = x % 0.0001; break;
+        case TWENTYFEET:     dif = x % 0.001;  break;
+        case TWOHUNDREDFEET: dif = x % 0.01; break;
+      }
+      return x - dif;
+    }
+    return x;
+  }
+  
+  public Precision getPrecision() {
+    return precise;
+  }
+  
+  public double getDistance(int docid, double centerLat, double centerLng, double lat, double lng){
+  
+    // check to see if we have distances
+    // if not calculate the distance
+    if(distances == null){
+      return DistanceUtils.getInstance().getDistanceMi(centerLat, centerLng, lat, lng);
+    }
+    
+    // check to see if the doc id has a cached distance
+    Double docd = distances.get( docid );
+    if (docd != null){
+      return docd.doubleValue();
+    }
+    
+    //check to see if we have a precision code
+    // and if another lat/long has been calculated at
+    // that rounded location
+    if (precise != null) {
+      double xLat = getPrecision(lat, precise);
+      double xLng = getPrecision(lng, precise);
+      
+      String k = new Double(xLat).toString() +","+ new Double(xLng).toString();
+    
+      Double d = (distances.get(k));
+      if (d != null){
+        return d.doubleValue();
+      }
+    }
+    
+    //all else fails calculate the distances    
+    return DistanceUtils.getInstance().getDistanceMi(centerLat, centerLng, lat, lng);
+  }
+  
+  
+  public static void main(String args[]){ 
+    DistanceHandler db = new DistanceHandler(new HashMap(), Precision.TWOHUNDREDFEET);
+    System.out.println(DistanceHandler.getPrecision(-1234.123456789, db.getPrecision()));
+  }
+}

Added: lucene/java/trunk/contrib/spatial/src/java/org/apache/lucene/spatial/tier/DistanceQuery.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/spatial/src/java/org/apache/lucene/spatial/tier/DistanceQuery.java?rev=730067&view=auto
==============================================================================
--- lucene/java/trunk/contrib/spatial/src/java/org/apache/lucene/spatial/tier/DistanceQuery.java (added)
+++ lucene/java/trunk/contrib/spatial/src/java/org/apache/lucene/spatial/tier/DistanceQuery.java Mon Dec 29 23:37:17 2008
@@ -0,0 +1,104 @@
+/**
+ * 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.tier;
+
+import org.apache.lucene.search.ConstantScoreQuery;
+import org.apache.lucene.search.Filter;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.QueryWrapperFilter;
+import org.apache.lucene.spatial.SerialChainFilter;
+
+
+public class DistanceQuery {
+
+  private static final long serialVersionUID = 1L;
+  
+  public BoundaryBoxFilter latFilter;
+  public BoundaryBoxFilter lngFilter;
+  public DistanceFilter distanceFilter;
+  
+  private double lat;
+  private double lng;
+  private double miles;
+  private Filter cartesianFilter;
+  
+  /**
+   * Create a distance query using
+   * a boundary box wrapper around a more precise
+   * DistanceFilter.
+   * 
+   * @see SerialChainFilter
+   * @param lat
+   * @param lng
+   * @param miles
+   */
+  public DistanceQuery (double lat, double lng, double miles, String latField, String lngField, boolean needPrecise){
+
+    this.lat = lat;
+    this.lng = lng;
+    this.miles = miles;
+    
+    
+    CartesianPolyFilter cpf = new CartesianPolyFilter();
+    cartesianFilter = cpf.getBoundingArea(lat, lng, (int)miles);
+
+    /* create precise distance filter */
+    if( needPrecise)
+    	distanceFilter = new DistanceFilter(lat, lng, miles, latField, lngField);
+    
+  }
+
+   /**
+  * Create a distance query using
+  * a boundary box wrapper around a more precise
+  * DistanceFilter.
+  * 
+  * @see SerialChainFilter
+  * @param lat
+  * @param lng
+  * @param miles
+  */
+  public Filter getFilter() {
+    
+      return new SerialChainFilter(new Filter[] {cartesianFilter, distanceFilter},
+                    new int[] {SerialChainFilter.AND,
+                           SerialChainFilter.SERIALAND});
+
+   
+  }
+  
+  public Filter getFilter(Query query) {
+    QueryWrapperFilter qf = new QueryWrapperFilter(query);
+    
+    
+    return new SerialChainFilter(new Filter[] {cartesianFilter, qf, distanceFilter},
+          new int[] {SerialChainFilter.AND, 
+              SerialChainFilter.AND,
+              SerialChainFilter.SERIALAND});
+  
+  }
+    
+  public Query getQuery() {
+      return new ConstantScoreQuery(getFilter());
+  }
+    
+  @Override
+  public String toString() {
+    return "DistanceQuery lat: " + lat + " lng: " + lng + " miles: "+ miles;
+  }
+}

Added: lucene/java/trunk/contrib/spatial/src/java/org/apache/lucene/spatial/tier/DistanceSortSource.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/spatial/src/java/org/apache/lucene/spatial/tier/DistanceSortSource.java?rev=730067&view=auto
==============================================================================
--- lucene/java/trunk/contrib/spatial/src/java/org/apache/lucene/spatial/tier/DistanceSortSource.java (added)
+++ lucene/java/trunk/contrib/spatial/src/java/org/apache/lucene/spatial/tier/DistanceSortSource.java Mon Dec 29 23:37:17 2008
@@ -0,0 +1,93 @@
+/**
+ * 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.tier;
+
+import java.io.IOException;
+
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.search.Filter;
+import org.apache.lucene.search.ScoreDoc;
+import org.apache.lucene.search.ScoreDocComparator;
+import org.apache.lucene.search.SortComparatorSource;
+import org.apache.lucene.search.SortField;
+
+public class DistanceSortSource implements SortComparatorSource {
+
+  private static final long serialVersionUID = 1L;
+
+  private DistanceFilter distanceFilter;
+  private DistanceScoreDocLookupComparator dsdlc;
+  
+  public DistanceSortSource (Filter distanceFilter){
+
+    this.distanceFilter = (DistanceFilter)distanceFilter;
+    
+  }
+  
+  public void cleanUp() {
+    distanceFilter = null;
+    
+    if (dsdlc !=null)
+      dsdlc.cleanUp();
+    
+    dsdlc = null;
+  }
+  
+  public ScoreDocComparator newComparator(IndexReader reader, String field) throws IOException {
+    dsdlc = new DistanceScoreDocLookupComparator(reader, distanceFilter);
+    return dsdlc;
+  }
+
+  
+
+  private class DistanceScoreDocLookupComparator implements ScoreDocComparator {
+
+    private DistanceFilter distanceFilter;
+    
+    public DistanceScoreDocLookupComparator(IndexReader reader, DistanceFilter distanceFilter) {
+      this.distanceFilter = distanceFilter;
+      return;
+    }
+    
+    
+    public int compare(ScoreDoc aDoc, ScoreDoc bDoc) {
+      
+//      if (this.distances == null) {
+//          distances = distanceFilter.getDistances();
+//      }
+      double a = distanceFilter.getDistance(aDoc.doc);
+      double b = distanceFilter.getDistance(bDoc.doc);
+      if (a > b) return 1;
+      if (a < b )return -1;
+      
+      return 0;
+    }
+
+    public int sortType() {
+      return SortField.DOUBLE;
+    }
+
+    public Comparable sortValue(ScoreDoc iDoc) {
+      return distanceFilter.getDistance(iDoc.doc);
+    }
+    
+    public void cleanUp() {
+      distanceFilter = null;
+    }
+  }
+}

Added: lucene/java/trunk/contrib/spatial/src/java/org/apache/lucene/spatial/tier/DistanceUtils.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/spatial/src/java/org/apache/lucene/spatial/tier/DistanceUtils.java?rev=730067&view=auto
==============================================================================
--- lucene/java/trunk/contrib/spatial/src/java/org/apache/lucene/spatial/tier/DistanceUtils.java (added)
+++ lucene/java/trunk/contrib/spatial/src/java/org/apache/lucene/spatial/tier/DistanceUtils.java Mon Dec 29 23:37:17 2008
@@ -0,0 +1,63 @@
+/**
+ * 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.tier;
+
+import org.apache.lucene.spatial.geometry.DistanceUnits;
+import org.apache.lucene.spatial.geometry.FloatLatLng;
+import org.apache.lucene.spatial.geometry.LatLng;
+import org.apache.lucene.spatial.geometry.shape.LLRect;
+import org.apache.lucene.spatial.geometry.shape.Rectangle;
+
+
+public class DistanceUtils {
+
+  static DistanceUtils instance = new DistanceUtils();
+  
+  public static DistanceUtils getInstance()
+  {
+    return instance;
+  }
+
+  
+  public double getDistanceMi(double x1, double y1, double x2, double y2) {
+    return getLLMDistance(x1, y1, x2, y2);
+  }
+
+  /**
+   * 
+   * @param x1
+   * @param y1
+   * @param miles
+   * @return boundary rectangle where getY/getX is top left, getMinY/getMinX is bottom right
+   */
+  public Rectangle getBoundary (double x1, double y1, double miles) {
+
+    LLRect box = LLRect.createBox( new FloatLatLng( x1, y1 ), miles, miles );
+    
+    //System.out.println("Box: "+maxX+" | "+ maxY+" | "+ minX + " | "+ minY);
+    return box.toRectangle();
+
+  }
+  
+  public double getLLMDistance (double x1, double y1, double x2, double y2) {  
+
+    LatLng p1 = new FloatLatLng( x1, y1 );
+    LatLng p2 = new FloatLatLng( x2, y2 );
+    return p1.arcDistance( p2, DistanceUnits.MILES );
+  }
+}

Added: lucene/java/trunk/contrib/spatial/src/java/org/apache/lucene/spatial/tier/InvalidGeoException.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/spatial/src/java/org/apache/lucene/spatial/tier/InvalidGeoException.java?rev=730067&view=auto
==============================================================================
--- lucene/java/trunk/contrib/spatial/src/java/org/apache/lucene/spatial/tier/InvalidGeoException.java (added)
+++ lucene/java/trunk/contrib/spatial/src/java/org/apache/lucene/spatial/tier/InvalidGeoException.java Mon Dec 29 23:37:17 2008
@@ -0,0 +1,30 @@
+/**
+ * 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.tier;
+
+public class InvalidGeoException extends Exception {
+
+  /**
+   * 
+   */
+  private static final long serialVersionUID = 1L;
+
+  public InvalidGeoException(String message){
+    super(message);
+  }
+}

Added: lucene/java/trunk/contrib/spatial/src/java/org/apache/lucene/spatial/tier/Shape.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/spatial/src/java/org/apache/lucene/spatial/tier/Shape.java?rev=730067&view=auto
==============================================================================
--- lucene/java/trunk/contrib/spatial/src/java/org/apache/lucene/spatial/tier/Shape.java (added)
+++ lucene/java/trunk/contrib/spatial/src/java/org/apache/lucene/spatial/tier/Shape.java Mon Dec 29 23:37:17 2008
@@ -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.tier;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * @author pjaol
+ *
+ */
+public class Shape {
+
+  private List<Double> area = new ArrayList<Double>();
+  private String tierId;
+  
+  public Shape (String tierId){
+    this.tierId = tierId;
+  }
+
+  public void addBox(double  boxId){
+    area.add(boxId);
+  }
+  
+  public List<Double> getArea(){
+    return area;
+  }
+  
+  public String getTierId(){
+    return tierId;
+  }
+  
+  public boolean isInside(double boxId){
+    return area.contains(boxId);
+  }
+}

Added: lucene/java/trunk/contrib/spatial/src/java/org/apache/lucene/spatial/tier/projections/CartesianTierPlotter.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/spatial/src/java/org/apache/lucene/spatial/tier/projections/CartesianTierPlotter.java?rev=730067&view=auto
==============================================================================
--- lucene/java/trunk/contrib/spatial/src/java/org/apache/lucene/spatial/tier/projections/CartesianTierPlotter.java (added)
+++ lucene/java/trunk/contrib/spatial/src/java/org/apache/lucene/spatial/tier/projections/CartesianTierPlotter.java Mon Dec 29 23:37:17 2008
@@ -0,0 +1,170 @@
+/**
+ * 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.tier.projections;
+
+/**
+ * 
+ */
+public class CartesianTierPlotter {
+  
+  int tierLevel;
+  int tierLength;
+  int tierBoxes;
+  int tierVerticalPosDivider;
+  IProjector projector;
+  final String fieldPrefix = "_localTier";
+  Double idd = new Double(180);
+  
+  public CartesianTierPlotter (int tierLevel, IProjector projector) {
+  
+    this.tierLevel  = tierLevel;
+    this.projector = projector;
+    
+    setTierLength();
+    setTierBoxes();
+    setTierVerticalPosDivider();
+  }
+  
+  
+  private void setTierLength (){
+    this.tierLength = (int) Math.pow(2 , this.tierLevel);
+  }
+  
+  private void setTierBoxes () {
+    this.tierBoxes = (int)Math.pow(this.tierLength, 2);
+  }
+  
+  /**
+   * Get nearest max power of 10 greater than
+   * the tierlen
+   * e.g
+   * tierId of 13 has tierLen 8192
+   * nearest max power of 10 greater than tierLen 
+   * would be 10,000
+   */
+  
+  private void setTierVerticalPosDivider() {
+    
+    // ceiling of log base 10 of tierLen
+    
+    tierVerticalPosDivider = new Double(Math.ceil(
+          Math.log10(new Integer(this.tierLength).doubleValue()))).intValue();
+    
+    // 
+    tierVerticalPosDivider = (int)Math.pow(10, tierVerticalPosDivider );
+    
+  }
+  
+  public double getTierVerticalPosDivider(){
+    return tierVerticalPosDivider;
+  }
+  
+  /**
+   * TierBoxId is latitude box id + longitude box id
+   * where latitude box id, and longitude box id are transposded in to position
+   * coordinates.
+   * 
+   * @param latitude
+   * @param longitude
+   * @return
+   */
+  public double getTierBoxId (double latitude, double longitude) {
+    
+    double[] coords = projector.coords(latitude, longitude);
+    
+    double id = getBoxId(coords[0]) + (getBoxId(coords[1]) / tierVerticalPosDivider);
+    return id ;
+  }
+  
+  
+  private double getBoxId (double coord){
+    
+    
+    return Math.floor(coord / (idd / this.tierLength));
+  }
+  
+  @SuppressWarnings("unused")
+  private double getBoxId (double coord, int tierLen){
+    return Math.floor(coord / (idd / tierLen) );
+  }
+  /**
+   * get the string name representing current tier
+   * _localTier&lt;tiedId&gt;
+   * @return
+   */
+  public String getTierFieldName (){
+    
+    return fieldPrefix + this.tierLevel;
+  }
+  
+  /**
+   * get the string name representing tierId
+   * _localTier&lt;tierId&gt;
+   * @param tierId
+   * @return
+   */
+  public String getTierFieldName (int tierId){
+    
+    return fieldPrefix + tierId;
+  }
+  
+  /**
+   * Find the tier with the best fit for a bounding box
+   * Best fit is defined as the ceiling of
+   *  log2 (circumference of earth / distance) 
+   *  distance is defined as the smallest box fitting
+   *  the corner between a radius and a bounding box.
+   *  
+   *  Distances less than a mile return 15, finer granularity is
+   *  in accurate
+   * 
+   * @param latitude
+   * @param longitude
+   * @return
+   */
+  public int bestFit(int miles){
+    
+    //28,892 a rough circumference of the earth
+    int circ = 28892;
+    
+    double r = miles / 2.0;
+    
+    double corner = r - Math.sqrt(Math.pow(r, 2) / 2.0d);
+    System.out.println("corner "+ corner);
+    double times = circ / corner;
+    int bestFit =  (int)Math.ceil(log2(times)) + 1;
+    
+    if (bestFit > 15) {
+      // 15 is the granularity of about 1 mile
+      // finer granularity isn't accurate with standard java math
+      return 15;
+    }
+    return bestFit;
+  }
+  
+  /**
+   * a log to the base 2 formula
+   * <code>Math.log(value) / Math.log(2)</code>
+   * @param value
+   * @return
+   */
+  public double log2(double value) {
+    
+    return Math.log(value) / Math.log(2);
+  }
+}

Added: lucene/java/trunk/contrib/spatial/src/java/org/apache/lucene/spatial/tier/projections/IProjector.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/spatial/src/java/org/apache/lucene/spatial/tier/projections/IProjector.java?rev=730067&view=auto
==============================================================================
--- lucene/java/trunk/contrib/spatial/src/java/org/apache/lucene/spatial/tier/projections/IProjector.java (added)
+++ lucene/java/trunk/contrib/spatial/src/java/org/apache/lucene/spatial/tier/projections/IProjector.java Mon Dec 29 23:37:17 2008
@@ -0,0 +1,23 @@
+/**
+ * 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.tier.projections;
+
+public interface IProjector {
+  public String coordsAsString(double latitude, double longitude);
+  public double[] coords(double latitude, double longitude);
+}
\ No newline at end of file

Added: lucene/java/trunk/contrib/spatial/src/java/org/apache/lucene/spatial/tier/projections/SinusoidalProjector.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/spatial/src/java/org/apache/lucene/spatial/tier/projections/SinusoidalProjector.java?rev=730067&view=auto
==============================================================================
--- lucene/java/trunk/contrib/spatial/src/java/org/apache/lucene/spatial/tier/projections/SinusoidalProjector.java (added)
+++ lucene/java/trunk/contrib/spatial/src/java/org/apache/lucene/spatial/tier/projections/SinusoidalProjector.java Mon Dec 29 23:37:17 2008
@@ -0,0 +1,41 @@
+/**
+ * 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.tier.projections;
+
+/**
+ * Based on Sinusoidal Projections
+ * Project a latitude / longitude on a 2D cartisian map
+ * 
+ */
+public class SinusoidalProjector implements IProjector {
+
+  
+  public String coordsAsString(double latitude, double longitude) {
+    return null;
+  }
+  
+  public double[] coords(double latitude, double longitude) {
+    double rlat = Math.toRadians(latitude);
+    double rlong = Math.toRadians(longitude);
+    double nlat = rlong * Math.cos(rlat);
+    double r[] = {nlat, rlong};
+    return r;
+    
+  }
+  
+}

Added: lucene/java/trunk/contrib/spatial/src/test/org/apache/lucene/spatial/tier/DistanceCheck.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/spatial/src/test/org/apache/lucene/spatial/tier/DistanceCheck.java?rev=730067&view=auto
==============================================================================
--- lucene/java/trunk/contrib/spatial/src/test/org/apache/lucene/spatial/tier/DistanceCheck.java (added)
+++ lucene/java/trunk/contrib/spatial/src/test/org/apache/lucene/spatial/tier/DistanceCheck.java Mon Dec 29 23:37:17 2008
@@ -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.tier;
+
+import java.text.DecimalFormat;
+
+import org.apache.lucene.spatial.tier.DistanceUtils;
+
+
+public class DistanceCheck {
+
+  /**
+   * @param args
+   */
+  public static void main(String[] args) {
+    double lat1 = 0;
+    double long1 = 0;
+    double lat2 = 0;
+    double long2 = 0;
+    
+    for (int i =0; i < 90; i++){
+      double dis = DistanceUtils.getInstance().getDistanceMi(lat1, long1, lat2, long2);
+      lat1 +=1;
+      lat2 = lat1 + 0.001;
+      
+      System.out.println(lat1+","+long1+","+lat2+","+long2+","+formatDistance(dis));
+      
+    }
+
+  }
+
+  public static String formatDistance (Double d){
+    DecimalFormat df1 = new DecimalFormat("####.000000");
+    return df1.format(d);
+  }
+  
+}

Added: lucene/java/trunk/contrib/spatial/src/test/org/apache/lucene/spatial/tier/PolyShape.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/spatial/src/test/org/apache/lucene/spatial/tier/PolyShape.java?rev=730067&view=auto
==============================================================================
--- lucene/java/trunk/contrib/spatial/src/test/org/apache/lucene/spatial/tier/PolyShape.java (added)
+++ lucene/java/trunk/contrib/spatial/src/test/org/apache/lucene/spatial/tier/PolyShape.java Mon Dec 29 23:37:17 2008
@@ -0,0 +1 @@
+/**
 * 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.tier;


/**
 *
 */
public class PolyShape {

  private static double lat = 38.969398; 
  private static double lng= -77.386398;
  private static int mil
 es = 1000;
  /**
   * @param args
   */
  public static void main(String[] args) {
  
    CartesianPolyFilter cpf = new CartesianPolyFilter();
    cpf.getBoxShape(lat, lng, miles);
    
  }

}
\ No newline at end of file

Added: lucene/java/trunk/contrib/spatial/src/test/org/apache/lucene/spatial/tier/TestCartesian.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/spatial/src/test/org/apache/lucene/spatial/tier/TestCartesian.java?rev=730067&view=auto
==============================================================================
--- lucene/java/trunk/contrib/spatial/src/test/org/apache/lucene/spatial/tier/TestCartesian.java (added)
+++ lucene/java/trunk/contrib/spatial/src/test/org/apache/lucene/spatial/tier/TestCartesian.java Mon Dec 29 23:37:17 2008
@@ -0,0 +1,225 @@
+/**
+ * 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.tier;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+
+import junit.framework.TestCase;
+
+import org.apache.lucene.analysis.WhitespaceAnalyzer;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.index.IndexWriter;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.Hits;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.Sort;
+import org.apache.lucene.search.SortField;
+import org.apache.lucene.search.TermQuery;
+import org.apache.lucene.spatial.tier.DistanceQuery;
+import org.apache.lucene.spatial.tier.DistanceSortSource;
+import org.apache.lucene.spatial.tier.DistanceUtils;
+import org.apache.lucene.spatial.tier.InvalidGeoException;
+import org.apache.lucene.spatial.tier.projections.CartesianTierPlotter;
+import org.apache.lucene.spatial.tier.projections.IProjector;
+import org.apache.lucene.spatial.tier.projections.SinusoidalProjector;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.RAMDirectory;
+import org.apache.lucene.spatial.NumberUtils;
+import org.apache.lucene.search.function.CustomScoreQuery;
+import org.apache.lucene.search.function.FieldScoreQuery;
+import org.apache.lucene.search.function.FieldScoreQuery.Type;
+
+
+/**
+ *
+ */
+public class TestCartesian extends TestCase{
+
+  /**
+   * @param args
+   */
+  
+  private Directory directory;
+  private IndexSearcher searcher;
+  // reston va
+  private double lat = 38.969398; 
+  private double lng= -77.386398;
+  private String latField = "lat";
+  private String lngField = "lng";
+  private List<CartesianTierPlotter> ctps = new LinkedList<CartesianTierPlotter>();
+  
+  private IProjector project = new SinusoidalProjector();
+  
+
+
+  @Override
+  protected void setUp() throws IOException {
+    directory = new RAMDirectory();
+
+    IndexWriter writer = new IndexWriter(directory, new WhitespaceAnalyzer(), true);
+    
+    setUpPlotter( 2, 15);
+    
+    addData(writer);
+    
+  }
+  
+  
+  private void setUpPlotter(int base, int top) {
+    
+    for (; base <= top; base ++){
+      ctps.add(new CartesianTierPlotter(base,project ));
+    
+    }
+  }
+  
+  private void addPoint(IndexWriter writer, String name, double lat, double lng) throws IOException{
+    
+    Document doc = new Document();
+    
+    doc.add(new Field("name", name,Field.Store.YES, Field.Index.TOKENIZED));
+    
+    // convert the lat / long to lucene fields
+    doc.add(new Field(latField, NumberUtils.double2sortableStr(lat),Field.Store.YES, Field.Index.UN_TOKENIZED));
+    doc.add(new Field(lngField, NumberUtils.double2sortableStr(lng),Field.Store.YES, Field.Index.UN_TOKENIZED));
+    
+    // add a default meta field to make searching all documents easy 
+    doc.add(new Field("metafile", "doc",Field.Store.YES, Field.Index.TOKENIZED));
+    
+    int ctpsize = ctps.size();
+    for (int i =0; i < ctpsize; i++){
+      CartesianTierPlotter ctp = ctps.get(i);
+      doc.add(new Field(ctp.getTierFieldName(), 
+          NumberUtils.double2sortableStr(ctp.getTierBoxId(lat,lng)),
+          Field.Store.YES, 
+          Field.Index.NO_NORMS));
+    }
+    writer.addDocument(doc);
+    
+  }
+  
+  
+  
+  private void addData(IndexWriter writer) throws IOException {
+    addPoint(writer,"McCormick &amp; Schmick's Seafood Restaurant",38.9579000,-77.3572000);
+    addPoint(writer,"Jimmy's Old Town Tavern",38.9690000,-77.3862000);
+    addPoint(writer,"Ned Devine's",38.9510000,-77.4107000);
+    addPoint(writer,"Old Brogue Irish Pub",38.9955000,-77.2884000);
+    addPoint(writer,"Alf Laylah Wa Laylah",38.8956000,-77.4258000);
+    addPoint(writer,"Sully's Restaurant &amp; Supper",38.9003000,-77.4467000);
+    addPoint(writer,"TGIFriday",38.8725000,-77.3829000);
+    addPoint(writer,"Potomac Swing Dance Club",38.9027000,-77.2639000);
+    addPoint(writer,"White Tiger Restaurant",38.9027000,-77.2638000);
+    addPoint(writer,"Jammin' Java",38.9039000,-77.2622000);
+    addPoint(writer,"Potomac Swing Dance Club",38.9027000,-77.2639000);
+    addPoint(writer,"WiseAcres Comedy Club",38.9248000,-77.2344000);
+    addPoint(writer,"Glen Echo Spanish Ballroom",38.9691000,-77.1400000);
+    addPoint(writer,"Whitlow's on Wilson",38.8889000,-77.0926000);
+    addPoint(writer,"Iota Club and Cafe",38.8890000,-77.0923000);
+    addPoint(writer,"Hilton Washington Embassy Row",38.9103000,-77.0451000);
+    addPoint(writer,"HorseFeathers, Bar & Grill", 39.01220000000001, -77.3942);
+    writer.flush();
+    writer.commit();
+    //writer.close();
+  }
+  
+  public void testRange() throws IOException, InvalidGeoException {
+    searcher = new IndexSearcher(directory);
+    
+    final double miles = 6.0;
+    
+    // create a distance query
+    final DistanceQuery dq = new DistanceQuery(lat, lng, miles, latField, lngField, true);
+     
+    System.out.println(dq);
+    //create a term query to search against all documents
+    Query tq = new TermQuery(new Term("metafile", "doc"));
+    
+    FieldScoreQuery fsQuery = new FieldScoreQuery("geo_distance", Type.FLOAT);
+    CustomScoreQuery customScore = new CustomScoreQuery(tq,fsQuery){
+      
+      @Override
+      public float customScore(int doc, float subQueryScore, float valSrcScore){
+        //System.out.println(doc);
+        if (dq.distanceFilter.getDistance(doc) == null)
+          return 0;
+        
+        double distance = dq.distanceFilter.getDistance(doc);
+        // boost score shouldn't exceed 1
+        if (distance < 1.0d)
+          distance = 1.0d;
+        //boost by distance is invertly proportional to
+        // to distance from center point to location
+        float score = new Float((miles - distance) / miles ).floatValue();
+        return score * subQueryScore;
+      }
+    };
+    // Create a distance sort
+    // As the radius filter has performed the distance calculations
+    // already, pass in the filter to reuse the results.
+    // 
+    DistanceSortSource dsort = new DistanceSortSource(dq.distanceFilter);
+    Sort sort = new Sort(new SortField("foo", dsort));
+    
+    // Perform the search, using the term query, the serial chain filter, and the
+    // distance sort
+    Hits hits = searcher.search(customScore, dq.getFilter()); //,sort);
+
+    int results = hits.length();
+    
+    // Get a list of distances 
+    Map<Integer,Double> distances = dq.distanceFilter.getDistances();
+    
+    // distances calculated from filter first pass must be less than total
+    // docs, from the above test of 20 items, 12 will come from the boundary box
+    // filter, but only 5 are actually in the radius of the results.
+    
+    // Note Boundary Box filtering, is not accurate enough for most systems.
+    
+    
+    System.out.println("Distance Filter filtered: " + distances.size());
+    System.out.println("Results: " + results);
+    System.out.println("=============================");
+    System.out.println("Distances should be 14 "+ distances.size());
+    System.out.println("Results should be 7 "+ results);
+
+    assertEquals(14, distances.size());
+    assertEquals(7, results);
+    
+    for(int i =0 ; i < results; i++){
+      Document d = hits.doc(i);
+      
+      String name = d.get("name");
+      double rsLat = NumberUtils.SortableStr2double(d.get(latField));
+      double rsLng = NumberUtils.SortableStr2double(d.get(lngField)); 
+      Double geo_distance = distances.get(hits.id(i));
+      
+      double distance = DistanceUtils.getInstance().getDistanceMi(lat, lng, rsLat, rsLng);
+      double llm = DistanceUtils.getInstance().getLLMDistance(lat, lng, rsLat, rsLng);
+      System.out.println("Name: "+ name +", Distance (res, ortho, harvesine):"+ distance +" |"+ geo_distance +"|"+ llm +" | score "+ hits.score(i));
+      assertTrue(Math.abs((distance - llm)) < 1);
+      assertTrue((distance < miles ));
+    }
+  }
+  
+}

Added: lucene/java/trunk/contrib/spatial/src/test/org/apache/lucene/spatial/tier/TestDistance.java
URL: http://svn.apache.org/viewvc/lucene/java/trunk/contrib/spatial/src/test/org/apache/lucene/spatial/tier/TestDistance.java?rev=730067&view=auto
==============================================================================
--- lucene/java/trunk/contrib/spatial/src/test/org/apache/lucene/spatial/tier/TestDistance.java (added)
+++ lucene/java/trunk/contrib/spatial/src/test/org/apache/lucene/spatial/tier/TestDistance.java Mon Dec 29 23:37:17 2008
@@ -0,0 +1,137 @@
+/**
+ * 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.tier;
+
+import java.io.IOException;
+import java.util.Map;
+
+import junit.framework.TestCase;
+
+import org.apache.lucene.analysis.WhitespaceAnalyzer;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+import org.apache.lucene.index.IndexWriter;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.search.Hits;
+import org.apache.lucene.search.IndexSearcher;
+import org.apache.lucene.search.Query;
+import org.apache.lucene.search.Sort;
+import org.apache.lucene.search.SortField;
+import org.apache.lucene.search.TermQuery;
+import org.apache.lucene.spatial.tier.DistanceQuery;
+import org.apache.lucene.spatial.tier.DistanceSortSource;
+import org.apache.lucene.spatial.tier.DistanceUtils;
+import org.apache.lucene.spatial.tier.InvalidGeoException;
+import org.apache.lucene.store.RAMDirectory;
+import org.apache.lucene.spatial.NumberUtils;
+
+
+/**
+ *
+ */
+public class TestDistance extends TestCase{
+
+  
+  private RAMDirectory directory;
+  private IndexSearcher searcher;
+  // reston va
+  private double lat = 38.969398; 
+  private double lng= -77.386398;
+  private String latField = "lat";
+  private String lngField = "lng";
+  
+  
+  @Override
+  protected void setUp() throws IOException {
+    directory = new RAMDirectory();
+    IndexWriter writer = new IndexWriter(directory, new WhitespaceAnalyzer(), true);
+    addData(writer);
+    
+  }
+  
+  
+  private void addPoint(IndexWriter writer, String name, double lat, double lng) throws IOException{
+    
+    Document doc = new Document();
+    
+    doc.add(new Field("name", name,Field.Store.YES, Field.Index.TOKENIZED));
+    
+    // convert the lat / long to lucene fields
+    doc.add(new Field(latField, NumberUtils.double2sortableStr(lat),Field.Store.YES, Field.Index.UN_TOKENIZED));
+    doc.add(new Field(lngField, NumberUtils.double2sortableStr(lng),Field.Store.YES, Field.Index.UN_TOKENIZED));
+    
+    // add a default meta field to make searching all documents easy 
+    doc.add(new Field("metafile", "doc",Field.Store.YES, Field.Index.TOKENIZED));
+    writer.addDocument(doc);
+    
+  }
+  
+
+  private void addData(IndexWriter writer) throws IOException {
+    addPoint(writer,"McCormick &amp; Schmick's Seafood Restaurant",38.9579000,-77.3572000);
+    addPoint(writer,"Jimmy's Old Town Tavern",38.9690000,-77.3862000);
+    addPoint(writer,"Ned Devine's",38.9510000,-77.4107000);
+    addPoint(writer,"Old Brogue Irish Pub",38.9955000,-77.2884000);
+    addPoint(writer,"Alf Laylah Wa Laylah",38.8956000,-77.4258000);
+    addPoint(writer,"Sully's Restaurant &amp; Supper",38.9003000,-77.4467000);
+    addPoint(writer,"TGIFriday",38.8725000,-77.3829000);
+    addPoint(writer,"Potomac Swing Dance Club",38.9027000,-77.2639000);
+    addPoint(writer,"White Tiger Restaurant",38.9027000,-77.2638000);
+    addPoint(writer,"Jammin' Java",38.9039000,-77.2622000);
+    addPoint(writer,"Potomac Swing Dance Club",38.9027000,-77.2639000);
+    addPoint(writer,"WiseAcres Comedy Club",38.9248000,-77.2344000);
+    addPoint(writer,"Glen Echo Spanish Ballroom",38.9691000,-77.1400000);
+    addPoint(writer,"Whitlow's on Wilson",38.8889000,-77.0926000);
+    addPoint(writer,"Iota Club and Cafe",38.8890000,-77.0923000);
+    addPoint(writer,"Hilton Washington Embassy Row",38.9103000,-77.0451000);
+    addPoint(writer,"HorseFeathers, Bar & Grill", 39.01220000000001, -77.3942);
+    writer.flush();
+  }
+  
+ 
+  
+  public void testMiles() {
+    double LLM = DistanceUtils.getInstance().getLLMDistance(lat, lng,39.012200001, -77.3942);
+    System.out.println(LLM);
+    System.out.println("-->"+DistanceUtils.getInstance().getDistanceMi(lat, lng, 39.0122, -77.3942));
+  }
+  
+  public void testMiles2(){
+    System.out.println("Test Miles 2");
+    double LLM = DistanceUtils.getInstance().getLLMDistance(44.30073, -78.32131,43.687267, -79.39842);
+    System.out.println(LLM);
+    System.out.println("-->"+DistanceUtils.getInstance().getDistanceMi(44.30073, -78.32131, 43.687267, -79.39842));
+    
+  }
+
+  
+//  public void testDistanceQueryCacheable() throws IOException {
+//
+//    // create two of the same distance queries
+//    double miles = 6.0;
+//    DistanceQuery dq1 = new DistanceQuery(lat, lng, miles, latField, lngField, true);
+//    DistanceQuery dq2 = new DistanceQuery(lat, lng, miles, latField, lngField, true);
+//
+//    /* ensure that they hash to the same code, which will cause a cache hit in solr */
+//    System.out.println("hash differences?");
+//    assertEquals(dq1.getQuery().hashCode(), dq2.getQuery().hashCode());
+//    
+//    /* ensure that changing the radius makes a different hash code, creating a cache miss in solr */
+//    DistanceQuery widerQuery = new DistanceQuery(lat, lng, miles + 5.0, latField, lngField, false);
+//    assertTrue(dq1.getQuery().hashCode() != widerQuery.getQuery().hashCode());
+//  }
+}