lucene-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From sim...@apache.org
Subject svn commit: r1144196 [2/5] - in /lucene/dev/trunk: ./ dev-tools/idea/lucene/contrib/ lucene/ lucene/contrib/misc/src/java/org/apache/lucene/index/codecs/appending/ lucene/contrib/misc/src/java/org/apache/lucene/store/ lucene/contrib/misc/src/test/org/a...
Date Fri, 08 Jul 2011 08:52:45 GMT
Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentMerger.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentMerger.java?rev=1144196&r1=1144195&r2=1144196&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentMerger.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentMerger.java Fri Jul  8 08:52:40 2011
@@ -33,6 +33,7 @@ import org.apache.lucene.index.codecs.Pe
 import org.apache.lucene.index.codecs.PerDocValues;
 import org.apache.lucene.store.CompoundFileDirectory;
 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.Bits;
@@ -68,8 +69,10 @@ final class SegmentMerger {
   private SegmentWriteState segmentWriteState;
 
   private PayloadProcessorProvider payloadProcessorProvider;
+  
+  private IOContext context;
 
-  SegmentMerger(Directory dir, int termIndexInterval, String name, MergePolicy.OneMerge merge, PayloadProcessorProvider payloadProcessorProvider, FieldInfos fieldInfos) {
+  SegmentMerger(Directory dir, int termIndexInterval, String name, MergePolicy.OneMerge merge, PayloadProcessorProvider payloadProcessorProvider, FieldInfos fieldInfos, IOContext context) {
     this.payloadProcessorProvider = payloadProcessorProvider;
     directory = dir;
     segment = name;
@@ -85,6 +88,7 @@ final class SegmentMerger {
       };
     }
     this.termIndexInterval = termIndexInterval;
+    this.context = context;
   }
 
   public FieldInfos fieldInfos() {
@@ -129,19 +133,19 @@ final class SegmentMerger {
    * deletion files, this SegmentInfo must not reference such files when this
    * method is called, because they are not allowed within a compound file.
    */
-  final Collection<String> createCompoundFile(String fileName, final SegmentInfo info)
+  final Collection<String> createCompoundFile(String fileName, final SegmentInfo info, IOContext context)
           throws IOException {
 
     // Now merge all added files
     Collection<String> files = info.files();
-    CompoundFileDirectory cfsDir = directory.createCompoundOutput(fileName);
+    CompoundFileDirectory cfsDir = directory.createCompoundOutput(fileName, context);
     try {
       for (String file : files) {
         assert !IndexFileNames.matchesExtension(file, IndexFileNames.DELETES_EXTENSION) 
                   : ".del file is not allowed in .cfs: " + file;
         assert !IndexFileNames.isSeparateNormsFile(file) 
                   : "separate norms file (.s[0-9]+) is not allowed in .cfs: " + file;
-        directory.copy(cfsDir, file, file);
+        directory.copy(cfsDir, file, file, context);
         checkAbort.work(directory.fileLength(file));
       }
     } finally {
@@ -236,9 +240,7 @@ final class SegmentMerger {
     int docCount = 0;
 
     setMatchingSegmentReaders();
-
-    final FieldsWriter fieldsWriter = new FieldsWriter(directory, segment);
-
+    final FieldsWriter fieldsWriter = new FieldsWriter(directory, segment, context);
     try {
       int idx = 0;
       for (IndexReader reader : readers) {
@@ -272,8 +274,7 @@ final class SegmentMerger {
       // entering the index.  See LUCENE-1282 for
       // details.
       throw new RuntimeException("mergeFields produced an invalid result: docCount is " + docCount + " but fdx file size is " + fdxFileLength + " file=" + fileName + " file exists?=" + directory.fileExists(fileName) + "; now aborting this merge to prevent index corruption");
-
-    segmentWriteState = new SegmentWriteState(null, directory, segment, fieldInfos, docCount, termIndexInterval, codecInfo, null);
+    segmentWriteState = new SegmentWriteState(null, directory, segment, fieldInfos, docCount, termIndexInterval, codecInfo, null, context);
 
     return docCount;
   }
@@ -360,7 +361,7 @@ final class SegmentMerger {
    */
   private final void mergeVectors() throws IOException {
     TermVectorsWriter termVectorsWriter =
-      new TermVectorsWriter(directory, segment, fieldInfos);
+      new TermVectorsWriter(directory, segment, fieldInfos, context);
 
     try {
       int idx = 0;
@@ -629,7 +630,7 @@ final class SegmentMerger {
       for (FieldInfo fi : fieldInfos) {
         if (fi.isIndexed && !fi.omitNorms) {
           if (output == null) {
-            output = directory.createOutput(IndexFileNames.segmentFileName(segment, "", IndexFileNames.NORMS_EXTENSION));
+            output = directory.createOutput(IndexFileNames.segmentFileName(segment, "", IndexFileNames.NORMS_EXTENSION), context);
             output.writeBytes(SegmentNorms.NORMS_HEADER, SegmentNorms.NORMS_HEADER.length);
           }
           for (IndexReader reader : readers) {

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentNorms.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentNorms.java?rev=1144196&r1=1144195&r2=1144196&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentNorms.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentNorms.java Fri Jul  8 08:52:40 2011
@@ -20,9 +20,10 @@ package org.apache.lucene.index;
 import java.io.IOException;
 import java.util.concurrent.atomic.AtomicInteger;
 
+import org.apache.lucene.store.FlushInfo;
+import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.store.IndexOutput;
-
 /**
  * Byte[] referencing is used because a new norm object needs 
  * to be created for each clone, and the byte array is all 
@@ -219,7 +220,7 @@ final class SegmentNorms implements Clon
     // NOTE: norms are re-written in regular directory, not cfs
     si.advanceNormGen(this.number);
     final String normFileName = si.getNormFileName(this.number);
-    IndexOutput out = owner.directory().createOutput(normFileName);
+    IndexOutput out = owner.directory().createOutput(normFileName, new IOContext(new FlushInfo(si.docCount, 0)));
     boolean success = false;
     try {
       try {

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentReadState.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentReadState.java?rev=1144196&r1=1144195&r2=1144196&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentReadState.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentReadState.java Fri Jul  8 08:52:40 2011
@@ -18,6 +18,7 @@ package org.apache.lucene.index;
  */
 
 import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
 
 /**
  * @lucene.experimental
@@ -26,7 +27,7 @@ public class SegmentReadState {
   public final Directory dir;
   public final SegmentInfo segmentInfo;
   public final FieldInfos fieldInfos;
-  public final int readBufferSize;
+  public final IOContext context;
 
   // NOTE: if this is < 0, that means "defer terms index
   // load until needed".  But if the codec must load the
@@ -37,20 +38,20 @@ public class SegmentReadState {
   public final int codecId;
 
   public SegmentReadState(Directory dir, SegmentInfo info,
-      FieldInfos fieldInfos, int readBufferSize, int termsIndexDivisor) {
-    this(dir, info, fieldInfos, readBufferSize, termsIndexDivisor, -1);
+      FieldInfos fieldInfos, IOContext context, int termsIndexDivisor) {
+    this(dir, info, fieldInfos,  context, termsIndexDivisor, -1);
   }
   
   public SegmentReadState(Directory dir,
                           SegmentInfo info,
                           FieldInfos fieldInfos,
-                          int readBufferSize,
+                          IOContext context,
                           int termsIndexDivisor,
                           int codecId) {
     this.dir = dir;
     this.segmentInfo = info;
     this.fieldInfos = fieldInfos;
-    this.readBufferSize = readBufferSize;
+    this.context = context;
     this.termsIndexDivisor = termsIndexDivisor;
     this.codecId = codecId;
   }

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentReader.java?rev=1144196&r1=1144195&r2=1144196&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentReader.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentReader.java Fri Jul  8 08:52:40 2011
@@ -32,6 +32,7 @@ import org.apache.lucene.document.FieldS
 import org.apache.lucene.index.codecs.PerDocValues;
 import org.apache.lucene.store.BufferedIndexInput;
 import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.util.BitVector;
 import org.apache.lucene.util.Bits;
@@ -46,7 +47,6 @@ public class SegmentReader extends Index
   protected boolean readOnly;
 
   private SegmentInfo si;
-  private int readBufferSize;
   private final ReaderContext readerContext = new AtomicReaderContext(this);
   CloseableThreadLocal<FieldsReader> fieldsReaderLocal = new FieldsReaderLocal();
   CloseableThreadLocal<TermVectorsReader> termVectorsLocal = new CloseableThreadLocal<TermVectorsReader>();
@@ -88,8 +88,9 @@ public class SegmentReader extends Index
    * @throws CorruptIndexException if the index is corrupt
    * @throws IOException if there is a low-level IO error
    */
-  public static SegmentReader get(boolean readOnly, SegmentInfo si, int termInfosIndexDivisor) throws CorruptIndexException, IOException {
-    return get(readOnly, si.dir, si, BufferedIndexInput.BUFFER_SIZE, true, termInfosIndexDivisor);
+  public static SegmentReader get(boolean readOnly, SegmentInfo si, int termInfosIndexDivisor, IOContext context) throws CorruptIndexException, IOException {
+    // TODO should we check if readOnly and context combination makes sense like asserting that if read only we don't get a default?
+    return get(readOnly, si.dir, si, true, termInfosIndexDivisor, context);
   }
 
   /**
@@ -99,25 +100,23 @@ public class SegmentReader extends Index
   public static SegmentReader get(boolean readOnly,
                                   Directory dir,
                                   SegmentInfo si,
-                                  int readBufferSize,
                                   boolean doOpenStores,
-                                  int termInfosIndexDivisor)
+                                  int termInfosIndexDivisor,
+                                  IOContext context)
     throws CorruptIndexException, IOException {
     
     SegmentReader instance = new SegmentReader();
     instance.readOnly = readOnly;
     instance.si = si;
-    instance.readBufferSize = readBufferSize;
-
     boolean success = false;
 
     try {
-      instance.core = new SegmentCoreReaders(instance, dir, si, readBufferSize, termInfosIndexDivisor);
+      instance.core = new SegmentCoreReaders(instance, dir, si, context, termInfosIndexDivisor);
       if (doOpenStores) {
         instance.core.openDocStores(si);
       }
       instance.loadLiveDocs();
-      instance.openNorms(instance.core.cfsDir, readBufferSize);
+      instance.openNorms(instance.core.cfsDir, context);
       success = true;
     } finally {
 
@@ -161,7 +160,7 @@ public class SegmentReader extends Index
   private void loadLiveDocs() throws IOException {
     // NOTE: the bitvector is stored using the regular directory, not cfs
     if (hasDeletions(si)) {
-      liveDocs = new BitVector(directory(), si.getDelFileName());
+      liveDocs = new BitVector(directory(), si.getDelFileName(), IOContext.DEFAULT);
       if (liveDocs.getVersion() < BitVector.VERSION_DGAPS_CLEARED) {
         liveDocs.invertAll();
       }
@@ -253,7 +252,6 @@ public class SegmentReader extends Index
       clone.core = core;
       clone.readOnly = openReadOnly;
       clone.si = si;
-      clone.readBufferSize = readBufferSize;
       clone.pendingDeleteCount = pendingDeleteCount;
       clone.readerFinishedListeners = readerFinishedListeners;
 
@@ -298,7 +296,7 @@ public class SegmentReader extends Index
 
       // If we are not cloning, then this will open anew
       // any norms that have changed:
-      clone.openNorms(si.getUseCompoundFile() ? core.getCFSReader() : directory(), readBufferSize);
+      clone.openNorms(si.getUseCompoundFile() ? core.getCFSReader() : directory(), IOContext.DEFAULT);
 
       success = true;
     } finally {
@@ -340,7 +338,7 @@ public class SegmentReader extends Index
       final String delFileName = si.getDelFileName();
       boolean success = false;
       try {
-        liveDocs.write(directory(), delFileName);
+        liveDocs.write(directory(), delFileName, IOContext.DEFAULT);
         success = true;
       } finally {
         if (!success) {
@@ -580,7 +578,7 @@ public class SegmentReader extends Index
     norm.copyOnWrite()[doc] = value;                    // set the value
   }
 
-  private void openNorms(Directory cfsDir, int readBufferSize) throws IOException {
+  private void openNorms(Directory cfsDir, IOContext context) throws IOException {
     long nextNormSeek = SegmentNorms.NORMS_HEADER.length; //skip header (header unused for now)
     int maxDoc = maxDoc();
     for (FieldInfo fi : core.fieldInfos) {
@@ -604,7 +602,7 @@ public class SegmentReader extends Index
         if (singleNormFile) {
           normSeek = nextNormSeek;
           if (singleNormStream == null) {
-            singleNormStream = d.openInput(fileName, readBufferSize);
+            singleNormStream = d.openInput(fileName, context);
             singleNormRef = new AtomicInteger(1);
           } else {
             singleNormRef.incrementAndGet();
@@ -614,7 +612,7 @@ public class SegmentReader extends Index
           // If this were to change in the future, a clone could be done here.
           normInput = singleNormStream;
         } else {
-          normInput = d.openInput(fileName);
+          normInput = d.openInput(fileName, context);
           // if the segment was created in 3.2 or after, we wrote the header for sure,
           // and don't need to do the sketchy file size check. otherwise, we check 
           // if the size is exactly equal to maxDoc to detect a headerless file.

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentWriteState.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentWriteState.java?rev=1144196&r1=1144195&r2=1144196&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentWriteState.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentWriteState.java Fri Jul  8 08:52:40 2011
@@ -20,6 +20,7 @@ package org.apache.lucene.index;
 import java.io.PrintStream;
 
 import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
 import org.apache.lucene.util.BitVector;
 
 /**
@@ -51,9 +52,11 @@ public class SegmentWriteState {
    * slower.  Searching is typically not dominated by dictionary lookup, so
    * tweaking this is rarely useful.*/
   public int termIndexInterval;                   // TODO: this should be private to the codec, not settable here or in IWC
+  
+  public final IOContext context;
 
   public SegmentWriteState(PrintStream infoStream, Directory directory, String segmentName, FieldInfos fieldInfos,
-      int numDocs, int termIndexInterval, SegmentCodecs segmentCodecs, BufferedDeletes segDeletes) {
+      int numDocs, int termIndexInterval, SegmentCodecs segmentCodecs, BufferedDeletes segDeletes, IOContext context) {
     this.infoStream = infoStream;
     this.segDeletes = segDeletes;
     this.directory = directory;
@@ -63,6 +66,7 @@ public class SegmentWriteState {
     this.termIndexInterval = termIndexInterval;
     this.segmentCodecs = segmentCodecs;
     codecId = -1;
+    this.context = context;
   }
   
   /**
@@ -76,6 +80,7 @@ public class SegmentWriteState {
     numDocs = state.numDocs;
     termIndexInterval = state.termIndexInterval;
     segmentCodecs = state.segmentCodecs;
+    context = state.context;
     this.codecId = codecId;
     segDeletes = state.segDeletes;
   }

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/StoredFieldsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/StoredFieldsWriter.java?rev=1144196&r1=1144195&r2=1144196&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/StoredFieldsWriter.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/StoredFieldsWriter.java Fri Jul  8 08:52:40 2011
@@ -20,6 +20,7 @@ package org.apache.lucene.index;
 import java.io.IOException;
 
 import org.apache.lucene.document.Fieldable;
+import org.apache.lucene.store.IOContext;
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.RamUsageEstimator;
 
@@ -59,7 +60,7 @@ final class StoredFieldsWriter {
       // 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 FieldsWriter:
-      initFieldsWriter();
+      initFieldsWriter(state.context);
       fill(state.numDocs);
     }
 
@@ -75,9 +76,9 @@ final class StoredFieldsWriter {
     }
   }
 
-  private synchronized void initFieldsWriter() throws IOException {
+  private synchronized void initFieldsWriter(IOContext context) throws IOException {
     if (fieldsWriter == null) {
-      fieldsWriter = new FieldsWriter(docWriter.directory, docWriter.getSegment());
+      fieldsWriter = new FieldsWriter(docWriter.directory, docWriter.getSegment(), context);
       lastDocID = 0;
     }
   }
@@ -107,7 +108,7 @@ final class StoredFieldsWriter {
   void finishDocument() throws IOException {
     assert docWriter.writer.testPoint("StoredFieldsWriter.finishDocument start");
 
-    initFieldsWriter();
+    initFieldsWriter(IOContext.DEFAULT);
     fill(docState.docID);
 
     if (fieldsWriter != null && numStoredFields > 0) {

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/TermVectorsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/TermVectorsReader.java?rev=1144196&r1=1144195&r2=1144196&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/TermVectorsReader.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/TermVectorsReader.java Fri Jul  8 08:52:40 2011
@@ -17,9 +17,12 @@ package org.apache.lucene.index;
  * limitations under the License.
  */
 
+import org.apache.lucene.index.MergePolicy.OneMerge;
 import org.apache.lucene.store.BufferedIndexInput;
 import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.store.IOContext.Context;
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.IOUtils;
@@ -63,29 +66,24 @@ class TermVectorsReader implements Clone
   
   private final int format;
 
-  TermVectorsReader(Directory d, String segment, FieldInfos fieldInfos)
+  TermVectorsReader(Directory d, String segment, FieldInfos fieldInfos, IOContext context)
     throws CorruptIndexException, IOException {
-    this(d, segment, fieldInfos, BufferedIndexInput.BUFFER_SIZE);
-  }
-
-  TermVectorsReader(Directory d, String segment, FieldInfos fieldInfos, int readBufferSize)
-    throws CorruptIndexException, IOException {
-    this(d, segment, fieldInfos, readBufferSize, -1, 0);
+    this(d, segment, fieldInfos, context, -1, 0);
   }
     
-  TermVectorsReader(Directory d, String segment, FieldInfos fieldInfos, int readBufferSize, int docStoreOffset, int size)
+  TermVectorsReader(Directory d, String segment, FieldInfos fieldInfos, IOContext context, int docStoreOffset, int size)
     throws CorruptIndexException, IOException {
     boolean success = false;
 
     try {
       String idxName = IndexFileNames.segmentFileName(segment, "", IndexFileNames.VECTORS_INDEX_EXTENSION);
-      tvx = d.openInput(idxName, readBufferSize);
+      tvx = d.openInput(idxName, context);
       format = checkValidFormat(tvx, idxName);
       String fn = IndexFileNames.segmentFileName(segment, "", IndexFileNames.VECTORS_DOCUMENTS_EXTENSION);
-      tvd = d.openInput(fn, readBufferSize);
+      tvd = d.openInput(fn, context);
       final int tvdFormat = checkValidFormat(tvd, fn);
       fn = IndexFileNames.segmentFileName(segment, "", IndexFileNames.VECTORS_FIELDS_EXTENSION);
-      tvf = d.openInput(fn, readBufferSize);
+      tvf = d.openInput(fn, context);
       final int tvfFormat = checkValidFormat(tvf, fn);
 
       assert format == tvdFormat;

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/TermVectorsTermsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/TermVectorsTermsWriter.java?rev=1144196&r1=1144195&r2=1144196&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/TermVectorsTermsWriter.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/TermVectorsTermsWriter.java Fri Jul  8 08:52:40 2011
@@ -20,7 +20,10 @@ package org.apache.lucene.index;
 import java.io.IOException;
 import java.util.Map;
 
+import org.apache.lucene.store.FlushInfo;
+import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.store.IOContext.Context;
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.IOUtils;
@@ -89,13 +92,14 @@ final class TermVectorsTermsWriter exten
     if (tvx == null) {
       boolean success = false;
       try {
+        IOContext context = new IOContext(new FlushInfo(docWriter.getNumDocsInRAM(), docWriter.bytesUsed()));
         // If we hit an exception while init'ing the term
         // vector output files, we must abort this segment
         // because those files will be in an unknown
         // state:
-        tvx = docWriter.directory.createOutput(IndexFileNames.segmentFileName(docWriter.getSegment(), "", IndexFileNames.VECTORS_INDEX_EXTENSION));
-        tvd = docWriter.directory.createOutput(IndexFileNames.segmentFileName(docWriter.getSegment(), "", IndexFileNames.VECTORS_DOCUMENTS_EXTENSION));
-        tvf = docWriter.directory.createOutput(IndexFileNames.segmentFileName(docWriter.getSegment(), "", IndexFileNames.VECTORS_FIELDS_EXTENSION));
+        tvx = docWriter.directory.createOutput(IndexFileNames.segmentFileName(docWriter.getSegment(), "", IndexFileNames.VECTORS_INDEX_EXTENSION), context);
+        tvd = docWriter.directory.createOutput(IndexFileNames.segmentFileName(docWriter.getSegment(), "", IndexFileNames.VECTORS_DOCUMENTS_EXTENSION), context);
+        tvf = docWriter.directory.createOutput(IndexFileNames.segmentFileName(docWriter.getSegment(), "", IndexFileNames.VECTORS_FIELDS_EXTENSION), context);
 
         tvx.writeInt(TermVectorsReader.FORMAT_CURRENT);
         tvd.writeInt(TermVectorsReader.FORMAT_CURRENT);

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/TermVectorsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/TermVectorsWriter.java?rev=1144196&r1=1144195&r2=1144196&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/TermVectorsWriter.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/TermVectorsWriter.java Fri Jul  8 08:52:40 2011
@@ -18,6 +18,7 @@ package org.apache.lucene.index;
  */
 
 import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexOutput;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.IOUtils;
@@ -31,15 +32,15 @@ final class TermVectorsWriter {
   private FieldInfos fieldInfos;
 
   public TermVectorsWriter(Directory directory, String segment,
-                           FieldInfos fieldInfos) throws IOException {
+                           FieldInfos fieldInfos, IOContext context) throws IOException {
     boolean success = false;
     try {
       // Open files for TermVector storage
-      tvx = directory.createOutput(IndexFileNames.segmentFileName(segment, "", IndexFileNames.VECTORS_INDEX_EXTENSION));
+      tvx = directory.createOutput(IndexFileNames.segmentFileName(segment, "", IndexFileNames.VECTORS_INDEX_EXTENSION), context);
       tvx.writeInt(TermVectorsReader.FORMAT_CURRENT);
-      tvd = directory.createOutput(IndexFileNames.segmentFileName(segment, "", IndexFileNames.VECTORS_DOCUMENTS_EXTENSION));
+      tvd = directory.createOutput(IndexFileNames.segmentFileName(segment, "", IndexFileNames.VECTORS_DOCUMENTS_EXTENSION), context);
       tvd.writeInt(TermVectorsReader.FORMAT_CURRENT);
-      tvf = directory.createOutput(IndexFileNames.segmentFileName(segment, "", IndexFileNames.VECTORS_FIELDS_EXTENSION));
+      tvf = directory.createOutput(IndexFileNames.segmentFileName(segment, "", IndexFileNames.VECTORS_FIELDS_EXTENSION), context);
       tvf.writeInt(TermVectorsReader.FORMAT_CURRENT);
       success = true;
     } finally {

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/BlockTermsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/BlockTermsReader.java?rev=1144196&r1=1144195&r2=1144196&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/BlockTermsReader.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/BlockTermsReader.java Fri Jul  8 08:52:40 2011
@@ -37,6 +37,7 @@ import org.apache.lucene.index.TermsEnum
 import org.apache.lucene.index.codecs.standard.StandardPostingsReader; // javadocs
 import org.apache.lucene.store.ByteArrayDataInput;
 import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.Bits;
@@ -107,7 +108,7 @@ public class BlockTermsReader extends Fi
   
   //private String segment;
   
-  public BlockTermsReader(TermsIndexReaderBase indexReader, Directory dir, FieldInfos fieldInfos, String segment, PostingsReaderBase postingsReader, int readBufferSize,
+  public BlockTermsReader(TermsIndexReaderBase indexReader, Directory dir, FieldInfos fieldInfos, String segment, PostingsReaderBase postingsReader, IOContext context,
                           int termsCacheSize, int codecId)
     throws IOException {
     
@@ -116,7 +117,7 @@ public class BlockTermsReader extends Fi
 
     //this.segment = segment;
     in = dir.openInput(IndexFileNames.segmentFileName(segment, codecId, BlockTermsWriter.TERMS_EXTENSION),
-                       readBufferSize);
+                       context);
 
     boolean success = false;
     try {

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/BlockTermsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/BlockTermsWriter.java?rev=1144196&r1=1144195&r2=1144196&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/BlockTermsWriter.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/BlockTermsWriter.java Fri Jul  8 08:52:40 2011
@@ -72,7 +72,7 @@ public class BlockTermsWriter extends Fi
       throws IOException {
     final String termsFileName = IndexFileNames.segmentFileName(state.segmentName, state.codecId, TERMS_EXTENSION);
     this.termsIndexWriter = termsIndexWriter;
-    out = state.directory.createOutput(termsFileName);
+    out = state.directory.createOutput(termsFileName, state.context);
     boolean success = false;
     try {
       fieldInfos = state.fieldInfos;

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/DefaultDocValuesConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/DefaultDocValuesConsumer.java?rev=1144196&r1=1144195&r2=1144196&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/DefaultDocValuesConsumer.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/DefaultDocValuesConsumer.java Fri Jul  8 08:52:40 2011
@@ -29,6 +29,7 @@ import org.apache.lucene.index.PerDocWri
 import org.apache.lucene.index.SegmentInfo;
 import org.apache.lucene.index.values.Writer;
 import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
 import org.apache.lucene.util.BytesRef;
 
 /**
@@ -42,13 +43,17 @@ public class DefaultDocValuesConsumer ex
   private final AtomicLong bytesUsed;
   private final Comparator<BytesRef> comparator;
   private boolean useCompoundFile;
+  private final IOContext context;
   
   public DefaultDocValuesConsumer(PerDocWriteState state, Comparator<BytesRef> comparator, boolean useCompoundFile) throws IOException {
     this.segmentName = state.segmentName;
     this.codecId = state.codecId;
     this.bytesUsed = state.bytesUsed;
+    this.context = state.context;
     //TODO maybe we should enable a global CFS that all codecs can pull on demand to further reduce the number of files?
-    this.directory = useCompoundFile ? state.directory.createCompoundOutput(IndexFileNames.segmentFileName(segmentName, state.codecId, IndexFileNames.COMPOUND_FILE_EXTENSION)) : state.directory;
+    this.directory = useCompoundFile ? state.directory.createCompoundOutput(
+        IndexFileNames.segmentFileName(segmentName, codecId,
+            IndexFileNames.COMPOUND_FILE_EXTENSION), context) : state.directory;
     this.comparator = comparator;
     this.useCompoundFile = useCompoundFile;
   }
@@ -63,7 +68,7 @@ public class DefaultDocValuesConsumer ex
   public DocValuesConsumer addValuesField(FieldInfo field) throws IOException {
     return Writer.create(field.getDocValues(),
         docValuesId(segmentName, codecId, field.number),
-        directory, comparator, bytesUsed);
+        directory, comparator, bytesUsed, context);
   }
   
   @SuppressWarnings("fallthrough")

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/DefaultDocValuesProducer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/DefaultDocValuesProducer.java?rev=1144196&r1=1144195&r2=1144196&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/DefaultDocValuesProducer.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/DefaultDocValuesProducer.java Fri Jul  8 08:52:40 2011
@@ -33,6 +33,7 @@ import org.apache.lucene.index.values.Fl
 import org.apache.lucene.index.values.Ints;
 import org.apache.lucene.index.values.ValueType;
 import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.IOUtils;
 
@@ -72,17 +73,17 @@ public class DefaultDocValuesProducer ex
    *           if an {@link IOException} occurs
    */
   public DefaultDocValuesProducer(SegmentInfo si, Directory dir, 
-      FieldInfos fieldInfo, int codecId, boolean useCompoundFile, Comparator<BytesRef> sortComparator) throws IOException {
+      FieldInfos fieldInfo, int codecId, boolean useCompoundFile, Comparator<BytesRef> sortComparator, IOContext context) throws IOException {
     this.useCompoundFile = useCompoundFile;
     this.sortComparator = sortComparator;
     final Directory directory;
     if (useCompoundFile) {
-      cfs = directory = dir.openCompoundInput(IndexFileNames.segmentFileName(si.name, codecId, IndexFileNames.COMPOUND_FILE_EXTENSION), 1024);
+      cfs = directory = dir.openCompoundInput(IndexFileNames.segmentFileName(si.name, codecId, IndexFileNames.COMPOUND_FILE_EXTENSION), context);
     } else {
       cfs = null;
       directory = dir;
     }
-    docValues = load(fieldInfo, si.name, si.docCount, directory, codecId);
+    docValues = load(fieldInfo, si.name, si.docCount, directory, codecId, context);
   }
 
   /**
@@ -96,7 +97,7 @@ public class DefaultDocValuesProducer ex
 
   // Only opens files... doesn't actually load any values
   protected TreeMap<String, IndexDocValues> load(FieldInfos fieldInfos,
-      String segment, int docCount, Directory dir, int codecId)
+      String segment, int docCount, Directory dir, int codecId, IOContext context)
       throws IOException {
     TreeMap<String, IndexDocValues> values = new TreeMap<String, IndexDocValues>();
     boolean success = false;
@@ -110,7 +111,7 @@ public class DefaultDocValuesProducer ex
           final String id = DefaultDocValuesConsumer.docValuesId(segment,
               codecId, fieldInfo.number);
           values.put(field,
-              loadDocValues(docCount, dir, id, fieldInfo.getDocValues(), sortComparator));
+              loadDocValues(docCount, dir, id, fieldInfo.getDocValues(), sortComparator, context));
         }
       }
       success = true;
@@ -145,30 +146,30 @@ public class DefaultDocValuesProducer ex
    *           if the given {@link ValueType} is not supported
    */
   protected IndexDocValues loadDocValues(int docCount, Directory dir, String id,
-      ValueType type, Comparator<BytesRef> sortComparator) throws IOException {
+      ValueType type, Comparator<BytesRef> sortComparator, IOContext context) throws IOException {
     switch (type) {
     case FIXED_INTS_16:
     case FIXED_INTS_32:
     case FIXED_INTS_64:
     case FIXED_INTS_8:
     case VAR_INTS:
-      return Ints.getValues(dir, id, docCount);
+      return Ints.getValues(dir, id, docCount, context);
     case FLOAT_32:
-      return Floats.getValues(dir, id, docCount);
+      return Floats.getValues(dir, id, docCount, context);
     case FLOAT_64:
-      return Floats.getValues(dir, id, docCount);
+      return Floats.getValues(dir, id, docCount, context);
     case BYTES_FIXED_STRAIGHT:
-      return Bytes.getValues(dir, id, Bytes.Mode.STRAIGHT, true, docCount, sortComparator);
+      return Bytes.getValues(dir, id, Bytes.Mode.STRAIGHT, true, docCount, sortComparator, context);
     case BYTES_FIXED_DEREF:
-      return Bytes.getValues(dir, id, Bytes.Mode.DEREF, true, docCount, sortComparator);
+      return Bytes.getValues(dir, id, Bytes.Mode.DEREF, true, docCount, sortComparator, context);
     case BYTES_FIXED_SORTED:
-      return Bytes.getValues(dir, id, Bytes.Mode.SORTED, true, docCount, sortComparator);
+      return Bytes.getValues(dir, id, Bytes.Mode.SORTED, true, docCount, sortComparator, context);
     case BYTES_VAR_STRAIGHT:
-      return Bytes.getValues(dir, id, Bytes.Mode.STRAIGHT, false, docCount, sortComparator);
+      return Bytes.getValues(dir, id, Bytes.Mode.STRAIGHT, false, docCount, sortComparator, context);
     case BYTES_VAR_DEREF:
-      return Bytes.getValues(dir, id, Bytes.Mode.DEREF, false, docCount, sortComparator);
+      return Bytes.getValues(dir, id, Bytes.Mode.DEREF, false, docCount, sortComparator, context);
     case BYTES_VAR_SORTED:
-      return Bytes.getValues(dir, id, Bytes.Mode.SORTED, false, docCount, sortComparator);
+      return Bytes.getValues(dir, id, Bytes.Mode.SORTED, false, docCount, sortComparator, context);
     default:
       throw new IllegalStateException("unrecognized index values mode " + type);
     }

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/DefaultSegmentInfosReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/DefaultSegmentInfosReader.java?rev=1144196&r1=1144195&r2=1144196&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/DefaultSegmentInfosReader.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/DefaultSegmentInfosReader.java Fri Jul  8 08:52:40 2011
@@ -28,6 +28,7 @@ import org.apache.lucene.index.SegmentIn
 import org.apache.lucene.index.SegmentInfos;
 import org.apache.lucene.store.ChecksumIndexInput;
 import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexInput;
 
 /**
@@ -38,10 +39,10 @@ public class DefaultSegmentInfosReader e
 
   @Override
   public void read(Directory directory, String segmentsFileName, CodecProvider codecs,
-          SegmentInfos infos) throws IOException {
+          SegmentInfos infos, IOContext context) throws IOException {
     IndexInput input = null;
     try {
-      input = openInput(directory, segmentsFileName);
+      input = openInput(directory, segmentsFileName, context);
       final int format = input.readInt();
       infos.setFormat(format);
   
@@ -69,11 +70,11 @@ public class DefaultSegmentInfosReader e
             if (si.getDocStoreIsCompoundFile()) {
               dir = dir.openCompoundInput(IndexFileNames.segmentFileName(
                   si.getDocStoreSegment(), "",
-                  IndexFileNames.COMPOUND_FILE_STORE_EXTENSION), 1024);
+                  IndexFileNames.COMPOUND_FILE_STORE_EXTENSION), context);
             }
           } else if (si.getUseCompoundFile()) {
             dir = dir.openCompoundInput(IndexFileNames.segmentFileName(
-                si.name, "", IndexFileNames.COMPOUND_FILE_EXTENSION), 1024);
+                si.name, "", IndexFileNames.COMPOUND_FILE_EXTENSION), context);
           }
 
           try {
@@ -107,8 +108,8 @@ public class DefaultSegmentInfosReader e
 
   }
   
-  public IndexInput openInput(Directory dir, String segmentsFileName) throws IOException {
-    IndexInput in = dir.openInput(segmentsFileName);
+  public IndexInput openInput(Directory dir, String segmentsFileName, IOContext context) throws IOException {
+    IndexInput in = dir.openInput(segmentsFileName, context);
     return new ChecksumIndexInput(in);
     
   }

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/DefaultSegmentInfosWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/DefaultSegmentInfosWriter.java?rev=1144196&r1=1144195&r2=1144196&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/DefaultSegmentInfosWriter.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/DefaultSegmentInfosWriter.java Fri Jul  8 08:52:40 2011
@@ -23,6 +23,8 @@ import org.apache.lucene.index.SegmentIn
 import org.apache.lucene.index.SegmentInfos;
 import org.apache.lucene.store.ChecksumIndexOutput;
 import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.FlushInfo;
+import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexOutput;
 import org.apache.lucene.util.IOUtils;
 
@@ -54,9 +56,9 @@ public class DefaultSegmentInfosWriter e
   public static final int FORMAT_MINIMUM = FORMAT_DIAGNOSTICS;
 
   @Override
-  public IndexOutput writeInfos(Directory dir, String segmentFileName, SegmentInfos infos)
+  public IndexOutput writeInfos(Directory dir, String segmentFileName, SegmentInfos infos, IOContext context)
           throws IOException {
-    IndexOutput out = createOutput(dir, segmentFileName);
+    IndexOutput out = createOutput(dir, segmentFileName, new IOContext(new FlushInfo(infos.size(), infos.totalDocCount())));
     boolean success = false;
     try {
       out.writeInt(FORMAT_CURRENT); // write FORMAT
@@ -77,9 +79,9 @@ public class DefaultSegmentInfosWriter e
     }
   }
   
-  protected IndexOutput createOutput(Directory dir, String segmentFileName)
+  protected IndexOutput createOutput(Directory dir, String segmentFileName, IOContext context)
       throws IOException {
-    IndexOutput plainOut = dir.createOutput(segmentFileName);
+    IndexOutput plainOut = dir.createOutput(segmentFileName, context);
     ChecksumIndexOutput out = new ChecksumIndexOutput(plainOut);
     return out;
   }

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/FixedGapTermsIndexReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/FixedGapTermsIndexReader.java?rev=1144196&r1=1144195&r2=1144196&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/FixedGapTermsIndexReader.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/FixedGapTermsIndexReader.java Fri Jul  8 08:52:40 2011
@@ -18,7 +18,9 @@ package org.apache.lucene.index.codecs;
  */
 
 import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.store.IOContext.Context;
 import org.apache.lucene.index.FieldInfos;
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.SegmentInfo;
@@ -68,12 +70,12 @@ public class FixedGapTermsIndexReader ex
   // start of the field info data
   protected long dirOffset;
 
-  public FixedGapTermsIndexReader(Directory dir, FieldInfos fieldInfos, String segment, int indexDivisor, Comparator<BytesRef> termComp, int codecId)
+  public FixedGapTermsIndexReader(Directory dir, FieldInfos fieldInfos, String segment, int indexDivisor, Comparator<BytesRef> termComp, int codecId, IOContext context)
     throws IOException {
 
     this.termComp = termComp;
 
-    in = dir.openInput(IndexFileNames.segmentFileName(segment, codecId, FixedGapTermsIndexWriter.TERMS_INDEX_EXTENSION));
+    in = dir.openInput(IndexFileNames.segmentFileName(segment, codecId, FixedGapTermsIndexWriter.TERMS_INDEX_EXTENSION), context);
     
     boolean success = false;
 

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/FixedGapTermsIndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/FixedGapTermsIndexWriter.java?rev=1144196&r1=1144195&r2=1144196&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/FixedGapTermsIndexWriter.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/FixedGapTermsIndexWriter.java Fri Jul  8 08:52:40 2011
@@ -58,7 +58,7 @@ public class FixedGapTermsIndexWriter ex
   public FixedGapTermsIndexWriter(SegmentWriteState state) throws IOException {
     final String indexFileName = IndexFileNames.segmentFileName(state.segmentName, state.codecId, TERMS_INDEX_EXTENSION);
     termIndexInterval = state.termIndexInterval;
-    out = state.directory.createOutput(indexFileName);
+    out = state.directory.createOutput(indexFileName, state.context);
     boolean success = false;
     try {
       fieldInfos = state.fieldInfos;

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/SegmentInfosReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/SegmentInfosReader.java?rev=1144196&r1=1144195&r2=1144196&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/SegmentInfosReader.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/SegmentInfosReader.java Fri Jul  8 08:52:40 2011
@@ -21,6 +21,7 @@ import java.io.IOException;
 
 import org.apache.lucene.index.SegmentInfos;
 import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
 
 /**
  * Specifies an API for classes that can read {@link SegmentInfos} information.
@@ -36,5 +37,5 @@ public abstract class SegmentInfosReader
    * @param infos empty instance to be populated with data
    * @throws IOException
    */
-  public abstract void read(Directory directory, String segmentsFileName, CodecProvider codecs, SegmentInfos infos) throws IOException;
+  public abstract void read(Directory directory, String segmentsFileName, CodecProvider codecs, SegmentInfos infos, IOContext context) throws IOException;
 }

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/SegmentInfosWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/SegmentInfosWriter.java?rev=1144196&r1=1144195&r2=1144196&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/SegmentInfosWriter.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/SegmentInfosWriter.java Fri Jul  8 08:52:40 2011
@@ -21,6 +21,7 @@ import java.io.IOException;
 
 import org.apache.lucene.index.SegmentInfos;
 import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexOutput;
 
 /**
@@ -41,7 +42,7 @@ public abstract class SegmentInfosWriter
    * phase commit" operations as described above.
    * @throws IOException
    */
-  public abstract IndexOutput writeInfos(Directory dir, String segmentsFileName, SegmentInfos infos) throws IOException;
+  public abstract IndexOutput writeInfos(Directory dir, String segmentsFileName, SegmentInfos infos, IOContext context) throws IOException;
   
   /**
    * First phase of the two-phase commit - ensure that all output can be

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/VariableGapTermsIndexReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/VariableGapTermsIndexReader.java?rev=1144196&r1=1144195&r2=1144196&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/VariableGapTermsIndexReader.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/VariableGapTermsIndexReader.java Fri Jul  8 08:52:40 2011
@@ -30,7 +30,9 @@ import org.apache.lucene.index.FieldInfo
 import org.apache.lucene.index.IndexFileNames;
 import org.apache.lucene.index.SegmentInfo;
 import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.store.IOContext.Context;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.CodecUtil;
 import org.apache.lucene.util.fst.Builder;
@@ -57,10 +59,10 @@ public class VariableGapTermsIndexReader
   protected long dirOffset;
 
   final String segment;
-  public VariableGapTermsIndexReader(Directory dir, FieldInfos fieldInfos, String segment, int indexDivisor, int codecId)
+  public VariableGapTermsIndexReader(Directory dir, FieldInfos fieldInfos, String segment, int indexDivisor, int codecId, IOContext context)
     throws IOException {
 
-    in = dir.openInput(IndexFileNames.segmentFileName(segment, codecId, VariableGapTermsIndexWriter.TERMS_INDEX_EXTENSION));
+    in = dir.openInput(IndexFileNames.segmentFileName(segment, codecId, VariableGapTermsIndexWriter.TERMS_INDEX_EXTENSION), context);
     this.segment = segment;
     boolean success = false;
 

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/VariableGapTermsIndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/VariableGapTermsIndexWriter.java?rev=1144196&r1=1144195&r2=1144196&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/VariableGapTermsIndexWriter.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/VariableGapTermsIndexWriter.java Fri Jul  8 08:52:40 2011
@@ -159,7 +159,7 @@ public class VariableGapTermsIndexWriter
 
   public VariableGapTermsIndexWriter(SegmentWriteState state, IndexTermSelector policy) throws IOException {
     final String indexFileName = IndexFileNames.segmentFileName(state.segmentName, state.codecId, TERMS_INDEX_EXTENSION);
-    out = state.directory.createOutput(indexFileName);
+    out = state.directory.createOutput(indexFileName, state.context);
     boolean success = false;
     try {
       fieldInfos = state.fieldInfos;

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/memory/MemoryCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/memory/MemoryCodec.java?rev=1144196&r1=1144195&r2=1144196&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/memory/MemoryCodec.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/memory/MemoryCodec.java Fri Jul  8 08:52:40 2011
@@ -48,6 +48,7 @@ import org.apache.lucene.index.codecs.Te
 import org.apache.lucene.index.codecs.TermsConsumer;
 import org.apache.lucene.store.ByteArrayDataInput;
 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.store.RAMOutputStream;
@@ -242,7 +243,7 @@ public class MemoryCodec extends Codec {
   public FieldsConsumer fieldsConsumer(SegmentWriteState state) throws IOException {
 
     final String fileName = IndexFileNames.segmentFileName(state.segmentName, state.codecId, EXTENSION);
-    final IndexOutput out = state.directory.createOutput(fileName);
+    final IndexOutput out = state.directory.createOutput(fileName, state.context);
     
     return new FieldsConsumer() {
       @Override
@@ -717,7 +718,7 @@ public class MemoryCodec extends Codec {
   @Override
   public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
     final String fileName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.codecId, EXTENSION);
-    final IndexInput in = state.dir.openInput(fileName);
+    final IndexInput in = state.dir.openInput(fileName, IOContext.READONCE);
 
     final SortedMap<String,TermsReader> fields = new TreeMap<String,TermsReader>();
 
@@ -794,6 +795,6 @@ public class MemoryCodec extends Codec {
 
   @Override
   public PerDocValues docsProducer(SegmentReadState state) throws IOException {
-    return new DefaultDocValuesProducer(state.segmentInfo, state.dir, state.fieldInfos, state.codecId, getDocValuesUseCFS(), getDocValuesSortComparator());
+    return new DefaultDocValuesProducer(state.segmentInfo, state.dir, state.fieldInfos, state.codecId, getDocValuesUseCFS(), getDocValuesSortComparator(), IOContext.READONCE);
   }
 }

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/preflex/PreFlexCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/preflex/PreFlexCodec.java?rev=1144196&r1=1144195&r2=1144196&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/preflex/PreFlexCodec.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/preflex/PreFlexCodec.java Fri Jul  8 08:52:40 2011
@@ -65,7 +65,7 @@ public class PreFlexCodec extends Codec 
 
   @Override
   public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
-    return new PreFlexFields(state.dir, state.fieldInfos, state.segmentInfo, state.readBufferSize, state.termsIndexDivisor);
+    return new PreFlexFields(state.dir, state.fieldInfos, state.segmentInfo, state.context, state.termsIndexDivisor);
   }
 
   @Override

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/preflex/PreFlexFields.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/preflex/PreFlexFields.java?rev=1144196&r1=1144195&r2=1144196&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/preflex/PreFlexFields.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/preflex/PreFlexFields.java Fri Jul  8 08:52:40 2011
@@ -38,6 +38,7 @@ 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;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
@@ -62,10 +63,10 @@ public class PreFlexFields extends Field
   final TreeMap<String,FieldInfo> fields = new TreeMap<String,FieldInfo>();
   final Map<String,Terms> preTerms = new HashMap<String,Terms>();
   private final Directory dir;
-  private final int readBufferSize;
+  private final IOContext context;
   private Directory cfsReader;
 
-  public PreFlexFields(Directory dir, FieldInfos fieldInfos, SegmentInfo info, int readBufferSize, int indexDivisor)
+  public PreFlexFields(Directory dir, FieldInfos fieldInfos, SegmentInfo info, IOContext context, int indexDivisor)
     throws IOException {
 
     si = info;
@@ -80,19 +81,19 @@ public class PreFlexFields extends Field
     
     boolean success = false;
     try {
-      TermInfosReader r = new TermInfosReader(dir, info.name, fieldInfos, readBufferSize, indexDivisor);    
+      TermInfosReader r = new TermInfosReader(dir, info.name, fieldInfos, context, indexDivisor);    
       if (indexDivisor == -1) {
         tisNoIndex = r;
       } else {
         tisNoIndex = null;
         tis = r;
       }
-      this.readBufferSize = readBufferSize;
+      this.context = context;
       this.fieldInfos = fieldInfos;
 
       // make sure that all index files have been read or are kept open
       // so that if an index update removes them we'll still have them
-      freqStream = dir.openInput(IndexFileNames.segmentFileName(info.name, "", PreFlexCodec.FREQ_EXTENSION), readBufferSize);
+      freqStream = dir.openInput(IndexFileNames.segmentFileName(info.name, "", PreFlexCodec.FREQ_EXTENSION), context);
       boolean anyProx = false;
       for (FieldInfo fi : fieldInfos) {
         if (fi.isIndexed) {
@@ -105,7 +106,7 @@ public class PreFlexFields extends Field
       }
 
       if (anyProx) {
-        proxStream = dir.openInput(IndexFileNames.segmentFileName(info.name, "", PreFlexCodec.PROX_EXTENSION), readBufferSize);
+        proxStream = dir.openInput(IndexFileNames.segmentFileName(info.name, "", PreFlexCodec.PROX_EXTENSION), context);
       } else {
         proxStream = null;
       }
@@ -178,7 +179,7 @@ public class PreFlexFields extends Field
         // to CFS
 
         if (!(dir instanceof CompoundFileDirectory)) {
-          dir0 = cfsReader = dir.openCompoundInput(IndexFileNames.segmentFileName(si.name, "", IndexFileNames.COMPOUND_FILE_EXTENSION), readBufferSize);
+          dir0 = cfsReader = dir.openCompoundInput(IndexFileNames.segmentFileName(si.name, "", IndexFileNames.COMPOUND_FILE_EXTENSION), context);
         } else {
           dir0 = dir;
         }
@@ -187,7 +188,7 @@ public class PreFlexFields extends Field
         dir0 = dir;
       }
 
-      tis = new TermInfosReader(dir0, si.name, fieldInfos, readBufferSize, indexDivisor);
+      tis = new TermInfosReader(dir0, si.name, fieldInfos, context, indexDivisor);
     }
   }
 

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/preflex/TermInfosReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/preflex/TermInfosReader.java?rev=1144196&r1=1144195&r2=1144196&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/preflex/TermInfosReader.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/preflex/TermInfosReader.java Fri Jul  8 08:52:40 2011
@@ -25,6 +25,7 @@ import org.apache.lucene.index.FieldInfo
 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;
@@ -96,7 +97,7 @@ public final class TermInfosReader {
     SegmentTermEnum termEnum;
   }
   
-  TermInfosReader(Directory dir, String seg, FieldInfos fis, int readBufferSize, int indexDivisor)
+  TermInfosReader(Directory dir, String seg, FieldInfos fis, IOContext context, int indexDivisor)
        throws CorruptIndexException, IOException {
     boolean success = false;
 
@@ -110,7 +111,7 @@ public final class TermInfosReader {
       fieldInfos = fis;
 
       origEnum = new SegmentTermEnum(directory.openInput(IndexFileNames.segmentFileName(segment, "", PreFlexCodec.TERMS_EXTENSION),
-                                                         readBufferSize), fieldInfos, false);
+                                                         context), fieldInfos, false);
       size = origEnum.size;
 
 
@@ -118,7 +119,7 @@ public final class TermInfosReader {
         // Load terms index
         totalIndexInterval = origEnum.indexInterval * indexDivisor;
         final SegmentTermEnum indexEnum = new SegmentTermEnum(directory.openInput(IndexFileNames.segmentFileName(segment, "", PreFlexCodec.TERMS_INDEX_EXTENSION),
-                                                                                  readBufferSize), fieldInfos, true);
+                                                                                  context), fieldInfos, true);
 
         try {
           int indexSize = 1+((int)indexEnum.size-1)/indexDivisor;  // otherwise read index

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/pulsing/PulsingCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/pulsing/PulsingCodec.java?rev=1144196&r1=1144195&r2=1144196&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/pulsing/PulsingCodec.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/pulsing/PulsingCodec.java Fri Jul  8 08:52:40 2011
@@ -118,7 +118,7 @@ public class PulsingCodec extends Codec 
 
     // We wrap StandardPostingsReader, but any StandardPostingsReader
     // will work:
-    PostingsReaderBase docsReader = new StandardPostingsReader(state.dir, state.segmentInfo, state.readBufferSize, state.codecId);
+    PostingsReaderBase docsReader = new StandardPostingsReader(state.dir, state.segmentInfo, state.context, state.codecId);
     PostingsReaderBase pulsingReader = new PulsingPostingsReaderImpl(docsReader);
 
     // Terms dict index reader
@@ -130,7 +130,7 @@ public class PulsingCodec extends Codec 
                                                     state.fieldInfos,
                                                     state.segmentInfo.name,
                                                     state.termsIndexDivisor,
-                                                    state.codecId);
+                                                    state.codecId, state.context);
       success = true;
     } finally {
       if (!success) {
@@ -144,7 +144,7 @@ public class PulsingCodec extends Codec 
       FieldsProducer ret = new BlockTermsReader(indexReader,
                                                 state.dir, state.fieldInfos, state.segmentInfo.name,
                                                 pulsingReader,
-                                                state.readBufferSize,
+                                                state.context,
                                                 StandardCodec.TERMS_CACHE_SIZE,
                                                 state.codecId);
       success = true;
@@ -181,6 +181,6 @@ public class PulsingCodec extends Codec 
 
   @Override
   public PerDocValues docsProducer(SegmentReadState state) throws IOException {
-    return new DefaultDocValuesProducer(state.segmentInfo, state.dir, state.fieldInfos, state.codecId, getDocValuesUseCFS(), getDocValuesSortComparator());
+    return new DefaultDocValuesProducer(state.segmentInfo, state.dir, state.fieldInfos, state.codecId, getDocValuesUseCFS(), getDocValuesSortComparator(), state.context);
   }
 }

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/sep/IntStreamFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/sep/IntStreamFactory.java?rev=1144196&r1=1144195&r2=1144196&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/sep/IntStreamFactory.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/sep/IntStreamFactory.java Fri Jul  8 08:52:40 2011
@@ -18,16 +18,16 @@ package org.apache.lucene.index.codecs.s
  */
 
 import org.apache.lucene.store.Directory;
-import org.apache.lucene.store.BufferedIndexInput;
+import org.apache.lucene.store.IOContext;
 
 import java.io.IOException;
 
 /** @lucene.experimental */
 public abstract class IntStreamFactory {
-  public IntIndexInput openInput(Directory dir, String fileName) throws IOException {
-    return openInput(dir, fileName, BufferedIndexInput.BUFFER_SIZE);
+  public IntIndexInput openInput(Directory dir, String fileName, IOContext context) throws IOException {
+    return openInput(dir, fileName, context);
   }
 
-  public abstract IntIndexInput openInput(Directory dir, String fileName, int readBufferSize) throws IOException;
-  public abstract IntIndexOutput createOutput(Directory dir, String fileName) throws IOException;
+//  public abstract IntIndexInput openInput(Directory dir, String fileName, IOContext context) throws IOException;
+  public abstract IntIndexOutput createOutput(Directory dir, String fileName, IOContext context) throws IOException;
 }

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/sep/SepPostingsReaderImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/sep/SepPostingsReaderImpl.java?rev=1144196&r1=1144195&r2=1144196&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/sep/SepPostingsReaderImpl.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/sep/SepPostingsReaderImpl.java Fri Jul  8 08:52:40 2011
@@ -30,6 +30,7 @@ import org.apache.lucene.index.codecs.Bl
 import org.apache.lucene.index.codecs.PostingsReaderBase;
 import org.apache.lucene.store.ByteArrayDataInput;
 import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.Bits;
@@ -58,19 +59,19 @@ public class SepPostingsReaderImpl exten
   int maxSkipLevels;
   int skipMinimum;
 
-  public SepPostingsReaderImpl(Directory dir, SegmentInfo segmentInfo, int readBufferSize, IntStreamFactory intFactory, int codecId) throws IOException {
+  public SepPostingsReaderImpl(Directory dir, SegmentInfo segmentInfo, IOContext context, IntStreamFactory intFactory, int codecId) throws IOException {
     boolean success = false;
     try {
 
       final String docFileName = IndexFileNames.segmentFileName(segmentInfo.name, codecId, SepPostingsWriterImpl.DOC_EXTENSION);
-      docIn = intFactory.openInput(dir, docFileName);
+      docIn = intFactory.openInput(dir, docFileName, context);
 
-      skipIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, codecId, SepPostingsWriterImpl.SKIP_EXTENSION), readBufferSize);
+      skipIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, codecId, SepPostingsWriterImpl.SKIP_EXTENSION), context);
 
       if (segmentInfo.getHasProx()) {
-        freqIn = intFactory.openInput(dir, IndexFileNames.segmentFileName(segmentInfo.name, codecId, SepPostingsWriterImpl.FREQ_EXTENSION));
-        posIn = intFactory.openInput(dir, IndexFileNames.segmentFileName(segmentInfo.name, codecId, SepPostingsWriterImpl.POS_EXTENSION), readBufferSize);
-        payloadIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, codecId, SepPostingsWriterImpl.PAYLOAD_EXTENSION), readBufferSize);
+        freqIn = intFactory.openInput(dir, IndexFileNames.segmentFileName(segmentInfo.name, codecId, SepPostingsWriterImpl.FREQ_EXTENSION), context);
+        posIn = intFactory.openInput(dir, IndexFileNames.segmentFileName(segmentInfo.name, codecId, SepPostingsWriterImpl.POS_EXTENSION), context);
+        payloadIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, codecId, SepPostingsWriterImpl.PAYLOAD_EXTENSION), context);
       } else {
         posIn = null;
         payloadIn = null;

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/sep/SepPostingsWriterImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/sep/SepPostingsWriterImpl.java?rev=1144196&r1=1144195&r2=1144196&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/sep/SepPostingsWriterImpl.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/sep/SepPostingsWriterImpl.java Fri Jul  8 08:52:40 2011
@@ -118,25 +118,25 @@ public final class SepPostingsWriterImpl
       this.skipInterval = skipInterval;
       this.skipMinimum = skipInterval; /* set to the same for now */
       final String docFileName = IndexFileNames.segmentFileName(state.segmentName, state.codecId, DOC_EXTENSION);
-      docOut = factory.createOutput(state.directory, docFileName);
+      docOut = factory.createOutput(state.directory, docFileName, state.context);
       docIndex = docOut.index();
       
       if (state.fieldInfos.hasProx()) {
         final String frqFileName = IndexFileNames.segmentFileName(state.segmentName, state.codecId, FREQ_EXTENSION);
-        freqOut = factory.createOutput(state.directory, frqFileName);
+        freqOut = factory.createOutput(state.directory, frqFileName, state.context);
         freqIndex = freqOut.index();
         
         final String posFileName = IndexFileNames.segmentFileName(state.segmentName, state.codecId, POS_EXTENSION);
-        posOut = factory.createOutput(state.directory, posFileName);
+        posOut = factory.createOutput(state.directory, posFileName, state.context);
         posIndex = posOut.index();
         
         // TODO: -- only if at least one field stores payloads?
         final String payloadFileName = IndexFileNames.segmentFileName(state.segmentName, state.codecId, PAYLOAD_EXTENSION);
-        payloadOut = state.directory.createOutput(payloadFileName);
+        payloadOut = state.directory.createOutput(payloadFileName, state.context);
       }
       
       final String skipFileName = IndexFileNames.segmentFileName(state.segmentName, state.codecId, SKIP_EXTENSION);
-      skipOut = state.directory.createOutput(skipFileName);
+      skipOut = state.directory.createOutput(skipFileName, state.context);
       
       totalNumDocs = state.numDocs;
       

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextCodec.java?rev=1144196&r1=1144195&r2=1144196&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextCodec.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextCodec.java Fri Jul  8 08:52:40 2011
@@ -86,6 +86,6 @@ public class SimpleTextCodec extends Cod
 
   @Override
   public PerDocValues docsProducer(SegmentReadState state) throws IOException {
-    return new DefaultDocValuesProducer(state.segmentInfo, state.dir, state.fieldInfos, state.codecId, getDocValuesUseCFS(), getDocValuesSortComparator());
+    return new DefaultDocValuesProducer(state.segmentInfo, state.dir, state.fieldInfos, state.codecId, getDocValuesUseCFS(), getDocValuesSortComparator(), state.context);
   }
 }

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextFieldsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextFieldsReader.java?rev=1144196&r1=1144195&r2=1144196&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextFieldsReader.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextFieldsReader.java Fri Jul  8 08:52:40 2011
@@ -57,7 +57,7 @@ class SimpleTextFieldsReader extends Fie
   final static BytesRef PAYLOAD = SimpleTextFieldsWriter.PAYLOAD;
 
   public SimpleTextFieldsReader(SegmentReadState state) throws IOException {
-    in = state.dir.openInput(SimpleTextCodec.getPostingsFileName(state.segmentInfo.name, state.codecId));
+    in = state.dir.openInput(SimpleTextCodec.getPostingsFileName(state.segmentInfo.name, state.codecId), state.context);
    
     fieldInfos = state.fieldInfos;
   }

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextFieldsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextFieldsWriter.java?rev=1144196&r1=1144195&r2=1144196&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextFieldsWriter.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextFieldsWriter.java Fri Jul  8 08:52:40 2011
@@ -46,7 +46,7 @@ class SimpleTextFieldsWriter extends Fie
 
   public SimpleTextFieldsWriter(SegmentWriteState state) throws IOException {
     final String fileName = SimpleTextCodec.getPostingsFileName(state.segmentName, state.codecId);
-    out = state.directory.createOutput(fileName);
+    out = state.directory.createOutput(fileName, state.context);
   }
 
   private void write(String s) throws IOException {

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardCodec.java?rev=1144196&r1=1144195&r2=1144196&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardCodec.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardCodec.java Fri Jul  8 08:52:40 2011
@@ -88,7 +88,7 @@ public class StandardCodec extends Codec
 
   @Override
   public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
-    PostingsReaderBase postings = new StandardPostingsReader(state.dir, state.segmentInfo, state.readBufferSize, state.codecId);
+    PostingsReaderBase postings = new StandardPostingsReader(state.dir, state.segmentInfo, state.context, state.codecId);
     TermsIndexReaderBase indexReader;
 
     boolean success = false;
@@ -97,7 +97,7 @@ public class StandardCodec extends Codec
                                                     state.fieldInfos,
                                                     state.segmentInfo.name,
                                                     state.termsIndexDivisor,
-                                                    state.codecId);
+                                                    state.codecId, state.context);
       success = true;
     } finally {
       if (!success) {
@@ -112,7 +112,7 @@ public class StandardCodec extends Codec
                                                 state.fieldInfos,
                                                 state.segmentInfo.name,
                                                 postings,
-                                                state.readBufferSize,
+                                                state.context,
                                                 TERMS_CACHE_SIZE,
                                                 state.codecId);
       success = true;
@@ -162,6 +162,6 @@ public class StandardCodec extends Codec
 
   @Override
   public PerDocValues docsProducer(SegmentReadState state) throws IOException {
-    return new DefaultDocValuesProducer(state.segmentInfo, state.dir, state.fieldInfos, state.codecId, getDocValuesUseCFS(), getDocValuesSortComparator());
+    return new DefaultDocValuesProducer(state.segmentInfo, state.dir, state.fieldInfos, state.codecId, getDocValuesUseCFS(), getDocValuesSortComparator(), state.context);
   }
 }

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardPostingsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardPostingsReader.java?rev=1144196&r1=1144195&r2=1144196&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardPostingsReader.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardPostingsReader.java Fri Jul  8 08:52:40 2011
@@ -30,6 +30,7 @@ import org.apache.lucene.index.codecs.Bl
 import org.apache.lucene.index.codecs.PostingsReaderBase;
 import org.apache.lucene.store.ByteArrayDataInput;
 import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.Bits;
@@ -51,15 +52,15 @@ public class StandardPostingsReader exte
 
   //private String segment;
 
-  public StandardPostingsReader(Directory dir, SegmentInfo segmentInfo, int readBufferSize, int codecId) throws IOException {
+  public StandardPostingsReader(Directory dir, SegmentInfo segmentInfo, IOContext context, int codecId) throws IOException {
     freqIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, codecId, StandardCodec.FREQ_EXTENSION),
-                           readBufferSize);
+                           context);
     //this.segment = segmentInfo.name;
     if (segmentInfo.getHasProx()) {
       boolean success = false;
       try {
         proxIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, codecId, StandardCodec.PROX_EXTENSION),
-                               readBufferSize);
+                               context);
         success = true;
       } finally {
         if (!success) {

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardPostingsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardPostingsWriter.java?rev=1144196&r1=1144195&r2=1144196&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardPostingsWriter.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/standard/StandardPostingsWriter.java Fri Jul  8 08:52:40 2011
@@ -92,14 +92,14 @@ public final class StandardPostingsWrite
     this.skipMinimum = skipInterval; /* set to the same for now */
     //this.segment = state.segmentName;
     String fileName = IndexFileNames.segmentFileName(state.segmentName, state.codecId, StandardCodec.FREQ_EXTENSION);
-    freqOut = state.directory.createOutput(fileName);
+    freqOut = state.directory.createOutput(fileName, state.context);
     boolean success = false;
     try {
       if (state.fieldInfos.hasProx()) {
         // At least one field does not omit TF, so create the
         // prox file
         fileName = IndexFileNames.segmentFileName(state.segmentName, state.codecId, StandardCodec.PROX_EXTENSION);
-        proxOut = state.directory.createOutput(fileName);
+        proxOut = state.directory.createOutput(fileName, state.context);
       } else {
         // Every field omits TF so we will write no prox file
         proxOut = null;

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/values/Bytes.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/values/Bytes.java?rev=1144196&r1=1144195&r2=1144196&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/values/Bytes.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/values/Bytes.java Fri Jul  8 08:52:40 2011
@@ -28,6 +28,7 @@ import org.apache.lucene.index.values.In
 import org.apache.lucene.index.values.IndexDocValues.Source;
 import org.apache.lucene.index.values.IndexDocValues.SourceEnum;
 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;
@@ -100,12 +101,13 @@ 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.
    */
   public static Writer getWriter(Directory dir, String id, Mode mode,
-      Comparator<BytesRef> comp, boolean fixedSize, AtomicLong bytesUsed)
+      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?
@@ -115,19 +117,19 @@ public final class Bytes {
 
     if (fixedSize) {
       if (mode == Mode.STRAIGHT) {
-        return new FixedStraightBytesImpl.Writer(dir, id, bytesUsed);
+        return new FixedStraightBytesImpl.Writer(dir, id, bytesUsed, context);
       } else if (mode == Mode.DEREF) {
-        return new FixedDerefBytesImpl.Writer(dir, id, bytesUsed);
+        return new FixedDerefBytesImpl.Writer(dir, id, bytesUsed, context);
       } else if (mode == Mode.SORTED) {
-        return new FixedSortedBytesImpl.Writer(dir, id, comp, bytesUsed);
+        return new FixedSortedBytesImpl.Writer(dir, id, comp, bytesUsed, context);
       }
     } else {
       if (mode == Mode.STRAIGHT) {
-        return new VarStraightBytesImpl.Writer(dir, id, bytesUsed);
+        return new VarStraightBytesImpl.Writer(dir, id, bytesUsed, context);
       } else if (mode == Mode.DEREF) {
-        return new VarDerefBytesImpl.Writer(dir, id, bytesUsed);
+        return new VarDerefBytesImpl.Writer(dir, id, bytesUsed, context);
       } else if (mode == Mode.SORTED) {
-        return new VarSortedBytesImpl.Writer(dir, id, comp, bytesUsed);
+        return new VarSortedBytesImpl.Writer(dir, id, comp, bytesUsed, context);
       }
     }
 
@@ -157,23 +159,24 @@ public final class Bytes {
    *           if an {@link IOException} occurs
    */
   public static IndexDocValues getValues(Directory dir, String id, Mode mode,
-      boolean fixedSize, int maxDoc, Comparator<BytesRef> sortComparator) throws IOException {
+      boolean fixedSize, int maxDoc, Comparator<BytesRef> sortComparator, IOContext context) throws IOException {
+
     // TODO -- I can peek @ header to determing fixed/mode?
     if (fixedSize) {
       if (mode == Mode.STRAIGHT) {
-        return new FixedStraightBytesImpl.Reader(dir, id, maxDoc);
+        return new FixedStraightBytesImpl.Reader(dir, id, maxDoc, context);
       } else if (mode == Mode.DEREF) {
-        return new FixedDerefBytesImpl.Reader(dir, id, maxDoc);
+        return new FixedDerefBytesImpl.Reader(dir, id, maxDoc, context);
       } else if (mode == Mode.SORTED) {
-        return new FixedSortedBytesImpl.Reader(dir, id, maxDoc);
+        return new FixedSortedBytesImpl.Reader(dir, id, maxDoc, context);
       }
     } else {
       if (mode == Mode.STRAIGHT) {
-        return new VarStraightBytesImpl.Reader(dir, id, maxDoc);
+        return new VarStraightBytesImpl.Reader(dir, id, maxDoc, context);
       } else if (mode == Mode.DEREF) {
-        return new VarDerefBytesImpl.Reader(dir, id, maxDoc);
+        return new VarDerefBytesImpl.Reader(dir, id, maxDoc, context);
       } else if (mode == Mode.SORTED) {
-        return new VarSortedBytesImpl.Reader(dir, id, maxDoc, sortComparator);
+        return new VarSortedBytesImpl.Reader(dir, id, maxDoc, sortComparator, context);
       }
     }
 
@@ -343,15 +346,16 @@ public final class Bytes {
     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,
-        AtomicLong bytesUsed) throws IOException {
+        int version, AtomicLong bytesUsed, IOContext context) throws IOException {
       super(bytesUsed);
       this.id = id;
       this.dir = dir;
       this.codecName = codecName;
       this.version = version;
+      this.context = context;
     }
     
     protected IndexOutput getDataOut() throws IOException {
@@ -359,7 +363,7 @@ public final class Bytes {
         boolean success = false;
         try {
           datOut = dir.createOutput(IndexFileNames.segmentFileName(id, "",
-              DATA_EXTENSION));
+              DATA_EXTENSION), context);
           CodecUtil.writeHeader(datOut, codecName, version);
           success = true;
         } finally {
@@ -376,7 +380,7 @@ public final class Bytes {
       try {
         if (idxOut == null) {
           idxOut = dir.createOutput(IndexFileNames.segmentFileName(id, "",
-              INDEX_EXTENSION));
+              INDEX_EXTENSION), context);
           CodecUtil.writeHeader(idxOut, codecName, version);
         }
         success = true;
@@ -439,16 +443,16 @@ public final class Bytes {
     protected final String id;
 
     protected BytesReaderBase(Directory dir, String id, String codecName,
-        int maxVersion, boolean doIndex) throws IOException {
+        int maxVersion, boolean doIndex, IOContext context) throws IOException {
       this.id = id;
       datIn = dir.openInput(IndexFileNames.segmentFileName(id, "",
-          Writer.DATA_EXTENSION));
+          Writer.DATA_EXTENSION), context);
       boolean success = false;
       try {
       version = CodecUtil.checkHeader(datIn, codecName, maxVersion, maxVersion);
       if (doIndex) {
         idxIn = dir.openInput(IndexFileNames.segmentFileName(id, "",
-            Writer.INDEX_EXTENSION));
+            Writer.INDEX_EXTENSION), context);
         final int version2 = CodecUtil.checkHeader(idxIn, codecName,
             maxVersion, maxVersion);
         assert version == version2;

Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/values/FixedDerefBytesImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/values/FixedDerefBytesImpl.java?rev=1144196&r1=1144195&r2=1144196&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/values/FixedDerefBytesImpl.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/values/FixedDerefBytesImpl.java Fri Jul  8 08:52:40 2011
@@ -24,6 +24,7 @@ import org.apache.lucene.index.values.By
 import org.apache.lucene.index.values.Bytes.BytesReaderBase;
 import org.apache.lucene.index.values.Bytes.BytesWriterBase;
 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;
@@ -54,15 +55,15 @@ class FixedDerefBytesImpl {
     private int size = -1;
     private int[] docToID;
     private final BytesRefHash hash;
-    public Writer(Directory dir, String id, AtomicLong bytesUsed)
+    public Writer(Directory dir, String id, AtomicLong bytesUsed, IOContext context)
         throws IOException {
       this(dir, id, new DirectTrackingAllocator(ByteBlockPool.BYTE_BLOCK_SIZE, bytesUsed),
-          bytesUsed);
+          bytesUsed, context);
     }
 
     public Writer(Directory dir, String id, Allocator allocator,
-        AtomicLong bytesUsed) throws IOException {
-      super(dir, id, CODEC_NAME, VERSION_CURRENT, bytesUsed);
+        AtomicLong bytesUsed, IOContext context) throws IOException {
+      super(dir, id, CODEC_NAME, VERSION_CURRENT, bytesUsed, context);
       hash = new BytesRefHash(new ByteBlockPool(allocator),
           BytesRefHash.DEFAULT_CAPACITY, new TrackingDirectBytesStartArray(
               BytesRefHash.DEFAULT_CAPACITY, bytesUsed));
@@ -144,8 +145,8 @@ class FixedDerefBytesImpl {
   public static class Reader extends BytesReaderBase {
     private final int size;
 
-    Reader(Directory dir, String id, int maxDoc) throws IOException {
-      super(dir, id, CODEC_NAME, VERSION_START, true);
+    Reader(Directory dir, String id, int maxDoc, IOContext context) throws IOException {
+      super(dir, id, CODEC_NAME, VERSION_START, true, context);
       size = datIn.readInt();
     }
 



Mime
View raw message