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/05/01 07:45:03 UTC

[GitHub] [lucene-solr] jpountz opened a new pull request #1473: LUCENE-9353: Move terms metadata to its own file.

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


   See https://issues.apache.org/jira/browse/LUCENE-9353.
   


----------------------------------------------------------------
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] jpountz commented on a change in pull request #1473: LUCENE-9353: Move terms metadata to its own file.

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



##########
File path: lucene/core/src/java/org/apache/lucene/codecs/blocktree/BlockTreeTermsWriter.java
##########
@@ -1060,36 +1052,35 @@ public void close() throws IOException {
       return;
     }
     closed = true;
-    
+
+    final String metaName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, BlockTreeTermsReader.TERMS_META_EXTENSION);
     boolean success = false;
-    try {
-      
-      final long dirStart = termsOut.getFilePointer();
-      final long indexDirStart = indexOut.getFilePointer();
+    try (IndexOutput metaOut = state.directory.createOutput(metaName, state.context)) {

Review comment:
       That would work too. I like keeping the index output open for as little time as possible when it doesn't make things worse otherwise.




----------------------------------------------------------------
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] jpountz commented on a change in pull request #1473: LUCENE-9353: Move terms metadata to its own file.

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



##########
File path: lucene/core/src/java/org/apache/lucene/codecs/blocktree/BlockTreeTermsReader.java
##########
@@ -148,56 +155,80 @@ public BlockTreeTermsReader(PostingsReaderBase postingsReader, SegmentReadState
       CodecUtil.retrieveChecksum(termsIn);
 
       // Read per-field details
-      seekDir(termsIn);
-      seekDir(indexIn);
+      String metaName = IndexFileNames.segmentFileName(segment, state.segmentSuffix, TERMS_META_EXTENSION);
+      Map<String, FieldReader> fieldMap = null;
+      Throwable priorE = null;
+      try (ChecksumIndexInput metaIn = version >= VERSION_META_FILE ? state.directory.openChecksumInput(metaName, state.context) : null) {
+        try {
+          final IndexInput indexMetaIn, termsMetaIn;
+          if (version >= VERSION_META_FILE) {
+            CodecUtil.checkIndexHeader(metaIn, TERMS_META_CODEC_NAME, version, version, state.segmentInfo.getId(), state.segmentSuffix);
+            indexMetaIn = termsMetaIn = metaIn;
+          } else {
+            seekDir(termsIn);
+            seekDir(indexIn);
+            indexMetaIn = indexIn;
+            termsMetaIn = termsIn;
+          }
 
-      final int numFields = termsIn.readVInt();
-      if (numFields < 0) {
-        throw new CorruptIndexException("invalid numFields: " + numFields, termsIn);
-      }
-      fieldMap = new HashMap<>((int) (numFields / 0.75f) + 1);
-      for (int i = 0; i < numFields; ++i) {
-        final int field = termsIn.readVInt();
-        final long numTerms = termsIn.readVLong();
-        if (numTerms <= 0) {
-          throw new CorruptIndexException("Illegal numTerms for field number: " + field, termsIn);
-        }
-        final BytesRef rootCode = readBytesRef(termsIn);
-        final FieldInfo fieldInfo = state.fieldInfos.fieldInfo(field);
-        if (fieldInfo == null) {
-          throw new CorruptIndexException("invalid field number: " + field, termsIn);
-        }
-        final long sumTotalTermFreq = termsIn.readVLong();
-        // when frequencies are omitted, sumDocFreq=sumTotalTermFreq and only one value is written.
-        final long sumDocFreq = fieldInfo.getIndexOptions() == IndexOptions.DOCS ? sumTotalTermFreq : termsIn.readVLong();
-        final int docCount = termsIn.readVInt();
-        if (version < VERSION_META_LONGS_REMOVED) {
-          final int longsSize = termsIn.readVInt();
-          if (longsSize < 0) {
-            throw new CorruptIndexException("invalid longsSize for field: " + fieldInfo.name + ", longsSize=" + longsSize, termsIn);
+          final int numFields = termsMetaIn.readVInt();
+          if (numFields < 0) {
+            throw new CorruptIndexException("invalid numFields: " + numFields, termsMetaIn);
+          }
+          fieldMap = new HashMap<>((int) (numFields / 0.75f) + 1);
+          for (int i = 0; i < numFields; ++i) {
+            final int field = termsMetaIn.readVInt();
+            final long numTerms = termsMetaIn.readVLong();
+            if (numTerms <= 0) {
+              throw new CorruptIndexException("Illegal numTerms for field number: " + field, termsMetaIn);
+            }
+            final BytesRef rootCode = readBytesRef(termsMetaIn);
+            final FieldInfo fieldInfo = state.fieldInfos.fieldInfo(field);
+            if (fieldInfo == null) {
+              throw new CorruptIndexException("invalid field number: " + field, termsMetaIn);
+            }
+            final long sumTotalTermFreq = termsMetaIn.readVLong();
+            // when frequencies are omitted, sumDocFreq=sumTotalTermFreq and only one value is written.
+            final long sumDocFreq = fieldInfo.getIndexOptions() == IndexOptions.DOCS ? sumTotalTermFreq : termsMetaIn.readVLong();
+            final int docCount = termsMetaIn.readVInt();
+            if (version < VERSION_META_LONGS_REMOVED) {
+              final int longsSize = termsMetaIn.readVInt();
+              if (longsSize < 0) {
+                throw new CorruptIndexException("invalid longsSize for field: " + fieldInfo.name + ", longsSize=" + longsSize, termsMetaIn);
+              }
+            }
+            BytesRef minTerm = readBytesRef(termsMetaIn);
+            BytesRef maxTerm = readBytesRef(termsMetaIn);
+            if (docCount < 0 || docCount > state.segmentInfo.maxDoc()) { // #docs with field must be <= #docs
+              throw new CorruptIndexException("invalid docCount: " + docCount + " maxDoc: " + state.segmentInfo.maxDoc(), termsMetaIn);

Review comment:
       Not directly, and these things are hard to test, though I agree we could do better. I opened https://issues.apache.org/jira/browse/LUCENE-9356 to try to improve the coverage of these code paths.




----------------------------------------------------------------
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] jpountz commented on a change in pull request #1473: LUCENE-9353: Move terms metadata to its own file.

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



##########
File path: lucene/core/src/java/org/apache/lucene/codecs/blocktree/BlockTreeTermsWriter.java
##########
@@ -1060,36 +1052,35 @@ public void close() throws IOException {
       return;
     }
     closed = true;
-    
+
+    final String metaName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, BlockTreeTermsReader.TERMS_META_EXTENSION);
     boolean success = false;
-    try {
-      
-      final long dirStart = termsOut.getFilePointer();
-      final long indexDirStart = indexOut.getFilePointer();
+    try (IndexOutput metaOut = state.directory.createOutput(metaName, state.context)) {
+      CodecUtil.writeIndexHeader(metaOut, BlockTreeTermsReader.TERMS_META_CODEC_NAME, BlockTreeTermsReader.VERSION_CURRENT,
+          state.segmentInfo.getId(), state.segmentSuffix);
 
-      termsOut.writeVInt(fields.size());
+      metaOut.writeVInt(fields.size());

Review comment:
       I prefer decoupling changes when possible.




----------------------------------------------------------------
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] juanka588 commented on a change in pull request #1473: LUCENE-9353: Move terms metadata to its own file.

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



##########
File path: lucene/core/src/java/org/apache/lucene/codecs/blocktree/BlockTreeTermsWriter.java
##########
@@ -1060,36 +1052,35 @@ public void close() throws IOException {
       return;
     }
     closed = true;
-    
+
+    final String metaName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, BlockTreeTermsReader.TERMS_META_EXTENSION);
     boolean success = false;
-    try {
-      
-      final long dirStart = termsOut.getFilePointer();
-      final long indexDirStart = indexOut.getFilePointer();
+    try (IndexOutput metaOut = state.directory.createOutput(metaName, state.context)) {

Review comment:
       why this file is not created at the same time with the indexOut, termOut?




----------------------------------------------------------------
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] jpountz commented on a change in pull request #1473: LUCENE-9353: Move terms metadata to its own file.

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



##########
File path: lucene/core/src/java/org/apache/lucene/codecs/blocktree/BlockTreeTermsWriter.java
##########
@@ -1060,36 +1052,35 @@ public void close() throws IOException {
       return;
     }
     closed = true;
-    
+
+    final String metaName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, BlockTreeTermsReader.TERMS_META_EXTENSION);
     boolean success = false;
-    try {
-      
-      final long dirStart = termsOut.getFilePointer();
-      final long indexDirStart = indexOut.getFilePointer();
+    try (IndexOutput metaOut = state.directory.createOutput(metaName, state.context)) {
+      CodecUtil.writeIndexHeader(metaOut, BlockTreeTermsReader.TERMS_META_CODEC_NAME, BlockTreeTermsReader.VERSION_CURRENT,
+          state.segmentInfo.getId(), state.segmentSuffix);
 
-      termsOut.writeVInt(fields.size());
+      metaOut.writeVInt(fields.size());

Review comment:
       Your proposal sounds orthogonal to this pull request to me?




----------------------------------------------------------------
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] juanka588 commented on a change in pull request #1473: LUCENE-9353: Move terms metadata to its own file.

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



##########
File path: lucene/core/src/java/org/apache/lucene/codecs/blocktree/BlockTreeTermsWriter.java
##########
@@ -1060,36 +1052,35 @@ public void close() throws IOException {
       return;
     }
     closed = true;
-    
+
+    final String metaName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, BlockTreeTermsReader.TERMS_META_EXTENSION);
     boolean success = false;
-    try {
-      
-      final long dirStart = termsOut.getFilePointer();
-      final long indexDirStart = indexOut.getFilePointer();
+    try (IndexOutput metaOut = state.directory.createOutput(metaName, state.context)) {
+      CodecUtil.writeIndexHeader(metaOut, BlockTreeTermsReader.TERMS_META_CODEC_NAME, BlockTreeTermsReader.VERSION_CURRENT,
+          state.segmentInfo.getId(), state.segmentSuffix);
 
-      termsOut.writeVInt(fields.size());
+      metaOut.writeVInt(fields.size());

Review comment:
       yes but maybe is the opportunity to move the code and add more testability in BlockTreeTermsReader.java 




----------------------------------------------------------------
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] msokolov commented on a change in pull request #1473: LUCENE-9353: Move terms metadata to its own file.

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



##########
File path: lucene/core/src/java/org/apache/lucene/codecs/blocktree/BlockTreeTermsReader.java
##########
@@ -148,56 +155,80 @@ public BlockTreeTermsReader(PostingsReaderBase postingsReader, SegmentReadState
       CodecUtil.retrieveChecksum(termsIn);
 
       // Read per-field details
-      seekDir(termsIn);
-      seekDir(indexIn);
+      String metaName = IndexFileNames.segmentFileName(segment, state.segmentSuffix, TERMS_META_EXTENSION);
+      Map<String, FieldReader> fieldMap = null;
+      Throwable priorE = null;
+      try (ChecksumIndexInput metaIn = version >= VERSION_META_FILE ? state.directory.openChecksumInput(metaName, state.context) : null) {
+        try {
+          final IndexInput indexMetaIn, termsMetaIn;
+          if (version >= VERSION_META_FILE) {
+            CodecUtil.checkIndexHeader(metaIn, TERMS_META_CODEC_NAME, version, version, state.segmentInfo.getId(), state.segmentSuffix);
+            indexMetaIn = termsMetaIn = metaIn;
+          } else {
+            seekDir(termsIn);
+            seekDir(indexIn);
+            indexMetaIn = indexIn;
+            termsMetaIn = termsIn;
+          }
 
-      final int numFields = termsIn.readVInt();
-      if (numFields < 0) {
-        throw new CorruptIndexException("invalid numFields: " + numFields, termsIn);
-      }
-      fieldMap = new HashMap<>((int) (numFields / 0.75f) + 1);
-      for (int i = 0; i < numFields; ++i) {
-        final int field = termsIn.readVInt();
-        final long numTerms = termsIn.readVLong();
-        if (numTerms <= 0) {
-          throw new CorruptIndexException("Illegal numTerms for field number: " + field, termsIn);
-        }
-        final BytesRef rootCode = readBytesRef(termsIn);
-        final FieldInfo fieldInfo = state.fieldInfos.fieldInfo(field);
-        if (fieldInfo == null) {
-          throw new CorruptIndexException("invalid field number: " + field, termsIn);
-        }
-        final long sumTotalTermFreq = termsIn.readVLong();
-        // when frequencies are omitted, sumDocFreq=sumTotalTermFreq and only one value is written.
-        final long sumDocFreq = fieldInfo.getIndexOptions() == IndexOptions.DOCS ? sumTotalTermFreq : termsIn.readVLong();
-        final int docCount = termsIn.readVInt();
-        if (version < VERSION_META_LONGS_REMOVED) {
-          final int longsSize = termsIn.readVInt();
-          if (longsSize < 0) {
-            throw new CorruptIndexException("invalid longsSize for field: " + fieldInfo.name + ", longsSize=" + longsSize, termsIn);
+          final int numFields = termsMetaIn.readVInt();
+          if (numFields < 0) {
+            throw new CorruptIndexException("invalid numFields: " + numFields, termsMetaIn);
+          }
+          fieldMap = new HashMap<>((int) (numFields / 0.75f) + 1);
+          for (int i = 0; i < numFields; ++i) {
+            final int field = termsMetaIn.readVInt();
+            final long numTerms = termsMetaIn.readVLong();
+            if (numTerms <= 0) {
+              throw new CorruptIndexException("Illegal numTerms for field number: " + field, termsMetaIn);
+            }
+            final BytesRef rootCode = readBytesRef(termsMetaIn);
+            final FieldInfo fieldInfo = state.fieldInfos.fieldInfo(field);
+            if (fieldInfo == null) {
+              throw new CorruptIndexException("invalid field number: " + field, termsMetaIn);
+            }
+            final long sumTotalTermFreq = termsMetaIn.readVLong();
+            // when frequencies are omitted, sumDocFreq=sumTotalTermFreq and only one value is written.
+            final long sumDocFreq = fieldInfo.getIndexOptions() == IndexOptions.DOCS ? sumTotalTermFreq : termsMetaIn.readVLong();
+            final int docCount = termsMetaIn.readVInt();
+            if (version < VERSION_META_LONGS_REMOVED) {
+              final int longsSize = termsMetaIn.readVInt();
+              if (longsSize < 0) {
+                throw new CorruptIndexException("invalid longsSize for field: " + fieldInfo.name + ", longsSize=" + longsSize, termsMetaIn);
+              }
+            }
+            BytesRef minTerm = readBytesRef(termsMetaIn);
+            BytesRef maxTerm = readBytesRef(termsMetaIn);
+            if (docCount < 0 || docCount > state.segmentInfo.maxDoc()) { // #docs with field must be <= #docs
+              throw new CorruptIndexException("invalid docCount: " + docCount + " maxDoc: " + state.segmentInfo.maxDoc(), termsMetaIn);
+            }
+            if (sumDocFreq < docCount) {  // #postings must be >= #docs with field
+              throw new CorruptIndexException("invalid sumDocFreq: " + sumDocFreq + " docCount: " + docCount, termsMetaIn);
+            }
+            if (sumTotalTermFreq < sumDocFreq) { // #positions must be >= #postings
+              throw new CorruptIndexException("invalid sumTotalTermFreq: " + sumTotalTermFreq + " sumDocFreq: " + sumDocFreq, termsMetaIn);
+            }
+            final long indexStartFP = indexMetaIn.readVLong();
+            FieldReader previous = fieldMap.put(fieldInfo.name,
+                new FieldReader(this, fieldInfo, numTerms, rootCode, sumTotalTermFreq, sumDocFreq, docCount,
+                    indexStartFP, indexIn, minTerm, maxTerm));
+            if (previous != null) {
+              throw new CorruptIndexException("duplicate field: " + fieldInfo.name, termsMetaIn);
+            }
+          }
+        } catch (Throwable exception) {
+          priorE = exception;
+        } finally {
+          if (metaIn != null) {
+            CodecUtil.checkFooter(metaIn, priorE);

Review comment:
       again, I don't know if we have test coverage for the corrputed metadata?




----------------------------------------------------------------
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] msokolov commented on a change in pull request #1473: LUCENE-9353: Move terms metadata to its own file.

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



##########
File path: lucene/core/src/java/org/apache/lucene/codecs/blocktree/BlockTreeTermsReader.java
##########
@@ -148,56 +155,80 @@ public BlockTreeTermsReader(PostingsReaderBase postingsReader, SegmentReadState
       CodecUtil.retrieveChecksum(termsIn);
 
       // Read per-field details
-      seekDir(termsIn);
-      seekDir(indexIn);
+      String metaName = IndexFileNames.segmentFileName(segment, state.segmentSuffix, TERMS_META_EXTENSION);
+      Map<String, FieldReader> fieldMap = null;
+      Throwable priorE = null;
+      try (ChecksumIndexInput metaIn = version >= VERSION_META_FILE ? state.directory.openChecksumInput(metaName, state.context) : null) {
+        try {
+          final IndexInput indexMetaIn, termsMetaIn;
+          if (version >= VERSION_META_FILE) {
+            CodecUtil.checkIndexHeader(metaIn, TERMS_META_CODEC_NAME, version, version, state.segmentInfo.getId(), state.segmentSuffix);
+            indexMetaIn = termsMetaIn = metaIn;
+          } else {
+            seekDir(termsIn);
+            seekDir(indexIn);
+            indexMetaIn = indexIn;
+            termsMetaIn = termsIn;
+          }
 
-      final int numFields = termsIn.readVInt();
-      if (numFields < 0) {
-        throw new CorruptIndexException("invalid numFields: " + numFields, termsIn);
-      }
-      fieldMap = new HashMap<>((int) (numFields / 0.75f) + 1);
-      for (int i = 0; i < numFields; ++i) {
-        final int field = termsIn.readVInt();
-        final long numTerms = termsIn.readVLong();
-        if (numTerms <= 0) {
-          throw new CorruptIndexException("Illegal numTerms for field number: " + field, termsIn);
-        }
-        final BytesRef rootCode = readBytesRef(termsIn);
-        final FieldInfo fieldInfo = state.fieldInfos.fieldInfo(field);
-        if (fieldInfo == null) {
-          throw new CorruptIndexException("invalid field number: " + field, termsIn);
-        }
-        final long sumTotalTermFreq = termsIn.readVLong();
-        // when frequencies are omitted, sumDocFreq=sumTotalTermFreq and only one value is written.
-        final long sumDocFreq = fieldInfo.getIndexOptions() == IndexOptions.DOCS ? sumTotalTermFreq : termsIn.readVLong();
-        final int docCount = termsIn.readVInt();
-        if (version < VERSION_META_LONGS_REMOVED) {
-          final int longsSize = termsIn.readVInt();
-          if (longsSize < 0) {
-            throw new CorruptIndexException("invalid longsSize for field: " + fieldInfo.name + ", longsSize=" + longsSize, termsIn);
+          final int numFields = termsMetaIn.readVInt();
+          if (numFields < 0) {
+            throw new CorruptIndexException("invalid numFields: " + numFields, termsMetaIn);
+          }
+          fieldMap = new HashMap<>((int) (numFields / 0.75f) + 1);
+          for (int i = 0; i < numFields; ++i) {
+            final int field = termsMetaIn.readVInt();
+            final long numTerms = termsMetaIn.readVLong();
+            if (numTerms <= 0) {
+              throw new CorruptIndexException("Illegal numTerms for field number: " + field, termsMetaIn);
+            }
+            final BytesRef rootCode = readBytesRef(termsMetaIn);
+            final FieldInfo fieldInfo = state.fieldInfos.fieldInfo(field);
+            if (fieldInfo == null) {
+              throw new CorruptIndexException("invalid field number: " + field, termsMetaIn);
+            }
+            final long sumTotalTermFreq = termsMetaIn.readVLong();
+            // when frequencies are omitted, sumDocFreq=sumTotalTermFreq and only one value is written.
+            final long sumDocFreq = fieldInfo.getIndexOptions() == IndexOptions.DOCS ? sumTotalTermFreq : termsMetaIn.readVLong();
+            final int docCount = termsMetaIn.readVInt();
+            if (version < VERSION_META_LONGS_REMOVED) {
+              final int longsSize = termsMetaIn.readVInt();
+              if (longsSize < 0) {
+                throw new CorruptIndexException("invalid longsSize for field: " + fieldInfo.name + ", longsSize=" + longsSize, termsMetaIn);
+              }
+            }
+            BytesRef minTerm = readBytesRef(termsMetaIn);
+            BytesRef maxTerm = readBytesRef(termsMetaIn);
+            if (docCount < 0 || docCount > state.segmentInfo.maxDoc()) { // #docs with field must be <= #docs
+              throw new CorruptIndexException("invalid docCount: " + docCount + " maxDoc: " + state.segmentInfo.maxDoc(), termsMetaIn);

Review comment:
       do we have a test that tickles these cases?




----------------------------------------------------------------
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] msokolov commented on a change in pull request #1473: LUCENE-9353: Move terms metadata to its own file.

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



##########
File path: lucene/core/src/java/org/apache/lucene/codecs/blocktree/BlockTreeTermsReader.java
##########
@@ -148,56 +155,80 @@ public BlockTreeTermsReader(PostingsReaderBase postingsReader, SegmentReadState
       CodecUtil.retrieveChecksum(termsIn);
 
       // Read per-field details
-      seekDir(termsIn);
-      seekDir(indexIn);
+      String metaName = IndexFileNames.segmentFileName(segment, state.segmentSuffix, TERMS_META_EXTENSION);
+      Map<String, FieldReader> fieldMap = null;
+      Throwable priorE = null;
+      try (ChecksumIndexInput metaIn = version >= VERSION_META_FILE ? state.directory.openChecksumInput(metaName, state.context) : null) {
+        try {
+          final IndexInput indexMetaIn, termsMetaIn;
+          if (version >= VERSION_META_FILE) {
+            CodecUtil.checkIndexHeader(metaIn, TERMS_META_CODEC_NAME, version, version, state.segmentInfo.getId(), state.segmentSuffix);
+            indexMetaIn = termsMetaIn = metaIn;
+          } else {
+            seekDir(termsIn);
+            seekDir(indexIn);
+            indexMetaIn = indexIn;
+            termsMetaIn = termsIn;
+          }
 
-      final int numFields = termsIn.readVInt();
-      if (numFields < 0) {
-        throw new CorruptIndexException("invalid numFields: " + numFields, termsIn);
-      }
-      fieldMap = new HashMap<>((int) (numFields / 0.75f) + 1);
-      for (int i = 0; i < numFields; ++i) {
-        final int field = termsIn.readVInt();
-        final long numTerms = termsIn.readVLong();
-        if (numTerms <= 0) {
-          throw new CorruptIndexException("Illegal numTerms for field number: " + field, termsIn);
-        }
-        final BytesRef rootCode = readBytesRef(termsIn);
-        final FieldInfo fieldInfo = state.fieldInfos.fieldInfo(field);
-        if (fieldInfo == null) {
-          throw new CorruptIndexException("invalid field number: " + field, termsIn);
-        }
-        final long sumTotalTermFreq = termsIn.readVLong();
-        // when frequencies are omitted, sumDocFreq=sumTotalTermFreq and only one value is written.
-        final long sumDocFreq = fieldInfo.getIndexOptions() == IndexOptions.DOCS ? sumTotalTermFreq : termsIn.readVLong();
-        final int docCount = termsIn.readVInt();
-        if (version < VERSION_META_LONGS_REMOVED) {
-          final int longsSize = termsIn.readVInt();
-          if (longsSize < 0) {
-            throw new CorruptIndexException("invalid longsSize for field: " + fieldInfo.name + ", longsSize=" + longsSize, termsIn);
+          final int numFields = termsMetaIn.readVInt();
+          if (numFields < 0) {
+            throw new CorruptIndexException("invalid numFields: " + numFields, termsMetaIn);
+          }
+          fieldMap = new HashMap<>((int) (numFields / 0.75f) + 1);
+          for (int i = 0; i < numFields; ++i) {
+            final int field = termsMetaIn.readVInt();
+            final long numTerms = termsMetaIn.readVLong();
+            if (numTerms <= 0) {
+              throw new CorruptIndexException("Illegal numTerms for field number: " + field, termsMetaIn);
+            }
+            final BytesRef rootCode = readBytesRef(termsMetaIn);
+            final FieldInfo fieldInfo = state.fieldInfos.fieldInfo(field);
+            if (fieldInfo == null) {
+              throw new CorruptIndexException("invalid field number: " + field, termsMetaIn);
+            }
+            final long sumTotalTermFreq = termsMetaIn.readVLong();
+            // when frequencies are omitted, sumDocFreq=sumTotalTermFreq and only one value is written.
+            final long sumDocFreq = fieldInfo.getIndexOptions() == IndexOptions.DOCS ? sumTotalTermFreq : termsMetaIn.readVLong();
+            final int docCount = termsMetaIn.readVInt();
+            if (version < VERSION_META_LONGS_REMOVED) {
+              final int longsSize = termsMetaIn.readVInt();
+              if (longsSize < 0) {
+                throw new CorruptIndexException("invalid longsSize for field: " + fieldInfo.name + ", longsSize=" + longsSize, termsMetaIn);
+              }
+            }
+            BytesRef minTerm = readBytesRef(termsMetaIn);
+            BytesRef maxTerm = readBytesRef(termsMetaIn);
+            if (docCount < 0 || docCount > state.segmentInfo.maxDoc()) { // #docs with field must be <= #docs
+              throw new CorruptIndexException("invalid docCount: " + docCount + " maxDoc: " + state.segmentInfo.maxDoc(), termsMetaIn);

Review comment:
       Thanks, Adrien




----------------------------------------------------------------
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] juanka588 commented on a change in pull request #1473: LUCENE-9353: Move terms metadata to its own file.

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



##########
File path: lucene/core/src/java/org/apache/lucene/codecs/blocktree/BlockTreeTermsWriter.java
##########
@@ -1060,36 +1052,35 @@ public void close() throws IOException {
       return;
     }
     closed = true;
-    
+
+    final String metaName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, BlockTreeTermsReader.TERMS_META_EXTENSION);
     boolean success = false;
-    try {
-      
-      final long dirStart = termsOut.getFilePointer();
-      final long indexDirStart = indexOut.getFilePointer();
+    try (IndexOutput metaOut = state.directory.createOutput(metaName, state.context)) {
+      CodecUtil.writeIndexHeader(metaOut, BlockTreeTermsReader.TERMS_META_CODEC_NAME, BlockTreeTermsReader.VERSION_CURRENT,
+          state.segmentInfo.getId(), state.segmentSuffix);
 
-      termsOut.writeVInt(fields.size());
+      metaOut.writeVInt(fields.size());

Review comment:
       @jpountz here I see the same lack of serializer write/read code, could it be possible to have such thing, It would improve readability and unit testing by only mocking fieldMetadatas and check serialization is correctly applied.




----------------------------------------------------------------
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] jpountz merged pull request #1473: LUCENE-9353: Move terms metadata to its own file.

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


   


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