|
@@ -24,6 +24,7 @@ import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.Result;
|
|
|
import org.apache.hadoop.hdds.client.BlockID;
|
|
|
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerNotOpenException;
|
|
|
import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline;
|
|
|
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
|
|
|
import org.apache.hadoop.hdds.scm.storage.BlockOutputStream;
|
|
|
import org.apache.hadoop.ozone.common.Checksum;
|
|
|
import org.apache.hadoop.ozone.om.helpers.*;
|
|
@@ -31,11 +32,11 @@ import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationType;
|
|
|
import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ReplicationFactor;
|
|
|
import org.apache.hadoop.ozone.om.protocolPB.OzoneManagerProtocolClientSideTranslatorPB;
|
|
|
import org.apache.hadoop.hdds.scm.XceiverClientManager;
|
|
|
-import org.apache.hadoop.hdds.scm.XceiverClientSpi;
|
|
|
import org.apache.hadoop.hdds.scm.container.common.helpers
|
|
|
.StorageContainerException;
|
|
|
import org.apache.hadoop.hdds.scm.protocolPB
|
|
|
.StorageContainerLocationProtocolClientSideTranslatorPB;
|
|
|
+import org.apache.ratis.protocol.AlreadyClosedException;
|
|
|
import org.apache.ratis.protocol.RaftRetryFailureException;
|
|
|
import org.slf4j.Logger;
|
|
|
import org.slf4j.LoggerFactory;
|
|
@@ -107,19 +108,6 @@ public class KeyOutputStream extends OutputStream {
|
|
|
this.checksum = new Checksum();
|
|
|
}
|
|
|
|
|
|
- /**
|
|
|
- * For testing purpose only. Not building output stream from blocks, but
|
|
|
- * taking from externally.
|
|
|
- *
|
|
|
- * @param outputStream
|
|
|
- * @param length
|
|
|
- */
|
|
|
- @VisibleForTesting
|
|
|
- public void addStream(OutputStream outputStream, long length) {
|
|
|
- streamEntries.add(
|
|
|
- new BlockOutputStreamEntry(outputStream, length, checksum));
|
|
|
- }
|
|
|
-
|
|
|
@VisibleForTesting
|
|
|
public List<BlockOutputStreamEntry> getStreamEntries() {
|
|
|
return streamEntries;
|
|
@@ -213,12 +201,11 @@ public class KeyOutputStream extends OutputStream {
|
|
|
throws IOException {
|
|
|
ContainerWithPipeline containerWithPipeline = scmClient
|
|
|
.getContainerWithPipeline(subKeyInfo.getContainerID());
|
|
|
- XceiverClientSpi xceiverClient =
|
|
|
- xceiverClientManager.acquireClient(containerWithPipeline.getPipeline());
|
|
|
streamEntries.add(new BlockOutputStreamEntry(subKeyInfo.getBlockID(),
|
|
|
- keyArgs.getKeyName(), xceiverClientManager, xceiverClient, requestID,
|
|
|
- chunkSize, subKeyInfo.getLength(), streamBufferFlushSize,
|
|
|
- streamBufferMaxSize, watchTimeout, bufferList, checksum));
|
|
|
+ keyArgs.getKeyName(), xceiverClientManager,
|
|
|
+ containerWithPipeline.getPipeline(), requestID, chunkSize,
|
|
|
+ subKeyInfo.getLength(), streamBufferFlushSize, streamBufferMaxSize,
|
|
|
+ watchTimeout, bufferList, checksum));
|
|
|
}
|
|
|
|
|
|
|
|
@@ -297,12 +284,14 @@ public class KeyOutputStream extends OutputStream {
|
|
|
current.write(b, off, writeLen);
|
|
|
}
|
|
|
} catch (IOException ioe) {
|
|
|
- if (checkIfContainerIsClosed(ioe) || checkIfTimeoutException(ioe)) {
|
|
|
+ boolean retryFailure = checkForRetryFailure(ioe);
|
|
|
+ if (checkIfContainerIsClosed(ioe) || checkIfTimeoutException(ioe)
|
|
|
+ || retryFailure) {
|
|
|
// for the current iteration, totalDataWritten - currentPos gives the
|
|
|
// amount of data already written to the buffer
|
|
|
writeLen = (int) (current.getWrittenDataLength() - currentPos);
|
|
|
LOG.debug("writeLen {}, total len {}", writeLen, len);
|
|
|
- handleException(current, currentStreamIndex);
|
|
|
+ handleException(current, currentStreamIndex, retryFailure);
|
|
|
} else {
|
|
|
throw ioe;
|
|
|
}
|
|
@@ -362,17 +351,19 @@ public class KeyOutputStream extends OutputStream {
|
|
|
*
|
|
|
* @param streamEntry StreamEntry
|
|
|
* @param streamIndex Index of the entry
|
|
|
+ * @param retryFailure if true the xceiverClient needs to be invalidated in
|
|
|
+ * the client cache.
|
|
|
* @throws IOException Throws IOException if Write fails
|
|
|
*/
|
|
|
private void handleException(BlockOutputStreamEntry streamEntry,
|
|
|
- int streamIndex) throws IOException {
|
|
|
+ int streamIndex, boolean retryFailure) throws IOException {
|
|
|
long totalSuccessfulFlushedData =
|
|
|
streamEntry.getTotalSuccessfulFlushedData();
|
|
|
//set the correct length for the current stream
|
|
|
streamEntry.currentPosition = totalSuccessfulFlushedData;
|
|
|
long bufferedDataLen = computeBufferData();
|
|
|
// just clean up the current stream.
|
|
|
- streamEntry.cleanup();
|
|
|
+ streamEntry.cleanup(retryFailure);
|
|
|
if (bufferedDataLen > 0) {
|
|
|
// If the data is still cached in the underlying stream, we need to
|
|
|
// allocate new block and write this data in the datanode.
|
|
@@ -390,7 +381,7 @@ public class KeyOutputStream extends OutputStream {
|
|
|
|
|
|
private boolean checkIfContainerIsClosed(IOException ioe) {
|
|
|
if (ioe.getCause() != null) {
|
|
|
- return checkIfContainerNotOpenOrRaftRetryFailureException(ioe) || Optional
|
|
|
+ return checkForException(ioe, ContainerNotOpenException.class) || Optional
|
|
|
.of(ioe.getCause())
|
|
|
.filter(e -> e instanceof StorageContainerException)
|
|
|
.map(e -> (StorageContainerException) e)
|
|
@@ -400,13 +391,23 @@ public class KeyOutputStream extends OutputStream {
|
|
|
return false;
|
|
|
}
|
|
|
|
|
|
- private boolean checkIfContainerNotOpenOrRaftRetryFailureException(
|
|
|
- IOException ioe) {
|
|
|
+ /**
|
|
|
+ * Checks if the provided exception signifies retry failure in ratis client.
|
|
|
+ * In case of retry failure, ratis client throws RaftRetryFailureException
|
|
|
+ * and all succeeding operations are failed with AlreadyClosedException.
|
|
|
+ */
|
|
|
+ private boolean checkForRetryFailure(IOException ioe) {
|
|
|
+ return checkForException(ioe, RaftRetryFailureException.class,
|
|
|
+ AlreadyClosedException.class);
|
|
|
+ }
|
|
|
+
|
|
|
+ private boolean checkForException(IOException ioe, Class... classes) {
|
|
|
Throwable t = ioe.getCause();
|
|
|
while (t != null) {
|
|
|
- if (t instanceof ContainerNotOpenException
|
|
|
- || t instanceof RaftRetryFailureException) {
|
|
|
- return true;
|
|
|
+ for (Class cls : classes) {
|
|
|
+ if (cls.isInstance(t)) {
|
|
|
+ return true;
|
|
|
+ }
|
|
|
}
|
|
|
t = t.getCause();
|
|
|
}
|
|
@@ -469,11 +470,13 @@ public class KeyOutputStream extends OutputStream {
|
|
|
entry.flush();
|
|
|
}
|
|
|
} catch (IOException ioe) {
|
|
|
- if (checkIfContainerIsClosed(ioe) || checkIfTimeoutException(ioe)) {
|
|
|
+ boolean retryFailure = checkForRetryFailure(ioe);
|
|
|
+ if (checkIfContainerIsClosed(ioe) || checkIfTimeoutException(ioe)
|
|
|
+ || retryFailure) {
|
|
|
// 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);
|
|
|
+ handleException(entry, streamIndex, retryFailure);
|
|
|
handleFlushOrClose(close);
|
|
|
} else {
|
|
|
throw ioe;
|
|
@@ -643,7 +646,7 @@ public class KeyOutputStream extends OutputStream {
|
|
|
private BlockID blockID;
|
|
|
private final String key;
|
|
|
private final XceiverClientManager xceiverClientManager;
|
|
|
- private final XceiverClientSpi xceiverClient;
|
|
|
+ private final Pipeline pipeline;
|
|
|
private final Checksum checksum;
|
|
|
private final String requestId;
|
|
|
private final int chunkSize;
|
|
@@ -660,14 +663,14 @@ public class KeyOutputStream extends OutputStream {
|
|
|
@SuppressWarnings("parameternumber")
|
|
|
BlockOutputStreamEntry(BlockID blockID, String key,
|
|
|
XceiverClientManager xceiverClientManager,
|
|
|
- XceiverClientSpi xceiverClient, String requestId, int chunkSize,
|
|
|
+ Pipeline pipeline, String requestId, int chunkSize,
|
|
|
long length, long streamBufferFlushSize, long streamBufferMaxSize,
|
|
|
long watchTimeout, List<ByteBuffer> bufferList, Checksum checksum) {
|
|
|
this.outputStream = null;
|
|
|
this.blockID = blockID;
|
|
|
this.key = key;
|
|
|
this.xceiverClientManager = xceiverClientManager;
|
|
|
- this.xceiverClient = xceiverClient;
|
|
|
+ this.pipeline = pipeline;
|
|
|
this.requestId = requestId;
|
|
|
this.chunkSize = chunkSize;
|
|
|
|
|
@@ -680,30 +683,6 @@ public class KeyOutputStream extends OutputStream {
|
|
|
this.bufferList = bufferList;
|
|
|
}
|
|
|
|
|
|
- /**
|
|
|
- * For testing purpose, taking a some random created stream instance.
|
|
|
- * @param outputStream a existing writable output stream
|
|
|
- * @param length the length of data to write to the stream
|
|
|
- */
|
|
|
- BlockOutputStreamEntry(OutputStream outputStream, long length,
|
|
|
- Checksum checksum) {
|
|
|
- this.outputStream = outputStream;
|
|
|
- this.blockID = null;
|
|
|
- this.key = null;
|
|
|
- this.xceiverClientManager = null;
|
|
|
- this.xceiverClient = null;
|
|
|
- this.requestId = null;
|
|
|
- this.chunkSize = -1;
|
|
|
-
|
|
|
- this.length = length;
|
|
|
- this.currentPosition = 0;
|
|
|
- streamBufferFlushSize = 0;
|
|
|
- streamBufferMaxSize = 0;
|
|
|
- bufferList = null;
|
|
|
- watchTimeout = 0;
|
|
|
- this.checksum = checksum;
|
|
|
- }
|
|
|
-
|
|
|
long getLength() {
|
|
|
return length;
|
|
|
}
|
|
@@ -712,11 +691,17 @@ public class KeyOutputStream extends OutputStream {
|
|
|
return length - currentPosition;
|
|
|
}
|
|
|
|
|
|
- private void checkStream() {
|
|
|
+ /**
|
|
|
+ * BlockOutputStream is initialized in this function. This makes sure that
|
|
|
+ * xceiverClient initialization is not done during preallocation and only
|
|
|
+ * done when data is written.
|
|
|
+ * @throws IOException if xceiverClient initialization fails
|
|
|
+ */
|
|
|
+ private void checkStream() throws IOException {
|
|
|
if (this.outputStream == null) {
|
|
|
this.outputStream =
|
|
|
new BlockOutputStream(blockID, key, xceiverClientManager,
|
|
|
- xceiverClient, requestId, chunkSize, streamBufferFlushSize,
|
|
|
+ pipeline, requestId, chunkSize, streamBufferFlushSize,
|
|
|
streamBufferMaxSize, watchTimeout, bufferList, checksum);
|
|
|
}
|
|
|
}
|
|
@@ -781,11 +766,11 @@ public class KeyOutputStream extends OutputStream {
|
|
|
throw new IOException("Invalid Output Stream for Key: " + key);
|
|
|
}
|
|
|
|
|
|
- void cleanup() {
|
|
|
+ void cleanup(boolean invalidateClient) throws IOException {
|
|
|
checkStream();
|
|
|
if (this.outputStream instanceof BlockOutputStream) {
|
|
|
BlockOutputStream out = (BlockOutputStream) this.outputStream;
|
|
|
- out.cleanup();
|
|
|
+ out.cleanup(invalidateClient);
|
|
|
}
|
|
|
}
|
|
|
|