Index: lucene/src/java/org/apache/lucene/index/ByteBlockPool.java
===================================================================
--- lucene/src/java/org/apache/lucene/index/ByteBlockPool.java	(revision 995529)
+++ lucene/src/java/org/apache/lucene/index/ByteBlockPool.java	(working copy)
@@ -33,11 +33,13 @@
  * its length and instead allocate a new slice once they
  * hit a non-zero byte. */
 
+import static org.apache.lucene.util.RamUsageEstimator.NUM_BYTES_OBJECT_REF;
+
 import java.util.Arrays;
-import org.apache.lucene.util.BytesRef;
 import java.util.List;
-import static org.apache.lucene.util.RamUsageEstimator.NUM_BYTES_OBJECT_REF;
+
 import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.BytesRef;
 
 final class ByteBlockPool {
 
@@ -47,9 +49,10 @@
     abstract byte[] getByteBlock();
   }
 
-  public byte[][] buffers = new byte[10][];
-
-  int bufferUpto = -1;                        // Which buffer we are upto
+  private byte[][] buffers = new byte[10][];
+  private boolean[] copyOnWrite = new boolean[10];
+  
+  public int bufferUpto = -1;                        // Which buffer we are upto
   public int byteUpto = DocumentsWriterRAMAllocator.BYTE_BLOCK_SIZE;             // Where we are in head buffer
 
   public byte[] buffer;                              // Current head buffer
@@ -60,8 +63,47 @@
   public ByteBlockPool(Allocator allocator) {
     this.allocator = allocator;
   }
-
-  public void reset() {
+  
+  public synchronized byte[] getBuffer(int i) {
+    return buffers[i];
+  }
+  
+  public synchronized byte[][] getBuffers() {
+    return buffers;
+  }
+  
+  public BytesReadOnly getBytesReadOnly(BytesReadOnly read) {
+    if (read == null) {
+      read = new BytesReadOnly();
+    }
+    if (read.buffers == null || buffers.length != read.buffers.length) {
+      read.buffers = new byte[buffers.length][];
+    }
+    System.arraycopy(buffers, 0, read.buffers, 0, buffers.length);
+    copyOnWrite = new boolean[buffers.length];
+    for (int x=0; x < buffers.length; x++) {
+      if (read.buffers[x] == buffers[x]) {
+        copyOnWrite[x] = true;
+      } else {
+        copyOnWrite[x] = false;
+      }
+    }
+    return read;
+  }
+  
+  // make a copy of the byte[] at the given index
+  // only if they are currently the same pointer.
+  // this method *must* be called prior to making 
+  // changes to byte[]s in the buffers array
+  public synchronized void makeCopy(int i) {
+    if (copyOnWrite[i]) {
+      buffers[i] = Arrays.copyOf(buffers[i], buffers[i].length);
+      copyOnWrite[i] = false;
+    }
+  }
+  
+  // nocommit: we cannot recycle buffers that IndexReaders may still be using
+  public synchronized void reset() {
     if (bufferUpto != -1) {
       // We allocated at least one buffer
 
@@ -84,12 +126,17 @@
     }
   }
 
-  public void nextBuffer() {
+  public synchronized void nextBuffer() {
     if (1+bufferUpto == buffers.length) {
       byte[][] newBuffers = new byte[ArrayUtil.oversize(buffers.length+1,
                                                         NUM_BYTES_OBJECT_REF)][];
       System.arraycopy(buffers, 0, newBuffers, 0, buffers.length);
       buffers = newBuffers;
+      
+      // enlarge the copyOnWrite array
+      boolean[] newCopyOnWrite = new boolean[newBuffers.length];
+      System.arraycopy(copyOnWrite, 0, newCopyOnWrite, 0, copyOnWrite.length);
+      copyOnWrite = newCopyOnWrite;
     }
     buffer = buffers[1+bufferUpto] = allocator.getByteBlock();
     bufferUpto++;
@@ -98,7 +145,7 @@
     byteOffset += DocumentsWriterRAMAllocator.BYTE_BLOCK_SIZE;
   }
 
-  public int newSlice(final int size) {
+  public synchronized int newSlice(final int size) {
     if (byteUpto > DocumentsWriterRAMAllocator.BYTE_BLOCK_SIZE-size)
       nextBuffer();
     final int upto = byteUpto;
@@ -116,7 +163,7 @@
   final static int[] levelSizeArray = {5, 14, 20, 30, 40, 40, 80, 80, 120, 200};
   final static int FIRST_LEVEL_SIZE = levelSizeArray[0];
 
-  public int allocSlice(final byte[] slice, final int upto) {
+  public synchronized int allocSlice(final byte[] slice, final int upto) {
 
     final int level = slice[upto] & 15;
     final int newLevel = nextLevelArray[level];
@@ -150,7 +197,7 @@
 
   // Fill in a BytesRef from term's length & bytes encoded in
   // byte block
-  final BytesRef setBytesRef(BytesRef term, int textStart) {
+  final synchronized BytesRef setBytesRef(BytesRef term, int textStart) {
     final byte[] bytes = term.bytes = buffers[textStart >> DocumentsWriterRAMAllocator.BYTE_BLOCK_SHIFT];
     int pos = textStart & DocumentsWriterRAMAllocator.BYTE_BLOCK_MASK;
     if ((bytes[pos] & 0x80) == 0) {
Index: lucene/src/java/org/apache/lucene/index/BytesReadOnly.java
===================================================================
--- lucene/src/java/org/apache/lucene/index/BytesReadOnly.java	(revision 0)
+++ lucene/src/java/org/apache/lucene/index/BytesReadOnly.java	(revision 0)
@@ -0,0 +1,31 @@
+package org.apache.lucene.index;
+
+import org.apache.lucene.util.BytesRef;
+
+public class BytesReadOnly {
+  public byte[][] buffers;
+  
+  BytesReadOnly() {}
+  
+  public BytesReadOnly(byte[][] buffers) {
+    this.buffers = buffers;
+  }
+  
+  //Fill in a BytesRef from term's length & bytes encoded in
+  // byte block
+  final synchronized BytesRef setBytesRef(BytesRef term, int textStart) {
+    final byte[] bytes = term.bytes = buffers[textStart >> DocumentsWriterRAMAllocator.BYTE_BLOCK_SHIFT];
+    int pos = textStart & DocumentsWriterRAMAllocator.BYTE_BLOCK_MASK;
+    if ((bytes[pos] & 0x80) == 0) {
+      // length is 1 byte
+      term.length = bytes[pos];
+      term.offset = pos+1;
+    } else {
+      // length is 2 bytes
+      term.length = (bytes[pos]&0x7f) + ((bytes[pos+1]&0xff)<<7);
+      term.offset = pos+2;
+    }
+    assert term.length >= 0;
+    return term;
+  }
+}
Index: lucene/src/java/org/apache/lucene/index/ByteSliceReader.java
===================================================================
--- lucene/src/java/org/apache/lucene/index/ByteSliceReader.java	(revision 995529)
+++ lucene/src/java/org/apache/lucene/index/ByteSliceReader.java	(working copy)
@@ -37,6 +37,8 @@
   public int bufferOffset;
 
   public int endIndex;
+  
+  byte[][] buffers;
 
   public void init(ByteBlockPool pool, int startIndex, int endIndex) {
 
@@ -50,7 +52,10 @@
     level = 0;
     bufferUpto = startIndex / DocumentsWriterRAMAllocator.BYTE_BLOCK_SIZE;
     bufferOffset = bufferUpto * DocumentsWriterRAMAllocator.BYTE_BLOCK_SIZE;
-    buffer = pool.buffers[bufferUpto];
+    buffers = pool.getBuffers();
+    //buffer = pool.buffers[bufferUpto];
+    buffer = buffers[bufferUpto];
+    //buffer = pool.getBuffer(bufferUpto);
     upto = startIndex & DocumentsWriterRAMAllocator.BYTE_BLOCK_MASK;
 
     final int firstSize = ByteBlockPool.levelSizeArray[0];
@@ -105,7 +110,8 @@
     bufferUpto = nextIndex / DocumentsWriterRAMAllocator.BYTE_BLOCK_SIZE;
     bufferOffset = bufferUpto * DocumentsWriterRAMAllocator.BYTE_BLOCK_SIZE;
 
-    buffer = pool.buffers[bufferUpto];
+    //buffer = pool.buffers[bufferUpto];
+    buffer = buffers[bufferUpto];
     upto = nextIndex & DocumentsWriterRAMAllocator.BYTE_BLOCK_MASK;
 
     if (nextIndex + newSize >= endIndex) {
Index: lucene/src/java/org/apache/lucene/index/ByteSliceReaderOnly.java
===================================================================
--- lucene/src/java/org/apache/lucene/index/ByteSliceReaderOnly.java	(revision 0)
+++ lucene/src/java/org/apache/lucene/index/ByteSliceReaderOnly.java	(revision 0)
@@ -0,0 +1,141 @@
+package org.apache.lucene.index;
+
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import java.io.IOException;
+
+import org.apache.lucene.store.DataInput;
+import org.apache.lucene.store.DataOutput;
+
+/* IndexInput that knows how to read the byte slices written
+ * by Posting and PostingVector.  We read the bytes in
+ * each slice until we hit the end of that slice at which
+ * point we read the forwarding address of the next slice
+ * and then jump to it.*/
+final class ByteSliceReaderOnly extends DataInput {
+  //ByteBlockPool pool;
+  BytesReadOnly pool;
+  int bufferUpto;
+  byte[] buffer;
+  public int upto;
+  int limit;
+  int level;
+  public int bufferOffset;
+
+  public int endIndex;
+
+  public void init(BytesReadOnly pool, int startIndex, int endIndex) {
+
+    assert endIndex-startIndex >= 0;
+    assert startIndex >= 0;
+    assert endIndex >= 0;
+
+    this.pool = pool;
+    this.endIndex = endIndex;
+
+    level = 0;
+    bufferUpto = startIndex / DocumentsWriterRAMAllocator.BYTE_BLOCK_SIZE;
+    bufferOffset = bufferUpto * DocumentsWriterRAMAllocator.BYTE_BLOCK_SIZE;
+    buffer = pool.buffers[bufferUpto];
+    upto = startIndex & DocumentsWriterRAMAllocator.BYTE_BLOCK_MASK;
+
+    final int firstSize = ByteBlockPool.levelSizeArray[0];
+
+    if (startIndex+firstSize >= endIndex) {
+      // There is only this one slice to read
+      limit = endIndex & DocumentsWriterRAMAllocator.BYTE_BLOCK_MASK;
+    } else
+      limit = upto+firstSize-4;
+  }
+
+  public boolean eof() {
+    assert upto + bufferOffset <= endIndex;
+    return upto + bufferOffset == endIndex;
+  }
+
+  @Override
+  public byte readByte() {
+    assert !eof();
+    assert upto <= limit;
+    if (upto == limit)
+      nextSlice();
+    return buffer[upto++];
+  }
+
+  public long writeTo(DataOutput out) throws IOException {
+    long size = 0;
+    while(true) {
+      if (limit + bufferOffset == endIndex) {
+        assert endIndex - bufferOffset >= upto;
+        out.writeBytes(buffer, upto, limit-upto);
+        size += limit-upto;
+        break;
+      } else {
+        out.writeBytes(buffer, upto, limit-upto);
+        size += limit-upto;
+        nextSlice();
+      }
+    }
+
+    return size;
+  }
+
+  public void nextSlice() {
+
+    // Skip to our next slice
+    final int nextIndex = ((buffer[limit]&0xff)<<24) + ((buffer[1+limit]&0xff)<<16) + ((buffer[2+limit]&0xff)<<8) + (buffer[3+limit]&0xff);
+
+    level = ByteBlockPool.nextLevelArray[level];
+    final int newSize = ByteBlockPool.levelSizeArray[level];
+
+    bufferUpto = nextIndex / DocumentsWriterRAMAllocator.BYTE_BLOCK_SIZE;
+    bufferOffset = bufferUpto * DocumentsWriterRAMAllocator.BYTE_BLOCK_SIZE;
+
+    buffer = pool.buffers[bufferUpto];
+    upto = nextIndex & DocumentsWriterRAMAllocator.BYTE_BLOCK_MASK;
+
+    if (nextIndex + newSize >= endIndex) {
+      // We are advancing to the final slice
+      assert endIndex - nextIndex > 0;
+      limit = endIndex - bufferOffset;
+    } else {
+      // This is not the final slice (subtract 4 for the
+      // forwarding address at the end of this new slice)
+      limit = upto+newSize-4;
+    }
+  }
+
+  @Override
+  public void readBytes(byte[] b, int offset, int len) {
+    while(len > 0) {
+      final int numLeft = limit-upto;
+      if (numLeft < len) {
+        // Read entire slice
+        System.arraycopy(buffer, upto, b, offset, numLeft);
+        offset += numLeft;
+        len -= numLeft;
+        nextSlice();
+      } else {
+        // This slice is the last one
+        System.arraycopy(buffer, upto, b, offset, len);
+        upto += len;
+        break;
+      }
+    }
+  }
+}
\ No newline at end of file
Index: lucene/src/java/org/apache/lucene/index/TermsHashPerField.java
===================================================================
--- lucene/src/java/org/apache/lucene/index/TermsHashPerField.java	(revision 995529)
+++ lucene/src/java/org/apache/lucene/index/TermsHashPerField.java	(working copy)
@@ -20,9 +20,11 @@
 import java.io.IOException;
 import java.util.Arrays;
 import java.util.Comparator;
+import java.util.concurrent.atomic.AtomicReference;
 
 import org.apache.lucene.analysis.tokenattributes.TermToBytesRefAttribute;
 import org.apache.lucene.document.Fieldable;
+import org.apache.lucene.index.FreqProxTermsWriterPerField.FreqProxPostingsArray;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.RamUsageEstimator;
 
@@ -55,8 +57,8 @@
   private int[] postingsHash;
  
   ParallelPostingsArray postingsArray;
-  private final BytesRef utf8;
-  private Comparator<BytesRef> termComp;
+  final BytesRef utf8;
+  Comparator<BytesRef> termComp;
 
   public TermsHashPerField(DocInverterPerField docInverterPerField, final TermsHash termsHash, final TermsHash nextTermsHash, final FieldInfo fieldInfo) {
     intPool = termsHash.intPool;
@@ -82,7 +84,23 @@
     else
       nextPerField = null;
   }
-
+  
+  // during the flush, no document additions will be occurring
+  // so there is no need to synchronize
+  public void flush(RAMBuffer buf) {
+    if (postingsArray != null) {
+      // nocommit: we need to make copying the PPA more GC friendly
+      // this probably means associating PPAs with an IndexReader
+      // and/or reusing them
+      //buf.postingsArray = new ParallelPostingsArray(postingsArray.size);
+      buf.postingsArray = new FreqProxPostingsArray(postingsArray.size);
+      postingsArray.copyTo(buf.postingsArray, postingsArray.size);
+    }
+    buf.bytePool = bytePool.getBytesReadOnly(buf.bytePool);
+    buf.termBytePool = termBytePool.getBytesReadOnly(buf.termBytePool);
+    buf.intPool = intPool.getIntsReadOnly(buf.intPool);
+  }
+  
   private void initPostingsArray() {
     postingsArray = consumer.createPostingsArray(2);
     bytesUsed(postingsArray.size * postingsArray.bytesPerPosting());
@@ -145,7 +163,8 @@
   public void initReader(ByteSliceReader reader, int termID, int stream) {
     assert stream < streamCount;
     int intStart = postingsArray.intStarts[termID];
-    final int[] ints = intPool.buffers[intStart >> DocumentsWriterRAMAllocator.INT_BLOCK_SHIFT];
+    //final int[] ints = intPool.buffers[intStart >> DocumentsWriterRAMAllocator.INT_BLOCK_SHIFT];
+    int[] ints = intPool.getBuffer(intStart >> DocumentsWriterRAMAllocator.INT_BLOCK_SHIFT);
     final int upto = intStart & DocumentsWriterRAMAllocator.INT_BLOCK_MASK;
     reader.init(bytePool,
                 postingsArray.byteStarts[termID]+stream*ByteBlockPool.FIRST_LEVEL_SIZE,
@@ -254,9 +273,10 @@
 
   /** Test whether the text for current RawPostingList p equals
    *  current tokenText in utf8. */
-  private boolean postingEquals(final int termID) {
+  private boolean postingEquals(final int termID, byte[][] buffers) {
     final int textStart = postingsArray.textStarts[termID];
-    final byte[] text = termBytePool.buffers[textStart >> DocumentsWriterRAMAllocator.BYTE_BLOCK_SHIFT];
+    //final byte[] text = termBytePool.buffers[textStart >> DocumentsWriterRAMAllocator.BYTE_BLOCK_SHIFT];
+    final byte[] text = buffers[textStart >> DocumentsWriterRAMAllocator.BYTE_BLOCK_SHIFT];
     assert text != null;
 
     int pos = textStart & DocumentsWriterRAMAllocator.BYTE_BLOCK_MASK;
@@ -377,7 +397,8 @@
 
     } else {
       int intStart = postingsArray.intStarts[termID];
-      intUptos = intPool.buffers[intStart >> DocumentsWriterRAMAllocator.INT_BLOCK_SHIFT];
+      //intUptos = intPool.buffers[intStart >> DocumentsWriterRAMAllocator.INT_BLOCK_SHIFT];
+      intUptos = intPool.getBuffer(intStart >> DocumentsWriterRAMAllocator.INT_BLOCK_SHIFT);
       intUptoStart = intStart & DocumentsWriterRAMAllocator.INT_BLOCK_MASK;
       consumer.addTerm(termID);
     }
@@ -399,8 +420,10 @@
 
     // Locate RawPostingList in hash
     int termID = postingsHash[hashPos];
-
-    if (termID != -1 && !postingEquals(termID)) {
+    
+    byte[][] buffers = termBytePool.getBuffers();
+    
+    if (termID != -1 && !postingEquals(termID, buffers)) {
       // Conflict: keep searching different locations in
       // the hash table.
       final int inc = ((code>>8)+code)|1;
@@ -408,7 +431,7 @@
         code += inc;
         hashPos = code & postingsHashMask;
         termID = postingsHash[hashPos];
-      } while (termID != -1 && !postingEquals(termID));
+      } while (termID != -1 && !postingEquals(termID, buffers));
     }
 
     if (termID == -1) {
@@ -505,7 +528,8 @@
 
     } else {
       final int intStart = postingsArray.intStarts[termID];
-      intUptos = intPool.buffers[intStart >> DocumentsWriterRAMAllocator.INT_BLOCK_SHIFT];
+      intUptos = intPool.getBuffer(intStart >> DocumentsWriterRAMAllocator.INT_BLOCK_SHIFT);
+      //intUptos = intPool.buffers[intStart >> DocumentsWriterRAMAllocator.INT_BLOCK_SHIFT];
       intUptoStart = intStart & DocumentsWriterRAMAllocator.INT_BLOCK_MASK;
       consumer.addTerm(termID);
     }
@@ -519,7 +543,8 @@
 
   void writeByte(int stream, byte b) {
     int upto = intUptos[intUptoStart+stream];
-    byte[] bytes = bytePool.buffers[upto >> DocumentsWriterRAMAllocator.BYTE_BLOCK_SHIFT];
+    byte[] bytes = bytePool.getBuffer(upto >> DocumentsWriterRAMAllocator.BYTE_BLOCK_SHIFT);
+    //byte[] bytes = bytePool.buffers[upto >> DocumentsWriterRAMAllocator.BYTE_BLOCK_SHIFT];
     assert bytes != null;
     int offset = upto & DocumentsWriterRAMAllocator.BYTE_BLOCK_MASK;
     if (bytes[offset] != 0) {
@@ -560,7 +585,9 @@
   void rehashPostings(final int newSize) {
 
     final int newMask = newSize-1;
-
+    
+    byte[][] buffers = bytePool.getBuffers();
+    
     int[] newHash = new int[newSize];
     Arrays.fill(newHash, -1);
     for(int i=0;i<postingsHashSize;i++) {
@@ -570,7 +597,8 @@
         if (termsHash.primary) {
           final int textStart = postingsArray.textStarts[termID];
           final int start = textStart & DocumentsWriterRAMAllocator.BYTE_BLOCK_MASK;
-          final byte[] text = bytePool.buffers[textStart >> DocumentsWriterRAMAllocator.BYTE_BLOCK_SHIFT];
+          //final byte[] text = bytePool.buffers[textStart >> DocumentsWriterRAMAllocator.BYTE_BLOCK_SHIFT];
+          final byte[] text = buffers[textStart >> DocumentsWriterRAMAllocator.BYTE_BLOCK_SHIFT];
           code = 0;
 
           final int len;
Index: lucene/src/java/org/apache/lucene/index/ByteSliceWriter.java
===================================================================
--- lucene/src/java/org/apache/lucene/index/ByteSliceWriter.java	(revision 995529)
+++ lucene/src/java/org/apache/lucene/index/ByteSliceWriter.java	(working copy)
@@ -31,18 +31,21 @@
   private byte[] slice;
   private int upto;
   private final ByteBlockPool pool;
+  byte[][] buffers;
 
   int offset0;
 
   public ByteSliceWriter(ByteBlockPool pool) {
     this.pool = pool;
+    buffers = pool.getBuffers();
   }
 
   /**
    * Set up the writer to write at address.
    */
   public void init(int address) {
-    slice = pool.buffers[address >> DocumentsWriterRAMAllocator.BYTE_BLOCK_SHIFT];
+    //slice = pool.buffers[address >> DocumentsWriterRAMAllocator.BYTE_BLOCK_SHIFT];
+    slice = buffers[address >> DocumentsWriterRAMAllocator.BYTE_BLOCK_SHIFT];
     assert slice != null;
     upto = address & DocumentsWriterRAMAllocator.BYTE_BLOCK_MASK;
     offset0 = address;
Index: lucene/src/java/org/apache/lucene/index/FreqProxTermsWriterPerField.java
===================================================================
--- lucene/src/java/org/apache/lucene/index/FreqProxTermsWriterPerField.java	(revision 995529)
+++ lucene/src/java/org/apache/lucene/index/FreqProxTermsWriterPerField.java	(working copy)
@@ -31,7 +31,7 @@
 // codecs; make separate container (tii/tis/skip/*) that can
 // be configured as any number of files 1..N
 final class FreqProxTermsWriterPerField extends TermsHashConsumerPerField implements Comparable<FreqProxTermsWriterPerField> {
-
+  static boolean FLUSH_READONLY_BUFFER = true;
   final FreqProxTermsWriter parent;
   final TermsHashPerField termsHashPerField;
   final FieldInfo fieldInfo;
@@ -223,7 +223,12 @@
    * into a single RAM segment. */
   void flush(FieldsConsumer consumer,  final SegmentWriteState state)
     throws CorruptIndexException, IOException {
-
+    
+    if (FLUSH_READONLY_BUFFER) {
+      flushRAMBuffer(consumer, state);
+      return;
+    }
+    
     final TermsConsumer termsConsumer = consumer.addField(fieldInfo);
     final Comparator<BytesRef> termComp = termsConsumer.getComparator();
 
@@ -233,10 +238,11 @@
     final int numTerms = termsHashPerField.numPostings;
     final BytesRef text = new BytesRef();
     final FreqProxPostingsArray postings = (FreqProxPostingsArray) termsHashPerField.postingsArray;
+    
+    
     final ByteSliceReader freq = new ByteSliceReader();
     final ByteSliceReader prox = new ByteSliceReader();
 
-    
     for (int i = 0; i < numTerms; i++) {
       final int termID = termIDs[i];
       // Get BytesRef
@@ -342,6 +348,133 @@
   
     termsConsumer.finish();
   }
+  
+  public void flushRAMBuffer(FieldsConsumer consumer,  final SegmentWriteState state)
+    throws CorruptIndexException, IOException {
+    final TermsConsumer termsConsumer = consumer.addField(fieldInfo);
+    final Comparator<BytesRef> termComp = termsConsumer.getComparator();
 
+    final boolean currentFieldOmitTermFreqAndPositions = fieldInfo.omitTermFreqAndPositions;
+    
+    final int[] termIDs = termsHashPerField.sortPostings(termComp);
+    final int numTerms = termsHashPerField.numPostings;
+    final BytesRef text = new BytesRef();
+    //final FreqProxPostingsArray postings = (FreqProxPostingsArray) termsHashPerField.postingsArray;
+    
+    RAMBuffer ramBuffer = new RAMBuffer(termsHashPerField);
+    termsHashPerField.flush(ramBuffer);
+    
+    final FreqProxPostingsArray postings = (FreqProxPostingsArray) ramBuffer.postingsArray;
+    
+    final ByteSliceReaderOnly freq = new ByteSliceReaderOnly();
+    final ByteSliceReaderOnly prox = new ByteSliceReaderOnly();
+
+    for (int i = 0; i < numTerms; i++) {
+      final int termID = termIDs[i];
+      // Get BytesRef
+      final int textStart = postings.textStarts[termID];
+      termsHashPerField.bytePool.setBytesRef(text, textStart);
+      
+      //termsHashPerField.initReader(freq, termID, 0);
+      ramBuffer.initReader(freq, termID, 0);
+      if (!fieldInfo.omitTermFreqAndPositions) {
+        //termsHashPerField.initReader(prox, termID, 1);
+        ramBuffer.initReader(prox, termID, 1);
+      }
+  
+      // TODO: really TermsHashPerField should take over most
+      // of this loop, including merge sort of terms from
+      // multiple threads and interacting with the
+      // TermsConsumer, only calling out to us (passing us the
+      // DocsConsumer) to handle delivery of docs/positions
+    
+      final PostingsConsumer postingsConsumer = termsConsumer.startTerm(text);
+  
+      // Now termStates has numToMerge FieldMergeStates
+      // which all share the same term.  Now we must
+      // interleave the docID streams.
+      int numDocs = 0;
+      int docID = 0;
+      int termFreq = 0;
+      
+      while(true) {
+        if (freq.eof()) {
+          if (postings.lastDocCodes[termID] != -1) {
+            // Return last doc
+            docID = postings.lastDocIDs[termID];
+            if (!omitTermFreqAndPositions) {
+              termFreq = postings.docFreqs[termID];
+            }
+            postings.lastDocCodes[termID] = -1;
+          } else {
+            // EOF
+            break;
+          }
+        } else {
+          final int code = freq.readVInt();
+          if (omitTermFreqAndPositions) {
+            docID += code;
+          } else {
+            docID += code >>> 1;
+            if ((code & 1) != 0) {
+              termFreq = 1;
+            } else {
+              termFreq = freq.readVInt();
+            }
+          }
+    
+          assert docID != postings.lastDocIDs[termID];
+        }
+        
+        numDocs++;
+        assert docID < state.numDocs: "doc=" + docID + " maxDoc=" + state.numDocs;
+        final int termDocFreq = termFreq;
+        postingsConsumer.startDoc(docID, termDocFreq);
+    
+        // Carefully copy over the prox + payload info,
+        // changing the format to match Lucene's segment
+        // format.
+        if (!currentFieldOmitTermFreqAndPositions) {
+          // omitTermFreqAndPositions == false so we do write positions &
+          // payload          
+          int position = 0;
+          for(int j=0;j<termDocFreq;j++) {
+            final int code = prox.readVInt();
+            position += code >> 1;
+    
+            final int payloadLength;
+            final BytesRef thisPayload;
+    
+            if ((code & 1) != 0) {
+              // This position has a payload
+              payloadLength = prox.readVInt();  
+              
+              if (payload == null) {
+                payload = new BytesRef();
+                payload.bytes = new byte[payloadLength];
+              } else if (payload.bytes.length < payloadLength) {
+                payload.grow(payloadLength);
+              }
+    
+              prox.readBytes(payload.bytes, 0, payloadLength);
+              payload.length = payloadLength;
+              thisPayload = payload;
+    
+            } else {
+              payloadLength = 0;
+              thisPayload = null;
+            }
+    
+            postingsConsumer.addPosition(position, thisPayload);
+          } 
+    
+          postingsConsumer.finishDoc();
+        }
+      } 
+      termsConsumer.finishTerm(text, numDocs);
+    }
+  
+    termsConsumer.finish();
+  }
 }
 
Index: lucene/src/java/org/apache/lucene/index/RAMBuffer.java
===================================================================
--- lucene/src/java/org/apache/lucene/index/RAMBuffer.java	(revision 0)
+++ lucene/src/java/org/apache/lucene/index/RAMBuffer.java	(revision 0)
@@ -0,0 +1,74 @@
+package org.apache.lucene.index;
+
+import java.util.Comparator;
+
+import org.apache.lucene.util.BytesRef;
+
+public class RAMBuffer {
+  ParallelPostingsArray postingsArray;
+  IntsReadOnly intPool;
+  BytesReadOnly bytePool;
+  BytesReadOnly termBytePool;
+  BytesRef utf8;
+  final BytesRef tr1 = new BytesRef();
+  final BytesRef tr2 = new BytesRef();
+  Comparator<BytesRef> termComp;
+  
+  public RAMBuffer(TermsHashPerField termsHashField) {
+    utf8 = termsHashField.utf8;
+    termComp = termsHashField.termComp;
+  }
+    
+  public void initReader(ByteSliceReaderOnly reader, int termID, int stream) {
+    //assert stream < streamCount;
+    int intStart = postingsArray.intStarts[termID];
+    final int[] ints = intPool.buffers[intStart >> DocumentsWriterRAMAllocator.INT_BLOCK_SHIFT];
+    final int upto = intStart & DocumentsWriterRAMAllocator.INT_BLOCK_MASK;
+    reader.init(bytePool,
+                postingsArray.byteStarts[termID]+stream*ByteBlockPool.FIRST_LEVEL_SIZE,
+                ints[upto+stream]);
+  }
+  
+  /** Compares term text for two term ids and
+   *  returns -1 if p1 < p2; 1 if p1 > p2; else 0. */
+  int comparePostings(int term1, int term2) {
+    if (term1 == term2) {
+      return 0;
+    }
+    termBytePool.setBytesRef(tr1, postingsArray.textStarts[term1]);
+    termBytePool.setBytesRef(tr2, postingsArray.textStarts[term2]);
+
+    return termComp.compare(tr1, tr2);
+  }
+  
+  private boolean postingEquals(final int termID) {
+    final int textStart = postingsArray.textStarts[termID];
+    final byte[] text = termBytePool.buffers[textStart >> DocumentsWriterRAMAllocator.BYTE_BLOCK_SHIFT];
+    assert text != null;
+
+    int pos = textStart & DocumentsWriterRAMAllocator.BYTE_BLOCK_MASK;
+    
+    final int len;
+    if ((text[pos] & 0x80) == 0) {
+      // length is 1 byte
+      len = text[pos];
+      pos += 1;
+    } else {
+      // length is 2 bytes
+      len = (text[pos]&0x7f) + ((text[pos+1]&0xff)<<7);
+      pos += 2;
+    }
+
+    if (len == utf8.length) {
+      final byte[] utf8Bytes = utf8.bytes;
+      for(int tokenPos=0;tokenPos<utf8.length;pos++,tokenPos++) {
+        if (utf8Bytes[tokenPos] != text[pos]) {
+          return false;
+        }
+      }
+      return true;
+    } else {
+      return false;
+    }
+  }
+}
Index: lucene/src/java/org/apache/lucene/index/IntBlockPool.java
===================================================================
--- lucene/src/java/org/apache/lucene/index/IntBlockPool.java	(revision 995529)
+++ lucene/src/java/org/apache/lucene/index/IntBlockPool.java	(working copy)
@@ -19,7 +19,8 @@
 
 final class IntBlockPool {
 
-  public int[][] buffers = new int[10][];
+  private int[][] buffers = new int[10][];
+  private boolean[] copyOnWrite;
 
   int bufferUpto = -1;                        // Which buffer we are upto
   public int intUpto = DocumentsWriterRAMAllocator.INT_BLOCK_SIZE;             // Where we are in head buffer
@@ -32,6 +33,29 @@
   public IntBlockPool(DocumentsWriterPerThread docWriter) {
     this.docWriter = docWriter;
   }
+  
+  public synchronized int[] getBuffer(int i) {
+    return buffers[i];
+  }
+  
+  public IntsReadOnly getIntsReadOnly(IntsReadOnly read) {
+    if (read == null) {
+      read = new IntsReadOnly();
+    }
+    if (read.buffers == null || buffers.length != read.buffers.length) {
+      read.buffers = new int[buffers.length][];
+    }
+    System.arraycopy(buffers, 0, read.buffers, 0, buffers.length);
+    copyOnWrite = new boolean[buffers.length];
+    for (int x=0; x < buffers.length; x++) {
+      if (read.buffers[x] == buffers[x]) {
+        copyOnWrite[x] = true;
+      } else {
+        copyOnWrite[x] = false;
+      }
+    }
+    return read;
+  }
 
   public void reset() {
     if (bufferUpto != -1) {
@@ -52,6 +76,11 @@
       int[][] newBuffers = new int[(int) (buffers.length*1.5)][];
       System.arraycopy(buffers, 0, newBuffers, 0, buffers.length);
       buffers = newBuffers;
+      
+      // enlarge the copyOnWrite array
+      boolean[] newCopyOnWrite = new boolean[newBuffers.length];
+      System.arraycopy(copyOnWrite, 0, newCopyOnWrite, 0, copyOnWrite.length);
+      copyOnWrite = newCopyOnWrite;
     }
     buffer = buffers[1+bufferUpto] = docWriter.ramAllocator.getIntBlock();
     bufferUpto++;
Index: lucene/src/java/org/apache/lucene/index/IntsReadOnly.java
===================================================================
--- lucene/src/java/org/apache/lucene/index/IntsReadOnly.java	(revision 0)
+++ lucene/src/java/org/apache/lucene/index/IntsReadOnly.java	(revision 0)
@@ -0,0 +1,5 @@
+package org.apache.lucene.index;
+
+public class IntsReadOnly {
+  public int[][] buffers;
+}
