Re: Performance issue associated with managed RocksDB memory

2020-09-15 Thread Yu Li
Thanks for the follow up Juha, I've just assigned FLINK-19238 to you. Let's further track this on JIRA. Best Regards, Yu On Tue, 15 Sep 2020 at 15:04, Juha Mynttinen wrote: > Hey > > I created this one https://issues.apache.org/jira/browse/FLINK-19238. > > Regards, > Juha > ---

Re: Disable WAL in RocksDB recovery

2020-09-18 Thread Yu Li
Thanks for bringing this up Juha, and good catch. We actually are disabling WAL for routine writes by default when using RocksDB and never encountered segment fault issues. However, from history in FLINK-8922, segment fault issue occurs during restore if WAL is disabled, so I guess the root cause

Re: [ANNOUNCE] Apache Flink 1.11.2 released

2020-09-20 Thread Yu Li
Thanks Zhu Zhu for being our release manager and everyone else who made the release possible! Best Regards, Yu On Thu, 17 Sep 2020 at 13:29, Zhu Zhu wrote: > The Apache Flink community is very happy to announce the release of Apache > Flink 1.11.2, which is the second bugfix release for the Ap

Re: Disable WAL in RocksDB recovery

2020-09-21 Thread Yu Li
> Regards, > Juha > ---------- > *From:* Yu Li > *Sent:* Friday, September 18, 2020 3:58 PM > *To:* Juha Mynttinen > *Cc:* user@flink.apache.org > *Subject:* Re: Disable WAL in RocksDB recovery > > Thanks for bringing this up Juha, and good catch. > > We actually are

Re: flink checkpoint timeout

2020-10-05 Thread Yu Li
I'm not 100% sure but from the given information this might be related to FLINK-14498 [1] and partially relieved by FLINK-16645 [2]. @Omkar Could you try the 1.11.0 release out and see whether the issue disappeared? @zhijiang @yingjie could you also take a look here? Thanks. Best Regards, Yu [

Re: The question about the FLIP-45

2020-03-23 Thread Yu Li
Hi LakeShen, Sorry for the late response. For the first question, literally, the stop command should be used if one means to stop the job instead of canceling it. For the second one, since FLIP-45 is still under discussion [1] [2] (although a little bit stalled due to priority), we still don't s

Re: Streaming Job eventually begins failing during checkpointing

2020-04-27 Thread Yu Li
Sorry, just noticed this thread... @Stephan I cannot remember the discussion but I think it's an interesting topic, will find some time to consider it (unregister states). @Eleanore Glad to know that Beam community has fixed it and thanks for the reference. Best Regards, Yu On Sun, 26 Apr 2020

[ANNOUNCE] Apache Flink 1.10.1 released

2020-05-13 Thread Yu Li
The Apache Flink community is very happy to announce the release of Apache Flink 1.10.1, which is the first bugfix release for the Apache Flink 1.10 series. Apache Flink® is an open-source stream processing framework for distributed, high-performing, always-available, and accurate data streaming a

Re: [DISCUSS] Upgrade HBase connector to 2.2.x

2020-06-19 Thread Yu Li
+1 on upgrading the HBase version of the connector, and 1.4.3 is indeed an old version. OTOH, AFAIK there're still quite some 1.x HBase clusters in production. We could also see that the HBase community is still maintaining 1.x release lines (with "stable-1 release" point to 1.4.13) [1] Please al

Re: [DISCUSS] Upgrade HBase connector to 2.2.x

2020-06-19 Thread Yu Li
1.x usages in production. Best Regards, Yu [1] http://mail-archives.apache.org/mod_mbox/hbase-dev/202003.mbox/%3c30180be2-bd93-d414-a158-16c9c8d01...@apache.org%3E On Fri, 19 Jun 2020 at 15:54, Yu Li wrote: > +1 on upgrading the HBase version of the connector, and 1.4.3 is indeed an

Re: Performance issue associated with managed RocksDB memory

2020-06-25 Thread Yu Li
Thanks for the ping Andrey. Hi Juha, Thanks for reporting the issue. I'd like to check the below things before further digging into it: 1. Could you let us know your configurations (especially memory related ones) when running the tests? 2. Did you watch the memory consumption before / after tu

Re: Performance issue associated with managed RocksDB memory

2020-06-26 Thread Yu Li
To clarify, that my questions were all against the very original issue instead of the WordCount job. The timers come from the window operator you mentioned as the source of the original issue: === bq. If I create a Flink job that has a single "heavy" operator

Re: [ANNOUNCE] Apache Flink 1.10.3 released

2021-01-28 Thread Yu Li
Thanks Xintong for being our release manager and everyone else who made the release possible! Best Regards, Yu On Fri, 29 Jan 2021 at 15:05, Xintong Song wrote: > The Apache Flink community is very happy to announce the release of Apache > Flink 1.10.3, which is the third bugfix release for th

Re: Re: [ANNOUNCE] Hequn becomes a Flink committer

2019-08-07 Thread Yu Li
Congratulations Hequn! Well deserved! Best Regards, Yu On Thu, 8 Aug 2019 at 03:53, Haibo Sun wrote: > Congratulations! > > Best, > Haibo > > At 2019-08-08 02:08:21, "Yun Tang" wrote: > >Congratulations Hequn. > > > >Best > >Yun Tang > > > >From: Rong Rong > >

Re: Capping RocksDb memory usage

2019-08-09 Thread Yu Li
Hi Cam, Which flink version are you using? Actually I don't think any existing flink release could take usage of the write buffer manager natively through some configuration magic, but requires some "developing" efforts, such as manually building flink with a higher version rocksdb to have the JN

Re: Capping RocksDb memory usage

2019-08-09 Thread Yu Li
e_buffer_manager after we read that Jira. > One quick question, I noticed that our disk usage (SSD) for RocksDb is > always stay around %2 (or 2.2 GB), which is not the case before we enable > RocksDb state backend. So wondering what stoping it? > > Thanks, > Cam > > > >

Re: Making broadcast state queryable?

2019-08-13 Thread Yu Li
Hi Oytun, Sorry but TBH such support will probably not be added in the foreseeable future due to lack of committer bandwidth (not only support queryable broadcast state but all about QueryableState module) as pointed out in other threads [1] [2]. However, I think you could open a JIRA for this so

Re: Making broadcast state queryable?

2019-08-14 Thread Yu Li
to that. > > Meanwhile, for the sake of opening our state to outside, we will put a > stupid simple operator in between to keep a *duplicate* of the state... > > Thanks again! > > > > > > --- > Oytun Tez > > *M O T A W O R D* > The World's Fastest Human Tr

Re: [ANNOUNCE] Apache Flink 1.9.0 released

2019-08-22 Thread Yu Li
Thanks for the update Gordon, and congratulations! Great thanks to all for making this release possible, especially to our release managers! Best Regards, Yu On Thu, 22 Aug 2019 at 14:55, Xintong Song wrote: > Congratulations! > Thanks Gordon and Kurt for being the release managers, and thank

Re: [SURVEY] Is the default restart delay of 0s causing problems?

2019-09-01 Thread Yu Li
-1 on increasing the default delay to none zero, with below reasons: a) I could see some concerns about setting the delay to zero in the very original JIRA (FLINK-2993 ) but later on in FLINK-9158

Re: [ANNOUNCE] Kinesis connector becomes part of Flink releases

2019-09-01 Thread Yu Li
Great to know, thanks for the efforts Bowen! And I believe it worth a release note in the original JIRA, wdyt? Thanks. Best Regards, Yu On Sat, 31 Aug 2019 at 11:01, Bowen Li wrote: > Hi all, > > I'm glad to announce that, as #9494 > was merged today

Re: Making broadcast state queryable?

2019-09-15 Thread Yu Li
with small teams. > > --- > Oytun Tez > > *M O T A W O R D* > The World's Fastest Human Translation Platform. > oy...@motaword.com — www.motaword.com > > > On Wed, Aug 14, 2019 at 3:13 AM Yu Li wrote: > >> Good to know your thoughts and the coming talk in F

Re: [PROPOSAL] Contribute Stateful Functions to Apache Flink

2019-10-12 Thread Yu Li
Hi Stephan, Big +1 for adding stateful functions to Flink. I believe a lot of user would be interested to try this out and I could imagine how this could contribute to reduce the TCO for business requiring both streaming processing and stateful functions. And my 2 cents is to put it into flink co

Re: Per Operator State Monitoring

2019-11-26 Thread Yu Li
Hi Aaron, I don't think we have such fine grained metrics on per operation state size, but from your description that "YARN kills containers who are exceeding their memory limits", I think the root cause is not the state size but related to the memory consumption of the state backend. My guess is

Re: Status of FLINK-12692 (Support disk spilling in HeapKeyedStateBackend)

2020-01-30 Thread Yu Li
once the package is ready. Best Regards, Yu [1] https://flink-packages.org/ On Thu, 30 Jan 2020 at 08:47, Ken Krugler wrote: > Hi Yu Li, > > It looks like this stalled out a bit, from May of last year, and won’t > make it into 1.10. > > I’m wondering if there’s a version in

Re: Status of FLINK-12692 (Support disk spilling in HeapKeyedStateBackend)

2020-01-30 Thread Yu Li
/jira/browse/FLINK-12697 [2] https://github.com/apache/flink/pull/9501 On Fri, 31 Jan 2020 at 15:27, Yu Li wrote: > Hi Ken, > > Thanks for watching this feature. > > Unfortunately yes this didn't make into 1.10, and we will try our best to > complete the upstreaming work

[ANNOUNCE] Apache Flink 1.10.0 released

2020-02-12 Thread Yu Li
The Apache Flink community is very happy to announce the release of Apache Flink 1.10.0, which is the latest major release. Apache Flink® is an open-source stream processing framework for distributed, high-performing, always-available, and accurate data streaming applications. The release is avai

Re: [ANNOUNCE] Apache Flink 1.10.0 released

2020-02-12 Thread Yu Li
gt; <https://issues.apache.org/jira/browse/FLINK-13884> J > > > > > > *From:* Yu Li > *Sent:* Wednesday, February 12, 2020 8:31 AM > *To:* dev ; user ; > annou...@apache.org > *Subject:* [ANNOUNCE] Apache Flink 1.10.0 released > > > > The Apache Flink co

Re: [ANNOUNCE] Apache Flink 1.10.0 released

2020-02-12 Thread Yu Li
Hi Kristoff, Thanks for the question. About Java 11 support, please allow me to quote from our release note [1]: Lastly, note that the connectors for Cassandra, Hive, HBase, and Kafka 0.8–0.11 have not been tested with Java 11 because the respective projects did not provide Java 11 support at th

Re: [ANNOUNCE] Jingsong Lee becomes a Flink committer

2020-02-23 Thread Yu Li
Congratulations Jingsong! Well deserved. Best Regards, Yu On Mon, 24 Feb 2020 at 14:10, Congxian Qiu wrote: > Congratulations Jingsong! > > Best, > Congxian > > > jincheng sun 于2020年2月24日周一 下午1:38写道: > >> Congratulations Jingsong! >> >> Best, >> Jincheng >> >> >> Zhu Zhu 于2020年2月24日周一 上午11:5

Re: FsStateBackend vs RocksDBStateBackend

2020-02-23 Thread Yu Li
Yes FsStateBackend would be the best fit for state access performance in this case. Just a reminder that FsStateBackend will upload the full dataset to DFS during checkpointing, so please watch the network bandwidth usage and make sure it won't become a new bottleneck. Best Regards, Yu On Fri, 2

Re: [DISCUSS] Drop Savepoint Compatibility with Flink 1.2

2020-02-23 Thread Yu Li
+1 for dropping savepoint compatibility with Flink 1.2. Best Regards, Yu On Sat, 22 Feb 2020 at 22:05, Ufuk Celebi wrote: > Hey Stephan, > > +1. > > Reading over the linked ticket and your description here, I think it makes > a lot of sense to go ahead with this. Since it's possible to upgrade

Re: [ANNOUNCE] Apache Flink 1.13.2 released

2021-08-09 Thread Yu Li
Thanks Yun Tang for being our release manager and everyone else who made the release possible! Best Regards, Yu On Fri, 6 Aug 2021 at 13:52, Yun Tang wrote: > > The Apache Flink community is very happy to announce the release of Apache > Flink 1.13.2, which is the second bugfix release for the

Re: [VOTE] Release 1.8.0, release candidate #3

2019-03-20 Thread Yu Li
-1, observed stably failure on streaming bucketing end-to-end test case in two different environments (Linux/MacOS) when running with both shaded hadoop-2.8.3 jar file

Re: [VOTE] Release 1.8.0, release candidate #3

2019-03-21 Thread Yu Li
incheng sun wrote: > Thanks for the quick fix, Yu. the PR of FLINK-11972 > <https://issues.apache.org/jira/browse/FLINK-11972> has been merged. > > Cheers, > Jincheng > > Yu Li 于2019年3月21日周四 上午7:23写道: > >> -1, observed stably failure on streaming bucketing end-to

Re: [VOTE] Release 1.8.0, release candidate #3

2019-03-21 Thread Yu Li
mment? > > I will also cancel this RC because of various issues. > > Best, > Aljoscha > > On 21. Mar 2019, at 12:23, Yu Li wrote: > > Thanks @jincheng > > @Aljoscha I've just opened FLINK-11990 > <https://issues.apache.org/jira/browse/FLINK-11990> for

Re: RocksDB local snapshot sliently disappears and cause checkpoint to fail

2019-03-28 Thread Yu Li
Hi Paul, Regarding "mistakenly uses the default filesystem scheme, which is specified to hdfs in the new cluster in my case", could you further clarify the configuration property and value you're using? Do you mean you're using an HDFS directory to store the local snapshot data? Thanks. Best Rega

Re: RocksDB local snapshot sliently disappears and cause checkpoint to fail

2019-03-28 Thread Yu Li
he local snapshot still goes > to YARN local cache dirs. > > Hope that answers your question. > > Best, > Paul Lam > > 在 2019年3月28日,15:34,Yu Li 写道: > > Hi Paul, > > Regarding "mistakenly uses the default filesystem scheme, which is > specified to h

Re: End to End Performance Testing

2019-04-03 Thread Yu Li
A short answer is no, the test platform is not included in the blink branch. FWIW (to make it clear, I'm not the decision maker so just some of my own opinions), the test platform is for production usage and includes simulation of online jobs and probably some confidential stuff, so I don't think

Re: [ANNOUNCE] Apache Flink 1.8.0 released

2019-04-12 Thread Yu Li
Thanks Aljoscha and all for making this happen, I believe 1.8.0 will be a great and successful release. Best Regards, Yu On Fri, 12 Apr 2019 at 21:23, Patrick Lucas wrote: > The Docker images for 1.8.0 are now available. > > Note that like Flink itself, starting with Flink 1.8 we are no longer

Re: [Discuss] Semantics of event time for state TTL

2019-04-15 Thread Yu Li
Thanks for initiating the discussion and wrap-up the conclusion Andrey, and thanks all for participating. Just to confirm, that for the out-of-order case, the conclusion is to update the data and timestamp with the currently-being-processed record w/o checking whether it's an old data, right? In t

[DISCUSS] Proposal to support disk spilling in HeapKeyedStateBackend

2019-05-24 Thread Yu Li
Hi All, As mentioned in our speak[1] given in FlinkForwardChina2018, we have improved HeapKeyedStateBackend to support disk spilling and put it in production here in Alibaba for last year's Singles' Day. Now we're ready to upstream our work and the design doc is up for review[2]. Please let us kno

Re: [DISCUSS] Deprecate previous Python APIs

2019-06-14 Thread Yu Li
+1 on removing plus an explicit NOTE thread, to prevent any neglection due to the current title (deprecation). Best Regards, Yu On Fri, 14 Jun 2019 at 18:09, Stephan Ewen wrote: > Okay, so we seem to have consensus for at least deprecating them, with a > suggestion to even directly remove them

Re: Queryable state and TTL

2019-07-03 Thread Yu Li
Thanks for the ping Andrey. For me the general answer is yes, but TBH it will probably not be added in the foreseeable future due to lack of committer bandwidth (not only QueryableState with TTL but all about QueryableState module) as per Aljoscha pointed out in another thread [1]. Although we co

Re: [ANNOUNCE] Rong Rong becomes a Flink committer

2019-07-11 Thread Yu Li
Congratulations Rong! Best Regards, Yu On Thu, 11 Jul 2019 at 22:54, zhijiang wrote: > Congratulations Rong! > > Best, > Zhijiang > > -- > From:Kurt Young > Send Time:2019年7月11日(星期四) 22:54 > To:Kostas Kloudas > Cc:Jark Wu ; Fabi

Re: Bandwidth throttling of checkpoints uploading to s3

2019-07-12 Thread Yu Li
Hi Pavel, Currently there's no such throttling functionality in Flink and I think it's a valid requirement. But before opening a JIRA for this, please allow me to ask for more details to better understand your scenario: 1. What kind of state backend are you using? Since you observe high load to ne

Re: Bandwidth throttling of checkpoints uploading to s3

2019-07-12 Thread Yu Li
e problem. >4. I think it is about 50 Mbit/s. > > Thanks. > > 12.07.2019, 17:27, "Yu Li" : > > Hi Pavel, > > Currently there's no such throttling functionality in Flink and I think > it's a valid requirement. But before opening a JIRA for this, pl

Re: Memory constrains running Flink on Kubernetes

2019-07-29 Thread Yu Li
For the memory usage of RocksDB, there's already some discussion in FLINK-7289 and a good suggestion

Re: Graceful Task Manager Termination and Replacement

2019-07-29 Thread Yu Li
Belated but FWIW, besides the region failover and best-efforts failover efforts, I believe stop with checkpoint as proposed in FLINK-12619 and FLIP-45 could also help here, FYI. W.r.t k8s, there're also some offline discussion about supporting local recovery with persistent volume even when task a

[ANNOUNCE] Call for Presentations for ApacheCon Asia 2022 streaming track

2022-05-18 Thread Yu Li
Hi everyone, ApacheCon Asia [1] will feature the Streaming track for the second year. Please don't hesitate to submit your proposal if there is an interesting project or Flink experience you would like to share with us! The conference will be online (virtual) and the talks will be pre-recorded. T

[ANNOUNCE] Flink Table Store Joins Apache Incubator as Apache Paimon(incubating)

2023-03-27 Thread Yu Li
Dear Flinkers, As you may have noticed, we are pleased to announce that Flink Table Store has joined the Apache Incubator as a separate project called Apache Paimon(incubating) [1] [2] [3]. The new project still aims at building a streaming data lake platform for high-speed data ingestion, change

Re: [ANNOUNCE] Apache Flink 1.19.0 released

2024-03-18 Thread Yu Li
Congrats and thanks all for the efforts! Best Regards, Yu On Tue, 19 Mar 2024 at 11:51, gongzhongqiang wrote: > > Congrats! Thanks to everyone involved! > > Best, > Zhongqiang Gong > > Lincoln Lee 于2024年3月18日周一 16:27写道: >> >> The Apache Flink community is very happy to announce the release of A

Re: [ANNOUNCE] Apache Paimon is graduated to Top Level Project

2024-03-28 Thread Yu Li
CC the Flink user and dev mailing list. Paimon originated within the Flink community, initially known as Flink Table Store, and all our incubating mentors are members of the Flink Project Management Committee. I am confident that the bonds of enduring friendship and close collaboration will contin