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

[lucene-solr] branch branch_8x updated: LUCENE-9260: Verify checksums of CFS files. (#1311)

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

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


The following commit(s) were added to refs/heads/branch_8x by this push:
     new 4a559ac  LUCENE-9260: Verify checksums of CFS files. (#1311)
4a559ac is described below

commit 4a559ac0c43ae40b9a70db679cc84d05a2e5f440
Author: Adrien Grand <jp...@gmail.com>
AuthorDate: Wed Apr 15 15:10:59 2020 +0200

    LUCENE-9260: Verify checksums of CFS files. (#1311)
---
 lucene/CHANGES.txt                                 |  3 +
 .../simpletext/SimpleTextCompoundFormat.java       | 31 ++------
 .../simpletext/TestSimpleTextCompoundFormat.java   |  5 ++
 .../apache/lucene/codecs/CompoundDirectory.java    | 83 ++++++++++++++++++++++
 .../org/apache/lucene/codecs/CompoundFormat.java   |  4 +-
 .../codecs/lucene50/Lucene50CompoundFormat.java    |  3 +-
 .../codecs/lucene50/Lucene50CompoundReader.java    | 49 ++-----------
 .../apache/lucene/index/SegmentCoreReaders.java    |  3 +-
 .../org/apache/lucene/index/SegmentReader.java     |  8 +++
 .../lucene/codecs/cranky/CrankyCompoundFormat.java |  3 +-
 .../lucene/index/BaseCompoundFormatTestCase.java   | 42 +++++++++++
 .../lucene/index/BaseIndexFileFormatTestCase.java  | 20 ++++--
 12 files changed, 176 insertions(+), 78 deletions(-)

diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index e3694d3..2611ad4 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -173,6 +173,9 @@ Optimizations
 
 * LUCENE-9237: Faster UniformSplit intersect TermsEnum. (Bruno Roustant)
 
+* LUCENE-9260: LeafReader#checkIntegrity verifies checksums of CFS files.
+  (Adrien Grand)
+
 * LUCENE-9068: FuzzyQuery builds its Automaton up-front (Alan Woodward, Mike Drob)
 
 * LUCENE-9113: Faster merging of SORTED/SORTED_SET doc values. (Adrien Grand)
diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextCompoundFormat.java b/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextCompoundFormat.java
index d04e99c..9e24877 100644
--- a/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextCompoundFormat.java
+++ b/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextCompoundFormat.java
@@ -24,11 +24,11 @@ import java.text.DecimalFormat;
 import java.text.DecimalFormatSymbols;
 import java.text.ParseException;
 import java.util.Arrays;
-import java.util.Collection;
 import java.util.Collections;
 import java.util.Locale;
 import java.util.Set;
 
+import org.apache.lucene.codecs.CompoundDirectory;
 import org.apache.lucene.codecs.CompoundFormat;
 import org.apache.lucene.index.CorruptIndexException;
 import org.apache.lucene.index.IndexFileNames;
@@ -37,7 +37,6 @@ 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.Lock;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.BytesRefBuilder;
 import org.apache.lucene.util.StringHelper;
@@ -55,7 +54,7 @@ public class SimpleTextCompoundFormat extends CompoundFormat {
   }
 
   @Override
-  public Directory getCompoundReader(Directory dir, SegmentInfo si, IOContext context) throws IOException {
+  public CompoundDirectory getCompoundReader(Directory dir, SegmentInfo si, IOContext context) throws IOException {
     String dataFile = IndexFileNames.segmentFileName(si.name, "", DATA_EXTENSION);
     final IndexInput in = dir.openInput(dataFile, context);
     
@@ -103,7 +102,7 @@ public class SimpleTextCompoundFormat extends CompoundFormat {
       endOffsets[i] = Long.parseLong(stripPrefix(scratch, TABLEEND));
     }
 
-    return new Directory() {
+    return new CompoundDirectory() {
 
       private int getIndex(String name) throws IOException {
         int index = Arrays.binarySearch(fileNames, name);
@@ -143,28 +142,10 @@ public class SimpleTextCompoundFormat extends CompoundFormat {
         return Collections.emptySet();
       }
 
-      // write methods: disabled
-      
-      @Override
-      public IndexOutput createOutput(String name, IOContext context) { throw new UnsupportedOperationException(); }
-
-      @Override
-      public IndexOutput createTempOutput(String prefix, String suffix, IOContext context) { throw new UnsupportedOperationException(); }
-      
-      @Override
-      public void sync(Collection<String> names) { throw new UnsupportedOperationException(); }
-      
       @Override
-      public void deleteFile(String name) { throw new UnsupportedOperationException(); }
-      
-      @Override
-      public void rename(String source, String dest) { throw new UnsupportedOperationException(); }
-
-      @Override
-      public void syncMetaData() { throw new UnsupportedOperationException(); }
-      
-      @Override
-      public Lock obtainLock(String name) { throw new UnsupportedOperationException(); }
+      public void checkIntegrity() throws IOException {
+        // No checksums for SimpleText
+      }
     };
   }
 
diff --git a/lucene/codecs/src/test/org/apache/lucene/codecs/simpletext/TestSimpleTextCompoundFormat.java b/lucene/codecs/src/test/org/apache/lucene/codecs/simpletext/TestSimpleTextCompoundFormat.java
index 2f54e2c..c5941c2 100644
--- a/lucene/codecs/src/test/org/apache/lucene/codecs/simpletext/TestSimpleTextCompoundFormat.java
+++ b/lucene/codecs/src/test/org/apache/lucene/codecs/simpletext/TestSimpleTextCompoundFormat.java
@@ -37,4 +37,9 @@ public class TestSimpleTextCompoundFormat extends BaseCompoundFormatTestCase {
   public void testMissingCodecHeadersAreCaught() {
     // SimpleText does not catch broken sub-files in CFS!
   }
+
+  @Override
+  public void testCheckIntegrity() {
+    // SimpleText does not catch broken sub-files in CFS!
+  }
 }
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/CompoundDirectory.java b/lucene/core/src/java/org/apache/lucene/codecs/CompoundDirectory.java
new file mode 100644
index 0000000..f063a12
--- /dev/null
+++ b/lucene/core/src/java/org/apache/lucene/codecs/CompoundDirectory.java
@@ -0,0 +1,83 @@
+/*
+ * 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.codecs;
+
+import java.io.IOException;
+import java.util.Collection;
+
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
+import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.store.Lock;
+
+/**
+ * A read-only {@link Directory} that consists of a view over a compound file.
+ * @see CompoundFormat
+ * @lucene.experimental
+ */
+public abstract class CompoundDirectory extends Directory {
+
+  /** Sole constructor. */
+  protected CompoundDirectory() {}
+
+  /**
+   * Checks consistency of this directory.
+   * <p>
+   * Note that this may be costly in terms of I/O, e.g.
+   * may involve computing a checksum value against large data files.
+   */
+  public abstract void checkIntegrity() throws IOException;
+
+  /** Not implemented
+   * @throws UnsupportedOperationException always: not supported by CFS */
+  @Override
+  public final void deleteFile(String name) {
+    throw new UnsupportedOperationException();
+  }
+  
+  /** Not implemented
+   * @throws UnsupportedOperationException always: not supported by CFS */
+  @Override
+  public final void rename(String from, String to) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public final void syncMetaData() {
+  }
+
+  @Override
+  public final IndexOutput createOutput(String name, IOContext context) throws IOException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public final IndexOutput createTempOutput(String prefix, String suffix, IOContext context) throws IOException {
+    throw new UnsupportedOperationException();
+  }
+  
+  @Override
+  public final void sync(Collection<String> names) {
+    throw new UnsupportedOperationException();
+  }
+  
+  @Override
+  public final Lock obtainLock(String name) {
+    throw new UnsupportedOperationException();
+  }
+
+}
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/CompoundFormat.java b/lucene/core/src/java/org/apache/lucene/codecs/CompoundFormat.java
index af1cc2a..5ab1c6a 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/CompoundFormat.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/CompoundFormat.java
@@ -40,8 +40,8 @@ public abstract class CompoundFormat {
   /**
    * Returns a Directory view (read-only) for the compound files in this segment
    */
-  public abstract Directory getCompoundReader(Directory dir, SegmentInfo si, IOContext context) throws IOException;
-  
+  public abstract CompoundDirectory getCompoundReader(Directory dir, SegmentInfo si, IOContext context) throws IOException;
+
   /**
    * Packs the provided segment's files into a compound format.  All files referenced
    * by the provided {@link SegmentInfo} must have {@link CodecUtil#writeIndexHeader}
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50CompoundFormat.java b/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50CompoundFormat.java
index 8fc314e..8e2cbb3 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50CompoundFormat.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/lucene50/Lucene50CompoundFormat.java
@@ -20,6 +20,7 @@ package org.apache.lucene.codecs.lucene50;
 import java.io.IOException;
 
 import org.apache.lucene.codecs.CodecUtil;
+import org.apache.lucene.codecs.CompoundDirectory;
 import org.apache.lucene.codecs.CompoundFormat;
 import org.apache.lucene.index.IndexFileNames;
 import org.apache.lucene.index.SegmentInfo;
@@ -66,7 +67,7 @@ public final class Lucene50CompoundFormat extends CompoundFormat {
   }
   
   @Override
-  public Directory getCompoundReader(Directory dir, SegmentInfo si, IOContext context) throws IOException {
+  public CompoundDirectory getCompoundReader(Directory dir, SegmentInfo si, IOContext context) throws IOException {
     return new Lucene50CompoundReader(dir, si, context);
   }
 
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 59e42d4..f4e7b9f 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
@@ -19,13 +19,13 @@ package org.apache.lucene.codecs.lucene50;
 
 import java.io.FileNotFoundException;
 import java.io.IOException;
-import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.Map;
 import java.util.Set;
 
 import org.apache.lucene.codecs.CodecUtil;
+import org.apache.lucene.codecs.CompoundDirectory;
 import org.apache.lucene.index.CorruptIndexException;
 import org.apache.lucene.index.IndexFileNames;
 import org.apache.lucene.index.SegmentInfo;
@@ -33,8 +33,6 @@ 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.store.IndexOutput;
-import org.apache.lucene.store.Lock;
 import org.apache.lucene.util.IOUtils;
 
 /**
@@ -43,7 +41,7 @@ import org.apache.lucene.util.IOUtils;
  * Directory methods that would normally modify data throw an exception.
  * @lucene.experimental
  */
-final class Lucene50CompoundReader extends Directory {
+final class Lucene50CompoundReader extends CompoundDirectory {
   
   /** Offset/Length for a slice inside of a compound file */
   public static final class FileEntry {
@@ -160,24 +158,6 @@ final class Lucene50CompoundReader extends Directory {
     return res;
   }
   
-  /** Not implemented
-   * @throws UnsupportedOperationException always: not supported by CFS */
-  @Override
-  public void deleteFile(String name) {
-    throw new UnsupportedOperationException();
-  }
-  
-  /** Not implemented
-   * @throws UnsupportedOperationException always: not supported by CFS */
-  @Override
-  public void rename(String from, String to) {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public void syncMetaData() {
-  }
-  
   /** Returns the length of a file in the directory.
    * @throws IOException if the file does not exist */
   @Override
@@ -188,26 +168,6 @@ final class Lucene50CompoundReader extends Directory {
       throw new FileNotFoundException(name);
     return e.length;
   }
-  
-  @Override
-  public IndexOutput createOutput(String name, IOContext context) throws IOException {
-    throw new UnsupportedOperationException();
-  }
-
-  @Override
-  public IndexOutput createTempOutput(String prefix, String suffix, IOContext context) throws IOException {
-    throw new UnsupportedOperationException();
-  }
-  
-  @Override
-  public void sync(Collection<String> names) {
-    throw new UnsupportedOperationException();
-  }
-  
-  @Override
-  public Lock obtainLock(String name) {
-    throw new UnsupportedOperationException();
-  }
 
   @Override
   public String toString() {
@@ -218,4 +178,9 @@ final class Lucene50CompoundReader extends Directory {
   public Set<String> getPendingDeletions() {
     return Collections.emptySet();
   }
+
+  @Override
+  public void checkIntegrity() throws IOException {
+    CodecUtil.checksumEntireFile(handle);
+  }
 }
diff --git a/lucene/core/src/java/org/apache/lucene/index/SegmentCoreReaders.java b/lucene/core/src/java/org/apache/lucene/index/SegmentCoreReaders.java
index 0e7b9e4..8da0e60 100644
--- a/lucene/core/src/java/org/apache/lucene/index/SegmentCoreReaders.java
+++ b/lucene/core/src/java/org/apache/lucene/index/SegmentCoreReaders.java
@@ -28,6 +28,7 @@ import java.util.Set;
 import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.lucene.codecs.Codec;
+import org.apache.lucene.codecs.CompoundDirectory;
 import org.apache.lucene.codecs.FieldsProducer;
 import org.apache.lucene.codecs.NormsProducer;
 import org.apache.lucene.codecs.PointsReader;
@@ -60,7 +61,7 @@ final class SegmentCoreReaders {
   final StoredFieldsReader fieldsReaderOrig;
   final TermVectorsReader termVectorsReaderOrig;
   final PointsReader pointsReader;
-  final Directory cfsReader;
+  final CompoundDirectory cfsReader;
   final String segment;
   /** 
    * fieldinfos for this core: means gen=-1.
diff --git a/lucene/core/src/java/org/apache/lucene/index/SegmentReader.java b/lucene/core/src/java/org/apache/lucene/index/SegmentReader.java
index b368b96..25145ff 100644
--- a/lucene/core/src/java/org/apache/lucene/index/SegmentReader.java
+++ b/lucene/core/src/java/org/apache/lucene/index/SegmentReader.java
@@ -366,4 +366,12 @@ public final class SegmentReader extends CodecReader {
   public Bits getHardLiveDocs() {
     return hardLiveDocs;
   }
+
+  @Override
+  public void checkIntegrity() throws IOException {
+    super.checkIntegrity();
+    if (core.cfsReader != null) {
+      core.cfsReader.checkIntegrity();
+    }
+  }
 }
diff --git a/lucene/test-framework/src/java/org/apache/lucene/codecs/cranky/CrankyCompoundFormat.java b/lucene/test-framework/src/java/org/apache/lucene/codecs/cranky/CrankyCompoundFormat.java
index 1b6b7f6..2114135 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/codecs/cranky/CrankyCompoundFormat.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/codecs/cranky/CrankyCompoundFormat.java
@@ -19,6 +19,7 @@ package org.apache.lucene.codecs.cranky;
 import java.io.IOException;
 import java.util.Random;
 
+import org.apache.lucene.codecs.CompoundDirectory;
 import org.apache.lucene.codecs.CompoundFormat;
 import org.apache.lucene.index.SegmentInfo;
 import org.apache.lucene.store.Directory;
@@ -34,7 +35,7 @@ class CrankyCompoundFormat extends CompoundFormat {
   }
   
   @Override
-  public Directory getCompoundReader(Directory dir, SegmentInfo si, IOContext context) throws IOException {
+  public CompoundDirectory getCompoundReader(Directory dir, SegmentInfo si, IOContext context) throws IOException {
     return delegate.getCompoundReader(dir, si, context);
   }
   
diff --git a/lucene/test-framework/src/java/org/apache/lucene/index/BaseCompoundFormatTestCase.java b/lucene/test-framework/src/java/org/apache/lucene/index/BaseCompoundFormatTestCase.java
index 1d28046..ebd53db 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/index/BaseCompoundFormatTestCase.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/index/BaseCompoundFormatTestCase.java
@@ -21,13 +21,17 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
 import java.util.List;
+import java.util.Map;
 import java.util.Random;
+import java.util.Set;
 
 import org.apache.lucene.codecs.Codec;
 import org.apache.lucene.codecs.CodecUtil;
+import org.apache.lucene.codecs.CompoundDirectory;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
 import org.apache.lucene.document.StoredField;
+import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.store.Directory;
 import org.apache.lucene.store.FilterDirectory;
 import org.apache.lucene.store.FlushInfo;
@@ -36,6 +40,7 @@ import org.apache.lucene.store.IndexInput;
 import org.apache.lucene.store.IndexOutput;
 import org.apache.lucene.store.MockDirectoryWrapper;
 import org.apache.lucene.store.NRTCachingDirectory;
+import org.apache.lucene.util.FixedBitSet;
 import org.apache.lucene.util.StringHelper;
 import org.apache.lucene.util.TestUtil;
 import org.apache.lucene.util.Version;
@@ -821,4 +826,41 @@ public abstract class BaseCompoundFormatTestCase extends BaseIndexFileFormatTest
     assertTrue(e.getMessage().contains("checksum failed (hardware problem?)"));
     dir.close();
   }
+
+  public void testCheckIntegrity() throws IOException {
+    Directory dir = newDirectory();
+    String subFile = "_123.xyz";
+    SegmentInfo si = newSegmentInfo(dir, "_123");
+    try (IndexOutput os = dir.createOutput(subFile, newIOContext(random()))) {
+      CodecUtil.writeIndexHeader(os, "Foo", 0, si.getId(), "suffix");
+      for (int i = 0; i < 1024; i++) {
+        os.writeByte((byte) i);
+      }
+      os.writeInt(CodecUtil.FOOTER_MAGIC);
+      os.writeInt(0);
+      long checksum = os.getChecksum();
+      os.writeLong(checksum);
+    }
+
+    si.setFiles(Collections.singletonList(subFile));
+    
+    FileTrackingDirectoryWrapper writeTrackingDir = new FileTrackingDirectoryWrapper(dir);
+    si.getCodec().compoundFormat().write(writeTrackingDir, si, IOContext.DEFAULT);
+    final Set<String> createdFiles = writeTrackingDir.getFiles();
+
+    ReadBytesDirectoryWrapper readTrackingDir = new ReadBytesDirectoryWrapper(dir);
+    CompoundDirectory compoundDir = si.getCodec().compoundFormat().getCompoundReader(readTrackingDir, si, IOContext.READ);
+    compoundDir.checkIntegrity();
+    Map<String,FixedBitSet> readBytes = readTrackingDir.getReadBytes();
+    assertEquals(createdFiles, readBytes.keySet());
+    for (Map.Entry<String, FixedBitSet> entry : readBytes.entrySet()) {
+      final String file = entry.getKey();
+      final FixedBitSet set = entry.getValue().clone();
+      set.flip(0, set.length());
+      final int next = set.nextSetBit(0);
+      assertEquals("Byte at offset " + next + " of " + file + " was not read", DocIdSetIterator.NO_MORE_DOCS, next);
+    }
+    compoundDir.close();
+    dir.close();
+  }
 }
diff --git a/lucene/test-framework/src/java/org/apache/lucene/index/BaseIndexFileFormatTestCase.java b/lucene/test-framework/src/java/org/apache/lucene/index/BaseIndexFileFormatTestCase.java
index ffdd8d6..1f1c593 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/index/BaseIndexFileFormatTestCase.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/index/BaseIndexFileFormatTestCase.java
@@ -80,7 +80,7 @@ import org.apache.lucene.util.Version;
 /**
  * Common tests to all index formats.
  */
-abstract class BaseIndexFileFormatTestCase extends LuceneTestCase {
+public abstract class BaseIndexFileFormatTestCase extends LuceneTestCase {
 
   // metadata or Directory-level objects
   private static final Set<Class<?>> EXCLUDED_CLASSES = Collections.newSetFromMap(new IdentityHashMap<Class<?>,Boolean>());
@@ -728,15 +728,20 @@ abstract class BaseIndexFileFormatTestCase extends LuceneTestCase {
     return r;
   }
 
-  private static class FileTrackingDirectoryWrapper extends FilterDirectory {
+  /**
+   * A directory that tracks created files that haven't been deleted.
+   */
+  protected static class FileTrackingDirectoryWrapper extends FilterDirectory {
 
     private final Set<String> files = Collections.newSetFromMap(new ConcurrentHashMap<String,Boolean>());
 
+    /** Sole constructor. */
     FileTrackingDirectoryWrapper(Directory in) {
       super(in);
     }
 
-    Set<String> getFiles() {
+    /** Get the set of created files. */
+    public Set<String> getFiles() {
       return Collections.unmodifiableSet(new HashSet<>(files));
     }
 
@@ -820,15 +825,18 @@ abstract class BaseIndexFileFormatTestCase extends LuceneTestCase {
 
   }
 
-  private static class ReadBytesDirectoryWrapper extends FilterDirectory {
+  /** A directory that tracks read bytes. */
+  protected static class ReadBytesDirectoryWrapper extends FilterDirectory {
 
-    ReadBytesDirectoryWrapper(Directory in) {
+    /** Sole constructor. */
+    public ReadBytesDirectoryWrapper(Directory in) {
       super(in);
     }
 
     private final Map<String, FixedBitSet> readBytes = new ConcurrentHashMap<>();
 
-    Map<String, FixedBitSet> getReadBytes() {
+    /** Get information about which bytes have been read. */
+    public Map<String, FixedBitSet> getReadBytes() {
       return Collections.unmodifiableMap(new HashMap<>(readBytes));
     }