|
@@ -19,18 +19,17 @@
|
|
|
|
|
|
package org.apache.hadoop.fs.s3a.read;
|
|
|
|
|
|
-import java.io.Closeable;
|
|
|
+
|
|
|
import java.io.IOException;
|
|
|
import java.io.InputStream;
|
|
|
-import java.util.ArrayList;
|
|
|
import java.util.IdentityHashMap;
|
|
|
-import java.util.List;
|
|
|
import java.util.Map;
|
|
|
|
|
|
import com.amazonaws.services.s3.model.GetObjectRequest;
|
|
|
import com.amazonaws.services.s3.model.S3Object;
|
|
|
+import org.slf4j.Logger;
|
|
|
+import org.slf4j.LoggerFactory;
|
|
|
|
|
|
-import org.apache.hadoop.fs.common.Io;
|
|
|
import org.apache.hadoop.fs.common.Validate;
|
|
|
import org.apache.hadoop.fs.s3a.Invoker;
|
|
|
import org.apache.hadoop.fs.s3a.S3AInputStream;
|
|
@@ -40,30 +39,56 @@ import org.apache.hadoop.fs.s3a.impl.ChangeTracker;
|
|
|
import org.apache.hadoop.fs.s3a.statistics.S3AInputStreamStatistics;
|
|
|
import org.apache.hadoop.fs.statistics.DurationTracker;
|
|
|
|
|
|
+import static org.apache.hadoop.fs.statistics.impl.IOStatisticsBinding.invokeTrackingDuration;
|
|
|
+import static org.apache.hadoop.io.IOUtils.cleanupWithLogger;
|
|
|
+
|
|
|
/**
|
|
|
* Encapsulates low level interactions with S3 object on AWS.
|
|
|
*/
|
|
|
-public class S3File implements Closeable {
|
|
|
+public class S3File {
|
|
|
+ private static final Logger LOG = LoggerFactory.getLogger(S3File.class);
|
|
|
|
|
|
- // Read-specific operation context.
|
|
|
+ /**
|
|
|
+ * Read-specific operation context.
|
|
|
+ */
|
|
|
private final S3AReadOpContext context;
|
|
|
|
|
|
- // S3 object attributes.
|
|
|
+ /**
|
|
|
+ * S3 object attributes.
|
|
|
+ */
|
|
|
private final S3ObjectAttributes s3Attributes;
|
|
|
|
|
|
- // Callbacks used for interacting with the underlying S3 client.
|
|
|
+ /**
|
|
|
+ * Callbacks used for interacting with the underlying S3 client.
|
|
|
+ */
|
|
|
private final S3AInputStream.InputStreamCallbacks client;
|
|
|
|
|
|
- // Used for reporting input stream access statistics.
|
|
|
+ /**
|
|
|
+ * Used for reporting input stream access statistics.
|
|
|
+ */
|
|
|
private final S3AInputStreamStatistics streamStatistics;
|
|
|
|
|
|
- // Enforces change tracking related policies.
|
|
|
+ /**
|
|
|
+ * Enforces change tracking related policies.
|
|
|
+ */
|
|
|
private final ChangeTracker changeTracker;
|
|
|
|
|
|
- // Maps a stream returned by openForRead() to the associated S3 object.
|
|
|
- // That allows us to close the object when closing the stream.
|
|
|
+ /**
|
|
|
+ * Maps a stream returned by openForRead() to the associated S3 object.
|
|
|
+ * That allows us to close the object when closing the stream.
|
|
|
+ */
|
|
|
private Map<InputStream, S3Object> s3Objects;
|
|
|
|
|
|
+ /**
|
|
|
+ * uri of the object being read.
|
|
|
+ */
|
|
|
+ private final String uri;
|
|
|
+
|
|
|
+ /**
|
|
|
+ * size of a buffer to create when draining the stream.
|
|
|
+ */
|
|
|
+ private static final int DRAIN_BUFFER_SIZE = 16384;
|
|
|
+
|
|
|
/**
|
|
|
* Initializes a new instance of the {@code S3File} class.
|
|
|
*
|
|
@@ -97,7 +122,8 @@ public class S3File implements Closeable {
|
|
|
this.client = client;
|
|
|
this.streamStatistics = streamStatistics;
|
|
|
this.changeTracker = changeTracker;
|
|
|
- this.s3Objects = new IdentityHashMap<InputStream, S3Object>();
|
|
|
+ this.s3Objects = new IdentityHashMap<>();
|
|
|
+ this.uri = this.getPath();
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -169,7 +195,6 @@ public class S3File implements Closeable {
|
|
|
.withRange(offset, offset + size - 1);
|
|
|
this.changeTracker.maybeApplyConstraint(request);
|
|
|
|
|
|
- String uri = this.getPath();
|
|
|
String operation = String.format(
|
|
|
"%s %s at %d", S3AInputStream.OPERATION_OPEN, uri, offset);
|
|
|
DurationTracker tracker = streamStatistics.initiateGetRequest();
|
|
@@ -193,18 +218,7 @@ public class S3File implements Closeable {
|
|
|
return stream;
|
|
|
}
|
|
|
|
|
|
- /**
|
|
|
- * Closes this stream and releases all acquired resources.
|
|
|
- */
|
|
|
- @Override
|
|
|
- public synchronized void close() {
|
|
|
- List<InputStream> streams = new ArrayList<InputStream>(this.s3Objects.keySet());
|
|
|
- for (InputStream stream : streams) {
|
|
|
- this.close(stream);
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- void close(InputStream inputStream) {
|
|
|
+ void close(InputStream inputStream, int numRemainingBytes) {
|
|
|
S3Object obj;
|
|
|
synchronized (this.s3Objects) {
|
|
|
obj = this.s3Objects.get(inputStream);
|
|
@@ -214,7 +228,91 @@ public class S3File implements Closeable {
|
|
|
this.s3Objects.remove(inputStream);
|
|
|
}
|
|
|
|
|
|
- Io.closeIgnoringIoException(inputStream);
|
|
|
- Io.closeIgnoringIoException(obj);
|
|
|
+ if (numRemainingBytes <= this.context.getAsyncDrainThreshold()) {
|
|
|
+ // don't bother with async io.
|
|
|
+ drain(false, "close() operation", numRemainingBytes, obj, inputStream);
|
|
|
+ } else {
|
|
|
+ LOG.debug("initiating asynchronous drain of {} bytes", numRemainingBytes);
|
|
|
+ // schedule an async drain/abort with references to the fields so they
|
|
|
+ // can be reused
|
|
|
+ client.submit(() -> drain(false, "close() operation", numRemainingBytes, obj, inputStream));
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * drain the stream. This method is intended to be
|
|
|
+ * used directly or asynchronously, and measures the
|
|
|
+ * duration of the operation in the stream statistics.
|
|
|
+ *
|
|
|
+ * @param shouldAbort force an abort; used if explicitly requested.
|
|
|
+ * @param reason reason for stream being closed; used in messages
|
|
|
+ * @param remaining remaining bytes
|
|
|
+ * @param requestObject http request object;
|
|
|
+ * @param inputStream stream to close.
|
|
|
+ * @return was the stream aborted?
|
|
|
+ */
|
|
|
+ private boolean drain(
|
|
|
+ final boolean shouldAbort,
|
|
|
+ final String reason,
|
|
|
+ final long remaining,
|
|
|
+ final S3Object requestObject,
|
|
|
+ final InputStream inputStream) {
|
|
|
+
|
|
|
+ try {
|
|
|
+ return invokeTrackingDuration(streamStatistics.initiateInnerStreamClose(shouldAbort),
|
|
|
+ () -> drainOrAbortHttpStream(shouldAbort, reason, remaining, requestObject, inputStream));
|
|
|
+ } catch (IOException e) {
|
|
|
+ // this is only here because invokeTrackingDuration() has it in its
|
|
|
+ // signature
|
|
|
+ return shouldAbort;
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Drain or abort the inner stream.
|
|
|
+ * Exceptions are swallowed.
|
|
|
+ * If a close() is attempted and fails, the operation escalates to
|
|
|
+ * an abort.
|
|
|
+ *
|
|
|
+ * @param shouldAbort force an abort; used if explicitly requested.
|
|
|
+ * @param reason reason for stream being closed; used in messages
|
|
|
+ * @param remaining remaining bytes
|
|
|
+ * @param requestObject http request object
|
|
|
+ * @param inputStream stream to close.
|
|
|
+ * @return was the stream aborted?
|
|
|
+ */
|
|
|
+ private boolean drainOrAbortHttpStream(
|
|
|
+ boolean shouldAbort,
|
|
|
+ final String reason,
|
|
|
+ final long remaining,
|
|
|
+ final S3Object requestObject,
|
|
|
+ final InputStream inputStream) {
|
|
|
+
|
|
|
+ if (!shouldAbort && remaining > 0) {
|
|
|
+ try {
|
|
|
+ long drained = 0;
|
|
|
+ byte[] buffer = new byte[DRAIN_BUFFER_SIZE];
|
|
|
+ while (true) {
|
|
|
+ final int count = inputStream.read(buffer);
|
|
|
+ if (count < 0) {
|
|
|
+ // no more data is left
|
|
|
+ break;
|
|
|
+ }
|
|
|
+ drained += count;
|
|
|
+ }
|
|
|
+ LOG.debug("Drained stream of {} bytes", drained);
|
|
|
+ } catch (Exception e) {
|
|
|
+ // exception escalates to an abort
|
|
|
+ LOG.debug("When closing {} stream for {}, will abort the stream", uri, reason, e);
|
|
|
+ shouldAbort = true;
|
|
|
+ }
|
|
|
+ }
|
|
|
+ cleanupWithLogger(LOG, inputStream);
|
|
|
+ cleanupWithLogger(LOG, requestObject);
|
|
|
+ streamStatistics.streamClose(shouldAbort, remaining);
|
|
|
+
|
|
|
+ LOG.debug("Stream {} {}: {}; remaining={}", uri, (shouldAbort ? "aborted" : "closed"), reason,
|
|
|
+ remaining);
|
|
|
+ return shouldAbort;
|
|
|
}
|
|
|
}
|