|
@@ -22,7 +22,6 @@ import java.io.EOFException;
|
|
|
import java.io.IOException;
|
|
|
import java.nio.ByteBuffer;
|
|
|
import java.util.ArrayList;
|
|
|
-import java.util.Arrays;
|
|
|
import java.util.List;
|
|
|
import java.util.concurrent.CompletableFuture;
|
|
|
import java.util.concurrent.CountDownLatch;
|
|
@@ -30,6 +29,7 @@ import java.util.concurrent.ExecutorService;
|
|
|
import java.util.concurrent.Executors;
|
|
|
import java.util.concurrent.TimeUnit;
|
|
|
import java.util.concurrent.TimeoutException;
|
|
|
+import java.util.concurrent.atomic.AtomicInteger;
|
|
|
import java.util.function.IntFunction;
|
|
|
|
|
|
import org.assertj.core.api.Assertions;
|
|
@@ -47,8 +47,8 @@ import org.apache.hadoop.fs.Path;
|
|
|
import org.apache.hadoop.io.ElasticByteBufferPool;
|
|
|
import org.apache.hadoop.io.WeakReferencedElasticByteBufferPool;
|
|
|
import org.apache.hadoop.util.concurrent.HadoopExecutors;
|
|
|
-import org.apache.hadoop.util.functional.FutureIO;
|
|
|
|
|
|
+import static java.util.Arrays.asList;
|
|
|
import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_LENGTH;
|
|
|
import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICY;
|
|
|
import static org.apache.hadoop.fs.Options.OpenFileOptions.FS_OPTION_OPENFILE_READ_POLICY_VECTOR;
|
|
@@ -58,10 +58,16 @@ import static org.apache.hadoop.fs.contract.ContractTestUtils.createFile;
|
|
|
import static org.apache.hadoop.fs.contract.ContractTestUtils.range;
|
|
|
import static org.apache.hadoop.fs.contract.ContractTestUtils.returnBuffersToPoolPostRead;
|
|
|
import static org.apache.hadoop.fs.contract.ContractTestUtils.validateVectoredReadResult;
|
|
|
-import static org.apache.hadoop.test.LambdaTestUtils.intercept;
|
|
|
+ import static org.apache.hadoop.test.LambdaTestUtils.intercept;
|
|
|
import static org.apache.hadoop.test.LambdaTestUtils.interceptFuture;
|
|
|
import static org.apache.hadoop.util.functional.FutureIO.awaitFuture;
|
|
|
|
|
|
+/**
|
|
|
+ * Test Vectored Reads.
|
|
|
+ * <p>
|
|
|
+ * Both the original readVectored(allocator) and the readVectored(allocator, release)
|
|
|
+ * operations are tested.
|
|
|
+ */
|
|
|
@RunWith(Parameterized.class)
|
|
|
public abstract class AbstractContractVectoredReadTest extends AbstractFSContractTestBase {
|
|
|
|
|
@@ -90,12 +96,19 @@ public abstract class AbstractContractVectoredReadTest extends AbstractFSContrac
|
|
|
*/
|
|
|
private Path vectorPath;
|
|
|
|
|
|
+ /**
|
|
|
+ * Counter of buffer releases.
|
|
|
+ * Because not all implementations release buffers on failures,
|
|
|
+ * this is not yet used in assertions.
|
|
|
+ */
|
|
|
+ private final AtomicInteger bufferReleases = new AtomicInteger();
|
|
|
+
|
|
|
@Parameterized.Parameters(name = "Buffer type : {0}")
|
|
|
public static List<String> params() {
|
|
|
- return Arrays.asList("direct", "array");
|
|
|
+ return asList("direct", "array");
|
|
|
}
|
|
|
|
|
|
- public AbstractContractVectoredReadTest(String bufferType) {
|
|
|
+ protected AbstractContractVectoredReadTest(String bufferType) {
|
|
|
this.bufferType = bufferType;
|
|
|
final boolean isDirect = !"array".equals(bufferType);
|
|
|
this.allocate = size -> pool.getBuffer(isDirect, size);
|
|
@@ -109,6 +122,15 @@ public abstract class AbstractContractVectoredReadTest extends AbstractFSContrac
|
|
|
return allocate;
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * The buffer release operation.
|
|
|
+ */
|
|
|
+ protected void release(ByteBuffer buffer) {
|
|
|
+ LOG.info("Released buffer {}", buffer);
|
|
|
+ bufferReleases.incrementAndGet();
|
|
|
+ pool.putBuffer(buffer);
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* Get the vector IO buffer pool.
|
|
|
* @return a pool.
|
|
@@ -164,7 +186,7 @@ public abstract class AbstractContractVectoredReadTest extends AbstractFSContrac
|
|
|
fileRanges.add(fileRange);
|
|
|
}
|
|
|
try (FSDataInputStream in = openVectorFile()) {
|
|
|
- in.readVectored(fileRanges, allocate);
|
|
|
+ in.readVectored(fileRanges, allocate, this::release);
|
|
|
CompletableFuture<?>[] completableFutures = new CompletableFuture<?>[fileRanges.size()];
|
|
|
int i = 0;
|
|
|
for (FileRange res : fileRanges) {
|
|
@@ -186,7 +208,7 @@ public abstract class AbstractContractVectoredReadTest extends AbstractFSContrac
|
|
|
in.readVectored(fileRanges, allocate);
|
|
|
byte[] readFullRes = new byte[100];
|
|
|
in.readFully(100, readFullRes);
|
|
|
- ByteBuffer vecRes = FutureIO.awaitFuture(fileRanges.get(0).getData());
|
|
|
+ ByteBuffer vecRes = awaitFuture(fileRanges.get(0).getData());
|
|
|
Assertions.assertThat(vecRes)
|
|
|
.describedAs("Result from vectored read and readFully must match")
|
|
|
.isEqualByComparingTo(ByteBuffer.wrap(readFullRes));
|
|
@@ -201,7 +223,7 @@ public abstract class AbstractContractVectoredReadTest extends AbstractFSContrac
|
|
|
range(fileRanges, 0, DATASET_LEN);
|
|
|
try (FSDataInputStream in = openVectorFile()) {
|
|
|
in.readVectored(fileRanges, allocate);
|
|
|
- ByteBuffer vecRes = FutureIO.awaitFuture(fileRanges.get(0).getData());
|
|
|
+ ByteBuffer vecRes = awaitFuture(fileRanges.get(0).getData());
|
|
|
Assertions.assertThat(vecRes)
|
|
|
.describedAs("Result from vectored read and readFully must match")
|
|
|
.isEqualByComparingTo(ByteBuffer.wrap(DATASET));
|
|
@@ -220,7 +242,7 @@ public abstract class AbstractContractVectoredReadTest extends AbstractFSContrac
|
|
|
range(fileRanges, 4_000 + 101, 100);
|
|
|
range(fileRanges, 16_000 + 101, 100);
|
|
|
try (FSDataInputStream in = openVectorFile()) {
|
|
|
- in.readVectored(fileRanges, allocate);
|
|
|
+ in.readVectored(fileRanges, allocate, this::release);
|
|
|
validateVectoredReadResult(fileRanges, DATASET, 0);
|
|
|
returnBuffersToPoolPostRead(fileRanges, pool);
|
|
|
}
|
|
@@ -263,7 +285,7 @@ public abstract class AbstractContractVectoredReadTest extends AbstractFSContrac
|
|
|
.withFileStatus(fileStatus)
|
|
|
.build();
|
|
|
try (FSDataInputStream in = builder.get()) {
|
|
|
- in.readVectored(fileRanges, allocate);
|
|
|
+ in.readVectored(fileRanges, allocate, this::release);
|
|
|
validateVectoredReadResult(fileRanges, DATASET, 0);
|
|
|
returnBuffersToPoolPostRead(fileRanges, pool);
|
|
|
}
|
|
@@ -301,7 +323,7 @@ public abstract class AbstractContractVectoredReadTest extends AbstractFSContrac
|
|
|
} else {
|
|
|
try (FSDataInputStream in = openVectorFile()) {
|
|
|
List<FileRange> fileRanges = getSampleSameRanges();
|
|
|
- in.readVectored(fileRanges, allocate);
|
|
|
+ in.readVectored(fileRanges, allocate, this::release);
|
|
|
validateVectoredReadResult(fileRanges, DATASET, 0);
|
|
|
returnBuffersToPoolPostRead(fileRanges, pool);
|
|
|
}
|
|
@@ -352,7 +374,7 @@ public abstract class AbstractContractVectoredReadTest extends AbstractFSContrac
|
|
|
range(fileRanges, offset, length);
|
|
|
range(fileRanges, offset + length, length);
|
|
|
try (FSDataInputStream in = openVectorFile()) {
|
|
|
- in.readVectored(fileRanges, allocate);
|
|
|
+ in.readVectored(fileRanges, allocate, this::release);
|
|
|
validateVectoredReadResult(fileRanges, DATASET, 0);
|
|
|
returnBuffersToPoolPostRead(fileRanges, pool);
|
|
|
}
|
|
@@ -408,12 +430,12 @@ public abstract class AbstractContractVectoredReadTest extends AbstractFSContrac
|
|
|
private void expectEOFinRead(final List<FileRange> fileRanges) throws Exception {
|
|
|
LOG.info("Expecting late EOF failure");
|
|
|
try (FSDataInputStream in = openVectorFile()) {
|
|
|
- in.readVectored(fileRanges, allocate);
|
|
|
+ in.readVectored(fileRanges, allocate, this::release);
|
|
|
for (FileRange res : fileRanges) {
|
|
|
CompletableFuture<ByteBuffer> data = res.getData();
|
|
|
interceptFuture(EOFException.class,
|
|
|
"",
|
|
|
- ContractTestUtils.VECTORED_READ_OPERATION_TEST_TIMEOUT_SECONDS,
|
|
|
+ VECTORED_READ_OPERATION_TEST_TIMEOUT_SECONDS,
|
|
|
TimeUnit.SECONDS,
|
|
|
data);
|
|
|
}
|
|
@@ -431,6 +453,17 @@ public abstract class AbstractContractVectoredReadTest extends AbstractFSContrac
|
|
|
verifyExceptionalVectoredRead(range(-1, 50), EOFException.class);
|
|
|
}
|
|
|
|
|
|
+ @Test
|
|
|
+ public void testNullReleaseOperation() throws Exception {
|
|
|
+
|
|
|
+ final List<FileRange> range = range(0, 10);
|
|
|
+
|
|
|
+ try (FSDataInputStream in = openVectorFile()) {
|
|
|
+ intercept(NullPointerException.class, () ->
|
|
|
+ in.readVectored(range, allocate, null));
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
@Test
|
|
|
public void testNormalReadAfterVectoredRead() throws Exception {
|
|
|
List<FileRange> fileRanges = createSampleNonOverlappingRanges();
|
|
@@ -469,7 +502,7 @@ public abstract class AbstractContractVectoredReadTest extends AbstractFSContrac
|
|
|
List<FileRange> fileRanges2 = createSampleNonOverlappingRanges();
|
|
|
try (FSDataInputStream in = openVectorFile()) {
|
|
|
in.readVectored(fileRanges1, allocate);
|
|
|
- in.readVectored(fileRanges2, allocate);
|
|
|
+ in.readVectored(fileRanges2, allocate, this::release);
|
|
|
validateVectoredReadResult(fileRanges2, DATASET, 0);
|
|
|
validateVectoredReadResult(fileRanges1, DATASET, 0);
|
|
|
returnBuffersToPoolPostRead(fileRanges1, pool);
|
|
@@ -524,7 +557,7 @@ public abstract class AbstractContractVectoredReadTest extends AbstractFSContrac
|
|
|
CompletableFuture<ByteBuffer> data = res.getData();
|
|
|
// Read the data and perform custom operation. Here we are just
|
|
|
// validating it with original data.
|
|
|
- FutureIO.awaitFuture(data.thenAccept(buffer -> {
|
|
|
+ awaitFuture(data.thenAccept(buffer -> {
|
|
|
assertDatasetEquals((int) res.getOffset(),
|
|
|
"vecRead", buffer, res.getLength(), DATASET);
|
|
|
// return buffer to the pool once read.
|