HADOOP-18392. Propagate vectored s3a input stream stats to file system stats. (#4704)
part of HADOOP-18103. Contributed By: Mukund Thakur
This commit is contained in:
parent
09c8084191
commit
93c4704b33
|
@ -24,11 +24,10 @@ import java.util.ArrayList;
|
||||||
import java.util.Arrays;
|
import java.util.Arrays;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.concurrent.CompletableFuture;
|
import java.util.concurrent.CompletableFuture;
|
||||||
import java.util.concurrent.ExecutionException;
|
import java.util.concurrent.TimeUnit;
|
||||||
import java.util.function.IntFunction;
|
import java.util.function.IntFunction;
|
||||||
|
|
||||||
import org.assertj.core.api.Assertions;
|
import org.assertj.core.api.Assertions;
|
||||||
import org.junit.Assert;
|
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
import org.junit.runner.RunWith;
|
import org.junit.runner.RunWith;
|
||||||
import org.junit.runners.Parameterized;
|
import org.junit.runners.Parameterized;
|
||||||
|
@ -43,13 +42,14 @@ import org.apache.hadoop.fs.FileSystem;
|
||||||
import org.apache.hadoop.fs.Path;
|
import org.apache.hadoop.fs.Path;
|
||||||
import org.apache.hadoop.fs.impl.FutureIOSupport;
|
import org.apache.hadoop.fs.impl.FutureIOSupport;
|
||||||
import org.apache.hadoop.io.WeakReferencedElasticByteBufferPool;
|
import org.apache.hadoop.io.WeakReferencedElasticByteBufferPool;
|
||||||
import org.apache.hadoop.test.LambdaTestUtils;
|
|
||||||
|
|
||||||
import static org.apache.hadoop.fs.contract.ContractTestUtils.assertCapabilities;
|
import static org.apache.hadoop.fs.contract.ContractTestUtils.assertCapabilities;
|
||||||
import static org.apache.hadoop.fs.contract.ContractTestUtils.assertDatasetEquals;
|
import static org.apache.hadoop.fs.contract.ContractTestUtils.assertDatasetEquals;
|
||||||
import static org.apache.hadoop.fs.contract.ContractTestUtils.createFile;
|
import static org.apache.hadoop.fs.contract.ContractTestUtils.createFile;
|
||||||
import static org.apache.hadoop.fs.contract.ContractTestUtils.returnBuffersToPoolPostRead;
|
import static org.apache.hadoop.fs.contract.ContractTestUtils.returnBuffersToPoolPostRead;
|
||||||
import static org.apache.hadoop.fs.contract.ContractTestUtils.validateVectoredReadResult;
|
import static org.apache.hadoop.fs.contract.ContractTestUtils.validateVectoredReadResult;
|
||||||
|
import static org.apache.hadoop.test.LambdaTestUtils.intercept;
|
||||||
|
import static org.apache.hadoop.test.LambdaTestUtils.interceptFuture;
|
||||||
|
|
||||||
@RunWith(Parameterized.class)
|
@RunWith(Parameterized.class)
|
||||||
public abstract class AbstractContractVectoredReadTest extends AbstractFSContractTestBase {
|
public abstract class AbstractContractVectoredReadTest extends AbstractFSContractTestBase {
|
||||||
|
@ -281,16 +281,11 @@ public abstract class AbstractContractVectoredReadTest extends AbstractFSContrac
|
||||||
in.readVectored(fileRanges, allocate);
|
in.readVectored(fileRanges, allocate);
|
||||||
for (FileRange res : fileRanges) {
|
for (FileRange res : fileRanges) {
|
||||||
CompletableFuture<ByteBuffer> data = res.getData();
|
CompletableFuture<ByteBuffer> data = res.getData();
|
||||||
try {
|
interceptFuture(EOFException.class,
|
||||||
ByteBuffer buffer = data.get();
|
"",
|
||||||
// Shouldn't reach here.
|
ContractTestUtils.VECTORED_READ_OPERATION_TEST_TIMEOUT_SECONDS,
|
||||||
Assert.fail("EOFException must be thrown while reading EOF");
|
TimeUnit.SECONDS,
|
||||||
} catch (ExecutionException ex) {
|
data);
|
||||||
// ignore as expected.
|
|
||||||
} catch (Exception ex) {
|
|
||||||
LOG.error("Exception while running vectored read ", ex);
|
|
||||||
Assert.fail("Exception while running vectored read " + ex);
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -410,7 +405,7 @@ public abstract class AbstractContractVectoredReadTest extends AbstractFSContrac
|
||||||
fs.openFile(path(VECTORED_READ_FILE_NAME))
|
fs.openFile(path(VECTORED_READ_FILE_NAME))
|
||||||
.build();
|
.build();
|
||||||
try (FSDataInputStream in = builder.get()) {
|
try (FSDataInputStream in = builder.get()) {
|
||||||
LambdaTestUtils.intercept(clazz,
|
intercept(clazz,
|
||||||
() -> in.readVectored(fileRanges, allocate));
|
() -> in.readVectored(fileRanges, allocate));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -308,6 +308,23 @@ public enum Statistic {
|
||||||
StreamStatisticNames.STREAM_READ_OPERATIONS,
|
StreamStatisticNames.STREAM_READ_OPERATIONS,
|
||||||
"Count of read() operations in an input stream",
|
"Count of read() operations in an input stream",
|
||||||
TYPE_COUNTER),
|
TYPE_COUNTER),
|
||||||
|
STREAM_READ_VECTORED_OPERATIONS(
|
||||||
|
StreamStatisticNames.STREAM_READ_VECTORED_OPERATIONS,
|
||||||
|
"Count of readVectored() operations in an input stream.",
|
||||||
|
TYPE_COUNTER),
|
||||||
|
STREAM_READ_VECTORED_READ_BYTES_DISCARDED(
|
||||||
|
StreamStatisticNames.STREAM_READ_VECTORED_READ_BYTES_DISCARDED,
|
||||||
|
"Count of bytes discarded during readVectored() operation." +
|
||||||
|
" in an input stream",
|
||||||
|
TYPE_COUNTER),
|
||||||
|
STREAM_READ_VECTORED_INCOMING_RANGES(
|
||||||
|
StreamStatisticNames.STREAM_READ_VECTORED_INCOMING_RANGES,
|
||||||
|
"Count of incoming file ranges during readVectored() operation.",
|
||||||
|
TYPE_COUNTER),
|
||||||
|
STREAM_READ_VECTORED_COMBINED_RANGES(
|
||||||
|
StreamStatisticNames.STREAM_READ_VECTORED_COMBINED_RANGES,
|
||||||
|
"Count of combined file ranges during readVectored() operation.",
|
||||||
|
TYPE_COUNTER),
|
||||||
STREAM_READ_REMOTE_STREAM_ABORTED(
|
STREAM_READ_REMOTE_STREAM_ABORTED(
|
||||||
StreamStatisticNames.STREAM_READ_REMOTE_STREAM_ABORTED,
|
StreamStatisticNames.STREAM_READ_REMOTE_STREAM_ABORTED,
|
||||||
"Duration of aborting a remote stream during stream IO",
|
"Duration of aborting a remote stream during stream IO",
|
||||||
|
|
|
@ -176,7 +176,8 @@ public class ITestS3AContractVectoredRead extends AbstractContractVectoredReadTe
|
||||||
* */
|
* */
|
||||||
@Test
|
@Test
|
||||||
public void testNormalReadVsVectoredReadStatsCollection() throws Exception {
|
public void testNormalReadVsVectoredReadStatsCollection() throws Exception {
|
||||||
FileSystem fs = getTestFileSystemWithReadAheadDisabled();
|
|
||||||
|
try (S3AFileSystem fs = getTestFileSystemWithReadAheadDisabled()) {
|
||||||
List<FileRange> fileRanges = new ArrayList<>();
|
List<FileRange> fileRanges = new ArrayList<>();
|
||||||
fileRanges.add(FileRange.createFileRange(10 * 1024, 100));
|
fileRanges.add(FileRange.createFileRange(10 * 1024, 100));
|
||||||
fileRanges.add(FileRange.createFileRange(8 * 1024, 100));
|
fileRanges.add(FileRange.createFileRange(8 * 1024, 100));
|
||||||
|
@ -261,11 +262,26 @@ public class ITestS3AContractVectoredRead extends AbstractContractVectoredReadTe
|
||||||
StreamStatisticNames.STREAM_READ_BYTES,
|
StreamStatisticNames.STREAM_READ_BYTES,
|
||||||
1424);
|
1424);
|
||||||
}
|
}
|
||||||
|
// validate stats are getting merged at fs instance level.
|
||||||
|
IOStatistics fsStats = fs.getIOStatistics();
|
||||||
|
// only 1 vectored io call is made in this fs instance.
|
||||||
|
verifyStatisticCounterValue(fsStats,
|
||||||
|
StreamStatisticNames.STREAM_READ_VECTORED_OPERATIONS,
|
||||||
|
1);
|
||||||
|
// 8 get requests were made in this fs instance.
|
||||||
|
verifyStatisticCounterValue(fsStats,
|
||||||
|
StoreStatisticNames.ACTION_HTTP_GET_REQUEST,
|
||||||
|
8);
|
||||||
|
|
||||||
|
verifyStatisticCounterValue(fsStats,
|
||||||
|
StreamStatisticNames.STREAM_READ_BYTES,
|
||||||
|
2848);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test
|
@Test
|
||||||
public void testMultiVectoredReadStatsCollection() throws Exception {
|
public void testMultiVectoredReadStatsCollection() throws Exception {
|
||||||
FileSystem fs = getTestFileSystemWithReadAheadDisabled();
|
try (S3AFileSystem fs = getTestFileSystemWithReadAheadDisabled()) {
|
||||||
List<FileRange> ranges1 = getConsecutiveRanges();
|
List<FileRange> ranges1 = getConsecutiveRanges();
|
||||||
List<FileRange> ranges2 = getConsecutiveRanges();
|
List<FileRange> ranges2 = getConsecutiveRanges();
|
||||||
FileStatus fileStatus = fs.getFileStatus(path(VECTORED_READ_FILE_NAME));
|
FileStatus fileStatus = fs.getFileStatus(path(VECTORED_READ_FILE_NAME));
|
||||||
|
@ -313,9 +329,19 @@ public class ITestS3AContractVectoredRead extends AbstractContractVectoredReadTe
|
||||||
StreamStatisticNames.STREAM_READ_BYTES,
|
StreamStatisticNames.STREAM_READ_BYTES,
|
||||||
2000);
|
2000);
|
||||||
}
|
}
|
||||||
|
IOStatistics fsStats = fs.getIOStatistics();
|
||||||
|
// 2 vectored io calls are made in this fs instance.
|
||||||
|
verifyStatisticCounterValue(fsStats,
|
||||||
|
StreamStatisticNames.STREAM_READ_VECTORED_OPERATIONS,
|
||||||
|
2);
|
||||||
|
// 2 get requests were made in this fs instance.
|
||||||
|
verifyStatisticCounterValue(fsStats,
|
||||||
|
StoreStatisticNames.ACTION_HTTP_GET_REQUEST,
|
||||||
|
2);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private FileSystem getTestFileSystemWithReadAheadDisabled() throws IOException {
|
private S3AFileSystem getTestFileSystemWithReadAheadDisabled() throws IOException {
|
||||||
Configuration conf = getFileSystem().getConf();
|
Configuration conf = getFileSystem().getConf();
|
||||||
// also resetting the min seek and max size values is important
|
// also resetting the min seek and max size values is important
|
||||||
// as this same test suite has test which overrides these params.
|
// as this same test suite has test which overrides these params.
|
||||||
|
|
Loading…
Reference in New Issue