Skip to content

Commit b28e4c6

Browse files
HADOOP-18392. Propagate vectored s3a input stream stats to file system stats. (#4704)
part of HADOOP-18103. Contributed By: Mukund Thakur
1 parent e9509ac commit b28e4c6

File tree

3 files changed

+172
-134
lines changed

3 files changed

+172
-134
lines changed

hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractVectoredReadTest.java

Lines changed: 9 additions & 14 deletions
Original file line numberDiff line numberDiff line change
@@ -24,11 +24,10 @@
2424
import java.util.Arrays;
2525
import java.util.List;
2626
import java.util.concurrent.CompletableFuture;
27-
import java.util.concurrent.ExecutionException;
27+
import java.util.concurrent.TimeUnit;
2828
import java.util.function.IntFunction;
2929

3030
import org.assertj.core.api.Assertions;
31-
import org.junit.Assert;
3231
import org.junit.Test;
3332
import org.junit.runner.RunWith;
3433
import org.junit.runners.Parameterized;
@@ -43,13 +42,14 @@
4342
import org.apache.hadoop.fs.Path;
4443
import org.apache.hadoop.fs.impl.FutureIOSupport;
4544
import org.apache.hadoop.io.WeakReferencedElasticByteBufferPool;
46-
import org.apache.hadoop.test.LambdaTestUtils;
4745

4846
import static org.apache.hadoop.fs.contract.ContractTestUtils.assertCapabilities;
4947
import static org.apache.hadoop.fs.contract.ContractTestUtils.assertDatasetEquals;
5048
import static org.apache.hadoop.fs.contract.ContractTestUtils.createFile;
5149
import static org.apache.hadoop.fs.contract.ContractTestUtils.returnBuffersToPoolPostRead;
5250
import static org.apache.hadoop.fs.contract.ContractTestUtils.validateVectoredReadResult;
51+
import static org.apache.hadoop.test.LambdaTestUtils.intercept;
52+
import static org.apache.hadoop.test.LambdaTestUtils.interceptFuture;
5353

5454
@RunWith(Parameterized.class)
5555
public abstract class AbstractContractVectoredReadTest extends AbstractFSContractTestBase {
@@ -281,16 +281,11 @@ public void testEOFRanges() throws Exception {
281281
in.readVectored(fileRanges, allocate);
282282
for (FileRange res : fileRanges) {
283283
CompletableFuture<ByteBuffer> data = res.getData();
284-
try {
285-
ByteBuffer buffer = data.get();
286-
// Shouldn't reach here.
287-
Assert.fail("EOFException must be thrown while reading EOF");
288-
} catch (ExecutionException ex) {
289-
// ignore as expected.
290-
} catch (Exception ex) {
291-
LOG.error("Exception while running vectored read ", ex);
292-
Assert.fail("Exception while running vectored read " + ex);
293-
}
284+
interceptFuture(EOFException.class,
285+
"",
286+
ContractTestUtils.VECTORED_READ_OPERATION_TEST_TIMEOUT_SECONDS,
287+
TimeUnit.SECONDS,
288+
data);
294289
}
295290
}
296291
}
@@ -410,7 +405,7 @@ protected <T extends Throwable> void verifyExceptionalVectoredRead(
410405
fs.openFile(path(VECTORED_READ_FILE_NAME))
411406
.build();
412407
try (FSDataInputStream in = builder.get()) {
413-
LambdaTestUtils.intercept(clazz,
408+
intercept(clazz,
414409
() -> in.readVectored(fileRanges, allocate));
415410
}
416411
}

hadoop-tools/hadoop-aws/src/main/java/org/apache/hadoop/fs/s3a/Statistic.java

Lines changed: 17 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -308,6 +308,23 @@ public enum Statistic {
308308
StreamStatisticNames.STREAM_READ_OPERATIONS,
309309
"Count of read() operations in an input stream",
310310
TYPE_COUNTER),
311+
STREAM_READ_VECTORED_OPERATIONS(
312+
StreamStatisticNames.STREAM_READ_VECTORED_OPERATIONS,
313+
"Count of readVectored() operations in an input stream.",
314+
TYPE_COUNTER),
315+
STREAM_READ_VECTORED_READ_BYTES_DISCARDED(
316+
StreamStatisticNames.STREAM_READ_VECTORED_READ_BYTES_DISCARDED,
317+
"Count of bytes discarded during readVectored() operation." +
318+
" in an input stream",
319+
TYPE_COUNTER),
320+
STREAM_READ_VECTORED_INCOMING_RANGES(
321+
StreamStatisticNames.STREAM_READ_VECTORED_INCOMING_RANGES,
322+
"Count of incoming file ranges during readVectored() operation.",
323+
TYPE_COUNTER),
324+
STREAM_READ_VECTORED_COMBINED_RANGES(
325+
StreamStatisticNames.STREAM_READ_VECTORED_COMBINED_RANGES,
326+
"Count of combined file ranges during readVectored() operation.",
327+
TYPE_COUNTER),
311328
STREAM_READ_REMOTE_STREAM_ABORTED(
312329
StreamStatisticNames.STREAM_READ_REMOTE_STREAM_ABORTED,
313330
"Duration of aborting a remote stream during stream IO",

hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/contract/s3a/ITestS3AContractVectoredRead.java

Lines changed: 146 additions & 120 deletions
Original file line numberDiff line numberDiff line change
@@ -176,146 +176,172 @@ public void testSameRanges() throws Exception {
176176
* */
177177
@Test
178178
public void testNormalReadVsVectoredReadStatsCollection() throws Exception {
179-
FileSystem fs = getTestFileSystemWithReadAheadDisabled();
180-
List<FileRange> fileRanges = new ArrayList<>();
181-
fileRanges.add(FileRange.createFileRange(10 * 1024, 100));
182-
fileRanges.add(FileRange.createFileRange(8 * 1024, 100));
183-
fileRanges.add(FileRange.createFileRange(14 * 1024, 100));
184-
fileRanges.add(FileRange.createFileRange(2 * 1024 - 101, 100));
185-
fileRanges.add(FileRange.createFileRange(40 * 1024, 1024));
186-
187-
FileStatus fileStatus = fs.getFileStatus(path(VECTORED_READ_FILE_NAME));
188-
CompletableFuture<FSDataInputStream> builder =
189-
fs.openFile(path(VECTORED_READ_FILE_NAME))
190-
.withFileStatus(fileStatus)
191-
.build();
192-
try (FSDataInputStream in = builder.get()) {
193-
in.readVectored(fileRanges, getAllocate());
194-
validateVectoredReadResult(fileRanges, DATASET);
195-
returnBuffersToPoolPostRead(fileRanges, getPool());
196-
197-
// audit the io statistics for this stream
198-
IOStatistics st = in.getIOStatistics();
199-
LOG.info("IOStats after readVectored operation {}", ioStatisticsToPrettyString(st));
200-
201-
// the vectored io operation must be tracked
202-
verifyStatisticCounterValue(st,
203-
StreamStatisticNames.STREAM_READ_VECTORED_OPERATIONS,
204-
1);
205-
206-
// the vectored io operation is being called with 5 input ranges.
207-
verifyStatisticCounterValue(st,
208-
StreamStatisticNames.STREAM_READ_VECTORED_INCOMING_RANGES,
209-
5);
210-
211-
// 5 input ranges got combined in 3 as some of them are close.
212-
verifyStatisticCounterValue(st,
213-
StreamStatisticNames.STREAM_READ_VECTORED_COMBINED_RANGES,
214-
3);
215-
216-
// number of bytes discarded will be based on the above input ranges.
217-
verifyStatisticCounterValue(st,
218-
StreamStatisticNames.STREAM_READ_VECTORED_READ_BYTES_DISCARDED,
219-
5944);
220-
221-
verifyStatisticCounterValue(st,
222-
StoreStatisticNames.ACTION_HTTP_GET_REQUEST,
223-
3);
224-
225-
// read bytes should match the sum of requested length for each input ranges.
226-
verifyStatisticCounterValue(st,
227-
StreamStatisticNames.STREAM_READ_BYTES,
228-
1424);
229-
230-
}
231179

232-
CompletableFuture<FSDataInputStream> builder1 =
233-
fs.openFile(path(VECTORED_READ_FILE_NAME))
234-
.withFileStatus(fileStatus)
235-
.build();
180+
try (S3AFileSystem fs = getTestFileSystemWithReadAheadDisabled()) {
181+
List<FileRange> fileRanges = new ArrayList<>();
182+
fileRanges.add(FileRange.createFileRange(10 * 1024, 100));
183+
fileRanges.add(FileRange.createFileRange(8 * 1024, 100));
184+
fileRanges.add(FileRange.createFileRange(14 * 1024, 100));
185+
fileRanges.add(FileRange.createFileRange(2 * 1024 - 101, 100));
186+
fileRanges.add(FileRange.createFileRange(40 * 1024, 1024));
187+
188+
FileStatus fileStatus = fs.getFileStatus(path(VECTORED_READ_FILE_NAME));
189+
CompletableFuture<FSDataInputStream> builder =
190+
fs.openFile(path(VECTORED_READ_FILE_NAME))
191+
.withFileStatus(fileStatus)
192+
.build();
193+
try (FSDataInputStream in = builder.get()) {
194+
in.readVectored(fileRanges, getAllocate());
195+
validateVectoredReadResult(fileRanges, DATASET);
196+
returnBuffersToPoolPostRead(fileRanges, getPool());
197+
198+
// audit the io statistics for this stream
199+
IOStatistics st = in.getIOStatistics();
200+
LOG.info("IOStats after readVectored operation {}", ioStatisticsToPrettyString(st));
201+
202+
// the vectored io operation must be tracked
203+
verifyStatisticCounterValue(st,
204+
StreamStatisticNames.STREAM_READ_VECTORED_OPERATIONS,
205+
1);
206+
207+
// the vectored io operation is being called with 5 input ranges.
208+
verifyStatisticCounterValue(st,
209+
StreamStatisticNames.STREAM_READ_VECTORED_INCOMING_RANGES,
210+
5);
211+
212+
// 5 input ranges got combined in 3 as some of them are close.
213+
verifyStatisticCounterValue(st,
214+
StreamStatisticNames.STREAM_READ_VECTORED_COMBINED_RANGES,
215+
3);
216+
217+
// number of bytes discarded will be based on the above input ranges.
218+
verifyStatisticCounterValue(st,
219+
StreamStatisticNames.STREAM_READ_VECTORED_READ_BYTES_DISCARDED,
220+
5944);
221+
222+
verifyStatisticCounterValue(st,
223+
StoreStatisticNames.ACTION_HTTP_GET_REQUEST,
224+
3);
225+
226+
// read bytes should match the sum of requested length for each input ranges.
227+
verifyStatisticCounterValue(st,
228+
StreamStatisticNames.STREAM_READ_BYTES,
229+
1424);
236230

237-
try (FSDataInputStream in = builder1.get()) {
238-
for (FileRange range : fileRanges) {
239-
byte[] temp = new byte[range.getLength()];
240-
in.readFully((int) range.getOffset(), temp, 0, range.getLength());
241231
}
242232

243-
// audit the statistics for this stream
244-
IOStatistics st = in.getIOStatistics();
245-
LOG.info("IOStats after read fully operation {}", ioStatisticsToPrettyString(st));
246-
247-
verifyStatisticCounterValue(st,
233+
CompletableFuture<FSDataInputStream> builder1 =
234+
fs.openFile(path(VECTORED_READ_FILE_NAME))
235+
.withFileStatus(fileStatus)
236+
.build();
237+
238+
try (FSDataInputStream in = builder1.get()) {
239+
for (FileRange range : fileRanges) {
240+
byte[] temp = new byte[range.getLength()];
241+
in.readFully((int) range.getOffset(), temp, 0, range.getLength());
242+
}
243+
244+
// audit the statistics for this stream
245+
IOStatistics st = in.getIOStatistics();
246+
LOG.info("IOStats after read fully operation {}", ioStatisticsToPrettyString(st));
247+
248+
verifyStatisticCounterValue(st,
249+
StreamStatisticNames.STREAM_READ_VECTORED_OPERATIONS,
250+
0);
251+
252+
// all other counter values consistent.
253+
verifyStatisticCounterValue(st,
254+
StreamStatisticNames.STREAM_READ_VECTORED_READ_BYTES_DISCARDED,
255+
0);
256+
verifyStatisticCounterValue(st,
257+
StoreStatisticNames.ACTION_HTTP_GET_REQUEST,
258+
5);
259+
260+
// read bytes should match the sum of requested length for each input ranges.
261+
verifyStatisticCounterValue(st,
262+
StreamStatisticNames.STREAM_READ_BYTES,
263+
1424);
264+
}
265+
// validate stats are getting merged at fs instance level.
266+
IOStatistics fsStats = fs.getIOStatistics();
267+
// only 1 vectored io call is made in this fs instance.
268+
verifyStatisticCounterValue(fsStats,
248269
StreamStatisticNames.STREAM_READ_VECTORED_OPERATIONS,
249-
0);
250-
251-
// all other counter values consistent.
252-
verifyStatisticCounterValue(st,
253-
StreamStatisticNames.STREAM_READ_VECTORED_READ_BYTES_DISCARDED,
254-
0);
255-
verifyStatisticCounterValue(st,
270+
1);
271+
// 8 get requests were made in this fs instance.
272+
verifyStatisticCounterValue(fsStats,
256273
StoreStatisticNames.ACTION_HTTP_GET_REQUEST,
257-
5);
274+
8);
258275

259-
// read bytes should match the sum of requested length for each input ranges.
260-
verifyStatisticCounterValue(st,
276+
verifyStatisticCounterValue(fsStats,
261277
StreamStatisticNames.STREAM_READ_BYTES,
262-
1424);
278+
2848);
263279
}
264280
}
265281

266282
@Test
267283
public void testMultiVectoredReadStatsCollection() throws Exception {
268-
FileSystem fs = getTestFileSystemWithReadAheadDisabled();
269-
List<FileRange> ranges1 = getConsecutiveRanges();
270-
List<FileRange> ranges2 = getConsecutiveRanges();
271-
FileStatus fileStatus = fs.getFileStatus(path(VECTORED_READ_FILE_NAME));
272-
CompletableFuture<FSDataInputStream> builder =
273-
fs.openFile(path(VECTORED_READ_FILE_NAME))
274-
.withFileStatus(fileStatus)
275-
.build();
276-
try (FSDataInputStream in = builder.get()) {
277-
in.readVectored(ranges1, getAllocate());
278-
in.readVectored(ranges2, getAllocate());
279-
validateVectoredReadResult(ranges1, DATASET);
280-
validateVectoredReadResult(ranges2, DATASET);
281-
returnBuffersToPoolPostRead(ranges1, getPool());
282-
returnBuffersToPoolPostRead(ranges2, getPool());
283-
284-
// audit the io statistics for this stream
285-
IOStatistics st = in.getIOStatistics();
286-
287-
// 2 vectored io calls are made above.
288-
verifyStatisticCounterValue(st,
284+
try (S3AFileSystem fs = getTestFileSystemWithReadAheadDisabled()) {
285+
List<FileRange> ranges1 = getConsecutiveRanges();
286+
List<FileRange> ranges2 = getConsecutiveRanges();
287+
FileStatus fileStatus = fs.getFileStatus(path(VECTORED_READ_FILE_NAME));
288+
CompletableFuture<FSDataInputStream> builder =
289+
fs.openFile(path(VECTORED_READ_FILE_NAME))
290+
.withFileStatus(fileStatus)
291+
.build();
292+
try (FSDataInputStream in = builder.get()) {
293+
in.readVectored(ranges1, getAllocate());
294+
in.readVectored(ranges2, getAllocate());
295+
validateVectoredReadResult(ranges1, DATASET);
296+
validateVectoredReadResult(ranges2, DATASET);
297+
returnBuffersToPoolPostRead(ranges1, getPool());
298+
returnBuffersToPoolPostRead(ranges2, getPool());
299+
300+
// audit the io statistics for this stream
301+
IOStatistics st = in.getIOStatistics();
302+
303+
// 2 vectored io calls are made above.
304+
verifyStatisticCounterValue(st,
305+
StreamStatisticNames.STREAM_READ_VECTORED_OPERATIONS,
306+
2);
307+
308+
// 2 vectored io operation is being called with 2 input ranges.
309+
verifyStatisticCounterValue(st,
310+
StreamStatisticNames.STREAM_READ_VECTORED_INCOMING_RANGES,
311+
4);
312+
313+
// 2 ranges are getting merged in 1 during both vectored io operation.
314+
verifyStatisticCounterValue(st,
315+
StreamStatisticNames.STREAM_READ_VECTORED_COMBINED_RANGES,
316+
2);
317+
318+
// number of bytes discarded will be 0 as the ranges are consecutive.
319+
verifyStatisticCounterValue(st,
320+
StreamStatisticNames.STREAM_READ_VECTORED_READ_BYTES_DISCARDED,
321+
0);
322+
// only 2 http get request will be made because ranges in both range list will be merged
323+
// to 1 because they are consecutive.
324+
verifyStatisticCounterValue(st,
325+
StoreStatisticNames.ACTION_HTTP_GET_REQUEST,
326+
2);
327+
// read bytes should match the sum of requested length for each input ranges.
328+
verifyStatisticCounterValue(st,
329+
StreamStatisticNames.STREAM_READ_BYTES,
330+
2000);
331+
}
332+
IOStatistics fsStats = fs.getIOStatistics();
333+
// 2 vectored io calls are made in this fs instance.
334+
verifyStatisticCounterValue(fsStats,
289335
StreamStatisticNames.STREAM_READ_VECTORED_OPERATIONS,
290336
2);
291-
292-
// 2 vectored io operation is being called with 2 input ranges.
293-
verifyStatisticCounterValue(st,
294-
StreamStatisticNames.STREAM_READ_VECTORED_INCOMING_RANGES,
295-
4);
296-
297-
// 2 ranges are getting merged in 1 during both vectored io operation.
298-
verifyStatisticCounterValue(st,
299-
StreamStatisticNames.STREAM_READ_VECTORED_COMBINED_RANGES,
300-
2);
301-
302-
// number of bytes discarded will be 0 as the ranges are consecutive.
303-
verifyStatisticCounterValue(st,
304-
StreamStatisticNames.STREAM_READ_VECTORED_READ_BYTES_DISCARDED,
305-
0);
306-
// only 2 http get request will be made because ranges in both range list will be merged
307-
// to 1 because they are consecutive.
308-
verifyStatisticCounterValue(st,
337+
// 2 get requests were made in this fs instance.
338+
verifyStatisticCounterValue(fsStats,
309339
StoreStatisticNames.ACTION_HTTP_GET_REQUEST,
310340
2);
311-
// read bytes should match the sum of requested length for each input ranges.
312-
verifyStatisticCounterValue(st,
313-
StreamStatisticNames.STREAM_READ_BYTES,
314-
2000);
315341
}
316342
}
317343

318-
private FileSystem getTestFileSystemWithReadAheadDisabled() throws IOException {
344+
private S3AFileSystem getTestFileSystemWithReadAheadDisabled() throws IOException {
319345
Configuration conf = getFileSystem().getConf();
320346
// also resetting the min seek and max size values is important
321347
// as this same test suite has test which overrides these params.

0 commit comments

Comments
 (0)