lucene-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From sim...@apache.org
Subject svn commit: r1143719 [7/20] - in /lucene/dev/branches/LUCENE2793: ./ dev-tools/eclipse/ dev-tools/idea/.idea/ dev-tools/idea/lucene/contrib/ dev-tools/idea/lucene/contrib/demo/ dev-tools/idea/lucene/contrib/highlighter/ dev-tools/idea/lucene/contrib/in...
Date Thu, 07 Jul 2011 09:04:29 GMT
Modified: lucene/dev/branches/LUCENE2793/lucene/src/java/org/apache/lucene/index/codecs/PostingsReaderBase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE2793/lucene/src/java/org/apache/lucene/index/codecs/PostingsReaderBase.java?rev=1143719&r1=1143718&r2=1143719&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE2793/lucene/src/java/org/apache/lucene/index/codecs/PostingsReaderBase.java (original)
+++ lucene/dev/branches/LUCENE2793/lucene/src/java/org/apache/lucene/index/codecs/PostingsReaderBase.java Thu Jul  7 09:03:58 2011
@@ -49,11 +49,11 @@ public abstract class PostingsReaderBase
 
   /** Must fully consume state, since after this call that
    *  TermState may be reused. */
-  public abstract DocsEnum docs(FieldInfo fieldInfo, BlockTermState state, Bits skipDocs, DocsEnum reuse) throws IOException;
+  public abstract DocsEnum docs(FieldInfo fieldInfo, BlockTermState state, Bits liveDocs, DocsEnum reuse) throws IOException;
 
   /** Must fully consume state, since after this call that
    *  TermState may be reused. */
-  public abstract DocsAndPositionsEnum docsAndPositions(FieldInfo fieldInfo, BlockTermState state, Bits skipDocs, DocsAndPositionsEnum reuse) throws IOException;
+  public abstract DocsAndPositionsEnum docsAndPositions(FieldInfo fieldInfo, BlockTermState state, Bits liveDocs, DocsAndPositionsEnum reuse) throws IOException;
 
   public abstract void close() throws IOException;
 

Modified: lucene/dev/branches/LUCENE2793/lucene/src/java/org/apache/lucene/index/codecs/TermsConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE2793/lucene/src/java/org/apache/lucene/index/codecs/TermsConsumer.java?rev=1143719&r1=1143718&r2=1143719&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE2793/lucene/src/java/org/apache/lucene/index/codecs/TermsConsumer.java (original)
+++ lucene/dev/branches/LUCENE2793/lucene/src/java/org/apache/lucene/index/codecs/TermsConsumer.java Thu Jul  7 09:03:58 2011
@@ -67,7 +67,7 @@ public abstract class TermsConsumer {
       MultiDocsEnum docsEnumIn = null;
 
       while((term = termsEnum.next()) != null) {
-        docsEnumIn = (MultiDocsEnum) termsEnum.docs(mergeState.multiDeletedDocs, docsEnumIn);
+        docsEnumIn = (MultiDocsEnum) termsEnum.docs(mergeState.multiLiveDocs, docsEnumIn);
         if (docsEnumIn != null) {
           docsEnum.reset(docsEnumIn);
           final PostingsConsumer postingsConsumer = startTerm(term);
@@ -89,7 +89,7 @@ public abstract class TermsConsumer {
       postingsEnum.setMergeState(mergeState);
       MultiDocsAndPositionsEnum postingsEnumIn = null;
       while((term = termsEnum.next()) != null) {
-        postingsEnumIn = (MultiDocsAndPositionsEnum) termsEnum.docsAndPositions(mergeState.multiDeletedDocs, postingsEnumIn);
+        postingsEnumIn = (MultiDocsAndPositionsEnum) termsEnum.docsAndPositions(mergeState.multiLiveDocs, postingsEnumIn);
         if (postingsEnumIn != null) {
           postingsEnum.reset(postingsEnumIn);
           // set PayloadProcessor

Modified: lucene/dev/branches/LUCENE2793/lucene/src/java/org/apache/lucene/index/codecs/memory/MemoryCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE2793/lucene/src/java/org/apache/lucene/index/codecs/memory/MemoryCodec.java?rev=1143719&r1=1143718&r2=1143719&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE2793/lucene/src/java/org/apache/lucene/index/codecs/memory/MemoryCodec.java (original)
+++ lucene/dev/branches/LUCENE2793/lucene/src/java/org/apache/lucene/index/codecs/memory/MemoryCodec.java Thu Jul  7 09:03:58 2011
@@ -270,7 +270,7 @@ public class MemoryCodec extends Codec {
     private byte[] buffer = new byte[16];
     private final ByteArrayDataInput in = new ByteArrayDataInput(buffer);
 
-    private Bits skipDocs;
+    private Bits liveDocs;
     private int docUpto;
     private int docID;
     private int freq;
@@ -286,14 +286,14 @@ public class MemoryCodec extends Codec {
       return omitTFAP == this.omitTFAP && storePayloads == this.storePayloads;
     }
     
-    public FSTDocsEnum reset(BytesRef bufferIn, Bits skipDocs, int numDocs) {
+    public FSTDocsEnum reset(BytesRef bufferIn, Bits liveDocs, int numDocs) {
       assert numDocs > 0;
       if (buffer.length < bufferIn.length - bufferIn.offset) {
         buffer = ArrayUtil.grow(buffer, bufferIn.length - bufferIn.offset);
       }
       in.reset(buffer, 0, bufferIn.length - bufferIn.offset);
       System.arraycopy(bufferIn.bytes, bufferIn.offset, buffer, 0, bufferIn.length - bufferIn.offset);
-      this.skipDocs = skipDocs;
+      this.liveDocs = liveDocs;
       docID = 0;
       docUpto = 0;
       payloadLen = 0;
@@ -340,7 +340,7 @@ public class MemoryCodec extends Codec {
           }
         }
 
-        if (skipDocs == null || !skipDocs.get(docID)) {
+        if (liveDocs == null || liveDocs.get(docID)) {
           if (VERBOSE) System.out.println("    return docID=" + docID + " freq=" + freq);
           return docID;
         }
@@ -376,7 +376,7 @@ public class MemoryCodec extends Codec {
     private byte[] buffer = new byte[16];
     private final ByteArrayDataInput in = new ByteArrayDataInput(buffer);
 
-    private Bits skipDocs;
+    private Bits liveDocs;
     private int docUpto;
     private int docID;
     private int freq;
@@ -397,7 +397,7 @@ public class MemoryCodec extends Codec {
       return omitTFAP == this.omitTFAP && storePayloads == this.storePayloads;
     }
     
-    public FSTDocsAndPositionsEnum reset(BytesRef bufferIn, Bits skipDocs, int numDocs) {
+    public FSTDocsAndPositionsEnum reset(BytesRef bufferIn, Bits liveDocs, int numDocs) {
       assert numDocs > 0;
       if (VERBOSE) {
         System.out.println("D&P reset bytes this=" + this);
@@ -410,7 +410,7 @@ public class MemoryCodec extends Codec {
       }
       in.reset(buffer, 0, bufferIn.length - bufferIn.offset);
       System.arraycopy(bufferIn.bytes, bufferIn.offset, buffer, 0, bufferIn.length - bufferIn.offset);
-      this.skipDocs = skipDocs;
+      this.liveDocs = liveDocs;
       docID = 0;
       docUpto = 0;
       payload.bytes = buffer;
@@ -447,7 +447,7 @@ public class MemoryCodec extends Codec {
           }
         }
 
-        if (skipDocs == null || !skipDocs.get(docID)) {
+        if (liveDocs == null || liveDocs.get(docID)) {
           pos = 0;
           posPending = freq;
           if (VERBOSE) System.out.println("    return docID=" + docID + " freq=" + freq);
@@ -472,7 +472,7 @@ public class MemoryCodec extends Codec {
 
     @Override
     public int nextPosition() {
-      if (VERBOSE) System.out.println("    nextPos storePayloads=" + storePayloads);
+      if (VERBOSE) System.out.println("    nextPos storePayloads=" + storePayloads + " this=" + this);
       assert posPending > 0;
       posPending--;
       if (!storePayloads) {
@@ -489,6 +489,9 @@ public class MemoryCodec extends Codec {
         payload.offset = in.getPosition();
         in.skipBytes(payloadLength);
         payload.length = payloadLength;
+        // Necessary, in case caller changed the
+        // payload.bytes from prior call:
+        payload.bytes = buffer;
         payloadRetrieved = false;
       }
 
@@ -534,7 +537,8 @@ public class MemoryCodec extends Codec {
   private final static class FSTTermsEnum extends TermsEnum {
     private final FieldInfo field;
     private final BytesRefFSTEnum<BytesRef> fstEnum;
-    private final ByteArrayDataInput buffer = new ByteArrayDataInput(null);
+    private final ByteArrayDataInput buffer = new ByteArrayDataInput();
+    private boolean didDecode;
 
     private int docFreq;
     private long totalTermFreq;
@@ -545,20 +549,31 @@ public class MemoryCodec extends Codec {
       fstEnum = new BytesRefFSTEnum<BytesRef>(fst);
     }
 
-    private void readTermStats() throws IOException {
-      buffer.reset(current.output.bytes, 0, current.output.length);
-      docFreq = buffer.readVInt();
-      if (!field.omitTermFreqAndPositions) {
-        totalTermFreq = docFreq + buffer.readVLong();
-      } else {
-        totalTermFreq = 0;
+    private void decodeMetaData() throws IOException {
+      if (!didDecode) {
+        buffer.reset(current.output.bytes, 0, current.output.length);
+        docFreq = buffer.readVInt();
+        if (!field.omitTermFreqAndPositions) {
+          totalTermFreq = docFreq + buffer.readVLong();
+        } else {
+          totalTermFreq = 0;
+        }
+        current.output.offset = buffer.getPosition();
+        if (VERBOSE) System.out.println("  df=" + docFreq + " totTF=" + totalTermFreq + " offset=" + buffer.getPosition() + " len=" + current.output.length);
+        didDecode = true;
       }
-      current.output.offset = buffer.getPosition();
-      if (VERBOSE) System.out.println("  df=" + docFreq + " totTF=" + totalTermFreq + " offset=" + buffer.getPosition() + " len=" + current.output.length);
     }
 
     @Override
-    public SeekStatus seek(BytesRef text, boolean useCache /* ignored */) throws IOException {
+    public boolean seekExact(BytesRef text, boolean useCache /* ignored */) throws IOException {
+      if (VERBOSE) System.out.println("te.seekExact text=" + field.name + ":" + text.utf8ToString() + " this=" + this);
+      current = fstEnum.seekExact(text);
+      didDecode = false;
+      return current != null;
+    }
+
+    @Override
+    public SeekStatus seekCeil(BytesRef text, boolean useCache /* ignored */) throws IOException {
       if (VERBOSE) System.out.println("te.seek text=" + field.name + ":" + text.utf8ToString() + " this=" + this);
       current = fstEnum.seekCeil(text);
       if (current == null) {
@@ -571,7 +586,8 @@ public class MemoryCodec extends Codec {
           }
         }
 
-        readTermStats();
+        didDecode = false;
+
         if (text.equals(current.input)) {
           if (VERBOSE) System.out.println("  found!");
           return SeekStatus.FOUND;
@@ -583,7 +599,8 @@ public class MemoryCodec extends Codec {
     }
     
     @Override
-    public DocsEnum docs(Bits skipDocs, DocsEnum reuse) {
+    public DocsEnum docs(Bits liveDocs, DocsEnum reuse) throws IOException {
+      decodeMetaData();
       FSTDocsEnum docsEnum;
       if (reuse == null || !(reuse instanceof FSTDocsEnum)) {
         docsEnum = new FSTDocsEnum(field.omitTermFreqAndPositions, field.storePayloads);
@@ -593,14 +610,15 @@ public class MemoryCodec extends Codec {
           docsEnum = new FSTDocsEnum(field.omitTermFreqAndPositions, field.storePayloads);
         }
       }
-      return docsEnum.reset(current.output, skipDocs, docFreq);
+      return docsEnum.reset(current.output, liveDocs, docFreq);
     }
 
     @Override
-    public DocsAndPositionsEnum docsAndPositions(Bits skipDocs, DocsAndPositionsEnum reuse) {
+    public DocsAndPositionsEnum docsAndPositions(Bits liveDocs, DocsAndPositionsEnum reuse) throws IOException {
       if (field.omitTermFreqAndPositions) {
         return null;
       }
+      decodeMetaData();
       FSTDocsAndPositionsEnum docsAndPositionsEnum;
       if (reuse == null || !(reuse instanceof FSTDocsAndPositionsEnum)) {
         docsAndPositionsEnum = new FSTDocsAndPositionsEnum(field.omitTermFreqAndPositions, field.storePayloads);
@@ -611,7 +629,7 @@ public class MemoryCodec extends Codec {
         }
       }
       if (VERBOSE) System.out.println("D&P reset this=" + this);
-      return docsAndPositionsEnum.reset(current.output, skipDocs, docFreq);
+      return docsAndPositionsEnum.reset(current.output, liveDocs, docFreq);
     }
 
     @Override
@@ -627,18 +645,20 @@ public class MemoryCodec extends Codec {
         if (VERBOSE) System.out.println("  END");
         return null;
       }
-      readTermStats();
+      didDecode = false;
       if (VERBOSE) System.out.println("  term=" + field.name + ":" + current.input.utf8ToString());
       return current.input;
     }
 
     @Override
-    public int docFreq() {
+    public int docFreq() throws IOException {
+      decodeMetaData();
       return docFreq;
     }
 
     @Override
-    public long totalTermFreq() {
+    public long totalTermFreq() throws IOException {
+      decodeMetaData();
       return totalTermFreq;
     }
 
@@ -648,7 +668,7 @@ public class MemoryCodec extends Codec {
     }
 
     @Override
-    public SeekStatus seek(long ord) {
+    public void seekExact(long ord) {
       // NOTE: we could add this...
       throw new UnsupportedOperationException();
     }
@@ -658,7 +678,6 @@ public class MemoryCodec extends Codec {
       // NOTE: we could add this...
       throw new UnsupportedOperationException();
     }
-
   }
 
   private final static class TermsReader extends Terms {

Modified: lucene/dev/branches/LUCENE2793/lucene/src/java/org/apache/lucene/index/codecs/preflex/PreFlexFields.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE2793/lucene/src/java/org/apache/lucene/index/codecs/preflex/PreFlexFields.java?rev=1143719&r1=1143718&r2=1143719&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE2793/lucene/src/java/org/apache/lucene/index/codecs/preflex/PreFlexFields.java (original)
+++ lucene/dev/branches/LUCENE2793/lucene/src/java/org/apache/lucene/index/codecs/preflex/PreFlexFields.java Thu Jul  7 09:03:58 2011
@@ -25,7 +25,6 @@ import java.util.Iterator;
 import java.util.Map;
 import java.util.TreeMap;
 
-import org.apache.lucene.index.CompoundFileReader;
 import org.apache.lucene.index.DocsAndPositionsEnum;
 import org.apache.lucene.index.DocsEnum;
 import org.apache.lucene.index.FieldInfo;
@@ -37,6 +36,7 @@ import org.apache.lucene.index.Term;
 import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.index.codecs.FieldsProducer;
+import org.apache.lucene.store.CompoundFileDirectory;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexInput;
@@ -178,8 +178,8 @@ public class PreFlexFields extends Field
         // terms reader with index, the segment has switched
         // to CFS
 
-        if (!(dir instanceof CompoundFileReader)) {
-          dir0 = cfsReader = new CompoundFileReader(dir, IndexFileNames.segmentFileName(si.name, "", IndexFileNames.COMPOUND_FILE_EXTENSION), context);
+        if (!(dir instanceof CompoundFileDirectory)) {
+          dir0 = cfsReader = dir.openCompoundInput(IndexFileNames.segmentFileName(si.name, "", IndexFileNames.COMPOUND_FILE_EXTENSION), context);
         } else {
           dir0 = dir;
         }
@@ -271,11 +271,11 @@ public class PreFlexFields extends Field
   private class PreTermsEnum extends TermsEnum {
     private SegmentTermEnum termEnum;
     private FieldInfo fieldInfo;
+    private String internedFieldName;
     private boolean skipNext;
     private BytesRef current;
 
     private SegmentTermEnum seekTermEnum;
-    private Term protoTerm;
     
     private static final byte UTF8_NON_BMP_LEAD = (byte) 0xf0;
     private static final byte UTF8_HIGH_BMP_LEAD = (byte) 0xee;
@@ -335,7 +335,7 @@ public class PreFlexFields extends Field
       }
 
       // Seek "back":
-      getTermsDict().seekEnum(te, protoTerm.createTerm(term), true);
+      getTermsDict().seekEnum(te, new Term(fieldInfo.name, term), true);
 
       // Test if the term we seek'd to in fact found a
       // surrogate pair at the same position as the E:
@@ -344,7 +344,7 @@ public class PreFlexFields extends Field
       // Cannot be null (or move to next field) because at
       // "worst" it'd seek to the same term we are on now,
       // unless we are being called from seek
-      if (t2 == null || t2.field() != fieldInfo.name) {
+      if (t2 == null || t2.field() != internedFieldName) {
         return false;
       }
 
@@ -462,13 +462,13 @@ public class PreFlexFields extends Field
           
         // TODO: more efficient seek?  can we simply swap
         // the enums?
-        getTermsDict().seekEnum(termEnum, protoTerm.createTerm(scratchTerm), true);
+        getTermsDict().seekEnum(termEnum, new Term(fieldInfo.name, scratchTerm), true);
 
         final Term t2 = termEnum.term();
 
         // We could hit EOF or different field since this
         // was a seek "forward":
-        if (t2 != null && t2.field() == fieldInfo.name) {
+        if (t2 != null && t2.field() == internedFieldName) {
 
           if (DEBUG_SURROGATES) {
             System.out.println("      got term=" + UnicodeUtil.toHexString(t2.text()) + " " + t2.bytes());
@@ -553,7 +553,7 @@ public class PreFlexFields extends Field
       // current term.
 
       // TODO: can we avoid this copy?
-      if (termEnum.term() == null || termEnum.term().field() != fieldInfo.name) {
+      if (termEnum.term() == null || termEnum.term().field() != internedFieldName) {
         scratchTerm.length = 0;
       } else {
         scratchTerm.copy(termEnum.term().bytes());
@@ -638,7 +638,7 @@ public class PreFlexFields extends Field
 
           // Seek "forward":
           // TODO: more efficient seek?
-          getTermsDict().seekEnum(seekTermEnum, protoTerm.createTerm(scratchTerm), true);
+          getTermsDict().seekEnum(seekTermEnum, new Term(fieldInfo.name, scratchTerm), true);
 
           scratchTerm.bytes[upTo] = scratch[0];
           scratchTerm.bytes[upTo+1] = scratch[1];
@@ -660,7 +660,7 @@ public class PreFlexFields extends Field
           // EOF or a different field:
           boolean matches;
 
-          if (t2 != null && t2.field() == fieldInfo.name) {
+          if (t2 != null && t2.field() == internedFieldName) {
             final BytesRef b2 = t2.bytes();
             assert b2.offset == 0;
             if (b2.length >= upTo+3 && isHighBMPChar(b2.bytes, upTo)) {
@@ -714,20 +714,21 @@ public class PreFlexFields extends Field
     void reset(FieldInfo fieldInfo) throws IOException {
       //System.out.println("pff.reset te=" + termEnum);
       this.fieldInfo = fieldInfo;
-      protoTerm = new Term(fieldInfo.name);
+      internedFieldName = fieldInfo.name.intern();
+      final Term term = new Term(internedFieldName);
       if (termEnum == null) {
-        termEnum = getTermsDict().terms(protoTerm);
-        seekTermEnum = getTermsDict().terms(protoTerm);
+        termEnum = getTermsDict().terms(term);
+        seekTermEnum = getTermsDict().terms(term);
         //System.out.println("  term=" + termEnum.term());
       } else {
-        getTermsDict().seekEnum(termEnum, protoTerm, true);
+        getTermsDict().seekEnum(termEnum, term, true);
       }
       skipNext = true;
 
       unicodeSortOrder = sortTermsByUnicode();
 
       final Term t = termEnum.term();
-      if (t != null && t.field() == fieldInfo.name) {
+      if (t != null && t.field() == internedFieldName) {
         newSuffixStart = 0;
         prevTerm.length = 0;
         surrogateDance();
@@ -746,7 +747,7 @@ public class PreFlexFields extends Field
     }
 
     @Override
-    public SeekStatus seek(long ord) throws IOException {
+    public void seekExact(long ord) throws IOException {
       throw new UnsupportedOperationException();
     }
 
@@ -756,13 +757,13 @@ public class PreFlexFields extends Field
     }
 
     @Override
-    public SeekStatus seek(BytesRef term, boolean useCache) throws IOException {
+    public SeekStatus seekCeil(BytesRef term, boolean useCache) throws IOException {
       if (DEBUG_SURROGATES) {
         System.out.println("TE.seek target=" + UnicodeUtil.toHexString(term.utf8ToString()));
       }
       skipNext = false;
       final TermInfosReader tis = getTermsDict();
-      final Term t0 = protoTerm.createTerm(term);
+      final Term t0 = new Term(fieldInfo.name, term);
 
       assert termEnum != null;
 
@@ -770,7 +771,7 @@ public class PreFlexFields extends Field
 
       final Term t = termEnum.term();
 
-      if (t != null && t.field() == fieldInfo.name && term.bytesEquals(t.bytes())) {
+      if (t != null && t.field() == internedFieldName && term.bytesEquals(t.bytes())) {
         // If we found an exact match, no need to do the
         // surrogate dance
         if (DEBUG_SURROGATES) {
@@ -778,7 +779,7 @@ public class PreFlexFields extends Field
         }
         current = t.bytes();
         return SeekStatus.FOUND;
-      } else if (t == null || t.field() != fieldInfo.name) {
+      } else if (t == null || t.field() != internedFieldName) {
 
         // TODO: maybe we can handle this like the next()
         // into null?  set term as prevTerm then dance?
@@ -841,8 +842,9 @@ public class PreFlexFields extends Field
         surrogateDance();
 
         final Term t2 = termEnum.term();
-        if (t2 == null || t2.field() != fieldInfo.name) {
-          assert t2 == null || !t2.field().equals(fieldInfo.name); // make sure fields are in fact interned
+        if (t2 == null || t2.field() != internedFieldName) {
+          // PreFlex codec interns field names; verify:
+          assert t2 == null || !t2.field().equals(internedFieldName);
           current = null;
           return SeekStatus.END;
         } else {
@@ -886,7 +888,8 @@ public class PreFlexFields extends Field
         skipNext = false;
         if (termEnum.term() == null) {
           return null;
-        } else if (termEnum.term().field() != fieldInfo.name) {
+        // PreFlex codec interns field names:
+        } else if (termEnum.term().field() != internedFieldName) {
           return null;
         } else {
           return current = termEnum.term().bytes();
@@ -896,15 +899,16 @@ public class PreFlexFields extends Field
       // TODO: can we use STE's prevBuffer here?
       prevTerm.copy(termEnum.term().bytes());
 
-      if (termEnum.next() && termEnum.term().field() == fieldInfo.name) {
+      if (termEnum.next() && termEnum.term().field() == internedFieldName) {
         newSuffixStart = termEnum.newSuffixStart;
         if (DEBUG_SURROGATES) {
           System.out.println("  newSuffixStart=" + newSuffixStart);
         }
         surrogateDance();
         final Term t = termEnum.term();
-        if (t == null || t.field() != fieldInfo.name) {
-          assert t == null || !t.field().equals(fieldInfo.name); // make sure fields are in fact interned
+        if (t == null || t.field() != internedFieldName) {
+          // PreFlex codec interns field names; verify:
+          assert t == null || !t.field().equals(internedFieldName);
           current = null;
         } else {
           current = t.bytes();
@@ -921,8 +925,9 @@ public class PreFlexFields extends Field
         surrogateDance();
         
         final Term t = termEnum.term();
-        if (t == null || t.field() != fieldInfo.name) {
-          assert t == null || !t.field().equals(fieldInfo.name); // make sure fields are in fact interned
+        if (t == null || t.field() != internedFieldName) {
+          // PreFlex codec interns field names; verify:
+          assert t == null || !t.field().equals(internedFieldName);
           return null;
         } else {
           current = t.bytes();
@@ -947,7 +952,7 @@ public class PreFlexFields extends Field
     }
 
     @Override
-    public DocsEnum docs(Bits skipDocs, DocsEnum reuse) throws IOException {
+    public DocsEnum docs(Bits liveDocs, DocsEnum reuse) throws IOException {
       PreDocsEnum docsEnum;
       if (reuse == null || !(reuse instanceof PreDocsEnum)) {
         docsEnum = new PreDocsEnum();
@@ -957,11 +962,11 @@ public class PreFlexFields extends Field
           docsEnum = new PreDocsEnum();
         }
       }
-      return docsEnum.reset(termEnum, skipDocs);
+      return docsEnum.reset(termEnum, liveDocs);
     }
 
     @Override
-    public DocsAndPositionsEnum docsAndPositions(Bits skipDocs, DocsAndPositionsEnum reuse) throws IOException {
+    public DocsAndPositionsEnum docsAndPositions(Bits liveDocs, DocsAndPositionsEnum reuse) throws IOException {
       PreDocsAndPositionsEnum docsPosEnum;
       if (fieldInfo.omitTermFreqAndPositions) {
         return null;
@@ -973,7 +978,7 @@ public class PreFlexFields extends Field
           docsPosEnum = new PreDocsAndPositionsEnum();
         }
       }
-      return docsPosEnum.reset(termEnum, skipDocs);        
+      return docsPosEnum.reset(termEnum, liveDocs);        
     }
   }
 
@@ -988,8 +993,8 @@ public class PreFlexFields extends Field
       return freqStream;
     }
 
-    public PreDocsEnum reset(SegmentTermEnum termEnum, Bits skipDocs) throws IOException {
-      docs.setSkipDocs(skipDocs);
+    public PreDocsEnum reset(SegmentTermEnum termEnum, Bits liveDocs) throws IOException {
+      docs.setLiveDocs(liveDocs);
       docs.seek(termEnum);
       return this;
     }
@@ -1044,8 +1049,8 @@ public class PreFlexFields extends Field
       return freqStream;
     }
 
-    public DocsAndPositionsEnum reset(SegmentTermEnum termEnum, Bits skipDocs) throws IOException {
-      pos.setSkipDocs(skipDocs);
+    public DocsAndPositionsEnum reset(SegmentTermEnum termEnum, Bits liveDocs) throws IOException {
+      pos.setLiveDocs(liveDocs);
       pos.seek(termEnum);
       return this;
     }

Modified: lucene/dev/branches/LUCENE2793/lucene/src/java/org/apache/lucene/index/codecs/preflex/SegmentTermDocs.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE2793/lucene/src/java/org/apache/lucene/index/codecs/preflex/SegmentTermDocs.java?rev=1143719&r1=1143718&r2=1143719&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE2793/lucene/src/java/org/apache/lucene/index/codecs/preflex/SegmentTermDocs.java (original)
+++ lucene/dev/branches/LUCENE2793/lucene/src/java/org/apache/lucene/index/codecs/preflex/SegmentTermDocs.java Thu Jul  7 09:03:58 2011
@@ -33,7 +33,7 @@ public class SegmentTermDocs {
   //protected SegmentReader parent;
   private final FieldInfos fieldInfos;
   private final TermInfosReader tis;
-  protected Bits skipDocs;
+  protected Bits liveDocs;
   protected IndexInput freqStream;
   protected int count;
   protected int df;
@@ -53,18 +53,6 @@ public class SegmentTermDocs {
   protected boolean currentFieldStoresPayloads;
   protected boolean currentFieldOmitTermFreqAndPositions;
   
-  /*
-  protected SegmentTermDocs(SegmentReader parent) {
-    this.parent = parent;
-    this.freqStream = (IndexInput) parent.core.freqStream.clone();
-    synchronized (parent) {
-      this.deletedDocs = parent.deletedDocs;
-    }
-    this.skipInterval = parent.core.getTermsReader().getSkipInterval();
-    this.maxSkipLevels = parent.core.getTermsReader().getMaxSkipLevels();
-  }
-  */
-
   public SegmentTermDocs(IndexInput freqStream, TermInfosReader tis, FieldInfos fieldInfos) {
     this.freqStream = (IndexInput) freqStream.clone();
     this.tis = tis;
@@ -78,8 +66,8 @@ public class SegmentTermDocs {
     seek(ti, term);
   }
 
-  public void setSkipDocs(Bits skipDocs) {
-    this.skipDocs = skipDocs;
+  public void setLiveDocs(Bits liveDocs) {
+    this.liveDocs = liveDocs;
   }
 
   public void seek(SegmentTermEnum segmentTermEnum) throws IOException {
@@ -149,7 +137,7 @@ public class SegmentTermDocs {
       
       count++;
 
-      if (skipDocs == null || !skipDocs.get(doc)) {
+      if (liveDocs == null || liveDocs.get(doc)) {
         break;
       }
       skippingDoc();
@@ -175,7 +163,7 @@ public class SegmentTermDocs {
           freq = freqStream.readVInt();     // else read freq
         count++;
 
-        if (skipDocs == null || !skipDocs.get(doc)) {
+        if (liveDocs == null || liveDocs.get(doc)) {
           docs[i] = doc;
           freqs[i] = freq;
           ++i;
@@ -192,7 +180,7 @@ public class SegmentTermDocs {
       doc += freqStream.readVInt();       
       count++;
 
-      if (skipDocs == null || !skipDocs.get(doc)) {
+      if (liveDocs == null || liveDocs.get(doc)) {
         docs[i] = doc;
         // Hardware freq to 1 when term freqs were not
         // stored in the index

Modified: lucene/dev/branches/LUCENE2793/lucene/src/java/org/apache/lucene/index/codecs/preflex/TermBuffer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE2793/lucene/src/java/org/apache/lucene/index/codecs/preflex/TermBuffer.java?rev=1143719&r1=1143718&r2=1143719&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE2793/lucene/src/java/org/apache/lucene/index/codecs/preflex/TermBuffer.java (original)
+++ lucene/dev/branches/LUCENE2793/lucene/src/java/org/apache/lucene/index/codecs/preflex/TermBuffer.java Thu Jul  7 09:03:58 2011
@@ -37,12 +37,17 @@ final class TermBuffer implements Clonea
 
   private BytesRef bytes = new BytesRef(10);
 
+  // Cannot be -1 since (strangely) we write that
+  // fieldNumber into index for first indexed term:
+  private int currentFieldNumber = -2;
+
   private static final Comparator<BytesRef> utf8AsUTF16Comparator = BytesRef.getUTF8SortedAsUTF16Comparator();
 
   int newSuffixStart;                             // only valid right after .read is called
 
   public int compareTo(TermBuffer other) {
     if (field == other.field) 	  // fields are interned
+                                  // (only by PreFlex codec)
       return utf8AsUTF16Comparator.compare(bytes, other.bytes);
     else
       return field.compareTo(other.field);
@@ -59,7 +64,13 @@ final class TermBuffer implements Clonea
     }
     bytes.length = totalLength;
     input.readBytes(bytes.bytes, newSuffixStart, length);
-    this.field = fieldInfos.fieldName(input.readVInt());
+    final int fieldNumber = input.readVInt();
+    if (fieldNumber != currentFieldNumber) {
+      currentFieldNumber = fieldNumber;
+      field = fieldInfos.fieldName(currentFieldNumber).intern();
+    } else {
+      assert field.equals(fieldInfos.fieldName(fieldNumber)): "currentFieldNumber=" + currentFieldNumber + " field=" + field + " vs " + fieldInfos.fieldName(fieldNumber);
+    }
   }
 
   public void set(Term term) {
@@ -68,12 +79,14 @@ final class TermBuffer implements Clonea
       return;
     }
     bytes.copy(term.bytes());
-    field = term.field();
+    field = term.field().intern();
+    currentFieldNumber = -1;
     this.term = term;
   }
 
   public void set(TermBuffer other) {
     field = other.field;
+    currentFieldNumber = other.currentFieldNumber;
     // dangerous to copy Term over, since the underlying
     // BytesRef could subsequently be modified:
     term = null;
@@ -83,6 +96,7 @@ final class TermBuffer implements Clonea
   public void reset() {
     field = null;
     term = null;
+    currentFieldNumber=  -1;
   }
 
   public Term toTerm() {
@@ -90,8 +104,7 @@ final class TermBuffer implements Clonea
       return null;
 
     if (term == null) {
-      term = new Term(field, new BytesRef(bytes), false);
-      //term = new Term(field, bytes, false);
+      term = new Term(field, new BytesRef(bytes));
     }
 
     return term;

Modified: lucene/dev/branches/LUCENE2793/lucene/src/java/org/apache/lucene/index/codecs/preflex/TermInfosReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE2793/lucene/src/java/org/apache/lucene/index/codecs/preflex/TermInfosReader.java?rev=1143719&r1=1143718&r2=1143719&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE2793/lucene/src/java/org/apache/lucene/index/codecs/preflex/TermInfosReader.java (original)
+++ lucene/dev/branches/LUCENE2793/lucene/src/java/org/apache/lucene/index/codecs/preflex/TermInfosReader.java Thu Jul  7 09:03:58 2011
@@ -18,13 +18,15 @@ package org.apache.lucene.index.codecs.p
  */
 
 import java.io.IOException;
+import java.util.Comparator;
 
 import org.apache.lucene.index.CorruptIndexException;
 import org.apache.lucene.index.FieldInfos;
-import org.apache.lucene.index.Term;
 import org.apache.lucene.index.IndexFileNames;
+import org.apache.lucene.index.Term;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
+import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.CloseableThreadLocal;
 import org.apache.lucene.util.DoubleBarrelLRUCache;
 
@@ -189,7 +191,17 @@ public final class TermInfosReader {
     }
     return resources;
   }
+  
+  private static final Comparator<BytesRef> legacyComparator = 
+    BytesRef.getUTF8SortedAsUTF16Comparator();
 
+  private final int compareAsUTF16(Term term1, Term term2) {
+    if (term1.field().equals(term2.field())) {
+      return legacyComparator.compare(term1.bytes(), term2.bytes());
+    } else {
+      return term1.field().compareTo(term2.field());
+    }
+  }
 
   /** Returns the offset of the greatest index entry which is less than or equal to term.*/
   private int getIndexOffset(Term term) {
@@ -199,7 +211,7 @@ public final class TermInfosReader {
     while (hi >= lo) {
       int mid = (lo + hi) >>> 1;
       assert indexTerms[mid] != null : "indexTerms = " + indexTerms.length + " mid=" + mid;
-      int delta = term.compareToUTF16(indexTerms[mid]);
+      int delta = compareAsUTF16(term, indexTerms[mid]);
       if (delta < 0)
 	hi = mid - 1;
       else if (delta > 0)
@@ -257,16 +269,16 @@ public final class TermInfosReader {
 
     // optimize sequential access: first try scanning cached enum w/o seeking
     if (enumerator.term() != null                 // term is at or past current
-	&& ((enumerator.prev() != null && term.compareToUTF16(enumerator.prev())> 0)
-	    || term.compareToUTF16(enumerator.term()) >= 0)) {
+	&& ((enumerator.prev() != null && compareAsUTF16(term, enumerator.prev())> 0)
+	    || compareAsUTF16(term, enumerator.term()) >= 0)) {
       int enumOffset = (int)(enumerator.position/totalIndexInterval)+1;
       if (indexTerms.length == enumOffset	  // but before end of block
-    || term.compareToUTF16(indexTerms[enumOffset]) < 0) {
+          || compareAsUTF16(term, indexTerms[enumOffset]) < 0) {
        // no need to seek
 
         final TermInfo ti;
         int numScans = enumerator.scanTo(term);
-        if (enumerator.term() != null && term.compareToUTF16(enumerator.term()) == 0) {
+        if (enumerator.term() != null && compareAsUTF16(term, enumerator.term()) == 0) {
           ti = enumerator.termInfo;
           if (numScans > 1) {
             // we only  want to put this TermInfo into the cache if
@@ -304,7 +316,7 @@ public final class TermInfosReader {
     enumerator.scanTo(term);
     final TermInfo ti;
 
-    if (enumerator.term() != null && term.compareToUTF16(enumerator.term()) == 0) {
+    if (enumerator.term() != null && compareAsUTF16(term, enumerator.term()) == 0) {
       ti = enumerator.termInfo;
       if (tiOrd == null) {
         if (useCache) {
@@ -361,9 +373,9 @@ public final class TermInfosReader {
     SegmentTermEnum enumerator = getThreadResources().termEnum;
     seekEnum(enumerator, indexOffset);
 
-    while(term.compareToUTF16(enumerator.term()) > 0 && enumerator.next()) {}
+    while(compareAsUTF16(term, enumerator.term()) > 0 && enumerator.next()) {}
 
-    if (term.compareToUTF16(enumerator.term()) == 0)
+    if (compareAsUTF16(term, enumerator.term()) == 0)
       return enumerator.position;
     else
       return -1;

Modified: lucene/dev/branches/LUCENE2793/lucene/src/java/org/apache/lucene/index/codecs/pulsing/PulsingPostingsReaderImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE2793/lucene/src/java/org/apache/lucene/index/codecs/pulsing/PulsingPostingsReaderImpl.java?rev=1143719&r1=1143718&r2=1143719&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE2793/lucene/src/java/org/apache/lucene/index/codecs/pulsing/PulsingPostingsReaderImpl.java (original)
+++ lucene/dev/branches/LUCENE2793/lucene/src/java/org/apache/lucene/index/codecs/pulsing/PulsingPostingsReaderImpl.java Thu Jul  7 09:03:58 2011
@@ -110,7 +110,7 @@ public class PulsingPostingsReaderImpl e
     final PulsingTermState termState = (PulsingTermState) _termState;
     if (termState.inlinedBytes == null) {
       termState.inlinedBytes = new byte[128];
-      termState.inlinedBytesReader = new ByteArrayDataInput(null);
+      termState.inlinedBytesReader = new ByteArrayDataInput();
     }
     int len = termsIn.readVInt();
     if (termState.inlinedBytes.length < len) {
@@ -167,7 +167,7 @@ public class PulsingPostingsReaderImpl e
   // TODO: we could actually reuse, by having TL that
   // holds the last wrapped reuse, and vice-versa
   @Override
-  public DocsEnum docs(FieldInfo field, BlockTermState _termState, Bits skipDocs, DocsEnum reuse) throws IOException {
+  public DocsEnum docs(FieldInfo field, BlockTermState _termState, Bits liveDocs, DocsEnum reuse) throws IOException {
     PulsingTermState termState = (PulsingTermState) _termState;
     if (termState.postingsSize != -1) {
       PulsingDocsEnum postings;
@@ -179,20 +179,20 @@ public class PulsingPostingsReaderImpl e
       } else {
         postings = new PulsingDocsEnum(field);
       }
-      return postings.reset(skipDocs, termState);
+      return postings.reset(liveDocs, termState);
     } else {
       // TODO: not great that we lose reuse of PulsingDocsEnum in this case:
       if (reuse instanceof PulsingDocsEnum) {
-        return wrappedPostingsReader.docs(field, termState.wrappedTermState, skipDocs, null);
+        return wrappedPostingsReader.docs(field, termState.wrappedTermState, liveDocs, null);
       } else {
-        return wrappedPostingsReader.docs(field, termState.wrappedTermState, skipDocs, reuse);
+        return wrappedPostingsReader.docs(field, termState.wrappedTermState, liveDocs, reuse);
       }
     }
   }
 
   // TODO: -- not great that we can't always reuse
   @Override
-  public DocsAndPositionsEnum docsAndPositions(FieldInfo field, BlockTermState _termState, Bits skipDocs, DocsAndPositionsEnum reuse) throws IOException {
+  public DocsAndPositionsEnum docsAndPositions(FieldInfo field, BlockTermState _termState, Bits liveDocs, DocsAndPositionsEnum reuse) throws IOException {
     if (field.omitTermFreqAndPositions) {
       return null;
     }
@@ -211,21 +211,21 @@ public class PulsingPostingsReaderImpl e
         postings = new PulsingDocsAndPositionsEnum(field);
       }
 
-      return postings.reset(skipDocs, termState);
+      return postings.reset(liveDocs, termState);
     } else {
       if (reuse instanceof PulsingDocsAndPositionsEnum) {
-        return wrappedPostingsReader.docsAndPositions(field, termState.wrappedTermState, skipDocs, null);
+        return wrappedPostingsReader.docsAndPositions(field, termState.wrappedTermState, liveDocs, null);
       } else {
-        return wrappedPostingsReader.docsAndPositions(field, termState.wrappedTermState, skipDocs, reuse);
+        return wrappedPostingsReader.docsAndPositions(field, termState.wrappedTermState, liveDocs, reuse);
       }
     }
   }
 
   private static class PulsingDocsEnum extends DocsEnum {
-    private final ByteArrayDataInput postings = new ByteArrayDataInput(null);
+    private final ByteArrayDataInput postings = new ByteArrayDataInput();
     private final boolean omitTF;
     private final boolean storePayloads;
-    private Bits skipDocs;
+    private Bits liveDocs;
     private int docID;
     private int freq;
     private int payloadLength;
@@ -235,7 +235,7 @@ public class PulsingPostingsReaderImpl e
       storePayloads = fieldInfo.storePayloads;
     }
 
-    public PulsingDocsEnum reset(Bits skipDocs, PulsingTermState termState) {
+    public PulsingDocsEnum reset(Bits liveDocs, PulsingTermState termState) {
       //System.out.println("PR docsEnum termState=" + termState + " docFreq=" + termState.docFreq);
       assert termState.postingsSize != -1;
       final byte[] bytes = new byte[termState.postingsSize];
@@ -244,7 +244,7 @@ public class PulsingPostingsReaderImpl e
       docID = 0;
       payloadLength = 0;
       freq = 1;
-      this.skipDocs = skipDocs;
+      this.liveDocs = liveDocs;
       return this;
     }
 
@@ -291,7 +291,7 @@ public class PulsingPostingsReaderImpl e
           }
         }
 
-        if (skipDocs == null || !skipDocs.get(docID)) {
+        if (liveDocs == null || liveDocs.get(docID)) {
           //System.out.println("  return docID=" + docID + " freq=" + freq);
           return docID;
         }
@@ -320,10 +320,10 @@ public class PulsingPostingsReaderImpl e
   }
 
   private static class PulsingDocsAndPositionsEnum extends DocsAndPositionsEnum {
-    private final ByteArrayDataInput postings = new ByteArrayDataInput(null);
+    private final ByteArrayDataInput postings = new ByteArrayDataInput();
     private final boolean storePayloads;
 
-    private Bits skipDocs;
+    private Bits liveDocs;
     private int docID;
     private int freq;
     private int posPending;
@@ -341,12 +341,12 @@ public class PulsingPostingsReaderImpl e
       return storePayloads == fieldInfo.storePayloads;
     }
 
-    public PulsingDocsAndPositionsEnum reset(Bits skipDocs, PulsingTermState termState) {
+    public PulsingDocsAndPositionsEnum reset(Bits liveDocs, PulsingTermState termState) {
       assert termState.postingsSize != -1;
       final byte[] bytes = new byte[termState.postingsSize];
       System.arraycopy(termState.postings, 0, bytes, 0, termState.postingsSize);
       postings.reset(bytes);
-      this.skipDocs = skipDocs;
+      this.liveDocs = liveDocs;
       payloadLength = 0;
       posPending = 0;
       docID = 0;
@@ -378,7 +378,7 @@ public class PulsingPostingsReaderImpl e
         }
         posPending = freq;
 
-        if (skipDocs == null || !skipDocs.get(docID)) {
+        if (liveDocs == null || liveDocs.get(docID)) {
           //System.out.println("  return docID=" + docID + " freq=" + freq);
           position = 0;
           return docID;

Modified: lucene/dev/branches/LUCENE2793/lucene/src/java/org/apache/lucene/index/codecs/sep/SepPostingsReaderImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE2793/lucene/src/java/org/apache/lucene/index/codecs/sep/SepPostingsReaderImpl.java?rev=1143719&r1=1143718&r2=1143719&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE2793/lucene/src/java/org/apache/lucene/index/codecs/sep/SepPostingsReaderImpl.java (original)
+++ lucene/dev/branches/LUCENE2793/lucene/src/java/org/apache/lucene/index/codecs/sep/SepPostingsReaderImpl.java Thu Jul  7 09:03:58 2011
@@ -257,7 +257,7 @@ public class SepPostingsReaderImpl exten
   }
 
   @Override
-  public DocsEnum docs(FieldInfo fieldInfo, BlockTermState _termState, Bits skipDocs, DocsEnum reuse) throws IOException {
+  public DocsEnum docs(FieldInfo fieldInfo, BlockTermState _termState, Bits liveDocs, DocsEnum reuse) throws IOException {
     final SepTermState termState = (SepTermState) _termState;
     SepDocsEnum docsEnum;
     if (reuse == null || !(reuse instanceof SepDocsEnum)) {
@@ -272,11 +272,11 @@ public class SepPostingsReaderImpl exten
       }
     }
 
-    return docsEnum.init(fieldInfo, termState, skipDocs);
+    return docsEnum.init(fieldInfo, termState, liveDocs);
   }
 
   @Override
-  public DocsAndPositionsEnum docsAndPositions(FieldInfo fieldInfo, BlockTermState _termState, Bits skipDocs, DocsAndPositionsEnum reuse) throws IOException {
+  public DocsAndPositionsEnum docsAndPositions(FieldInfo fieldInfo, BlockTermState _termState, Bits liveDocs, DocsAndPositionsEnum reuse) throws IOException {
     assert !fieldInfo.omitTermFreqAndPositions;
     final SepTermState termState = (SepTermState) _termState;
     SepDocsAndPositionsEnum postingsEnum;
@@ -292,7 +292,7 @@ public class SepPostingsReaderImpl exten
       }
     }
 
-    return postingsEnum.init(fieldInfo, termState, skipDocs);
+    return postingsEnum.init(fieldInfo, termState, liveDocs);
   }
 
   class SepDocsEnum extends DocsEnum {
@@ -305,7 +305,7 @@ public class SepPostingsReaderImpl exten
     // TODO: -- should we do omitTF with 2 different enum classes?
     private boolean omitTF;
     private boolean storePayloads;
-    private Bits skipDocs;
+    private Bits liveDocs;
     private final IntIndexInput.Reader docReader;
     private final IntIndexInput.Reader freqReader;
     private long skipFP;
@@ -338,8 +338,8 @@ public class SepPostingsReaderImpl exten
       }
     }
 
-    SepDocsEnum init(FieldInfo fieldInfo, SepTermState termState, Bits skipDocs) throws IOException {
-      this.skipDocs = skipDocs;
+    SepDocsEnum init(FieldInfo fieldInfo, SepTermState termState, Bits liveDocs) throws IOException {
+      this.liveDocs = liveDocs;
       omitTF = fieldInfo.omitTermFreqAndPositions;
       storePayloads = fieldInfo.storePayloads;
 
@@ -384,7 +384,7 @@ public class SepPostingsReaderImpl exten
           freq = freqReader.next();
         }
 
-        if (skipDocs == null || !skipDocs.get(doc)) {
+        if (liveDocs == null || liveDocs.get(doc)) {
           break;
         }
       }
@@ -409,7 +409,7 @@ public class SepPostingsReaderImpl exten
           freq = freqReader.next();
         }
 
-        if (skipDocs == null || !skipDocs.get(doc)) {
+        if (liveDocs == null || liveDocs.get(doc)) {
           docs[i] = doc;
           freqs[i] = freq;
           //System.out.println("  docs[" + i + "]=" + doc + " count=" + count + " dF=" + docFreq);
@@ -494,7 +494,7 @@ public class SepPostingsReaderImpl exten
     long freqStart;
 
     private boolean storePayloads;
-    private Bits skipDocs;
+    private Bits liveDocs;
     private final IntIndexInput.Reader docReader;
     private final IntIndexInput.Reader freqReader;
     private final IntIndexInput.Reader posReader;
@@ -529,8 +529,8 @@ public class SepPostingsReaderImpl exten
       payloadIn = (IndexInput) SepPostingsReaderImpl.this.payloadIn.clone();
     }
 
-    SepDocsAndPositionsEnum init(FieldInfo fieldInfo, SepTermState termState, Bits skipDocs) throws IOException {
-      this.skipDocs = skipDocs;
+    SepDocsAndPositionsEnum init(FieldInfo fieldInfo, SepTermState termState, Bits liveDocs) throws IOException {
+      this.liveDocs = liveDocs;
       storePayloads = fieldInfo.storePayloads;
       //System.out.println("Sep D&P init");
 
@@ -585,7 +585,7 @@ public class SepPostingsReaderImpl exten
 
         pendingPosCount += freq;
 
-        if (skipDocs == null || !skipDocs.get(doc)) {
+        if (liveDocs == null || liveDocs.get(doc)) {
           break;
         }
       }

Modified: lucene/dev/branches/LUCENE2793/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextFieldsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE2793/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextFieldsReader.java?rev=1143719&r1=1143718&r2=1143719&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE2793/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextFieldsReader.java (original)
+++ lucene/dev/branches/LUCENE2793/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextFieldsReader.java Thu Jul  7 09:03:58 2011
@@ -30,7 +30,6 @@ import org.apache.lucene.store.IndexInpu
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.CharsRef;
-import org.apache.lucene.util.StringHelper;
 import org.apache.lucene.util.UnicodeUtil;
 import org.apache.lucene.util.fst.Builder;
 import org.apache.lucene.util.fst.BytesRefFSTEnum;
@@ -102,9 +101,7 @@ class SimpleTextFieldsReader extends Fie
           return null;
         }
         if (scratch.startsWith(FIELD)) {
-          String field = StringHelper.intern(new String(scratch.bytes, scratch.offset + FIELD.length, scratch.length - FIELD.length, "UTF-8"));
-          current = field;
-          return field;
+          return current = new String(scratch.bytes, scratch.offset + FIELD.length, scratch.length - FIELD.length, "UTF-8");
         }
       }
     }
@@ -131,7 +128,23 @@ class SimpleTextFieldsReader extends Fie
     }
 
     @Override
-    public SeekStatus seek(BytesRef text, boolean useCache /* ignored */) throws IOException {
+    public boolean seekExact(BytesRef text, boolean useCache /* ignored */) throws IOException {
+
+      final BytesRefFSTEnum.InputOutput<PairOutputs.Pair<Long,PairOutputs.Pair<Long,Long>>> result = fstEnum.seekExact(text);
+      if (result != null) {
+        PairOutputs.Pair<Long,PairOutputs.Pair<Long,Long>> pair1 = result.output;
+        PairOutputs.Pair<Long,Long> pair2 = pair1.output2;
+        docsStart = pair1.output1;
+        docFreq = pair2.output1.intValue();
+        totalTermFreq = pair2.output2;
+        return true;
+      } else {
+        return false;
+      }
+    }
+
+    @Override
+    public SeekStatus seekCeil(BytesRef text, boolean useCache /* ignored */) throws IOException {
 
       //System.out.println("seek to text=" + text.utf8ToString());
       final BytesRefFSTEnum.InputOutput<PairOutputs.Pair<Long,PairOutputs.Pair<Long,Long>>> result = fstEnum.seekCeil(text);
@@ -183,7 +196,7 @@ class SimpleTextFieldsReader extends Fie
     }
 
     @Override
-    public SeekStatus seek(long ord) {
+    public void seekExact(long ord) {
       throw new UnsupportedOperationException();
     }
 
@@ -198,18 +211,18 @@ class SimpleTextFieldsReader extends Fie
     }
  
     @Override
-    public DocsEnum docs(Bits skipDocs, DocsEnum reuse) throws IOException {
+    public DocsEnum docs(Bits liveDocs, DocsEnum reuse) throws IOException {
       SimpleTextDocsEnum docsEnum;
       if (reuse != null && reuse instanceof SimpleTextDocsEnum && ((SimpleTextDocsEnum) reuse).canReuse(in)) {
         docsEnum = (SimpleTextDocsEnum) reuse;
       } else {
         docsEnum = new SimpleTextDocsEnum();
       }
-      return docsEnum.reset(docsStart, skipDocs, omitTF);
+      return docsEnum.reset(docsStart, liveDocs, omitTF);
     }
 
     @Override
-    public DocsAndPositionsEnum docsAndPositions(Bits skipDocs, DocsAndPositionsEnum reuse) throws IOException {
+    public DocsAndPositionsEnum docsAndPositions(Bits liveDocs, DocsAndPositionsEnum reuse) throws IOException {
       if (omitTF) {
         return null;
       }
@@ -220,7 +233,7 @@ class SimpleTextFieldsReader extends Fie
       } else {
         docsAndPositionsEnum = new SimpleTextDocsAndPositionsEnum();
       } 
-      return docsAndPositionsEnum.reset(docsStart, skipDocs);
+      return docsAndPositionsEnum.reset(docsStart, liveDocs);
     }
     
     @Override
@@ -235,7 +248,7 @@ class SimpleTextFieldsReader extends Fie
     private boolean omitTF;
     private int docID;
     private int tf;
-    private Bits skipDocs;
+    private Bits liveDocs;
     private final BytesRef scratch = new BytesRef(10);
     private final CharsRef scratchUTF16 = new CharsRef(10);
     
@@ -248,8 +261,8 @@ class SimpleTextFieldsReader extends Fie
       return in == inStart;
     }
 
-    public SimpleTextDocsEnum reset(long fp, Bits skipDocs, boolean omitTF) throws IOException {
-      this.skipDocs = skipDocs;
+    public SimpleTextDocsEnum reset(long fp, Bits liveDocs, boolean omitTF) throws IOException {
+      this.liveDocs = liveDocs;
       in.seek(fp);
       this.omitTF = omitTF;
       if (omitTF) {
@@ -279,7 +292,7 @@ class SimpleTextFieldsReader extends Fie
         final long lineStart = in.getFilePointer();
         readLine(in, scratch);
         if (scratch.startsWith(DOC)) {
-          if (!first && (skipDocs == null || !skipDocs.get(docID))) {
+          if (!first && (liveDocs == null || liveDocs.get(docID))) {
             in.seek(lineStart);
             if (!omitTF) {
               tf = termFreq;
@@ -296,7 +309,7 @@ class SimpleTextFieldsReader extends Fie
           // skip
         } else {
           assert scratch.startsWith(TERM) || scratch.startsWith(FIELD) || scratch.startsWith(END): "scratch=" + scratch.utf8ToString();
-          if (!first && (skipDocs == null || !skipDocs.get(docID))) {
+          if (!first && (liveDocs == null || liveDocs.get(docID))) {
             in.seek(lineStart);
             if (!omitTF) {
               tf = termFreq;
@@ -321,7 +334,7 @@ class SimpleTextFieldsReader extends Fie
     private final IndexInput in;
     private int docID;
     private int tf;
-    private Bits skipDocs;
+    private Bits liveDocs;
     private final BytesRef scratch = new BytesRef(10);
     private final BytesRef scratch2 = new BytesRef(10);
     private final CharsRef scratchUTF16 = new CharsRef(10);
@@ -338,8 +351,8 @@ class SimpleTextFieldsReader extends Fie
       return in == inStart;
     }
 
-    public SimpleTextDocsAndPositionsEnum reset(long fp, Bits skipDocs) {
-      this.skipDocs = skipDocs;
+    public SimpleTextDocsAndPositionsEnum reset(long fp, Bits liveDocs) {
+      this.liveDocs = liveDocs;
       nextDocStart = fp;
       return this;
     }
@@ -363,7 +376,7 @@ class SimpleTextFieldsReader extends Fie
         final long lineStart = in.getFilePointer();
         readLine(in, scratch);
         if (scratch.startsWith(DOC)) {
-          if (!first && (skipDocs == null || !skipDocs.get(docID))) {
+          if (!first && (liveDocs == null || liveDocs.get(docID))) {
             nextDocStart = lineStart;
             in.seek(posStart);
             return docID;
@@ -379,7 +392,7 @@ class SimpleTextFieldsReader extends Fie
           // skip
         } else {
           assert scratch.startsWith(TERM) || scratch.startsWith(FIELD) || scratch.startsWith(END);
-          if (!first && (skipDocs == null || !skipDocs.get(docID))) {
+          if (!first && (liveDocs == null || liveDocs.get(docID))) {
             nextDocStart = lineStart;
             in.seek(posStart);
             return docID;

Modified: lucene/dev/branches/LUCENE2793/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardPostingsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE2793/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardPostingsReader.java?rev=1143719&r1=1143718&r2=1143719&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE2793/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardPostingsReader.java (original)
+++ lucene/dev/branches/LUCENE2793/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardPostingsReader.java Thu Jul  7 09:03:58 2011
@@ -157,7 +157,7 @@ public class StandardPostingsReader exte
     //System.out.println("SPR.readTermsBlock termsIn.fp=" + termsIn.getFilePointer());
     if (termState.bytes == null) {
       termState.bytes = new byte[ArrayUtil.oversize(len, 1)];
-      termState.bytesReader = new ByteArrayDataInput(null);
+      termState.bytesReader = new ByteArrayDataInput();
     } else if (termState.bytes.length < len) {
       termState.bytes = new byte[ArrayUtil.oversize(len, 1)];
     }
@@ -201,7 +201,7 @@ public class StandardPostingsReader exte
   }
     
   @Override
-  public DocsEnum docs(FieldInfo fieldInfo, BlockTermState termState, Bits skipDocs, DocsEnum reuse) throws IOException {
+  public DocsEnum docs(FieldInfo fieldInfo, BlockTermState termState, Bits liveDocs, DocsEnum reuse) throws IOException {
     SegmentDocsEnum docsEnum;
     if (reuse == null || !(reuse instanceof SegmentDocsEnum)) {
       docsEnum = new SegmentDocsEnum(freqIn);
@@ -214,11 +214,11 @@ public class StandardPostingsReader exte
         docsEnum = new SegmentDocsEnum(freqIn);
       }
     }
-    return docsEnum.reset(fieldInfo, (StandardTermState) termState, skipDocs);
+    return docsEnum.reset(fieldInfo, (StandardTermState) termState, liveDocs);
   }
 
   @Override
-  public DocsAndPositionsEnum docsAndPositions(FieldInfo fieldInfo, BlockTermState termState, Bits skipDocs, DocsAndPositionsEnum reuse) throws IOException {
+  public DocsAndPositionsEnum docsAndPositions(FieldInfo fieldInfo, BlockTermState termState, Bits liveDocs, DocsAndPositionsEnum reuse) throws IOException {
     if (fieldInfo.omitTermFreqAndPositions) {
       return null;
     }
@@ -237,7 +237,7 @@ public class StandardPostingsReader exte
           docsEnum = new SegmentDocsAndPositionsAndPayloadsEnum(freqIn, proxIn);
         }
       }
-      return docsEnum.reset(fieldInfo, (StandardTermState) termState, skipDocs);
+      return docsEnum.reset(fieldInfo, (StandardTermState) termState, liveDocs);
     } else {
       SegmentDocsAndPositionsEnum docsEnum;
       if (reuse == null || !(reuse instanceof SegmentDocsAndPositionsEnum)) {
@@ -251,7 +251,7 @@ public class StandardPostingsReader exte
           docsEnum = new SegmentDocsAndPositionsEnum(freqIn, proxIn);
         }
       }
-      return docsEnum.reset(fieldInfo, (StandardTermState) termState, skipDocs);
+      return docsEnum.reset(fieldInfo, (StandardTermState) termState, liveDocs);
     }
   }
 
@@ -268,7 +268,7 @@ public class StandardPostingsReader exte
     int doc;                                      // doc we last read
     int freq;                                     // freq we last read
 
-    Bits skipDocs;
+    Bits liveDocs;
 
     long freqOffset;
     int skipOffset;
@@ -281,13 +281,13 @@ public class StandardPostingsReader exte
       this.freqIn = (IndexInput) freqIn.clone();
     }
 
-    public SegmentDocsEnum reset(FieldInfo fieldInfo, StandardTermState termState, Bits skipDocs) throws IOException {
+    public SegmentDocsEnum reset(FieldInfo fieldInfo, StandardTermState termState, Bits liveDocs) throws IOException {
       omitTF = fieldInfo.omitTermFreqAndPositions;
       if (omitTF) {
         freq = 1;
       }
       storePayloads = fieldInfo.storePayloads;
-      this.skipDocs = skipDocs;
+      this.liveDocs = liveDocs;
       freqOffset = termState.freqOffset;
       skipOffset = termState.skipOffset;
 
@@ -328,7 +328,7 @@ public class StandardPostingsReader exte
           }
         }
 
-        if (skipDocs == null || !skipDocs.get(doc)) {
+        if (liveDocs == null || liveDocs.get(doc)) {
           break;
         }
       }
@@ -358,7 +358,7 @@ public class StandardPostingsReader exte
           }
         }
 
-        if (skipDocs == null || !skipDocs.get(doc)) {
+        if (liveDocs == null || liveDocs.get(doc)) {
           docs[i] = doc;
           freqs[i] = freq;
           ++i;
@@ -436,7 +436,7 @@ public class StandardPostingsReader exte
     int freq;                                     // freq we last read
     int position;
 
-    Bits skipDocs;
+    Bits liveDocs;
 
     long freqOffset;
     int skipOffset;
@@ -454,11 +454,11 @@ public class StandardPostingsReader exte
       this.proxIn = (IndexInput) proxIn.clone();
     }
 
-    public SegmentDocsAndPositionsEnum reset(FieldInfo fieldInfo, StandardTermState termState, Bits skipDocs) throws IOException {
+    public SegmentDocsAndPositionsEnum reset(FieldInfo fieldInfo, StandardTermState termState, Bits liveDocs) throws IOException {
       assert !fieldInfo.omitTermFreqAndPositions;
       assert !fieldInfo.storePayloads;
 
-      this.skipDocs = skipDocs;
+      this.liveDocs = liveDocs;
 
       // TODO: for full enum case (eg segment merging) this
       // seek is unnecessary; maybe we can avoid in such
@@ -505,7 +505,7 @@ public class StandardPostingsReader exte
         }
         posPendingCount += freq;
 
-        if (skipDocs == null || !skipDocs.get(doc)) {
+        if (liveDocs == null || liveDocs.get(doc)) {
           break;
         }
       }
@@ -627,7 +627,7 @@ public class StandardPostingsReader exte
     int freq;                                     // freq we last read
     int position;
 
-    Bits skipDocs;
+    Bits liveDocs;
 
     long freqOffset;
     int skipOffset;
@@ -648,7 +648,7 @@ public class StandardPostingsReader exte
       this.proxIn = (IndexInput) proxIn.clone();
     }
 
-    public SegmentDocsAndPositionsAndPayloadsEnum reset(FieldInfo fieldInfo, StandardTermState termState, Bits skipDocs) throws IOException {
+    public SegmentDocsAndPositionsAndPayloadsEnum reset(FieldInfo fieldInfo, StandardTermState termState, Bits liveDocs) throws IOException {
       assert !fieldInfo.omitTermFreqAndPositions;
       assert fieldInfo.storePayloads;
       if (payload == null) {
@@ -656,7 +656,7 @@ public class StandardPostingsReader exte
         payload.bytes = new byte[1];
       }
 
-      this.skipDocs = skipDocs;
+      this.liveDocs = liveDocs;
 
       // TODO: for full enum case (eg segment merging) this
       // seek is unnecessary; maybe we can avoid in such
@@ -702,7 +702,7 @@ public class StandardPostingsReader exte
         }
         posPendingCount += freq;
 
-        if (skipDocs == null || !skipDocs.get(doc)) {
+        if (liveDocs == null || liveDocs.get(doc)) {
           break;
         }
       }

Modified: lucene/dev/branches/LUCENE2793/lucene/src/java/org/apache/lucene/index/values/Bytes.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE2793/lucene/src/java/org/apache/lucene/index/values/Bytes.java?rev=1143719&r1=1143718&r2=1143719&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE2793/lucene/src/java/org/apache/lucene/index/values/Bytes.java (original)
+++ lucene/dev/branches/LUCENE2793/lucene/src/java/org/apache/lucene/index/values/Bytes.java Thu Jul  7 09:03:58 2011
@@ -32,7 +32,6 @@ import org.apache.lucene.store.IOContext
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.store.IndexOutput;
 import org.apache.lucene.util.AttributeSource;
-import org.apache.lucene.util.ByteBlockPool;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.CodecUtil;
 import org.apache.lucene.util.IOUtils;
@@ -102,6 +101,7 @@ public final class Bytes {
    *          {@link Writer}. A call to {@link Writer#finish(int)} will release
    *          all internally used resources and frees the memeory tracking
    *          reference.
+   * @param context 
    * @return a new {@link Writer} instance
    * @throws IOException
    *           if the files for the writer can not be created.
@@ -109,7 +109,6 @@ public final class Bytes {
   public static Writer getWriter(Directory dir, String id, Mode mode,
       Comparator<BytesRef> comp, boolean fixedSize, AtomicLong bytesUsed, IOContext context)
       throws IOException {
-
     // TODO -- i shouldn't have to specify fixed? can
     // track itself & do the write thing at write time?
     if (comp == null) {
@@ -118,7 +117,7 @@ public final class Bytes {
 
     if (fixedSize) {
       if (mode == Mode.STRAIGHT) {
-        return new FixedStraightBytesImpl.Writer(dir, id, context);
+        return new FixedStraightBytesImpl.Writer(dir, id, bytesUsed, context);
       } else if (mode == Mode.DEREF) {
         return new FixedDerefBytesImpl.Writer(dir, id, bytesUsed, context);
       } else if (mode == Mode.SORTED) {
@@ -340,37 +339,57 @@ public final class Bytes {
   // TODO: open up this API?!
   static abstract class BytesWriterBase extends Writer {
     private final String id;
-    protected IndexOutput idxOut;
-    protected IndexOutput datOut;
+    private IndexOutput idxOut;
+    private IndexOutput datOut;
     protected BytesRef bytesRef;
-    protected final ByteBlockPool pool;
+    private final Directory dir;
+    private final String codecName;
+    private final int version;
+    private final IOContext context;
 
     protected BytesWriterBase(Directory dir, String id, String codecName,
-        int version, boolean initIndex, ByteBlockPool pool,
-        AtomicLong bytesUsed, IOContext context) throws IOException {
+        int version, AtomicLong bytesUsed, IOContext context) throws IOException {
       super(bytesUsed);
       this.id = id;
-      this.pool = pool;
-      datOut = dir.createOutput(IndexFileNames.segmentFileName(id, "",
-            DATA_EXTENSION), context);
+      this.dir = dir;
+      this.codecName = codecName;
+      this.version = version;
+      this.context = context;
+    }
+    
+    protected IndexOutput getDataOut() throws IOException {
+      if (datOut == null) {
+        boolean success = false;
+        try {
+          datOut = dir.createOutput(IndexFileNames.segmentFileName(id, "",
+              DATA_EXTENSION), context);
+          CodecUtil.writeHeader(datOut, codecName, version);
+          success = true;
+        } finally {
+          if (!success) {
+            IOUtils.closeSafely(true, datOut);
+          }
+        }
+      }
+      return datOut;
+    }
+
+    protected IndexOutput getIndexOut() throws IOException {
       boolean success = false;
       try {
-        CodecUtil.writeHeader(datOut, codecName, version);
-        if (initIndex) {
+        if (idxOut == null) {
           idxOut = dir.createOutput(IndexFileNames.segmentFileName(id, "",
               INDEX_EXTENSION), context);
           CodecUtil.writeHeader(idxOut, codecName, version);
-        } else {
-          idxOut = null;
         }
         success = true;
       } finally {
         if (!success) {
-          IOUtils.closeSafely(true, datOut, idxOut);
+          IOUtils.closeSafely(true, idxOut);
         }
       }
+      return idxOut;
     }
-
     /**
      * Must be called only with increasing docIDs. It's OK for some docIDs to be
      * skipped; they will be filled with 0 bytes.
@@ -379,18 +398,10 @@ public final class Bytes {
     public abstract void add(int docID, BytesRef bytes) throws IOException;
 
     @Override
-    public void finish(int docCount) throws IOException {
-      try {
-        IOUtils.closeSafely(false, datOut, idxOut);
-      } finally {
-        if (pool != null) {
-          pool.reset();
-        }
-      }
-    }
+    public abstract void finish(int docCount) throws IOException;
 
     @Override
-    protected void add(int docID) throws IOException {
+    protected void mergeDoc(int docID) throws IOException {
       add(docID, bytesRef);
     }
 

Modified: lucene/dev/branches/LUCENE2793/lucene/src/java/org/apache/lucene/index/values/FixedDerefBytesImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE2793/lucene/src/java/org/apache/lucene/index/values/FixedDerefBytesImpl.java?rev=1143719&r1=1143718&r2=1143719&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE2793/lucene/src/java/org/apache/lucene/index/values/FixedDerefBytesImpl.java (original)
+++ lucene/dev/branches/LUCENE2793/lucene/src/java/org/apache/lucene/index/values/FixedDerefBytesImpl.java Thu Jul  7 09:03:58 2011
@@ -26,11 +26,13 @@ import org.apache.lucene.index.values.By
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.store.IndexOutput;
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.AttributeSource;
 import org.apache.lucene.util.ByteBlockPool;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefHash;
+import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.PagedBytes;
 import org.apache.lucene.util.RamUsageEstimator;
 import org.apache.lucene.util.ByteBlockPool.Allocator;
@@ -52,9 +54,7 @@ class FixedDerefBytesImpl {
   static class Writer extends BytesWriterBase {
     private int size = -1;
     private int[] docToID;
-    private final BytesRefHash hash = new BytesRefHash(pool,
-        BytesRefHash.DEFAULT_CAPACITY, new TrackingDirectBytesStartArray(
-            BytesRefHash.DEFAULT_CAPACITY, bytesUsed));
+    private final BytesRefHash hash;
     public Writer(Directory dir, String id, AtomicLong bytesUsed, IOContext context)
         throws IOException {
       this(dir, id, new DirectTrackingAllocator(ByteBlockPool.BYTE_BLOCK_SIZE, bytesUsed),
@@ -63,11 +63,12 @@ class FixedDerefBytesImpl {
 
     public Writer(Directory dir, String id, Allocator allocator,
         AtomicLong bytesUsed, IOContext context) throws IOException {
-      super(dir, id, CODEC_NAME, VERSION_CURRENT, true,
-          new ByteBlockPool(allocator), bytesUsed, context);
+      super(dir, id, CODEC_NAME, VERSION_CURRENT, bytesUsed, context);
+      hash = new BytesRefHash(new ByteBlockPool(allocator),
+          BytesRefHash.DEFAULT_CAPACITY, new TrackingDirectBytesStartArray(
+              BytesRefHash.DEFAULT_CAPACITY, bytesUsed));
       docToID = new int[1];
-      bytesUsed.addAndGet(RamUsageEstimator.NUM_BYTES_INT); // TODO BytesRefHash
-                                                            // uses bytes too!
+      bytesUsed.addAndGet(RamUsageEstimator.NUM_BYTES_INT);
     }
 
     @Override
@@ -76,20 +77,14 @@ class FixedDerefBytesImpl {
         return;
       if (size == -1) {
         size = bytes.length;
-        datOut.writeInt(size);
       } else if (bytes.length != size) {
         throw new IllegalArgumentException("expected bytes size=" + size
             + " but got " + bytes.length);
       }
       int ord = hash.add(bytes);
-
-      if (ord >= 0) {
-        // new added entry
-        datOut.writeBytes(bytes.bytes, bytes.offset, bytes.length);
-      } else {
+      if (ord < 0) {
         ord = (-ord) - 1;
       }
-
       if (docID >= docToID.length) {
         final int size = docToID.length;
         docToID = ArrayUtil.grow(docToID, 1 + docID);
@@ -103,11 +98,27 @@ class FixedDerefBytesImpl {
     // some last docs that we didn't see
     @Override
     public void finish(int docCount) throws IOException {
+      boolean success = false;
+      final int numValues = hash.size();
+      final IndexOutput datOut = getDataOut();
       try {
-        if (size == -1) {
-          datOut.writeInt(size);
+        datOut.writeInt(size);
+        if (size != -1) {
+          final BytesRef bytesRef = new BytesRef(size);
+          for (int i = 0; i < numValues; i++) {
+            hash.get(i, bytesRef);
+            datOut.writeBytes(bytesRef.bytes, bytesRef.offset, bytesRef.length);
+          }
         }
-        final int count = 1 + hash.size();
+        success = true;
+      } finally {
+        IOUtils.closeSafely(!success, datOut);
+        hash.close();
+      }
+      success = false;
+      final IndexOutput idxOut = getIndexOut();
+      try {
+        final int count = 1 + numValues;
         idxOut.writeInt(count - 1);
         // write index
         final PackedInts.Writer w = PackedInts.getWriter(idxOut, docCount,
@@ -121,9 +132,9 @@ class FixedDerefBytesImpl {
           w.add(0);
         }
         w.finish();
+        success = true;
       } finally {
-        hash.close();
-        super.finish(docCount);
+        IOUtils.closeSafely(!success, idxOut);
         bytesUsed
             .addAndGet((-docToID.length) * RamUsageEstimator.NUM_BYTES_INT);
         docToID = null;

Modified: lucene/dev/branches/LUCENE2793/lucene/src/java/org/apache/lucene/index/values/FixedSortedBytesImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE2793/lucene/src/java/org/apache/lucene/index/values/FixedSortedBytesImpl.java?rev=1143719&r1=1143718&r2=1143719&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE2793/lucene/src/java/org/apache/lucene/index/values/FixedSortedBytesImpl.java (original)
+++ lucene/dev/branches/LUCENE2793/lucene/src/java/org/apache/lucene/index/values/FixedSortedBytesImpl.java Thu Jul  7 09:03:58 2011
@@ -28,12 +28,14 @@ import org.apache.lucene.index.values.Fi
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.store.IndexOutput;
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.AttributeSource;
 import org.apache.lucene.util.ByteBlockPool;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefHash;
 import org.apache.lucene.util.CodecUtil;
+import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.PagedBytes;
 import org.apache.lucene.util.RamUsageEstimator;
 import org.apache.lucene.util.ByteBlockPool.Allocator;
@@ -57,10 +59,7 @@ class FixedSortedBytesImpl {
     private int size = -1;
     private int[] docToEntry;
     private final Comparator<BytesRef> comp;
-
-    private final BytesRefHash hash = new BytesRefHash(pool,
-        BytesRefHash.DEFAULT_CAPACITY, new TrackingDirectBytesStartArray(
-            BytesRefHash.DEFAULT_CAPACITY, bytesUsed));
+    private final BytesRefHash hash;
 
     public Writer(Directory dir, String id, Comparator<BytesRef> comp,
         AtomicLong bytesUsed, IOContext context) throws IOException {
@@ -70,10 +69,12 @@ class FixedSortedBytesImpl {
 
     public Writer(Directory dir, String id, Comparator<BytesRef> comp,
         Allocator allocator, AtomicLong bytesUsed, IOContext context) throws IOException {
-      super(dir, id, CODEC_NAME, VERSION_CURRENT, true,
-          new ByteBlockPool(allocator), bytesUsed, context);
+      super(dir, id, CODEC_NAME, VERSION_CURRENT, bytesUsed, context);
+      ByteBlockPool pool = new ByteBlockPool(allocator);
+      hash = new BytesRefHash(pool, BytesRefHash.DEFAULT_CAPACITY,
+          new TrackingDirectBytesStartArray(BytesRefHash.DEFAULT_CAPACITY,
+              bytesUsed));
       docToEntry = new int[1];
-      // docToEntry[0] = -1;
       bytesUsed.addAndGet(RamUsageEstimator.NUM_BYTES_INT);
       this.comp = comp;
     }
@@ -84,7 +85,6 @@ class FixedSortedBytesImpl {
         return; // default - skip it
       if (size == -1) {
         size = bytes.length;
-        datOut.writeInt(size);
       } else if (bytes.length != size) {
         throw new IllegalArgumentException("expected bytes size=" + size
             + " but got " + bytes.length);
@@ -105,26 +105,36 @@ class FixedSortedBytesImpl {
     // some last docs that we didn't see
     @Override
     public void finish(int docCount) throws IOException {
+      final IndexOutput datOut = getDataOut();
+      boolean success = false;
+      final int count = hash.size();
+      final int[] address = new int[count];
+
       try {
-        if (size == -1) {// no data added
-          datOut.writeInt(size);
-        }
-        final int[] sortedEntries = hash.sort(comp);
-        final int count = hash.size();
-        int[] address = new int[count];
-        // first dump bytes data, recording address as we go
-        for (int i = 0; i < count; i++) {
-          final int e = sortedEntries[i];
-          final BytesRef bytes = hash.get(e, new BytesRef());
-          assert bytes.length == size;
-          datOut.writeBytes(bytes.bytes, bytes.offset, bytes.length);
-          address[e] = 1 + i;
+        datOut.writeInt(size);
+        if (size != -1) {
+          final int[] sortedEntries = hash.sort(comp);
+          // first dump bytes data, recording address as we go
+          final BytesRef bytesRef = new BytesRef(size);
+          for (int i = 0; i < count; i++) {
+            final int e = sortedEntries[i];
+            final BytesRef bytes = hash.get(e, bytesRef);
+            assert bytes.length == size;
+            datOut.writeBytes(bytes.bytes, bytes.offset, bytes.length);
+            address[e] = 1 + i;
+          }
         }
-
+        success = true;
+      } finally {
+        IOUtils.closeSafely(!success, datOut);
+        hash.close();
+      }
+      final IndexOutput idxOut = getIndexOut();
+      success = false;
+      try {
         idxOut.writeInt(count);
-
         // next write index
-        PackedInts.Writer w = PackedInts.getWriter(idxOut, docCount,
+        final PackedInts.Writer w = PackedInts.getWriter(idxOut, docCount,
             PackedInts.bitsRequired(count));
         final int limit;
         if (docCount > docToEntry.length) {
@@ -149,11 +159,10 @@ class FixedSortedBytesImpl {
         }
         w.finish();
       } finally {
-        super.finish(docCount);
+        IOUtils.closeSafely(!success, idxOut);
         bytesUsed.addAndGet((-docToEntry.length)
             * RamUsageEstimator.NUM_BYTES_INT);
         docToEntry = null;
-        hash.close();
       }
     }
   }

Modified: lucene/dev/branches/LUCENE2793/lucene/src/java/org/apache/lucene/index/values/FixedStraightBytesImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE2793/lucene/src/java/org/apache/lucene/index/values/FixedStraightBytesImpl.java?rev=1143719&r1=1143718&r2=1143719&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE2793/lucene/src/java/org/apache/lucene/index/values/FixedStraightBytesImpl.java (original)
+++ lucene/dev/branches/LUCENE2793/lucene/src/java/org/apache/lucene/index/values/FixedStraightBytesImpl.java Thu Jul  7 09:03:58 2011
@@ -17,7 +17,10 @@ package org.apache.lucene.index.values;
  * limitations under the License.
  */
 
+import static org.apache.lucene.util.ByteBlockPool.BYTE_BLOCK_SIZE;
+
 import java.io.IOException;
+import java.util.concurrent.atomic.AtomicLong;
 
 import org.apache.lucene.index.values.Bytes.BytesBaseSource;
 import org.apache.lucene.index.values.Bytes.BytesReaderBase;
@@ -25,7 +28,10 @@ import org.apache.lucene.index.values.By
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.store.IndexOutput;
 import org.apache.lucene.util.AttributeSource;
+import org.apache.lucene.util.ByteBlockPool;
+import org.apache.lucene.util.ByteBlockPool.DirectTrackingAllocator;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.PagedBytes;
@@ -45,31 +51,60 @@ class FixedStraightBytesImpl {
     private int size = -1;
     // start at -1 if the first added value is > 0
     private int lastDocID = -1;
-    private byte[] oneRecord;
-
-    public Writer(Directory dir, String id, IOContext context) throws IOException {
-      super(dir, id, CODEC_NAME, VERSION_CURRENT, false, null, null, context);
+    private final ByteBlockPool pool;
+    private boolean merge;
+    private final int byteBlockSize;
+    private IndexOutput datOut;
+
+    public Writer(Directory dir, String id, AtomicLong bytesUsed, IOContext context) throws IOException {
+      super(dir, id, CODEC_NAME, VERSION_CURRENT, bytesUsed, context);
+      pool = new ByteBlockPool(new DirectTrackingAllocator(bytesUsed));
+      byteBlockSize = BYTE_BLOCK_SIZE;
     }
 
-
     @Override
     public void add(int docID, BytesRef bytes) throws IOException {
+      assert lastDocID < docID;
+      assert !merge;
       if (size == -1) {
+        if (bytes.length > BYTE_BLOCK_SIZE) {
+          throw new IllegalArgumentException("bytes arrays > " + Short.MAX_VALUE + " are not supported");
+        }
         size = bytes.length;
-        datOut.writeInt(size);
-        oneRecord = new byte[size];
+        pool.nextBuffer();
       } else if (bytes.length != size) {
         throw new IllegalArgumentException("expected bytes size=" + size
             + " but got " + bytes.length);
       }
-      fill(docID);
-      assert bytes.bytes.length >= bytes.length;
-      datOut.writeBytes(bytes.bytes, bytes.offset, bytes.length);
+      if (lastDocID+1 < docID) {
+        advancePool(docID);
+      }
+      pool.copy(bytes);
+      lastDocID = docID;
+    }
+    
+    private final void advancePool(int docID) {
+      assert !merge;
+      long numBytes = (docID - (lastDocID+1))*size;
+      while(numBytes > 0) {
+        if (numBytes + pool.byteUpto < byteBlockSize) {
+          pool.byteUpto += numBytes;
+          numBytes = 0;
+        } else {
+          numBytes -= byteBlockSize - pool.byteUpto;
+          pool.nextBuffer();
+        }
+      }
+      assert numBytes == 0;
     }
 
     @Override
     protected void merge(MergeState state) throws IOException {
-      if (state.bits == null && state.reader instanceof Reader) {
+      merge = true;
+      datOut = getDataOut();
+      boolean success = false;
+      try {
+      if (state.liveDocs == null && state.reader instanceof Reader) {
         Reader reader = (Reader) state.reader;
         final int maxDocs = reader.maxDoc;
         if (maxDocs == 0) {
@@ -78,48 +113,92 @@ class FixedStraightBytesImpl {
         if (size == -1) {
           size = reader.size;
           datOut.writeInt(size);
-          oneRecord = new byte[size];
         }
-        fill(state.docBase);
+        if (lastDocID+1 < state.docBase) {
+          fill(datOut, state.docBase);
+          lastDocID = state.docBase-1;
+        }
         // TODO should we add a transfer to API to each reader?
         final IndexInput cloneData = reader.cloneData();
         try {
           datOut.copyBytes(cloneData, size * maxDocs);
         } finally {
-          cloneData.close();  
+          IOUtils.closeSafely(true, cloneData);  
         }
         
-        lastDocID += maxDocs - 1;
+        lastDocID += maxDocs;
       } else {
         super.merge(state);
       }
+      success = true;
+      } finally {
+        if (!success) {
+          IOUtils.closeSafely(!success, datOut);
+        }
+      }
+    }
+    
+    
+
+    @Override
+    protected void mergeDoc(int docID) throws IOException {
+      assert lastDocID < docID;
+      if (size == -1) {
+        size = bytesRef.length;
+        datOut.writeInt(size);
+      }
+      assert size == bytesRef.length;
+      if (lastDocID+1 < docID) {
+        fill(datOut, docID);
+      }
+      datOut.writeBytes(bytesRef.bytes, bytesRef.offset, bytesRef.length);
+      lastDocID = docID;
     }
 
+
+
     // Fills up to but not including this docID
-    private void fill(int docID) throws IOException {
+    private void fill(IndexOutput datOut, int docID) throws IOException {
       assert size >= 0;
-      for (int i = lastDocID + 1; i < docID; i++) {
-        datOut.writeBytes(oneRecord, size);
+      final long numBytes = (docID - (lastDocID+1))*size;
+      final byte zero = 0;
+      for (long i = 0; i < numBytes; i++) {
+        datOut.writeByte(zero);
       }
-      lastDocID = docID;
     }
 
     @Override
     public void finish(int docCount) throws IOException {
+      boolean success = false;
       try {
-        if (size == -1) {// no data added
-          datOut.writeInt(0);
+        if (!merge) {
+          // indexing path - no disk IO until here
+          assert datOut == null;
+          datOut = getDataOut();
+          if (size == -1) {
+            datOut.writeInt(0);
+          } else {
+            datOut.writeInt(size);
+            pool.writePool(datOut);
+          }
+          if (lastDocID + 1 < docCount) {
+            fill(datOut, docCount);
+          }
         } else {
-          fill(docCount);
+          // merge path - datOut should be initialized
+          assert datOut != null;
+          if (size == -1) {// no data added
+            datOut.writeInt(0);
+          } else {
+            fill(datOut, docCount);
+          }
         }
+        success = true;
       } finally {
-        super.finish(docCount);
+        pool.dropBuffersAndReset();
+        IOUtils.closeSafely(!success, datOut);
       }
     }
-
-    public long ramBytesUsed() {
-      return oneRecord == null ? 0 : oneRecord.length;
-    }
   }
   
   public static class Reader extends BytesReaderBase {



Mime
View raw message