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 [5/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/index/MergePolicy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/MergePolicy.java?rev=1499601&r1=1499600&r2=1499601&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/MergePolicy.java (original)
+++ lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/MergePolicy.java Wed Jul  3 23:26:32 2013
@@ -57,7 +57,6 @@ import org.apache.lucene.util.SetOnce;
  *
  * @lucene.experimental
  */
-
 public abstract class MergePolicy implements java.io.Closeable, Cloneable {
 
   /** A map of doc IDs. */
@@ -361,9 +360,29 @@ public abstract class MergePolicy implem
       super(message);
     }
   }
+  
+  /**
+   * Default ratio for compound file system usage. Set to <tt>1.0</tt>, always use 
+   * compound file system.
+   */
+  protected static final double DEFAULT_NO_CFS_RATIO = 1.0;
+
+  /**
+   * Default max segment size in order to use compound file system. Set to {@link Long#MAX_VALUE}.
+   */
+  protected static final long DEFAULT_MAX_CFS_SEGMENT_SIZE = Long.MAX_VALUE;
 
   /** {@link IndexWriter} that contains this instance. */
   protected SetOnce<IndexWriter> writer;
+  
+  /** If the size of the merge segment exceeds this ratio of
+   *  the total index size then it will remain in
+   *  non-compound format */
+  protected double noCFSRatio = DEFAULT_NO_CFS_RATIO;
+  
+  /** If the size of the merged segment exceeds
+   *  this value then it will not use compound file format. */
+  protected long maxCFSSegmentSize = DEFAULT_MAX_CFS_SEGMENT_SIZE;
 
   @Override
   public MergePolicy clone() {
@@ -384,7 +403,18 @@ public abstract class MergePolicy implem
    * {@link #setIndexWriter(IndexWriter)}.
    */
   public MergePolicy() {
+    this(DEFAULT_NO_CFS_RATIO, DEFAULT_MAX_CFS_SEGMENT_SIZE);
+  }
+  
+  /**
+   * Creates a new merge policy instance with default settings for noCFSRatio
+   * and maxCFSSegmentSize. This ctor should be used by subclasses using different
+   * defaults than the {@link MergePolicy}
+   */
+  protected MergePolicy(double defaultNoCFSRatio, long defaultMaxCFSSegmentSize) {
     writer = new SetOnce<IndexWriter>();
+    this.noCFSRatio = defaultNoCFSRatio;
+    this.maxCFSSegmentSize = defaultMaxCFSSegmentSize;
   }
 
   /**
@@ -451,12 +481,91 @@ public abstract class MergePolicy implem
   @Override
   public abstract void close();
   
-  
   /**
-   * Returns true if a new segment (regardless of its origin) should use the compound file format.
+   * Returns true if a new segment (regardless of its origin) should use the
+   * compound file format. The default implementation returns <code>true</code>
+   * iff the size of the given mergedInfo is less or equal to
+   * {@link #getMaxCFSSegmentSizeMB()} and the size is less or equal to the
+   * TotalIndexSize * {@link #getNoCFSRatio()} otherwise <code>false</code>.
    */
-  public abstract boolean useCompoundFile(SegmentInfos segments, SegmentInfoPerCommit newSegment) throws IOException;
+  public boolean useCompoundFile(SegmentInfos infos, SegmentInfoPerCommit mergedInfo) throws IOException {
+    if (getNoCFSRatio() == 0.0) {
+      return false;
+    }
+    long mergedInfoSize = size(mergedInfo);
+    if (mergedInfoSize > maxCFSSegmentSize) {
+      return false;
+    }
+    if (getNoCFSRatio() >= 1.0) {
+      return true;
+    }
+    long totalSize = 0;
+    for (SegmentInfoPerCommit info : infos) {
+      totalSize += size(info);
+    }
+    return mergedInfoSize <= getNoCFSRatio() * totalSize;
+  }
+  
+  /** Return the byte size of the provided {@link
+   *  SegmentInfoPerCommit}, pro-rated by percentage of
+   *  non-deleted documents is set. */
+  protected long size(SegmentInfoPerCommit info) throws IOException {
+    long byteSize = info.sizeInBytes();
+    int delCount = writer.get().numDeletedDocs(info);
+    double delRatio = (info.info.getDocCount() <= 0 ? 0.0f : ((float)delCount / (float)info.info.getDocCount()));
+    assert delRatio <= 1.0;
+    return (info.info.getDocCount() <= 0 ?  byteSize : (long)(byteSize * (1.0 - delRatio)));
+  }
   
+  /** Returns true if this single info is already fully merged (has no
+   *  pending deletes, is in the same dir as the
+   *  writer, and matches the current compound file setting */
+  protected final boolean isMerged(SegmentInfoPerCommit info) {
+    IndexWriter w = writer.get();
+    assert w != null;
+    boolean hasDeletions = w.numDeletedDocs(info) > 0;
+    return !hasDeletions &&
+      info.info.dir == w.getDirectory() &&
+      ((noCFSRatio > 0.0 && noCFSRatio < 1.0) || maxCFSSegmentSize < Long.MAX_VALUE);
+  }
+  
+  /** Returns current {@code noCFSRatio}.
+   *
+   *  @see #setNoCFSRatio */
+  public final double getNoCFSRatio() {
+    return noCFSRatio;
+  }
+
+  /** If a merged segment will be more than this percentage
+   *  of the total size of the index, leave the segment as
+   *  non-compound file even if compound file is enabled.
+   *  Set to 1.0 to always use CFS regardless of merge
+   *  size. */
+  public final void setNoCFSRatio(double noCFSRatio) {
+    if (noCFSRatio < 0.0 || noCFSRatio > 1.0) {
+      throw new IllegalArgumentException("noCFSRatio must be 0.0 to 1.0 inclusive; got " + noCFSRatio);
+    }
+    this.noCFSRatio = noCFSRatio;
+  }
+
+  /** Returns the largest size allowed for a compound file segment */
+  public final double getMaxCFSSegmentSizeMB() {
+    return maxCFSSegmentSize/1024/1024.;
+  }
+
+  /** If a merged segment will be more than this value,
+   *  leave the segment as
+   *  non-compound file even if compound file is enabled.
+   *  Set this to Double.POSITIVE_INFINITY (default) and noCFSRatio to 1.0
+   *  to always use CFS regardless of merge size. */
+  public final void setMaxCFSSegmentSizeMB(double v) {
+    if (v < 0.0) {
+      throw new IllegalArgumentException("maxCFSSegmentSizeMB must be >=0 (got " + v + ")");
+    }
+    v *= 1024 * 1024;
+    this.maxCFSSegmentSize = (v > Long.MAX_VALUE) ? Long.MAX_VALUE : (long) v;
+  }
+
   /**
    * MergeTrigger is passed to
    * {@link MergePolicy#findMerges(MergeTrigger, SegmentInfos)} to indicate the

Modified: lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/MergeScheduler.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/MergeScheduler.java?rev=1499601&r1=1499600&r2=1499601&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/MergeScheduler.java (original)
+++ lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/MergeScheduler.java Wed Jul  3 23:26:32 2013
@@ -24,7 +24,9 @@ import java.io.IOException;
  *  implementing this interface to execute the merges
  *  selected by a {@link MergePolicy}.  The default
  *  MergeScheduler is {@link ConcurrentMergeScheduler}.</p>
- *
+ *  <p>Implementers of sub-classes should make sure that {@link #clone()}
+ *  returns an independent instance able to work with any {@link IndexWriter}
+ *  instance.</p>
  * @lucene.experimental
 */
 public abstract class MergeScheduler implements Closeable, Cloneable {

Modified: lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/NoMergePolicy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/NoMergePolicy.java?rev=1499601&r1=1499600&r2=1499601&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/NoMergePolicy.java (original)
+++ lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/NoMergePolicy.java Wed Jul  3 23:26:32 2013
@@ -20,8 +20,6 @@ package org.apache.lucene.index;
 import java.io.IOException;
 import java.util.Map;
 
-import org.apache.lucene.index.MergePolicy.MergeTrigger;
-import org.apache.lucene.index.MergePolicy.MergeSpecification;
 
 /**
  * A {@link MergePolicy} which never returns merges to execute (hence it's
@@ -49,6 +47,7 @@ public final class NoMergePolicy extends
   private final boolean useCompoundFile;
   
   private NoMergePolicy(boolean useCompoundFile) {
+    super(useCompoundFile ? 1.0 : 0.0, 0);
     // prevent instantiation
     this.useCompoundFile = useCompoundFile;
   }
@@ -71,6 +70,11 @@ public final class NoMergePolicy extends
 
   @Override
   public void setIndexWriter(IndexWriter writer) {}
+  
+  @Override
+  protected long size(SegmentInfoPerCommit info) throws IOException {
+      return Long.MAX_VALUE;
+  }
 
   @Override
   public String toString() {

Modified: lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java?rev=1499601&r1=1499600&r2=1499601&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java (original)
+++ lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java Wed Jul  3 23:26:32 2013
@@ -17,7 +17,6 @@ package org.apache.lucene.index;
  * limitations under the License.
  */
 
-import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.PrintStream;
 import java.util.ArrayList;
@@ -613,10 +612,6 @@ public final class SegmentInfos implemen
           IndexInput genInput = null;
           try {
             genInput = directory.openInput(IndexFileNames.SEGMENTS_GEN, IOContext.READONCE);
-          } catch (FileNotFoundException e) {
-            if (infoStream != null) {
-              message("segments.gen open: FileNotFoundException " + e);
-            }
           } catch (IOException e) {
             if (infoStream != null) {
               message("segments.gen open: IOException " + e);

Modified: lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/StoredFieldsProcessor.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/StoredFieldsProcessor.java?rev=1499601&r1=1499600&r2=1499601&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/StoredFieldsProcessor.java (original)
+++ lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/StoredFieldsProcessor.java Wed Jul  3 23:26:32 2013
@@ -18,11 +18,13 @@ package org.apache.lucene.index;
  */
 
 import java.io.IOException;
+import java.util.Arrays;
 
 import org.apache.lucene.codecs.Codec;
 import org.apache.lucene.codecs.StoredFieldsWriter;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.RamUsageEstimator;
 
 /** This is a StoredFieldsConsumer that writes stored fields. */
@@ -32,8 +34,6 @@ final class StoredFieldsProcessor extend
   final DocumentsWriterPerThread docWriter;
   int lastDocID;
 
-  int freeCount;
-
   final DocumentsWriterPerThread.DocState docState;
   final Codec codec;
 
@@ -44,13 +44,13 @@ final class StoredFieldsProcessor extend
   }
 
   private int numStoredFields;
-  private StorableField[] storedFields;
-  private FieldInfo[] fieldInfos;
+  private StorableField[] storedFields = new StorableField[1];
+  private FieldInfo[] fieldInfos = new FieldInfo[1];
 
   public void reset() {
     numStoredFields = 0;
-    storedFields = new StorableField[1];
-    fieldInfos = new FieldInfo[1];
+    Arrays.fill(storedFields, null);
+    Arrays.fill(fieldInfos, null);
   }
   
   @Override
@@ -61,7 +61,6 @@ final class StoredFieldsProcessor extend
   @Override
   public void flush(SegmentWriteState state) throws IOException {
     int numDocs = state.segmentInfo.getDocCount();
-
     if (numDocs > 0) {
       // It's possible that all documents seen in this segment
       // hit non-aborting exceptions, in which case we will
@@ -69,14 +68,17 @@ final class StoredFieldsProcessor extend
       initFieldsWriter(state.context);
       fill(numDocs);
     }
-
     if (fieldsWriter != null) {
-      try {
-        fieldsWriter.finish(state.fieldInfos, numDocs);
-      } finally {
-        fieldsWriter.close();
-        fieldsWriter = null;
-        lastDocID = 0;
+        boolean success = false;
+        try {
+          fieldsWriter.finish(state.fieldInfos, numDocs);
+          success = true;
+        } finally {
+          if (success) {
+            IOUtils.close(fieldsWriter);
+          } else {
+            IOUtils.closeWhileHandlingException(fieldsWriter);
+          }
       }
     }
   }
@@ -88,7 +90,6 @@ final class StoredFieldsProcessor extend
     }
   }
 
-  int allocCount;
 
   @Override
   void abort() {
@@ -114,7 +115,7 @@ final class StoredFieldsProcessor extend
 
   @Override
   void finishDocument() throws IOException {
-    assert docWriter.writer.testPoint("StoredFieldsWriter.finishDocument start");
+    assert docWriter.testPoint("StoredFieldsWriter.finishDocument start");
 
     initFieldsWriter(IOContext.DEFAULT);
     fill(docState.docID);
@@ -129,7 +130,7 @@ final class StoredFieldsProcessor extend
     }
 
     reset();
-    assert docWriter.writer.testPoint("StoredFieldsWriter.finishDocument end");
+    assert docWriter.testPoint("StoredFieldsWriter.finishDocument end");
   }
 
   @Override

Modified: lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/TermVectorsConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/TermVectorsConsumer.java?rev=1499601&r1=1499600&r2=1499601&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/TermVectorsConsumer.java (original)
+++ lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/TermVectorsConsumer.java Wed Jul  3 23:26:32 2013
@@ -18,6 +18,7 @@ package org.apache.lucene.index;
  */
 
 import java.io.IOException;
+import java.util.Arrays;
 import java.util.Map;
 
 import org.apache.lucene.codecs.TermVectorsWriter;
@@ -32,9 +33,6 @@ final class TermVectorsConsumer extends 
 
   TermVectorsWriter writer;
   final DocumentsWriterPerThread docWriter;
-  int freeCount;
-  int lastDocID;
-
   final DocumentsWriterPerThread.DocState docState;
   final BytesRef flushTerm = new BytesRef();
 
@@ -42,6 +40,9 @@ final class TermVectorsConsumer extends 
   final ByteSliceReader vectorSliceReaderPos = new ByteSliceReader();
   final ByteSliceReader vectorSliceReaderOff = new ByteSliceReader();
   boolean hasVectors;
+  int numVectorFields;
+  int lastDocID;
+  private TermVectorsConsumerPerField[] perFields = new TermVectorsConsumerPerField[1];
 
   public TermVectorsConsumer(DocumentsWriterPerThread docWriter) {
     this.docWriter = docWriter;
@@ -52,6 +53,7 @@ final class TermVectorsConsumer extends 
   void flush(Map<String, TermsHashConsumerPerField> fieldsToFlush, final SegmentWriteState state) throws IOException {
     if (writer != null) {
       int numDocs = state.segmentInfo.getDocCount();
+      assert numDocs > 0;
       // At least one doc in this run had term vectors enabled
       try {
         fill(numDocs);
@@ -60,7 +62,6 @@ final class TermVectorsConsumer extends 
       } finally {
         IOUtils.close(writer);
         writer = null;
-
         lastDocID = 0;
         hasVectors = false;
       }
@@ -94,7 +95,7 @@ final class TermVectorsConsumer extends 
   @Override
   void finishDocument(TermsHash termsHash) throws IOException {
 
-    assert docWriter.writer.testPoint("TermVectorsTermsWriter.finishDocument start");
+    assert docWriter.testPoint("TermVectorsTermsWriter.finishDocument start");
 
     if (!hasVectors) {
       return;
@@ -117,7 +118,7 @@ final class TermVectorsConsumer extends 
 
     termsHash.reset();
     reset();
-    assert docWriter.writer.testPoint("TermVectorsTermsWriter.finishDocument end");
+    assert docWriter.testPoint("TermVectorsTermsWriter.finishDocument end");
   }
 
   @Override
@@ -130,17 +131,12 @@ final class TermVectorsConsumer extends 
     }
 
     lastDocID = 0;
-
     reset();
   }
 
-  int numVectorFields;
-
-  TermVectorsConsumerPerField[] perFields;
-
   void reset() {
+    Arrays.fill(perFields, null);// don't hang onto stuff from previous doc
     numVectorFields = 0;
-    perFields = new TermVectorsConsumerPerField[1];
   }
 
   @Override
@@ -175,10 +171,7 @@ final class TermVectorsConsumer extends 
   String lastVectorFieldName;
   final boolean vectorFieldsInOrder(FieldInfo fi) {
     try {
-      if (lastVectorFieldName != null)
-        return lastVectorFieldName.compareTo(fi.name) < 0;
-      else
-        return true;
+      return lastVectorFieldName != null ? lastVectorFieldName.compareTo(fi.name) < 0 : true; 
     } finally {
       lastVectorFieldName = fi.name;
     }

Modified: lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/TieredMergePolicy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/TieredMergePolicy.java?rev=1499601&r1=1499600&r2=1499601&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/TieredMergePolicy.java (original)
+++ lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/index/TieredMergePolicy.java Wed Jul  3 23:26:32 2013
@@ -27,9 +27,6 @@ import java.util.Comparator;
 import java.util.List;
 import java.util.ArrayList;
 
-import org.apache.lucene.index.MergePolicy.MergeTrigger;
-
-
 /**
  *  Merges segments of approximately equal size, subject to
  *  an allowed number of segments per tier.  This is similar
@@ -76,7 +73,11 @@ import org.apache.lucene.index.MergePoli
 //     maybe CMS should do so)
 
 public class TieredMergePolicy extends MergePolicy {
-
+  /** Default noCFSRatio.  If a merge's size is >= 10% of
+   *  the index, then we disable compound file for it.
+   *  @see MergePolicy#setNoCFSRatio */
+  public static final double DEFAULT_NO_CFS_RATIO = 0.1;
+  
   private int maxMergeAtOnce = 10;
   private long maxMergedSegmentBytes = 5*1024*1024*1024L;
   private int maxMergeAtOnceExplicit = 30;
@@ -84,14 +85,12 @@ public class TieredMergePolicy extends M
   private long floorSegmentBytes = 2*1024*1024L;
   private double segsPerTier = 10.0;
   private double forceMergeDeletesPctAllowed = 10.0;
-  private boolean useCompoundFile = true;
-  private double noCFSRatio = 0.1;
-  private long maxCFSSegmentSize = Long.MAX_VALUE;
   private double reclaimDeletesWeight = 2.0;
 
   /** Sole constructor, setting all settings to their
    *  defaults. */
   public TieredMergePolicy() {
+    super(DEFAULT_NO_CFS_RATIO, MergePolicy.DEFAULT_MAX_CFS_SEGMENT_SIZE);
   }
 
   /** Maximum number of segments to be merged at a time
@@ -233,41 +232,6 @@ public class TieredMergePolicy extends M
     return segsPerTier;
   }
 
-  /** Sets whether compound file format should be used for
-   *  newly flushed and newly merged segments.  Default
-   *  true. */
-  public TieredMergePolicy setUseCompoundFile(boolean useCompoundFile) {
-    this.useCompoundFile = useCompoundFile;
-    return this;
-  }
-
-  /** Returns the current useCompoundFile setting.
-   *
-   * @see  #setUseCompoundFile */
-  public boolean getUseCompoundFile() {
-    return useCompoundFile;
-  }
-
-  /** If a merged segment will be more than this percentage
-   *  of the total size of the index, leave the segment as
-   *  non-compound file even if compound file is enabled.
-   *  Set to 1.0 to always use CFS regardless of merge
-   *  size.  Default is 0.1. */
-  public TieredMergePolicy setNoCFSRatio(double noCFSRatio) {
-    if (noCFSRatio < 0.0 || noCFSRatio > 1.0) {
-      throw new IllegalArgumentException("noCFSRatio must be 0.0 to 1.0 inclusive; got " + noCFSRatio);
-    }
-    this.noCFSRatio = noCFSRatio;
-    return this;
-  }
-  
-  /** Returns the current noCFSRatio setting.
-   *
-   * @see #setNoCFSRatio */
-  public double getNoCFSRatio() {
-    return noCFSRatio;
-  }
-
   private class SegmentByteSizeDescending implements Comparator<SegmentInfoPerCommit> {
     @Override
     public int compare(SegmentInfoPerCommit o1, SegmentInfoPerCommit o2) {
@@ -637,46 +601,9 @@ public class TieredMergePolicy extends M
   }
 
   @Override
-  public boolean useCompoundFile(SegmentInfos infos, SegmentInfoPerCommit mergedInfo) throws IOException {
-    if (!getUseCompoundFile()) {
-        return false;
-    }
-    long mergedInfoSize = size(mergedInfo);
-    if (mergedInfoSize > maxCFSSegmentSize) {
-        return false;
-    }
-    if (getNoCFSRatio() >= 1.0) {
-        return true;
-    }
-    long totalSize = 0;
-    for (SegmentInfoPerCommit info : infos) {
-        totalSize += size(info);
-    }
-    return mergedInfoSize <= getNoCFSRatio() * totalSize;
-  }
-
-  @Override
   public void close() {
   }
 
-  private boolean isMerged(SegmentInfoPerCommit info) {
-    IndexWriter w = writer.get();
-    assert w != null;
-    boolean hasDeletions = w.numDeletedDocs(info) > 0;
-    return !hasDeletions &&
-      info.info.dir == w.getDirectory() &&
-      (info.info.getUseCompoundFile() == useCompoundFile || noCFSRatio < 1.0 || maxCFSSegmentSize < Long.MAX_VALUE);
-  }
-
-  // Segment size in bytes, pro-rated by % deleted
-  private long size(SegmentInfoPerCommit info) throws IOException {
-    final long byteSize = info.sizeInBytes();    
-    final int delCount = writer.get().numDeletedDocs(info);
-    final double delRatio = (info.info.getDocCount() <= 0 ? 0.0f : ((double)delCount / (double)info.info.getDocCount()));    
-    assert delRatio <= 1.0;
-    return (long) (byteSize * (1.0-delRatio));
-  }
-
   private long floorSize(long bytes) {
     return Math.max(floorSegmentBytes, bytes);
   }
@@ -699,28 +626,8 @@ public class TieredMergePolicy extends M
     sb.append("floorSegmentMB=").append(floorSegmentBytes/1024/1024.).append(", ");
     sb.append("forceMergeDeletesPctAllowed=").append(forceMergeDeletesPctAllowed).append(", ");
     sb.append("segmentsPerTier=").append(segsPerTier).append(", ");
-    sb.append("useCompoundFile=").append(useCompoundFile).append(", ");
     sb.append("maxCFSSegmentSizeMB=").append(getMaxCFSSegmentSizeMB()).append(", ");
     sb.append("noCFSRatio=").append(noCFSRatio);
     return sb.toString();
   }
-
-  /** Returns the largest size allowed for a compound file segment */
-  public final double getMaxCFSSegmentSizeMB() {
-    return maxCFSSegmentSize/1024/1024.;
-  }
-
-  /** If a merged segment will be more than this value,
-   *  leave the segment as
-   *  non-compound file even if compound file is enabled.
-   *  Set this to Double.POSITIVE_INFINITY (default) and noCFSRatio to 1.0
-   *  to always use CFS regardless of merge size. */
-  public final TieredMergePolicy setMaxCFSSegmentSizeMB(double v) {
-    if (v < 0.0) {
-      throw new IllegalArgumentException("maxCFSSegmentSizeMB must be >=0 (got " + v + ")");
-    }
-    v *= 1024 * 1024;
-    this.maxCFSSegmentSize = (v > Long.MAX_VALUE) ? Long.MAX_VALUE : (long) v;
-    return this;
-  }
 }

Modified: lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/ExactPhraseScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/ExactPhraseScorer.java?rev=1499601&r1=1499600&r2=1499601&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/ExactPhraseScorer.java (original)
+++ lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/ExactPhraseScorer.java Wed Jul  3 23:26:32 2013
@@ -56,10 +56,10 @@ final class ExactPhraseScorer extends Sc
   private int docID = -1;
   private int freq;
 
-  private final Similarity.ExactSimScorer docScorer;
+  private final Similarity.SimScorer docScorer;
   
   ExactPhraseScorer(Weight weight, PhraseQuery.PostingsAndFreq[] postings,
-                    Similarity.ExactSimScorer docScorer) throws IOException {
+                    Similarity.SimScorer docScorer) throws IOException {
     super(weight);
     this.docScorer = docScorer;
 

Modified: lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/FieldCache.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/FieldCache.java?rev=1499601&r1=1499600&r2=1499601&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/FieldCache.java (original)
+++ lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/FieldCache.java Wed Jul  3 23:26:32 2013
@@ -49,34 +49,6 @@ import org.apache.lucene.util.RamUsageEs
  */
 public interface FieldCache {
 
-  /** Field values as 8-bit signed bytes */
-  public static abstract class Bytes {
-    /** Return a single Byte representation of this field's value. */
-    public abstract byte get(int docID);
-    
-    /** Zero value for every document */
-    public static final Bytes EMPTY = new Bytes() {
-      @Override
-      public byte get(int docID) {
-        return 0;
-      }
-    };
-  }
-
-  /** Field values as 16-bit signed shorts */
-  public static abstract class Shorts {
-    /** Return a short representation of this field's value. */
-    public abstract short get(int docID);
-    
-    /** Zero value for every document */
-    public static final Shorts EMPTY = new Shorts() {
-      @Override
-      public short get(int docID) {
-        return 0;
-      }
-    };
-  }
-
   /** Field values as 32-bit signed integers */
   public static abstract class Ints {
     /** Return an integer representation of this field's value. */
@@ -178,22 +150,6 @@ public interface FieldCache {
     public TermsEnum termsEnum(Terms terms) throws IOException;
   }
 
-  /** Interface to parse bytes from document fields.
-   * @see FieldCache#getBytes(AtomicReader, String, FieldCache.ByteParser, boolean)
-   */
-  public interface ByteParser extends Parser {
-    /** Return a single Byte representation of this field's value. */
-    public byte parseByte(BytesRef term);
-  }
-
-  /** Interface to parse shorts from document fields.
-   * @see FieldCache#getShorts(AtomicReader, String, FieldCache.ShortParser, boolean)
-   */
-  public interface ShortParser extends Parser {
-    /** Return a short representation of this field's value. */
-    public short parseShort(BytesRef term);
-  }
-
   /** Interface to parse ints from document fields.
    * @see FieldCache#getInts(AtomicReader, String, FieldCache.IntParser, boolean)
    */
@@ -229,135 +185,6 @@ public interface FieldCache {
   /** Expert: The cache used internally by sorting and range query classes. */
   public static FieldCache DEFAULT = new FieldCacheImpl();
 
-  /** The default parser for byte values, which are encoded by {@link Byte#toString(byte)} */
-  public static final ByteParser DEFAULT_BYTE_PARSER = new ByteParser() {
-    @Override
-    public byte parseByte(BytesRef term) {
-      // TODO: would be far better to directly parse from
-      // UTF8 bytes... but really users should use
-      // IntField, instead, which already decodes
-      // directly from byte[]
-      return Byte.parseByte(term.utf8ToString());
-    }
-    @Override
-    public String toString() { 
-      return FieldCache.class.getName()+".DEFAULT_BYTE_PARSER"; 
-    }
-    @Override
-    public TermsEnum termsEnum(Terms terms) throws IOException {
-      return terms.iterator(null);
-    }
-  };
-
-  /** The default parser for short values, which are encoded by {@link Short#toString(short)} */
-  public static final ShortParser DEFAULT_SHORT_PARSER = new ShortParser() {
-    @Override
-    public short parseShort(BytesRef term) {
-      // TODO: would be far better to directly parse from
-      // UTF8 bytes... but really users should use
-      // IntField, instead, which already decodes
-      // directly from byte[]
-      return Short.parseShort(term.utf8ToString());
-    }
-    @Override
-    public String toString() { 
-      return FieldCache.class.getName()+".DEFAULT_SHORT_PARSER"; 
-    }
-    
-    @Override
-    public TermsEnum termsEnum(Terms terms) throws IOException {
-      return terms.iterator(null);
-    }
-  };
-
-  /** The default parser for int values, which are encoded by {@link Integer#toString(int)} */
-  public static final IntParser DEFAULT_INT_PARSER = new IntParser() {
-    @Override
-    public int parseInt(BytesRef term) {
-      // TODO: would be far better to directly parse from
-      // UTF8 bytes... but really users should use
-      // IntField, instead, which already decodes
-      // directly from byte[]
-      return Integer.parseInt(term.utf8ToString());
-    }
-    
-    @Override
-    public TermsEnum termsEnum(Terms terms) throws IOException {
-      return terms.iterator(null);
-    }
-    
-    @Override
-    public String toString() { 
-      return FieldCache.class.getName()+".DEFAULT_INT_PARSER"; 
-    }
-  };
-
-  /** The default parser for float values, which are encoded by {@link Float#toString(float)} */
-  public static final FloatParser DEFAULT_FLOAT_PARSER = new FloatParser() {
-    @Override
-    public float parseFloat(BytesRef term) {
-      // TODO: would be far better to directly parse from
-      // UTF8 bytes... but really users should use
-      // FloatField, instead, which already decodes
-      // directly from byte[]
-      return Float.parseFloat(term.utf8ToString());
-    }
-    
-    @Override
-    public TermsEnum termsEnum(Terms terms) throws IOException {
-      return terms.iterator(null);
-    }
-    
-    @Override
-    public String toString() { 
-      return FieldCache.class.getName()+".DEFAULT_FLOAT_PARSER"; 
-    }
-  };
-
-  /** The default parser for long values, which are encoded by {@link Long#toString(long)} */
-  public static final LongParser DEFAULT_LONG_PARSER = new LongParser() {
-    @Override
-    public long parseLong(BytesRef term) {
-      // TODO: would be far better to directly parse from
-      // UTF8 bytes... but really users should use
-      // LongField, instead, which already decodes
-      // directly from byte[]
-      return Long.parseLong(term.utf8ToString());
-    }
-    
-    @Override
-    public TermsEnum termsEnum(Terms terms) throws IOException {
-      return terms.iterator(null);
-    }
-    
-    @Override
-    public String toString() { 
-      return FieldCache.class.getName()+".DEFAULT_LONG_PARSER"; 
-    }
-  };
-
-  /** The default parser for double values, which are encoded by {@link Double#toString(double)} */
-  public static final DoubleParser DEFAULT_DOUBLE_PARSER = new DoubleParser() {
-    @Override
-    public double parseDouble(BytesRef term) {
-      // TODO: would be far better to directly parse from
-      // UTF8 bytes... but really users should use
-      // DoubleField, instead, which already decodes
-      // directly from byte[]
-      return Double.parseDouble(term.utf8ToString());
-    }
-    
-    @Override
-    public TermsEnum termsEnum(Terms terms) throws IOException {
-      return terms.iterator(null);
-    }
-    
-    @Override
-    public String toString() { 
-      return FieldCache.class.getName()+".DEFAULT_DOUBLE_PARSER"; 
-    }
-  };
-
   /**
    * A parser instance for int values encoded by {@link NumericUtils}, e.g. when indexed
    * via {@link IntField}/{@link NumericTokenStream}.
@@ -450,60 +277,6 @@ public interface FieldCache {
   public Bits getDocsWithField(AtomicReader reader, String field) throws IOException;
 
   /** Checks the internal cache for an appropriate entry, and if none is
-   * found, reads the terms in <code>field</code> as a single byte and returns an array
-   * of size <code>reader.maxDoc()</code> of the value each document
-   * has in the given field.
-   * @param reader  Used to get field values.
-   * @param field   Which field contains the single byte values.
-   * @param setDocsWithField  If true then {@link #getDocsWithField} will
-   *        also be computed and stored in the FieldCache.
-   * @return The values in the given field for each document.
-   * @throws IOException  If any error occurs.
-   */
-  public Bytes getBytes(AtomicReader reader, String field, boolean setDocsWithField) throws IOException;
-
-  /** Checks the internal cache for an appropriate entry, and if none is found,
-   * reads the terms in <code>field</code> as bytes and returns an array of
-   * size <code>reader.maxDoc()</code> of the value each document has in the
-   * given field.
-   * @param reader  Used to get field values.
-   * @param field   Which field contains the bytes.
-   * @param parser  Computes byte for string values.
-   * @param setDocsWithField  If true then {@link #getDocsWithField} will
-   *        also be computed and stored in the FieldCache.
-   * @return The values in the given field for each document.
-   * @throws IOException  If any error occurs.
-   */
-  public Bytes getBytes(AtomicReader reader, String field, ByteParser parser, boolean setDocsWithField) throws IOException;
-
-  /** Checks the internal cache for an appropriate entry, and if none is
-   * found, reads the terms in <code>field</code> as shorts and returns an array
-   * of size <code>reader.maxDoc()</code> of the value each document
-   * has in the given field.
-   * @param reader  Used to get field values.
-   * @param field   Which field contains the shorts.
-   * @param setDocsWithField  If true then {@link #getDocsWithField} will
-   *        also be computed and stored in the FieldCache.
-   * @return The values in the given field for each document.
-   * @throws IOException  If any error occurs.
-   */
-  public Shorts getShorts (AtomicReader reader, String field, boolean setDocsWithField) throws IOException;
-
-  /** Checks the internal cache for an appropriate entry, and if none is found,
-   * reads the terms in <code>field</code> as shorts and returns an array of
-   * size <code>reader.maxDoc()</code> of the value each document has in the
-   * given field.
-   * @param reader  Used to get field values.
-   * @param field   Which field contains the shorts.
-   * @param parser  Computes short for string values.
-   * @param setDocsWithField  If true then {@link #getDocsWithField} will
-   *        also be computed and stored in the FieldCache.
-   * @return The values in the given field for each document.
-   * @throws IOException  If any error occurs.
-   */
-  public Shorts getShorts (AtomicReader reader, String field, ShortParser parser, boolean setDocsWithField) throws IOException;
-  
-  /** Checks the internal cache for an appropriate entry, and if none is
    * found, reads the terms in <code>field</code> as integers and returns an array
    * of size <code>reader.maxDoc()</code> of the value each document
    * has in the given field.

Modified: lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/FieldCacheImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/FieldCacheImpl.java?rev=1499601&r1=1499600&r2=1499601&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/FieldCacheImpl.java (original)
+++ lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/FieldCacheImpl.java Wed Jul  3 23:26:32 2013
@@ -38,13 +38,13 @@ import org.apache.lucene.index.SortedDoc
 import org.apache.lucene.index.SortedSetDocValues;
 import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.TermsEnum;
-import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.FieldCacheSanityChecker;
 import org.apache.lucene.util.FixedBitSet;
 import org.apache.lucene.util.PagedBytes;
 import org.apache.lucene.util.packed.GrowableWriter;
+import org.apache.lucene.util.packed.MonotonicAppendingLongBuffer;
 import org.apache.lucene.util.packed.PackedInts;
 
 /**
@@ -62,8 +62,6 @@ class FieldCacheImpl implements FieldCac
 
   private synchronized void init() {
     caches = new HashMap<Class<?>,Cache>(9);
-    caches.put(Byte.TYPE, new ByteCache(this));
-    caches.put(Short.TYPE, new ShortCache(this));
     caches.put(Integer.TYPE, new IntCache(this));
     caches.put(Float.TYPE, new FloatCache(this));
     caches.put(Long.TYPE, new LongCache(this));
@@ -354,192 +352,6 @@ class FieldCacheImpl implements FieldCac
     }
     caches.get(DocsWithFieldCache.class).put(reader, new CacheKey(field, null), bits);
   }
-  
-  // inherit javadocs
-  public Bytes getBytes (AtomicReader reader, String field, boolean setDocsWithField) throws IOException {
-    return getBytes(reader, field, null, setDocsWithField);
-  }
-
-  // inherit javadocs
-  public Bytes getBytes(AtomicReader reader, String field, ByteParser parser, boolean setDocsWithField)
-      throws IOException {
-    final NumericDocValues valuesIn = reader.getNumericDocValues(field);
-    if (valuesIn != null) {
-      // Not cached here by FieldCacheImpl (cached instead
-      // per-thread by SegmentReader):
-      return new Bytes() {
-        @Override
-        public byte get(int docID) {
-          return (byte) valuesIn.get(docID);
-        }
-      };
-    } else {
-      final FieldInfo info = reader.getFieldInfos().fieldInfo(field);
-      if (info == null) {
-        return Bytes.EMPTY;
-      } else if (info.hasDocValues()) {
-        throw new IllegalStateException("Type mismatch: " + field + " was indexed as " + info.getDocValuesType());
-      } else if (!info.isIndexed()) {
-        return Bytes.EMPTY;
-      }
-      return (Bytes) caches.get(Byte.TYPE).get(reader, new CacheKey(field, parser), setDocsWithField);
-    }
-  }
-
-  static class BytesFromArray extends Bytes {
-    private final byte[] values;
-
-    public BytesFromArray(byte[] values) {
-      this.values = values;
-    }
-    
-    @Override
-    public byte get(int docID) {
-      return values[docID];
-    }
-  }
-
-  static final class ByteCache extends Cache {
-    ByteCache(FieldCacheImpl wrapper) {
-      super(wrapper);
-    }
-
-    @Override
-    protected Object createValue(AtomicReader reader, CacheKey key, boolean setDocsWithField)
-        throws IOException {
-
-      int maxDoc = reader.maxDoc();
-      final byte[] values;
-      final ByteParser parser = (ByteParser) key.custom;
-      if (parser == null) {
-        // Confusing: must delegate to wrapper (vs simply
-        // setting parser = DEFAULT_SHORT_PARSER) so cache
-        // key includes DEFAULT_SHORT_PARSER:
-        return wrapper.getBytes(reader, key.field, DEFAULT_BYTE_PARSER, setDocsWithField);
-      }
-
-      values = new byte[maxDoc];
-
-      Uninvert u = new Uninvert() {
-          private byte currentValue;
-
-          @Override
-          public void visitTerm(BytesRef term) {
-            currentValue = parser.parseByte(term);
-          }
-
-          @Override
-          public void visitDoc(int docID) {
-            values[docID] = currentValue;
-          }
-
-          @Override
-          protected TermsEnum termsEnum(Terms terms) throws IOException {
-            return parser.termsEnum(terms);
-          }
-        };
-
-      u.uninvert(reader, key.field, setDocsWithField);
-
-      if (setDocsWithField) {
-        wrapper.setDocsWithField(reader, key.field, u.docsWithField);
-      }
-
-      return new BytesFromArray(values);
-    }
-  }
-  
-  // inherit javadocs
-  public Shorts getShorts (AtomicReader reader, String field, boolean setDocsWithField) throws IOException {
-    return getShorts(reader, field, null, setDocsWithField);
-  }
-
-  // inherit javadocs
-  public Shorts getShorts(AtomicReader reader, String field, ShortParser parser, boolean setDocsWithField)
-      throws IOException {
-    final NumericDocValues valuesIn = reader.getNumericDocValues(field);
-    if (valuesIn != null) {
-      // Not cached here by FieldCacheImpl (cached instead
-      // per-thread by SegmentReader):
-      return new Shorts() {
-        @Override
-        public short get(int docID) {
-          return (short) valuesIn.get(docID);
-        }
-      };
-    } else {
-      final FieldInfo info = reader.getFieldInfos().fieldInfo(field);
-      if (info == null) {
-        return Shorts.EMPTY;
-      } else if (info.hasDocValues()) {
-        throw new IllegalStateException("Type mismatch: " + field + " was indexed as " + info.getDocValuesType());
-      } else if (!info.isIndexed()) {
-        return Shorts.EMPTY;
-      }
-      return (Shorts) caches.get(Short.TYPE).get(reader, new CacheKey(field, parser), setDocsWithField);
-    }
-  }
-
-  static class ShortsFromArray extends Shorts {
-    private final short[] values;
-
-    public ShortsFromArray(short[] values) {
-      this.values = values;
-    }
-    
-    @Override
-    public short get(int docID) {
-      return values[docID];
-    }
-  }
-
-  static final class ShortCache extends Cache {
-    ShortCache(FieldCacheImpl wrapper) {
-      super(wrapper);
-    }
-
-    @Override
-    protected Object createValue(AtomicReader reader, CacheKey key, boolean setDocsWithField)
-        throws IOException {
-
-      int maxDoc = reader.maxDoc();
-      final short[] values;
-      final ShortParser parser = (ShortParser) key.custom;
-      if (parser == null) {
-        // Confusing: must delegate to wrapper (vs simply
-        // setting parser = DEFAULT_SHORT_PARSER) so cache
-        // key includes DEFAULT_SHORT_PARSER:
-        return wrapper.getShorts(reader, key.field, DEFAULT_SHORT_PARSER, setDocsWithField);
-      }
-
-      values = new short[maxDoc];
-      Uninvert u = new Uninvert() {
-          private short currentValue;
-
-          @Override
-          public void visitTerm(BytesRef term) {
-            currentValue = parser.parseShort(term);
-          }
-
-          @Override
-          public void visitDoc(int docID) {
-            values[docID] = currentValue;
-          }
-          
-          @Override
-          protected TermsEnum termsEnum(Terms terms) throws IOException {
-            return parser.termsEnum(terms);
-          }
-        };
-
-      u.uninvert(reader, key.field, setDocsWithField);
-
-      if (setDocsWithField) {
-        wrapper.setDocsWithField(reader, key.field, u.docsWithField);
-      }
-      return new ShortsFromArray(values);
-    }
-  }
 
   // inherit javadocs
   public Ints getInts (AtomicReader reader, String field, boolean setDocsWithField) throws IOException {
@@ -573,15 +385,19 @@ class FieldCacheImpl implements FieldCac
   }
 
   static class IntsFromArray extends Ints {
-    private final int[] values;
+    private final PackedInts.Reader values;
+    private final int minValue;
 
-    public IntsFromArray(int[] values) {
+    public IntsFromArray(PackedInts.Reader values, int minValue) {
+      assert values.getBitsPerValue() <= 32;
       this.values = values;
+      this.minValue = minValue;
     }
     
     @Override
     public int get(int docID) {
-      return values[docID];
+      final long delta = values.get(docID);
+      return minValue + (int) delta;
     }
   }
 
@@ -597,6 +413,15 @@ class FieldCacheImpl implements FieldCac
     }
   }
 
+  private static class GrowableWriterAndMinValue {
+    GrowableWriterAndMinValue(GrowableWriter array, long minValue) {
+      this.writer = array;
+      this.minValue = minValue;
+    }
+    public GrowableWriter writer;
+    public long minValue;
+  }
+
   static final class IntCache extends Cache {
     IntCache(FieldCacheImpl wrapper) {
       super(wrapper);
@@ -609,22 +434,17 @@ class FieldCacheImpl implements FieldCac
       final IntParser parser = (IntParser) key.custom;
       if (parser == null) {
         // Confusing: must delegate to wrapper (vs simply
-        // setting parser =
-        // DEFAULT_INT_PARSER/NUMERIC_UTILS_INT_PARSER) so
-        // cache key includes
-        // DEFAULT_INT_PARSER/NUMERIC_UTILS_INT_PARSER:
-        try {
-          return wrapper.getInts(reader, key.field, DEFAULT_INT_PARSER, setDocsWithField);
-        } catch (NumberFormatException ne) {
-          return wrapper.getInts(reader, key.field, NUMERIC_UTILS_INT_PARSER, setDocsWithField);
-        }
+        // setting parser = NUMERIC_UTILS_INT_PARSER) so
+        // cache key includes NUMERIC_UTILS_INT_PARSER:
+        return wrapper.getInts(reader, key.field, NUMERIC_UTILS_INT_PARSER, setDocsWithField);
       }
 
-      final HoldsOneThing<int[]> valuesRef = new HoldsOneThing<int[]>();
+      final HoldsOneThing<GrowableWriterAndMinValue> valuesRef = new HoldsOneThing<GrowableWriterAndMinValue>();
 
       Uninvert u = new Uninvert() {
+          private int minValue;
           private int currentValue;
-          private int[] values;
+          private GrowableWriter values;
 
           @Override
           public void visitTerm(BytesRef term) {
@@ -634,16 +454,28 @@ class FieldCacheImpl implements FieldCac
               // (which will hit a NumberFormatException
               // when we first try the DEFAULT_INT_PARSER),
               // we don't double-alloc:
-              values = new int[reader.maxDoc()];
-              valuesRef.set(values);
+              int startBitsPerValue;
+              // Make sure than missing values (0) can be stored without resizing
+              if (currentValue < 0) {
+                minValue = currentValue;
+                startBitsPerValue = PackedInts.bitsRequired((-minValue) & 0xFFFFFFFFL);
+              } else {
+                minValue = 0;
+                startBitsPerValue = PackedInts.bitsRequired(currentValue);
+              }
+              values = new GrowableWriter(startBitsPerValue, reader.maxDoc(), PackedInts.FAST);
+              if (minValue != 0) {
+                values.fill(0, values.size(), (-minValue) & 0xFFFFFFFFL); // default value must be 0
+              }
+              valuesRef.set(new GrowableWriterAndMinValue(values, minValue));
             }
           }
 
           @Override
           public void visitDoc(int docID) {
-            values[docID] = currentValue;
+            values.set(docID, (currentValue - minValue) & 0xFFFFFFFFL);
           }
-          
+
           @Override
           protected TermsEnum termsEnum(Terms terms) throws IOException {
             return parser.termsEnum(terms);
@@ -655,11 +487,11 @@ class FieldCacheImpl implements FieldCac
       if (setDocsWithField) {
         wrapper.setDocsWithField(reader, key.field, u.docsWithField);
       }
-      int[] values = valuesRef.get();
+      GrowableWriterAndMinValue values = valuesRef.get();
       if (values == null) {
-        values = new int[reader.maxDoc()];
+        return new IntsFromArray(new PackedInts.NullReader(reader.maxDoc()), 0);
       }
-      return new IntsFromArray(values);
+      return new IntsFromArray(values.writer.getMutable(), (int) values.minValue);
     }
   }
 
@@ -791,15 +623,9 @@ class FieldCacheImpl implements FieldCac
       final FloatParser parser = (FloatParser) key.custom;
       if (parser == null) {
         // Confusing: must delegate to wrapper (vs simply
-        // setting parser =
-        // DEFAULT_FLOAT_PARSER/NUMERIC_UTILS_FLOAT_PARSER) so
-        // cache key includes
-        // DEFAULT_FLOAT_PARSER/NUMERIC_UTILS_FLOAT_PARSER:
-        try {
-          return wrapper.getFloats(reader, key.field, DEFAULT_FLOAT_PARSER, setDocsWithField);
-        } catch (NumberFormatException ne) {
-          return wrapper.getFloats(reader, key.field, NUMERIC_UTILS_FLOAT_PARSER, setDocsWithField);
-        }
+        // setting parser = NUMERIC_UTILS_FLOAT_PARSER) so
+        // cache key includes NUMERIC_UTILS_FLOAT_PARSER:
+        return wrapper.getFloats(reader, key.field, NUMERIC_UTILS_FLOAT_PARSER, setDocsWithField);
       }
 
       final HoldsOneThing<float[]> valuesRef = new HoldsOneThing<float[]>();
@@ -878,15 +704,17 @@ class FieldCacheImpl implements FieldCac
   }
 
   static class LongsFromArray extends Longs {
-    private final long[] values;
+    private final PackedInts.Reader values;
+    private final long minValue;
 
-    public LongsFromArray(long[] values) {
+    public LongsFromArray(PackedInts.Reader values, long minValue) {
       this.values = values;
+      this.minValue = minValue;
     }
     
     @Override
     public long get(int docID) {
-      return values[docID];
+      return minValue + values.get(docID);
     }
   }
 
@@ -902,22 +730,17 @@ class FieldCacheImpl implements FieldCac
       final LongParser parser = (LongParser) key.custom;
       if (parser == null) {
         // Confusing: must delegate to wrapper (vs simply
-        // setting parser =
-        // DEFAULT_LONG_PARSER/NUMERIC_UTILS_LONG_PARSER) so
-        // cache key includes
-        // DEFAULT_LONG_PARSER/NUMERIC_UTILS_LONG_PARSER:
-        try {
-          return wrapper.getLongs(reader, key.field, DEFAULT_LONG_PARSER, setDocsWithField);
-        } catch (NumberFormatException ne) {
-          return wrapper.getLongs(reader, key.field, NUMERIC_UTILS_LONG_PARSER, setDocsWithField);
-        }
+        // setting parser = NUMERIC_UTILS_LONG_PARSER) so
+        // cache key includes NUMERIC_UTILS_LONG_PARSER:
+        return wrapper.getLongs(reader, key.field, NUMERIC_UTILS_LONG_PARSER, setDocsWithField);
       }
 
-      final HoldsOneThing<long[]> valuesRef = new HoldsOneThing<long[]>();
+      final HoldsOneThing<GrowableWriterAndMinValue> valuesRef = new HoldsOneThing<GrowableWriterAndMinValue>();
 
       Uninvert u = new Uninvert() {
+          private long minValue;
           private long currentValue;
-          private long[] values;
+          private GrowableWriter values;
 
           @Override
           public void visitTerm(BytesRef term) {
@@ -927,14 +750,26 @@ class FieldCacheImpl implements FieldCac
               // (which will hit a NumberFormatException
               // when we first try the DEFAULT_INT_PARSER),
               // we don't double-alloc:
-              values = new long[reader.maxDoc()];
-              valuesRef.set(values);
+              int startBitsPerValue;
+              // Make sure than missing values (0) can be stored without resizing
+              if (currentValue < 0) {
+                minValue = currentValue;
+                startBitsPerValue = minValue == Long.MIN_VALUE ? 64 : PackedInts.bitsRequired(-minValue);
+              } else {
+                minValue = 0;
+                startBitsPerValue = PackedInts.bitsRequired(currentValue);
+              }
+              values = new GrowableWriter(startBitsPerValue, reader.maxDoc(), PackedInts.FAST);
+              if (minValue != 0) {
+                values.fill(0, values.size(), -minValue); // default value must be 0
+              }
+              valuesRef.set(new GrowableWriterAndMinValue(values, minValue));
             }
           }
 
           @Override
           public void visitDoc(int docID) {
-            values[docID] = currentValue;
+            values.set(docID, currentValue - minValue);
           }
           
           @Override
@@ -948,11 +783,11 @@ class FieldCacheImpl implements FieldCac
       if (setDocsWithField) {
         wrapper.setDocsWithField(reader, key.field, u.docsWithField);
       }
-      long[] values = valuesRef.get();
+      GrowableWriterAndMinValue values = valuesRef.get();
       if (values == null) {
-        values = new long[reader.maxDoc()];
+        return new LongsFromArray(new PackedInts.NullReader(reader.maxDoc()), 0L);
       }
-      return new LongsFromArray(values);
+      return new LongsFromArray(values.writer.getMutable(), values.minValue);
     }
   }
 
@@ -1013,15 +848,9 @@ class FieldCacheImpl implements FieldCac
       final DoubleParser parser = (DoubleParser) key.custom;
       if (parser == null) {
         // Confusing: must delegate to wrapper (vs simply
-        // setting parser =
-        // DEFAULT_DOUBLE_PARSER/NUMERIC_UTILS_DOUBLE_PARSER) so
-        // cache key includes
-        // DEFAULT_DOUBLE_PARSER/NUMERIC_UTILS_DOUBLE_PARSER:
-        try {
-          return wrapper.getDoubles(reader, key.field, DEFAULT_DOUBLE_PARSER, setDocsWithField);
-        } catch (NumberFormatException ne) {
-          return wrapper.getDoubles(reader, key.field, NUMERIC_UTILS_DOUBLE_PARSER, setDocsWithField);
-        }
+        // setting parser = NUMERIC_UTILS_DOUBLE_PARSER) so
+        // cache key includes NUMERIC_UTILS_DOUBLE_PARSER:
+        return wrapper.getDoubles(reader, key.field, NUMERIC_UTILS_DOUBLE_PARSER, setDocsWithField);
       }
 
       final HoldsOneThing<double[]> valuesRef = new HoldsOneThing<double[]>();
@@ -1069,11 +898,11 @@ class FieldCacheImpl implements FieldCac
 
   public static class SortedDocValuesImpl extends SortedDocValues {
     private final PagedBytes.Reader bytes;
-    private final PackedInts.Reader termOrdToBytesOffset;
+    private final MonotonicAppendingLongBuffer termOrdToBytesOffset;
     private final PackedInts.Reader docToTermOrd;
     private final int numOrd;
 
-    public SortedDocValuesImpl(PagedBytes.Reader bytes, PackedInts.Reader termOrdToBytesOffset, PackedInts.Reader docToTermOrd, int numOrd) {
+    public SortedDocValuesImpl(PagedBytes.Reader bytes, MonotonicAppendingLongBuffer termOrdToBytesOffset, PackedInts.Reader docToTermOrd, int numOrd) {
       this.bytes = bytes;
       this.docToTermOrd = docToTermOrd;
       this.termOrdToBytesOffset = termOrdToBytesOffset;
@@ -1144,9 +973,7 @@ class FieldCacheImpl implements FieldCac
 
       final PagedBytes bytes = new PagedBytes(15);
 
-      int startBytesBPV;
       int startTermsBPV;
-      int startNumUniqueTerms;
 
       final int termCountHardLimit;
       if (maxDoc == Integer.MAX_VALUE) {
@@ -1169,22 +996,15 @@ class FieldCacheImpl implements FieldCac
             numUniqueTerms = termCountHardLimit;
           }
 
-          startBytesBPV = PackedInts.bitsRequired(numUniqueTerms*4);
           startTermsBPV = PackedInts.bitsRequired(numUniqueTerms);
-
-          startNumUniqueTerms = (int) numUniqueTerms;
         } else {
-          startBytesBPV = 1;
           startTermsBPV = 1;
-          startNumUniqueTerms = 1;
         }
       } else {
-        startBytesBPV = 1;
         startTermsBPV = 1;
-        startNumUniqueTerms = 1;
       }
 
-      GrowableWriter termOrdToBytesOffset = new GrowableWriter(startBytesBPV, 1+startNumUniqueTerms, acceptableOverheadRatio);
+      MonotonicAppendingLongBuffer termOrdToBytesOffset = new MonotonicAppendingLongBuffer();
       final GrowableWriter docToTermOrd = new GrowableWriter(startTermsBPV, maxDoc, acceptableOverheadRatio);
 
       int termOrd = 0;
@@ -1204,13 +1024,7 @@ class FieldCacheImpl implements FieldCac
             break;
           }
 
-          if (termOrd == termOrdToBytesOffset.size()) {
-            // NOTE: this code only runs if the incoming
-            // reader impl doesn't implement
-            // size (which should be uncommon)
-            termOrdToBytesOffset = termOrdToBytesOffset.resize(ArrayUtil.oversize(1+termOrd, 1));
-          }
-          termOrdToBytesOffset.set(termOrd, bytes.copyUsingLengthPrefix(term));
+          termOrdToBytesOffset.add(bytes.copyUsingLengthPrefix(term));
           docs = termsEnum.docs(null, docs, DocsEnum.FLAG_NONE);
           while (true) {
             final int docID = docs.nextDoc();
@@ -1222,14 +1036,10 @@ class FieldCacheImpl implements FieldCac
           }
           termOrd++;
         }
-
-        if (termOrdToBytesOffset.size() > termOrd) {
-          termOrdToBytesOffset = termOrdToBytesOffset.resize(termOrd);
-        }
       }
 
       // maybe an int-only impl?
-      return new SortedDocValuesImpl(bytes.freeze(true), termOrdToBytesOffset.getMutable(), docToTermOrd.getMutable(), termOrd);
+      return new SortedDocValuesImpl(bytes.freeze(true), termOrdToBytesOffset, docToTermOrd.getMutable(), termOrd);
     }
   }
 

Modified: lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/FieldCacheRangeFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/FieldCacheRangeFilter.java?rev=1499601&r1=1499600&r2=1499601&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/FieldCacheRangeFilter.java (original)
+++ lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/FieldCacheRangeFilter.java Wed Jul  3 23:26:32 2013
@@ -193,109 +193,7 @@ public abstract class FieldCacheRangeFil
       }
     };
   }
-  
-  /**
-   * Creates a numeric range filter using {@link FieldCache#getBytes(AtomicReader,String,boolean)}. This works with all
-   * byte fields containing exactly one numeric term in the field. The range can be half-open by setting one
-   * of the values to <code>null</code>.
-   */
-  public static FieldCacheRangeFilter<Byte> newByteRange(String field, Byte lowerVal, Byte upperVal, boolean includeLower, boolean includeUpper) {
-    return newByteRange(field, null, lowerVal, upperVal, includeLower, includeUpper);
-  }
-  
-  /**
-   * Creates a numeric range filter using {@link FieldCache#getBytes(AtomicReader,String,FieldCache.ByteParser,boolean)}. This works with all
-   * byte fields containing exactly one numeric term in the field. The range can be half-open by setting one
-   * of the values to <code>null</code>.
-   */
-  public static FieldCacheRangeFilter<Byte> newByteRange(String field, FieldCache.ByteParser parser, Byte lowerVal, Byte upperVal, boolean includeLower, boolean includeUpper) {
-    return new FieldCacheRangeFilter<Byte>(field, parser, lowerVal, upperVal, includeLower, includeUpper) {
-      @Override
-      public DocIdSet getDocIdSet(AtomicReaderContext context, Bits acceptDocs) throws IOException {
-        final byte inclusiveLowerPoint, inclusiveUpperPoint;
-        if (lowerVal != null) {
-          final byte i = lowerVal.byteValue();
-          if (!includeLower && i == Byte.MAX_VALUE)
-            return null;
-          inclusiveLowerPoint = (byte) (includeLower ?  i : (i + 1));
-        } else {
-          inclusiveLowerPoint = Byte.MIN_VALUE;
-        }
-        if (upperVal != null) {
-          final byte i = upperVal.byteValue();
-          if (!includeUpper && i == Byte.MIN_VALUE)
-            return null;
-          inclusiveUpperPoint = (byte) (includeUpper ? i : (i - 1));
-        } else {
-          inclusiveUpperPoint = Byte.MAX_VALUE;
-        }
-        
-        if (inclusiveLowerPoint > inclusiveUpperPoint)
-          return null;
-        
-        final FieldCache.Bytes values = FieldCache.DEFAULT.getBytes(context.reader(), field, (FieldCache.ByteParser) parser, false);
-        return new FieldCacheDocIdSet(context.reader().maxDoc(), acceptDocs) {
-          @Override
-          protected boolean matchDoc(int doc) {
-            final byte value = values.get(doc);
-            return value >= inclusiveLowerPoint && value <= inclusiveUpperPoint;
-          }
-        };
-      }
-    };
-  }
-  
-  /**
-   * Creates a numeric range filter using {@link FieldCache#getShorts(AtomicReader,String,boolean)}. This works with all
-   * short fields containing exactly one numeric term in the field. The range can be half-open by setting one
-   * of the values to <code>null</code>.
-   */
-  public static FieldCacheRangeFilter<Short> newShortRange(String field, Short lowerVal, Short upperVal, boolean includeLower, boolean includeUpper) {
-    return newShortRange(field, null, lowerVal, upperVal, includeLower, includeUpper);
-  }
-  
-  /**
-   * Creates a numeric range filter using {@link FieldCache#getShorts(AtomicReader,String,FieldCache.ShortParser,boolean)}. This works with all
-   * short fields containing exactly one numeric term in the field. The range can be half-open by setting one
-   * of the values to <code>null</code>.
-   */
-  public static FieldCacheRangeFilter<Short> newShortRange(String field, FieldCache.ShortParser parser, Short lowerVal, Short upperVal, boolean includeLower, boolean includeUpper) {
-    return new FieldCacheRangeFilter<Short>(field, parser, lowerVal, upperVal, includeLower, includeUpper) {
-      @Override
-      public DocIdSet getDocIdSet(AtomicReaderContext context, Bits acceptDocs) throws IOException {
-        final short inclusiveLowerPoint, inclusiveUpperPoint;
-        if (lowerVal != null) {
-          short i = lowerVal.shortValue();
-          if (!includeLower && i == Short.MAX_VALUE)
-            return null;
-          inclusiveLowerPoint = (short) (includeLower ? i : (i + 1));
-        } else {
-          inclusiveLowerPoint = Short.MIN_VALUE;
-        }
-        if (upperVal != null) {
-          short i = upperVal.shortValue();
-          if (!includeUpper && i == Short.MIN_VALUE)
-            return null;
-          inclusiveUpperPoint = (short) (includeUpper ? i : (i - 1));
-        } else {
-          inclusiveUpperPoint = Short.MAX_VALUE;
-        }
-        
-        if (inclusiveLowerPoint > inclusiveUpperPoint)
-          return null;
-        
-        final FieldCache.Shorts values = FieldCache.DEFAULT.getShorts(context.reader(), field, (FieldCache.ShortParser) parser, false);
-        return new FieldCacheDocIdSet(context.reader().maxDoc(), acceptDocs) {
-          @Override
-          protected boolean matchDoc(int doc) {
-            final short value = values.get(doc);
-            return value >= inclusiveLowerPoint && value <= inclusiveUpperPoint;
-          }
-        };
-      }
-    };
-  }
-  
+
   /**
    * Creates a numeric range filter using {@link FieldCache#getInts(AtomicReader,String,boolean)}. This works with all
    * int fields containing exactly one numeric term in the field. The range can be half-open by setting one
@@ -521,7 +419,7 @@ public abstract class FieldCacheRangeFil
   }
 
   @Override
-  @SuppressWarnings({"unchecked","rawtypes"})
+  @SuppressWarnings({"rawtypes"})
   public final boolean equals(Object o) {
     if (this == o) return true;
     if (!(o instanceof FieldCacheRangeFilter)) return false;

Modified: lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/FieldComparator.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/FieldComparator.java?rev=1499601&r1=1499600&r2=1499601&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/FieldComparator.java (original)
+++ lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/FieldComparator.java Wed Jul  3 23:26:32 2013
@@ -22,12 +22,10 @@ import java.io.IOException;
 import org.apache.lucene.index.AtomicReaderContext;
 import org.apache.lucene.index.BinaryDocValues;
 import org.apache.lucene.index.SortedDocValues;
-import org.apache.lucene.search.FieldCache.ByteParser;
 import org.apache.lucene.search.FieldCache.DoubleParser;
 import org.apache.lucene.search.FieldCache.FloatParser;
 import org.apache.lucene.search.FieldCache.IntParser;
 import org.apache.lucene.search.FieldCache.LongParser;
-import org.apache.lucene.search.FieldCache.ShortParser;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
 
@@ -218,78 +216,6 @@ public abstract class FieldComparator<T>
     }
   }
 
-  /** Parses field's values as byte (using {@link
-   *  FieldCache#getBytes} and sorts by ascending value */
-  public static final class ByteComparator extends NumericComparator<Byte> {
-    private final byte[] values;
-    private final ByteParser parser;
-    private FieldCache.Bytes currentReaderValues;
-    private byte bottom;
-
-    ByteComparator(int numHits, String field, FieldCache.Parser parser, Byte missingValue) {
-      super(field, missingValue);
-      values = new byte[numHits];
-      this.parser = (ByteParser) parser;
-    }
-
-    @Override
-    public int compare(int slot1, int slot2) {
-      return Byte.compare(values[slot1], values[slot2]);
-    }
-
-    @Override
-    public int compareBottom(int doc) {
-      byte v2 = currentReaderValues.get(doc);
-      // Test for v2 == 0 to save Bits.get method call for
-      // the common case (doc has value and value is non-zero):
-      if (docsWithField != null && v2 == 0 && !docsWithField.get(doc)) {
-        v2 = missingValue;
-      }
-
-      return Byte.compare(bottom, v2);
-    }
-
-    @Override
-    public void copy(int slot, int doc) {
-      byte v2 = currentReaderValues.get(doc);
-      // Test for v2 == 0 to save Bits.get method call for
-      // the common case (doc has value and value is non-zero):
-      if (docsWithField != null && v2 == 0 && !docsWithField.get(doc)) {
-        v2 = missingValue;
-      }
-      values[slot] = v2;
-    }
-
-    @Override
-    public FieldComparator<Byte> setNextReader(AtomicReaderContext context) throws IOException {
-      // NOTE: must do this before calling super otherwise
-      // we compute the docsWithField Bits twice!
-      currentReaderValues = FieldCache.DEFAULT.getBytes(context.reader(), field, parser, missingValue != null);
-      return super.setNextReader(context);
-    }
-    
-    @Override
-    public void setBottom(final int bottom) {
-      this.bottom = values[bottom];
-    }
-
-    @Override
-    public Byte value(int slot) {
-      return Byte.valueOf(values[slot]);
-    }
-
-    @Override
-    public int compareDocToValue(int doc, Byte value) {
-      byte docValue = currentReaderValues.get(doc);
-      // Test for docValue == 0 to save Bits.get method call for
-      // the common case (doc has value and value is non-zero):
-      if (docsWithField != null && docValue == 0 && !docsWithField.get(doc)) {
-        docValue = missingValue;
-      }
-      return Byte.compare(docValue, value.byteValue());
-    }
-  }
-
   /** Parses field's values as double (using {@link
    *  FieldCache#getDoubles} and sorts by ascending value */
   public static final class DoubleComparator extends NumericComparator<Double> {
@@ -439,80 +365,6 @@ public abstract class FieldComparator<T>
     }
   }
 
-  /** Parses field's values as short (using {@link
-   *  FieldCache#getShorts} and sorts by ascending value */
-  public static final class ShortComparator extends NumericComparator<Short> {
-    private final short[] values;
-    private final ShortParser parser;
-    private FieldCache.Shorts currentReaderValues;
-    private short bottom;
-
-    ShortComparator(int numHits, String field, FieldCache.Parser parser, Short missingValue) {
-      super(field, missingValue);
-      values = new short[numHits];
-      this.parser = (ShortParser) parser;
-    }
-
-    @Override
-    public int compare(int slot1, int slot2) {
-      return Short.compare(values[slot1], values[slot2]);
-    }
-
-    @Override
-    public int compareBottom(int doc) {
-      short v2 = currentReaderValues.get(doc);
-      // Test for v2 == 0 to save Bits.get method call for
-      // the common case (doc has value and value is non-zero):
-      if (docsWithField != null && v2 == 0 && !docsWithField.get(doc)) {
-        v2 = missingValue;
-      }
-
-      return Short.compare(bottom, v2);
-    }
-
-    @Override
-    public void copy(int slot, int doc) {
-      short v2 = currentReaderValues.get(doc);
-      // Test for v2 == 0 to save Bits.get method call for
-      // the common case (doc has value and value is non-zero):
-      if (docsWithField != null && v2 == 0 && !docsWithField.get(doc)) {
-        v2 = missingValue;
-      }
-
-      values[slot] = v2;
-    }
-
-    @Override
-    public FieldComparator<Short> setNextReader(AtomicReaderContext context) throws IOException {
-      // NOTE: must do this before calling super otherwise
-      // we compute the docsWithField Bits twice!
-      currentReaderValues = FieldCache.DEFAULT.getShorts(context.reader(), field, parser, missingValue != null);
-      return super.setNextReader(context);
-    }
-
-    @Override
-    public void setBottom(final int bottom) {
-      this.bottom = values[bottom];
-    }
-
-    @Override
-    public Short value(int slot) {
-      return Short.valueOf(values[slot]);
-    }
-
-    @Override
-    public int compareDocToValue(int doc, Short valueObj) {
-      final short value = valueObj.shortValue();
-      short docValue = currentReaderValues.get(doc);
-      // Test for docValue == 0 to save Bits.get method call for
-      // the common case (doc has value and value is non-zero):
-      if (docsWithField != null && docValue == 0 && !docsWithField.get(doc)) {
-        docValue = missingValue;
-      }
-      return Short.compare(docValue, value);
-    }
-  }
-
   /** Parses field's values as int (using {@link
    *  FieldCache#getInts} and sorts by ascending value */
   public static final class IntComparator extends NumericComparator<Integer> {

Modified: lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/MultiPhraseQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/MultiPhraseQuery.java?rev=1499601&r1=1499600&r2=1499601&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/MultiPhraseQuery.java (original)
+++ lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/MultiPhraseQuery.java Wed Jul  3 23:26:32 2013
@@ -31,7 +31,7 @@ import org.apache.lucene.index.TermConte
 import org.apache.lucene.index.TermState;
 import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.TermsEnum;
-import org.apache.lucene.search.similarities.Similarity.SloppySimScorer;
+import org.apache.lucene.search.similarities.Similarity.SimScorer;
 import org.apache.lucene.search.similarities.Similarity;
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.Bits;
@@ -245,14 +245,14 @@ public class MultiPhraseQuery extends Qu
       }
 
       if (slop == 0) {
-        ExactPhraseScorer s = new ExactPhraseScorer(this, postingsFreqs, similarity.exactSimScorer(stats, context));
+        ExactPhraseScorer s = new ExactPhraseScorer(this, postingsFreqs, similarity.simScorer(stats, context));
         if (s.noDocs) {
           return null;
         } else {
           return s;
         }
       } else {
-        return new SloppyPhraseScorer(this, postingsFreqs, slop, similarity.sloppySimScorer(stats, context));
+        return new SloppyPhraseScorer(this, postingsFreqs, slop, similarity.simScorer(stats, context));
       }
     }
 
@@ -263,7 +263,7 @@ public class MultiPhraseQuery extends Qu
         int newDoc = scorer.advance(doc);
         if (newDoc == doc) {
           float freq = slop == 0 ? scorer.freq() : ((SloppyPhraseScorer)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/search/PhraseQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/PhraseQuery.java?rev=1499601&r1=1499600&r2=1499601&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/PhraseQuery.java (original)
+++ lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/PhraseQuery.java Wed Jul  3 23:26:32 2013
@@ -33,7 +33,7 @@ import org.apache.lucene.index.TermConte
 import org.apache.lucene.index.TermState;
 import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.TermsEnum;
-import org.apache.lucene.search.similarities.Similarity.SloppySimScorer;
+import org.apache.lucene.search.similarities.Similarity.SimScorer;
 import org.apache.lucene.search.similarities.Similarity;
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.Bits;
@@ -282,7 +282,7 @@ public class PhraseQuery extends Query {
       }
 
       if (slop == 0) {  // optimize exact case
-        ExactPhraseScorer s = new ExactPhraseScorer(this, postingsFreqs, similarity.exactSimScorer(stats, context));
+        ExactPhraseScorer s = new ExactPhraseScorer(this, postingsFreqs, similarity.simScorer(stats, context));
         if (s.noDocs) {
           return null;
         } else {
@@ -290,7 +290,7 @@ public class PhraseQuery extends Query {
         }
       } else {
         return
-          new SloppyPhraseScorer(this, postingsFreqs, slop, similarity.sloppySimScorer(stats, context));
+          new SloppyPhraseScorer(this, postingsFreqs, slop, similarity.simScorer(stats, context));
       }
     }
     
@@ -306,7 +306,7 @@ public class PhraseQuery extends Query {
         int newDoc = scorer.advance(doc);
         if (newDoc == doc) {
           float freq = slop == 0 ? scorer.freq() : ((SloppyPhraseScorer)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/search/SloppyPhraseScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/SloppyPhraseScorer.java?rev=1499601&r1=1499600&r2=1499601&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/SloppyPhraseScorer.java (original)
+++ lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/SloppyPhraseScorer.java Wed Jul  3 23:26:32 2013
@@ -34,7 +34,7 @@ final class SloppyPhraseScorer extends S
 
   private float sloppyFreq; //phrase frequency in current doc as computed by phraseFreq().
 
-  private final Similarity.SloppySimScorer docScorer;
+  private final Similarity.SimScorer docScorer;
   
   private final int slop;
   private final int numPostings;
@@ -52,7 +52,7 @@ final class SloppyPhraseScorer extends S
   private final long cost;
   
   SloppyPhraseScorer(Weight weight, PhraseQuery.PostingsAndFreq[] postings,
-      int slop, Similarity.SloppySimScorer docScorer) {
+      int slop, Similarity.SimScorer docScorer) {
     super(weight);
     this.docScorer = docScorer;
     this.slop = slop;

Modified: lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/SortField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/SortField.java?rev=1499601&r1=1499600&r2=1499601&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/SortField.java (original)
+++ lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/SortField.java Wed Jul  3 23:26:32 2013
@@ -71,18 +71,10 @@ public class SortField {
      * lower values are at the front. */
     DOUBLE,
 
-    /** Sort using term values as encoded Shorts.  Sort values are Short and
-     * lower values are at the front. */
-    SHORT,
-
     /** Sort using a custom Comparator.  Sort values are any Comparable and
      * sorting is done according to natural order. */
     CUSTOM,
 
-    /** Sort using term values as encoded Bytes.  Sort values are Byte and
-     * lower values are at the front. */
-    BYTE,
-
     /** Sort using term values as Strings, but comparing by
      * value (using String.compareTo) for all comparisons.
      * This is typically slower than {@link #STRING}, which
@@ -164,8 +156,6 @@ public class SortField {
   public SortField(String field, FieldCache.Parser parser, boolean reverse) {
     if (parser instanceof FieldCache.IntParser) initFieldType(field, Type.INT);
     else if (parser instanceof FieldCache.FloatParser) initFieldType(field, Type.FLOAT);
-    else if (parser instanceof FieldCache.ShortParser) initFieldType(field, Type.SHORT);
-    else if (parser instanceof FieldCache.ByteParser) initFieldType(field, Type.BYTE);
     else if (parser instanceof FieldCache.LongParser) initFieldType(field, Type.LONG);
     else if (parser instanceof FieldCache.DoubleParser) initFieldType(field, Type.DOUBLE);
     else {
@@ -177,7 +167,7 @@ public class SortField {
   }
   
   public SortField setMissingValue(Object missingValue) {
-    if (type != Type.BYTE && type != Type.SHORT && type != Type.INT && type != Type.FLOAT && type != Type.LONG && type != Type.DOUBLE) {
+    if (type != Type.INT && type != Type.FLOAT && type != Type.LONG && type != Type.DOUBLE) {
       throw new IllegalArgumentException( "Missing value only works for numeric types" );
     }
     this.missingValue = missingValue;
@@ -274,14 +264,6 @@ public class SortField {
         buffer.append("<string_val" + ": \"").append(field).append("\">");
         break;
 
-      case BYTE:
-        buffer.append("<byte: \"").append(field).append("\">");
-        break;
-
-      case SHORT:
-        buffer.append("<short: \"").append(field).append("\">");
-        break;
-
       case INT:
         buffer.append("<int" + ": \"").append(field).append("\">");
         break;
@@ -389,12 +371,6 @@ public class SortField {
     case DOUBLE:
       return new FieldComparator.DoubleComparator(numHits, field, parser, (Double) missingValue);
 
-    case BYTE:
-      return new FieldComparator.ByteComparator(numHits, field, parser, (Byte) missingValue);
-
-    case SHORT:
-      return new FieldComparator.ShortComparator(numHits, field, parser, (Short) missingValue);
-
     case CUSTOM:
       assert comparatorSource != null;
       return comparatorSource.newComparator(field, numHits, sortPos, reverse);

Modified: lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/TermQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/TermQuery.java?rev=1499601&r1=1499600&r2=1499601&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/TermQuery.java (original)
+++ lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/TermQuery.java Wed Jul  3 23:26:32 2013
@@ -29,7 +29,7 @@ import org.apache.lucene.index.Term;
 import org.apache.lucene.index.TermContext;
 import org.apache.lucene.index.TermState;
 import org.apache.lucene.index.TermsEnum;
-import org.apache.lucene.search.similarities.Similarity.ExactSimScorer;
+import org.apache.lucene.search.similarities.Similarity.SimScorer;
 import org.apache.lucene.search.similarities.Similarity;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.ToStringUtils;
@@ -84,7 +84,7 @@ public class TermQuery extends Query {
       }
       DocsEnum docs = termsEnum.docs(acceptDocs, null);
       assert docs != null;
-      return new TermScorer(this, docs, similarity.exactSimScorer(stats, context));
+      return new TermScorer(this, docs, similarity.simScorer(stats, context));
     }
     
     /**
@@ -116,7 +116,7 @@ public class TermQuery extends Query {
         int newDoc = scorer.advance(doc);
         if (newDoc == doc) {
           float freq = scorer.freq();
-          ExactSimScorer docScorer = similarity.exactSimScorer(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, "termFreq=" + freq));

Modified: lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/TermScorer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/TermScorer.java?rev=1499601&r1=1499600&r2=1499601&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/TermScorer.java (original)
+++ lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/TermScorer.java Wed Jul  3 23:26:32 2013
@@ -26,7 +26,7 @@ import org.apache.lucene.search.similari
  */
 final class TermScorer extends Scorer {
   private final DocsEnum docsEnum;
-  private final Similarity.ExactSimScorer docScorer;
+  private final Similarity.SimScorer docScorer;
   
   /**
    * Construct a <code>TermScorer</code>.
@@ -36,10 +36,10 @@ final class TermScorer extends Scorer {
    * @param td
    *          An iterator over the documents matching the <code>Term</code>.
    * @param docScorer
-   *          The </code>Similarity.ExactSimScorer</code> implementation 
+   *          The </code>Similarity.SimScorer</code> implementation 
    *          to be used for score computations.
    */
-  TermScorer(Weight weight, DocsEnum td, Similarity.ExactSimScorer docScorer) {
+  TermScorer(Weight weight, DocsEnum td, Similarity.SimScorer docScorer) {
     super(weight);
     this.docScorer = docScorer;
     this.docsEnum = td;

Modified: lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/package.html
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/package.html?rev=1499601&r1=1499600&r2=1499601&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/package.html (original)
+++ lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/package.html Wed Jul  3 23:26:32 2013
@@ -441,9 +441,8 @@ on the built-in available scoring models
                   explain(AtomicReaderContext context, int doc)} &mdash; Provide a means for explaining why a given document was
                 scored the way it was.
                 Typically a weight such as TermWeight
-                that scores via a {@link org.apache.lucene.search.similarities.Similarity Similarity} will make use of the Similarity's implementations:
-                {@link org.apache.lucene.search.similarities.Similarity.ExactSimScorer#explain(int, Explanation) ExactSimScorer#explain(int doc, Explanation freq)},
-                and {@link org.apache.lucene.search.similarities.Similarity.SloppySimScorer#explain(int, Explanation) SloppySimScorer#explain(int doc, Explanation freq)}
+                that scores via a {@link org.apache.lucene.search.similarities.Similarity Similarity} will make use of the Similarity's implementation:
+                {@link org.apache.lucene.search.similarities.Similarity.SimScorer#explain(int, Explanation) SimScorer#explain(int doc, Explanation freq)}.
                 </li>
              </li>
         </ol>
@@ -468,7 +467,7 @@ on the built-in available scoring models
                 {@link org.apache.lucene.search.Scorer#score score()} &mdash; Return the score of the
                 current document. This value can be determined in any appropriate way for an application. For instance, the
                 {@link org.apache.lucene.search.TermScorer TermScorer} simply defers to the configured Similarity:
-                {@link org.apache.lucene.search.similarities.Similarity.ExactSimScorer#score(int, int) ExactSimScorer.score(int doc, int freq)}.
+                {@link org.apache.lucene.search.similarities.Similarity.SimScorer#score(int, float) SimScorer.score(int doc, float freq)}.
             </li>
             <li>
                 {@link org.apache.lucene.search.Scorer#freq freq()} &mdash; Returns the number of matches

Modified: lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/payloads/PayloadNearQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/payloads/PayloadNearQuery.java?rev=1499601&r1=1499600&r2=1499601&view=diff
==============================================================================
--- lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/payloads/PayloadNearQuery.java (original)
+++ lucene/dev/branches/security/lucene/core/src/java/org/apache/lucene/search/payloads/PayloadNearQuery.java Wed Jul  3 23:26:32 2013
@@ -25,7 +25,7 @@ import org.apache.lucene.search.IndexSea
 import org.apache.lucene.search.Weight;
 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.NearSpansOrdered;
 import org.apache.lucene.search.spans.NearSpansUnordered;
 import org.apache.lucene.search.spans.SpanNearQuery;
@@ -53,7 +53,7 @@ import java.util.Iterator;
  * <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 PayloadNearQuery extends SpanNearQuery {
   protected String fieldName;
@@ -151,7 +151,7 @@ public class PayloadNearQuery extends Sp
     public Scorer scorer(AtomicReaderContext context, boolean scoreDocsInOrder,
         boolean topScorer, Bits acceptDocs) throws IOException {
       return new PayloadNearSpanScorer(query.getSpans(context, acceptDocs, termContexts), this,
-          similarity, similarity.sloppySimScorer(stats, context));
+          similarity, similarity.simScorer(stats, context));
     }
     
     @Override
@@ -161,7 +161,7 @@ public class PayloadNearQuery extends Sp
         int newDoc = scorer.advance(doc);
         if (newDoc == doc) {
           float freq = scorer.freq();
-          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));
@@ -190,7 +190,7 @@ public class PayloadNearQuery extends Sp
     private int payloadsSeen;
 
     protected PayloadNearSpanScorer(Spans spans, Weight weight,
-        Similarity similarity, Similarity.SloppySimScorer docScorer) throws IOException {
+        Similarity similarity, Similarity.SimScorer docScorer) throws IOException {
       super(spans, weight, docScorer);
       this.spans = spans;
     }