[HUDI-786] Fixing read beyond inline length in InlineFS (#1616)
This commit is contained in:
committed by
GitHub
parent
3574a89232
commit
5a0d3f1cf9
@@ -37,15 +37,19 @@ public class InLineFsDataInputStream extends FSDataInputStream {
|
||||
private final FSDataInputStream outerStream;
|
||||
private final int length;
|
||||
|
||||
public InLineFsDataInputStream(int startOffset, FSDataInputStream outerStream, int length) {
|
||||
public InLineFsDataInputStream(int startOffset, FSDataInputStream outerStream, int length) throws IOException {
|
||||
super(outerStream.getWrappedStream());
|
||||
this.startOffset = startOffset;
|
||||
this.outerStream = outerStream;
|
||||
this.length = length;
|
||||
outerStream.seek(startOffset);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void seek(long desired) throws IOException {
|
||||
if (desired > length) {
|
||||
throw new IOException("Attempting to seek past inline content");
|
||||
}
|
||||
outerStream.seek(startOffset + desired);
|
||||
}
|
||||
|
||||
@@ -56,24 +60,32 @@ public class InLineFsDataInputStream extends FSDataInputStream {
|
||||
|
||||
@Override
|
||||
public int read(long position, byte[] buffer, int offset, int length) throws IOException {
|
||||
if ((length + offset) > this.length) {
|
||||
throw new IOException("Attempting to read past inline content");
|
||||
}
|
||||
return outerStream.read(startOffset + position, buffer, offset, length);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void readFully(long position, byte[] buffer, int offset, int length) throws IOException {
|
||||
if ((length + offset) > this.length) {
|
||||
throw new IOException("Attempting to read past inline content");
|
||||
}
|
||||
outerStream.readFully(startOffset + position, buffer, offset, length);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void readFully(long position, byte[] buffer)
|
||||
throws IOException {
|
||||
outerStream.readFully(startOffset + position, buffer, 0, buffer.length);
|
||||
readFully(position, buffer, 0, buffer.length);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean seekToNewSource(long targetPos) throws IOException {
|
||||
boolean toReturn = outerStream.seekToNewSource(startOffset + targetPos);
|
||||
return toReturn;
|
||||
if (targetPos > this.length) {
|
||||
throw new IOException("Attempting to seek past inline content");
|
||||
}
|
||||
return outerStream.seekToNewSource(startOffset + targetPos);
|
||||
}
|
||||
|
||||
@Override
|
||||
@@ -88,6 +100,9 @@ public class InLineFsDataInputStream extends FSDataInputStream {
|
||||
|
||||
@Override
|
||||
public void setReadahead(Long readahead) throws IOException, UnsupportedOperationException {
|
||||
if (readahead > this.length) {
|
||||
throw new IOException("Attempting to set read ahead past inline content");
|
||||
}
|
||||
outerStream.setReadahead(readahead);
|
||||
}
|
||||
|
||||
@@ -99,6 +114,9 @@ public class InLineFsDataInputStream extends FSDataInputStream {
|
||||
@Override
|
||||
public ByteBuffer read(ByteBufferPool bufferPool, int maxLength, EnumSet<ReadOption> opts)
|
||||
throws IOException, UnsupportedOperationException {
|
||||
if (maxLength > this.length) {
|
||||
throw new IOException("Attempting to read max length beyond inline content");
|
||||
}
|
||||
return outerStream.read(bufferPool, maxLength, opts);
|
||||
}
|
||||
|
||||
|
||||
Reference in New Issue
Block a user