22
22
import org .apache .hadoop .fs .FSInputStream ;
23
23
import org .apache .hadoop .fs .FileSystem ;
24
24
import org .apache .hadoop .fs .FilterFileSystem ;
25
+ import org .apache .hadoop .fs .FutureDataInputStreamBuilder ;
25
26
import org .apache .hadoop .fs .Path ;
27
+ import org .apache .hadoop .fs .PathHandle ;
28
+ import org .apache .hadoop .fs .impl .OpenFileParameters ;
29
+ import org .jetbrains .annotations .NotNull ;
26
30
import org .slf4j .Logger ;
27
31
import org .slf4j .LoggerFactory ;
28
32
32
36
import java .nio .channels .Channels ;
33
37
import java .nio .channels .SeekableByteChannel ;
34
38
import java .util .Map ;
39
+ import java .util .concurrent .CompletableFuture ;
40
+ import java .util .concurrent .CompletionException ;
35
41
36
42
/**
37
43
* A hadoop {@link FileSystem} that support files decryption (encryption is currently not supported).
@@ -42,6 +48,7 @@ public class EncryptedFileSystem extends FilterFileSystem {
42
48
private static final String FS_SCHEME = CONF_PREFIX + "scheme" ;
43
49
private static final String FS_IMPL = CONF_PREFIX + "impl" ;
44
50
private static final String DECRYPTOR_IMPL = CONF_PREFIX + "decryptor.impl" ;
51
+ private static final int DEFAULT_BUFFER_SIZE = 4096 ;
45
52
46
53
private static final Logger LOG = LoggerFactory .getLogger (EncryptedFileSystem .class );
47
54
@@ -103,6 +110,82 @@ public FSDataInputStream open(Path path, int bufferSize) throws IOException {
103
110
return new FSDataInputStream (new SeekableByteChannelFSInputStream (decryptor .open (fs , path , bufferSize )));
104
111
}
105
112
113
+ /**
114
+ * Opens a file asynchronously and returns a {@link FutureDataInputStreamBuilder}
115
+ * to build a {@link FSDataInputStream} for the specified {@link Path}.
116
+ *
117
+ * <p>This implementation returns a builder that constructs an input stream by using a decryptor
118
+ * to open the file through a {@link SeekableByteChannelFSInputStream}. The file is read
119
+ * with a buffer size of 4096 bytes.</p>
120
+ *
121
+ * @param path the {@link Path} of the file to open
122
+ * @return a {@link FutureDataInputStreamBuilder} that asynchronously builds a {@link FSDataInputStream}
123
+ * @throws UnsupportedOperationException if the operation is not supported
124
+ */
125
+ @ Override
126
+ public FutureDataInputStreamBuilder openFile (Path path ) throws UnsupportedOperationException {
127
+ return new FutureDataInputStreamBuilder () {
128
+ @ Override
129
+ public CompletableFuture <FSDataInputStream > build ()
130
+ throws IllegalArgumentException , UnsupportedOperationException {
131
+ return CompletableFuture .supplyAsync (() -> {
132
+ try {
133
+ return new FSDataInputStream (
134
+ new SeekableByteChannelFSInputStream (decryptor .open (fs , path , DEFAULT_BUFFER_SIZE )));
135
+ } catch (Exception e ) {
136
+ throw new CompletionException (e );
137
+ }
138
+ });
139
+ }
140
+
141
+ @ Override
142
+ public FutureDataInputStreamBuilder opt (@ NotNull String s , @ NotNull String s1 ) {
143
+ return this ;
144
+ }
145
+
146
+ @ Override
147
+ public FutureDataInputStreamBuilder opt (@ NotNull String s , @ NotNull String ... strings ) {
148
+ return this ;
149
+ }
150
+
151
+ @ Override
152
+ public FutureDataInputStreamBuilder must (@ NotNull String s , @ NotNull String s1 ) {
153
+ return this ;
154
+ }
155
+
156
+ @ Override
157
+ public FutureDataInputStreamBuilder must (@ NotNull String s , @ NotNull String ... strings ) {
158
+ return this ;
159
+ }
160
+ };
161
+ }
162
+
163
+ /**
164
+ * Opens a file asynchronously using the provided {@link Path}, and returns
165
+ * a {@link CompletableFuture} that supplies a {@link FSDataInputStream}.
166
+ *
167
+ * <p>This method uses a decryptor to open the file and wraps it in a {@link SeekableByteChannelFSInputStream}.
168
+ * It uses the buffer size specified in the {@code parameters}; if the buffer size is not greater than zero,
169
+ * a default of 4096 bytes is used.</p>
170
+ *
171
+ * @param path the {@link Path} to the file to open
172
+ * @param parameters the {@link OpenFileParameters} containing optional configuration, such as buffer size
173
+ * @return a {@link CompletableFuture} that will complete with the {@link FSDataInputStream}
174
+ * @throws CompletionException if an exception occurs during file opening
175
+ */
176
+ @ Override
177
+ protected CompletableFuture <FSDataInputStream > openFileWithOptions (Path path , OpenFileParameters parameters ) {
178
+ return CompletableFuture .supplyAsync (() -> {
179
+ try {
180
+ int bufferSize = parameters .getBufferSize () > 0 ? parameters .getBufferSize () : 4096 ;
181
+ return new FSDataInputStream (
182
+ new SeekableByteChannelFSInputStream (decryptor .open (fs , path , bufferSize )));
183
+ } catch (Exception e ) {
184
+ throw new CompletionException (e );
185
+ }
186
+ });
187
+ }
188
+
106
189
/**
107
190
* A {@link FSInputStream} implementation backed by a {@link SeekableByteChannel}.
108
191
*/
0 commit comments