You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by dw...@apache.org on 2019/01/15 12:44:25 UTC

[lucene-solr] branch master updated: LUCENE-8474: (partial) removal of accesses to RAMFile and RAMDirectory streams. Removal of GrowableByteArrayDataOutput.

This is an automated email from the ASF dual-hosted git repository.

dweiss pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/lucene-solr.git


The following commit(s) were added to refs/heads/master by this push:
     new d4e016a  LUCENE-8474: (partial) removal of accesses to RAMFile and RAMDirectory streams. Removal of GrowableByteArrayDataOutput.
d4e016a is described below

commit d4e016afdf41baf0104f79e82d953a4650df42aa
Author: Dawid Weiss <dw...@apache.org>
AuthorDate: Tue Jan 15 13:42:25 2019 +0100

    LUCENE-8474: (partial) removal of accesses to RAMFile and RAMDirectory streams. Removal of GrowableByteArrayDataOutput.
---
 .../lucene/codecs/blockterms/BlockTermsWriter.java |  22 +--
 .../blockterms/FixedGapTermsIndexWriter.java       |  23 +--
 .../blocktreeords/OrdsBlockTreeTermsWriter.java    |  36 ++--
 .../lucene/codecs/memory/DirectPostingsFormat.java |  14 +-
 .../lucene/codecs/memory/FSTOrdTermsWriter.java    |  49 +++---
 .../lucene/codecs/memory/FSTTermsWriter.java       |  12 +-
 .../lucene/codecs/MultiLevelSkipListWriter.java    |  19 ++-
 .../codecs/blocktree/BlockTreeTermsWriter.java     |  35 ++--
 .../compressing/CompressingStoredFieldsWriter.java |  36 ++--
 .../compressing/CompressingTermVectorsWriter.java  |  20 ++-
 .../lucene/codecs/lucene50/Lucene50SkipWriter.java |  13 +-
 .../codecs/lucene70/Lucene70DocValuesConsumer.java |  31 ++--
 .../org/apache/lucene/index/PrefixCodedTerms.java  |  69 ++++----
 .../org/apache/lucene/index/SortingLeafReader.java |  29 ++--
 .../lucene/store/GrowableByteArrayDataOutput.java  | 102 -----------
 .../src/java/org/apache/lucene/util/IOUtils.java   |   3 +-
 .../java/org/apache/lucene/util/bkd/BKDWriter.java |  38 ++---
 .../src/java/org/apache/lucene/util/fst/FST.java   |  10 +-
 .../org/apache/lucene/codecs/TestCodecUtil.java    | 102 +++++------
 .../lucene70/TestLucene70DocValuesFormat.java      |  35 ++--
 .../lucene/index/TestIndexWriterOnDiskFull.java    |   2 +-
 .../apache/lucene/search/TermInSetQueryTest.java   |   4 +-
 .../lucene/store/BaseDataOutputTestCase.java       |   1 -
 .../lucene/store/TestByteBuffersDataInput.java     |   3 +-
 .../lucene/store/TestByteBuffersDataOutput.java    |   1 -
 .../store/TestGrowableByteArrayDataOutput.java     |  80 ---------
 .../org/apache/lucene/store/TestHugeRamFile.java   | 102 -----------
 .../org/apache/lucene/store/TestRAMDirectory.java  | 189 ---------------------
 .../apache/lucene/replicator/nrt/PrimaryNode.java  |  24 +--
 .../lucene/replicator/nrt/SimpleTransLog.java      |  15 +-
 .../idversion/VersionBlockTreeTermsWriter.java     |  34 ++--
 .../apache/lucene/store/BaseDirectoryTestCase.java |  26 +--
 .../apache/lucene/store/MockDirectoryWrapper.java  |  47 +----
 .../lucene/store/MockIndexOutputWrapper.java       |   6 +-
 .../org/apache/lucene/util/RamUsageTester.java     |   3 +
 35 files changed, 364 insertions(+), 871 deletions(-)

diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/blockterms/BlockTermsWriter.java b/lucene/codecs/src/java/org/apache/lucene/codecs/blockterms/BlockTermsWriter.java
index 9de9d73..f620bd8 100644
--- a/lucene/codecs/src/java/org/apache/lucene/codecs/blockterms/BlockTermsWriter.java
+++ b/lucene/codecs/src/java/org/apache/lucene/codecs/blockterms/BlockTermsWriter.java
@@ -28,16 +28,16 @@ import org.apache.lucene.codecs.FieldsConsumer;
 import org.apache.lucene.codecs.NormsProducer;
 import org.apache.lucene.codecs.PostingsWriterBase;
 import org.apache.lucene.codecs.TermStats;
-import org.apache.lucene.index.IndexOptions;
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.FieldInfos;
 import org.apache.lucene.index.Fields;
 import org.apache.lucene.index.IndexFileNames;
+import org.apache.lucene.index.IndexOptions;
 import org.apache.lucene.index.SegmentWriteState;
 import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.TermsEnum;
+import org.apache.lucene.store.ByteBuffersDataOutput;
 import org.apache.lucene.store.IndexOutput;
-import org.apache.lucene.store.RAMOutputStream;
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefBuilder;
@@ -306,8 +306,8 @@ public class BlockTermsWriter extends FieldsConsumer implements Closeable {
       return pos1;
     }
 
-    private final RAMOutputStream bytesWriter = new RAMOutputStream();
-    private final RAMOutputStream bufferWriter = new RAMOutputStream();
+    private final ByteBuffersDataOutput bytesWriter = ByteBuffersDataOutput.newResettableInstance();
+    private final ByteBuffersDataOutput bufferWriter = ByteBuffersDataOutput.newResettableInstance();
 
     private void flushBlock() throws IOException {
       //System.out.println("BTW.flushBlock seg=" + segment + " pendingCount=" + pendingCount + " fp=" + out.getFilePointer());
@@ -333,8 +333,8 @@ public class BlockTermsWriter extends FieldsConsumer implements Closeable {
         bytesWriter.writeVInt(suffix);
         bytesWriter.writeBytes(pendingTerms[termCount].term.bytes(), commonPrefix, suffix);
       }
-      out.writeVInt((int) bytesWriter.getFilePointer());
-      bytesWriter.writeTo(out);
+      out.writeVInt(Math.toIntExact(bytesWriter.size()));
+      bytesWriter.copyTo(out);
       bytesWriter.reset();
 
       // 3rd pass: write the freqs as byte[] blob
@@ -348,8 +348,8 @@ public class BlockTermsWriter extends FieldsConsumer implements Closeable {
           bytesWriter.writeVLong(state.totalTermFreq-state.docFreq);
         }
       }
-      out.writeVInt((int) bytesWriter.getFilePointer());
-      bytesWriter.writeTo(out);
+      out.writeVInt(Math.toIntExact(bytesWriter.size()));
+      bytesWriter.copyTo(out);
       bytesWriter.reset();
 
       // 4th pass: write the metadata 
@@ -361,12 +361,12 @@ public class BlockTermsWriter extends FieldsConsumer implements Closeable {
         for (int i = 0; i < longsSize; i++) {
           bytesWriter.writeVLong(longs[i]);
         }
-        bufferWriter.writeTo(bytesWriter);
+        bufferWriter.copyTo(bytesWriter);
         bufferWriter.reset();
         absolute = false;
       }
-      out.writeVInt((int) bytesWriter.getFilePointer());
-      bytesWriter.writeTo(out);
+      out.writeVInt(Math.toIntExact(bytesWriter.size()));
+      bytesWriter.copyTo(out);
       bytesWriter.reset();
 
       lastPrevTerm.copyBytes(pendingTerms[pendingCount-1].term);
diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/blockterms/FixedGapTermsIndexWriter.java b/lucene/codecs/src/java/org/apache/lucene/codecs/blockterms/FixedGapTermsIndexWriter.java
index abb4cb0..7bbded8 100644
--- a/lucene/codecs/src/java/org/apache/lucene/codecs/blockterms/FixedGapTermsIndexWriter.java
+++ b/lucene/codecs/src/java/org/apache/lucene/codecs/blockterms/FixedGapTermsIndexWriter.java
@@ -17,13 +17,17 @@
 package org.apache.lucene.codecs.blockterms;
 
 
-import org.apache.lucene.store.IndexOutput;
-import org.apache.lucene.store.RAMOutputStream;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
 import org.apache.lucene.codecs.CodecUtil;
 import org.apache.lucene.codecs.TermStats;
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.IndexFileNames;
 import org.apache.lucene.index.SegmentWriteState;
+import org.apache.lucene.store.ByteBuffersDataOutput;
+import org.apache.lucene.store.IndexOutput;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefBuilder;
 import org.apache.lucene.util.IOUtils;
@@ -31,10 +35,6 @@ import org.apache.lucene.util.StringHelper;
 import org.apache.lucene.util.packed.MonotonicBlockPackedWriter;
 import org.apache.lucene.util.packed.PackedInts;
 
-import java.util.List;
-import java.util.ArrayList;
-import java.io.IOException;
-
 /**
  * Selects every Nth term as and index term, and hold term
  * bytes (mostly) fully expanded in memory.  This terms index
@@ -111,11 +111,11 @@ public class FixedGapTermsIndexWriter extends TermsIndexWriterBase {
     long packedOffsetsStart;
     private long numTerms;
 
-    private RAMOutputStream offsetsBuffer = new RAMOutputStream();
+    private ByteBuffersDataOutput offsetsBuffer = ByteBuffersDataOutput.newResettableInstance();
     private MonotonicBlockPackedWriter termOffsets = new MonotonicBlockPackedWriter(offsetsBuffer, BLOCKSIZE);
     private long currentOffset;
 
-    private RAMOutputStream addressBuffer = new RAMOutputStream();
+    private ByteBuffersDataOutput addressBuffer = ByteBuffersDataOutput.newResettableInstance();
     private MonotonicBlockPackedWriter termAddresses = new MonotonicBlockPackedWriter(addressBuffer, BLOCKSIZE);
 
     private final BytesRefBuilder lastTerm = new BytesRefBuilder();
@@ -183,18 +183,19 @@ public class FixedGapTermsIndexWriter extends TermsIndexWriterBase {
 
       // relative to our indexStart
       termAddresses.finish();
-      addressBuffer.writeTo(out);
+      addressBuffer.copyTo(out);
 
       packedOffsetsStart = out.getFilePointer();
 
       // write offsets into the byte[] terms
       termOffsets.finish();
-      offsetsBuffer.writeTo(out);
+      offsetsBuffer.copyTo(out);
 
       // our referrer holds onto us, while other fields are
       // being written, so don't tie up this RAM:
       termOffsets = termAddresses = null;
-      addressBuffer = offsetsBuffer = null;
+      addressBuffer = null;
+      offsetsBuffer = null;
     }
   }
 
diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/blocktreeords/OrdsBlockTreeTermsWriter.java b/lucene/codecs/src/java/org/apache/lucene/codecs/blocktreeords/OrdsBlockTreeTermsWriter.java
index bdc0738..769c732 100644
--- a/lucene/codecs/src/java/org/apache/lucene/codecs/blocktreeords/OrdsBlockTreeTermsWriter.java
+++ b/lucene/codecs/src/java/org/apache/lucene/codecs/blocktreeords/OrdsBlockTreeTermsWriter.java
@@ -36,8 +36,8 @@ import org.apache.lucene.index.IndexOptions;
 import org.apache.lucene.index.SegmentWriteState;
 import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.TermsEnum;
+import org.apache.lucene.store.ByteBuffersDataOutput;
 import org.apache.lucene.store.IndexOutput;
-import org.apache.lucene.store.RAMOutputStream;
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefBuilder;
@@ -330,12 +330,12 @@ public final class OrdsBlockTreeTermsWriter extends FieldsConsumer {
       return "BLOCK: " + brToString(prefix);
     }
 
-    public void compileIndex(List<PendingBlock> blocks, RAMOutputStream scratchBytes, IntsRefBuilder scratchIntsRef) throws IOException {
+    public void compileIndex(List<PendingBlock> blocks, ByteBuffersDataOutput scratchBytes, IntsRefBuilder scratchIntsRef) throws IOException {
 
       assert (isFloor && blocks.size() > 1) || (isFloor == false && blocks.size() == 1): "isFloor=" + isFloor + " blocks=" + blocks;
       assert this == blocks.get(0);
 
-      assert scratchBytes.getFilePointer() == 0;
+      assert scratchBytes.size() == 0;
 
       // TODO: try writing the leading vLong in MSB order
       // (opposite of what Lucene does today), for better
@@ -368,10 +368,8 @@ public final class OrdsBlockTreeTermsWriter extends FieldsConsumer {
       //  System.out.println("  compile index for prefix=" + prefix);
       //}
       //indexBuilder.DEBUG = false;
-      final byte[] bytes = new byte[(int) scratchBytes.getFilePointer()];
+      final byte[] bytes = scratchBytes.toArrayCopy();
       assert bytes.length > 0;
-      // System.out.println("  bytes=" + bytes.length);
-      scratchBytes.writeTo(bytes, 0);
       indexBuilder.add(Util.toIntsRef(prefix, scratchIntsRef),
                        FST_OUTPUTS.newOutput(new BytesRef(bytes, 0, bytes.length),
                                              0, Long.MAX_VALUE-(sumTotalTermCount-1)));
@@ -423,7 +421,7 @@ public final class OrdsBlockTreeTermsWriter extends FieldsConsumer {
     }
   }
 
-  private final RAMOutputStream scratchBytes = new RAMOutputStream();
+  private final ByteBuffersDataOutput scratchBytes = ByteBuffersDataOutput.newResettableInstance();
   private final IntsRefBuilder scratchIntsRef = new IntsRefBuilder();
 
   class TermsWriter {
@@ -642,7 +640,7 @@ public final class OrdsBlockTreeTermsWriter extends FieldsConsumer {
             assert longs[pos] >= 0;
             metaWriter.writeVLong(longs[pos]);
           }
-          bytesWriter.writeTo(metaWriter);
+          bytesWriter.copyTo(metaWriter);
           bytesWriter.reset();
           absolute = false;
         }
@@ -693,7 +691,7 @@ public final class OrdsBlockTreeTermsWriter extends FieldsConsumer {
               assert longs[pos] >= 0;
               metaWriter.writeVLong(longs[pos]);
             }
-            bytesWriter.writeTo(metaWriter);
+            bytesWriter.copyTo(metaWriter);
             bytesWriter.reset();
             absolute = false;
 
@@ -738,18 +736,18 @@ public final class OrdsBlockTreeTermsWriter extends FieldsConsumer {
       // search on lookup
 
       // Write suffixes byte[] blob to terms dict output:
-      out.writeVInt((int) (suffixWriter.getFilePointer() << 1) | (isLeafBlock ? 1:0));
-      suffixWriter.writeTo(out);
+      out.writeVInt((int) (suffixWriter.size() << 1) | (isLeafBlock ? 1:0));
+      suffixWriter.copyTo(out);
       suffixWriter.reset();
 
       // Write term stats byte[] blob
-      out.writeVInt((int) statsWriter.getFilePointer());
-      statsWriter.writeTo(out);
+      out.writeVInt((int) statsWriter.size());
+      statsWriter.copyTo(out);
       statsWriter.reset();
 
       // Write term meta data byte[] blob
-      out.writeVInt((int) metaWriter.getFilePointer());
-      metaWriter.writeTo(out);
+      out.writeVInt((int) metaWriter.size());
+      metaWriter.copyTo(out);
       metaWriter.reset();
 
       // if (DEBUG) {
@@ -885,10 +883,10 @@ public final class OrdsBlockTreeTermsWriter extends FieldsConsumer {
       }
     }
 
-    private final RAMOutputStream suffixWriter = new RAMOutputStream();
-    private final RAMOutputStream statsWriter = new RAMOutputStream();
-    private final RAMOutputStream metaWriter = new RAMOutputStream();
-    private final RAMOutputStream bytesWriter = new RAMOutputStream();
+    private final ByteBuffersDataOutput suffixWriter = ByteBuffersDataOutput.newResettableInstance();
+    private final ByteBuffersDataOutput statsWriter = ByteBuffersDataOutput.newResettableInstance();
+    private final ByteBuffersDataOutput metaWriter = ByteBuffersDataOutput.newResettableInstance();
+    private final ByteBuffersDataOutput bytesWriter = ByteBuffersDataOutput.newResettableInstance();
   }
 
   private boolean closed;
diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/memory/DirectPostingsFormat.java b/lucene/codecs/src/java/org/apache/lucene/codecs/memory/DirectPostingsFormat.java
index 6c8853d..15c0025 100644
--- a/lucene/codecs/src/java/org/apache/lucene/codecs/memory/DirectPostingsFormat.java
+++ b/lucene/codecs/src/java/org/apache/lucene/codecs/memory/DirectPostingsFormat.java
@@ -39,8 +39,8 @@ import org.apache.lucene.index.SlowImpactsEnum;
 import org.apache.lucene.index.TermState;
 import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.TermsEnum;
+import org.apache.lucene.store.ByteBuffersDataOutput;
 import org.apache.lucene.store.IOContext;
-import org.apache.lucene.store.RAMOutputStream;
 import org.apache.lucene.util.Accountable;
 import org.apache.lucene.util.Accountables;
 import org.apache.lucene.util.ArrayUtil;
@@ -333,7 +333,7 @@ public final class DirectPostingsFormat extends PostingsFormat {
       final IntArrayWriter scratch = new IntArrayWriter();
 
       // Used for payloads, if any:
-      final RAMOutputStream ros = new RAMOutputStream();
+      final ByteBuffersDataOutput ros = ByteBuffersDataOutput.newResettableInstance();
 
       // if (DEBUG) {
       //   System.out.println("\nLOAD terms seg=" + state.segmentInfo.name + " field=" + field + " hasOffsets=" + hasOffsets + " hasFreq=" + hasFreq + " hasPos=" + hasPos + " hasPayloads=" + hasPayloads);
@@ -374,7 +374,6 @@ public final class DirectPostingsFormat extends PostingsFormat {
         int docID;
 
         if (docFreq <= lowFreqCutoff) {
-
           ros.reset();
 
           // Pack postings for low-freq terms into a single int[]:
@@ -404,14 +403,7 @@ public final class DirectPostingsFormat extends PostingsFormat {
             }
           }
 
-          final byte[] payloads;
-          if (hasPayloads) {
-            payloads = new byte[(int) ros.getFilePointer()];
-            ros.writeTo(payloads, 0);
-          } else {
-            payloads = null;
-          }
-
+          final byte[] payloads = hasPayloads ? ros.toArrayCopy() : null;
           final int[] postings = scratch.get();
 
           ent = new LowFreqTerm(postings, payloads, docFreq, (int) totalTermFreq);
diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTOrdTermsWriter.java b/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTOrdTermsWriter.java
index b59d41d..0077045 100644
--- a/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTOrdTermsWriter.java
+++ b/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTOrdTermsWriter.java
@@ -34,9 +34,9 @@ import org.apache.lucene.index.IndexOptions;
 import org.apache.lucene.index.SegmentWriteState;
 import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.TermsEnum;
+import org.apache.lucene.store.ByteBuffersDataOutput;
 import org.apache.lucene.store.DataOutput;
 import org.apache.lucene.store.IndexOutput;
-import org.apache.lucene.store.RAMOutputStream;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.FixedBitSet;
 import org.apache.lucene.util.IOUtils;
@@ -236,14 +236,14 @@ public class FSTOrdTermsWriter extends FieldsConsumer {
           blockOut.writeVLong(field.sumDocFreq);
           blockOut.writeVInt(field.docCount);
           blockOut.writeVInt(field.longsSize);
-          blockOut.writeVLong(field.statsOut.getFilePointer());
-          blockOut.writeVLong(field.metaLongsOut.getFilePointer());
-          blockOut.writeVLong(field.metaBytesOut.getFilePointer());
+          blockOut.writeVLong(field.statsOut.size());
+          blockOut.writeVLong(field.metaLongsOut.size());
+          blockOut.writeVLong(field.metaBytesOut.size());
           
-          field.skipOut.writeTo(blockOut);
-          field.statsOut.writeTo(blockOut);
-          field.metaLongsOut.writeTo(blockOut);
-          field.metaBytesOut.writeTo(blockOut);
+          field.skipOut.copyTo(blockOut);
+          field.statsOut.copyTo(blockOut);
+          field.metaLongsOut.copyTo(blockOut);
+          field.metaBytesOut.copyTo(blockOut);
           field.dict.save(indexOut);
         }
         writeTrailer(blockOut, blockDirStart);
@@ -277,13 +277,13 @@ public class FSTOrdTermsWriter extends FieldsConsumer {
     // TODO: block encode each part 
 
     // vint encode next skip point (fully decoded when reading)
-    public RAMOutputStream skipOut;
+    public ByteBuffersDataOutput skipOut;
     // vint encode df, (ttf-df)
-    public RAMOutputStream statsOut;
+    public ByteBuffersDataOutput statsOut;
     // vint encode monotonic long[] and length for corresponding byte[]
-    public RAMOutputStream metaLongsOut;
+    public ByteBuffersDataOutput metaLongsOut;
     // generic byte[]
-    public RAMOutputStream metaBytesOut;
+    public ByteBuffersDataOutput metaBytesOut;
   }
 
   final class TermsWriter {
@@ -294,11 +294,10 @@ public class FSTOrdTermsWriter extends FieldsConsumer {
     private long numTerms;
 
     private final IntsRefBuilder scratchTerm = new IntsRefBuilder();
-    private final RAMOutputStream statsOut = new RAMOutputStream();
-    private final RAMOutputStream metaLongsOut = new RAMOutputStream();
-    private final RAMOutputStream metaBytesOut = new RAMOutputStream();
-
-    private final RAMOutputStream skipOut = new RAMOutputStream();
+    private final ByteBuffersDataOutput statsOut = new ByteBuffersDataOutput();
+    private final ByteBuffersDataOutput metaLongsOut = new ByteBuffersDataOutput();
+    private final ByteBuffersDataOutput metaBytesOut = new ByteBuffersDataOutput();
+    private final ByteBuffersDataOutput skipOut = new ByteBuffersDataOutput();
     private long lastBlockStatsFP;
     private long lastBlockMetaLongsFP;
     private long lastBlockMetaBytesFP;
@@ -345,12 +344,12 @@ public class FSTOrdTermsWriter extends FieldsConsumer {
         metaLongsOut.writeVLong(longs[i] - lastLongs[i]);
         lastLongs[i] = longs[i];
       }
-      metaLongsOut.writeVLong(metaBytesOut.getFilePointer() - lastMetaBytesFP);
+      metaLongsOut.writeVLong(metaBytesOut.size() - lastMetaBytesFP);
 
       builder.add(Util.toIntsRef(text, scratchTerm), numTerms);
       numTerms++;
 
-      lastMetaBytesFP = metaBytesOut.getFilePointer();
+      lastMetaBytesFP = metaBytesOut.size();
     }
 
     public void finish(long sumTotalTermFreq, long sumDocFreq, int docCount) throws IOException {
@@ -372,15 +371,15 @@ public class FSTOrdTermsWriter extends FieldsConsumer {
     }
 
     private void bufferSkip() throws IOException {
-      skipOut.writeVLong(statsOut.getFilePointer() - lastBlockStatsFP);
-      skipOut.writeVLong(metaLongsOut.getFilePointer() - lastBlockMetaLongsFP);
-      skipOut.writeVLong(metaBytesOut.getFilePointer() - lastBlockMetaBytesFP);
+      skipOut.writeVLong(statsOut.size() - lastBlockStatsFP);
+      skipOut.writeVLong(metaLongsOut.size() - lastBlockMetaLongsFP);
+      skipOut.writeVLong(metaBytesOut.size() - lastBlockMetaBytesFP);
       for (int i = 0; i < longsSize; i++) {
         skipOut.writeVLong(lastLongs[i] - lastBlockLongs[i]);
       }
-      lastBlockStatsFP = statsOut.getFilePointer();
-      lastBlockMetaLongsFP = metaLongsOut.getFilePointer();
-      lastBlockMetaBytesFP = metaBytesOut.getFilePointer();
+      lastBlockStatsFP = statsOut.size();
+      lastBlockMetaLongsFP = metaLongsOut.size();
+      lastBlockMetaBytesFP = metaBytesOut.size();
       System.arraycopy(lastLongs, 0, lastBlockLongs, 0, longsSize);
     }
   }
diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTTermsWriter.java b/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTTermsWriter.java
index 8e55d41..22eff0b 100644
--- a/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTTermsWriter.java
+++ b/lucene/codecs/src/java/org/apache/lucene/codecs/memory/FSTTermsWriter.java
@@ -26,17 +26,17 @@ import org.apache.lucene.codecs.CodecUtil;
 import org.apache.lucene.codecs.FieldsConsumer;
 import org.apache.lucene.codecs.NormsProducer;
 import org.apache.lucene.codecs.PostingsWriterBase;
-import org.apache.lucene.index.IndexOptions;
 import org.apache.lucene.index.FieldInfo;
 import org.apache.lucene.index.FieldInfos;
 import org.apache.lucene.index.Fields;
 import org.apache.lucene.index.IndexFileNames;
+import org.apache.lucene.index.IndexOptions;
 import org.apache.lucene.index.SegmentWriteState;
 import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.TermsEnum;
+import org.apache.lucene.store.ByteBuffersDataOutput;
 import org.apache.lucene.store.DataOutput;
 import org.apache.lucene.store.IndexOutput;
-import org.apache.lucene.store.RAMOutputStream;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.FixedBitSet;
 import org.apache.lucene.util.IOUtils;
@@ -254,7 +254,7 @@ public class FSTTermsWriter extends FieldsConsumer {
     private long numTerms;
 
     private final IntsRefBuilder scratchTerm = new IntsRefBuilder();
-    private final RAMOutputStream metaWriter = new RAMOutputStream();
+    private final ByteBuffersDataOutput metaWriter = ByteBuffersDataOutput.newResettableInstance();
 
     TermsWriter(FieldInfo fieldInfo) {
       this.numTerms = 0;
@@ -272,10 +272,8 @@ public class FSTTermsWriter extends FieldsConsumer {
       meta.docFreq = state.docFreq;
       meta.totalTermFreq = state.totalTermFreq;
       postingsWriter.encodeTerm(meta.longs, metaWriter, fieldInfo, state, true);
-      final int bytesSize = (int)metaWriter.getFilePointer();
-      if (bytesSize > 0) {
-        meta.bytes = new byte[bytesSize];
-        metaWriter.writeTo(meta.bytes, 0);
+      if (metaWriter.size() > 0) {
+        meta.bytes = metaWriter.toArrayCopy();
         metaWriter.reset();
       }
       builder.add(Util.toIntsRef(text, scratchTerm), meta);
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/MultiLevelSkipListWriter.java b/lucene/core/src/java/org/apache/lucene/codecs/MultiLevelSkipListWriter.java
index 8e090be..7cf04a0 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/MultiLevelSkipListWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/MultiLevelSkipListWriter.java
@@ -19,8 +19,9 @@ package org.apache.lucene.codecs;
 
 import java.io.IOException;
 
+import org.apache.lucene.store.ByteBuffersDataOutput;
+import org.apache.lucene.store.DataOutput;
 import org.apache.lucene.store.IndexOutput;
-import org.apache.lucene.store.RAMOutputStream;
 import org.apache.lucene.util.MathUtil;
 
 /**
@@ -62,7 +63,7 @@ public abstract class MultiLevelSkipListWriter {
   private final int skipMultiplier;
   
   /** for every skip level a different buffer is used  */
-  private RAMOutputStream[] skipBuffer;
+  private ByteBuffersDataOutput[] skipBuffer;
 
   /** Creates a {@code MultiLevelSkipListWriter}. */
   protected MultiLevelSkipListWriter(int skipInterval, int skipMultiplier, int maxSkipLevels, int df) {
@@ -93,9 +94,9 @@ public abstract class MultiLevelSkipListWriter {
 
   /** Allocates internal skip buffers. */
   protected void init() {
-    skipBuffer = new RAMOutputStream[numberOfSkipLevels];
+    skipBuffer = new ByteBuffersDataOutput [numberOfSkipLevels];
     for (int i = 0; i < numberOfSkipLevels; i++) {
-      skipBuffer[i] = new RAMOutputStream();
+      skipBuffer[i] = ByteBuffersDataOutput.newResettableInstance();
     }
   }
 
@@ -116,7 +117,7 @@ public abstract class MultiLevelSkipListWriter {
    * @param level the level skip data shall be writing for
    * @param skipBuffer the skip buffer to write to
    */
-  protected abstract void writeSkipData(int level, IndexOutput skipBuffer) throws IOException;
+  protected abstract void writeSkipData(int level, DataOutput skipBuffer) throws IOException;
 
   /**
    * Writes the current skip data to the buffers. The current document frequency determines
@@ -142,7 +143,7 @@ public abstract class MultiLevelSkipListWriter {
     for (int level = 0; level < numLevels; level++) {
       writeSkipData(level, skipBuffer[level]);
       
-      long newChildPointer = skipBuffer[level].getFilePointer();
+      long newChildPointer = skipBuffer[level].size();
       
       if (level != 0) {
         // store child pointers for all levels except the lowest
@@ -166,13 +167,13 @@ public abstract class MultiLevelSkipListWriter {
     if (skipBuffer == null || skipBuffer.length == 0) return skipPointer;
     
     for (int level = numberOfSkipLevels - 1; level > 0; level--) {
-      long length = skipBuffer[level].getFilePointer();
+      long length = skipBuffer[level].size();
       if (length > 0) {
         output.writeVLong(length);
-        skipBuffer[level].writeTo(output);
+        skipBuffer[level].copyTo(output);
       }
     }
-    skipBuffer[0].writeTo(output);
+    skipBuffer[0].copyTo(output);
     
     return skipPointer;
   }
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/blocktree/BlockTreeTermsWriter.java b/lucene/core/src/java/org/apache/lucene/codecs/blocktree/BlockTreeTermsWriter.java
index ec3f6e6..3059d5a 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/blocktree/BlockTreeTermsWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/blocktree/BlockTreeTermsWriter.java
@@ -34,9 +34,9 @@ import org.apache.lucene.index.IndexOptions;
 import org.apache.lucene.index.SegmentWriteState;
 import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.TermsEnum;
+import org.apache.lucene.store.ByteBuffersDataOutput;
 import org.apache.lucene.store.DataOutput;
 import org.apache.lucene.store.IndexOutput;
-import org.apache.lucene.store.RAMOutputStream;
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefBuilder;
@@ -428,12 +428,12 @@ public final class BlockTreeTermsWriter extends FieldsConsumer {
       return "BLOCK: prefix=" + brToString(prefix);
     }
 
-    public void compileIndex(List<PendingBlock> blocks, RAMOutputStream scratchBytes, IntsRefBuilder scratchIntsRef) throws IOException {
+    public void compileIndex(List<PendingBlock> blocks, ByteBuffersDataOutput scratchBytes, IntsRefBuilder scratchIntsRef) throws IOException {
 
       assert (isFloor && blocks.size() > 1) || (isFloor == false && blocks.size() == 1): "isFloor=" + isFloor + " blocks=" + blocks;
       assert this == blocks.get(0);
 
-      assert scratchBytes.getFilePointer() == 0;
+      assert scratchBytes.size() == 0;
 
       // TODO: try writing the leading vLong in MSB order
       // (opposite of what Lucene does today), for better
@@ -461,9 +461,8 @@ public final class BlockTreeTermsWriter extends FieldsConsumer {
       //  System.out.println("  compile index for prefix=" + prefix);
       //}
       //indexBuilder.DEBUG = false;
-      final byte[] bytes = new byte[(int) scratchBytes.getFilePointer()];
+      final byte[] bytes = scratchBytes.toArrayCopy();
       assert bytes.length > 0;
-      scratchBytes.writeTo(bytes, 0);
       indexBuilder.add(Util.toIntsRef(prefix, scratchIntsRef), new BytesRef(bytes, 0, bytes.length));
       scratchBytes.reset();
 
@@ -504,7 +503,7 @@ public final class BlockTreeTermsWriter extends FieldsConsumer {
     }
   }
 
-  private final RAMOutputStream scratchBytes = new RAMOutputStream();
+  private final ByteBuffersDataOutput scratchBytes = ByteBuffersDataOutput.newResettableInstance();
   private final IntsRefBuilder scratchIntsRef = new IntsRefBuilder();
 
   static final BytesRef EMPTY_BYTES_REF = new BytesRef();
@@ -727,7 +726,7 @@ public final class BlockTreeTermsWriter extends FieldsConsumer {
             assert longs[pos] >= 0;
             metaWriter.writeVLong(longs[pos]);
           }
-          bytesWriter.writeTo(metaWriter);
+          bytesWriter.copyTo(metaWriter);
           bytesWriter.reset();
           absolute = false;
         }
@@ -778,7 +777,7 @@ public final class BlockTreeTermsWriter extends FieldsConsumer {
               assert longs[pos] >= 0;
               metaWriter.writeVLong(longs[pos]);
             }
-            bytesWriter.writeTo(metaWriter);
+            bytesWriter.copyTo(metaWriter);
             bytesWriter.reset();
             absolute = false;
           } else {
@@ -817,18 +816,18 @@ public final class BlockTreeTermsWriter extends FieldsConsumer {
       // search on lookup
 
       // Write suffixes byte[] blob to terms dict output:
-      termsOut.writeVInt((int) (suffixWriter.getFilePointer() << 1) | (isLeafBlock ? 1:0));
-      suffixWriter.writeTo(termsOut);
+      termsOut.writeVInt((int) (suffixWriter.size() << 1) | (isLeafBlock ? 1:0));
+      suffixWriter.copyTo(termsOut);
       suffixWriter.reset();
 
       // Write term stats byte[] blob
-      termsOut.writeVInt((int) statsWriter.getFilePointer());
-      statsWriter.writeTo(termsOut);
+      termsOut.writeVInt((int) statsWriter.size());
+      statsWriter.copyTo(termsOut);
       statsWriter.reset();
 
       // Write term meta data byte[] blob
-      termsOut.writeVInt((int) metaWriter.getFilePointer());
-      metaWriter.writeTo(termsOut);
+      termsOut.writeVInt((int) metaWriter.size());
+      metaWriter.copyTo(termsOut);
       metaWriter.reset();
 
       // if (DEBUG) {
@@ -976,10 +975,10 @@ public final class BlockTreeTermsWriter extends FieldsConsumer {
       }
     }
 
-    private final RAMOutputStream suffixWriter = new RAMOutputStream();
-    private final RAMOutputStream statsWriter = new RAMOutputStream();
-    private final RAMOutputStream metaWriter = new RAMOutputStream();
-    private final RAMOutputStream bytesWriter = new RAMOutputStream();
+    private final ByteBuffersDataOutput suffixWriter = ByteBuffersDataOutput.newResettableInstance();
+    private final ByteBuffersDataOutput statsWriter = ByteBuffersDataOutput.newResettableInstance();
+    private final ByteBuffersDataOutput metaWriter = ByteBuffersDataOutput.newResettableInstance();
+    private final ByteBuffersDataOutput bytesWriter = ByteBuffersDataOutput.newResettableInstance();
   }
 
   private boolean closed;
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsWriter.java b/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsWriter.java
index 5b8eb9e..5d779ab4 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingStoredFieldsWriter.java
@@ -17,6 +17,8 @@
 package org.apache.lucene.codecs.compressing;
 
 
+import static org.apache.lucene.search.DocIdSetIterator.*;
+
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
@@ -33,9 +35,9 @@ import org.apache.lucene.index.IndexFileNames;
 import org.apache.lucene.index.IndexableField;
 import org.apache.lucene.index.MergeState;
 import org.apache.lucene.index.SegmentInfo;
+import org.apache.lucene.store.ByteBuffersDataOutput;
 import org.apache.lucene.store.DataOutput;
 import org.apache.lucene.store.Directory;
-import org.apache.lucene.store.GrowableByteArrayDataOutput;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.store.IndexOutput;
@@ -46,8 +48,6 @@ import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.packed.PackedInts;
 
-import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
-
 /**
  * {@link StoredFieldsWriter} impl for {@link CompressingStoredFieldsFormat}.
  * @lucene.experimental
@@ -84,7 +84,7 @@ public final class CompressingStoredFieldsWriter extends StoredFieldsWriter {
   private final int chunkSize;
   private final int maxDocsPerChunk;
 
-  private final GrowableByteArrayDataOutput bufferedDocs;
+  private final ByteBuffersDataOutput bufferedDocs;
   private int[] numStoredFields; // number of stored fields
   private int[] endOffsets; // end offsets in bufferedDocs
   private int docBase; // doc ID at the beginning of the chunk
@@ -103,7 +103,7 @@ public final class CompressingStoredFieldsWriter extends StoredFieldsWriter {
     this.chunkSize = chunkSize;
     this.maxDocsPerChunk = maxDocsPerChunk;
     this.docBase = 0;
-    this.bufferedDocs = new GrowableByteArrayDataOutput(chunkSize);
+    this.bufferedDocs = ByteBuffersDataOutput.newResettableInstance();
     this.numStoredFields = new int[16];
     this.endOffsets = new int[16];
     this.numBufferedDocs = 0;
@@ -162,7 +162,7 @@ public final class CompressingStoredFieldsWriter extends StoredFieldsWriter {
     }
     this.numStoredFields[numBufferedDocs] = numStoredFieldsInDoc;
     numStoredFieldsInDoc = 0;
-    endOffsets[numBufferedDocs] = bufferedDocs.getPosition();
+    endOffsets[numBufferedDocs] = Math.toIntExact(bufferedDocs.size());
     ++numBufferedDocs;
     if (triggerFlush()) {
       flush();
@@ -215,7 +215,7 @@ public final class CompressingStoredFieldsWriter extends StoredFieldsWriter {
   }
 
   private boolean triggerFlush() {
-    return bufferedDocs.getPosition() >= chunkSize || // chunks of at least chunkSize bytes
+    return bufferedDocs.size() >= chunkSize || // chunks of at least chunkSize bytes
         numBufferedDocs >= maxDocsPerChunk;
   }
 
@@ -228,17 +228,23 @@ public final class CompressingStoredFieldsWriter extends StoredFieldsWriter {
       lengths[i] = endOffsets[i] - endOffsets[i - 1];
       assert lengths[i] >= 0;
     }
-    final boolean sliced = bufferedDocs.getPosition() >= 2 * chunkSize;
+    final boolean sliced = bufferedDocs.size() >= 2 * chunkSize;
     writeHeader(docBase, numBufferedDocs, numStoredFields, lengths, sliced);
 
-    // compress stored fields to fieldsStream
+    // compress stored fields to fieldsStream.
+    //
+    // TODO: do we need to slice it since we already have the slices in the buffer? Perhaps
+    // we should use max-block-bits restriction on the buffer itself, then we won't have to check it here.
+    byte [] content = bufferedDocs.toArrayCopy();
+    bufferedDocs.reset();
+
     if (sliced) {
       // big chunk, slice it
-      for (int compressed = 0; compressed < bufferedDocs.getPosition(); compressed += chunkSize) {
-        compressor.compress(bufferedDocs.getBytes(), compressed, Math.min(chunkSize, bufferedDocs.getPosition() - compressed), fieldsStream);
+      for (int compressed = 0; compressed < content.length; compressed += chunkSize) {
+        compressor.compress(content, compressed, Math.min(chunkSize, content.length - compressed), fieldsStream);
       }
     } else {
-      compressor.compress(bufferedDocs.getBytes(), 0, bufferedDocs.getPosition(), fieldsStream);
+      compressor.compress(content, 0, content.length, fieldsStream);
     }
 
     // reset
@@ -464,7 +470,7 @@ public final class CompressingStoredFieldsWriter extends StoredFieldsWriter {
       flush();
       numDirtyChunks++; // incomplete: we had to force this flush
     } else {
-      assert bufferedDocs.getPosition() == 0;
+      assert bufferedDocs.size() == 0;
     }
     if (docBase != numDocs) {
       throw new RuntimeException("Wrote " + docBase + " docs, finish called with numDocs=" + numDocs);
@@ -473,9 +479,9 @@ public final class CompressingStoredFieldsWriter extends StoredFieldsWriter {
     fieldsStream.writeVLong(numChunks);
     fieldsStream.writeVLong(numDirtyChunks);
     CodecUtil.writeFooter(fieldsStream);
-    assert bufferedDocs.getPosition() == 0;
+    assert bufferedDocs.size() == 0;
   }
-  
+
   // bulk merge is scary: its caused corruption bugs in the past.
   // we try to be extra safe with this impl, but add an escape hatch to
   // have a workaround for undiscovered bugs.
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingTermVectorsWriter.java b/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingTermVectorsWriter.java
index 4f8d004..46ac128 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingTermVectorsWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/compressing/CompressingTermVectorsWriter.java
@@ -35,9 +35,9 @@ import org.apache.lucene.index.Fields;
 import org.apache.lucene.index.IndexFileNames;
 import org.apache.lucene.index.MergeState;
 import org.apache.lucene.index.SegmentInfo;
+import org.apache.lucene.store.ByteBuffersDataOutput;
 import org.apache.lucene.store.DataInput;
 import org.apache.lucene.store.Directory;
-import org.apache.lucene.store.GrowableByteArrayDataOutput;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.store.IndexOutput;
@@ -198,8 +198,8 @@ public final class CompressingTermVectorsWriter extends TermVectorsWriter {
   private FieldData curField; // current field
   private final BytesRef lastTerm;
   private int[] positionsBuf, startOffsetsBuf, lengthsBuf, payloadLengthsBuf;
-  private final GrowableByteArrayDataOutput termSuffixes; // buffered term suffixes
-  private final GrowableByteArrayDataOutput payloadBytes; // buffered term payloads
+  private final ByteBuffersDataOutput termSuffixes; // buffered term suffixes
+  private final ByteBuffersDataOutput payloadBytes; // buffered term payloads
   private final BlockPackedWriter writer;
 
   /** Sole constructor. */
@@ -213,8 +213,8 @@ public final class CompressingTermVectorsWriter extends TermVectorsWriter {
 
     numDocs = 0;
     pendingDocs = new ArrayDeque<>();
-    termSuffixes = new GrowableByteArrayDataOutput(ArrayUtil.oversize(chunkSize, 1));
-    payloadBytes = new GrowableByteArrayDataOutput(ArrayUtil.oversize(1, 1));
+    termSuffixes = ByteBuffersDataOutput.newResettableInstance();
+    payloadBytes = ByteBuffersDataOutput.newResettableInstance();
     lastTerm = new BytesRef(ArrayUtil.oversize(30, 1));
 
     boolean success = false;
@@ -269,7 +269,7 @@ public final class CompressingTermVectorsWriter extends TermVectorsWriter {
   @Override
   public void finishDocument() throws IOException {
     // append the payload bytes of the doc after its terms
-    termSuffixes.writeBytes(payloadBytes.getBytes(), payloadBytes.getPosition());
+    payloadBytes.copyTo(termSuffixes);
     payloadBytes.reset();
     ++numDocs;
     if (triggerFlush()) {
@@ -322,7 +322,7 @@ public final class CompressingTermVectorsWriter extends TermVectorsWriter {
   }
 
   private boolean triggerFlush() {
-    return termSuffixes.getPosition() >= chunkSize
+    return termSuffixes.size() >= chunkSize
         || pendingDocs.size() >= MAX_DOCUMENTS_PER_CHUNK;
   }
 
@@ -361,7 +361,11 @@ public final class CompressingTermVectorsWriter extends TermVectorsWriter {
       flushPayloadLengths();
 
       // compress terms and payloads and write them to the output
-      compressor.compress(termSuffixes.getBytes(), 0, termSuffixes.getPosition(), vectorsStream);
+      //
+      // TODO: We could compress in the slices we already have in the buffer (min/max slice
+      // can be set on the buffer itself).
+      byte[] content = termSuffixes.toArrayCopy();
+      compressor.compress(content, 0, content.length, vectorsStream);
     }
 
     // reset
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50SkipWriter.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50SkipWriter.java
index 37044b5..9b9b4de 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50SkipWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50SkipWriter.java
@@ -25,8 +25,9 @@ import java.util.SortedSet;
 import org.apache.lucene.codecs.CompetitiveImpactAccumulator;
 import org.apache.lucene.codecs.MultiLevelSkipListWriter;
 import org.apache.lucene.index.Impact;
+import org.apache.lucene.store.ByteBuffersDataOutput;
+import org.apache.lucene.store.DataOutput;
 import org.apache.lucene.store.IndexOutput;
-import org.apache.lucene.store.RAMOutputStream;
 
 /**
  * Write skip lists with multiple levels, and support skip within block ints.
@@ -162,10 +163,10 @@ final class Lucene50SkipWriter extends MultiLevelSkipListWriter {
     bufferSkip(numDocs);
   }
 
-  private final RAMOutputStream freqNormOut = new RAMOutputStream();
+  private final ByteBuffersDataOutput freqNormOut = ByteBuffersDataOutput.newResettableInstance();
 
   @Override
-  protected void writeSkipData(int level, IndexOutput skipBuffer) throws IOException {
+  protected void writeSkipData(int level, DataOutput skipBuffer) throws IOException {
 
     int delta = curDoc - lastSkipDoc[level];
 
@@ -197,13 +198,13 @@ final class Lucene50SkipWriter extends MultiLevelSkipListWriter {
       curCompetitiveFreqNorms[level + 1].addAll(competitiveFreqNorms);
     }
     writeImpacts(competitiveFreqNorms, freqNormOut);
-    skipBuffer.writeVInt(Math.toIntExact(freqNormOut.getFilePointer()));
-    freqNormOut.writeTo(skipBuffer);
+    skipBuffer.writeVInt(Math.toIntExact(freqNormOut.size()));
+    freqNormOut.copyTo(skipBuffer);
     freqNormOut.reset();
     competitiveFreqNorms.clear();
   }
 
-  static void writeImpacts(CompetitiveImpactAccumulator acc, IndexOutput out) throws IOException {
+  static void writeImpacts(CompetitiveImpactAccumulator acc, DataOutput out) throws IOException {
     SortedSet<Impact> impacts = acc.getCompetitiveFreqNormPairs();
     Impact previous = new Impact(0, 0);
     for (Impact impact : impacts) {
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene70/Lucene70DocValuesConsumer.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene70/Lucene70DocValuesConsumer.java
index e25e8e7..6db3cca 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene70/Lucene70DocValuesConsumer.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene70/Lucene70DocValuesConsumer.java
@@ -44,9 +44,9 @@ import org.apache.lucene.index.SortedSetDocValues;
 import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.search.SortedSetSelector;
-import org.apache.lucene.store.GrowableByteArrayDataOutput;
+import org.apache.lucene.store.ByteBuffersDataOutput;
+import org.apache.lucene.store.ByteBuffersIndexOutput;
 import org.apache.lucene.store.IndexOutput;
-import org.apache.lucene.store.RAMOutputStream;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefBuilder;
 import org.apache.lucene.util.IOUtils;
@@ -285,7 +285,7 @@ final class Lucene70DocValuesConsumer extends DocValuesConsumer implements Close
  
   private void writeValuesMultipleBlocks(SortedNumericDocValues values, long gcd) throws IOException {
     final long[] buffer = new long[NUMERIC_BLOCK_SIZE];
-    final GrowableByteArrayDataOutput encodeBuffer = new GrowableByteArrayDataOutput(NUMERIC_BLOCK_SIZE);
+    final ByteBuffersDataOutput encodeBuffer = ByteBuffersDataOutput.newResettableInstance();
     int upTo = 0;
     for (int doc = values.nextDoc(); doc != DocIdSetIterator.NO_MORE_DOCS; doc = values.nextDoc()) {
       for (int i = 0, count = values.docValueCount(); i < count; ++i) {
@@ -301,7 +301,7 @@ final class Lucene70DocValuesConsumer extends DocValuesConsumer implements Close
     }
   }
 
-  private void writeBlock(long[] values, int length, long gcd, GrowableByteArrayDataOutput buffer) throws IOException {
+  private void writeBlock(long[] values, int length, long gcd, ByteBuffersDataOutput buffer) throws IOException {
     assert length > 0;
     long min = values[0];
     long max = values[0];
@@ -317,7 +317,7 @@ final class Lucene70DocValuesConsumer extends DocValuesConsumer implements Close
     } else {
       final int bitsPerValue = DirectWriter.unsignedBitsRequired(max - min);
       buffer.reset();
-      assert buffer.getPosition() == 0;
+      assert buffer.size() == 0;
       final DirectWriter w = DirectWriter.getInstance(buffer, length, bitsPerValue);
       for (int i = 0; i < length; ++i) {
         w.add((values[i] - min) / gcd);
@@ -325,8 +325,8 @@ final class Lucene70DocValuesConsumer extends DocValuesConsumer implements Close
       w.finish();
       data.writeByte((byte) bitsPerValue);
       data.writeLong(min);
-      data.writeInt(buffer.getPosition());
-      data.writeBytes(buffer.getBytes(), buffer.getPosition());
+      data.writeInt(Math.toIntExact(buffer.size()));
+      buffer.copyTo(data);
     }
   }
 
@@ -442,10 +442,11 @@ final class Lucene70DocValuesConsumer extends DocValuesConsumer implements Close
     meta.writeVLong(size);
     meta.writeInt(Lucene70DocValuesFormat.TERMS_DICT_BLOCK_SHIFT);
 
-    RAMOutputStream addressBuffer = new RAMOutputStream();
+    ByteBuffersDataOutput addressBuffer = new ByteBuffersDataOutput();
+    ByteBuffersIndexOutput addressOutput = new ByteBuffersIndexOutput(addressBuffer, "temp", "temp");
     meta.writeInt(DIRECT_MONOTONIC_BLOCK_SHIFT);
     long numBlocks = (size + Lucene70DocValuesFormat.TERMS_DICT_BLOCK_MASK) >>> Lucene70DocValuesFormat.TERMS_DICT_BLOCK_SHIFT;
-    DirectMonotonicWriter writer = DirectMonotonicWriter.getInstance(meta, addressBuffer, numBlocks, DIRECT_MONOTONIC_BLOCK_SHIFT);
+    DirectMonotonicWriter writer = DirectMonotonicWriter.getInstance(meta, addressOutput, numBlocks, DIRECT_MONOTONIC_BLOCK_SHIFT);
 
     BytesRefBuilder previous = new BytesRefBuilder();
     long ord = 0;
@@ -480,7 +481,7 @@ final class Lucene70DocValuesConsumer extends DocValuesConsumer implements Close
     meta.writeLong(start);
     meta.writeLong(data.getFilePointer() - start);
     start = data.getFilePointer();
-    addressBuffer.writeTo(data);
+    addressBuffer.copyTo(data);
     meta.writeLong(start);
     meta.writeLong(data.getFilePointer() - start);
 
@@ -494,9 +495,10 @@ final class Lucene70DocValuesConsumer extends DocValuesConsumer implements Close
     long start = data.getFilePointer();
 
     long numBlocks = 1L + ((size + Lucene70DocValuesFormat.TERMS_DICT_REVERSE_INDEX_MASK) >>> Lucene70DocValuesFormat.TERMS_DICT_REVERSE_INDEX_SHIFT);
-    RAMOutputStream addressBuffer = new RAMOutputStream();
-    DirectMonotonicWriter writer = DirectMonotonicWriter.getInstance(meta, addressBuffer, numBlocks, DIRECT_MONOTONIC_BLOCK_SHIFT);
-
+    ByteBuffersDataOutput addressBuffer = new ByteBuffersDataOutput();
+    DirectMonotonicWriter writer;
+    try (ByteBuffersIndexOutput addressOutput = new ByteBuffersIndexOutput(addressBuffer, "temp", "temp")) {
+      writer = DirectMonotonicWriter.getInstance(meta, addressOutput, numBlocks, DIRECT_MONOTONIC_BLOCK_SHIFT);
     TermsEnum iterator = values.termsEnum();
     BytesRefBuilder previous = new BytesRefBuilder();
     long offset = 0;
@@ -523,10 +525,11 @@ final class Lucene70DocValuesConsumer extends DocValuesConsumer implements Close
     meta.writeLong(start);
     meta.writeLong(data.getFilePointer() - start);
     start = data.getFilePointer();
-    addressBuffer.writeTo(data);
+      addressBuffer.copyTo(data);
     meta.writeLong(start);
     meta.writeLong(data.getFilePointer() - start);
   }
+  }
 
   @Override
   public void addSortedNumericField(FieldInfo field, DocValuesProducer valuesProducer) throws IOException {
diff --git a/lucene/core/src/java/org/apache/lucene/index/PrefixCodedTerms.java b/lucene/core/src/java/org/apache/lucene/index/PrefixCodedTerms.java
index 7a65d71..6820561 100644
--- a/lucene/core/src/java/org/apache/lucene/index/PrefixCodedTerms.java
+++ b/lucene/core/src/java/org/apache/lucene/index/PrefixCodedTerms.java
@@ -17,12 +17,12 @@
 package org.apache.lucene.index;
 
 import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
 import java.util.Objects;
 
-import org.apache.lucene.store.IndexInput;
-import org.apache.lucene.store.RAMFile;
-import org.apache.lucene.store.RAMInputStream;
-import org.apache.lucene.store.RAMOutputStream;
+import org.apache.lucene.store.ByteBuffersDataInput;
+import org.apache.lucene.store.ByteBuffersDataOutput;
 import org.apache.lucene.util.Accountable;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefBuilder;
@@ -35,18 +35,19 @@ import org.apache.lucene.util.StringHelper;
  * @lucene.internal
  */
 public class PrefixCodedTerms implements Accountable {
-  final RAMFile buffer;
+  private final List<ByteBuffer> content;
   private final long size;
   private long delGen;
+  private int lazyHash;
 
-  private PrefixCodedTerms(RAMFile buffer, long size) {
-    this.buffer = Objects.requireNonNull(buffer);
+  private PrefixCodedTerms(List<ByteBuffer> content, long size) {
+    this.content = Objects.requireNonNull(content);
     this.size = size;
   }
 
   @Override
   public long ramBytesUsed() {
-    return buffer.ramBytesUsed() + 2 * Long.BYTES;
+    return content.stream().mapToLong(buf -> buf.capacity()).sum() + 2 * Long.BYTES; 
   }
 
   /** Records del gen for this packet. */
@@ -56,8 +57,7 @@ public class PrefixCodedTerms implements Accountable {
   
   /** Builds a PrefixCodedTerms: call add repeatedly, then finish. */
   public static class Builder {
-    private RAMFile buffer = new RAMFile();
-    private RAMOutputStream output = new RAMOutputStream(buffer, false);
+    private ByteBuffersDataOutput output = new ByteBuffersDataOutput();
     private Term lastTerm = new Term("");
     private BytesRefBuilder lastTermBytes = new BytesRefBuilder();
     private long size;
@@ -101,37 +101,28 @@ public class PrefixCodedTerms implements Accountable {
     
     /** return finalized form */
     public PrefixCodedTerms finish() {
-      try {
-        output.close();
-        return new PrefixCodedTerms(buffer, size);
-      } catch (IOException e) {
-        throw new RuntimeException(e);
-      }
+      return new PrefixCodedTerms(output.toBufferList(), size);
     }
   }
 
   /** An iterator over the list of terms stored in a {@link PrefixCodedTerms}. */
   public static class TermIterator extends FieldTermIterator {
-    final IndexInput input;
+    final ByteBuffersDataInput input;
     final BytesRefBuilder builder = new BytesRefBuilder();
     final BytesRef bytes = builder.get();
     final long end;
     final long delGen;
     String field = "";
 
-    private TermIterator(long delGen, RAMFile buffer) {
-      try {
-        input = new RAMInputStream("PrefixCodedTermsIterator", buffer);
-      } catch (IOException e) {
-        throw new RuntimeException(e);
-      }
-      end = input.length();
+    private TermIterator(long delGen, ByteBuffersDataInput input) {
+      this.input = input;
+      end = input.size();
       this.delGen = delGen;
     }
 
     @Override
     public BytesRef next() {
-      if (input.getFilePointer() < end) {
+      if (input.position() < end) {
         try {
           int code = input.readVInt();
           boolean newField = (code & 1) != 0;
@@ -171,7 +162,7 @@ public class PrefixCodedTerms implements Accountable {
 
   /** Return an iterator over the terms stored in this {@link PrefixCodedTerms}. */
   public TermIterator iterator() {
-    return new TermIterator(delGen, buffer);
+    return new TermIterator(delGen, new ByteBuffersDataInput(content));
   }
 
   /** Return the number of terms stored in this {@link PrefixCodedTerms}. */
@@ -181,17 +172,29 @@ public class PrefixCodedTerms implements Accountable {
 
   @Override
   public int hashCode() {
-    int h = buffer.hashCode();
-    h = 31 * h + (int) (delGen ^ (delGen >>> 32));
-    return h;
+    if (lazyHash == 0) {
+      int h = 1;
+      for (ByteBuffer bb : content) {
+        h = h + 31 * bb.hashCode();
+      }
+      h = 31 * h + (int) (delGen ^ (delGen >>> 32));
+      lazyHash = h;
+    }
+    return lazyHash;
   }
 
   @Override
   public boolean equals(Object obj) {
-    if (this == obj) return true;
-    if (obj == null) return false;
-    if (getClass() != obj.getClass()) return false;
+    if (this == obj) {
+      return true;
+    }
+
+    if (obj == null || getClass() != obj.getClass()) { 
+      return false;
+    }
+
     PrefixCodedTerms other = (PrefixCodedTerms) obj;
-    return buffer.equals(other.buffer) && delGen == other.delGen;
+    return delGen == other.delGen &&
+           this.content.equals(other.content);
   }
 }
diff --git a/lucene/core/src/java/org/apache/lucene/index/SortingLeafReader.java b/lucene/core/src/java/org/apache/lucene/index/SortingLeafReader.java
index d9ed94b..104ef06 100644
--- a/lucene/core/src/java/org/apache/lucene/index/SortingLeafReader.java
+++ b/lucene/core/src/java/org/apache/lucene/index/SortingLeafReader.java
@@ -24,11 +24,9 @@ import java.util.Map;
 
 import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.search.Sort;
-import org.apache.lucene.store.IndexInput;
-import org.apache.lucene.store.IndexOutput;
-import org.apache.lucene.store.RAMFile;
-import org.apache.lucene.store.RAMInputStream;
-import org.apache.lucene.store.RAMOutputStream;
+import org.apache.lucene.store.ByteBuffersDataInput;
+import org.apache.lucene.store.ByteBuffersDataOutput;
+import org.apache.lucene.store.DataOutput;
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.BitSet;
 import org.apache.lucene.util.Bits;
@@ -856,7 +854,7 @@ class SortingLeafReader extends FilterLeafReader {
     private long[] offsets;
     private final int upto;
 
-    private final IndexInput postingInput;
+    private final ByteBuffersDataInput postingInput;
     private final boolean storeOffsets;
 
     private int docIt = -1;
@@ -866,7 +864,7 @@ class SortingLeafReader extends FilterLeafReader {
     private final BytesRef payload;
     private int currFreq;
 
-    private final RAMFile file;
+    private final ByteBuffersDataOutput buffer;
 
     SortingPostingsEnum(int maxDoc, SortingPostingsEnum reuse, final PostingsEnum in, Sorter.DocMap docMap, boolean storeOffsets) throws IOException {
       super(in);
@@ -876,7 +874,8 @@ class SortingLeafReader extends FilterLeafReader {
         docs = reuse.docs;
         offsets = reuse.offsets;
         payload = reuse.payload;
-        file = reuse.file;
+        buffer = reuse.buffer;
+        buffer.reset();
         if (reuse.maxDoc == maxDoc) {
           sorter = reuse.sorter;
         } else {
@@ -886,10 +885,10 @@ class SortingLeafReader extends FilterLeafReader {
         docs = new int[32];
         offsets = new long[32];
         payload = new BytesRef(32);
-        file = new RAMFile();
+        buffer = ByteBuffersDataOutput.newResettableInstance();
         sorter = new DocOffsetSorter(maxDoc);
       }
-      final IndexOutput out = new RAMOutputStream(file, false);
+
       int doc;
       int i = 0;
       while ((doc = in.nextDoc()) != DocIdSetIterator.NO_MORE_DOCS) {
@@ -899,15 +898,15 @@ class SortingLeafReader extends FilterLeafReader {
           offsets = ArrayUtil.growExact(offsets, newLength);
         }
         docs[i] = docMap.oldToNew(doc);
-        offsets[i] = out.getFilePointer();
-        addPositions(in, out);
+        offsets[i] = buffer.size();
+        addPositions(in, buffer);
         i++;
       }
       upto = i;
       sorter.reset(docs, offsets);
       sorter.sort(0, upto);
-      out.close();
-      this.postingInput = new RAMInputStream("", file);
+
+      this.postingInput = buffer.toDataInput();
     }
 
     // for testing
@@ -918,7 +917,7 @@ class SortingLeafReader extends FilterLeafReader {
       return docs == ((SortingPostingsEnum) other).docs;
     }
 
-    private void addPositions(final PostingsEnum in, final IndexOutput out) throws IOException {
+    private void addPositions(final PostingsEnum in, final DataOutput out) throws IOException {
       int freq = in.freq();
       out.writeVInt(freq);
       int previousPosition = 0;
diff --git a/lucene/core/src/java/org/apache/lucene/store/GrowableByteArrayDataOutput.java b/lucene/core/src/java/org/apache/lucene/store/GrowableByteArrayDataOutput.java
deleted file mode 100644
index c74d4ef..0000000
--- a/lucene/core/src/java/org/apache/lucene/store/GrowableByteArrayDataOutput.java
+++ /dev/null
@@ -1,102 +0,0 @@
-/*
- * 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.
- */
-
-package org.apache.lucene.store;
-
-import java.io.IOException;
-
-import org.apache.lucene.util.ArrayUtil;
-import org.apache.lucene.util.UnicodeUtil;
-
-/**
- * A {@link DataOutput} that can be used to build a byte[].
- *
- * @lucene.internal
- */
-public final class GrowableByteArrayDataOutput extends DataOutput {
-
-  /** Minimum utf8 byte size of a string over which double pass over string is to save memory during encode */
-  static final int MIN_UTF8_SIZE_TO_ENABLE_DOUBLE_PASS_ENCODING = 65536;
-
-  /** The bytes */
-  private byte[] bytes;
-
-  /** The length */
-  private int length;
-
-  // scratch for utf8 encoding of small strings
-  private byte[] scratchBytes;
-
-  /** Create a {@link GrowableByteArrayDataOutput} with the given initial capacity. */
-  public GrowableByteArrayDataOutput(int cp) {
-    this.bytes = new byte[ArrayUtil.oversize(cp, 1)];
-    this.length = 0;
-  }
-
-  @Override
-  public void writeByte(byte b) {
-    if (length >= bytes.length) {
-      bytes = ArrayUtil.grow(bytes);
-    }
-    bytes[length++] = b;
-  }
-
-  @Override
-  public void writeBytes(byte[] b, int off, int len) {
-    final int newLength = length + len;
-    if (newLength > bytes.length) {
-      bytes = ArrayUtil.grow(bytes, newLength);
-    }
-    System.arraycopy(b, off, bytes, length, len);
-    length = newLength;
-  }
-
-  @Override
-  public void writeString(String string) throws IOException {
-    int maxLen = UnicodeUtil.maxUTF8Length(string.length());
-    if (maxLen <= MIN_UTF8_SIZE_TO_ENABLE_DOUBLE_PASS_ENCODING)  {
-      // string is small enough that we don't need to save memory by falling back to double-pass approach
-      // this is just an optimized writeString() that re-uses scratchBytes.
-      if (scratchBytes == null) {
-        scratchBytes = new byte[ArrayUtil.oversize(maxLen, Character.BYTES)];
-      } else {
-        scratchBytes = ArrayUtil.grow(scratchBytes, maxLen);
-      }
-      int len = UnicodeUtil.UTF16toUTF8(string, 0, string.length(), scratchBytes);
-      writeVInt(len);
-      writeBytes(scratchBytes, len);
-    } else  {
-      // use a double pass approach to avoid allocating a large intermediate buffer for string encoding
-      int numBytes = UnicodeUtil.calcUTF16toUTF8Length(string, 0, string.length());
-      writeVInt(numBytes);
-      bytes = ArrayUtil.grow(bytes, length + numBytes);
-      length = UnicodeUtil.UTF16toUTF8(string, 0, string.length(), bytes, length);
-    }
-  }
-
-  public byte[] getBytes() {
-    return bytes;
-  }
-
-  public int getPosition() {
-    return length;
-  }
-
-  public void reset() {
-    length = 0;
-  }
-}
diff --git a/lucene/core/src/java/org/apache/lucene/util/IOUtils.java b/lucene/core/src/java/org/apache/lucene/util/IOUtils.java
index a556789..ba0be42 100644
--- a/lucene/core/src/java/org/apache/lucene/util/IOUtils.java
+++ b/lucene/core/src/java/org/apache/lucene/util/IOUtils.java
@@ -41,6 +41,7 @@ import java.util.LinkedHashMap;
 import java.util.Map;
 import java.util.Objects;
 
+import org.apache.lucene.store.ByteBuffersDirectory;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.FSDirectory;
 import org.apache.lucene.store.FileSwitchDirectory;
@@ -486,7 +487,7 @@ public final class IOUtils {
       FileSwitchDirectory fsd = (FileSwitchDirectory) dir;
       // Spinning is contagious:
       return spins(fsd.getPrimaryDir()) || spins(fsd.getSecondaryDir());
-    } else if (dir instanceof RAMDirectory) {
+    } else if (dir instanceof RAMDirectory || dir instanceof ByteBuffersDirectory) {
       return false;
     } else if (dir instanceof FSDirectory) {
       return spins(((FSDirectory) dir).getDirectory());
diff --git a/lucene/core/src/java/org/apache/lucene/util/bkd/BKDWriter.java b/lucene/core/src/java/org/apache/lucene/util/bkd/BKDWriter.java
index 676896f..82c490a 100644
--- a/lucene/core/src/java/org/apache/lucene/util/bkd/BKDWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/util/bkd/BKDWriter.java
@@ -29,13 +29,12 @@ import org.apache.lucene.codecs.MutablePointValues;
 import org.apache.lucene.index.MergeState;
 import org.apache.lucene.index.PointValues.IntersectVisitor;
 import org.apache.lucene.index.PointValues.Relation;
+import org.apache.lucene.store.ByteBuffersDataOutput;
 import org.apache.lucene.store.ChecksumIndexInput;
 import org.apache.lucene.store.DataOutput;
 import org.apache.lucene.store.Directory;
-import org.apache.lucene.store.GrowableByteArrayDataOutput;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexOutput;
-import org.apache.lucene.store.RAMOutputStream;
 import org.apache.lucene.store.TrackingDirectoryWrapper;
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.BytesRef;
@@ -572,8 +571,8 @@ public class BKDWriter implements Closeable {
     return oneDimWriter.finish();
   }
 
-  // reused when writing leaf blocks
-  private final GrowableByteArrayDataOutput scratchOut = new GrowableByteArrayDataOutput(32*1024);
+  // Reused when writing leaf blocks
+  private final ByteBuffersDataOutput scratchOut = ByteBuffersDataOutput.newResettableInstance();
 
   private class OneDimensionBKDWriter {
 
@@ -686,7 +685,7 @@ public class BKDWriter implements Closeable {
 
       commonPrefixLengths[0] = prefix;
 
-      assert scratchOut.getPosition() == 0;
+      assert scratchOut.size() == 0;
       writeLeafBlockDocs(scratchOut, leafDocs, 0, leafCount);
       writeCommonPrefixes(scratchOut, commonPrefixLengths, leafValues);
 
@@ -704,7 +703,7 @@ public class BKDWriter implements Closeable {
           ArrayUtil.copyOfSubArray(leafValues, (leafCount - 1) * packedBytesLength, leafCount * packedBytesLength),
           packedValues, leafDocs, 0);
       writeLeafBlockPackedValues(scratchOut, commonPrefixLengths, leafCount, 0, packedValues);
-      out.writeBytes(scratchOut.getBytes(), 0, scratchOut.getPosition());
+      scratchOut.copyTo(out);
       scratchOut.reset();
     }
   }
@@ -1094,7 +1093,7 @@ public class BKDWriter implements Closeable {
     }
 
     /** Reused while packing the index */
-    RAMOutputStream writeBuffer = new RAMOutputStream();
+    ByteBuffersDataOutput writeBuffer = ByteBuffersDataOutput.newResettableInstance();
 
     // This is the "file" we append the byte[] to:
     List<byte[]> blocks = new ArrayList<>();
@@ -1115,19 +1114,17 @@ public class BKDWriter implements Closeable {
   }
 
   /** Appends the current contents of writeBuffer as another block on the growing in-memory file */
-  private int appendBlock(RAMOutputStream writeBuffer, List<byte[]> blocks) throws IOException {
-    int pos = Math.toIntExact(writeBuffer.getFilePointer());
-    byte[] bytes = new byte[pos];
-    writeBuffer.writeTo(bytes, 0);
+  private int appendBlock(ByteBuffersDataOutput writeBuffer, List<byte[]> blocks) throws IOException {
+    byte[] block = writeBuffer.toArrayCopy();
+    blocks.add(block);
     writeBuffer.reset();
-    blocks.add(bytes);
-    return pos;
+    return block.length;
   }
 
   /**
    * lastSplitValues is per-dimension split value previously seen; we use this to prefix-code the split byte[] on each inner node
    */
-  private int recursePackIndex(RAMOutputStream writeBuffer, long[] leafBlockFPs, byte[] splitPackedValues, long minBlockFP, List<byte[]> blocks,
+  private int recursePackIndex(ByteBuffersDataOutput writeBuffer, long[] leafBlockFPs, byte[] splitPackedValues, long minBlockFP, List<byte[]> blocks,
                                int nodeID, byte[] lastSplitValues, boolean[] negativeDeltas, boolean isLeft) throws IOException {
     if (nodeID >= leafBlockFPs.length) {
       int leafID = nodeID - leafBlockFPs.length;
@@ -1225,9 +1222,8 @@ public class BKDWriter implements Closeable {
       } else {
         assert leftNumBytes == 0: "leftNumBytes=" + leftNumBytes;
       }
-      int numBytes2 = Math.toIntExact(writeBuffer.getFilePointer());
-      byte[] bytes2 = new byte[numBytes2];
-      writeBuffer.writeTo(bytes2, 0);
+      
+      byte[] bytes2 = writeBuffer.toArrayCopy();
       writeBuffer.reset();
       // replace our placeholder:
       blocks.set(idxSav, bytes2);
@@ -1242,7 +1238,7 @@ public class BKDWriter implements Closeable {
 
       assert Arrays.equals(lastSplitValues, cmp);
       
-      return numBytes + numBytes2 + leftNumBytes + rightNumBytes;
+      return numBytes + bytes2.length + leftNumBytes + rightNumBytes;
     }
   }
 
@@ -1605,7 +1601,7 @@ public class BKDWriter implements Closeable {
       // Save the block file pointer:
       leafBlockFPs[nodeID - leafNodeOffset] = out.getFilePointer();
 
-      assert scratchOut.getPosition() == 0;
+      assert scratchOut.size() == 0;
 
       // Write doc IDs
       int[] docIDs = spareDocIds;
@@ -1631,10 +1627,8 @@ public class BKDWriter implements Closeable {
       assert valuesInOrderAndBounds(count, sortedDim, minPackedValue, maxPackedValue, packedValues,
           docIDs, 0);
       writeLeafBlockPackedValues(scratchOut, commonPrefixLengths, count, sortedDim, packedValues);
-
-      out.writeBytes(scratchOut.getBytes(), 0, scratchOut.getPosition());
+      scratchOut.copyTo(out);
       scratchOut.reset();
-
     } else {
       // inner node
 
diff --git a/lucene/core/src/java/org/apache/lucene/util/fst/FST.java b/lucene/core/src/java/org/apache/lucene/util/fst/FST.java
index 92973f6..8e84e3c 100644
--- a/lucene/core/src/java/org/apache/lucene/util/fst/FST.java
+++ b/lucene/core/src/java/org/apache/lucene/util/fst/FST.java
@@ -27,11 +27,11 @@ import java.nio.file.Path;
 
 import org.apache.lucene.codecs.CodecUtil;
 import org.apache.lucene.store.ByteArrayDataOutput;
+import org.apache.lucene.store.ByteBuffersDataOutput;
 import org.apache.lucene.store.DataInput;
 import org.apache.lucene.store.DataOutput;
 import org.apache.lucene.store.InputStreamDataInput;
 import org.apache.lucene.store.OutputStreamDataOutput;
-import org.apache.lucene.store.RAMOutputStream;
 import org.apache.lucene.util.Accountable;
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.Constants;
@@ -432,16 +432,14 @@ public final class FST<T> implements Accountable {
       out.writeByte((byte) 1);
 
       // Serialize empty-string output:
-      RAMOutputStream ros = new RAMOutputStream();
+      ByteBuffersDataOutput ros = new ByteBuffersDataOutput();
       outputs.writeFinalOutput(emptyOutput, ros);
-      
-      byte[] emptyOutputBytes = new byte[(int) ros.getFilePointer()];
-      ros.writeTo(emptyOutputBytes, 0);
+      byte[] emptyOutputBytes = ros.toArrayCopy();
 
       // reverse
       final int stopAt = emptyOutputBytes.length/2;
       int upto = 0;
-      while(upto < stopAt) {
+      while (upto < stopAt) {
         final byte b = emptyOutputBytes[upto];
         emptyOutputBytes[upto] = emptyOutputBytes[emptyOutputBytes.length-upto-1];
         emptyOutputBytes[emptyOutputBytes.length-upto-1] = b;
diff --git a/lucene/core/src/test/org/apache/lucene/codecs/TestCodecUtil.java b/lucene/core/src/test/org/apache/lucene/codecs/TestCodecUtil.java
index a15475b..0a11a9b 100644
--- a/lucene/core/src/test/org/apache/lucene/codecs/TestCodecUtil.java
+++ b/lucene/core/src/test/org/apache/lucene/codecs/TestCodecUtil.java
@@ -22,12 +22,12 @@ import java.util.concurrent.atomic.AtomicLong;
 
 import org.apache.lucene.index.CorruptIndexException;
 import org.apache.lucene.store.BufferedChecksumIndexInput;
+import org.apache.lucene.store.ByteBuffersDataOutput;
+import org.apache.lucene.store.ByteBuffersIndexInput;
+import org.apache.lucene.store.ByteBuffersIndexOutput;
 import org.apache.lucene.store.ChecksumIndexInput;
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.store.IndexOutput;
-import org.apache.lucene.store.RAMFile;
-import org.apache.lucene.store.RAMInputStream;
-import org.apache.lucene.store.RAMOutputStream;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.StringHelper;
 
@@ -35,13 +35,13 @@ import org.apache.lucene.util.StringHelper;
 public class TestCodecUtil extends LuceneTestCase {
   
   public void testHeaderLength() throws Exception {
-    RAMFile file = new RAMFile();
-    IndexOutput output = new RAMOutputStream(file, true);
+    ByteBuffersDataOutput out = new ByteBuffersDataOutput();
+    IndexOutput output = new ByteBuffersIndexOutput(out, "temp", "temp");
     CodecUtil.writeHeader(output, "FooBar", 5);
     output.writeString("this is the data");
     output.close();
     
-    IndexInput input = new RAMInputStream("file", file);
+    IndexInput input = new ByteBuffersIndexInput(out.toDataInput(), "temp");
     input.seek(CodecUtil.headerLength("FooBar"));
     assertEquals("this is the data", input.readString());
     input.close();
@@ -52,55 +52,55 @@ public class TestCodecUtil extends LuceneTestCase {
     for (int i = 0; i < 128; i++) {
       tooLong.append('a');
     }
-    RAMFile file = new RAMFile();
-    IndexOutput output = new RAMOutputStream(file, true);
+    ByteBuffersDataOutput out = new ByteBuffersDataOutput();
+    IndexOutput output = new ByteBuffersIndexOutput(out, "temp", "temp");
     expectThrows(IllegalArgumentException.class, () -> {
       CodecUtil.writeHeader(output, tooLong.toString(), 5);
     });
   }
   
   public void testWriteNonAsciiHeader() throws Exception {
-    RAMFile file = new RAMFile();
-    IndexOutput output = new RAMOutputStream(file, true);
+    ByteBuffersDataOutput out = new ByteBuffersDataOutput();
+    IndexOutput output = new ByteBuffersIndexOutput(out, "temp", "temp");
     expectThrows(IllegalArgumentException.class, () -> {
       CodecUtil.writeHeader(output, "\u1234", 5);
     });
   }
   
   public void testReadHeaderWrongMagic() throws Exception {
-    RAMFile file = new RAMFile();
-    IndexOutput output = new RAMOutputStream(file, true);
+    ByteBuffersDataOutput out = new ByteBuffersDataOutput();
+    IndexOutput output = new ByteBuffersIndexOutput(out, "temp", "temp");
     output.writeInt(1234);
     output.close();
     
-    IndexInput input = new RAMInputStream("file", file);
+    IndexInput input = new ByteBuffersIndexInput(out.toDataInput(), "temp");
     expectThrows(CorruptIndexException.class, () -> {
       CodecUtil.checkHeader(input, "bogus", 1, 1);
     });
   }
   
   public void testChecksumEntireFile() throws Exception {
-    RAMFile file = new RAMFile();
-    IndexOutput output = new RAMOutputStream(file, true);
+    ByteBuffersDataOutput out = new ByteBuffersDataOutput();
+    IndexOutput output = new ByteBuffersIndexOutput(out, "temp", "temp");
     CodecUtil.writeHeader(output, "FooBar", 5);
     output.writeString("this is the data");
     CodecUtil.writeFooter(output);
     output.close();
     
-    IndexInput input = new RAMInputStream("file", file);
+    IndexInput input = new ByteBuffersIndexInput(out.toDataInput(), "temp");
     CodecUtil.checksumEntireFile(input);
     input.close();
   }
   
   public void testCheckFooterValid() throws Exception {
-    RAMFile file = new RAMFile();
-    IndexOutput output = new RAMOutputStream(file, true);
+    ByteBuffersDataOutput out = new ByteBuffersDataOutput();
+    IndexOutput output = new ByteBuffersIndexOutput(out, "temp", "temp");
     CodecUtil.writeHeader(output, "FooBar", 5);
     output.writeString("this is the data");
     CodecUtil.writeFooter(output);
     output.close();
     
-    ChecksumIndexInput input = new BufferedChecksumIndexInput(new RAMInputStream("file", file));
+    ChecksumIndexInput input = new BufferedChecksumIndexInput(new ByteBuffersIndexInput(out.toDataInput(), "temp"));
     Exception mine = new RuntimeException("fake exception");
     RuntimeException expected = expectThrows(RuntimeException.class, () -> {
       CodecUtil.checkFooter(input, mine);
@@ -113,14 +113,14 @@ public class TestCodecUtil extends LuceneTestCase {
   }
   
   public void testCheckFooterValidAtFooter() throws Exception {
-    RAMFile file = new RAMFile();
-    IndexOutput output = new RAMOutputStream(file, true);
+    ByteBuffersDataOutput out = new ByteBuffersDataOutput();
+    IndexOutput output = new ByteBuffersIndexOutput(out, "temp", "temp");
     CodecUtil.writeHeader(output, "FooBar", 5);
     output.writeString("this is the data");
     CodecUtil.writeFooter(output);
     output.close();
     
-    ChecksumIndexInput input = new BufferedChecksumIndexInput(new RAMInputStream("file", file));
+    ChecksumIndexInput input = new BufferedChecksumIndexInput(new ByteBuffersIndexInput(out.toDataInput(), "temp"));
     CodecUtil.checkHeader(input, "FooBar", 5, 5);
     assertEquals("this is the data", input.readString());
     Exception mine = new RuntimeException("fake exception");
@@ -135,14 +135,14 @@ public class TestCodecUtil extends LuceneTestCase {
   }
   
   public void testCheckFooterValidPastFooter() throws Exception {
-    RAMFile file = new RAMFile();
-    IndexOutput output = new RAMOutputStream(file, true);
+    ByteBuffersDataOutput out = new ByteBuffersDataOutput();
+    IndexOutput output = new ByteBuffersIndexOutput(out, "temp", "temp");
     CodecUtil.writeHeader(output, "FooBar", 5);
     output.writeString("this is the data");
     CodecUtil.writeFooter(output);
     output.close();
     
-    ChecksumIndexInput input = new BufferedChecksumIndexInput(new RAMInputStream("file", file));
+    ChecksumIndexInput input = new BufferedChecksumIndexInput(new ByteBuffersIndexInput(out.toDataInput(), "temp"));
     CodecUtil.checkHeader(input, "FooBar", 5, 5);
     assertEquals("this is the data", input.readString());
     // bogusly read a byte too far (can happen)
@@ -159,16 +159,16 @@ public class TestCodecUtil extends LuceneTestCase {
   }
   
   public void testCheckFooterInvalid() throws Exception {
-    RAMFile file = new RAMFile();
-    IndexOutput output = new RAMOutputStream(file, true);
+    ByteBuffersDataOutput out = new ByteBuffersDataOutput();
+    IndexOutput output = new ByteBuffersIndexOutput(out, "temp", "temp");
     CodecUtil.writeHeader(output, "FooBar", 5);
     output.writeString("this is the data");
     output.writeInt(CodecUtil.FOOTER_MAGIC);
     output.writeInt(0);
     output.writeLong(1234567); // write a bogus checksum
     output.close();
-    
-    ChecksumIndexInput input = new BufferedChecksumIndexInput(new RAMInputStream("file", file));
+
+    ChecksumIndexInput input = new BufferedChecksumIndexInput(new ByteBuffersIndexInput(out.toDataInput(), "temp"));
     CodecUtil.checkHeader(input, "FooBar", 5, 5);
     assertEquals("this is the data", input.readString());
     Exception mine = new RuntimeException("fake exception");
@@ -183,13 +183,13 @@ public class TestCodecUtil extends LuceneTestCase {
   }
   
   public void testSegmentHeaderLength() throws Exception {
-    RAMFile file = new RAMFile();
-    IndexOutput output = new RAMOutputStream(file, true);
+    ByteBuffersDataOutput out = new ByteBuffersDataOutput();
+    IndexOutput output = new ByteBuffersIndexOutput(out, "temp", "temp");
     CodecUtil.writeIndexHeader(output, "FooBar", 5, StringHelper.randomId(), "xyz");
     output.writeString("this is the data");
     output.close();
     
-    IndexInput input = new RAMInputStream("file", file);
+    IndexInput input = new ByteBuffersIndexInput(out.toDataInput(), "temp");
     input.seek(CodecUtil.indexHeaderLength("FooBar", "xyz"));
     assertEquals("this is the data", input.readString());
     input.close();
@@ -200,8 +200,8 @@ public class TestCodecUtil extends LuceneTestCase {
     for (int i = 0; i < 256; i++) {
       tooLong.append('a');
     }
-    RAMFile file = new RAMFile();
-    IndexOutput output = new RAMOutputStream(file, true);
+    ByteBuffersDataOutput out = new ByteBuffersDataOutput();
+    IndexOutput output = new ByteBuffersIndexOutput(out, "temp", "temp");
     expectThrows(IllegalArgumentException.class, () -> {
       CodecUtil.writeIndexHeader(output, "foobar", 5, StringHelper.randomId(), tooLong.toString());
     });
@@ -212,13 +212,13 @@ public class TestCodecUtil extends LuceneTestCase {
     for (int i = 0; i < 255; i++) {
       justLongEnough.append('a');
     }
-    RAMFile file = new RAMFile();
-    IndexOutput output = new RAMOutputStream(file, true);
+    ByteBuffersDataOutput out = new ByteBuffersDataOutput();
+    IndexOutput output = new ByteBuffersIndexOutput(out, "temp", "temp");
     byte[] id = StringHelper.randomId();
     CodecUtil.writeIndexHeader(output, "foobar", 5, id, justLongEnough.toString());
     output.close();
     
-    IndexInput input = new RAMInputStream("file", file);
+    IndexInput input = new ByteBuffersIndexInput(out.toDataInput(), "temp");
     CodecUtil.checkIndexHeader(input, "foobar", 5, 5, id, justLongEnough.toString());
     assertEquals(input.getFilePointer(), input.length());
     assertEquals(input.getFilePointer(), CodecUtil.indexHeaderLength("foobar", justLongEnough.toString()));
@@ -226,22 +226,22 @@ public class TestCodecUtil extends LuceneTestCase {
   }
   
   public void testWriteNonAsciiSuffix() throws Exception {
-    RAMFile file = new RAMFile();
-    IndexOutput output = new RAMOutputStream(file, true);
+    ByteBuffersDataOutput out = new ByteBuffersDataOutput();
+    IndexOutput output = new ByteBuffersIndexOutput(out, "temp", "temp");
     expectThrows(IllegalArgumentException.class, () -> {
       CodecUtil.writeIndexHeader(output, "foobar", 5, StringHelper.randomId(), "\u1234");
     });
   }
   
   public void testReadBogusCRC() throws Exception {
-    RAMFile file = new RAMFile();
-    IndexOutput output = new RAMOutputStream(file, false);
+    ByteBuffersDataOutput out = new ByteBuffersDataOutput();
+    IndexOutput output = new ByteBuffersIndexOutput(out, "temp", "temp");
     output.writeLong(-1L); // bad
     output.writeLong(1L << 32); // bad
     output.writeLong(-(1L << 32)); // bad
     output.writeLong((1L << 32) - 1); // ok
     output.close();
-    IndexInput input = new RAMInputStream("file", file);
+    IndexInput input = new BufferedChecksumIndexInput(new ByteBuffersIndexInput(out.toDataInput(), "temp"));
     // read 3 bogus values
     for (int i = 0; i < 3; i++) {
       expectThrows(CorruptIndexException.class, () -> {
@@ -253,9 +253,10 @@ public class TestCodecUtil extends LuceneTestCase {
   }
   
   public void testWriteBogusCRC() throws Exception {
-    RAMFile file = new RAMFile();
-    final IndexOutput output = new RAMOutputStream(file, false);
+    ByteBuffersDataOutput out = new ByteBuffersDataOutput();
+    IndexOutput output = new ByteBuffersIndexOutput(out, "temp", "temp");
     AtomicLong fakeChecksum = new AtomicLong();
+
     // wrap the index input where we control the checksum for mocking
     IndexOutput fakeOutput = new IndexOutput("fake", "fake") {
       @Override
@@ -304,15 +305,18 @@ public class TestCodecUtil extends LuceneTestCase {
   }
 
   public void testTruncatedFileThrowsCorruptIndexException() throws IOException {
-    RAMFile file = new RAMFile();
-    IndexOutput output = new RAMOutputStream(file, false);
+    ByteBuffersDataOutput out = new ByteBuffersDataOutput();
+    IndexOutput output = new ByteBuffersIndexOutput(out, "temp", "temp");
     output.close();
-    IndexInput input = new RAMInputStream("file", file);
+
+    IndexInput input = new ByteBuffersIndexInput(out.toDataInput(), "temp");
+    
     CorruptIndexException e = expectThrows(CorruptIndexException.class,
         () -> CodecUtil.checksumEntireFile(input));
-    assertEquals("misplaced codec footer (file truncated?): length=0 but footerLength==16 (resource=RAMInputStream(name=file))", e.getMessage());
+    assertTrue(e.getMessage(), e.getMessage().contains("misplaced codec footer (file truncated?): length=0 but footerLength==16 (resource"));
+
     e = expectThrows(CorruptIndexException.class,
         () -> CodecUtil.retrieveChecksum(input));
-    assertEquals("misplaced codec footer (file truncated?): length=0 but footerLength==16 (resource=RAMInputStream(name=file))", e.getMessage());
+    assertTrue(e.getMessage(), e.getMessage().contains("misplaced codec footer (file truncated?): length=0 but footerLength==16 (resource"));
   }
 }
diff --git a/lucene/core/src/test/org/apache/lucene/codecs/lucene70/TestLucene70DocValuesFormat.java b/lucene/core/src/test/org/apache/lucene/codecs/lucene70/TestLucene70DocValuesFormat.java
index 1355a9d..9ed3173 100644
--- a/lucene/core/src/test/org/apache/lucene/codecs/lucene70/TestLucene70DocValuesFormat.java
+++ b/lucene/core/src/test/org/apache/lucene/codecs/lucene70/TestLucene70DocValuesFormat.java
@@ -60,13 +60,12 @@ import org.apache.lucene.index.SortedNumericDocValues;
 import org.apache.lucene.index.SortedSetDocValues;
 import org.apache.lucene.index.Term;
 import org.apache.lucene.index.Terms;
+import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.index.TermsEnum.SeekStatus;
 import org.apache.lucene.search.DocIdSetIterator;
-import org.apache.lucene.index.TermsEnum;
+import org.apache.lucene.store.ByteBuffersDataInput;
+import org.apache.lucene.store.ByteBuffersDataOutput;
 import org.apache.lucene.store.Directory;
-import org.apache.lucene.store.RAMFile;
-import org.apache.lucene.store.RAMInputStream;
-import org.apache.lucene.store.RAMOutputStream;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefBuilder;
 import org.apache.lucene.util.TestUtil;
@@ -446,8 +445,7 @@ public class TestLucene70DocValuesFormat extends BaseCompressingDocValuesFormatT
     for (int maxDoc = frontier - 1; maxDoc <= frontier + 1; ++maxDoc) {
       final Directory dir = newDirectory();
       IndexWriter w = new IndexWriter(dir, newIndexWriterConfig().setMergePolicy(newLogMergePolicy()));
-      RAMFile buffer = new RAMFile();
-      RAMOutputStream out = new RAMOutputStream(buffer, false);
+      ByteBuffersDataOutput buffer = new ByteBuffersDataOutput();
       Document doc = new Document();
       SortedSetDocValuesField field1 = new SortedSetDocValuesField("sset", new BytesRef());
       doc.add(field1);
@@ -460,13 +458,12 @@ public class TestLucene70DocValuesFormat extends BaseCompressingDocValuesFormatT
         field2.setBytesValue(s2);
         w.addDocument(doc);
         Set<BytesRef> set = new TreeSet<>(Arrays.asList(s1, s2));
-        out.writeVInt(set.size());
+        buffer.writeVInt(set.size());
         for (BytesRef ref : set) {
-          out.writeVInt(ref.length);
-          out.writeBytes(ref.bytes, ref.offset, ref.length);
+          buffer.writeVInt(ref.length);
+          buffer.writeBytes(ref.bytes, ref.offset, ref.length);
         }
       }
-      out.close();
       w.forceMerge(1);
       DirectoryReader r = DirectoryReader.open(w);
       w.close();
@@ -474,7 +471,7 @@ public class TestLucene70DocValuesFormat extends BaseCompressingDocValuesFormatT
       assertEquals(maxDoc, sr.maxDoc());
       SortedSetDocValues values = sr.getSortedSetDocValues("sset");
       assertNotNull(values);
-      RAMInputStream in = new RAMInputStream("", buffer);
+      ByteBuffersDataInput in = buffer.toDataInput();
       BytesRefBuilder b = new BytesRefBuilder();
       for (int i = 0; i < maxDoc; ++i) {
         assertEquals(i, values.nextDoc());
@@ -500,8 +497,8 @@ public class TestLucene70DocValuesFormat extends BaseCompressingDocValuesFormatT
     for (int maxDoc = frontier - 1; maxDoc <= frontier + 1; ++maxDoc) {
       final Directory dir = newDirectory();
       IndexWriter w = new IndexWriter(dir, newIndexWriterConfig().setMergePolicy(newLogMergePolicy()));
-      RAMFile buffer = new RAMFile();
-      RAMOutputStream out = new RAMOutputStream(buffer, false);
+      ByteBuffersDataOutput buffer = new ByteBuffersDataOutput();
+
       Document doc = new Document();
       SortedNumericDocValuesField field1 = new SortedNumericDocValuesField("snum", 0L);
       doc.add(field1);
@@ -513,10 +510,10 @@ public class TestLucene70DocValuesFormat extends BaseCompressingDocValuesFormatT
         field1.setLongValue(s1);
         field2.setLongValue(s2);
         w.addDocument(doc);
-        out.writeVLong(Math.min(s1, s2));
-        out.writeVLong(Math.max(s1, s2));
+        buffer.writeVLong(Math.min(s1, s2));
+        buffer.writeVLong(Math.max(s1, s2));
       }
-      out.close();
+
       w.forceMerge(1);
       DirectoryReader r = DirectoryReader.open(w);
       w.close();
@@ -524,12 +521,12 @@ public class TestLucene70DocValuesFormat extends BaseCompressingDocValuesFormatT
       assertEquals(maxDoc, sr.maxDoc());
       SortedNumericDocValues values = sr.getSortedNumericDocValues("snum");
       assertNotNull(values);
-      RAMInputStream in = new RAMInputStream("", buffer);
+      ByteBuffersDataInput dataInput = buffer.toDataInput();
       for (int i = 0; i < maxDoc; ++i) {
         assertEquals(i, values.nextDoc());
         assertEquals(2, values.docValueCount());
-        assertEquals(in.readVLong(), values.nextValue());
-        assertEquals(in.readVLong(), values.nextValue());
+        assertEquals(dataInput.readVLong(), values.nextValue());
+        assertEquals(dataInput.readVLong(), values.nextValue());
       }
       r.close();
       dir.close();
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterOnDiskFull.java b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterOnDiskFull.java
index 3629721..a00e01b 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterOnDiskFull.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterOnDiskFull.java
@@ -547,7 +547,7 @@ public class TestIndexWriterOnDiskFull extends LuceneTestCase {
                                                 .setMergeScheduler(new ConcurrentMergeScheduler())
                                                 .setCommitOnClose(false));
     writer.commit(); // empty commit, to not create confusing situation with first commit
-    dir.setMaxSizeInBytes(Math.max(1, dir.getRecomputedActualSizeInBytes()));
+    dir.setMaxSizeInBytes(Math.max(1, dir.sizeInBytes()));
     final Document doc = new Document();
     FieldType customType = new FieldType(TextField.TYPE_STORED);
     doc.add(newField("field", "aaa bbb ccc ddd eee fff ggg hhh iii jjj", customType));
diff --git a/lucene/core/src/test/org/apache/lucene/search/TermInSetQueryTest.java b/lucene/core/src/test/org/apache/lucene/search/TermInSetQueryTest.java
index 9633ea4..caabfef 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TermInSetQueryTest.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TermInSetQueryTest.java
@@ -175,7 +175,7 @@ public class TermInSetQueryTest extends LuceneTestCase {
 
   public void testRamBytesUsed() {
     List<BytesRef> terms = new ArrayList<>();
-    final int numTerms = 1000 + random().nextInt(1000);
+    final int numTerms = 10000 + random().nextInt(1000);
     for (int i = 0; i < numTerms; ++i) {
       terms.add(new BytesRef(RandomStrings.randomUnicodeOfLength(random(), 10)));
     }
@@ -183,7 +183,7 @@ public class TermInSetQueryTest extends LuceneTestCase {
     final long actualRamBytesUsed = RamUsageTester.sizeOf(query);
     final long expectedRamBytesUsed = query.ramBytesUsed();
     // error margin within 5%
-    assertEquals(actualRamBytesUsed, expectedRamBytesUsed, actualRamBytesUsed / 20);
+    assertEquals(expectedRamBytesUsed, actualRamBytesUsed, actualRamBytesUsed / 20);
   }
 
   private static class TermsCountingDirectoryReaderWrapper extends FilterDirectoryReader {
diff --git a/lucene/core/src/test/org/apache/lucene/store/BaseDataOutputTestCase.java b/lucene/core/src/test/org/apache/lucene/store/BaseDataOutputTestCase.java
index 4578a4f..d77e133 100644
--- a/lucene/core/src/test/org/apache/lucene/store/BaseDataOutputTestCase.java
+++ b/lucene/core/src/test/org/apache/lucene/store/BaseDataOutputTestCase.java
@@ -24,7 +24,6 @@ import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Random;
-
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.IOUtils.IOConsumer;
 import org.junit.Test;
diff --git a/lucene/core/src/test/org/apache/lucene/store/TestByteBuffersDataInput.java b/lucene/core/src/test/org/apache/lucene/store/TestByteBuffersDataInput.java
index 20e3bd2..3cd76fe 100644
--- a/lucene/core/src/test/org/apache/lucene/store/TestByteBuffersDataInput.java
+++ b/lucene/core/src/test/org/apache/lucene/store/TestByteBuffersDataInput.java
@@ -21,8 +21,8 @@ import static org.junit.Assert.*;
 import java.io.EOFException;
 import java.io.IOException;
 import java.nio.ByteBuffer;
-import java.util.ArrayList;
 import java.util.List;
+import java.util.ArrayList;
 
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.IOUtils.IOConsumer;
@@ -157,6 +157,7 @@ public final class TestByteBuffersDataInput extends RandomizedTest {
   
       byte [] array = dst.toArrayCopy();
       array = ArrayUtil.copyOfSubArray(array, prefix.length, array.length);
+
       for (int i = 0; i < 1000; i++) {
         int offs = randomIntBetween(0, array.length - 1);
         in.seek(offs);
diff --git a/lucene/core/src/test/org/apache/lucene/store/TestByteBuffersDataOutput.java b/lucene/core/src/test/org/apache/lucene/store/TestByteBuffersDataOutput.java
index 893aa37..42e10b0 100644
--- a/lucene/core/src/test/org/apache/lucene/store/TestByteBuffersDataOutput.java
+++ b/lucene/core/src/test/org/apache/lucene/store/TestByteBuffersDataOutput.java
@@ -22,7 +22,6 @@ import java.io.IOException;
 import java.nio.ByteBuffer;
 import java.util.Random;
 import java.util.concurrent.atomic.AtomicInteger;
-
 import org.apache.lucene.util.ArrayUtil;
 import org.junit.Assert;
 import org.junit.Test;
diff --git a/lucene/core/src/test/org/apache/lucene/store/TestGrowableByteArrayDataOutput.java b/lucene/core/src/test/org/apache/lucene/store/TestGrowableByteArrayDataOutput.java
deleted file mode 100644
index 10992b7..0000000
--- a/lucene/core/src/test/org/apache/lucene/store/TestGrowableByteArrayDataOutput.java
+++ /dev/null
@@ -1,80 +0,0 @@
-/*
- * 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.
- */
-
-package org.apache.lucene.store;
-
-import org.apache.lucene.util.LuceneTestCase;
-import org.apache.lucene.util.TestUtil;
-import org.apache.lucene.util.UnicodeUtil;
-import org.junit.Test;
-
-/**
- * Test for {@link GrowableByteArrayDataOutput}
- */
-public class TestGrowableByteArrayDataOutput extends LuceneTestCase {
-
-  @Test
-  public void testWriteSmallStrings() throws Exception {
-    int minSizeForDoublePass = GrowableByteArrayDataOutput.MIN_UTF8_SIZE_TO_ENABLE_DOUBLE_PASS_ENCODING;
-
-    // a simple string encoding test
-    int num = atLeast(1000);
-    for (int i = 0; i < num; i++) {
-      // create a small string such that the single pass approach is used
-      int length = TestUtil.nextInt(random(), 1, minSizeForDoublePass - 1);
-      String unicode = TestUtil.randomFixedByteLengthUnicodeString(random(), length);
-      byte[] utf8 = new byte[UnicodeUtil.maxUTF8Length(unicode.length())];
-      int len = UnicodeUtil.UTF16toUTF8(unicode, 0, unicode.length(), utf8);
-
-      GrowableByteArrayDataOutput dataOutput = new GrowableByteArrayDataOutput(1 << 8);
-      //explicitly write utf8 len so that we know how many bytes it occupies
-      dataOutput.writeVInt(len);
-      int vintLen = dataOutput.getPosition();
-      // now write the string which will internally write number of bytes as a vint and then utf8 bytes
-      dataOutput.writeString(unicode);
-
-      assertEquals("GrowableByteArrayDataOutput wrote the wrong length after encode", len + vintLen * 2, dataOutput.getPosition());
-      for (int j = 0, k = vintLen * 2; j < len; j++, k++) {
-        assertEquals(utf8[j], dataOutput.getBytes()[k]);
-      }
-    }
-  }
-
-  @Test
-  public void testWriteLargeStrings() throws Exception {
-    int minSizeForDoublePass = GrowableByteArrayDataOutput.MIN_UTF8_SIZE_TO_ENABLE_DOUBLE_PASS_ENCODING;
-
-    int num = atLeast(100);
-    for (int i = 0; i < num; i++) {
-      String unicode = TestUtil.randomRealisticUnicodeString(random(), minSizeForDoublePass, 10 * minSizeForDoublePass);
-      byte[] utf8 = new byte[UnicodeUtil.maxUTF8Length(unicode.length())];
-      int len = UnicodeUtil.UTF16toUTF8(unicode, 0, unicode.length(), utf8);
-
-      GrowableByteArrayDataOutput dataOutput = new GrowableByteArrayDataOutput(1 << 8);
-      //explicitly write utf8 len so that we know how many bytes it occupies
-      dataOutput.writeVInt(len);
-      int vintLen = dataOutput.getPosition();
-      // now write the string which will internally write number of bytes as a vint and then utf8 bytes
-      dataOutput.writeString(unicode);
-
-      assertEquals("GrowableByteArrayDataOutput wrote the wrong length after encode", len + vintLen * 2, dataOutput.getPosition());
-      for (int j = 0, k = vintLen * 2; j < len; j++, k++) {
-        assertEquals(utf8[j], dataOutput.getBytes()[k]);
-      }
-    }
-  }
-}
diff --git a/lucene/core/src/test/org/apache/lucene/store/TestHugeRamFile.java b/lucene/core/src/test/org/apache/lucene/store/TestHugeRamFile.java
deleted file mode 100644
index 04c5900..0000000
--- a/lucene/core/src/test/org/apache/lucene/store/TestHugeRamFile.java
+++ /dev/null
@@ -1,102 +0,0 @@
-/*
- * 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.
- */
-package org.apache.lucene.store;
-
-
-import java.io.IOException;
-import java.util.HashMap;
-
-import org.apache.lucene.util.LuceneTestCase;
-
-/** Test huge RAMFile with more than Integer.MAX_VALUE bytes. */
-public class TestHugeRamFile extends LuceneTestCase {
-  
-  private static final long MAX_VALUE = (long) 2 * (long) Integer.MAX_VALUE;
-
-  /** Fake a huge ram file by using the same byte buffer for all 
-   * buffers under maxint. */
-  private static class DenseRAMFile extends RAMFile {
-    private long capacity = 0;
-    private HashMap<Integer,byte[]> singleBuffers = new HashMap<>();
-    @Override
-    protected byte[] newBuffer(int size) {
-      capacity += size;
-      if (capacity <= MAX_VALUE) {
-        // below maxint we reuse buffers
-        byte buf[] = singleBuffers.get(Integer.valueOf(size));
-        if (buf==null) {
-          buf = new byte[size]; 
-          //System.out.println("allocate: "+size);
-          singleBuffers.put(Integer.valueOf(size),buf);
-        }
-        return buf;
-      }
-      //System.out.println("allocate: "+size); System.out.flush();
-      return new byte[size];
-    }
-  }
-  
-  /** Test huge RAMFile with more than Integer.MAX_VALUE bytes. (LUCENE-957) */
-  public void testHugeFile() throws IOException {
-    DenseRAMFile f = new DenseRAMFile();
-    // output part
-    RAMOutputStream out = new RAMOutputStream(f, true);
-    byte b1[] = new byte[RAMOutputStream.BUFFER_SIZE];
-    byte b2[] = new byte[RAMOutputStream.BUFFER_SIZE / 3];
-    for (int i = 0; i < b1.length; i++) {
-      b1[i] = (byte) (i & 0x0007F);
-    }
-    for (int i = 0; i < b2.length; i++) {
-      b2[i] = (byte) (i & 0x0003F);
-    }
-    long n = 0;
-    assertEquals("output length must match",n,out.getFilePointer());
-    while (n <= MAX_VALUE - b1.length) {
-      out.writeBytes(b1,0,b1.length);
-      out.flush();
-      n += b1.length;
-      assertEquals("output length must match",n,out.getFilePointer());
-    }
-    //System.out.println("after writing b1's, length = "+out.length()+" (MAX_VALUE="+MAX_VALUE+")");
-    int m = b2.length;
-    long L = 12;
-    for (int j=0; j<L; j++) {
-      for (int i = 0; i < b2.length; i++) {
-        b2[i]++;
-      }
-      out.writeBytes(b2,0,m);
-      out.flush();
-      n += m;
-      assertEquals("output length must match",n,out.getFilePointer());
-    }
-    out.close();
-    // input part
-    RAMInputStream in = new RAMInputStream("testcase", f);
-    assertEquals("input length must match",n,in.length());
-    //System.out.println("input length = "+in.length()+" % 1024 = "+in.length()%1024);
-    for (int j=0; j<L; j++) {
-      long loc = n - (L-j)*m; 
-      in.seek(loc/3);
-      in.seek(loc);
-      for (int i=0; i<m; i++) {
-        byte bt = in.readByte();
-        byte expected = (byte) (1 + j + (i & 0x0003F));
-        assertEquals("must read same value that was written! j="+j+" i="+i,expected,bt);
-      }
-    }
-  }
-}
diff --git a/lucene/core/src/test/org/apache/lucene/store/TestRAMDirectory.java b/lucene/core/src/test/org/apache/lucene/store/TestRAMDirectory.java
deleted file mode 100644
index 02ea9d7..0000000
--- a/lucene/core/src/test/org/apache/lucene/store/TestRAMDirectory.java
+++ /dev/null
@@ -1,189 +0,0 @@
-/*
- * 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.
- */
-package org.apache.lucene.store;
-
-
-import java.io.EOFException;
-import java.io.IOException;
-import java.nio.file.Files;
-import java.nio.file.Path;
-import java.util.Arrays;
-import java.util.List;
-import java.util.Random;
-
-import org.apache.lucene.analysis.MockAnalyzer;
-import org.apache.lucene.document.Document;
-import org.apache.lucene.document.Field;
-import org.apache.lucene.index.DirectoryReader;
-import org.apache.lucene.index.IndexReader;
-import org.apache.lucene.index.IndexWriter;
-import org.apache.lucene.index.IndexWriterConfig.OpenMode;
-import org.apache.lucene.index.IndexWriterConfig;
-import org.apache.lucene.search.IndexSearcher;
-import org.apache.lucene.util.English;
-import org.apache.lucene.util.IOUtils;
-
-/**
- * JUnit testcase to test RAMDirectory. RAMDirectory itself is used in many testcases,
- * but not one of them uses an different constructor other than the default constructor.
- */
-public class TestRAMDirectory extends BaseDirectoryTestCase {
-  
-  @Override
-  protected Directory getDirectory(Path path) {
-    return new RAMDirectory();
-  }
-  
-  // add enough document so that the index will be larger than RAMDirectory.READ_BUFFER_SIZE
-  private static final int DOCS_TO_ADD = 500;
-
-  private Path buildIndex() throws IOException {
-    Path path = createTempDir("buildIndex");
-    
-    Directory dir = newFSDirectory(path);
-    IndexWriter writer = new IndexWriter(dir, new IndexWriterConfig(
-        new MockAnalyzer(random())).setOpenMode(OpenMode.CREATE));
-    // add some documents
-    Document doc = null;
-    for (int i = 0; i < DOCS_TO_ADD; i++) {
-      doc = new Document();
-      doc.add(newStringField("content", English.intToEnglish(i).trim(), Field.Store.YES));
-      writer.addDocument(doc);
-    }
-    assertEquals(DOCS_TO_ADD, writer.getDocStats().maxDoc);
-    writer.close();
-    dir.close();
-
-    return path;
-  }
-  
-  // LUCENE-1468
-  public void testCopySubdir() throws Throwable {
-    Path path = createTempDir("testsubdir");
-    FSDirectory fsDir = null;
-    try {
-      Files.createDirectory(path.resolve("subdir"));
-      fsDir = new SimpleFSDirectory(path);
-      RAMDirectory ramDir = new RAMDirectory(fsDir, newIOContext(random()));
-      List<String> files = Arrays.asList(ramDir.listAll());
-      assertFalse(files.contains("subdir"));
-    } finally {
-      IOUtils.close(fsDir);
-    }
-  }
-
-  public void testRAMDirectory () throws IOException {
-    Path indexDir = buildIndex();
-    
-    FSDirectory dir = new SimpleFSDirectory(indexDir);
-    MockDirectoryWrapper ramDir = new MockDirectoryWrapper(random(), new RAMDirectory(dir, newIOContext(random())));
-    
-    // close the underlaying directory
-    dir.close();
-    
-    // Check size
-    assertEquals(ramDir.sizeInBytes(), ramDir.getRecomputedSizeInBytes());
-    
-    // open reader to test document count
-    IndexReader reader = DirectoryReader.open(ramDir);
-    assertEquals(DOCS_TO_ADD, reader.numDocs());
-    
-    // open search zo check if all doc's are there
-    IndexSearcher searcher = newSearcher(reader);
-    
-    // search for all documents
-    for (int i = 0; i < DOCS_TO_ADD; i++) {
-      Document doc = searcher.doc(i);
-      assertTrue(doc.getField("content") != null);
-    }
-
-    // cleanup
-    reader.close();
-  }
-  
-  private static final int NUM_THREADS = 10;
-  private static final int DOCS_PER_THREAD = 40;
-  
-  public void testRAMDirectorySize() throws IOException, InterruptedException {
-
-    Path indexDir = buildIndex();
-      
-    FSDirectory dir = new SimpleFSDirectory(indexDir);
-    final MockDirectoryWrapper ramDir = new MockDirectoryWrapper(random(), new RAMDirectory(dir, newIOContext(random())));
-    dir.close();
-    
-    final IndexWriter writer = new IndexWriter(ramDir, new IndexWriterConfig(
-        new MockAnalyzer(random())).setOpenMode(OpenMode.APPEND));
-    writer.forceMerge(1);
-    
-    assertEquals(ramDir.sizeInBytes(), ramDir.getRecomputedSizeInBytes());
-    
-    Thread[] threads = new Thread[NUM_THREADS];
-    for (int i = 0; i< NUM_THREADS; i++) {
-      final int num = i;
-      threads[i] = new Thread(){
-        @Override
-        public void run() {
-          for (int j = 1; j< DOCS_PER_THREAD; j++) {
-            Document doc = new Document();
-            doc.add(newStringField("sizeContent", English.intToEnglish(num* DOCS_PER_THREAD +j).trim(), Field.Store.YES));
-            try {
-              writer.addDocument(doc);
-            } catch (IOException e) {
-              throw new RuntimeException(e);
-            }
-          }
-        }
-      };
-    }
-    for (int i = 0; i< NUM_THREADS; i++) {
-      threads[i].start();
-    }
-    for (int i = 0; i< NUM_THREADS; i++) {
-      threads[i].join();
-    }
-
-    writer.forceMerge(1);
-    assertEquals(ramDir.sizeInBytes(), ramDir.getRecomputedSizeInBytes());
-    
-    writer.close();
-  }
-
-  public void testShouldThrowEOFException() throws Exception {
-    final Random random = random();
-
-    try (Directory dir = newDirectory()) {
-      final int len = 16 + random().nextInt(2048) / 16 * 16;
-      final byte[] bytes = new byte[len];
-
-      try (IndexOutput os = dir.createOutput("foo", newIOContext(random))) {
-        os.writeBytes(bytes, bytes.length);
-      }
-
-      try (IndexInput is = dir.openInput("foo", newIOContext(random))) {
-        expectThrows(EOFException.class, () -> {
-          is.seek(0);
-          // Here, I go past EOF.
-          is.seek(len + random().nextInt(2048));
-          // since EOF is not enforced by the previous call in RAMInputStream
-          // this call to readBytes should throw the exception.
-          is.readBytes(bytes, 0, 16);
-        });
-      }
-    }
-  }
-}
diff --git a/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/PrimaryNode.java b/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/PrimaryNode.java
index 2d3e4d6..6b90c78 100644
--- a/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/PrimaryNode.java
+++ b/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/PrimaryNode.java
@@ -34,21 +34,22 @@ import org.apache.lucene.search.IndexSearcher;
 import org.apache.lucene.search.SearcherFactory;
 import org.apache.lucene.search.SearcherManager;
 import org.apache.lucene.store.AlreadyClosedException;
-import org.apache.lucene.store.RAMFile;
-import org.apache.lucene.store.RAMOutputStream;
+import org.apache.lucene.store.ByteBuffersDataOutput;
+import org.apache.lucene.store.ByteBuffersIndexOutput;
 import org.apache.lucene.util.ThreadInterruptedException;
 
 /*
  * This just asks IndexWriter to open new NRT reader, in order to publish a new NRT point.  This could be improved, if we separated out 1)
  * nrt flush (and incRef the SIS) from 2) opening a new reader, but this is tricky with IW's concurrency, and it would also be hard-ish to share
- * IW's reader pool with our searcher manager.  So we do the simpler solution now, but that adds some unecessary latency to NRT refresh on
+ * IW's reader pool with our searcher manager.  So we do the simpler solution now, but that adds some unnecessary latency to NRT refresh on
  * replicas since step 2) could otherwise be done concurrently with replicas copying files over.
  */
 
-/** Node that holds an IndexWriter, indexing documents into its local index.
+/** 
+ * Node that holds an IndexWriter, indexing documents into its local index.
  *
- * @lucene.experimental */
-
+ * @lucene.experimental 
+ */
 public abstract class PrimaryNode extends Node {
 
   // Current NRT segment infos, incRef'd with IndexWriter.deleter:
@@ -241,11 +242,12 @@ public abstract class PrimaryNode extends Node {
 
     message("top: switch to infos=" + infos.toString() + " version=" + infos.getVersion());
 
-    // Serialize the SegmentInfos:
-    RAMOutputStream out = new RAMOutputStream(new RAMFile(), true);
-    infos.write(dir, out);
-    byte[] infosBytes = new byte[(int) out.getFilePointer()];
-    out.writeTo(infosBytes, 0);
+    // Serialize the SegmentInfos.
+    ByteBuffersDataOutput buffer = new ByteBuffersDataOutput();
+    try (ByteBuffersIndexOutput tmpIndexOutput = new ByteBuffersIndexOutput(buffer, "temporary", "temporary")) {
+      infos.write(dir, tmpIndexOutput);
+    }
+    byte[] infosBytes = buffer.toArrayCopy();
 
     Map<String,FileMetaData> filesMetaData = new HashMap<String,FileMetaData>();
     for(SegmentCommitInfo info : infos) {
diff --git a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleTransLog.java b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleTransLog.java
index ce75a29..9968bd8 100644
--- a/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleTransLog.java
+++ b/lucene/replicator/src/test/org/apache/lucene/replicator/nrt/SimpleTransLog.java
@@ -31,8 +31,8 @@ import org.apache.lucene.document.StringField;
 import org.apache.lucene.document.TextField;
 import org.apache.lucene.index.CorruptIndexException;
 import org.apache.lucene.store.ByteArrayDataInput;
+import org.apache.lucene.store.ByteBuffersDataOutput;
 import org.apache.lucene.store.DataInput;
-import org.apache.lucene.store.RAMOutputStream;
 
 /** This is a stupid yet functional transaction log: it never fsync's, never prunes, it's over-synchronized, it hard-wires id field name to "docid", can
  *  only handle specific docs/fields used by this test, etc.  It's just barely enough to show how a translog could work on top of NRT
@@ -41,7 +41,7 @@ import org.apache.lucene.store.RAMOutputStream;
 class SimpleTransLog implements Closeable {
 
   final FileChannel channel;
-  final RAMOutputStream buffer = new RAMOutputStream();
+  final ByteBuffersDataOutput buffer = ByteBuffersDataOutput.newResettableInstance();
   final byte[] intBuffer = new byte[4];
   final ByteBuffer intByteBuffer = ByteBuffer.wrap(intBuffer);
 
@@ -59,7 +59,7 @@ class SimpleTransLog implements Closeable {
 
   /** Appends an addDocument op */
   public synchronized long addDocument(String id, Document doc) throws IOException {
-    assert buffer.getFilePointer() == 0;
+    assert buffer.size() == 0;
     buffer.writeByte(OP_ADD_DOCUMENT);
     encode(id, doc);
     return flushBuffer();
@@ -67,7 +67,7 @@ class SimpleTransLog implements Closeable {
 
   /** Appends an updateDocument op */
   public synchronized long updateDocument(String id, Document doc) throws IOException {
-    assert buffer.getFilePointer() == 0;
+    assert buffer.size() == 0;
     buffer.writeByte(OP_UPDATE_DOCUMENT);
     encode(id, doc);
     return flushBuffer();
@@ -75,7 +75,7 @@ class SimpleTransLog implements Closeable {
 
   /** Appends a deleteDocuments op */
   public synchronized long deleteDocuments(String id) throws IOException {
-    assert buffer.getFilePointer() == 0;
+    assert buffer.size() == 0;
     buffer.writeByte(OP_DELETE_DOCUMENTS);
     buffer.writeString(id);
     return flushBuffer();
@@ -84,9 +84,8 @@ class SimpleTransLog implements Closeable {
   /** Writes buffer to the file and returns the start position. */
   private synchronized long flushBuffer() throws IOException {
     long pos = channel.position();
-    int len = (int) buffer.getFilePointer();
-    byte[] bytes = new byte[len];
-    buffer.writeTo(bytes, 0);
+    int len = Math.toIntExact(buffer.size());
+    byte[] bytes = buffer.toArrayCopy();
     buffer.reset();
 
     intBuffer[0] = (byte) (len >> 24);
diff --git a/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/VersionBlockTreeTermsWriter.java b/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/VersionBlockTreeTermsWriter.java
index 078ca8b..3f0c2bd 100644
--- a/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/VersionBlockTreeTermsWriter.java
+++ b/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/VersionBlockTreeTermsWriter.java
@@ -34,8 +34,8 @@ import org.apache.lucene.index.IndexOptions;
 import org.apache.lucene.index.SegmentWriteState;
 import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.TermsEnum;
+import org.apache.lucene.store.ByteBuffersDataOutput;
 import org.apache.lucene.store.IndexOutput;
-import org.apache.lucene.store.RAMOutputStream;
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefBuilder;
@@ -47,8 +47,8 @@ import org.apache.lucene.util.fst.Builder;
 import org.apache.lucene.util.fst.ByteSequenceOutputs;
 import org.apache.lucene.util.fst.BytesRefFSTEnum;
 import org.apache.lucene.util.fst.FST;
-import org.apache.lucene.util.fst.PairOutputs.Pair;
 import org.apache.lucene.util.fst.PairOutputs;
+import org.apache.lucene.util.fst.PairOutputs.Pair;
 import org.apache.lucene.util.fst.PositiveIntOutputs;
 import org.apache.lucene.util.fst.Util;
 
@@ -162,7 +162,6 @@ public final class VersionBlockTreeTermsWriter extends FieldsConsumer {
   }
 
   private final List<FieldMetaData> fields = new ArrayList<>();
-  private final String segment;
 
   /** Create a new writer.  The number of items (terms or
    *  sub-blocks) per block will aim to be between
@@ -176,8 +175,6 @@ public final class VersionBlockTreeTermsWriter extends FieldsConsumer {
     throws IOException
   {
     BlockTreeTermsWriter.validateSettings(minItemsInBlock, maxItemsInBlock);
-    segment = state.segmentInfo.name;
-
     maxDoc = state.segmentInfo.maxDoc();
 
     final String termsFileName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, TERMS_EXTENSION);
@@ -326,12 +323,12 @@ public final class VersionBlockTreeTermsWriter extends FieldsConsumer {
       return "BLOCK: " + brToString(prefix);
     }
 
-    public void compileIndex(List<PendingBlock> blocks, RAMOutputStream scratchBytes, IntsRefBuilder scratchIntsRef) throws IOException {
+    public void compileIndex(List<PendingBlock> blocks, ByteBuffersDataOutput scratchBytes, IntsRefBuilder scratchIntsRef) throws IOException {
 
       assert (isFloor && blocks.size() > 1) || (isFloor == false && blocks.size() == 1): "isFloor=" + isFloor + " blocks=" + blocks;
       assert this == blocks.get(0);
 
-      assert scratchBytes.getFilePointer() == 0;
+      assert scratchBytes.size() == 0;
 
       long maxVersionIndex = maxVersion;
 
@@ -360,9 +357,8 @@ public final class VersionBlockTreeTermsWriter extends FieldsConsumer {
       //  System.out.println("  compile index for prefix=" + prefix);
       //}
       //indexBuilder.DEBUG = false;
-      final byte[] bytes = new byte[(int) scratchBytes.getFilePointer()];
+      final byte[] bytes = scratchBytes.toArrayCopy();
       assert bytes.length > 0;
-      scratchBytes.writeTo(bytes, 0);
       indexBuilder.add(Util.toIntsRef(prefix, scratchIntsRef), FST_OUTPUTS.newPair(new BytesRef(bytes, 0, bytes.length), Long.MAX_VALUE - maxVersionIndex));
       scratchBytes.reset();
 
@@ -403,7 +399,7 @@ public final class VersionBlockTreeTermsWriter extends FieldsConsumer {
     }
   }
 
-  private final RAMOutputStream scratchBytes = new RAMOutputStream();
+  private final ByteBuffersDataOutput scratchBytes = ByteBuffersDataOutput.newResettableInstance();
   private final IntsRefBuilder scratchIntsRef = new IntsRefBuilder();
 
   class TermsWriter {
@@ -615,7 +611,7 @@ public final class VersionBlockTreeTermsWriter extends FieldsConsumer {
             assert longs[pos] >= 0;
             metaWriter.writeVLong(longs[pos]);
           }
-          bytesWriter.writeTo(metaWriter);
+          bytesWriter.copyTo(metaWriter);
           bytesWriter.reset();
           absolute = false;
         }
@@ -658,7 +654,7 @@ public final class VersionBlockTreeTermsWriter extends FieldsConsumer {
               assert longs[pos] >= 0;
               metaWriter.writeVLong(longs[pos]);
             }
-            bytesWriter.writeTo(metaWriter);
+            bytesWriter.copyTo(metaWriter);
             bytesWriter.reset();
             absolute = false;
           } else {
@@ -700,13 +696,13 @@ public final class VersionBlockTreeTermsWriter extends FieldsConsumer {
       // search on lookup
 
       // Write suffixes byte[] blob to terms dict output:
-      out.writeVInt((int) (suffixWriter.getFilePointer() << 1) | (isLeafBlock ? 1:0));
-      suffixWriter.writeTo(out);
+      out.writeVInt((int) (suffixWriter.size() << 1) | (isLeafBlock ? 1:0));
+      suffixWriter.copyTo(out);
       suffixWriter.reset();
 
       // Write term meta data byte[] blob
-      out.writeVInt((int) metaWriter.getFilePointer());
-      metaWriter.writeTo(out);
+      out.writeVInt((int) metaWriter.size());
+      metaWriter.copyTo(out);
       metaWriter.reset();
 
       // if (DEBUG) {
@@ -831,9 +827,9 @@ public final class VersionBlockTreeTermsWriter extends FieldsConsumer {
       }
     }
 
-    private final RAMOutputStream suffixWriter = new RAMOutputStream();
-    private final RAMOutputStream metaWriter = new RAMOutputStream();
-    private final RAMOutputStream bytesWriter = new RAMOutputStream();
+    private final ByteBuffersDataOutput suffixWriter = ByteBuffersDataOutput.newResettableInstance();
+    private final ByteBuffersDataOutput metaWriter = ByteBuffersDataOutput.newResettableInstance();
+    private final ByteBuffersDataOutput bytesWriter = ByteBuffersDataOutput.newResettableInstance();
   }
 
   private boolean closed;
diff --git a/lucene/test-framework/src/java/org/apache/lucene/store/BaseDirectoryTestCase.java b/lucene/test-framework/src/java/org/apache/lucene/store/BaseDirectoryTestCase.java
index d9eeb88..4d12f0a 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/store/BaseDirectoryTestCase.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/store/BaseDirectoryTestCase.java
@@ -40,9 +40,6 @@ import java.util.stream.Collectors;
 import java.util.stream.IntStream;
 import java.util.zip.CRC32;
 
-import com.carrotsearch.randomizedtesting.RandomizedTest;
-import com.carrotsearch.randomizedtesting.generators.RandomBytes;
-import com.carrotsearch.randomizedtesting.generators.RandomPicks;
 import org.apache.lucene.index.DirectoryReader;
 import org.apache.lucene.index.IndexFileNames;
 import org.apache.lucene.index.IndexNotFoundException;
@@ -52,8 +49,12 @@ import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.TestUtil;
 import org.junit.Assert;
 
+import com.carrotsearch.randomizedtesting.RandomizedTest;
+import com.carrotsearch.randomizedtesting.generators.RandomBytes;
+import com.carrotsearch.randomizedtesting.generators.RandomPicks;
+
 /**
- * Base class for per-Directory tests.
+ * Base class for {@link Directory} implementations.
  */
 public abstract class BaseDirectoryTestCase extends LuceneTestCase {
 
@@ -410,7 +411,6 @@ public abstract class BaseDirectoryTestCase extends LuceneTestCase {
       AtomicBoolean stop = new AtomicBoolean();
       Thread writer = new Thread(() -> {
         try {
-          Random rnd = new Random(RandomizedTest.randomLong() + 1);
           for (int i = 0, max = RandomizedTest.randomIntBetween(500, 1000); i < max; i++) {
             String fileName = "file-" + i;
             try (IndexOutput output = dir.createOutput(fileName, newIOContext(random()))) {
@@ -640,7 +640,8 @@ public abstract class BaseDirectoryTestCase extends LuceneTestCase {
   // LUCENE-3541
   public void testCopyBytesWithThreads() throws Exception {
     try (Directory d = getDirectory(createTempDir("testCopyBytesWithThreads"))) {
-      byte data[] = RandomBytes.randomBytesOfLengthBetween(random(), 101, 10000);
+      int headerLen = 100;
+      byte data[] = RandomBytes.randomBytesOfLengthBetween(random(), headerLen + 1, 10000);
 
       IndexOutput output = d.createOutput("data", IOContext.DEFAULT);
       output.writeBytes(data, 0, data.length);
@@ -648,8 +649,8 @@ public abstract class BaseDirectoryTestCase extends LuceneTestCase {
 
       IndexInput input = d.openInput("data", IOContext.DEFAULT);
       IndexOutput outputHeader = d.createOutput("header", IOContext.DEFAULT);
-      // copy our 100-byte header
-      outputHeader.copyBytes(input, 100);
+      // copy our header
+      outputHeader.copyBytes(input, headerLen);
       outputHeader.close();
 
       // now make N copies of the remaining bytes
@@ -662,7 +663,7 @@ public abstract class BaseDirectoryTestCase extends LuceneTestCase {
               try {
                 start.await();
                 IndexOutput dst = d.createOutput("copy" + i, IOContext.DEFAULT);
-                dst.copyBytes(src, src.length() - 100);
+                dst.copyBytes(src, src.length() - headerLen);
                 dst.close();
               } catch (Exception e) {
                 throw new RuntimeException(e);
@@ -680,8 +681,8 @@ public abstract class BaseDirectoryTestCase extends LuceneTestCase {
       for (int i = 0; i < threads; i++) {
         try (IndexInput copiedData = d.openInput("copy" + i, IOContext.DEFAULT)) {
           byte[] dataCopy = new byte[data.length];
-          System.arraycopy(data, 0, dataCopy, 0, 100);
-          copiedData.readBytes(dataCopy, 100, data.length - 100);
+          System.arraycopy(data, 0, dataCopy, 0, headerLen);
+          copiedData.readBytes(dataCopy, headerLen, data.length - headerLen);
           assertArrayEquals(data, dataCopy);
         }
       }
@@ -723,7 +724,6 @@ public abstract class BaseDirectoryTestCase extends LuceneTestCase {
   }
   
   // random access APIs
-
   public void testRandomLong() throws Exception {
     try (Directory dir = getDirectory(createTempDir("testLongs"))) {
       IndexOutput output = dir.createOutput("longs", newIOContext(random()));
@@ -935,7 +935,7 @@ public abstract class BaseDirectoryTestCase extends LuceneTestCase {
       output.close();
 
       IndexInput input = dir.openInput("bytes", newIOContext(random()));
-      // seek to a random spot shouldnt impact slicing.
+      // seek to a random spot should not impact slicing.
       input.seek(TestUtil.nextLong(random(), 0, input.length()));
       for (int i = 0; i < num; i += 16) {
         IndexInput slice1 = input.slice("slice1", i, num - i);
diff --git a/lucene/test-framework/src/java/org/apache/lucene/store/MockDirectoryWrapper.java b/lucene/test-framework/src/java/org/apache/lucene/store/MockDirectoryWrapper.java
index 15f6454..316796a 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/store/MockDirectoryWrapper.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/store/MockDirectoryWrapper.java
@@ -255,20 +255,15 @@ public class MockDirectoryWrapper extends BaseDirectoryWrapper {
   }
 
   public synchronized final long sizeInBytes() throws IOException {
-    if (in instanceof RAMDirectory)
-      return ((RAMDirectory) in).ramBytesUsed();
-    else {
-      // hack
-      long size = 0;
-      for (String file : in.listAll()) {
-        // hack 2: see TODO in ExtrasFS (ideally it would always return 0 byte
-        // size for extras it creates, even though the size of non-regular files is not defined)
-        if (!file.startsWith("extra")) {
-          size += in.fileLength(file);
-        }
+    long size = 0;
+    for (String file : in.listAll()) {
+      // hack 2: see TODO in ExtrasFS (ideally it would always return 0 byte
+      // size for extras it creates, even though the size of non-regular files is not defined)
+      if (!file.startsWith("extra")) {
+        size += in.fileLength(file);
       }
-      return size;
     }
+    return size;
   }
 
   public synchronized void corruptUnknownFiles() throws IOException {
@@ -510,7 +505,7 @@ public class MockDirectoryWrapper extends BaseDirectoryWrapper {
     return this.maxUsedSize;
   }
   public void resetMaxUsedSizeInBytes() throws IOException {
-    this.maxUsedSize = getRecomputedActualSizeInBytes();
+    this.maxUsedSize = sizeInBytes();
   }
 
   /**
@@ -773,32 +768,6 @@ public class MockDirectoryWrapper extends BaseDirectoryWrapper {
     addFileHandle(ii, name, Handle.Input);
     return ii;
   }
-  
-  /** Provided for testing purposes.  Use sizeInBytes() instead. */
-  public synchronized final long getRecomputedSizeInBytes() throws IOException {
-    if (!(in instanceof RAMDirectory))
-      return sizeInBytes();
-    long size = 0;
-    for(final RAMFile file: ((RAMDirectory)in).fileMap.values()) {
-      size += file.ramBytesUsed();
-    }
-    return size;
-  }
-
-  /** Like getRecomputedSizeInBytes(), but, uses actual file
-   * lengths rather than buffer allocations (which are
-   * quantized up to nearest
-   * RAMOutputStream.BUFFER_SIZE (now 1024) bytes.
-   */
-
-  public final synchronized long getRecomputedActualSizeInBytes() throws IOException {
-    if (!(in instanceof RAMDirectory))
-      return sizeInBytes();
-    long size = 0;
-    for (final RAMFile file : ((RAMDirectory)in).fileMap.values())
-      size += file.length;
-    return size;
-  }
 
   // NOTE: This is off by default; see LUCENE-5574
   private volatile boolean assertNoUnreferencedFilesOnClose;
diff --git a/lucene/test-framework/src/java/org/apache/lucene/store/MockIndexOutputWrapper.java b/lucene/test-framework/src/java/org/apache/lucene/store/MockIndexOutputWrapper.java
index 510e930..17fb07e 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/store/MockIndexOutputWrapper.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/store/MockIndexOutputWrapper.java
@@ -59,7 +59,7 @@ public class MockIndexOutputWrapper extends IndexOutput {
     if (dir.maxSize != 0 && freeSpace <= len) {
       // Compute the real disk free.  This will greatly slow
       // down our test but makes it more accurate:
-      realUsage = dir.getRecomputedActualSizeInBytes();
+      realUsage = dir.sizeInBytes();
       freeSpace = dir.maxSize - realUsage;
     }
 
@@ -75,7 +75,7 @@ public class MockIndexOutputWrapper extends IndexOutput {
       if (realUsage > dir.maxUsedSize) {
         dir.maxUsedSize = realUsage;
       }
-      String message = "fake disk full at " + dir.getRecomputedActualSizeInBytes() + " bytes when writing " + name + " (file length=" + delegate.getFilePointer();
+      String message = "fake disk full at " + dir.sizeInBytes() + " bytes when writing " + name + " (file length=" + delegate.getFilePointer();
       if (freeSpace > 0) {
         message += "; wrote " + freeSpace + " of " + len + " bytes";
       }
@@ -106,7 +106,7 @@ public class MockIndexOutputWrapper extends IndexOutput {
       if (dir.trackDiskUsage) {
         // Now compute actual disk usage & track the maxUsedSize
         // in the MockDirectoryWrapper:
-        long size = dir.getRecomputedActualSizeInBytes();
+        long size = dir.sizeInBytes();
         if (size > dir.maxUsedSize) {
           dir.maxUsedSize = size;
         }
diff --git a/lucene/test-framework/src/java/org/apache/lucene/util/RamUsageTester.java b/lucene/test-framework/src/java/org/apache/lucene/util/RamUsageTester.java
index 6437d8e..9b7b666 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/util/RamUsageTester.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/util/RamUsageTester.java
@@ -21,6 +21,7 @@ import java.io.File;
 import java.lang.reflect.Array;
 import java.lang.reflect.Field;
 import java.lang.reflect.Modifier;
+import java.nio.ByteBuffer;
 import java.nio.file.Path;
 import java.security.AccessController;
 import java.security.PrivilegedAction;
@@ -211,6 +212,8 @@ public final class RamUsageTester {
       a(StringBuffer.class, v -> charArraySize(v.capacity()));
       // Types with large buffers:
       a(ByteArrayOutputStream.class, v -> byteArraySize(v.size()));
+      // Approximate ByteBuffers with their underling storage (ignores field overhead).
+      a(ByteBuffer.class, v -> byteArraySize(v.capacity()));
       // For File and Path, we just take the length of String representation as approximation:
       a(File.class, v -> charArraySize(v.toString().length()));
       a(Path.class, v -> charArraySize(v.toString().length()));