[RESULT][VOTE] FLIP-447: Upgrade FRocksDB from 6.20.3 to 8.10.0

2024-05-08 Thread yue ma
Hi everyone, Thanks for your review and the votes! I am happy to announce that FLIP-447: Upgrade FRocksDB from 6.20.3 to 8.10.0 [1]. has been accepted. The proposal has been accepted with 15 approving votes (9 binding) and there is no disapproval: - Zakelly Lan (binding) - Yanfei Lei (binding)

[VOTE] FLIP-447: Upgrade FRocksDB from 6.20.3 to 8.10.0

2024-05-05 Thread yue ma
Hi everyone, Thanks for all the feedback, I'd like to start a vote on the FLIP-447: Upgrade FRocksDB from 6.20.3 to 8.10.0 [1]. The discussion thread is here [2]. The vote will be open for at least 72 hours unless there is an objection or insufficient votes. [1]

Re: [DISCUSS] FLIP-447: Upgrade FRocksDB from 6.20.3 to 8.10.0

2024-04-26 Thread yue ma
Hi all, Thank you all for your feedback. If there are no extra comments, I will start voting in three days, thank you :) yue ma 于2024年4月22日周一 14:09写道: > Hi Flink devs, > > I would like to start a discussion on FLIP-447: Upgrade FRocksDB from > 6.20.3 to 8.10.0 > > > htt

Re: [DISCUSS] FLIP-447: Upgrade FRocksDB from 6.20.3 to 8.10.0

2024-04-24 Thread yue ma
kes > > > > > > > FRocksDB easier to maintain, +1 for upgrading. > > > > > > > > > > > > > > I read the FLIP and have a minor comment, it would be better > to add > > > > > > > some description about the environment/configuration of th

Re: [DISCUSS] FLIP-447: Upgrade FRocksDB from 6.20.3 to 8.10.0

2024-04-24 Thread yue ma
and the > new > > APIs are necessary for faster rescaling. > > We're already using a similar version internally. > > > > I reviewed the FLIP and it looks good to me (disclaimer: I took part in > > some steps of this effort). > > > > > > Regards, > &g

Re: [DISCUSS] FLIP-447: Upgrade FRocksDB from 6.20.3 to 8.10.0

2024-04-24 Thread yue ma
anks for writing the proposal and preparing the upgrade. > > > > > > FRocksDB definitely needs to be kept in sync with the upstream and the > > new > > > APIs are necessary for faster rescaling. > > > We're already using a similar version internally. > >

Re: [DISCUSS] FLIP-447: Upgrade FRocksDB from 6.20.3 to 8.10.0

2024-04-24 Thread yue ma
efinitely needs to be kept in sync with the upstream and the new > APIs are necessary for faster rescaling. > We're already using a similar version internally. > > I reviewed the FLIP and it looks good to me (disclaimer: I took part in > some steps of this effort). > > > Regards, >

[DISCUSS] FLIP-447: Upgrade FRocksDB from 6.20.3 to 8.10.0

2024-04-22 Thread yue ma
Hi Flink devs, I would like to start a discussion on FLIP-447: Upgrade FRocksDB from 6.20.3 to 8.10.0 https://cwiki.apache.org/confluence/display/FLINK/FLIP-447%3A+Upgrade+FRocksDB+from+6.20.3++to+8.10.0 This FLIP proposes upgrading the version of FRocksDB in the Flink Project from 6.20.3 to

Re: [ANNOUNCE] New Apache Flink Committer - Zakelly Lan

2024-04-14 Thread yue ma
Congratulations Zakelly! Best, Yue

Re: [VOTE] FLIP-433: State Access on DataStream API V2

2024-03-29 Thread yue ma
+1 (non-binding) weijie guo 于2024年3月20日周三 20:35写道: > Hi everyone, > > > Thanks for all the feedback about the FLIP-433: State Access on > DataStream API V2 [1]. The discussion thread is here [2]. > > > The vote will be open for at least 72 hours unless there is an > objection or insufficient

Re: [VOTE] FLIP-428: Fault Tolerance/Rescale Integration for Disaggregated State

2024-03-29 Thread yue ma
+1(non-binding) Jinzhong Li 于2024年3月27日周三 19:31写道: > Hi devs, > > > I'd like to start a vote on the FLIP-428: Fault Tolerance/Rescale > Integration for Disaggregated State [1]. The discussion thread is here [2]. > > > The vote will be open for at least 72 hours unless there is an objection or >

Re: [VOTE] FLIP-427: Disaggregated state Store

2024-03-29 Thread yue ma
+1(non-binding) Hangxiang Yu 于2024年3月27日周三 18:37写道: > Hi devs, > > Thanks all for your valuable feedback about FLIP-427: Disaggregated state > Store [1]. > I'd like to start a vote on it. The discussion thread is here [2]. > > The vote will be open for at least 72 hours unless there is an

Re: [VOTE] FLIP-426: Grouping Remote State Access

2024-03-29 Thread yue ma
+1 (non-binding) Jinzhong Li 于2024年3月27日周三 18:57写道: > Hi devs, > > I'd like to start a vote on the FLIP-426: Grouping Remote State Access [1]. > The discussion thread is here [2]. > > The vote will be open for at least 72 hours unless there is an objection or > insufficient votes. > > > [1]

Re: [VOTE] FLIP-423: Disaggregated State Storage and Management (Umbrella FLIP)

2024-03-29 Thread yue ma
+1 (non-binding) Best, Yue

Re: [VOTE] FLIP-425: Asynchronous Execution Model

2024-03-29 Thread yue ma
+1 (non-binding) Yanfei Lei 于2024年3月27日周三 18:28写道: > Hi everyone, > > Thanks for all the feedback about the FLIP-425: Asynchronous Execution > Model [1]. The discussion thread is here [2]. > > The vote will be open for at least 72 hours unless there is an > objection or insufficient votes. > >

Re: [DISCUSS] FLIP-428: Fault Tolerance/Rescale Integration for Disaggregated State

2024-03-22 Thread yue ma
pointing and > restoring. > > [1] > > https://github.com/facebook/rocksdb/blob/6ddfa5f06140c8d0726b561e16dc6894138bcfa0/db/db_filesnapshot.cc#L77 > [2] > > https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=293046855#FLIP423:DisaggregatedStateStorageandM

Re: [DISCUSS] FLIP-426: Grouping Remote State Access

2024-03-20 Thread yue ma
Access#FLIP426:GroupingRemoteStateAccess-CodeExampleonHowtoAccessStateUsingBatch > > Best, > Jinzhong Li > > > On Tue, Mar 19, 2024 at 5:52 PM yue ma wrote: > > > Hi Jinzhong, > > > > Thanks for the FLIP. I have the following questions: > > > >

Re: [DISCUSS] FLIP-428: Fault Tolerance/Rescale Integration for Disaggregated State

2024-03-20 Thread yue ma
Hi Jinzhong Thank you for initiating this FLIP. I have just some minor question: 1. If we choice Option-3 for ForSt , how would we handle Manifest File ? Should we take snapshot of the Manifest during the synchronization phase? Otherwise, may the Manifest and MetaInfo information be

Re: [DISCUSS] FLIP-427: Disaggregated State Store

2024-03-19 Thread yue ma
Hi Hangxiang, Thanks for bringing this discussion. I have a few questions about the Proposal you mentioned in the FLIP. The current conclusion is to use proposal 2, which is okay for me. My point is whether we should retain the potential of proposal 1 in the design. There are the following

Re: [DISCUSS] FLIP-426: Grouping Remote State Access

2024-03-19 Thread yue ma
Hi Jinzhong, Thanks for the FLIP. I have the following questions: 1. Does Grouping Remote State Access only support asynchronous interfaces? --If it is: IIUC, MultiGet can also greatly improve performance for synchronous access modes. Do we need to support it ? --If not, how can we

Re: [DISCUSS] FLIP-424: Asynchronous State APIs

2024-03-19 Thread yue ma
Hi Zakelly, Thanks for your proposal. The FLIP looks good to me +1! I'd like to ask some minor questions I found that there is also a definition of class `FutureUtils` under `org. apache. flink. util. concurrent` which seems to offer more interfaces. My question is: 1. Is it possible for all

[jira] [Created] (FLINK-34210) DefaultExecutionGraphBuilder#isCheckpointingEnabled may return Wrong Value when checkpoint disabled

2024-01-23 Thread Yue Ma (Jira)
Yue Ma created FLINK-34210: -- Summary: DefaultExecutionGraphBuilder#isCheckpointingEnabled may return Wrong Value when checkpoint disabled Key: FLINK-34210 URL: https://issues.apache.org/jira/browse/FLINK-34210

Re: [DISCUSS] FLIP-414: Support Retry Mechanism in RocksDBStateDataTransfer

2024-01-10 Thread yue ma
Thanks for driving this effort, xiangyu! The proposal overall LGTM. I just have a small question. There are other places in Flink that interact with external storage. Should we consider adding a general retry mechanism to them? xiangyu feng 于2024年1月8日周一 11:31写道: > Hi devs, > > I'm opening this

[jira] [Created] (FLINK-33946) RocksDb sets setAvoidFlushDuringShutdown to true to speed up Task Cancel

2023-12-26 Thread Yue Ma (Jira)
Yue Ma created FLINK-33946: -- Summary: RocksDb sets setAvoidFlushDuringShutdown to true to speed up Task Cancel Key: FLINK-33946 URL: https://issues.apache.org/jira/browse/FLINK-33946 Project: Flink

[jira] [Created] (FLINK-33819) Suppor setting CompressType in RocksDBStateBackend

2023-12-13 Thread Yue Ma (Jira)
Yue Ma created FLINK-33819: -- Summary: Suppor setting CompressType in RocksDBStateBackend Key: FLINK-33819 URL: https://issues.apache.org/jira/browse/FLINK-33819 Project: Flink Issue Type

[jira] [Created] (FLINK-33126) Fix EventTimeAllWindowCheckpointingITCase jobName typo

2023-09-20 Thread Yue Ma (Jira)
Yue Ma created FLINK-33126: -- Summary: Fix EventTimeAllWindowCheckpointingITCase jobName typo Key: FLINK-33126 URL: https://issues.apache.org/jira/browse/FLINK-33126 Project: Flink Issue Type

[jira] [Created] (FLINK-32833) Rocksdb CacheIndexAndFilterBlocks must be true when using shared memory

2023-08-11 Thread Yue Ma (Jira)
Yue Ma created FLINK-32833: -- Summary: Rocksdb CacheIndexAndFilterBlocks must be true when using shared memory Key: FLINK-32833 URL: https://issues.apache.org/jira/browse/FLINK-32833 Project: Flink

Re: [ANNOUNCE] New Apache Flink Committer - Weihua Hu

2023-08-03 Thread yue ma
Congratulations, Weihua!

[jira] [Created] (FLINK-31238) Use IngestDB to speed up Rocksdb rescaling recovery

2023-02-27 Thread Yue Ma (Jira)
Yue Ma created FLINK-31238: -- Summary: Use IngestDB to speed up Rocksdb rescaling recovery Key: FLINK-31238 URL: https://issues.apache.org/jira/browse/FLINK-31238 Project: Flink Issue Type

Re: [VOTE] FLIP-228: Support Within between events in CEP Pattern

2022-06-14 Thread yue ma
Thanks for Nicholas driving this. +1 Nicholas 于2022年6月13日周一 10:16写道: > Hi everyone, > > > > > Thanks for feedback for FLIP-228: Support Within between events in CEP > Pattern[1] on the discussion thread[2]. I'd like to start a VOTE thread for > FLIP-228. > > The vote will be open for at least

Re: [DISCUSS] FLIP-228: Support Within between events in CEP Pattern

2022-05-06 Thread yue ma
hi Nicholas, Thanks for bringing this discussion, we also think it's a useful feature. Some fine-grained timeout pattern matching can be implemented in CEP which makes Flink CEP more powerful Nicholas 于2022年5月5日周四 14:28写道: > Hi everyone, > > > > > Pattern#withIn interface in CEP defines the

Re: [DISCUSS] FLIP-228: Support Within between events in CEP Pattern

2022-05-06 Thread yue ma
hi Nicholas , Nicholas 于2022年5月5日周四 14:28写道: > Hi everyone, > > > > > Pattern#withIn interface in CEP defines the maximum time interval in which > a matching pattern has to be completed in order to be considered valid, > which interval corresponds to the maximum time gap between first and the

[jira] [Created] (FLINK-27218) Serializer in OperatorState has not been updated when new Serializers are NOT incompatible

2022-04-13 Thread Yue Ma (Jira)
Yue Ma created FLINK-27218: -- Summary: Serializer in OperatorState has not been updated when new Serializers are NOT incompatible Key: FLINK-27218 URL: https://issues.apache.org/jira/browse/FLINK-27218

[jira] [Created] (FLINK-26941) Support Pattern end with notFollowedBy with window

2022-03-31 Thread Yue Ma (Jira)
Yue Ma created FLINK-26941: -- Summary: Support Pattern end with notFollowedBy with window Key: FLINK-26941 URL: https://issues.apache.org/jira/browse/FLINK-26941 Project: Flink Issue Type

Re: [DISCUSS] FLIP-200: Support Multiple Rule and Dynamic Rule Changing (Flink CEP)

2021-12-17 Thread yue ma
Glad to see the Community's progress in Flink CEP. After reading this Flip, I have few questions, would you please take a look ? 1. About Pattern Updating. If we use PatternProcessoerDiscoverer to update the rules, will it increase the load of JM? For example, if the user wants the updated rule

[jira] [Created] (FLINK-24597) RocksdbStateBackend getKeysAndNamespaces would return duplicate data when using MapState

2021-10-20 Thread Yue Ma (Jira)
Yue Ma created FLINK-24597: -- Summary: RocksdbStateBackend getKeysAndNamespaces would return duplicate data when using MapState Key: FLINK-24597 URL: https://issues.apache.org/jira/browse/FLINK-24597

[jira] [Created] (FLINK-23890) CepOperator may create a large number of timers and cause performance problems

2021-08-20 Thread Yue Ma (Jira)
Yue Ma created FLINK-23890: -- Summary: CepOperator may create a large number of timers and cause performance problems Key: FLINK-23890 URL: https://issues.apache.org/jira/browse/FLINK-23890 Project: Flink

[jira] [Created] (FLINK-22888) Matches results may be wrong when using notNext as the last part of the pattern with Window

2021-06-06 Thread Yue Ma (Jira)
Yue Ma created FLINK-22888: -- Summary: Matches results may be wrong when using notNext as the last part of the pattern with Window Key: FLINK-22888 URL: https://issues.apache.org/jira/browse/FLINK-22888