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 2021/02/09 11:34:35 UTC

[GitHub] [lucene-solr] iverase opened a new pull request #2334: LUCENE-9705: Create Lucene90TermVectorsFormat

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


   For now this is just a copy of Lucene90TermVectorsFormat. The existing
   Lucene50TermVectorsFormat was moved to backwards-codecs, along with its utility
   classes.


----------------------------------------------------------------
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] muse-dev[bot] commented on a change in pull request #2334: LUCENE-9705: Create Lucene90TermVectorsFormat

Posted by GitBox <gi...@apache.org>.
muse-dev[bot] commented on a change in pull request #2334:
URL: https://github.com/apache/lucene-solr/pull/2334#discussion_r572873185



##########
File path: lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/compressing/Lucene50CompressingTermVectorsReader.java
##########
@@ -0,0 +1,1367 @@
+/*
+ * 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.backward_codecs.compressing;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.NoSuchElementException;
+import org.apache.lucene.codecs.CodecUtil;
+import org.apache.lucene.codecs.TermVectorsReader;
+import org.apache.lucene.codecs.compressing.CompressionMode;
+import org.apache.lucene.codecs.compressing.Decompressor;
+import org.apache.lucene.index.BaseTermsEnum;
+import org.apache.lucene.index.CorruptIndexException;
+import org.apache.lucene.index.FieldInfo;
+import org.apache.lucene.index.FieldInfos;
+import org.apache.lucene.index.Fields;
+import org.apache.lucene.index.ImpactsEnum;
+import org.apache.lucene.index.IndexFileNames;
+import org.apache.lucene.index.PostingsEnum;
+import org.apache.lucene.index.SegmentInfo;
+import org.apache.lucene.index.SlowImpactsEnum;
+import org.apache.lucene.index.Terms;
+import org.apache.lucene.index.TermsEnum;
+import org.apache.lucene.store.AlreadyClosedException;
+import org.apache.lucene.store.ByteArrayDataInput;
+import org.apache.lucene.store.ChecksumIndexInput;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
+import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.util.Accountable;
+import org.apache.lucene.util.Accountables;
+import org.apache.lucene.util.ArrayUtil;
+import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.IOUtils;
+import org.apache.lucene.util.LongsRef;
+import org.apache.lucene.util.packed.BlockPackedReaderIterator;
+import org.apache.lucene.util.packed.PackedInts;
+
+/**
+ * {@link TermVectorsReader} for {@link Lucene50CompressingTermVectorsFormat}.
+ *
+ * @lucene.experimental
+ */
+public final class Lucene50CompressingTermVectorsReader extends TermVectorsReader
+    implements Closeable {
+
+  // hard limit on the maximum number of documents per chunk
+  static final int MAX_DOCUMENTS_PER_CHUNK = 128;
+
+  static final String VECTORS_EXTENSION = "tvd";
+  static final String VECTORS_INDEX_EXTENSION = "tvx";
+  static final String VECTORS_META_EXTENSION = "tvm";
+  static final String VECTORS_INDEX_CODEC_NAME = "Lucene85TermVectorsIndex";
+
+  static final int VERSION_START = 1;
+  static final int VERSION_OFFHEAP_INDEX = 2;
+  /** Version where all metadata were moved to the meta file. */
+  static final int VERSION_META = 3;
+
+  static final int VERSION_CURRENT = VERSION_META;
+  static final int META_VERSION_START = 0;
+
+  static final int PACKED_BLOCK_SIZE = 64;
+
+  static final int POSITIONS = 0x01;
+  static final int OFFSETS = 0x02;
+  static final int PAYLOADS = 0x04;
+  static final int FLAGS_BITS = PackedInts.bitsRequired(POSITIONS | OFFSETS | PAYLOADS);
+
+  private final FieldInfos fieldInfos;
+  final FieldsIndex indexReader;
+  final IndexInput vectorsStream;
+  private final int version;
+  private final int packedIntsVersion;
+  private final CompressionMode compressionMode;
+  private final Decompressor decompressor;
+  private final int chunkSize;
+  private final int numDocs;
+  private boolean closed;
+  private final BlockPackedReaderIterator reader;
+  private final long numDirtyChunks; // number of incomplete compressed blocks written
+  private final long numDirtyDocs; // cumulative number of missing docs in incomplete chunks
+  private final long maxPointer; // end of the data section
+
+  // used by clone
+  private Lucene50CompressingTermVectorsReader(Lucene50CompressingTermVectorsReader reader) {
+    this.fieldInfos = reader.fieldInfos;
+    this.vectorsStream = reader.vectorsStream.clone();
+    this.indexReader = reader.indexReader.clone();
+    this.packedIntsVersion = reader.packedIntsVersion;
+    this.compressionMode = reader.compressionMode;
+    this.decompressor = reader.decompressor.clone();
+    this.chunkSize = reader.chunkSize;
+    this.numDocs = reader.numDocs;
+    this.reader =
+        new BlockPackedReaderIterator(vectorsStream, packedIntsVersion, PACKED_BLOCK_SIZE, 0);
+    this.version = reader.version;
+    this.numDirtyChunks = reader.numDirtyChunks;
+    this.numDirtyDocs = reader.numDirtyDocs;
+    this.maxPointer = reader.maxPointer;
+    this.closed = false;
+  }
+
+  /** Sole constructor. */
+  public Lucene50CompressingTermVectorsReader(
+      Directory d,
+      SegmentInfo si,
+      String segmentSuffix,
+      FieldInfos fn,
+      IOContext context,
+      String formatName,
+      CompressionMode compressionMode)
+      throws IOException {
+    this.compressionMode = compressionMode;
+    final String segment = si.name;
+    boolean success = false;
+    fieldInfos = fn;
+    numDocs = si.maxDoc();
+
+    ChecksumIndexInput metaIn = null;
+    try {
+      // Open the data file
+      final String vectorsStreamFN =
+          IndexFileNames.segmentFileName(segment, segmentSuffix, VECTORS_EXTENSION);
+      vectorsStream = d.openInput(vectorsStreamFN, context);
+      version =
+          CodecUtil.checkIndexHeader(
+              vectorsStream, formatName, VERSION_START, VERSION_CURRENT, si.getId(), segmentSuffix);
+      assert CodecUtil.indexHeaderLength(formatName, segmentSuffix)
+          == vectorsStream.getFilePointer();
+
+      if (version >= VERSION_OFFHEAP_INDEX) {
+        final String metaStreamFN =
+            IndexFileNames.segmentFileName(segment, segmentSuffix, VECTORS_META_EXTENSION);
+        metaIn = d.openChecksumInput(metaStreamFN, IOContext.READONCE);
+        CodecUtil.checkIndexHeader(
+            metaIn,
+            VECTORS_INDEX_CODEC_NAME + "Meta",
+            META_VERSION_START,
+            version,
+            si.getId(),
+            segmentSuffix);
+      }
+
+      if (version >= VERSION_META) {
+        packedIntsVersion = metaIn.readVInt();
+        chunkSize = metaIn.readVInt();
+      } else {
+        packedIntsVersion = vectorsStream.readVInt();
+        chunkSize = vectorsStream.readVInt();
+      }
+
+      // NOTE: data file is too costly to verify checksum against all the bytes on open,
+      // but for now we at least verify proper structure of the checksum footer: which looks
+      // for FOOTER_MAGIC + algorithmID. This is cheap and can detect some forms of corruption
+      // such as file truncation.
+      CodecUtil.retrieveChecksum(vectorsStream);
+
+      FieldsIndex indexReader = null;
+      long maxPointer = -1;
+
+      if (version < VERSION_OFFHEAP_INDEX) {
+        // Load the index into memory
+        final String indexName = IndexFileNames.segmentFileName(segment, segmentSuffix, "tvx");
+        try (ChecksumIndexInput indexStream = d.openChecksumInput(indexName, context)) {
+          Throwable priorE = null;
+          try {
+            assert formatName.endsWith("Data");
+            final String codecNameIdx =
+                formatName.substring(0, formatName.length() - "Data".length()) + "Index";
+            final int version2 =
+                CodecUtil.checkIndexHeader(
+                    indexStream,
+                    codecNameIdx,
+                    VERSION_START,
+                    VERSION_CURRENT,
+                    si.getId(),
+                    segmentSuffix);
+            if (version != version2) {
+              throw new CorruptIndexException(
+                  "Version mismatch between stored fields index and data: "
+                      + version
+                      + " != "
+                      + version2,
+                  indexStream);
+            }
+            assert CodecUtil.indexHeaderLength(codecNameIdx, segmentSuffix)
+                == indexStream.getFilePointer();
+            indexReader = new LegacyFieldsIndexReader(indexStream, si);
+            maxPointer = indexStream.readVLong(); // the end of the data section
+          } catch (Throwable exception) {
+            priorE = exception;
+          } finally {
+            CodecUtil.checkFooter(indexStream, priorE);
+          }
+        }
+      } else {
+        FieldsIndexReader fieldsIndexReader =
+            new FieldsIndexReader(
+                d,
+                si.name,
+                segmentSuffix,
+                VECTORS_INDEX_EXTENSION,
+                VECTORS_INDEX_CODEC_NAME,
+                si.getId(),
+                metaIn);
+        indexReader = fieldsIndexReader;
+        maxPointer = fieldsIndexReader.getMaxPointer();
+      }
+
+      this.indexReader = indexReader;
+      this.maxPointer = maxPointer;
+
+      if (version >= VERSION_META) {
+        numDirtyChunks = metaIn.readVLong();
+        numDirtyDocs = metaIn.readVLong();
+      } else {
+        // Old versions of this format did not record numDirtyDocs. Since bulk
+        // merges are disabled on version increments anyway, we make no effort
+        // to get valid values of numDirtyChunks and numDirtyDocs.
+        numDirtyChunks = numDirtyDocs = -1;
+      }
+
+      decompressor = compressionMode.newDecompressor();
+      this.reader =
+          new BlockPackedReaderIterator(vectorsStream, packedIntsVersion, PACKED_BLOCK_SIZE, 0);
+
+      if (metaIn != null) {
+        CodecUtil.checkFooter(metaIn, null);
+        metaIn.close();
+      }
+
+      success = true;
+    } catch (Throwable t) {
+      if (metaIn != null) {
+        CodecUtil.checkFooter(metaIn, t);
+        throw new AssertionError("unreachable");
+      } else {
+        throw t;
+      }
+    } finally {
+      if (!success) {
+        IOUtils.closeWhileHandlingException(this, metaIn);
+      }
+    }
+  }
+
+  CompressionMode getCompressionMode() {
+    return compressionMode;
+  }
+
+  int getChunkSize() {
+    return chunkSize;
+  }
+
+  int getPackedIntsVersion() {
+    return packedIntsVersion;
+  }
+
+  int getVersion() {
+    return version;
+  }
+
+  FieldsIndex getIndexReader() {
+    return indexReader;
+  }
+
+  IndexInput getVectorsStream() {
+    return vectorsStream;
+  }
+
+  long getMaxPointer() {
+    return maxPointer;
+  }
+
+  long getNumDirtyDocs() {
+    if (version != VERSION_CURRENT) {
+      throw new IllegalStateException(
+          "getNumDirtyDocs should only ever get called when the reader is on the current version");
+    }
+    assert numDirtyDocs >= 0;
+    return numDirtyDocs;
+  }
+
+  long getNumDirtyChunks() {
+    if (version != VERSION_CURRENT) {
+      throw new IllegalStateException(
+          "getNumDirtyChunks should only ever get called when the reader is on the current version");
+    }
+    assert numDirtyChunks >= 0;
+    return numDirtyChunks;
+  }
+
+  int getNumDocs() {
+    return numDocs;
+  }
+
+  /** @throws AlreadyClosedException if this TermVectorsReader is closed */
+  private void ensureOpen() throws AlreadyClosedException {
+    if (closed) {
+      throw new AlreadyClosedException("this FieldsReader is closed");
+    }
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (!closed) {
+      IOUtils.close(indexReader, vectorsStream);
+      closed = true;
+    }
+  }
+
+  @Override
+  public TermVectorsReader clone() {
+    return new Lucene50CompressingTermVectorsReader(this);
+  }
+
+  @Override
+  public Fields get(int doc) throws IOException {
+    ensureOpen();
+
+    // seek to the right place
+    {
+      final long startPointer = indexReader.getStartPointer(doc);
+      vectorsStream.seek(startPointer);
+    }
+
+    // decode
+    // - docBase: first doc ID of the chunk
+    // - chunkDocs: number of docs of the chunk
+    final int docBase = vectorsStream.readVInt();
+    final int chunkDocs = vectorsStream.readVInt();
+    if (doc < docBase || doc >= docBase + chunkDocs || docBase + chunkDocs > numDocs) {
+      throw new CorruptIndexException(
+          "docBase=" + docBase + ",chunkDocs=" + chunkDocs + ",doc=" + doc, vectorsStream);
+    }
+
+    final int skip; // number of fields to skip
+    final int numFields; // number of fields of the document we're looking for
+    final int totalFields; // total number of fields of the chunk (sum for all docs)
+    if (chunkDocs == 1) {
+      skip = 0;
+      numFields = totalFields = vectorsStream.readVInt();
+    } else {
+      reader.reset(vectorsStream, chunkDocs);
+      int sum = 0;
+      for (int i = docBase; i < doc; ++i) {
+        sum += reader.next();
+      }
+      skip = sum;
+      numFields = (int) reader.next();
+      sum += numFields;
+      for (int i = doc + 1; i < docBase + chunkDocs; ++i) {
+        sum += reader.next();
+      }
+      totalFields = sum;
+    }
+
+    if (numFields == 0) {
+      // no vectors
+      return null;
+    }
+
+    // read field numbers that have term vectors
+    final int[] fieldNums;
+    {
+      final int token = vectorsStream.readByte() & 0xFF;
+      assert token != 0; // means no term vectors, cannot happen since we checked for numFields == 0
+      final int bitsPerFieldNum = token & 0x1F;
+      int totalDistinctFields = token >>> 5;
+      if (totalDistinctFields == 0x07) {
+        totalDistinctFields += vectorsStream.readVInt();
+      }
+      ++totalDistinctFields;
+      final PackedInts.ReaderIterator it =
+          PackedInts.getReaderIteratorNoHeader(
+              vectorsStream,
+              PackedInts.Format.PACKED,
+              packedIntsVersion,
+              totalDistinctFields,
+              bitsPerFieldNum,
+              1);
+      fieldNums = new int[totalDistinctFields];
+      for (int i = 0; i < totalDistinctFields; ++i) {
+        fieldNums[i] = (int) it.next();
+      }
+    }
+
+    // read field numbers and flags
+    final int[] fieldNumOffs = new int[numFields];
+    final PackedInts.Reader flags;
+    {
+      final int bitsPerOff = PackedInts.bitsRequired(fieldNums.length - 1);
+      final PackedInts.Reader allFieldNumOffs =
+          PackedInts.getReaderNoHeader(
+              vectorsStream, PackedInts.Format.PACKED, packedIntsVersion, totalFields, bitsPerOff);
+      switch (vectorsStream.readVInt()) {
+        case 0:
+          final PackedInts.Reader fieldFlags =
+              PackedInts.getReaderNoHeader(
+                  vectorsStream,
+                  PackedInts.Format.PACKED,
+                  packedIntsVersion,
+                  fieldNums.length,
+                  FLAGS_BITS);
+          PackedInts.Mutable f = PackedInts.getMutable(totalFields, FLAGS_BITS, PackedInts.COMPACT);
+          for (int i = 0; i < totalFields; ++i) {
+            final int fieldNumOff = (int) allFieldNumOffs.get(i);
+            assert fieldNumOff >= 0 && fieldNumOff < fieldNums.length;
+            final int fgs = (int) fieldFlags.get(fieldNumOff);
+            f.set(i, fgs);
+          }
+          flags = f;
+          break;
+        case 1:
+          flags =
+              PackedInts.getReaderNoHeader(
+                  vectorsStream,
+                  PackedInts.Format.PACKED,
+                  packedIntsVersion,
+                  totalFields,
+                  FLAGS_BITS);
+          break;
+        default:
+          throw new AssertionError();
+      }
+      for (int i = 0; i < numFields; ++i) {
+        fieldNumOffs[i] = (int) allFieldNumOffs.get(skip + i);
+      }
+    }
+
+    // number of terms per field for all fields
+    final PackedInts.Reader numTerms;
+    final int totalTerms;
+    {
+      final int bitsRequired = vectorsStream.readVInt();
+      numTerms =
+          PackedInts.getReaderNoHeader(
+              vectorsStream,
+              PackedInts.Format.PACKED,
+              packedIntsVersion,
+              totalFields,
+              bitsRequired);
+      int sum = 0;
+      for (int i = 0; i < totalFields; ++i) {
+        sum += numTerms.get(i);
+      }
+      totalTerms = sum;
+    }
+
+    // term lengths
+    int docOff = 0, docLen = 0, totalLen;
+    final int[] fieldLengths = new int[numFields];
+    final int[][] prefixLengths = new int[numFields][];
+    final int[][] suffixLengths = new int[numFields][];
+    {
+      reader.reset(vectorsStream, totalTerms);
+      // skip
+      int toSkip = 0;
+      for (int i = 0; i < skip; ++i) {
+        toSkip += numTerms.get(i);
+      }
+      reader.skip(toSkip);
+      // read prefix lengths
+      for (int i = 0; i < numFields; ++i) {
+        final int termCount = (int) numTerms.get(skip + i);
+        final int[] fieldPrefixLengths = new int[termCount];
+        prefixLengths[i] = fieldPrefixLengths;
+        for (int j = 0; j < termCount; ) {
+          final LongsRef next = reader.next(termCount - j);
+          for (int k = 0; k < next.length; ++k) {
+            fieldPrefixLengths[j++] = (int) next.longs[next.offset + k];
+          }
+        }
+      }
+      reader.skip(totalTerms - reader.ord());
+
+      reader.reset(vectorsStream, totalTerms);
+      // skip
+      toSkip = 0;
+      for (int i = 0; i < skip; ++i) {
+        for (int j = 0; j < numTerms.get(i); ++j) {
+          docOff += reader.next();
+        }
+      }
+      for (int i = 0; i < numFields; ++i) {
+        final int termCount = (int) numTerms.get(skip + i);
+        final int[] fieldSuffixLengths = new int[termCount];
+        suffixLengths[i] = fieldSuffixLengths;
+        for (int j = 0; j < termCount; ) {
+          final LongsRef next = reader.next(termCount - j);
+          for (int k = 0; k < next.length; ++k) {
+            fieldSuffixLengths[j++] = (int) next.longs[next.offset + k];
+          }
+        }
+        fieldLengths[i] = sum(suffixLengths[i]);
+        docLen += fieldLengths[i];
+      }
+      totalLen = docOff + docLen;
+      for (int i = skip + numFields; i < totalFields; ++i) {
+        for (int j = 0; j < numTerms.get(i); ++j) {
+          totalLen += reader.next();
+        }
+      }
+    }
+
+    // term freqs
+    final int[] termFreqs = new int[totalTerms];
+    {
+      reader.reset(vectorsStream, totalTerms);
+      for (int i = 0; i < totalTerms; ) {
+        final LongsRef next = reader.next(totalTerms - i);
+        for (int k = 0; k < next.length; ++k) {
+          termFreqs[i++] = 1 + (int) next.longs[next.offset + k];
+        }
+      }
+    }
+
+    // total number of positions, offsets and payloads
+    int totalPositions = 0, totalOffsets = 0, totalPayloads = 0;
+    for (int i = 0, termIndex = 0; i < totalFields; ++i) {
+      final int f = (int) flags.get(i);
+      final int termCount = (int) numTerms.get(i);
+      for (int j = 0; j < termCount; ++j) {
+        final int freq = termFreqs[termIndex++];
+        if ((f & POSITIONS) != 0) {
+          totalPositions += freq;
+        }
+        if ((f & OFFSETS) != 0) {
+          totalOffsets += freq;
+        }
+        if ((f & PAYLOADS) != 0) {
+          totalPayloads += freq;
+        }
+      }
+      assert i != totalFields - 1 || termIndex == totalTerms : termIndex + " " + totalTerms;
+    }
+
+    final int[][] positionIndex = positionIndex(skip, numFields, numTerms, termFreqs);
+    final int[][] positions, startOffsets, lengths;
+    if (totalPositions > 0) {
+      positions =
+          readPositions(
+              skip,
+              numFields,
+              flags,
+              numTerms,
+              termFreqs,
+              POSITIONS,
+              totalPositions,
+              positionIndex);
+    } else {
+      positions = new int[numFields][];
+    }
+
+    if (totalOffsets > 0) {
+      // average number of chars per term
+      final float[] charsPerTerm = new float[fieldNums.length];
+      for (int i = 0; i < charsPerTerm.length; ++i) {
+        charsPerTerm[i] = Float.intBitsToFloat(vectorsStream.readInt());
+      }
+      startOffsets =
+          readPositions(
+              skip, numFields, flags, numTerms, termFreqs, OFFSETS, totalOffsets, positionIndex);
+      lengths =
+          readPositions(
+              skip, numFields, flags, numTerms, termFreqs, OFFSETS, totalOffsets, positionIndex);
+
+      for (int i = 0; i < numFields; ++i) {
+        final int[] fStartOffsets = startOffsets[i];
+        final int[] fPositions = positions[i];
+        // patch offsets from positions
+        if (fStartOffsets != null && fPositions != null) {
+          final float fieldCharsPerTerm = charsPerTerm[fieldNumOffs[i]];
+          for (int j = 0; j < startOffsets[i].length; ++j) {
+            fStartOffsets[j] += (int) (fieldCharsPerTerm * fPositions[j]);
+          }
+        }
+        if (fStartOffsets != null) {
+          final int[] fPrefixLengths = prefixLengths[i];
+          final int[] fSuffixLengths = suffixLengths[i];
+          final int[] fLengths = lengths[i];
+          for (int j = 0, end = (int) numTerms.get(skip + i); j < end; ++j) {
+            // delta-decode start offsets and  patch lengths using term lengths
+            final int termLength = fPrefixLengths[j] + fSuffixLengths[j];
+            lengths[i][positionIndex[i][j]] += termLength;
+            for (int k = positionIndex[i][j] + 1; k < positionIndex[i][j + 1]; ++k) {
+              fStartOffsets[k] += fStartOffsets[k - 1];
+              fLengths[k] += termLength;
+            }
+          }
+        }
+      }
+    } else {
+      startOffsets = lengths = new int[numFields][];
+    }
+    if (totalPositions > 0) {
+      // delta-decode positions
+      for (int i = 0; i < numFields; ++i) {
+        final int[] fPositions = positions[i];
+        final int[] fpositionIndex = positionIndex[i];
+        if (fPositions != null) {
+          for (int j = 0, end = (int) numTerms.get(skip + i); j < end; ++j) {
+            // delta-decode start offsets
+            for (int k = fpositionIndex[j] + 1; k < fpositionIndex[j + 1]; ++k) {
+              fPositions[k] += fPositions[k - 1];
+            }
+          }
+        }
+      }
+    }
+
+    // payload lengths
+    final int[][] payloadIndex = new int[numFields][];
+    int totalPayloadLength = 0;
+    int payloadOff = 0;
+    int payloadLen = 0;
+    if (totalPayloads > 0) {
+      reader.reset(vectorsStream, totalPayloads);
+      // skip
+      int termIndex = 0;
+      for (int i = 0; i < skip; ++i) {
+        final int f = (int) flags.get(i);
+        final int termCount = (int) numTerms.get(i);
+        if ((f & PAYLOADS) != 0) {
+          for (int j = 0; j < termCount; ++j) {
+            final int freq = termFreqs[termIndex + j];
+            for (int k = 0; k < freq; ++k) {
+              final int l = (int) reader.next();
+              payloadOff += l;
+            }
+          }
+        }
+        termIndex += termCount;
+      }
+      totalPayloadLength = payloadOff;
+      // read doc payload lengths
+      for (int i = 0; i < numFields; ++i) {
+        final int f = (int) flags.get(skip + i);
+        final int termCount = (int) numTerms.get(skip + i);
+        if ((f & PAYLOADS) != 0) {
+          final int totalFreq = positionIndex[i][termCount];
+          payloadIndex[i] = new int[totalFreq + 1];
+          int posIdx = 0;
+          payloadIndex[i][posIdx] = payloadLen;
+          for (int j = 0; j < termCount; ++j) {
+            final int freq = termFreqs[termIndex + j];
+            for (int k = 0; k < freq; ++k) {
+              final int payloadLength = (int) reader.next();
+              payloadLen += payloadLength;
+              payloadIndex[i][posIdx + 1] = payloadLen;
+              ++posIdx;
+            }
+          }
+          assert posIdx == totalFreq;
+        }
+        termIndex += termCount;
+      }
+      totalPayloadLength += payloadLen;
+      for (int i = skip + numFields; i < totalFields; ++i) {
+        final int f = (int) flags.get(i);
+        final int termCount = (int) numTerms.get(i);
+        if ((f & PAYLOADS) != 0) {
+          for (int j = 0; j < termCount; ++j) {
+            final int freq = termFreqs[termIndex + j];
+            for (int k = 0; k < freq; ++k) {
+              totalPayloadLength += reader.next();
+            }
+          }
+        }
+        termIndex += termCount;
+      }
+      assert termIndex == totalTerms : termIndex + " " + totalTerms;
+    }
+
+    // decompress data
+    final BytesRef suffixBytes = new BytesRef();
+    decompressor.decompress(
+        vectorsStream,
+        totalLen + totalPayloadLength,
+        docOff + payloadOff,
+        docLen + payloadLen,
+        suffixBytes);
+    suffixBytes.length = docLen;
+    final BytesRef payloadBytes =
+        new BytesRef(suffixBytes.bytes, suffixBytes.offset + docLen, payloadLen);
+
+    final int[] fieldFlags = new int[numFields];
+    for (int i = 0; i < numFields; ++i) {
+      fieldFlags[i] = (int) flags.get(skip + i);
+    }
+
+    final int[] fieldNumTerms = new int[numFields];
+    for (int i = 0; i < numFields; ++i) {
+      fieldNumTerms[i] = (int) numTerms.get(skip + i);
+    }
+
+    final int[][] fieldTermFreqs = new int[numFields][];
+    {
+      int termIdx = 0;
+      for (int i = 0; i < skip; ++i) {
+        termIdx += numTerms.get(i);
+      }
+      for (int i = 0; i < numFields; ++i) {
+        final int termCount = (int) numTerms.get(skip + i);
+        fieldTermFreqs[i] = new int[termCount];
+        for (int j = 0; j < termCount; ++j) {
+          fieldTermFreqs[i][j] = termFreqs[termIdx++];
+        }
+      }
+    }
+
+    assert sum(fieldLengths) == docLen : sum(fieldLengths) + " != " + docLen;
+
+    return new TVFields(
+        fieldNums,
+        fieldFlags,
+        fieldNumOffs,
+        fieldNumTerms,
+        fieldLengths,
+        prefixLengths,
+        suffixLengths,
+        fieldTermFreqs,
+        positionIndex,
+        positions,
+        startOffsets,
+        lengths,
+        payloadBytes,
+        payloadIndex,
+        suffixBytes);
+  }
+
+  // field -> term index -> position index
+  private int[][] positionIndex(
+      int skip, int numFields, PackedInts.Reader numTerms, int[] termFreqs) {
+    final int[][] positionIndex = new int[numFields][];
+    int termIndex = 0;
+    for (int i = 0; i < skip; ++i) {
+      final int termCount = (int) numTerms.get(i);
+      termIndex += termCount;
+    }
+    for (int i = 0; i < numFields; ++i) {
+      final int termCount = (int) numTerms.get(skip + i);
+      positionIndex[i] = new int[termCount + 1];
+      for (int j = 0; j < termCount; ++j) {
+        final int freq = termFreqs[termIndex + j];
+        positionIndex[i][j + 1] = positionIndex[i][j] + freq;
+      }
+      termIndex += termCount;
+    }
+    return positionIndex;
+  }
+
+  private int[][] readPositions(
+      int skip,
+      int numFields,
+      PackedInts.Reader flags,
+      PackedInts.Reader numTerms,
+      int[] termFreqs,
+      int flag,
+      final int totalPositions,
+      int[][] positionIndex)
+      throws IOException {
+    final int[][] positions = new int[numFields][];
+    reader.reset(vectorsStream, totalPositions);
+    // skip
+    int toSkip = 0;
+    int termIndex = 0;
+    for (int i = 0; i < skip; ++i) {
+      final int f = (int) flags.get(i);
+      final int termCount = (int) numTerms.get(i);
+      if ((f & flag) != 0) {
+        for (int j = 0; j < termCount; ++j) {
+          final int freq = termFreqs[termIndex + j];
+          toSkip += freq;
+        }
+      }
+      termIndex += termCount;
+    }
+    reader.skip(toSkip);
+    // read doc positions
+    for (int i = 0; i < numFields; ++i) {
+      final int f = (int) flags.get(skip + i);
+      final int termCount = (int) numTerms.get(skip + i);
+      if ((f & flag) != 0) {
+        final int totalFreq = positionIndex[i][termCount];
+        final int[] fieldPositions = new int[totalFreq];
+        positions[i] = fieldPositions;
+        for (int j = 0; j < totalFreq; ) {
+          final LongsRef nextPositions = reader.next(totalFreq - j);
+          for (int k = 0; k < nextPositions.length; ++k) {
+            fieldPositions[j++] = (int) nextPositions.longs[nextPositions.offset + k];
+          }
+        }
+      }
+      termIndex += termCount;
+    }
+    reader.skip(totalPositions - reader.ord());
+    return positions;
+  }
+
+  private class TVFields extends Fields {
+
+    private final int[] fieldNums, fieldFlags, fieldNumOffs, numTerms, fieldLengths;
+    private final int[][] prefixLengths,
+        suffixLengths,
+        termFreqs,
+        positionIndex,
+        positions,
+        startOffsets,
+        lengths,
+        payloadIndex;
+    private final BytesRef suffixBytes, payloadBytes;
+
+    public TVFields(
+        int[] fieldNums,
+        int[] fieldFlags,
+        int[] fieldNumOffs,
+        int[] numTerms,
+        int[] fieldLengths,
+        int[][] prefixLengths,
+        int[][] suffixLengths,
+        int[][] termFreqs,
+        int[][] positionIndex,
+        int[][] positions,
+        int[][] startOffsets,
+        int[][] lengths,
+        BytesRef payloadBytes,
+        int[][] payloadIndex,
+        BytesRef suffixBytes) {
+      this.fieldNums = fieldNums;
+      this.fieldFlags = fieldFlags;
+      this.fieldNumOffs = fieldNumOffs;
+      this.numTerms = numTerms;
+      this.fieldLengths = fieldLengths;
+      this.prefixLengths = prefixLengths;
+      this.suffixLengths = suffixLengths;
+      this.termFreqs = termFreqs;
+      this.positionIndex = positionIndex;
+      this.positions = positions;
+      this.startOffsets = startOffsets;
+      this.lengths = lengths;
+      this.payloadBytes = payloadBytes;
+      this.payloadIndex = payloadIndex;
+      this.suffixBytes = suffixBytes;
+    }
+
+    @Override
+    public Iterator<String> iterator() {
+      return new Iterator<String>() {
+        int i = 0;
+
+        @Override
+        public boolean hasNext() {
+          return i < fieldNumOffs.length;
+        }
+
+        @Override
+        public String next() {
+          if (!hasNext()) {
+            throw new NoSuchElementException();
+          }
+          final int fieldNum = fieldNums[fieldNumOffs[i++]];
+          return fieldInfos.fieldInfo(fieldNum).name;

Review comment:
       *NULL_DEREFERENCE:*  object returned by `Lucene50CompressingTermVectorsReader$TVFields$1.this$1.this$0.fieldInfos.fieldInfo(fieldNum)` could be null and is dereferenced at line 881.




----------------------------------------------------------------
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] jtibshirani commented on a change in pull request #2334: LUCENE-9705: Create Lucene90TermVectorsFormat

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



##########
File path: lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/compressing/Lucene50CompressingTermVectorsFormat.java
##########
@@ -0,0 +1,113 @@
+/*
+ * 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.backward_codecs.compressing;
+
+import java.io.IOException;
+import org.apache.lucene.codecs.CodecUtil;
+import org.apache.lucene.codecs.StoredFieldsFormat;
+import org.apache.lucene.codecs.TermVectorsFormat;
+import org.apache.lucene.codecs.TermVectorsReader;
+import org.apache.lucene.codecs.TermVectorsWriter;
+import org.apache.lucene.codecs.compressing.CompressionMode;
+import org.apache.lucene.index.FieldInfos;
+import org.apache.lucene.index.SegmentInfo;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
+
+/**
+ * A {@link TermVectorsFormat} that compresses chunks of documents together in order to improve the
+ * compression ratio.
+ *
+ * @lucene.experimental
+ */
+public class Lucene50CompressingTermVectorsFormat extends TermVectorsFormat {

Review comment:
       This seems like a good name change to me. Thinking about the general pattern (so our other PRs are consistent):
   * For substantial codec classes like 'format', 'reader', 'writer', we add `LuceneXX` to the name if it's not already there. This makes it clear the class follows our backwards compatibility scheme.
   * For smaller helper classes like `FieldsReader` we move them but keep the name as-is.
   
   Does this make sense to you? A couple other questions:
   * For consistency, we could also rename the current `CompressingTermVectorsFormat` to `Lucene90CompressingTermVectorsFormat`.
   * Should these move to version-specific packages like `lucene50` since they're versioned?




----------------------------------------------------------------
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] jtibshirani commented on a change in pull request #2334: LUCENE-9705: Create Lucene90TermVectorsFormat

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



##########
File path: lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/compressing/Lucene50CompressingTermVectorsFormat.java
##########
@@ -0,0 +1,113 @@
+/*
+ * 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.backward_codecs.compressing;
+
+import java.io.IOException;
+import org.apache.lucene.codecs.CodecUtil;
+import org.apache.lucene.codecs.StoredFieldsFormat;
+import org.apache.lucene.codecs.TermVectorsFormat;
+import org.apache.lucene.codecs.TermVectorsReader;
+import org.apache.lucene.codecs.TermVectorsWriter;
+import org.apache.lucene.codecs.compressing.CompressionMode;
+import org.apache.lucene.index.FieldInfos;
+import org.apache.lucene.index.SegmentInfo;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
+
+/**
+ * A {@link TermVectorsFormat} that compresses chunks of documents together in order to improve the
+ * compression ratio.
+ *
+ * @lucene.experimental
+ */
+public class Lucene50CompressingTermVectorsFormat extends TermVectorsFormat {

Review comment:
       We discussed in #2310 and agreed on this package/ naming pattern.




----------------------------------------------------------------
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] iverase commented on a change in pull request #2334: LUCENE-9705: Create Lucene90TermVectorsFormat

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



##########
File path: lucene/core/src/java/org/apache/lucene/codecs/compressing/Lucene90CompressingTermVectorsFormat.java
##########
@@ -33,7 +33,7 @@
  *
  * @lucene.experimental
  */
-public class CompressingTermVectorsFormat extends TermVectorsFormat {
+public class Lucene90CompressingTermVectorsFormat extends TermVectorsFormat {

Review comment:
       I will do that once we move Lucene87StoredFieldFormat




----------------------------------------------------------------
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] jtibshirani commented on a change in pull request #2334: LUCENE-9705: Create Lucene90TermVectorsFormat

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



##########
File path: lucene/core/src/java/org/apache/lucene/codecs/compressing/Lucene90CompressingTermVectorsFormat.java
##########
@@ -33,7 +33,7 @@
  *
  * @lucene.experimental
  */
-public class CompressingTermVectorsFormat extends TermVectorsFormat {
+public class Lucene90CompressingTermVectorsFormat extends TermVectorsFormat {

Review comment:
       I think all these classes should be moved under the package `lucene90.compressing`. That way all of our "versioned" format files (plus their helper classes) will be in a package with the version name, ready to move to backwards-codecs when needed. This will be consistent with #2310.

##########
File path: lucene/backward-codecs/src/test/org/apache/lucene/backward_codecs/lucene50/compressing/Lucene50RWCompressingTermVectorsFormat.java
##########
@@ -0,0 +1,121 @@
+/*
+ * 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.backward_codecs.lucene50.compressing;
+
+import java.io.IOException;
+import org.apache.lucene.codecs.CodecUtil;
+import org.apache.lucene.codecs.StoredFieldsFormat;
+import org.apache.lucene.codecs.TermVectorsFormat;
+import org.apache.lucene.codecs.TermVectorsReader;
+import org.apache.lucene.codecs.TermVectorsWriter;
+import org.apache.lucene.codecs.compressing.CompressionMode;
+import org.apache.lucene.index.FieldInfos;
+import org.apache.lucene.index.SegmentInfo;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
+
+/**
+ * A {@link TermVectorsFormat} that compresses chunks of documents together in order to improve the
+ * compression ratio.
+ *
+ * @lucene.experimental
+ */
+public class Lucene50RWCompressingTermVectorsFormat extends TermVectorsFormat {

Review comment:
       Could this extend `Lucene50CompressingTermVectorsFormat`? This seems like a common set-up, for the read-write version to extend the read-only version.

##########
File path: lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/lucene50/compressing/package-info.java
##########
@@ -15,8 +15,5 @@
  * limitations under the License.
  */
 
-/**
- * Components from the Lucene 5.0 index format See {@link org.apache.lucene.codecs.lucene90} for an
- * overview of the index format.
- */
-package org.apache.lucene.codecs.lucene50;
+/** Lucene50 TermsVectorFormat . */

Review comment:
       I think this will hold other classes later related to `StoredFieldsFormat`, maybe we could remove this javadoc.




----------------------------------------------------------------
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] iverase merged pull request #2334: LUCENE-9705: Create Lucene90TermVectorsFormat

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


   


----------------------------------------------------------------
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] iverase commented on a change in pull request #2334: LUCENE-9705: Create Lucene90TermVectorsFormat

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



##########
File path: lucene/backward-codecs/src/java/org/apache/lucene/backward_codecs/compressing/Lucene50CompressingTermVectorsFormat.java
##########
@@ -0,0 +1,113 @@
+/*
+ * 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.backward_codecs.compressing;
+
+import java.io.IOException;
+import org.apache.lucene.codecs.CodecUtil;
+import org.apache.lucene.codecs.StoredFieldsFormat;
+import org.apache.lucene.codecs.TermVectorsFormat;
+import org.apache.lucene.codecs.TermVectorsReader;
+import org.apache.lucene.codecs.TermVectorsWriter;
+import org.apache.lucene.codecs.compressing.CompressionMode;
+import org.apache.lucene.index.FieldInfos;
+import org.apache.lucene.index.SegmentInfo;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
+
+/**
+ * A {@link TermVectorsFormat} that compresses chunks of documents together in order to improve the
+ * compression ratio.
+ *
+ * @lucene.experimental
+ */
+public class Lucene50CompressingTermVectorsFormat extends TermVectorsFormat {

Review comment:
       @jtibshirani I hope I addressed the suggestions here. I think this is ready for another review. Thanks!




----------------------------------------------------------------
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