You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by dw...@apache.org on 2021/03/10 10:03:53 UTC

[lucene] 16/18: LUCENE-6998: fix a couple places to better detect truncated index files; improve corruption testing

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

dweiss pushed a commit to branch branch_5_4
in repository https://gitbox.apache.org/repos/asf/lucene.git

commit df30bc6c5b4855fcd95c3660fdd2991d0e9c58bf
Author: Mike McCandless <mi...@apache.org>
AuthorDate: Thu Jan 28 05:01:15 2016 -0500

    LUCENE-6998: fix a couple places to better detect truncated index files; improve corruption testing
    
    Conflicts:
    	lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60PointFormat.java
    	lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60PointReader.java
    	lucene/core/src/java/org/apache/lucene/codecs/lucene60/Lucene60PointWriter.java
    
    Conflicts:
    	lucene/CHANGES.txt
---
 lucene/CHANGES.txt                                 |   3 +
 .../java/org/apache/lucene/codecs/CodecUtil.java   |   4 +-
 .../codecs/lucene50/Lucene50CompoundReader.java    |  14 +++
 .../lucene/index/TestAllFilesCheckIndexHeader.java | 132 +++++++++++++++++++++
 .../lucene/index/TestAllFilesDetectTruncation.java | 127 ++++++++++++++++++++
 .../index/TestAllFilesHaveChecksumFooter.java      |  24 ++--
 .../lucene/index/TestAllFilesHaveCodecHeader.java  |  31 ++---
 .../apache/lucene/index/TestSwappedIndexFiles.java | 127 ++++++++++++++++++++
 8 files changed, 422 insertions(+), 40 deletions(-)

diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index 9795dae..c93d601 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -18,6 +18,9 @@ Bug Fixes
   EOFException in some cases (Stéphane Campinas, Adrien Grand via Mike
   McCandless)
 
+* LUCENE-6998: Fix a couple places to better detect truncated index files
+  as corruption.  (Robert Muir, Mike McCandless)
+
 ======================= Lucene 5.4.1 =======================
 
 Bug Fixes
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/CodecUtil.java b/lucene/core/src/java/org/apache/lucene/codecs/CodecUtil.java
index ca6497a..3ca7569 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/CodecUtil.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/CodecUtil.java
@@ -17,7 +17,6 @@ package org.apache.lucene.codecs;
  * limitations under the License.
  */
 
-
 import java.io.IOException;
 import java.nio.charset.StandardCharsets;
 import java.util.Arrays;
@@ -397,6 +396,9 @@ public final class CodecUtil {
    * @throws IOException if the footer is invalid
    */
   public static long retrieveChecksum(IndexInput in) throws IOException {
+    if (in.length() < footerLength()) {
+      throw new CorruptIndexException("misplaced codec footer (file truncated?): length=" + in.length() + " but footerLength==" + footerLength(), in);
+    }
     in.seek(in.length() - footerLength());
     validateFooter(in);
     return readCRC(in);
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50CompoundReader.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50CompoundReader.java
index 5c14b86..99df7f2 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50CompoundReader.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50CompoundReader.java
@@ -68,6 +68,13 @@ final class Lucene50CompoundReader extends Directory {
     String entriesFileName = IndexFileNames.segmentFileName(segmentName, "", Lucene50CompoundFormat.ENTRIES_EXTENSION);
     this.entries = readEntries(si.getId(), directory, entriesFileName);
     boolean success = false;
+
+    long expectedLength = CodecUtil.indexHeaderLength(Lucene50CompoundFormat.DATA_CODEC, "");
+    for(Map.Entry<String,FileEntry> ent : entries.entrySet()) {
+      expectedLength += ent.getValue().length;
+    }
+    expectedLength += CodecUtil.footerLength(); 
+
     handle = directory.openInput(dataFileName, context);
     try {
       CodecUtil.checkIndexHeader(handle, Lucene50CompoundFormat.DATA_CODEC, version, version, si.getId(), "");
@@ -77,6 +84,13 @@ final class Lucene50CompoundReader extends Directory {
       // for FOOTER_MAGIC + algorithmID. This is cheap and can detect some forms of corruption
       // such as file truncation.
       CodecUtil.retrieveChecksum(handle);
+
+      // We also validate length, because e.g. if you strip 16 bytes off the .cfs we otherwise
+      // would not detect it:
+      if (handle.length() != expectedLength) {
+        throw new CorruptIndexException("length should be " + expectedLength + " bytes, but is " + handle.length() + " instead", handle);
+      }
+
       success = true;
     } finally {
       if (!success) {
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestAllFilesCheckIndexHeader.java b/lucene/core/src/test/org/apache/lucene/index/TestAllFilesCheckIndexHeader.java
new file mode 100644
index 0000000..9dffaa8
--- /dev/null
+++ b/lucene/core/src/test/org/apache/lucene/index/TestAllFilesCheckIndexHeader.java
@@ -0,0 +1,132 @@
+package org.apache.lucene.index;
+
+/*
+ * 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.
+ */
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.util.Collections;
+
+import org.apache.lucene.analysis.MockAnalyzer;
+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.store.MockDirectoryWrapper;
+import org.apache.lucene.util.LineFileDocs;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.TestUtil;
+
+/**
+ * Test that a plain default detects broken index headers early (on opening a reader).
+ */
+public class TestAllFilesCheckIndexHeader extends LuceneTestCase {
+  public void test() throws Exception {
+    Directory dir = newDirectory();
+
+    if (dir instanceof MockDirectoryWrapper) {
+      // otherwise we can have unref'd files left in the index that won't be visited when opening a reader and lead to scary looking false failures:
+      ((MockDirectoryWrapper) dir).setEnableVirusScanner(false);
+    }
+
+    IndexWriterConfig conf = newIndexWriterConfig(new MockAnalyzer(random()));
+    conf.setCodec(TestUtil.getDefaultCodec());
+
+    // Disable CFS 80% of the time so we can truncate individual files, but the other 20% of the time we test truncation of .cfs/.cfe too:
+    if (random().nextInt(5) != 1) {
+      conf.setUseCompoundFile(false);
+      conf.getMergePolicy().setNoCFSRatio(0.0);
+    }
+
+    RandomIndexWriter riw = new RandomIndexWriter(random(), dir, conf);
+    // Use LineFileDocs so we (hopefully) get most Lucene features
+    // tested, e.g. IntPoint was recently added to it:
+    LineFileDocs docs = new LineFileDocs(random());
+    for (int i = 0; i < 100; i++) {
+      riw.addDocument(docs.nextDoc());
+      if (random().nextInt(7) == 0) {
+        riw.commit();
+      }
+      if (random().nextInt(20) == 0) {
+        riw.deleteDocuments(new Term("docid", Integer.toString(i)));
+      }
+      if (random().nextInt(15) == 0) {
+        riw.updateNumericDocValue(new Term("docid", Integer.toString(i)), "docid_intDV", Long.valueOf(i));
+      }
+    }
+
+    if (TEST_NIGHTLY == false) {
+      riw.forceMerge(1);
+    }
+    riw.close();
+    checkIndexHeader(dir);
+    dir.close();
+  }
+  
+  private void checkIndexHeader(Directory dir) throws IOException {
+    for(String name : dir.listAll()) {
+      checkOneFile(dir, name);
+    }
+  }
+  
+  private void checkOneFile(Directory dir, String victim) throws IOException {
+    try (BaseDirectoryWrapper dirCopy = newDirectory()) {
+      dirCopy.setCheckIndexOnClose(false);
+      long victimLength = dir.fileLength(victim);
+      int wrongBytes = TestUtil.nextInt(random(), 1, (int) Math.min(100, victimLength));
+      assert victimLength > 0;
+
+      if (VERBOSE) {
+        System.out.println("TEST: now break file " + victim + " by randomizing first " + wrongBytes + " of " + victimLength);
+      }
+
+      for(String name : dir.listAll()) {
+        if (name.equals(victim) == false) {
+          dirCopy.copyFrom(dir, name, name, IOContext.DEFAULT);
+        } else {
+          try(IndexOutput out = dirCopy.createOutput(name, IOContext.DEFAULT);
+              IndexInput in = dir.openInput(name, IOContext.DEFAULT)) {
+              // keeps same file length, but replaces the first wrongBytes with random bytes:
+              byte[] bytes = new byte[wrongBytes];
+              random().nextBytes(bytes);
+              out.writeBytes(bytes, 0, bytes.length);
+              in.seek(wrongBytes);
+              out.copyBytes(in, victimLength - wrongBytes);
+            }
+        }
+        dirCopy.sync(Collections.singleton(name));
+      }
+
+      try {
+        // NOTE: we .close so that if the test fails (truncation not detected) we don't also get all these confusing errors about open files:
+        DirectoryReader.open(dirCopy).close();
+        fail("wrong bytes not detected after randomizing first " + wrongBytes + " bytes out of " + victimLength + " for file " + victim);
+      } catch (CorruptIndexException | EOFException | IndexFormatTooOldException e) {
+        // expected
+      }
+
+      // CheckIndex should also fail:
+      try {
+        TestUtil.checkIndex(dirCopy, true, true);
+        fail("wrong bytes not detected after randomizing first " + wrongBytes + " bytes out of " + victimLength + " for file " + victim);
+      } catch (CorruptIndexException | EOFException | IndexFormatTooOldException e) {
+        // expected
+      }
+    }
+  }
+}
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestAllFilesDetectTruncation.java b/lucene/core/src/test/org/apache/lucene/index/TestAllFilesDetectTruncation.java
new file mode 100644
index 0000000..bda5857
--- /dev/null
+++ b/lucene/core/src/test/org/apache/lucene/index/TestAllFilesDetectTruncation.java
@@ -0,0 +1,127 @@
+package org.apache.lucene.index;
+
+/*
+ * 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.
+ */
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.Collections;
+
+import org.apache.lucene.analysis.MockAnalyzer;
+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.store.MockDirectoryWrapper;
+import org.apache.lucene.util.LineFileDocs;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.TestUtil;
+
+/**
+ * Test that a plain default detects index file truncation early (on opening a reader).
+ */
+public class TestAllFilesDetectTruncation extends LuceneTestCase {
+  public void test() throws Exception {
+    Directory dir = newDirectory();
+
+    if (dir instanceof MockDirectoryWrapper) {
+      // otherwise we can have unref'd files left in the index that won't be visited when opening a reader and lead to scary looking false failures:
+      ((MockDirectoryWrapper) dir).setEnableVirusScanner(false);
+    }
+
+    IndexWriterConfig conf = newIndexWriterConfig(new MockAnalyzer(random()));
+    conf.setCodec(TestUtil.getDefaultCodec());
+
+    // Disable CFS 80% of the time so we can truncate individual files, but the other 20% of the time we test truncation of .cfs/.cfe too:
+    if (random().nextInt(5) != 1) {
+      conf.setUseCompoundFile(false);
+      conf.getMergePolicy().setNoCFSRatio(0.0);
+    }
+
+    RandomIndexWriter riw = new RandomIndexWriter(random(), dir, conf);
+    // Use LineFileDocs so we (hopefully) get most Lucene features
+    // tested, e.g. IntPoint was recently added to it:
+    LineFileDocs docs = new LineFileDocs(random());
+    for (int i = 0; i < 100; i++) {
+      riw.addDocument(docs.nextDoc());
+      if (random().nextInt(7) == 0) {
+        riw.commit();
+      }
+      if (random().nextInt(20) == 0) {
+        riw.deleteDocuments(new Term("docid", Integer.toString(i)));
+      }
+      if (random().nextInt(15) == 0) {
+        riw.updateNumericDocValue(new Term("docid", Integer.toString(i)), "docid_intDV", Long.valueOf(i));
+      }
+    }
+    if (TEST_NIGHTLY == false) {
+      riw.forceMerge(1);
+    }
+    riw.close();
+    checkTruncation(dir);
+    dir.close();
+  }
+  
+  private void checkTruncation(Directory dir) throws IOException {
+    for(String name : dir.listAll()) {
+      truncateOneFile(dir, name);
+    }
+  }
+  
+  private void truncateOneFile(Directory dir, String victim) throws IOException {
+    try (BaseDirectoryWrapper dirCopy = newDirectory()) {
+      dirCopy.setCheckIndexOnClose(false);
+      long victimLength = dir.fileLength(victim);
+      int lostBytes = TestUtil.nextInt(random(), 1, (int) Math.min(100, victimLength));
+      assert victimLength > 0;
+
+      if (VERBOSE) {
+        System.out.println("TEST: now truncate file " + victim + " by removing " + lostBytes + " of " + victimLength + " bytes");
+      }
+
+      for(String name : dir.listAll()) {
+        if (name.equals(victim) == false) {
+          dirCopy.copyFrom(dir, name, name, IOContext.DEFAULT);
+        } else {
+          try(IndexOutput out = dirCopy.createOutput(name, IOContext.DEFAULT);
+              IndexInput in = dir.openInput(name, IOContext.DEFAULT)) {
+              out.copyBytes(in, victimLength - lostBytes);
+            }
+        }
+        dirCopy.sync(Collections.singleton(name));
+      }
+
+      try {
+        // NOTE: we .close so that if the test fails (truncation not detected) we don't also get all these confusing errors about open files:
+        DirectoryReader.open(dirCopy).close();
+        fail("truncation not detected after removing " + lostBytes + " bytes out of " + victimLength + " for file " + victim);
+      } catch (CorruptIndexException | EOFException e) {
+        // expected
+      }
+
+      // CheckIndex should also fail:
+      try {
+        TestUtil.checkIndex(dirCopy, true, true);
+        fail("truncation not detected after removing " + lostBytes + " bytes out of " + victimLength + " for file " + victim);
+      } catch (CorruptIndexException | EOFException e) {
+        // expected
+      }
+    }
+  }
+}
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestAllFilesHaveChecksumFooter.java b/lucene/core/src/test/org/apache/lucene/index/TestAllFilesHaveChecksumFooter.java
index 66eb343..710d20f 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestAllFilesHaveChecksumFooter.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestAllFilesHaveChecksumFooter.java
@@ -21,11 +21,9 @@ import java.io.IOException;
 
 import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.codecs.CodecUtil;
-import org.apache.lucene.document.Document;
-import org.apache.lucene.document.Field;
-import org.apache.lucene.document.NumericDocValuesField;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.util.LineFileDocs;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.TestUtil;
 
@@ -38,23 +36,19 @@ public class TestAllFilesHaveChecksumFooter extends LuceneTestCase {
     IndexWriterConfig conf = newIndexWriterConfig(new MockAnalyzer(random()));
     conf.setCodec(TestUtil.getDefaultCodec());
     RandomIndexWriter riw = new RandomIndexWriter(random(), dir, conf);
-    Document doc = new Document();
-    // these fields should sometimes get term vectors, etc
-    Field idField = newStringField("id", "", Field.Store.NO);
-    Field bodyField = newTextField("body", "", Field.Store.NO);
-    Field dvField = new NumericDocValuesField("dv", 5);
-    doc.add(idField);
-    doc.add(bodyField);
-    doc.add(dvField);
+    // Use LineFileDocs so we (hopefully) get most Lucene features
+    // tested, e.g. IntPoint was recently added to it:
+    LineFileDocs docs = new LineFileDocs(random());
     for (int i = 0; i < 100; i++) {
-      idField.setStringValue(Integer.toString(i));
-      bodyField.setStringValue(TestUtil.randomUnicodeString(random()));
-      riw.addDocument(doc);
+      riw.addDocument(docs.nextDoc());
       if (random().nextInt(7) == 0) {
         riw.commit();
       }
       if (random().nextInt(20) == 0) {
-        riw.deleteDocuments(new Term("id", Integer.toString(i)));
+        riw.deleteDocuments(new Term("docid", Integer.toString(i)));
+      }
+      if (random().nextInt(15) == 0) {
+        riw.updateNumericDocValue(new Term("docid", Integer.toString(i)), "docid_intDV", Long.valueOf(i));
       }
     }
     riw.close();
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestAllFilesHaveCodecHeader.java b/lucene/core/src/test/org/apache/lucene/index/TestAllFilesHaveCodecHeader.java
index c2b515d..d726019 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestAllFilesHaveCodecHeader.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestAllFilesHaveCodecHeader.java
@@ -23,13 +23,9 @@ import java.util.Map;
 
 import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.codecs.CodecUtil;
-import org.apache.lucene.document.Document;
-import org.apache.lucene.document.Field;
-import org.apache.lucene.document.FieldType;
-import org.apache.lucene.document.NumericDocValuesField;
-import org.apache.lucene.document.TextField;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.util.LineFileDocs;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.TestUtil;
 
@@ -43,32 +39,19 @@ public class TestAllFilesHaveCodecHeader extends LuceneTestCase {
     IndexWriterConfig conf = newIndexWriterConfig(new MockAnalyzer(random()));
     conf.setCodec(TestUtil.getDefaultCodec());
     RandomIndexWriter riw = new RandomIndexWriter(random(), dir, conf);
-    Document doc = new Document();
-    Field idField = newStringField("id", "", Field.Store.YES);
-    Field bodyField = newTextField("body", "", Field.Store.YES);
-    FieldType vectorsType = new FieldType(TextField.TYPE_STORED);
-    vectorsType.setStoreTermVectors(true);
-    vectorsType.setStoreTermVectorPositions(true);
-    Field vectorsField = new Field("vectors", "", vectorsType);
-    Field dvField = new NumericDocValuesField("dv", 5);
-    doc.add(idField);
-    doc.add(bodyField);
-    doc.add(vectorsField);
-    doc.add(dvField);
+    // Use LineFileDocs so we (hopefully) get most Lucene features
+    // tested, e.g. IntPoint was recently added to it:
+    LineFileDocs docs = new LineFileDocs(random());
     for (int i = 0; i < 100; i++) {
-      idField.setStringValue(Integer.toString(i));
-      bodyField.setStringValue(TestUtil.randomUnicodeString(random()));
-      dvField.setLongValue(random().nextInt(5));
-      vectorsField.setStringValue(TestUtil.randomUnicodeString(random()));
-      riw.addDocument(doc);
+      riw.addDocument(docs.nextDoc());
       if (random().nextInt(7) == 0) {
         riw.commit();
       }
       if (random().nextInt(20) == 0) {
-        riw.deleteDocuments(new Term("id", Integer.toString(i)));
+        riw.deleteDocuments(new Term("docid", Integer.toString(i)));
       }
       if (random().nextInt(15) == 0) {
-        riw.updateNumericDocValue(new Term("id"), "dv", Long.valueOf(i));
+        riw.updateNumericDocValue(new Term("docid", Integer.toString(i)), "docid_intDV", Long.valueOf(i));
       }
     }
     riw.close();
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestSwappedIndexFiles.java b/lucene/core/src/test/org/apache/lucene/index/TestSwappedIndexFiles.java
new file mode 100644
index 0000000..008e9a6
--- /dev/null
+++ b/lucene/core/src/test/org/apache/lucene/index/TestSwappedIndexFiles.java
@@ -0,0 +1,127 @@
+package org.apache.lucene.index;
+
+/*
+ * 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.
+ */
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.util.Collections;
+import java.util.Random;
+
+import org.apache.lucene.analysis.MockAnalyzer;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.store.BaseDirectoryWrapper;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
+import org.apache.lucene.store.MockDirectoryWrapper;
+import org.apache.lucene.util.LineFileDocs;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.lucene.util.TestUtil;
+
+/**
+ * Test that the same file name, but from a different index, is detected as foreign.
+ */
+public class TestSwappedIndexFiles extends LuceneTestCase {
+  public void test() throws Exception {
+    Directory dir1 = newDirectory();
+    Directory dir2 = newDirectory();
+
+    if (dir1 instanceof MockDirectoryWrapper) {
+      // otherwise we can have unref'd files left in the index that won't be visited when opening a reader and lead to scary looking false failures:
+      ((MockDirectoryWrapper) dir1).setEnableVirusScanner(false);
+    }
+    if (dir2 instanceof MockDirectoryWrapper) {
+      // otherwise we can have unref'd files left in the index that won't be visited when opening a reader and lead to scary looking false failures:
+      ((MockDirectoryWrapper) dir2).setEnableVirusScanner(false);
+    }
+
+    // Disable CFS 80% of the time so we can truncate individual files, but the other 20% of the time we test truncation of .cfs/.cfe too:
+    boolean useCFS = random().nextInt(5) == 1;
+
+    // Use LineFileDocs so we (hopefully) get most Lucene features
+    // tested, e.g. IntPoint was recently added to it:
+    LineFileDocs docs = new LineFileDocs(random());
+    Document doc = docs.nextDoc();
+    long seed = random().nextLong();
+
+    indexOneDoc(seed, dir1, doc, useCFS);
+    indexOneDoc(seed, dir2, doc, useCFS);
+
+    swapFiles(dir1, dir2);
+    dir1.close();
+    dir2.close();
+  }
+
+  private void indexOneDoc(long seed, Directory dir, Document doc, boolean useCFS) throws IOException {
+    Random random = new Random(seed);
+    IndexWriterConfig conf = newIndexWriterConfig(new MockAnalyzer(random));
+    conf.setCodec(TestUtil.getDefaultCodec());
+
+    if (useCFS == false) {
+      conf.setUseCompoundFile(false);
+      conf.getMergePolicy().setNoCFSRatio(0.0);
+    } else {
+      conf.setUseCompoundFile(true);
+      conf.getMergePolicy().setNoCFSRatio(1.0);
+    }
+
+    RandomIndexWriter w = new RandomIndexWriter(random, dir, conf);
+    w.addDocument(doc);
+    w.close();
+  }
+  
+  private void swapFiles(Directory dir1, Directory dir2) throws IOException {
+    for(String name : dir1.listAll()) {
+      if (name.equals(IndexWriter.WRITE_LOCK_NAME)) {
+        continue;
+      }
+      swapOneFile(dir1, dir2, name);
+    }
+  }
+  
+  private void swapOneFile(Directory dir1, Directory dir2, String victim) throws IOException {
+    try (BaseDirectoryWrapper dirCopy = newDirectory()) {
+      dirCopy.setCheckIndexOnClose(false);
+
+      // Copy all files from dir1 to dirCopy, except victim which we copy from dir2:
+      for(String name : dir1.listAll()) {
+        if (name.equals(victim) == false) {
+          dirCopy.copyFrom(dir1, name, name, IOContext.DEFAULT);
+        } else {
+          dirCopy.copyFrom(dir2, name, name, IOContext.DEFAULT);
+        }
+        dirCopy.sync(Collections.singleton(name));
+      }
+
+      try {
+        // NOTE: we .close so that if the test fails (truncation not detected) we don't also get all these confusing errors about open files:
+        DirectoryReader.open(dirCopy).close();
+        fail("wrong file " + victim + " not detected");
+      } catch (CorruptIndexException | EOFException | IndexFormatTooOldException e) {
+        // expected
+      }
+
+      // CheckIndex should also fail:
+      try {
+        TestUtil.checkIndex(dirCopy, true, true);
+        fail("wrong file " + victim + " not detected");
+      } catch (CorruptIndexException | EOFException | IndexFormatTooOldException e) {
+        // expected
+      }
+    }
+  }
+}