Skip to content

Commit bb32aec

Browse files
HADOOP-19043. S3A: Regression: ITestS3AOpenCost fails on prefetch test runs (#6465)
Disables the new tests added in: HADOOP-19027. S3A: S3AInputStream doesn't recover from HTTP/channel exceptions #6425 The underlying issue here is that the block prefetch code can identify when there's a mismatch between declared and actual length, and doesn't store any of the incomplete buffer. This should be addressed in HADOOP-18184. Contributed by Steve Loughran
1 parent fc166d3 commit bb32aec

File tree

2 files changed

+82
-26
lines changed

2 files changed

+82
-26
lines changed

hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/AbstractS3ACostTest.java

Lines changed: 8 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -370,6 +370,14 @@ protected OperationCostValidator.ExpectedProbe always(
370370
return expect(true, cost);
371371
}
372372

373+
/**
374+
* Always run a metrics operation.
375+
* @return a probe.
376+
*/
377+
protected OperationCostValidator.ExpectedProbe always() {
378+
return OperationCostValidator.always();
379+
}
380+
373381
/**
374382
* A metric diff which must hold when the fs is keeping markers.
375383
* @param cost expected cost

hadoop-tools/hadoop-aws/src/test/java/org/apache/hadoop/fs/s3a/performance/ITestS3AOpenCost.java

Lines changed: 74 additions & 26 deletions
Original file line numberDiff line numberDiff line change
@@ -52,6 +52,8 @@
5252
import static org.apache.hadoop.fs.contract.ContractTestUtils.skip;
5353
import static org.apache.hadoop.fs.contract.ContractTestUtils.writeTextFile;
5454
import static org.apache.hadoop.fs.s3a.Constants.CHECKSUM_VALIDATION;
55+
import static org.apache.hadoop.fs.s3a.Constants.PREFETCH_ENABLED_DEFAULT;
56+
import static org.apache.hadoop.fs.s3a.Constants.PREFETCH_ENABLED_KEY;
5557
import static org.apache.hadoop.fs.s3a.S3ATestUtils.assertStreamIsNotChecksummed;
5658
import static org.apache.hadoop.fs.s3a.S3ATestUtils.disableFilesystemCaching;
5759
import static org.apache.hadoop.fs.s3a.S3ATestUtils.getS3AInputStream;
@@ -60,10 +62,12 @@
6062
import static org.apache.hadoop.fs.s3a.Statistic.STREAM_READ_OPENED;
6163
import static org.apache.hadoop.fs.s3a.Statistic.STREAM_READ_SEEK_BYTES_SKIPPED;
6264
import static org.apache.hadoop.fs.s3a.performance.OperationCost.NO_HEAD_OR_LIST;
65+
import static org.apache.hadoop.fs.s3a.performance.OperationCostValidator.probe;
6366
import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.assertDurationRange;
6467
import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.extractStatistics;
6568
import static org.apache.hadoop.fs.statistics.IOStatisticAssertions.verifyStatisticCounterValue;
6669
import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.demandStringifyIOStatistics;
70+
import static org.apache.hadoop.fs.statistics.IOStatisticsLogging.ioStatisticsToPrettyString;
6771
import static org.apache.hadoop.fs.statistics.StoreStatisticNames.ACTION_FILE_OPENED;
6872
import static org.apache.hadoop.test.LambdaTestUtils.intercept;
6973
import static org.apache.hadoop.test.LambdaTestUtils.interceptFuture;
@@ -84,6 +88,11 @@ public class ITestS3AOpenCost extends AbstractS3ACostTest {
8488

8589
private int fileLength;
8690

91+
/**
92+
* Is prefetching enabled?
93+
*/
94+
private boolean prefetching;
95+
8796
public ITestS3AOpenCost() {
8897
super(true);
8998
}
@@ -111,6 +120,7 @@ public void setup() throws Exception {
111120
writeTextFile(fs, testFile, TEXT, true);
112121
testFileStatus = fs.getFileStatus(testFile);
113122
fileLength = (int)testFileStatus.getLen();
123+
prefetching = prefetching();
114124
}
115125

116126
/**
@@ -161,7 +171,11 @@ public void testOpenFileWithStatusOfOtherFS() throws Throwable {
161171
@Test
162172
public void testStreamIsNotChecksummed() throws Throwable {
163173
describe("Verify that an opened stream is not checksummed");
174+
175+
// if prefetching is enabled, skip this test
176+
assumeNoPrefetching();
164177
S3AFileSystem fs = getFileSystem();
178+
165179
// open the file
166180
try (FSDataInputStream in = verifyMetrics(() ->
167181
fs.openFile(testFile)
@@ -173,12 +187,6 @@ public void testStreamIsNotChecksummed() throws Throwable {
173187
always(NO_HEAD_OR_LIST),
174188
with(STREAM_READ_OPENED, 0))) {
175189

176-
// if prefetching is enabled, skip this test
177-
final InputStream wrapped = in.getWrappedStream();
178-
if (!(wrapped instanceof S3AInputStream)) {
179-
skip("Not an S3AInputStream: " + wrapped);
180-
}
181-
182190
// open the stream.
183191
in.read();
184192
// now examine the innermost stream and make sure it doesn't have a checksum
@@ -239,16 +247,20 @@ public void testOpenFileLongerLengthReadFully() throws Throwable {
239247
try (FSDataInputStream in = openFile(longLen,
240248
FS_OPTION_OPENFILE_READ_POLICY_SEQUENTIAL)) {
241249
byte[] out = new byte[(int) (longLen)];
242-
intercept(EOFException.class, () -> in.readFully(0, out));
250+
intercept(EOFException.class, () -> {
251+
in.readFully(0, out);
252+
return in;
253+
});
243254
in.seek(longLen - 1);
244255
assertEquals("read past real EOF on " + in, -1, in.read());
245256
return in.toString();
246257
}
247258
},
259+
always(),
248260
// two GET calls were made, one for readFully,
249261
// the second on the read() past the EOF
250262
// the operation has got as far as S3
251-
with(STREAM_READ_OPENED, 1 + 1));
263+
probe(!prefetching(), STREAM_READ_OPENED, 1 + 1));
252264

253265
// now on a new stream, try a full read from after the EOF
254266
verifyMetrics(() -> {
@@ -293,15 +305,19 @@ private FSDataInputStream openFile(final long longLen, String policy)
293305
public void testReadPastEOF() throws Throwable {
294306

295307
// set a length past the actual file length
308+
describe("read() up to the end of the real file");
309+
assumeNoPrefetching();
310+
296311
final int extra = 10;
297312
int longLen = fileLength + extra;
298313
try (FSDataInputStream in = openFile(longLen,
299314
FS_OPTION_OPENFILE_READ_POLICY_RANDOM)) {
300315
for (int i = 0; i < fileLength; i++) {
301316
Assertions.assertThat(in.read())
302-
.describedAs("read() at %d", i)
317+
.describedAs("read() at %d from stream %s", i, in)
303318
.isEqualTo(TEXT.charAt(i));
304319
}
320+
LOG.info("Statistics after EOF {}", ioStatisticsToPrettyString(in.getIOStatistics()));
305321
}
306322

307323
// now open and read after the EOF; this is
@@ -323,10 +339,12 @@ public void testReadPastEOF() throws Throwable {
323339
.describedAs("read() at %d", p)
324340
.isEqualTo(-1);
325341
}
342+
LOG.info("Statistics after EOF {}", ioStatisticsToPrettyString(in.getIOStatistics()));
326343
return in.toString();
327344
}
328345
},
329-
with(Statistic.ACTION_HTTP_GET_REQUEST, extra));
346+
always(),
347+
probe(!prefetching, Statistic.ACTION_HTTP_GET_REQUEST, extra));
330348
}
331349

332350
/**
@@ -353,10 +371,12 @@ public void testPositionedReadableReadFullyPastEOF() throws Throwable {
353371
return in;
354372
});
355373
assertS3StreamClosed(in);
356-
return "readFully past EOF";
374+
return "readFully past EOF with statistics"
375+
+ ioStatisticsToPrettyString(in.getIOStatistics());
357376
}
358377
},
359-
with(Statistic.ACTION_HTTP_GET_REQUEST, 1)); // no attempt to re-open
378+
always(),
379+
probe(!prefetching, Statistic.ACTION_HTTP_GET_REQUEST, 1)); // no attempt to re-open
360380
}
361381

362382
/**
@@ -370,6 +390,7 @@ public void testPositionedReadableReadPastEOF() throws Throwable {
370390
int longLen = fileLength + extra;
371391

372392
describe("PositionedReadable.read() past the end of the file");
393+
assumeNoPrefetching();
373394

374395
verifyMetrics(() -> {
375396
try (FSDataInputStream in =
@@ -388,10 +409,11 @@ public void testPositionedReadableReadPastEOF() throws Throwable {
388409
// stream is closed as part of this failure
389410
assertS3StreamClosed(in);
390411

391-
return "PositionedReadable.read()) past EOF";
412+
return "PositionedReadable.read()) past EOF with " + in;
392413
}
393414
},
394-
with(Statistic.ACTION_HTTP_GET_REQUEST, 1)); // no attempt to re-open
415+
always(),
416+
probe(!prefetching, Statistic.ACTION_HTTP_GET_REQUEST, 1)); // no attempt to re-open
395417
}
396418

397419
/**
@@ -405,7 +427,8 @@ public void testVectorReadPastEOF() throws Throwable {
405427
final int extra = 10;
406428
int longLen = fileLength + extra;
407429

408-
describe("Vector read past the end of the file");
430+
describe("Vector read past the end of the file, expecting an EOFException");
431+
409432
verifyMetrics(() -> {
410433
try (FSDataInputStream in =
411434
openFile(longLen, FS_OPTION_OPENFILE_READ_POLICY_RANDOM)) {
@@ -420,31 +443,56 @@ public void testVectorReadPastEOF() throws Throwable {
420443
TimeUnit.SECONDS,
421444
range.getData());
422445
assertS3StreamClosed(in);
423-
return "vector read past EOF";
446+
return "vector read past EOF with " + in;
424447
}
425448
},
426-
with(Statistic.ACTION_HTTP_GET_REQUEST, 1));
449+
always(),
450+
probe(!prefetching, Statistic.ACTION_HTTP_GET_REQUEST, 1));
451+
}
452+
453+
/**
454+
* Probe the FS for supporting prefetching.
455+
* @return true if the fs has prefetching enabled.
456+
*/
457+
private boolean prefetching() {
458+
return getFileSystem().getConf().getBoolean(
459+
PREFETCH_ENABLED_KEY, PREFETCH_ENABLED_DEFAULT);
460+
}
461+
462+
/**
463+
* Skip the test if prefetching is enabled.
464+
*/
465+
private void assumeNoPrefetching(){
466+
if (prefetching) {
467+
skip("Prefetching is enabled");
468+
}
427469
}
428470

429471
/**
430472
* Assert that the inner S3 Stream is closed.
431473
* @param in input stream
432474
*/
433475
private static void assertS3StreamClosed(final FSDataInputStream in) {
434-
S3AInputStream s3ain = (S3AInputStream) in.getWrappedStream();
435-
Assertions.assertThat(s3ain.isObjectStreamOpen())
436-
.describedAs("stream is open")
437-
.isFalse();
476+
final InputStream wrapped = in.getWrappedStream();
477+
if (wrapped instanceof S3AInputStream) {
478+
S3AInputStream s3ain = (S3AInputStream) wrapped;
479+
Assertions.assertThat(s3ain.isObjectStreamOpen())
480+
.describedAs("stream is open: %s", s3ain)
481+
.isFalse();
482+
}
438483
}
439484

440485
/**
441-
* Assert that the inner S3 Stream is open.
486+
* Assert that the inner S3 Stream is closed.
442487
* @param in input stream
443488
*/
444489
private static void assertS3StreamOpen(final FSDataInputStream in) {
445-
S3AInputStream s3ain = (S3AInputStream) in.getWrappedStream();
446-
Assertions.assertThat(s3ain.isObjectStreamOpen())
447-
.describedAs("stream is closed")
448-
.isTrue();
490+
final InputStream wrapped = in.getWrappedStream();
491+
if (wrapped instanceof S3AInputStream) {
492+
S3AInputStream s3ain = (S3AInputStream) wrapped;
493+
Assertions.assertThat(s3ain.isObjectStreamOpen())
494+
.describedAs("stream is closed: %s", s3ain)
495+
.isTrue();
496+
}
449497
}
450498
}

0 commit comments

Comments
 (0)