HDDS-870. Avoid creating block sized buffer in ChunkGroupOutputStream. Contributed by Shashikant Banerjee.

This commit is contained in:
Jitendra Pandey 2018-12-08 09:23:10 -08:00
parent 5e773efd78
commit 1afba83f2c
12 changed files with 505 additions and 385 deletions

View File

@ -290,12 +290,16 @@ public class XceiverClientGrpc extends XceiverClientSpi {
} }
@Override @Override
public void watchForCommit(long index, long timeout) public long watchForCommit(long index, long timeout)
throws InterruptedException, ExecutionException, TimeoutException, throws InterruptedException, ExecutionException, TimeoutException,
IOException { IOException {
// there is no notion of watch for commit index in standalone pipeline // there is no notion of watch for commit index in standalone pipeline
return 0;
}; };
public long getReplicatedMinCommitIndex() {
return 0;
}
/** /**
* Returns pipeline Type. * Returns pipeline Type.
* *

View File

@ -18,7 +18,9 @@
package org.apache.hadoop.hdds.scm; package org.apache.hadoop.hdds.scm;
import com.google.common.base.Preconditions;
import org.apache.hadoop.hdds.HddsUtils; import org.apache.hadoop.hdds.HddsUtils;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
import org.apache.ratis.proto.RaftProtos; import org.apache.ratis.proto.RaftProtos;
import org.apache.ratis.protocol.RaftRetryFailureException; import org.apache.ratis.protocol.RaftRetryFailureException;
import org.apache.ratis.retry.RetryPolicy; import org.apache.ratis.retry.RetryPolicy;
@ -42,15 +44,14 @@ import org.slf4j.Logger;
import org.slf4j.LoggerFactory; import org.slf4j.LoggerFactory;
import java.io.IOException; import java.io.IOException;
import java.util.ArrayList; import java.util.*;
import java.util.Objects;
import java.util.Collection;
import java.util.concurrent.CompletableFuture; import java.util.concurrent.CompletableFuture;
import java.util.concurrent.CompletionException; import java.util.concurrent.CompletionException;
import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeUnit;
import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutionException;
import java.util.concurrent.TimeoutException; import java.util.concurrent.TimeoutException;
import java.util.concurrent.atomic.AtomicReference; import java.util.concurrent.atomic.AtomicReference;
import java.util.concurrent.ConcurrentHashMap;
/** /**
* An abstract implementation of {@link XceiverClientSpi} using Ratis. * An abstract implementation of {@link XceiverClientSpi} using Ratis.
@ -79,6 +80,12 @@ public final class XceiverClientRatis extends XceiverClientSpi {
private final int maxOutstandingRequests; private final int maxOutstandingRequests;
private final RetryPolicy retryPolicy; private final RetryPolicy retryPolicy;
// Map to track commit index at every server
private final ConcurrentHashMap<String, Long> commitInfoMap;
// create a separate RaftClient for watchForCommit API
private RaftClient watchClient;
/** /**
* Constructs a client. * Constructs a client.
*/ */
@ -89,6 +96,30 @@ public final class XceiverClientRatis extends XceiverClientSpi {
this.rpcType = rpcType; this.rpcType = rpcType;
this.maxOutstandingRequests = maxOutStandingChunks; this.maxOutstandingRequests = maxOutStandingChunks;
this.retryPolicy = retryPolicy; this.retryPolicy = retryPolicy;
commitInfoMap = new ConcurrentHashMap<>();
watchClient = null;
}
private void updateCommitInfosMap(
Collection<RaftProtos.CommitInfoProto> commitInfoProtos) {
// if the commitInfo map is empty, just update the commit indexes for each
// of the servers
if (commitInfoMap.isEmpty()) {
commitInfoProtos.forEach(proto -> commitInfoMap
.put(proto.getServer().getAddress(), proto.getCommitIndex()));
// In case the commit is happening 2 way, just update the commitIndex
// for the servers which have been successfully updating the commit
// indexes. This is important because getReplicatedMinCommitIndex()
// should always return the min commit index out of the nodes which have
// been replicating data successfully.
} else {
commitInfoProtos.forEach(proto -> commitInfoMap
.computeIfPresent(proto.getServer().getAddress(),
(address, index) -> {
index = proto.getCommitIndex();
return index;
}));
}
} }
/** /**
@ -125,6 +156,9 @@ public final class XceiverClientRatis extends XceiverClientSpi {
if (c != null) { if (c != null) {
closeRaftClient(c); closeRaftClient(c);
} }
if (watchClient != null) {
closeRaftClient(watchClient);
}
} }
private void closeRaftClient(RaftClient raftClient) { private void closeRaftClient(RaftClient raftClient) {
@ -148,39 +182,73 @@ public final class XceiverClientRatis extends XceiverClientSpi {
getClient().sendAsync(() -> byteString); getClient().sendAsync(() -> byteString);
} }
// gets the minimum log index replicated to all servers
@Override @Override
public void watchForCommit(long index, long timeout) public long getReplicatedMinCommitIndex() {
OptionalLong minIndex =
commitInfoMap.values().parallelStream().mapToLong(v -> v).min();
return minIndex.isPresent() ? minIndex.getAsLong() : 0;
}
private void getFailedServer(
Collection<RaftProtos.CommitInfoProto> commitInfos) {
for (RaftProtos.CommitInfoProto proto : commitInfos) {
}
}
@Override
public long watchForCommit(long index, long timeout)
throws InterruptedException, ExecutionException, TimeoutException, throws InterruptedException, ExecutionException, TimeoutException,
IOException { IOException {
long commitIndex = getReplicatedMinCommitIndex();
if (commitIndex >= index) {
// return the min commit index till which the log has been replicated to
// all servers
return commitIndex;
}
LOG.debug("commit index : {} watch timeout : {}", index, timeout); LOG.debug("commit index : {} watch timeout : {}", index, timeout);
// create a new RaftClient instance for watch request // create a new RaftClient instance for watch request
RaftClient raftClient = if (watchClient == null) {
RatisHelper.newRaftClient(rpcType, getPipeline(), retryPolicy); watchClient =
CompletableFuture<RaftClientReply> replyFuture = raftClient RatisHelper.newRaftClient(rpcType, getPipeline(), retryPolicy);
}
CompletableFuture<RaftClientReply> replyFuture = watchClient
.sendWatchAsync(index, RaftProtos.ReplicationLevel.ALL_COMMITTED); .sendWatchAsync(index, RaftProtos.ReplicationLevel.ALL_COMMITTED);
RaftClientReply reply;
try { try {
replyFuture.get(timeout, TimeUnit.MILLISECONDS); reply = replyFuture.get(timeout, TimeUnit.MILLISECONDS);
} catch (TimeoutException toe) { } catch (TimeoutException toe) {
LOG.warn("3 way commit failed ", toe); LOG.warn("3 way commit failed ", toe);
closeRaftClient(raftClient); closeRaftClient(watchClient);
// generate a new raft client instance again so that next watch request // generate a new raft client instance again so that next watch request
// does not get blocked for the previous one // does not get blocked for the previous one
// TODO : need to remove the code to create the new RaftClient instance // TODO : need to remove the code to create the new RaftClient instance
// here once the watch request bypassing sliding window in Raft Client // here once the watch request bypassing sliding window in Raft Client
// gets fixed. // gets fixed.
raftClient = watchClient =
RatisHelper.newRaftClient(rpcType, getPipeline(), retryPolicy); RatisHelper.newRaftClient(rpcType, getPipeline(), retryPolicy);
raftClient reply = watchClient
.sendWatchAsync(index, RaftProtos.ReplicationLevel.MAJORITY_COMMITTED) .sendWatchAsync(index, RaftProtos.ReplicationLevel.MAJORITY_COMMITTED)
.get(timeout, TimeUnit.MILLISECONDS); .get(timeout, TimeUnit.MILLISECONDS);
LOG.info("Could not commit " + index + " to all the nodes." Optional<RaftProtos.CommitInfoProto>
+ "Committed by majority."); proto = reply.getCommitInfos().stream().min(Comparator.comparing(
} finally { RaftProtos.CommitInfoProto :: getCommitIndex));
closeRaftClient(raftClient); Preconditions.checkState(proto.isPresent());
String address = proto.get().getServer().getAddress();
// since 3 way commit has failed, the updated map from now on will
// only store entries for those datanodes which have had successful
// replication.
commitInfoMap.remove(address);
LOG.info(
"Could not commit " + index + " to all the nodes. Server " + address
+ " has failed" + "Committed by majority.");
} }
return index;
} }
/** /**
* Sends a given command to server gets a waitable future back. * Sends a given command to server gets a waitable future back.
* *
@ -193,8 +261,6 @@ public final class XceiverClientRatis extends XceiverClientSpi {
XceiverClientAsyncReply asyncReply = new XceiverClientAsyncReply(null); XceiverClientAsyncReply asyncReply = new XceiverClientAsyncReply(null);
CompletableFuture<RaftClientReply> raftClientReply = CompletableFuture<RaftClientReply> raftClientReply =
sendRequestAsync(request); sendRequestAsync(request);
Collection<XceiverClientAsyncReply.CommitInfo> commitInfos =
new ArrayList<>();
CompletableFuture<ContainerCommandResponseProto> containerCommandResponse = CompletableFuture<ContainerCommandResponseProto> containerCommandResponse =
raftClientReply.whenComplete((reply, e) -> LOG.debug( raftClientReply.whenComplete((reply, e) -> LOG.debug(
"received reply {} for request: cmdType={} containerID={}" "received reply {} for request: cmdType={} containerID={}"
@ -212,14 +278,10 @@ public final class XceiverClientRatis extends XceiverClientSpi {
ContainerCommandResponseProto response = ContainerCommandResponseProto response =
ContainerCommandResponseProto ContainerCommandResponseProto
.parseFrom(reply.getMessage().getContent()); .parseFrom(reply.getMessage().getContent());
reply.getCommitInfos().forEach(e -> { if (response.getResult() == ContainerProtos.Result.SUCCESS) {
XceiverClientAsyncReply.CommitInfo commitInfo = updateCommitInfosMap(reply.getCommitInfos());
new XceiverClientAsyncReply.CommitInfo(
e.getServer().getAddress(), e.getCommitIndex());
commitInfos.add(commitInfo);
asyncReply.setCommitInfos(commitInfos);
asyncReply.setLogIndex(reply.getLogIndex()); asyncReply.setLogIndex(reply.getLogIndex());
}); }
return response; return response;
} catch (InvalidProtocolBufferException e) { } catch (InvalidProtocolBufferException e) {
throw new CompletionException(e); throw new CompletionException(e);

View File

@ -37,15 +37,13 @@ import org.slf4j.LoggerFactory;
import java.io.IOException; import java.io.IOException;
import java.io.OutputStream; import java.io.OutputStream;
import java.nio.Buffer;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.util.UUID; import java.util.UUID;
import java.util.List; import java.util.List;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.concurrent.CompletableFuture; import java.util.concurrent.*;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.TimeoutException;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.Executors;
import static org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls import static org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls
.putBlockAsync; .putBlockAsync;
import static org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls import static org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls
@ -84,25 +82,30 @@ public class ChunkOutputStream extends OutputStream {
private final long streamBufferFlushSize; private final long streamBufferFlushSize;
private final long streamBufferMaxSize; private final long streamBufferMaxSize;
private final long watchTimeout; private final long watchTimeout;
private ByteBuffer buffer; private List<ByteBuffer> bufferList;
// The IOException will be set by response handling thread in case there is an // The IOException will be set by response handling thread in case there is an
// exception received in the response. If the exception is set, the next // exception received in the response. If the exception is set, the next
// request will fail upfront. // request will fail upfront.
private IOException ioException; private IOException ioException;
private ExecutorService responseExecutor; private ExecutorService responseExecutor;
// position of the buffer where the last flush was attempted // the effective length of data flushed so far
private int lastFlushPos; private long totalDataFlushedLength;
// position of the buffer till which the flush was successfully // effective data write attempted so far for the block
// acknowledged by all nodes in pipeline private long writtenDataLength;
private int lastSuccessfulFlushIndex;
// total data which has been successfully flushed and acknowledged
// by all servers
private long totalAckDataLength;
// list to hold up all putBlock futures // list to hold up all putBlock futures
private List<CompletableFuture<ContainerProtos.ContainerCommandResponseProto>> private List<CompletableFuture<ContainerProtos.ContainerCommandResponseProto>>
futureList; futureList;
// list maintaining commit indexes for putBlocks // map containing mapping for putBlock logIndex to to flushedDataLength Map.
private List<Long> commitIndexList; private ConcurrentHashMap<Long, Long> commitIndex2flushedDataMap;
private int currentBufferIndex;
/** /**
* Creates a new ChunkOutputStream. * Creates a new ChunkOutputStream.
@ -113,12 +116,17 @@ public class ChunkOutputStream extends OutputStream {
* @param xceiverClient client to perform container calls * @param xceiverClient client to perform container calls
* @param traceID container protocol call args * @param traceID container protocol call args
* @param chunkSize chunk size * @param chunkSize chunk size
* @param bufferList list of byte buffers
* @param streamBufferFlushSize flush size
* @param streamBufferMaxSize max size of the currentBuffer
* @param watchTimeout watch timeout
* @param checksum checksum
*/ */
public ChunkOutputStream(BlockID blockID, String key, public ChunkOutputStream(BlockID blockID, String key,
XceiverClientManager xceiverClientManager, XceiverClientSpi xceiverClient, XceiverClientManager xceiverClientManager, XceiverClientSpi xceiverClient,
String traceID, int chunkSize, long streamBufferFlushSize, String traceID, int chunkSize, long streamBufferFlushSize,
long streamBufferMaxSize, long watchTimeout, ByteBuffer buffer, long streamBufferMaxSize, long watchTimeout,
Checksum checksum) { List<ByteBuffer> bufferList, Checksum checksum) {
this.blockID = blockID; this.blockID = blockID;
this.key = key; this.key = key;
this.traceID = traceID; this.traceID = traceID;
@ -135,24 +143,36 @@ public class ChunkOutputStream extends OutputStream {
this.streamBufferFlushSize = streamBufferFlushSize; this.streamBufferFlushSize = streamBufferFlushSize;
this.streamBufferMaxSize = streamBufferMaxSize; this.streamBufferMaxSize = streamBufferMaxSize;
this.watchTimeout = watchTimeout; this.watchTimeout = watchTimeout;
this.buffer = buffer; this.bufferList = bufferList;
this.ioException = null;
this.checksum = checksum; this.checksum = checksum;
// A single thread executor handle the responses of async requests // A single thread executor handle the responses of async requests
responseExecutor = Executors.newSingleThreadExecutor(); responseExecutor = Executors.newSingleThreadExecutor();
commitIndexList = new ArrayList<>(); commitIndex2flushedDataMap = new ConcurrentHashMap<>();
lastSuccessfulFlushIndex = 0; totalAckDataLength = 0;
futureList = new ArrayList<>(); futureList = new ArrayList<>();
lastFlushPos = 0; totalDataFlushedLength = 0;
currentBufferIndex = 0;
writtenDataLength = 0;
} }
public BlockID getBlockID() { public BlockID getBlockID() {
return blockID; return blockID;
} }
public int getLastSuccessfulFlushIndex() { public long getTotalSuccessfulFlushedData() {
return lastSuccessfulFlushIndex; return totalAckDataLength;
}
public long getWrittenDataLength() {
return writtenDataLength;
}
private long computeBufferData() {
int dataLength =
bufferList.stream().mapToInt(Buffer::position).sum();
Preconditions.checkState(dataLength <= streamBufferMaxSize);
return dataLength;
} }
@ -176,139 +196,172 @@ public class ChunkOutputStream extends OutputStream {
if (len == 0) { if (len == 0) {
return; return;
} }
checkOpen();
while (len > 0) { while (len > 0) {
checkOpen();
int writeLen; int writeLen;
writeLen = Math.min(chunkSize - buffer.position() % chunkSize, len); allocateBuffer();
buffer.put(b, off, writeLen); ByteBuffer currentBuffer = getCurrentBuffer();
if (buffer.position() % chunkSize == 0) { writeLen =
int pos = buffer.position() - chunkSize; Math.min(chunkSize - currentBuffer.position() % chunkSize, len);
int limit = buffer.position(); currentBuffer.put(b, off, writeLen);
if (currentBuffer.position() % chunkSize == 0) {
int pos = currentBuffer.position() - chunkSize;
int limit = currentBuffer.position();
writeChunk(pos, limit); writeChunk(pos, limit);
} }
off += writeLen; off += writeLen;
len -= writeLen; len -= writeLen;
if (buffer.position() >= streamBufferFlushSize writtenDataLength += writeLen;
&& buffer.position() % streamBufferFlushSize == 0) { if (currentBuffer.position() == streamBufferFlushSize) {
totalDataFlushedLength += streamBufferFlushSize;
lastFlushPos = buffer.position(); handlePartialFlush();
futureList.add(handlePartialFlush());
} }
if (buffer.position() >= streamBufferMaxSize long bufferedData = computeBufferData();
&& buffer.position() % streamBufferMaxSize == 0) { // Data in the bufferList can not exceed streamBufferMaxSize
if (bufferedData == streamBufferMaxSize) {
handleFullBuffer(); handleFullBuffer();
} }
} }
} }
private ByteBuffer getCurrentBuffer() {
ByteBuffer buffer = bufferList.get(currentBufferIndex);
if (!buffer.hasRemaining()) {
currentBufferIndex =
currentBufferIndex < getMaxNumBuffers() - 1 ? ++currentBufferIndex :
0;
}
return bufferList.get(currentBufferIndex);
}
private int getMaxNumBuffers() {
return (int)(streamBufferMaxSize/streamBufferFlushSize);
}
private void allocateBuffer() {
for (int i = bufferList.size(); i < getMaxNumBuffers(); i++) {
bufferList.add(ByteBuffer.allocate((int)streamBufferFlushSize));
}
}
/** /**
* Will be called on the retryPath in case closedContainerException/ * Will be called on the retryPath in case closedContainerException/
* TimeoutException. * TimeoutException.
* @param len length of data to write * @param len length of data to write
* @throws IOException if error occured * @throws IOException if error occurred
*/ */
// In this case, the data is already cached in the buffer. // In this case, the data is already cached in the currentBuffer.
public void writeOnRetry(int len) throws IOException { public void writeOnRetry(long len) throws IOException {
if (len == 0) { if (len == 0) {
return; return;
} }
int off = 0; int off = 0;
checkOpen(); int pos = off;
while (len > 0) { while (len > 0) {
int writeLen; long writeLen;
writeLen = Math.min(chunkSize, len); writeLen = Math.min(chunkSize, len);
if (writeLen == chunkSize) { if (writeLen == chunkSize) {
int pos = off;
int limit = pos + chunkSize; int limit = pos + chunkSize;
writeChunk(pos, limit); writeChunk(pos, limit);
} }
off += writeLen; off += writeLen;
len -= writeLen; len -= writeLen;
writtenDataLength += writeLen;
if (off % streamBufferFlushSize == 0) { if (off % streamBufferFlushSize == 0) {
lastFlushPos = off; // reset the position to zero as now we wll readng thhe next buffer in
futureList.add(handlePartialFlush()); // the list
pos = 0;
totalDataFlushedLength += streamBufferFlushSize;
handlePartialFlush();
} }
if (off % streamBufferMaxSize == 0) { if (computeBufferData() % streamBufferMaxSize == 0) {
handleFullBuffer(); handleFullBuffer();
} }
} }
} }
private void handleResponse(
ContainerProtos.ContainerCommandResponseProto response,
XceiverClientAsyncReply asyncReply) {
validateResponse(response);
discardBuffer(asyncReply);
}
private void discardBuffer(XceiverClientAsyncReply asyncReply) {
if (!commitIndexList.isEmpty()) {
long index = commitIndexList.get(0);
if (checkIfBufferDiscardRequired(asyncReply, index)) {
updateFlushIndex();
}
}
}
/** /**
* just update the lastSuccessfulFlushIndex. Since we have allocated * just update the totalAckDataLength. Since we have allocated
* the buffer more than the streamBufferMaxSize, we can keep on writing * the currentBuffer more than the streamBufferMaxSize, we can keep on writing
* to the buffer. In case of failure, we will read the data starting from * to the currentBuffer. In case of failure, we will read the data starting
* lastSuccessfulFlushIndex. * from totalAckDataLength.
*/ */
private void updateFlushIndex() { private void updateFlushIndex(long index) {
lastSuccessfulFlushIndex += streamBufferFlushSize; if (!commitIndex2flushedDataMap.isEmpty()) {
LOG.debug("Discarding buffer till pos " + lastSuccessfulFlushIndex); Preconditions.checkState(commitIndex2flushedDataMap.containsKey(index));
if (!commitIndexList.isEmpty()) { totalAckDataLength = commitIndex2flushedDataMap.remove(index);
commitIndexList.remove(0); LOG.debug("Total data successfully replicated: " + totalAckDataLength);
futureList.remove(0); futureList.remove(0);
} // Flush has been committed to required servers successful.
// just swap the bufferList head and tail after clearing.
} ByteBuffer currentBuffer = bufferList.remove(0);
/** currentBuffer.clear();
* Check if the last commitIndex stored at the beginning of the if (currentBufferIndex != 0) {
* commitIndexList is less than equal to current commitInfo indexes. currentBufferIndex--;
* If its true, the buffer has been successfully flushed till the
* last position where flush happened.
*/
private boolean checkIfBufferDiscardRequired(
XceiverClientAsyncReply asyncReply, long commitIndex) {
if (asyncReply.getCommitInfos() != null) {
for (XceiverClientAsyncReply.CommitInfo info : asyncReply
.getCommitInfos()) {
if (info.getCommitIndex() < commitIndex) {
return false;
}
} }
bufferList.add(currentBuffer);
} }
return true;
} }
/** /**
* This is a blocking call.It will wait for the flush till the commit index * This is a blocking call. It will wait for the flush till the commit index
* at the head of the commitIndexList gets replicated to all or majority. * at the head of the commitIndex2flushedDataMap gets replicated to all or
* majority.
* @throws IOException * @throws IOException
*/ */
private void handleFullBuffer() throws IOException { private void handleFullBuffer() throws IOException {
if (!commitIndexList.isEmpty()) { try {
watchForCommit(commitIndexList.get(0)); checkOpen();
if (!futureList.isEmpty()) {
waitOnFlushFutures();
}
} catch (InterruptedException | ExecutionException e) {
adjustBuffersOnException();
throw new IOException(
"Unexpected Storage Container Exception: " + e.toString(), e);
} }
if (!commitIndex2flushedDataMap.isEmpty()) {
watchForCommit(
commitIndex2flushedDataMap.keySet().stream().mapToLong(v -> v)
.min().getAsLong());
}
}
private void adjustBuffers(long commitIndex) {
commitIndex2flushedDataMap.keySet().stream().forEach(index -> {
if (index <= commitIndex) {
updateFlushIndex(index);
} else {
return;
}
});
}
// It may happen that once the exception is encountered , we still might
// have successfully flushed up to a certain index. Make sure the buffers
// only contain data which have not been sufficiently replicated
private void adjustBuffersOnException() {
adjustBuffers(xceiverClient.getReplicatedMinCommitIndex());
} }
/** /**
* calls watchForCommit API of the Ratis Client. For Standalone client, * calls watchForCommit API of the Ratis Client. For Standalone client,
* it is a no op. * it is a no op.
* @param commitIndex log index to watch for * @param commitIndex log index to watch for
* @return minimum commit index replicated to all nodes
* @throws IOException IOException in case watch gets timed out * @throws IOException IOException in case watch gets timed out
*/ */
private void watchForCommit(long commitIndex) throws IOException { private void watchForCommit(long commitIndex) throws IOException {
checkOpen(); checkOpen();
Preconditions.checkState(!commitIndexList.isEmpty()); Preconditions.checkState(!commitIndex2flushedDataMap.isEmpty());
try { try {
xceiverClient.watchForCommit(commitIndex, watchTimeout); long index =
xceiverClient.watchForCommit(commitIndex, watchTimeout);
adjustBuffers(index);
} catch (TimeoutException | InterruptedException | ExecutionException e) { } catch (TimeoutException | InterruptedException | ExecutionException e) {
LOG.warn("watchForCommit failed for index " + commitIndex, e); LOG.warn("watchForCommit failed for index " + commitIndex, e);
adjustBuffersOnException();
throw new IOException( throw new IOException(
"Unexpected Storage Container Exception: " + e.toString(), e); "Unexpected Storage Container Exception: " + e.toString(), e);
} }
@ -317,68 +370,79 @@ public class ChunkOutputStream extends OutputStream {
private CompletableFuture<ContainerProtos. private CompletableFuture<ContainerProtos.
ContainerCommandResponseProto> handlePartialFlush() ContainerCommandResponseProto> handlePartialFlush()
throws IOException { throws IOException {
checkOpen();
long flushPos = totalDataFlushedLength;
String requestId = String requestId =
traceID + ContainerProtos.Type.PutBlock + chunkIndex + blockID; traceID + ContainerProtos.Type.PutBlock + chunkIndex + blockID;
CompletableFuture<ContainerProtos.
ContainerCommandResponseProto> flushFuture;
try { try {
XceiverClientAsyncReply asyncReply = XceiverClientAsyncReply asyncReply =
putBlockAsync(xceiverClient, containerBlockData.build(), requestId); putBlockAsync(xceiverClient, containerBlockData.build(), requestId);
CompletableFuture<ContainerProtos.ContainerCommandResponseProto> future = CompletableFuture<ContainerProtos.ContainerCommandResponseProto> future =
asyncReply.getResponse(); asyncReply.getResponse();
flushFuture = future.thenApplyAsync(e -> {
return future.thenApplyAsync(e -> { try {
handleResponse(e, asyncReply); validateResponse(e);
} catch (IOException sce) {
future.completeExceptionally(sce);
return e;
}
// if the ioException is not set, putBlock is successful // if the ioException is not set, putBlock is successful
if (ioException == null) { if (ioException == null) {
LOG.debug( LOG.debug(
"Adding index " + asyncReply.getLogIndex() + " commitList size " "Adding index " + asyncReply.getLogIndex() + " commitMap size "
+ commitIndexList.size()); + commitIndex2flushedDataMap.size());
BlockID responseBlockID = BlockID.getFromProtobuf( BlockID responseBlockID = BlockID.getFromProtobuf(
e.getPutBlock().getCommittedBlockLength().getBlockID()); e.getPutBlock().getCommittedBlockLength().getBlockID());
Preconditions.checkState(blockID.getContainerBlockID() Preconditions.checkState(blockID.getContainerBlockID()
.equals(responseBlockID.getContainerBlockID())); .equals(responseBlockID.getContainerBlockID()));
// updates the bcsId of the block // updates the bcsId of the block
blockID = responseBlockID; blockID = responseBlockID;
long index = asyncReply.getLogIndex();
// for standalone protocol, logIndex will always be 0. // for standalone protocol, logIndex will always be 0.
if (index != 0) { commitIndex2flushedDataMap.put(asyncReply.getLogIndex(), flushPos);
commitIndexList.add(index);
} else {
updateFlushIndex();
}
} }
return e; return e;
}, responseExecutor); }, responseExecutor).exceptionally(e -> {
LOG.debug(
"putBlock failed for blockID " + blockID + " with exception " + e
.getLocalizedMessage());
CompletionException ce = new CompletionException(e);
setIoException(ce);
throw ce;
});
} catch (IOException | InterruptedException | ExecutionException e) { } catch (IOException | InterruptedException | ExecutionException e) {
throw new IOException( throw new IOException(
"Unexpected Storage Container Exception: " + e.toString(), e); "Unexpected Storage Container Exception: " + e.toString(), e);
} }
futureList.add(flushFuture);
return flushFuture;
} }
@Override @Override
public void flush() throws IOException { public void flush() throws IOException {
if (xceiverClientManager != null && xceiverClient != null if (xceiverClientManager != null && xceiverClient != null
&& buffer != null) { && bufferList != null) {
checkOpen(); checkOpen();
if (buffer.position() > 0 && lastSuccessfulFlushIndex != buffer int bufferSize = bufferList.size();
.position()) { if (bufferSize > 0) {
try { try {
// flush the last chunk data residing on the currentBuffer
// flush the last chunk data residing on the buffer if (totalDataFlushedLength < writtenDataLength) {
if (buffer.position() % chunkSize > 0) { ByteBuffer currentBuffer = getCurrentBuffer();
int pos = buffer.position() - (buffer.position() % chunkSize); int pos = currentBuffer.position() - (currentBuffer.position()
writeChunk(pos, buffer.position()); % chunkSize);
} int limit = currentBuffer.position() - pos;
if (lastFlushPos != buffer.position()) { writeChunk(pos, currentBuffer.position());
lastFlushPos = buffer.position(); totalDataFlushedLength += limit;
handlePartialFlush(); handlePartialFlush();
} }
CompletableFuture<Void> combinedFuture = CompletableFuture.allOf( waitOnFlushFutures();
futureList.toArray(new CompletableFuture[futureList.size()]));
combinedFuture.get();
// just check again if the exception is hit while waiting for the // just check again if the exception is hit while waiting for the
// futures to ensure flush has indeed succeeded // futures to ensure flush has indeed succeeded
checkOpen(); checkOpen();
} catch (InterruptedException | ExecutionException e) { } catch (InterruptedException | ExecutionException e) {
adjustBuffersOnException();
throw new IOException( throw new IOException(
"Unexpected Storage Container Exception: " + e.toString(), e); "Unexpected Storage Container Exception: " + e.toString(), e);
} }
@ -388,11 +452,11 @@ public class ChunkOutputStream extends OutputStream {
private void writeChunk(int pos, int limit) throws IOException { private void writeChunk(int pos, int limit) throws IOException {
// Please note : We are not flipping the slice when we write since // Please note : We are not flipping the slice when we write since
// the slices are pointing the buffer start and end as needed for // the slices are pointing the currentBuffer start and end as needed for
// the chunk write. Also please note, Duplicate does not create a // the chunk write. Also please note, Duplicate does not create a
// copy of data, it only creates metadata that points to the data // copy of data, it only creates metadata that points to the data
// stream. // stream.
ByteBuffer chunk = buffer.duplicate(); ByteBuffer chunk = bufferList.get(currentBufferIndex).duplicate();
chunk.position(pos); chunk.position(pos);
chunk.limit(limit); chunk.limit(limit);
writeChunkToContainer(chunk); writeChunkToContainer(chunk);
@ -401,49 +465,78 @@ public class ChunkOutputStream extends OutputStream {
@Override @Override
public void close() throws IOException { public void close() throws IOException {
if (xceiverClientManager != null && xceiverClient != null if (xceiverClientManager != null && xceiverClient != null
&& buffer != null) { && bufferList != null) {
try { int bufferSize = bufferList.size();
if (buffer.position() > lastFlushPos) { if (bufferSize > 0) {
int pos = buffer.position() - (buffer.position() % chunkSize); try {
writeChunk(pos, buffer.position()); // flush the last chunk data residing on the currentBuffer
futureList.add(handlePartialFlush()); if (totalDataFlushedLength < writtenDataLength) {
ByteBuffer currentBuffer = getCurrentBuffer();
int pos = currentBuffer.position() - (currentBuffer.position()
% chunkSize);
int limit = currentBuffer.position() - pos;
writeChunk(pos, currentBuffer.position());
totalDataFlushedLength += limit;
handlePartialFlush();
}
waitOnFlushFutures();
// irrespective of whether the commitIndex2flushedDataMap is empty
// or not, ensure there is no exception set
checkOpen();
if (!commitIndex2flushedDataMap.isEmpty()) {
// wait for the last commit index in the commitIndex2flushedDataMap
// to get committed to all or majority of nodes in case timeout
// happens.
long lastIndex =
commitIndex2flushedDataMap.keySet().stream()
.mapToLong(v -> v).max().getAsLong();
LOG.debug(
"waiting for last flush Index " + lastIndex + " to catch up");
watchForCommit(lastIndex);
}
} catch (InterruptedException | ExecutionException e) {
adjustBuffersOnException();
throw new IOException(
"Unexpected Storage Container Exception: " + e.toString(), e);
} finally {
cleanup();
} }
CompletableFuture<Void> combinedFuture = CompletableFuture.allOf(
futureList.toArray(new CompletableFuture[futureList.size()]));
// wait for all the transactions to complete
combinedFuture.get();
// irrespective of whether the commitIndexList is empty or not,
// ensure there is no exception set(For Standalone Protocol)
checkOpen();
if (!commitIndexList.isEmpty()) {
// wait for the last commit index in the commitIndexList to get
// committed to all or majority of nodes in case timeout happens.
long lastIndex = commitIndexList.get(commitIndexList.size() - 1);
LOG.debug(
"waiting for last flush Index " + lastIndex + " to catch up");
watchForCommit(lastIndex);
updateFlushIndex();
}
} catch (InterruptedException | ExecutionException e) {
throw new IOException(
"Unexpected Storage Container Exception: " + e.toString(), e);
} finally {
cleanup();
} }
// clear the currentBuffer
bufferList.stream().forEach(ByteBuffer::clear);
} }
// clear the buffer }
buffer.clear();
private void waitOnFlushFutures()
throws InterruptedException, ExecutionException {
CompletableFuture<Void> combinedFuture = CompletableFuture
.allOf(futureList.toArray(new CompletableFuture[futureList.size()]));
// wait for all the transactions to complete
combinedFuture.get();
} }
private void validateResponse( private void validateResponse(
ContainerProtos.ContainerCommandResponseProto responseProto) { ContainerProtos.ContainerCommandResponseProto responseProto)
throws IOException {
try { try {
// if the ioException is already set, it means a prev request has failed
// just throw the exception. The current operation will fail with the
// original error
if (ioException != null) {
throw ioException;
}
ContainerProtocolCalls.validateContainerResponse(responseProto); ContainerProtocolCalls.validateContainerResponse(responseProto);
} catch (StorageContainerException sce) { } catch (StorageContainerException sce) {
ioException = new IOException( LOG.error("Unexpected Storage Container Exception: ", sce);
"Unexpected Storage Container Exception: " + sce.toString(), sce); setIoException(sce);
throw sce;
}
}
private void setIoException(Exception e) {
if (ioException != null) {
ioException = new IOException(
"Unexpected Storage Container Exception: " + e.toString(), e);
} }
} }
@ -457,7 +550,10 @@ public class ChunkOutputStream extends OutputStream {
futureList.clear(); futureList.clear();
} }
futureList = null; futureList = null;
commitIndexList = null; if (commitIndex2flushedDataMap != null) {
commitIndex2flushedDataMap.clear();
}
commitIndex2flushedDataMap = null;
responseExecutor.shutdown(); responseExecutor.shutdown();
} }
@ -471,6 +567,7 @@ public class ChunkOutputStream extends OutputStream {
if (xceiverClient == null) { if (xceiverClient == null) {
throw new IOException("ChunkOutputStream has been closed."); throw new IOException("ChunkOutputStream has been closed.");
} else if (ioException != null) { } else if (ioException != null) {
adjustBuffersOnException();
throw ioException; throw ioException;
} }
} }
@ -504,16 +601,27 @@ public class ChunkOutputStream extends OutputStream {
CompletableFuture<ContainerProtos.ContainerCommandResponseProto> future = CompletableFuture<ContainerProtos.ContainerCommandResponseProto> future =
asyncReply.getResponse(); asyncReply.getResponse();
future.thenApplyAsync(e -> { future.thenApplyAsync(e -> {
handleResponse(e, asyncReply); try {
validateResponse(e);
} catch (IOException sce) {
future.completeExceptionally(sce);
}
return e; return e;
}, responseExecutor); }, responseExecutor).exceptionally(e -> {
LOG.debug(
"writing chunk failed " + chunkInfo.getChunkName() + " blockID "
+ blockID + " with exception " + e.getLocalizedMessage());
CompletionException ce = new CompletionException(e);
setIoException(ce);
throw ce;
});
} catch (IOException | InterruptedException | ExecutionException e) { } catch (IOException | InterruptedException | ExecutionException e) {
throw new IOException( throw new IOException(
"Unexpected Storage Container Exception: " + e.toString(), e); "Unexpected Storage Container Exception: " + e.toString(), e);
} }
LOG.debug( LOG.debug(
"writing chunk " + chunkInfo.getChunkName() + " blockID " + blockID "writing chunk " + chunkInfo.getChunkName() + " blockID " + blockID
+ " length " + chunk.remaining()); + " length " + effectiveChunkSize);
containerBlockData.addChunks(chunkInfo); containerBlockData.addChunks(chunkInfo);
} }
} }

View File

@ -21,8 +21,6 @@ package org.apache.hadoop.hdds.scm;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
.ContainerCommandResponseProto; .ContainerCommandResponseProto;
import java.util.Collection;
import java.util.concurrent.CompletableFuture; import java.util.concurrent.CompletableFuture;
/** /**
@ -32,49 +30,13 @@ public class XceiverClientAsyncReply {
private CompletableFuture<ContainerCommandResponseProto> response; private CompletableFuture<ContainerCommandResponseProto> response;
private Long logIndex; private Long logIndex;
private Collection<CommitInfo> commitInfos;
public XceiverClientAsyncReply( public XceiverClientAsyncReply(
CompletableFuture<ContainerCommandResponseProto> response) { CompletableFuture<ContainerCommandResponseProto> response) {
this(response, 0, null); this.logIndex = (long)0;
}
public XceiverClientAsyncReply(
CompletableFuture<ContainerCommandResponseProto> response, long index,
Collection<CommitInfo> commitInfos) {
this.commitInfos = commitInfos;
this.logIndex = index;
this.response = response; this.response = response;
} }
/**
* A class having details about latest commitIndex for each server in the
* Ratis pipeline. For Standalone pipeline, commitInfo will be null.
*/
public static class CommitInfo {
private final String server;
private final Long commitIndex;
public CommitInfo(String server, long commitIndex) {
this.server = server;
this.commitIndex = commitIndex;
}
public String getServer() {
return server;
}
public long getCommitIndex() {
return commitIndex;
}
}
public Collection<CommitInfo> getCommitInfos() {
return commitInfos;
}
public CompletableFuture<ContainerCommandResponseProto> getResponse() { public CompletableFuture<ContainerCommandResponseProto> getResponse() {
return response; return response;
} }
@ -83,10 +45,6 @@ public class XceiverClientAsyncReply {
return logIndex; return logIndex;
} }
public void setCommitInfos(Collection<CommitInfo> commitInfos) {
this.commitInfos = commitInfos;
}
public void setLogIndex(Long logIndex) { public void setLogIndex(Long logIndex) {
this.logIndex = logIndex; this.logIndex = logIndex;
} }

View File

@ -125,7 +125,24 @@ public abstract class XceiverClientSpi implements Closeable {
*/ */
public abstract HddsProtos.ReplicationType getPipelineType(); public abstract HddsProtos.ReplicationType getPipelineType();
public abstract void watchForCommit(long index, long timeout) /**
* Check if an specfic commitIndex is replicated to majority/all servers.
* @param index index to watch for
* @param timeout timeout provided for the watch ipeartion to complete
* @return the min commit index replicated to all or majority servers
* in case of a failure
* @throws InterruptedException
* @throws ExecutionException
* @throws TimeoutException
* @throws IOException
*/
public abstract long watchForCommit(long index, long timeout)
throws InterruptedException, ExecutionException, TimeoutException, throws InterruptedException, ExecutionException, TimeoutException,
IOException; IOException;
/**
* returns the min commit index replicated to all servers.
* @return min commit index replicated to all servers.
*/
public abstract long getReplicatedMinCommitIndex();
} }

View File

@ -189,7 +189,6 @@ public final class ContainerProtocolCalls {
.setContainerID(containerBlockData.getBlockID().getContainerID()) .setContainerID(containerBlockData.getBlockID().getContainerID())
.setTraceID(traceID).setDatanodeUuid(id) .setTraceID(traceID).setDatanodeUuid(id)
.setPutBlock(createBlockRequest).build(); .setPutBlock(createBlockRequest).build();
xceiverClient.sendCommand(request);
return xceiverClient.sendCommandAsync(request); return xceiverClient.sendCommandAsync(request);
} }

View File

@ -365,8 +365,8 @@
<name>ozone.client.stream.buffer.flush.size</name> <name>ozone.client.stream.buffer.flush.size</name>
<value>64MB</value> <value>64MB</value>
<tag>OZONE, CLIENT</tag> <tag>OZONE, CLIENT</tag>
<description>Size in mb which determines at what buffer position , a partial <description>Size which determines at what buffer position , a partial
flush will be initiated during write. It should be ideally a mutiple flush will be initiated during write. It should be ideally a multiple
of chunkSize. of chunkSize.
</description> </description>
</property> </property>
@ -374,8 +374,8 @@
<name>ozone.client.stream.buffer.max.size</name> <name>ozone.client.stream.buffer.max.size</name>
<value>128MB</value> <value>128MB</value>
<tag>OZONE, CLIENT</tag> <tag>OZONE, CLIENT</tag>
<description>Size in mb which determines at what buffer position , <description>Size which determines at what buffer position,
write call be blocked till acknowledgement of the fisrt partial flush write call be blocked till acknowledgement of the first partial flush
happens by all servers. happens by all servers.
</description> </description>
</property> </property>

View File

@ -21,7 +21,7 @@ import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.classification.InterfaceStability;
/** /**
* DispatcherContext class holds transport protocol specfic context info * DispatcherContext class holds transport protocol specific context info
* required for execution of container commands over the container dispatcher. * required for execution of container commands over the container dispatcher.
*/ */
@InterfaceAudience.Private @InterfaceAudience.Private
@ -121,7 +121,7 @@ public class DispatcherContext {
} }
/** /**
* Builds and returns DatanodeDetails instance. * Builds and returns DispatcherContext instance.
* *
* @return DispatcherContext * @return DispatcherContext
*/ */

View File

@ -83,8 +83,8 @@ public class ChunkGroupOutputStream extends OutputStream {
private final long streamBufferMaxSize; private final long streamBufferMaxSize;
private final long watchTimeout; private final long watchTimeout;
private final long blockSize; private final long blockSize;
private ByteBuffer buffer;
private final Checksum checksum; private final Checksum checksum;
private List<ByteBuffer> bufferList;
/** /**
* A constructor for testing purpose only. * A constructor for testing purpose only.
*/ */
@ -101,7 +101,9 @@ public class ChunkGroupOutputStream extends OutputStream {
closed = false; closed = false;
streamBufferFlushSize = 0; streamBufferFlushSize = 0;
streamBufferMaxSize = 0; streamBufferMaxSize = 0;
buffer = ByteBuffer.allocate(1); bufferList = new ArrayList<>(1);
ByteBuffer buffer = ByteBuffer.allocate(1);
bufferList.add(buffer);
watchTimeout = 0; watchTimeout = 0;
blockSize = 0; blockSize = 0;
this.checksum = new Checksum(); this.checksum = new Checksum();
@ -177,15 +179,7 @@ public class ChunkGroupOutputStream extends OutputStream {
Preconditions.checkState(streamBufferFlushSize % chunkSize == 0); Preconditions.checkState(streamBufferFlushSize % chunkSize == 0);
Preconditions.checkState(streamBufferMaxSize % streamBufferFlushSize == 0); Preconditions.checkState(streamBufferMaxSize % streamBufferFlushSize == 0);
Preconditions.checkState(blockSize % streamBufferMaxSize == 0); Preconditions.checkState(blockSize % streamBufferMaxSize == 0);
this.bufferList = new ArrayList<>();
// This byteBuffer will be used to cache data until all the blockCommits
// (putBlock) gets replicated to all/majority servers. The idea here is to
// allocate the buffer of size blockSize so that as and when a chunk is
// is replicated to all servers, as a part of discarding the buffer, we
// don't necessarily need to run compaction(buffer.compact() on the buffer
// to actually discard the acknowledged data. Compaction is inefficient so
// it would be a better choice to avoid compaction on the happy I/O path.
this.buffer = ByteBuffer.allocate((int) blockSize);
} }
/** /**
@ -222,12 +216,7 @@ public class ChunkGroupOutputStream extends OutputStream {
streamEntries.add(new ChunkOutputStreamEntry(subKeyInfo.getBlockID(), streamEntries.add(new ChunkOutputStreamEntry(subKeyInfo.getBlockID(),
keyArgs.getKeyName(), xceiverClientManager, xceiverClient, requestID, keyArgs.getKeyName(), xceiverClientManager, xceiverClient, requestID,
chunkSize, subKeyInfo.getLength(), streamBufferFlushSize, chunkSize, subKeyInfo.getLength(), streamBufferFlushSize,
streamBufferMaxSize, watchTimeout, buffer, checksum)); streamBufferMaxSize, watchTimeout, bufferList, checksum));
}
@VisibleForTesting
public long getByteOffset() {
return getKeyLength();
} }
@ -254,11 +243,6 @@ public class ChunkGroupOutputStream extends OutputStream {
public void write(byte[] b, int off, int len) public void write(byte[] b, int off, int len)
throws IOException { throws IOException {
checkNotClosed(); checkNotClosed();
handleWrite(b, off, len, false, buffer.position());
}
private void handleWrite(byte[] b, int off, int len, boolean retry,
int pos) throws IOException {
if (b == null) { if (b == null) {
throw new NullPointerException(); throw new NullPointerException();
} }
@ -269,8 +253,17 @@ public class ChunkGroupOutputStream extends OutputStream {
if (len == 0) { if (len == 0) {
return; return;
} }
handleWrite(b, off, len, false);
}
private long computeBufferData() {
return bufferList.stream().mapToInt(value -> value.position())
.sum();
}
private void handleWrite(byte[] b, int off, long len, boolean retry)
throws IOException {
int succeededAllocates = 0; int succeededAllocates = 0;
int initialPos;
while (len > 0) { while (len > 0) {
if (streamEntries.size() <= currentStreamIndex) { if (streamEntries.size() <= currentStreamIndex) {
Preconditions.checkNotNull(omClient); Preconditions.checkNotNull(omClient);
@ -289,8 +282,12 @@ public class ChunkGroupOutputStream extends OutputStream {
// still do a sanity check. // still do a sanity check.
Preconditions.checkArgument(currentStreamIndex < streamEntries.size()); Preconditions.checkArgument(currentStreamIndex < streamEntries.size());
ChunkOutputStreamEntry current = streamEntries.get(currentStreamIndex); ChunkOutputStreamEntry current = streamEntries.get(currentStreamIndex);
int writeLen = Math.min(len, (int) current.getRemaining());
initialPos = pos < buffer.position() ? pos : buffer.position(); // length(len) will be in int range if the call is happening through
// write API of chunkOutputStream. Length can be in long range if it comes
// via Exception path.
int writeLen = Math.min((int)len, (int) current.getRemaining());
long currentPos = current.getWrittenDataLength();
try { try {
if (retry) { if (retry) {
current.writeOnRetry(len); current.writeOnRetry(len);
@ -299,9 +296,10 @@ public class ChunkGroupOutputStream extends OutputStream {
} }
} catch (IOException ioe) { } catch (IOException ioe) {
if (checkIfContainerIsClosed(ioe) || checkIfTimeoutException(ioe)) { if (checkIfContainerIsClosed(ioe) || checkIfTimeoutException(ioe)) {
// for the current iteration, current pos - initialPos gives the // for the current iteration, totalDataWritten - currentPos gives the
// amount of data already written to the buffer // amount of data already written to the buffer
writeLen = buffer.position() - initialPos; writeLen = (int) (current.getWrittenDataLength() - currentPos);
LOG.debug("writeLen {}, total len {}", writeLen, len);
handleException(current, currentStreamIndex); handleException(current, currentStreamIndex);
} else { } else {
throw ioe; throw ioe;
@ -366,30 +364,20 @@ public class ChunkGroupOutputStream extends OutputStream {
*/ */
private void handleException(ChunkOutputStreamEntry streamEntry, private void handleException(ChunkOutputStreamEntry streamEntry,
int streamIndex) throws IOException { int streamIndex) throws IOException {
int lastSuccessfulFlushIndex = streamEntry.getLastSuccessfulFlushIndex(); long totalSuccessfulFlushedData =
int currentPos = buffer.position(); streamEntry.getTotalSuccessfulFlushedData();
//set the correct length for the current stream
// In case of a failure, read the data from the position till the last streamEntry.currentPosition = totalSuccessfulFlushedData;
// acknowledgement happened. long bufferedDataLen = computeBufferData();
if (lastSuccessfulFlushIndex > 0) { // just clean up the current stream.
buffer.position(lastSuccessfulFlushIndex); streamEntry.cleanup();
buffer.limit(currentPos); if (bufferedDataLen > 0) {
buffer.compact();
}
if (buffer.position() > 0) {
//set the correct length for the current stream
streamEntry.currentPosition = lastSuccessfulFlushIndex;
// If the data is still cached in the underlying stream, we need to // If the data is still cached in the underlying stream, we need to
// allocate new block and write this data in the datanode. // allocate new block and write this data in the datanode.
currentStreamIndex += 1; currentStreamIndex += 1;
handleWrite(buffer.array(), 0, buffer.position(), true, handleWrite(null, 0, bufferedDataLen, true);
lastSuccessfulFlushIndex);
} }
if (totalSuccessfulFlushedData == 0) {
// just clean up the current stream.
streamEntry.cleanup();
if (lastSuccessfulFlushIndex == 0) {
streamEntries.remove(streamIndex); streamEntries.remove(streamIndex);
currentStreamIndex -= 1; currentStreamIndex -= 1;
} }
@ -433,7 +421,7 @@ public class ChunkGroupOutputStream extends OutputStream {
} }
private long getKeyLength() { private long getKeyLength() {
return streamEntries.parallelStream().mapToLong(e -> e.currentPosition) return streamEntries.stream().mapToLong(e -> e.currentPosition)
.sum(); .sum();
} }
@ -517,10 +505,10 @@ public class ChunkGroupOutputStream extends OutputStream {
} catch (IOException ioe) { } catch (IOException ioe) {
throw ioe; throw ioe;
} finally { } finally {
if (buffer != null) { if (bufferList != null) {
buffer.clear(); bufferList.stream().forEach(e -> e.clear());
} }
buffer = null; bufferList = null;
} }
} }
@ -633,13 +621,13 @@ public class ChunkGroupOutputStream extends OutputStream {
private final long streamBufferFlushSize; private final long streamBufferFlushSize;
private final long streamBufferMaxSize; private final long streamBufferMaxSize;
private final long watchTimeout; private final long watchTimeout;
private ByteBuffer buffer; private List<ByteBuffer> bufferList;
ChunkOutputStreamEntry(BlockID blockID, String key, ChunkOutputStreamEntry(BlockID blockID, String key,
XceiverClientManager xceiverClientManager, XceiverClientManager xceiverClientManager,
XceiverClientSpi xceiverClient, String requestId, int chunkSize, XceiverClientSpi xceiverClient, String requestId, int chunkSize,
long length, long streamBufferFlushSize, long streamBufferMaxSize, long length, long streamBufferFlushSize, long streamBufferMaxSize,
long watchTimeout, ByteBuffer buffer, Checksum checksum) { long watchTimeout, List<ByteBuffer> bufferList, Checksum checksum) {
this.outputStream = null; this.outputStream = null;
this.blockID = blockID; this.blockID = blockID;
this.key = key; this.key = key;
@ -653,8 +641,8 @@ public class ChunkGroupOutputStream extends OutputStream {
this.streamBufferFlushSize = streamBufferFlushSize; this.streamBufferFlushSize = streamBufferFlushSize;
this.streamBufferMaxSize = streamBufferMaxSize; this.streamBufferMaxSize = streamBufferMaxSize;
this.watchTimeout = watchTimeout; this.watchTimeout = watchTimeout;
this.buffer = buffer;
this.checksum = checksum; this.checksum = checksum;
this.bufferList = bufferList;
} }
/** /**
@ -676,7 +664,7 @@ public class ChunkGroupOutputStream extends OutputStream {
this.currentPosition = 0; this.currentPosition = 0;
streamBufferFlushSize = 0; streamBufferFlushSize = 0;
streamBufferMaxSize = 0; streamBufferMaxSize = 0;
buffer = null; bufferList = null;
watchTimeout = 0; watchTimeout = 0;
this.checksum = checksum; this.checksum = checksum;
} }
@ -694,7 +682,7 @@ public class ChunkGroupOutputStream extends OutputStream {
this.outputStream = this.outputStream =
new ChunkOutputStream(blockID, key, xceiverClientManager, new ChunkOutputStream(blockID, key, xceiverClientManager,
xceiverClient, requestId, chunkSize, streamBufferFlushSize, xceiverClient, requestId, chunkSize, streamBufferFlushSize,
streamBufferMaxSize, watchTimeout, buffer, checksum); streamBufferMaxSize, watchTimeout, bufferList, checksum);
} }
} }
@ -731,11 +719,24 @@ public class ChunkGroupOutputStream extends OutputStream {
} }
} }
int getLastSuccessfulFlushIndex() throws IOException { long getTotalSuccessfulFlushedData() throws IOException {
if (this.outputStream instanceof ChunkOutputStream) { if (this.outputStream instanceof ChunkOutputStream) {
ChunkOutputStream out = (ChunkOutputStream) this.outputStream; ChunkOutputStream out = (ChunkOutputStream) this.outputStream;
blockID = out.getBlockID(); blockID = out.getBlockID();
return out.getLastSuccessfulFlushIndex(); return out.getTotalSuccessfulFlushedData();
} else if (outputStream == null) {
// For a pre allocated block for which no write has been initiated,
// the OutputStream will be null here.
// In such cases, the default blockCommitSequenceId will be 0
return 0;
}
throw new IOException("Invalid Output Stream for Key: " + key);
}
long getWrittenDataLength() throws IOException {
if (this.outputStream instanceof ChunkOutputStream) {
ChunkOutputStream out = (ChunkOutputStream) this.outputStream;
return out.getWrittenDataLength();
} else if (outputStream == null) { } else if (outputStream == null) {
// For a pre allocated block for which no write has been initiated, // For a pre allocated block for which no write has been initiated,
// the OutputStream will be null here. // the OutputStream will be null here.
@ -753,7 +754,7 @@ public class ChunkGroupOutputStream extends OutputStream {
} }
} }
void writeOnRetry(int len) throws IOException { void writeOnRetry(long len) throws IOException {
checkStream(); checkStream();
if (this.outputStream instanceof ChunkOutputStream) { if (this.outputStream instanceof ChunkOutputStream) {
ChunkOutputStream out = (ChunkOutputStream) this.outputStream; ChunkOutputStream out = (ChunkOutputStream) this.outputStream;

View File

@ -109,6 +109,10 @@ public class TestCloseContainerHandlingByClient {
objectStore.getVolume(volumeName).createBucket(bucketName); objectStore.getVolume(volumeName).createBucket(bucketName);
} }
private String getKeyName() {
return UUID.randomUUID().toString();
}
/** /**
* Shutdown MiniDFSCluster. * Shutdown MiniDFSCluster.
*/ */
@ -121,7 +125,7 @@ public class TestCloseContainerHandlingByClient {
@Test @Test
public void testBlockWritesWithFlushAndClose() throws Exception { public void testBlockWritesWithFlushAndClose() throws Exception {
String keyName = "standalone"; String keyName = getKeyName();
OzoneOutputStream key = createKey(keyName, ReplicationType.RATIS, 0); OzoneOutputStream key = createKey(keyName, ReplicationType.RATIS, 0);
// write data more than 1 chunk // write data more than 1 chunk
byte[] data = ContainerTestHelper byte[] data = ContainerTestHelper
@ -153,7 +157,7 @@ public class TestCloseContainerHandlingByClient {
@Test @Test
public void testBlockWritesCloseConsistency() throws Exception { public void testBlockWritesCloseConsistency() throws Exception {
String keyName = "standalone2"; String keyName = getKeyName();
OzoneOutputStream key = createKey(keyName, ReplicationType.RATIS, 0); OzoneOutputStream key = createKey(keyName, ReplicationType.RATIS, 0);
// write data more than 1 chunk // write data more than 1 chunk
byte[] data = ContainerTestHelper byte[] data = ContainerTestHelper
@ -181,7 +185,7 @@ public class TestCloseContainerHandlingByClient {
@Test @Test
public void testMultiBlockWrites() throws Exception { public void testMultiBlockWrites() throws Exception {
String keyName = "standalone3"; String keyName = getKeyName();
OzoneOutputStream key = OzoneOutputStream key =
createKey(keyName, ReplicationType.RATIS, (4 * blockSize)); createKey(keyName, ReplicationType.RATIS, (4 * blockSize));
ChunkGroupOutputStream groupOutputStream = ChunkGroupOutputStream groupOutputStream =
@ -227,8 +231,7 @@ public class TestCloseContainerHandlingByClient {
@Test @Test
public void testMultiBlockWrites2() throws Exception { public void testMultiBlockWrites2() throws Exception {
String keyName = "ratis2"; String keyName = getKeyName();
long dataLength;
OzoneOutputStream key = OzoneOutputStream key =
createKey(keyName, ReplicationType.RATIS, 4 * blockSize); createKey(keyName, ReplicationType.RATIS, 4 * blockSize);
ChunkGroupOutputStream groupOutputStream = ChunkGroupOutputStream groupOutputStream =
@ -272,7 +275,7 @@ public class TestCloseContainerHandlingByClient {
@Test @Test
public void testMultiBlockWrites3() throws Exception { public void testMultiBlockWrites3() throws Exception {
String keyName = "standalone5"; String keyName = getKeyName();
int keyLen = 4 * blockSize; int keyLen = 4 * blockSize;
OzoneOutputStream key = createKey(keyName, ReplicationType.RATIS, keyLen); OzoneOutputStream key = createKey(keyName, ReplicationType.RATIS, keyLen);
ChunkGroupOutputStream groupOutputStream = ChunkGroupOutputStream groupOutputStream =
@ -391,7 +394,7 @@ public class TestCloseContainerHandlingByClient {
// on the datanode. // on the datanode.
@Test @Test
public void testDiscardPreallocatedBlocks() throws Exception { public void testDiscardPreallocatedBlocks() throws Exception {
String keyName = "discardpreallocatedblocks"; String keyName = getKeyName();
OzoneOutputStream key = OzoneOutputStream key =
createKey(keyName, ReplicationType.RATIS, 2 * blockSize); createKey(keyName, ReplicationType.RATIS, 2 * blockSize);
ChunkGroupOutputStream groupOutputStream = ChunkGroupOutputStream groupOutputStream =
@ -447,7 +450,7 @@ public class TestCloseContainerHandlingByClient {
@Test @Test
public void testBlockWriteViaRatis() throws Exception { public void testBlockWriteViaRatis() throws Exception {
String keyName = "ratis"; String keyName = getKeyName();
OzoneOutputStream key = createKey(keyName, ReplicationType.RATIS, 0); OzoneOutputStream key = createKey(keyName, ReplicationType.RATIS, 0);
byte[] data = ContainerTestHelper byte[] data = ContainerTestHelper
.getFixedLengthString(keyString, chunkSize + chunkSize / 2) .getFixedLengthString(keyString, chunkSize + chunkSize / 2)

View File

@ -65,10 +65,6 @@ public class TestFailureHandlingByClient {
private static String keyString; private static String keyString;
private static int maxRetries; private static int maxRetries;
/**
* TODO: we will spawn new MiniOzoneCluster every time for each unit test
* invocation. Need to use the same instance for all tests.
*/
/** /**
* Create a MiniDFSCluster for testing. * Create a MiniDFSCluster for testing.
* <p> * <p>
@ -86,6 +82,11 @@ public class TestFailureHandlingByClient {
TimeUnit.SECONDS); TimeUnit.SECONDS);
conf.setTimeDuration(HDDS_SCM_WATCHER_TIMEOUT, 1000, TimeUnit.MILLISECONDS); conf.setTimeDuration(HDDS_SCM_WATCHER_TIMEOUT, 1000, TimeUnit.MILLISECONDS);
conf.setTimeDuration(OZONE_SCM_STALENODE_INTERVAL, 3, TimeUnit.SECONDS); conf.setTimeDuration(OZONE_SCM_STALENODE_INTERVAL, 3, TimeUnit.SECONDS);
conf.setInt(OzoneConfigKeys.DFS_RATIS_CLIENT_REQUEST_MAX_RETRIES_KEY, 5);
conf.setTimeDuration(
OzoneConfigKeys.DFS_RATIS_CLIENT_REQUEST_RETRY_INTERVAL_KEY,
1, TimeUnit.SECONDS);
conf.setQuietMode(false); conf.setQuietMode(false);
cluster = MiniOzoneCluster.newBuilder(conf) cluster = MiniOzoneCluster.newBuilder(conf)
.setNumDatanodes(6).build(); .setNumDatanodes(6).build();
@ -112,7 +113,7 @@ public class TestFailureHandlingByClient {
@Test @Test
public void testBlockWritesWithDnFailures() throws Exception { public void testBlockWritesWithDnFailures() throws Exception {
String keyName = "ratis3"; String keyName = UUID.randomUUID().toString();
OzoneOutputStream key = createKey(keyName, ReplicationType.RATIS, 0); OzoneOutputStream key = createKey(keyName, ReplicationType.RATIS, 0);
byte[] data = byte[] data =
ContainerTestHelper ContainerTestHelper
@ -189,6 +190,51 @@ public class TestFailureHandlingByClient {
validateData(keyName, data.concat(data).getBytes()); validateData(keyName, data.concat(data).getBytes());
} }
@Test
public void testMultiBlockWritesWithIntermittentDnFailures()
throws Exception {
String keyName = UUID.randomUUID().toString();
OzoneOutputStream key =
createKey(keyName, ReplicationType.RATIS, 6 * blockSize);
String data = ContainerTestHelper
.getFixedLengthString(keyString, blockSize + chunkSize);
key.write(data.getBytes());
// get the name of a valid container
Assert.assertTrue(key.getOutputStream() instanceof ChunkGroupOutputStream);
ChunkGroupOutputStream groupOutputStream =
(ChunkGroupOutputStream) key.getOutputStream();
List<OmKeyLocationInfo> locationInfoList =
groupOutputStream.getLocationInfoList();
Assert.assertTrue(locationInfoList.size() == 6);
long containerId = locationInfoList.get(1).getContainerID();
ContainerInfo container =
cluster.getStorageContainerManager().getContainerManager()
.getContainer(ContainerID.valueof(containerId));
Pipeline pipeline =
cluster.getStorageContainerManager().getPipelineManager()
.getPipeline(container.getPipelineID());
List<DatanodeDetails> datanodes = pipeline.getNodes();
cluster.shutdownHddsDatanode(datanodes.get(0));
// The write will fail but exception will be handled and length will be
// updated correctly in OzoneManager once the steam is closed
key.write(data.getBytes());
// shutdown the second datanode
cluster.shutdownHddsDatanode(datanodes.get(1));
key.write(data.getBytes());
key.close();
OmKeyArgs keyArgs = new OmKeyArgs.Builder().setVolumeName(volumeName)
.setBucketName(bucketName).setType(HddsProtos.ReplicationType.RATIS)
.setFactor(HddsProtos.ReplicationFactor.THREE).setKeyName(keyName)
.build();
OmKeyInfo keyInfo = cluster.getOzoneManager().lookupKey(keyArgs);
Assert.assertEquals(3 * data.getBytes().length, keyInfo.getDataSize());
validateData(keyName, data.concat(data).concat(data).getBytes());
}
private OzoneOutputStream createKey(String keyName, ReplicationType type, private OzoneOutputStream createKey(String keyName, ReplicationType type,
long size) throws Exception { long size) throws Exception {
return ContainerTestHelper return ContainerTestHelper

View File

@ -18,15 +18,12 @@ package org.apache.hadoop.ozone.om;
import org.apache.commons.lang3.RandomStringUtils; import org.apache.commons.lang3.RandomStringUtils;
import org.apache.hadoop.ozone.client.io.ChunkGroupInputStream; import org.apache.hadoop.ozone.client.io.ChunkGroupInputStream;
import org.apache.hadoop.ozone.client.io.ChunkGroupOutputStream;
import org.apache.hadoop.hdds.scm.storage.ChunkInputStream; import org.apache.hadoop.hdds.scm.storage.ChunkInputStream;
import org.junit.Rule; import org.junit.Rule;
import org.junit.Test; import org.junit.Test;
import org.junit.rules.ExpectedException; import org.junit.rules.ExpectedException;
import java.io.ByteArrayInputStream; import java.io.ByteArrayInputStream;
import java.io.ByteArrayOutputStream;
import java.io.OutputStream;
import java.io.IOException; import java.io.IOException;
import java.util.ArrayList; import java.util.ArrayList;
@ -41,81 +38,9 @@ public class TestChunkStreams {
@Rule @Rule
public ExpectedException exception = ExpectedException.none(); public ExpectedException exception = ExpectedException.none();
/**
* This test uses ByteArrayOutputStream as the underlying stream to test
* the correctness of ChunkGroupOutputStream.
*
* @throws Exception
*/
@Test
public void testWriteGroupOutputStream() throws Exception {
try (ChunkGroupOutputStream groupOutputStream =
new ChunkGroupOutputStream()) {
ArrayList<OutputStream> outputStreams = new ArrayList<>();
// 5 byte streams, each 100 bytes. write 500 bytes means writing to each
// of them with 100 bytes.
for (int i = 0; i < 5; i++) {
ByteArrayOutputStream out = new ByteArrayOutputStream(100);
outputStreams.add(out);
groupOutputStream.addStream(out, 100);
}
assertEquals(0, groupOutputStream.getByteOffset());
String dataString = RandomStringUtils.randomAscii(500);
byte[] data = dataString.getBytes(UTF_8);
groupOutputStream.write(data, 0, data.length);
assertEquals(500, groupOutputStream.getByteOffset());
String res = "";
int offset = 0;
for (OutputStream stream : outputStreams) {
String subString = stream.toString();
res += subString;
assertEquals(dataString.substring(offset, offset + 100), subString);
offset += 100;
}
assertEquals(dataString, res);
}
}
@Test
public void testErrorWriteGroupOutputStream() throws Exception {
try (ChunkGroupOutputStream groupOutputStream =
new ChunkGroupOutputStream()) {
ArrayList<OutputStream> outputStreams = new ArrayList<>();
// 5 byte streams, each 100 bytes. write 500 bytes means writing to each
// of them with 100 bytes. all 5 streams makes up a ChunkGroupOutputStream
// with a total of 500 bytes in size
for (int i = 0; i < 5; i++) {
ByteArrayOutputStream out = new ByteArrayOutputStream(100);
outputStreams.add(out);
groupOutputStream.addStream(out, 100);
}
assertEquals(0, groupOutputStream.getByteOffset());
// first writes of 100 bytes should succeed
groupOutputStream.write(RandomStringUtils.randomAscii(100)
.getBytes(UTF_8));
assertEquals(100, groupOutputStream.getByteOffset());
// second writes of 500 bytes should fail, as there should be only 400
// bytes space left
// TODO : if we decide to take the 400 bytes instead in the future,
// other add more informative error code rather than exception, need to
// change this part.
exception.expect(Exception.class);
groupOutputStream.write(RandomStringUtils.randomAscii(500)
.getBytes(UTF_8));
assertEquals(100, groupOutputStream.getByteOffset());
}
}
@Test @Test
public void testReadGroupInputStream() throws Exception { public void testReadGroupInputStream() throws Exception {
try (ChunkGroupInputStream groupInputStream = new ChunkGroupInputStream()) { try (ChunkGroupInputStream groupInputStream = new ChunkGroupInputStream()) {
ArrayList<ChunkInputStream> inputStreams = new ArrayList<>();
String dataString = RandomStringUtils.randomAscii(500); String dataString = RandomStringUtils.randomAscii(500);
byte[] buf = dataString.getBytes(UTF_8); byte[] buf = dataString.getBytes(UTF_8);
@ -157,7 +82,6 @@ public class TestChunkStreams {
return readLen; return readLen;
} }
}; };
inputStreams.add(in);
offset += 100; offset += 100;
groupInputStream.addStream(in, 100); groupInputStream.addStream(in, 100);
} }
@ -173,7 +97,6 @@ public class TestChunkStreams {
@Test @Test
public void testErrorReadGroupInputStream() throws Exception { public void testErrorReadGroupInputStream() throws Exception {
try (ChunkGroupInputStream groupInputStream = new ChunkGroupInputStream()) { try (ChunkGroupInputStream groupInputStream = new ChunkGroupInputStream()) {
ArrayList<ChunkInputStream> inputStreams = new ArrayList<>();
String dataString = RandomStringUtils.randomAscii(500); String dataString = RandomStringUtils.randomAscii(500);
byte[] buf = dataString.getBytes(UTF_8); byte[] buf = dataString.getBytes(UTF_8);
@ -215,7 +138,6 @@ public class TestChunkStreams {
return readLen; return readLen;
} }
}; };
inputStreams.add(in);
offset += 100; offset += 100;
groupInputStream.addStream(in, 100); groupInputStream.addStream(in, 100);
} }