32
32
import org .elasticsearch .common .blobstore .BlobContainer ;
33
33
import org .elasticsearch .common .blobstore .BlobPath ;
34
34
import org .elasticsearch .common .bytes .BytesReference ;
35
+ import org .elasticsearch .common .collect .Tuple ;
35
36
import org .elasticsearch .common .io .Streams ;
36
37
import org .elasticsearch .common .lucene .store .ByteArrayIndexInput ;
37
38
import org .elasticsearch .common .lucene .store .InputStreamIndexInput ;
58
59
import java .util .Arrays ;
59
60
import java .util .Locale ;
60
61
import java .util .Objects ;
62
+ import java .util .Optional ;
61
63
import java .util .concurrent .atomic .AtomicBoolean ;
62
64
import java .util .concurrent .atomic .AtomicInteger ;
63
65
import java .util .concurrent .atomic .AtomicLong ;
70
72
import static org .elasticsearch .repositories .s3 .S3ClientSettings .READ_TIMEOUT_SETTING ;
71
73
import static org .hamcrest .Matchers .anyOf ;
72
74
import static org .hamcrest .Matchers .containsString ;
75
+ import static org .hamcrest .Matchers .either ;
73
76
import static org .hamcrest .Matchers .equalTo ;
77
+ import static org .hamcrest .Matchers .greaterThanOrEqualTo ;
74
78
import static org .hamcrest .Matchers .instanceOf ;
75
79
import static org .hamcrest .Matchers .is ;
76
80
import static org .hamcrest .Matchers .lessThan ;
81
+ import static org .hamcrest .Matchers .lessThanOrEqualTo ;
77
82
78
83
/**
79
84
* This class tests how a {@link S3BlobContainer} and its underlying AWS S3 client are retrying requests when reading or writing blobs.
80
85
*/
81
86
@ SuppressForbidden (reason = "use a http server" )
82
87
public class S3BlobContainerRetriesTests extends ESTestCase {
83
88
89
+ private static final long MAX_RANGE_VAL = Long .MAX_VALUE - 1 ;
90
+
84
91
private HttpServer httpServer ;
85
92
private S3Service service ;
86
93
@@ -139,8 +146,19 @@ private BlobContainer createBlobContainer(final @Nullable Integer maxRetries,
139
146
140
147
public void testReadNonexistentBlobThrowsNoSuchFileException () {
141
148
final BlobContainer blobContainer = createBlobContainer (between (1 , 5 ), null , null , null );
142
- final Exception exception = expectThrows (NoSuchFileException .class , () -> blobContainer .readBlob ("read_nonexistent_blob" ));
149
+ final long position = randomLongBetween (0 , MAX_RANGE_VAL );
150
+ final int length = randomIntBetween (0 , Math .toIntExact (Math .min (Integer .MAX_VALUE , MAX_RANGE_VAL - position )));
151
+ final Exception exception = expectThrows (NoSuchFileException .class ,
152
+ () -> {
153
+ if (randomBoolean ()) {
154
+ blobContainer .readBlob ("read_nonexistent_blob" );
155
+ } else {
156
+ blobContainer .readBlob ("read_nonexistent_blob" , 0 , 1 );
157
+ }
158
+ });
143
159
assertThat (exception .getMessage ().toLowerCase (Locale .ROOT ), containsString ("blob object [read_nonexistent_blob] not found" ));
160
+ assertThat (expectThrows (NoSuchFileException .class , () -> blobContainer .readBlob ("read_nonexistent_blob" , position , length ))
161
+ .getMessage ().toLowerCase (Locale .ROOT ), containsString ("blob object [read_nonexistent_blob] not found" ));
144
162
}
145
163
146
164
public void testReadBlobWithRetries () throws Exception {
@@ -153,6 +171,7 @@ public void testReadBlobWithRetries() throws Exception {
153
171
if (countDown .countDown ()) {
154
172
final int rangeStart = getRangeStart (exchange );
155
173
assertThat (rangeStart , lessThan (bytes .length ));
174
+ assertEquals (Optional .empty (), getRangeEnd (exchange ));
156
175
exchange .getResponseHeaders ().add ("Content-Type" , "text/plain; charset=utf-8" );
157
176
exchange .sendResponseHeaders (HttpStatus .SC_OK , bytes .length - rangeStart );
158
177
exchange .getResponseBody ().write (bytes , rangeStart , bytes .length - rangeStart );
@@ -173,8 +192,85 @@ public void testReadBlobWithRetries() throws Exception {
173
192
final TimeValue readTimeout = TimeValue .timeValueMillis (between (100 , 500 ));
174
193
final BlobContainer blobContainer = createBlobContainer (maxRetries , readTimeout , null , null );
175
194
try (InputStream inputStream = blobContainer .readBlob ("read_blob_max_retries" )) {
176
- assertArrayEquals (bytes , BytesReference .toBytes (Streams .readFully (inputStream )));
177
- assertThat (countDown .isCountedDown (), is (true ));
195
+ final int readLimit ;
196
+ final InputStream wrappedStream ;
197
+ if (randomBoolean ()) {
198
+ // read stream only partly
199
+ readLimit = randomIntBetween (0 , bytes .length );
200
+ wrappedStream = Streams .limitStream (inputStream , readLimit );
201
+ } else {
202
+ readLimit = bytes .length ;
203
+ wrappedStream = inputStream ;
204
+ }
205
+ final byte [] bytesRead = BytesReference .toBytes (Streams .readFully (wrappedStream ));
206
+ logger .info ("maxRetries={}, readLimit={}, byteSize={}, bytesRead={}" ,
207
+ maxRetries , readLimit , bytes .length , bytesRead .length );
208
+ assertArrayEquals (Arrays .copyOfRange (bytes , 0 , readLimit ), bytesRead );
209
+ if (readLimit < bytes .length ) {
210
+ // we might have completed things based on an incomplete response, and we're happy with that
211
+ } else {
212
+ assertTrue (countDown .isCountedDown ());
213
+ }
214
+ }
215
+ }
216
+
217
+ public void testReadRangeBlobWithRetries () throws Exception {
218
+ final int maxRetries = randomInt (5 );
219
+ final CountDown countDown = new CountDown (maxRetries + 1 );
220
+
221
+ final byte [] bytes = randomBlobContent ();
222
+ httpServer .createContext ("/bucket/read_range_blob_max_retries" , exchange -> {
223
+ Streams .readFully (exchange .getRequestBody ());
224
+ if (countDown .countDown ()) {
225
+ final int rangeStart = getRangeStart (exchange );
226
+ assertThat (rangeStart , lessThan (bytes .length ));
227
+ assertTrue (getRangeEnd (exchange ).isPresent ());
228
+ final int rangeEnd = getRangeEnd (exchange ).get ();
229
+ assertThat (rangeEnd , greaterThanOrEqualTo (rangeStart ));
230
+ // adapt range end to be compliant to https://www.w3.org/Protocols/rfc2616/rfc2616-sec14.html#sec14.35
231
+ final int effectiveRangeEnd = Math .min (bytes .length - 1 , rangeEnd );
232
+ final int length = (effectiveRangeEnd - rangeStart ) + 1 ;
233
+ exchange .getResponseHeaders ().add ("Content-Type" , "text/plain; charset=utf-8" );
234
+ exchange .sendResponseHeaders (HttpStatus .SC_OK , length );
235
+ exchange .getResponseBody ().write (bytes , rangeStart , length );
236
+ exchange .close ();
237
+ return ;
238
+ }
239
+ if (randomBoolean ()) {
240
+ exchange .sendResponseHeaders (randomFrom (HttpStatus .SC_INTERNAL_SERVER_ERROR , HttpStatus .SC_BAD_GATEWAY ,
241
+ HttpStatus .SC_SERVICE_UNAVAILABLE , HttpStatus .SC_GATEWAY_TIMEOUT ), -1 );
242
+ } else if (randomBoolean ()) {
243
+ sendIncompleteContent (exchange , bytes );
244
+ }
245
+ if (randomBoolean ()) {
246
+ exchange .close ();
247
+ }
248
+ });
249
+
250
+ final TimeValue readTimeout = TimeValue .timeValueMillis (between (100 , 500 ));
251
+ final BlobContainer blobContainer = createBlobContainer (maxRetries , readTimeout , null , null );
252
+ final int position = randomIntBetween (0 , bytes .length - 1 );
253
+ final int length = randomIntBetween (0 , randomBoolean () ? bytes .length : Integer .MAX_VALUE );
254
+ try (InputStream inputStream = blobContainer .readBlob ("read_range_blob_max_retries" , position , length )) {
255
+ final int readLimit ;
256
+ final InputStream wrappedStream ;
257
+ if (randomBoolean ()) {
258
+ // read stream only partly
259
+ readLimit = randomIntBetween (0 , length );
260
+ wrappedStream = Streams .limitStream (inputStream , readLimit );
261
+ } else {
262
+ readLimit = length ;
263
+ wrappedStream = inputStream ;
264
+ }
265
+ final byte [] bytesRead = BytesReference .toBytes (Streams .readFully (wrappedStream ));
266
+ logger .info ("maxRetries={}, position={}, length={}, readLimit={}, byteSize={}, bytesRead={}" ,
267
+ maxRetries , position , length , readLimit , bytes .length , bytesRead .length );
268
+ assertArrayEquals (Arrays .copyOfRange (bytes , position , Math .min (bytes .length , position + readLimit )), bytesRead );
269
+ if (readLimit == 0 || (readLimit < length && readLimit == bytesRead .length )) {
270
+ // we might have completed things based on an incomplete response, and we're happy with that
271
+ } else {
272
+ assertTrue (countDown .isCountedDown ());
273
+ }
178
274
}
179
275
}
180
276
@@ -194,12 +290,18 @@ public void testReadBlobWithReadTimeouts() {
194
290
final byte [] bytes = randomBlobContent ();
195
291
httpServer .createContext ("/bucket/read_blob_incomplete" , exchange -> sendIncompleteContent (exchange , bytes ));
196
292
197
- exception = expectThrows (SocketTimeoutException .class , () -> {
198
- try (InputStream stream = blobContainer .readBlob ("read_blob_incomplete" )) {
293
+ final int position = randomIntBetween (0 , bytes .length - 1 );
294
+ final int length = randomIntBetween (1 , randomBoolean () ? bytes .length : Integer .MAX_VALUE );
295
+ exception = expectThrows (IOException .class , () -> {
296
+ try (InputStream stream = randomBoolean () ?
297
+ blobContainer .readBlob ("read_blob_incomplete" ) :
298
+ blobContainer .readBlob ("read_blob_incomplete" , position , length )) {
199
299
Streams .readFully (stream );
200
300
}
201
301
});
202
- assertThat (exception .getMessage ().toLowerCase (Locale .ROOT ), containsString ("read timed out" ));
302
+ assertThat (exception , either (instanceOf (SocketTimeoutException .class )).or (instanceOf (ConnectionClosedException .class )));
303
+ assertThat (exception .getMessage ().toLowerCase (Locale .ROOT ), either (containsString ("read timed out" )).or (
304
+ containsString ("premature end of chunk coded message body: closing chunk expected" )));
203
305
assertThat (exception .getSuppressed ().length , equalTo (maxRetries ));
204
306
}
205
307
@@ -209,7 +311,14 @@ public void testReadBlobWithNoHttpResponse() {
209
311
// HTTP server closes connection immediately
210
312
httpServer .createContext ("/bucket/read_blob_no_response" , HttpExchange ::close );
211
313
212
- Exception exception = expectThrows (SdkClientException .class , () -> blobContainer .readBlob ("read_blob_no_response" ));
314
+ Exception exception = expectThrows (SdkClientException .class ,
315
+ () -> {
316
+ if (randomBoolean ()) {
317
+ blobContainer .readBlob ("read_blob_no_response" );
318
+ } else {
319
+ blobContainer .readBlob ("read_blob_no_response" , 0 , 1 );
320
+ }
321
+ });
213
322
assertThat (exception .getMessage ().toLowerCase (Locale .ROOT ), containsString ("the target server failed to respond" ));
214
323
assertThat (exception .getCause (), instanceOf (NoHttpResponseException .class ));
215
324
assertThat (exception .getSuppressed ().length , equalTo (0 ));
@@ -227,12 +336,15 @@ public void testReadBlobWithPrematureConnectionClose() {
227
336
});
228
337
229
338
final Exception exception = expectThrows (ConnectionClosedException .class , () -> {
230
- try (InputStream stream = blobContainer .readBlob ("read_blob_incomplete" )) {
339
+ try (InputStream stream = randomBoolean () ?
340
+ blobContainer .readBlob ("read_blob_incomplete" , 0 , 1 ):
341
+ blobContainer .readBlob ("read_blob_incomplete" )) {
231
342
Streams .readFully (stream );
232
343
}
233
344
});
234
345
assertThat (exception .getMessage ().toLowerCase (Locale .ROOT ),
235
- containsString ("premature end of content-length delimited message body" ));
346
+ either (containsString ("premature end of chunk coded message body: closing chunk expected" ))
347
+ .or (containsString ("premature end of content-length delimited message body" )));
236
348
assertThat (exception .getSuppressed ().length , equalTo (Math .min (S3RetryingInputStream .MAX_SUPPRESSED_EXCEPTIONS , maxRetries )));
237
349
}
238
350
@@ -397,23 +509,47 @@ private static byte[] randomBlobContent() {
397
509
return randomByteArrayOfLength (randomIntBetween (1 , frequently () ? 512 : 1 << 20 )); // rarely up to 1mb
398
510
}
399
511
400
- private static int getRangeStart (HttpExchange exchange ) {
512
+ private static Tuple < Long , Long > getRange (HttpExchange exchange ) {
401
513
final String rangeHeader = exchange .getRequestHeaders ().getFirst ("Range" );
402
514
if (rangeHeader == null ) {
403
- return 0 ;
515
+ return Tuple . tuple ( 0L , MAX_RANGE_VAL ) ;
404
516
}
405
517
406
- final Matcher matcher = Pattern .compile ("^bytes=([0-9]+)-9223372036854775806 $" ).matcher (rangeHeader );
518
+ final Matcher matcher = Pattern .compile ("^bytes=([0-9]+)-([0-9]+) $" ).matcher (rangeHeader );
407
519
assertTrue (rangeHeader + " matches expected pattern" , matcher .matches ());
408
- return Math .toIntExact (Long .parseLong (matcher .group (1 )));
520
+ long rangeStart = Long .parseLong (matcher .group (1 ));
521
+ long rangeEnd = Long .parseLong (matcher .group (2 ));
522
+ assertThat (rangeStart , lessThanOrEqualTo (rangeEnd ));
523
+ return Tuple .tuple (rangeStart , rangeEnd );
524
+ }
525
+
526
+ private static int getRangeStart (HttpExchange exchange ) {
527
+ return Math .toIntExact (getRange (exchange ).v1 ());
528
+ }
529
+
530
+ private static Optional <Integer > getRangeEnd (HttpExchange exchange ) {
531
+ final long rangeEnd = getRange (exchange ).v2 ();
532
+ if (rangeEnd == MAX_RANGE_VAL ) {
533
+ return Optional .empty ();
534
+ }
535
+ return Optional .of (Math .toIntExact (rangeEnd ));
409
536
}
410
537
411
538
private static void sendIncompleteContent (HttpExchange exchange , byte [] bytes ) throws IOException {
412
539
final int rangeStart = getRangeStart (exchange );
413
540
assertThat (rangeStart , lessThan (bytes .length ));
541
+ final Optional <Integer > rangeEnd = getRangeEnd (exchange );
542
+ final int length ;
543
+ if (rangeEnd .isPresent ()) {
544
+ // adapt range end to be compliant to https://www.w3.org/Protocols/rfc2616/rfc2616-sec14.html#sec14.35
545
+ final int effectiveRangeEnd = Math .min (rangeEnd .get (), bytes .length - 1 );
546
+ length = effectiveRangeEnd - rangeStart ;
547
+ } else {
548
+ length = bytes .length - rangeStart - 1 ;
549
+ }
414
550
exchange .getResponseHeaders ().add ("Content-Type" , "text/plain; charset=utf-8" );
415
- exchange .sendResponseHeaders (HttpStatus .SC_OK , bytes . length - rangeStart );
416
- final int bytesToSend = randomIntBetween (0 , bytes . length - rangeStart - 1 );
551
+ exchange .sendResponseHeaders (HttpStatus .SC_OK , length );
552
+ final int bytesToSend = randomIntBetween (0 , length - 1 );
417
553
if (bytesToSend > 0 ) {
418
554
exchange .getResponseBody ().write (bytes , rangeStart , bytesToSend );
419
555
}
0 commit comments