### Eclipse Workspace Patch 1.0
#P lucene
Index: lucene/src/java/org/apache/lucene/util/BitVector.java
===================================================================
--- lucene/src/java/org/apache/lucene/util/BitVector.java	(revision 1128825)
+++ lucene/src/java/org/apache/lucene/util/BitVector.java	(working copy)
@@ -19,6 +19,9 @@
 
 import java.io.IOException;
 
+import org.apache.lucene.index.IOContext;
+import org.apache.lucene.index.IOContext.Context;
+import org.apache.lucene.index.MergePolicy.OneMerge;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.store.IndexOutput;
@@ -163,7 +166,7 @@
     <code>d</code>, in a format that can be read by the constructor {@link
     #BitVector(Directory, String)}.  */
   public final void write(Directory d, String name) throws IOException {
-    IndexOutput output = d.createOutput(name);
+    IndexOutput output = d.createOutput(name, IOContext.DEFAULT_IOCONTEXT);
     try {
       if (isSparse()) { 
         writeDgaps(output); // sparse bit-set more efficiently saved as d-gaps.
@@ -221,7 +224,7 @@
     <code>d</code>, as written by the {@link #write} method.
     */
   public BitVector(Directory d, String name) throws IOException {
-    IndexInput input = d.openInput(name);
+    IndexInput input = d.openInput(name, IOContext.DEFAULT_IOCONTEXT);
     try {
       size = input.readInt();       // read size
       if (size == -1) {
Index: lucene/src/java/org/apache/lucene/index/MergeInfo.java
===================================================================
--- lucene/src/java/org/apache/lucene/index/MergeInfo.java	(revision 0)
+++ lucene/src/java/org/apache/lucene/index/MergeInfo.java	(revision 0)
@@ -0,0 +1,32 @@
+package org.apache.lucene.index;
+
+import java.io.IOException;
+
+public abstract class MergeInfo implements java.io.Closeable {
+  
+  public static class OneMergeInfo {
+    
+    boolean optimize;               // used by IndexWriter
+    
+    boolean registerDone;           // used by IndexWriter
+    
+    boolean isExternal;             // used by IndexWriter
+    
+    public long estimatedMergeBytes;       // used by IndexWriter
+    
+    public final int totalDocCount;
+
+    public OneMergeInfo (MergePolicy.OneMerge oneMerge) {
+      this.optimize = oneMerge.optimize;
+      this.registerDone = oneMerge.registerDone;
+      this.isExternal = oneMerge.isExternal;
+      this.estimatedMergeBytes = oneMerge.estimatedMergeBytes;
+      this.totalDocCount = oneMerge.totalDocCount;    
+    }
+    
+  }
+  
+  public abstract void close()
+      throws IOException;
+
+}
Index: lucene/src/java/org/apache/lucene/index/CheckIndex.java
===================================================================
--- lucene/src/java/org/apache/lucene/index/CheckIndex.java	(revision 1128825)
+++ lucene/src/java/org/apache/lucene/index/CheckIndex.java	(working copy)
@@ -25,6 +25,8 @@
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.document.AbstractField;  // for javadocs
 import org.apache.lucene.document.Document;
+import org.apache.lucene.index.IOContext.Context;
+import org.apache.lucene.index.MergePolicy.OneMerge;
 import org.apache.lucene.index.codecs.CodecProvider;
 import org.apache.lucene.index.codecs.DefaultSegmentInfosWriter;
 import org.apache.lucene.util.Bits;
@@ -321,7 +323,7 @@
     final String segmentsFileName = sis.getCurrentSegmentFileName();
     IndexInput input = null;
     try {
-      input = dir.openInput(segmentsFileName);
+      input = dir.openInput(segmentsFileName, IOContext.DEFAULT_IOCONTEXT);
     } catch (Throwable t) {
       msg("ERROR: could not open segments file in directory");
       if (infoStream != null)
Index: lucene/src/java/org/apache/lucene/index/codecs/sep/SepPostingsReaderImpl.java
===================================================================
--- lucene/src/java/org/apache/lucene/index/codecs/sep/SepPostingsReaderImpl.java	(revision 1128825)
+++ lucene/src/java/org/apache/lucene/index/codecs/sep/SepPostingsReaderImpl.java	(working copy)
@@ -23,6 +23,7 @@
 import org.apache.lucene.index.DocsAndPositionsEnum;
 import org.apache.lucene.index.DocsEnum;
 import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.IOContext;
 import org.apache.lucene.index.IndexFileNames;
 import org.apache.lucene.index.SegmentInfo;
 import org.apache.lucene.index.TermState;
@@ -58,7 +59,7 @@
   int maxSkipLevels;
   int skipMinimum;
 
-  public SepPostingsReaderImpl(Directory dir, SegmentInfo segmentInfo, int readBufferSize, IntStreamFactory intFactory, String codecId) throws IOException {
+  public SepPostingsReaderImpl(Directory dir, SegmentInfo segmentInfo, IOContext context, IntStreamFactory intFactory, String codecId) throws IOException {
 
     boolean success = false;
     try {
@@ -66,12 +67,12 @@
       final String docFileName = IndexFileNames.segmentFileName(segmentInfo.name, codecId, SepPostingsWriterImpl.DOC_EXTENSION);
       docIn = intFactory.openInput(dir, docFileName);
 
-      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);
+        posIn = intFactory.openInput(dir, IndexFileNames.segmentFileName(segmentInfo.name, codecId, SepPostingsWriterImpl.POS_EXTENSION));
+        payloadIn = dir.openInput(IndexFileNames.segmentFileName(segmentInfo.name, codecId, SepPostingsWriterImpl.PAYLOAD_EXTENSION), context);
       } else {
         posIn = null;
         payloadIn = null;
Index: lucene/src/java/org/apache/lucene/store/MMapDirectory.java
===================================================================
--- lucene/src/java/org/apache/lucene/store/MMapDirectory.java	(revision 1128825)
+++ lucene/src/java/org/apache/lucene/store/MMapDirectory.java	(working copy)
@@ -31,6 +31,7 @@
 import java.security.PrivilegedActionException;
 import java.lang.reflect.Method;
 
+import org.apache.lucene.index.IOContext;
 import org.apache.lucene.util.Constants;
 
 /** File-based {@link Directory} implementation that uses
@@ -201,7 +202,7 @@
 
   /** Creates an IndexInput for the file with the given name. */
   @Override
-  public IndexInput openInput(String name, int bufferSize) throws IOException {
+  public IndexInput openInput(String name, IOContext context) throws IOException {
     ensureOpen();
     File f = new File(getDirectory(), name);
     RandomAccessFile raf = new RandomAccessFile(f, "r");
@@ -477,4 +478,5 @@
       }
     }
   }
+
 }
Index: lucene/src/java/org/apache/lucene/index/codecs/pulsing/PulsingCodec.java
===================================================================
--- lucene/src/java/org/apache/lucene/index/codecs/pulsing/PulsingCodec.java	(revision 1128825)
+++ lucene/src/java/org/apache/lucene/index/codecs/pulsing/PulsingCodec.java	(working copy)
@@ -107,7 +107,7 @@
 
     // 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
@@ -133,7 +133,7 @@
       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;
Index: lucene/contrib/misc/src/java/org/apache/lucene/store/WindowsDirectory.java
===================================================================
--- lucene/contrib/misc/src/java/org/apache/lucene/store/WindowsDirectory.java	(revision 1128825)
+++ lucene/contrib/misc/src/java/org/apache/lucene/store/WindowsDirectory.java	(working copy)
@@ -19,6 +19,8 @@
 
 import java.io.File;
 import java.io.IOException;
+
+import org.apache.lucene.index.IOContext;
 import org.apache.lucene.store.Directory; // javadoc
 import org.apache.lucene.store.NativeFSLockFactory; // javadoc
 
@@ -67,9 +69,9 @@
   }
 
   @Override
-  public IndexInput openInput(String name, int bufferSize) throws IOException {
+  public IndexInput openInput(String name, IOContext context) throws IOException {
     ensureOpen();
-    return new WindowsIndexInput(new File(getDirectory(), name), Math.max(bufferSize, DEFAULT_BUFFERSIZE));
+    return new WindowsIndexInput(new File(getDirectory(), name), DEFAULT_BUFFERSIZE);
   }
   
   protected static class WindowsIndexInput extends BufferedIndexInput {
Index: lucene/src/java/org/apache/lucene/index/IndexReader.java
===================================================================
--- lucene/src/java/org/apache/lucene/index/IndexReader.java	(revision 1128825)
+++ lucene/src/java/org/apache/lucene/index/IndexReader.java	(working copy)
@@ -1411,13 +1411,15 @@
 
     Directory dir = null;
     CompoundFileReader cfr = null;
+    IOContext context=null;
+    //TODO: Fill in IOContext
 
     try {
       File file = new File(filename);
       String dirname = file.getAbsoluteFile().getParent();
       filename = file.getName();
       dir = FSDirectory.open(new File(dirname));
-      cfr = new CompoundFileReader(dir, filename);
+      cfr = new CompoundFileReader(dir, filename, context);
 
       String [] files = cfr.listAll();
       ArrayUtil.mergeSort(files);   // sort the array of filename so that the output is more readable
@@ -1427,7 +1429,7 @@
 
         if (extract) {
           System.out.println("extract " + files[i] + " with " + len + " bytes to local directory...");
-          IndexInput ii = cfr.openInput(files[i]);
+          IndexInput ii = cfr.openInput(files[i], context);
 
           FileOutputStream f = new FileOutputStream(files[i]);
 
Index: lucene/src/java/org/apache/lucene/index/SegmentCoreReaders.java
===================================================================
--- lucene/src/java/org/apache/lucene/index/SegmentCoreReaders.java	(revision 1128825)
+++ lucene/src/java/org/apache/lucene/index/SegmentCoreReaders.java	(working copy)
@@ -42,7 +42,7 @@
   
   final Directory dir;
   final Directory cfsDir;
-  final int readBufferSize;
+  final IOContext context;
   final int termsIndexDivisor;
   
   private final SegmentReader owner;
@@ -52,7 +52,7 @@
   CompoundFileReader cfsReader;
   CompoundFileReader storeCFSReader;
   
-  SegmentCoreReaders(SegmentReader owner, Directory dir, SegmentInfo si, int readBufferSize, int termsIndexDivisor) throws IOException {
+  SegmentCoreReaders(SegmentReader owner, Directory dir, SegmentInfo si, IOContext context, int termsIndexDivisor) throws IOException {
     
     if (termsIndexDivisor == 0) {
       throw new IllegalArgumentException("indexDivisor must be < 0 (don't load terms index) or greater than 0 (got 0)");
@@ -60,7 +60,7 @@
     
     segment = si.name;
     final SegmentCodecs segmentCodecs = si.getSegmentCodecs();
-    this.readBufferSize = readBufferSize;
+    this.context = context;
     this.dir = dir;
     
     boolean success = false;
@@ -68,7 +68,7 @@
     try {
       Directory dir0 = dir;
       if (si.getUseCompoundFile()) {
-        cfsReader = new CompoundFileReader(dir, IndexFileNames.segmentFileName(segment, "", IndexFileNames.COMPOUND_FILE_EXTENSION), readBufferSize);
+        cfsReader = new CompoundFileReader(dir, IndexFileNames.segmentFileName(segment, "", IndexFileNames.COMPOUND_FILE_EXTENSION), context);
         dir0 = cfsReader;
       }
       cfsDir = dir0;
@@ -78,7 +78,7 @@
       this.termsIndexDivisor = termsIndexDivisor;
       
       // Ask codec for its Fields
-      fields = segmentCodecs.codec().fieldsProducer(new SegmentReadState(cfsDir, si, fieldInfos, readBufferSize, termsIndexDivisor));
+      fields = segmentCodecs.codec().fieldsProducer(new SegmentReadState(cfsDir, si, fieldInfos, context, termsIndexDivisor));
       assert fields != null;
       
       success = true;
@@ -153,7 +153,7 @@
           assert storeCFSReader == null;
           storeCFSReader = new CompoundFileReader(dir,
               IndexFileNames.segmentFileName(si.getDocStoreSegment(), "", IndexFileNames.COMPOUND_FILE_STORE_EXTENSION),
-              readBufferSize);
+              context);
           storeDir = storeCFSReader;
           assert storeDir != null;
         } else {
@@ -165,7 +165,7 @@
         // was not used, but then we are asked to open doc
         // stores after the segment has switched to CFS
         if (cfsReader == null) {
-          cfsReader = new CompoundFileReader(dir, IndexFileNames.segmentFileName(segment, "", IndexFileNames.COMPOUND_FILE_EXTENSION), readBufferSize);
+          cfsReader = new CompoundFileReader(dir, IndexFileNames.segmentFileName(segment, "", IndexFileNames.COMPOUND_FILE_EXTENSION), context);
         }
         storeDir = cfsReader;
         assert storeDir != null;
@@ -175,7 +175,7 @@
       }
       
       final String storesSegment = si.getDocStoreSegment();
-      fieldsReaderOrig = new FieldsReader(storeDir, storesSegment, fieldInfos, readBufferSize,
+      fieldsReaderOrig = new FieldsReader(storeDir, storesSegment, fieldInfos, context,
           si.getDocStoreOffset(), si.docCount);
       
       // Verify two sources of "maxDoc" agree:
@@ -184,7 +184,7 @@
       }
       
       if (si.getHasVectors()) { // open term vector files only as needed
-        termVectorsReaderOrig = new TermVectorsReader(storeDir, storesSegment, fieldInfos, readBufferSize, si.getDocStoreOffset(), si.docCount);
+        termVectorsReaderOrig = new TermVectorsReader(storeDir, storesSegment, fieldInfos, context, si.getDocStoreOffset(), si.docCount);
       }
     }
   }
Index: lucene/src/java/org/apache/lucene/index/SegmentInfo.java
===================================================================
--- lucene/src/java/org/apache/lucene/index/SegmentInfo.java	(revision 1128825)
+++ lucene/src/java/org/apache/lucene/index/SegmentInfo.java	(working copy)
@@ -244,7 +244,7 @@
       }
       final Directory dirToTest;
       if (isCompoundFile) {
-        dirToTest = new CompoundFileReader(dir, IndexFileNames.segmentFileName(storesSegment, "", ext));
+        dirToTest = new CompoundFileReader(dir, IndexFileNames.segmentFileName(storesSegment, "", ext), new IOContext(true));
       } else {
         dirToTest = dir;
       }
@@ -263,7 +263,7 @@
       Directory dir0 = dir;
       if (isCompoundFile && checkCompoundFile) {
         dir0 = new CompoundFileReader(dir, IndexFileNames.segmentFileName(name,
-            "", IndexFileNames.COMPOUND_FILE_EXTENSION));
+            "", IndexFileNames.COMPOUND_FILE_EXTENSION), new IOContext(true));
       }
       try {
         fieldInfos = new FieldInfos(dir0, IndexFileNames.segmentFileName(name,
Index: lucene/src/java/org/apache/lucene/index/DocFieldProcessor.java
===================================================================
--- lucene/src/java/org/apache/lucene/index/DocFieldProcessor.java	(revision 1128825)
+++ lucene/src/java/org/apache/lucene/index/DocFieldProcessor.java	(working copy)
@@ -27,6 +27,8 @@
 
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Fieldable;
+import org.apache.lucene.index.IOContext.Context;
+import org.apache.lucene.index.MergePolicy.OneMerge;
 import org.apache.lucene.util.ArrayUtil;
 
 
Index: lucene/src/java/org/apache/lucene/index/codecs/DefaultSegmentInfosWriter.java
===================================================================
--- lucene/src/java/org/apache/lucene/index/codecs/DefaultSegmentInfosWriter.java	(revision 1128825)
+++ lucene/src/java/org/apache/lucene/index/codecs/DefaultSegmentInfosWriter.java	(working copy)
@@ -19,8 +19,10 @@
 
 import java.io.IOException;
 
+import org.apache.lucene.index.IOContext;
 import org.apache.lucene.index.SegmentInfo;
 import org.apache.lucene.index.SegmentInfos;
+import org.apache.lucene.index.IOContext.Context;
 import org.apache.lucene.store.ChecksumIndexOutput;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IndexOutput;
@@ -70,7 +72,7 @@
   
   protected IndexOutput createOutput(Directory dir, String segmentFileName)
       throws IOException {
-    IndexOutput plainOut = dir.createOutput(segmentFileName);
+    IndexOutput plainOut = dir.createOutput(segmentFileName, new IOContext(Context.Default));
     ChecksumIndexOutput out = new ChecksumIndexOutput(plainOut);
     return out;
   }
Index: lucene/src/java/org/apache/lucene/index/SegmentWriteState.java
===================================================================
--- lucene/src/java/org/apache/lucene/index/SegmentWriteState.java	(revision 1128825)
+++ lucene/src/java/org/apache/lucene/index/SegmentWriteState.java	(working copy)
@@ -51,9 +51,12 @@
    * 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 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 @@
     this.termIndexInterval = termIndexInterval;
     this.segmentCodecs = segmentCodecs;
     codecId = "";
+    this.context=context;
   }
   
   /**
@@ -76,6 +80,7 @@
     numDocs = state.numDocs;
     termIndexInterval = state.termIndexInterval;
     segmentCodecs = state.segmentCodecs;
+    context=state.context;
     this.codecId = codecId;
     segDeletes = state.segDeletes;
   }
Index: lucene/src/java/org/apache/lucene/index/IOContext.java
===================================================================
--- lucene/src/java/org/apache/lucene/index/IOContext.java	(revision 0)
+++ lucene/src/java/org/apache/lucene/index/IOContext.java	(revision 0)
@@ -0,0 +1,73 @@
+package org.apache.lucene.index;
+
+import org.apache.lucene.index.MergePolicy.OneMerge;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+* Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+IOContext holds additional details on the merge/search context.
+A IOContext object can never be initialised as null as passed as a parameter to either 
+{@link #org.apache.lucene.store.Directory.openInput()} or {@link #org.apache.lucene.store.Directory.createInput()}
+ */
+public class IOContext {
+
+  /**
+   * Context is a enumerator which specifies the context in which the Directory is being used for.
+   */
+  public enum Context {Merge,Read,Flush,Default};
+  
+  /**
+   * An object of a enumerator Context type
+   */
+  public final Context context;
+  
+  /**
+   * An object of {@link #org.apache.lucene.index.MergePolicy.OneMerge}
+   */
+//  public final OneMerge onemerge;
+  public final MergeInfo.OneMergeInfo mergeInfo;
+  
+  public final boolean readOnce;
+  
+  public static final IOContext DEFAULT_IOCONTEXT = new IOContext(Context.Default);
+  
+  public IOContext () {
+    this.context = Context.Default;
+    this.mergeInfo = null;
+    this.readOnce = false;
+  }
+  
+  public IOContext(Context context) {
+    this.context = context;
+    this.mergeInfo = null;
+    this.readOnce = false;
+  }
+  
+  public IOContext(boolean readOnce) {
+    this.context = Context.Default;
+    this.mergeInfo = null;    
+    this.readOnce = readOnce;
+  }
+  
+  public IOContext (MergeInfo.OneMergeInfo mergeInfo) {
+    this.context=Context.Merge;
+    this.mergeInfo=mergeInfo;
+    this.readOnce=false;
+  }
+ 
+}
\ No newline at end of file
Index: lucene/contrib/misc/src/java/org/apache/lucene/store/DirectIOLinuxDirectory.java
===================================================================
--- lucene/contrib/misc/src/java/org/apache/lucene/store/DirectIOLinuxDirectory.java	(revision 1128825)
+++ lucene/contrib/misc/src/java/org/apache/lucene/store/DirectIOLinuxDirectory.java	(working copy)
@@ -25,6 +25,7 @@
 import java.nio.ByteBuffer;
 import java.nio.channels.FileChannel;
 
+import org.apache.lucene.index.IOContext;
 import org.apache.lucene.store.Directory; // javadoc
 import org.apache.lucene.store.NativeFSLockFactory; // javadoc
 
@@ -69,13 +70,13 @@
   }
 
   @Override
-  public IndexInput openInput(String name, int bufferSize) throws IOException {
+  public IndexInput openInput(String name, IOContext context) throws IOException {
     ensureOpen();
-    return new DirectIOLinuxIndexInput(new File(getDirectory(), name), forcedBufferSize == 0 ? bufferSize : forcedBufferSize);
+    return new DirectIOLinuxIndexInput(new File(getDirectory(), name), forcedBufferSize == 0 ? BufferedIndexInput.BUFFER_SIZE : forcedBufferSize);
   }
 
   @Override
-  public IndexOutput createOutput(String name) throws IOException {
+  public IndexOutput createOutput(String name,IOContext context) throws IOException {
     ensureOpen();
     ensureCanWrite(name);
     return new DirectIOLinuxIndexOutput(new File(getDirectory(), name), forcedBufferSize == 0 ? BufferedIndexOutput.BUFFER_SIZE : forcedBufferSize);
Index: lucene/src/java/org/apache/lucene/index/SegmentNorms.java
===================================================================
--- lucene/src/java/org/apache/lucene/index/SegmentNorms.java	(revision 1128825)
+++ lucene/src/java/org/apache/lucene/index/SegmentNorms.java	(working copy)
@@ -20,6 +20,8 @@
 import java.io.IOException;
 import java.util.concurrent.atomic.AtomicInteger;
 
+import org.apache.lucene.index.IOContext.Context;
+import org.apache.lucene.index.MergePolicy.OneMerge;
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.store.IndexOutput;
 
@@ -219,7 +221,7 @@
     // 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(Context.Default));
     boolean success = false;
     try {
       try {
Index: lucene/src/java/org/apache/lucene/store/SimpleFSDirectory.java
===================================================================
--- lucene/src/java/org/apache/lucene/store/SimpleFSDirectory.java	(revision 1128825)
+++ lucene/src/java/org/apache/lucene/store/SimpleFSDirectory.java	(working copy)
@@ -21,6 +21,8 @@
 import java.io.IOException;
 import java.io.RandomAccessFile;
 
+import org.apache.lucene.index.IOContext;
+
 /** A straightforward implementation of {@link FSDirectory}
  *  using java.io.RandomAccessFile.  However, this class has
  *  poor concurrent performance (multiple threads will
@@ -51,9 +53,9 @@
 
   /** Creates an IndexInput for the file with the given name. */
   @Override
-  public IndexInput openInput(String name, int bufferSize) throws IOException {
+  public IndexInput openInput(String name, IOContext context) throws IOException {
     ensureOpen();
-    return new SimpleFSIndexInput(new File(directory, name), bufferSize, getReadChunkSize());
+    return new SimpleFSIndexInput(new File(directory, name), context, getReadChunkSize());
   }
 
   protected static class SimpleFSIndexInput extends BufferedIndexInput {
@@ -85,8 +87,9 @@
     //  LUCENE-1566 - maximum read length on a 32bit JVM to prevent incorrect OOM 
     protected final int chunkSize;
     
-    public SimpleFSIndexInput(File path, int bufferSize, int chunkSize) throws IOException {
-      super(bufferSize);
+    public SimpleFSIndexInput(File path, IOContext context, int chunkSize) throws IOException {
+      //TODO Use IOContext to decide bufferSize instead of BufferedIndexInput.BUFFER_SIZE
+      super(BufferedIndexInput.BUFFER_SIZE);
       file = new Descriptor(path, "r");
       this.chunkSize = chunkSize;
     }
Index: lucene/contrib/misc/src/java/org/apache/lucene/store/NRTCachingDirectory.java
===================================================================
--- lucene/contrib/misc/src/java/org/apache/lucene/store/NRTCachingDirectory.java	(revision 1128825)
+++ lucene/contrib/misc/src/java/org/apache/lucene/store/NRTCachingDirectory.java	(working copy)
@@ -24,8 +24,11 @@
 import java.util.concurrent.ConcurrentHashMap;
 
 import org.apache.lucene.index.ConcurrentMergeScheduler;
+import org.apache.lucene.index.IOContext;
+import org.apache.lucene.index.IOContext.Context;
 import org.apache.lucene.index.IndexFileNames;
 import org.apache.lucene.index.IndexWriter;       // javadocs
+import org.apache.lucene.index.MergeInfo;
 import org.apache.lucene.index.MergePolicy;
 import org.apache.lucene.index.MergeScheduler;
 import org.apache.lucene.store.RAMDirectory;      // javadocs
@@ -158,17 +161,17 @@
   }
 
   @Override
-  public IndexOutput createOutput(String name) throws IOException {
+  public IndexOutput createOutput(String name, IOContext context) throws IOException {
     if (VERBOSE) {
       System.out.println("nrtdir.createOutput name=" + name);
     }
-    if (doCacheWrite(name)) {
+    if (doCacheWrite(name, context)) {
       if (VERBOSE) {
         System.out.println("  to cache");
       }
-      return cache.createOutput(name);
+      return cache.createOutput(name, context);
     } else {
-      return delegate.createOutput(name);
+      return delegate.createOutput(name, context);
     }
   }
 
@@ -184,7 +187,7 @@
   }
 
   @Override
-  public synchronized IndexInput openInput(String name) throws IOException {
+  public synchronized IndexInput openInput(String name, IOContext context) throws IOException {
     if (VERBOSE) {
       System.out.println("nrtdir.openInput name=" + name);
     }
@@ -192,22 +195,13 @@
       if (VERBOSE) {
         System.out.println("  from cache");
       }
-      return cache.openInput(name);
+      return cache.openInput(name, context);
     } else {
-      return delegate.openInput(name);
+      return delegate.openInput(name, context);
     }
   }
-
+  
   @Override
-  public synchronized IndexInput openInput(String name, int bufferSize) throws IOException {
-    if (cache.fileExists(name)) {
-      return cache.openInput(name, bufferSize);
-    } else {
-      return delegate.openInput(name, bufferSize);
-    }
-  }
-
-  @Override
   public Lock makeLock(String name) {
     return delegate.makeLock(name);
   }
@@ -246,18 +240,19 @@
 
   /** Subclass can override this to customize logic; return
    *  true if this file should be written to the RAMDirectory. */
-  protected boolean doCacheWrite(String name) {
-    final MergePolicy.OneMerge merge = merges.get(Thread.currentThread());
+  protected boolean doCacheWrite(String name, IOContext context) {
+    final MergeInfo.OneMergeInfo merge = context.mergeInfo;
     //System.out.println(Thread.currentThread().getName() + ": CACHE check merge=" + merge + " size=" + (merge==null ? 0 : merge.estimatedMergeBytes));
     return !name.equals(IndexFileNames.SEGMENTS_GEN) && (merge == null || merge.estimatedMergeBytes <= maxMergeSizeBytes) && cache.sizeInBytes() <= maxCachedBytes;
   }
 
   private void unCache(String fileName) throws IOException {
     final IndexOutput out;
+    IOContext context=new IOContext(Context.Default);
     synchronized(this) {
       if (!delegate.fileExists(fileName)) {
         assert cache.fileExists(fileName);
-        out = delegate.createOutput(fileName);
+        out = delegate.createOutput(fileName, context);
       } else {
         out = null;
       }
@@ -266,7 +261,7 @@
     if (out != null) {
       IndexInput in = null;
       try {
-        in = cache.openInput(fileName);
+        in = cache.openInput(fileName, context);
         in.copyBytes(out, in.length());
       } finally {
         IOUtils.closeSafely(in, out);
Index: lucene/src/java/org/apache/lucene/store/RAMDirectory.java
===================================================================
--- lucene/src/java/org/apache/lucene/store/RAMDirectory.java	(revision 1128825)
+++ lucene/src/java/org/apache/lucene/store/RAMDirectory.java	(working copy)
@@ -27,6 +27,8 @@
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.atomic.AtomicLong;
 
+import org.apache.lucene.index.IOContext;
+
 /**
  * A memory-resident {@link Directory} implementation.  Locking
  * implementation is by default the {@link SingleInstanceLockFactory}
@@ -38,7 +40,7 @@
   
   // *****
   // Lock acquisition sequence:  RAMDirectory, then RAMFile
-  // *****
+  // ***** 
 
   /** Constructs an empty {@link Directory}. */
   public RAMDirectory() {
@@ -65,14 +67,14 @@
    * @param dir a <code>Directory</code> value
    * @exception IOException if an error occurs
    */
-  public RAMDirectory(Directory dir) throws IOException {
-    this(dir, false);
+  public RAMDirectory(Directory dir, IOContext context) throws IOException {
+    this(dir, false, context);    
   }
   
-  private RAMDirectory(Directory dir, boolean closeDir) throws IOException {
+  private RAMDirectory(Directory dir, boolean closeDir, IOContext context) throws IOException {
     this();
     for (String file : dir.listAll()) {
-      dir.copy(this, file, file);
+      dir.copy(this, file, file, context);
     }
     if (closeDir) {
       dir.close();
@@ -149,7 +151,7 @@
 
   /** Creates a new, empty file in the directory with the given name. Returns a stream writing this file. */
   @Override
-  public IndexOutput createOutput(String name) throws IOException {
+  public IndexOutput createOutput(String name, IOContext context) throws IOException {
     ensureOpen();
     RAMFile file = newRAMFile();
     RAMFile existing = fileMap.remove(name);
@@ -176,7 +178,7 @@
 
   /** Returns a stream reading an existing file. */
   @Override
-  public IndexInput openInput(String name) throws IOException {
+  public IndexInput openInput(String name, IOContext context) throws IOException {
     ensureOpen();
     RAMFile file = fileMap.get(name);
     if (file == null) {
Index: lucene/src/java/org/apache/lucene/index/codecs/BlockTermsReader.java
===================================================================
--- lucene/src/java/org/apache/lucene/index/codecs/BlockTermsReader.java	(revision 1128825)
+++ lucene/src/java/org/apache/lucene/index/codecs/BlockTermsReader.java	(working copy)
@@ -29,6 +29,7 @@
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.FieldInfos;
 import org.apache.lucene.index.FieldsEnum;
+import org.apache.lucene.index.IOContext;
 import org.apache.lucene.index.IndexFileNames;
 import org.apache.lucene.index.SegmentInfo;
 import org.apache.lucene.index.TermState;
@@ -107,7 +108,7 @@
   
   //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, String codecId)
     throws IOException {
     
@@ -116,7 +117,7 @@
 
     //this.segment = segment;
     in = dir.openInput(IndexFileNames.segmentFileName(segment, codecId, BlockTermsWriter.TERMS_EXTENSION),
-                       readBufferSize);
+                       context);
 
     boolean success = false;
     try {
Index: lucene/src/java/org/apache/lucene/index/codecs/preflex/TermInfosReader.java
===================================================================
--- lucene/src/java/org/apache/lucene/index/codecs/preflex/TermInfosReader.java	(revision 1128825)
+++ lucene/src/java/org/apache/lucene/index/codecs/preflex/TermInfosReader.java	(working copy)
@@ -21,6 +21,7 @@
 
 import org.apache.lucene.index.CorruptIndexException;
 import org.apache.lucene.index.FieldInfos;
+import org.apache.lucene.index.IOContext;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.index.IndexFileNames;
 import org.apache.lucene.store.Directory;
@@ -93,7 +94,7 @@
     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;
 
@@ -107,7 +108,7 @@
       fieldInfos = fis;
 
       origEnum = new SegmentTermEnum(directory.openInput(IndexFileNames.segmentFileName(segment, "", PreFlexCodec.TERMS_EXTENSION),
-                                                         readBufferSize), fieldInfos, false);
+                                                         context), fieldInfos, false);
       size = origEnum.size;
 
 
@@ -115,7 +116,7 @@
         // 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
Index: lucene/src/java/org/apache/lucene/index/NormsWriter.java
===================================================================
--- lucene/src/java/org/apache/lucene/index/NormsWriter.java	(revision 1128825)
+++ lucene/src/java/org/apache/lucene/index/NormsWriter.java	(working copy)
@@ -21,6 +21,7 @@
 import java.util.Collection;
 import java.util.Map;
 
+import org.apache.lucene.index.IOContext.Context;
 import org.apache.lucene.store.IndexOutput;
 
 // TODO FI: norms could actually be stored as doc store
@@ -48,7 +49,7 @@
     }
 
     final String normsFileName = IndexFileNames.segmentFileName(state.segmentName, "", IndexFileNames.NORMS_EXTENSION);
-    IndexOutput normsOut = state.directory.createOutput(normsFileName);
+    IndexOutput normsOut = state.directory.createOutput(normsFileName, new IOContext(true));
 
     try {
       normsOut.writeBytes(SegmentNorms.NORMS_HEADER, 0, SegmentNorms.NORMS_HEADER.length);
Index: lucene/src/java/org/apache/lucene/index/TermVectorsTermsWriter.java
===================================================================
--- lucene/src/java/org/apache/lucene/index/TermVectorsTermsWriter.java	(revision 1128825)
+++ lucene/src/java/org/apache/lucene/index/TermVectorsTermsWriter.java	(working copy)
@@ -20,6 +20,8 @@
 import java.io.IOException;
 import java.util.Map;
 
+import org.apache.lucene.index.IOContext.Context;
+import org.apache.lucene.index.MergePolicy.OneMerge;
 import org.apache.lucene.store.IndexOutput;
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.BytesRef;
@@ -89,14 +91,15 @@
 
   private final void initTermVectorsWriter() throws IOException {
     if (tvx == null) {
+      IOContext context = new IOContext(Context.Merge);
 
       // 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);
Index: lucene/src/java/org/apache/lucene/index/SegmentInfos.java
===================================================================
--- lucene/src/java/org/apache/lucene/index/SegmentInfos.java	(revision 1128825)
+++ lucene/src/java/org/apache/lucene/index/SegmentInfos.java	(working copy)
@@ -600,7 +600,7 @@
           for(int i=0;i<defaultGenFileRetryCount;i++) {
             IndexInput genInput = null;
             try {
-              genInput = directory.openInput(IndexFileNames.SEGMENTS_GEN);
+              genInput = directory.openInput(IndexFileNames.SEGMENTS_GEN, new IOContext(true));
             } catch (FileNotFoundException e) {
               if (infoStream != null) {
                 message("segments.gen open: FileNotFoundException " + e);
@@ -817,7 +817,7 @@
   }
   
   private final long writeGlobalFieldMap(FieldNumberBiMap map, Directory dir, String name) throws IOException {
-    final IndexOutput output = dir.createOutput(name);
+    final IndexOutput output = dir.createOutput(name, new IOContext(true));
     boolean success = false;
     long version;
     try {
@@ -846,7 +846,7 @@
   
   private void readGlobalFieldMap(FieldNumberBiMap map, Directory dir) throws IOException {
     final String name = getGlobalFieldNumberName(lastGlobalFieldMapVersion);
-    final IndexInput input = dir.openInput(name);
+    final IndexInput input = dir.openInput(name, new IOContext(true));
     try {
       map.read(input);
     } finally {
@@ -937,7 +937,7 @@
     }
 
     try {
-      IndexOutput genOutput = dir.createOutput(IndexFileNames.SEGMENTS_GEN);
+      IndexOutput genOutput = dir.createOutput(IndexFileNames.SEGMENTS_GEN, new IOContext(true));
       try {
         genOutput.writeInt(FORMAT_SEGMENTS_GEN_CURRENT);
         genOutput.writeLong(generation);
Index: lucene/src/java/org/apache/lucene/store/FSDirectory.java
===================================================================
--- lucene/src/java/org/apache/lucene/store/FSDirectory.java	(revision 1128825)
+++ lucene/src/java/org/apache/lucene/store/FSDirectory.java	(working copy)
@@ -31,6 +31,7 @@
 import java.util.Set;
 import java.util.concurrent.Future;
 
+import org.apache.lucene.index.IOContext;
 import org.apache.lucene.util.ThreadInterruptedException;
 import org.apache.lucene.util.Constants;
 
@@ -297,7 +298,7 @@
 
   /** Creates an IndexOutput for the file with the given name. */
   @Override
-  public IndexOutput createOutput(String name) throws IOException {
+  public IndexOutput createOutput(String name, IOContext context) throws IOException {
     ensureOpen();
 
     ensureCanWrite(name);
@@ -332,9 +333,9 @@
 
   // Inherit javadoc
   @Override
-  public IndexInput openInput(String name) throws IOException {
+  public IndexInput openInput(String name, IOContext context) throws IOException {
     ensureOpen();
-    return openInput(name, BufferedIndexInput.BUFFER_SIZE);
+    return openInput(name, context);
   }
 
   /**
Index: lucene/src/java/org/apache/lucene/index/codecs/DefaultSegmentInfosReader.java
===================================================================
--- lucene/src/java/org/apache/lucene/index/codecs/DefaultSegmentInfosReader.java	(revision 1128825)
+++ lucene/src/java/org/apache/lucene/index/codecs/DefaultSegmentInfosReader.java	(working copy)
@@ -22,11 +22,13 @@
 import org.apache.lucene.index.CompoundFileReader;
 import org.apache.lucene.index.CorruptIndexException;
 import org.apache.lucene.index.FieldsReader;
+import org.apache.lucene.index.IOContext;
 import org.apache.lucene.index.IndexFileNames;
 import org.apache.lucene.index.IndexFormatTooOldException;
 import org.apache.lucene.index.IndexFormatTooNewException;
 import org.apache.lucene.index.SegmentInfo;
 import org.apache.lucene.index.SegmentInfos;
+import org.apache.lucene.index.IOContext.Context;
 import org.apache.lucene.store.ChecksumIndexInput;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IndexInput;
@@ -70,11 +72,11 @@
             if (si.getDocStoreIsCompoundFile()) {
               dir = new CompoundFileReader(dir, IndexFileNames.segmentFileName(
                   si.getDocStoreSegment(), "",
-                  IndexFileNames.COMPOUND_FILE_STORE_EXTENSION), 1024);
+                  IndexFileNames.COMPOUND_FILE_STORE_EXTENSION), new IOContext(Context.Default));
             }
           } else if (si.getUseCompoundFile()) {
             dir = new CompoundFileReader(dir, IndexFileNames.segmentFileName(
-                si.name, "", IndexFileNames.COMPOUND_FILE_EXTENSION), 1024);
+                si.name, "", IndexFileNames.COMPOUND_FILE_EXTENSION), new IOContext(Context.Default));
           }
 
           try {
@@ -109,7 +111,7 @@
   }
   
   public IndexInput openInput(Directory dir, String segmentsFileName) throws IOException {
-    IndexInput in = dir.openInput(segmentsFileName);
+    IndexInput in = dir.openInput(segmentsFileName, new IOContext(Context.Default));
     return new ChecksumIndexInput(in);
     
   }
Index: lucene/src/java/org/apache/lucene/index/TermVectorsReader.java
===================================================================
--- lucene/src/java/org/apache/lucene/index/TermVectorsReader.java	(revision 1128825)
+++ lucene/src/java/org/apache/lucene/index/TermVectorsReader.java	(working copy)
@@ -17,6 +17,8 @@
  * limitations under the License.
  */
 
+import org.apache.lucene.index.IOContext.Context;
+import org.apache.lucene.index.MergePolicy.OneMerge;
 import org.apache.lucene.store.BufferedIndexInput;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IndexInput;
@@ -63,27 +65,27 @@
 
   TermVectorsReader(Directory d, String segment, FieldInfos fieldInfos)
     throws CorruptIndexException, IOException {
-    this(d, segment, fieldInfos, BufferedIndexInput.BUFFER_SIZE);
+    this(d, segment, fieldInfos, new IOContext(Context.Merge));
   }
 
-  TermVectorsReader(Directory d, String segment, FieldInfos fieldInfos, int readBufferSize)
+  TermVectorsReader(Directory d, String segment, FieldInfos fieldInfos, IOContext context)
     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;
Index: lucene/src/java/org/apache/lucene/index/SegmentReader.java
===================================================================
--- lucene/src/java/org/apache/lucene/index/SegmentReader.java	(revision 1128825)
+++ lucene/src/java/org/apache/lucene/index/SegmentReader.java	(working copy)
@@ -29,7 +29,6 @@
 
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.FieldSelector;
-import org.apache.lucene.store.BufferedIndexInput;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.util.BitVector;
@@ -88,7 +87,7 @@
    * @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);
+    return get(readOnly, si.dir, si, true, termInfosIndexDivisor);
   }
 
   /**
@@ -98,7 +97,6 @@
   public static SegmentReader get(boolean readOnly,
                                   Directory dir,
                                   SegmentInfo si,
-                                  int readBufferSize,
                                   boolean doOpenStores,
                                   int termInfosIndexDivisor)
     throws CorruptIndexException, IOException {
@@ -106,17 +104,17 @@
     SegmentReader instance = new SegmentReader();
     instance.readOnly = readOnly;
     instance.si = si;
-    instance.readBufferSize = readBufferSize;
+//    instance.readBufferSize = readBufferSize;
 
     boolean success = false;
 
     try {
-      instance.core = new SegmentCoreReaders(instance, dir, si, readBufferSize, termInfosIndexDivisor);
+      instance.core = new SegmentCoreReaders(instance, dir, si, IOContext.DEFAULT_IOCONTEXT, termInfosIndexDivisor);
       if (doOpenStores) {
         instance.core.openDocStores(si);
       }
       instance.loadDeletedDocs();
-      instance.openNorms(instance.core.cfsDir, readBufferSize);
+      instance.openNorms(instance.core.cfsDir, IOContext.DEFAULT_IOCONTEXT);
       success = true;
     } finally {
 
@@ -294,7 +292,7 @@
 
       // 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_IOCONTEXT);
 
       success = true;
     } finally {
@@ -573,7 +571,7 @@
     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) {
@@ -597,7 +595,7 @@
         if (singleNormFile) {
           normSeek = nextNormSeek;
           if (singleNormStream == null) {
-            singleNormStream = d.openInput(fileName, readBufferSize);
+            singleNormStream = d.openInput(fileName, context);
             singleNormRef = new AtomicInteger(1);
           } else {
             singleNormRef.incrementAndGet();
@@ -607,7 +605,7 @@
           // 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.
Index: lucene/src/java/org/apache/lucene/index/codecs/VariableGapTermsIndexReader.java
===================================================================
--- lucene/src/java/org/apache/lucene/index/codecs/VariableGapTermsIndexReader.java	(revision 1128825)
+++ lucene/src/java/org/apache/lucene/index/codecs/VariableGapTermsIndexReader.java	(working copy)
@@ -27,6 +27,8 @@
 
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.FieldInfos;
+import org.apache.lucene.index.IOContext;
+import org.apache.lucene.index.IOContext.Context;
 import org.apache.lucene.index.IndexFileNames;
 import org.apache.lucene.index.SegmentInfo;
 import org.apache.lucene.store.Directory;
@@ -61,7 +63,7 @@
   public VariableGapTermsIndexReader(Directory dir, FieldInfos fieldInfos, String segment, int indexDivisor, String codecId)
     throws IOException {
 
-    in = dir.openInput(IndexFileNames.segmentFileName(segment, codecId, VariableGapTermsIndexWriter.TERMS_INDEX_EXTENSION));
+    in = dir.openInput(IndexFileNames.segmentFileName(segment, codecId, VariableGapTermsIndexWriter.TERMS_INDEX_EXTENSION), new IOContext(Context.Default));
     this.segment = segment;
     boolean success = false;
 
Index: lucene/src/java/org/apache/lucene/index/CompoundFileWriter.java
===================================================================
--- lucene/src/java/org/apache/lucene/index/CompoundFileWriter.java	(revision 1128825)
+++ lucene/src/java/org/apache/lucene/index/CompoundFileWriter.java	(working copy)
@@ -78,6 +78,7 @@
 
     private Directory directory;
     private String fileName;
+    private IOContext context;
     private HashSet<String> ids;
     private LinkedList<FileEntry> entries;
     private boolean merged = false;
@@ -87,11 +88,11 @@
      *  entire name (no extensions are added).
      *  @throws NullPointerException if <code>dir</code> or <code>name</code> is null
      */
-    public CompoundFileWriter(Directory dir, String name) {
-      this(dir, name, null);
+    public CompoundFileWriter(Directory dir, String name, IOContext context) {
+      this(dir, name, context, null);
     }
 
-    CompoundFileWriter(Directory dir, String name, MergeState.CheckAbort checkAbort) {
+    CompoundFileWriter(Directory dir, String name, IOContext context, MergeState.CheckAbort checkAbort) {
         if (dir == null)
             throw new NullPointerException("directory cannot be null");
         if (name == null)
@@ -101,6 +102,7 @@
         fileName = name;
         ids = new HashSet<String>();
         entries = new LinkedList<FileEntry>();
+        this.context=context;
     }
 
     /** Returns the directory of the compound file. */
@@ -164,7 +166,7 @@
         merged = true;
 
         // open the compound stream
-        IndexOutput os = directory.createOutput(fileName);
+        IndexOutput os = directory.createOutput(fileName, context);
         IOException priorException = null;
         try {
             // Write the Version info - must be a VInt because CFR reads a VInt
@@ -228,7 +230,7 @@
    * output stream.
    */
   private void copyFile(FileEntry source, IndexOutput os) throws IOException {
-    IndexInput is = source.dir.openInput(source.file);
+    IndexInput is = source.dir.openInput(source.file, context);
     try {
       long startPtr = os.getFilePointer();
       long length = is.length();
Index: lucene/src/java/org/apache/lucene/index/SegmentMerger.java
===================================================================
--- lucene/src/java/org/apache/lucene/index/SegmentMerger.java	(revision 1128825)
+++ lucene/src/java/org/apache/lucene/index/SegmentMerger.java	(working copy)
@@ -25,6 +25,7 @@
 import java.util.regex.Pattern; // for assert
 
 import org.apache.lucene.document.Document;
+import org.apache.lucene.index.IOContext.Context;
 import org.apache.lucene.index.IndexReader.FieldOption;
 import org.apache.lucene.index.MergePolicy.MergeAbortedException;
 import org.apache.lucene.index.codecs.Codec;
@@ -131,7 +132,7 @@
 
     // Now merge all added files
     Collection<String> files = info.files();
-    CompoundFileWriter cfsWriter = new CompoundFileWriter(directory, fileName, checkAbort);
+    CompoundFileWriter cfsWriter = new CompoundFileWriter(directory, fileName, new IOContext(Context.Merge), checkAbort);
     for (String file : files) {
       assert !IndexFileNames.matchesExtension(file, IndexFileNames.DELETES_EXTENSION) 
                 : ".del file is not allowed in .cfs: " + file;
@@ -267,7 +268,7 @@
       // 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, new IOContext(Context.Merge));
 
     return docCount;
   }
@@ -583,7 +584,7 @@
       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), new IOContext(Context.Merge));
             output.writeBytes(SegmentNorms.NORMS_HEADER, SegmentNorms.NORMS_HEADER.length);
           }
           for (IndexReader reader : readers) {
Index: lucene/src/java/org/apache/lucene/index/FieldsWriter.java
===================================================================
--- lucene/src/java/org/apache/lucene/index/FieldsWriter.java	(revision 1128825)
+++ lucene/src/java/org/apache/lucene/index/FieldsWriter.java	(working copy)
@@ -22,6 +22,8 @@
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Fieldable;
 import org.apache.lucene.document.NumericField;
+import org.apache.lucene.index.IOContext.Context;
+import org.apache.lucene.index.MergePolicy.OneMerge;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.store.IndexOutput;
@@ -71,8 +73,8 @@
 
     boolean success = false;
     try {
-      fieldsStream = directory.createOutput(IndexFileNames.segmentFileName(segment, "", IndexFileNames.FIELDS_EXTENSION));
-      indexStream = directory.createOutput(IndexFileNames.segmentFileName(segment, "", IndexFileNames.FIELDS_INDEX_EXTENSION));
+      fieldsStream = directory.createOutput(IndexFileNames.segmentFileName(segment, "", IndexFileNames.FIELDS_EXTENSION), new IOContext(Context.Default));
+      indexStream = directory.createOutput(IndexFileNames.segmentFileName(segment, "", IndexFileNames.FIELDS_INDEX_EXTENSION), new IOContext(Context.Default));
 
       fieldsStream.writeInt(FORMAT_CURRENT);
       indexStream.writeInt(FORMAT_CURRENT);
Index: lucene/src/java/org/apache/lucene/index/codecs/BlockTermsWriter.java
===================================================================
--- lucene/src/java/org/apache/lucene/index/codecs/BlockTermsWriter.java	(revision 1128825)
+++ lucene/src/java/org/apache/lucene/index/codecs/BlockTermsWriter.java	(working copy)
@@ -74,7 +74,7 @@
   {
     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);
     fieldInfos = state.fieldInfos;
     writeHeader(out);
     currentField = null;
Index: lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextFieldsReader.java
===================================================================
--- lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextFieldsReader.java	(revision 1128825)
+++ lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextFieldsReader.java	(working copy)
@@ -58,7 +58,7 @@
   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;
   }
Index: lucene/src/java/org/apache/lucene/store/FileSwitchDirectory.java
===================================================================
--- lucene/src/java/org/apache/lucene/store/FileSwitchDirectory.java	(revision 1128825)
+++ lucene/src/java/org/apache/lucene/store/FileSwitchDirectory.java	(working copy)
@@ -25,6 +25,8 @@
 import java.util.Set;
 import java.util.HashSet;
 
+import org.apache.lucene.index.IOContext;
+
 /**
  * Expert: A Directory instance that switches files between
  * two other Directory instances.
@@ -125,8 +127,8 @@
   }
 
   @Override
-  public IndexOutput createOutput(String name) throws IOException {
-    return getDirectory(name).createOutput(name);
+  public IndexOutput createOutput(String name, IOContext context) throws IOException {
+    return getDirectory(name).createOutput(name, context);
   }
 
   @Override
@@ -145,7 +147,7 @@
   }
 
   @Override
-  public IndexInput openInput(String name) throws IOException {
-    return getDirectory(name).openInput(name);
+  public IndexInput openInput(String name, IOContext context) throws IOException {
+    return getDirectory(name).openInput(name, context);
   }
 }
Index: lucene/src/java/org/apache/lucene/index/FieldInfos.java
===================================================================
--- lucene/src/java/org/apache/lucene/index/FieldInfos.java	(revision 1128825)
+++ lucene/src/java/org/apache/lucene/index/FieldInfos.java	(working copy)
@@ -267,7 +267,7 @@
    */
   public FieldInfos(Directory d, String name) throws IOException {
     this((FieldNumberBiMap)null, null); // use null here to make this FIs Read-Only
-    final IndexInput input = d.openInput(name);
+    final IndexInput input = d.openInput(name, new IOContext(true));
     try {
       read(input, name);
     } finally {
@@ -560,7 +560,7 @@
   }
 
   public void write(Directory d, String name) throws IOException {
-    IndexOutput output = d.createOutput(name);
+    IndexOutput output = d.createOutput(name, new IOContext(true));
     try {
       write(output);
     } finally {
Index: lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextFieldsWriter.java
===================================================================
--- lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextFieldsWriter.java	(revision 1128825)
+++ lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextFieldsWriter.java	(working copy)
@@ -46,7 +46,7 @@
 
   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 {
Index: lucene/src/java/org/apache/lucene/index/codecs/preflex/PreFlexFields.java
===================================================================
--- lucene/src/java/org/apache/lucene/index/codecs/preflex/PreFlexFields.java	(revision 1128825)
+++ lucene/src/java/org/apache/lucene/index/codecs/preflex/PreFlexFields.java	(working copy)
@@ -31,6 +31,7 @@
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.FieldInfos;
 import org.apache.lucene.index.FieldsEnum;
+import org.apache.lucene.index.IOContext;
 import org.apache.lucene.index.IndexFileNames;
 import org.apache.lucene.index.SegmentInfo;
 import org.apache.lucene.index.Term;
@@ -62,10 +63,10 @@
   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 @@
     
     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(info.name + ".frq", readBufferSize);
+      freqStream = dir.openInput(info.name + ".frq", context);
       boolean anyProx = false;
       for (FieldInfo fi : fieldInfos) {
         if (fi.isIndexed) {
@@ -105,7 +106,7 @@
       }
 
       if (anyProx) {
-        proxStream = dir.openInput(info.name + ".prx", readBufferSize);
+        proxStream = dir.openInput(info.name + ".prx", context);
       } else {
         proxStream = null;
       }
@@ -178,7 +179,7 @@
         // to CFS
 
         if (!(dir instanceof CompoundFileReader)) {
-          dir0 = cfsReader = new CompoundFileReader(dir, IndexFileNames.segmentFileName(si.name, "", IndexFileNames.COMPOUND_FILE_EXTENSION), readBufferSize);
+          dir0 = cfsReader = new CompoundFileReader(dir, IndexFileNames.segmentFileName(si.name, "", IndexFileNames.COMPOUND_FILE_EXTENSION), context);
         } else {
           dir0 = dir;
         }
@@ -187,7 +188,7 @@
         dir0 = dir;
       }
 
-      tis = new TermInfosReader(dir0, si.name, fieldInfos, readBufferSize, indexDivisor);
+      tis = new TermInfosReader(dir0, si.name, fieldInfos, context, indexDivisor);
     }
   }
 
Index: lucene/src/java/org/apache/lucene/index/PerFieldCodecWrapper.java
===================================================================
--- lucene/src/java/org/apache/lucene/index/PerFieldCodecWrapper.java	(revision 1128825)
+++ lucene/src/java/org/apache/lucene/index/PerFieldCodecWrapper.java	(working copy)
@@ -25,6 +25,7 @@
 import java.util.Set;
 import java.util.TreeSet;
 
+import org.apache.lucene.index.IOContext.Context;
 import org.apache.lucene.index.codecs.Codec;
 import org.apache.lucene.index.codecs.FieldsConsumer;
 import org.apache.lucene.index.codecs.FieldsProducer;
@@ -99,7 +100,7 @@
     private final Map<String, FieldsProducer> codecs = new HashMap<String, FieldsProducer>();
 
     public FieldsReader(Directory dir, FieldInfos fieldInfos, SegmentInfo si,
-        int readBufferSize, int indexDivisor) throws IOException {
+        IOContext context, int indexDivisor) throws IOException {
 
       final Map<Codec, FieldsProducer> producers = new HashMap<Codec, FieldsProducer>();
       boolean success = false;
@@ -111,7 +112,7 @@
             Codec codec = segmentCodecs.codecs[fi.getCodecId()];
             if (!producers.containsKey(codec)) {
               producers.put(codec, codec.fieldsProducer(new SegmentReadState(dir,
-                                                                             si, fieldInfos, readBufferSize, indexDivisor, ""+fi.getCodecId())));
+                                                                             si, fieldInfos, new IOContext(Context.Default), indexDivisor, ""+fi.getCodecId())));
             }
             codecs.put(fi.name, producers.get(codec));
           }
@@ -208,7 +209,7 @@
   public FieldsProducer fieldsProducer(SegmentReadState state)
       throws IOException {
     return new FieldsReader(state.dir, state.fieldInfos, state.segmentInfo,
-        state.readBufferSize, state.termsIndexDivisor);
+        state.context, state.termsIndexDivisor);
   }
 
   @Override
Index: lucene/src/java/org/apache/lucene/index/codecs/FixedGapTermsIndexReader.java
===================================================================
--- lucene/src/java/org/apache/lucene/index/codecs/FixedGapTermsIndexReader.java	(revision 1128825)
+++ lucene/src/java/org/apache/lucene/index/codecs/FixedGapTermsIndexReader.java	(working copy)
@@ -21,6 +21,8 @@
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.index.FieldInfos;
 import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.IOContext;
+import org.apache.lucene.index.IOContext.Context;
 import org.apache.lucene.index.SegmentInfo;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.CodecUtil;
@@ -72,7 +74,7 @@
 
     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), new IOContext(Context.Default));
     
     boolean success = false;
 
Index: lucene/src/java/org/apache/lucene/index/codecs/standard/StandardCodec.java
===================================================================
--- lucene/src/java/org/apache/lucene/index/codecs/standard/StandardCodec.java	(revision 1128825)
+++ lucene/src/java/org/apache/lucene/index/codecs/standard/StandardCodec.java	(working copy)
@@ -83,7 +83,7 @@
 
   @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;
@@ -107,7 +107,7 @@
                                                 state.fieldInfos,
                                                 state.segmentInfo.name,
                                                 postings,
-                                                state.readBufferSize,
+                                                state.context,
                                                 TERMS_CACHE_SIZE,
                                                 state.codecId);
       success = true;
Index: lucene/src/java/org/apache/lucene/index/codecs/standard/StandardPostingsReader.java
===================================================================
--- lucene/src/java/org/apache/lucene/index/codecs/standard/StandardPostingsReader.java	(revision 1128825)
+++ lucene/src/java/org/apache/lucene/index/codecs/standard/StandardPostingsReader.java	(working copy)
@@ -23,6 +23,7 @@
 import org.apache.lucene.index.DocsAndPositionsEnum;
 import org.apache.lucene.index.DocsEnum;
 import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.IOContext;
 import org.apache.lucene.index.IndexFileNames;
 import org.apache.lucene.index.SegmentInfo;
 import org.apache.lucene.index.TermState;
@@ -51,15 +52,15 @@
 
   //private String segment;
 
-  public StandardPostingsReader(Directory dir, SegmentInfo segmentInfo, int readBufferSize, String codecId) throws IOException {
+  public StandardPostingsReader(Directory dir, SegmentInfo segmentInfo, IOContext context, String 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) {
Index: lucene/src/java/org/apache/lucene/index/codecs/VariableGapTermsIndexWriter.java
===================================================================
--- lucene/src/java/org/apache/lucene/index/codecs/VariableGapTermsIndexWriter.java	(revision 1128825)
+++ lucene/src/java/org/apache/lucene/index/codecs/VariableGapTermsIndexWriter.java	(working copy)
@@ -158,7 +158,7 @@
 
   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);
     fieldInfos = state.fieldInfos;
     this.policy = policy;
     writeHeader(out);
Index: lucene/src/java/org/apache/lucene/index/IndexWriter.java
===================================================================
--- lucene/src/java/org/apache/lucene/index/IndexWriter.java	(revision 1128825)
+++ lucene/src/java/org/apache/lucene/index/IndexWriter.java	(working copy)
@@ -39,7 +39,9 @@
 import org.apache.lucene.document.Document;
 import org.apache.lucene.index.DocumentsWriterPerThread.FlushedSegment;
 import org.apache.lucene.index.FieldInfos.FieldNumberBiMap;
+import org.apache.lucene.index.IOContext.Context;
 import org.apache.lucene.index.IndexWriterConfig.OpenMode;
+import org.apache.lucene.index.MergePolicy.OneMerge;
 import org.apache.lucene.index.PayloadProcessorProvider.DirPayloadProcessor;
 import org.apache.lucene.index.SegmentCodecs.SegmentCodecsBuilder;
 import org.apache.lucene.index.codecs.CodecProvider;
@@ -591,8 +593,8 @@
      * enrolled in the pool, so you should simply close()
      * it when you're done (ie, do not call release()).
      */
-    public synchronized SegmentReader getReadOnlyClone(SegmentInfo info, boolean doOpenStores, int termInfosIndexDivisor) throws IOException {
-      SegmentReader sr = get(info, doOpenStores, BufferedIndexInput.BUFFER_SIZE, termInfosIndexDivisor);
+    public synchronized SegmentReader getReadOnlyClone(SegmentInfo info, boolean doOpenStores, int termInfosIndexDivisor, IOContext context) throws IOException {
+      SegmentReader sr = get(info, doOpenStores, context, termInfosIndexDivisor);
       try {
         return (SegmentReader) sr.clone(true);
       } finally {
@@ -608,8 +610,8 @@
      * @param doOpenStores
      * @throws IOException
      */
-    public synchronized SegmentReader get(SegmentInfo info, boolean doOpenStores) throws IOException {
-      return get(info, doOpenStores, BufferedIndexInput.BUFFER_SIZE, config.getReaderTermsIndexDivisor());
+    public synchronized SegmentReader get(SegmentInfo info, boolean doOpenStores, IOContext context) throws IOException {
+      return get(info, doOpenStores, context, config.getReaderTermsIndexDivisor());
     }
 
     /**
@@ -623,18 +625,20 @@
      * @param termsIndexDivisor
      * @throws IOException
      */
-    public synchronized SegmentReader get(SegmentInfo info, boolean doOpenStores, int readBufferSize, int termsIndexDivisor) throws IOException {
+    public synchronized SegmentReader get(SegmentInfo info, boolean doOpenStores, IOContext context, int termsIndexDivisor) throws IOException {
 
-      if (poolReaders) {
-        readBufferSize = BufferedIndexInput.BUFFER_SIZE;
-      }
+      //      if (poolReaders) {
+      //        readBufferSize = BufferedIndexInput.BUFFER_SIZE;
+      //      }
+      
+      // nocommit context should be part of the key used to cache that reader in the pool.
 
       SegmentReader sr = readerMap.get(info);
       if (sr == null) {
         // TODO: we may want to avoid doing this while
         // synchronized
         // Returns a ref, which we xfer to readerMap:
-        sr = SegmentReader.get(false, info.dir, info, readBufferSize, doOpenStores, termsIndexDivisor);
+        sr = SegmentReader.get(false, info.dir, info, doOpenStores, termsIndexDivisor);
         sr.readerFinishedListeners = readerFinishedListeners;
 
         if (info.dir == directory) {
@@ -2179,7 +2183,7 @@
         String compoundFileName = IndexFileNames.segmentFileName(newSegment.name, "", IndexFileNames.COMPOUND_FILE_EXTENSION);
         message("creating compound file " + compoundFileName);
         // Now build compound file
-        CompoundFileWriter cfsWriter = new CompoundFileWriter(directory, compoundFileName);
+        CompoundFileWriter cfsWriter = new CompoundFileWriter(directory, compoundFileName, new IOContext(Context.Default));
         for(String fileName : newSegment.files()) {
           cfsWriter.addFile(fileName);
         }
@@ -2381,10 +2385,12 @@
                 && versionComparator.compare(info.getVersion(), "3.1") >= 0;
           }
 
+          IOContext context=new IOContext(Context.Merge);
+          
           if (createCFS) {
-            copySegmentIntoCFS(info, newSegName);
+            copySegmentIntoCFS(info, newSegName, context);
           } else {
-            copySegmentAsIs(info, newSegName, dsNames, dsFilesCopied);
+            copySegmentAsIs(info, newSegName, dsNames, dsFilesCopied, context);
           }
 
           infos.add(info);
@@ -2476,10 +2482,10 @@
   }
 
   /** Copies the segment into the IndexWriter's directory, as a compound segment. */
-  private void copySegmentIntoCFS(SegmentInfo info, String segName) throws IOException {
+  private void copySegmentIntoCFS(SegmentInfo info, String segName, IOContext context) throws IOException {
     String segFileName = IndexFileNames.segmentFileName(segName, "", IndexFileNames.COMPOUND_FILE_EXTENSION);
     Collection<String> files = info.files();
-    CompoundFileWriter cfsWriter = new CompoundFileWriter(directory, segFileName);
+    CompoundFileWriter cfsWriter = new CompoundFileWriter(directory, segFileName, context);
     for (String file : files) {
       String newFileName = segName + IndexFileNames.stripSegmentName(file);
       if (!IndexFileNames.matchesExtension(file, IndexFileNames.DELETES_EXTENSION)
@@ -2487,7 +2493,7 @@
         cfsWriter.addFile(file, info.dir);
       } else {
         assert !directory.fileExists(newFileName): "file \"" + newFileName + "\" already exists";
-        info.dir.copy(directory, file, newFileName);
+        info.dir.copy(directory, file, newFileName, context);
       }
     }
     
@@ -2501,7 +2507,7 @@
   
   /** Copies the segment files as-is into the IndexWriter's directory. */
   private void copySegmentAsIs(SegmentInfo info, String segName,
-      Map<String, String> dsNames, Set<String> dsFilesCopied)
+      Map<String, String> dsNames, Set<String> dsFilesCopied, IOContext context)
       throws IOException {
     // Determine if the doc store of this segment needs to be copied. It's
     // only relevant for segments that share doc store with others,
@@ -2537,7 +2543,7 @@
       }
       
       assert !directory.fileExists(newFileName): "file \"" + newFileName + "\" already exists";
-      info.dir.copy(directory, file, newFileName);
+      info.dir.copy(directory, file, newFileName, context);
     }
     
     info.setDocStore(info.getDocStoreOffset(), newDsName, info.getDocStoreIsCompoundFile());
@@ -3412,7 +3418,7 @@
         // Hold onto the "live" reader; we will use this to
         // commit merged deletes
         final SegmentReader reader = readerPool.get(info, true,
-                                                    MERGE_READ_BUFFER_SIZE,
+                                                    new IOContext(Context.Merge),
                                                     -config.getReaderTermsIndexDivisor());
         merge.readers.add(reader);
 
@@ -3536,7 +3542,7 @@
       // keep deletes (it's costly to open entire reader
       // when we just need deletes)
 
-      final SegmentReader mergedReader = readerPool.get(merge.info, loadDocStores, BufferedIndexInput.BUFFER_SIZE, termsIndexDivisor);
+      final SegmentReader mergedReader = readerPool.get(merge.info, loadDocStores, new IOContext(Context.Default), termsIndexDivisor);
       try {
         if (poolReaders && mergedSegmentWarmer != null) {
           mergedSegmentWarmer.warm(mergedReader);
Index: lucene/src/java/org/apache/lucene/index/CompoundFileReader.java
===================================================================
--- lucene/src/java/org/apache/lucene/index/CompoundFileReader.java	(revision 1128825)
+++ lucene/src/java/org/apache/lucene/index/CompoundFileReader.java	(working copy)
@@ -37,8 +37,6 @@
  */
 public class CompoundFileReader extends Directory {
 
-    private int readBufferSize;
-
     private static final class FileEntry {
         long offset;
         long length;
@@ -52,20 +50,14 @@
     private IndexInput stream;
     private HashMap<String,FileEntry> entries = new HashMap<String,FileEntry>();
 
-
-  public CompoundFileReader(Directory dir, String name) throws IOException {
-    this(dir, name, BufferedIndexInput.BUFFER_SIZE);
-  }
-
-  public CompoundFileReader(Directory dir, String name, int readBufferSize) throws IOException {
+  public CompoundFileReader(Directory dir, String name, IOContext context) throws IOException {
         directory = dir;
         fileName = name;
-        this.readBufferSize = readBufferSize;
 
         boolean success = false;
 
         try {
-            stream = dir.openInput(name, readBufferSize);
+            stream = dir.openInput(name, context);
 
             // read the first VInt. If it is negative, it's the version number
             // otherwise it's the count (pre-3.1 indexes)
@@ -143,17 +135,9 @@
     }
 
     @Override
-    public synchronized IndexInput openInput(String id)
+    public synchronized IndexInput openInput(String id, IOContext context)
     throws IOException
     {
-      // Default to readBufferSize passed in when we were opened
-      return openInput(id, readBufferSize);
-    }
-
-    @Override
-    public synchronized IndexInput openInput(String id, int readBufferSize)
-    throws IOException
-    {
         if (stream == null)
             throw new IOException("Stream closed");
         
@@ -162,7 +146,7 @@
         if (entry == null)
           throw new IOException("No sub-file with id " + id + " found (files: " + entries.keySet() + ")");
 
-        return new CSIndexInput(stream, entry.offset, entry.length, readBufferSize);
+        return new CSIndexInput(stream, entry.offset, entry.length, BufferedIndexInput.BUFFER_SIZE);
     }
 
     /** Returns an array of strings, one for each file in the directory. */
@@ -217,7 +201,7 @@
     /** Not implemented
      * @throws UnsupportedOperationException */
     @Override
-    public IndexOutput createOutput(String name)
+    public IndexOutput createOutput(String name, IOContext context)
     {
         throw new UnsupportedOperationException();
     }
Index: lucene/src/java/org/apache/lucene/store/Directory.java
===================================================================
--- lucene/src/java/org/apache/lucene/store/Directory.java	(revision 1128825)
+++ lucene/src/java/org/apache/lucene/store/Directory.java	(working copy)
@@ -22,6 +22,7 @@
 import java.io.Closeable;
 import java.util.Collection; // for javadocs
 
+import org.apache.lucene.index.IOContext;
 import org.apache.lucene.util.IOUtils;
 
 /** A Directory is a flat list of files.  Files may be written once, when they
@@ -87,7 +88,7 @@
 
   /** Creates a new, empty file in the directory with the given name.
       Returns a stream writing this file. */
-  public abstract IndexOutput createOutput(String name)
+  public abstract IndexOutput createOutput(String name, IOContext context)
        throws IOException;
 
   /**
@@ -103,10 +104,6 @@
    */
   public abstract void sync(Collection<String> names) throws IOException;
 
-  /** Returns a stream reading an existing file. */
-  public abstract IndexInput openInput(String name)
-    throws IOException;
-
   /** Returns a stream reading an existing file, with the
    * specified read buffer size.  The particular Directory
    * implementation may ignore the buffer size.  Currently
@@ -114,8 +111,8 @@
    * parameter are {@link FSDirectory} and {@link
    * org.apache.lucene.index.CompoundFileReader}.
   */
-  public IndexInput openInput(String name, int bufferSize) throws IOException {
-    return openInput(name);
+  public IndexInput openInput(String name, IOContext context) throws IOException {
+    return openInput(name, context);
   }
 
   /** Construct a {@link Lock}.
@@ -199,9 +196,9 @@
    * <b>NOTE:</b> this method does not check whether <i>dest<i> exist and will
    * overwrite it if it does.
    */
-  public void copy(Directory to, String src, String dest) throws IOException {
-    IndexOutput os = to.createOutput(dest);
-    IndexInput is = openInput(src);
+  public void copy(Directory to, String src, String dest, IOContext context) throws IOException {
+    IndexOutput os = to.createOutput(dest, context);
+    IndexInput is = openInput(src, context);
     IOException priorException = null;
     try {
       is.copyBytes(os, is.length());
Index: lucene/src/java/org/apache/lucene/index/TermVectorsWriter.java
===================================================================
--- lucene/src/java/org/apache/lucene/index/TermVectorsWriter.java	(revision 1128825)
+++ lucene/src/java/org/apache/lucene/index/TermVectorsWriter.java	(working copy)
@@ -17,6 +17,7 @@
  * limitations under the License.
  */
 
+import org.apache.lucene.index.IOContext.Context;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IndexOutput;
 import org.apache.lucene.util.BytesRef;
@@ -33,12 +34,14 @@
   public TermVectorsWriter(Directory directory, String segment,
                            FieldInfos fieldInfos)
     throws IOException {
+    
+    IOContext context = new IOContext(Context.Merge);
     // 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);
 
     this.fieldInfos = fieldInfos;
Index: lucene/src/java/org/apache/lucene/index/DirectoryReader.java
===================================================================
--- lucene/src/java/org/apache/lucene/index/DirectoryReader.java	(revision 1128825)
+++ lucene/src/java/org/apache/lucene/index/DirectoryReader.java	(working copy)
@@ -34,6 +34,7 @@
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.Lock;
 import org.apache.lucene.store.LockObtainFailedException;
+import org.apache.lucene.index.IOContext.Context;
 import org.apache.lucene.index.codecs.CodecProvider;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
@@ -169,7 +170,8 @@
       try {
         final SegmentInfo info = infos.info(i);
         assert info.dir == dir;
-        final SegmentReader reader = writer.readerPool.getReadOnlyClone(info, true, termInfosIndexDivisor);
+        final SegmentReader reader = writer.readerPool.getReadOnlyClone(info, true, termInfosIndexDivisor,
+                                                                        new IOContext(Context.Read));
         if (reader.numDocs() > 0 || writer.getKeepFullyDeletedSegments()) {
           reader.readerFinishedListeners = readerFinishedListeners;
           readers.add(reader);
Index: lucene/src/java/org/apache/lucene/index/SegmentReadState.java
===================================================================
--- lucene/src/java/org/apache/lucene/index/SegmentReadState.java	(revision 1128825)
+++ lucene/src/java/org/apache/lucene/index/SegmentReadState.java	(working copy)
@@ -26,7 +26,7 @@
   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 +37,20 @@
   public final String codecId;
 
   public SegmentReadState(Directory dir, SegmentInfo info,
-      FieldInfos fieldInfos, int readBufferSize, int termsIndexDivisor) {
-    this(dir, info, fieldInfos, readBufferSize, termsIndexDivisor, "");
+      FieldInfos fieldInfos, IOContext context, int termsIndexDivisor) {
+    this(dir, info, fieldInfos, context, termsIndexDivisor, "");
   }
   
   public SegmentReadState(Directory dir,
                           SegmentInfo info,
                           FieldInfos fieldInfos,
-                          int readBufferSize,
+                          IOContext context,
                           int termsIndexDivisor,
                           String codecId) {
     this.dir = dir;
     this.segmentInfo = info;
     this.fieldInfos = fieldInfos;
-    this.readBufferSize = readBufferSize;
+    this.context = context;
     this.termsIndexDivisor = termsIndexDivisor;
     this.codecId = codecId;
   }
Index: lucene/src/java/org/apache/lucene/index/codecs/FixedGapTermsIndexWriter.java
===================================================================
--- lucene/src/java/org/apache/lucene/index/codecs/FixedGapTermsIndexWriter.java	(revision 1128825)
+++ lucene/src/java/org/apache/lucene/index/codecs/FixedGapTermsIndexWriter.java	(working copy)
@@ -57,7 +57,7 @@
   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);
     fieldInfos = state.fieldInfos;
     writeHeader(out);
     out.writeInt(termIndexInterval);
Index: lucene/src/java/org/apache/lucene/index/codecs/standard/StandardPostingsWriter.java
===================================================================
--- lucene/src/java/org/apache/lucene/index/codecs/standard/StandardPostingsWriter.java	(revision 1128825)
+++ lucene/src/java/org/apache/lucene/index/codecs/standard/StandardPostingsWriter.java	(working copy)
@@ -91,13 +91,13 @@
     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);
 
     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;
Index: lucene/src/java/org/apache/lucene/index/FieldsReader.java
===================================================================
--- lucene/src/java/org/apache/lucene/index/FieldsReader.java	(revision 1128825)
+++ lucene/src/java/org/apache/lucene/index/FieldsReader.java	(working copy)
@@ -25,6 +25,8 @@
 import org.apache.lucene.document.FieldSelectorResult;
 import org.apache.lucene.document.Fieldable;
 import org.apache.lucene.document.NumericField;
+import org.apache.lucene.index.IOContext.Context;
+import org.apache.lucene.index.MergePolicy.OneMerge;
 import org.apache.lucene.store.AlreadyClosedException;
 import org.apache.lucene.store.BufferedIndexInput;
 import org.apache.lucene.store.Directory;
@@ -81,7 +83,8 @@
   /** Verifies that the code version which wrote the segment is supported. */
   public static void checkCodeVersion(Directory dir, String segment) throws IOException {
     final String indexStreamFN = IndexFileNames.segmentFileName(segment, "", IndexFileNames.FIELDS_INDEX_EXTENSION);
-    IndexInput idxStream = dir.openInput(indexStreamFN, 1024);
+    final IOContext context = new IOContext(Context.Default);
+    IndexInput idxStream = dir.openInput(indexStreamFN, context);
     
     try {
       int format = idxStream.readInt();
@@ -110,18 +113,18 @@
   }
   
   public FieldsReader(Directory d, String segment, FieldInfos fn) throws IOException {
-    this(d, segment, fn, BufferedIndexInput.BUFFER_SIZE, -1, 0);
+    this(d, segment, fn, new IOContext(Context.Default), -1, 0);
   }
 
-  public FieldsReader(Directory d, String segment, FieldInfos fn, int readBufferSize, int docStoreOffset, int size) throws IOException {
+  public FieldsReader(Directory d, String segment, FieldInfos fn, IOContext context, int docStoreOffset, int size) throws IOException {
     boolean success = false;
     isOriginal = true;
     try {
       fieldInfos = fn;
 
-      cloneableFieldsStream = d.openInput(IndexFileNames.segmentFileName(segment, "", IndexFileNames.FIELDS_EXTENSION), readBufferSize);
+      cloneableFieldsStream = d.openInput(IndexFileNames.segmentFileName(segment, "", IndexFileNames.FIELDS_EXTENSION), context);
       final String indexStreamFN = IndexFileNames.segmentFileName(segment, "", IndexFileNames.FIELDS_INDEX_EXTENSION);
-      cloneableIndexStream = d.openInput(indexStreamFN, readBufferSize);
+      cloneableIndexStream = d.openInput(indexStreamFN, context);
       
       format = cloneableIndexStream.readInt();
 
Index: lucene/src/java/org/apache/lucene/index/codecs/preflex/PreFlexCodec.java
===================================================================
--- lucene/src/java/org/apache/lucene/index/codecs/preflex/PreFlexCodec.java	(revision 1128825)
+++ lucene/src/java/org/apache/lucene/index/codecs/preflex/PreFlexCodec.java	(working copy)
@@ -62,7 +62,7 @@
 
   @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
Index: lucene/src/java/org/apache/lucene/store/NIOFSDirectory.java
===================================================================
--- lucene/src/java/org/apache/lucene/store/NIOFSDirectory.java	(revision 1128825)
+++ lucene/src/java/org/apache/lucene/store/NIOFSDirectory.java	(working copy)
@@ -23,6 +23,7 @@
 import java.nio.channels.ClosedChannelException; // javadoc @link
 import java.nio.channels.FileChannel;
 import java.util.concurrent.Future; // javadoc
+import org.apache.lucene.index.IOContext;
 
 /**
  * An {@link FSDirectory} implementation that uses java.nio's FileChannel's
@@ -73,9 +74,9 @@
 
   /** Creates an IndexInput for the file with the given name. */
   @Override
-  public IndexInput openInput(String name, int bufferSize) throws IOException {
+  public IndexInput openInput(String name, IOContext context) throws IOException {
     ensureOpen();
-    return new NIOFSIndexInput(new File(getDirectory(), name), bufferSize, getReadChunkSize());
+    return new NIOFSIndexInput(new File(getDirectory(), name), context, getReadChunkSize());
   }
 
   protected static class NIOFSIndexInput extends SimpleFSDirectory.SimpleFSIndexInput {
@@ -87,8 +88,8 @@
 
     final FileChannel channel;
 
-    public NIOFSIndexInput(File path, int bufferSize, int chunkSize) throws IOException {
-      super(path, bufferSize, chunkSize);
+    public NIOFSIndexInput(File path, IOContext context, int chunkSize) throws IOException {
+      super(path, context, chunkSize);
       channel = file.getChannel();
     }
 
@@ -178,4 +179,5 @@
       }
     }
   }
+
 }
Index: lucene/src/java/org/apache/lucene/index/codecs/sep/SepPostingsWriterImpl.java
===================================================================
--- lucene/src/java/org/apache/lucene/index/codecs/sep/SepPostingsWriterImpl.java	(revision 1128825)
+++ lucene/src/java/org/apache/lucene/index/codecs/sep/SepPostingsWriterImpl.java	(working copy)
@@ -125,7 +125,7 @@
 
       // 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);
 
     } else {
       freqOut = null;
@@ -136,7 +136,7 @@
     }
 
     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;
 
