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/07 11:30:56 UTC

[lucene-solr] branch branch_8x updated: LUCENE-9271: Move BufferedIndexInput to the ByteBuffer API.

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 4ccbf19  LUCENE-9271: Move BufferedIndexInput to the ByteBuffer API.
4ccbf19 is described below

commit 4ccbf19eb785705b80f3ecb10fc8643dbee53783
Author: Adrien Grand <jp...@gmail.com>
AuthorDate: Tue Apr 7 12:44:00 2020 +0200

    LUCENE-9271: Move BufferedIndexInput to the ByteBuffer API.
    
    Closes #1338
---
 lucene/CHANGES.txt                                 |   3 +
 .../apache/lucene/store/BufferedIndexInput.java    | 223 ++++++++-------------
 .../org/apache/lucene/store/NIOFSDirectory.java    |  34 +---
 .../org/apache/lucene/store/SimpleFSDirectory.java |  32 +--
 .../org/apache/lucene/index/TestFieldsReader.java  |   6 +-
 .../lucene/store/TestBufferedIndexInput.java       |  16 +-
 .../java/org/apache/lucene/store/RAFDirectory.java |  18 +-
 .../org/apache/lucene/store/WindowsDirectory.java  |   5 +-
 8 files changed, 123 insertions(+), 214 deletions(-)

diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index f0736e8..6b0a2e5 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -67,6 +67,9 @@ Other
 * LUCENE-9244: Adjust TestLucene60PointsFormat#testEstimatePointCount2Dims so it does not fail when a point
   is shared by multiple leaves. (Ignacio Vera)
 
+* LUCENE-9271: ByteBufferIndexInput was refactored to work on top of the
+  ByteBuffer API. (Adrien Grand)
+
 ======================= Lucene 8.5.0 =======================
 
 API Changes
diff --git a/lucene/core/src/java/org/apache/lucene/store/BufferedIndexInput.java b/lucene/core/src/java/org/apache/lucene/store/BufferedIndexInput.java
index 3f15eea..fa06b0a 100644
--- a/lucene/core/src/java/org/apache/lucene/store/BufferedIndexInput.java
+++ b/lucene/core/src/java/org/apache/lucene/store/BufferedIndexInput.java
@@ -19,10 +19,14 @@ package org.apache.lucene.store;
 
 import java.io.EOFException;
 import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
 
 /** Base implementation class for buffered {@link IndexInput}. */
 public abstract class BufferedIndexInput extends IndexInput implements RandomAccessInput {
 
+  private static final ByteBuffer EMPTY_BYTEBUFFER = ByteBuffer.allocate(0);
+
   /** Default buffer size set to {@value #BUFFER_SIZE}. */
   public static final int BUFFER_SIZE = 1024;
   
@@ -42,17 +46,16 @@ public abstract class BufferedIndexInput extends IndexInput implements RandomAcc
 
   private int bufferSize = BUFFER_SIZE;
   
-  protected byte[] buffer;
-  
+  private ByteBuffer buffer = EMPTY_BYTEBUFFER;
+
   private long bufferStart = 0;       // position in file of buffer
-  private int bufferLength = 0;       // end of valid bytes
-  private int bufferPosition = 0;     // next byte to read
 
   @Override
   public final byte readByte() throws IOException {
-    if (bufferPosition >= bufferLength)
+    if (buffer.hasRemaining() == false) {
       refill();
-    return buffer[bufferPosition++];
+    }
+    return buffer.get();
   }
 
   public BufferedIndexInput(String resourceDesc) {
@@ -72,7 +75,7 @@ public abstract class BufferedIndexInput extends IndexInput implements RandomAcc
 
   /** Change the buffer size used by this IndexInput */
   public final void setBufferSize(int newSize) {
-    assert buffer == null || bufferSize == buffer.length: "buffer=" + buffer + " bufferSize=" + bufferSize + " buffer.length=" + (buffer != null ? buffer.length : 0);
+    assert buffer == null || bufferSize == buffer.capacity(): "buffer=" + buffer + " bufferSize=" + bufferSize + " buffer.length=" + (buffer != null ? buffer.capacity() : 0);
     if (newSize != bufferSize) {
       checkBufferSize(newSize);
       bufferSize = newSize;
@@ -80,27 +83,19 @@ public abstract class BufferedIndexInput extends IndexInput implements RandomAcc
         // Resize the existing buffer and carefully save as
         // many bytes as possible starting from the current
         // bufferPosition
-        byte[] newBuffer = new byte[newSize];
-        final int leftInBuffer = bufferLength-bufferPosition;
-        final int numToCopy;
-        if (leftInBuffer > newSize)
-          numToCopy = newSize;
-        else
-          numToCopy = leftInBuffer;
-        System.arraycopy(buffer, bufferPosition, newBuffer, 0, numToCopy);
-        bufferStart += bufferPosition;
-        bufferPosition = 0;
-        bufferLength = numToCopy;
-        newBuffer(newBuffer);
+        ByteBuffer newBuffer = ByteBuffer.allocate(newSize);
+        assert newBuffer.order() == ByteOrder.BIG_ENDIAN;
+        if (buffer.remaining() > newBuffer.capacity()) {
+          buffer.limit(buffer.position() + newBuffer.capacity());
+        }
+        assert buffer.remaining() <= newBuffer.capacity();
+        newBuffer.put(buffer);
+        newBuffer.flip();
+        buffer = newBuffer;
       }
     }
   }
 
-  protected void newBuffer(byte[] newBuffer) {
-    // Subclasses can do something here
-    buffer = newBuffer;
-  }
-
   /** Returns buffer size.  @see #setBufferSize */
   public final int getBufferSize() {
     return bufferSize;
@@ -118,19 +113,17 @@ public abstract class BufferedIndexInput extends IndexInput implements RandomAcc
 
   @Override
   public final void readBytes(byte[] b, int offset, int len, boolean useBuffer) throws IOException {
-    int available = bufferLength - bufferPosition;
+    int available = buffer.remaining();
     if(len <= available){
       // the buffer contains enough data to satisfy this request
       if(len>0) // to allow b to be null if len is 0...
-        System.arraycopy(buffer, bufferPosition, b, offset, len);
-      bufferPosition+=len;
+        buffer.get(b, offset, len);
     } else {
       // the buffer does not have enough data. First serve all we've got.
       if(available > 0){
-        System.arraycopy(buffer, bufferPosition, b, offset, available);
+        buffer.get(b, offset, available);
         offset += available;
         len -= available;
-        bufferPosition += available;
       }
       // and now, read the remaining 'len' bytes:
       if (useBuffer && len<bufferSize){
@@ -138,13 +131,12 @@ public abstract class BufferedIndexInput extends IndexInput implements RandomAcc
         // we are allowed to use our buffer, do it in the usual
         // buffered way: fill the buffer and copy from it:
         refill();
-        if(bufferLength<len){
+        if(buffer.remaining()<len){
           // Throw an exception when refill() could not read len bytes:
-          System.arraycopy(buffer, 0, b, offset, bufferLength);
+          buffer.get(b, offset, buffer.remaining());
           throw new EOFException("read past EOF: " + this);
         } else {
-          System.arraycopy(buffer, 0, b, offset, len);
-          bufferPosition=len;
+          buffer.get(b, offset, len);
         }
       } else {
         // The amount left to read is larger than the buffer
@@ -154,21 +146,20 @@ public abstract class BufferedIndexInput extends IndexInput implements RandomAcc
         // this function, there is no need to do a seek
         // here, because there's no need to reread what we
         // had in the buffer.
-        long after = bufferStart+bufferPosition+len;
+        long after = bufferStart+buffer.position()+len;
         if(after > length())
           throw new EOFException("read past EOF: " + this);
-        readInternal(b, offset, len);
+        readInternal(ByteBuffer.wrap(b, offset, len));
         bufferStart = after;
-        bufferPosition = 0;
-        bufferLength = 0;                    // trigger refill() on read
+        buffer.limit(0);  // trigger refill() on read
       }
     }
   }
 
   @Override
   public final short readShort() throws IOException {
-    if (2 <= (bufferLength-bufferPosition)) {
-      return (short) (((buffer[bufferPosition++] & 0xFF) <<  8) |  (buffer[bufferPosition++] & 0xFF));
+    if (Short.BYTES <= buffer.remaining()) {
+      return buffer.getShort();
     } else {
       return super.readShort();
     }
@@ -176,9 +167,8 @@ public abstract class BufferedIndexInput extends IndexInput implements RandomAcc
   
   @Override
   public final int readInt() throws IOException {
-    if (4 <= (bufferLength-bufferPosition)) {
-      return ((buffer[bufferPosition++] & 0xFF) << 24) | ((buffer[bufferPosition++] & 0xFF) << 16)
-        | ((buffer[bufferPosition++] & 0xFF) <<  8) |  (buffer[bufferPosition++] & 0xFF);
+    if (Integer.BYTES <= buffer.remaining()) {
+      return buffer.getInt();
     } else {
       return super.readInt();
     }
@@ -186,12 +176,8 @@ public abstract class BufferedIndexInput extends IndexInput implements RandomAcc
   
   @Override
   public final long readLong() throws IOException {
-    if (8 <= (bufferLength-bufferPosition)) {
-      final int i1 = ((buffer[bufferPosition++] & 0xff) << 24) | ((buffer[bufferPosition++] & 0xff) << 16) |
-        ((buffer[bufferPosition++] & 0xff) << 8) | (buffer[bufferPosition++] & 0xff);
-      final int i2 = ((buffer[bufferPosition++] & 0xff) << 24) | ((buffer[bufferPosition++] & 0xff) << 16) |
-        ((buffer[bufferPosition++] & 0xff) << 8) | (buffer[bufferPosition++] & 0xff);
-      return (((long)i1) << 32) | (i2 & 0xFFFFFFFFL);
+    if (Long.BYTES <= buffer.remaining()) {
+      return buffer.getLong();
     } else {
       return super.readLong();
     }
@@ -199,20 +185,20 @@ public abstract class BufferedIndexInput extends IndexInput implements RandomAcc
 
   @Override
   public final int readVInt() throws IOException {
-    if (5 <= (bufferLength-bufferPosition)) {
-      byte b = buffer[bufferPosition++];
+    if (5 <= buffer.remaining()) {
+      byte b = buffer.get();
       if (b >= 0) return b;
       int i = b & 0x7F;
-      b = buffer[bufferPosition++];
+      b = buffer.get();
       i |= (b & 0x7F) << 7;
       if (b >= 0) return i;
-      b = buffer[bufferPosition++];
+      b = buffer.get();
       i |= (b & 0x7F) << 14;
       if (b >= 0) return i;
-      b = buffer[bufferPosition++];
+      b = buffer.get();
       i |= (b & 0x7F) << 21;
       if (b >= 0) return i;
-      b = buffer[bufferPosition++];
+      b = buffer.get();
       // Warning: the next ands use 0x0F / 0xF0 - beware copy/paste errors:
       i |= (b & 0x0F) << 28;
       if ((b & 0xF0) == 0) return i;
@@ -224,32 +210,32 @@ public abstract class BufferedIndexInput extends IndexInput implements RandomAcc
   
   @Override
   public final long readVLong() throws IOException {
-    if (9 <= bufferLength-bufferPosition) {
-      byte b = buffer[bufferPosition++];
+    if (9 <= buffer.remaining()) {
+      byte b = buffer.get();
       if (b >= 0) return b;
       long i = b & 0x7FL;
-      b = buffer[bufferPosition++];
+      b = buffer.get();
       i |= (b & 0x7FL) << 7;
       if (b >= 0) return i;
-      b = buffer[bufferPosition++];
+      b = buffer.get();
       i |= (b & 0x7FL) << 14;
       if (b >= 0) return i;
-      b = buffer[bufferPosition++];
+      b = buffer.get();
       i |= (b & 0x7FL) << 21;
       if (b >= 0) return i;
-      b = buffer[bufferPosition++];
+      b = buffer.get();
       i |= (b & 0x7FL) << 28;
       if (b >= 0) return i;
-      b = buffer[bufferPosition++];
+      b = buffer.get();
       i |= (b & 0x7FL) << 35;
       if (b >= 0) return i;
-      b = buffer[bufferPosition++];
+      b = buffer.get();
       i |= (b & 0x7FL) << 42;
       if (b >= 0) return i;
-      b = buffer[bufferPosition++];
+      b = buffer.get();
       i |= (b & 0x7FL) << 49;
       if (b >= 0) return i;
-      b = buffer[bufferPosition++];
+      b = buffer.get();
       i |= (b & 0x7FL) << 56;
       if (b >= 0) return i;
       throw new IOException("Invalid vLong detected (negative values disallowed)");
@@ -261,73 +247,57 @@ public abstract class BufferedIndexInput extends IndexInput implements RandomAcc
   @Override
   public final byte readByte(long pos) throws IOException {
     long index = pos - bufferStart;
-    if (index < 0 || index >= bufferLength) {
+    if (index < 0 || index >= buffer.limit()) {
       bufferStart = pos;
-      bufferPosition = 0;
-      bufferLength = 0;  // trigger refill() on read()
+      buffer.limit(0);  // trigger refill() on read
       seekInternal(pos);
       refill();
       index = 0;
     }
-    return buffer[(int)index];
+    return buffer.get((int) index);
   }
 
   @Override
   public final short readShort(long pos) throws IOException {
     long index = pos - bufferStart;
-    if (index < 0 || index >= bufferLength-1) {
+    if (index < 0 || index >= buffer.limit()-1) {
       bufferStart = pos;
-      bufferPosition = 0;
-      bufferLength = 0;  // trigger refill() on read()
+      buffer.limit(0);  // trigger refill() on read
       seekInternal(pos);
       refill();
       index = 0;
     }
-    return (short) (((buffer[(int)index]   & 0xFF) << 8) | 
-                     (buffer[(int)index+1] & 0xFF));
+    return buffer.getShort((int) index);
   }
 
   @Override
   public final int readInt(long pos) throws IOException {
     long index = pos - bufferStart;
-    if (index < 0 || index >= bufferLength-3) {
+    if (index < 0 || index >= buffer.limit()-3) {
       bufferStart = pos;
-      bufferPosition = 0;
-      bufferLength = 0;  // trigger refill() on read()
+      buffer.limit(0);  // trigger refill() on read
       seekInternal(pos);
       refill();
       index = 0;
     }
-    return ((buffer[(int)index]   & 0xFF) << 24) | 
-           ((buffer[(int)index+1] & 0xFF) << 16) |
-           ((buffer[(int)index+2] & 0xFF) << 8)  |
-            (buffer[(int)index+3] & 0xFF);
+    return buffer.getInt((int) index);
   }
 
   @Override
   public final long readLong(long pos) throws IOException {
     long index = pos - bufferStart;
-    if (index < 0 || index >= bufferLength-7) {
+    if (index < 0 || index >= buffer.limit()-7) {
       bufferStart = pos;
-      bufferPosition = 0;
-      bufferLength = 0;  // trigger refill() on read()
+      buffer.limit(0);  // trigger refill() on read
       seekInternal(pos);
       refill();
       index = 0;
     }
-    final int i1 = ((buffer[(int)index]   & 0xFF) << 24) | 
-                   ((buffer[(int)index+1] & 0xFF) << 16) |
-                   ((buffer[(int)index+2] & 0xFF) << 8)  | 
-                    (buffer[(int)index+3] & 0xFF);
-    final int i2 = ((buffer[(int)index+4] & 0xFF) << 24) | 
-                   ((buffer[(int)index+5] & 0xFF) << 16) |
-                   ((buffer[(int)index+6] & 0xFF) << 8)  | 
-                    (buffer[(int)index+7] & 0xFF);
-    return (((long)i1) << 32) | (i2 & 0xFFFFFFFFL);
+    return buffer.getLong((int) index);
   }
   
   private void refill() throws IOException {
-    long start = bufferStart + bufferPosition;
+    long start = bufferStart + buffer.position();
     long end = start + bufferSize;
     if (end > length())  // don't read past EOF
       end = length();
@@ -335,43 +305,44 @@ public abstract class BufferedIndexInput extends IndexInput implements RandomAcc
     if (newLength <= 0)
       throw new EOFException("read past EOF: " + this);
 
-    if (buffer == null) {
-      newBuffer(new byte[bufferSize]);  // allocate buffer lazily
+    if (buffer == EMPTY_BYTEBUFFER) {
+      buffer = ByteBuffer.allocate(bufferSize);  // allocate buffer lazily
       seekInternal(bufferStart);
     }
-    readInternal(buffer, 0, newLength);
-    bufferLength = newLength;
+    buffer.position(0);
+    buffer.limit(newLength);
     bufferStart = start;
-    bufferPosition = 0;
+    readInternal(buffer);
+    // Make sure sub classes don't mess up with the buffer.
+    assert buffer.order() == ByteOrder.BIG_ENDIAN : buffer.order();
+    assert buffer.remaining() == 0 : "should have thrown EOFException";
+    assert buffer.position() == newLength;
+    buffer.flip();
   }
 
   /** Expert: implements buffer refill.  Reads bytes from the current position
    * in the input.
-   * @param b the array to read bytes into
-   * @param offset the offset in the array to start storing bytes
-   * @param length the number of bytes to read
+   * @param b the buffer to read bytes into
    */
-  protected abstract void readInternal(byte[] b, int offset, int length)
-          throws IOException;
+  protected abstract void readInternal(ByteBuffer b) throws IOException;
 
   @Override
-  public final long getFilePointer() { return bufferStart + bufferPosition; }
+  public final long getFilePointer() { return bufferStart + buffer.position(); }
 
   @Override
   public final void seek(long pos) throws IOException {
-    if (pos >= bufferStart && pos < (bufferStart + bufferLength))
-      bufferPosition = (int)(pos - bufferStart);  // seek within buffer
+    if (pos >= bufferStart && pos < (bufferStart + buffer.limit()))
+      buffer.position((int)(pos - bufferStart));  // seek within buffer
     else {
       bufferStart = pos;
-      bufferPosition = 0;
-      bufferLength = 0;  // trigger refill() on read()
+      buffer.limit(0);  // trigger refill() on read
       seekInternal(pos);
     }
   }
 
   /** Expert: implements seek.  Sets current position in this file, where the
-   * next {@link #readInternal(byte[],int,int)} will occur.
-   * @see #readInternal(byte[],int,int)
+   * next {@link #readInternal(ByteBuffer)} will occur.
+   * @see #readInternal(ByteBuffer)
    */
   protected abstract void seekInternal(long pos) throws IOException;
 
@@ -379,9 +350,7 @@ public abstract class BufferedIndexInput extends IndexInput implements RandomAcc
   public BufferedIndexInput clone() {
     BufferedIndexInput clone = (BufferedIndexInput)super.clone();
 
-    clone.buffer = null;
-    clone.bufferLength = 0;
-    clone.bufferPosition = 0;
+    clone.buffer = EMPTY_BYTEBUFFER;
     clone.bufferStart = getFilePointer();
 
     return clone;
@@ -391,27 +360,6 @@ public abstract class BufferedIndexInput extends IndexInput implements RandomAcc
   public IndexInput slice(String sliceDescription, long offset, long length) throws IOException {
     return wrap(sliceDescription, this, offset, length);
   }
-
-  /**
-   * Flushes the in-memory buffer to the given output, copying at most
-   * <code>numBytes</code>.
-   * <p>
-   * <b>NOTE:</b> this method does not refill the buffer, however it does
-   * advance the buffer position.
-   * 
-   * @return the number of bytes actually flushed from the in-memory buffer.
-   */
-  protected final int flushBuffer(IndexOutput out, long numBytes) throws IOException {
-    int toCopy = bufferLength - bufferPosition;
-    if (toCopy > numBytes) {
-      toCopy = (int) numBytes;
-    }
-    if (toCopy > 0) {
-      out.writeBytes(buffer, bufferPosition, toCopy);
-      bufferPosition += toCopy;
-    }
-    return toCopy;
-  }
   
   /**
    * Returns default buffer sizes for the given {@link IOContext}
@@ -461,13 +409,14 @@ public abstract class BufferedIndexInput extends IndexInput implements RandomAcc
     }
     
     @Override
-    protected void readInternal(byte[] b, int offset, int len) throws IOException {
+    protected void readInternal(ByteBuffer b) throws IOException {
       long start = getFilePointer();
-      if (start + len > length) {
+      if (start + b.remaining() > length) {
         throw new EOFException("read past EOF: " + this);
       }
       base.seek(fileOffset + start);
-      base.readBytes(b, offset, len, false);
+      base.readBytes(b.array(), b.position(), b.remaining());
+      b.position(b.position() + b.remaining());
     }
     
     @Override
diff --git a/lucene/core/src/java/org/apache/lucene/store/NIOFSDirectory.java b/lucene/core/src/java/org/apache/lucene/store/NIOFSDirectory.java
index 27a90e8..dd44025 100644
--- a/lucene/core/src/java/org/apache/lucene/store/NIOFSDirectory.java
+++ b/lucene/core/src/java/org/apache/lucene/store/NIOFSDirectory.java
@@ -99,8 +99,6 @@ public class NIOFSDirectory extends FSDirectory {
     protected final long off;
     /** end offset (start+length) */
     protected final long end;
-    
-    private ByteBuffer byteBuf; // wraps the buffer for NIO
 
     public NIOFSIndexInput(String resourceDesc, FileChannel fc, IOContext context) throws IOException {
       super(resourceDesc, context);
@@ -145,40 +143,22 @@ public class NIOFSDirectory extends FSDirectory {
     }
 
     @Override
-    protected void newBuffer(byte[] newBuffer) {
-      super.newBuffer(newBuffer);
-      byteBuf = ByteBuffer.wrap(newBuffer);
-    }
-
-    @Override
-    protected void readInternal(byte[] b, int offset, int len) throws IOException {
-      final ByteBuffer bb;
-
-      // Determine the ByteBuffer we should use
-      if (b == buffer) {
-        // Use our own pre-wrapped byteBuf:
-        assert byteBuf != null;
-        bb = byteBuf;
-        byteBuf.clear().position(offset);
-      } else {
-        bb = ByteBuffer.wrap(b, offset, len);
-      }
-
+    protected void readInternal(ByteBuffer b) throws IOException {
       long pos = getFilePointer() + off;
       
-      if (pos + len > end) {
+      if (pos + b.remaining() > end) {
         throw new EOFException("read past EOF: " + this);
       }
 
       try {
-        int readLength = len;
+        int readLength = b.remaining();
         while (readLength > 0) {
           final int toRead = Math.min(CHUNK_SIZE, readLength);
-          bb.limit(bb.position() + toRead);
-          assert bb.remaining() == toRead;
-          final int i = channel.read(bb, pos);
+          b.limit(b.position() + toRead);
+          assert b.remaining() == toRead;
+          final int i = channel.read(b, pos);
           if (i < 0) { // be defensive here, even though we checked before hand, something could have changed
-            throw new EOFException("read past EOF: " + this + " off: " + offset + " len: " + len + " pos: " + pos + " chunkLen: " + toRead + " end: " + end);
+            throw new EOFException("read past EOF: " + this + " buffer: " + b + " chunkLen: " + toRead + " end: " + end);
           }
           assert i > 0 : "FileChannel.read with non zero-length bb.remaining() must always read at least one byte (FileChannel is in blocking mode, see spec of ReadableByteChannel)";
           pos += i;
diff --git a/lucene/core/src/java/org/apache/lucene/store/SimpleFSDirectory.java b/lucene/core/src/java/org/apache/lucene/store/SimpleFSDirectory.java
index a06a5ba..373e975 100644
--- a/lucene/core/src/java/org/apache/lucene/store/SimpleFSDirectory.java
+++ b/lucene/core/src/java/org/apache/lucene/store/SimpleFSDirectory.java
@@ -143,43 +143,25 @@ public class SimpleFSDirectory extends FSDirectory {
     }
 
     @Override
-    protected void newBuffer(byte[] newBuffer) {
-      super.newBuffer(newBuffer);
-      byteBuf = ByteBuffer.wrap(newBuffer);
-    }
-
-    @Override
-    protected void readInternal(byte[] b, int offset, int len) throws IOException {
-      final ByteBuffer bb;
-
-      // Determine the ByteBuffer we should use
-      if (b == buffer) {
-        // Use our own pre-wrapped byteBuf:
-        assert byteBuf != null;
-        bb = byteBuf;
-        byteBuf.clear().position(offset);
-      } else {
-        bb = ByteBuffer.wrap(b, offset, len);
-      }
-
+    protected void readInternal(ByteBuffer b) throws IOException {
       synchronized(channel) {
         long pos = getFilePointer() + off;
         
-        if (pos + len > end) {
+        if (pos + b.remaining() > end) {
           throw new EOFException("read past EOF: " + this);
         }
                
         try {
           channel.position(pos);
 
-          int readLength = len;
+          int readLength = b.remaining();
           while (readLength > 0) {
             final int toRead = Math.min(CHUNK_SIZE, readLength);
-            bb.limit(bb.position() + toRead);
-            assert bb.remaining() == toRead;
-            final int i = channel.read(bb);
+            b.limit(b.position() + toRead);
+            assert b.remaining() == toRead;
+            final int i = channel.read(b);
             if (i < 0) { // be defensive here, even though we checked before hand, something could have changed
-              throw new EOFException("read past EOF: " + this + " off: " + offset + " len: " + len + " pos: " + pos + " chunkLen: " + toRead + " end: " + end);
+              throw new EOFException("read past EOF: " + this + " buffer: " + b + " chunkLen: " + toRead + " end: " + end);
             }
             assert i > 0 : "SeekableByteChannel.read with non zero-length bb.remaining() must always read at least one byte (Channel is in blocking mode, see spec of ReadableByteChannel)";
             pos += i;
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestFieldsReader.java b/lucene/core/src/test/org/apache/lucene/index/TestFieldsReader.java
index ce24b7f..b34cfe5 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestFieldsReader.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestFieldsReader.java
@@ -18,6 +18,7 @@ package org.apache.lucene.index;
 
 
 import java.io.IOException;
+import java.nio.ByteBuffer;
 import java.nio.file.Path;
 import java.util.*;
 import java.util.concurrent.atomic.AtomicBoolean;
@@ -144,10 +145,11 @@ public class TestFieldsReader extends LuceneTestCase {
     }
 
     @Override
-    public void readInternal(byte[] b, int offset, int length) throws IOException {
+    public void readInternal(ByteBuffer b) throws IOException {
       simOutage();
       delegate.seek(getFilePointer());
-      delegate.readBytes(b, offset, length);
+      delegate.readBytes(b.array(), b.position(), b.remaining());
+      b.position(b.limit());
     }
     
     @Override
diff --git a/lucene/core/src/test/org/apache/lucene/store/TestBufferedIndexInput.java b/lucene/core/src/test/org/apache/lucene/store/TestBufferedIndexInput.java
index 87762df..4ea1b2e 100644
--- a/lucene/core/src/test/org/apache/lucene/store/TestBufferedIndexInput.java
+++ b/lucene/core/src/test/org/apache/lucene/store/TestBufferedIndexInput.java
@@ -18,6 +18,7 @@ package org.apache.lucene.store;
 
 
 import java.io.IOException;
+import java.nio.ByteBuffer;
 import java.nio.file.Path;
 import java.util.ArrayList;
 import java.util.List;
@@ -61,14 +62,6 @@ public class TestBufferedIndexInput extends LuceneTestCase {
     MyBufferedIndexInput input = new MyBufferedIndexInput();
     runReadBytes(input, BufferedIndexInput.BUFFER_SIZE, random());
   }
-
-  private void runReadBytesAndClose(IndexInput input, int bufferSize, Random r) throws IOException {
-    try {
-      runReadBytes(input, bufferSize, r);
-    } finally {
-      input.close();
-    }
-  }
   
   private void runReadBytes(IndexInput input, int bufferSize, Random r)
       throws IOException {
@@ -180,9 +173,10 @@ public class TestBufferedIndexInput extends LuceneTestCase {
         this(Long.MAX_VALUE);
       }
       @Override
-      protected void readInternal(byte[] b, int offset, int length) throws IOException {
-        for(int i=offset; i<offset+length; i++)
-          b[i] = byten(pos++);
+      protected void readInternal(ByteBuffer b) throws IOException {
+        while (b.hasRemaining()) {
+          b.put(byten(pos++));
+        }
       }
 
       @Override
diff --git a/lucene/misc/src/java/org/apache/lucene/store/RAFDirectory.java b/lucene/misc/src/java/org/apache/lucene/store/RAFDirectory.java
index 96963ae..abc523d 100644
--- a/lucene/misc/src/java/org/apache/lucene/store/RAFDirectory.java
+++ b/lucene/misc/src/java/org/apache/lucene/store/RAFDirectory.java
@@ -20,6 +20,7 @@ import java.io.EOFException;
 import java.io.File;
 import java.io.IOException;
 import java.io.RandomAccessFile;
+import java.nio.ByteBuffer;
 import java.nio.file.Path;
 
 import org.apache.lucene.util.SuppressForbidden;
@@ -136,28 +137,25 @@ public class RAFDirectory extends FSDirectory {
   
     /** IndexInput methods */
     @Override
-    protected void readInternal(byte[] b, int offset, int len)
-         throws IOException {
+    protected void readInternal(ByteBuffer b) throws IOException {
       synchronized (file) {
         long position = off + getFilePointer();
         file.seek(position);
-        int total = 0;
 
-        if (position + len > end) {
+        if (position + b.remaining() > end) {
           throw new EOFException("read past EOF: " + this);
         }
 
         try {
-          while (total < len) {
-            final int toRead = Math.min(CHUNK_SIZE, len - total);
-            final int i = file.read(b, offset + total, toRead);
+          while (b.hasRemaining()) {
+            final int toRead = Math.min(CHUNK_SIZE, b.remaining());
+            final int i = file.read(b.array(), b.position(), toRead);
             if (i < 0) { // be defensive here, even though we checked before hand, something could have changed
-             throw new EOFException("read past EOF: " + this + " off: " + offset + " len: " + len + " total: " + total + " chunkLen: " + toRead + " end: " + end);
+             throw new EOFException("read past EOF: " + this + " off: " + b.position() + " len: " + b.remaining() + " chunkLen: " + toRead + " end: " + end);
             }
             assert i > 0 : "RandomAccessFile.read with non zero-length toRead must always read at least one byte";
-            total += i;
+            b.position(b.position() + i);
           }
-          assert total == len;
         } catch (IOException ioe) {
           throw new IOException(ioe.getMessage() + ": " + this, ioe);
         }
diff --git a/lucene/misc/src/java/org/apache/lucene/store/WindowsDirectory.java b/lucene/misc/src/java/org/apache/lucene/store/WindowsDirectory.java
index ff4c15f..43b8e12 100644
--- a/lucene/misc/src/java/org/apache/lucene/store/WindowsDirectory.java
+++ b/lucene/misc/src/java/org/apache/lucene/store/WindowsDirectory.java
@@ -18,6 +18,7 @@ package org.apache.lucene.store;
 
 import java.io.IOException;
 import java.io.EOFException;
+import java.nio.ByteBuffer;
 import java.nio.file.Path;
 
 /**
@@ -83,10 +84,10 @@ public class WindowsDirectory extends FSDirectory {
     }
     
     @Override
-    protected void readInternal(byte[] b, int offset, int length) throws IOException {
+    protected void readInternal(ByteBuffer b) throws IOException {
       int bytesRead;
       try {
-        bytesRead = WindowsDirectory.read(fd, b, offset, length, getFilePointer());
+        bytesRead = WindowsDirectory.read(fd, b.array(), b.position(), b.remaining(), getFilePointer());
       } catch (IOException ioe) {
         throw new IOException(ioe.getMessage() + ": " + this, ioe);
       }