Re: [Discussion] externalize Hive connector

2023-01-19 Thread Chen Qin
now. > > Best regards, > Yuxia > > - 原始邮件 - > 发件人: "Chen Qin" > 收件人: "dev" > 发送时间: 星期三, 2023年 1 月 11日 上午 11:24:48 > 主题: Re: [Discussion] externalize Hive connector > > Yuxia, > > thanks for details context, I see with your point

[jira] [Created] (FLINK-30667) remove the planner dependency in flink-connector-hive

2023-01-12 Thread Chen Qin (Jira)
Chen Qin created FLINK-30667: Summary: remove the planner dependency in flink-connector-hive Key: FLINK-30667 URL: https://issues.apache.org/jira/browse/FLINK-30667 Project: Flink Issue Type

[jira] [Created] (FLINK-30664) [Connector/Hive] cleanup hive/haoop package ambiguous package dependencies

2023-01-12 Thread Chen Qin (Jira)
Chen Qin created FLINK-30664: Summary: [Connector/Hive] cleanup hive/haoop package ambiguous package dependencies Key: FLINK-30664 URL: https://issues.apache.org/jira/browse/FLINK-30664 Project: Flink

[jira] [Created] (FLINK-30660) move SQLClientHiveITCase and TestHiveCatalogFactory to flink-connector-hive e2e

2023-01-12 Thread Chen Qin (Jira)
Chen Qin created FLINK-30660: Summary: move SQLClientHiveITCase and TestHiveCatalogFactory to flink-connector-hive e2e Key: FLINK-30660 URL: https://issues.apache.org/jira/browse/FLINK-30660 Project

[jira] [Created] (FLINK-30659) move Flink-sql-parser-hive to flink-connector-hive

2023-01-12 Thread Chen Qin (Jira)
Chen Qin created FLINK-30659: Summary: move Flink-sql-parser-hive to flink-connector-hive Key: FLINK-30659 URL: https://issues.apache.org/jira/browse/FLINK-30659 Project: Flink Issue Type: Sub

[jira] [Created] (FLINK-30658) remove Flink-sql-parser-hive dependency on table-planner

2023-01-12 Thread Chen Qin (Jira)
Chen Qin created FLINK-30658: Summary: remove Flink-sql-parser-hive dependency on table-planner Key: FLINK-30658 URL: https://issues.apache.org/jira/browse/FLINK-30658 Project: Flink Issue Type

Re: [Discussion] externalize Hive connector

2023-01-10 Thread Chen Qin
connector can't be released as offical > Hive connector unless we resolve the problem I refered above. > So, why not wait any thing is ready before externalize the Hive connector? > > Best regards, > Yuxia > > - 原始邮件 - > 发件人: "Chen Qin" > 收件人: &q

Re: [Discussion] externalize Hive connector

2023-01-09 Thread Chen Qin
his hasn't been done > yet is because it's blocked by > https://issues.apache.org/jira/browse/FLINK-26603. Is that still the case? > > Best regards, > > Martijn > > [1] https://lists.apache.org/thread/bk9f91o6wk66zdh353j1n7sfshh262tr > > On Mon, Jan

[Discussion] externalize Hive connector

2023-01-09 Thread Chen Qin
Hi there, Following community guidance Externalized+Connector+development , We would like to initiate discussion on moving connector/hive to apache/flink-connector-hive

[jira] [Created] (FLINK-30362) Flink-connector-hive can't build with maven 3.8

2022-12-11 Thread Chen Qin (Jira)
Chen Qin created FLINK-30362: Summary: Flink-connector-hive can't build with maven 3.8 Key: FLINK-30362 URL: https://issues.apache.org/jira/browse/FLINK-30362 Project: Flink Issue Type: Bug

Re: Table API thrift support

2022-12-06 Thread Chen Qin
erything on connector externalization is documented at > > https://cwiki.apache.org/confluence/display/FLINK/Externalized+Connector+development > , > including the links to the relevant discussions on that topic in the > community. > > Thanks, > > Martijn > > On Mon, Nov

Re: Table API thrift support

2022-11-28 Thread Chen Qin
o root > level config then available anymore. How would it then work? > > Best regards, > > Martijn > > On Sun, Nov 27, 2022 at 4:02 AM Chen Qin wrote: > > > Hi Martjin, > > > > "shading Thrift libraries from the Hive connector" > > Hivemetas

Re: Table API thrift support

2022-11-26 Thread Chen Qin
done for Protobuf? > > https://nightlies.apache.org/flink/flink-docs-stable/docs/connectors/table/formats/protobuf/ > > Best regards, > > Martijn > > On Tue, Nov 22, 2022 at 6:44 PM Chen Qin wrote: > > > Hi Yuxia, Martijin, > > > > Thanks for your feed

Re: Table API thrift support

2022-11-22 Thread Chen Qin
upport to keep code consistency Please give another pass and let me know if you have any questions. Chen On Mon, May 30, 2022 at 6:34 PM Chen Qin wrote: > > > On Mon, May 30, 2022 at 7:35 AM Martijn Visser > wrote: > >> Hi Chen, >> >> I think the best star

Re: [DISCUSS] FLIP-271: Autoscaling

2022-11-20 Thread Chen Qin
min,5 mins) as input features > > Regards, > Gyula > > On Sun, Nov 20, 2022 at 12:28 AM Chen Qin wrote: > > > Hi Gyula, > > > > Do we think the scaler could be a plugin or hard coded ? > > We observed some cases scaler can't address (e.g async io depend

Re: [DISCUSS] FLIP-271: Autoscaling

2022-11-19 Thread Chen Qin
Hi Gyula, Do we think the scaler could be a plugin or hard coded ? We observed some cases scaler can't address (e.g async io dependency service degradation or small spike that doesn't worth restarting job) Thanks, Chen On Fri, Nov 18, 2022 at 1:03 AM Gyula Fóra wrote: > Hi Dong! > > Could you

Re: [DISCUSS] FLIP-237: Thrift Format Support in Flink

2022-06-10 Thread Chen Qin
> with Catalogs in general, not with Hive only. What type of > compatibility guarantees (backward, forward, full) does Thrift offer? > > Best regards, > > Martijn > > Op di 7 jun. 2022 om 18:56 schreef Chen Qin : > > > Thanks for the pointers, I moved the proposal to th

Re: [DISCUSS] FLIP-237: Thrift Format Support in Flink

2022-06-07 Thread Chen Qin
gt; https://cwiki.apache.org/confluence/display/FLINK/Flink+Improvement+Proposals#FlinkImprovementProposals-WhatshouldbeincludedinaFLIP > > ? > > > > On Tue, Jun 7, 2022 at 6:41 AM Chen Qin wrote: > > > > > Hi there, > > > > > > I want to kick off the

[DISCUSS] FLIP-237: Thrift Format Support in Flink

2022-06-06 Thread Chen Qin
Hi there, I want to kick off the first round of FLIP-237 : Thrift Format Support discussion. Notice for the area marked as WIP, we are looking for more feedback from folks, those areas would either stay in the

Re: Table API thrift support

2022-05-30 Thread Chen Qin
/display/FLINK/Flink+Improvement+Proposals > > Op zo 29 mei 2022 om 19:06 schreef Chen Qin : > > > Hi there, > > > > We would like to discuss and potentially upstream our thrift support > > patches to flink. > > > > For some context, we have been internally

Table API thrift support

2022-05-29 Thread Chen Qin
Hi there, We would like to discuss and potentially upstream our thrift support patches to flink. For some context, we have been internally patched flink-1.11.2 to support FlinkSQL jobs read/write to thrift encoded kafka source/sink. Over the course of last 12 months, those patches supports a few

[jira] [Created] (FLINK-27726) shad thrift and fb303 in hive connector

2022-05-20 Thread Chen Qin (Jira)
Chen Qin created FLINK-27726: Summary: shad thrift and fb303 in hive connector Key: FLINK-27726 URL: https://issues.apache.org/jira/browse/FLINK-27726 Project: Flink Issue Type: Improvement

FLINK-11746 work

2022-04-11 Thread Chen Qin
Hi there, I would like to reboot discussion on FLINK-11746 work. Over the course of last two years, we managed to run a large number of critical flink apps (table/datastream) with underlying thrift format. It would be great if folks assign this

is task reassignment possible

2022-01-09 Thread Chen Qin
Hi there, We ran multiple large scale applications YARN clusters, one observation were those jobs often CPU skewed due to topology or data skew on subtasks. And for better or worse, the skew leads to a few task managers consuming large vcores while majority task managers consume much less. Our

Re: handle SUSPENDED in ZooKeeperLeaderRetrievalService

2021-04-23 Thread Chen Qin
wrote: > The original log (section) is preferred over rephrasing. > Best, > tison. > > > tison 于2021年4月23日周五 下午1:15写道: > > > Could you show the log about which unhandled exception was thrown? > > > > Best, > > tison. > > > >

Re: handle SUSPENDED in ZooKeeperLeaderRetrievalService

2021-04-22 Thread Chen Qin
e reactions on FLINK-10052 > otherwise > >> it is highly possibly in vain again from my perspective. > >> > >> Best, > >> tison. > >> > >> [1] https://github.com/apache/flink/pull/15675 > >> [2] https://github.com/apache/flink/pull/113

Re: handle SUSPENDED in ZooKeeperLeaderRetrievalService

2021-04-22 Thread Chen Qin
d work > and really hope the result of more stringent load testing. > > After then I think we could revive this ticket. > > > Best, > Yang > > Chen Qin 于2021年4月16日周五 上午2:01写道: > >> Hi there, >> >> Thanks for providing points to related changes and jira.

Re: handle SUSPENDED in ZooKeeperLeaderRetrievalService

2021-04-15 Thread Chen Qin
Hi there, Thanks for providing points to related changes and jira. Some updates from our side, we applied a path by merging FLINK-10052 with master as well as only handling lost state leveraging SessionConnectionStateErrorPolicy FLINK-10052

[jira] [Created] (FLINK-22081) Entropy key not resolved if flink-s3-fs-hadoop is added as a plugin

2021-03-31 Thread Chen Qin (Jira)
Chen Qin created FLINK-22081: Summary: Entropy key not resolved if flink-s3-fs-hadoop is added as a plugin Key: FLINK-22081 URL: https://issues.apache.org/jira/browse/FLINK-22081 Project: Flink

flink 1.11 class loading question

2021-03-13 Thread Chen Qin
Hi there, We were using flink 1.11.2 in production with a large setting. The job runs fine for a couple of days and ends up with a restart loop caused by YARN container memory kill. This is not observed while running against 1.9.1 with the same setting. Here is JVM environment passed to 1.11 as

Hive Streaming write compaction

2020-11-18 Thread Chen Qin
Hi there, We are testing out writing Kafka to hive table as parquet format. Currently, we have seen user has to choose to create lots of small files in min level folder to gain latency benefits. I recall FF2020 Global folks mentioned implement compaction logic during the checkpointing time.

Re: thrift support

2020-07-22 Thread Chen Qin
support both (de)ser with > > > > thrift format, and accessing > > > > the thrift RPC service. > > > > > > > > Jeff Zhang 于2020年7月19日周日 上午9:43写道: > > > > > > > >> Hi Chen, > > > >> > > > >>

Re: thrift support

2020-07-20 Thread Chen Qin
t the > > desr/ser work ? Giving a concrete example or usage scenario would be > > helpful. > > > > > > > > Chen Qin 于2020年7月18日周六 下午11:09写道: > > > > > Jeff, > > > > > > Are you referring something like this SPIP? > >

Re: thrift support

2020-07-18 Thread Chen Qin
areas and align. Chen On Sat, Jul 18, 2020 at 6:41 AM Jeff Zhang wrote: > Hi Chen, > > Are building something like hive thrift server ? > > Chen Qin 于2020年7月18日周六 上午8:50写道: > > > Hi there, > > > > Here in Pinterest, we utilize thrift end to end in our tech

thrift support

2020-07-17 Thread Chen Qin
Hi there, Here in Pinterest, we utilize thrift end to end in our tech stack. As we have been building Flink as a service platform, the team spent time working on supporting Flink jobs with thrift format and successfully launched a good number of important jobs in Production in H1. In H2, we are

Re: [Discuss] IntervalJoin one side sorted cache

2020-03-06 Thread Chen Qin
lier, > why not just change [lowerbound, upbound] shorter directly? > Looking forward to your response, thanks. > > > ---------- > 发件人:Chen Qin > 发送时间:2020年3月6日(星期五) 10:54 > 收件人:dev > 主 题:[Discuss] IntervalJoin one si

[Discuss] IntervalJoin one side sorted cache

2020-03-05 Thread Chen Qin
for that post arrives, flink save cost of rocksdb access. Thanks, Chen Qin

[jira] [Created] (FLINK-16392) oneside sorted cache in intervaljoin

2020-03-02 Thread Chen Qin (Jira)
Chen Qin created FLINK-16392: Summary: oneside sorted cache in intervaljoin Key: FLINK-16392 URL: https://issues.apache.org/jira/browse/FLINK-16392 Project: Flink Issue Type: Improvement

Re: FsStateBackend vs RocksDBStateBackend

2020-02-29 Thread Chen Qin
nchronous checkpoints enabled, they will also slow down the > processing. > > > On Sun, Feb 23, 2020 at 8:27 PM Chen Qin <mailto:c...@pinterest.com>> wrote: > Just follow up on this thread, it accurately caused by key skew. Given single > subtask is single threaded 5%

Large intervaljoin related question

2019-12-13 Thread Chen Qin
Hi there, We had seen growing interest of using large window and interval join operation. What is recommended way of handling these use cases?(e.g DeltaLake in Spark) After some benchmark, we found performance seems a bottleneck (still) on support those use cases. How is performance

Re: set job level TTL

2019-11-27 Thread Chen Qin
aml so that we could also achieve #1 but leave window state > not > > limited to TTL. > > I think this deserves an issue to track this request. > > > > Best > > Yun Tang > > > > On 11/25/19, 8:04 AM, "Chen Qin" wrote: > > > > I am a bi

Re: set job level TTL

2019-11-24 Thread Chen Qin
tting is per-state scoped, and there is no > job-level TTL setting so far. Do you want similar settings in flink-conf to > enable Flink SQL could also benefit from state TTL? > > Best > Yun Tang > > On 11/25/19, 12:16 AM, "Chen Qin" wrote: > > Hi there, >

set job level TTL

2019-11-24 Thread Chen Qin
Hi there, We have use case of allowing ad hoc user submit a Flink job (mostly SQL) and user come back and verify results after period of time (eg 24hours) without manual intervention. The idea is to have something like set job level TTL configuration to 24hours. Jobmanager will honor 24hours of

Re: [DISCUSS] FLIP-39: Flink ML pipeline and ML libs

2019-04-28 Thread Chen Qin
Just share some of insights from operating SparkML side at scale - map reduce may not best way to iterative sync partitioned workers. - native hardware accelerations is key to adopt rapid changes in ML improvements in foreseeable future. Chen On Apr 29, 2019, at 11:02, jincheng sun wrote: >

Re: Hbase state backend in Flink

2018-12-27 Thread Chen Qin
wse/HBASE-17743> and the design doc > there for more details. And for sure we will upstream our work when ready > to (smile). > > Best Regards, > Yu > > > On Fri, 28 Dec 2018 at 13:12, Chen Qin wrote: > >> Hi Naveen, >> >> AFAIK, there are two

Re: Hbase state backend in Flink

2018-12-27 Thread Chen Qin
Hi Naveen, AFAIK, there are two level of storage in typical statebackend (local/remote). I think it kinda similar to what PC main memory and disk analogy. Take RocksDB Statebackend as example, window state (typical very large ListState) persisted in partitioned local rocksdb files, adding

Re: [DISCUSS] Embracing Table API in Flink ML

2018-11-20 Thread Chen Qin
Hi Yun, Very excited to see Flink ML forward! There are many touch points your document touched. I couldn't agree more the value of having a (unified) table API could bring to Flink ecosystem towards running ML workload. Most ML pipelines we observed starts from single box python scripts or adhoc

Re: [ANNOUNCE] New committer Gary Yao

2018-09-07 Thread Chen Qin
Congrats! Chen > On Sep 7, 2018, at 10:51, Xingcan Cui wrote: > > Congratulations, Gary! > > Xingcan > >> On Sep 7, 2018, at 11:20 PM, Hequn Cheng wrote: >> >> Congratulations Gary! >> >> Hequn >> >>> On Fri, Sep 7, 2018 at 11:16 PM Matthias J. Sax wrote: >>> >>> Congrats! >>> On

Re: is Flink's recovery speed still slow?

2018-07-24 Thread Chen Qin
As far as I learned from folks with better understanding than myself , barrier alignment might be only path to get deterministic output. Any state or outcome between barrier alignments requires second thought(like UDP packages from network). Currently, alignment is used only do heavyweight

Re: [ANNOUNCE] New committer: Sihua Zhou

2018-06-22 Thread Chen Qin
Congrats! > On Jun 22, 2018, at 9:48 AM, Ted Yu wrote: > > Congratulations Sihua! > >> On Fri, Jun 22, 2018 at 6:42 AM, zhangminglei <18717838...@163.com> wrote: >> >> Congrats! Sihua >> >> Cheers >> Minglei. >> >>> 在 2018年6月22日,下午9:17,Till Rohrmann 写道: >>> >>> Hi everybody, >>> >>> On

Re: [ANNOUNCE] New committer: Haohui Mai

2017-11-01 Thread Chen Qin
Congratulations! On Wed, Nov 1, 2017 at 2:41 AM, Aljoscha Krettek wrote: > Congratulations!  > > > On 1. Nov 2017, at 10:13, Shaoxuan Wang wrote: > > > > Congratulations! > > > > On Wed, Nov 1, 2017 at 4:36 PM, Till Rohrmann >

[jira] [Created] (FLINK-7954) sideoutput in async function

2017-10-31 Thread Chen Qin (JIRA)
Chen Qin created FLINK-7954: --- Summary: sideoutput in async function Key: FLINK-7954 URL: https://issues.apache.org/jira/browse/FLINK-7954 Project: Flink Issue Type: Improvement

Re: Unable to write snapshots to S3 on EMR

2017-10-07 Thread Chen Qin
Attached my side project verified working to deploy jobmanager and taskmanager as stateless service(non yarn/mesos), configuration here https://github.com/chenqin/flink-jar/tree/master/config/hadoop more detail here

Re: partial upgrade

2017-10-03 Thread Chen Qin
I would recommend a different architecture by using connect() > and stream in a new logic dynamically. This is especially interesting for > ML models etc. > > Regards, > Timo > > > Am 10/1/17 um 3:03 AM schrieb Chen Qin: > >> Hi there, >> >> So far, fli

partial upgrade

2017-09-30 Thread Chen Qin
Hi there, So far, flink job is interpreted and deployed during bootstrap phase. Once pipeline runs, it's very hard to do partial upgrade without stop execution. (like savepoint is heavy) Is there any plan to allow upload annotated jar package which hints which stream tasks implementation CAN BE

[jira] [Created] (FLINK-7635) support sideOutput in ProcessWindowFunciton & ProcessAllWindowFunction

2017-09-17 Thread Chen Qin (JIRA)
Chen Qin created FLINK-7635: --- Summary: support sideOutput in ProcessWindowFunciton & ProcessAllWindowFunction Key: FLINK-7635 URL: https://issues.apache.org/jira/browse/FLINK-7635 Project: F

Re: [DISCUSS] Flink 1.4 and time based release

2017-08-22 Thread Chen Qin
I would be great to avoid immediate 1.x1 bug fixing release. It cause confusion and raise quality concerns. Also, is there already way to communicate with Amazon EMR for latest release speedy available? I may try to find someone work there is needed. Thanks Chen > On Aug 22, 2017, at 9:32 AM,

Re: Make SubmittedJobGraphStore configurable

2017-07-25 Thread Chen Qin
> On Jul 25, 2017, at 6:07 AM, Till Rohrmann <trohrm...@apache.org> wrote: > > If there is a need for this, then we can definitely make this configurable. > The interface SubmittedJobGraphStore is already there. > > Cheers, > Till > > >> On Fri, Jul 7, 2017 a

Re: Make SubmittedJobGraphStore configurable

2017-07-06 Thread Chen Qin
re broken into multiple lines. > > Can you send the config again with one config on one line ? > > Cheers > > On Wed, Jul 5, 2017 at 10:19 PM, Chen Qin <qinnc...@gmail.com> wrote: > > > ​Hi there, > > > > ​I would like to propose/discuss median level refacto

Make SubmittedJobGraphStore configurable

2017-07-05 Thread Chen Qin
​Hi there, ​I would like to propose/discuss median level refactor work to make submittedJobGraphStore configurable and extensible. The rationale behind is to allow users offload those meta data to durable cross dc read after write strong consistency storage and decouple with zk quorum. ​

[jira] [Created] (FLINK-7106) Make SubmittedJobGraphStore configurable

2017-07-05 Thread Chen Qin (JIRA)
Chen Qin created FLINK-7106: --- Summary: Make SubmittedJobGraphStore configurable Key: FLINK-7106 URL: https://issues.apache.org/jira/browse/FLINK-7106 Project: Flink Issue Type: Improvement

Re: Flink as a Service (FaaS)

2017-03-24 Thread Chen Qin
Here is a working draft doc, feel free to comment out :) https://docs.google.com/document/d/1MSsTOz7xUu50dAf_8v3gsQFfJFFy9LKnULdIl26yj0o/edit?usp=sharing On Thu, Mar 23, 2017 at 5:00 PM, Chen Qin <qinnc...@gmail.com> wrote: > Quick capture comments on FLINK-6085, we want to have r

Re: Flink as a Service (FaaS)

2017-03-23 Thread Chen Qin
Quick capture comments on FLINK-6085, we want to have rpc source that accept requests from clients and reroute response (callback to corresponding rpc source) ​ On Tue, Mar 21, 2017 at 10:47 PM, Chen Qin <qinnc...@gmail.com> wrote: > Hi Radu/jinkui, > > Thanks for your inp

Re: Flink as a Service (FaaS)

2017-03-21 Thread Chen Qin
ink API Gateway. I would propose to support a couple of calls > option to ingest also events. I am thinking of: > - callback mechanism > - REST > - RPC > > > > > -Original Message- > From: Chen Qin [mailto:qinnc...@gmail.com] > Sent: Wednesday, March 15, 20

Re: [ANNOUNCE] New committer: Theodore Vasiloudis

2017-03-21 Thread Chen Qin
Congrats and looking forward to see great stuff in flink ml side! Chen > On Mar 21, 2017, at 19:13, 伍翀(云邪) wrote: > > Congrats, Theo! > >> 在 2017年3月22日,上午8:30,Chiwan Park 写道: >> >> Congratulations, Theo! >> >> Regards, >> Chiwan Park >>

[jira] [Created] (FLINK-6113) Implement split/select with OutputTag

2017-03-18 Thread Chen Qin (JIRA)
Chen Qin created FLINK-6113: --- Summary: Implement split/select with OutputTag Key: FLINK-6113 URL: https://issues.apache.org/jira/browse/FLINK-6113 Project: Flink Issue Type: Improvement

Re: Machine Learning on Flink - Next steps

2017-03-17 Thread Chen Qin
Hi there, I am not a machine learning expert :) But in recent, I see more and more adoption and trends towards tensor flow[1], which is backed by google and big vendors. If flink and somehow compatible and run tensor flow pipelines (with some modifications is fine) I think the adoption would be

[jira] [Created] (FLINK-6085) flink as micro service

2017-03-16 Thread Chen Qin (JIRA)
Chen Qin created FLINK-6085: --- Summary: flink as micro service Key: FLINK-6085 URL: https://issues.apache.org/jira/browse/FLINK-6085 Project: Flink Issue Type: Improvement Components

Re: Flink as a Service (FaaS)

2017-03-15 Thread Chen Qin
jin...@huawei.com> wrote: > Hi, Chen Qin > > We also met your end-to-end use case. A RPC Source and Sink such as netty > source sink can fit such requirements. I’ve submit a natty module in > bahir-flink project which only a demo. > If use connector source instead of Kafka,

Flink as a Service (FaaS)

2017-03-14 Thread Chen Qin
Hi there, I am very happy about Flink 1.2 release. It was much more robust and feature rich compare to previous versions. In the following section, I would like to discuss a non typical use case in flink community. With ever increasing popularity of micro services[1] to scale out popular online

Re: [DISCUSS] Side Outputs and Split/Select

2017-02-24 Thread Chen Qin
Hi Jamie, I think it does make consuming late arriving events more explicit! At cost of fix a predefined OutputTag which user have no control nor definition an extra UDF which essentially filter out all mainOutputs and only let sideOutput pass (like filterFunction) Thanks, Chen > On Feb 24,

Re: [ANNOUNCE] Welcome Jark Wu and Kostas Kloudas as committers

2017-02-07 Thread Chen Qin
Congrats! > On Feb 7, 2017, at 17:52, Zhuoluo Yang wrote: > > Congrats! Good job guys! > > Thanks, > > Zhuoluo  > > > > > >> 在 2017年2月8日,上午4:59,Greg Hogan 写道: >> >> Welcome Jark and Kostas! Thank you for your contributions and many

Re: States split over to external storage

2017-01-25 Thread Chen Qin
On Fri, Jan 20, 2017 at 3:25 AM, Stephan Ewen <se...@apache.org> wrote: > There are works on different approaches of incremental policies underways > (more soon in some design proposals), > but the point raised here sounded different to me. > > Maybe Chen Qin can describe in

Re: 答复: States split over to external storage

2017-01-17 Thread Chen Qin
egy. In original idea, we plan to only backup > the new coming element, and that means a whole window may span several > checkpoints, and we have develop this idea in our private SPS. But in > Flink, the window may not keep raw data(for example, ReducingState and > FoldingState). The idea of C

States split over to external storage

2017-01-16 Thread Chen Qin
thing of keeping HDFS to store snapshots would be avoid versioning conflicts. Once checkpoint restore happens, partial write data will be overwritten with previously checkpointed value. Comments? -- -Chen Qin

Minor Cleanup of StreamGraphGeneator

2016-12-12 Thread Chen Qin
Hi There, I am thinking of doing a minor refactor of StreamGraphGenerator. There are some transforms share same logic that can potentially reusable. Also, long list of if else seems hard to read, might better off abstract interface & offer default implementation. Bonus point would be allow

Re: [Discuss] FLIP-13 Side Outputs in Flink

2016-12-09 Thread Chen Qin
Dear Flink community members, Please review and comment on https://github.com/apache/flink/pull/2982. Thanks, Chen -- View this message in context: http://apache-flink-mailing-list-archive.1008284.n3.nabble.com/Discuss-FLIP-13-Side-Outputs-in-Flink-tp14204p14938.html Sent from the Apache

Re: [Discuss] State Backend use external HBase storage

2016-11-19 Thread Chen Qin
kui...@163.com> wrote: > Hi, Chen Qin > I fount this issue. Does it kicked off? What’s the current progress? > https://issues.apache.org/jira/browse/FLINK-4266 > > On Nov 16, 2016, at 19:35, Till Rohrmann <trohrm...@apache.org> wrote: > > Hi Jinkui, > > the fi

Re: [Discuss] FLIP-13 Side Outputs in Flink

2016-11-03 Thread Chen Qin
that implement Collector interface from graph generator related classes. For streamrecord ones, we might be able to implement *collect(T out)* by calling * void collect(OutputTag tag, S value). *For graph generator keep it as it is. On Wed, Nov 2, 2016 at 8:14 PM, Chen Qin <c...@uber.com> wrote: >

Re: [Discuss] FLIP-13 Side Outputs in Flink

2016-11-02 Thread Chen Qin
internals of the DataStream API, so I leave comments on that to other. Best, Fabian On Tue, Oct 25, 2016 at 9:00 AM, Chen Qin <c...@uber.com> wrote: > Hey folks, > > Please give feedback on FLIP-13! > https://cwiki.apache.org/confluence/display/FLINK/FLIP- > 13+Side+Out

[Discuss] FLIP-13 Side Outputs in Flink

2016-10-25 Thread Chen Qin
Hey folks, Please give feedback on FLIP-13! https://cwiki.apache.org/confluence/display/FLINK/FLIP-13+Side+Outputs+in+Flink JIRA task link to google doc https://issues.apache.org/jira/browse/FLINK-4460 Thanks, Chen Qin

Re: expose side output stream

2016-08-13 Thread Chen Qin
he.org/confluence/display/FLINK/ > Flink+Improvement+Proposals > > > > Greetings, > > Stephan > > > > > > On Thu, Aug 11, 2016 at 12:41 AM, Chen Qin <qinnc...@gmail.com> wrote: > > > > > Hi there, > > > > > > I am th

Re: expose side output stream

2016-08-11 Thread Chen Qin
roposals > > Greetings, > Stephan > > > On Thu, Aug 11, 2016 at 12:41 AM, Chen Qin <qinnc...@gmail.com> wrote: > > > Hi there, > > > > I am thinking of implement sideOutput into Flink which seems missing > > support. > > https://cloud.google.com/da

expose side output stream

2016-08-10 Thread Chen Qin
Hi there, I am thinking of implement sideOutput into Flink which seems missing support. https://cloud.google.com/dataflow/model/par-do#side-outputs It is useful because it will help pipeline author redirect corrputed input/ code bug to a side stream or write to a table and reconsile afterwards.

Re: Cassandra statebackend

2016-08-08 Thread Chen Qin
as. > > > > We might have to revisit the specifics once the re-sharding/key-group > > changes are in Flink and once you actually want to start working on this. > > > > Cheers, > > Aljoscha > > > > On Sat, 6 Aug 2016 at 07:32 Chen Qin <qinnc...@gmail.co

Re: Cassandra statebackend

2016-08-05 Thread Chen Qin
have a > StateBackend component running in the JobManager that we can invoke to > delete ranges of checkpoints. Right now, a StateBackend only lives on the > TaskManager, in the operators. Cleanup of time ranges, however, should > probably happen in some centralized location. > > Cheer

Re: Cassandra statebackend

2016-07-25 Thread Chen Qin
Aljoscha > > On Mon, 25 Jul 2016 at 08:31 Tai Gordon <tzuli...@gmail.com> wrote: > > > Hi Chen, > > > > AFAIK, there currently isn’t any FLIP / JIRA / work currently for a > > Cassandra state backend. I think it’ll definitely by interesting to have > > one in Fli

[jira] [Created] (FLINK-4266) Cassandra Statebackend

2016-07-25 Thread Chen Qin (JIRA)
Chen Qin created FLINK-4266: --- Summary: Cassandra Statebackend Key: FLINK-4266 URL: https://issues.apache.org/jira/browse/FLINK-4266 Project: Flink Issue Type: New Feature Components

Cassandra statebackend

2016-07-24 Thread Chen Qin
​Hi there, Is there any design docs or on going efforts there? Thanks, Chen ​

Re: [DISCUSS] Allowed Lateness in Flink

2016-07-18 Thread Chen Qin
to > > the > > > elements that are dropped as late. Right now, the section just mentions > > > that we have to do this but there is no real proposal yet for how to do > > it. > > > Only a rough sketch so that we don't forget about it. > > > > > >

Re: [DISCUSS] Allowed Lateness in Flink

2016-07-18 Thread Chen Qin
2+Extending+Window+Function+Metadata > With > this, the WindowFunction can get meta data about the window firing so it > could be informed that this is the last firing before a cleanup and that > there already was an earlier, on-time firing. > > Does this cover your concerns, Chen?

custom control messages from source

2016-07-17 Thread Chen Qin
Hi there, So far, checkpoint trigger is hardcoded in CheckpointCorrdinator which triggered periodically and push control messages to task managers. It was implemented orthogonal to business logics implemented in jobs. Our scenario requires master pipeline flow control messages along with events

Re: [DISCUSS] Allowed Lateness in Flink

2016-07-07 Thread Chen Qin
+1 for allowedLateness scenario. The rationale behind is there are backfills or data issues hold in-window data till watermark pass end time. It cause sink write partial output. Allow high allowedLateness threshold makes life easier to merge those results and overwrite partial output with

support launch more than one pipeline in a yarn session

2016-06-27 Thread Chen Qin
Hi there, We are researching launch more than one pipeline in a flink yarn session. It's sort of like cluster mode where user pick off multiple pipelines on a job manager. Is there any plan to support this use case? If not, why not? Is it because of better isolation per pipeline or some

Re: incremental Checkpointing , Rocksdb HA

2016-06-18 Thread Chen Qin
gt; > for > > > information to schedule spark tasks. > > > > > > On 9 June 2016 at 19:55, Nick Dimiduk <ndimi...@gmail.com> wrote: > > > > > > > You might also consider support for a Bigtable > > > > backend: HBase/Accumulo/Cassandra. The dat

incremental Checkpointing , Rocksdb HA

2016-06-09 Thread Chen Qin
Hi there, What is progress on incremental checkpointing? Does flink dev has plan to work on this or JIRA to track this? super interested to know. I also research and consider use rocksdbstatebackend without running HDFS cluster nor talk to S3. Some primitive idea is to use ZK to store / notify