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/14 18:30:02 UTC

[lucene-solr] branch branch_8x updated: LUCENE-9514: Include TermVectorsWriter in DWPT accounting (#1847)

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 597d8f6  LUCENE-9514: Include TermVectorsWriter in DWPT accounting (#1847)
597d8f6 is described below

commit 597d8f69522d1c1a85917b126cd5e2cacfcb35e4
Author: Simon Willnauer <si...@apache.org>
AuthorDate: Mon Sep 14 11:11:06 2020 +0200

    LUCENE-9514: Include TermVectorsWriter in DWPT accounting (#1847)
    
    TermVectorsWriter 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 TermVectorsWriter since it's part of the
    DWPT lifecycle and not just present during flush.
---
 lucene/CHANGES.txt                                        |  4 ++++
 .../codecs/simpletext/SimpleTextTermVectorsWriter.java    |  9 +++++----
 .../java/org/apache/lucene/codecs/TermVectorsWriter.java  |  3 ++-
 .../codecs/compressing/CompressingTermVectorsWriter.java  | 15 ++++++++++++++-
 .../org/apache/lucene/index/DefaultIndexingChain.java     | 13 +++++++++++--
 .../apache/lucene/index/SortingTermVectorsConsumer.java   |  2 ++
 .../org/apache/lucene/index/StoredFieldsConsumer.java     | 15 ++++++---------
 .../java/org/apache/lucene/index/TermVectorsConsumer.java | 12 ++++++------
 .../core/src/java/org/apache/lucene/util/Accountable.java |  4 ++++
 .../codecs/asserting/AssertingTermVectorsFormat.java      | 10 ++++++++++
 .../lucene/codecs/cranky/CrankyTermVectorsFormat.java     | 13 +++++++++++++
 11 files changed, 77 insertions(+), 23 deletions(-)

diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index bdf62dd..2333661 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -53,6 +53,10 @@ Improvements
   heap consumption is taken into account when IndexWriter stalls or should flush
   DWPTs. (Simon Willnauer)
 
+* LUCENE-9514: Include TermVectorsWriter 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/SimpleTextTermVectorsWriter.java b/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextTermVectorsWriter.java
index 117ec19..43ae34e 100644
--- a/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextTermVectorsWriter.java
+++ b/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextTermVectorsWriter.java
@@ -56,8 +56,6 @@ public class SimpleTextTermVectorsWriter extends TermVectorsWriter {
 
   static final String VECTORS_EXTENSION = "vec";
   
-  private final Directory directory;
-  private final String segment;
   private IndexOutput out;
   private int numDocsWritten = 0;
   private final BytesRefBuilder scratch = new BytesRefBuilder();
@@ -66,8 +64,6 @@ public class SimpleTextTermVectorsWriter extends TermVectorsWriter {
   private boolean payloads;
 
   public SimpleTextTermVectorsWriter(Directory directory, String segment, IOContext context) throws IOException {
-    this.directory = directory;
-    this.segment = segment;
     boolean success = false;
     try {
       out = directory.createOutput(IndexFileNames.segmentFileName(segment, "", VECTORS_EXTENSION), context);
@@ -193,4 +189,9 @@ public class SimpleTextTermVectorsWriter extends TermVectorsWriter {
   private void newLine() throws IOException {
     SimpleTextUtil.writeNewline(out);
   }
+
+  @Override
+  public long ramBytesUsed() {
+    return scratch.get().bytes.length;
+  }
 }
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/TermVectorsWriter.java b/lucene/core/src/java/org/apache/lucene/codecs/TermVectorsWriter.java
index b84065a..7865c08 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/TermVectorsWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/TermVectorsWriter.java
@@ -32,6 +32,7 @@ import org.apache.lucene.index.Terms;
 import org.apache.lucene.index.TermsEnum;
 import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.store.DataInput;
+import org.apache.lucene.util.Accountable;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefBuilder;
 
@@ -58,7 +59,7 @@ import static org.apache.lucene.search.DocIdSetIterator.NO_MORE_DOCS;
  * 
  * @lucene.experimental
  */
-public abstract class TermVectorsWriter implements Closeable {
+public abstract class TermVectorsWriter implements Closeable, Accountable {
   
   /** Sole constructor. (For invocation by subclass 
    *  constructors, typically implicit.) */
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 fb607b8..3dc2ff6 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
@@ -20,6 +20,7 @@ package org.apache.lucene.codecs.compressing;
 import java.io.IOException;
 import java.util.ArrayDeque;
 import java.util.Arrays;
+import java.util.Collection;
 import java.util.Deque;
 import java.util.Iterator;
 import java.util.SortedSet;
@@ -41,6 +42,7 @@ import org.apache.lucene.store.GrowableByteArrayDataOutput;
 import org.apache.lucene.store.IOContext;
 import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.util.Accountable;
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
@@ -206,7 +208,7 @@ public final class CompressingTermVectorsWriter extends TermVectorsWriter {
   private final BlockPackedWriter writer;
 
   /** Sole constructor. */
-  public CompressingTermVectorsWriter(Directory directory, SegmentInfo si, String segmentSuffix, IOContext context,
+  CompressingTermVectorsWriter(Directory directory, SegmentInfo si, String segmentSuffix, IOContext context,
       String formatName, CompressionMode compressionMode, int chunkSize, int blockShift) throws IOException {
     assert directory != null;
     this.segment = si.name;
@@ -853,4 +855,15 @@ public final class CompressingTermVectorsWriter extends TermVectorsWriter {
     return candidate.getNumDirtyChunks() > 1024 || 
            candidate.getNumDirtyChunks() * 100 > candidate.getNumChunks();
   }
+
+  @Override
+  public long ramBytesUsed() {
+    return positionsBuf.length + startOffsetsBuf.length + lengthsBuf.length + payloadLengthsBuf.length
+        + termSuffixes.ramBytesUsed() + payloadBytes.ramBytesUsed() + lastTerm.bytes.length;
+  }
+
+  @Override
+  public Collection<Accountable> getChildResources() {
+    return Arrays.asList(termSuffixes, payloadBytes);
+  }
 }
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 b3208c2..3ea577a 100644
--- a/lucene/core/src/java/org/apache/lucene/index/DefaultIndexingChain.java
+++ b/lucene/core/src/java/org/apache/lucene/index/DefaultIndexingChain.java
@@ -21,6 +21,7 @@ import java.io.Closeable;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collection;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
@@ -43,6 +44,7 @@ import org.apache.lucene.search.SortField;
 import org.apache.lucene.search.similarities.Similarity;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
+import org.apache.lucene.util.Accountable;
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.ByteBlockPool;
 import org.apache.lucene.util.BytesRef;
@@ -65,6 +67,8 @@ final class DefaultIndexingChain extends DocConsumer {
   final TermsHash termsHash;
   // Writes stored fields
   final StoredFieldsConsumer storedFieldsConsumer;
+  final TermVectorsConsumer termVectorsWriter;
+
 
   // NOTE: I tried using Hash Map<String,PerField>
   // but it was ~2% slower on Wiki and Geonames with Java
@@ -95,7 +99,6 @@ final class DefaultIndexingChain extends DocConsumer {
     this.infoStream = indexWriterConfig.getInfoStream();
     this.abortingExceptionConsumer = abortingExceptionConsumer;
 
-    final TermsHash termVectorsWriter;
     if (segmentInfo.getIndexSort() == null) {
       storedFieldsConsumer = new StoredFieldsConsumer(indexWriterConfig.getCodec(), directory, segmentInfo);
       termVectorsWriter = new TermVectorsConsumer(intBlockAllocator, byteBlockAllocator, directory, segmentInfo, indexWriterConfig.getCodec());
@@ -795,7 +798,13 @@ final class DefaultIndexingChain extends DocConsumer {
 
   @Override
   public long ramBytesUsed() {
-    return bytesUsed.get() + storedFieldsConsumer.ramBytesUsed();
+    return bytesUsed.get() + storedFieldsConsumer.accountable.ramBytesUsed()
+        + termVectorsWriter.accountable.ramBytesUsed();
+  }
+
+  @Override
+  public Collection<Accountable> getChildResources() {
+    return Arrays.asList(storedFieldsConsumer.accountable, termVectorsWriter.accountable);
   }
 
   /** NOTE: not static: accesses at least docState, termsHash. */
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 c005ede..5162f39 100644
--- a/lucene/core/src/java/org/apache/lucene/index/SortingTermVectorsConsumer.java
+++ b/lucene/core/src/java/org/apache/lucene/index/SortingTermVectorsConsumer.java
@@ -185,4 +185,6 @@ final class SortingTermVectorsConsumer extends TermVectorsConsumer {
     assert fieldCount == numFields;
     writer.finishDocument();
   }
+
+
 }
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 f73bb75..883c459 100644
--- a/lucene/core/src/java/org/apache/lucene/index/StoredFieldsConsumer.java
+++ b/lucene/core/src/java/org/apache/lucene/index/StoredFieldsConsumer.java
@@ -23,6 +23,7 @@ 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.Accountable;
 import org.apache.lucene.util.IOUtils;
 
 class StoredFieldsConsumer {
@@ -30,6 +31,9 @@ class StoredFieldsConsumer {
   final Directory directory;
   final SegmentInfo info;
   StoredFieldsWriter writer;
+  // this accountable either holds the writer or one that returns null.
+  // it's cleaner than checking if the writer is null all over the place
+  Accountable accountable = Accountable.NULL_ACCOUNTABLE;
   private int lastDoc;
 
   StoredFieldsConsumer(Codec codec, Directory directory, SegmentInfo info) {
@@ -42,6 +46,7 @@ class StoredFieldsConsumer {
   protected void initStoredFieldsWriter() throws IOException {
     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);
+      accountable = writer;
     }
   }
 
@@ -76,18 +81,10 @@ class StoredFieldsConsumer {
       writer.finish(state.fieldInfos, state.segmentInfo.maxDoc());
     } finally {
       IOUtils.close(writer);
-      writer = null;
     }
   }
 
   void abort() {
-    if (writer != null) {
-      IOUtils.closeWhileHandlingException(writer);
-      writer = null;
-    }
-  }
-
-  long ramBytesUsed() {
-    return writer == null ? 0 : writer.ramBytesUsed();
+    IOUtils.closeWhileHandlingException(writer);
   }
 }
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 1f6b197..703a8bd 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.codecs.TermVectorsWriter;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.FlushInfo;
 import org.apache.lucene.store.IOContext;
+import org.apache.lucene.util.Accountable;
 import org.apache.lucene.util.ArrayUtil;
 import org.apache.lucene.util.ByteBlockPool;
 import org.apache.lucene.util.BytesRef;
@@ -54,6 +55,9 @@ class TermVectorsConsumer extends TermsHash {
   private int numVectorFields;
   int lastDocID;
   private TermVectorsConsumerPerField[] perFields = new TermVectorsConsumerPerField[1];
+  // this accountable either holds the writer or one that returns null.
+  // it's cleaner than checking if the writer is null all over the place
+  Accountable accountable = Accountable.NULL_ACCOUNTABLE;
 
   TermVectorsConsumer(final IntBlockPool.Allocator intBlockAllocator, final ByteBlockPool.Allocator byteBlockAllocator, Directory directory, SegmentInfo info, Codec codec) {
     super(intBlockAllocator, byteBlockAllocator, Counter.newCounter(), null);
@@ -74,9 +78,6 @@ class TermVectorsConsumer extends TermsHash {
         writer.finish(state.fieldInfos, numDocs);
       } finally {
         IOUtils.close(writer);
-        writer = null;
-        lastDocID = 0;
-        hasVectors = false;
       }
     }
   }
@@ -96,6 +97,7 @@ class TermVectorsConsumer extends TermsHash {
       IOContext context = new IOContext(new FlushInfo(lastDocID, bytesUsed.get()));
       writer = codec.termVectorsFormat().vectorsWriter(directory, info, context);
       lastDocID = 0;
+      accountable = writer;
     }
   }
 
@@ -130,13 +132,10 @@ class TermVectorsConsumer extends TermsHash {
 
   @Override
   public void abort() {
-    hasVectors = false;
     try {
       super.abort();
     } finally {
       IOUtils.closeWhileHandlingException(writer);
-      writer = null;
-      lastDocID = 0;
       reset();
     }
   }
@@ -167,4 +166,5 @@ class TermVectorsConsumer extends TermsHash {
     resetFields();
     numVectorFields = 0;
   }
+
 }
diff --git a/lucene/core/src/java/org/apache/lucene/util/Accountable.java b/lucene/core/src/java/org/apache/lucene/util/Accountable.java
index 901d805..a61ab34 100644
--- a/lucene/core/src/java/org/apache/lucene/util/Accountable.java
+++ b/lucene/core/src/java/org/apache/lucene/util/Accountable.java
@@ -41,4 +41,8 @@ public interface Accountable {
     return Collections.emptyList();
   }
 
+  /**
+   * An accountable that always returns 0
+   */
+  Accountable NULL_ACCOUNTABLE = () -> 0;
 }
diff --git a/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingTermVectorsFormat.java b/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingTermVectorsFormat.java
index 8594adc..b8ea510 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingTermVectorsFormat.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/codecs/asserting/AssertingTermVectorsFormat.java
@@ -18,6 +18,7 @@ package org.apache.lucene.codecs.asserting;
 
 import java.io.IOException;
 import java.util.Collection;
+import java.util.Collections;
 
 import org.apache.lucene.codecs.TermVectorsFormat;
 import org.apache.lucene.codecs.TermVectorsReader;
@@ -210,5 +211,14 @@ public class AssertingTermVectorsFormat extends TermVectorsFormat {
       in.close(); // close again
     }
 
+    @Override
+    public long ramBytesUsed() {
+      return in.ramBytesUsed();
+    }
+
+    @Override
+    public Collection<Accountable> getChildResources() {
+      return Collections.singleton(in);
+    }
   }
 }
diff --git a/lucene/test-framework/src/java/org/apache/lucene/codecs/cranky/CrankyTermVectorsFormat.java b/lucene/test-framework/src/java/org/apache/lucene/codecs/cranky/CrankyTermVectorsFormat.java
index 0537f25..375c042 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/codecs/cranky/CrankyTermVectorsFormat.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/codecs/cranky/CrankyTermVectorsFormat.java
@@ -17,6 +17,8 @@
 package org.apache.lucene.codecs.cranky;
 
 import java.io.IOException;
+import java.util.Collection;
+import java.util.Collections;
 import java.util.Random;
 
 import org.apache.lucene.codecs.TermVectorsFormat;
@@ -29,6 +31,7 @@ import org.apache.lucene.index.SegmentInfo;
 import org.apache.lucene.store.DataInput;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
+import org.apache.lucene.util.Accountable;
 import org.apache.lucene.util.BytesRef;
 
 class CrankyTermVectorsFormat extends TermVectorsFormat {
@@ -151,5 +154,15 @@ class CrankyTermVectorsFormat extends TermVectorsFormat {
       }
       super.addProx(numProx, positions, offsets);
     }
+
+    @Override
+    public long ramBytesUsed() {
+      return delegate.ramBytesUsed();
+    }
+
+    @Override
+    public Collection<Accountable> getChildResources() {
+      return Collections.singleton(delegate);
+    }
   }
 }