|
@@ -32,6 +32,8 @@ import org.apache.hadoop.hdds.scm.storage.BufferPool;
|
|
|
import org.apache.hadoop.ozone.OzoneConfigKeys;
|
|
|
import org.apache.hadoop.hdds.scm.container.common.helpers.ExcludeList;
|
|
|
import org.apache.hadoop.hdds.scm.pipeline.PipelineID;
|
|
|
+import org.apache.hadoop.io.retry.RetryPolicies;
|
|
|
+import org.apache.hadoop.io.retry.RetryPolicy;
|
|
|
import org.apache.hadoop.ozone.client.OzoneClientUtils;
|
|
|
import org.apache.hadoop.ozone.om.helpers.*;
|
|
|
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType;
|
|
@@ -45,6 +47,7 @@ import org.slf4j.Logger;
|
|
|
import org.slf4j.LoggerFactory;
|
|
|
|
|
|
import java.io.IOException;
|
|
|
+import java.io.InterruptedIOException;
|
|
|
import java.io.OutputStream;
|
|
|
import java.util.ArrayList;
|
|
|
import java.util.List;
|
|
@@ -87,6 +90,8 @@ public class KeyOutputStream extends OutputStream {
|
|
|
private OmMultipartCommitUploadPartInfo commitUploadPartInfo;
|
|
|
private FileEncryptionInfo feInfo;
|
|
|
private ExcludeList excludeList;
|
|
|
+ private final RetryPolicy retryPolicy;
|
|
|
+ private int retryCount;
|
|
|
/**
|
|
|
* A constructor for testing purpose only.
|
|
|
*/
|
|
@@ -111,6 +116,8 @@ public class KeyOutputStream extends OutputStream {
|
|
|
OzoneConfigKeys.OZONE_CLIENT_CHECKSUM_TYPE_DEFAULT);
|
|
|
this.bytesPerChecksum = OzoneConfigKeys
|
|
|
.OZONE_CLIENT_BYTES_PER_CHECKSUM_DEFAULT_BYTES; // Default is 1MB
|
|
|
+ this.retryPolicy = RetryPolicies.TRY_ONCE_THEN_FAIL;
|
|
|
+ retryCount = 0;
|
|
|
}
|
|
|
|
|
|
@VisibleForTesting
|
|
@@ -147,7 +154,7 @@ public class KeyOutputStream extends OutputStream {
|
|
|
String requestId, ReplicationFactor factor, ReplicationType type,
|
|
|
long bufferFlushSize, long bufferMaxSize, long size, long watchTimeout,
|
|
|
ChecksumType checksumType, int bytesPerChecksum,
|
|
|
- String uploadID, int partNumber, boolean isMultipart) {
|
|
|
+ String uploadID, int partNumber, boolean isMultipart, int maxRetryCount) {
|
|
|
this.streamEntries = new ArrayList<>();
|
|
|
this.currentStreamIndex = 0;
|
|
|
this.omClient = omClient;
|
|
@@ -183,6 +190,8 @@ public class KeyOutputStream extends OutputStream {
|
|
|
this.bufferPool =
|
|
|
new BufferPool(chunkSize, (int)streamBufferMaxSize / chunkSize);
|
|
|
this.excludeList = new ExcludeList();
|
|
|
+ this.retryPolicy = OzoneClientUtils.createRetryPolicy(maxRetryCount);
|
|
|
+ this.retryCount = 0;
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -308,23 +317,18 @@ public class KeyOutputStream extends OutputStream {
|
|
|
current.write(b, off, writeLen);
|
|
|
}
|
|
|
} catch (IOException ioe) {
|
|
|
- Throwable t = checkForException(ioe);
|
|
|
- if (t != null) {
|
|
|
- // for the current iteration, totalDataWritten - currentPos gives the
|
|
|
- // amount of data already written to the buffer
|
|
|
-
|
|
|
- // In the retryPath, the total data to be written will always be equal
|
|
|
- // to or less than the max length of the buffer allocated.
|
|
|
- // The len specified here is the combined sum of the data length of
|
|
|
- // the buffers
|
|
|
- Preconditions.checkState(!retry || len <= streamBufferMaxSize);
|
|
|
- writeLen = retry ? (int) len :
|
|
|
- (int) (current.getWrittenDataLength() - currentPos);
|
|
|
- LOG.debug("writeLen {}, total len {}", writeLen, len);
|
|
|
- handleException(current, currentStreamIndex, t);
|
|
|
- } else {
|
|
|
- throw ioe;
|
|
|
- }
|
|
|
+ // for the current iteration, totalDataWritten - currentPos gives the
|
|
|
+ // amount of data already written to the buffer
|
|
|
+
|
|
|
+ // In the retryPath, the total data to be written will always be equal
|
|
|
+ // to or less than the max length of the buffer allocated.
|
|
|
+ // The len specified here is the combined sum of the data length of
|
|
|
+ // the buffers
|
|
|
+ Preconditions.checkState(!retry || len <= streamBufferMaxSize);
|
|
|
+ writeLen = retry ? (int) len :
|
|
|
+ (int) (current.getWrittenDataLength() - currentPos);
|
|
|
+ LOG.debug("writeLen {}, total len {}", writeLen, len);
|
|
|
+ handleException(current, currentStreamIndex, ioe);
|
|
|
}
|
|
|
if (current.getRemaining() <= 0) {
|
|
|
// since the current block is already written close the stream.
|
|
@@ -390,7 +394,8 @@ public class KeyOutputStream extends OutputStream {
|
|
|
* @throws IOException Throws IOException if Write fails
|
|
|
*/
|
|
|
private void handleException(BlockOutputStreamEntry streamEntry,
|
|
|
- int streamIndex, Throwable exception) throws IOException {
|
|
|
+ int streamIndex, IOException exception) throws IOException {
|
|
|
+ Throwable t = checkForException(exception);
|
|
|
boolean retryFailure = checkForRetryFailure(exception);
|
|
|
boolean closedContainerException = false;
|
|
|
if (!retryFailure) {
|
|
@@ -413,9 +418,9 @@ public class KeyOutputStream extends OutputStream {
|
|
|
if (!failedServers.isEmpty()) {
|
|
|
excludeList.addDatanodes(failedServers);
|
|
|
}
|
|
|
- if (checkIfContainerIsClosed(exception)) {
|
|
|
+ if (checkIfContainerIsClosed(t)) {
|
|
|
excludeList.addConatinerId(ContainerID.valueof(containerId));
|
|
|
- } else if (retryFailure || exception instanceof TimeoutException) {
|
|
|
+ } else if (retryFailure || t instanceof TimeoutException) {
|
|
|
pipelineId = streamEntry.getPipeline().getId();
|
|
|
excludeList.addPipeline(pipelineId);
|
|
|
}
|
|
@@ -425,7 +430,7 @@ public class KeyOutputStream extends OutputStream {
|
|
|
// If the data is still cached in the underlying stream, we need to
|
|
|
// allocate new block and write this data in the datanode.
|
|
|
currentStreamIndex += 1;
|
|
|
- handleWrite(null, 0, bufferedDataLen, true);
|
|
|
+ handleRetry(exception, bufferedDataLen);
|
|
|
}
|
|
|
if (totalSuccessfulFlushedData == 0) {
|
|
|
streamEntries.remove(streamIndex);
|
|
@@ -448,6 +453,43 @@ public class KeyOutputStream extends OutputStream {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ private void handleRetry(IOException exception, long len) throws IOException {
|
|
|
+ RetryPolicy.RetryAction action;
|
|
|
+ try {
|
|
|
+ action = retryPolicy
|
|
|
+ .shouldRetry(exception, retryCount, 0, true);
|
|
|
+ } catch (Exception e) {
|
|
|
+ throw e instanceof IOException ? (IOException) e : new IOException(e);
|
|
|
+ }
|
|
|
+ if (action.action == RetryPolicy.RetryAction.RetryDecision.FAIL) {
|
|
|
+ if (action.reason != null) {
|
|
|
+ LOG.error("Retry request failed. " + action.reason,
|
|
|
+ exception);
|
|
|
+ }
|
|
|
+ throw exception;
|
|
|
+ }
|
|
|
+
|
|
|
+ // Throw the exception if the thread is interrupted
|
|
|
+ if (Thread.currentThread().isInterrupted()) {
|
|
|
+ LOG.warn("Interrupted while trying for retry");
|
|
|
+ throw exception;
|
|
|
+ }
|
|
|
+ Preconditions.checkArgument(
|
|
|
+ action.action == RetryPolicy.RetryAction.RetryDecision.RETRY);
|
|
|
+ if (action.delayMillis > 0) {
|
|
|
+ try {
|
|
|
+ Thread.sleep(action.delayMillis);
|
|
|
+ } catch (InterruptedException e) {
|
|
|
+ throw (IOException) new InterruptedIOException(
|
|
|
+ "Interrupted: action=" + action + ", retry policy=" + retryPolicy)
|
|
|
+ .initCause(e);
|
|
|
+ }
|
|
|
+ }
|
|
|
+ retryCount++;
|
|
|
+ LOG.trace("Retrying Write request. Already tried "
|
|
|
+ + retryCount + " time(s); retry policy is " + retryPolicy);
|
|
|
+ handleWrite(null, 0, len, true);
|
|
|
+ }
|
|
|
/**
|
|
|
* Checks if the provided exception signifies retry failure in ratis client.
|
|
|
* In case of retry failure, ratis client throws RaftRetryFailureException
|
|
@@ -462,7 +504,7 @@ public class KeyOutputStream extends OutputStream {
|
|
|
return t instanceof ContainerNotOpenException;
|
|
|
}
|
|
|
|
|
|
- private Throwable checkForException(IOException ioe) {
|
|
|
+ private Throwable checkForException(IOException ioe) throws IOException {
|
|
|
Throwable t = ioe.getCause();
|
|
|
while (t != null) {
|
|
|
for (Class<? extends Exception> cls : OzoneClientUtils
|
|
@@ -473,7 +515,7 @@ public class KeyOutputStream extends OutputStream {
|
|
|
}
|
|
|
t = t.getCause();
|
|
|
}
|
|
|
- return null;
|
|
|
+ throw ioe;
|
|
|
}
|
|
|
|
|
|
private long getKeyLength() {
|
|
@@ -512,36 +554,30 @@ public class KeyOutputStream extends OutputStream {
|
|
|
if (streamEntries.size() == 0) {
|
|
|
return;
|
|
|
}
|
|
|
- int size = streamEntries.size();
|
|
|
- int streamIndex =
|
|
|
- currentStreamIndex >= size ? size - 1 : currentStreamIndex;
|
|
|
- BlockOutputStreamEntry entry = streamEntries.get(streamIndex);
|
|
|
- if (entry != null) {
|
|
|
- try {
|
|
|
- Collection<DatanodeDetails> failedServers = entry.getFailedServers();
|
|
|
-
|
|
|
- // failed servers can be null in case there is no data written in the
|
|
|
- // stream
|
|
|
- if (failedServers != null && !failedServers.isEmpty()) {
|
|
|
- excludeList.addDatanodes(failedServers);
|
|
|
- }
|
|
|
- if (close) {
|
|
|
- entry.close();
|
|
|
- } else {
|
|
|
- entry.flush();
|
|
|
- }
|
|
|
- } catch (IOException ioe) {
|
|
|
- Throwable t = checkForException(ioe);
|
|
|
- if (t != null) {
|
|
|
- // This call will allocate a new streamEntry and write the Data.
|
|
|
- // Close needs to be retried on the newly allocated streamEntry as
|
|
|
- // as well.
|
|
|
- handleException(entry, streamIndex, t);
|
|
|
- handleFlushOrClose(close);
|
|
|
- } else {
|
|
|
- throw ioe;
|
|
|
+ while (true) {
|
|
|
+ int size = streamEntries.size();
|
|
|
+ int streamIndex =
|
|
|
+ currentStreamIndex >= size ? size - 1 : currentStreamIndex;
|
|
|
+ BlockOutputStreamEntry entry = streamEntries.get(streamIndex);
|
|
|
+ if (entry != null) {
|
|
|
+ try {
|
|
|
+ Collection<DatanodeDetails> failedServers = entry.getFailedServers();
|
|
|
+ // failed servers can be null in case there is no data written in the
|
|
|
+ // stream
|
|
|
+ if (failedServers != null && !failedServers.isEmpty()) {
|
|
|
+ excludeList.addDatanodes(failedServers);
|
|
|
+ }
|
|
|
+ if (close) {
|
|
|
+ entry.close();
|
|
|
+ } else {
|
|
|
+ entry.flush();
|
|
|
+ }
|
|
|
+ } catch (IOException ioe) {
|
|
|
+ handleException(entry, streamIndex, ioe);
|
|
|
+ continue;
|
|
|
}
|
|
|
}
|
|
|
+ break;
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -616,6 +652,7 @@ public class KeyOutputStream extends OutputStream {
|
|
|
private String multipartUploadID;
|
|
|
private int multipartNumber;
|
|
|
private boolean isMultipartKey;
|
|
|
+ private int maxRetryCount;
|
|
|
|
|
|
|
|
|
public Builder setMultipartUploadID(String uploadID) {
|
|
@@ -704,11 +741,17 @@ public class KeyOutputStream extends OutputStream {
|
|
|
return this;
|
|
|
}
|
|
|
|
|
|
+ public Builder setMaxRetryCount(int maxCount) {
|
|
|
+ this.maxRetryCount = maxCount;
|
|
|
+ return this;
|
|
|
+ }
|
|
|
+
|
|
|
public KeyOutputStream build() throws IOException {
|
|
|
return new KeyOutputStream(openHandler, xceiverManager, scmClient,
|
|
|
omClient, chunkSize, requestID, factor, type, streamBufferFlushSize,
|
|
|
streamBufferMaxSize, blockSize, watchTimeout, checksumType,
|
|
|
- bytesPerChecksum, multipartUploadID, multipartNumber, isMultipartKey);
|
|
|
+ bytesPerChecksum, multipartUploadID, multipartNumber, isMultipartKey,
|
|
|
+ maxRetryCount);
|
|
|
}
|
|
|
}
|
|
|
|