21
21
22
22
import java .io .File ;
23
23
import java .io .IOException ;
24
- import java .nio .charset .Charset ;
25
- import java .util .UUID ;
24
+ import java .io .InputStream ;
26
25
27
26
import org .junit .Before ;
28
27
import org .junit .Test ;
29
28
30
- import org .apache .commons .io .FileUtils ;
31
29
import org .apache .hadoop .conf .Configuration ;
32
30
import org .apache .hadoop .fs .FSDataInputStream ;
33
31
import org .apache .hadoop .fs .FileStatus ;
34
32
import org .apache .hadoop .fs .FileSystem ;
35
33
import org .apache .hadoop .fs .Path ;
36
- import org .apache .hadoop .fs .PathHandle ;
37
- import org .apache .hadoop .fs .s3a .impl .AwsSdkWorkarounds ;
34
+ import org .apache .hadoop .fs .s3a . impl . streams . InputStreamType ;
35
+ import org .apache .hadoop .fs .s3a .impl .streams . ObjectInputStream ;
38
36
import org .apache .hadoop .fs .statistics .IOStatistics ;
39
- import org .apache .hadoop .test .GenericTestUtils ;
40
37
41
- import static org .apache .commons .io .FileUtils .ONE_KB ;
42
38
import static org .apache .hadoop .fs .s3a .Constants .*;
43
39
import static org .apache .hadoop .fs .s3a .S3ATestUtils .enableAnalyticsAccelerator ;
44
- import static org .apache .hadoop .fs .s3a .S3ATestUtils .enablePrefetching ;
45
40
import static org .apache .hadoop .fs .s3a .S3ATestUtils .removeBaseAndBucketOverrides ;
46
41
import static org .apache .hadoop .fs .s3a .test .PublicDatasetTestUtils .getExternalData ;
47
42
import static org .apache .hadoop .fs .s3a .test .PublicDatasetTestUtils .isUsingDefaultExternalDataFile ;
48
43
import static org .apache .hadoop .fs .statistics .IOStatisticAssertions .verifyStatisticCounterValue ;
49
44
import static org .apache .hadoop .fs .statistics .StreamStatisticNames .STREAM_READ_ANALYTICS_OPENED ;
50
- import static org .apache .hadoop .fs .statistics .StreamStatisticNames .STREAM_READ_OPENED ;
51
- import static org .apache .hadoop .fs .statistics .StreamStatisticNames .STREAM_READ_PREFETCH_OPERATIONS ;
52
- import static org .apache .hadoop .test .GenericTestUtils .LogCapturer .captureLogs ;
45
+
53
46
54
47
import org .assertj .core .api .Assertions ;
55
48
56
- import org . slf4j . LoggerFactory ;
49
+
57
50
import software .amazon .s3 .analyticsaccelerator .S3SeekableInputStreamConfiguration ;
58
51
import software .amazon .s3 .analyticsaccelerator .common .ConnectorConfiguration ;
59
- import software .amazon .s3 .analyticsaccelerator .io .logical .parquet .ParquetMetadataParsingTask ;
60
52
import software .amazon .s3 .analyticsaccelerator .util .PrefetchMode ;
61
53
62
54
public class ITestS3AAnalyticsAcceleratorStream extends AbstractS3ATestBase {
@@ -90,6 +82,9 @@ public Configuration createConfiguration() {
90
82
public void testConnectorFrameWorkIntegration () throws IOException {
91
83
describe ("Verify S3 connector framework integration" );
92
84
85
+ removeBaseAndBucketOverrides (conf , INPUT_FADVISE );
86
+ conf .set (INPUT_FADVISE , "whole-file" );
87
+
93
88
S3AFileSystem fs =
94
89
(S3AFileSystem ) FileSystem .get (testFile .toUri (), conf );
95
90
byte [] buffer = new byte [500 ];
@@ -99,7 +94,13 @@ public void testConnectorFrameWorkIntegration() throws IOException {
99
94
ioStats = inputStream .getIOStatistics ();
100
95
inputStream .seek (5 );
101
96
inputStream .read (buffer , 0 , 500 );
97
+
98
+ final InputStream wrappedStream = inputStream .getWrappedStream ();
99
+ ObjectInputStream objectInputStream = (ObjectInputStream ) wrappedStream ;
100
+ assertEquals (objectInputStream .streamType (), InputStreamType .Analytics );
101
+ assertEquals (objectInputStream .getInputPolicy (), S3AInputPolicy .Sequential );
102
102
}
103
+
103
104
verifyStatisticCounterValue (ioStats , STREAM_READ_ANALYTICS_OPENED , 1 );
104
105
}
105
106
0 commit comments