You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@lucene.apache.org by GitBox <gi...@apache.org> on 2020/06/12 15:18:34 UTC

[GitHub] [lucene-solr] s1monw opened a new pull request #1573: Cleanup TermsHashPerField

s1monw opened a new pull request #1573:
URL: https://github.com/apache/lucene-solr/pull/1573


   Several classes within the IndexWriter indexing chain haven't been touched for several years. Most of these classes expose their internals through public members and are difficult to construct in tests since they depend on many other classes. This change tries to clean up TermsHashPerField and adds a dedicated standalone test for it to make it more accessible for other developers since it's simpler to understand. There are also attempts to make documentation better as a result of this refactoring.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene-solr] s1monw commented on a change in pull request #1573: Cleanup TermsHashPerField

Posted by GitBox <gi...@apache.org>.
s1monw commented on a change in pull request #1573:
URL: https://github.com/apache/lucene-solr/pull/1573#discussion_r439804443



##########
File path: lucene/core/src/java/org/apache/lucene/index/TermsHashPerField.java
##########
@@ -19,203 +19,207 @@
 
 import java.io.IOException;
 
-import org.apache.lucene.analysis.tokenattributes.TermFrequencyAttribute;
-import org.apache.lucene.analysis.tokenattributes.TermToBytesRefAttribute;
 import org.apache.lucene.util.ByteBlockPool;
+import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefHash.BytesStartArray;
 import org.apache.lucene.util.BytesRefHash;
 import org.apache.lucene.util.Counter;
 import org.apache.lucene.util.IntBlockPool;
 
+/**
+ * This class allows to store streams of information per term without knowing
+ * the size of the stream ahead of time. Each stream typically encodes one level
+ * of information like term frequency per document or term proximity. Internally
+ * this class allocates a linked list of slices that can be read by a {@link ByteSliceReader}
+ * for each term. Terms are first deduplicated in a {@link BytesRefHash} once this is done
+ * internal data-structures point to the current offset of each stream that can be written to.
+ */
 abstract class TermsHashPerField implements Comparable<TermsHashPerField> {
   private static final int HASH_INIT_SIZE = 4;
 
-  final TermsHash termsHash;
-
-  final TermsHashPerField nextPerField;
-  protected final DocumentsWriterPerThread.DocState docState;
-  protected final FieldInvertState fieldState;
-  TermToBytesRefAttribute termAtt;
-  protected TermFrequencyAttribute termFreqAtt;
-
-  // Copied from our perThread
-  final IntBlockPool intPool;
+  private final TermsHashPerField nextPerField;
+  private final IntBlockPool intPool;
   final ByteBlockPool bytePool;
-  final ByteBlockPool termBytePool;
-
-  final int streamCount;
-  final int numPostingInt;
-
-  protected final FieldInfo fieldInfo;
-
-  final BytesRefHash bytesHash;
+  // for each term we store an integer per stream that points into the bytePool above
+  // the address is updated once data is written to the stream to point to the next free offset
+  // this the terms stream. The start address for the stream is stored in postingsArray.byteStarts[termId]
+  // This is initialized in the #addTerm method, either to a brand new per term stream if the term is new or
+  // to the addresses where the term stream was written to when we saw it the last time.
+  private int[] termStreamAddressBuffer;
+  private int streamAddressOffset;
+  private final int streamCount;
+  private final String fieldName;
+  final IndexOptions indexOptions;
+  /* This stores the actual term bytes for postings and offsets into the parent hash in the case that this
+  * TermsHashPerField is hashing term vectors.*/
+  private final BytesRefHash bytesHash;
 
   ParallelPostingsArray postingsArray;
-  private final Counter bytesUsed;
+  private int lastDocID; // only with assert
 
   /** streamCount: how many streams this field stores per term.
    * E.g. doc(+freq) is 1 stream, prox+offset is a second. */
-
-  public TermsHashPerField(int streamCount, FieldInvertState fieldState, TermsHash termsHash, TermsHashPerField nextPerField, FieldInfo fieldInfo) {
-    intPool = termsHash.intPool;
-    bytePool = termsHash.bytePool;
-    termBytePool = termsHash.termBytePool;
-    docState = termsHash.docState;
-    this.termsHash = termsHash;
-    bytesUsed = termsHash.bytesUsed;
-    this.fieldState = fieldState;
+  TermsHashPerField(int streamCount, IntBlockPool intPool, ByteBlockPool bytePool, ByteBlockPool termBytePool,
+                    Counter bytesUsed, TermsHashPerField nextPerField, String fieldName, IndexOptions indexOptions) {
+    this.intPool = intPool;
+    this.bytePool = bytePool;
     this.streamCount = streamCount;
-    numPostingInt = 2*streamCount;
-    this.fieldInfo = fieldInfo;
+    this.fieldName = fieldName;
     this.nextPerField = nextPerField;
+    assert indexOptions != IndexOptions.NONE;

Review comment:
       yeah :)




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene-solr] s1monw commented on pull request #1573: Cleanup TermsHashPerField

Posted by GitBox <gi...@apache.org>.
s1monw commented on pull request #1573:
URL: https://github.com/apache/lucene-solr/pull/1573#issuecomment-643735813


   > I think we should run indexing throughput benchmark to make sure there's no real impact here. I'll do that.
   
   +1 thanks @mikemccand 
   
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene-solr] s1monw commented on a change in pull request #1573: Cleanup TermsHashPerField

Posted by GitBox <gi...@apache.org>.
s1monw commented on a change in pull request #1573:
URL: https://github.com/apache/lucene-solr/pull/1573#discussion_r439804315



##########
File path: lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriterPerField.java
##########
@@ -56,12 +56,6 @@ public FreqProxTermsWriterPerField(FieldInvertState invertState, TermsHash terms
   @Override
   void finish() throws IOException {
     super.finish();
-    sumDocFreq += fieldState.uniqueTermCount;
-    sumTotalTermFreq += fieldState.length;

Review comment:
       `sumDocFreq` and `sumTotalTermFreq` are unused. They were used in `FreqProxFields` in the past but not anymore for a while now. I removed their commented out usage so you can see it in a followup commit




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene-solr] s1monw commented on a change in pull request #1573: Cleanup TermsHashPerField

Posted by GitBox <gi...@apache.org>.
s1monw commented on a change in pull request #1573:
URL: https://github.com/apache/lucene-solr/pull/1573#discussion_r439803910



##########
File path: lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriterPerField.java
##########
@@ -207,8 +202,6 @@ public void newPostingsArray() {
 
   @Override
   ParallelPostingsArray createPostingsArray(int size) {
-    IndexOptions indexOptions = fieldInfo.getIndexOptions();
-    assert indexOptions != IndexOptions.NONE;

Review comment:
       I moved it in a better place in the ctor of the base class. I think that's enough?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene-solr] s1monw merged pull request #1573: Cleanup TermsHashPerField

Posted by GitBox <gi...@apache.org>.
s1monw merged pull request #1573:
URL: https://github.com/apache/lucene-solr/pull/1573


   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene-solr] s1monw commented on a change in pull request #1573: Cleanup TermsHashPerField

Posted by GitBox <gi...@apache.org>.
s1monw commented on a change in pull request #1573:
URL: https://github.com/apache/lucene-solr/pull/1573#discussion_r439804390



##########
File path: lucene/core/src/java/org/apache/lucene/index/TermVectorsConsumerPerField.java
##########
@@ -222,7 +234,7 @@ void writeProx(TermVectorsPostingsArray postings, int termID) {
   }
 
   @Override
-  void newTerm(final int termID) {
+  void newTerm(final int termID, final int docID) {

Review comment:
       that's correct.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene-solr] mikemccand commented on pull request #1573: Cleanup TermsHashPerField

Posted by GitBox <gi...@apache.org>.
mikemccand commented on pull request #1573:
URL: https://github.com/apache/lucene-solr/pull/1573#issuecomment-644734269


   > thanks @mikemccand - I will run tests again and push.
   
   ++


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene-solr] mikemccand commented on a change in pull request #1573: Cleanup TermsHashPerField

Posted by GitBox <gi...@apache.org>.
mikemccand commented on a change in pull request #1573:
URL: https://github.com/apache/lucene-solr/pull/1573#discussion_r439751347



##########
File path: lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriterPerField.java
##########
@@ -207,8 +202,6 @@ public void newPostingsArray() {
 
   @Override
   ParallelPostingsArray createPostingsArray(int size) {
-    IndexOptions indexOptions = fieldInfo.getIndexOptions();
-    assert indexOptions != IndexOptions.NONE;

Review comment:
       Hmm why not keep this assertion (to confirm that if the field is not somehow indexed we are not accidentally/incorrectly running this code)?

##########
File path: lucene/core/src/java/org/apache/lucene/index/ParallelPostingsArray.java
##########
@@ -22,14 +22,14 @@
   final static int BYTES_PER_POSTING = 3 * Integer.BYTES;
 
   final int size;
-  final int[] textStarts;
-  final int[] intStarts;
-  final int[] byteStarts;
+  final int[] textStarts; // maps term ID to the terms text start in the bytesHash
+  final int[] addressOffset; // maps term ID to current stream address

Review comment:
       +1 for this renaming!

##########
File path: lucene/core/src/java/org/apache/lucene/index/TermsHashPerField.java
##########
@@ -19,203 +19,207 @@
 
 import java.io.IOException;
 
-import org.apache.lucene.analysis.tokenattributes.TermFrequencyAttribute;
-import org.apache.lucene.analysis.tokenattributes.TermToBytesRefAttribute;
 import org.apache.lucene.util.ByteBlockPool;
+import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefHash.BytesStartArray;
 import org.apache.lucene.util.BytesRefHash;
 import org.apache.lucene.util.Counter;
 import org.apache.lucene.util.IntBlockPool;
 
+/**
+ * This class allows to store streams of information per term without knowing
+ * the size of the stream ahead of time. Each stream typically encodes one level
+ * of information like term frequency per document or term proximity. Internally
+ * this class allocates a linked list of slices that can be read by a {@link ByteSliceReader}
+ * for each term. Terms are first deduplicated in a {@link BytesRefHash} once this is done
+ * internal data-structures point to the current offset of each stream that can be written to.
+ */
 abstract class TermsHashPerField implements Comparable<TermsHashPerField> {
   private static final int HASH_INIT_SIZE = 4;
 
-  final TermsHash termsHash;
-
-  final TermsHashPerField nextPerField;
-  protected final DocumentsWriterPerThread.DocState docState;
-  protected final FieldInvertState fieldState;
-  TermToBytesRefAttribute termAtt;
-  protected TermFrequencyAttribute termFreqAtt;
-
-  // Copied from our perThread
-  final IntBlockPool intPool;
+  private final TermsHashPerField nextPerField;
+  private final IntBlockPool intPool;
   final ByteBlockPool bytePool;
-  final ByteBlockPool termBytePool;
-
-  final int streamCount;
-  final int numPostingInt;
-
-  protected final FieldInfo fieldInfo;
-
-  final BytesRefHash bytesHash;
+  // for each term we store an integer per stream that points into the bytePool above
+  // the address is updated once data is written to the stream to point to the next free offset
+  // this the terms stream. The start address for the stream is stored in postingsArray.byteStarts[termId]
+  // This is initialized in the #addTerm method, either to a brand new per term stream if the term is new or
+  // to the addresses where the term stream was written to when we saw it the last time.
+  private int[] termStreamAddressBuffer;
+  private int streamAddressOffset;
+  private final int streamCount;
+  private final String fieldName;
+  final IndexOptions indexOptions;
+  /* This stores the actual term bytes for postings and offsets into the parent hash in the case that this
+  * TermsHashPerField is hashing term vectors.*/
+  private final BytesRefHash bytesHash;
 
   ParallelPostingsArray postingsArray;
-  private final Counter bytesUsed;
+  private int lastDocID; // only with assert
 
   /** streamCount: how many streams this field stores per term.
    * E.g. doc(+freq) is 1 stream, prox+offset is a second. */
-
-  public TermsHashPerField(int streamCount, FieldInvertState fieldState, TermsHash termsHash, TermsHashPerField nextPerField, FieldInfo fieldInfo) {
-    intPool = termsHash.intPool;
-    bytePool = termsHash.bytePool;
-    termBytePool = termsHash.termBytePool;
-    docState = termsHash.docState;
-    this.termsHash = termsHash;
-    bytesUsed = termsHash.bytesUsed;
-    this.fieldState = fieldState;
+  TermsHashPerField(int streamCount, IntBlockPool intPool, ByteBlockPool bytePool, ByteBlockPool termBytePool,
+                    Counter bytesUsed, TermsHashPerField nextPerField, String fieldName, IndexOptions indexOptions) {
+    this.intPool = intPool;
+    this.bytePool = bytePool;
     this.streamCount = streamCount;
-    numPostingInt = 2*streamCount;
-    this.fieldInfo = fieldInfo;
+    this.fieldName = fieldName;
     this.nextPerField = nextPerField;
+    assert indexOptions != IndexOptions.NONE;
+    this.indexOptions = indexOptions;
     PostingsBytesStartArray byteStarts = new PostingsBytesStartArray(this, bytesUsed);
     bytesHash = new BytesRefHash(termBytePool, HASH_INIT_SIZE, byteStarts);
   }
 
   void reset() {
     bytesHash.clear(false);
+    sortedTermIDs = null;
     if (nextPerField != null) {
       nextPerField.reset();
     }
   }
 
-  public void initReader(ByteSliceReader reader, int termID, int stream) {
+  final void initReader(ByteSliceReader reader, int termID, int stream) {
     assert stream < streamCount;
-    int intStart = postingsArray.intStarts[termID];
-    final int[] ints = intPool.buffers[intStart >> IntBlockPool.INT_BLOCK_SHIFT];
-    final int upto = intStart & IntBlockPool.INT_BLOCK_MASK;
+    int streamStartOffset = postingsArray.addressOffset[termID];
+    final int[] streamAddressBuffer = intPool.buffers[streamStartOffset >> IntBlockPool.INT_BLOCK_SHIFT];
+    final int offsetInAddressBuffer = streamStartOffset & IntBlockPool.INT_BLOCK_MASK;
     reader.init(bytePool,
                 postingsArray.byteStarts[termID]+stream*ByteBlockPool.FIRST_LEVEL_SIZE,
-                ints[upto+stream]);
+                streamAddressBuffer[offsetInAddressBuffer+stream]);
   }
 
-  int[] sortedTermIDs;
+  private int[] sortedTermIDs;
 
   /** Collapse the hash table and sort in-place; also sets
-   * this.sortedTermIDs to the results */
-  public int[] sortPostings() {
+   * this.sortedTermIDs to the results
+   * This method should not be called twice unless {@link #reset()}
+   * or {@link #reinitHash()} was called. */
+  final void sortTerms() {
+    assert sortedTermIDs == null;
     sortedTermIDs = bytesHash.sort();
+  }
+
+  /**
+   * Returns the sorted term IDs. {@link #sortTerms()} must be called before
+   */
+  final int[] getSortedTermIDs() {
+    assert sortedTermIDs != null;
     return sortedTermIDs;
   }
 
+  final void reinitHash() {
+    sortedTermIDs = null;
+    bytesHash.reinit();
+  }
+
   private boolean doNextCall;
 
   // Secondary entry point (for 2nd & subsequent TermsHash),
   // because token text has already been "interned" into
   // textStart, so we hash by textStart.  term vectors use
   // this API.
-  public void add(int textStart) throws IOException {
+  private void add(int textStart, final int docID) throws IOException {
     int termID = bytesHash.addByPoolOffset(textStart);
     if (termID >= 0) {      // New posting
       // First time we are seeing this token since we last
       // flushed the hash.
-      // Init stream slices
-      if (numPostingInt + intPool.intUpto > IntBlockPool.INT_BLOCK_SIZE) {
-        intPool.nextBuffer();
-      }
-
-      if (ByteBlockPool.BYTE_BLOCK_SIZE - bytePool.byteUpto < numPostingInt*ByteBlockPool.FIRST_LEVEL_SIZE) {
-        bytePool.nextBuffer();
-      }
+      initStreamSlices(termID, docID);
+    } else {
+      positionStreamSlice(termID, docID);
+    }
+  }
 
-      intUptos = intPool.buffer;
-      intUptoStart = intPool.intUpto;
-      intPool.intUpto += streamCount;
+  private void initStreamSlices(int termID, int docID) throws IOException {
+    // Init stream slices
+    // TODO: figure out why this is 2*streamCount here. streamCount should be enough?
+    if ((2*streamCount) + intPool.intUpto > IntBlockPool.INT_BLOCK_SIZE) {
+      // can we fit all the streams in the current buffer?
+      intPool.nextBuffer();
+    }
 
-      postingsArray.intStarts[termID] = intUptoStart + intPool.intOffset;
+    if (ByteBlockPool.BYTE_BLOCK_SIZE - bytePool.byteUpto < (2*streamCount) * ByteBlockPool.FIRST_LEVEL_SIZE) {
+      // can we fit at least one byte per stream in the current buffer, if not allocated a new one

Review comment:
       s/`allocated`/`allocate`

##########
File path: lucene/core/src/java/org/apache/lucene/index/TermsHashPerField.java
##########
@@ -19,203 +19,207 @@
 
 import java.io.IOException;
 
-import org.apache.lucene.analysis.tokenattributes.TermFrequencyAttribute;
-import org.apache.lucene.analysis.tokenattributes.TermToBytesRefAttribute;
 import org.apache.lucene.util.ByteBlockPool;
+import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefHash.BytesStartArray;
 import org.apache.lucene.util.BytesRefHash;
 import org.apache.lucene.util.Counter;
 import org.apache.lucene.util.IntBlockPool;
 
+/**
+ * This class allows to store streams of information per term without knowing
+ * the size of the stream ahead of time. Each stream typically encodes one level
+ * of information like term frequency per document or term proximity. Internally
+ * this class allocates a linked list of slices that can be read by a {@link ByteSliceReader}
+ * for each term. Terms are first deduplicated in a {@link BytesRefHash} once this is done
+ * internal data-structures point to the current offset of each stream that can be written to.
+ */
 abstract class TermsHashPerField implements Comparable<TermsHashPerField> {
   private static final int HASH_INIT_SIZE = 4;
 
-  final TermsHash termsHash;
-
-  final TermsHashPerField nextPerField;
-  protected final DocumentsWriterPerThread.DocState docState;
-  protected final FieldInvertState fieldState;
-  TermToBytesRefAttribute termAtt;
-  protected TermFrequencyAttribute termFreqAtt;
-
-  // Copied from our perThread
-  final IntBlockPool intPool;
+  private final TermsHashPerField nextPerField;
+  private final IntBlockPool intPool;
   final ByteBlockPool bytePool;
-  final ByteBlockPool termBytePool;
-
-  final int streamCount;
-  final int numPostingInt;
-
-  protected final FieldInfo fieldInfo;
-
-  final BytesRefHash bytesHash;
+  // for each term we store an integer per stream that points into the bytePool above
+  // the address is updated once data is written to the stream to point to the next free offset
+  // this the terms stream. The start address for the stream is stored in postingsArray.byteStarts[termId]
+  // This is initialized in the #addTerm method, either to a brand new per term stream if the term is new or
+  // to the addresses where the term stream was written to when we saw it the last time.
+  private int[] termStreamAddressBuffer;
+  private int streamAddressOffset;
+  private final int streamCount;
+  private final String fieldName;
+  final IndexOptions indexOptions;
+  /* This stores the actual term bytes for postings and offsets into the parent hash in the case that this
+  * TermsHashPerField is hashing term vectors.*/
+  private final BytesRefHash bytesHash;
 
   ParallelPostingsArray postingsArray;
-  private final Counter bytesUsed;
+  private int lastDocID; // only with assert
 
   /** streamCount: how many streams this field stores per term.
    * E.g. doc(+freq) is 1 stream, prox+offset is a second. */
-
-  public TermsHashPerField(int streamCount, FieldInvertState fieldState, TermsHash termsHash, TermsHashPerField nextPerField, FieldInfo fieldInfo) {
-    intPool = termsHash.intPool;
-    bytePool = termsHash.bytePool;
-    termBytePool = termsHash.termBytePool;
-    docState = termsHash.docState;
-    this.termsHash = termsHash;
-    bytesUsed = termsHash.bytesUsed;
-    this.fieldState = fieldState;
+  TermsHashPerField(int streamCount, IntBlockPool intPool, ByteBlockPool bytePool, ByteBlockPool termBytePool,
+                    Counter bytesUsed, TermsHashPerField nextPerField, String fieldName, IndexOptions indexOptions) {
+    this.intPool = intPool;
+    this.bytePool = bytePool;
     this.streamCount = streamCount;
-    numPostingInt = 2*streamCount;
-    this.fieldInfo = fieldInfo;
+    this.fieldName = fieldName;
     this.nextPerField = nextPerField;
+    assert indexOptions != IndexOptions.NONE;

Review comment:
       Ahh I see, we just moved the assertion to a better place, awesome.

##########
File path: lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriterPerField.java
##########
@@ -56,12 +56,6 @@ public FreqProxTermsWriterPerField(FieldInvertState invertState, TermsHash terms
   @Override
   void finish() throws IOException {
     super.finish();
-    sumDocFreq += fieldState.uniqueTermCount;
-    sumTotalTermFreq += fieldState.length;

Review comment:
       Hmm, did these aggregations move somewhere else?  Oh, they look entirely removed?  Were they redundant (computed elsewhere) and these ones were unused?

##########
File path: lucene/core/src/java/org/apache/lucene/index/ParallelPostingsArray.java
##########
@@ -22,14 +22,14 @@
   final static int BYTES_PER_POSTING = 3 * Integer.BYTES;
 
   final int size;
-  final int[] textStarts;
-  final int[] intStarts;
-  final int[] byteStarts;
+  final int[] textStarts; // maps term ID to the terms text start in the bytesHash

Review comment:
       s/`terms text`/`terms's text`

##########
File path: lucene/core/src/java/org/apache/lucene/index/TermVectorsConsumerPerField.java
##########
@@ -222,7 +234,7 @@ void writeProx(TermVectorsPostingsArray postings, int termID) {
   }
 
   @Override
-  void newTerm(final int termID) {
+  void newTerm(final int termID, final int docID) {

Review comment:
       Hmm `docID` is unused in this method?  But I guess the other impl (normal postings) needs it?

##########
File path: lucene/core/src/java/org/apache/lucene/index/TermsHashPerField.java
##########
@@ -19,203 +19,207 @@
 
 import java.io.IOException;
 
-import org.apache.lucene.analysis.tokenattributes.TermFrequencyAttribute;
-import org.apache.lucene.analysis.tokenattributes.TermToBytesRefAttribute;
 import org.apache.lucene.util.ByteBlockPool;
+import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefHash.BytesStartArray;
 import org.apache.lucene.util.BytesRefHash;
 import org.apache.lucene.util.Counter;
 import org.apache.lucene.util.IntBlockPool;
 
+/**
+ * This class allows to store streams of information per term without knowing

Review comment:
       Thank you for the javadocs/comments for such cryptic and ancient code ;)
   
   Maybe just `stores` instead of `allows to store`?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene-solr] mikemccand commented on pull request #1573: Cleanup TermsHashPerField

Posted by GitBox <gi...@apache.org>.
mikemccand commented on pull request #1573:
URL: https://github.com/apache/lucene-solr/pull/1573#issuecomment-644731914


   I tested indexing throughput on `luceneutil` with `wikimediumall`, single thread for indexing `SerialMergeScheduler`:
   
   ```
   [mike@beast3 facet]$ grep "GB/hour" /l/logs/simon?
   /l/logs/simon0:Indexer: 46.44432470391602 GB/hour plain text
   /l/logs/simon1:Indexer: 46.267723012921515 GB/hour plain text
   /l/logs/simon2:Indexer: 46.26414201429784 GB/hour plain text
   [mike@beast3 facet]$ grep "GB/hour" /l/logs/trunk?
   /l/logs/trunk0:Indexer: 45.632881600179495 GB/hour plain text
   /l/logs/trunk1:Indexer: 46.09383252131896 GB/hour plain text
   /l/logs/trunk2:Indexer: 45.439666582156924 GB/hour plain text
   ```
   
   Net/net this change might be a bit faster, or just noise, so all clear to push!  Thanks @s1monw.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene-solr] s1monw commented on pull request #1573: Cleanup TermsHashPerField

Posted by GitBox <gi...@apache.org>.
s1monw commented on pull request #1573:
URL: https://github.com/apache/lucene-solr/pull/1573#issuecomment-644732898


   thanks @mikemccand - I will run tests again and push.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene-solr] dweiss commented on a change in pull request #1573: Cleanup TermsHashPerField

Posted by GitBox <gi...@apache.org>.
dweiss commented on a change in pull request #1573:
URL: https://github.com/apache/lucene-solr/pull/1573#discussion_r439858046



##########
File path: lucene/core/src/java/org/apache/lucene/index/TermsHashPerField.java
##########
@@ -19,203 +19,207 @@
 
 import java.io.IOException;
 
-import org.apache.lucene.analysis.tokenattributes.TermFrequencyAttribute;
-import org.apache.lucene.analysis.tokenattributes.TermToBytesRefAttribute;
 import org.apache.lucene.util.ByteBlockPool;
+import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefHash.BytesStartArray;
 import org.apache.lucene.util.BytesRefHash;
 import org.apache.lucene.util.Counter;
 import org.apache.lucene.util.IntBlockPool;
 
+/**
+ * This class stores streams of information per term without knowing
+ * the size of the stream ahead of time. Each stream typically encodes one level
+ * of information like term frequency per document or term proximity. Internally
+ * this class allocates a linked list of slices that can be read by a {@link ByteSliceReader}
+ * for each term. Terms are first deduplicated in a {@link BytesRefHash} once this is done
+ * internal data-structures point to the current offset of each stream that can be written to.
+ */
 abstract class TermsHashPerField implements Comparable<TermsHashPerField> {
   private static final int HASH_INIT_SIZE = 4;
 
-  final TermsHash termsHash;
-
-  final TermsHashPerField nextPerField;
-  protected final DocumentsWriterPerThread.DocState docState;
-  protected final FieldInvertState fieldState;
-  TermToBytesRefAttribute termAtt;
-  protected TermFrequencyAttribute termFreqAtt;
-
-  // Copied from our perThread
-  final IntBlockPool intPool;
+  private final TermsHashPerField nextPerField;
+  private final IntBlockPool intPool;
   final ByteBlockPool bytePool;
-  final ByteBlockPool termBytePool;
-
-  final int streamCount;
-  final int numPostingInt;
-
-  protected final FieldInfo fieldInfo;
-
-  final BytesRefHash bytesHash;
+  // for each term we store an integer per stream that points into the bytePool above
+  // the address is updated once data is written to the stream to point to the next free offset
+  // this the terms stream. The start address for the stream is stored in postingsArray.byteStarts[termId]
+  // This is initialized in the #addTerm method, either to a brand new per term stream if the term is new or
+  // to the addresses where the term stream was written to when we saw it the last time.
+  private int[] termStreamAddressBuffer;
+  private int streamAddressOffset;
+  private final int streamCount;
+  private final String fieldName;
+  final IndexOptions indexOptions;
+  /* This stores the actual term bytes for postings and offsets into the parent hash in the case that this
+  * TermsHashPerField is hashing term vectors.*/
+  private final BytesRefHash bytesHash;
 
   ParallelPostingsArray postingsArray;
-  private final Counter bytesUsed;
+  private int lastDocID; // only with assert
 
   /** streamCount: how many streams this field stores per term.
    * E.g. doc(+freq) is 1 stream, prox+offset is a second. */
-
-  public TermsHashPerField(int streamCount, FieldInvertState fieldState, TermsHash termsHash, TermsHashPerField nextPerField, FieldInfo fieldInfo) {
-    intPool = termsHash.intPool;
-    bytePool = termsHash.bytePool;
-    termBytePool = termsHash.termBytePool;
-    docState = termsHash.docState;
-    this.termsHash = termsHash;
-    bytesUsed = termsHash.bytesUsed;
-    this.fieldState = fieldState;
+  TermsHashPerField(int streamCount, IntBlockPool intPool, ByteBlockPool bytePool, ByteBlockPool termBytePool,
+                    Counter bytesUsed, TermsHashPerField nextPerField, String fieldName, IndexOptions indexOptions) {
+    this.intPool = intPool;
+    this.bytePool = bytePool;
     this.streamCount = streamCount;
-    numPostingInt = 2*streamCount;
-    this.fieldInfo = fieldInfo;
+    this.fieldName = fieldName;
     this.nextPerField = nextPerField;
+    assert indexOptions != IndexOptions.NONE;
+    this.indexOptions = indexOptions;
     PostingsBytesStartArray byteStarts = new PostingsBytesStartArray(this, bytesUsed);
     bytesHash = new BytesRefHash(termBytePool, HASH_INIT_SIZE, byteStarts);
   }
 
   void reset() {
     bytesHash.clear(false);
+    sortedTermIDs = null;
     if (nextPerField != null) {
       nextPerField.reset();
     }
   }
 
-  public void initReader(ByteSliceReader reader, int termID, int stream) {
+  final void initReader(ByteSliceReader reader, int termID, int stream) {
     assert stream < streamCount;
-    int intStart = postingsArray.intStarts[termID];
-    final int[] ints = intPool.buffers[intStart >> IntBlockPool.INT_BLOCK_SHIFT];
-    final int upto = intStart & IntBlockPool.INT_BLOCK_MASK;
+    int streamStartOffset = postingsArray.addressOffset[termID];
+    final int[] streamAddressBuffer = intPool.buffers[streamStartOffset >> IntBlockPool.INT_BLOCK_SHIFT];
+    final int offsetInAddressBuffer = streamStartOffset & IntBlockPool.INT_BLOCK_MASK;
     reader.init(bytePool,
                 postingsArray.byteStarts[termID]+stream*ByteBlockPool.FIRST_LEVEL_SIZE,
-                ints[upto+stream]);
+                streamAddressBuffer[offsetInAddressBuffer+stream]);
   }
 
-  int[] sortedTermIDs;
+  private int[] sortedTermIDs;
 
   /** Collapse the hash table and sort in-place; also sets
-   * this.sortedTermIDs to the results */
-  public int[] sortPostings() {
+   * this.sortedTermIDs to the results
+   * This method should not be called twice unless {@link #reset()}

Review comment:
       should not -> must not perhaps?

##########
File path: lucene/core/src/test/org/apache/lucene/index/TestTermsHashPerField.java
##########
@@ -0,0 +1,210 @@
+/*
+ * 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.index;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.TreeMap;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+
+import com.carrotsearch.randomizedtesting.generators.RandomPicks;
+import com.carrotsearch.randomizedtesting.generators.RandomStrings;
+import org.apache.lucene.util.ByteBlockPool;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.Counter;
+import org.apache.lucene.util.IntBlockPool;
+import org.apache.lucene.util.LuceneTestCase;
+
+public class TestTermsHashPerField extends LuceneTestCase  {
+
+  private static TermsHashPerField createNewHash(AtomicInteger newCalled, AtomicInteger addCalled) {
+    IntBlockPool intBlockPool = new IntBlockPool();
+    ByteBlockPool byteBlockPool = new ByteBlockPool(new ByteBlockPool.DirectAllocator());
+    ByteBlockPool termBlockPool = new ByteBlockPool(new ByteBlockPool.DirectAllocator());
+
+    TermsHashPerField hash = new TermsHashPerField(1, intBlockPool, byteBlockPool, termBlockPool, Counter.newCounter(),
+        null, "testfield", IndexOptions.DOCS_AND_FREQS) {
+
+      private FreqProxTermsWriterPerField.FreqProxPostingsArray freqProxPostingsArray;
+
+      @Override
+      void newTerm(int termID, int docID) {
+        newCalled.incrementAndGet();
+        FreqProxTermsWriterPerField.FreqProxPostingsArray postings = freqProxPostingsArray;
+        postings.lastDocIDs[termID] = docID;
+        postings.lastDocCodes[termID] = docID << 1;
+        postings.termFreqs[termID] = 1;
+      }
+
+      @Override
+      void addTerm(int termID, int docID) {
+        addCalled.incrementAndGet();
+        FreqProxTermsWriterPerField.FreqProxPostingsArray postings = freqProxPostingsArray;
+        if (docID != postings.lastDocIDs[termID]) {
+          if (1 == postings.termFreqs[termID]) {
+            writeVInt(0, postings.lastDocCodes[termID]|1);
+          } else {
+            writeVInt(0, postings.lastDocCodes[termID]);
+            writeVInt(0, postings.termFreqs[termID]);
+          }
+          postings.termFreqs[termID] = 1;
+          postings.lastDocCodes[termID] = (docID - postings.lastDocIDs[termID]) << 1;
+          postings.lastDocIDs[termID] = docID;
+        } else {
+          postings.termFreqs[termID] = Math.addExact(postings.termFreqs[termID], 1);
+        }
+      }
+
+      @Override
+      void newPostingsArray() {
+        freqProxPostingsArray = (FreqProxTermsWriterPerField.FreqProxPostingsArray) postingsArray;
+

Review comment:
       extra space?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene-solr] dweiss commented on a change in pull request #1573: Cleanup TermsHashPerField

Posted by GitBox <gi...@apache.org>.
dweiss commented on a change in pull request #1573:
URL: https://github.com/apache/lucene-solr/pull/1573#discussion_r439857921



##########
File path: lucene/core/src/java/org/apache/lucene/index/TermsHashPerField.java
##########
@@ -19,203 +19,207 @@
 
 import java.io.IOException;
 
-import org.apache.lucene.analysis.tokenattributes.TermFrequencyAttribute;
-import org.apache.lucene.analysis.tokenattributes.TermToBytesRefAttribute;
 import org.apache.lucene.util.ByteBlockPool;
+import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefHash.BytesStartArray;
 import org.apache.lucene.util.BytesRefHash;
 import org.apache.lucene.util.Counter;
 import org.apache.lucene.util.IntBlockPool;
 
+/**
+ * This class stores streams of information per term without knowing
+ * the size of the stream ahead of time. Each stream typically encodes one level
+ * of information like term frequency per document or term proximity. Internally
+ * this class allocates a linked list of slices that can be read by a {@link ByteSliceReader}
+ * for each term. Terms are first deduplicated in a {@link BytesRefHash} once this is done
+ * internal data-structures point to the current offset of each stream that can be written to.
+ */
 abstract class TermsHashPerField implements Comparable<TermsHashPerField> {
   private static final int HASH_INIT_SIZE = 4;
 
-  final TermsHash termsHash;
-
-  final TermsHashPerField nextPerField;
-  protected final DocumentsWriterPerThread.DocState docState;
-  protected final FieldInvertState fieldState;
-  TermToBytesRefAttribute termAtt;
-  protected TermFrequencyAttribute termFreqAtt;
-
-  // Copied from our perThread
-  final IntBlockPool intPool;
+  private final TermsHashPerField nextPerField;
+  private final IntBlockPool intPool;
   final ByteBlockPool bytePool;
-  final ByteBlockPool termBytePool;
-
-  final int streamCount;
-  final int numPostingInt;
-
-  protected final FieldInfo fieldInfo;
-
-  final BytesRefHash bytesHash;
+  // for each term we store an integer per stream that points into the bytePool above
+  // the address is updated once data is written to the stream to point to the next free offset
+  // this the terms stream. The start address for the stream is stored in postingsArray.byteStarts[termId]

Review comment:
       this the?




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org


[GitHub] [lucene-solr] s1monw commented on a change in pull request #1573: Cleanup TermsHashPerField

Posted by GitBox <gi...@apache.org>.
s1monw commented on a change in pull request #1573:
URL: https://github.com/apache/lucene-solr/pull/1573#discussion_r439805381



##########
File path: lucene/core/src/java/org/apache/lucene/index/FreqProxTermsWriterPerField.java
##########
@@ -56,12 +56,6 @@ public FreqProxTermsWriterPerField(FieldInvertState invertState, TermsHash terms
   @Override
   void finish() throws IOException {
     super.finish();
-    sumDocFreq += fieldState.uniqueTermCount;
-    sumTotalTermFreq += fieldState.length;

Review comment:
       see this https://github.com/apache/lucene-solr/pull/1573/files#diff-aa6c5376b6b755262430916164fd0088L84




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@lucene.apache.org
For additional commands, e-mail: issues-help@lucene.apache.org