diff --git a/src/main/java/com/github/dbmdz/solrocr/lucene/filters/ByteSeekableReader.java b/src/main/java/com/github/dbmdz/solrocr/lucene/filters/ByteSeekableReader.java new file mode 100644 index 00000000..e6e3a44e --- /dev/null +++ b/src/main/java/com/github/dbmdz/solrocr/lucene/filters/ByteSeekableReader.java @@ -0,0 +1,46 @@ +package com.github.dbmdz.solrocr.lucene.filters; + +import com.github.dbmdz.solrocr.reader.StreamDecoder; +import java.io.IOException; +import java.io.Reader; +import java.nio.channels.SeekableByteChannel; +import java.nio.charset.StandardCharsets; + +/** + * A Reader implementation that reads from a SeekableByteChannel and allows repositioning the + * reader. + */ +public class ByteSeekableReader extends Reader { + private final SeekableByteChannel channel; + private StreamDecoder decoder; + + public ByteSeekableReader(SeekableByteChannel channel) { + this.channel = channel; + this.decoder = StreamDecoder.forDecoder(channel, StandardCharsets.UTF_8.newDecoder(), -1); + } + + @Override + public int read(char[] cbuf, int off, int len) throws IOException { + return this.decoder.read(cbuf, off, len); + } + + /** Return the current byte position in the underlying channel. */ + public int position() throws IOException { + return (int) this.channel.position(); + } + + /** + * Reposition the reader to the given byte position. + * + *
This will also reset the decoder. + */ + public void position(int newPosition) throws IOException { + this.channel.position(newPosition); + this.decoder = StreamDecoder.forDecoder(channel, StandardCharsets.UTF_8.newDecoder(), -1); + } + + @Override + public void close() throws IOException { + this.channel.close(); + } +} diff --git a/src/main/java/com/github/dbmdz/solrocr/lucene/filters/ExternalUtf8ContentFilter.java b/src/main/java/com/github/dbmdz/solrocr/lucene/filters/ExternalUtf8ContentFilter.java index 3570446f..b790fdab 100644 --- a/src/main/java/com/github/dbmdz/solrocr/lucene/filters/ExternalUtf8ContentFilter.java +++ b/src/main/java/com/github/dbmdz/solrocr/lucene/filters/ExternalUtf8ContentFilter.java @@ -1,10 +1,13 @@ package com.github.dbmdz.solrocr.lucene.filters; import com.github.dbmdz.solrocr.model.SourcePointer; +import com.github.dbmdz.solrocr.model.SourcePointer.Region; import com.github.dbmdz.solrocr.util.SourceAwareReader; import com.github.dbmdz.solrocr.util.Utf8; +import com.google.common.collect.ImmutableList; import java.io.IOException; -import java.io.Reader; +import java.nio.CharBuffer; +import java.nio.channels.SeekableByteChannel; import java.util.LinkedList; import java.util.List; import java.util.Optional; @@ -17,98 +20,132 @@ public class ExternalUtf8ContentFilter extends BaseCharFilter implements SourceA * The cumulative offset difference between the input (bytes) and the output (chars) at the * current position. * + *
Used to calculate the byte offset in the input, given a + * char offset from the output of this filter. + * *
- * current actual byte offset in input = currentOutOffset + cumulative + * currentInputByteOffset = currentOutCharOffset + cumulativeOffsetDifference **/ - private int cumulative; + private int cumulativeOffsetDifference; - /** The current char offset in the full file; */ - private int currentInOffset; + /** + * The current byte offset in the full input, i.e. the + * concatenated content of all files in the source pointer. + */ + private int currentInByteOffset; - /** The current char offset in the output. */ - private int currentOutOffset; + /** + * The current char offset in the output, i.e. the concatenated and decoded + * content of all regions in the source pointer. + */ + private int currentOutCharOffset; /** Source pointer of this reader, used for debugging and error reporting. */ private final String pointer; - private boolean nextIsOffset = false; + /** Whether the last seen character had more than 1 byte for a char */ + private boolean lastCharHadMultipleBytes = false; + private final Queue
Keeps track of the current byte offset in the input and the current char offset in the
+ * output.
*/
@Override
- public int read(char[] cbuf, int off, int len) throws IOException {
- if (currentInOffset == currentRegion.end) {
+ public int read(char[] outputBuffer, int outputCharOffset, int requestedCharLen)
+ throws IOException {
+ if (currentInByteOffset == currentRegion.end) {
return -1;
}
int numCharsRead = 0;
- while (len - numCharsRead > 0) {
- int charsRemainingInRegion = currentRegion.end - currentInOffset;
- int charsToRead = len - numCharsRead;
- if (charsToRead > charsRemainingInRegion) {
- charsToRead = charsRemainingInRegion;
+ while (requestedCharLen - numCharsRead > 0) {
+ int bytesRemainingInRegion = currentRegion.end - currentInByteOffset;
+ int charsToRead = requestedCharLen - numCharsRead;
+ if (charsToRead > bytesRemainingInRegion) {
+ charsToRead = bytesRemainingInRegion;
}
- int read = this.input.read(cbuf, off, charsToRead);
- if (read < 0) {
+ int charsRead = this.input.read(outputBuffer, outputCharOffset, charsToRead);
+ if (charsRead < 0) {
break;
}
- correctOffsets(cbuf, off, read);
- numCharsRead += read;
- off += read;
+ while (Utf8.encodedLength(CharBuffer.wrap(outputBuffer, outputCharOffset, charsRead))
+ > bytesRemainingInRegion) {
+ charsRead--;
+ }
+ correctOffsets(outputBuffer, outputCharOffset, charsRead);
+ numCharsRead += charsRead;
+ outputCharOffset += charsRead;
- if (currentInOffset == currentRegion.end) {
+ if (currentInByteOffset == currentRegion.end) {
if (remainingRegions.isEmpty()) {
break;
}
currentRegion = remainingRegions.remove();
- cumulative = currentRegion.startOffset - currentOutOffset;
- this.addOffCorrectMap(currentOutOffset, cumulative);
- int toSkip = this.currentRegion.start - this.currentInOffset;
- if (toSkip > 0) {
- this.input.skip(this.currentRegion.start - this.currentInOffset);
+ cumulativeOffsetDifference = currentRegion.start - currentOutCharOffset;
+ this.addOffCorrectMap(currentOutCharOffset, cumulativeOffsetDifference);
+ if (this.currentRegion.start > this.currentInByteOffset) {
+ this.currentInByteOffset = currentRegion.start;
}
- this.currentInOffset = currentRegion.start;
+ ((ByteSeekableReader) this.input).position(this.currentInByteOffset);
}
}
return numCharsRead > 0 ? numCharsRead : -1;
}
- private void correctOffsets(char[] cbuf, int off, int len) {
- for (int i = off; i < off + len; i++) {
- if (nextIsOffset) {
- this.addOffCorrectMap(currentOutOffset, cumulative);
- nextIsOffset = false;
+ /**
+ * Updates the current input and output offsets based on the characters read from the input.
+ *
+ * @param decodedChars Buffer of characters that were read from the input
+ * @param bufOffset Offset in decodedChars, where the stored characters start
+ * @param numChars Number of characters stored in decodedChars
+ */
+ private void correctOffsets(char[] decodedChars, int bufOffset, int numChars) {
+ for (int i = bufOffset; i < bufOffset + numChars; ) {
+ if (lastCharHadMultipleBytes) {
+ this.addOffCorrectMap(currentOutCharOffset, cumulativeOffsetDifference);
+ lastCharHadMultipleBytes = false;
}
- currentInOffset += 1;
- currentOutOffset += 1;
- int cp = Character.codePointAt(cbuf, i);
- int increment = Utf8.encodedLength(cp) - 1;
- if (increment > 0) {
- cumulative += increment;
- nextIsOffset = true;
+ currentOutCharOffset += 1;
+ int cp = Character.codePointAt(decodedChars, i);
+ i += Character.charCount(cp);
+ int encodedLen = Utf8.encodedLength(cp);
+ currentInByteOffset += encodedLen;
+ if (encodedLen > 1) {
+ cumulativeOffsetDifference += (encodedLen - 1);
+ lastCharHadMultipleBytes = true;
}
}
}
diff --git a/src/main/java/com/github/dbmdz/solrocr/model/SourcePointer.java b/src/main/java/com/github/dbmdz/solrocr/model/SourcePointer.java
index fd916695..22c347a3 100644
--- a/src/main/java/com/github/dbmdz/solrocr/model/SourcePointer.java
+++ b/src/main/java/com/github/dbmdz/solrocr/model/SourcePointer.java
@@ -3,13 +3,13 @@
import com.github.dbmdz.solrocr.reader.FileSourceReader;
import com.github.dbmdz.solrocr.reader.MultiFileSourceReader;
import com.github.dbmdz.solrocr.reader.SourceReader;
-import com.google.common.collect.ImmutableList;
import java.io.FileNotFoundException;
import java.io.IOException;
import java.lang.invoke.MethodHandles;
import java.nio.file.Files;
import java.nio.file.Path;
import java.nio.file.Paths;
+import java.util.ArrayList;
import java.util.Arrays;
import java.util.Comparator;
import java.util.List;
@@ -75,7 +75,7 @@ static Source parse(String pointer) {
throw new RuntimeException("Could not parse source pointer from '" + pointer + ".");
}
String target = m.group("target");
- List Vendoring was necessary due to package-private constructors that don't allow subclassing.
+ */
+public class StreamDecoder extends Reader {
+
+ private static final int MIN_BYTE_BUFFER_SIZE = 32;
+ private static final int DEFAULT_BYTE_BUFFER_SIZE = 8192;
+
+ private volatile boolean closed;
+
+ private void ensureOpen() throws IOException {
+ if (closed) throw new IOException("Stream closed");
+ }
+
+ // In order to handle surrogates properly we must never try to produce
+ // fewer than two characters at a time. If we're only asked to return one
+ // character then the other is saved here to be returned later.
+ //
+ private boolean haveLeftoverChar = false;
+ private char leftoverChar;
+
+ // Factories for java.io.InputStreamReader
+
+ public static StreamDecoder forInputStreamReader(InputStream in, Object lock, String charsetName)
+ throws UnsupportedEncodingException {
+ try {
+ return new StreamDecoder(in, lock, Charset.forName(charsetName));
+ } catch (IllegalCharsetNameException | UnsupportedCharsetException x) {
+ throw new UnsupportedEncodingException(charsetName);
+ }
+ }
+
+ public static StreamDecoder forInputStreamReader(InputStream in, Object lock, Charset cs) {
+ return new StreamDecoder(in, lock, cs);
+ }
+
+ public static StreamDecoder forInputStreamReader(
+ InputStream in, Object lock, CharsetDecoder dec) {
+ return new StreamDecoder(in, lock, dec);
+ }
+
+ // Factory for java.nio.channels.Channels.newReader
+
+ public static StreamDecoder forDecoder(
+ ReadableByteChannel ch, CharsetDecoder dec, int minBufferCap) {
+ return new StreamDecoder(ch, dec, minBufferCap);
+ }
+
+ // -- Public methods corresponding to those in InputStreamReader --
+
+ // All synchronization and state/argument checking is done in these public
+ // methods; the concrete stream-decoder subclasses defined below need not
+ // do any such checking.
+
+ public String getEncoding() {
+ if (isOpen()) return encodingName();
+ return null;
+ }
+
+ public int read() throws IOException {
+ return read0();
+ }
+
+ private int read0() throws IOException {
+ Object lock = this.lock;
+ synchronized (lock) {
+ return lockedRead0();
+ }
+ }
+
+ @SuppressWarnings("fallthrough")
+ private int lockedRead0() throws IOException {
+ // Return the leftover char, if there is one
+ if (haveLeftoverChar) {
+ haveLeftoverChar = false;
+ return leftoverChar;
+ }
+
+ // Convert more bytes
+ char[] cb = new char[2];
+ int n = read(cb, 0, 2);
+ switch (n) {
+ case -1:
+ return -1;
+ case 2:
+ leftoverChar = cb[1];
+ haveLeftoverChar = true;
+ // FALL THROUGH
+ case 1:
+ return cb[0];
+ default:
+ assert false : n;
+ return -1;
+ }
+ }
+
+ public int read(char[] cbuf, int offset, int length) throws IOException {
+ Object lock = this.lock;
+ synchronized (lock) {
+ return lockedRead(cbuf, offset, length);
+ }
+ }
+
+ private int lockedRead(char[] cbuf, int offset, int length) throws IOException {
+ int off = offset;
+ int len = length;
+
+ ensureOpen();
+ if ((off < 0)
+ || (off > cbuf.length)
+ || (len < 0)
+ || ((off + len) > cbuf.length)
+ || ((off + len) < 0)) {
+ throw new IndexOutOfBoundsException();
+ }
+ if (len == 0) return 0;
+
+ int n = 0;
+
+ if (haveLeftoverChar) {
+ // Copy the leftover char into the buffer
+ cbuf[off] = leftoverChar;
+ off++;
+ len--;
+ haveLeftoverChar = false;
+ n = 1;
+ if ((len == 0) || !implReady())
+ // Return now if this is all we can produce w/o blocking
+ return n;
+ }
+
+ if (len == 1) {
+ // Treat single-character array reads just like read()
+ int c = read0();
+ if (c == -1) return (n == 0) ? -1 : n;
+ cbuf[off] = (char) c;
+ return n + 1;
+ }
+
+ // Read remaining characters
+ int nr = implRead(cbuf, off, off + len);
+
+ // At this point, n is either 1 if a leftover character was read,
+ // or 0 if no leftover character was read. If n is 1 and nr is -1,
+ // indicating EOF, then we don't return their sum as this loses data.
+ return (nr < 0) ? (n == 1 ? 1 : nr) : (n + nr);
+ }
+
+ public boolean ready() throws IOException {
+ Object lock = this.lock;
+ synchronized (lock) {
+ return lockedReady();
+ }
+ }
+
+ private boolean lockedReady() throws IOException {
+ ensureOpen();
+ return haveLeftoverChar || implReady();
+ }
+
+ public void close() throws IOException {
+ Object lock = this.lock;
+ synchronized (lock) {
+ lockedClose();
+ }
+ }
+
+ private void lockedClose() throws IOException {
+ if (closed) return;
+ try {
+ implClose();
+ } finally {
+ closed = true;
+ }
+ }
+
+ private boolean isOpen() {
+ return !closed;
+ }
+
+ public void fillZeroToPosition() throws IOException {
+ Object lock = this.lock;
+ synchronized (lock) {
+ lockedFillZeroToPosition();
+ }
+ }
+
+ private void lockedFillZeroToPosition() {
+ Arrays.fill(bb.array(), bb.arrayOffset(), bb.arrayOffset() + bb.position(), (byte) 0);
+ }
+
+ // -- Charset-based stream decoder impl --
+
+ private final Charset cs;
+ private final CharsetDecoder decoder;
+ private final ByteBuffer bb;
+
+ // Exactly one of these is non-null
+ private final InputStream in;
+ private final ReadableByteChannel ch;
+
+ StreamDecoder(InputStream in, Object lock, Charset cs) {
+ this(
+ in,
+ lock,
+ cs.newDecoder()
+ .onMalformedInput(CodingErrorAction.REPLACE)
+ .onUnmappableCharacter(CodingErrorAction.REPLACE));
+ }
+
+ StreamDecoder(InputStream in, Object lock, CharsetDecoder dec) {
+ super(lock);
+ this.cs = dec.charset();
+ this.decoder = dec;
+ this.in = in;
+ this.ch = null;
+ this.bb = ByteBuffer.allocate(DEFAULT_BYTE_BUFFER_SIZE);
+ bb.flip(); // So that bb is initially empty
+ }
+
+ StreamDecoder(ReadableByteChannel ch, CharsetDecoder dec, int mbc) {
+ this.in = null;
+ this.ch = ch;
+ this.decoder = dec;
+ this.cs = dec.charset();
+ this.bb =
+ ByteBuffer.allocate(
+ mbc < 0
+ ? DEFAULT_BYTE_BUFFER_SIZE
+ : (mbc < MIN_BYTE_BUFFER_SIZE ? MIN_BYTE_BUFFER_SIZE : mbc));
+ bb.flip();
+ }
+
+ private int readBytes() throws IOException {
+ bb.compact();
+ try {
+ if (ch != null) {
+ // Read from the channel
+ int n = ch.read(bb);
+ if (n < 0) return n;
+ } else {
+ // Read from the input stream, and then update the buffer
+ int lim = bb.limit();
+ int pos = bb.position();
+ assert (pos <= lim);
+ int rem = (pos <= lim ? lim - pos : 0);
+ int n = in.read(bb.array(), bb.arrayOffset() + pos, rem);
+ if (n < 0) return n;
+ if (n == 0) throw new IOException("Underlying input stream returned zero bytes");
+ assert (n <= rem) : "n = " + n + ", rem = " + rem;
+ bb.position(pos + n);
+ }
+ } finally {
+ // Flip even when an IOException is thrown,
+ // otherwise the stream will stutter
+ bb.flip();
+ }
+
+ int rem = bb.remaining();
+ assert (rem != 0) : rem;
+ return rem;
+ }
+
+ int implRead(char[] cbuf, int off, int end) throws IOException {
+
+ // In order to handle surrogate pairs, this method requires that
+ // the invoker attempt to read at least two characters. Saving the
+ // extra character, if any, at a higher level is easier than trying
+ // to deal with it here.
+ assert (end - off > 1);
+
+ CharBuffer cb = CharBuffer.wrap(cbuf, off, end - off);
+ if (cb.position() != 0) {
+ // Ensure that cb[0] == cbuf[off]
+ cb = cb.slice();
+ }
+
+ boolean eof = false;
+ for (; ; ) {
+ CoderResult cr = decoder.decode(bb, cb, eof);
+ if (cr.isUnderflow()) {
+ if (eof) break;
+ if (!cb.hasRemaining()) break;
+ if ((cb.position() > 0) && !inReady()) break; // Block at most once
+ int n = readBytes();
+ if (n < 0) {
+ eof = true;
+ if ((cb.position() == 0) && (!bb.hasRemaining())) break;
+ }
+ continue;
+ }
+ if (cr.isOverflow()) {
+ assert cb.position() > 0;
+ break;
+ }
+ cr.throwException();
+ }
+
+ if (eof) {
+ // ## Need to flush decoder
+ decoder.reset();
+ }
+
+ if (cb.position() == 0) {
+ if (eof) {
+ return -1;
+ }
+ assert false;
+ }
+ return cb.position();
+ }
+
+ String encodingName() {
+ return cs.name();
+ }
+
+ private boolean inReady() {
+ try {
+ return (((in != null) && (in.available() > 0))
+ || (ch instanceof FileChannel)); // ## RBC.available()?
+ } catch (IOException x) {
+ return false;
+ }
+ }
+
+ boolean implReady() {
+ return bb.hasRemaining() || inReady();
+ }
+
+ void implClose() throws IOException {
+ if (ch != null) {
+ ch.close();
+ } else {
+ in.close();
+ }
+ }
+}
diff --git a/src/main/java/solrocr/ExternalUtf8ContentFilterFactory.java b/src/main/java/solrocr/ExternalUtf8ContentFilterFactory.java
index 385eb319..8b4ce919 100644
--- a/src/main/java/solrocr/ExternalUtf8ContentFilterFactory.java
+++ b/src/main/java/solrocr/ExternalUtf8ContentFilterFactory.java
@@ -2,21 +2,18 @@
import com.github.dbmdz.solrocr.lucene.filters.ExternalUtf8ContentFilter;
import com.github.dbmdz.solrocr.model.SourcePointer;
+import com.github.dbmdz.solrocr.model.SourcePointer.Region;
import com.github.dbmdz.solrocr.model.SourcePointer.Source;
import com.github.dbmdz.solrocr.model.SourcePointer.SourceType;
import com.github.dbmdz.solrocr.reader.SourceReader;
-import com.github.dbmdz.solrocr.util.Utf8;
-import com.google.common.collect.ImmutableList;
-import java.io.BufferedReader;
import java.io.File;
import java.io.IOException;
import java.io.Reader;
import java.io.StringReader;
import java.nio.ByteBuffer;
-import java.nio.channels.Channels;
import java.nio.channels.SeekableByteChannel;
-import java.nio.charset.StandardCharsets;
import java.nio.file.Paths;
+import java.util.Arrays;
import java.util.List;
import java.util.Locale;
import java.util.Map;
@@ -63,11 +60,6 @@ public Reader create(Reader input) {
}
pointer.sources.forEach(this::validateSource);
- // Regions contained in source pointers are defined by byte offsets.
- // We need to convert these to Java character offsets, so they can be used by the filter.
- // This is very expensive, but we need this since all IO from here on out is character-based.
- toCharOffsets(pointer);
-
if (pointer.sources.isEmpty()) {
throw new RuntimeException(
"No source files could be determined from pointer. "
@@ -75,12 +67,13 @@ public Reader create(Reader input) {
+ "Pointer was: "
+ ptrStr);
}
- Reader r =
- Channels.newReader(
- pointer.getReader(512 * 1024, 0).getByteChannel(), StandardCharsets.UTF_8.name());
- List UTF8-encoded files may contain a 3 byte byte-order-marker at the beginning of the file. Its
+ * use is discouraged and not needed for UTF8 (since the byte order is pre-defined), but we've
+ * encountered OCR files in the wild that have it, so we check for it and adjust regions starting
+ * on the beginning of the file to account for it.
+ */
+ private void adjustRegions(SourcePointer ptr) throws IOException {
+ int outByteOffset = 0;
+ byte[] bomBuf = new byte[3];
for (SourcePointer.Source src : ptr.sources) {
- SourceReader reader = src.getReader(512 * 1024, 0);
- try {
- SeekableByteChannel chan = reader.getByteChannel();
- final int size = (int) chan.size();
+ // Again, section size and cache size don't matter, since we don't use sectioned reads during
+ // indexing.
+ try (SourceReader reader = src.getReader(512, 0)) {
+ int inputLen = reader.length();
- int bomOffset = 0;
- if (!src.isAscii) {
- // Check for BOM without modifying channel position, we need to skip it as to not break
- // mult-file parsing
- ByteBuffer bomBuf = ByteBuffer.allocate(3);
- chan.read(bomBuf);
+ if (src.regions.isEmpty()) {
+ src.regions.add(new Region(0, inputLen));
+ }
+
+ if (!src.isAscii && src.regions.stream().anyMatch(r -> r.start == 0)) {
+ SeekableByteChannel chan = reader.getByteChannel();
+ chan.read(ByteBuffer.wrap(bomBuf));
chan.position(0);
- bomBuf.flip();
- if (bomBuf.equals(ByteBuffer.wrap(new byte[] {(byte) 0xEF, (byte) 0xBB, (byte) 0xBF}))) {
- bomOffset = 3;
+ boolean hasBOM =
+ Arrays.equals(bomBuf, new byte[] {(byte) 0xEF, (byte) 0xBB, (byte) 0xBF});
+ if (hasBOM) {
+ src.regions.stream().filter(r -> r.start == 0).forEach(r -> r.start += 3);
}
}
- // Byte offset of the current file from the beginning of the first file
- final int baseOffset = byteOffset;
- if (src.regions.isEmpty()) {
- src.regions = ImmutableList.of(new SourcePointer.Region(0, size));
- }
- for (SourcePointer.Region region : src.regions) {
- if (src.isAscii) {
- // Optimization for pure-ASCII sources, where we don't need to do any mapping
- region.start += baseOffset;
- region.end = Math.min(region.end + baseOffset, size + baseOffset);
- continue;
- }
- if (region.start == 0) {
- // Skip the BOM at the start of a file, if present
- region.start += bomOffset;
- }
- // Make region offsets relative to the beginning of the first file
- region.start += baseOffset;
- if (region.end < 0) {
- region.end = size;
+ for (Region region : src.regions) {
+ if (region.end == -1) {
+ region.end = inputLen;
}
- region.end = Math.min(region.end + baseOffset, size + baseOffset);
- // Read until the start of the region
- if (byteOffset != region.start) {
- // Read the data between the current offset and the start of the region
- int len = region.start - byteOffset;
- charOffset += (int) getUtf8DecodedLength(chan, buf, len);
- byteOffset += len;
- }
-
- int regionSize = region.end - region.start;
- region.start = charOffset;
- region.startOffset = byteOffset;
- // Read region, determine character offset of region end
- charOffset += (int) getUtf8DecodedLength(chan, buf, regionSize);
- byteOffset += regionSize;
- region.end = charOffset;
+ region.start += outByteOffset;
+ region.end += outByteOffset;
}
- // Determine character offset of the end of the file
- if (src.isAscii) {
- byteOffset += size;
- } else if (byteOffset != baseOffset + size) {
- int len = (baseOffset + size) - byteOffset;
- charOffset += (int) getUtf8DecodedLength(chan, buf, len);
- byteOffset += len;
- }
- } finally {
- reader.close();
+
+ outByteOffset += inputLen;
}
}
}
diff --git a/src/test/java/com/github/dbmdz/solrocr/lucene/ExternalUtf8ContentFilterTest.java b/src/test/java/com/github/dbmdz/solrocr/lucene/ExternalUtf8ContentFilterTest.java
index ce8b4d50..db3463b6 100644
--- a/src/test/java/com/github/dbmdz/solrocr/lucene/ExternalUtf8ContentFilterTest.java
+++ b/src/test/java/com/github/dbmdz/solrocr/lucene/ExternalUtf8ContentFilterTest.java
@@ -6,8 +6,6 @@
import com.github.dbmdz.solrocr.model.SourcePointer.Region;
import com.github.dbmdz.solrocr.util.Utf8;
import com.google.common.collect.ImmutableList;
-import java.io.BufferedReader;
-import java.io.FileReader;
import java.io.IOException;
import java.io.StringReader;
import java.nio.ByteBuffer;
@@ -57,7 +55,7 @@ public void extractFully() throws IOException {
Path p = Paths.get("src/test/resources/data/hocr.html");
CharFilter filter =
new ExternalUtf8ContentFilter(
- new BufferedReader(new FileReader(p.toFile())),
+ Files.newByteChannel(p),
ImmutableList.of(new Region(0, (int) p.toFile().length())),
p.toString());
String full = new String(Files.readAllBytes(p), StandardCharsets.UTF_8);