Hi Lorenzo,
> As I mentioned above, in the end I think hardcoding in Flink support for
the `cpulimit` doesn't seem appropriate and at the same time it is not
actually necessary.
I agree.
What's suggested is similar to the way where Flink currently allow users to
limit resource usage on s3a files
I meant to address Piotr on the above, apologies!
Best regards
Keith Lee
On Tue, Jun 18, 2024 at 8:06 AM Keith Lee
wrote:
> Hi Lorenzo,
>
> > As I mentioned above, in the end I think hardcoding in Flink support for
> the `cpulimit` doesn't seem appropriate and at the same time it is not
> actu
Hi all!
>> Do you know if there is a way to limit the CPU utilisation of s5cmd? I
see
>> worker and concurrency configuration but these do not map directly to cap
>> in CPU usage. The experience for feature user in this case will be one of
>> trial and error.
>
> Those are good points. As a matter
Perfectly agree with all your considerations.
Wee said.
Thank you!
On May 16, 2024 at 10:53 +0200, Piotr Nowojski , wrote:
> Hi Lorenzo,
>
> > • concerns about memory and CPU used out of Flink's control
>
> Please note that using AWS SDKv2 would have the same concerns. In both
> cases Flink can co
Hi Lorenzo,
> • concerns about memory and CPU used out of Flink's control
Please note that using AWS SDKv2 would have the same concerns. In both
cases Flink can control only to a certain extent what either the SDKv2
library does under the hood or the s5cmd process, via configuration
parameters. S
Hello Piotr and thanks for this proposal!
The idea sounds smart and very well grounded thank you!
Also here, as others, I have some doubts about invoking an external binary
(namely s5cmd):
• concerns about memory and CPU used out of Flink's control
• deployment concerns from the usability perspe
>
> Note that for both recovery and checkpoints, there are no retring
> mechanisms. If any part of downloading or
> uploading fails, the job fails over, so actually using such interface
> extension would be out of scope of this FLIP. In
> that case, maybe if this could be extended in the future wi
Hi!
Thanks for your suggestions!
> I'd prefer a unified one interface
I have updated the FLIP to take that into account. In this case, I would
also propose to completely drop `DuplicatingFileSystem` in favour of a
basically renamed version of it `PathsCopyingFileSystem`.
`DuplicatingFileSystem`
Hi Piotr.
Thanks for your proposal.
I have some comments, PTAL:
1. +1 about unifying the interface with DuplicatingFileSystem.
IIUC, DuplicatingFileSystem also covers the logic from/to both local and
remote paths.
The implementations could define their own logic about how to fast
copy/duplicate fi
Hi Piotr,
Thank you for the proposal. Looks great.
Along similar line of Aleks' question on memory usage.
The proposal mentions that s5cmd utilises 100% of CPU similar to Flink
1.18. However, this will be a native process outside of the JVM. Are there
risk of large/long state download starving t
Hi Piotr
+1 for the proposal, it seems to have a lot of gains.
Best Regards
Ahmed Hamdy
On Mon, 6 May 2024 at 12:06, Zakelly Lan wrote:
> Hi Piotrek,
>
> Thanks for your answers!
>
> Good question. The intention and use case behind `DuplicatingFileSystem` is
> > different. It marks if `FileSys
Hi Piotrek,
Thanks for your answers!
Good question. The intention and use case behind `DuplicatingFileSystem` is
> different. It marks if `FileSystem` can quickly copy/duplicate files
> in the remote `FileSystem`. For example an equivalent of a hard link or
> bumping a reference count in the remo
Hi All!
Thanks for your comments.
Muhammet and Hong, about the config options.
> Could you please also add the configuration property for this? An example
showing how users would set this parameter would be helpful.
> 1/ Configure the implementation of PathsCopyingFileSystem used
> 2/ Configure
Hi Piotr,
Thanks for the proposal. It's meaningful to speed up the state download. I
get into some questions:
1. What is the semantic of `canCopyPath`? Should it be associated with a
specific destination path? e.g. It can be copied to local, but not to the
remote FS.
2. Is the existing interface
Hi Piotr,
Thanks for the proposal.
How adding a s5cmd will affect memory footprint? Since this is a native
binary, memory consumption will not be controlled by JVM or Flink.
Thanks,
Aleksandr
On Thu, 2 May 2024 at 11:12, Hong Liang wrote:
> Hi Piotr,
>
> Thanks for the FLIP! Nice to see work t
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
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
Hi,
Thanks for the proposal, I think improving download speed in this way is a
great idea. Hope we can have similar improvements for other clouds as well.
Best,
Stefan
> On 30. Apr 2024, at 15:15, Piotr Nowojski wrote:
>
> Hi all!
>
> I would like to put under discussion:
>
> FLIP-444: Nat
Hi Piotr,
+1 for the proposal, the recovery time improvements are significant IMO
Thanks for pushing this
Regards,
Roman
On Tue, Apr 30, 2024 at 3:15 PM Piotr Nowojski wrote:
> Hi all!
>
> I would like to put under discussion:
>
> FLIP-444: Native file copy support
> https://cwiki.apache.org
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 u
20 matches
Mail list logo