[ 
https://issues.apache.org/jira/browse/FLINK-39811?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Efrat Levitan updated FLINK-39811:
----------------------------------
    Description: 
Unlike the upload path (recoverableWriter) which sets 
{{S3_MULTIPART_MIN_PART_SIZE}} of [5 
MiB|https://github.com/apache/flink/blob/master/flink-filesystems/flink-s3-fs-base/src/main/java/org/apache/flink/fs/s3/common/FlinkS3FileSystem.java#L93],
 we don't set any part size for s5cmd so downloads part size fall back to its 
default of 
[50MiB|https://github.com/peak/s5cmd/blob/54d6a8a955688f07e5acc40d61f9c42ceac6c33b/command/cp.go#L30].

As a result, the effective concurrency is capped to batchSize / 50MiB, 
underutilizing the worker pool for downloaded batchSizes below (50*concurrency 
MiB)

- s3 sdk source 
[ref|https://github.com/aws/aws-sdk-go/blob/070853e88d22854d2355c2543d0958a5f76ad407/service/s3/s3manager/download.go#L329-L338]
 N(=concurrency) channels are initialized but only (batchsize/partsize) of them 
are assigned a range to read.


  was:
Unlike the upload path (recoverableWriter) which sets 
{{S3_MULTIPART_MIN_PART_SIZE}} of [5 
MiB|https://github.com/apache/flink/blob/master/flink-filesystems/flink-s3-fs-base/src/main/java/org/apache/flink/fs/s3/common/FlinkS3FileSystem.java#L93],
 we don't set any part size for s5cmd so downloads part size fall back to its 
default of 
[50MiB|https://github.com/peak/s5cmd/blob/54d6a8a955688f07e5acc40d61f9c42ceac6c33b/command/cp.go#L30].

As a result, the effective concurrency is capped to batchSize / 50MiB, 
underutilizing the worker pool for downloaded batchSizes below (50*concurrency 
MiB)

- s3 sdk source 
[ref|https://github.com/aws/aws-sdk-go/blob/070853e88d22854d2355c2543d0958a5f76ad407/service/s3/s3manager/download.go#L329-L338]
 N(=concurrency) channels are initialized but only batchsize/partsize are 
assigned a range to read.



> S5cmd effective concurrency capped by static part size for small downloads
> --------------------------------------------------------------------------
>
>                 Key: FLINK-39811
>                 URL: https://issues.apache.org/jira/browse/FLINK-39811
>             Project: Flink
>          Issue Type: Bug
>          Components: FileSystems
>            Reporter: Efrat Levitan
>            Priority: Major
>
> Unlike the upload path (recoverableWriter) which sets 
> {{S3_MULTIPART_MIN_PART_SIZE}} of [5 
> MiB|https://github.com/apache/flink/blob/master/flink-filesystems/flink-s3-fs-base/src/main/java/org/apache/flink/fs/s3/common/FlinkS3FileSystem.java#L93],
>  we don't set any part size for s5cmd so downloads part size fall back to its 
> default of 
> [50MiB|https://github.com/peak/s5cmd/blob/54d6a8a955688f07e5acc40d61f9c42ceac6c33b/command/cp.go#L30].
> As a result, the effective concurrency is capped to batchSize / 50MiB, 
> underutilizing the worker pool for downloaded batchSizes below 
> (50*concurrency MiB)
> - s3 sdk source 
> [ref|https://github.com/aws/aws-sdk-go/blob/070853e88d22854d2355c2543d0958a5f76ad407/service/s3/s3manager/download.go#L329-L338]
>  N(=concurrency) channels are initialized but only (batchsize/partsize) of 
> them are assigned a range to read.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

Reply via email to