You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ja...@apache.org on 2013/07/04 01:26:45 UTC

svn commit: r1499601 [6/20] - in /lucene/dev/branches/security: ./ dev-tools/ dev-tools/eclipse/ dev-tools/idea/.idea/libraries/ dev-tools/idea/solr/core/src/test/ dev-tools/maven/ dev-tools/maven/lucene/ dev-tools/maven/lucene/analysis/stempel/ dev-to...

Modified: lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/payloads/PayloadTermQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/payloads/PayloadTermQuery.java?rev=1499601&r1=1499600&r2=1499601&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/payloads/PayloadTermQuery.java (original)
+++ lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/payloads/PayloadTermQuery.java Wed Jul  3 23:26:32 2013
@@ -27,7 +27,7 @@ import org.apache.lucene.search.Explanat
 import org.apache.lucene.search.ComplexExplanation;
 import org.apache.lucene.search.similarities.DefaultSimilarity;
 import org.apache.lucene.search.similarities.Similarity;
-import org.apache.lucene.search.similarities.Similarity.SloppySimScorer;
+import org.apache.lucene.search.similarities.Similarity.SimScorer;
 import org.apache.lucene.search.spans.SpanQuery;
 import org.apache.lucene.search.spans.TermSpans;
 import org.apache.lucene.search.spans.SpanTermQuery;
@@ -49,7 +49,7 @@ import java.io.IOException;
  * which returns 1 by default.
  * <p/>
  * Payload scores are aggregated using a pluggable {@link PayloadFunction}.
- * @see org.apache.lucene.search.similarities.Similarity.SloppySimScorer#computePayloadFactor(int, int, int, BytesRef)
+ * @see org.apache.lucene.search.similarities.Similarity.SimScorer#computePayloadFactor(int, int, int, BytesRef)
  **/
 public class PayloadTermQuery extends SpanTermQuery {
   protected PayloadFunction function;
@@ -82,7 +82,7 @@ public class PayloadTermQuery extends Sp
     public Scorer scorer(AtomicReaderContext context, boolean scoreDocsInOrder,
         boolean topScorer, Bits acceptDocs) throws IOException {
       return new PayloadTermSpanScorer((TermSpans) query.getSpans(context, acceptDocs, termContexts),
-          this, similarity.sloppySimScorer(stats, context));
+          this, similarity.simScorer(stats, context));
     }
 
     protected class PayloadTermSpanScorer extends SpanScorer {
@@ -91,7 +91,7 @@ public class PayloadTermQuery extends Sp
       protected int payloadsSeen;
       private final TermSpans termSpans;
 
-      public PayloadTermSpanScorer(TermSpans spans, Weight weight, Similarity.SloppySimScorer docScorer) throws IOException {
+      public PayloadTermSpanScorer(TermSpans spans, Weight weight, Similarity.SimScorer docScorer) throws IOException {
         super(spans, weight, docScorer);
         termSpans = spans;
       }
@@ -182,7 +182,7 @@ public class PayloadTermQuery extends Sp
         int newDoc = scorer.advance(doc);
         if (newDoc == doc) {
           float freq = scorer.sloppyFreq();
-          SloppySimScorer docScorer = similarity.sloppySimScorer(stats, context);
+          SimScorer docScorer = similarity.simScorer(stats, context);
           Explanation expl = new Explanation();
           expl.setDescription("weight("+getQuery()+" in "+doc+") [" + similarity.getClass().getSimpleName() + "], result of:");
           Explanation scoreExplanation = docScorer.explain(doc, new Explanation(freq, "phraseFreq=" + freq));

Modified: lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/similarities/BM25Similarity.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/similarities/BM25Similarity.java?rev=1499601&r1=1499600&r2=1499601&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/similarities/BM25Similarity.java (original)
+++ lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/similarities/BM25Similarity.java Wed Jul  3 23:26:32 2013
@@ -212,80 +212,18 @@ public class BM25Similarity extends Simi
   }
 
   @Override
-  public final ExactSimScorer exactSimScorer(SimWeight stats, AtomicReaderContext context) throws IOException {
+  public final SimScorer simScorer(SimWeight stats, AtomicReaderContext context) throws IOException {
     BM25Stats bm25stats = (BM25Stats) stats;
-    final NumericDocValues norms = context.reader().getNormValues(bm25stats.field);
-    return norms == null 
-      ? new ExactBM25DocScorerNoNorms(bm25stats)
-      : new ExactBM25DocScorer(bm25stats, norms);
-  }
-
-  @Override
-  public final SloppySimScorer sloppySimScorer(SimWeight stats, AtomicReaderContext context) throws IOException {
-    BM25Stats bm25stats = (BM25Stats) stats;
-    return new SloppyBM25DocScorer(bm25stats, context.reader().getNormValues(bm25stats.field));
-  }
-  
-  private class ExactBM25DocScorer extends ExactSimScorer {
-    private final BM25Stats stats;
-    private final float weightValue;
-    private final NumericDocValues norms;
-    private final float[] cache;
-    
-    ExactBM25DocScorer(BM25Stats stats, NumericDocValues norms) throws IOException {
-      assert norms != null;
-      this.stats = stats;
-      this.weightValue = stats.weight * (k1 + 1); // boost * idf * (k1 + 1)
-      this.cache = stats.cache;
-      this.norms = norms;
-    }
-    
-    @Override
-    public float score(int doc, int freq) {
-      return weightValue * freq / (freq + cache[(byte)norms.get(doc) & 0xFF]);
-    }
-    
-    @Override
-    public Explanation explain(int doc, Explanation freq) {
-      return explainScore(doc, freq, stats, norms);
-    }
-  }
-  
-  /** there are no norms, we act as if b=0 */
-  private class ExactBM25DocScorerNoNorms extends ExactSimScorer {
-    private final BM25Stats stats;
-    private final float weightValue;
-    private static final int SCORE_CACHE_SIZE = 32;
-    private float[] scoreCache = new float[SCORE_CACHE_SIZE];
-
-    ExactBM25DocScorerNoNorms(BM25Stats stats) {
-      this.stats = stats;
-      this.weightValue = stats.weight * (k1 + 1); // boost * idf * (k1 + 1)
-      for (int i = 0; i < SCORE_CACHE_SIZE; i++)
-        scoreCache[i] = weightValue * i / (i + k1);
-    }
-    
-    @Override
-    public float score(int doc, int freq) {
-      // TODO: maybe score cache is more trouble than its worth?
-      return freq < SCORE_CACHE_SIZE        // check cache
-        ? scoreCache[freq]                  // cache hit
-        : weightValue * freq / (freq + k1); // cache miss
-    }
-    
-    @Override
-    public Explanation explain(int doc, Explanation freq) {
-      return explainScore(doc, freq, stats, null);
-    }
+    return new BM25DocScorer(bm25stats, context.reader().getNormValues(bm25stats.field));
   }
   
-  private class SloppyBM25DocScorer extends SloppySimScorer {
+  private class BM25DocScorer extends SimScorer {
     private final BM25Stats stats;
     private final float weightValue; // boost * idf * (k1 + 1)
     private final NumericDocValues norms;
     private final float[] cache;
     
-    SloppyBM25DocScorer(BM25Stats stats, NumericDocValues norms) throws IOException {
+    BM25DocScorer(BM25Stats stats, NumericDocValues norms) throws IOException {
       this.stats = stats;
       this.weightValue = stats.weight * (k1 + 1);
       this.cache = stats.cache;

Modified: lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/similarities/DefaultSimilarity.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/similarities/DefaultSimilarity.java?rev=1499601&r1=1499600&r2=1499601&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/similarities/DefaultSimilarity.java (original)
+++ lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/similarities/DefaultSimilarity.java Wed Jul  3 23:26:32 2013
@@ -19,10 +19,40 @@ package org.apache.lucene.search.similar
 
 import org.apache.lucene.index.FieldInvertState;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.SmallFloat;
 
-/** Expert: Default scoring implementation. */
+/**
+ * Expert: Default scoring implementation which {@link #encodeNormValue(float)
+ * encodes} norm values as a single byte before being stored. At search time,
+ * the norm byte value is read from the index
+ * {@link org.apache.lucene.store.Directory directory} and
+ * {@link #decodeNormValue(long) decoded} back to a float <i>norm</i> value.
+ * This encoding/decoding, while reducing index size, comes with the price of
+ * precision loss - it is not guaranteed that <i>decode(encode(x)) = x</i>. For
+ * instance, <i>decode(encode(0.89)) = 0.75</i>.
+ * <p>
+ * Compression of norm values to a single byte saves memory at search time,
+ * because once a field is referenced at search time, its norms - for all
+ * documents - are maintained in memory.
+ * <p>
+ * The rationale supporting such lossy compression of norm values is that given
+ * the difficulty (and inaccuracy) of users to express their true information
+ * need by a query, only big differences matter. <br>
+ * &nbsp;<br>
+ * Last, note that search time is too late to modify this <i>norm</i> part of
+ * scoring, e.g. by using a different {@link Similarity} for search.
+ */
 public class DefaultSimilarity extends TFIDFSimilarity {
   
+  /** Cache of decoded bytes. */
+  private static final float[] NORM_TABLE = new float[256];
+
+  static {
+    for (int i = 0; i < 256; i++) {
+      NORM_TABLE[i] = SmallFloat.byte315ToFloat((byte)i);
+    }
+  }
+
   /** Sole constructor: parameter-free */
   public DefaultSimilarity() {}
   
@@ -38,6 +68,35 @@ public class DefaultSimilarity extends T
     return (float)(1.0 / Math.sqrt(sumOfSquaredWeights));
   }
   
+  /**
+   * Encodes a normalization factor for storage in an index.
+   * <p>
+   * The encoding uses a three-bit mantissa, a five-bit exponent, and the
+   * zero-exponent point at 15, thus representing values from around 7x10^9 to
+   * 2x10^-9 with about one significant decimal digit of accuracy. Zero is also
+   * represented. Negative numbers are rounded up to zero. Values too large to
+   * represent are rounded down to the largest representable value. Positive
+   * values too small to represent are rounded up to the smallest positive
+   * representable value.
+   * 
+   * @see org.apache.lucene.document.Field#setBoost(float)
+   * @see org.apache.lucene.util.SmallFloat
+   */
+  @Override
+  public final long encodeNormValue(float f) {
+    return SmallFloat.floatToByte315(f);
+  }
+
+  /**
+   * Decodes the norm value, assuming it is a single byte.
+   * 
+   * @see #encodeNormValue(float)
+   */
+  @Override
+  public final float decodeNormValue(long norm) {
+    return NORM_TABLE[(int) (norm & 0xFF)];  // & 0xFF maps negative bytes to positive above 127
+  }
+
   /** Implemented as
    *  <code>state.getBoost()*lengthNorm(numTerms)</code>, where
    *  <code>numTerms</code> is {@link FieldInvertState#getLength()} if {@link

Modified: lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/similarities/MultiSimilarity.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/similarities/MultiSimilarity.java?rev=1499601&r1=1499600&r2=1499601&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/similarities/MultiSimilarity.java (original)
+++ lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/similarities/MultiSimilarity.java Wed Jul  3 23:26:32 2013
@@ -57,60 +57,25 @@ public class MultiSimilarity extends Sim
   }
 
   @Override
-  public ExactSimScorer exactSimScorer(SimWeight stats, AtomicReaderContext context) throws IOException {
-    ExactSimScorer subScorers[] = new ExactSimScorer[sims.length];
+  public SimScorer simScorer(SimWeight stats, AtomicReaderContext context) throws IOException {
+    SimScorer subScorers[] = new SimScorer[sims.length];
     for (int i = 0; i < subScorers.length; i++) {
-      subScorers[i] = sims[i].exactSimScorer(((MultiStats)stats).subStats[i], context);
-    }
-    return new MultiExactDocScorer(subScorers);
-  }
-
-  @Override
-  public SloppySimScorer sloppySimScorer(SimWeight stats, AtomicReaderContext context) throws IOException {
-    SloppySimScorer subScorers[] = new SloppySimScorer[sims.length];
-    for (int i = 0; i < subScorers.length; i++) {
-      subScorers[i] = sims[i].sloppySimScorer(((MultiStats)stats).subStats[i], context);
-    }
-    return new MultiSloppyDocScorer(subScorers);
-  }
-  
-  static class MultiExactDocScorer extends ExactSimScorer {
-    private final ExactSimScorer subScorers[];
-    
-    MultiExactDocScorer(ExactSimScorer subScorers[]) {
-      this.subScorers = subScorers;
-    }
-    
-    @Override
-    public float score(int doc, int freq) {
-      float sum = 0.0f;
-      for (ExactSimScorer subScorer : subScorers) {
-        sum += subScorer.score(doc, freq);
-      }
-      return sum;
-    }
-
-    @Override
-    public Explanation explain(int doc, Explanation freq) {
-      Explanation expl = new Explanation(score(doc, (int)freq.getValue()), "sum of:");
-      for (ExactSimScorer subScorer : subScorers) {
-        expl.addDetail(subScorer.explain(doc, freq));
-      }
-      return expl;
+      subScorers[i] = sims[i].simScorer(((MultiStats)stats).subStats[i], context);
     }
+    return new MultiSimScorer(subScorers);
   }
   
-  static class MultiSloppyDocScorer extends SloppySimScorer {
-    private final SloppySimScorer subScorers[];
+  static class MultiSimScorer extends SimScorer {
+    private final SimScorer subScorers[];
     
-    MultiSloppyDocScorer(SloppySimScorer subScorers[]) {
+    MultiSimScorer(SimScorer subScorers[]) {
       this.subScorers = subScorers;
     }
     
     @Override
     public float score(int doc, float freq) {
       float sum = 0.0f;
-      for (SloppySimScorer subScorer : subScorers) {
+      for (SimScorer subScorer : subScorers) {
         sum += subScorer.score(doc, freq);
       }
       return sum;
@@ -119,7 +84,7 @@ public class MultiSimilarity extends Sim
     @Override
     public Explanation explain(int doc, Explanation freq) {
       Explanation expl = new Explanation(score(doc, freq.getValue()), "sum of:");
-      for (SloppySimScorer subScorer : subScorers) {
+      for (SimScorer subScorer : subScorers) {
         expl.addDetail(subScorer.explain(doc, freq));
       }
       return expl;

Modified: lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/similarities/PerFieldSimilarityWrapper.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/similarities/PerFieldSimilarityWrapper.java?rev=1499601&r1=1499600&r2=1499601&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/similarities/PerFieldSimilarityWrapper.java (original)
+++ lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/similarities/PerFieldSimilarityWrapper.java Wed Jul  3 23:26:32 2013
@@ -54,15 +54,9 @@ public abstract class PerFieldSimilarity
   }
 
   @Override
-  public final ExactSimScorer exactSimScorer(SimWeight weight, AtomicReaderContext context) throws IOException {
+  public final SimScorer simScorer(SimWeight weight, AtomicReaderContext context) throws IOException {
     PerFieldSimWeight perFieldWeight = (PerFieldSimWeight) weight;
-    return perFieldWeight.delegate.exactSimScorer(perFieldWeight.delegateWeight, context);
-  }
-
-  @Override
-  public final SloppySimScorer sloppySimScorer(SimWeight weight, AtomicReaderContext context) throws IOException {
-    PerFieldSimWeight perFieldWeight = (PerFieldSimWeight) weight;
-    return perFieldWeight.delegate.sloppySimScorer(perFieldWeight.delegateWeight, context);
+    return perFieldWeight.delegate.simScorer(perFieldWeight.delegateWeight, context);
   }
   
   /** 

Modified: lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/similarities/Similarity.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/similarities/Similarity.java?rev=1499601&r1=1499600&r2=1499601&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/similarities/Similarity.java (original)
+++ lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/similarities/Similarity.java Wed Jul  3 23:26:32 2013
@@ -88,10 +88,8 @@ import org.apache.lucene.util.SmallFloat
  *       is called for each query leaf node, {@link Similarity#queryNorm(float)} is called for the top-level
  *       query, and finally {@link Similarity.SimWeight#normalize(float, float)} passes down the normalization value
  *       and any top-level boosts (e.g. from enclosing {@link BooleanQuery}s).
- *   <li>For each segment in the index, the Query creates a {@link #exactSimScorer(SimWeight, AtomicReaderContext)}
- *       (for queries with exact frequencies such as TermQuerys and exact PhraseQueries) or a 
- *       {@link #sloppySimScorer(SimWeight, AtomicReaderContext)} (for queries with sloppy frequencies such as
- *       SpanQuerys and sloppy PhraseQueries). The score() method is called for each matching document.
+ *   <li>For each segment in the index, the Query creates a {@link #simScorer(SimWeight, AtomicReaderContext)}
+ *       The score() method is called for each matching document.
  * </ol>
  * <p>
  * <a name="explaintime"/>
@@ -166,76 +164,31 @@ public abstract class Similarity {
    * @return SimWeight object with the information this Similarity needs to score a query.
    */
   public abstract SimWeight computeWeight(float queryBoost, CollectionStatistics collectionStats, TermStatistics... termStats);
-  
-  /**
-   * Creates a new {@link Similarity.ExactSimScorer} to score matching documents from a segment of the inverted index.
-   * @param weight collection information from {@link #computeWeight(float, CollectionStatistics, TermStatistics...)}
-   * @param context segment of the inverted index to be scored.
-   * @return ExactSimScorer for scoring documents across <code>context</code>
-   * @throws IOException if there is a low-level I/O error
-   */
-  public abstract ExactSimScorer exactSimScorer(SimWeight weight, AtomicReaderContext context) throws IOException;
-  
+
   /**
-   * Creates a new {@link Similarity.SloppySimScorer} to score matching documents from a segment of the inverted index.
+   * Creates a new {@link Similarity.SimScorer} to score matching documents from a segment of the inverted index.
    * @param weight collection information from {@link #computeWeight(float, CollectionStatistics, TermStatistics...)}
    * @param context segment of the inverted index to be scored.
    * @return SloppySimScorer for scoring documents across <code>context</code>
    * @throws IOException if there is a low-level I/O error
    */
-  public abstract SloppySimScorer sloppySimScorer(SimWeight weight, AtomicReaderContext context) throws IOException;
-  
-  /**
-   * API for scoring exact queries such as {@link TermQuery} and 
-   * exact {@link PhraseQuery}.
-   * <p>
-   * Frequencies are integers (the term or phrase frequency within the document)
-   */
-  public static abstract class ExactSimScorer {
-    
-    /**
-     * Sole constructor. (For invocation by subclass 
-     * constructors, typically implicit.)
-     */
-    public ExactSimScorer() {}
-
-    /**
-     * Score a single document
-     * @param doc document id
-     * @param freq term frequency
-     * @return document's score
-     */
-    public abstract float score(int doc, int freq);
-    
-    /**
-     * Explain the score for a single document
-     * @param doc document id
-     * @param freq Explanation of how the term frequency was computed
-     * @return document's score
-     */
-    public Explanation explain(int doc, Explanation freq) {
-      Explanation result = new Explanation(score(doc, (int)freq.getValue()), 
-          "score(doc=" + doc + ",freq=" + freq.getValue() +"), with freq of:");
-      result.addDetail(freq);
-      return result;
-    }
-  }
+  public abstract SimScorer simScorer(SimWeight weight, AtomicReaderContext context) throws IOException;
   
   /**
-   * API for scoring "sloppy" queries such as {@link SpanQuery} and 
-   * sloppy {@link PhraseQuery}.
+   * API for scoring "sloppy" queries such as {@link TermQuery},
+   * {@link SpanQuery}, and {@link PhraseQuery}.
    * <p>
    * Frequencies are floating-point values: an approximate 
    * within-document frequency adjusted for "sloppiness" by 
-   * {@link SloppySimScorer#computeSlopFactor(int)}.
+   * {@link SimScorer#computeSlopFactor(int)}.
    */
-  public static abstract class SloppySimScorer {
+  public static abstract class SimScorer {
     
     /**
      * Sole constructor. (For invocation by subclass 
      * constructors, typically implicit.)
      */
-    public SloppySimScorer() {}
+    public SimScorer() {}
 
     /**
      * Score a single document

Modified: lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/similarities/SimilarityBase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/similarities/SimilarityBase.java?rev=1499601&r1=1499600&r2=1499601&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/similarities/SimilarityBase.java (original)
+++ lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/similarities/SimilarityBase.java Wed Jul  3 23:26:32 2013
@@ -190,38 +190,20 @@ public abstract class SimilarityBase ext
   }
   
   @Override
-  public ExactSimScorer exactSimScorer(SimWeight stats, AtomicReaderContext context) throws IOException {
+  public SimScorer simScorer(SimWeight stats, AtomicReaderContext context) throws IOException {
     if (stats instanceof MultiSimilarity.MultiStats) {
       // a multi term query (e.g. phrase). return the summation, 
       // scoring almost as if it were boolean query
       SimWeight subStats[] = ((MultiSimilarity.MultiStats) stats).subStats;
-      ExactSimScorer subScorers[] = new ExactSimScorer[subStats.length];
+      SimScorer subScorers[] = new SimScorer[subStats.length];
       for (int i = 0; i < subScorers.length; i++) {
         BasicStats basicstats = (BasicStats) subStats[i];
-        subScorers[i] = new BasicExactDocScorer(basicstats, context.reader().getNormValues(basicstats.field));
+        subScorers[i] = new BasicSimScorer(basicstats, context.reader().getNormValues(basicstats.field));
       }
-      return new MultiSimilarity.MultiExactDocScorer(subScorers);
+      return new MultiSimilarity.MultiSimScorer(subScorers);
     } else {
       BasicStats basicstats = (BasicStats) stats;
-      return new BasicExactDocScorer(basicstats, context.reader().getNormValues(basicstats.field));
-    }
-  }
-  
-  @Override
-  public SloppySimScorer sloppySimScorer(SimWeight stats, AtomicReaderContext context) throws IOException {
-    if (stats instanceof MultiSimilarity.MultiStats) {
-      // a multi term query (e.g. phrase). return the summation, 
-      // scoring almost as if it were boolean query
-      SimWeight subStats[] = ((MultiSimilarity.MultiStats) stats).subStats;
-      SloppySimScorer subScorers[] = new SloppySimScorer[subStats.length];
-      for (int i = 0; i < subScorers.length; i++) {
-        BasicStats basicstats = (BasicStats) subStats[i];
-        subScorers[i] = new BasicSloppyDocScorer(basicstats, context.reader().getNormValues(basicstats.field));
-      }
-      return new MultiSimilarity.MultiSloppyDocScorer(subScorers);
-    } else {
-      BasicStats basicstats = (BasicStats) stats;
-      return new BasicSloppyDocScorer(basicstats, context.reader().getNormValues(basicstats.field));
+      return new BasicSimScorer(basicstats, context.reader().getNormValues(basicstats.field));
     }
   }
   
@@ -277,46 +259,17 @@ public abstract class SimilarityBase ext
   
   // --------------------------------- Classes ---------------------------------
   
-  /** Delegates the {@link #score(int, int)} and
-   * {@link #explain(int, Explanation)} methods to
-   * {@link SimilarityBase#score(BasicStats, float, float)} and
-   * {@link SimilarityBase#explain(BasicStats, int, Explanation, float)},
-   * respectively.
-   */
-  private class BasicExactDocScorer extends ExactSimScorer {
-    private final BasicStats stats;
-    private final NumericDocValues norms;
-    
-    BasicExactDocScorer(BasicStats stats, NumericDocValues norms) throws IOException {
-      this.stats = stats;
-      this.norms = norms;
-    }
-    
-    @Override
-    public float score(int doc, int freq) {
-      // We have to supply something in case norms are omitted
-      return SimilarityBase.this.score(stats, freq,
-          norms == null ? 1F : decodeNormValue((byte)norms.get(doc)));
-    }
-    
-    @Override
-    public Explanation explain(int doc, Explanation freq) {
-      return SimilarityBase.this.explain(stats, doc, freq,
-          norms == null ? 1F : decodeNormValue((byte)norms.get(doc)));
-    }
-  }
-  
   /** Delegates the {@link #score(int, float)} and
    * {@link #explain(int, Explanation)} methods to
    * {@link SimilarityBase#score(BasicStats, float, float)} and
    * {@link SimilarityBase#explain(BasicStats, int, Explanation, float)},
    * respectively.
    */
-  private class BasicSloppyDocScorer extends SloppySimScorer {
+  private class BasicSimScorer extends SimScorer {
     private final BasicStats stats;
     private final NumericDocValues norms;
     
-    BasicSloppyDocScorer(BasicStats stats, NumericDocValues norms) throws IOException {
+    BasicSimScorer(BasicStats stats, NumericDocValues norms) throws IOException {
       this.stats = stats;
       this.norms = norms;
     }

Modified: lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/similarities/TFIDFSimilarity.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/similarities/TFIDFSimilarity.java?rev=1499601&r1=1499600&r2=1499601&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/similarities/TFIDFSimilarity.java (original)
+++ lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/similarities/TFIDFSimilarity.java Wed Jul  3 23:26:32 2013
@@ -28,7 +28,6 @@ import org.apache.lucene.search.IndexSea
 import org.apache.lucene.search.PhraseQuery;
 import org.apache.lucene.search.TermStatistics;
 import org.apache.lucene.util.BytesRef;
-import org.apache.lucene.util.SmallFloat;
 
 
 /**
@@ -496,27 +495,8 @@ import org.apache.lucene.util.SmallFloat
  *          <td></td>
  *        </tr>
  *      </table>
- *      <br>&nbsp;<br>
- *      However the resulted <i>norm</i> value is {@link #encodeNormValue(float) encoded} as a single byte
- *      before being stored.
- *      At search time, the norm byte value is read from the index
- *      {@link org.apache.lucene.store.Directory directory} and
- *      {@link #decodeNormValue(byte) decoded} back to a float <i>norm</i> value.
- *      This encoding/decoding, while reducing index size, comes with the price of
- *      precision loss - it is not guaranteed that <i>decode(encode(x)) = x</i>.
- *      For instance, <i>decode(encode(0.89)) = 0.75</i>.
- *      <br>&nbsp;<br>
- *      Compression of norm values to a single byte saves memory at search time, 
- *      because once a field is referenced at search time, its norms - for 
- *      all documents - are maintained in memory.
- *      <br>&nbsp;<br>
- *      The rationale supporting such lossy compression of norm values is that
- *      given the difficulty (and inaccuracy) of users to express their true information
- *      need by a query, only big differences matter.
- *      <br>&nbsp;<br>
- *      Last, note that search time is too late to modify this <i>norm</i> part of scoring, e.g. by
- *      using a different {@link Similarity} for search.
- *      <br>&nbsp;<br>
+ *      Note that search time is too late to modify this <i>norm</i> part of scoring, 
+ *      e.g. by using a different {@link Similarity} for search.
  *    </li>
  * </ol>
  *
@@ -572,25 +552,6 @@ public abstract class TFIDFSimilarity ex
    * when <code>freq</code> is large, and smaller values when <code>freq</code>
    * is small.
    *
-   * <p>The default implementation calls {@link #tf(float)}.
-   *
-   * @param freq the frequency of a term within a document
-   * @return a score factor based on a term's within-document frequency
-   */
-  public float tf(int freq) {
-    return tf((float)freq);
-  }
-
-  /** Computes a score factor based on a term or phrase's frequency in a
-   * document.  This value is multiplied by the {@link #idf(long, long)}
-   * factor for each term in the query and these products are then summed to
-   * form the initial score for a document.
-   *
-   * <p>Terms and phrases repeated in a document indicate the topic of the
-   * document, so implementations of this method usually return larger values
-   * when <code>freq</code> is large, and smaller values when <code>freq</code>
-   * is small.
-   *
    * @param freq the frequency of a term within a document
    * @return a score factor based on a term's within-document frequency
    */
@@ -655,7 +616,7 @@ public abstract class TFIDFSimilarity ex
 
   /** Computes a score factor based on a term's document frequency (the number
    * of documents which contain the term).  This value is multiplied by the
-   * {@link #tf(int)} factor for each term in the query and these products are
+   * {@link #tf(float)} factor for each term in the query and these products are
    * then summed to form the initial score for a document.
    *
    * <p>Terms that occur in fewer documents are better indicators of topic, so
@@ -685,38 +646,15 @@ public abstract class TFIDFSimilarity ex
     return encodeNormValue(normValue);
   }
   
-  /** Cache of decoded bytes. */
-  private static final float[] NORM_TABLE = new float[256];
-
-  static {
-    for (int i = 0; i < 256; i++) {
-      NORM_TABLE[i] = SmallFloat.byte315ToFloat((byte)i);
-    }
-  }
-
-  /** Decodes a normalization factor stored in an index.
+  /**
+   * Decodes a normalization factor stored in an index.
+   * 
    * @see #encodeNormValue(float)
    */
-  public float decodeNormValue(byte b) {
-    return NORM_TABLE[b & 0xFF];  // & 0xFF maps negative bytes to positive above 127
-  }
+  public abstract float decodeNormValue(long norm);
 
-  /** Encodes a normalization factor for storage in an index.
-  *
-  * <p>The encoding uses a three-bit mantissa, a five-bit exponent, and
-  * the zero-exponent point at 15, thus
-  * representing values from around 7x10^9 to 2x10^-9 with about one
-  * significant decimal digit of accuracy.  Zero is also represented.
-  * Negative numbers are rounded up to zero.  Values too large to represent
-  * are rounded down to the largest representable value.  Positive values too
-  * small to represent are rounded up to the smallest positive representable
-  * value.
-  * @see org.apache.lucene.document.Field#setBoost(float)
-  * @see org.apache.lucene.util.SmallFloat
-  */
-  public byte encodeNormValue(float f) {
-    return SmallFloat.floatToByte315(f);
-  }
+  /** Encodes a normalization factor for storage in an index. */
+  public abstract long encodeNormValue(float f);
  
   /** Computes the amount of a sloppy phrase match, based on an edit distance.
    * This value is summed for each sloppy phrase match in a document to form
@@ -755,49 +693,17 @@ public abstract class TFIDFSimilarity ex
   }
 
   @Override
-  public final ExactSimScorer exactSimScorer(SimWeight stats, AtomicReaderContext context) throws IOException {
+  public final SimScorer simScorer(SimWeight stats, AtomicReaderContext context) throws IOException {
     IDFStats idfstats = (IDFStats) stats;
-    return new ExactTFIDFDocScorer(idfstats, context.reader().getNormValues(idfstats.field));
-  }
-
-  @Override
-  public final SloppySimScorer sloppySimScorer(SimWeight stats, AtomicReaderContext context) throws IOException {
-    IDFStats idfstats = (IDFStats) stats;
-    return new SloppyTFIDFDocScorer(idfstats, context.reader().getNormValues(idfstats.field));
-  }
-  
-  // TODO: we can specialize these for omitNorms up front, but we should test that it doesn't confuse stupid hotspot.
-
-  private final class ExactTFIDFDocScorer extends ExactSimScorer {
-    private final IDFStats stats;
-    private final float weightValue;
-    private final NumericDocValues norms;
-    
-    ExactTFIDFDocScorer(IDFStats stats, NumericDocValues norms) throws IOException {
-      this.stats = stats;
-      this.weightValue = stats.value;
-      this.norms = norms; 
-    }
-    
-    @Override
-    public float score(int doc, int freq) {
-      final float raw = tf(freq)*weightValue;  // compute tf(f)*weight
-
-      return norms == null ? raw : raw * decodeNormValue((byte)norms.get(doc)); // normalize for field
-    }
-
-    @Override
-    public Explanation explain(int doc, Explanation freq) {
-      return explainScore(doc, freq, stats, norms);
-    }
+    return new TFIDFSimScorer(idfstats, context.reader().getNormValues(idfstats.field));
   }
   
-  private final class SloppyTFIDFDocScorer extends SloppySimScorer {
+  private final class TFIDFSimScorer extends SimScorer {
     private final IDFStats stats;
     private final float weightValue;
     private final NumericDocValues norms;
     
-    SloppyTFIDFDocScorer(IDFStats stats, NumericDocValues norms) throws IOException {
+    TFIDFSimScorer(IDFStats stats, NumericDocValues norms) throws IOException {
       this.stats = stats;
       this.weightValue = stats.value;
       this.norms = norms;
@@ -807,7 +713,7 @@ public abstract class TFIDFSimilarity ex
     public float score(int doc, float freq) {
       final float raw = tf(freq) * weightValue; // compute tf(f)*weight
       
-      return norms == null ? raw : raw * decodeNormValue((byte)norms.get(doc));  // normalize for field
+      return norms == null ? raw : raw * decodeNormValue(norms.get(doc));  // normalize for field
     }
     
     @Override
@@ -894,8 +800,7 @@ public abstract class TFIDFSimilarity ex
     fieldExpl.addDetail(stats.idf);
 
     Explanation fieldNormExpl = new Explanation();
-    float fieldNorm =
-      norms!=null ? decodeNormValue((byte) norms.get(doc)) : 1.0f;
+    float fieldNorm = norms != null ? decodeNormValue(norms.get(doc)) : 1.0f;
     fieldNormExpl.setValue(fieldNorm);
     fieldNormExpl.setDescription("fieldNorm(doc="+doc+")");
     fieldExpl.addDetail(fieldNormExpl);

Modified: lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/spans/SpanScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/spans/SpanScorer.java?rev=1499601&r1=1499600&r2=1499601&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/spans/SpanScorer.java (original)
+++ lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/spans/SpanScorer.java Wed Jul  3 23:26:32 2013
@@ -34,9 +34,9 @@ public class SpanScorer extends Scorer {
   protected int doc;
   protected float freq;
   protected int numMatches;
-  protected final Similarity.SloppySimScorer docScorer;
+  protected final Similarity.SimScorer docScorer;
   
-  protected SpanScorer(Spans spans, Weight weight, Similarity.SloppySimScorer docScorer)
+  protected SpanScorer(Spans spans, Weight weight, Similarity.SimScorer docScorer)
   throws IOException {
     super(weight);
     this.docScorer = docScorer;

Modified: lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/spans/SpanWeight.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/spans/SpanWeight.java?rev=1499601&r1=1499600&r2=1499601&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/spans/SpanWeight.java (original)
+++ lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/spans/SpanWeight.java Wed Jul  3 23:26:32 2013
@@ -23,7 +23,7 @@ import org.apache.lucene.index.Term;
 import org.apache.lucene.index.TermContext;
 import org.apache.lucene.search.*;
 import org.apache.lucene.search.similarities.Similarity;
-import org.apache.lucene.search.similarities.Similarity.SloppySimScorer;
+import org.apache.lucene.search.similarities.Similarity.SimScorer;
 import org.apache.lucene.util.Bits;
 
 import java.io.IOException;
@@ -86,7 +86,7 @@ public class SpanWeight extends Weight {
     if (stats == null) {
       return null;
     } else {
-      return new SpanScorer(query.getSpans(context, acceptDocs, termContexts), this, similarity.sloppySimScorer(stats, context));
+      return new SpanScorer(query.getSpans(context, acceptDocs, termContexts), this, similarity.simScorer(stats, context));
     }
   }
 
@@ -97,7 +97,7 @@ public class SpanWeight extends Weight {
       int newDoc = scorer.advance(doc);
       if (newDoc == doc) {
         float freq = scorer.sloppyFreq();
-        SloppySimScorer docScorer = similarity.sloppySimScorer(stats, context);
+        SimScorer docScorer = similarity.simScorer(stats, context);
         ComplexExplanation result = new ComplexExplanation();
         result.setDescription("weight("+getQuery()+" in "+doc+") [" + similarity.getClass().getSimpleName() + "], result of:");
         Explanation scoreExplanation = docScorer.explain(doc, new Explanation(freq, "phraseFreq=" + freq));

Modified: lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/store/Directory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/store/Directory.java?rev=1499601&r1=1499600&r2=1499601&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/store/Directory.java (original)
+++ lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/store/Directory.java Wed Jul  3 23:26:32 2013
@@ -21,6 +21,7 @@ import java.io.EOFException;
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.Closeable;
+import java.nio.file.NoSuchFileException;
 import java.util.Collection; // for javadocs
 
 import org.apache.lucene.util.IOUtils;
@@ -70,12 +71,12 @@ public abstract class Directory implemen
    * Returns the length of a file in the directory. This method follows the
    * following contract:
    * <ul>
-   * <li>Throws {@link FileNotFoundException} if the file does not exist
+   * <li>Throws {@link FileNotFoundException} or {@link NoSuchFileException}
+   * if the file does not exist.
    * <li>Returns a value &ge;0 if the file exists, which specifies its length.
    * </ul>
    * 
    * @param name the name of the file for which to return the length.
-   * @throws FileNotFoundException if the file does not exist.
    * @throws IOException if there was an IO error while retrieving the file's
    *         length.
    */
@@ -106,7 +107,9 @@ public abstract class Directory implemen
    * the only Directory implementations that respect this
    * parameter are {@link FSDirectory} and {@link
    * CompoundFileDirectory}.
-  */
+   * <p>Throws {@link FileNotFoundException} or {@link NoSuchFileException}
+   * if the file does not exist.
+   */
   public abstract IndexInput openInput(String name, IOContext context) throws IOException; 
   
   /** Construct a {@link Lock}.
@@ -223,6 +226,8 @@ public abstract class Directory implemen
    * efficiently open one or more sliced {@link IndexInput} instances from a
    * single file handle. The underlying file handle is kept open until the
    * {@link IndexInputSlicer} is closed.
+   * <p>Throws {@link FileNotFoundException} or {@link NoSuchFileException}
+   * if the file does not exist.
    *
    * @throws IOException
    *           if an {@link IOException} occurs

Modified: lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/util/Constants.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/util/Constants.java?rev=1499601&r1=1499600&r2=1499601&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/util/Constants.java (original)
+++ lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/util/Constants.java Wed Jul  3 23:26:32 2013
@@ -46,6 +46,8 @@ public final class Constants {
   public static final boolean SUN_OS = OS_NAME.startsWith("SunOS");
   /** True iff running on Mac OS X */
   public static final boolean MAC_OS_X = OS_NAME.startsWith("Mac OS X");
+  /** True iff running on FreeBSD */
+  public static final boolean FREE_BSD = OS_NAME.startsWith("FreeBSD");
 
   public static final String OS_ARCH = System.getProperty("os.arch");
   public static final String OS_VERSION = System.getProperty("os.version");

Modified: lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/util/RollingBuffer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/util/RollingBuffer.java?rev=1499601&r1=1499600&r2=1499601&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/util/RollingBuffer.java (original)
+++ lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/util/RollingBuffer.java Wed Jul  3 23:26:32 2013
@@ -17,9 +17,6 @@ package org.apache.lucene.util;
  * limitations under the License.
  */
 
-// TODO: probably move this to core at some point (eg,
-// cutover kuromoji, synfilter, LookaheadTokenFilter)
-
 /** Acts like forever growing T[], but internally uses a
  *  circular buffer to reuse instances of T.
  * 

Modified: lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/util/automaton/SpecialOperations.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/util/automaton/SpecialOperations.java?rev=1499601&r1=1499600&r2=1499601&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/util/automaton/SpecialOperations.java (original)
+++ lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/util/automaton/SpecialOperations.java Wed Jul  3 23:26:32 2013
@@ -219,7 +219,7 @@ final public class SpecialOperations {
   /**
    * Returns the set of accepted strings, assuming that at most
    * <code>limit</code> strings are accepted. If more than <code>limit</code> 
-   * strings are accepted, null is returned. If <code>limit</code>&lt;0, then 
+   * strings are accepted, the first limit strings found are returned. If <code>limit</code>&lt;0, then 
    * the limit is infinite.
    */
   public static Set<IntsRef> getFiniteStrings(Automaton a, int limit) {
@@ -227,11 +227,9 @@ final public class SpecialOperations {
     if (a.isSingleton()) {
       if (limit > 0) {
         strings.add(Util.toUTF32(a.singleton, new IntsRef()));
-      } else {
-        return null;
       }
     } else if (!getFiniteStrings(a.initial, new HashSet<State>(), strings, new IntsRef(), limit)) {
-      return null;
+      return strings;
     }
     return strings;
   }

Modified: lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/util/fst/NodeHash.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/util/fst/NodeHash.java?rev=1499601&r1=1499600&r2=1499601&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/util/fst/NodeHash.java (original)
+++ lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/util/fst/NodeHash.java Wed Jul  3 23:26:32 2013
@@ -19,21 +19,21 @@ package org.apache.lucene.util.fst;
 
 import java.io.IOException;
 
-import org.apache.lucene.util.packed.GrowableWriter;
 import org.apache.lucene.util.packed.PackedInts;
+import org.apache.lucene.util.packed.PagedGrowableWriter;
 
 // Used to dedup states (lookup already-frozen states)
 final class NodeHash<T> {
 
-  private GrowableWriter table;
-  private int count;
-  private int mask;
+  private PagedGrowableWriter table;
+  private long count;
+  private long mask;
   private final FST<T> fst;
   private final FST.Arc<T> scratchArc = new FST.Arc<T>();
   private final FST.BytesReader in;
 
   public NodeHash(FST<T> fst, FST.BytesReader in) {
-    table = new GrowableWriter(8, 16, PackedInts.COMPACT);
+    table = new PagedGrowableWriter(16, 1<<30, 8, PackedInts.COMPACT);
     mask = 15;
     this.fst = fst;
     this.in = in;
@@ -69,10 +69,10 @@ final class NodeHash<T> {
 
   // hash code for an unfrozen node.  This must be identical
   // to the un-frozen case (below)!!
-  private int hash(Builder.UnCompiledNode<T> node) {
+  private long hash(Builder.UnCompiledNode<T> node) {
     final int PRIME = 31;
     //System.out.println("hash unfrozen");
-    int h = 0;
+    long h = 0;
     // TODO: maybe if number of arcs is high we can safely subsample?
     for(int arcIdx=0;arcIdx<node.numArcs;arcIdx++) {
       final Builder.Arc<T> arc = node.arcs[arcIdx];
@@ -87,14 +87,14 @@ final class NodeHash<T> {
       }
     }
     //System.out.println("  ret " + (h&Integer.MAX_VALUE));
-    return h & Integer.MAX_VALUE;
+    return h & Long.MAX_VALUE;
   }
 
   // hash code for a frozen node
-  private int hash(long node) throws IOException {
+  private long hash(long node) throws IOException {
     final int PRIME = 31;
     //System.out.println("hash frozen node=" + node);
-    int h = 0;
+    long h = 0;
     fst.readFirstRealTargetArc(node, scratchArc, in);
     while(true) {
       //System.out.println("  label=" + scratchArc.label + " target=" + scratchArc.target + " h=" + h + " output=" + fst.outputs.outputToString(scratchArc.output) + " next?=" + scratchArc.flag(4) + " final?=" + scratchArc.isFinal() + " pos=" + in.getPosition());
@@ -111,13 +111,13 @@ final class NodeHash<T> {
       fst.readNextRealArc(scratchArc, in);
     }
     //System.out.println("  ret " + (h&Integer.MAX_VALUE));
-    return h & Integer.MAX_VALUE;
+    return h & Long.MAX_VALUE;
   }
 
   public long add(Builder.UnCompiledNode<T> nodeIn) throws IOException {
-    // System.out.println("hash: add count=" + count + " vs " + table.size());
-    final int h = hash(nodeIn);
-    int pos = h & mask;
+    //System.out.println("hash: add count=" + count + " vs " + table.size() + " mask=" + mask);
+    final long h = hash(nodeIn);
+    long pos = h & mask;
     int c = 0;
     while(true) {
       final long v = table.get(pos);
@@ -128,7 +128,8 @@ final class NodeHash<T> {
         assert hash(node) == h : "frozenHash=" + hash(node) + " vs h=" + h;
         count++;
         table.set(pos, node);
-        if (table.size() < 2*count) {
+        // Rehash at 2/3 occupancy:
+        if (count > 2*table.size()/3) {
           rehash();
         }
         return node;
@@ -144,7 +145,7 @@ final class NodeHash<T> {
 
   // called only by rehash
   private void addNew(long address) throws IOException {
-    int pos = hash(address) & mask;
+    long pos = hash(address) & mask;
     int c = 0;
     while(true) {
       if (table.get(pos) == 0) {
@@ -158,23 +159,15 @@ final class NodeHash<T> {
   }
 
   private void rehash() throws IOException {
-    final GrowableWriter oldTable = table;
+    final PagedGrowableWriter oldTable = table;
 
-    if (oldTable.size() >= Integer.MAX_VALUE/2) {
-      throw new IllegalStateException("FST too large (> 2.1 GB)");
-    }
-
-    table = new GrowableWriter(oldTable.getBitsPerValue(), 2*oldTable.size(), PackedInts.COMPACT);
+    table = new PagedGrowableWriter(2*oldTable.size(), 1<<30, PackedInts.bitsRequired(count), PackedInts.COMPACT);
     mask = table.size()-1;
-    for(int idx=0;idx<oldTable.size();idx++) {
+    for(long idx=0;idx<oldTable.size();idx++) {
       final long address = oldTable.get(idx);
       if (address != 0) {
         addNew(address);
       }
     }
   }
-
-  public int count() {
-    return count;
-  }
 }

Modified: lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/util/fst/PositiveIntOutputs.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/util/fst/PositiveIntOutputs.java?rev=1499601&r1=1499600&r2=1499601&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/util/fst/PositiveIntOutputs.java (original)
+++ lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/util/fst/PositiveIntOutputs.java Wed Jul  3 23:26:32 2013
@@ -33,26 +33,13 @@ public final class PositiveIntOutputs ex
   
   private final static Long NO_OUTPUT = new Long(0);
 
-  private final boolean doShare;
+  private final static PositiveIntOutputs singleton = new PositiveIntOutputs();
 
-  private final static PositiveIntOutputs singletonShare = new PositiveIntOutputs(true);
-  private final static PositiveIntOutputs singletonNoShare = new PositiveIntOutputs(false);
-
-  private PositiveIntOutputs(boolean doShare) {
-    this.doShare = doShare;
+  private PositiveIntOutputs() {
   }
 
-  /** Returns the instance of PositiveIntOutputs. */
   public static PositiveIntOutputs getSingleton() {
-    return getSingleton(true);
-  }
-
-  /** Expert: pass doShare=false to disable output sharing.
-   *  In some cases this may result in a smaller FST,
-   *  however it will also break methods like {@link
-   *  Util#getByOutput} and {@link Util#shortestPaths}. */
-  public static PositiveIntOutputs getSingleton(boolean doShare) {
-    return doShare ? singletonShare : singletonNoShare;
+    return singleton;
   }
 
   @Override
@@ -61,14 +48,10 @@ public final class PositiveIntOutputs ex
     assert valid(output2);
     if (output1 == NO_OUTPUT || output2 == NO_OUTPUT) {
       return NO_OUTPUT;
-    } else if (doShare) {
+    } else {
       assert output1 > 0;
       assert output2 > 0;
       return Math.min(output1, output2);
-    } else if (output1.equals(output2)) {
-      return output1;
-    } else {
-      return NO_OUTPUT;
     }
   }
 
@@ -134,6 +117,6 @@ public final class PositiveIntOutputs ex
 
   @Override
   public String toString() {
-    return "PositiveIntOutputs(doShare=" + doShare + ")";
+    return "PositiveIntOutputs";
   }
 }

Modified: lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/util/fst/Util.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/util/fst/Util.java?rev=1499601&r1=1499600&r2=1499601&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/util/fst/Util.java (original)
+++ lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/util/fst/Util.java Wed Jul  3 23:26:32 2013
@@ -93,9 +93,7 @@ public final class Util {
    *
    *  <p>NOTE: this only works with {@code FST<Long>}, only
    *  works when the outputs are ascending in order with
-   *  the inputs and only works when you shared
-   *  the outputs (pass doShare=true to {@link
-   *  PositiveIntOutputs#getSingleton}).
+   *  the inputs.
    *  For example, simple ordinals (0, 1,
    *  2, ...), or file offets (when appending to a file)
    *  fit this. */
@@ -517,11 +515,7 @@ public final class Util {
   }
 
   /** Starting from node, find the top N min cost 
-   *  completions to a final node.
-   *
-   *  <p>NOTE: you must share the outputs when you build the
-   *  FST (pass doShare=true to {@link
-   *  PositiveIntOutputs#getSingleton}). */
+   *  completions to a final node. */
   public static <T> MinResult<T>[] shortestPaths(FST<T> fst, FST.Arc<T> fromNode, T startOutput, Comparator<T> comparator, int topN,
                                                  boolean allowEmptyString) throws IOException {
 
@@ -814,7 +808,7 @@ public final class Util {
     final int charLimit = offset + length;
     while(charIdx < charLimit) {
       scratch.grow(intIdx+1);
-      final int utf32 = Character.codePointAt(s, charIdx);
+      final int utf32 = Character.codePointAt(s, charIdx, charLimit);
       scratch.ints[intIdx] = utf32;
       charIdx += Character.charCount(utf32);
       intIdx++;

Modified: lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/util/fst/package.html
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/util/fst/package.html?rev=1499601&r1=1499600&r2=1499601&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/util/fst/package.html (original)
+++ lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/util/fst/package.html Wed Jul  3 23:26:32 2013
@@ -43,7 +43,7 @@ FST Construction example:
     String inputValues[] = {"cat", "dog", "dogs"};
     long outputValues[] = {5, 7, 12};
     
-    PositiveIntOutputs outputs = PositiveIntOutputs.getSingleton(true);
+    PositiveIntOutputs outputs = PositiveIntOutputs.getSingleton();
     Builder&lt;Long&gt; builder = new Builder&lt;Long&gt;(INPUT_TYPE.BYTE1, outputs);
     BytesRef scratchBytes = new BytesRef();
     IntsRef scratchInts = new IntsRef();
@@ -60,8 +60,7 @@ Retrieval by key:
 </pre>
 Retrieval by value:
 <pre class="prettyprint">
-    // Only works because outputs are also in sorted order, and
-    // we passed 'true' for sharing to PositiveIntOutputs.getSingleton
+    // Only works because outputs are also in sorted order
     IntsRef key = Util.getByOutput(fst, 12);
     System.out.println(Util.toBytesRef(key, scratchBytes).utf8ToString()); // dogs
 </pre>
@@ -77,7 +76,6 @@ Iterate over key-value pairs in sorted o
 </pre>
 N-shortest paths by weight:
 <pre class="prettyprint">
-    // Only works because we passed 'true' for sharing to PositiveIntOutputs.getSingleton
     Comparator&lt;Long&gt; comparator = new Comparator&lt;Long&gt;() {
       public int compare(Long left, Long right) {
         return left.compareTo(right);

Modified: lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/util/packed/AbstractAppendingLongBuffer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/util/packed/AbstractAppendingLongBuffer.java?rev=1499601&r1=1499600&r2=1499601&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/util/packed/AbstractAppendingLongBuffer.java (original)
+++ lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/util/packed/AbstractAppendingLongBuffer.java Wed Jul  3 23:26:32 2013
@@ -17,6 +17,8 @@ package org.apache.lucene.util.packed;
  * limitations under the License.
  */
 
+import static org.apache.lucene.util.packed.PackedInts.checkBlockSize;
+
 import java.util.Arrays;
 
 import org.apache.lucene.util.ArrayUtil;
@@ -25,33 +27,37 @@ import org.apache.lucene.util.RamUsageEs
 /** Common functionality shared by {@link AppendingLongBuffer} and {@link MonotonicAppendingLongBuffer}. */
 abstract class AbstractAppendingLongBuffer {
 
-  static final int BLOCK_BITS = 10;
-  static final int MAX_PENDING_COUNT = 1 << BLOCK_BITS;
-  static final int BLOCK_MASK = MAX_PENDING_COUNT - 1;
+  static final int MIN_PAGE_SIZE = 64;
+  // More than 1M doesn't really makes sense with these appending buffers
+  // since their goal is to try to have small numbers of bits per value
+  static final int MAX_PAGE_SIZE = 1 << 20;
 
+  final int pageShift, pageMask;
   long[] minValues;
   PackedInts.Reader[] deltas;
   private long deltasBytes;
   int valuesOff;
-  long[] pending;
+  final long[] pending;
   int pendingOff;
 
-  AbstractAppendingLongBuffer(int initialBlockCount) {
-    minValues = new long[16];
-    deltas = new PackedInts.Reader[16];
-    pending = new long[MAX_PENDING_COUNT];
+  AbstractAppendingLongBuffer(int initialBlockCount, int pageSize) {
+    minValues = new long[initialBlockCount];
+    deltas = new PackedInts.Reader[initialBlockCount];
+    pending = new long[pageSize];
+    pageShift = checkBlockSize(pageSize, MIN_PAGE_SIZE, MAX_PAGE_SIZE);
+    pageMask = pageSize - 1;
     valuesOff = 0;
     pendingOff = 0;
   }
 
   /** Get the number of values that have been added to the buffer. */
   public final long size() {
-    return valuesOff * (long) MAX_PENDING_COUNT + pendingOff;
+    return valuesOff * (long) pending.length + pendingOff;
   }
 
   /** Append a value to this buffer. */
   public final void add(long l) {
-    if (pendingOff == MAX_PENDING_COUNT) {
+    if (pendingOff == pending.length) {
       // check size
       if (deltas.length == valuesOff) {
         final int newLength = ArrayUtil.oversize(valuesOff + 1, 8);
@@ -80,8 +86,8 @@ abstract class AbstractAppendingLongBuff
     if (index < 0 || index >= size()) {
       throw new IndexOutOfBoundsException("" + index);
     }
-    int block = (int) (index >> BLOCK_BITS);
-    int element = (int) (index & BLOCK_MASK);
+    final int block = (int) (index >> pageShift);
+    final int element = (int) (index & pageMask);
     return get(block, element);
   }
 
@@ -99,7 +105,7 @@ abstract class AbstractAppendingLongBuff
       if (valuesOff == 0) {
         currentValues = pending;
       } else {
-        currentValues = new long[MAX_PENDING_COUNT];
+        currentValues = new long[pending.length];
         fillValues();
       }
     }
@@ -115,7 +121,7 @@ abstract class AbstractAppendingLongBuff
     public final long next() {
       assert hasNext();
       long result = currentValues[pOff++];
-      if (pOff == MAX_PENDING_COUNT) {
+      if (pOff == pending.length) {
         vOff += 1;
         pOff = 0;
         if (vOff <= valuesOff) {
@@ -139,6 +145,7 @@ abstract class AbstractAppendingLongBuff
   public long ramBytesUsed() {
     // TODO: this is called per-doc-per-norms/dv-field, can we optimize this?
     long bytesUsed = RamUsageEstimator.alignObjectSize(baseRamBytesUsed())
+        + 2 * RamUsageEstimator.NUM_BYTES_INT // pageShift, pageMask
         + RamUsageEstimator.NUM_BYTES_LONG // valuesBytes
         + RamUsageEstimator.sizeOf(pending)
         + RamUsageEstimator.sizeOf(minValues)

Modified: lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/util/packed/AbstractBlockPackedWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/util/packed/AbstractBlockPackedWriter.java?rev=1499601&r1=1499600&r2=1499601&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/util/packed/AbstractBlockPackedWriter.java (original)
+++ lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/util/packed/AbstractBlockPackedWriter.java Wed Jul  3 23:26:32 2013
@@ -17,6 +17,8 @@ package org.apache.lucene.util.packed;
  * limitations under the License.
  */
 
+import static org.apache.lucene.util.packed.PackedInts.checkBlockSize;
+
 import java.io.IOException;
 import java.util.Arrays;
 
@@ -24,22 +26,11 @@ import org.apache.lucene.store.DataOutpu
 
 abstract class AbstractBlockPackedWriter {
 
+  static final int MIN_BLOCK_SIZE = 64;
   static final int MAX_BLOCK_SIZE = 1 << (30 - 3);
   static final int MIN_VALUE_EQUALS_0 = 1 << 0;
   static final int BPV_SHIFT = 1;
 
-  static void checkBlockSize(int blockSize) {
-    if (blockSize <= 0 || blockSize > MAX_BLOCK_SIZE) {
-      throw new IllegalArgumentException("blockSize must be > 0 and < " + MAX_BLOCK_SIZE + ", got " + blockSize);
-    }
-    if (blockSize < 64) {
-      throw new IllegalArgumentException("blockSize must be >= 64, got " + blockSize);
-    }
-    if ((blockSize & (blockSize - 1)) != 0) {
-      throw new IllegalArgumentException("blockSize must be a power of two, got " + blockSize);
-    }
-  }
-
   static long zigZagEncode(long n) {
     return (n >> 63) ^ (n << 1);
   }
@@ -66,7 +57,7 @@ abstract class AbstractBlockPackedWriter
    * @param blockSize the number of values of a single block, must be a multiple of <tt>64</tt>
    */
   public AbstractBlockPackedWriter(DataOutput out, int blockSize) {
-    checkBlockSize(blockSize);
+    checkBlockSize(blockSize, MIN_BLOCK_SIZE, MAX_BLOCK_SIZE);
     reset(out);
     values = new long[blockSize];
   }

Modified: lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/util/packed/AppendingLongBuffer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/util/packed/AppendingLongBuffer.java?rev=1499601&r1=1499600&r2=1499601&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/util/packed/AppendingLongBuffer.java (original)
+++ lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/util/packed/AppendingLongBuffer.java Wed Jul  3 23:26:32 2013
@@ -27,9 +27,16 @@ import java.util.Arrays;
  */
 public final class AppendingLongBuffer extends AbstractAppendingLongBuffer {
 
-  /** Sole constructor. */
+  /** @param initialPageCount the initial number of pages
+   *  @param pageSize         the size of a single page */
+  public AppendingLongBuffer(int initialPageCount, int pageSize) {
+    super(initialPageCount, pageSize);
+  }
+
+  /** Create an {@link AppendingLongBuffer} with initialPageCount=16 and
+   *  pageSize=1024. */
   public AppendingLongBuffer() {
-    super(16);
+    this(16, 1024);
   }
 
   @Override
@@ -43,8 +50,9 @@ public final class AppendingLongBuffer e
     }
   }
 
+  @Override
   void packPendingValues() {
-    assert pendingOff == MAX_PENDING_COUNT;
+    assert pendingOff == pending.length;
 
     // compute max delta
     long minValue = pending[0];
@@ -71,6 +79,7 @@ public final class AppendingLongBuffer e
   }
 
   /** Return an iterator over the values of this buffer. */
+  @Override
   public Iterator iterator() {
     return new Iterator();
   }
@@ -78,20 +87,21 @@ public final class AppendingLongBuffer e
   /** A long iterator. */
   public final class Iterator extends AbstractAppendingLongBuffer.Iterator {
 
-    private Iterator() {
+    Iterator() {
       super();
     }
 
+    @Override
     void fillValues() {
       if (vOff == valuesOff) {
         currentValues = pending;
       } else if (deltas[vOff] == null) {
         Arrays.fill(currentValues, minValues[vOff]);
       } else {
-        for (int k = 0; k < MAX_PENDING_COUNT; ) {
-          k += deltas[vOff].get(k, currentValues, k, MAX_PENDING_COUNT - k);
+        for (int k = 0; k < pending.length; ) {
+          k += deltas[vOff].get(k, currentValues, k, pending.length - k);
         }
-        for (int k = 0; k < MAX_PENDING_COUNT; ++k) {
+        for (int k = 0; k < pending.length; ++k) {
           currentValues[k] += minValues[vOff];
         }
       }

Modified: lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/util/packed/BlockPackedReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/util/packed/BlockPackedReader.java?rev=1499601&r1=1499600&r2=1499601&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/util/packed/BlockPackedReader.java (original)
+++ lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/util/packed/BlockPackedReader.java Wed Jul  3 23:26:32 2013
@@ -17,11 +17,14 @@ package org.apache.lucene.util.packed;
  * limitations under the License.
  */
 
+import static org.apache.lucene.util.packed.AbstractBlockPackedWriter.BPV_SHIFT;
+import static org.apache.lucene.util.packed.AbstractBlockPackedWriter.MAX_BLOCK_SIZE;
+import static org.apache.lucene.util.packed.AbstractBlockPackedWriter.MIN_BLOCK_SIZE;
+import static org.apache.lucene.util.packed.AbstractBlockPackedWriter.MIN_VALUE_EQUALS_0;
 import static org.apache.lucene.util.packed.BlockPackedReaderIterator.readVLong;
 import static org.apache.lucene.util.packed.BlockPackedReaderIterator.zigZagDecode;
-import static org.apache.lucene.util.packed.BlockPackedWriter.BPV_SHIFT;
-import static org.apache.lucene.util.packed.BlockPackedWriter.MIN_VALUE_EQUALS_0;
-import static org.apache.lucene.util.packed.BlockPackedWriter.checkBlockSize;
+import static org.apache.lucene.util.packed.PackedInts.checkBlockSize;
+import static org.apache.lucene.util.packed.PackedInts.numBlocks;
 
 import java.io.IOException;
 
@@ -40,14 +43,10 @@ public final class BlockPackedReader {
 
   /** Sole constructor. */
   public BlockPackedReader(IndexInput in, int packedIntsVersion, int blockSize, long valueCount, boolean direct) throws IOException {
-    checkBlockSize(blockSize);
     this.valueCount = valueCount;
-    blockShift = Integer.numberOfTrailingZeros(blockSize);
+    blockShift = checkBlockSize(blockSize, MIN_BLOCK_SIZE, MAX_BLOCK_SIZE);
     blockMask = blockSize - 1;
-    final int numBlocks = (int) (valueCount / blockSize) + (valueCount % blockSize == 0 ? 0 : 1);
-    if ((long) numBlocks * blockSize < valueCount) {
-      throw new IllegalArgumentException("valueCount is too large for this block size");
-    }
+    final int numBlocks = numBlocks(valueCount, blockSize);
     long[] minValues = null;
     subReaders = new PackedInts.Reader[numBlocks];
     for (int i = 0; i < numBlocks; ++i) {

Modified: lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/util/packed/BlockPackedReaderIterator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/util/packed/BlockPackedReaderIterator.java?rev=1499601&r1=1499600&r2=1499601&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/util/packed/BlockPackedReaderIterator.java (original)
+++ lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/util/packed/BlockPackedReaderIterator.java Wed Jul  3 23:26:32 2013
@@ -17,9 +17,13 @@ package org.apache.lucene.util.packed;
  * limitations under the License.
  */
 
-import static org.apache.lucene.util.packed.BlockPackedWriter.BPV_SHIFT;
-import static org.apache.lucene.util.packed.BlockPackedWriter.MIN_VALUE_EQUALS_0;
-import static org.apache.lucene.util.packed.BlockPackedWriter.checkBlockSize;
+import static org.apache.lucene.util.packed.AbstractBlockPackedWriter.BPV_SHIFT;
+import static org.apache.lucene.util.packed.AbstractBlockPackedWriter.MAX_BLOCK_SIZE;
+import static org.apache.lucene.util.packed.AbstractBlockPackedWriter.MIN_BLOCK_SIZE;
+import static org.apache.lucene.util.packed.AbstractBlockPackedWriter.MIN_VALUE_EQUALS_0;
+import static org.apache.lucene.util.packed.BlockPackedReaderIterator.readVLong;
+import static org.apache.lucene.util.packed.BlockPackedReaderIterator.zigZagDecode;
+import static org.apache.lucene.util.packed.PackedInts.checkBlockSize;
 
 import java.io.EOFException;
 import java.io.IOException;
@@ -87,7 +91,7 @@ public final class BlockPackedReaderIter
    *                  been used to write the stream
    */
   public BlockPackedReaderIterator(DataInput in, int packedIntsVersion, int blockSize, long valueCount) {
-    checkBlockSize(blockSize);
+    checkBlockSize(blockSize, MIN_BLOCK_SIZE, MAX_BLOCK_SIZE);
     this.packedIntsVersion = packedIntsVersion;
     this.blockSize = blockSize;
     this.values = new long[blockSize];

Modified: lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/util/packed/GrowableWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/util/packed/GrowableWriter.java?rev=1499601&r1=1499600&r2=1499601&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/util/packed/GrowableWriter.java (original)
+++ lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/util/packed/GrowableWriter.java Wed Jul  3 23:26:32 2013
@@ -20,24 +20,35 @@ package org.apache.lucene.util.packed;
 import java.io.IOException;
 
 import org.apache.lucene.store.DataOutput;
+import org.apache.lucene.util.RamUsageEstimator;
 
 /**     
  * Implements {@link PackedInts.Mutable}, but grows the
  * bit count of the underlying packed ints on-demand.
+ * <p>Beware that this class will accept to set negative values but in order
+ * to do this, it will grow the number of bits per value to 64.
  *
  * <p>@lucene.internal</p>
  */
-
 public class GrowableWriter implements PackedInts.Mutable {
 
-  private long currentMaxValue;
+  private long currentMask;
   private PackedInts.Mutable current;
   private final float acceptableOverheadRatio;
 
+  /**
+   * @param startBitsPerValue       the initial number of bits per value, may grow depending on the data
+   * @param valueCount              the number of values
+   * @param acceptableOverheadRatio an acceptable overhead ratio
+   */
   public GrowableWriter(int startBitsPerValue, int valueCount, float acceptableOverheadRatio) {
     this.acceptableOverheadRatio = acceptableOverheadRatio;
     current = PackedInts.getMutable(valueCount, startBitsPerValue, this.acceptableOverheadRatio);
-    currentMaxValue = PackedInts.maxValue(current.getBitsPerValue());
+    currentMask = mask(current.getBitsPerValue());
+  }
+
+  private static long mask(int bitsPerValue) {
+    return bitsPerValue == 64 ? ~0L : PackedInts.maxValue(bitsPerValue);
   }
 
   @Override
@@ -70,16 +81,16 @@ public class GrowableWriter implements P
   }
 
   private void ensureCapacity(long value) {
-    assert value >= 0;
-    if (value <= currentMaxValue) {
+    if ((value & currentMask) == value) {
       return;
     }
-    final int bitsRequired = PackedInts.bitsRequired(value);
+    final int bitsRequired = value < 0 ? 64 : PackedInts.bitsRequired(value);
+    assert bitsRequired > current.getBitsPerValue();
     final int valueCount = size();
     PackedInts.Mutable next = PackedInts.getMutable(valueCount, bitsRequired, acceptableOverheadRatio);
     PackedInts.copy(current, 0, next, 0, valueCount, PackedInts.DEFAULT_BUFFER_SIZE);
     current = next;
-    currentMaxValue = PackedInts.maxValue(current.getBitsPerValue());
+    currentMask = mask(current.getBitsPerValue());
   }
 
   @Override
@@ -109,6 +120,10 @@ public class GrowableWriter implements P
   public int set(int index, long[] arr, int off, int len) {
     long max = 0;
     for (int i = off, end = off + len; i < end; ++i) {
+      // bitwise or is nice because either all values are positive and the
+      // or-ed result will require as many bits per value as the max of the
+      // values, or one of them is negative and the result will be negative,
+      // forcing GrowableWriter to use 64 bits per value
       max |= arr[i];
     }
     ensureCapacity(max);
@@ -123,7 +138,12 @@ public class GrowableWriter implements P
 
   @Override
   public long ramBytesUsed() {
-    return current.ramBytesUsed();
+    return RamUsageEstimator.alignObjectSize(
+        RamUsageEstimator.NUM_BYTES_OBJECT_HEADER
+        + RamUsageEstimator.NUM_BYTES_OBJECT_REF
+        + RamUsageEstimator.NUM_BYTES_LONG
+        + RamUsageEstimator.NUM_BYTES_FLOAT)
+        + current.ramBytesUsed();
   }
 
   @Override

Modified: lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/util/packed/MonotonicAppendingLongBuffer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/util/packed/MonotonicAppendingLongBuffer.java?rev=1499601&r1=1499600&r2=1499601&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/util/packed/MonotonicAppendingLongBuffer.java (original)
+++ lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/util/packed/MonotonicAppendingLongBuffer.java Wed Jul  3 23:26:32 2013
@@ -37,14 +37,22 @@ public final class MonotonicAppendingLon
     return (n >> 63) ^ (n << 1);
   }
 
-  private float[] averages;
+  float[] averages;
 
-  /** Sole constructor. */
+  /** @param initialPageCount the initial number of pages
+   *  @param pageSize         the size of a single page */
+  public MonotonicAppendingLongBuffer(int initialPageCount, int pageSize) {
+    super(initialPageCount, pageSize);
+    averages = new float[pending.length];
+  }
+
+  /** Create an {@link MonotonicAppendingLongBuffer} with initialPageCount=16
+   *  and pageSize=1024. */
   public MonotonicAppendingLongBuffer() {
-    super(16);
-    averages = new float[16];
+    this(16, 1024);
   }
-  
+
+  @Override
   long get(int block, int element) {
     if (block == valuesOff) {
       return pending[element];
@@ -66,16 +74,16 @@ public final class MonotonicAppendingLon
 
   @Override
   void packPendingValues() {
-    assert pendingOff == MAX_PENDING_COUNT;
+    assert pendingOff == pending.length;
 
     minValues[valuesOff] = pending[0];
-    averages[valuesOff] = (float) (pending[BLOCK_MASK] - pending[0]) / BLOCK_MASK;
+    averages[valuesOff] = (float) (pending[pending.length - 1] - pending[0]) / (pending.length - 1);
 
-    for (int i = 0; i < MAX_PENDING_COUNT; ++i) {
+    for (int i = 0; i < pending.length; ++i) {
       pending[i] = zigZagEncode(pending[i] - minValues[valuesOff] - (long) (averages[valuesOff] * (long) i));
     }
     long maxDelta = 0;
-    for (int i = 0; i < MAX_PENDING_COUNT; ++i) {
+    for (int i = 0; i < pending.length; ++i) {
       if (pending[i] < 0) {
         maxDelta = -1;
         break;
@@ -94,6 +102,7 @@ public final class MonotonicAppendingLon
   }
 
   /** Return an iterator over the values of this buffer. */
+  @Override
   public Iterator iterator() {
     return new Iterator();
   }
@@ -105,18 +114,19 @@ public final class MonotonicAppendingLon
       super();
     }
 
+    @Override
     void fillValues() {
       if (vOff == valuesOff) {
         currentValues = pending;
       } else if (deltas[vOff] == null) {
-        for (int k = 0; k < MAX_PENDING_COUNT; ++k) {
+        for (int k = 0; k < pending.length; ++k) {
           currentValues[k] = minValues[vOff] + (long) (averages[vOff] * (long) k);
         }
       } else {
-        for (int k = 0; k < MAX_PENDING_COUNT; ) {
-          k += deltas[vOff].get(k, currentValues, k, MAX_PENDING_COUNT - k);
+        for (int k = 0; k < pending.length; ) {
+          k += deltas[vOff].get(k, currentValues, k, pending.length - k);
         }
-        for (int k = 0; k < MAX_PENDING_COUNT; ++k) {
+        for (int k = 0; k < pending.length; ++k) {
           currentValues[k] = minValues[vOff] + (long) (averages[vOff] * (long) k) + zigZagDecode(currentValues[k]);
         }
       }

Modified: lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/util/packed/MonotonicBlockPackedReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/util/packed/MonotonicBlockPackedReader.java?rev=1499601&r1=1499600&r2=1499601&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/util/packed/MonotonicBlockPackedReader.java (original)
+++ lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/util/packed/MonotonicBlockPackedReader.java Wed Jul  3 23:26:32 2013
@@ -17,8 +17,11 @@ package org.apache.lucene.util.packed;
  * limitations under the License.
  */
 
-import static org.apache.lucene.util.packed.AbstractBlockPackedWriter.checkBlockSize;
+import static org.apache.lucene.util.packed.AbstractBlockPackedWriter.MAX_BLOCK_SIZE;
+import static org.apache.lucene.util.packed.AbstractBlockPackedWriter.MIN_BLOCK_SIZE;
 import static org.apache.lucene.util.packed.BlockPackedReaderIterator.zigZagDecode;
+import static org.apache.lucene.util.packed.PackedInts.checkBlockSize;
+import static org.apache.lucene.util.packed.PackedInts.numBlocks;
 
 import java.io.IOException;
 
@@ -39,14 +42,10 @@ public final class MonotonicBlockPackedR
 
   /** Sole constructor. */
   public MonotonicBlockPackedReader(IndexInput in, int packedIntsVersion, int blockSize, long valueCount, boolean direct) throws IOException {
-    checkBlockSize(blockSize);
     this.valueCount = valueCount;
-    blockShift = Integer.numberOfTrailingZeros(blockSize);
+    blockShift = checkBlockSize(blockSize, MIN_BLOCK_SIZE, MAX_BLOCK_SIZE);
     blockMask = blockSize - 1;
-    final int numBlocks = (int) (valueCount / blockSize) + (valueCount % blockSize == 0 ? 0 : 1);
-    if ((long) numBlocks * blockSize < valueCount) {
-      throw new IllegalArgumentException("valueCount is too large for this block size");
-    }
+    final int numBlocks = numBlocks(valueCount, blockSize);
     minValues = new long[numBlocks];
     averages = new float[numBlocks];
     subReaders = new PackedInts.Reader[numBlocks];

Modified: lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/util/packed/PackedInts.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/util/packed/PackedInts.java?rev=1499601&r1=1499600&r2=1499601&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/util/packed/PackedInts.java (original)
+++ lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/util/packed/PackedInts.java Wed Jul  3 23:26:32 2013
@@ -213,6 +213,11 @@ public class PackedInts {
       this.format = format;
       this.bitsPerValue = bitsPerValue;
     }
+
+    @Override
+    public String toString() {
+      return "FormatAndBits(format=" + format + " bitsPerValue=" + bitsPerValue + ")";
+    }
   }
 
   /**
@@ -1036,14 +1041,21 @@ public class PackedInts {
    */
   public static Mutable getMutable(int valueCount,
       int bitsPerValue, float acceptableOverheadRatio) {
-    assert valueCount >= 0;
-
     final FormatAndBits formatAndBits = fastestFormatAndBits(valueCount, bitsPerValue, acceptableOverheadRatio);
-    switch (formatAndBits.format) {
+    return getMutable(valueCount, formatAndBits.bitsPerValue, formatAndBits.format);
+  }
+
+  /** Same as {@link #getMutable(int, int, float)} with a pre-computed number
+   *  of bits per value and format.
+   *  @lucene.internal */
+  public static Mutable getMutable(int valueCount,
+      int bitsPerValue, PackedInts.Format format) {
+    assert valueCount >= 0;
+    switch (format) {
       case PACKED_SINGLE_BLOCK:
-        return Packed64SingleBlock.create(valueCount, formatAndBits.bitsPerValue);
+        return Packed64SingleBlock.create(valueCount, bitsPerValue);
       case PACKED:
-        switch (formatAndBits.bitsPerValue) {
+        switch (bitsPerValue) {
           case 8:
             return new Direct8(valueCount);
           case 16:
@@ -1063,7 +1075,7 @@ public class PackedInts {
             }
             break;
         }
-        return new Packed64(valueCount, formatAndBits.bitsPerValue);
+        return new Packed64(valueCount, bitsPerValue);
       default:
         throw new AssertionError();
     }
@@ -1198,33 +1210,39 @@ public class PackedInts {
       for (int i = 0; i < len; ++i) {
         dest.set(destPos++, src.get(srcPos++));
       }
-    } else {
+    } else if (len > 0) {
       // use bulk operations
-      long[] buf = new long[Math.min(capacity, len)];
-      int remaining = 0;
-      while (len > 0) {
-        final int read = src.get(srcPos, buf, remaining, Math.min(len, buf.length - remaining));
-        assert read > 0;
-        srcPos += read;
-        len -= read;
-        remaining += read;
-        final int written = dest.set(destPos, buf, 0, remaining);
-        assert written > 0;
-        destPos += written;
-        if (written < remaining) {
-          System.arraycopy(buf, written, buf, 0, remaining - written);
-        }
-        remaining -= written;
-      }
-      while (remaining > 0) {
-        final int written = dest.set(destPos, buf, 0, remaining);
-        destPos += written;
-        remaining -= written;
-        System.arraycopy(buf, written, buf, 0, remaining);
+      final long[] buf = new long[Math.min(capacity, len)];
+      copy(src, srcPos, dest, destPos, len, buf);
+    }
+  }
+
+  /** Same as {@link #copy(Reader, int, Mutable, int, int, int)} but using a pre-allocated buffer. */
+  static void copy(Reader src, int srcPos, Mutable dest, int destPos, int len, long[] buf) {
+    assert buf.length > 0;
+    int remaining = 0;
+    while (len > 0) {
+      final int read = src.get(srcPos, buf, remaining, Math.min(len, buf.length - remaining));
+      assert read > 0;
+      srcPos += read;
+      len -= read;
+      remaining += read;
+      final int written = dest.set(destPos, buf, 0, remaining);
+      assert written > 0;
+      destPos += written;
+      if (written < remaining) {
+        System.arraycopy(buf, written, buf, 0, remaining - written);
       }
+      remaining -= written;
+    }
+    while (remaining > 0) {
+      final int written = dest.set(destPos, buf, 0, remaining);
+      destPos += written;
+      remaining -= written;
+      System.arraycopy(buf, written, buf, 0, remaining);
     }
   }
-  
+
   /**
    * Expert: reads only the metadata from a stream. This is useful to later
    * restore a stream or open a direct reader via 
@@ -1261,4 +1279,26 @@ public class PackedInts {
     }    
   }
 
-}
\ No newline at end of file
+  /** Check that the block size is a power of 2, in the right bounds, and return
+   *  its log in base 2. */
+  static int checkBlockSize(int blockSize, int minBlockSize, int maxBlockSize) {
+    if (blockSize < minBlockSize || blockSize > maxBlockSize) {
+      throw new IllegalArgumentException("blockSize must be >= " + minBlockSize + " and <= " + maxBlockSize + ", got " + blockSize);
+    }
+    if ((blockSize & (blockSize - 1)) != 0) {
+      throw new IllegalArgumentException("blockSize must be a power of two, got " + blockSize);
+    }
+    return Integer.numberOfTrailingZeros(blockSize);
+  }
+
+  /** Return the number of blocks required to store <code>size</code> values on
+   *  <code>blockSize</code>. */
+  static int numBlocks(long size, int blockSize) {
+    final int numBlocks = (int) (size / blockSize) + (size % blockSize == 0 ? 0 : 1);
+    if ((long) numBlocks * blockSize < size) {
+      throw new IllegalArgumentException("size is too large for this block size");
+    }
+    return numBlocks;
+  }
+
+}

Modified: lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/util/packed/package.html
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/util/packed/package.html?rev=1499601&r1=1499600&r2=1499601&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/util/packed/package.html (original)
+++ lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/util/packed/package.html Wed Jul  3 23:26:32 2013
@@ -47,6 +47,11 @@
         <li>Same as PackedInts.Mutable but grows the number of bits per values when needed.</li>
         <li>Useful to build a PackedInts.Mutable from a read-once stream of longs.</li>
     </ul></li>
+    <li><b>{@link org.apache.lucene.util.packed.PagedGrowableWriter}</b><ul>
+        <li>Slices data into fixed-size blocks stored in GrowableWriters.</li>
+        <li>Supports more than 2B values.</li>
+        <li>You should use AppendingLongBuffer instead if you don't need random write access.</li>
+    </ul></li>
     <li><b>{@link org.apache.lucene.util.packed.AppendingLongBuffer}</b><ul>
         <li>Can store any sequence of longs.</li>
         <li>Compression is good when values are close to each other.</li>

Modified: lucene/dev/branches/security/lucene/core/src/test/org/apache/lucene/TestSearch.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/core/src/test/org/apache/lucene/TestSearch.java?rev=1499601&r1=1499600&r2=1499601&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/core/src/test/org/apache/lucene/TestSearch.java (original)
+++ lucene/dev/branches/security/lucene/core/src/test/org/apache/lucene/TestSearch.java Wed Jul  3 23:26:32 2013
@@ -112,10 +112,7 @@ public class TestSearch extends LuceneTe
       Analyzer analyzer = new MockAnalyzer(random);
       IndexWriterConfig conf = newIndexWriterConfig(TEST_VERSION_CURRENT, analyzer);
       MergePolicy mp = conf.getMergePolicy();
-      if (mp instanceof LogMergePolicy) {
-        ((LogMergePolicy) mp).setUseCompoundFile(useCompoundFile);
-      }
-      
+      mp.setNoCFSRatio(useCompoundFile ? 1.0 : 0.0);
       IndexWriter writer = new IndexWriter(directory, conf);
 
       String[] docs = {
@@ -130,7 +127,7 @@ public class TestSearch extends LuceneTe
       for (int j = 0; j < docs.length; j++) {
         Document d = new Document();
         d.add(newTextField("contents", docs[j], Field.Store.YES));
-        d.add(newStringField("id", ""+j, Field.Store.NO));
+        d.add(new IntField("id", j, Field.Store.NO));
         writer.addDocument(d);
       }
       writer.close();

Modified: lucene/dev/branches/security/lucene/core/src/test/org/apache/lucene/TestSearchForDuplicates.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/core/src/test/org/apache/lucene/TestSearchForDuplicates.java?rev=1499601&r1=1499600&r2=1499601&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/core/src/test/org/apache/lucene/TestSearchForDuplicates.java (original)
+++ lucene/dev/branches/security/lucene/core/src/test/org/apache/lucene/TestSearchForDuplicates.java Wed Jul  3 23:26:32 2013
@@ -72,9 +72,7 @@ public class TestSearchForDuplicates ext
       Analyzer analyzer = new MockAnalyzer(random);
       IndexWriterConfig conf = newIndexWriterConfig(TEST_VERSION_CURRENT, analyzer);
       final MergePolicy mp = conf.getMergePolicy();
-      if (mp instanceof LogMergePolicy) {
-        ((LogMergePolicy) mp).setUseCompoundFile(useCompoundFiles);
-      }
+      mp.setNoCFSRatio(useCompoundFiles ? 1.0 : 0.0);
       IndexWriter writer = new IndexWriter(directory, conf);
       if (VERBOSE) {
         System.out.println("TEST: now build index MAX_DOCS=" + MAX_DOCS);
@@ -83,7 +81,7 @@ public class TestSearchForDuplicates ext
       for (int j = 0; j < MAX_DOCS; j++) {
         Document d = new Document();
         d.add(newTextField(PRIORITY_FIELD, HIGH_PRIORITY, Field.Store.YES));
-        d.add(newTextField(ID_FIELD, Integer.toString(j), Field.Store.YES));
+        d.add(new IntField(ID_FIELD, j, Field.Store.YES));
         writer.addDocument(d);
       }
       writer.close();

Modified: lucene/dev/branches/security/lucene/core/src/test/org/apache/lucene/codecs/compressing/TestCompressingStoredFieldsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/core/src/test/org/apache/lucene/codecs/compressing/TestCompressingStoredFieldsFormat.java?rev=1499601&r1=1499600&r2=1499601&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/core/src/test/org/apache/lucene/codecs/compressing/TestCompressingStoredFieldsFormat.java (original)
+++ lucene/dev/branches/security/lucene/core/src/test/org/apache/lucene/codecs/compressing/TestCompressingStoredFieldsFormat.java Wed Jul  3 23:26:32 2013
@@ -49,6 +49,7 @@ public class TestCompressingStoredFields
     iwConf.setCodec(CompressingCodec.randomInstance(random()));
     // disable CFS because this test checks file names
     iwConf.setMergePolicy(newLogMergePolicy(false));
+    iwConf.setUseCompoundFile(false);
     RandomIndexWriter iw = new RandomIndexWriter(random(), dir, iwConf);
 
     final Document validDoc = new Document();