You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by cm...@apache.org on 2013/08/11 14:19:39 UTC

svn commit: r1512909 [20/38] - in /lucene/dev/branches/lucene4956: ./ dev-tools/ dev-tools/eclipse/ dev-tools/idea/.idea/libraries/ dev-tools/idea/lucene/suggest/ dev-tools/idea/solr/contrib/dataimporthandler/ dev-tools/idea/solr/core/src/test/ dev-too...

Modified: lucene/dev/branches/lucene4956/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/ContainsPrefixTreeFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/ContainsPrefixTreeFilter.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/ContainsPrefixTreeFilter.java (original)
+++ lucene/dev/branches/lucene4956/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/ContainsPrefixTreeFilter.java Sun Aug 11 12:19:13 2013
@@ -41,8 +41,35 @@ import java.util.Collection;
  */
 public class ContainsPrefixTreeFilter extends AbstractPrefixTreeFilter {
 
-  public ContainsPrefixTreeFilter(Shape queryShape, String fieldName, SpatialPrefixTree grid, int detailLevel) {
+  /*
+  Future optimizations:
+    Instead of seekExact, use seekCeil with some leap-frogging, like Intersects does.
+  */
+
+  /**
+   * If the spatial data for a document is comprised of multiple overlapping or adjacent parts,
+   * it might fail to match a query shape when doing the CONTAINS predicate when the sum of
+   * those shapes contain the query shape but none do individually.  Set this to false to
+   * increase performance if you don't care about that circumstance (such as if your indexed
+   * data doesn't even have such conditions).  See LUCENE-5062.
+   */
+  protected final boolean multiOverlappingIndexedShapes;
+
+  public ContainsPrefixTreeFilter(Shape queryShape, String fieldName, SpatialPrefixTree grid, int detailLevel, boolean multiOverlappingIndexedShapes) {
     super(queryShape, fieldName, grid, detailLevel);
+    this.multiOverlappingIndexedShapes = multiOverlappingIndexedShapes;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (!super.equals(o))
+      return false;
+    return multiOverlappingIndexedShapes == ((ContainsPrefixTreeFilter)o).multiOverlappingIndexedShapes;
+  }
+
+  @Override
+  public int hashCode() {
+    return super.hashCode() + (multiOverlappingIndexedShapes ? 1 : 0);
   }
 
   @Override
@@ -65,18 +92,25 @@ public class ContainsPrefixTreeFilter ex
       if (termsEnum == null)//signals all done
         return null;
 
-      //Leaf docs match all query shape
+      // Leaf docs match all query shape
       SmallDocSet leafDocs = getLeafDocs(cell, acceptContains);
 
-      // Get the AND of all child results
+      // Get the AND of all child results (into combinedSubResults)
       SmallDocSet combinedSubResults = null;
-      Collection<Cell> subCells = cell.getSubCells(queryShape);
+      //   Optimization: use null subCellsFilter when we know cell is within the query shape.
+      Shape subCellsFilter = queryShape;
+      if (cell.getLevel() != 0 && ((cell.getShapeRel() == null || cell.getShapeRel() == SpatialRelation.WITHIN))) {
+        subCellsFilter = null;
+        assert cell.getShape().relate(queryShape) == SpatialRelation.WITHIN;
+      }
+      Collection <Cell> subCells = cell.getSubCells(subCellsFilter);
       for (Cell subCell : subCells) {
         if (!seekExact(subCell))
           combinedSubResults = null;
         else if (subCell.getLevel() == detailLevel)
           combinedSubResults = getDocs(subCell, acceptContains);
-        else if (subCell.getShapeRel() == SpatialRelation.WITHIN)
+        else if (!multiOverlappingIndexedShapes &&
+            subCell.getShapeRel() == SpatialRelation.WITHIN)
           combinedSubResults = getLeafDocs(subCell, acceptContains);
         else
           combinedSubResults = visit(subCell, acceptContains); //recursion
@@ -90,17 +124,18 @@ public class ContainsPrefixTreeFilter ex
       if (combinedSubResults != null) {
         if (leafDocs == null)
           return combinedSubResults;
-        return leafDocs.union(combinedSubResults);
+        return leafDocs.union(combinedSubResults);//union is 'or'
       }
       return leafDocs;
     }
 
     private boolean seekExact(Cell cell) throws IOException {
       assert new BytesRef(cell.getTokenBytes()).compareTo(termBytes) > 0;
-
       termBytes.bytes = cell.getTokenBytes();
       termBytes.length = termBytes.bytes.length;
-      return termsEnum.seekExact(termBytes, cell.getLevel() <= 2);
+      if (termsEnum == null)
+        return false;
+      return termsEnum.seekExact(termBytes);
     }
 
     private SmallDocSet getDocs(Cell cell, Bits acceptContains) throws IOException {
@@ -109,9 +144,15 @@ public class ContainsPrefixTreeFilter ex
       return collectDocs(acceptContains);
     }
 
+    private Cell lastLeaf = null;//just for assertion
+
     private SmallDocSet getLeafDocs(Cell leafCell, Bits acceptContains) throws IOException {
       assert new BytesRef(leafCell.getTokenBytes()).equals(termBytes);
+      assert ! leafCell.equals(lastLeaf);//don't call for same leaf again
+      lastLeaf = leafCell;
 
+      if (termsEnum == null)
+        return null;
       BytesRef nextTerm = termsEnum.next();
       if (nextTerm == null) {
         termsEnum = null;//signals all done

Modified: lucene/dev/branches/lucene4956/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/PrefixTreeStrategy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/PrefixTreeStrategy.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/PrefixTreeStrategy.java (original)
+++ lucene/dev/branches/lucene4956/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/PrefixTreeStrategy.java Sun Aug 11 12:19:13 2013
@@ -56,7 +56,7 @@ import java.util.concurrent.ConcurrentHa
  * <li>Only {@link org.apache.lucene.spatial.query.SpatialOperation#Intersects}
  * is supported.  If only points are indexed then this is effectively equivalent
  * to IsWithin.</li>
- * <li>The strategy supports {@link #makeDistanceValueSource(com.spatial4j.core.shape.Point)}
+ * <li>The strategy supports {@link #makeDistanceValueSource(com.spatial4j.core.shape.Point,double)}
  * even for multi-valued data, so long as the indexed data is all points; the
  * behavior is undefined otherwise.  However, <em>it will likely be removed in
  * the future</em> in lieu of using another strategy with a more scalable
@@ -182,7 +182,7 @@ public abstract class PrefixTreeStrategy
   }
 
   @Override
-  public ValueSource makeDistanceValueSource(Point queryPoint) {
+  public ValueSource makeDistanceValueSource(Point queryPoint, double multiplier) {
     PointPrefixTreeFieldCacheProvider p = provider.get( getFieldName() );
     if( p == null ) {
       synchronized (this) {//double checked locking idiom is okay since provider is threadsafe
@@ -194,7 +194,7 @@ public abstract class PrefixTreeStrategy
       }
     }
 
-    return new ShapeFieldCacheDistanceValueSource(ctx, p, queryPoint);
+    return new ShapeFieldCacheDistanceValueSource(ctx, p, queryPoint, multiplier);
   }
 
   public SpatialPrefixTree getGrid() {

Modified: lucene/dev/branches/lucene4956/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/RecursivePrefixTreeStrategy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/RecursivePrefixTreeStrategy.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/RecursivePrefixTreeStrategy.java (original)
+++ lucene/dev/branches/lucene4956/lucene/spatial/src/java/org/apache/lucene/spatial/prefix/RecursivePrefixTreeStrategy.java Sun Aug 11 12:19:13 2013
@@ -38,6 +38,13 @@ public class RecursivePrefixTreeStrategy
 
   private int prefixGridScanLevel;
 
+  /** True if only indexed points shall be supported.  See
+   *  {@link IntersectsPrefixTreeFilter#hasIndexedLeaves}. */
+  protected boolean pointsOnly = false;
+
+  /** See {@link ContainsPrefixTreeFilter#multiOverlappingIndexedShapes}. */
+  protected boolean multiOverlappingIndexedShapes = true;
+
   public RecursivePrefixTreeStrategy(SpatialPrefixTree grid, String fieldName) {
     super(grid, fieldName,
         true);//simplify indexed cells
@@ -69,18 +76,17 @@ public class RecursivePrefixTreeStrategy
 
     Shape shape = args.getShape();
     int detailLevel = grid.getLevelForDistance(args.resolveDistErr(ctx, distErrPct));
-    final boolean hasIndexedLeaves = true;
 
-    if (op == SpatialOperation.Intersects) {
+    if (pointsOnly || op == SpatialOperation.Intersects) {
       return new IntersectsPrefixTreeFilter(
-          shape, getFieldName(), grid, detailLevel, prefixGridScanLevel,
-          hasIndexedLeaves);
+          shape, getFieldName(), grid, detailLevel, prefixGridScanLevel, !pointsOnly);
     } else if (op == SpatialOperation.IsWithin) {
       return new WithinPrefixTreeFilter(
           shape, getFieldName(), grid, detailLevel, prefixGridScanLevel,
           -1);//-1 flag is slower but ensures correct results
     } else if (op == SpatialOperation.Contains) {
-      return new ContainsPrefixTreeFilter(shape, getFieldName(), grid, detailLevel);
+      return new ContainsPrefixTreeFilter(shape, getFieldName(), grid, detailLevel,
+          multiOverlappingIndexedShapes);
     }
     throw new UnsupportedSpatialOperation(op);
   }

Modified: lucene/dev/branches/lucene4956/lucene/spatial/src/java/org/apache/lucene/spatial/util/ShapeFieldCacheDistanceValueSource.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/spatial/src/java/org/apache/lucene/spatial/util/ShapeFieldCacheDistanceValueSource.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/spatial/src/java/org/apache/lucene/spatial/util/ShapeFieldCacheDistanceValueSource.java (original)
+++ lucene/dev/branches/lucene4956/lucene/spatial/src/java/org/apache/lucene/spatial/util/ShapeFieldCacheDistanceValueSource.java Sun Aug 11 12:19:13 2013
@@ -38,14 +38,17 @@ import java.util.Map;
  */
 public class ShapeFieldCacheDistanceValueSource extends ValueSource {
 
-  private final ShapeFieldCacheProvider<Point> provider;
   private final SpatialContext ctx;
   private final Point from;
+  private final ShapeFieldCacheProvider<Point> provider;
+  private final double multiplier;
 
-  public ShapeFieldCacheDistanceValueSource(SpatialContext ctx, ShapeFieldCacheProvider<Point> provider, Point from) {
+  public ShapeFieldCacheDistanceValueSource(SpatialContext ctx,
+      ShapeFieldCacheProvider<Point> provider, Point from, double multiplier) {
     this.ctx = ctx;
     this.from = from;
     this.provider = provider;
+    this.multiplier = multiplier;
   }
 
   @Override
@@ -60,7 +63,7 @@ public class ShapeFieldCacheDistanceValu
           provider.getCache(readerContext.reader());
       private final Point from = ShapeFieldCacheDistanceValueSource.this.from;
       private final DistanceCalculator calculator = ctx.getDistCalc();
-      private final double nullValue = (ctx.isGeo() ? 180 : Double.MAX_VALUE);
+      private final double nullValue = (ctx.isGeo() ? 180 * multiplier : Double.MAX_VALUE);
 
       @Override
       public float floatVal(int doc) {
@@ -69,13 +72,14 @@ public class ShapeFieldCacheDistanceValu
 
       @Override
       public double doubleVal(int doc) {
+
         List<Point> vals = cache.getShapes( doc );
         if( vals != null ) {
           double v = calculator.distance(from, vals.get(0));
           for( int i=1; i<vals.size(); i++ ) {
             v = Math.min(v, calculator.distance(from, vals.get(i)));
           }
-          return v;
+          return v * multiplier;
         }
         return nullValue;
       }
@@ -97,6 +101,7 @@ public class ShapeFieldCacheDistanceValu
     if (!ctx.equals(that.ctx)) return false;
     if (!from.equals(that.from)) return false;
     if (!provider.equals(that.provider)) return false;
+    if (multiplier != that.multiplier) return false;
 
     return true;
   }

Modified: lucene/dev/branches/lucene4956/lucene/spatial/src/java/org/apache/lucene/spatial/vector/DistanceValueSource.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/spatial/src/java/org/apache/lucene/spatial/vector/DistanceValueSource.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/spatial/src/java/org/apache/lucene/spatial/vector/DistanceValueSource.java (original)
+++ lucene/dev/branches/lucene4956/lucene/spatial/src/java/org/apache/lucene/spatial/vector/DistanceValueSource.java Sun Aug 11 12:19:13 2013
@@ -39,13 +39,15 @@ public class DistanceValueSource extends
 
   private PointVectorStrategy strategy;
   private final Point from;
+  private final double multiplier;
 
   /**
    * Constructor.
    */
-  public DistanceValueSource(PointVectorStrategy strategy, Point from) {
+  public DistanceValueSource(PointVectorStrategy strategy, Point from, double multiplier) {
     this.strategy = strategy;
     this.from = from;
+    this.multiplier = multiplier;
   }
 
   /**
@@ -72,7 +74,8 @@ public class DistanceValueSource extends
 
       private final Point from = DistanceValueSource.this.from;
       private final DistanceCalculator calculator = strategy.getSpatialContext().getDistCalc();
-      private final double nullValue = (strategy.getSpatialContext().isGeo() ? 180 : Double.MAX_VALUE);
+      private final double nullValue =
+          (strategy.getSpatialContext().isGeo() ? 180 * multiplier : Double.MAX_VALUE);
 
       @Override
       public float floatVal(int doc) {
@@ -84,7 +87,7 @@ public class DistanceValueSource extends
         // make sure it has minX and area
         if (validX.get(doc)) {
           assert validY.get(doc);
-          return calculator.distance(from, ptX.get(doc), ptY.get(doc));
+          return calculator.distance(from, ptX.get(doc), ptY.get(doc)) * multiplier;
         }
         return nullValue;
       }
@@ -105,6 +108,7 @@ public class DistanceValueSource extends
 
     if (!from.equals(that.from)) return false;
     if (!strategy.equals(that.strategy)) return false;
+    if (multiplier != that.multiplier) return false;
 
     return true;
   }

Modified: lucene/dev/branches/lucene4956/lucene/spatial/src/java/org/apache/lucene/spatial/vector/PointVectorStrategy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/spatial/src/java/org/apache/lucene/spatial/vector/PointVectorStrategy.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/spatial/src/java/org/apache/lucene/spatial/vector/PointVectorStrategy.java (original)
+++ lucene/dev/branches/lucene4956/lucene/spatial/src/java/org/apache/lucene/spatial/vector/PointVectorStrategy.java Sun Aug 11 12:19:13 2013
@@ -120,8 +120,8 @@ public class PointVectorStrategy extends
   }
 
   @Override
-  public ValueSource makeDistanceValueSource(Point queryPoint) {
-    return new DistanceValueSource(this, queryPoint);
+  public ValueSource makeDistanceValueSource(Point queryPoint, double multiplier) {
+    return new DistanceValueSource(this, queryPoint, multiplier);
   }
 
   @Override

Modified: lucene/dev/branches/lucene4956/lucene/spatial/src/test/org/apache/lucene/spatial/DistanceStrategyTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/spatial/src/test/org/apache/lucene/spatial/DistanceStrategyTest.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/spatial/src/test/org/apache/lucene/spatial/DistanceStrategyTest.java (original)
+++ lucene/dev/branches/lucene4956/lucene/spatial/src/test/org/apache/lucene/spatial/DistanceStrategyTest.java Sun Aug 11 12:19:13 2013
@@ -34,6 +34,7 @@ import org.junit.Test;
 
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.List;
 
 public class DistanceStrategyTest extends StrategyTestCase {
@@ -121,6 +122,11 @@ public class DistanceStrategyTest extend
 
   void checkDistValueSource(String ptStr, float... distances) throws IOException {
     Point pt = (Point) ctx.readShape(ptStr);
-    checkValueSource(strategy.makeDistanceValueSource(pt), distances, 1.0e-4f);
+    float multiplier = random().nextFloat() * 100f;
+    float[] dists2 = Arrays.copyOf(distances, distances.length);
+    for (int i = 0; i < dists2.length; i++) {
+      dists2[i] *= multiplier;
+    }
+    checkValueSource(strategy.makeDistanceValueSource(pt, multiplier), dists2, 1.0e-3f);
   }
 }

Modified: lucene/dev/branches/lucene4956/lucene/spatial/src/test/org/apache/lucene/spatial/SpatialExample.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/spatial/src/test/org/apache/lucene/spatial/SpatialExample.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/spatial/src/test/org/apache/lucene/spatial/SpatialExample.java (original)
+++ lucene/dev/branches/lucene4956/lucene/spatial/src/test/org/apache/lucene/spatial/SpatialExample.java Sun Aug 11 12:19:13 2013
@@ -162,7 +162,8 @@ public class SpatialExample extends Luce
     //--Match all, order by distance ascending
     {
       Point pt = ctx.makePoint(60, -50);
-      ValueSource valueSource = strategy.makeDistanceValueSource(pt);//the distance (in degrees)
+      double degToKm = DistanceUtils.degrees2Dist(1, DistanceUtils.EARTH_MEAN_RADIUS_KM);
+      ValueSource valueSource = strategy.makeDistanceValueSource(pt, degToKm);//the distance (in km)
       Sort distSort = new Sort(valueSource.getSortField(false)).rewrite(indexSearcher);//false=asc dist
       TopDocs docs = indexSearcher.search(new MatchAllDocsQuery(), 10, distSort);
       assertDocMatchedIds(indexSearcher, docs, 4, 20, 2);

Modified: lucene/dev/branches/lucene4956/lucene/spatial/src/test/org/apache/lucene/spatial/SpatialTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/spatial/src/test/org/apache/lucene/spatial/SpatialTestCase.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/spatial/src/test/org/apache/lucene/spatial/SpatialTestCase.java (original)
+++ lucene/dev/branches/lucene4956/lucene/spatial/src/test/org/apache/lucene/spatial/SpatialTestCase.java Sun Aug 11 12:19:13 2013
@@ -58,6 +58,8 @@ public abstract class SpatialTestCase ex
 
     directory = newDirectory();
     indexWriter = new RandomIndexWriter(random(),directory);
+    indexReader = indexWriter.getReader();
+    indexSearcher = newSearcher(indexReader);
   }
 
   @Override

Modified: lucene/dev/branches/lucene4956/lucene/spatial/src/test/org/apache/lucene/spatial/prefix/SpatialOpRecursivePrefixTreeTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/spatial/src/test/org/apache/lucene/spatial/prefix/SpatialOpRecursivePrefixTreeTest.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/spatial/src/test/org/apache/lucene/spatial/prefix/SpatialOpRecursivePrefixTreeTest.java (original)
+++ lucene/dev/branches/lucene4956/lucene/spatial/src/test/org/apache/lucene/spatial/prefix/SpatialOpRecursivePrefixTreeTest.java Sun Aug 11 12:19:13 2013
@@ -58,6 +58,8 @@ import static com.spatial4j.core.shape.S
 
 public class SpatialOpRecursivePrefixTreeTest extends StrategyTestCase {
 
+  static final int ITERATIONS = 10;//Test Iterations
+
   private SpatialPrefixTree grid;
 
   @Before
@@ -81,33 +83,45 @@ public class SpatialOpRecursivePrefixTre
   }
 
   @Test
-  @Repeat(iterations = 10)
+  @Repeat(iterations = ITERATIONS)
   public void testIntersects() throws IOException {
     mySetup(-1);
     doTest(SpatialOperation.Intersects);
   }
 
   @Test
-  @Repeat(iterations = 10)
+  @Repeat(iterations = ITERATIONS)
   public void testWithin() throws IOException {
     mySetup(-1);
     doTest(SpatialOperation.IsWithin);
   }
 
   @Test
-  @Repeat(iterations = 10)
+  @Repeat(iterations = ITERATIONS)
   public void testContains() throws IOException {
     mySetup(-1);
     doTest(SpatialOperation.Contains);
   }
 
   @Test
-  @Repeat(iterations = 10)
+  @Repeat(iterations = ITERATIONS)
   public void testDisjoint() throws IOException {
     mySetup(-1);
     doTest(SpatialOperation.IsDisjointTo);
   }
 
+  /** See LUCENE-5062, {@link ContainsPrefixTreeFilter#multiOverlappingIndexedShapes}. */
+  @Test
+  public void testContainsPairOverlap() throws IOException {
+    mySetup(3);
+    adoc("0", new ShapePair(ctx.makeRectangle(0, 33, -128, 128), ctx.makeRectangle(33, 128, -128, 128), true));
+    commit();
+    Query query = strategy.makeQuery(new SpatialArgs(SpatialOperation.Contains,
+        ctx.makeRectangle(0, 128, -16, 128)));
+    SearchResults searchResults = executeQuery(query, 1);
+    assertEquals(1, searchResults.numFound);
+  }
+
   @Test
   public void testWithinDisjointParts() throws IOException {
     mySetup(7);
@@ -172,6 +186,13 @@ public class SpatialOpRecursivePrefixTre
   }
 
   private void doTest(final SpatialOperation operation) throws IOException {
+    //first show that when there's no data, a query will result in no results
+    {
+      Query query = strategy.makeQuery(new SpatialArgs(operation, randomRectangle()));
+      SearchResults searchResults = executeQuery(query, 1);
+      assertEquals(0, searchResults.numFound);
+    }
+
     final boolean biasContains = (operation == SpatialOperation.Contains);
 
     Map<String, Shape> indexedShapes = new LinkedHashMap<String, Shape>();
@@ -182,10 +203,10 @@ public class SpatialOpRecursivePrefixTre
       Shape indexedShape;
       Shape indexedShapeGS; //(grid-snapped)
       int R = random().nextInt(12);
-      if (R == 0) {//1 in 10
+      if (R == 0) {//1 in 12
         indexedShape = null; //no shape for this doc
         indexedShapeGS = null;
-      } else if (R % 4 == 0) {//3 in 12
+      } else if (R % 3 == 0) {//4-1 in 12
         //comprised of more than one shape
         Rectangle shape1 = randomRectangle();
         Rectangle shape2 = randomRectangle();
@@ -334,9 +355,10 @@ public class SpatialOpRecursivePrefixTre
     @Override
     public SpatialRelation relate(Shape other) {
       SpatialRelation r = relateApprox(other);
-      if (r != INTERSECTS)
+      if (r != INTERSECTS && !(r == WITHIN && biasContainsThenWithin))
         return r;
-      //See if the correct answer is actually Contains
+      //See if the correct answer is actually Contains, when the indexed shapes are adjacent,
+      // creating a larger shape that contains the input shape.
       Rectangle oRect = (Rectangle)other;
       boolean pairTouches = shape1.relate(shape2).intersects();
       if (!pairTouches)

Modified: lucene/dev/branches/lucene4956/lucene/suggest/build.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/suggest/build.xml?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/suggest/build.xml (original)
+++ lucene/dev/branches/lucene4956/lucene/suggest/build.xml Sun Aug 11 12:19:13 2013
@@ -25,5 +25,15 @@
 	
   <!-- just a list of words for testing suggesters -->
   <property name="rat.excludes" value="**/Top50KWiki.utf8"/>
+
   <import file="../module-build.xml"/>
+
+  <path id="classpath">
+    <pathelement path="${analyzers-common.jar}"/>
+    <pathelement path="${misc.jar}"/>
+    <path refid="base.classpath"/>
+  </path>
+
+  <target name="compile-core" depends="jar-misc, jar-analyzers-common, common.compile-core" />
+
 </project>

Modified: lucene/dev/branches/lucene4956/lucene/suggest/src/java/org/apache/lucene/search/spell/SpellChecker.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/suggest/src/java/org/apache/lucene/search/spell/SpellChecker.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/suggest/src/java/org/apache/lucene/search/spell/SpellChecker.java (original)
+++ lucene/dev/branches/lucene4956/lucene/suggest/src/java/org/apache/lucene/search/spell/SpellChecker.java Sun Aug 11 12:19:13 2013
@@ -525,7 +525,7 @@ public class SpellChecker implements jav
   
           if (!isEmpty) {
             for (TermsEnum te : termsEnums) {
-              if (te.seekExact(currentTerm, false)) {
+              if (te.seekExact(currentTerm)) {
                 continue terms;
               }
             }

Modified: lucene/dev/branches/lucene4956/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingSuggester.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingSuggester.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingSuggester.java (original)
+++ lucene/dev/branches/lucene4956/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/AnalyzingSuggester.java Sun Aug 11 12:19:13 2013
@@ -21,7 +21,6 @@ import java.io.File;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
-import java.io.StringReader;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.Comparator;
@@ -160,7 +159,7 @@ public class AnalyzingSuggester extends 
 
   /** Represents the separation between tokens, if
    *  PRESERVE_SEP was specified */
-  private static final int SEP_LABEL = 0xff;
+  private static final int SEP_LABEL = '\u001F';
 
   /** Marks end of the analyzed input and start of dedup
    *  byte. */
@@ -307,44 +306,14 @@ public class AnalyzingSuggester extends 
     }
   }
 
-  /** Just escapes the 0xff byte (which we still for SEP). */
-  private static final class  EscapingTokenStreamToAutomaton extends TokenStreamToAutomaton {
-
-    final BytesRef spare = new BytesRef();
-
-    @Override
-    protected BytesRef changeToken(BytesRef in) {
-      int upto = 0;
-      for(int i=0;i<in.length;i++) {
-        byte b = in.bytes[in.offset+i];
-        if (b == (byte) 0xff) {
-          if (spare.bytes.length == upto) {
-            spare.grow(upto+2);
-          }
-          spare.bytes[upto++] = (byte) 0xff;
-          spare.bytes[upto++] = b;
-        } else {
-          if (spare.bytes.length == upto) {
-            spare.grow(upto+1);
-          }
-          spare.bytes[upto++] = b;
-        }
-      }
-      spare.offset = 0;
-      spare.length = upto;
-      return spare;
-    }
+  /** Used by subclass to change the lookup automaton, if
+   *  necessary. */
+  protected Automaton convertAutomaton(Automaton a) {
+    return a;
   }
-
+  
   TokenStreamToAutomaton getTokenStreamToAutomaton() {
-    final TokenStreamToAutomaton tsta;
-    if (preserveSep) {
-      tsta = new EscapingTokenStreamToAutomaton();
-    } else {
-      // When we're not preserving sep, we don't steal 0xff
-      // byte, so we don't need to do any escaping:
-      tsta = new TokenStreamToAutomaton();
-    }
+    final TokenStreamToAutomaton tsta = new TokenStreamToAutomaton();
     tsta.setPreservePositionIncrements(preservePositionIncrements);
     return tsta;
   }
@@ -380,11 +349,14 @@ public class AnalyzingSuggester extends 
       if (cmp != 0) {
         return cmp;
       }
+      readerA.skipBytes(scratchA.length);
+      readerB.skipBytes(scratchB.length);
 
       // Next by cost:
       long aCost = readerA.readInt();
       long bCost = readerB.readInt();
-
+      assert decodeWeight(aCost) >= 0;
+      assert decodeWeight(bCost) >= 0;
       if (aCost < bCost) {
         return -1;
       } else if (aCost > bCost) {
@@ -393,27 +365,20 @@ public class AnalyzingSuggester extends 
 
       // Finally by surface form:
       if (hasPayloads) {
-        readerA.setPosition(readerA.getPosition() + scratchA.length);
         scratchA.length = readerA.readShort();
-        scratchA.offset = readerA.getPosition();
-        readerB.setPosition(readerB.getPosition() + scratchB.length);
         scratchB.length = readerB.readShort();
+        scratchA.offset = readerA.getPosition();
         scratchB.offset = readerB.getPosition();
       } else {
         scratchA.offset = readerA.getPosition();
-        scratchA.length = a.length - scratchA.offset;
         scratchB.offset = readerB.getPosition();
+        scratchA.length = a.length - scratchA.offset;
         scratchB.length = b.length - scratchB.offset;
       }
-
-      cmp = scratchA.compareTo(scratchB);
-      if (cmp != 0) {
-        return cmp;
-      }
-
-      return 0;
+   
+      return scratchA.compareTo(scratchB);
     }
-  };
+  }
 
   @Override
   public void build(TermFreqIterator iterator) throws IOException {
@@ -512,7 +477,7 @@ public class AnalyzingSuggester extends 
 
       reader = new Sort.ByteSequencesReader(tempSorted);
      
-      PairOutputs<Long,BytesRef> outputs = new PairOutputs<Long,BytesRef>(PositiveIntOutputs.getSingleton(true), ByteSequenceOutputs.getSingleton());
+      PairOutputs<Long,BytesRef> outputs = new PairOutputs<Long,BytesRef>(PositiveIntOutputs.getSingleton(), ByteSequenceOutputs.getSingleton());
       Builder<Pair<Long,BytesRef>> builder = new Builder<Pair<Long,BytesRef>>(FST.INPUT_TYPE.BYTE1, outputs);
 
       // Build FST:
@@ -634,7 +599,7 @@ public class AnalyzingSuggester extends 
   public boolean load(InputStream input) throws IOException {
     DataInput dataIn = new InputStreamDataInput(input);
     try {
-      this.fst = new FST<Pair<Long,BytesRef>>(dataIn, new PairOutputs<Long,BytesRef>(PositiveIntOutputs.getSingleton(true), ByteSequenceOutputs.getSingleton()));
+      this.fst = new FST<Pair<Long,BytesRef>>(dataIn, new PairOutputs<Long,BytesRef>(PositiveIntOutputs.getSingleton(), ByteSequenceOutputs.getSingleton()));
       maxAnalyzedPathsForOneInput = dataIn.readVInt();
       hasPayloads = dataIn.readByte() == 1;
     } finally {
@@ -655,9 +620,8 @@ public class AnalyzingSuggester extends 
       }
       assert sepIndex != -1;
       spare.grow(sepIndex);
-      int payloadLen = output2.length - sepIndex - 1;
-      output2.length = sepIndex;
-      UnicodeUtil.UTF8toUTF16(output2, spare);
+      final int payloadLen = output2.length - sepIndex - 1;
+      UnicodeUtil.UTF8toUTF16(output2.bytes, output2.offset, sepIndex, spare);
       BytesRef payload = new BytesRef(payloadLen);
       System.arraycopy(output2.bytes, sepIndex+1, payload.bytes, 0, payloadLen);
       payload.length = payloadLen;
@@ -700,6 +664,14 @@ public class AnalyzingSuggester extends 
     }
 
     //System.out.println("lookup key=" + key + " num=" + num);
+    for (int i = 0; i < key.length(); i++) {
+      if (key.charAt(i) == 0x1E) {
+        throw new IllegalArgumentException("lookup key cannot contain HOLE character U+001E; this character is reserved");
+      }
+      if (key.charAt(i) == 0x1F) {
+        throw new IllegalArgumentException("lookup key cannot contain unit separator character U+001F; this character is reserved");
+      }
+    }
     final BytesRef utf8Key = new BytesRef(key);
     try {
 
@@ -721,7 +693,7 @@ public class AnalyzingSuggester extends 
 
       final List<LookupResult> results = new ArrayList<LookupResult>();
 
-      List<FSTUtil.Path<Pair<Long,BytesRef>>> prefixPaths = FSTUtil.intersectPrefixPaths(lookupAutomaton, fst);
+      List<FSTUtil.Path<Pair<Long,BytesRef>>> prefixPaths = FSTUtil.intersectPrefixPaths(convertAutomaton(lookupAutomaton), fst);
 
       if (exactFirst) {
 
@@ -856,7 +828,7 @@ public class AnalyzingSuggester extends 
   
   final Set<IntsRef> toFiniteStrings(final BytesRef surfaceForm, final TokenStreamToAutomaton ts2a) throws IOException {
  // Analyze surface form:
-    TokenStream ts = indexAnalyzer.tokenStream("", new StringReader(surfaceForm.utf8ToString()));
+    TokenStream ts = indexAnalyzer.tokenStream("", surfaceForm.utf8ToString());
 
     // Create corresponding automaton: labels are bytes
     // from each analyzed token, with byte 0 used as
@@ -865,6 +837,7 @@ public class AnalyzingSuggester extends 
     ts.close();
 
     replaceSep(automaton);
+    automaton = convertAutomaton(automaton);
 
     assert SpecialOperations.isFinite(automaton);
 
@@ -881,7 +854,7 @@ public class AnalyzingSuggester extends 
   final Automaton toLookupAutomaton(final CharSequence key) throws IOException {
     // TODO: is there a Reader from a CharSequence?
     // Turn tokenstream into automaton:
-    TokenStream ts = queryAnalyzer.tokenStream("", new StringReader(key.toString()));
+    TokenStream ts = queryAnalyzer.tokenStream("", key.toString());
     Automaton automaton = (getTokenStreamToAutomaton()).toAutomaton(ts);
     ts.close();
 

Modified: lucene/dev/branches/lucene4956/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/FuzzySuggester.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/FuzzySuggester.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/FuzzySuggester.java (original)
+++ lucene/dev/branches/lucene4956/lucene/suggest/src/java/org/apache/lucene/search/suggest/analyzing/FuzzySuggester.java Sun Aug 11 12:19:13 2013
@@ -15,16 +15,15 @@ package org.apache.lucene.search.suggest
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-import java.io.FileOutputStream;
+
 import java.io.IOException;
-import java.io.OutputStreamWriter;
-import java.io.Writer;
 import java.util.Arrays;
 import java.util.List;
 import java.util.Set;
 
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.TokenStream;
+import org.apache.lucene.analysis.TokenStreamToAutomaton;
 import org.apache.lucene.analysis.tokenattributes.TermToBytesRefAttribute; // javadocs
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.IntsRef;
@@ -33,6 +32,7 @@ import org.apache.lucene.util.automaton.
 import org.apache.lucene.util.automaton.BasicOperations;
 import org.apache.lucene.util.automaton.LevenshteinAutomata;
 import org.apache.lucene.util.automaton.SpecialOperations;
+import org.apache.lucene.util.automaton.UTF32ToUTF8;
 import org.apache.lucene.util.fst.FST;
 import org.apache.lucene.util.fst.PairOutputs.Pair;
 
@@ -54,6 +54,9 @@ import org.apache.lucene.util.fst.PairOu
  * #DEFAULT_NON_FUZZY_PREFIX} byte is not allowed to be
  * edited.  We allow up to 1 (@link
  * #DEFAULT_MAX_EDITS} edit.
+ * If {@link #unicodeAware} parameter in the constructor is set to true, maxEdits,
+ * minFuzzyLength, transpositions and nonFuzzyPrefix are measured in Unicode code 
+ * points (actual letters) instead of bytes. 
  *
  * <p>
  * NOTE: This suggester does not boost suggestions that
@@ -66,12 +69,20 @@ import org.apache.lucene.util.fst.PairOu
  * like synonyms to keep the complexity of the prefix intersection low for good
  * lookup performance. At index time, complex analyzers can safely be used.
  * </p>
+ *
+ * @lucene.experimental
  */
 public final class FuzzySuggester extends AnalyzingSuggester {
   private final int maxEdits;
   private final boolean transpositions;
   private final int nonFuzzyPrefix;
   private final int minFuzzyLength;
+  private final boolean unicodeAware;
+
+  /** Measure maxEdits, minFuzzyLength, transpositions and nonFuzzyPrefix 
+   *  parameters in Unicode code points (actual letters)
+   *  instead of bytes. */
+  public static final boolean DEFAULT_UNICODE_AWARE = false;
 
   /**
    * The default minimum length of the key passed to {@link
@@ -114,7 +125,7 @@ public final class FuzzySuggester extend
    */
   public FuzzySuggester(Analyzer indexAnalyzer, Analyzer queryAnalyzer) {
     this(indexAnalyzer, queryAnalyzer, EXACT_FIRST | PRESERVE_SEP, 256, -1, DEFAULT_MAX_EDITS, DEFAULT_TRANSPOSITIONS,
-         DEFAULT_NON_FUZZY_PREFIX, DEFAULT_MIN_FUZZY_LENGTH);
+         DEFAULT_NON_FUZZY_PREFIX, DEFAULT_MIN_FUZZY_LENGTH, DEFAULT_UNICODE_AWARE);
   }
 
   /**
@@ -138,11 +149,12 @@ public final class FuzzySuggester extend
    *        Levenshtein algorithm.
    * @param nonFuzzyPrefix length of common (non-fuzzy) prefix (see default {@link #DEFAULT_NON_FUZZY_PREFIX}
    * @param minFuzzyLength minimum length of lookup key before any edits are allowed (see default {@link #DEFAULT_MIN_FUZZY_LENGTH})
+   * @param unicodeAware operate Unicode code points instead of bytes.
    */
   public FuzzySuggester(Analyzer indexAnalyzer, Analyzer queryAnalyzer,
                         int options, int maxSurfaceFormsPerAnalyzedForm, int maxGraphExpansions,
                         int maxEdits, boolean transpositions, int nonFuzzyPrefix,
-                        int minFuzzyLength) {
+                        int minFuzzyLength, boolean unicodeAware) {
     super(indexAnalyzer, queryAnalyzer, options, maxSurfaceFormsPerAnalyzedForm, maxGraphExpansions);
     if (maxEdits < 0 || maxEdits > LevenshteinAutomata.MAXIMUM_SUPPORTED_DISTANCE) {
       throw new IllegalArgumentException("maxEdits must be between 0 and " + LevenshteinAutomata.MAXIMUM_SUPPORTED_DISTANCE);
@@ -158,6 +170,7 @@ public final class FuzzySuggester extend
     this.transpositions = transpositions;
     this.nonFuzzyPrefix = nonFuzzyPrefix;
     this.minFuzzyLength = minFuzzyLength;
+    this.unicodeAware = unicodeAware;
   }
   
   @Override
@@ -176,7 +189,7 @@ public final class FuzzySuggester extend
     // "compete") ... in which case I think the wFST needs
     // to be log weights or something ...
 
-    Automaton levA = toLevenshteinAutomata(lookupAutomaton);
+    Automaton levA = convertAutomaton(toLevenshteinAutomata(lookupAutomaton));
     /*
       Writer w = new OutputStreamWriter(new FileOutputStream("out.dot"), "UTF-8");
       w.write(levA.toDot());
@@ -186,6 +199,24 @@ public final class FuzzySuggester extend
     return FSTUtil.intersectPrefixPaths(levA, fst);
   }
 
+  @Override
+  protected Automaton convertAutomaton(Automaton a) {
+    if (unicodeAware) {
+      Automaton utf8automaton = new UTF32ToUTF8().convert(a);
+      BasicOperations.determinize(utf8automaton);
+      return utf8automaton;
+    } else {
+      return a;
+    }
+  }
+
+  @Override
+  TokenStreamToAutomaton getTokenStreamToAutomaton() {
+    final TokenStreamToAutomaton tsta = super.getTokenStreamToAutomaton();
+    tsta.setUnicodeArcs(unicodeAware);
+    return tsta;
+  }
+
   Automaton toLevenshteinAutomata(Automaton automaton) {
     final Set<IntsRef> ref = SpecialOperations.getFiniteStrings(automaton, -1);
     Automaton subs[] = new Automaton[ref.size()];
@@ -203,7 +234,7 @@ public final class FuzzySuggester extend
         // to allow the trailing dedup bytes to be
         // edited... but then 0 byte is "in general" allowed
         // on input (but not in UTF8).
-        LevenshteinAutomata lev = new LevenshteinAutomata(ints, 255, transpositions);
+        LevenshteinAutomata lev = new LevenshteinAutomata(ints, unicodeAware ? Character.MAX_CODE_POINT : 255, transpositions);
         Automaton levAutomaton = lev.toAutomaton(maxEdits);
         Automaton combined = BasicOperations.concatenate(Arrays.asList(prefix, levAutomaton));
         combined.setDeterministic(true); // its like the special case in concatenate itself, except we cloneExpanded already

Modified: lucene/dev/branches/lucene4956/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/WFSTCompletionLookup.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/WFSTCompletionLookup.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/WFSTCompletionLookup.java (original)
+++ lucene/dev/branches/lucene4956/lucene/suggest/src/java/org/apache/lucene/search/suggest/fst/WFSTCompletionLookup.java Sun Aug 11 12:19:13 2013
@@ -101,7 +101,7 @@ public class WFSTCompletionLookup extend
     TermFreqIterator iter = new WFSTTermFreqIteratorWrapper(iterator);
     IntsRef scratchInts = new IntsRef();
     BytesRef previous = null;
-    PositiveIntOutputs outputs = PositiveIntOutputs.getSingleton(true);
+    PositiveIntOutputs outputs = PositiveIntOutputs.getSingleton();
     Builder<Long> builder = new Builder<Long>(FST.INPUT_TYPE.BYTE1, outputs);
     while ((scratch = iter.next()) != null) {
       long cost = iter.weight();
@@ -136,7 +136,7 @@ public class WFSTCompletionLookup extend
   @Override
   public boolean load(InputStream input) throws IOException {
     try {
-      this.fst = new FST<Long>(new InputStreamDataInput(input), PositiveIntOutputs.getSingleton(true));
+      this.fst = new FST<Long>(new InputStreamDataInput(input), PositiveIntOutputs.getSingleton());
     } finally {
       IOUtils.close(input);
     }

Modified: lucene/dev/branches/lucene4956/lucene/suggest/src/test/org/apache/lucene/search/suggest/LookupBenchmarkTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/suggest/src/test/org/apache/lucene/search/suggest/LookupBenchmarkTest.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/suggest/src/test/org/apache/lucene/search/suggest/LookupBenchmarkTest.java (original)
+++ lucene/dev/branches/lucene4956/lucene/suggest/src/test/org/apache/lucene/search/suggest/LookupBenchmarkTest.java Sun Aug 11 12:19:13 2013
@@ -30,18 +30,18 @@ import java.util.Locale;
 import java.util.Random;
 import java.util.concurrent.Callable;
 
-import org.apache.lucene.util.*;
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.analysis.MockTokenizer;
 import org.apache.lucene.search.suggest.Lookup; // javadocs
+import org.apache.lucene.search.suggest.analyzing.AnalyzingInfixSuggester;
 import org.apache.lucene.search.suggest.analyzing.AnalyzingSuggester;
 import org.apache.lucene.search.suggest.analyzing.FuzzySuggester;
 import org.apache.lucene.search.suggest.fst.FSTCompletionLookup;
 import org.apache.lucene.search.suggest.fst.WFSTCompletionLookup;
 import org.apache.lucene.search.suggest.jaspell.JaspellLookup;
 import org.apache.lucene.search.suggest.tst.TSTLookup;
-
+import org.apache.lucene.util.*;
 import org.junit.BeforeClass;
 import org.junit.Ignore;
 
@@ -54,11 +54,11 @@ public class LookupBenchmarkTest extends
   private final List<Class<? extends Lookup>> benchmarkClasses = Arrays.asList(
       FuzzySuggester.class,
       AnalyzingSuggester.class,
+      AnalyzingInfixSuggester.class,
       JaspellLookup.class, 
       TSTLookup.class,
       FSTCompletionLookup.class,
       WFSTCompletionLookup.class
-      
       );
 
   private final static int rounds = 15;
@@ -168,8 +168,13 @@ public class LookupBenchmarkTest extends
     try {
       lookup = cls.newInstance();
     } catch (InstantiationException e) {
-      Constructor<? extends Lookup> ctor = cls.getConstructor(Analyzer.class);
-      lookup = ctor.newInstance(new MockAnalyzer(random, MockTokenizer.KEYWORD, false));
+      Analyzer a = new MockAnalyzer(random, MockTokenizer.KEYWORD, false);
+      if (cls == AnalyzingInfixSuggester.class) {
+        lookup = new AnalyzingInfixSuggester(TEST_VERSION_CURRENT, _TestUtil.getTempDir("LookupBenchmarkTest"), a);
+      } else {
+        Constructor<? extends Lookup> ctor = cls.getConstructor(Analyzer.class);
+        lookup = ctor.newInstance(a);
+      }
     }
     lookup.build(new TermFreqArrayIterator(input));
     return lookup;

Modified: lucene/dev/branches/lucene4956/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/AnalyzingSuggesterTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/AnalyzingSuggesterTest.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/AnalyzingSuggesterTest.java (original)
+++ lucene/dev/branches/lucene4956/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/AnalyzingSuggesterTest.java Sun Aug 11 12:19:13 2013
@@ -24,13 +24,15 @@ import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
 import java.io.Reader;
-import java.io.StringReader;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
 import java.util.Comparator;
+import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
+import java.util.Map;
+import java.util.Random;
 import java.util.Set;
 import java.util.TreeSet;
 
@@ -48,14 +50,14 @@ import org.apache.lucene.analysis.TokenS
 import org.apache.lucene.analysis.Tokenizer;
 import org.apache.lucene.analysis.tokenattributes.CharTermAttribute;
 import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
-import org.apache.lucene.analysis.tokenattributes.PositionLengthAttribute;
-import org.apache.lucene.analysis.tokenattributes.TermToBytesRefAttribute;
+import org.apache.lucene.document.Document;
 import org.apache.lucene.search.suggest.Lookup.LookupResult;
 import org.apache.lucene.search.suggest.TermFreq;
 import org.apache.lucene.search.suggest.TermFreqArrayIterator;
 import org.apache.lucene.search.suggest.TermFreqPayload;
 import org.apache.lucene.search.suggest.TermFreqPayloadArrayIterator;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.LineFileDocs;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util._TestUtil;
 
@@ -63,13 +65,16 @@ public class AnalyzingSuggesterTest exte
   
   /** this is basically the WFST test ported to KeywordAnalyzer. so it acts the same */
   public void testKeyword() throws Exception {
-    TermFreq keys[] = new TermFreq[] {
+    Iterable<TermFreq> keys = shuffle(
         new TermFreq("foo", 50),
         new TermFreq("bar", 10),
+        new TermFreq("barbar", 10),
         new TermFreq("barbar", 12),
-        new TermFreq("barbara", 6)
-    };
-    
+        new TermFreq("barbara", 6),
+        new TermFreq("bar", 5),
+        new TermFreq("barbara", 1)
+    );
+
     AnalyzingSuggester suggester = new AnalyzingSuggester(new MockAnalyzer(random(), MockTokenizer.KEYWORD, false));
     suggester.build(new TermFreqArrayIterator(keys));
     
@@ -106,53 +111,99 @@ public class AnalyzingSuggesterTest exte
   }
   
   public void testKeywordWithPayloads() throws Exception {
-    TermFreqPayload keys[] = new TermFreqPayload[] {
+    Iterable<TermFreqPayload> keys = shuffle(
       new TermFreqPayload("foo", 50, new BytesRef("hello")),
       new TermFreqPayload("bar", 10, new BytesRef("goodbye")),
       new TermFreqPayload("barbar", 12, new BytesRef("thank you")),
-      new TermFreqPayload("barbara", 6, new BytesRef("for all the fish"))
-    };
+      new TermFreqPayload("bar", 9, new BytesRef("should be deduplicated")),
+      new TermFreqPayload("bar", 8, new BytesRef("should also be deduplicated")),
+      new TermFreqPayload("barbara", 6, new BytesRef("for all the fish")));
     
     AnalyzingSuggester suggester = new AnalyzingSuggester(new MockAnalyzer(random(), MockTokenizer.KEYWORD, false));
     suggester.build(new TermFreqPayloadArrayIterator(keys));
+    for (int i = 0; i < 2; i++) {
+      // top N of 2, but only foo is available
+      List<LookupResult> results = suggester.lookup(_TestUtil.stringToCharSequence("f", random()), false, 2);
+      assertEquals(1, results.size());
+      assertEquals("foo", results.get(0).key.toString());
+      assertEquals(50, results.get(0).value, 0.01F);
+      assertEquals(new BytesRef("hello"), results.get(0).payload);
+      
+      // top N of 1 for 'bar': we return this even though
+      // barbar is higher because exactFirst is enabled:
+      results = suggester.lookup(_TestUtil.stringToCharSequence("bar", random()), false, 1);
+      assertEquals(1, results.size());
+      assertEquals("bar", results.get(0).key.toString());
+      assertEquals(10, results.get(0).value, 0.01F);
+      assertEquals(new BytesRef("goodbye"), results.get(0).payload);
+      
+      // top N Of 2 for 'b'
+      results = suggester.lookup(_TestUtil.stringToCharSequence("b", random()), false, 2);
+      assertEquals(2, results.size());
+      assertEquals("barbar", results.get(0).key.toString());
+      assertEquals(12, results.get(0).value, 0.01F);
+      assertEquals(new BytesRef("thank you"), results.get(0).payload);
+      assertEquals("bar", results.get(1).key.toString());
+      assertEquals(10, results.get(1).value, 0.01F);
+      assertEquals(new BytesRef("goodbye"), results.get(1).payload);
+      
+      // top N of 3 for 'ba'
+      results = suggester.lookup(_TestUtil.stringToCharSequence("ba", random()), false, 3);
+      assertEquals(3, results.size());
+      assertEquals("barbar", results.get(0).key.toString());
+      assertEquals(12, results.get(0).value, 0.01F);
+      assertEquals(new BytesRef("thank you"), results.get(0).payload);
+      assertEquals("bar", results.get(1).key.toString());
+      assertEquals(10, results.get(1).value, 0.01F);
+      assertEquals(new BytesRef("goodbye"), results.get(1).payload);
+      assertEquals("barbara", results.get(2).key.toString());
+      assertEquals(6, results.get(2).value, 0.01F);
+      assertEquals(new BytesRef("for all the fish"), results.get(2).payload);
+    }
+  }
+  
+  public void testRandomRealisticKeys() throws IOException {
+    LineFileDocs lineFile = new LineFileDocs(random());
+    Map<String, Long> mapping = new HashMap<>();
+    List<TermFreq> keys = new ArrayList<>();
     
-    // top N of 2, but only foo is available
-    List<LookupResult> results = suggester.lookup(_TestUtil.stringToCharSequence("f", random()), false, 2);
-    assertEquals(1, results.size());
-    assertEquals("foo", results.get(0).key.toString());
-    assertEquals(50, results.get(0).value, 0.01F);
-    assertEquals(new BytesRef("hello"), results.get(0).payload);
+    int howMany = atLeast(100); // this might bring up duplicates
+    for (int i = 0; i < howMany; i++) {
+      Document nextDoc = lineFile.nextDoc();
+      String title = nextDoc.getField("title").stringValue();
+      int randomWeight = random().nextInt(100);
+      keys.add(new TermFreq(title, randomWeight));
+      if (!mapping.containsKey(title) || mapping.get(title) < randomWeight) {
+          mapping.put(title, Long.valueOf(randomWeight));
+      }
+    }
     
-    // top N of 1 for 'bar': we return this even though
-    // barbar is higher because exactFirst is enabled:
-    results = suggester.lookup(_TestUtil.stringToCharSequence("bar", random()), false, 1);
-    assertEquals(1, results.size());
-    assertEquals("bar", results.get(0).key.toString());
-    assertEquals(10, results.get(0).value, 0.01F);
-    assertEquals(new BytesRef("goodbye"), results.get(0).payload);
+    AnalyzingSuggester analyzingSuggester = new AnalyzingSuggester(new MockAnalyzer(random()));
+    analyzingSuggester.setPreservePositionIncrements(random().nextBoolean());
+    boolean doPayloads = random().nextBoolean();
+    if (doPayloads) {
+      List<TermFreqPayload> keysAndPayloads = new ArrayList<>();
+      for (TermFreq termFreq : keys) {
+        keysAndPayloads.add(new TermFreqPayload(termFreq.term, termFreq.v, new BytesRef(Long.toString(termFreq.v))));
+      }
+      analyzingSuggester.build(new TermFreqPayloadArrayIterator(keysAndPayloads));
+    } else {
+      analyzingSuggester.build(new TermFreqArrayIterator(keys));  
+    }
     
-    // top N Of 2 for 'b'
-    results = suggester.lookup(_TestUtil.stringToCharSequence("b", random()), false, 2);
-    assertEquals(2, results.size());
-    assertEquals("barbar", results.get(0).key.toString());
-    assertEquals(12, results.get(0).value, 0.01F);
-    assertEquals(new BytesRef("thank you"), results.get(0).payload);
-    assertEquals("bar", results.get(1).key.toString());
-    assertEquals(10, results.get(1).value, 0.01F);
-    assertEquals(new BytesRef("goodbye"), results.get(1).payload);
+    for (TermFreq termFreq : keys) {
+      List<LookupResult> lookup = analyzingSuggester.lookup(termFreq.term.utf8ToString(), false, keys.size());
+      for (LookupResult lookupResult : lookup) {
+        assertEquals(mapping.get(lookupResult.key), Long.valueOf(lookupResult.value));
+        if (doPayloads) {
+          assertEquals(lookupResult.payload.utf8ToString(), Long.toString(lookupResult.value));
+        } else {
+          assertNull(lookupResult.payload);
+        }
+      }
+    }
     
-    // top N of 3 for 'ba'
-    results = suggester.lookup(_TestUtil.stringToCharSequence("ba", random()), false, 3);
-    assertEquals(3, results.size());
-    assertEquals("barbar", results.get(0).key.toString());
-    assertEquals(12, results.get(0).value, 0.01F);
-    assertEquals(new BytesRef("thank you"), results.get(0).payload);
-    assertEquals("bar", results.get(1).key.toString());
-    assertEquals(10, results.get(1).value, 0.01F);
-    assertEquals(new BytesRef("goodbye"), results.get(1).payload);
-    assertEquals("barbara", results.get(2).key.toString());
-    assertEquals(6, results.get(2).value, 0.01F);
-    assertEquals(new BytesRef("for all the fish"), results.get(2).payload);
+    lineFile.close();
   }
   
   // TODO: more tests
@@ -594,7 +645,7 @@ public class AnalyzingSuggesterTest exte
     }
   }
 
-  private static char SEP = '\uFFFF';
+  private static char SEP = '\u001F';
 
   public void testRandom() throws Exception {
 
@@ -705,9 +756,9 @@ public class AnalyzingSuggesterTest exte
     AnalyzingSuggester suggester = new AnalyzingSuggester(a, a,
                                                           preserveSep ? AnalyzingSuggester.PRESERVE_SEP : 0, 256, -1);
     if (doPayloads) {
-      suggester.build(new TermFreqPayloadArrayIterator(payloadKeys));
+      suggester.build(new TermFreqPayloadArrayIterator(shuffle(payloadKeys)));
     } else {
-      suggester.build(new TermFreqArrayIterator(keys));
+      suggester.build(new TermFreqArrayIterator(shuffle(keys)));
     }
 
     for (String prefix : allPrefixes) {
@@ -822,82 +873,11 @@ public class AnalyzingSuggesterTest exte
     }
   }
 
-  public void testStolenBytes() throws Exception {
-
-    // First time w/ preserveSep, second time without:
-    for(int i=0;i<2;i++) {
-      
-      final Analyzer analyzer = new Analyzer() {
-          @Override
-          protected TokenStreamComponents createComponents(String fieldName, Reader reader) {
-            Tokenizer tokenizer = new MockTokenizer(reader, MockTokenizer.SIMPLE, true);
-        
-            // TokenStream stream = new SynonymFilter(tokenizer, map, true);
-            // return new TokenStreamComponents(tokenizer, new RemoveDuplicatesTokenFilter(stream));
-            return new TokenStreamComponents(tokenizer) {
-              int tokenStreamCounter = 0;
-              final TokenStream[] tokenStreams = new TokenStream[] {
-                new CannedBinaryTokenStream(new BinaryToken[] {
-                    token(new BytesRef(new byte[] {0x61, (byte) 0xff, 0x61})),
-                  }),
-                new CannedTokenStream(new Token[] {
-                    token("a",1,1),          
-                    token("a",1,1)
-                  }),
-                new CannedTokenStream(new Token[] {
-                    token("a",1,1),
-                    token("a",1,1)
-                  }),
-                new CannedBinaryTokenStream(new BinaryToken[] {
-                    token(new BytesRef(new byte[] {0x61, (byte) 0xff, 0x61})),
-                  })
-              };
-
-              @Override
-              public TokenStream getTokenStream() {
-                TokenStream result = tokenStreams[tokenStreamCounter];
-                tokenStreamCounter++;
-                return result;
-              }
-         
-              @Override
-              protected void setReader(final Reader reader) throws IOException {
-              }
-            };
-          }
-        };
-
-      TermFreq keys[] = new TermFreq[] {
-        new TermFreq("a a", 50),
-        new TermFreq("a b", 50),
-      };
-
-      AnalyzingSuggester suggester = new AnalyzingSuggester(analyzer, analyzer, AnalyzingSuggester.EXACT_FIRST | (i==0 ? AnalyzingSuggester.PRESERVE_SEP : 0), 256, -1);
-      suggester.build(new TermFreqArrayIterator(keys));
-      List<LookupResult> results = suggester.lookup("a a", false, 5);
-      assertEquals(1, results.size());
-      assertEquals("a b", results.get(0).key);
-      assertEquals(50, results.get(0).value);
-
-      results = suggester.lookup("a a", false, 5);
-      assertEquals(1, results.size());
-      assertEquals("a a", results.get(0).key);
-      assertEquals(50, results.get(0).value);
-    }
-  }
-
   public void testMaxSurfaceFormsPerAnalyzedForm() throws Exception {
     Analyzer a = new MockAnalyzer(random());
     AnalyzingSuggester suggester = new AnalyzingSuggester(a, a, 0, 2, -1);
-
-    List<TermFreq> keys = Arrays.asList(new TermFreq[] {
-        new TermFreq("a", 40),
-        new TermFreq("a ", 50),
-        new TermFreq(" a", 60),
-      });
-
-    Collections.shuffle(keys, random());
-    suggester.build(new TermFreqArrayIterator(keys));
+    suggester.build(new TermFreqArrayIterator(shuffle(new TermFreq("a", 40),
+        new TermFreq("a ", 50), new TermFreq(" a", 60))));
 
     List<LookupResult> results = suggester.lookup("a", false, 5);
     assertEquals(2, results.size());
@@ -992,10 +972,9 @@ public class AnalyzingSuggesterTest exte
 
     AnalyzingSuggester suggester = new AnalyzingSuggester(a, a, 0, 256, -1);
 
-    suggester.build(new TermFreqArrayIterator(new TermFreq[] {
+    suggester.build(new TermFreqArrayIterator(shuffle(
           new TermFreq("hambone", 6),
-          new TermFreq("nellie", 5),
-        }));
+          new TermFreq("nellie", 5))));
 
     List<LookupResult> results = suggester.lookup("nellie", false, 2);
     assertEquals(2, results.size());
@@ -1164,4 +1143,63 @@ public class AnalyzingSuggesterTest exte
     assertEquals("[isla de muerta/8, i love lucy/7]", suggester.lookup("i", false, 3).toString());
     assertEquals("[i love lucy/7]", suggester.lookup("i ", false, 3).toString());
   }
+
+  public void testTooManyExpansions() throws Exception {
+
+    final Analyzer a = new Analyzer() {
+        @Override
+        protected TokenStreamComponents createComponents(String fieldName, Reader reader) {
+          Tokenizer tokenizer = new MockTokenizer(reader, MockTokenizer.SIMPLE, true);
+        
+          return new TokenStreamComponents(tokenizer) {
+            @Override
+            public TokenStream getTokenStream() {
+              Token a = new Token("a", 0, 1);
+              a.setPositionIncrement(1);
+              Token b = new Token("b", 0, 1);
+              b.setPositionIncrement(0);
+              return new CannedTokenStream(new Token[] {a, b});
+            }
+         
+            @Override
+            protected void setReader(final Reader reader) throws IOException {
+            }
+          };
+        }
+      };
+
+    AnalyzingSuggester suggester = new AnalyzingSuggester(a, a, 0, 256, 1);
+    suggester.build(new TermFreqArrayIterator(new TermFreq[] {new TermFreq("a", 1)}));
+    assertEquals("[a/1]", suggester.lookup("a", false, 1).toString());
+  }
+  
+  public void testIllegalLookupArgument() throws Exception {
+    Analyzer a = new MockAnalyzer(random());
+    AnalyzingSuggester suggester = new AnalyzingSuggester(a, a, 0, 256, -1);
+    suggester.build(new TermFreqArrayIterator(new TermFreq[] {
+        new TermFreq("а где Люси?", 7),
+    }));
+    try {
+      suggester.lookup("а\u001E", false, 3);
+      fail("should throw IllegalArgumentException");
+    } catch (IllegalArgumentException e) {
+      // expected
+    }
+    try {
+      suggester.lookup("а\u001F", false, 3);
+      fail("should throw IllegalArgumentException");
+    } catch (IllegalArgumentException e) {
+      // expected
+    }
+  }
+
+  @SafeVarargs
+  public final <T> Iterable<T> shuffle(T...values) {
+    final List<T> asList = new ArrayList<T>(values.length);
+    for (T value : values) {
+      asList.add(value);
+    }
+    Collections.shuffle(asList, random());
+    return asList;
+  }
 }

Modified: lucene/dev/branches/lucene4956/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/FuzzySuggesterTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/FuzzySuggesterTest.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/FuzzySuggesterTest.java (original)
+++ lucene/dev/branches/lucene4956/lucene/suggest/src/test/org/apache/lucene/search/suggest/analyzing/FuzzySuggesterTest.java Sun Aug 11 12:19:13 2013
@@ -60,7 +60,9 @@ public class FuzzySuggesterTest extends 
       keys.add(new TermFreq("boo" + _TestUtil.randomSimpleString(random()), 1 + random().nextInt(100)));
     }
     keys.add(new TermFreq("foo bar boo far", 12));
-    FuzzySuggester suggester = new FuzzySuggester(new MockAnalyzer(random(), MockTokenizer.KEYWORD, false));
+    MockAnalyzer analyzer = new MockAnalyzer(random(), MockTokenizer.KEYWORD, false);
+    FuzzySuggester suggester = new FuzzySuggester(analyzer, analyzer, FuzzySuggester.EXACT_FIRST | FuzzySuggester.PRESERVE_SEP, 256, -1, FuzzySuggester.DEFAULT_MAX_EDITS, FuzzySuggester.DEFAULT_TRANSPOSITIONS,
+                                                  0, FuzzySuggester.DEFAULT_MIN_FUZZY_LENGTH, FuzzySuggester.DEFAULT_UNICODE_AWARE);
     suggester.build(new TermFreqArrayIterator(keys));
     int numIters = atLeast(10);
     for (int i = 0; i < numIters; i++) {
@@ -72,6 +74,27 @@ public class FuzzySuggesterTest extends 
     }
   }
   
+  public void testNonLatinRandomEdits() throws IOException {
+    List<TermFreq> keys = new ArrayList<TermFreq>();
+    int numTerms = atLeast(100);
+    for (int i = 0; i < numTerms; i++) {
+      keys.add(new TermFreq("буу" + _TestUtil.randomSimpleString(random()), 1 + random().nextInt(100)));
+    }
+    keys.add(new TermFreq("фуу бар буу фар", 12));
+    MockAnalyzer analyzer = new MockAnalyzer(random(), MockTokenizer.KEYWORD, false);
+    FuzzySuggester suggester = new FuzzySuggester(analyzer, analyzer, FuzzySuggester.EXACT_FIRST | FuzzySuggester.PRESERVE_SEP, 256, -1, FuzzySuggester.DEFAULT_MAX_EDITS, FuzzySuggester.DEFAULT_TRANSPOSITIONS,
+        0, FuzzySuggester.DEFAULT_MIN_FUZZY_LENGTH, true);
+    suggester.build(new TermFreqArrayIterator(keys));
+    int numIters = atLeast(10);
+    for (int i = 0; i < numIters; i++) {
+      String addRandomEdit = addRandomEdit("фуу бар буу", 0);
+      List<LookupResult> results = suggester.lookup(_TestUtil.stringToCharSequence(addRandomEdit, random()), false, 2);
+      assertEquals(addRandomEdit, 1, results.size());
+      assertEquals("фуу бар буу фар", results.get(0).key.toString());
+      assertEquals(12, results.get(0).value, 0.01F);
+    }
+  }
+
   /** this is basically the WFST test ported to KeywordAnalyzer. so it acts the same */
   public void testKeyword() throws Exception {
     TermFreq keys[] = new TermFreq[] {
@@ -185,7 +208,7 @@ public class FuzzySuggesterTest extends 
     int options = 0;
 
     Analyzer a = new MockAnalyzer(random());
-    FuzzySuggester suggester = new FuzzySuggester(a, a, options, 256, -1, 1, true, 1, 3);
+    FuzzySuggester suggester = new FuzzySuggester(a, a, options, 256, -1, 1, true, 1, 3, false);
     suggester.build(new TermFreqArrayIterator(keys));
     // TODO: would be nice if "ab " would allow the test to
     // pass, and more generally if the analyzer can know
@@ -394,7 +417,7 @@ public class FuzzySuggesterTest extends 
   public void testExactFirst() throws Exception {
 
     Analyzer a = getUnusualAnalyzer();
-    FuzzySuggester suggester = new FuzzySuggester(a, a, AnalyzingSuggester.EXACT_FIRST | AnalyzingSuggester.PRESERVE_SEP, 256, -1, 1, true, 1, 3);
+    FuzzySuggester suggester = new FuzzySuggester(a, a, AnalyzingSuggester.EXACT_FIRST | AnalyzingSuggester.PRESERVE_SEP, 256, -1, 1, true, 1, 3, false);
     suggester.build(new TermFreqArrayIterator(new TermFreq[] {
           new TermFreq("x y", 1),
           new TermFreq("x y z", 3),
@@ -433,7 +456,7 @@ public class FuzzySuggesterTest extends 
   public void testNonExactFirst() throws Exception {
 
     Analyzer a = getUnusualAnalyzer();
-    FuzzySuggester suggester = new FuzzySuggester(a, a, AnalyzingSuggester.PRESERVE_SEP, 256, -1, 1, true, 1, 3);
+    FuzzySuggester suggester = new FuzzySuggester(a, a, AnalyzingSuggester.PRESERVE_SEP, 256, -1, 1, true, 1, 3, false);
 
     suggester.build(new TermFreqArrayIterator(new TermFreq[] {
           new TermFreq("x y", 1),
@@ -580,12 +603,13 @@ public class FuzzySuggesterTest extends 
     TermFreq[] keys = new TermFreq[numQueries];
 
     boolean preserveSep = random().nextBoolean();
+    boolean unicodeAware = random().nextBoolean();
 
     final int numStopChars = random().nextInt(10);
     final boolean preserveHoles = random().nextBoolean();
 
     if (VERBOSE) {
-      System.out.println("TEST: " + numQueries + " words; preserveSep=" + preserveSep + " numStopChars=" + numStopChars + " preserveHoles=" + preserveHoles);
+      System.out.println("TEST: " + numQueries + " words; preserveSep=" + preserveSep + " ; unicodeAware=" + unicodeAware + " numStopChars=" + numStopChars + " preserveHoles=" + preserveHoles);
     }
     
     for (int i = 0; i < numQueries; i++) {
@@ -606,7 +630,7 @@ public class FuzzySuggesterTest extends 
               if (token > 0) {
                 key += " ";
               }
-              if (preserveSep && analyzedKey.length() > 0 && analyzedKey.charAt(analyzedKey.length()-1) != ' ') {
+              if (preserveSep && analyzedKey.length() > 0 && (unicodeAware ? analyzedKey.codePointAt(analyzedKey.codePointCount(0, analyzedKey.length())-1) != ' ' : analyzedKey.charAt(analyzedKey.length()-1) != ' ')) {
                 analyzedKey += " ";
               }
               key += s;
@@ -659,7 +683,7 @@ public class FuzzySuggesterTest extends 
 
     Analyzer a = new MockTokenEatingAnalyzer(numStopChars, preserveHoles);
     FuzzySuggester suggester = new FuzzySuggester(a, a,
-                                                  preserveSep ? AnalyzingSuggester.PRESERVE_SEP : 0, 256, -1, 1, false, 1, 3);
+                                                  preserveSep ? AnalyzingSuggester.PRESERVE_SEP : 0, 256, -1, 1, false, 1, 3, unicodeAware);
     suggester.build(new TermFreqArrayIterator(keys));
 
     for (String prefix : allPrefixes) {
@@ -728,7 +752,7 @@ public class FuzzySuggesterTest extends 
       // us the "answer key" (ie maybe we have a bug in
       // suggester.toLevA ...) ... but testRandom2() fixes
       // this:
-      Automaton automaton = suggester.toLevenshteinAutomata(suggester.toLookupAutomaton(analyzedKey));
+      Automaton automaton = suggester.convertAutomaton(suggester.toLevenshteinAutomata(suggester.toLookupAutomaton(analyzedKey)));
       assertTrue(automaton.isDeterministic());
       // TODO: could be faster... but its slowCompletor for a reason
       BytesRef spare = new BytesRef();
@@ -799,7 +823,7 @@ public class FuzzySuggesterTest extends 
 
   public void testMaxSurfaceFormsPerAnalyzedForm() throws Exception {
     Analyzer a = new MockAnalyzer(random());
-    FuzzySuggester suggester = new FuzzySuggester(a, a, 0, 2, -1, 1, true, 1, 3);
+    FuzzySuggester suggester = new FuzzySuggester(a, a, 0, 2, -1, 1, true, 1, 3, false);
 
     List<TermFreq> keys = Arrays.asList(new TermFreq[] {
         new TermFreq("a", 40),
@@ -820,7 +844,7 @@ public class FuzzySuggesterTest extends 
 
   public void testEditSeps() throws Exception {
     Analyzer a = new MockAnalyzer(random());
-    FuzzySuggester suggester = new FuzzySuggester(a, a, FuzzySuggester.PRESERVE_SEP, 2, -1, 2, true, 1, 3);
+    FuzzySuggester suggester = new FuzzySuggester(a, a, FuzzySuggester.PRESERVE_SEP, 2, -1, 2, true, 1, 3, false);
 
     List<TermFreq> keys = Arrays.asList(new TermFreq[] {
         new TermFreq("foo bar", 40),
@@ -878,7 +902,8 @@ public class FuzzySuggesterTest extends 
             // NOTE: can only use ascii here so that, in
             // UTF8 byte space it's still a single
             // insertion:
-            int x = random().nextInt(128);
+            // bytes 0x1e and 0x1f are reserved
+            int x = random().nextBoolean() ? random().nextInt(30) :  32 + random().nextInt(128 - 32);
             builder.append((char) x);
             for (int j = i; j < input.length; j++) {
               builder.append(input[j]);  
@@ -933,7 +958,7 @@ public class FuzzySuggesterTest extends 
     boolean transpositions = random().nextBoolean();
     // TODO: test graph analyzers
     // TODO: test exactFirst / preserveSep permutations
-    FuzzySuggester suggest = new FuzzySuggester(a, a, 0, 256, -1, maxEdits, transpositions, prefixLen, prefixLen);
+    FuzzySuggester suggest = new FuzzySuggester(a, a, 0, 256, -1, maxEdits, transpositions, prefixLen, prefixLen, false);
 
     if (VERBOSE) {
       System.out.println("TEST: maxEdits=" + maxEdits + " prefixLen=" + prefixLen + " transpositions=" + transpositions + " num=" + NUM);

Modified: lucene/dev/branches/lucene4956/lucene/test-framework/src/java/org/apache/lucene/analysis/BaseTokenStreamTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/test-framework/src/java/org/apache/lucene/analysis/BaseTokenStreamTestCase.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/test-framework/src/java/org/apache/lucene/analysis/BaseTokenStreamTestCase.java (original)
+++ lucene/dev/branches/lucene4956/lucene/test-framework/src/java/org/apache/lucene/analysis/BaseTokenStreamTestCase.java Sun Aug 11 12:19:13 2013
@@ -295,15 +295,15 @@ public abstract class BaseTokenStreamTes
   }
   
   public static void assertAnalyzesTo(Analyzer a, String input, String[] output, int startOffsets[], int endOffsets[], String types[], int posIncrements[]) throws IOException {
-    assertTokenStreamContents(a.tokenStream("dummy", new StringReader(input)), output, startOffsets, endOffsets, types, posIncrements, null, input.length());
+    assertTokenStreamContents(a.tokenStream("dummy", input), output, startOffsets, endOffsets, types, posIncrements, null, input.length());
   }
   
   public static void assertAnalyzesTo(Analyzer a, String input, String[] output, int startOffsets[], int endOffsets[], String types[], int posIncrements[], int posLengths[]) throws IOException {
-    assertTokenStreamContents(a.tokenStream("dummy", new StringReader(input)), output, startOffsets, endOffsets, types, posIncrements, posLengths, input.length());
+    assertTokenStreamContents(a.tokenStream("dummy", input), output, startOffsets, endOffsets, types, posIncrements, posLengths, input.length());
   }
 
   public static void assertAnalyzesTo(Analyzer a, String input, String[] output, int startOffsets[], int endOffsets[], String types[], int posIncrements[], int posLengths[], boolean offsetsAreCorrect) throws IOException {
-    assertTokenStreamContents(a.tokenStream("dummy", new StringReader(input)), output, startOffsets, endOffsets, types, posIncrements, posLengths, input.length(), offsetsAreCorrect);
+    assertTokenStreamContents(a.tokenStream("dummy", input), output, startOffsets, endOffsets, types, posIncrements, posLengths, input.length(), offsetsAreCorrect);
   }
   
   public static void assertAnalyzesTo(Analyzer a, String input, String[] output) throws IOException {
@@ -332,7 +332,7 @@ public abstract class BaseTokenStreamTes
   
 
   public static void assertAnalyzesToReuse(Analyzer a, String input, String[] output, int startOffsets[], int endOffsets[], String types[], int posIncrements[]) throws IOException {
-    assertTokenStreamContents(a.tokenStream("dummy", new StringReader(input)), output, startOffsets, endOffsets, types, posIncrements, null, input.length());
+    assertTokenStreamContents(a.tokenStream("dummy", input), output, startOffsets, endOffsets, types, posIncrements, null, input.length());
   }
   
   public static void assertAnalyzesToReuse(Analyzer a, String input, String[] output) throws IOException {
@@ -891,7 +891,7 @@ public abstract class BaseTokenStreamTes
 
   protected String toDot(Analyzer a, String inputText) throws IOException {
     final StringWriter sw = new StringWriter();
-    final TokenStream ts = a.tokenStream("field", new StringReader(inputText));
+    final TokenStream ts = a.tokenStream("field", inputText);
     ts.reset();
     new TokenStreamToDot(inputText, ts, new PrintWriter(sw)).toDot();
     return sw.toString();
@@ -899,7 +899,7 @@ public abstract class BaseTokenStreamTes
 
   protected void toDotFile(Analyzer a, String inputText, String localFileName) throws IOException {
     Writer w = new OutputStreamWriter(new FileOutputStream(localFileName), "UTF-8");
-    final TokenStream ts = a.tokenStream("field", new StringReader(inputText));
+    final TokenStream ts = a.tokenStream("field", inputText);
     ts.reset();
     new TokenStreamToDot(inputText, ts, new PrintWriter(w)).toDot();
     w.close();

Modified: lucene/dev/branches/lucene4956/lucene/test-framework/src/java/org/apache/lucene/analysis/CollationTestBase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/test-framework/src/java/org/apache/lucene/analysis/CollationTestBase.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/test-framework/src/java/org/apache/lucene/analysis/CollationTestBase.java (original)
+++ lucene/dev/branches/lucene4956/lucene/test-framework/src/java/org/apache/lucene/analysis/CollationTestBase.java Sun Aug 11 12:19:13 2013
@@ -19,7 +19,6 @@ package org.apache.lucene.analysis;
 
 
 import java.io.IOException;
-import java.io.StringReader;
 import java.util.HashMap;
 import java.util.Map;
 
@@ -33,7 +32,6 @@ import org.apache.lucene.index.Directory
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.IndexWriter;
 import org.apache.lucene.index.IndexWriterConfig;
-import org.apache.lucene.index.IndexableField;
 import org.apache.lucene.index.StorableField;
 import org.apache.lucene.index.StoredDocument;
 import org.apache.lucene.index.Term;
@@ -199,13 +197,13 @@ public abstract class CollationTestBase 
       doc.add(new Field("tracer", sortData[i][0], customType));
       doc.add(new TextField("contents", sortData[i][1], Field.Store.NO));
       if (sortData[i][2] != null) 
-        doc.add(new TextField("US", usAnalyzer.tokenStream("US", new StringReader(sortData[i][2]))));
+        doc.add(new TextField("US", usAnalyzer.tokenStream("US", sortData[i][2])));
       if (sortData[i][3] != null) 
-        doc.add(new TextField("France", franceAnalyzer.tokenStream("France", new StringReader(sortData[i][3]))));
+        doc.add(new TextField("France", franceAnalyzer.tokenStream("France", sortData[i][3])));
       if (sortData[i][4] != null)
-        doc.add(new TextField("Sweden", swedenAnalyzer.tokenStream("Sweden", new StringReader(sortData[i][4]))));
+        doc.add(new TextField("Sweden", swedenAnalyzer.tokenStream("Sweden", sortData[i][4])));
       if (sortData[i][5] != null) 
-        doc.add(new TextField("Denmark", denmarkAnalyzer.tokenStream("Denmark", new StringReader(sortData[i][5]))));
+        doc.add(new TextField("Denmark", denmarkAnalyzer.tokenStream("Denmark", sortData[i][5])));
       writer.addDocument(doc);
     }
     writer.forceMerge(1);
@@ -260,7 +258,7 @@ public abstract class CollationTestBase 
 
     for (int i = 0; i < numTestPoints; i++) {
       String term = _TestUtil.randomSimpleString(random());
-      TokenStream ts = analyzer.tokenStream("fake", new StringReader(term));
+      TokenStream ts = analyzer.tokenStream("fake", term);
       TermToBytesRefAttribute termAtt = ts.addAttribute(TermToBytesRefAttribute.class);
       BytesRef bytes = termAtt.getBytesRef();
       ts.reset();
@@ -279,7 +277,7 @@ public abstract class CollationTestBase 
             for (Map.Entry<String,BytesRef> mapping : map.entrySet()) {
               String term = mapping.getKey();
               BytesRef expected = mapping.getValue();
-              TokenStream ts = analyzer.tokenStream("fake", new StringReader(term));
+              TokenStream ts = analyzer.tokenStream("fake", term);
               TermToBytesRefAttribute termAtt = ts.addAttribute(TermToBytesRefAttribute.class);
               BytesRef bytes = termAtt.getBytesRef();
               ts.reset();

Modified: lucene/dev/branches/lucene4956/lucene/test-framework/src/java/org/apache/lucene/codecs/cheapbastard/CheapBastardDocValuesFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/test-framework/src/java/org/apache/lucene/codecs/cheapbastard/CheapBastardDocValuesFormat.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/test-framework/src/java/org/apache/lucene/codecs/cheapbastard/CheapBastardDocValuesFormat.java (original)
+++ lucene/dev/branches/lucene4956/lucene/test-framework/src/java/org/apache/lucene/codecs/cheapbastard/CheapBastardDocValuesFormat.java Sun Aug 11 12:19:13 2013
@@ -24,8 +24,10 @@ import org.apache.lucene.codecs.DocValue
 import org.apache.lucene.codecs.DocValuesFormat;
 import org.apache.lucene.codecs.diskdv.DiskDocValuesConsumer;
 import org.apache.lucene.codecs.diskdv.DiskDocValuesFormat;
+import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.SegmentReadState;
 import org.apache.lucene.index.SegmentWriteState;
+import org.apache.lucene.util.BytesRef;
 
 /**
  * DocValues format that keeps everything on disk.
@@ -53,7 +55,13 @@ public final class CheapBastardDocValues
     return new DiskDocValuesConsumer(state, DiskDocValuesFormat.DATA_CODEC, 
                                             DiskDocValuesFormat.DATA_EXTENSION, 
                                             DiskDocValuesFormat.META_CODEC, 
-                                            DiskDocValuesFormat.META_EXTENSION);
+                                            DiskDocValuesFormat.META_EXTENSION) {
+      // don't ever write an index, we dont want to use RAM :)
+      @Override
+      protected void addTermsDict(FieldInfo field, Iterable<BytesRef> values) throws IOException {
+        addBinaryField(field, values);
+      }  
+    };
   }
 
   @Override

Modified: lucene/dev/branches/lucene4956/lucene/test-framework/src/java/org/apache/lucene/codecs/cheapbastard/CheapBastardDocValuesProducer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/test-framework/src/java/org/apache/lucene/codecs/cheapbastard/CheapBastardDocValuesProducer.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/test-framework/src/java/org/apache/lucene/codecs/cheapbastard/CheapBastardDocValuesProducer.java (original)
+++ lucene/dev/branches/lucene4956/lucene/test-framework/src/java/org/apache/lucene/codecs/cheapbastard/CheapBastardDocValuesProducer.java Sun Aug 11 12:19:13 2013
@@ -27,6 +27,7 @@ import java.util.Map;
 
 import org.apache.lucene.codecs.CodecUtil;
 import org.apache.lucene.codecs.DocValuesProducer;
+import org.apache.lucene.codecs.diskdv.DiskDocValuesConsumer;
 import org.apache.lucene.codecs.diskdv.DiskDocValuesFormat;
 import org.apache.lucene.index.BinaryDocValues;
 import org.apache.lucene.index.CorruptIndexException;
@@ -58,7 +59,7 @@ class CheapBastardDocValuesProducer exte
     final int version;
     try {
       version = CodecUtil.checkHeader(in, metaCodec, 
-                                      DiskDocValuesFormat.VERSION_START,
+                                      DiskDocValuesFormat.VERSION_CURRENT,
                                       DiskDocValuesFormat.VERSION_CURRENT);
       numerics = new HashMap<Integer,NumericEntry>();
       ords = new HashMap<Integer,NumericEntry>();
@@ -80,7 +81,7 @@ class CheapBastardDocValuesProducer exte
       String dataName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, dataExtension);
       data = state.directory.openInput(dataName, state.context);
       final int version2 = CodecUtil.checkHeader(data, dataCodec, 
-                                                 DiskDocValuesFormat.VERSION_START,
+                                                 DiskDocValuesFormat.VERSION_CURRENT,
                                                  DiskDocValuesFormat.VERSION_CURRENT);
       if (version != version2) {
         throw new CorruptIndexException("Versions mismatch");
@@ -193,6 +194,10 @@ class CheapBastardDocValuesProducer exte
   
   static BinaryEntry readBinaryEntry(IndexInput meta) throws IOException {
     BinaryEntry entry = new BinaryEntry();
+    int format = meta.readVInt();
+    if (format != DiskDocValuesConsumer.BINARY_FIXED_UNCOMPRESSED && format != DiskDocValuesConsumer.BINARY_VARIABLE_UNCOMPRESSED) {
+      throw new CorruptIndexException("Unexpected format for binary entry: " + format + ", input=" + meta);
+    }
     entry.minLength = meta.readVInt();
     entry.maxLength = meta.readVInt();
     entry.count = meta.readVLong();

Modified: lucene/dev/branches/lucene4956/lucene/test-framework/src/java/org/apache/lucene/codecs/compressing/FastCompressingCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/test-framework/src/java/org/apache/lucene/codecs/compressing/FastCompressingCodec.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/test-framework/src/java/org/apache/lucene/codecs/compressing/FastCompressingCodec.java (original)
+++ lucene/dev/branches/lucene4956/lucene/test-framework/src/java/org/apache/lucene/codecs/compressing/FastCompressingCodec.java Sun Aug 11 12:19:13 2013
@@ -1,5 +1,11 @@
 package org.apache.lucene.codecs.compressing;
 
+import org.apache.lucene.codecs.DocValuesFormat;
+import org.apache.lucene.codecs.NormsFormat;
+import org.apache.lucene.codecs.lucene42.Lucene42DocValuesFormat;
+import org.apache.lucene.codecs.lucene42.Lucene42NormsFormat;
+import org.apache.lucene.util.packed.PackedInts;
+
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
  * contributor license agreements.  See the NOTICE file distributed with
@@ -32,4 +38,13 @@ public class FastCompressingCodec extend
     this(1 << 14, false);
   }
 
+  @Override
+  public NormsFormat normsFormat() {
+    return new Lucene42NormsFormat(PackedInts.FAST);
+  }
+
+  @Override
+  public DocValuesFormat docValuesFormat() {
+    return new Lucene42DocValuesFormat(PackedInts.FAST);
+  }
 }

Modified: lucene/dev/branches/lucene4956/lucene/test-framework/src/java/org/apache/lucene/codecs/compressing/FastDecompressionCompressingCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/test-framework/src/java/org/apache/lucene/codecs/compressing/FastDecompressionCompressingCodec.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/test-framework/src/java/org/apache/lucene/codecs/compressing/FastDecompressionCompressingCodec.java (original)
+++ lucene/dev/branches/lucene4956/lucene/test-framework/src/java/org/apache/lucene/codecs/compressing/FastDecompressionCompressingCodec.java Sun Aug 11 12:19:13 2013
@@ -1,5 +1,11 @@
 package org.apache.lucene.codecs.compressing;
 
+import org.apache.lucene.codecs.DocValuesFormat;
+import org.apache.lucene.codecs.NormsFormat;
+import org.apache.lucene.codecs.lucene42.Lucene42DocValuesFormat;
+import org.apache.lucene.codecs.lucene42.Lucene42NormsFormat;
+import org.apache.lucene.util.packed.PackedInts;
+
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
  * contributor license agreements.  See the NOTICE file distributed with
@@ -32,4 +38,13 @@ public class FastDecompressionCompressin
     this(1 << 14, false);
   }
 
+  @Override
+  public NormsFormat normsFormat() {
+    return new Lucene42NormsFormat(PackedInts.DEFAULT);
+  }
+
+  @Override
+  public DocValuesFormat docValuesFormat() {
+    return new Lucene42DocValuesFormat(PackedInts.DEFAULT);
+  }
 }

Modified: lucene/dev/branches/lucene4956/lucene/test-framework/src/java/org/apache/lucene/codecs/compressing/HighCompressionCompressingCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene4956/lucene/test-framework/src/java/org/apache/lucene/codecs/compressing/HighCompressionCompressingCodec.java?rev=1512909&r1=1512908&r2=1512909&view=diff
==============================================================================
--- lucene/dev/branches/lucene4956/lucene/test-framework/src/java/org/apache/lucene/codecs/compressing/HighCompressionCompressingCodec.java (original)
+++ lucene/dev/branches/lucene4956/lucene/test-framework/src/java/org/apache/lucene/codecs/compressing/HighCompressionCompressingCodec.java Sun Aug 11 12:19:13 2013
@@ -1,5 +1,9 @@
 package org.apache.lucene.codecs.compressing;
 
+import org.apache.lucene.codecs.NormsFormat;
+import org.apache.lucene.codecs.lucene42.Lucene42NormsFormat;
+import org.apache.lucene.util.packed.PackedInts;
+
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
  * contributor license agreements.  See the NOTICE file distributed with
@@ -32,4 +36,8 @@ public class HighCompressionCompressingC
     this(1 << 14, false);
   }
 
+  @Override
+  public NormsFormat normsFormat() {
+    return new Lucene42NormsFormat(PackedInts.COMPACT);
+  }
 }