You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by si...@apache.org on 2020/09/08 16:48:59 UTC

[lucene-solr] branch branch_8x updated: LUCENE-9511: Include StoredFieldsWriter in DWPT accounting (#1839)

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

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


The following commit(s) were added to refs/heads/branch_8x by this push:
     new 72448a9  LUCENE-9511: Include StoredFieldsWriter in DWPT accounting (#1839)
72448a9 is described below

commit 72448a9f77c9b6bd9458ddcb0322d2dd5b3c890c
Author: Simon Willnauer <si...@apache.org>
AuthorDate: Tue Sep 8 18:18:13 2020 +0200

    LUCENE-9511: Include StoredFieldsWriter in DWPT accounting (#1839)
    
    StoredFieldsWriter might consume some heap space memory that
    can have a significant impact on decisions made in the IW if
    writers should be stalled or DWPTs should be flushed if memory
    settings are small in IWC and flushes are frequent. This change adds
    RAM accounting to the StoredFieldsWriter since it's part of the
    DWPT lifecycle and not just present during flush.
---
 lucene/CHANGES.txt                                 |  4 +++
 .../simpletext/SimpleTextStoredFieldsWriter.java   |  9 +++---
 .../apache/lucene/codecs/StoredFieldsWriter.java   |  3 +-
 .../compressing/CompressingStoredFieldsWriter.java |  7 ++++-
 .../apache/lucene/index/DefaultIndexingChain.java  | 19 +++++++------
 .../java/org/apache/lucene/index/DocConsumer.java  |  3 +-
 .../lucene/index/DocumentsWriterFlushControl.java  |  4 +--
 .../lucene/index/DocumentsWriterPerThread.java     | 33 ++++++++++++++--------
 .../apache/lucene/index/FreqProxTermsWriter.java   |  5 ++--
 .../org/apache/lucene/index/PointValuesWriter.java |  6 ++--
 .../lucene/index/SortingStoredFieldsConsumer.java  | 15 +++++-----
 .../lucene/index/SortingTermVectorsConsumer.java   |  4 +--
 .../apache/lucene/index/StoredFieldsConsumer.java  | 24 ++++++++++------
 .../apache/lucene/index/TermVectorsConsumer.java   |  5 ++--
 .../java/org/apache/lucene/index/TermsHash.java    |  6 ++--
 .../lucene/store/GrowableByteArrayDataOutput.java  |  8 +++++-
 .../lucene/index/TestFlushByRamOrCountsPolicy.java |  2 +-
 .../apache/lucene/index/TestIndexWriterDelete.java |  2 +-
 .../asserting/AssertingStoredFieldsFormat.java     | 10 +++++++
 .../codecs/cranky/CrankyStoredFieldsFormat.java    | 12 ++++++++
 20 files changed, 121 insertions(+), 60 deletions(-)

diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 7b52d37..b436f68 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -43,6 +43,10 @@ Improvements
 
 * LUCENE-9501: Improve how Asserting* test classes handle singleton doc values.
 
+* LUCENE-9511: Include StoredFieldsWriter in DWPT accounting to ensure that it's 
+  heap consumption is taken into account when IndexWriter stalls or should flush
+  DWPTs. (Simon Willnauer)
+
 Optimizations
 ---------------------
 
diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextStoredFieldsWriter.java b/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextStoredFieldsWriter.java
index b59114a..7929b53 100644
--- a/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextStoredFieldsWriter.java
+++ b/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextStoredFieldsWriter.java
@@ -39,8 +39,6 @@ import org.apache.lucene.util.IOUtils;
  */
 public class SimpleTextStoredFieldsWriter extends StoredFieldsWriter {
   private int numDocsWritten = 0;
-  private final Directory directory;
-  private final String segment;
   private IndexOutput out;
   
   final static String FIELDS_EXTENSION = "fld";
@@ -62,8 +60,6 @@ public class SimpleTextStoredFieldsWriter extends StoredFieldsWriter {
   private final BytesRefBuilder scratch = new BytesRefBuilder();
   
   public SimpleTextStoredFieldsWriter(Directory directory, String segment, IOContext context) throws IOException {
-    this.directory = directory;
-    this.segment = segment;
     boolean success = false;
     try {
       out = directory.createOutput(IndexFileNames.segmentFileName(segment, "", FIELDS_EXTENSION), context);
@@ -181,4 +177,9 @@ public class SimpleTextStoredFieldsWriter extends StoredFieldsWriter {
   private void newLine() throws IOException {
     SimpleTextUtil.writeNewline(out);
   }
+
+  @Override
+  public long ramBytesUsed() {
+    return Integer.BYTES; // something > 0
+  }
 }
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/StoredFieldsWriter.java b/lucene/core/src/java/org/apache/lucene/codecs/StoredFieldsWriter.java
index 39ade42..1dcf20e 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/StoredFieldsWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/StoredFieldsWriter.java
@@ -33,6 +33,7 @@ import org.apache.lucene.index.IndexableField;
 import org.apache.lucene.index.IndexableFieldType;
 import org.apache.lucene.index.MergeState;
 import org.apache.lucene.index.StoredFieldVisitor;
+import org.apache.lucene.util.Accountable;
 import org.apache.lucene.util.BytesRef;
 
 import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
@@ -51,7 +52,7 @@ import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
  * 
  * @lucene.experimental
  */
-public abstract class StoredFieldsWriter implements Closeable {
+public abstract class StoredFieldsWriter implements Closeable, Accountable {
   
   /** Sole constructor. (For invocation by subclass 
    *  constructors, typically implicit.) */
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 a9ee2e8..8d64841 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
@@ -671,7 +671,7 @@ public final class CompressingStoredFieldsWriter extends StoredFieldsWriter {
     private final int maxDoc;
     int docID = -1;
 
-    public CompressingStoredFieldsMergeSub(CompressingStoredFieldsReader reader, MergeState.DocMap docMap, int maxDoc) {
+    CompressingStoredFieldsMergeSub(CompressingStoredFieldsReader reader, MergeState.DocMap docMap, int maxDoc) {
       super(docMap);
       this.maxDoc = maxDoc;
       this.reader = reader;
@@ -687,4 +687,9 @@ public final class CompressingStoredFieldsWriter extends StoredFieldsWriter {
       }
     }
   }
+
+  @Override
+  public long ramBytesUsed() {
+    return bufferedDocs.ramBytesUsed() + numStoredFields.length * Integer.BYTES + endOffsets.length * Integer.BYTES;
+  }
 }
diff --git a/lucene/core/src/java/org/apache/lucene/index/DefaultIndexingChain.java b/lucene/core/src/java/org/apache/lucene/index/DefaultIndexingChain.java
index ab35cc2..dbe6ccf 100644
--- a/lucene/core/src/java/org/apache/lucene/index/DefaultIndexingChain.java
+++ b/lucene/core/src/java/org/apache/lucene/index/DefaultIndexingChain.java
@@ -52,7 +52,7 @@ import org.apache.lucene.util.RamUsageEstimator;
 /** Default general purpose indexing chain, which handles
  *  indexing all types of fields. */
 final class DefaultIndexingChain extends DocConsumer {
-  final Counter bytesUsed;
+  final Counter bytesUsed = Counter.newCounter();
   final DocumentsWriterPerThread docWriter;
   final FieldInfos.Builder fieldInfos;
 
@@ -74,21 +74,20 @@ final class DefaultIndexingChain extends DocConsumer {
   private PerField[] fields = new PerField[1];
   private final InfoStream infoStream;
 
-  public DefaultIndexingChain(DocumentsWriterPerThread docWriter) {
+  DefaultIndexingChain(DocumentsWriterPerThread docWriter) {
     this.docWriter = docWriter;
     this.fieldInfos = docWriter.getFieldInfosBuilder();
-    this.bytesUsed = docWriter.bytesUsed;
     this.infoStream = docWriter.getIndexWriterConfig().getInfoStream();
 
     final TermsHash termVectorsWriter;
     if (docWriter.getSegmentInfo().getIndexSort() == null) {
-      storedFieldsConsumer = new StoredFieldsConsumer(docWriter);
+      storedFieldsConsumer = new StoredFieldsConsumer(docWriter.codec, docWriter.directory, docWriter.getSegmentInfo());
       termVectorsWriter = new TermVectorsConsumer(docWriter);
     } else {
-      storedFieldsConsumer = new SortingStoredFieldsConsumer(docWriter);
+      storedFieldsConsumer = new SortingStoredFieldsConsumer(docWriter.codec, docWriter.directory, docWriter.getSegmentInfo());
       termVectorsWriter = new SortingTermVectorsConsumer(docWriter);
     }
-    termsHash = new FreqProxTermsWriter(docWriter, termVectorsWriter);
+    termsHash = new FreqProxTermsWriter(docWriter, bytesUsed, termVectorsWriter);
   }
 
   private LeafReader getDocValuesLeafReader() {
@@ -302,7 +301,6 @@ final class DefaultIndexingChain extends DocConsumer {
 
   /** Writes all buffered doc values (called from {@link #flush}). */
   private void writeDocValues(SegmentWriteState state, Sorter.DocMap sortMap) throws IOException {
-    int maxDoc = state.segmentInfo.maxDoc();
     DocValuesConsumer dvConsumer = null;
     boolean success = false;
     try {
@@ -582,7 +580,7 @@ final class DefaultIndexingChain extends DocConsumer {
     fp.fieldInfo.setPointDimensions(pointDimensionCount, pointIndexDimensionCount, dimensionNumBytes);
 
     if (fp.pointValuesWriter == null) {
-      fp.pointValuesWriter = new PointValuesWriter(docWriter, fp.fieldInfo);
+      fp.pointValuesWriter = new PointValuesWriter(docWriter.byteBlockAllocator, bytesUsed, fp.fieldInfo);
     }
     fp.pointValuesWriter.addPackedValue(docID, field.binaryValue());
   }
@@ -774,6 +772,11 @@ final class DefaultIndexingChain extends DocConsumer {
     info.setIndexOptions(indexOptions);
   }
 
+  @Override
+  public long ramBytesUsed() {
+    return bytesUsed.get() + storedFieldsConsumer.ramBytesUsed();
+  }
+
   /** NOTE: not static: accesses at least docState, termsHash. */
   private final class PerField implements Comparable<PerField> {
 
diff --git a/lucene/core/src/java/org/apache/lucene/index/DocConsumer.java b/lucene/core/src/java/org/apache/lucene/index/DocConsumer.java
index 90a98f4..eff4433 100644
--- a/lucene/core/src/java/org/apache/lucene/index/DocConsumer.java
+++ b/lucene/core/src/java/org/apache/lucene/index/DocConsumer.java
@@ -20,8 +20,9 @@ package org.apache.lucene.index;
 import java.io.IOException;
 
 import org.apache.lucene.search.DocIdSetIterator;
+import org.apache.lucene.util.Accountable;
 
-abstract class DocConsumer {
+abstract class DocConsumer implements Accountable {
   abstract void processDocument(int docId, Iterable<? extends IndexableField> document) throws IOException;
   abstract Sorter.DocMap flush(final SegmentWriteState state) throws IOException;
   abstract void abort() throws IOException;
diff --git a/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterFlushControl.java b/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterFlushControl.java
index 0e51dff..a88c4fe 100644
--- a/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterFlushControl.java
+++ b/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterFlushControl.java
@@ -180,7 +180,7 @@ final class DocumentsWriterFlushControl implements Accountable, Closeable {
         } else {
           flushPolicy.onInsert(this, perThread);
         }
-        if (!perThread.isFlushPending() && perThread.bytesUsed() > hardMaxBytesPerDWPT) {
+        if (!perThread.isFlushPending() && perThread.ramBytesUsed() > hardMaxBytesPerDWPT) {
           // Safety check to prevent a single DWPT exceeding its RAM limit. This
           // is super important since we can not address more than 2048 MB per DWPT
           setFlushPending(perThread);
@@ -670,7 +670,7 @@ final class DocumentsWriterFlushControl implements Accountable, Closeable {
     int count = 0;
     for (DocumentsWriterPerThread next : perThreadPool) {
       if (next.isFlushPending() == false && next.getNumDocsInRAM() > 0) {
-        final long nextRam = next.bytesUsed();
+        final long nextRam = next.ramBytesUsed();
         if (infoStream.isEnabled("FP")) {
           infoStream.message("FP", "thread state has " + nextRam + " bytes; docInRAM=" + next.getNumDocsInRAM());
         }
diff --git a/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java b/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java
index 48f676e..fd75a02 100644
--- a/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java
+++ b/lucene/core/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java
@@ -18,6 +18,8 @@ package org.apache.lucene.index;
 
 import java.io.IOException;
 import java.text.NumberFormat;
+import java.util.Arrays;
+import java.util.Collection;
 import java.util.Collections;
 import java.util.HashSet;
 import java.util.Locale;
@@ -33,6 +35,7 @@ import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.FlushInfo;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.TrackingDirectoryWrapper;
+import org.apache.lucene.util.Accountable;
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.ByteBlockPool.Allocator;
@@ -48,7 +51,7 @@ import org.apache.lucene.util.Version;
 import static org.apache.lucene.util.ByteBlockPool.BYTE_BLOCK_MASK;
 import static org.apache.lucene.util.ByteBlockPool.BYTE_BLOCK_SIZE;
 
-final class DocumentsWriterPerThread {
+final class DocumentsWriterPerThread implements Accountable {
 
   LiveIndexWriterConfig getIndexWriterConfig() {
     return indexWriterConfig;
@@ -132,7 +135,7 @@ final class DocumentsWriterPerThread {
   final Codec codec;
   final TrackingDirectoryWrapper directory;
   private final DocConsumer consumer;
-  final Counter bytesUsed;
+  private final Counter bytesUsed;
   
   // Updates for our still-in-RAM (to be flushed next) segment
   private final BufferedUpdates pendingUpdates;
@@ -292,7 +295,7 @@ final class DocumentsWriterPerThread {
     for (int docId = from; docId < to; docId++) {
       deleteDocIDs[numDeletedDocIds++] = docId;
     }
-    bytesUsed.addAndGet((deleteDocIDs.length - size) * Integer.SIZE);
+    bytesUsed.addAndGet((deleteDocIDs.length - size) * Integer.BYTES);
     // NOTE: we do not trigger flush here.  This is
     // potentially a RAM leak, if you have an app that tries
     // to add docs but every single doc always hits a
@@ -338,8 +341,8 @@ final class DocumentsWriterPerThread {
     assert deleteSlice.isEmpty() : "all deletes must be applied in prepareFlush";
     segmentInfo.setMaxDoc(numDocsInRAM);
     final SegmentWriteState flushState = new SegmentWriteState(infoStream, directory, segmentInfo, fieldInfos.finish(),
-        pendingUpdates, new IOContext(new FlushInfo(numDocsInRAM, bytesUsed())));
-    final double startMBUsed = bytesUsed() / 1024. / 1024.;
+        pendingUpdates, new IOContext(new FlushInfo(numDocsInRAM, ramBytesUsed())));
+    final double startMBUsed = ramBytesUsed() / 1024. / 1024.;
 
     // Apply delete-by-docID now (delete-byDocID only
     // happens when an exception is hit processing that
@@ -351,7 +354,7 @@ final class DocumentsWriterPerThread {
         flushState.liveDocs.clear(deleteDocIDs[i]);
       }
       flushState.delCountOnFlush = numDeletedDocIds;
-      bytesUsed.addAndGet(-(deleteDocIDs.length * Integer.SIZE));
+      bytesUsed.addAndGet(-(deleteDocIDs.length * Integer.BYTES));
       deleteDocIDs = null;
 
     }
@@ -545,12 +548,18 @@ final class DocumentsWriterPerThread {
     return segmentInfo;
   }
 
-  long bytesUsed() {
-    return bytesUsed.get() + pendingUpdates.ramBytesUsed();
+  @Override
+  public long ramBytesUsed() {
+    return bytesUsed.get() + pendingUpdates.ramBytesUsed() + consumer.ramBytesUsed();
+  }
+
+  @Override
+  public Collection<Accountable> getChildResources() {
+    return Collections.unmodifiableList(Arrays.asList(pendingUpdates, consumer));
   }
 
   /* Initial chunks size of the shared byte[] blocks used to
-     store postings data */
+       store postings data */
   final static int BYTE_BLOCK_NOT_MASK = ~BYTE_BLOCK_MASK;
 
   /* if you increase this, you must fix field cache impl for
@@ -613,13 +622,13 @@ final class DocumentsWriterPerThread {
   }
 
   /**
-   * Commits the current {@link #bytesUsed()} and stores it's value for later reuse.
+   * Commits the current {@link #ramBytesUsed()} and stores it's value for later reuse.
    * The last committed bytes used can be retrieved via {@link #getLastCommittedBytesUsed()}
-   * @return the delta between the current {@link #bytesUsed()} and the current {@link #getLastCommittedBytesUsed()}
+   * @return the delta between the current {@link #ramBytesUsed()} and the current {@link #getLastCommittedBytesUsed()}
    */
   long commitLastBytesUsed() {
     assert isHeldByCurrentThread();
-    long delta = bytesUsed() - lastCommittedBytesUsed;
+    long delta = ramBytesUsed() - lastCommittedBytesUsed;
     lastCommittedBytesUsed += delta;
     return delta;
   }
diff --git a/lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriter.java b/lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriter.java
index 702218b..07a5c5f 100644
--- a/lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriter.java
@@ -32,6 +32,7 @@ import org.apache.lucene.store.DataOutput;
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.CollectionUtil;
+import org.apache.lucene.util.Counter;
 import org.apache.lucene.util.FixedBitSet;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.TimSorter;
@@ -39,8 +40,8 @@ import org.apache.lucene.util.automaton.CompiledAutomaton;
 
 final class FreqProxTermsWriter extends TermsHash {
 
-  public FreqProxTermsWriter(DocumentsWriterPerThread docWriter, TermsHash termVectors) {
-    super(docWriter, true, termVectors);
+  FreqProxTermsWriter(DocumentsWriterPerThread docWriter, Counter bytesUsed, TermsHash termVectors) {
+    super(docWriter, bytesUsed, termVectors);
   }
 
   private void applyDeletes(SegmentWriteState state, Fields fields) throws IOException {
diff --git a/lucene/core/src/java/org/apache/lucene/index/PointValuesWriter.java b/lucene/core/src/java/org/apache/lucene/index/PointValuesWriter.java
index dd99393..d715872 100644
--- a/lucene/core/src/java/org/apache/lucene/index/PointValuesWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/index/PointValuesWriter.java
@@ -37,10 +37,10 @@ class PointValuesWriter {
   private int lastDocID = -1;
   private final int packedBytesLength;
 
-  public PointValuesWriter(DocumentsWriterPerThread docWriter, FieldInfo fieldInfo) {
+  PointValuesWriter(ByteBlockPool.Allocator allocator, Counter bytesUsed, FieldInfo fieldInfo) {
     this.fieldInfo = fieldInfo;
-    this.iwBytesUsed = docWriter.bytesUsed;
-    this.bytes = new ByteBlockPool(docWriter.byteBlockAllocator);
+    this.iwBytesUsed = bytesUsed;
+    this.bytes = new ByteBlockPool(allocator);
     docIDs = new int[16];
     iwBytesUsed.addAndGet(16 * Integer.BYTES);
     packedBytesLength = fieldInfo.getPointDimensionCount() * fieldInfo.getPointNumBytes();
diff --git a/lucene/core/src/java/org/apache/lucene/index/SortingStoredFieldsConsumer.java b/lucene/core/src/java/org/apache/lucene/index/SortingStoredFieldsConsumer.java
index 97253a5..a861f72 100644
--- a/lucene/core/src/java/org/apache/lucene/index/SortingStoredFieldsConsumer.java
+++ b/lucene/core/src/java/org/apache/lucene/index/SortingStoredFieldsConsumer.java
@@ -24,9 +24,11 @@ import java.util.Map;
 
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.TokenStream;
+import org.apache.lucene.codecs.Codec;
 import org.apache.lucene.codecs.StoredFieldsReader;
 import org.apache.lucene.codecs.StoredFieldsWriter;
 import org.apache.lucene.document.StoredField;
+import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.IOUtils;
@@ -34,16 +36,15 @@ import org.apache.lucene.util.IOUtils;
 final class SortingStoredFieldsConsumer extends StoredFieldsConsumer {
   TrackingTmpOutputDirectoryWrapper tmpDirectory;
 
-  SortingStoredFieldsConsumer(DocumentsWriterPerThread docWriter) {
-    super(docWriter);
+  SortingStoredFieldsConsumer(Codec codec, Directory directory, SegmentInfo info) {
+    super(codec, directory, info);
   }
 
   @Override
   protected void initStoredFieldsWriter() throws IOException {
     if (writer == null) {
-      this.tmpDirectory = new TrackingTmpOutputDirectoryWrapper(docWriter.directory);
-      this.writer = docWriter.codec.storedFieldsFormat().fieldsWriter(tmpDirectory, docWriter.getSegmentInfo(),
-          IOContext.DEFAULT);
+      this.tmpDirectory = new TrackingTmpOutputDirectoryWrapper(directory);
+      this.writer = codec.storedFieldsFormat().fieldsWriter(tmpDirectory, info, IOContext.DEFAULT);
     }
   }
 
@@ -57,10 +58,10 @@ final class SortingStoredFieldsConsumer extends StoredFieldsConsumer {
       }
       return;
     }
-    StoredFieldsReader reader = docWriter.codec.storedFieldsFormat()
+    StoredFieldsReader reader = codec.storedFieldsFormat()
         .fieldsReader(tmpDirectory, state.segmentInfo, state.fieldInfos, IOContext.DEFAULT);
     StoredFieldsReader mergeReader = reader.getMergeInstance();
-    StoredFieldsWriter sortWriter = docWriter.codec.storedFieldsFormat()
+    StoredFieldsWriter sortWriter = codec.storedFieldsFormat()
         .fieldsWriter(state.directory, state.segmentInfo, IOContext.DEFAULT);
     try {
       reader.checkIntegrity();
diff --git a/lucene/core/src/java/org/apache/lucene/index/SortingTermVectorsConsumer.java b/lucene/core/src/java/org/apache/lucene/index/SortingTermVectorsConsumer.java
index 955dd8a..5c0fc87 100644
--- a/lucene/core/src/java/org/apache/lucene/index/SortingTermVectorsConsumer.java
+++ b/lucene/core/src/java/org/apache/lucene/index/SortingTermVectorsConsumer.java
@@ -33,7 +33,7 @@ import org.apache.lucene.util.IOUtils;
 final class SortingTermVectorsConsumer extends TermVectorsConsumer {
   TrackingTmpOutputDirectoryWrapper tmpDirectory;
 
-  public SortingTermVectorsConsumer(DocumentsWriterPerThread docWriter) {
+  SortingTermVectorsConsumer(DocumentsWriterPerThread docWriter) {
     super(docWriter);
   }
 
@@ -71,7 +71,7 @@ final class SortingTermVectorsConsumer extends TermVectorsConsumer {
   @Override
   void initTermVectorsWriter() throws IOException {
     if (writer == null) {
-      IOContext context = new IOContext(new FlushInfo(docWriter.getNumDocsInRAM(), docWriter.bytesUsed()));
+      IOContext context = new IOContext(new FlushInfo(docWriter.getNumDocsInRAM(), docWriter.ramBytesUsed()));
       tmpDirectory = new TrackingTmpOutputDirectoryWrapper(docWriter.directory);
       writer = docWriter.codec.termVectorsFormat().vectorsWriter(tmpDirectory, docWriter.getSegmentInfo(), context);
       lastDocID = 0;
diff --git a/lucene/core/src/java/org/apache/lucene/index/StoredFieldsConsumer.java b/lucene/core/src/java/org/apache/lucene/index/StoredFieldsConsumer.java
index 56b3e4d..f73bb75 100644
--- a/lucene/core/src/java/org/apache/lucene/index/StoredFieldsConsumer.java
+++ b/lucene/core/src/java/org/apache/lucene/index/StoredFieldsConsumer.java
@@ -19,25 +19,29 @@ package org.apache.lucene.index;
 
 import java.io.IOException;
 
+import org.apache.lucene.codecs.Codec;
 import org.apache.lucene.codecs.StoredFieldsWriter;
+import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.util.IOUtils;
 
 class StoredFieldsConsumer {
-  final DocumentsWriterPerThread docWriter;
+  final Codec codec;
+  final Directory directory;
+  final SegmentInfo info;
   StoredFieldsWriter writer;
-  int lastDoc;
+  private int lastDoc;
 
-  StoredFieldsConsumer(DocumentsWriterPerThread docWriter) {
-    this.docWriter = docWriter;
+  StoredFieldsConsumer(Codec codec, Directory directory, SegmentInfo info) {
+    this.codec = codec;
+    this.directory = directory;
+    this.info = info;
     this.lastDoc = -1;
   }
 
   protected void initStoredFieldsWriter() throws IOException {
-    if (writer == null) {
-      this.writer =
-          docWriter.codec.storedFieldsFormat().fieldsWriter(docWriter.directory, docWriter.getSegmentInfo(),
-              IOContext.DEFAULT);
+    if (writer == null) { // TODO can we allocate this in the ctor? we call start document for every doc anyway
+      this.writer = codec.storedFieldsFormat().fieldsWriter(directory, info, IOContext.DEFAULT);
     }
   }
 
@@ -82,4 +86,8 @@ class StoredFieldsConsumer {
       writer = null;
     }
   }
+
+  long ramBytesUsed() {
+    return writer == null ? 0 : writer.ramBytesUsed();
+  }
 }
diff --git a/lucene/core/src/java/org/apache/lucene/index/TermVectorsConsumer.java b/lucene/core/src/java/org/apache/lucene/index/TermVectorsConsumer.java
index 80213f9..4f042d2 100644
--- a/lucene/core/src/java/org/apache/lucene/index/TermVectorsConsumer.java
+++ b/lucene/core/src/java/org/apache/lucene/index/TermVectorsConsumer.java
@@ -27,6 +27,7 @@ import org.apache.lucene.store.FlushInfo;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.Counter;
 import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.RamUsageEstimator;
 
@@ -49,7 +50,7 @@ class TermVectorsConsumer extends TermsHash {
   private TermVectorsConsumerPerField[] perFields = new TermVectorsConsumerPerField[1];
 
   TermVectorsConsumer(DocumentsWriterPerThread docWriter) {
-    super(docWriter, false, null);
+    super(docWriter, Counter.newCounter(), null);
     this.docWriter = docWriter;
   }
 
@@ -84,7 +85,7 @@ class TermVectorsConsumer extends TermsHash {
 
   void initTermVectorsWriter() throws IOException {
     if (writer == null) {
-      IOContext context = new IOContext(new FlushInfo(docWriter.getNumDocsInRAM(), docWriter.bytesUsed()));
+      IOContext context = new IOContext(new FlushInfo(docWriter.getNumDocsInRAM(), docWriter.ramBytesUsed()));
       writer = docWriter.codec.termVectorsFormat().vectorsWriter(docWriter.directory, docWriter.getSegmentInfo(), context);
       lastDocID = 0;
     }
diff --git a/lucene/core/src/java/org/apache/lucene/index/TermsHash.java b/lucene/core/src/java/org/apache/lucene/index/TermsHash.java
index 0f702d9..d2e6b28 100644
--- a/lucene/core/src/java/org/apache/lucene/index/TermsHash.java
+++ b/lucene/core/src/java/org/apache/lucene/index/TermsHash.java
@@ -40,12 +40,10 @@ abstract class TermsHash {
   final ByteBlockPool bytePool;
   ByteBlockPool termBytePool;
   final Counter bytesUsed;
-  final boolean trackAllocations;
 
-  TermsHash(final DocumentsWriterPerThread docWriter, boolean trackAllocations, TermsHash nextTermsHash) {
-    this.trackAllocations = trackAllocations;
+  TermsHash(final DocumentsWriterPerThread docWriter, Counter bytesUsed, TermsHash nextTermsHash) {
     this.nextTermsHash = nextTermsHash;
-    this.bytesUsed = trackAllocations ? docWriter.bytesUsed : Counter.newCounter();
+    this.bytesUsed = bytesUsed;
     intPool = new IntBlockPool(docWriter.intBlockAllocator);
     bytePool = new ByteBlockPool(docWriter.byteBlockAllocator);
 
diff --git a/lucene/core/src/java/org/apache/lucene/store/GrowableByteArrayDataOutput.java b/lucene/core/src/java/org/apache/lucene/store/GrowableByteArrayDataOutput.java
index c74d4ef..9319e9b 100644
--- a/lucene/core/src/java/org/apache/lucene/store/GrowableByteArrayDataOutput.java
+++ b/lucene/core/src/java/org/apache/lucene/store/GrowableByteArrayDataOutput.java
@@ -19,6 +19,7 @@ package org.apache.lucene.store;
 
 import java.io.IOException;
 
+import org.apache.lucene.util.Accountable;
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.UnicodeUtil;
 
@@ -27,7 +28,7 @@ import org.apache.lucene.util.UnicodeUtil;
  *
  * @lucene.internal
  */
-public final class GrowableByteArrayDataOutput extends DataOutput {
+public final class GrowableByteArrayDataOutput extends DataOutput implements Accountable {
 
   /** 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;
@@ -99,4 +100,9 @@ public final class GrowableByteArrayDataOutput extends DataOutput {
   public void reset() {
     length = 0;
   }
+
+  @Override
+  public long ramBytesUsed() {
+    return bytes.length + (scratchBytes == null ? 0 : scratchBytes.length);
+  }
 }
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestFlushByRamOrCountsPolicy.java b/lucene/core/src/test/org/apache/lucene/index/TestFlushByRamOrCountsPolicy.java
index b07456b..7450267 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestFlushByRamOrCountsPolicy.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestFlushByRamOrCountsPolicy.java
@@ -267,7 +267,7 @@ public class TestFlushByRamOrCountsPolicy extends LuceneTestCase {
     long bytesUsed = 0;
     while (allActiveWriter.hasNext()) {
       DocumentsWriterPerThread next = allActiveWriter.next();
-      bytesUsed += next.bytesUsed();
+      bytesUsed += next.ramBytesUsed();
     }
     assertEquals(bytesUsed, flushControl.activeBytes());
   }
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterDelete.java b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterDelete.java
index 1f9781a..12fb840 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterDelete.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterDelete.java
@@ -1044,7 +1044,7 @@ public class TestIndexWriterDelete extends LuceneTestCase {
     // note: tiny rambuffer used, as with a 1MB buffer the test is too slow (flush @ 128,999)
     IndexWriter w = new IndexWriter(dir,
                                     newIndexWriterConfig(new MockAnalyzer(random()))
-                                      .setRAMBufferSizeMB(0.1f)
+                                      .setRAMBufferSizeMB(0.8f)
                                       .setMaxBufferedDocs(1000)
                                       .setMergePolicy(NoMergePolicy.INSTANCE)
                                       .setReaderPooling(false));
diff --git a/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingStoredFieldsFormat.java b/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingStoredFieldsFormat.java
index f5455f5..4a55228 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingStoredFieldsFormat.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingStoredFieldsFormat.java
@@ -161,5 +161,15 @@ public class AssertingStoredFieldsFormat extends StoredFieldsFormat {
       in.close();
       in.close(); // close again
     }
+
+    @Override
+    public long ramBytesUsed() {
+      return in.ramBytesUsed();
+    }
+
+    @Override
+    public Collection<Accountable> getChildResources() {
+      return in.getChildResources();
+    }
   }
 }
diff --git a/lucene/test-framework/src/java/org/apache/lucene/codecs/cranky/CrankyStoredFieldsFormat.java b/lucene/test-framework/src/java/org/apache/lucene/codecs/cranky/CrankyStoredFieldsFormat.java
index 6504931..edd5983 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/codecs/cranky/CrankyStoredFieldsFormat.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/codecs/cranky/CrankyStoredFieldsFormat.java
@@ -17,6 +17,7 @@
 package org.apache.lucene.codecs.cranky;
 
 import java.io.IOException;
+import java.util.Collection;
 import java.util.Random;
 
 import org.apache.lucene.codecs.StoredFieldsFormat;
@@ -29,6 +30,7 @@ import org.apache.lucene.index.MergeState;
 import org.apache.lucene.index.SegmentInfo;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
+import org.apache.lucene.util.Accountable;
 
 class CrankyStoredFieldsFormat extends StoredFieldsFormat {
   final StoredFieldsFormat delegate;
@@ -111,5 +113,15 @@ class CrankyStoredFieldsFormat extends StoredFieldsFormat {
       }
       delegate.writeField(info, field);
     }
+
+    @Override
+    public long ramBytesUsed() {
+      return delegate.ramBytesUsed();
+    }
+
+    @Override
+    public Collection<Accountable> getChildResources() {
+      return delegate.getChildResources();
+    }
   }
 }