Skip to content

Conversation

@chungen0126
Copy link
Contributor

@chungen0126 chungen0126 commented Apr 30, 2024

What changes were proposed in this pull request?

Introduce a new BlockInputStream to allow a block to be streamed to a client with a single RPC call.

Upon receiving the request to stream a block, the server seeks the block file to the given offset and starts sending pieces of the block to the client. Each piece of the block sent is a checksum sized piece of the block.

For example, if the checksum size is 16kb (the current default in Ozone), and a client requests a block starting at offset 17kb. Then the server will align the read to the checksum boundary and return 16kb of data starting at the 16kb offset and include the correct checksum. The server will then keep sending the subsequent pieces of the block to the client until it is blocked from doing so, where it will stall in the GRPC onNext() method. The block file and file position are maintained in the server thread while the file remains open on the client side. This means the server handler thread is also blocked.

On the client side, it will buffer the first response and wait for it to be consumed by any wrapping block streams. A blocking queue is used to prevent the server from sending too much data and hence provide back pressure to the server to prevent it sending any more until the client can consume it. In my testing, I found the server will send pieces of the block up to about 1MB which block on the TCP/IP socket until the client is able to read it off.

Compared to the current code in Ozone, with 16kb checksums, the time taken to cat a 1GB file on a docker cluster on my macbook as about 30 seconds:

# Original code, 16kb checksums.
$ time ozone sh key cat sodonnell/bucket/1gb > /dev/null

real	0m30.409s
user	0m15.489s
sys	0m6.544s

With this change, the response time dropped to about 7.6 seconds:

bash-5.1$ time ozone sh key cat sodonnell/bucket/1gb > /dev/null

real	0m7.600s
user	0m8.302s
sys	0m2.948s

Even with 1MB checksums, the speed up was 4.39s to 2.6s.

This approach has some potential problems, the main one being that it blocks server threads in the aim of efficiency. This means that this approach will require scaling the server handler pool accordingly, and possibly creating a new handler pool for stream reads to prevent them impacting other short lived calls (eg put block, traditional read block).

There are also a few pieces of this change that will need further work, eg:

  1. A timeout on the client side to unbuffer if the reader is idle for too long, hence preventing the thread being blocked on the server indefinitely. This can be added in a followup PR.
  2. The client will not currently retry on a failed read after it has a successful initial read from a DN. This will also need to handle the server thread timing out. This can be added in a followup PR.

What is the link to the Apache JIRA

https://issues.apache.org/jira/browse/HDDS-10338

How was this patch tested?

Various new tests added

@szetszwo
Copy link
Contributor

szetszwo commented Nov 3, 2025

I still have reservations that the chatty nature of the client having to request each piece of the block will add latency and overhead we don't need limiting any potential performance gains, especially with the size of reads I have seen when testing from the haoop shell and spark.

This is for avoiding the deadlock but not the performance.

BTW, how large is enough? Would it slows down other operations for creating such a large handler pool?

I don't know, but I have seen HDFS work just fine with 32k threads active, provided the open file limits are high enough.

So, we have to start 32k threads for each client? It does not sound right. (Sorry that I missed your reply previously.)

Also, it probably won't work well for YARN or S3Gateway since they need to start a client for each user and it has to support thousands of users.

@sodonnel
Copy link
Contributor

sodonnel commented Nov 4, 2025

So, we have to start 32k threads for each client? It does not sound right. (Sorry that I missed your reply previously.)

The thread overhead is on the datanode side, as streaming a block to a client ties up the server handler thread. So if you have lots of blocks being read slowly it will use up the handlers. I don't think there is an issue with the number of threads needed on the client side. For any given block read, it will need 1 extra thread while the block is being read.

@sodonnel
Copy link
Contributor

sodonnel commented Nov 4, 2025

I may have misunderstood "small read". What do you mean by small read? (1) or (2) below?

1. client opens a file, reads a small amount of data (say 4KB) and then the close the file; or

2. client opens a file with a small buffer (say 4KB), keep looping calling read(buffer) for many times and then the close the file.

I have been primarily looking at 2, which I believe is the most common spark, Hive type workload, where blocks are largely read completely.

1 is more like an HBase use case, where it will scatter positional reads all over the file based on which part of the block its index tells it to read.

*/
public class StreamingReader implements StreamingReaderSpi {

private final BlockingQueue<ContainerProtos.ReadBlockResponseProto> responseQueue = new LinkedBlockingQueue<>(1);
Copy link
Contributor

@szetszwo szetszwo Nov 4, 2025

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The thread overhead is on the datanode side, as streaming a block to a client ties up the server handler thread. ...

@sodonnel , This BlockingQueue<> responseQueue is at the client side. So, the thread blocking problem is at the client side.

Server side gRPC threads won't block in the current implementation. Since the server side onNext() implementation does not block. We may print some debug messages to verify.

It would be much better if the blocking thread problem is at the server side.

Copy link
Contributor

@szetszwo szetszwo Nov 4, 2025

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Added client side to #6613 (comment)

... the implementation uses a BlockingQueue responseQueue. When the responses are not removed from the queue, the client side gRPC thread is blocked in StreamingReader.onNext(..) and the gRPC streaming mechanism will throttle and slow down the server side.


ReadBlockResponseProto readBlock;
try {
readBlock = responseQueue.poll(30, TimeUnit.SECONDS);

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can we add param for this timeout value?

return new ECBlockInputStreamProxy((ECReplicationConfig)repConfig,
blockInfo, xceiverFactory, refreshFunction,
ecBlockStreamFactory, config);
} else if (config.isStreamReadBlock() && allDataNodesSupportStreamBlock(pipeline)) {

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can we do streaming for EC?

adjustedOffset += bytesPerChecksum;
}
}
// TODO metrics.incContainerBytesStats(Type.ReadBlock, readBlock.getLen());

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

can we add metrics to see how it performs vs. non-streamed implementation?


private static final Logger LOG = LoggerFactory.getLogger(
KeyValueHandler.class);
private static final int STREAMING_BYTES_PER_CHUNK = 1024 * 64;

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can we have a configuration param for the size?


@Override
public synchronized int read() throws IOException {
checkOpen();

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

it would be good to have open telemetry trace here.

}

@Override
public synchronized int read(byte[] b, int off, int len) throws IOException {

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

it would be good to have open telemetry trace here.

}

@Override
public synchronized int read(ByteBuffer targetBuf) throws IOException {

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

it would be good to have open telemetry trace here.

}
}

private int fillBuffer() throws IOException {

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

it would be good to have open telemetry trace here.

ByteString checksum = blockData.getChunks().get(chunkIndex).getChecksumData().getChecksums(checksumIndex);
checksumData = new ChecksumData(checksumType, bytesPerChecksum, Collections.singletonList(checksum));
}
streamObserver.onNext(getReadBlockResponse(request, checksumData, buffer, adjustedOffset));

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

here we need to check if streamObserver.isReady(), and if its not ready we should use setOnReadyHandler​(Runnable onReadyHandler) for transferring the remaining data.

@szetszwo
Copy link
Contributor

@sodonnel , Tried to add length to ReadBlockRequestProto. It works well. See https://github.com/szetszwo/ozone/commits/HDDS-10338b/ (please feel free to use the code.)

Tested it with various client buffer sizes (17MB, 3.5MB, 4KB). Below is the case 3.5MB (=3565158).

XXX ---------------------------------------------------------
XXX read 17825792 bytes with bufferSize 3565158
XXX ---------------------------------------------------------
XXX initStreamRead conID: 1 locID: 115816896921600001 on datanode a85ea168-0eb0-46ed-9090-10c44ebd9c10(10.96.128.134/10.96.128.134)
XXX position 0, length 3565158, requested 0, diff 3565158, rounded 3670016
XXX XceiverClientGrpc-1 -> dn-10c44ebd9c10_stream, send onNext request blockID { containerID: 1 localID: 115816896921600001 blockCommitSequenceId: 16 } offset: 0 length: 3670016
XXX server readBlock blockID { containerID: 1 localID: 115816896921600001 blockCommitSequenceId: 16 } offset: 0 length: 3670016
XXX server onNext response 0: dataLength=262144
XXX server onNext response 1: dataLength=262144
XXX server onNext response 2: dataLength=262144
XXX server onNext response 3: dataLength=262144
XXX server onNext response 4: dataLength=262144
XXX server onNext response 5: dataLength=262144
XXX server onNext response 6: dataLength=262144
XXX server onNext response 7: dataLength=262144
XXX server onNext response 8: dataLength=262144
XXX server onNext response 9: dataLength=262144
XXX server onNext response 10: dataLength=262144
XXX server onNext response 11: dataLength=262144
XXX server onNext response 12: dataLength=262144
XXX server onNext response 13: dataLength=262144
XXX server response ended: totalDataLength=3670016, numResponse=14
XXX position 3670016, length 3460300, requested 3670016, diff 3460300, rounded 3670016
XXX XceiverClientGrpc-1 -> dn-10c44ebd9c10_stream, send onNext request blockID { containerID: 1 localID: 115816896921600001 blockCommitSequenceId: 16 } offset: 3670016 length: 3670016
XXX server readBlock blockID { containerID: 1 localID: 115816896921600001 blockCommitSequenceId: 16 } offset: 3670016 length: 3670016
XXX server onNext response 0: dataLength=262144
XXX server onNext response 1: dataLength=262144
XXX server onNext response 2: dataLength=262144
XXX server onNext response 3: dataLength=262144
XXX server onNext response 4: dataLength=262144
XXX server onNext response 5: dataLength=262144
XXX server onNext response 6: dataLength=262144
XXX server onNext response 7: dataLength=262144
XXX server onNext response 8: dataLength=262144
XXX server onNext response 9: dataLength=262144
XXX server onNext response 10: dataLength=262144
XXX server onNext response 11: dataLength=262144
XXX server onNext response 12: dataLength=262144
XXX server onNext response 13: dataLength=262144
XXX server response ended: totalDataLength=3670016, numResponse=14
XXX position 7340032, length 1048576, requested 7340032, diff 1048576, rounded 1048576
XXX XceiverClientGrpc-1 -> dn-10c44ebd9c10_stream, send onNext request blockID { containerID: 1 localID: 115816896921600001 blockCommitSequenceId: 16 } offset: 7340032 length: 1048576
XXX server readBlock blockID { containerID: 1 localID: 115816896921600001 blockCommitSequenceId: 16 } offset: 7340032 length: 1048576
XXX server onNext response 0: dataLength=262144
XXX server onNext response 1: dataLength=262144
XXX server onNext response 2: dataLength=262144
XXX server onNext response 3: dataLength=262144
XXX server response ended: totalDataLength=1048576, numResponse=4
XXX initStreamRead conID: 2 locID: 115816896921600002 on datanode a85ea168-0eb0-46ed-9090-10c44ebd9c10(10.96.128.134/10.96.128.134)
XXX position 0, length 2306866, requested 0, diff 2306866, rounded 2359296
XXX XceiverClientGrpc-1 -> dn-10c44ebd9c10_stream, send onNext request blockID { containerID: 2 localID: 115816896921600002 blockCommitSequenceId: 39 } offset: 0 length: 2359296
XXX server readBlock blockID { containerID: 2 localID: 115816896921600002 blockCommitSequenceId: 39 } offset: 0 length: 2359296
XXX server onNext response 0: dataLength=262144
XXX server onNext response 1: dataLength=262144
XXX server onNext response 2: dataLength=262144
XXX server onNext response 3: dataLength=262144
XXX server onNext response 4: dataLength=262144
XXX server onNext response 5: dataLength=262144
XXX server onNext response 6: dataLength=262144
XXX server onNext response 7: dataLength=262144
XXX server onNext response 8: dataLength=262144
XXX server response ended: totalDataLength=2359296, numResponse=9
XXX position 2359296, length 3512728, requested 2359296, diff 3512728, rounded 3670016
XXX XceiverClientGrpc-1 -> dn-10c44ebd9c10_stream, send onNext request blockID { containerID: 2 localID: 115816896921600002 blockCommitSequenceId: 39 } offset: 2359296 length: 3670016
XXX server readBlock blockID { containerID: 2 localID: 115816896921600002 blockCommitSequenceId: 39 } offset: 2359296 length: 3670016
XXX server onNext response 0: dataLength=262144
XXX server onNext response 1: dataLength=262144
XXX server onNext response 2: dataLength=262144
XXX server onNext response 3: dataLength=262144
XXX server onNext response 4: dataLength=262144
XXX server onNext response 5: dataLength=262144
XXX server onNext response 6: dataLength=262144
XXX server onNext response 7: dataLength=262144
XXX server onNext response 8: dataLength=262144
XXX server onNext response 9: dataLength=262144
XXX server onNext response 10: dataLength=262144
XXX server onNext response 11: dataLength=262144
XXX server onNext response 12: dataLength=262144
XXX server onNext response 13: dataLength=262144
XXX server response ended: totalDataLength=3670016, numResponse=14
XXX position 6029312, length 2359296, requested 6029312, diff 2359296, rounded 2359296
XXX XceiverClientGrpc-1 -> dn-10c44ebd9c10_stream, send onNext request blockID { containerID: 2 localID: 115816896921600002 blockCommitSequenceId: 39 } offset: 6029312 length: 2359296
XXX server readBlock blockID { containerID: 2 localID: 115816896921600002 blockCommitSequenceId: 39 } offset: 6029312 length: 2359296
XXX server onNext response 0: dataLength=262144
XXX server onNext response 1: dataLength=262144
XXX server onNext response 2: dataLength=262144
XXX server onNext response 3: dataLength=262144
XXX server onNext response 4: dataLength=262144
XXX server onNext response 5: dataLength=262144
XXX server onNext response 6: dataLength=262144
XXX server onNext response 7: dataLength=262144
XXX server onNext response 8: dataLength=262144
XXX server response ended: totalDataLength=2359296, numResponse=9
XXX initStreamRead conID: 3 locID: 115816896921600003 on datanode a85ea168-0eb0-46ed-9090-10c44ebd9c10(10.96.128.134/10.96.128.134)
XXX position 0, length 1048574, requested 0, diff 1048574, rounded 1048576
XXX XceiverClientGrpc-1 -> dn-10c44ebd9c10_stream, send onNext request blockID { containerID: 3 localID: 115816896921600003 blockCommitSequenceId: 47 } offset: 0 length: 1048576
XXX server readBlock blockID { containerID: 3 localID: 115816896921600003 blockCommitSequenceId: 47 } offset: 0 length: 1048576
XXX server onNext response 0: dataLength=262144
XXX server onNext response 1: dataLength=262144
XXX server onNext response 2: dataLength=262144
XXX server onNext response 3: dataLength=262144
XXX server response ended: totalDataLength=1048576, numResponse=4

@szetszwo
Copy link
Contributor

All three cases: test.log

@szetszwo
Copy link
Contributor

  1. Changed a single response to contain multiple checksums: bytesPerChecksum only affects the performance slightly. Even 512 bytesPerChecksum can get good performance.
  2. Added pre-read supported: the client buffer size does not matter .
  • 512 bytesPerChecksum
XXX ---------------------------------------------------------
XXX key4 with 1G (=1073741824) bytes and 512 bytesPerChecksum
XXX  createStreamKey:   142.14 MB/s (  7.204 s, buffer   8MB (=8388608), keySize  1024.00 MB, md5=61 17 A5 9D AD F1 E1 DD 3B 3D 37 E3 2B 6A 5A 4C)
XXX    readStreamKey:  2622.17 MB/s (  0.391 s, buffer       4k (=4096), keySize  1024.00 MB, md5=null)
XXX    readStreamKey:   767.54 MB/s (  1.334 s, buffer       4k (=4096), keySize  1024.00 MB, md5=61 17 A5 9D AD F1 E1 DD 3B 3D 37 E3 2B 6A 5A 4C)
XXX    readStreamKey:  2659.68 MB/s (  0.385 s, buffer    1M (=1048576), keySize  1024.00 MB, md5=null)
XXX    readStreamKey:   762.32 MB/s (  1.343 s, buffer    1M (=1048576), keySize  1024.00 MB, md5=61 17 A5 9D AD F1 E1 DD 3B 3D 37 E3 2B 6A 5A 4C)
XXX    readStreamKey:  2668.96 MB/s (  0.384 s, buffer    8M (=8388608), keySize  1024.00 MB, md5=null)
XXX    readStreamKey:   727.99 MB/s (  1.407 s, buffer    8M (=8388608), keySize  1024.00 MB, md5=61 17 A5 9D AD F1 E1 DD 3B 3D 37 E3 2B 6A 5A 4C)
XXX    readStreamKey:  2720.65 MB/s (  0.376 s, buffer  32M (=33554432), keySize  1024.00 MB, md5=null)
XXX    readStreamKey:   680.19 MB/s (  1.505 s, buffer  32M (=33554432), keySize  1024.00 MB, md5=61 17 A5 9D AD F1 E1 DD 3B 3D 37 E3 2B 6A 5A 4C)
  • 256k bytesPerChecksum
XXX ---------------------------------------------------------
XXX key3 with 1G (=1073741824) bytes and 256k (=262144) bytesPerChecksum
XXX  createStreamKey:   145.81 MB/s (  7.023 s, buffer   8MB (=8388608), keySize  1024.00 MB, md5=5B 98 A8 2B 30 74 F2 C7 40 59 9B D9 98 8E 59 9C)
XXX    readStreamKey:  2769.22 MB/s (  0.370 s, buffer       4k (=4096), keySize  1024.00 MB, md5=null)
XXX    readStreamKey:   780.02 MB/s (  1.313 s, buffer       4k (=4096), keySize  1024.00 MB, md5=5B 98 A8 2B 30 74 F2 C7 40 59 9B D9 98 8E 59 9C)
XXX    readStreamKey:  2726.07 MB/s (  0.376 s, buffer    1M (=1048576), keySize  1024.00 MB, md5=null)
XXX    readStreamKey:   778.18 MB/s (  1.316 s, buffer    1M (=1048576), keySize  1024.00 MB, md5=5B 98 A8 2B 30 74 F2 C7 40 59 9B D9 98 8E 59 9C)
XXX    readStreamKey:  2786.75 MB/s (  0.367 s, buffer    8M (=8388608), keySize  1024.00 MB, md5=null)
XXX    readStreamKey:   736.60 MB/s (  1.390 s, buffer    8M (=8388608), keySize  1024.00 MB, md5=5B 98 A8 2B 30 74 F2 C7 40 59 9B D9 98 8E 59 9C)
XXX    readStreamKey:  2810.36 MB/s (  0.364 s, buffer  32M (=33554432), keySize  1024.00 MB, md5=null)
XXX    readStreamKey:   699.14 MB/s (  1.465 s, buffer  32M (=33554432), keySize  1024.00 MB, md5=5B 98 A8 2B 30 74 F2 C7 40 59 9B D9 98 8E 59 9C)

stub.withDeadlineAfter(timeout, TimeUnit.SECONDS)
.streamBlock(request, streamObserver);
streamObserver.setStreamingDatanode(dn);
return;

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

do we do that only for one node?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yes, it iterates over the list of available datanodes, and uses the first one that works. If the first one fails to connect, it will try to use the next one etc.

@chungen0126
Copy link
Contributor Author

Thanks @sodonnel for the update and @szetszwo for the review. After reviewing the discussion above , have we considered implementing gRPC flow control here? I think that could effectively free up the blocked threads.

Please refer to:
https://grpc.io/docs/guides/flow-control/

Additionally, since we are optimizing the read path by switching from a unary request to streaming, the size of the TCP socket buffer might impact performance. I think we should expose this configuration to allow users to tune it.

@szetszwo
Copy link
Contributor

... have we considered implementing gRPC flow control here? ...

@chungen0126 , Let's merge the current code first and then work on the further improvements. It is already a lot of code.

@szetszwo
Copy link
Contributor

szetszwo commented Nov 21, 2025

@chungen0126 , @sodonnel , we should plan how to merge the code. How about we merge it in three parts in the following order?

  1. @chungen0126 's changes up to fc95079
  2. @sodonnel 's changes up to 726f8e4
  3. My changes up to a45fa13 in https://github.com/szetszwo/ozone/commits/HDDS-10338b/

If you agree, please submit a new pull request against the current master. This PR is very long and hard to follow. (We may also consider merging to a dev branch first. However, since it is configurable and the default is disabled, we probably don't need a dev branch.)

We may also create subtasks under HDDS-10338.

@chungen0126
Copy link
Contributor Author

@chungen0126 , Let's merge the current code first and then work on the further improvements. It is already a lot of code.

Sure, I agree.

@chungen0126 , @sodonnel , we should plan how to merge the code. How about we merge it in three parts in the following order?

I have no idea how to merge this pr. If it was an acceptable way, let's do it.

@szetszwo
Copy link
Contributor

szetszwo commented Nov 21, 2025

  1. @chungen0126 , filed HDDS-13973 "The ground work to support stream block" for your changes.
  2. @sodonnel , filed HDDS-13974 "Use the same RPC steam for reading the same block".
  3. For myself, filed HDDS-13975 Limit the number of responses in stream read block

Please feel free to update the Summary and the Description of the JRIAs.

@sodonnel
Copy link
Contributor

I am happy for @szetszwo to simply pushed his latest changes onto this branch and just commit this one as a single PR. Its just more work and more confusing for future people to see the change split into 3, and if we want to cherry pick the change to other places for testing etc, its harder with 3 commits than 1.

@swamirishi
Copy link
Contributor

I am not able to take out time to do a complete review on this PR. I don't wanna block the PR please go ahead and merge the changes if other reviewers are good with the changes. I will review the changes offline and I will bring up bugs if at all I find any. Apologies for the delayed response.

@szetszwo
Copy link
Contributor

... since you have requested changes, could you review the latest changes?

@devabhishekpal , could you follow up on this?

@chungen0126
Copy link
Contributor Author

@sodonnel @szetszwo Here's a commit that implements client-side gRPC flow control: chungen0126@73de01a

This effectively prevents blocking the gRPC event loop thread. Since it's just a few lines of code, could you please take a look? If you agree with the change, I think it would be better to include it in this PR.

@sodonnel
Copy link
Contributor

@chungen0126 the goal is to get to the point where we commit https://github.com/szetszwo/ozone/commits/HDDS-10338b/ which @szetszwo created based on the work you and I did previously. Ideally we should just close this PR and open a new one with the change above and work to get it committed. It has changed the approach to implement the flow control by asking for the next chunks as it needs them. In some initial testing it appears to perform better than the latest commit on this PR, which also implemented flow control at the server side.

…orage/StreamBlockInputStream.java

Co-authored-by: Chung En Lee <[email protected]>
@szetszwo
Copy link
Contributor

szetszwo commented Dec 1, 2025

With some change in HDDS-14055 (HDDS-14035, will submit a PR soon), the performance can be further improved to 3300 MB/s for larger buffer sizes. It may be hitting the SSD read speed limit in my machine. (It is reading from OS cache, direct file read can be >10 GB/s.)

---------------------------------------------------------
key8 with 256M (=268435456) bytes and 16k (=16384) bytesPerChecksum
 createStreamKey:   145.00 MB/s (  1.765 s, buffer   8MB (=8388608), keySize   256.00 MB, md5=9E 4E A5 A0 51 07 0A 17 5E E2 B2 07 09 60 95 6E)
   readStreamKey:  3273.01 MB/s (  0.078 s, buffer  32M (=33554432), keySize   256.00 MB, md5=null)
   readStreamKey:   743.80 MB/s (  0.344 s, buffer  32M (=33554432), keySize   256.00 MB, md5=9E 4E A5 A0 51 07 0A 17 5E E2 B2 07 09 60 95 6E)
   readStreamKey:  3300.60 MB/s (  0.078 s, buffer    8M (=8388608), keySize   256.00 MB, md5=null)
   readStreamKey:   776.45 MB/s (  0.330 s, buffer    8M (=8388608), keySize   256.00 MB, md5=9E 4E A5 A0 51 07 0A 17 5E E2 B2 07 09 60 95 6E)
   readStreamKey:  2282.57 MB/s (  0.112 s, buffer    1M (=1048576), keySize   256.00 MB, md5=null)
   readStreamKey:   757.36 MB/s (  0.338 s, buffer    1M (=1048576), keySize   256.00 MB, md5=9E 4E A5 A0 51 07 0A 17 5E E2 B2 07 09 60 95 6E)
   readStreamKey:  2259.09 MB/s (  0.113 s, buffer       4k (=4096), keySize   256.00 MB, md5=null)
   readStreamKey:   751.97 MB/s (  0.340 s, buffer       4k (=4096), keySize   256.00 MB, md5=9E 4E A5 A0 51 07 0A 17 5E E2 B2 07 09 60 95 6E)

All JIRAs in #6613 (comment) have been merged. Closing this ...

@szetszwo szetszwo closed this Dec 1, 2025
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Projects

None yet

Development

Successfully merging this pull request may close these issues.