You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by si...@apache.org on 2010/12/06 01:47:28 UTC

svn commit: r1042501 [4/13] - in /lucene/dev/branches/docvalues: ./ lucene/ lucene/contrib/ lucene/contrib/benchmark/src/java/org/apache/lucene/benchmark/byTask/feeds/ lucene/contrib/benchmark/src/java/org/apache/lucene/benchmark/byTask/tasks/ lucene/c...

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/Term.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/Term.java?rev=1042501&r1=1042500&r2=1042501&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/Term.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/Term.java Mon Dec  6 00:47:16 2010
@@ -168,7 +168,7 @@ public final class Term implements Compa
     BytesRef.getUTF8SortedAsUTF16Comparator();
 
   /** 
-   * @deprecated For internal backwards compatibility use only
+   * @deprecated (4.0) For internal backwards compatibility use only
    * @lucene.internal
    */
   @Deprecated

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/TermVectorsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/TermVectorsReader.java?rev=1042501&r1=1042500&r2=1042501&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/TermVectorsReader.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/TermVectorsReader.java Mon Dec  6 00:47:16 2010
@@ -90,7 +90,6 @@ class TermVectorsReader implements Clone
         assert format == tvdFormat;
         assert format == tvfFormat;
 
-        assert (tvx.length()-FORMAT_SIZE) % 16 == 0;
         numTotalDocs = (int) (tvx.length() >> 4);
 
         if (-1 == docStoreOffset) {
@@ -105,9 +104,10 @@ class TermVectorsReader implements Clone
           assert numTotalDocs >= size + docStoreOffset: "numTotalDocs=" + numTotalDocs + " size=" + size + " docStoreOffset=" + docStoreOffset;
         }
       } else {
-        // TODO: understand why FieldInfos.hasVectors() can
-        // return true yet the term vectors files don't
-        // exist...
+        // If all documents flushed in a segment had hit
+        // non-aborting exceptions, it's possible that
+        // FieldInfos.hasVectors returns true yet the term
+        // vector files don't exist.
         format = 0;
       }
 

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/TermVectorsTermsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/TermVectorsTermsWriter.java?rev=1042501&r1=1042500&r2=1042501&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/TermVectorsTermsWriter.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/TermVectorsTermsWriter.java Mon Dec  6 00:47:16 2010
@@ -50,12 +50,13 @@ final class TermVectorsTermsWriter exten
   @Override
   synchronized void flush(Map<TermsHashConsumerPerThread,Collection<TermsHashConsumerPerField>> threadsAndFields, final SegmentWriteState state) throws IOException {
 
-    if (state.numDocsInStore > 0) {
-      // It's possible that all documents seen in this segment
-      // hit non-aborting exceptions, in which case we will
-      // not have yet init'd the TermVectorsWriter:
-      initTermVectorsWriter();
-    }
+    // NOTE: it's possible that all documents seen in this segment
+    // hit non-aborting exceptions, in which case we will
+    // not have yet init'd the TermVectorsWriter.  This is
+    // actually OK (unlike in the stored fields case)
+    // because, although IieldInfos.hasVectors() will return
+    // true, the TermVectorsReader gracefully handles
+    // non-existence of the term vectors files.
 
     if (tvx != null) {
 

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/CodecProvider.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/CodecProvider.java?rev=1042501&r1=1042500&r2=1042501&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/CodecProvider.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/CodecProvider.java Mon Dec  6 00:47:16 2010
@@ -39,7 +39,7 @@ import org.apache.lucene.index.codecs.st
 public class CodecProvider {
   private SegmentInfosWriter infosWriter = new DefaultSegmentInfosWriter();
   private SegmentInfosReader infosReader = new DefaultSegmentInfosReader();
-  private String defaultFieldCodec = defaultCodec;
+  private String defaultFieldCodec = "Standard";
   private final Map<String, String> perFieldMap = new HashMap<String, String>();
 
   
@@ -47,7 +47,6 @@ public class CodecProvider {
 
   private final Set<String> knownExtensions = new HashSet<String>();
 
-  private static String defaultCodec = "Standard";
 
   public final static String[] CORE_CODECS = new String[] {"Standard", "Pulsing", "PreFlex", "SimpleText"};
 
@@ -97,25 +96,22 @@ public class CodecProvider {
     return infosReader;
   }
 
-  static private final CodecProvider defaultCodecs = new DefaultCodecProvider();
+  static private CodecProvider defaultCodecs = new DefaultCodecProvider();
 
   public static CodecProvider getDefault() {
     return defaultCodecs;
   }
 
-  /** Used for testing. @lucene.internal */
-  public synchronized static void setDefaultCodec(String s) {
-    defaultCodec = s;
-  }
-  /** Used for testing. @lucene.internal */
-  public synchronized static String getDefaultCodec() {
-    return defaultCodec;
+  /** For testing only
+   *  @lucene.internal */
+  public static void setDefault(CodecProvider cp) {
+    defaultCodecs = cp;
   }
   
   /**
-   * Sets the {@link Codec} for a given field. Not that setting a fields code is
-   * write-once. If the fields codec is already set this method will throw an
-   * {@link IllegalArgumentException}
+   * Sets the {@link Codec} for a given field. Not that setting a field's codec is
+   * write-once. If the field's codec is already set this method will throw an
+   * {@link IllegalArgumentException}.
    * 
    * @param field
    *          the name of the field
@@ -175,6 +171,5 @@ class DefaultCodecProvider extends Codec
     register(new PreFlexCodec());
     register(new PulsingCodec(1));
     register(new SimpleTextCodec());
-    setDefaultFieldCodec(CodecProvider.getDefaultCodec());
   }
 }

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/FixedGapTermsIndexReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/FixedGapTermsIndexReader.java?rev=1042501&r1=1042500&r2=1042501&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/FixedGapTermsIndexReader.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/FixedGapTermsIndexReader.java Mon Dec  6 00:47:16 2010
@@ -358,7 +358,7 @@ public class FixedGapTermsIndexReader ex
       private void fillResult(int idx, TermsIndexResult result) {
         final long offset = termOffsets.get(idx);
         final int length = (int) (termOffsets.get(1+idx) - offset);
-        termBytesReader.fill(result.term, termBytesStart + offset, length);
+        termBytesReader.fillSlice(result.term, termBytesStart + offset, length);
         result.position = idx * totalIndexInterval;
         result.offset = termsStart + termsDictOffsets.get(idx);
       }
@@ -373,7 +373,7 @@ public class FixedGapTermsIndexReader ex
 
           final long offset = termOffsets.get(mid);
           final int length = (int) (termOffsets.get(1+mid) - offset);
-          termBytesReader.fill(result.term, termBytesStart + offset, length);
+          termBytesReader.fillSlice(result.term, termBytesStart + offset, length);
 
           int delta = termComp.compare(term, result.term);
           if (delta < 0) {
@@ -394,7 +394,7 @@ public class FixedGapTermsIndexReader ex
 
         final long offset = termOffsets.get(hi);
         final int length = (int) (termOffsets.get(1+hi) - offset);
-        termBytesReader.fill(result.term, termBytesStart + offset, length);
+        termBytesReader.fillSlice(result.term, termBytesStart + offset, length);
 
         result.position = hi*totalIndexInterval;
         result.offset = termsStart + termsDictOffsets.get(hi);

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/docvalues/DocValuesCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/docvalues/DocValuesCodec.java?rev=1042501&r1=1042500&r2=1042501&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/docvalues/DocValuesCodec.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/docvalues/DocValuesCodec.java Mon Dec  6 00:47:16 2010
@@ -22,7 +22,6 @@ import java.util.HashSet;
 import java.util.Iterator;
 import java.util.Set;
 import java.util.Map.Entry;
-import java.util.concurrent.atomic.AtomicLong;
 
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.FieldsEnum;
@@ -43,6 +42,7 @@ import org.apache.lucene.util.BytesRef;
 
 /**
  * A codec that adds DocValues support to a given codec transparently.
+ * @lucene.experimental
  */
 public class DocValuesCodec extends Codec {
   private final Codec other;

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/docvalues/DocValuesConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/docvalues/DocValuesConsumer.java?rev=1042501&r1=1042500&r2=1042501&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/docvalues/DocValuesConsumer.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/docvalues/DocValuesConsumer.java Mon Dec  6 00:47:16 2010
@@ -24,7 +24,7 @@ import java.util.concurrent.atomic.Atomi
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.values.DocValues;
-import org.apache.lucene.index.values.ValuesAttribute;
+import org.apache.lucene.index.values.PerDocFieldValues;
 import org.apache.lucene.index.values.Writer;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.Bits;
@@ -47,7 +47,7 @@ public abstract class DocValuesConsumer 
     return this.bytesUsed.get();
   }
 
-  public abstract void add(int docID, ValuesAttribute attr) throws IOException;
+  public abstract void add(int docID, PerDocFieldValues docValues) throws IOException;
 
   public abstract void finish(int docCount) throws IOException;
 

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/docvalues/DocValuesProducerBase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/docvalues/DocValuesProducerBase.java?rev=1042501&r1=1042500&r2=1042501&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/docvalues/DocValuesProducerBase.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/docvalues/DocValuesProducerBase.java Mon Dec  6 00:47:16 2010
@@ -21,7 +21,6 @@ import java.util.TreeMap;
 
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.FieldInfos;
-import org.apache.lucene.index.IndexFileNames;
 import org.apache.lucene.index.SegmentInfo;
 import org.apache.lucene.index.codecs.FieldsProducer;
 import org.apache.lucene.index.values.Bytes;
@@ -29,10 +28,12 @@ import org.apache.lucene.index.values.Do
 import org.apache.lucene.index.values.Floats;
 import org.apache.lucene.index.values.Ints;
 import org.apache.lucene.index.values.Values;
-import org.apache.lucene.index.values.Writer;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.util.IntsRef;
 
+/**
+ * @lucene.experimental
+ */
 public abstract class DocValuesProducerBase extends FieldsProducer{
   
   protected final TreeMap<String, DocValues> docValues = new TreeMap<String, DocValues>();

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/preflex/PreFlexCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/preflex/PreFlexCodec.java?rev=1042501&r1=1042500&r2=1042501&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/preflex/PreFlexCodec.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/preflex/PreFlexCodec.java Mon Dec  6 00:47:16 2010
@@ -32,7 +32,7 @@ import org.apache.lucene.index.codecs.Fi
  *  format.  It does not provide a writer because newly
  *  written segments should use StandardCodec.
  *
- * @deprecated This is only used to read indexes created
+ * @deprecated (4.0) This is only used to read indexes created
  * before 4.0.
  * @lucene.experimental
  */

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/preflex/PreFlexFields.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/preflex/PreFlexFields.java?rev=1042501&r1=1042500&r2=1042501&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/preflex/PreFlexFields.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/preflex/PreFlexFields.java Mon Dec  6 00:47:16 2010
@@ -45,7 +45,10 @@ import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.UnicodeUtil;
 
 /** Exposes flex API on a pre-flex index, as a codec. 
- * @lucene.experimental */
+ * @lucene.experimental
+ * @deprecated (4.0)
+ */
+@Deprecated
 public class PreFlexFields extends FieldsProducer {
   
   private static final boolean DEBUG_SURROGATES = false;

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/preflex/SegmentTermDocs.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/preflex/SegmentTermDocs.java?rev=1042501&r1=1042500&r2=1042501&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/preflex/SegmentTermDocs.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/preflex/SegmentTermDocs.java Mon Dec  6 00:47:16 2010
@@ -26,7 +26,7 @@ import org.apache.lucene.index.codecs.st
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.util.Bits;
 
-/** @deprecated 
+/** @deprecated (4.0)
  *  @lucene.experimental */
 @Deprecated
 public class SegmentTermDocs {

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/preflex/SegmentTermEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/preflex/SegmentTermEnum.java?rev=1042501&r1=1042500&r2=1042501&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/preflex/SegmentTermEnum.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/preflex/SegmentTermEnum.java Mon Dec  6 00:47:16 2010
@@ -26,7 +26,7 @@ import org.apache.lucene.index.IndexForm
 import org.apache.lucene.index.IndexFormatTooNewException;
 
 /**
- * @deprecated No longer used with flex indexing, except for
+ * @deprecated (4.0) No longer used with flex indexing, except for
  * reading old segments 
  * @lucene.experimental */
 

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/preflex/SegmentTermPositions.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/preflex/SegmentTermPositions.java?rev=1042501&r1=1042500&r2=1042501&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/preflex/SegmentTermPositions.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/preflex/SegmentTermPositions.java Mon Dec  6 00:47:16 2010
@@ -23,7 +23,11 @@ import org.apache.lucene.index.FieldInfo
 import org.apache.lucene.index.Term;
 import org.apache.lucene.store.IndexInput;
 
-/** @lucene.experimental */
+/**
+ * @lucene.experimental
+ * @deprecated (4.0)
+ */
+@Deprecated
 public final class SegmentTermPositions
 extends SegmentTermDocs  {
   private IndexInput proxStream;

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/preflex/TermBuffer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/preflex/TermBuffer.java?rev=1042501&r1=1042500&r2=1042501&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/preflex/TermBuffer.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/preflex/TermBuffer.java Mon Dec  6 00:47:16 2010
@@ -25,6 +25,11 @@ import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.index.FieldInfos;
 
+/**
+ * @lucene.experimental
+ * @deprecated (4.0)
+ */
+@Deprecated
 final class TermBuffer implements Cloneable {
 
   private String field;

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/preflex/TermInfo.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/preflex/TermInfo.java?rev=1042501&r1=1042500&r2=1042501&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/preflex/TermInfo.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/preflex/TermInfo.java Mon Dec  6 00:47:16 2010
@@ -19,7 +19,7 @@ package org.apache.lucene.index.codecs.p
 
 /** A TermInfo is the record of information stored for a
  * term
- * @deprecated This class is no longer used in flexible
+ * @deprecated (4.0) This class is no longer used in flexible
  * indexing. */
 
 @Deprecated

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/preflex/TermInfosReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/preflex/TermInfosReader.java?rev=1042501&r1=1042500&r2=1042501&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/preflex/TermInfosReader.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/codecs/preflex/TermInfosReader.java Mon Dec  6 00:47:16 2010
@@ -30,7 +30,7 @@ import org.apache.lucene.util.DoubleBarr
 /** This stores a monotonically increasing set of <Term, TermInfo> pairs in a
  * Directory.  Pairs are accessed either by Term or by ordinal position the
  * set
- * @deprecated This class has been replaced by
+ * @deprecated (4.0) This class has been replaced by
  * FormatPostingsTermsDictReader, except for reading old segments. 
  * @lucene.experimental
  */

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/Bytes.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/Bytes.java?rev=1042501&r1=1042500&r2=1042501&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/Bytes.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/Bytes.java Mon Dec  6 00:47:16 2010
@@ -49,6 +49,7 @@ import org.apache.lucene.util.PagedBytes
  * <p>
  * NOTE: Each byte[] must be <= 32768 bytes in length
  * </p>
+ * @lucene.experimental
  */
 // TODO - add bulk copy where possible
 public final class Bytes {
@@ -164,8 +165,6 @@ public final class Bytes {
     public ValuesEnum getEnum(AttributeSource attrSource) throws IOException {
       final MissingValue missing = getMissing();
       return new SourceEnum(attrSource, type(), this, maxDoc()) {
-        final BytesRef bytesRef = attr.bytes();
-
         @Override
         public int advance(int target) throws IOException {
           if (target >= numDocs) {
@@ -250,7 +249,6 @@ public final class Bytes {
     public ValuesEnum getEnum(AttributeSource attrSource) throws IOException {
       final MissingValue missing = getMissing();
       return new SourceEnum(attrSource, type(), this, maxDoc()) {
-        final BytesRef bytesRef = attr.bytes();
 
         @Override
         public int advance(int target) throws IOException {
@@ -332,19 +330,19 @@ public final class Bytes {
       add(docID, bytesRef);
     }
 
-    @Override
-    protected void setNextAttribute(ValuesAttribute attr) {
-      bytesRef = attr.bytes();
-      assert bytesRef != null;
-    }
 
     @Override
-    public void add(int docID, ValuesAttribute attr) throws IOException {
+    public void add(int docID, PerDocFieldValues docValues) throws IOException {
       final BytesRef ref;
-      if ((ref = attr.bytes()) != null) {
+      if ((ref = docValues.getBytes()) != null) {
         add(docID, ref);
       }
     }
+    
+    @Override
+    protected void setNextEnum(ValuesEnum valuesEnum) {
+      bytesRef = valuesEnum.bytes();
+    }
 
     @Override
     public void files(Collection<String> files) throws IOException {

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/DocValues.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/DocValues.java?rev=1042501&r1=1042500&r2=1042501&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/DocValues.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/DocValues.java Mon Dec  6 00:47:16 2010
@@ -22,7 +22,10 @@ import java.util.Comparator;
 
 import org.apache.lucene.util.AttributeSource;
 import org.apache.lucene.util.BytesRef;
-
+/**
+ * 
+ * @lucene.experimental
+ */
 public abstract class DocValues implements Closeable {
 
   public static final DocValues[] EMPTY_ARRAY = new DocValues[0];
@@ -93,7 +96,7 @@ public abstract class DocValues implemen
     }
 
     public ValuesEnum getEnum() throws IOException {
-      return getEnum(new AttributeSource());
+      return getEnum(null);
     }
     
     public MissingValue getMissing() {
@@ -114,7 +117,6 @@ public abstract class DocValues implemen
 
     SourceEnum(AttributeSource attrs, Values type, Source source, int numDocs) {
       super(attrs, type);
-      
       this.source = source;
       this.numDocs = numDocs;
     }

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/FixedDerefBytesImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/FixedDerefBytesImpl.java?rev=1042501&r1=1042500&r2=1042501&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/FixedDerefBytesImpl.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/FixedDerefBytesImpl.java Mon Dec  6 00:47:16 2010
@@ -39,7 +39,9 @@ import org.apache.lucene.util.packed.Pac
 
 // Stores fixed-length byte[] by deref, ie when two docs
 // have the same value, they store only 1 byte[]
-
+/**
+ * @lucene.experimental
+ */
 class FixedDerefBytesImpl {
 
   static final String CODEC_NAME = "FixedDerefBytes";
@@ -161,7 +163,7 @@ class FixedDerefBytesImpl {
         if (id == 0) {
           return null;
         }
-        return data.fill(bytesRef, ((id - 1) * size), size);
+        return data.fillSlice(bytesRef, ((id - 1) * size), size);
       }
 
       @Override
@@ -191,7 +193,6 @@ class FixedDerefBytesImpl {
       private final PackedInts.ReaderIterator idx;
       protected final long fp;
       private final int size;
-      protected final BytesRef ref;
       private final int valueCount;
       private int pos = -1;
 
@@ -204,17 +205,26 @@ class FixedDerefBytesImpl {
           IndexInput idxIn, int size, Values enumType)
           throws IOException {
         super(source, enumType);
-        ref = attr.bytes();
         this.datIn = datIn;
-        this.size = size == -1 ? 128 : size;
+        this.size = size;
         idxIn.readInt();// read valueCount
         idx = PackedInts.getReaderIterator(idxIn);
         fp = datIn.getFilePointer();
-        ref.grow(this.size);
-        ref.length = this.size;
-        ref.offset = 0;
+        bytesRef.grow(this.size);
+        bytesRef.length = this.size;
+        bytesRef.offset = 0;
         valueCount = idx.size();
       }
+      
+
+      protected void copyReferences(ValuesEnum valuesEnum) {
+        bytesRef = valuesEnum.bytesRef;
+        if(bytesRef.bytes.length < size) {
+          bytesRef.grow(size);
+        }
+        bytesRef.length = size;
+        bytesRef.offset = 0;
+      }
 
       @Override
       public int advance(int target) throws IOException {
@@ -226,7 +236,7 @@ class FixedDerefBytesImpl {
             }
           }
           pos = idx.ord();
-          fill(address, ref);
+          fill(address, bytesRef);
           return pos;
         }
         return pos = NO_MORE_DOCS;

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/FixedSortedBytesImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/FixedSortedBytesImpl.java?rev=1042501&r1=1042500&r2=1042501&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/FixedSortedBytesImpl.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/FixedSortedBytesImpl.java Mon Dec  6 00:47:16 2010
@@ -43,6 +43,9 @@ import org.apache.lucene.util.packed.Pac
 // Stores fixed-length byte[] by deref, ie when two docs
 // have the same value, they store only 1 byte[]
 
+/**
+ * @lucene.experimental
+ */
 class FixedSortedBytesImpl {
 
   static final String CODEC_NAME = "FixedSortedBytes";
@@ -208,7 +211,7 @@ class FixedSortedBytesImpl {
       }
       @Override
       protected BytesRef deref(int ord, BytesRef bytesRef) {
-        return data.fill(bytesRef, (ord* size), size);
+        return data.fillSlice(bytesRef, (ord* size), size);
       }
 
       @Override

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/FixedStraightBytesImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/FixedStraightBytesImpl.java?rev=1042501&r1=1042500&r2=1042501&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/FixedStraightBytesImpl.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/FixedStraightBytesImpl.java Mon Dec  6 00:47:16 2010
@@ -30,7 +30,9 @@ import org.apache.lucene.util.PagedBytes
 
 // Simplest storage: stores fixed length byte[] per
 // document, with no dedup and no sorting.
-
+/**
+ * @lucene.experimental
+ */
 class FixedStraightBytesImpl {
 
   static final String CODEC_NAME = "FixedStraightBytes";
@@ -145,7 +147,7 @@ class FixedStraightBytesImpl {
       
       @Override
       public BytesRef getBytes(int docID, BytesRef bytesRef) { 
-        return data.fill(bytesRef, docID * size, size);
+        return data.fillSlice(bytesRef, docID * size, size);
       }
 
       @Override
@@ -175,19 +177,26 @@ class FixedStraightBytesImpl {
       private final int maxDoc;
       private int pos = -1;
       private final long fp;
-      private final BytesRef ref;
 
       public FixedStraightBytesEnum(AttributeSource source, IndexInput datIn, int size, int maxDoc) throws IOException{
         super(source, Values.BYTES_FIXED_STRAIGHT);
         this.datIn = datIn;
         this.size = size;
         this.maxDoc = maxDoc;
-        ref = attr.bytes();
-        ref.grow(size);
-        ref.length = size;
-        ref.offset = 0;
+        bytesRef.grow(size);
+        bytesRef.length = size;
+        bytesRef.offset = 0;
         fp = datIn.getFilePointer();
       }
+      
+      protected void copyReferences(ValuesEnum valuesEnum) {
+        bytesRef = valuesEnum.bytesRef;
+        if(bytesRef.bytes.length < size) {
+          bytesRef.grow(size);
+        }
+        bytesRef.length = size;
+        bytesRef.offset = 0;
+      }
      
       public void close() throws IOException {
         datIn.close();
@@ -200,7 +209,7 @@ class FixedStraightBytesImpl {
         }
         if((target-1) != pos) // pos inc == 1
           datIn.seek(fp + target * size);
-        datIn.readBytes(ref.bytes, 0, size);
+        datIn.readBytes(bytesRef.bytes, 0, size);
         return pos = target;
       }
       

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/Floats.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/Floats.java?rev=1042501&r1=1042500&r2=1042501&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/Floats.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/Floats.java Mon Dec  6 00:47:16 2010
@@ -29,11 +29,11 @@ import org.apache.lucene.store.IndexOutp
 import org.apache.lucene.util.AttributeSource;
 import org.apache.lucene.util.CodecUtil;
 import org.apache.lucene.util.FloatsRef;
-import org.apache.lucene.util.RamUsageEstimator;
 
 /**
  * Exposes writer/reader for floating point values. You can specify 4 (java
  * float) or 8 (java double) byte precision.
+ * @lucene.experimental
  */
 // TODO - add bulk copy where possible
 public class Floats {
@@ -97,15 +97,13 @@ public class Floats {
     }
 
     @Override
-    public void add(int docID, ValuesAttribute attr) throws IOException {
-      final FloatsRef ref;
-      if ((ref = attr.floats()) != null)
-        add(docID, ref.get());
+    public void add(int docID, PerDocFieldValues docValues) throws IOException {
+        add(docID, docValues.getFloat());
     }
 
     @Override
-    protected void setNextAttribute(ValuesAttribute attr) {
-      floatsRef = attr.floats();
+    protected void setNextEnum(ValuesEnum valuesEnum) {
+      floatsRef = valuesEnum.getFloat();
     }
 
     protected abstract int fillDefault(int num) throws IOException;
@@ -289,16 +287,10 @@ public class Floats {
         return values.get(docID);
       }
 
-      public long ramBytesUsed() {
-        return RamUsageEstimator.NUM_BYTES_ARRAY_HEADER + values.limit()
-            * RamUsageEstimator.NUM_BYTES_FLOAT;
-      }
-
       @Override
       public ValuesEnum getEnum(AttributeSource attrSource) throws IOException {
         final MissingValue missing = getMissing();
         return new SourceEnum(attrSource, Values.SIMPLE_FLOAT_4BYTE, this, maxDoc) {
-          private final FloatsRef ref = attr.floats();
           @Override
           public int advance(int target) throws IOException {
             if (target >= numDocs)
@@ -308,7 +300,7 @@ public class Floats {
                 return pos = NO_MORE_DOCS;
               }
             }
-            ref.floats[ref.offset] = source.getFloat(target);
+            floatsRef.floats[floatsRef.offset] = source.getFloat(target);
             return pos = target;
           }
         };
@@ -334,16 +326,10 @@ public class Floats {
         return values.get(docID);
       }
 
-      public long ramBytesUsed() {
-        return RamUsageEstimator.NUM_BYTES_ARRAY_HEADER + values.limit()
-            * RamUsageEstimator.NUM_BYTES_DOUBLE;
-      }
-
       @Override
       public ValuesEnum getEnum(AttributeSource attrSource) throws IOException {
         final MissingValue missing = getMissing();
         return new SourceEnum(attrSource, type(), this, maxDoc) {
-          private final FloatsRef ref = attr.floats();
           @Override
           public int advance(int target) throws IOException {
             if (target >= numDocs)
@@ -353,7 +339,7 @@ public class Floats {
                 return pos = NO_MORE_DOCS;
               }
             }
-            ref.floats[ref.offset] = source.getFloat(target);
+            floatsRef.floats[floatsRef.offset] = source.getFloat(target);
             return pos = target;
           }
         };
@@ -405,8 +391,8 @@ public class Floats {
         if (++target >= maxDoc)
           return pos = NO_MORE_DOCS;
       }
-      ref.floats[0] = Float.intBitsToFloat(intBits);
-      ref.offset = 0;
+      floatsRef.floats[0] = Float.intBitsToFloat(intBits);
+      floatsRef.offset = 0;
       return pos = target;
     }
 
@@ -442,8 +428,8 @@ public class Floats {
         if (++target >= maxDoc)
           return pos = NO_MORE_DOCS;
       }
-      ref.floats[0] = Double.longBitsToDouble(value);
-      ref.offset = 0;
+      floatsRef.floats[0] = Double.longBitsToDouble(value);
+      floatsRef.offset = 0;
       return pos = target;
     }
 
@@ -467,7 +453,6 @@ public class Floats {
     protected final int precision;
     protected final int maxDoc;
     protected final long fp;
-    protected final FloatsRef ref;
 
     FloatsEnumImpl(AttributeSource source, IndexInput dataIn, int precision,
         int maxDoc, Values type) throws IOException {
@@ -477,8 +462,7 @@ public class Floats {
       this.precision = precision;
       this.maxDoc = maxDoc;
       fp = dataIn.getFilePointer();
-      this.ref = attr.floats();
-      this.ref.offset = 0;
+      floatsRef.offset = 0;
     }
 
     @Override

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/Ints.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/Ints.java?rev=1042501&r1=1042500&r2=1042501&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/Ints.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/Ints.java Mon Dec  6 00:47:16 2010
@@ -1,4 +1,5 @@
 package org.apache.lucene.index.values;
+
 /**
  * Licensed to the Apache Software Foundation (ASF) under one or more
  * contributor license agreements.  See the NOTICE file distributed with
@@ -23,14 +24,17 @@ import org.apache.lucene.index.values.Pa
 import org.apache.lucene.index.values.PackedIntsImpl.IntsWriter;
 import org.apache.lucene.store.Directory;
 
-//TODO - add bulk copy where possible
+/**
+ * @lucene.experimental
+ */
 public class Ints {
+//TODO - add bulk copy where possible
 
   private Ints() {
   }
 
-  public static Writer getWriter(Directory dir, String id, boolean useFixedArray, AtomicLong bytesUsed)
-      throws IOException {
+  public static Writer getWriter(Directory dir, String id,
+      boolean useFixedArray, AtomicLong bytesUsed) throws IOException {
     // TODO - implement fixed?!
     return new IntsWriter(dir, id, bytesUsed);
   }

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/MultiDocValues.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/MultiDocValues.java?rev=1042501&r1=1042500&r2=1042501&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/MultiDocValues.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/MultiDocValues.java Mon Dec  6 00:47:16 2010
@@ -23,6 +23,9 @@ import org.apache.lucene.util.AttributeS
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.ReaderUtil;
 
+/**
+ * @lucene.experimental
+ */
 public class MultiDocValues extends DocValues {
 
   public static class DocValuesIndex {
@@ -121,6 +124,8 @@ public class MultiDocValues extends DocV
       maxDoc = last.start + last.length;
       final DocValuesIndex idx = docValuesIdx[0];
       currentEnum = idx.docValues.getEnum(this.attributes());
+      currentEnum.copyReferences(this);
+      intsRef = currentEnum.intsRef;
       currentMax = idx.length;
       currentStart = 0;
       this.starts = starts;
@@ -143,6 +148,7 @@ public class MultiDocValues extends DocV
           final int idx = ReaderUtil.subIndex(target, starts);
           currentEnum.close();
           currentEnum = docValuesIdx[idx].docValues.getEnum(this.attributes());
+          currentEnum.copyReferences(this);
           currentStart = docValuesIdx[idx].start;
           currentMax = currentStart + docValuesIdx[idx].length;
           relativeDoc = target - currentStart;
@@ -250,10 +256,6 @@ public class MultiDocValues extends DocV
       return missingValue.longValue;
     }
 
-    public long ramBytesUsed() {
-      return 0;
-    }
-
     @Override
     public ValuesEnum getEnum(AttributeSource attrSource) throws IOException {
       return ValuesEnum.emptyEnum(type);

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/PackedIntsImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/PackedIntsImpl.java?rev=1042501&r1=1042500&r2=1042501&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/PackedIntsImpl.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/PackedIntsImpl.java Mon Dec  6 00:47:16 2010
@@ -32,7 +32,10 @@ import org.apache.lucene.util.OpenBitSet
 import org.apache.lucene.util.RamUsageEstimator;
 import org.apache.lucene.util.packed.PackedInts;
 
-/** Stores ints packed with fixed-bit precision. */
+/** Stores ints packed with fixed-bit precision.
+ * 
+ * @lucene.experimental
+ * */
 class PackedIntsImpl {
 
   private static final String CODEC_NAME = "PackedInts";
@@ -130,16 +133,13 @@ class PackedIntsImpl {
     }
 
     @Override
-    protected void setNextAttribute(ValuesAttribute attr) {
-      intsRef = attr.ints();
+    protected void setNextEnum(ValuesEnum valuesEnum) {
+      intsRef = valuesEnum.getInt();
     }
 
     @Override
-    public void add(int docID, ValuesAttribute attr) throws IOException {
-      final LongsRef ref;
-      if ((ref = attr.ints()) != null) {
-        add(docID, ref.get());
-      }
+    public void add(int docID, PerDocFieldValues docValues) throws IOException {
+        add(docID, docValues.getInt());
     }
 
     @Override
@@ -191,17 +191,10 @@ class PackedIntsImpl {
         return minValue + values.get(docID);
       }
 
-      public long ramBytesUsed() {
-        // TODO(simonw): move that to PackedInts?
-        return RamUsageEstimator.NUM_BYTES_ARRAY_HEADER
-            + values.getBitsPerValue() * values.size();
-      }
-
       @Override
       public ValuesEnum getEnum(AttributeSource attrSource) throws IOException {
         final MissingValue missing = getMissing();
         return new SourceEnum(attrSource, type(), this, values.size()) {
-          private final LongsRef ref = attr.ints();
           @Override
           public int advance(int target) throws IOException {
             if (target >= numDocs)
@@ -211,7 +204,7 @@ class PackedIntsImpl {
                 return pos = NO_MORE_DOCS;
               }
             }
-            ref.ints[ref.offset] = source.getInt(target);
+            intsRef.ints[intsRef.offset] = source.getInt(target);
             return pos = target;
           }
         };
@@ -246,15 +239,13 @@ class PackedIntsImpl {
     private long minValue;
     private final IndexInput dataIn;
     private final long defaultValue;
-    private LongsRef ref;
     private final int maxDoc;
     private int pos = -1;
 
     private IntsEnumImpl(AttributeSource source, IndexInput dataIn)
         throws IOException {
       super(source, Values.PACKED_INTS);
-      this.ref = attr.ints();
-      this.ref.offset = 0;
+      intsRef.offset = 0;
       this.dataIn = dataIn;
       dataIn.seek(CodecUtil.headerLength(CODEC_NAME));
       minValue = dataIn.readLong();
@@ -281,8 +272,8 @@ class PackedIntsImpl {
         }
         val = ints.advance(target);
       }
-      ref.ints[0] = minValue + val;
-      ref.offset = 0; // can we skip this?
+      intsRef.ints[0] = minValue + val;
+      intsRef.offset = 0; // can we skip this?
       return pos = target;
     }
 

Added: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/PerDocFieldValues.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/PerDocFieldValues.java?rev=1042501&view=auto
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/PerDocFieldValues.java (added)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/PerDocFieldValues.java Mon Dec  6 00:47:16 2010
@@ -0,0 +1,53 @@
+package org.apache.lucene.index.values;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+import java.util.Comparator;
+
+import org.apache.lucene.util.BytesRef;
+
+/**
+ * 
+ * @lucene.experimental
+ */
+public interface PerDocFieldValues {
+
+  public void setInt(long value);
+
+  public void setFloat(float value);
+
+  public void setFloat(double value);
+
+  public void setBytes(BytesRef value, Values type);
+
+  public void setBytes(BytesRef value, Values type, Comparator<BytesRef> comp);
+
+  public BytesRef getBytes();
+
+  public Comparator<BytesRef> bytesComparator();
+
+  public double getFloat();
+
+  public long getInt();
+
+  public void setBytesComparator(Comparator<BytesRef> comp);
+
+  public void setType(Values type);
+
+  public Values type();
+
+}
\ No newline at end of file

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/SourceCache.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/SourceCache.java?rev=1042501&r1=1042500&r2=1042501&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/SourceCache.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/SourceCache.java Mon Dec  6 00:47:16 2010
@@ -1,12 +1,5 @@
 package org.apache.lucene.index.values;
 
-import java.io.IOException;
-import java.util.Comparator;
-
-import org.apache.lucene.index.values.DocValues.SortedSource;
-import org.apache.lucene.index.values.DocValues.Source;
-import org.apache.lucene.util.BytesRef;
-
 /**
  * Licensed to the Apache Software Foundation (ASF) under one or more
  * contributor license agreements.  See the NOTICE file distributed with
@@ -24,6 +17,13 @@ import org.apache.lucene.util.BytesRef;
  * limitations under the License.
  */
 
+import java.io.IOException;
+import java.util.Comparator;
+
+import org.apache.lucene.index.values.DocValues.SortedSource;
+import org.apache.lucene.index.values.DocValues.Source;
+import org.apache.lucene.util.BytesRef;
+
 /**
  * Per {@link DocValues} {@link Source} cache.
  */

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/Values.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/Values.java?rev=1042501&r1=1042500&r2=1042501&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/Values.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/Values.java Mon Dec  6 00:47:16 2010
@@ -22,7 +22,7 @@ package org.apache.lucene.index.values;
  *  values into RAM, exposing a random access API, when
  *  loaded.
  *
- * @lucene.experimenta 
+ * @lucene.experimental 
  */
 public enum Values {
 

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/ValuesEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/ValuesEnum.java?rev=1042501&r1=1042500&r2=1042501&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/ValuesEnum.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/ValuesEnum.java Mon Dec  6 00:47:16 2010
@@ -24,10 +24,16 @@ import org.apache.lucene.util.AttributeS
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.FloatsRef;
 import org.apache.lucene.util.LongsRef;
-
+/**
+ * 
+ * @lucene.experimental
+ */
 public abstract class ValuesEnum extends DocIdSetIterator {
   private AttributeSource source;
-  protected final ValuesAttribute attr;
+  private Values enumType;
+  protected BytesRef bytesRef;
+  protected FloatsRef floatsRef;
+  protected LongsRef intsRef;
 
   protected ValuesEnum(Values enumType) {
     this(null, enumType);
@@ -35,26 +41,46 @@ public abstract class ValuesEnum extends
 
   protected ValuesEnum(AttributeSource source, Values enumType) {
     this.source = source;
-    boolean setType = !hasAttribute(ValuesAttribute.class);
-    attr = addAttribute(ValuesAttribute.class);
-    if (setType)
-      attr.setType(enumType);
+    this.enumType = enumType;
+    switch (enumType) {
+    case BYTES_FIXED_DEREF:
+    case BYTES_FIXED_SORTED:
+    case BYTES_FIXED_STRAIGHT:
+    case BYTES_VAR_DEREF:
+    case BYTES_VAR_SORTED:
+    case BYTES_VAR_STRAIGHT:
+      bytesRef = new BytesRef();
+      break;
+    case PACKED_INTS:
+      intsRef = new LongsRef(1);
+      break;
+    case SIMPLE_FLOAT_4BYTE:
+    case SIMPLE_FLOAT_8BYTE:
+      floatsRef = new FloatsRef(1);
+      break;  
+    }
   }
 
   public Values type() {
-    return attr.type();
+    return enumType;
   }
 
   public BytesRef bytes() {
-    return attr.bytes();
+    return bytesRef;
   }
 
   public FloatsRef getFloat() {
-    return attr.floats();
+    return floatsRef;
   }
 
   public LongsRef getInt() {
-    return attr.ints();
+    return intsRef;
+  }
+  
+  protected void copyReferences(ValuesEnum valuesEnum) {
+    intsRef = valuesEnum.intsRef;
+    floatsRef = valuesEnum.floatsRef;
+    bytesRef = valuesEnum.bytesRef;
   }
 
   public AttributeSource attributes() {

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/VarDerefBytesImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/VarDerefBytesImpl.java?rev=1042501&r1=1042500&r2=1042501&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/VarDerefBytesImpl.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/VarDerefBytesImpl.java Mon Dec  6 00:47:16 2010
@@ -44,6 +44,9 @@ import org.apache.lucene.util.packed.Pac
 // have the same value, they store only 1 byte[] and both
 // docs reference that single source
 
+/**
+ * @lucene.experimental
+ */
 class VarDerefBytesImpl {
 
   static final String CODEC_NAME = "VarDerefBytes";

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/VarSortedBytesImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/VarSortedBytesImpl.java?rev=1042501&r1=1042500&r2=1042501&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/VarSortedBytesImpl.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/VarSortedBytesImpl.java Mon Dec  6 00:47:16 2010
@@ -43,6 +43,9 @@ import org.apache.lucene.util.packed.Pac
 // have the same value, they store only 1 byte[] and both
 // docs reference that single source
 
+/**
+ * @lucene.experimental
+ */
 class VarSortedBytesImpl {
 
   static final String CODEC_NAME = "VarDerefBytes";
@@ -212,7 +215,7 @@ class VarSortedBytesImpl {
           nextOffset = ordToOffsetIndex.get(1 + ord);
         }
         final long offset = ordToOffsetIndex.get(ord);
-        data.fill(bytesRef, offset , (int)(nextOffset - offset));
+        data.fillSlice(bytesRef, offset , (int)(nextOffset - offset));
         return bytesRef;
       }
 
@@ -233,12 +236,10 @@ class VarSortedBytesImpl {
     }
 
     private static class VarSortedBytesEnum extends ValuesEnum {
-
       private PackedInts.Reader docToOrdIndex;
       private PackedInts.Reader ordToOffsetIndex;
       private IndexInput idxIn;
       private IndexInput datIn;
-      private final BytesRef bytesRef;
       private int valueCount;
       private long totBytes;
       private int docCount;
@@ -248,7 +249,6 @@ class VarSortedBytesImpl {
       protected VarSortedBytesEnum(AttributeSource source, IndexInput datIn,
           IndexInput idxIn) throws IOException {
         super(source, Values.BYTES_VAR_SORTED);
-        bytesRef = attr.bytes();
         totBytes = idxIn.readLong();
         // keep that in memory to prevent lots of disk seeks
         docToOrdIndex = PackedInts.getReader(idxIn);

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/VarStraightBytesImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/VarStraightBytesImpl.java?rev=1042501&r1=1042500&r2=1042501&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/VarStraightBytesImpl.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/VarStraightBytesImpl.java Mon Dec  6 00:47:16 2010
@@ -34,6 +34,9 @@ import org.apache.lucene.util.packed.Pac
 
 // Variable length byte[] per document, no sharing
 
+/**
+ * @lucene.experimental
+ */
 class VarStraightBytesImpl {
 
   static final String CODEC_NAME = "VarStraightBytes";
@@ -135,7 +138,7 @@ class VarStraightBytesImpl {
         final long address = addresses.get(docID);
         final int length = docID == maxDoc - 1 ? (int) (totalLengthInBytes - address)
             : (int) (addresses.get(1 + docID) - address);
-        return data.fill(bytesRef, address, length);
+        return data.fillSlice(bytesRef, address, length);
       }
 
       @Override
@@ -165,7 +168,6 @@ class VarStraightBytesImpl {
       private final IndexInput idxIn;
       private final long fp;
       private final int totBytes;
-      private final BytesRef ref;
       private int pos = -1;
 
       protected VarStraightBytesEnum(AttributeSource source, IndexInput datIn,
@@ -176,8 +178,6 @@ class VarStraightBytesImpl {
         addresses = PackedInts.getReader(idxIn);
         this.datIn = datIn;
         this.idxIn = idxIn;
-        ref = attr.bytes();
-
       }
 
       @Override
@@ -193,17 +193,18 @@ class VarStraightBytesImpl {
         }
         final long addr = addresses.get(target);
         if (addr == totBytes) { // empty values at the end
-          ref.length = 0;
-          ref.offset = 0;
+          bytesRef.length = 0;
+          bytesRef.offset = 0;
           return pos = target;
         }
         datIn.seek(fp + addr);
         final int size = (int) (target == maxDoc - 1 ? totBytes - addr
             : addresses.get(target + 1) - addr);
-        if (ref.bytes.length < size)
-          ref.grow(size);
-        ref.length = size;
-        datIn.readBytes(ref.bytes, 0, size);
+        if (bytesRef.bytes.length < size) {
+          bytesRef.grow(size);
+        }
+        bytesRef.length = size;
+        datIn.readBytes(bytesRef.bytes, 0, size);
         return pos = target;
       }
 

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/Writer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/Writer.java?rev=1042501&r1=1042500&r2=1042501&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/Writer.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/index/values/Writer.java Mon Dec  6 00:47:16 2010
@@ -25,6 +25,9 @@ import org.apache.lucene.store.Directory
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
 
+/**
+ * @lucene.experimental
+ */
 public abstract class Writer extends DocValuesConsumer {
 
   protected Writer(AtomicLong bytesUsed) {
@@ -51,8 +54,8 @@ public abstract class Writer extends Doc
 
   /** Records the specfied value for the docID */
   protected abstract void add(int docID) throws IOException;
-
-  protected abstract void setNextAttribute(ValuesAttribute attr);
+  
+  protected abstract void setNextEnum(ValuesEnum valuesEnum);
 
   /** Finish writing, close any files */
   public abstract void finish(int docCount) throws IOException;
@@ -63,8 +66,7 @@ public abstract class Writer extends Doc
     final ValuesEnum valEnum = state.reader.getEnum();
     assert valEnum != null;
     try {
-      final ValuesAttribute attr = valEnum.addAttribute(ValuesAttribute.class);
-      setNextAttribute(attr);
+      setNextEnum(valEnum);
       int docID = state.docBase;
       final Bits bits = state.bits;
       final int docCount = state.docCount;

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/queryParser/CharStream.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/queryParser/CharStream.java?rev=1042501&r1=1042500&r2=1042501&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/queryParser/CharStream.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/queryParser/CharStream.java Mon Dec  6 00:47:16 2010
@@ -28,14 +28,14 @@ public interface CharStream {
 
   /**
    * Returns the column position of the character last read.
-   * @deprecated
+   * @deprecated (gen)
    * @see #getEndColumn
    */
   int getColumn();
 
   /**
    * Returns the line number of the character last read.
-   * @deprecated
+   * @deprecated (gen)
    * @see #getEndLine
    */
   int getLine();

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/queryParser/QueryParser.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/queryParser/QueryParser.java?rev=1042501&r1=1042500&r2=1042501&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/queryParser/QueryParser.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/queryParser/QueryParser.java Mon Dec  6 00:47:16 2010
@@ -18,7 +18,6 @@ import org.apache.lucene.analysis.Cachin
 import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
 import org.apache.lucene.analysis.tokenattributes.TermToBytesRefAttribute;
-import org.apache.lucene.document.DateField;
 import org.apache.lucene.document.DateTools;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.search.BooleanClause;
@@ -77,10 +76,8 @@ import org.apache.lucene.util.VirtualMet
  * <tt>date:[6/1/2005 TO 6/4/2005]</tt> produces a range query that searches
  * for "date" fields between 2005-06-01 and 2005-06-04. Note that the format
  * of the accepted input depends on {@link #setLocale(Locale) the locale}.
- * By default a date is converted into a search term using the deprecated
- * {@link DateField} for compatibility reasons.
- * To use the new {@link DateTools} to convert dates, a
- * {@link org.apache.lucene.document.DateTools.Resolution} has to be set.
+ * A {@link org.apache.lucene.document.DateTools.Resolution} has to be set,
+ * if you want to use {@link DateTools} for date conversion.
  * </p>
  * <p>
  * The date resolution that shall be used for RangeQueries can be set
@@ -91,10 +88,9 @@ import org.apache.lucene.util.VirtualMet
  * resolutions take, if set, precedence over the default date resolution.
  * </p>
  * <p>
- * If you use neither {@link DateField} nor {@link DateTools} in your
- * index, you can create your own
+ * If you don't use {@link DateTools} in your index, you can create your own
  * query parser that inherits QueryParser and overwrites
- * {@link #getRangeQuery(String, String, String, boolean)} to
+ * {@link #getRangeQuery(String, String, String, boolean, boolean)} to
  * use a different method for date conversion.
  * </p>
  *
@@ -108,8 +104,6 @@ import org.apache.lucene.util.VirtualMet
  * <p><b>NOTE</b>: You must specify the required {@link Version}
  * compatibility when creating QueryParser:
  * <ul>
- *    <li> As of 2.9, {@link #setEnablePositionIncrements} is true by
- *         default.
  *    <li> As of 3.1, {@link #setAutoGeneratePhraseQueries} is false by
  *         default.
  * </ul>

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/queryParser/QueryParser.jj
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/queryParser/QueryParser.jj?rev=1042501&r1=1042500&r2=1042501&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/queryParser/QueryParser.jj (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/queryParser/QueryParser.jj Mon Dec  6 00:47:16 2010
@@ -42,7 +42,6 @@ import org.apache.lucene.analysis.Cachin
 import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
 import org.apache.lucene.analysis.tokenattributes.TermToBytesRefAttribute;
-import org.apache.lucene.document.DateField;
 import org.apache.lucene.document.DateTools;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.search.BooleanClause;
@@ -101,10 +100,8 @@ import org.apache.lucene.util.VirtualMet
  * <tt>date:[6/1/2005 TO 6/4/2005]</tt> produces a range query that searches
  * for "date" fields between 2005-06-01 and 2005-06-04. Note that the format
  * of the accepted input depends on {@link #setLocale(Locale) the locale}.
- * By default a date is converted into a search term using the deprecated
- * {@link DateField} for compatibility reasons.
- * To use the new {@link DateTools} to convert dates, a
- * {@link org.apache.lucene.document.DateTools.Resolution} has to be set.
+ * A {@link org.apache.lucene.document.DateTools.Resolution} has to be set,
+ * if you want to use {@link DateTools} for date conversion.
  * </p>
  * <p>
  * The date resolution that shall be used for RangeQueries can be set
@@ -115,10 +112,9 @@ import org.apache.lucene.util.VirtualMet
  * resolutions take, if set, precedence over the default date resolution.
  * </p>
  * <p>
- * If you use neither {@link DateField} nor {@link DateTools} in your
- * index, you can create your own
+ * If you don't use {@link DateTools} in your index, you can create your own
  * query parser that inherits QueryParser and overwrites
- * {@link #getRangeQuery(String, String, String, boolean)} to
+ * {@link #getRangeQuery(String, String, String, boolean, boolean)} to
  * use a different method for date conversion.
  * </p>
  *
@@ -132,8 +128,6 @@ import org.apache.lucene.util.VirtualMet
  * <p><b>NOTE</b>: You must specify the required {@link Version}
  * compatibility when creating QueryParser:
  * <ul>
- *    <li> As of 2.9, {@link #setEnablePositionIncrements} is true by
- *         default.
  *    <li> As of 3.1, {@link #setAutoGeneratePhraseQueries} is false by
  *         default.
  * </ul>

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/queryParser/QueryParserBase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/queryParser/QueryParserBase.java?rev=1042501&r1=1042500&r2=1042501&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/queryParser/QueryParserBase.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/queryParser/QueryParserBase.java Mon Dec  6 00:47:16 2010
@@ -17,25 +17,23 @@
 
 package org.apache.lucene.queryParser;
 
+import java.io.IOException;
+import java.io.StringReader;
+import java.text.Collator;
+import java.text.DateFormat;
+import java.util.*;
+
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.CachingTokenFilter;
 import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
 import org.apache.lucene.analysis.tokenattributes.TermToBytesRefAttribute;
-import org.apache.lucene.document.DateField;
 import org.apache.lucene.document.DateTools;
 import org.apache.lucene.index.Term;
+import org.apache.lucene.queryParser.QueryParser.Operator;
 import org.apache.lucene.search.*;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.Version;
-import org.apache.lucene.util.VirtualMethod;
-import org.apache.lucene.queryParser.QueryParser.Operator;
-
-import java.io.IOException;
-import java.io.StringReader;
-import java.text.Collator;
-import java.text.DateFormat;
-import java.util.*;
 
 /** This class is overridden by QueryParser in QueryParser.jj
  * and acts to eparate the majority of the Java code from the .jj grammar file. 
@@ -84,20 +82,6 @@ public abstract class QueryParserBase {
   // for use when constructing RangeQuerys.
   Collator rangeCollator = null;
 
-  /** @deprecated remove when getFieldQuery is removed */
-  @Deprecated
-  static final VirtualMethod<QueryParserBase> getFieldQueryMethod =
-    new VirtualMethod<QueryParserBase>(QueryParserBase.class, "getFieldQuery", String.class, String.class);
-  /** @deprecated remove when getFieldQuery is removed */
-  @Deprecated
-  static final VirtualMethod<QueryParserBase> getFieldQueryWithQuotedMethod =
-    new VirtualMethod<QueryParserBase>(QueryParserBase.class, "getFieldQuery", String.class, String.class, boolean.class);
-  /** @deprecated remove when getFieldQuery is removed */
-  @Deprecated
-  final boolean hasNewAPI =
-    VirtualMethod.compareImplementationDistance(getClass(),
-        getFieldQueryWithQuotedMethod, getFieldQueryMethod) >= 0; // its ok for both to be overridden
-
   boolean autoGeneratePhraseQueries;
 
   // So the generated QueryParser(CharStream) won't error out
@@ -112,11 +96,6 @@ public abstract class QueryParserBase {
   public void init(Version matchVersion, String f, Analyzer a) {
     analyzer = a;
     field = f;
-    if (matchVersion.onOrAfter(Version.LUCENE_29)) {
-      enablePositionIncrements = true;
-    } else {
-      enablePositionIncrements = false;
-    }
     if (matchVersion.onOrAfter(Version.LUCENE_31)) {
       setAutoGeneratePhraseQueries(false);
     } else {
@@ -190,9 +169,6 @@ public abstract class QueryParserBase {
    * surrounded by double quotes.
    */
   public final void setAutoGeneratePhraseQueries(boolean value) {
-    if (value == false && !hasNewAPI)
-      throw new IllegalArgumentException("You must implement the new API: getFieldQuery(String,String,boolean)"
-       + " to use setAutoGeneratePhraseQueries(false)");
     this.autoGeneratePhraseQueries = value;
   }
 
@@ -272,7 +248,7 @@ public abstract class QueryParserBase {
    * Useful when e.g. a StopFilter increases the position increment of
    * the token that follows an omitted token.
    * <p>
-   * Default: false.
+   * Default: true.
    */
   public void setEnablePositionIncrements(boolean enable) {
     this.enablePositionIncrements = enable;
@@ -489,15 +465,6 @@ public abstract class QueryParserBase {
   }
 
   /**
-   * @deprecated Use {@link #getFieldQuery(String,String,boolean)} instead.
-   */
-  @Deprecated
-  protected Query getFieldQuery(String field, String queryText) throws ParseException {
-    // treat the text as if it was quoted, to drive phrase logic with old versions.
-    return getFieldQuery(field, queryText, true);
-  }
-
-  /**
    * @exception org.apache.lucene.queryParser.ParseException throw in overridden method to disallow
    */
   protected Query getFieldQuery(String field, String queryText, boolean quoted)  throws ParseException {
@@ -684,7 +651,7 @@ public abstract class QueryParserBase {
    */
   protected Query getFieldQuery(String field, String queryText, int slop)
         throws ParseException {
-    Query query = hasNewAPI ? getFieldQuery(field, queryText, true) : getFieldQuery(field, queryText);
+    Query query = getFieldQuery(field, queryText, true);
 
     if (query instanceof PhraseQuery) {
       ((PhraseQuery) query).setSlop(slop);
@@ -696,11 +663,6 @@ public abstract class QueryParserBase {
     return query;
   }
 
-
-  @Deprecated
-  protected final Query getRangeQuery(String field, String part1, String part2, boolean inclusive) throws MethodRemovedUseAnother {return null;}
-
-
   /**
    *
    * @exception org.apache.lucene.queryParser.ParseException
@@ -722,15 +684,7 @@ public abstract class QueryParserBase {
     DateTools.Resolution resolution = getDateResolution(field);
     
     try {
-      Date d1 = df.parse(part1);
-      if (resolution == null) {
-        // no default or field specific date resolution has been set,
-        // use deprecated DateField to maintain compatibility with
-        // pre-1.9 Lucene versions.
-        part1 = DateField.dateToString(d1);
-      } else {
-        part1 = DateTools.dateToString(d1, resolution);
-      }
+      part1 = DateTools.dateToString(df.parse(part1), resolution);
     } catch (Exception e) { }
 
     try {
@@ -747,14 +701,7 @@ public abstract class QueryParserBase {
         cal.set(Calendar.MILLISECOND, 999);
         d2 = cal.getTime();
       }
-      if (resolution == null) {
-        // no default or field specific date resolution has been set,
-        // use deprecated DateField to maintain compatibility with
-        // pre-1.9 Lucene versions.
-        part2 = DateField.dateToString(d2);
-      } else {
-        part2 = DateTools.dateToString(d2, resolution);
-      }
+      part2 = DateTools.dateToString(d2, resolution);
     } catch (Exception e) { }
 
     return newRangeQuery(field, part1, part2, startInclusive, endInclusive);
@@ -838,10 +785,6 @@ public abstract class QueryParserBase {
     return new FuzzyQuery(term,minimumSimilarity,prefixLength);
   }
 
-  @Deprecated
-  protected final Query newRangeQuery(String field, String part1, String part2, boolean inclusive) throws MethodRemovedUseAnother {return null;}
-
-
   /**
    * Builds a new TermRangeQuery instance
    * @param field Field
@@ -1064,7 +1007,7 @@ public abstract class QueryParserBase {
       }
       q = getFuzzyQuery(qfield, termImage, fms);
     } else {
-      q = hasNewAPI ? getFieldQuery(qfield, termImage, false) : getFieldQuery(qfield, termImage);
+      q = getFieldQuery(qfield, termImage, false);
     }
     return q;
   }

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/queryParser/QueryParserTokenManager.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/queryParser/QueryParserTokenManager.java?rev=1042501&r1=1042500&r2=1042501&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/queryParser/QueryParserTokenManager.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/queryParser/QueryParserTokenManager.java Mon Dec  6 00:47:16 2010
@@ -16,7 +16,6 @@ import org.apache.lucene.analysis.Cachin
 import org.apache.lucene.analysis.TokenStream;
 import org.apache.lucene.analysis.tokenattributes.PositionIncrementAttribute;
 import org.apache.lucene.analysis.tokenattributes.TermToBytesRefAttribute;
-import org.apache.lucene.document.DateField;
 import org.apache.lucene.document.DateTools;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.search.BooleanClause;

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/AutomatonQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/AutomatonQuery.java?rev=1042501&r1=1042500&r2=1042501&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/AutomatonQuery.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/AutomatonQuery.java Mon Dec  6 00:47:16 2010
@@ -19,10 +19,9 @@ package org.apache.lucene.search;
 
 import java.io.IOException;
 
-import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.Term;
+import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.TermsEnum;
-import org.apache.lucene.index.MultiFields;
 import org.apache.lucene.util.ToStringUtils;
 import org.apache.lucene.util.AttributeSource;
 import org.apache.lucene.util.BytesRef;
@@ -86,43 +85,43 @@ public class AutomatonQuery extends Mult
   }
 
   @Override
-  protected TermsEnum getTermsEnum(IndexReader reader, AttributeSource atts) throws IOException {
+  protected TermsEnum getTermsEnum(Terms terms, AttributeSource atts) throws IOException {
     // matches nothing
     if (BasicOperations.isEmpty(automaton)) {
       return TermsEnum.EMPTY;
     }
     
+    TermsEnum tenum = terms.iterator();
+    
     // matches all possible strings
     if (BasicOperations.isTotal(automaton)) {
-      // NOTE: for now, MultiTermQuery enums terms at the
-      // MultiReader level, so we must use MultiFields here:
-      return MultiFields.getTerms(reader, getField()).iterator();
+      return tenum;
     }
     
     // matches a fixed string in singleton representation
     String singleton = automaton.getSingleton();
     if (singleton != null)
-      return new SingleTermsEnum(reader, term.createTerm(singleton));
+      return new SingleTermsEnum(tenum, term.createTerm(singleton));
 
     // matches a fixed string in expanded representation
     final String commonPrefix = SpecialOperations.getCommonPrefix(automaton);
 
     if (commonPrefix.length() > 0) {
       if (BasicOperations.sameLanguage(automaton, BasicAutomata.makeString(commonPrefix))) {
-        return new SingleTermsEnum(reader, term.createTerm(commonPrefix));
+        return new SingleTermsEnum(tenum, term.createTerm(commonPrefix));
       }
     
       // matches a constant prefix
       Automaton prefixAutomaton = BasicOperations.concatenate(BasicAutomata
                                                               .makeString(commonPrefix), BasicAutomata.makeAnyString());
       if (BasicOperations.sameLanguage(automaton, prefixAutomaton)) {
-        return new PrefixTermsEnum(reader, term.createTerm(commonPrefix));
+        return new PrefixTermsEnum(tenum, term.createTerm(commonPrefix));
       }
     }
 
     compileAutomaton();
     
-    return new AutomatonTermsEnum(runAutomaton, term.field(), reader, isFinite, commonSuffixRef);
+    return new AutomatonTermsEnum(runAutomaton, tenum, isFinite, commonSuffixRef);
   }
 
   @Override

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/AutomatonTermsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/AutomatonTermsEnum.java?rev=1042501&r1=1042500&r2=1042501&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/AutomatonTermsEnum.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/AutomatonTermsEnum.java Mon Dec  6 00:47:16 2010
@@ -20,13 +20,12 @@ package org.apache.lucene.search;
 import java.io.IOException;
 import java.util.Comparator;
 
-import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.IntsRef;
 import org.apache.lucene.util.automaton.Automaton;
 import org.apache.lucene.util.automaton.ByteRunAutomaton;
 import org.apache.lucene.util.automaton.SpecialOperations;
-import org.apache.lucene.util.automaton.State;
 import org.apache.lucene.util.automaton.Transition;
 
 /**
@@ -74,7 +73,7 @@ public class AutomatonTermsEnum extends 
   /**
    * Expert ctor:
    * Construct an enumerator based upon an automaton, enumerating the specified
-   * field, working on a supplied reader.
+   * field, working on a supplied TermsEnum
    * <p>
    * @lucene.experimental 
    * <p>
@@ -82,10 +81,10 @@ public class AutomatonTermsEnum extends 
    * @param finite true if the automaton accepts a finite language
    */
   public AutomatonTermsEnum(ByteRunAutomaton runAutomaton,
-                     String field, IndexReader reader,
+                     TermsEnum tenum,
                      boolean finite, BytesRef commonSuffixRef)
       throws IOException {
-    super(reader, field);
+    super(tenum);
     this.automaton = runAutomaton.getAutomaton();
     this.finite = finite;
 
@@ -102,12 +101,7 @@ public class AutomatonTermsEnum extends 
     }
 
     // build a cache of sorted transitions for every state
-    allTransitions = new Transition[runAutomaton.getSize()][];
-    for (State state : this.automaton.getNumberedStates()) {
-      state.sortTransitions(Transition.CompareByMinMaxThenDest);
-      state.trimTransitionsArray();
-      allTransitions[state.getNumber()] = state.transitionsArray;
-    }
+    allTransitions = this.automaton.getSortedTransitions();
     // used for path tracking, where each bit is a numbered state.
     visited = new long[runAutomaton.getSize()];
 
@@ -116,13 +110,13 @@ public class AutomatonTermsEnum extends 
   
   /**
    * Construct an enumerator based upon an automaton, enumerating the specified
-   * field, working on a supplied reader.
+   * field, working on a supplied TermsEnum
    * <p>
    * It will automatically calculate whether or not the automaton is finite
    */
-  public AutomatonTermsEnum(Automaton automaton, String field, IndexReader reader)
+  public AutomatonTermsEnum(Automaton automaton, TermsEnum tenum)
     throws IOException {
-    this(new ByteRunAutomaton(automaton), field, reader, SpecialOperations.isFinite(automaton), null);
+    this(new ByteRunAutomaton(automaton), tenum, SpecialOperations.isFinite(automaton), null);
   }
  
   /**

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/BooleanScorer2.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/BooleanScorer2.java?rev=1042501&r1=1042500&r2=1042501&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/BooleanScorer2.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/BooleanScorer2.java Mon Dec  6 00:47:16 2010
@@ -193,7 +193,7 @@ class BooleanScorer2 extends Scorer {
   }
 
   private Scorer dualConjunctionSumScorer(Scorer req1, Scorer req2) throws IOException { // non counting.
-    return new ConjunctionScorer(defaultSimilarity, new Scorer[]{req1, req2});
+    return new ConjunctionScorer(defaultSimilarity, req1, req2);
     // All scorers match, so defaultSimilarity always has 1 as
     // the coordination factor.
     // Therefore the sum of the scores of two scorers

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/FilteredTermsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/FilteredTermsEnum.java?rev=1042501&r1=1042500&r2=1042501&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/FilteredTermsEnum.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/FilteredTermsEnum.java Mon Dec  6 00:47:16 2010
@@ -20,11 +20,8 @@ package org.apache.lucene.search;
 import java.io.IOException;
 import java.util.Comparator;
 
-import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.index.TermsEnum;
-import org.apache.lucene.index.Terms;
-import org.apache.lucene.index.MultiFields;
 import org.apache.lucene.index.DocsEnum;
 import org.apache.lucene.index.DocsAndPositionsEnum;
 import org.apache.lucene.util.AttributeSource;
@@ -62,22 +59,11 @@ public abstract class FilteredTermsEnum 
   protected abstract AcceptStatus accept(BytesRef term) throws IOException;
 
   /**
-   * Creates a filtered {@link TermsEnum} for the given field name and reader.
-   */
-  public FilteredTermsEnum(final IndexReader reader, final String field) throws IOException {
-    final Terms terms = MultiFields.getTerms(reader, field);
-    if (terms != null) {
-      tenum = terms.iterator();
-    } else {
-      tenum = null;
-    }
-  }
-
-  /**
    * Creates a filtered {@link TermsEnum} on a terms enum.
-   * @param tenum the terms enumeration to filter, if {@code null} this is the null iterator.
+   * @param tenum the terms enumeration to filter.
    */
   public FilteredTermsEnum(final TermsEnum tenum) {
+    assert tenum != null;
     this.tenum = tenum;
   }
 
@@ -121,26 +107,21 @@ public abstract class FilteredTermsEnum 
    */
   @Override
   public AttributeSource attributes() {
-    /* if we have no tenum, we return a new attributes instance,
-     * to prevent NPE in subclasses that use attributes.
-     * in all other cases we share the attributes with our delegate. */
-    return (tenum == null) ? super.attributes() : tenum.attributes();
+    return tenum.attributes();
   }
   
   @Override
   public BytesRef term() throws IOException {
-    assert tenum != null;
     return tenum.term();
   }
 
   @Override
   public Comparator<BytesRef> getComparator() throws IOException {
-    return (tenum == null) ? null : tenum.getComparator();
+    return tenum.getComparator();
   }
     
   @Override
   public int docFreq() {
-    assert tenum != null;
     return tenum.docFreq();
   }
 
@@ -162,19 +143,16 @@ public abstract class FilteredTermsEnum 
 
   @Override
   public long ord() throws IOException {
-    assert tenum != null;
     return tenum.ord();
   }
 
   @Override
   public DocsEnum docs(Bits bits, DocsEnum reuse) throws IOException {
-    assert tenum != null;
     return tenum.docs(bits, reuse);
   }
     
   @Override
   public DocsAndPositionsEnum docsAndPositions(Bits bits, DocsAndPositionsEnum reuse) throws IOException {
-    assert tenum != null;
     return tenum.docsAndPositions(bits, reuse);
   }
 
@@ -186,8 +164,6 @@ public abstract class FilteredTermsEnum 
   @SuppressWarnings("fallthrough")
   @Override
   public BytesRef next() throws IOException {
-    if (tenum == null)
-      return null;
     for (;;) {
       // Seek or forward the iterator
       if (doSeek) {

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/FuzzyQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/FuzzyQuery.java?rev=1042501&r1=1042500&r2=1042501&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/FuzzyQuery.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/FuzzyQuery.java Mon Dec  6 00:47:16 2010
@@ -17,8 +17,8 @@ package org.apache.lucene.search;
  * limitations under the License.
  */
 
-import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.Term;
+import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.util.AttributeSource;
 import org.apache.lucene.util.ToStringUtils;
@@ -136,11 +136,13 @@ public class FuzzyQuery extends MultiTer
   }
 
   @Override
-  protected TermsEnum getTermsEnum(IndexReader reader, AttributeSource atts) throws IOException {
+  protected TermsEnum getTermsEnum(Terms terms, AttributeSource atts) throws IOException {
+    TermsEnum tenum = terms.iterator();
+    
     if (!termLongEnough) {  // can only match if it's exact
-      return new SingleTermsEnum(reader, term);
+      return new SingleTermsEnum(tenum, term);
     }
-    return new FuzzyTermsEnum(reader, atts, getTerm(), minimumSimilarity, prefixLength);
+    return new FuzzyTermsEnum(tenum, atts, getTerm(), minimumSimilarity, prefixLength);
   }
   
   /**

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/FuzzyTermsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/FuzzyTermsEnum.java?rev=1042501&r1=1042500&r2=1042501&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/FuzzyTermsEnum.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/FuzzyTermsEnum.java Mon Dec  6 00:47:16 2010
@@ -19,7 +19,6 @@ package org.apache.lucene.search;
 
 import org.apache.lucene.index.DocsAndPositionsEnum;
 import org.apache.lucene.index.DocsEnum;
-import org.apache.lucene.index.IndexReader;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.util.Attribute;
@@ -71,7 +70,7 @@ public final class FuzzyTermsEnum extend
   private int maxEdits;
   private final boolean raw;
 
-  private final IndexReader reader;
+  private final TermsEnum tenum;
   private final Term term;
   private final int termText[];
   private final int realPrefixLength;
@@ -84,7 +83,7 @@ public final class FuzzyTermsEnum extend
    * After calling the constructor the enumeration is already pointing to the first 
    * valid term if such a term exists. 
    * 
-   * @param reader Delivers terms.
+   * @param tenum Delivers terms.
    * @param atts {@link AttributeSource} created by the rewrite method of {@link MultiTermQuery}
    * thats contains information about competitive boosts during rewrite. It is also used
    * to cache DFAs between segment transitions.
@@ -93,7 +92,7 @@ public final class FuzzyTermsEnum extend
    * @param prefixLength Length of required common prefix. Default value is 0.
    * @throws IOException
    */
-  public FuzzyTermsEnum(IndexReader reader, AttributeSource atts, Term term, 
+  public FuzzyTermsEnum(TermsEnum tenum, AttributeSource atts, Term term, 
       final float minSimilarity, final int prefixLength) throws IOException {
     if (minSimilarity >= 1.0f && minSimilarity != (int)minSimilarity)
       throw new IllegalArgumentException("fractional edit distances are not allowed");
@@ -101,7 +100,7 @@ public final class FuzzyTermsEnum extend
       throw new IllegalArgumentException("minimumSimilarity cannot be less than 0");
     if(prefixLength < 0)
       throw new IllegalArgumentException("prefixLength cannot be less than 0");
-    this.reader = reader;
+    this.tenum = tenum;
     this.term = term;
 
     // convert the string into a utf32 int[] representation for fast comparisons
@@ -304,7 +303,7 @@ public final class FuzzyTermsEnum extend
     
     public AutomatonFuzzyTermsEnum(ByteRunAutomaton matchers[], 
         BytesRef lastTerm) throws IOException {
-      super(matchers[matchers.length - 1], term.field(), reader, true, null);
+      super(matchers[matchers.length - 1], tenum, true, null);
       this.matchers = matchers;
       this.lastTerm = lastTerm;
       termRef = new BytesRef(term.text());
@@ -380,7 +379,7 @@ public final class FuzzyTermsEnum extend
      * @throws IOException
      */
     public LinearFuzzyTermsEnum() throws IOException {
-      super(reader, term.field());
+      super(tenum);
 
       this.text = new int[termLength - realPrefixLength];
       System.arraycopy(termText, realPrefixLength, text, 0, text.length);

Modified: lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/IndexSearcher.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/IndexSearcher.java?rev=1042501&r1=1042500&r2=1042501&view=diff
==============================================================================
--- lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/IndexSearcher.java (original)
+++ lucene/dev/branches/docvalues/lucene/src/java/org/apache/lucene/search/IndexSearcher.java Mon Dec  6 00:47:16 2010
@@ -158,11 +158,11 @@ public class IndexSearcher extends Searc
   @Override
   public TopDocs search(Weight weight, Filter filter, int nDocs) throws IOException {
 
-    if (nDocs <= 0) {
-      throw new IllegalArgumentException("nDocs must be > 0");
+    int limit = reader.maxDoc();
+    if (limit == 0) {
+      limit = 1;
     }
-
-    nDocs = Math.min(nDocs, reader.maxDoc());
+    nDocs = Math.min(nDocs, limit);
 
     TopScoreDocCollector collector = TopScoreDocCollector.create(nDocs, !weight.scoresDocsOutOfOrder());
     search(weight, filter, collector);
@@ -190,7 +190,11 @@ public class IndexSearcher extends Searc
                              Sort sort, boolean fillFields)
       throws IOException {
 
-    nDocs = Math.min(nDocs, reader.maxDoc());
+    int limit = reader.maxDoc();
+    if (limit == 0) {
+      limit = 1;
+    }
+    nDocs = Math.min(nDocs, limit);
 
     TopFieldCollector collector = TopFieldCollector.create(sort, nDocs,
         fillFields, fieldSortDoTrackScores, fieldSortDoMaxScore, !weight.scoresDocsOutOfOrder());