Re: Flink stateful functions and Agentic Architecture

2024-05-05 Thread Yunfeng Zhou
Hi David, I'm not very familiar with stateful functions, but I participated in Flink ML, a machine learning infrastructure and algorithm library based on Flink. There we developed functions like iteration based model-training process and hot updating ML model during online prediction. You may

Re: Two potential bugs in Flink ML

2024-04-09 Thread Yunfeng Zhou
"statedescriptor", // the state > name > BasicTypeInfo.DOUBLE_TYPE_INFO, > BasicTypeInfo.STRING_TYPE_INFO > ); >

[jira] [Created] (FLINK-35066) TwoInputOperator in IterationBody cannot use keyBy

2024-04-09 Thread Yunfeng Zhou (Jira)
Yunfeng Zhou created FLINK-35066: Summary: TwoInputOperator in IterationBody cannot use keyBy Key: FLINK-35066 URL: https://issues.apache.org/jira/browse/FLINK-35066 Project: Flink Issue

Re: Two potential bugs in Flink ML

2024-04-06 Thread Yunfeng Zhou
Hi Komal, For the first question, could you please provide a simple program that could help reproduce this exception? That could help us better find out the bugs (if any) in Flink ML. For the second question, there have been Functions implementing the IterationListener interface in Flink ML[1]

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

2024-03-28 Thread Yunfeng Zhou
+1 (non-binding) Best, Yunfeng On Wed, Mar 27, 2024 at 6:23 PM Zakelly Lan wrote: > > Hi devs, > > I'd like to start a vote on the FLIP-424: Asynchronous State APIs [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-425: Asynchronous Execution Model

2024-03-28 Thread Yunfeng Zhou
+1 (non-bindling) Best, Yunfeng On Wed, Mar 27, 2024 at 6:28 PM Yanfei Lei wrote: > > 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 >

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

2024-03-25 Thread Yunfeng Zhou
+1 (non-binding) Best, Yunfeng On Wed, Mar 20, 2024 at 8:29 PM weijie guo wrote: > > 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

Re: [DISCUSS] FLIP-423 ~FLIP-428: Introduce Disaggregated State Storage and Management in Flink 2.0

2024-03-19 Thread Yunfeng Zhou
óra > > > > > wrote: > > > > > > > > > > > > > > > Hey all! > > > > > > > > > > > > > > > > This is a massive improvement / work. I just started going >

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

2024-03-12 Thread Yunfeng Zhou
eamOperatorV2, the current package name would be acceptable >> to me if other reviewers of this FLIP agrees on it. > > Actually, I don't like 'v2' either. So if there is another good name, I'd be > happy to apply. This is a compromise to the current situation. Maybe we could > refin

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

2024-03-11 Thread Yunfeng Zhou
s. While for state APIs, we > should stay in the flink-core(-api) module alongside with other > apis, currently I tend to compromise at the expense of naming style. > > > Looking forward to hearing from you again! > > Thanks & Best, > Zakelly > > On Mon, Mar 11

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

2024-03-11 Thread Yunfeng Zhou
;, and the existing state APIs to "org.apache.flink.api.common.state" or "org.apache.flink.api.common.state.sync". Best regards, Yunfeng Zhou On Thu, Mar 7, 2024 at 4:48 PM Zakelly Lan wrote: > > Hi devs, > > I'd like to start a discussion on a sub-FLIP of FLIP-423: Disaggregated > S

Re: [DISCUSS] FLIP-423 ~FLIP-428: Introduce Disaggregated State Storage and Management in Flink 2.0

2024-03-04 Thread Yunfeng Zhou
I suggest we discuss these existing metrics as well as some new > metrics that should be introduced in FLIP-431 later in milestone 2, since we > have basically finished the framework thus we will have a better view of what > metrics should be like afterwards. WDYT? > > > Best, &

Re: [DISCUSS] FLIP-423 ~FLIP-428: Introduce Disaggregated State Storage and Management in Flink 2.0

2024-03-04 Thread Yunfeng Zhou
reasonable. Faster checkpointing is always a core >> > advantage of disaggregated state, but only for the async phase. There will >> > be some complexity introduced by in-flight requests, but I'd suggest a >> > checkpoint containing those in-flight state requests as part of

Re: [DISCUSS] FLIP-423 ~FLIP-428: Introduce Disaggregated State Storage and Management in Flink 2.0

2024-03-01 Thread Yunfeng Zhou
ksDB", the link points to a document in flink-1.15. It might be better to verify the referenced content is still valid in the latest Flink and update the link accordingly. Same for other references if any. Best, Yunfeng Zhou On Thu, Feb 29, 2024 at 2:17 PM Yuan Mei wrote: > > Hi

[jira] [Created] (FLINK-34519) Refine checkpoint scheduling and canceling logic

2024-02-26 Thread Yunfeng Zhou (Jira)
Yunfeng Zhou created FLINK-34519: Summary: Refine checkpoint scheduling and canceling logic Key: FLINK-34519 URL: https://issues.apache.org/jira/browse/FLINK-34519 Project: Flink Issue Type

[jira] [Created] (FLINK-34371) FLIP-331: Support EndOfStreamTrigger and isOutputOnlyAfterEndOfStream operator attribute to optimize task deployment

2024-02-05 Thread Yunfeng Zhou (Jira)
Yunfeng Zhou created FLINK-34371: Summary: FLIP-331: Support EndOfStreamTrigger and isOutputOnlyAfterEndOfStream operator attribute to optimize task deployment Key: FLINK-34371 URL: https://issues.apache.org/jira

Re: [DISCUSS] FLIP-410: Config, Context and Processing Timer Service of DataStream API V2

2024-01-25 Thread Yunfeng Zhou
ty of configurations. Looking forward to your response. Best regards, Yunfeng Zhou On Tue, Dec 26, 2023 at 2:47 PM weijie guo wrote: > > Hi devs, > > > I'd like to start a discussion about FLIP-410: Config, Context and > Processing Timer Service of DataStream API V2 [1]. This is the

Re: [DISCUSS] FLIP-409: DataStream V2 Building Blocks: DataStream, Partitioning and ProcessFunction

2024-01-25 Thread Yunfeng Zhou
forward to discussing these questions with you. Best regards, Yunfeng Zhou On Tue, Dec 26, 2023 at 2:44 PM weijie guo wrote: > > Hi devs, > > > I'd like to start a discussion about FLIP-409: DataStream V2 Building > Blocks: DataStream, Partitioning and ProcessFunction [1]. >

[jira] [Created] (FLINK-34077) Sphinx version needs upgrade

2024-01-14 Thread Yunfeng Zhou (Jira)
Yunfeng Zhou created FLINK-34077: Summary: Sphinx version needs upgrade Key: FLINK-34077 URL: https://issues.apache.org/jira/browse/FLINK-34077 Project: Flink Issue Type: Bug

Re: Fink 1.17 vs 1.18

2023-11-08 Thread Yunfeng Zhou
Hi Amir, You can find the major differences in Flink 1.18's release note[1], which has also described the performance improvements in 1.18. [1] https://flink.apache.org/2023/10/24/announcing-the-release-of-apache-flink-1.18/ Best, Yunfeng Zhou On Wed, Nov 8, 2023 at 11:50 PM Amir Hossein

Re: [DISCUSS] FLIP-365: Introduce flush interval to adjust the interval of emitting results with idempotent semantics

2023-09-21 Thread Yunfeng Zhou
re CPU resources for checkpointing and compaction. > > > Looking forward to your thoughts. > > > Best, > Zakelly > > > On Mon, Sep 11, 2023 at 1:39 PM Yunfeng Zhou > wrote: > > > > Hi all, > > > > Dong(cc'ed) and I are opening this thread

[DISCUSS] FLIP-365: Introduce flush interval to adjust the interval of emitting results with idempotent semantics

2023-09-10 Thread Yunfeng Zhou
Hi all, Dong(cc'ed) and I are opening this thread to discuss our proposal to support buffering & flushing the output of operators with idempotent semantics, which has been documented in

Re: [DISCUSS] FLIP-330: Support specifying record timestamp requirement

2023-08-10 Thread Yunfeng Zhou
ount (where the record size is just 1 > string token + 1 integer, typically 20B or so?) shows only 2-3% throughput > improvement. Not even mention the workloads with larger record size or > higher computation load. > > Given the above, I personally don't think the benefit of thi

Re: [DISCUSS] FLIP-330: Support specifying record timestamp requirement

2023-08-07 Thread Yunfeng Zhou
TrackingInterval is set too small(like 5ms), it will have a large > impact on performance. But if the latencyTrackingInterval is configured to be > relatively large, such as 10s, this impact can be ignored. > > > > -- > > Best, > Matt Wang > > > Replied Mes

Re: [DISCUSS] FLIP-330: Support specifying record timestamp requirement

2023-08-02 Thread Yunfeng Zhou
y > > limit > > > the usage scenarios. Whether the solution design can retain the > > capability > > > of the latency marker; > > > 3. The data of the POC test is of long type. Here I want to see how much > > > profit it will have if it is a string with a length o

Re: [DISCUSS] FLIP-330: Support specifying record timestamp requirement

2023-07-14 Thread Yunfeng Zhou
hether the solution design can retain the capability of > the latency marker; > 3. The data of the POC test is of long type. Here I want to see how much > profit it will have if it is a string with a length of 100B or 1KB. > > > -- > > Best, > Matt Wang > > > --

Re: [DISCUSS] FLIP-330: Support specifying record timestamp requirement

2023-07-13 Thread Yunfeng Zhou
out the benchmark example, why did you enable object reuse? Since It > is an optimization of serde, will the benchmark be better if it is > disabled? > > Best regards, > Jing > > On Mon, Jul 10, 2023 at 11:54 AM Yunfeng Zhou > wrote: > > > Hi all, > >

[DISCUSS] FLIP-330: Support specifying record timestamp requirement

2023-07-10 Thread Yunfeng Zhou
Hi all, Dong(cc'ed) and I are opening this thread to discuss our proposal to support optimizing StreamRecord's serialization performance. Currently, a StreamRecord would be converted into a 1-byte tag (+ 8-byte timestamp) + N-byte serialized value during the serialization process. In scenarios

[jira] [Created] (FLINK-32514) FLIP-309: Support using larger checkpointing interval when source is processing backlog

2023-07-03 Thread Yunfeng Zhou (Jira)
Yunfeng Zhou created FLINK-32514: Summary: FLIP-309: Support using larger checkpointing interval when source is processing backlog Key: FLINK-32514 URL: https://issues.apache.org/jira/browse/FLINK-32514

[jira] [Created] (FLINK-32494) Cannot convert list literal to Table with PyFlink

2023-06-29 Thread Yunfeng Zhou (Jira)
Yunfeng Zhou created FLINK-32494: Summary: Cannot convert list literal to Table with PyFlink Key: FLINK-32494 URL: https://issues.apache.org/jira/browse/FLINK-32494 Project: Flink Issue Type

[jira] [Created] (FLINK-32464) AssertionError when converting between Table and SQL with selection and type cast

2023-06-28 Thread Yunfeng Zhou (Jira)
Yunfeng Zhou created FLINK-32464: Summary: AssertionError when converting between Table and SQL with selection and type cast Key: FLINK-32464 URL: https://issues.apache.org/jira/browse/FLINK-32464

[DISCUSS] FLIP-325: Support configuring end-to-end allowed latency

2023-06-24 Thread Yunfeng Zhou
Hi all, Dong(cc'ed) and I are opening this thread to discuss our proposal to support configuring end-to-end allowed latency for Flink jobs, which has been documented in FLIP-325 . By

[DISCUSS] FLIP-309: Enable operators to trigger checkpoints dynamically

2023-05-09 Thread Yunfeng Zhou
Hi all, Dong(cc'ed) and I are opening this thread to discuss our proposal to support dynamically triggering checkpoints from operators, which has been documented in FLIP-309 . With the help of the ability proposed in

[jira] [Created] (FLINK-31605) Table#to_pandas check table boundedness

2023-03-24 Thread Yunfeng Zhou (Jira)
Yunfeng Zhou created FLINK-31605: Summary: Table#to_pandas check table boundedness Key: FLINK-31605 URL: https://issues.apache.org/jira/browse/FLINK-31605 Project: Flink Issue Type

[jira] [Created] (FLINK-30959) UNIX_TIMESTAMP's return value does not meet expected

2023-02-07 Thread Yunfeng Zhou (Jira)
Yunfeng Zhou created FLINK-30959: Summary: UNIX_TIMESTAMP's return value does not meet expected Key: FLINK-30959 URL: https://issues.apache.org/jira/browse/FLINK-30959 Project: Flink Issue

[jira] [Created] (FLINK-30753) Py4J cannot acquire Table.explain() method

2023-01-19 Thread Yunfeng Zhou (Jira)
Yunfeng Zhou created FLINK-30753: Summary: Py4J cannot acquire Table.explain() method Key: FLINK-30753 URL: https://issues.apache.org/jira/browse/FLINK-30753 Project: Flink Issue Type

[jira] [Created] (FLINK-30532) Add benchmark for DCT, SQLTransformer and StopWordsRemover algorithm

2022-12-28 Thread Yunfeng Zhou (Jira)
Yunfeng Zhou created FLINK-30532: Summary: Add benchmark for DCT, SQLTransformer and StopWordsRemover algorithm Key: FLINK-30532 URL: https://issues.apache.org/jira/browse/FLINK-30532 Project: Flink

[jira] [Created] (FLINK-30292) Better support for conversion between DataType and TypeInformation

2022-12-04 Thread Yunfeng Zhou (Jira)
Yunfeng Zhou created FLINK-30292: Summary: Better support for conversion between DataType and TypeInformation Key: FLINK-30292 URL: https://issues.apache.org/jira/browse/FLINK-30292 Project: Flink

[jira] [Created] (FLINK-30241) Flink ML Iteration ConcurrentModificationException

2022-11-29 Thread Yunfeng Zhou (Jira)
Yunfeng Zhou created FLINK-30241: Summary: Flink ML Iteration ConcurrentModificationException Key: FLINK-30241 URL: https://issues.apache.org/jira/browse/FLINK-30241 Project: Flink Issue

[jira] [Created] (FLINK-30168) PyFlink Deserialization Error with Object Array

2022-11-23 Thread Yunfeng Zhou (Jira)
Yunfeng Zhou created FLINK-30168: Summary: PyFlink Deserialization Error with Object Array Key: FLINK-30168 URL: https://issues.apache.org/jira/browse/FLINK-30168 Project: Flink Issue Type

[jira] [Created] (FLINK-30144) Guarantee Flink ML operators function correctly with object-reuse enabled

2022-11-22 Thread Yunfeng Zhou (Jira)
Yunfeng Zhou created FLINK-30144: Summary: Guarantee Flink ML operators function correctly with object-reuse enabled Key: FLINK-30144 URL: https://issues.apache.org/jira/browse/FLINK-30144 Project

[jira] [Created] (FLINK-30142) Flink ML operators lose table watermark after transform()

2022-11-22 Thread Yunfeng Zhou (Jira)
Yunfeng Zhou created FLINK-30142: Summary: Flink ML operators lose table watermark after transform() Key: FLINK-30142 URL: https://issues.apache.org/jira/browse/FLINK-30142 Project: Flink

[jira] [Created] (FLINK-30130) Table.select lose watermark

2022-11-21 Thread Yunfeng Zhou (Jira)
Yunfeng Zhou created FLINK-30130: Summary: Table.select lose watermark Key: FLINK-30130 URL: https://issues.apache.org/jira/browse/FLINK-30130 Project: Flink Issue Type: Bug

[jira] [Created] (FLINK-30124) GenericType is not supported in PyFlink currently

2022-11-21 Thread Yunfeng Zhou (Jira)
Yunfeng Zhou created FLINK-30124: Summary: GenericType is not supported in PyFlink currently Key: FLINK-30124 URL: https://issues.apache.org/jira/browse/FLINK-30124 Project: Flink Issue

[jira] [Created] (FLINK-30122) Flink ML KMeans getting model data throws TypeError

2022-11-21 Thread Yunfeng Zhou (Jira)
Yunfeng Zhou created FLINK-30122: Summary: Flink ML KMeans getting model data throws TypeError Key: FLINK-30122 URL: https://issues.apache.org/jira/browse/FLINK-30122 Project: Flink Issue

[jira] [Created] (FLINK-30037) Improve the efficiency of Flink ML Python CI

2022-11-15 Thread Yunfeng Zhou (Jira)
Yunfeng Zhou created FLINK-30037: Summary: Improve the efficiency of Flink ML Python CI Key: FLINK-30037 URL: https://issues.apache.org/jira/browse/FLINK-30037 Project: Flink Issue Type

[jira] [Created] (FLINK-30009) OperatorCoordinator.start()'s JavaDoc mismatches its behavior

2022-11-13 Thread Yunfeng Zhou (Jira)
Yunfeng Zhou created FLINK-30009: Summary: OperatorCoordinator.start()'s JavaDoc mismatches its behavior Key: FLINK-30009 URL: https://issues.apache.org/jira/browse/FLINK-30009 Project: Flink

[jira] [Created] (FLINK-29843) Euclidean Distance Measure generates NAN distance values

2022-11-01 Thread Yunfeng Zhou (Jira)
Yunfeng Zhou created FLINK-29843: Summary: Euclidean Distance Measure generates NAN distance values Key: FLINK-29843 URL: https://issues.apache.org/jira/browse/FLINK-29843 Project: Flink

[jira] [Created] (FLINK-29763) TaskManager heatbeat timeout exception in Github CI for python tests

2022-10-25 Thread Yunfeng Zhou (Jira)
Yunfeng Zhou created FLINK-29763: Summary: TaskManager heatbeat timeout exception in Github CI for python tests Key: FLINK-29763 URL: https://issues.apache.org/jira/browse/FLINK-29763 Project: Flink

[jira] [Created] (FLINK-29604) Add Estimator and Transformer for CountVectorizer

2022-10-12 Thread Yunfeng Zhou (Jira)
Yunfeng Zhou created FLINK-29604: Summary: Add Estimator and Transformer for CountVectorizer Key: FLINK-29604 URL: https://issues.apache.org/jira/browse/FLINK-29604 Project: Flink Issue Type

[jira] [Created] (FLINK-29602) Add Transformer for SQLTransformer

2022-10-12 Thread Yunfeng Zhou (Jira)
Yunfeng Zhou created FLINK-29602: Summary: Add Transformer for SQLTransformer Key: FLINK-29602 URL: https://issues.apache.org/jira/browse/FLINK-29602 Project: Flink Issue Type: New Feature

[jira] [Created] (FLINK-29603) Add Transformer for StopWordsRemover

2022-10-12 Thread Yunfeng Zhou (Jira)
Yunfeng Zhou created FLINK-29603: Summary: Add Transformer for StopWordsRemover Key: FLINK-29603 URL: https://issues.apache.org/jira/browse/FLINK-29603 Project: Flink Issue Type: New Feature

[jira] [Created] (FLINK-29600) Add Estimator and Transformer for BucketedRandomProjectionLSH

2022-10-12 Thread Yunfeng Zhou (Jira)
Yunfeng Zhou created FLINK-29600: Summary: Add Estimator and Transformer for BucketedRandomProjectionLSH Key: FLINK-29600 URL: https://issues.apache.org/jira/browse/FLINK-29600 Project: Flink

[jira] [Created] (FLINK-29601) Add Estimator and Transformer for UnivariateFeatureSelector

2022-10-12 Thread Yunfeng Zhou (Jira)
Yunfeng Zhou created FLINK-29601: Summary: Add Estimator and Transformer for UnivariateFeatureSelector Key: FLINK-29601 URL: https://issues.apache.org/jira/browse/FLINK-29601 Project: Flink

[jira] [Created] (FLINK-29599) Add Estimator and Transformer for MinHashLSH

2022-10-12 Thread Yunfeng Zhou (Jira)
Yunfeng Zhou created FLINK-29599: Summary: Add Estimator and Transformer for MinHashLSH Key: FLINK-29599 URL: https://issues.apache.org/jira/browse/FLINK-29599 Project: Flink Issue Type: New

[jira] [Created] (FLINK-29598) Add Estimator and Transformer for Imputer

2022-10-12 Thread Yunfeng Zhou (Jira)
Yunfeng Zhou created FLINK-29598: Summary: Add Estimator and Transformer for Imputer Key: FLINK-29598 URL: https://issues.apache.org/jira/browse/FLINK-29598 Project: Flink Issue Type: New

[jira] [Created] (FLINK-29597) Add Estimator and Transformer for QuantileDiscretizer

2022-10-12 Thread Yunfeng Zhou (Jira)
Yunfeng Zhou created FLINK-29597: Summary: Add Estimator and Transformer for QuantileDiscretizer Key: FLINK-29597 URL: https://issues.apache.org/jira/browse/FLINK-29597 Project: Flink Issue

[jira] [Created] (FLINK-29596) Add Estimator and Transformer for RFormula

2022-10-12 Thread Yunfeng Zhou (Jira)
Yunfeng Zhou created FLINK-29596: Summary: Add Estimator and Transformer for RFormula Key: FLINK-29596 URL: https://issues.apache.org/jira/browse/FLINK-29596 Project: Flink Issue Type: New

[jira] [Created] (FLINK-29595) Add Estimator and Transformer for ChiSqSelector

2022-10-12 Thread Yunfeng Zhou (Jira)
Yunfeng Zhou created FLINK-29595: Summary: Add Estimator and Transformer for ChiSqSelector Key: FLINK-29595 URL: https://issues.apache.org/jira/browse/FLINK-29595 Project: Flink Issue Type

[jira] [Created] (FLINK-29591) Add built-in UDFs to convert between arrays and vectors

2022-10-12 Thread Yunfeng Zhou (Jira)
Yunfeng Zhou created FLINK-29591: Summary: Add built-in UDFs to convert between arrays and vectors Key: FLINK-29591 URL: https://issues.apache.org/jira/browse/FLINK-29591 Project: Flink

[jira] [Created] (FLINK-29429) Add DataType for Flink ML linear algorithm classes

2022-09-27 Thread Yunfeng Zhou (Jira)
Yunfeng Zhou created FLINK-29429: Summary: Add DataType for Flink ML linear algorithm classes Key: FLINK-29429 URL: https://issues.apache.org/jira/browse/FLINK-29429 Project: Flink Issue

[jira] [Created] (FLINK-29115) Improve the quickstart of Flink ML python API

2022-08-26 Thread Yunfeng Zhou (Jira)
Yunfeng Zhou created FLINK-29115: Summary: Improve the quickstart of Flink ML python API Key: FLINK-29115 URL: https://issues.apache.org/jira/browse/FLINK-29115 Project: Flink Issue Type

[jira] [Created] (FLINK-29044) Add Transformer for DCT

2022-08-19 Thread Yunfeng Zhou (Jira)
Yunfeng Zhou created FLINK-29044: Summary: Add Transformer for DCT Key: FLINK-29044 URL: https://issues.apache.org/jira/browse/FLINK-29044 Project: Flink Issue Type: New Feature

[jira] [Created] (FLINK-29043) Improve ML iteration efficiency

2022-08-19 Thread Yunfeng Zhou (Jira)
Yunfeng Zhou created FLINK-29043: Summary: Improve ML iteration efficiency Key: FLINK-29043 URL: https://issues.apache.org/jira/browse/FLINK-29043 Project: Flink Issue Type: Improvement

[jira] [Created] (FLINK-28941) Savepoint ignores MaxConcurrentCheckpoint limit in aligned checkpoint case

2022-08-11 Thread Yunfeng Zhou (Jira)
Yunfeng Zhou created FLINK-28941: Summary: Savepoint ignores MaxConcurrentCheckpoint limit in aligned checkpoint case Key: FLINK-28941 URL: https://issues.apache.org/jira/browse/FLINK-28941 Project

[jira] [Created] (FLINK-28761) BinaryClassificationEvaluator cannot work with unaligned checkpoint

2022-08-01 Thread Yunfeng Zhou (Jira)
Yunfeng Zhou created FLINK-28761: Summary: BinaryClassificationEvaluator cannot work with unaligned checkpoint Key: FLINK-28761 URL: https://issues.apache.org/jira/browse/FLINK-28761 Project: Flink

[jira] [Created] (FLINK-28673) Migrate Flink ML to Flink 1.15.1

2022-07-25 Thread Yunfeng Zhou (Jira)
Yunfeng Zhou created FLINK-28673: Summary: Migrate Flink ML to Flink 1.15.1 Key: FLINK-28673 URL: https://issues.apache.org/jira/browse/FLINK-28673 Project: Flink Issue Type: Improvement

[jira] [Created] (FLINK-28639) Preserve distributed consistency of OperatorEvents from subtasks to OperatorCoordinator

2022-07-21 Thread Yunfeng Zhou (Jira)
Yunfeng Zhou created FLINK-28639: Summary: Preserve distributed consistency of OperatorEvents from subtasks to OperatorCoordinator Key: FLINK-28639 URL: https://issues.apache.org/jira/browse/FLINK-28639

[jira] [Created] (FLINK-28606) Preserve distributed consistency of OperatorEvents from OperatorCoordinator to subtasks

2022-07-19 Thread Yunfeng Zhou (Jira)
Yunfeng Zhou created FLINK-28606: Summary: Preserve distributed consistency of OperatorEvents from OperatorCoordinator to subtasks Key: FLINK-28606 URL: https://issues.apache.org/jira/browse/FLINK-28606

Re: [VOTE] Apache Flink ML Release 2.1.0, release candidate #2

2022-07-04 Thread Yunfeng Zhou
Thanks for raising this release candidate. +1 (non-binding) - Verified that the checksums and GPG files match the corresponding release files. - Verified that the source distributions do not contain any binaries. - Built the source distribution and ensured that all source files have Apache

[jira] [Created] (FLINK-28224) Add document for algorithms and features in Flink ML 2.1

2022-06-23 Thread Yunfeng Zhou (Jira)
Yunfeng Zhou created FLINK-28224: Summary: Add document for algorithms and features in Flink ML 2.1 Key: FLINK-28224 URL: https://issues.apache.org/jira/browse/FLINK-28224 Project: Flink

[jira] [Created] (FLINK-27798) Migrate Flink ML to Flink 1.15.0

2022-05-26 Thread Yunfeng Zhou (Jira)
Yunfeng Zhou created FLINK-27798: Summary: Migrate Flink ML to Flink 1.15.0 Key: FLINK-27798 URL: https://issues.apache.org/jira/browse/FLINK-27798 Project: Flink Issue Type: Improvement

[jira] [Created] (FLINK-27797) PythonTableUtils.getCollectionInputFormat cannot correctly handle None values

2022-05-26 Thread Yunfeng Zhou (Jira)
Yunfeng Zhou created FLINK-27797: Summary: PythonTableUtils.getCollectionInputFormat cannot correctly handle None values Key: FLINK-27797 URL: https://issues.apache.org/jira/browse/FLINK-27797

[jira] [Created] (FLINK-27742) Fix Compatibility Issues Between Flink ML Operators.

2022-05-23 Thread Yunfeng Zhou (Jira)
Yunfeng Zhou created FLINK-27742: Summary: Fix Compatibility Issues Between Flink ML Operators. Key: FLINK-27742 URL: https://issues.apache.org/jira/browse/FLINK-27742 Project: Flink Issue

[jira] [Created] (FLINK-27084) Perround mode recreating operator fails

2022-04-06 Thread Yunfeng Zhou (Jira)
Yunfeng Zhou created FLINK-27084: Summary: Perround mode recreating operator fails Key: FLINK-27084 URL: https://issues.apache.org/jira/browse/FLINK-27084 Project: Flink Issue Type: Bug

[jira] [Created] (FLINK-26313) Support Online KMeans in Flink ML

2022-02-22 Thread Yunfeng Zhou (Jira)
Yunfeng Zhou created FLINK-26313: Summary: Support Online KMeans in Flink ML Key: FLINK-26313 URL: https://issues.apache.org/jira/browse/FLINK-26313 Project: Flink Issue Type: New Feature

[jira] [Created] (FLINK-26263) Check data size in LogisticRegression

2022-02-20 Thread Yunfeng Zhou (Jira)
Yunfeng Zhou created FLINK-26263: Summary: Check data size in LogisticRegression Key: FLINK-26263 URL: https://issues.apache.org/jira/browse/FLINK-26263 Project: Flink Issue Type: Bug

[jira] [Created] (FLINK-26100) Set up Flink ML Document Website

2022-02-13 Thread Yunfeng Zhou (Jira)
Yunfeng Zhou created FLINK-26100: Summary: Set up Flink ML Document Website Key: FLINK-26100 URL: https://issues.apache.org/jira/browse/FLINK-26100 Project: Flink Issue Type: New Feature

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

2021-12-19 Thread Yunfeng Zhou
nd Dynamic Rule Changing . > But we are facing such a problem, some users’ usage scenarios are that they > want to have their own pattern for each user_id, which means that there > could be thousands of patterns, which would make the performance of Pattern > Matching very poor. We are also try

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

2021-12-14 Thread Yunfeng Zhou
artijn Visser > > >> wrote: > > >> > > >>> Hi Yunfeng, > > >>> > > >>> Thanks for creating the FLIP. I don't see any mention of SQL's > > >>> MATCH_RECOGNIZE implementation in the FLIP and I think that any > > >

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

2021-12-13 Thread Yunfeng Zhou
Hi Konstantin, Thanks for your suggestion. For the first idea, I agree that adding a timestamp field and making users able to schedule a rule is a useful feature. This might not require too much implementation work and I believe it can be achieved in this FLIP. As for the second idea, Rule is a

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

2021-12-10 Thread Yunfeng Zhou
Hi all, I'm opening this thread to propose the design to support multiple rule & dynamic rule changing in the Flink-CEP project, as described in FLIP-200 [1] . Currently Flink CEP only supports having a single pattern inside a CepOperator and does not support changing the pattern dynamically. In

[jira] [Created] (FLINK-24955) Add One-hot Encoder to Flink ML

2021-11-18 Thread Yunfeng Zhou (Jira)
Yunfeng Zhou created FLINK-24955: Summary: Add One-hot Encoder to Flink ML Key: FLINK-24955 URL: https://issues.apache.org/jira/browse/FLINK-24955 Project: Flink Issue Type: New Feature

[jira] [Created] (FLINK-24817) Support Naive Bayes algorithm in Flink ML

2021-11-08 Thread Yunfeng Zhou (Jira)
Yunfeng Zhou created FLINK-24817: Summary: Support Naive Bayes algorithm in Flink ML Key: FLINK-24817 URL: https://issues.apache.org/jira/browse/FLINK-24817 Project: Flink Issue Type: New

Re: [VOTE] FLIP-176: Unified Iteration to Support Algorithms (Flink ML)

2021-10-25 Thread Yunfeng Zhou
Sorry that I misunderstood the usage of "binding". I am not a Flink committer so my vote should be a non-binding one. Best, Yunfeng On Mon, Oct 25, 2021 at 4:33 PM Yunfeng Zhou wrote: > Excellent work to support iteration for Flink. > > +1 (binding) > > Best regards,

Re: [VOTE] FLIP-176: Unified Iteration to Support Algorithms (Flink ML)

2021-10-25 Thread Yunfeng Zhou
Excellent work to support iteration for Flink. +1 (binding) Best regards, Yunfeng On Sat, Oct 23, 2021 at 12:27 AM Till Rohrmann wrote: > Thanks for creating this FLIP. > > +1 (binding) > > Cheers, > Till > > > On Fri, Oct 22, 2021 at 6:29 AM Guowei Ma wrote: > > > +1 (binding) > > > > Best,

[jira] [Created] (FLINK-22689) Table API Documentation Row-Based Operations Example Fails

2021-05-17 Thread Yunfeng Zhou (Jira)
Yunfeng Zhou created FLINK-22689: Summary: Table API Documentation Row-Based Operations Example Fails Key: FLINK-22689 URL: https://issues.apache.org/jira/browse/FLINK-22689 Project: Flink