Re: [VOTE] FLIP-123: DDL and DML compatibility for Hive connector

2020-04-14 Thread Rong Rong
Great feature, I kinda like how the dialect / conformance is handled. +1. also could you please update the FLIP-123 link and add a summary/description row in the main page[1] please? Thanks, Rong -- [1] https://cwiki.apache.org/confluence/display/FLINK/Flink+Improvement+Proposals On Tue, Apr 14

Re: [ANNOUNCE] New Committers and PMC member

2020-04-01 Thread Rong Rong
Congratulations to all!!! -- Rong On Wed, Apr 1, 2020 at 2:27 PM Thomas Weise wrote: > Congratulations! > > > On Wed, Apr 1, 2020 at 9:31 AM Fabian Hueske wrote: > > > Congrats everyone! > > > > Cheers, Fabian > > > > Am Mi., 1. Apr. 2020 um 18:26 Uhr schrieb Yun Tang : > > > > > Congratulatio

[jira] [Created] (FLINK-16789) Support JMX RMI via JMXConnectorServer

2020-03-25 Thread Rong Rong (Jira)
Rong Rong created FLINK-16789: - Summary: Support JMX RMI via JMXConnectorServer Key: FLINK-16789 URL: https://issues.apache.org/jira/browse/FLINK-16789 Project: Flink Issue Type: New Feature

Re: [DISCUSS] JMX remote monitoring integration with Flink

2020-03-23 Thread Rong Rong
Thanks @Till for sharing the JIRA information. I thought as well that this should not be an isolated case to our situation. We would continue to follow up on the JIRA ticket. Best, Rong On Fri, Mar 20, 2020 at 7:30 AM Till Rohrmann wrote: > Hi Rong Rong, > > you are right that it JMX

Re: Flink Kafka consumer auto-commit timeout

2020-03-13 Thread Rong Rong
am not sure that would eventually happen. -- Rong On Fri, Mar 13, 2020 at 6:43 AM Aljoscha Krettek wrote: > Thanks for the update! > > On 13.03.20 13:47, Rong Rong wrote: > > 1. I think we have finally pinpointed what the root cause to this issue > is: > > When partitio

[DISCUSS] JMX remote monitoring integration with Flink

2020-03-13 Thread Rong Rong
Hi All, Has anyone tried to manage production Flink applications through JMX remote monitoring & management[1]? We were experimenting to enable JMXRMI on Flink by default in production and would like to share some of our thoughts: ** Is there any straightforward way to dynamically allocate JMXRMI

Re: Flink Kafka consumer auto-commit timeout

2020-03-13 Thread Rong Rong
but somehow still want to avoid a certain amount of data loss. Most of our analytics use cases falls into this category. -- Rong [1] https://issues.apache.org/jira/browse/KAFKA-6362 [2] https://github.com/apache/kafka/pull/4326 On Wed, Mar 11, 2020 at 10:16 AM Aljoscha Krettek wrote: >

Flink Kafka consumer auto-commit timeout

2020-03-08 Thread Rong Rong
Hi All, I would like to bring back this discussion which I saw multiple times in previous ML threads [1], but there seem to have no solution if checkpointing is disabled. All of these ML reported exceptions have one common pattern: > *INFO* org.apache.kafka.clients.consumer.internals.AbstractCoo

Re: Flink dev blog

2020-03-07 Thread Rong Rong
nificant > > >>>>>> portion is spent on polishing the language. If the blog does not > > >>>> require > > >>>>>> such formal and high quality languages, I believe it will make > > >>> things a > > >>>>>

Re: [VOTE] [FLIP-85] Flink Application Mode

2020-03-07 Thread Rong Rong
+1 (binding) -- Rong On Fri, Mar 6, 2020 at 6:43 PM Peter Huang wrote: > Hi Kostas, > > Thanks for the effort of combing our needs with the long term goal of > execution mode of Flink application. > Looking forward to the feature in Flink > > +1 (non-binding) from my side > > > Best Regards > P

Re: Flink dev blog

2020-03-03 Thread Rong Rong
Big +1 on this. Some of these topics are not only for contributors, but would also be super useful for advance users. One topic I can think of in addition is: Security/Kerberos. Echo on Both Seth's idea, we could have both wiki and PR submission: As Robert mentioned - wiki submission would make th

[jira] [Created] (FLINK-16236) Fix YARNSessionFIFOSecuredITCase not loading the correct security context factory

2020-02-22 Thread Rong Rong (Jira)
Rong Rong created FLINK-16236: - Summary: Fix YARNSessionFIFOSecuredITCase not loading the correct security context factory Key: FLINK-16236 URL: https://issues.apache.org/jira/browse/FLINK-16236 Project

Re: [ANNOUNCE] Jingsong Lee becomes a Flink committer

2020-02-21 Thread Rong Rong
Congratulations Jingsong!! Cheers, Rong On Fri, Feb 21, 2020 at 8:45 AM Bowen Li wrote: > Congrats, Jingsong! > > On Fri, Feb 21, 2020 at 7:28 AM Till Rohrmann > wrote: > >> Congratulations Jingsong! >> >> Cheers, >> Till >> >> On Fri, Feb 21, 2020 at 4:03 PM Yun Gao wrote: >> >>> Congr

[jira] [Created] (FLINK-16224) Refine Hadoop Delegation Token based testing framework

2020-02-21 Thread Rong Rong (Jira)
Rong Rong created FLINK-16224: - Summary: Refine Hadoop Delegation Token based testing framework Key: FLINK-16224 URL: https://issues.apache.org/jira/browse/FLINK-16224 Project: Flink Issue Type

Re: [DISCUSS] Improve history server with log support

2020-02-17 Thread Rong Rong
t this is only a thought and still have many details to iron out. We would share the design doc soon, and we would love to hear more of your ideas and looking forward to your feedbacks. Thanks, Rong On Sun, Feb 16, 2020 at 7:02 PM Yang Wang wrote: > Hi Rong Rong, > > > Thanks for s

Re: [DISCUSS] Improve history server with log support

2020-02-14 Thread Rong Rong
> > Best, > Aljoscha > > [1] https://issues.apache.org/jira/browse/FLINK-14317 > > On 13.02.20 03:47, SHI Xiaogang wrote: > > Hi Rong Rong, > > > > Thanks for the proposal. We are also suffering from some pains brought by > > history server. To address them

Re: [VOTE] Support Python ML Pipeline API

2020-02-14 Thread Rong Rong
+1 (binding) On Fri, Feb 14, 2020 at 7:15 AM Zhijiang wrote: > +1 (binding), it is valuable to enhance the python API for supporting ML > well. > > Best, > Zhijiang > > > -- > From:Dian Fu > Send Time:2020 Feb. 14 (Fri.) 15:07 > To

Re: [DISCUSS] Support Python ML Pipeline API

2020-02-12 Thread Rong Rong
Thanks for driving this initiative @Hequn Cheng . Moving towards python based ML is definitely a huge win consider how large the python-ML community is. a big +1 on my side! Regarding the doc, I only left a few comments on the specific APIs. overall the architecture looks very good! Looking forwa

[DISCUSS] Improve history server with log support

2020-02-12 Thread Rong Rong
Hi All, Recently we have been experimenting using Flink’s history server as a centralized debugging service for completed streaming jobs. Specifically, we dynamically generate links to access log files on the YARN host; in the meantime, we use the Flink history server to show job graphs, exceptio

Re: [DISCUSS] Include flink-ml-api and flink-ml-lib in opt

2020-02-10 Thread Rong Rong
can revisit this > easily instead of blocking the improvement for users. What do you think? > > Best, > Hequn > > [1] > http://apache-flink-mailing-list-archive.1008284.n3.nabble.com/DISCUSS-Support-Python-ML-Pipeline-API-td37291.html > > On Sat, Feb 8, 2020 at 1:57 AM Rong Ro

Re: [DISCUSS] Include flink-ml-api and flink-ml-lib in opt

2020-02-07 Thread Rong Rong
CC @Xu Yang Thanks for starting the discussion @Hequn Cheng and sorry for joining the discussion late. I've mainly helped merging the code in flink-ml-api and flink-ml-lib in the past several months. IMO the flink-ml-api are an extension on top of the table API and agree that it should be treat

Re: [VOTE] Integrate Flink Docker image publication into Flink release process

2020-01-29 Thread Rong Rong
+1 -- Rong On Wed, Jan 29, 2020 at 8:51 AM Ismaël Mejía wrote: > +1 (non-binding) > > No more maintenance work for us Patrick! Just kidding :), it was mostly > done by Patrick, all kudos to him. > Just one question, we will be able to still be featured as an official > docker image in this case

[jira] [Created] (FLINK-15561) Improve Kerberos delegation token login

2020-01-12 Thread Rong Rong (Jira)
Rong Rong created FLINK-15561: - Summary: Improve Kerberos delegation token login Key: FLINK-15561 URL: https://issues.apache.org/jira/browse/FLINK-15561 Project: Flink Issue Type: Bug

[jira] [Created] (FLINK-15475) Add isOutputTypeUsed() API to Transformation

2020-01-03 Thread Rong Rong (Jira)
Rong Rong created FLINK-15475: - Summary: Add isOutputTypeUsed() API to Transformation Key: FLINK-15475 URL: https://issues.apache.org/jira/browse/FLINK-15475 Project: Flink Issue Type

Re: [ANNOUNCE] Zhu Zhu becomes a Flink committer

2019-12-14 Thread Rong Rong
Congrats Zhu Zhu :-) -- Rong On Sat, Dec 14, 2019 at 4:47 AM tison wrote: > Congratulations!:) > > Best, > tison. > > > OpenInx 于2019年12月14日周六 下午7:34写道: > > > Congrats Zhu Zhu! > > > > On Sat, Dec 14, 2019 at 2:38 PM Jeff Zhang wrote: > > > > > Congrats, Zhu Zhu! > > > > > > Paul Lam 于2019年1

Re: [VOTE] Multi-topics consuming from KafkaTableSource

2019-12-06 Thread Rong Rong
Hi Leo, Thanks for sharing the JIRA ticket and the idea of supporting multiple Kafka topic in KafkaTableSource. I've also commented on the JIRA ticket so sorry for joining the discussion late. My question was similar to @Jingsong's: 1. can you share some concrete examples on how this could benefi

Re: [DISCUSS] Support configure remote flink jar

2019-11-23 Thread Rong Rong
Thanks @Tison for starting the discussion and sorry for joining so late. Yes, I think this is a very good idea. we already tweak the flink-yarn package internally to support something similar to what @Thomas mentioned: to support registering a Jar that has already uploaded to some DFS (needless to

Re: [ANNOUNCE] Jark Wu is now part of the Flink PMC

2019-11-08 Thread Rong Rong
Congratulations Jark!!! On Fri, Nov 8, 2019 at 10:03 AM Xuefu Z wrote: > Many congratulations, Jark! > > On Fri, Nov 8, 2019 at 2:31 AM wenlong.lwl > wrote: > > > Congratulations Jark, well deserved! > > > > > > Best, > > Wenlong Lyu > > > > On Fri, 8 Nov 2019 at 18:22, tison wrote: > > > > >

Re: [ANNOUNCE] Becket Qin joins the Flink PMC

2019-10-28 Thread Rong Rong
Congratulations Becket!! -- Rong On Mon, Oct 28, 2019, 7:53 AM Jark Wu wrote: > Congratulations Becket! > > Best, > Jark > > On Mon, 28 Oct 2019 at 20:26, Benchao Li wrote: > > > Congratulations Becket. > > > > Dian Fu 于2019年10月28日周一 下午7:22写道: > > > > > Congrats, Becket. > > > > > > > 在 2019年

Re: [VOTE] FLIP-65: New type inference for Table API UDFs

2019-10-21 Thread Rong Rong
+1 (binding) Thanks Timo for driving this. -- Rong On Mon, Oct 21, 2019 at 8:19 AM wrote: > +1 (binding) > > Best, > Xingcan > > -Original Message- > From: jincheng sun > Sent: Monday, October 21, 2019 5:04 > To: dev > Subject: Re: [VOTE] FLIP-65: New type inference for Table API UDF

Re: [DISCUSS] FLIP-72: Introduce Pulsar Connector

2019-10-17 Thread Rong Rong
Hi Yijie, I also agree with Jark on separating the Catalog part into another FLIP. With FLIP-27[1] also in the air, it is also probably great to split and unblock the sink implementation contribution. I would suggest either putting in a detail implementation plan section in the doc, or (maybe too

Re: [DISCUSS] Improve Flink logging with contextual information

2019-10-14 Thread Rong Rong
Hi Gyula, Sorry for the late reply. I think it is definitely a challenge in terms of log visibility. However, for your requirement I think you can customize your Flink job by utilizing a customized log formatter/encoder (e.g. log4j.properties or logback.xml) and a suitable logger implementation.

Re: [COMMITTER] repo locked due to synchronization issues

2019-10-06 Thread Rong Rong
> > > > Thanks, > > Thomas > > > > On Fri, Sep 27, 2019 at 9:29 AM Rong Rong wrote: > > > > > +1 on to state with one recommendation method in the wiki. > > > I haven't encountered this often, so I do not have a preference > regarding

Re: [VOTE] FLIP-66: Support Time Attribute in SQL DDL (#2)

2019-10-04 Thread Rong Rong
inding? Committer votes are > binding :-) > > Thanks, > Timo > > On 29.09.19 02:17, Rong Rong wrote: > > +1 (non-binding). Thanks for the effort and leading the discussions @Jark > > > > -- > > Rong > > > > On Thu, Sep 26, 2019 at 7:36 PM Ji

Re: [VOTE] FLIP-66: Support Time Attribute in SQL DDL (#2)

2019-09-28 Thread Rong Rong
+1 (non-binding). Thanks for the effort and leading the discussions @Jark -- Rong On Thu, Sep 26, 2019 at 7:36 PM JingsongLee wrote: > +1 non-binding > (nit: Add a road map?) > > Best, > Jingsong Lee > > > -- > From:Kurt Young > S

Re: [COMMITTER] repo locked due to synchronization issues

2019-09-27 Thread Rong Rong
+1 on to state with one recommendation method in the wiki. I haven't encountered this often, so I do not have a preference regarding which way to go (Gitbox or Github). However, I've experienced some different issues on both Github and Gitbox when setting up new committer credentials. If possible,

Re: [DISCUSS] Contribute Pulsar Flink connector back to Flink

2019-09-18 Thread Rong Rong
ed a FLIP that describes the current design of the Pulsar > connector: > > > https://docs.google.com/document/d/1rES79eKhkJxrRfQp1b3u8LB2aPaq-6JaDHDPJIA8kMY/edit# > > Please take a look and let me know what you think. > > Thanks, > Yijie > > On Sat, Sep 14, 20

Re: [DISCUSS] Contribute Pulsar Flink connector back to Flink

2019-09-13 Thread Rong Rong
Hi All, Sorry for joining the discussion late and thanks Yijie & Sijie for driving the discussion. I also think the Pulsar connector would be a very valuable addition to Flink. I can also help out a bit on the review side :-) Regarding the timeline, I also share concerns with Becket on the relati

Re: [ANNOUNCE] Zili Chen becomes a Flink committer

2019-09-11 Thread Rong Rong
Congratulations Zili! -- Rong On Wed, Sep 11, 2019 at 6:26 PM Hequn Cheng wrote: > Congratulations! > > Best, Hequn > > On Thu, Sep 12, 2019 at 9:24 AM Jark Wu wrote: > >> Congratulations Zili! >> >> Best, >> Jark >> >> On Wed, 11 Sep 2019 at 23:06, wrote: >> >> > Congratulations, Zili. >> >

Re: [DISCUSS] FLIP-55: Introduction of a Table API Java Expression DSL

2019-09-03 Thread Rong Rong
Thanks for putting together the proposal @Timo and sorry for joining the discussion thread late. I also share the same thought with Fabian on the ease-of-use front. However I was wondering if we need to start the expression design with them? One thing I can think of is: is it possible to support "

Re: [ANNOUNCE] Andrey Zagrebin becomes a Flink committer

2019-08-14 Thread Rong Rong
Congratulations Andrey! On Wed, Aug 14, 2019 at 10:14 PM chaojianok wrote: > Congratulations Andrey! > At 2019-08-14 21:26:37, "Till Rohrmann" wrote: > >Hi everyone, > > > >I'm very happy to announce that Andrey Zagrebin accepted the offer of the > >Flink PMC to become a committer of the Flink

Re: [ANNOUNCE] Hequn becomes a Flink committer

2019-08-07 Thread Rong Rong
Congratulations Hequn, well deserved! -- Rong On Wed, Aug 7, 2019 at 8:30 AM wrote: > Congratulations, Hequn! > > > > *From:* Xintong Song > *Sent:* Wednesday, August 07, 2019 10:41 AM > *To:* dev@flink.apache.org > *Cc:* user > *Subject:* Re: [ANNOUNCE] Hequn becomes a Flink committer > > >

Re: [DISCUSS] Removing the flink-mapr-fs module

2019-07-29 Thread Rong Rong
We've also experienced some issues with our internal JFrog artifactory. I am suspecting some sort of mirroring problem but somehow it only occur to the mapr-fs module. So +1 to remove. On Mon, Jul 29, 2019 at 12:47 PM Stephan Ewen wrote: > It should be fairly straightforward to rewrite the code

Re: Timestamp(timezone) conversion bug in non blink Table/SQL runtime

2019-07-23 Thread Rong Rong
Hi Shuyi, I think there were some discussions in the mailing list [1,2] and JIRA tickets [3,4] that might be related. Since the table-blink planner doesn't produce such error, I think this problem is valid and should be fixed. Thanks, Rong [1] http://apache-flink-user-mailing-list-archive.233605

[jira] [Created] (FLINK-13389) Setting DataStream return type breaks some type conversion between Table and DataStream

2019-07-23 Thread Rong Rong (JIRA)
Rong Rong created FLINK-13389: - Summary: Setting DataStream return type breaks some type conversion between Table and DataStream Key: FLINK-13389 URL: https://issues.apache.org/jira/browse/FLINK-13389

Re: [ANNOUNCE] Kete Young is now part of the Flink PMC

2019-07-23 Thread Rong Rong
Congratulations Kurt!! -- Rong On Tue, Jul 23, 2019 at 7:31 AM zhijiang wrote: > Congrats Kurt! > > Best, > Zhijiang > -- > From:Till Rohrmann > Send Time:2019年7月23日(星期二) 21:08 > To:dev > Subject:Re: [ANNOUNCE] Kete Young is now

Re: [ANNOUNCE] Jiangjie (Becket) Qin has been added as a committer to the Flink project

2019-07-18 Thread Rong Rong
Congratulations Becket! -- Rong On Thu, Jul 18, 2019 at 7:05 AM Xingcan Cui wrote: > Congrats Becket! > > Best, > Xingcan > > On Thu, Jul 18, 2019, 07:17 Dian Fu wrote: > > > Congrats Becket! > > > > > 在 2019年7月18日,下午6:42,Danny Chan 写道: > > > > > >> Congratulations! > > > > > > Best, > > > Da

Re: Confused by the `projectFields` Method in `ProjectableTableSource` Interface

2019-07-13 Thread Rong Rong
see when "PushProjectIntoTableSourceScanRule" gets invoked. -- Rong -- Rong On Fri, Jul 12, 2019 at 8:33 PM Caizhi Weng wrote: > Hi Rong, > > Thanks for your explanation. What I'm wondering when implementing this > interface is that, will `projectFields` be called twice

Re: Confused by the `projectFields` Method in `ProjectableTableSource` Interface

2019-07-12 Thread Rong Rong
Hi Caizhi, from my understanding, the "ProjectableTableSource" interface is used for something like predicator push-down scenarios: where your produced output should be the same as how your SELECT statement requires. For example, in the case of: SourceSchema: {a: Int, b: Double, c: String, d: Lon

Re: [ANNOUNCE] Rong Rong becomes a Flink committer

2019-07-11 Thread Rong Rong
gt;>> Best, >>> >>>> Xingcan >>> >>>> >>> >>>> On Jul 11, 2019, at 1:08 PM, Shuyi Chen wrote: >>> >>>> >>> >>>> Congratulations, Rong! >>> >>>> >>> >>>>

Re: [DISCUSS] Connectors and NULL handling

2019-06-21 Thread Rong Rong
Hi Aljoscha, Sorry for the late reply, I think the solution makes sense. Using the NULL return value to mark a message is corrupted is not a valid way since NULL value has semantic meaning in not just Kafka but also in a lot of other contexts. I was wondering if we can have a more meaningful inte

Re: Flink internals

2019-05-30 Thread Rong Rong
Hi Fan, John, The flink internal link[1] seems to be not updated in the last years. I found out some higher level pages in the official documentation here[2] however they are still very limited. Is there any other well maintained, internal documentations for contributors? I think it might be a go

Re: [SURVEY] Usage of flink-ml and [DISCUSS] Delete flink-ml

2019-05-23 Thread Rong Rong
+1 for the deletion. Also I think it also might be a good idea to update the roadmap for the plan of removal/development since we've reached the consensus on FLIP-39. Thanks, Rong On Wed, May 22, 2019 at 8:26 AM Shaoxuan Wang wrote: > Hi Chesnay, > Yes, you are right. There is not any active

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

2019-05-06 Thread Rong Rong
's table API approach), I will follow up on the discussion thread and the PR on FLIP-23 with you and Boris :-) -- Rong On Mon, May 6, 2019 at 3:30 AM Shaoxuan Wang wrote: > > Thanks for the feedback, Rong and Flavio. > > @Rong Rong > > There's another thread regard

Re: [DISCUSS] FLIP-23 Model Serving

2019-05-05 Thread Rong Rong
scalanlp/breeze [2] https://spark.apache.org/docs/latest/ml-guide.html On Tue, Apr 30, 2019 at 2:33 AM Robert Metzger wrote: > Hey all, > > I'm wondering if somebody on the list can take a look at the PR from > FLIP-23: https://github.com/apache/flink/pull/7446 > > > On Mon, Oct 1,

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

2019-05-01 Thread Rong Rong
Hi Shaoxuan/Weihua, Thanks for the proposal and driving the effort. I also replied to the original discussion thread, and still a +1 on moving towards the ski-learn model. I just left a few comments on the API details and some general questions. Please kindly take a look. There's another thread r

Re: The contradiction between event time and natural time from EventTimeTrigger

2019-04-30 Thread Rong Rong
Hi Zhipeng, Please see my explanation below: >From the default EventTimeTrigger source code, I found that only onElement > method (will judge the watermark) and onEventTime method only have a chance > to trigger TriggerResult.FIRE; > Therefore, the default EventTimeTrigger is assumed and must be

Re: [DISCUSS] [FLINK SQL] External catalog for Confluent Kafka

2019-04-17 Thread Rong Rong
Thanks Artsem for looking into this problem and Thanks Dawid for bringing up the discussion on FLIP-30. We've observe similar scenarios when we also would like to reuse the schema registry of both Kafka stream as well as the raw ingested kafka messages in datalake. FYI another more catalog-oriente

Re: Introducing Flink's Plugin mechanism

2019-04-16 Thread Rong Rong
Hi All, Sorry for joining the discussion late. Thanks Piotrek for initiating this effort. I recall reporting a very similar bug years ago[1] that was not easily solvable at the time, so +1 on this feature goes beyond just FileSystem :-) I think this would definitely be beneficial as a useful way

Re: [ANNOUNCE] Apache Flink 1.8.0 released

2019-04-10 Thread Rong Rong
Congrats! Thanks Aljoscha for being the release manager and all for making the release possible. -- Rong On Wed, Apr 10, 2019 at 4:23 AM Stefan Richter wrote: > Congrats and thanks to Aljoscha for managing the release! > > Best, > Stefan > > > On 10. Apr 2019, at 13:01, Biao Liu wrote: > > >

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

2019-04-04 Thread Rong Rong
+1 (non-binding) * Verified checksums and GPG files matches release files * Verified that the source archives do not contain any binaries * Built the source with Maven to ensure all source files have Apache headers * Checked that all POM files point to the same version * `mvn clean verify` against

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

2019-03-30 Thread Rong Rong
Hi @Aljoscha, Based on the previous commit [1] that adds the random port selection code, it seems like the important part is to unset whatever 'rest.port' setting previously done. I don't think the current way of setting the BIND_PORT actually overrides any existing PORT setting. However, I wasn't

Re: [DISCUSS] FLIP-37: Rework of the Table API Type System (Part 1)

2019-03-28 Thread Rong Rong
Thanks @Timo for starting this effort and preparing the document :-) I took a pass and left some comments. I also very much like the idea of the DataType and LogicalType separation. As explained in the doc, we've also been looking into ways to improve the type system so a huge +1 on our side. One

Re: [DISCUSS] Improvement to Flink Window Operator with Slicing

2019-03-21 Thread Rong Rong
take a look at the initial design document here [1]. Any comments or suggestions are highly appreciated! Thanks, Rong -- [1] https://docs.google.com/document/d/1CvjPJl1Fm1PCpsuuZ4Qc-p_iUzUosBePX_rWNUt8lRw/edit# On Thu, Feb 28, 2019 at 2:24 PM Rong Rong wrote: > Hi Kurt, > > Thanks for

Re: [DISCUSS] Flink Kerberos Improvement

2019-03-21 Thread Rong Rong
e generic solution. Thanks! > Tao Yang > > > ------ > 发件人:Rong Rong > 发送时间:2018年12月19日(星期三) 03:06 > 收件人:dev > 主 题:Re: [DISCUSS] Flink Kerberos Improvement > > Hi Shuyi, > > Yes. I think the impersonation is a very

Re: Backoff strategies for async IO functions?

2019-03-19 Thread Rong Rong
; Cheers, > Till > > On Wed, Mar 13, 2019 at 5:42 PM Rong Rong wrote: > >> Thanks for raising the concern @shuyi and the explanation @konstantin. >> >> Upon glancing on the Flink document, it seems like user have full control >> on the timeout behavior [1]. But u

Re: [DISCUSS] Introduction of a Table API Java Expression DSL

2019-03-19 Thread Rong Rong
Thanks for sharing the initiative of improving Java side Table expression DSL. I agree as in the doc stated that Java DSL was always a "3rd class citizen" and we've run into many hand holding scenarios with our Flink developers trying to get the Stringify syntax working. Overall I am a +1 on this,

[jira] [Created] (FLINK-11936) Remove AuxiliaryConverter pull-in from calcite and fix auxiliary converter issue.

2019-03-15 Thread Rong Rong (JIRA)
Rong Rong created FLINK-11936: - Summary: Remove AuxiliaryConverter pull-in from calcite and fix auxiliary converter issue. Key: FLINK-11936 URL: https://issues.apache.org/jira/browse/FLINK-11936 Project

[jira] [Created] (FLINK-11935) Remove DateTimeUtils pull-in and fix datetime casting problem

2019-03-15 Thread Rong Rong (JIRA)
Rong Rong created FLINK-11935: - Summary: Remove DateTimeUtils pull-in and fix datetime casting problem Key: FLINK-11935 URL: https://issues.apache.org/jira/browse/FLINK-11935 Project: Flink

Re: Backoff strategies for async IO functions?

2019-03-13 Thread Rong Rong
Thanks for raising the concern @shuyi and the explanation @konstantin. Upon glancing on the Flink document, it seems like user have full control on the timeout behavior [1]. But unlike AsyncWaitOperator, it is not straightforward to access the internal state of the operator to, for example, put th

[jira] [Created] (FLINK-11909) Provide default failure/timeout handling strategy for AsyncIO functions

2019-03-13 Thread Rong Rong (JIRA)
Rong Rong created FLINK-11909: - Summary: Provide default failure/timeout handling strategy for AsyncIO functions Key: FLINK-11909 URL: https://issues.apache.org/jira/browse/FLINK-11909 Project: Flink

Re: [DISCUSS] Improvement to Flink Window Operator with Slicing

2019-02-28 Thread Rong Rong
operator internally when we > decide to add new APIs, which you have covered a lot in your proposal. > Actually, the approaches you proposed looks good to me, take it step by > step is a more practical way. > > Best, > Kurt > > > On Fri, Feb 22, 2019 at 2:58 AM Rong Rong wro

Re: [DISCUSS] Adding a mid-term roadmap to the Flink website

2019-02-21 Thread Rong Rong
Rong On Thu, Feb 21, 2019 at 8:50 AM Stephan Ewen wrote: > Hi Rong Rong! > > I would add the security / kerberos threads to the roadmap. They seem to > be advanced enough in the discussions so that there is clarity what will > come. > > For the window operator with slicing, I

Re: [DISCUSS] Improvement to Flink Window Operator with Slicing

2019-02-21 Thread Rong Rong
ows whose window size > can't be divided by step, such as 10 seconds window and slide with 3 > seconds? > > Best, > Kurt > > > On Wed, Jan 30, 2019 at 2:12 AM Fabian Hueske wrote: > > > Thank you Rong! > > The performance of sliding windows is an

Re: [DISCUSS] Adding a mid-term roadmap to the Flink website

2019-02-14 Thread Rong Rong
t;>> Because it is easier to update the roadmap on wiki compared to on flink web >>> site. And I guess we may need to update the roadmap very often at the >>> beginning as there's so many discussions and proposals in community >>> recently. We can move it into fli

Re: [DISCUSS] Flink Kerberos Improvement

2019-02-14 Thread Rong Rong
-list-archive.1008284.n3.nabble.com/DISCUSS-Flink-security-improvements-td21068.html > > On Tue, Dec 18, 2018 at 8:06 PM Rong Rong wrote: > > > Hi Shuyi, > > > > Yes. I think the impersonation is a very much valid question! This can > > actually be considered as

Re: [DISCUSS] Adding a mid-term roadmap to the Flink website

2019-02-13 Thread Rong Rong
Thanks Stephan for the great proposal. This would not only be beneficial for new users but also for contributors to keep track on all upcoming features. I think that better window operator support can also be separately group into its own category, as they affects both future DataStream API and b

[jira] [Created] (FLINK-11589) Introduce service provider pattern for user to dynamically load SecurityFactory classes

2019-02-12 Thread Rong Rong (JIRA)
Rong Rong created FLINK-11589: - Summary: Introduce service provider pattern for user to dynamically load SecurityFactory classes Key: FLINK-11589 URL: https://issues.apache.org/jira/browse/FLINK-11589

Re: [DISCUSS] Contributing Chinese website and docs to Apache Flink

2019-02-11 Thread Rong Rong
Hi all, Thanks @Jark for creating the translation terminology document! I think the idea of keeping some of the terminologies untranslated and keeping such document in place is absolutely good. I actually share the same concern with @Shaoxuan regarding the discrepancy getting larger. The dilemma i

[jira] [Created] (FLINK-11569) Row type does not serialize in to readable format when invoke "toString" method

2019-02-08 Thread Rong Rong (JIRA)
Rong Rong created FLINK-11569: - Summary: Row type does not serialize in to readable format when invoke "toString" method Key: FLINK-11569 URL: https://issues.apache.org/jira/browse/FLINK-11569

Re: [DISCUSS] Contributing Chinese website and docs to Apache Flink

2019-02-06 Thread Rong Rong
+1. Thanks for the proposal @Jark. I think utilizing review-bot will definitely be a good plus. I remember @Robert mentioned that there was an auto checklist functionality, maybe we can utilize that to flag/tag a specific PR is ready for documentation parity review? I would also like to follow up

[jira] [Created] (FLINK-11456) Improve window operator with sliding window assigners

2019-01-29 Thread Rong Rong (JIRA)
Rong Rong created FLINK-11456: - Summary: Improve window operator with sliding window assigners Key: FLINK-11456 URL: https://issues.apache.org/jira/browse/FLINK-11456 Project: Flink Issue Type

Re: [DISCUSS] Improvement to Flink Window Operator with Slicing

2019-01-29 Thread Rong Rong
On Wed, Dec 5, 2018 at 6:17 PM Rong Rong wrote: > Hi all, > > Various discussion in the mailing list & JIRA tickets [2] had been brought > up in the past regarding the windowing operation performance. As we > experiment internally with some of our extreme use cases, we found

[jira] [Created] (FLINK-11455) Support evictor operations on slicing and merging operators

2019-01-29 Thread Rong Rong (JIRA)
Rong Rong created FLINK-11455: - Summary: Support evictor operations on slicing and merging operators Key: FLINK-11455 URL: https://issues.apache.org/jira/browse/FLINK-11455 Project: Flink Issue

[jira] [Created] (FLINK-11454) Support MergedStream operation

2019-01-29 Thread Rong Rong (JIRA)
Rong Rong created FLINK-11454: - Summary: Support MergedStream operation Key: FLINK-11454 URL: https://issues.apache.org/jira/browse/FLINK-11454 Project: Flink Issue Type: Sub-task

[jira] [Created] (FLINK-11453) Support SliceWindow with forwardable pane info

2019-01-29 Thread Rong Rong (JIRA)
Rong Rong created FLINK-11453: - Summary: Support SliceWindow with forwardable pane info Key: FLINK-11453 URL: https://issues.apache.org/jira/browse/FLINK-11453 Project: Flink Issue Type: Sub

[jira] [Created] (FLINK-11276) Slicing Window Optimization

2019-01-07 Thread Rong Rong (JIRA)
Rong Rong created FLINK-11276: - Summary: Slicing Window Optimization Key: FLINK-11276 URL: https://issues.apache.org/jira/browse/FLINK-11276 Project: Flink Issue Type: New Feature

[jira] [Created] (FLINK-11271) Improve Kerberos Credential Distribution

2019-01-04 Thread Rong Rong (JIRA)
Rong Rong created FLINK-11271: - Summary: Improve Kerberos Credential Distribution Key: FLINK-11271 URL: https://issues.apache.org/jira/browse/FLINK-11271 Project: Flink Issue Type: Improvement

Re: [DISCUSS] Flink Kerberos Improvement

2018-12-18 Thread Rong Rong
n invoke Flink client to deploy the job. Thanks a lot. > > Shuyi > > [1] > > https://docs.google.com/document/d/10V7LiNlUJKeKZ58mkR7oVv1t6BrC6TZi3FGf2Dm6-i8/edit > > On Mon, Dec 17, 2018 at 5:49 PM Rong Rong wrote: > > > Hi All, > > > > We have been exper

[DISCUSS] Flink Kerberos Improvement

2018-12-17 Thread Rong Rong
Hi All, We have been experimenting integration of Kerberos with Flink in our Corp environment and found out some limitations on the current Flink-Kerberos security mechanism running with Apache YARN. Based on the Hadoop Kerberos security guide [1]. Apparently there are only a subset of the sugges

Re: [DISCUSS] Flink SQL DDL Design

2018-12-12 Thread Rong Rong
Thanks for the summary effort @shuyi. Sorry for jumping in the discussion so late. As of the scope of MVP, I think we might want to consider adding "table update mode" problem to it. I agree with @timo that might not be easily changed in the future if the flags has to be part of the schema/column

[jira] [Created] (FLINK-11088) Improve Kerberos Authentication using Keytab in YARN proxy user mode

2018-12-06 Thread Rong Rong (JIRA)
Rong Rong created FLINK-11088: - Summary: Improve Kerberos Authentication using Keytab in YARN proxy user mode Key: FLINK-11088 URL: https://issues.apache.org/jira/browse/FLINK-11088 Project: Flink

[DISCUSS] Improvement to Flink Window Operator with Slicing

2018-12-05 Thread Rong Rong
Hi all, Various discussion in the mailing list & JIRA tickets [2] had been brought up in the past regarding the windowing operation performance. As we experiment internally with some of our extreme use cases, we found out that using a slice-based implementation can optimize Flink's windowing mecha

[jira] [Created] (FLINK-11006) Update Calcite dependency to 1.18

2018-11-26 Thread Rong Rong (JIRA)
Rong Rong created FLINK-11006: - Summary: Update Calcite dependency to 1.18 Key: FLINK-11006 URL: https://issues.apache.org/jira/browse/FLINK-11006 Project: Flink Issue Type: Improvement

Re: [DISCUSS] Enhancing the functionality and productivity of Table API

2018-11-04 Thread Rong Rong
Hi Jincheng, Thank you for the proposal! I think being able to define a process / co-process function in table API definitely opens up a whole new level of applications using a unified API. In addition, as Tzu-Li and Hequn have mentioned, the benefit of optimization layer of Table API will alread

Re: [DISCUSS] Flink SQL DDL Design

2018-11-03 Thread Rong Rong
+1. Thanks for putting the proposal together Shuyi. DDL has been brought up in a couple of times previously [1,2]. Utilizing DDL will definitely be a great extension to the current Flink SQL to systematically support some of the previously brought up features such as [3]. And it will also be benef

Re: [DISCUSS] Integrate Flink SQL well with Hive ecosystem

2018-10-11 Thread Rong Rong
Hi Xuefu, Thanks for putting together the overview. I would like to add some more on top of Timo's comments. 1,2. I agree with Timo that a proper catalog support should also address the metadata compatibility issues. I was actually wondering if you are referring to something like utilizing table s

Re: Handling burst I/O when using tumbling/sliding windows

2018-10-10 Thread Rong Rong
easy to do without > the need for API changes. > > Piotrek > > On 1 Oct 2018, at 17:48, Rong Rong wrote: > > Hi Piotrek, > > Thanks for the quick response. To follow up with the questions: > Re 1). Yes it is causing network I/O issues on Kafka itself. > > Re

Re: [DISCUSS] Improvements to the Unified SQL Connector API

2018-10-04 Thread Rong Rong
Hi Timo, Thanks for putting together the proposal! I really love the idea to combining solution for historic and recent data and left some suggestions on that part. Regarding the table type, e.g. for kafka streams, I agree with @hequn's idea that it should be pretty much inferable from the SQL co

Re: [DISCUSS] FLIP-23 Model Serving

2018-10-01 Thread Rong Rong
Thanks for the contribution Boris!! I've been playing around with the basic model for a while back and loved it. +1 and really looking forward to having the feature merging back to Flink ML. -- Rong On Mon, Oct 1, 2018 at 7:55 AM Fabian Hueske wrote: > Hi everybody, > > The question of how to s

  1   2   >