Hi all,
Thanks very much for all of the feedbacks. It seems that we have reached a
consensus. I will start a vote soon.
Best,
Yingjie
Yun Gao 于2022年1月5日周三 16:08写道:
> Very thanks @Yingjie for completing the experiments!
>
> Also +1 for changing the default config values. From the ex
my limited knowledge, I guess that increasing
the total TaskManager size and network memory size is important for
performance, because more memory (managed and network) can make operators
and shuffle faster.
Best,
Yingjie
Yingjie Cao 于2021年12月15日周三 12:19写道:
> Hi Till,
>
> Thanks fo
Hi Till,
Thanks for the suggestion. I think it makes a lot of sense to also extend
the documentation for the sort shuffle to include a tuning guide.
Best,
Yingjie
Till Rohrmann 于2021年12月14日周二 18:57写道:
> As part of this FLIP, does it make sense to also extend the documentation
> for th
st if a rough table could be provided.
I think this is a good suggestion, we can provide those suggestions in the
document.
Best,
Yingjie
Jingsong Li 于2021年12月14日周二 14:39写道:
> Hi Yingjie,
>
> +1 for this FLIP. I'm pretty sure this will greatly improve the ease
> of batch jobs.
>
mory.framework.off-heap.batch-shuffle.size, you can increase
these values for large-scale batch jobs.
BTW, I am still running TPCDS tests these days and I can share these
results soon.
Best,
Yingjie
刘建刚 于2021年12月10日周五 18:30写道:
> Glad to see the suggestion. In our test, we found that small jobs with t
the second issue, I will try to find the cause and solve it
in 1.15.
I am open for your suggestion, but I still need some more tests and
analysis to guarantee that it works well.
Best,
Yingjie
Yun Gao 于2021年12月10日周五 17:19写道:
> Hi Yingjie,
>
> Very thanks for drafting the FLIP and initia
Hi dev & users:
I have created a FLIP [1] for it, feedbacks are highly appreciated.
Best,
Yingjie
[1]
https://cwiki.apache.org/confluence/display/FLINK/FLIP-199%3A+Change+some+default+config+values+of+blocking+shuffle+for+better+usability
Yingjie Cao 于2021年12月3日周五 17:02写道:
> Hi dev
. It can be a very short one, though.
Yes, you are right. I will prepare a simple FLIP soon.
Best,
Yingjie
Till Rohrmann 于2021年12月3日周五 18:39写道:
> Thanks for starting this discussion Yingjie,
>
> How will our tests be affected by these changes? Will Flink require more
> res
nd both the performance and stability improved a lot. These
changes can help to improve the out-of-box experience of blocking shuffle.
What do you think about these changes? Is there any concern? If there are
no objections, I will make these changes soon.
Best,
Yingjie
Hi Jiangang,
Great to hear that, welcome to work together to make the project better.
Best,
Yingjie
刘建刚 于2021年12月1日周三 下午3:27写道:
> Good work for flink's batch processing!
> Remote shuffle service can resolve the container lost problem and reduce
> the running time for batch jobs
project, please refer to
[1].
Before going open source, the project has been used widely in production
and it behaves well on both stability and performance. We hope you enjoy
it. Collaborations and feedbacks are highly appreciated.
Best,
Yingjie on behalf of all contributors
[1] https://gi
.
[2]
https://cwiki.apache.org/confluence/display/FLINK/FLIP-184%3A+Refine+ShuffleMaster+lifecycle+management+for+pluggable+shuffle+service+framework
.
Best,
Yingjie
Maybe you can try to
increase taskmanager.network.retries,
taskmanager.network.netty.server.backlog and
taskmanager.network.netty.sendReceiveBufferSize. These options are useful
for our jobs.
yidan zhao 于2021年6月16日周三 下午7:10写道:
> Hi, yingjie.
> If the network is not stable, which
/ops/batch/blocking_shuffle/
[2]
https://ci.apache.org/projects/flink/flink-docs-release-1.13/docs/deployment/config/
[3] https://issues.apache.org/jira/browse/FLINK-22643
Best,
Yingjie
yidan zhao 于2021年6月16日周三 下午3:36写道:
> Attachment is the exception stack from flink's web-ui. Does anyone
> have
Hi Till,
Thanks for the suggestion. The blog post is already on the way.
Best,
Yingjie
Till Rohrmann 于2021年6月8日周二 下午5:30写道:
> Thanks for the update Yingjie. Would it make sense to write a short blog
> post about this feature including some performance improvement numbers? I
> think t
nt[2] which
contains more implementation details. FYI.
[1]
https://cwiki.apache.org/confluence/display/FLINK/FLIP-148%3A+Introduce+Sort-Based+Blocking+Shuffle+to+Flink
[2]
https://docs.google.com/document/d/1j12TkSqgf6dg3J48udA2MFrDOQccW24tzjn5pJlTQaQ/edit?usp=sharing
Best,
Yingjie
Yingjie Ca
Hi Haihang,
After scanning the user mailing list, I found some users have reported
checkpoint timeout when using unaligned checkpoint, can you share which
checkpoint mode do you use? (The information can be found in log or the
checkpoint -> configuration tab in webui)
Best,
Yingjie
Yingjie
share some more information, for example, the stack of the task
which can not finish the checkpoint?
Best,
Yingjie
Haihang Jing 于2021年3月25日周四 上午10:58写道:
> Hi,Congxian ,thanks for your replay.
> job run on Flink1.9 (checkpoint interval 3min)
> <
> http://apache-flink-user-mailing
Hi Felipe,
That depends on what do you mean by 'bandwidth'. If you mean the capability
of the network stack, the answer would be no.
Here is a post about Flink network stack which may help:
https://flink.apache.org/2019/06/05/flink-network-stack.html.
Thanks,
Yingjie
Felipe Gutierrez
Piotr is right, that depend on the data size you are reading and the memory
pressure. Those memory occupied by mmapped region can be recycled and used
by other processes if memory pressure is high, that is, other process or
service on the same node won't be affected because the OS will recycle the
The new BlockingSubpartition implementation in 1.9 uses mmap for data reading
by default which means it steals memory from OS. The mmaped region memory is
managed by JVM, so there should be no OutOfMemory problem reported by JVM
and the OS memory is also not exhausted, so there should be no kernal
21 matches
Mail list logo