-
Notifications
You must be signed in to change notification settings - Fork 592
HDDS-6039.Define a minimum packet size during streaming writes. #2883
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
946897d
fb5c4d0
9c34e32
d74d07d
d4bedad
c0b76a9
2d85b61
494d9cb
2383a28
f791369
30388bc
0aab669
2feac65
3623c99
5f7951a
93afa56
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -29,6 +29,8 @@ | |
| import org.apache.hadoop.hdds.ratis.ContainerCommandRequestMessage; | ||
| import org.apache.hadoop.hdds.scm.OzoneClientConfig; | ||
| import org.apache.hadoop.hdds.scm.XceiverClientFactory; | ||
| import org.apache.hadoop.hdds.scm.XceiverClientManager; | ||
| import org.apache.hadoop.hdds.scm.XceiverClientMetrics; | ||
| import org.apache.hadoop.hdds.scm.XceiverClientRatis; | ||
| import org.apache.hadoop.hdds.scm.XceiverClientReply; | ||
| import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException; | ||
|
|
@@ -125,7 +127,8 @@ public class BlockDataStreamOutput implements ByteBufferStreamOutput { | |
| private List<CompletableFuture<DataStreamReply>> futures = new ArrayList<>(); | ||
| private final long syncSize = 0; // TODO: disk sync is disabled for now | ||
| private long syncPosition = 0; | ||
|
|
||
| private StreamBuffer currentBuffer; | ||
| private XceiverClientMetrics metrics; | ||
|
||
| /** | ||
| * Creates a new BlockDataStreamOutput. | ||
| * | ||
|
|
@@ -172,6 +175,7 @@ public BlockDataStreamOutput( | |
| ioException = new AtomicReference<>(null); | ||
| checksum = new Checksum(config.getChecksumType(), | ||
| config.getBytesPerChecksum()); | ||
| metrics = XceiverClientManager.getXceiverClientMetrics(); | ||
| } | ||
|
|
||
| private DataStreamOutput setupStream(Pipeline pipeline) throws IOException { | ||
|
|
@@ -257,27 +261,47 @@ public void write(ByteBuffer b, int off, int len) throws IOException { | |
| if (len == 0) { | ||
| return; | ||
| } | ||
| int curLen = len; | ||
| // set limit on the number of bytes that a ByteBuffer(StreamBuffer) can hold | ||
| int maxBufferLen = config.getDataStreamMaxBufferSize(); | ||
| while (curLen > 0) { | ||
| int writeLen = Math.min(curLen, maxBufferLen); | ||
| while (len > 0) { | ||
| allocateNewBufferIfNeeded(); | ||
| int writeLen = Math.min(len, currentBuffer.length()); | ||
| final StreamBuffer buf = new StreamBuffer(b, off, writeLen); | ||
| currentBuffer.put(buf); | ||
| writeChunkIfNeeded(); | ||
| off += writeLen; | ||
| bufferList.add(buf); | ||
| writeChunkToContainer(buf.duplicate()); | ||
| curLen -= writeLen; | ||
| writtenDataLength += writeLen; | ||
| len -= writeLen; | ||
| doFlushIfNeeded(); | ||
| } | ||
| } | ||
|
|
||
| private void writeChunkIfNeeded() throws IOException { | ||
| if (currentBuffer.length()==0) { | ||
| writeChunk(currentBuffer); | ||
| currentBuffer = null; | ||
| } | ||
| } | ||
|
|
||
| private void writeChunk(StreamBuffer sb) throws IOException { | ||
| bufferList.add(sb); | ||
| ByteBuffer dup = sb.duplicate(); | ||
| dup.position(0); | ||
| dup.limit(sb.position()); | ||
| writeChunkToContainer(dup); | ||
| } | ||
|
|
||
| private void allocateNewBufferIfNeeded() { | ||
| if (currentBuffer==null) { | ||
| currentBuffer = | ||
| StreamBuffer.allocate(config.getDataStreamMinPacketSize()); | ||
| } | ||
| } | ||
|
|
||
| private void doFlushIfNeeded() throws IOException { | ||
| Preconditions.checkArgument(config.getDataStreamBufferFlushSize() > config | ||
| .getDataStreamMaxBufferSize()); | ||
| long boundary = config.getDataStreamBufferFlushSize() / config | ||
| .getDataStreamMaxBufferSize(); | ||
| if (bufferList.size() % boundary == 0) { | ||
| if (!bufferList.isEmpty() && bufferList.size() % boundary == 0) { | ||
| updateFlushLength(); | ||
| executePutBlock(false, false); | ||
| } | ||
|
|
@@ -308,11 +332,10 @@ public void writeOnRetry(long len) throws IOException { | |
| int count = 0; | ||
| while (len > 0) { | ||
| final StreamBuffer buf = bufferList.get(count); | ||
| final long writeLen = Math.min(buf.length(), len); | ||
| final long writeLen = Math.min(buf.position(), len); | ||
| final ByteBuffer duplicated = buf.duplicate(); | ||
| if (writeLen != buf.length()) { | ||
| duplicated.limit(Math.toIntExact(len)); | ||
| } | ||
| duplicated.position(0); | ||
|
||
| duplicated.limit(buf.position()); | ||
| writeChunkToContainer(duplicated); | ||
| len -= writeLen; | ||
| count++; | ||
|
|
@@ -449,6 +472,11 @@ private void handleFlush(boolean close) | |
| // This can be a partially filled chunk. Since we are flushing the buffer | ||
| // here, we just limit this buffer to the current position. So that next | ||
| // write will happen in new buffer | ||
|
|
||
| if (currentBuffer!=null) { | ||
| writeChunk(currentBuffer); | ||
| currentBuffer = null; | ||
| } | ||
| updateFlushLength(); | ||
| executePutBlock(close, false); | ||
| } else if (close) { | ||
|
|
@@ -584,6 +612,7 @@ private void writeChunkToContainer(ByteBuffer buf) | |
| .setLen(effectiveChunkSize) | ||
| .setChecksumData(checksumData.getProtoBufMessage()) | ||
| .build(); | ||
| metrics.incrPendingContainerOpsMetrics(ContainerProtos.Type.WriteChunk); | ||
|
|
||
| if (LOG.isDebugEnabled()) { | ||
| LOG.debug("Writing chunk {} length {} at offset {}", | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -27,7 +27,7 @@ public class StreamBuffer { | |
| private final ByteBuffer buffer; | ||
|
|
||
| public StreamBuffer(ByteBuffer buffer) { | ||
| this.buffer = buffer.asReadOnlyBuffer(); | ||
| this.buffer = buffer; | ||
| } | ||
|
|
||
| public StreamBuffer(ByteBuffer buffer, int offset, int length) { | ||
|
|
@@ -43,4 +43,17 @@ public int length() { | |
| return buffer.limit() - buffer.position(); | ||
| } | ||
|
|
||
| public int position() { | ||
| return buffer.position(); | ||
| } | ||
|
|
||
|
|
||
| public void put(StreamBuffer sb){ | ||
| buffer.put(sb.buffer); | ||
| } | ||
|
|
||
| public static StreamBuffer allocate(int size){ | ||
|
||
| return new StreamBuffer(ByteBuffer.allocate(size)); | ||
| } | ||
|
|
||
| } | ||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Let's set this size to 1 MB == bytesPerCheckSum by default.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Set to 1Mb
Uh oh!
There was an error while loading. Please reload this page.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
In performance test, I found that the performance of 512K is better than that of IMB. Can we change the default value to 512K?