mirror of https://github.com/apache/lucene.git
Fixed a bug where RAMInputStream could not read across more than
across a single buffer boundary. git-svn-id: https://svn.apache.org/repos/asf/lucene/java/trunk@149615 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
b91b113e86
commit
992d1ade00
|
@ -160,21 +160,22 @@ final class RAMInputStream extends InputStream implements Cloneable {
|
||||||
|
|
||||||
/** InputStream methods */
|
/** InputStream methods */
|
||||||
public final void readInternal(byte[] dest, int destOffset, int len) {
|
public final void readInternal(byte[] dest, int destOffset, int len) {
|
||||||
int bufferNumber = pointer/InputStream.BUFFER_SIZE;
|
int remainder = len;
|
||||||
int bufferOffset = pointer%InputStream.BUFFER_SIZE;
|
int start = pointer;
|
||||||
int bytesInBuffer = InputStream.BUFFER_SIZE - bufferOffset;
|
while (remainder != 0) {
|
||||||
int bytesToCopy = bytesInBuffer >= len ? len : bytesInBuffer;
|
int bufferNumber = start/InputStream.BUFFER_SIZE;
|
||||||
byte[] buffer = (byte[])file.buffers.elementAt(bufferNumber);
|
int bufferOffset = start%InputStream.BUFFER_SIZE;
|
||||||
System.arraycopy(buffer, bufferOffset, dest, destOffset, bytesToCopy);
|
int bytesInBuffer = InputStream.BUFFER_SIZE - bufferOffset;
|
||||||
|
int bytesToCopy = bytesInBuffer >= remainder ? remainder : bytesInBuffer;
|
||||||
if (bytesToCopy < len) { // not all in one buffer
|
byte[] buffer = (byte[])file.buffers.elementAt(bufferNumber);
|
||||||
|
System.arraycopy(buffer, bufferOffset, dest, destOffset, bytesToCopy);
|
||||||
destOffset += bytesToCopy;
|
destOffset += bytesToCopy;
|
||||||
bytesToCopy = len - bytesToCopy; // remaining bytes
|
start += bytesToCopy;
|
||||||
buffer = (byte[])file.buffers.elementAt(bufferNumber+1);
|
remainder -= bytesToCopy;
|
||||||
System.arraycopy(buffer, 0, dest, destOffset, bytesToCopy);
|
|
||||||
}
|
}
|
||||||
pointer += len;
|
pointer += len;
|
||||||
}
|
}
|
||||||
|
|
||||||
public final void close() {
|
public final void close() {
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
Loading…
Reference in New Issue