Skip to content
Merged
Show file tree
Hide file tree
Changes from 3 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@
package org.apache.hadoop.fs.ozone;

import java.io.BufferedInputStream;
import java.io.EOFException;
import java.io.File;
import java.io.FileInputStream;
import java.io.IOException;
Expand Down Expand Up @@ -58,6 +59,7 @@
import static org.apache.hadoop.hdds.StringUtils.string2Bytes;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertArrayEquals;
import static org.junit.jupiter.api.Assertions.assertThrows;

/**
* Test OzoneFSInputStream by reading through multiple interfaces.
Expand Down Expand Up @@ -162,6 +164,76 @@ public void testO3FSSingleByteRead() throws IOException {
}
}

@Test
public void testByteBufferPositionedRead() throws IOException {
try (FSDataInputStream inputStream = fs.open(filePath)) {
ByteBuffer buffer = ByteBuffer.allocate(20);
// Read positional from 50th index
int readBytes = inputStream.read(50, buffer);
byte[] value1 = new byte[readBytes];
System.arraycopy(buffer.array(), 0, value1, 0, readBytes);
byte[] value2 = new byte[readBytes];
System.arraycopy(data, 50, value2, 0, readBytes);
// Verify input and positional read data
assertArrayEquals(value1, value2, "value mismatch");

buffer.clear();
// Verify offset of file didn't change
// Read positional from 8th index again using same inputStream
readBytes = inputStream.read(8, buffer);
byte[] value3 = new byte[readBytes];
System.arraycopy(buffer.array(), 0, value3, 0, readBytes);
byte[] value4 = new byte[readBytes];
System.arraycopy(data, 8, value4, 0, readBytes);
// Verify input and positional read data
assertArrayEquals(value3, value4, "value mismatch");

// Buffer size more than actual data, still read should succeed
ByteBuffer buffer1 = ByteBuffer.allocate(30 * 1024 * 1024 * 2);
readBytes = inputStream.read(12, buffer1);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can you add readBytes check, to make sure the bytes read value is as expected?

Copy link
Contributor

@ChenSammi ChenSammi Mar 14, 2024

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Also can you add

  1. file position check, make sure file position is not changed after read or readFully.
  2. boundary check, such as illegal position value.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Done

byte[] value5 = new byte[readBytes];
System.arraycopy(buffer1.array(), 0, value5, 0, readBytes);
byte[] value6 = new byte[readBytes];
System.arraycopy(data, 12, value6, 0, readBytes);
// Verify input and positional read data
assertArrayEquals(value5, value6, "value mismatch");
}
}


@Test
public void testByteBufferPositionedReadFully() throws IOException {
try (FSDataInputStream inputStream = fs.open(filePath)) {
int bufferCapacity = 20;
ByteBuffer buffer = ByteBuffer.allocate(bufferCapacity);
// Read positional from 50th index
inputStream.readFully(50, buffer);
byte[] value1 = new byte[bufferCapacity];
System.arraycopy(buffer.array(), 0, value1, 0, bufferCapacity);
byte[] value2 = new byte[bufferCapacity];
System.arraycopy(data, 50, value2, 0, bufferCapacity);
// Verify input and positional read data
assertArrayEquals(value1, value2, "value mismatch");

buffer.clear();
// Verify offset of file didn't change
// Read positional from 8th index again using same inputStream
inputStream.readFully(8, buffer);
byte[] value3 = new byte[bufferCapacity];
System.arraycopy(buffer.array(), 0, value3, 0, bufferCapacity);
byte[] value4 = new byte[bufferCapacity];
System.arraycopy(data, 8, value4, 0, bufferCapacity);
// Verify input and positional read data
assertArrayEquals(value3, value4, "value mismatch");

// Buffer size is more than actual data, readFully should fail in this case
ByteBuffer buffer1 = ByteBuffer.allocate(30 * 1024 * 1024 * 2);
assertThrows(EOFException.class, () -> inputStream.readFully(12, buffer1));
}
}



@Test
public void testO3FSMultiByteRead() throws IOException {
try (FSDataInputStream inputStream = fs.open(filePath)) {
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,92 @@
/**
* 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.hadoop.fs;

import java.io.EOFException;
import java.io.IOException;
import java.nio.ByteBuffer;

import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;

/**
* FIXME: Hack: This is copied from Hadoop 3.3.6. Remove this interface once
* we drop Hadoop 3.1, 3.2 support.
* Implementers of this interface provide a positioned read API that writes to a
* {@link ByteBuffer} rather than a {@code byte[]}.
*
* @see PositionedReadable
* @see ByteBufferReadable
*/
@InterfaceAudience.Public
@InterfaceStability.Evolving
public interface ByteBufferPositionedReadable {
/**
* Reads up to {@code buf.remaining()} bytes into buf from a given position
* in the file and returns the number of bytes read. Callers should use
* {@code buf.limit(...)} to control the size of the desired read and
* {@code buf.position(...)} to control the offset into the buffer the data
* should be written to.
* <p>
* After a successful call, {@code buf.position()} will be advanced by the
* number of bytes read and {@code buf.limit()} will be unchanged.
* <p>
* In the case of an exception, the state of the buffer (the contents of the
* buffer, the {@code buf.position()}, the {@code buf.limit()}, etc.) is
* undefined, and callers should be prepared to recover from this
* eventuality.
* <p>
* Callers should use {@link StreamCapabilities#hasCapability(String)} with
* {@link StreamCapabilities#PREADBYTEBUFFER} to check if the underlying
* stream supports this interface, otherwise they might get a
* {@link UnsupportedOperationException}.
* <p>
* Implementations should treat 0-length requests as legitimate, and must not
* signal an error upon their receipt.
* <p>
* This does not change the current offset of a file, and is thread-safe.
*
* @param position position within file
* @param buf the ByteBuffer to receive the results of the read operation.
* @return the number of bytes read, possibly zero, or -1 if reached
* end-of-stream
* @throws IOException if there is some error performing the read
*/
int read(long position, ByteBuffer buf) throws IOException;

/**
* Reads {@code buf.remaining()} bytes into buf from a given position in
* the file or until the end of the data was reached before the read
* operation completed. Callers should use {@code buf.limit(...)} to
* control the size of the desired read and {@code buf.position(...)} to
* control the offset into the buffer the data should be written to.
* <p>
* This operation provides similar semantics to
* {@link #read(long, ByteBuffer)}, the difference is that this method is
* guaranteed to read data until the {@link ByteBuffer} is full, or until
* the end of the data stream is reached.
*
* @param position position within file
* @param buf the ByteBuffer to receive the results of the read operation.
* @throws IOException if there is some error performing the read
* @throws EOFException the end of the data was reached before
* the read operation completed
* @see #read(long, ByteBuffer)
*/
void readFully(long position, ByteBuffer buf) throws IOException;
}
Copy link
Contributor

@ChenSammi ChenSammi Mar 14, 2024

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We will use the ByteBufferPositionedReadable interface defined in hadoop-common directly. This file can be removed. Is there dependency problem for this?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The problem is that some of the acceptance tests run with Hadoop 3.2 runtime which does not have this API.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This interface is added in Hadoop 3.3, so the older hadoop version client fails to load ozone fs jar. This file can be removed when we drop support for older hadoop version.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I see. Oh, there are comments explaining this.

Original file line number Diff line number Diff line change
Expand Up @@ -35,6 +35,7 @@ public boolean hasCapability(String capability) {
switch (StringUtils.toLowerCase(capability)) {
case StreamCapabilities.READBYTEBUFFER:
case StreamCapabilities.UNBUFFER:
case StreamCapabilities.PREADBYTEBUFFER:
return true;
default:
return false;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@

package org.apache.hadoop.fs.ozone;

import java.io.EOFException;
import java.io.IOException;
import java.io.InputStream;
import java.nio.ByteBuffer;
Expand All @@ -27,6 +28,7 @@
import org.apache.hadoop.hdds.annotation.InterfaceAudience;
import org.apache.hadoop.hdds.annotation.InterfaceStability;
import org.apache.hadoop.fs.ByteBufferReadable;
import org.apache.hadoop.fs.ByteBufferPositionedReadable;
import org.apache.hadoop.fs.FSInputStream;
import org.apache.hadoop.fs.FileSystem.Statistics;
import org.apache.hadoop.fs.Seekable;
Expand All @@ -40,7 +42,7 @@
@InterfaceAudience.Private
@InterfaceStability.Evolving
public class OzoneFSInputStream extends FSInputStream
implements ByteBufferReadable, CanUnbuffer {
implements ByteBufferReadable, CanUnbuffer, ByteBufferPositionedReadable {

private final InputStream inputStream;
private final Statistics statistics;
Expand Down Expand Up @@ -137,4 +139,49 @@ public void unbuffer() {
((CanUnbuffer) inputStream).unbuffer();
}
}

/**
* @param buf the ByteBuffer to receive the results of the read operation.
* @param position offset
* @return the number of bytes read, possibly zero, or -1 if
* reach end-of-stream
* @throws IOException if there is some error performing the read
*/
@Override
public int read(long position, ByteBuffer buf) throws IOException {
if (!buf.hasRemaining()) {
return 0;
}
long oldPos = this.getPos();
int bytesRead;
try {
((Seekable) inputStream).seek(position);
bytesRead = ((ByteBufferReadable) inputStream).read(buf);
} catch (EOFException e) {
// Either position is negative or it has reached EOF
return -1;
} finally {
((Seekable) inputStream).seek(oldPos);
}
return bytesRead;
}

/**
* @param buf the ByteBuffer to receive the results of the read operation.
* @param position offset
* @return void
* @throws IOException if there is some error performing the read
* @throws EOFException if end of file reched before reading fully
*/
@Override
public void readFully(long position, ByteBuffer buf) throws IOException {
int bytesRead;
for (int readCount = 0; buf.hasRemaining(); readCount += bytesRead) {
bytesRead = this.read(position + (long)readCount, buf);
if (bytesRead < 0) {
// Still buffer had space to read but stream has already reached EOF
throw new EOFException("End of file reached before reading fully.");
}
}
}
}