Hi Piotr, Thanks for the FLIP! Nice to see work to improve the filesystem performance. +1 to future work to improve the upload speed as well. This would be useful for jobs with large state and high Async checkpointing times.
Some thoughts on the configuration, it might be good for us to introduce 2x points of configurability for future proofing: 1/ Configure the implementation of PathsCopyingFileSystem used, maybe by config, or by ServiceResources (this would allow us to use this for alternative clouds/Implement S3 SDKv2 support if we want this in the future). Also this could be used as a feature flag to determine if we should be using this new native file copy support. 2/ Configure the location of the s5cmd binary (version control etc.), as you have mentioned in the FLIP. Regards, Hong On Thu, May 2, 2024 at 9:40 AM Muhammet Orazov <mor+fl...@morazow.com.invalid> wrote: > Hey Piotr, > > Thanks for the proposal! It would be great improvement! > > Some questions from my side: > > > In order to configure s5cmd Flink’s user would need > > to specify path to the s5cmd binary. > > Could you please also add the configuration property > for this? An example showing how users would set this > parameter would be helpful. > > Would this affect any filesystem connectors that use > FileSystem[1][2] dependencies? > > Best, > Muhammet > > [1]: > > https://nightlies.apache.org/flink/flink-docs-master/docs/deployment/filesystems/s3/ > [2]: > > https://nightlies.apache.org/flink/flink-docs-master/docs/connectors/datastream/filesystem/ > > On 2024-04-30 13:15, Piotr Nowojski wrote: > > Hi all! > > > > I would like to put under discussion: > > > > FLIP-444: Native file copy support > > https://cwiki.apache.org/confluence/x/rAn9EQ > > > > This proposal aims to speed up Flink recovery times, by speeding up > > state > > download times. However in the future, the same mechanism could be also > > used to speed up state uploading (checkpointing/savepointing). > > > > I'm curious to hear your thoughts. > > > > Best, > > Piotrek >