You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by so...@apache.org on 2020/06/03 19:14:08 UTC

[lucene-solr] 39/47: LUCENE-9359: Always call checkFooter in SegmentInfos#readCommit. (#1483)

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

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

commit 2a224cb08cca9b2ea8086741552c81adc9ceb7b2
Author: Adrien Grand <jp...@gmail.com>
AuthorDate: Fri May 29 14:59:36 2020 +0200

    LUCENE-9359: Always call checkFooter in SegmentInfos#readCommit. (#1483)
---
 lucene/CHANGES.txt                                 |   3 +
 .../java/org/apache/lucene/index/SegmentInfos.java | 236 +++++++++++----------
 .../org/apache/lucene/index/TestSegmentInfos.java  |  61 ++++++
 3 files changed, 187 insertions(+), 113 deletions(-)

diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index cd42f6e..50b7f7b 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -195,6 +195,9 @@ Improvements
 * LUCENE-9342: TotalHits' relation will be EQUAL_TO when the number of hits is lower than TopDocsColector's numHits
   (Tomás Fernández Löbbe)
 
+* LUCENE-9359: SegmentInfos#readCommit now always returns a
+  CorruptIndexException if the content of the file is invalid. (Adrien Grand)
+
 Optimizations
 ---------------------
 
diff --git a/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java b/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java
index f9edccd..dc379ab 100644
--- a/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java
+++ b/lucene/core/src/java/org/apache/lucene/index/SegmentInfos.java
@@ -304,136 +304,146 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentCommitInfo
 
   /** Read the commit from the provided {@link ChecksumIndexInput}. */
   public static final SegmentInfos readCommit(Directory directory, ChecksumIndexInput input, long generation) throws IOException {
+    Throwable priorE = null;
+    int format = -1;
+    try {
+      // NOTE: as long as we want to throw indexformattooold (vs corruptindexexception), we need
+      // to read the magic ourselves.
+      int magic = input.readInt();
+      if (magic != CodecUtil.CODEC_MAGIC) {
+        throw new IndexFormatTooOldException(input, magic, CodecUtil.CODEC_MAGIC, CodecUtil.CODEC_MAGIC);
+      }
+      format = CodecUtil.checkHeaderNoMagic(input, "segments", VERSION_70, VERSION_CURRENT);
+      byte id[] = new byte[StringHelper.ID_LENGTH];
+      input.readBytes(id, 0, id.length);
+      CodecUtil.checkIndexHeaderSuffix(input, Long.toString(generation, Character.MAX_RADIX));
+
+      Version luceneVersion = Version.fromBits(input.readVInt(), input.readVInt(), input.readVInt());
+      int indexCreatedVersion = input.readVInt();
+      if (luceneVersion.major < indexCreatedVersion) {
+        throw new CorruptIndexException("Creation version [" + indexCreatedVersion
+            + ".x] can't be greater than the version that wrote the segment infos: [" + luceneVersion + "]" , input);
+      }
 
-    // NOTE: as long as we want to throw indexformattooold (vs corruptindexexception), we need
-    // to read the magic ourselves.
-    int magic = input.readInt();
-    if (magic != CodecUtil.CODEC_MAGIC) {
-      throw new IndexFormatTooOldException(input, magic, CodecUtil.CODEC_MAGIC, CodecUtil.CODEC_MAGIC);
-    }
-    int format = CodecUtil.checkHeaderNoMagic(input, "segments", VERSION_70, VERSION_CURRENT);
-    byte id[] = new byte[StringHelper.ID_LENGTH];
-    input.readBytes(id, 0, id.length);
-    CodecUtil.checkIndexHeaderSuffix(input, Long.toString(generation, Character.MAX_RADIX));
-
-    Version luceneVersion = Version.fromBits(input.readVInt(), input.readVInt(), input.readVInt());
-    int indexCreatedVersion = input.readVInt();
-    if (luceneVersion.major < indexCreatedVersion) {
-      throw new CorruptIndexException("Creation version [" + indexCreatedVersion
-          + ".x] can't be greater than the version that wrote the segment infos: [" + luceneVersion + "]" , input);
-    }
-
-    if (indexCreatedVersion < Version.LATEST.major - 1) {
-      throw new IndexFormatTooOldException(input, "This index was initially created with Lucene "
-          + indexCreatedVersion + ".x while the current version is " + Version.LATEST
-          + " and Lucene only supports reading the current and previous major versions.");
-    }
-
-    SegmentInfos infos = new SegmentInfos(indexCreatedVersion);
-    infos.id = id;
-    infos.generation = generation;
-    infos.lastGeneration = generation;
-    infos.luceneVersion = luceneVersion;
-
-    infos.version = input.readLong();
-    //System.out.println("READ sis version=" + infos.version);
-    if (format > VERSION_70) {
-      infos.counter = input.readVLong();
-    } else {
-      infos.counter = input.readInt();
-    }
-    int numSegments = input.readInt();
-    if (numSegments < 0) {
-      throw new CorruptIndexException("invalid segment count: " + numSegments, input);
-    }
+      if (indexCreatedVersion < Version.LATEST.major - 1) {
+        throw new IndexFormatTooOldException(input, "This index was initially created with Lucene "
+            + indexCreatedVersion + ".x while the current version is " + Version.LATEST
+            + " and Lucene only supports reading the current and previous major versions.");
+      }
 
-    if (numSegments > 0) {
-      infos.minSegmentLuceneVersion = Version.fromBits(input.readVInt(), input.readVInt(), input.readVInt());
-    } else {
-      // else leave as null: no segments
-    }
+      SegmentInfos infos = new SegmentInfos(indexCreatedVersion);
+      infos.id = id;
+      infos.generation = generation;
+      infos.lastGeneration = generation;
+      infos.luceneVersion = luceneVersion;
 
-    long totalDocs = 0;
-    for (int seg = 0; seg < numSegments; seg++) {
-      String segName = input.readString();
-      byte[] segmentID = new byte[StringHelper.ID_LENGTH];
-      input.readBytes(segmentID, 0, segmentID.length);
-      Codec codec = readCodec(input);
-      SegmentInfo info = codec.segmentInfoFormat().read(directory, segName, segmentID, IOContext.READ);
-      info.setCodec(codec);
-      totalDocs += info.maxDoc();
-      long delGen = input.readLong();
-      int delCount = input.readInt();
-      if (delCount < 0 || delCount > info.maxDoc()) {
-        throw new CorruptIndexException("invalid deletion count: " + delCount + " vs maxDoc=" + info.maxDoc(), input);
-      }
-      long fieldInfosGen = input.readLong();
-      long dvGen = input.readLong();
-      int softDelCount = format > VERSION_72 ? input.readInt() : 0;
-      if (softDelCount < 0 || softDelCount > info.maxDoc()) {
-        throw new CorruptIndexException("invalid deletion count: " + softDelCount + " vs maxDoc=" + info.maxDoc(), input);
+      infos.version = input.readLong();
+      //System.out.println("READ sis version=" + infos.version);
+      if (format > VERSION_70) {
+        infos.counter = input.readVLong();
+      } else {
+        infos.counter = input.readInt();
       }
-      if (softDelCount + delCount > info.maxDoc()) {
-        throw new CorruptIndexException("invalid deletion count: " + softDelCount + delCount + " vs maxDoc=" + info.maxDoc(), input);
+      int numSegments = input.readInt();
+      if (numSegments < 0) {
+        throw new CorruptIndexException("invalid segment count: " + numSegments, input);
       }
-      final byte[] sciId;
-      if (format > VERSION_74) {
-        byte marker = input.readByte();
-        switch (marker) {
-          case 1:
-            sciId = new byte[StringHelper.ID_LENGTH];
-            input.readBytes(sciId, 0, sciId.length);
-            break;
-          case 0:
-            sciId = null;
-            break;
-          default:
-            throw new CorruptIndexException("invalid SegmentCommitInfo ID marker: " + marker, input);
-        }
+
+      if (numSegments > 0) {
+        infos.minSegmentLuceneVersion = Version.fromBits(input.readVInt(), input.readVInt(), input.readVInt());
       } else {
-        sciId = null;
+        // else leave as null: no segments
       }
-      SegmentCommitInfo siPerCommit = new SegmentCommitInfo(info, delCount, softDelCount, delGen, fieldInfosGen, dvGen, sciId);
-      siPerCommit.setFieldInfosFiles(input.readSetOfStrings());
-      final Map<Integer,Set<String>> dvUpdateFiles;
-      final int numDVFields = input.readInt();
-      if (numDVFields == 0) {
-        dvUpdateFiles = Collections.emptyMap();
-      } else {
-        Map<Integer,Set<String>> map = new HashMap<>(numDVFields);
-        for (int i = 0; i < numDVFields; i++) {
-          map.put(input.readInt(), input.readSetOfStrings());
+
+      long totalDocs = 0;
+      for (int seg = 0; seg < numSegments; seg++) {
+        String segName = input.readString();
+        byte[] segmentID = new byte[StringHelper.ID_LENGTH];
+        input.readBytes(segmentID, 0, segmentID.length);
+        Codec codec = readCodec(input);
+        SegmentInfo info = codec.segmentInfoFormat().read(directory, segName, segmentID, IOContext.READ);
+        info.setCodec(codec);
+        totalDocs += info.maxDoc();
+        long delGen = input.readLong();
+        int delCount = input.readInt();
+        if (delCount < 0 || delCount > info.maxDoc()) {
+          throw new CorruptIndexException("invalid deletion count: " + delCount + " vs maxDoc=" + info.maxDoc(), input);
         }
-        dvUpdateFiles = Collections.unmodifiableMap(map);
-      }
-      siPerCommit.setDocValuesUpdatesFiles(dvUpdateFiles);
-      infos.add(siPerCommit);
+        long fieldInfosGen = input.readLong();
+        long dvGen = input.readLong();
+        int softDelCount = format > VERSION_72 ? input.readInt() : 0;
+        if (softDelCount < 0 || softDelCount > info.maxDoc()) {
+          throw new CorruptIndexException("invalid deletion count: " + softDelCount + " vs maxDoc=" + info.maxDoc(), input);
+        }
+        if (softDelCount + delCount > info.maxDoc()) {
+          throw new CorruptIndexException("invalid deletion count: " + softDelCount + delCount + " vs maxDoc=" + info.maxDoc(), input);
+        }
+        final byte[] sciId;
+        if (format > VERSION_74) {
+          byte marker = input.readByte();
+          switch (marker) {
+            case 1:
+              sciId = new byte[StringHelper.ID_LENGTH];
+              input.readBytes(sciId, 0, sciId.length);
+              break;
+            case 0:
+              sciId = null;
+              break;
+            default:
+              throw new CorruptIndexException("invalid SegmentCommitInfo ID marker: " + marker, input);
+          }
+        } else {
+          sciId = null;
+        }
+        SegmentCommitInfo siPerCommit = new SegmentCommitInfo(info, delCount, softDelCount, delGen, fieldInfosGen, dvGen, sciId);
+        siPerCommit.setFieldInfosFiles(input.readSetOfStrings());
+        final Map<Integer,Set<String>> dvUpdateFiles;
+        final int numDVFields = input.readInt();
+        if (numDVFields == 0) {
+          dvUpdateFiles = Collections.emptyMap();
+        } else {
+          Map<Integer,Set<String>> map = new HashMap<>(numDVFields);
+          for (int i = 0; i < numDVFields; i++) {
+            map.put(input.readInt(), input.readSetOfStrings());
+          }
+          dvUpdateFiles = Collections.unmodifiableMap(map);
+        }
+        siPerCommit.setDocValuesUpdatesFiles(dvUpdateFiles);
+        infos.add(siPerCommit);
 
-      Version segmentVersion = info.getVersion();
+        Version segmentVersion = info.getVersion();
 
-      if (segmentVersion.onOrAfter(infos.minSegmentLuceneVersion) == false) {
-        throw new CorruptIndexException("segments file recorded minSegmentLuceneVersion=" + infos.minSegmentLuceneVersion + " but segment=" + info + " has older version=" + segmentVersion, input);
-      }
+        if (segmentVersion.onOrAfter(infos.minSegmentLuceneVersion) == false) {
+          throw new CorruptIndexException("segments file recorded minSegmentLuceneVersion=" + infos.minSegmentLuceneVersion + " but segment=" + info + " has older version=" + segmentVersion, input);
+        }
 
-      if (infos.indexCreatedVersionMajor >= 7 && segmentVersion.major < infos.indexCreatedVersionMajor) {
-        throw new CorruptIndexException("segments file recorded indexCreatedVersionMajor=" + infos.indexCreatedVersionMajor + " but segment=" + info + " has older version=" + segmentVersion, input);
-      }
+        if (infos.indexCreatedVersionMajor >= 7 && segmentVersion.major < infos.indexCreatedVersionMajor) {
+          throw new CorruptIndexException("segments file recorded indexCreatedVersionMajor=" + infos.indexCreatedVersionMajor + " but segment=" + info + " has older version=" + segmentVersion, input);
+        }
 
-      if (infos.indexCreatedVersionMajor >= 7 && info.getMinVersion() == null) {
-        throw new CorruptIndexException("segments infos must record minVersion with indexCreatedVersionMajor=" + infos.indexCreatedVersionMajor, input);
+        if (infos.indexCreatedVersionMajor >= 7 && info.getMinVersion() == null) {
+          throw new CorruptIndexException("segments infos must record minVersion with indexCreatedVersionMajor=" + infos.indexCreatedVersionMajor, input);
+        }
       }
-    }
 
-    infos.userData = input.readMapOfStrings();
+      infos.userData = input.readMapOfStrings();
 
-    CodecUtil.checkFooter(input);
+      // LUCENE-6299: check we are in bounds
+      if (totalDocs > IndexWriter.getActualMaxDocs()) {
+        throw new CorruptIndexException("Too many documents: an index cannot exceed " + IndexWriter.getActualMaxDocs() + " but readers have total maxDoc=" + totalDocs, input);
+      }
 
-    // LUCENE-6299: check we are in bounds
-    if (totalDocs > IndexWriter.getActualMaxDocs()) {
-      throw new CorruptIndexException("Too many documents: an index cannot exceed " + IndexWriter.getActualMaxDocs() + " but readers have total maxDoc=" + totalDocs, input);
+      return infos;
+    } catch (Throwable t) {
+      priorE = t;
+    } finally {
+      if (format >= VERSION_70) { // oldest supported version
+        CodecUtil.checkFooter(input, priorE);
+      } else {
+        throw IOUtils.rethrowAlways(priorE);
+      }
     }
-
-    return infos;
+    throw new Error("Unreachable code");
   }
 
   private static Codec readCodec(DataInput input) throws IOException {
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestSegmentInfos.java b/lucene/core/src/test/org/apache/lucene/index/TestSegmentInfos.java
index 19d8214..23c98ad 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestSegmentInfos.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestSegmentInfos.java
@@ -18,12 +18,16 @@ package org.apache.lucene.index;
 
 
 import org.apache.lucene.codecs.Codec;
+import org.apache.lucene.codecs.CodecUtil;
 import org.apache.lucene.search.Sort;
 import org.apache.lucene.store.BaseDirectoryWrapper;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IOContext;
+import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.store.IndexOutput;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.StringHelper;
+import org.apache.lucene.util.TestUtil;
 import org.apache.lucene.util.Version;
 
 import java.io.IOException;
@@ -178,5 +182,62 @@ public class TestSegmentInfos extends LuceneTestCase {
       assertEquals("clone changed but shouldn't", StringHelper.idToString(id), StringHelper.idToString(clone.getId()));
     }
   }
+
+  public void testBitFlippedTriggersCorruptIndexException() throws IOException {
+    BaseDirectoryWrapper dir = newDirectory();
+    dir.setCheckIndexOnClose(false);
+    byte id[] = StringHelper.randomId();
+    Codec codec = Codec.getDefault();
+
+    SegmentInfos sis = new SegmentInfos(Version.LATEST.major);
+    SegmentInfo info = new SegmentInfo(dir, Version.LATEST, Version.LATEST, "_0", 1, false, Codec.getDefault(),
+                                       Collections.<String,String>emptyMap(), id, Collections.<String,String>emptyMap(), null);
+    info.setFiles(Collections.<String>emptySet());
+    codec.segmentInfoFormat().write(dir, info, IOContext.DEFAULT);
+    SegmentCommitInfo commitInfo = new SegmentCommitInfo(info, 0, 0, -1, -1, -1, StringHelper.randomId());
+    sis.add(commitInfo);
+
+    info = new SegmentInfo(dir, Version.LATEST, Version.LATEST, "_1", 1, false, Codec.getDefault(),
+                           Collections.<String,String>emptyMap(), id, Collections.<String,String>emptyMap(), null);
+    info.setFiles(Collections.<String>emptySet());
+    codec.segmentInfoFormat().write(dir, info, IOContext.DEFAULT);
+    commitInfo = new SegmentCommitInfo(info, 0, 0,-1, -1, -1, StringHelper.randomId());
+    sis.add(commitInfo);
+
+    sis.commit(dir);
+
+    BaseDirectoryWrapper corruptDir = newDirectory();
+    corruptDir.setCheckIndexOnClose(false);
+    boolean corrupt = false;
+    for (String file : dir.listAll()) {
+      if (file.startsWith(IndexFileNames.SEGMENTS)) {
+        try (IndexInput in = dir.openInput(file, IOContext.DEFAULT);
+            IndexOutput out = corruptDir.createOutput(file, IOContext.DEFAULT)) {
+          final long corruptIndex = TestUtil.nextLong(random(), 0, in.length() - 1);
+          out.copyBytes(in, corruptIndex);
+          final int b = Byte.toUnsignedInt(in.readByte()) + TestUtil.nextInt(random(), 0x01, 0xff);
+          out.writeByte((byte) b);
+          out.copyBytes(in, in.length() - in.getFilePointer());
+        }
+        try (IndexInput in = corruptDir.openInput(file, IOContext.DEFAULT)) {
+          CodecUtil.checksumEntireFile(in);
+          if (VERBOSE) {
+            System.out.println("TEST: Altering the file did not update the checksum, aborting...");
+          }
+          return;
+        } catch (CorruptIndexException e) {
+          // ok
+        }
+        corrupt = true;
+      } else if (slowFileExists(corruptDir, file) == false) { // extraFS
+        corruptDir.copyFrom(dir, file, file, IOContext.DEFAULT);
+      }
+    }
+    assertTrue("No segments file found", corrupt);
+
+    expectThrows(CorruptIndexException.class, () -> SegmentInfos.readLatestCommit(corruptDir));
+    dir.close();
+    corruptDir.close();
+  }
 }