Re: [DISCUSS] Releasing Flink 1.6.4

2019-02-12 Thread fudian.fd
+1

> 在 2019年2月12日,下午5:05,Ufuk Celebi  写道:
> 
> +1 to release 1.6.4
> 
> On Tue, Feb 12, 2019 at 10:01 AM jincheng sun  
> wrote:
>> 
>> Hi Flink devs,
>> 
>> It has been a long time since the release of 1.6.3 (December 23, 2018).
>> There have been a lot of valuable bug fixes during this period.
>> What do you think about releasing Flink 1.6.4 soon?
>> 
>> We already have some critical fixes in the release-1.6 branch(such as):
>> 
>> - FLINK-11235: Solve Elasticsearch connector thread leaks
>> - FLINK-11207: security vulnerability with currently used
>> Apachecommons-compress version
>> - FLINK-10761: do not acquire lock for getAllVariables
>> - FLINK-10761: potential deadlock with metrics system
>> - FLINK-11140: fix empty child path check in Buckets
>> - FLINK-10774: connection leak in FlinkKafkaConsumer
>> - FLINK-10848: problem with resource allocation in YARN mode
>> - FLINK-11419: restore issue with StreamingFileSink
>> - FLINK-10774: connection leak in FlinkKafkaConsumer
>> 
>> Please let me know what you think. Ideally, we can kick off the release
>> vote for the first RC early next week.
>> 
>> I have a preliminary analysis of the JIRAs on 1.6.4. There are currently 4
>> in progress, and there are 23 need to do.
>> I have write the Google doc
>> 
>> for how to processing of these JIRAs. Welcome to comment in the email or in
>> the Google doc
>> 
>> .
>> 
>> If there are some other critical fixes for 1.6.4 that are almost completed
>> (already have a PR opened and review is in progress),
>> please let me know here by the end of this week.
>> 
>> Cheers,
>> Jincheng



smime.p7s
Description: S/MIME cryptographic signature


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

2019-01-28 Thread fudian.fd
Hi Jark,

Thanks a lot for starting the discussion! It would be great to have an official 
Flink Chinese doc.  For the long term maintaining problem, I think creating a 
JIRA when the English documentation is updated is a good idea. Should we add 
one item such as "Does this pull request updated the documentation? If yes, is 
the Chinese documentation is updated?" in the pull-request-template to remind 
the contributors to create the JIRA if Chinese documentation should be updated?

Regards,
Dian

> 在 2019年1月29日,上午11:13,jincheng sun  写道:
> 
> Thanks Jark starting this discussion!
> 
> Hi Fabian, very glad to hear that you like this proposal.
> As far as I know, `zh` is the language, `cn` is the territory, such as:
> `zh-cn` representative
> Simplified Chinese (China) ,  `zh-tw` representative Traditional Chinese
> (Taiwan), So i like the naming the mailing list and website as follows:
> 
> maillist: user...@flink.apache.org
> website: https://flink.apache.org/zh/
> 
> BTW:
> I completely agree with translating resources is a great way to contribute
> without writing source code.
> Welcome and thank every contributors to the translation resources!
> 
> Best, Jincheng
> 
> Fabian Hueske  于2019年1月28日周一 下午9:49写道:
> 
>> Hi Jark,
>> 
>> Thank you for starting this discussion!
>> I'm very happy about the various efforts to support the Chinese Flink
>> community.
>> Offering a translated website and documentation gives Flink a lot more
>> reach and will help many users.
>> 
>> I think integrating the website and documentation as subdirectories into
>> the existing website and docs is a very good approach.
>> Regarding the name, does it make sense to keep the URLs in sync with the
>> newly created mailing list (user...@flink.apache.org), i.e.,
>> https://flink.apache.org/zh/  etc?
>> 
>> I think integrating the translated website/documentation into Apache
>> Flink's repositories might also help to grow the number of Chinese non-code
>> contributors.
>> Translating resources is IMO a great way to contribute without writing
>> source code.
>> 
>> I'm very much looking forward to this.
>> 
>> Best, Fabian
>> 
>> Am Mo., 28. Jan. 2019 um 12:59 Uhr schrieb Jark Wu :
>> 
>>> Hi all,
>>> 
>>> In the past year, the Chinese community is working on building a Chinese
>>> translated Flink website (http://flink.apache.org) and documents (
>>> http://ci.apache.org/projects/flink/flink-docs-master/) in order to help
>>> Chinese speaking users. This is http://flink-china.org and it has
>> received
>>> a lot of praise since online.
>>> 
>>> In order to follow the Apache Way and grow Apache Flink community, we
>> want
>>> to contribute it to Apache Flink. It contains two parts to contribute:
>>> (1) the Chinese translated version of the Flink website
>>> (2) the Chinese translated version of the Flink documentation.
>>> 
>>> But there are some questions are up to discuss:
>>> 
>>> ## The Address of the translated version
>>> 
>>> I think we can add a Chinese channel on official flink website, such as "
>>> https://flink.apache.org/cn/";, which is similar as "
>>> http://kylin.apache.org/cn/";. And use "
>>> https://ci.apache.org/projects/flink/flink-docs-zh-master/"; to put the
>>> Chinese translated docs.
>>> 
>>> ## Add a link to the translated version
>>> 
>>> It would be great if we can add links to each other in both Chinese
>> version
>>> and English version. For example, we can add a link to the translated
>>> website on the sidebar of the Flink website. We can also add a dropdown
>>> button for the Chinese document version under the "Pick Docs Version" in
>>> Flink document.
>>> 
>>> ## How to contribute the translation in a long term
>>> 
>>> This is a more important problem. Because translation is a huge and
>>> long-term work. We need a healthy mechanism to ensure the sustainability
>> of
>>> contributions and the quality of translations.
>>> 
>>> I would suggest to put the Chinese version document in flink repo (such
>> as
>>> "doc-zh" folder) and update with the master. Once we modify the English
>>> doc, we have to update the Chinese doc together, or create a JIRA
>> (contains
>>> git commit id refer to the English modification) to do that. This will
>>> increase some workload when we update the doc. But this will keep the
>>> Chinese doc up to date. We can attract more Chinese contributors to help
>>> build the doc. And the modification is small enough and easy to review.
>>> 
>>> Maybe there is a better solution and we can also learn how the other
>>> projects do it.
>>> 
>>> Any feedbacks are welcome!
>>> 
>>> Best,
>>> Jark Wu
>>> 
>> 



smime.p7s
Description: S/MIME cryptographic signature


Re: [DISCUSS] Start a user...@flink.apache.org mailing list for the Chinese-speaking community?

2019-01-24 Thread fudian.fd
+1. I noticed that many folks from China are requesting the JIRA permission in 
the past year. It reflects that more and more developers from China are using 
Flink. A Chinese oriented mailing list will definitely be helpful for the 
growth of Flink in China.


> 在 2019年1月24日,下午7:42,Stephan Ewen  写道:
> 
> +1, a very nice idea
> 
> On Thu, Jan 24, 2019 at 12:41 PM Robert Metzger  wrote:
> 
>> Thanks for your response.
>> 
>> You are right, I'm proposing "user...@flink.apache.org" as the mailing
>> list's name!
>> 
>> On Thu, Jan 24, 2019 at 12:37 PM Tzu-Li (Gordon) Tai 
>> wrote:
>> 
>>> Hi Robert,
>>> 
>>> Thanks a lot for starting this discussion!
>>> 
>>> +1 to a user-zh@flink.a.o mailing list (you mentioned -zh in the title,
>>> but
>>> -cn in the opening email content.
>>> I think -zh would be better as we are establishing the tool for general
>>> Chinese-speaking users).
>>> All dev@ discussions / JIRAs should still be in a single English mailing
>>> list.
>>> 
>>> From what I've seen in the DingTalk Flink user group, there's quite a bit
>>> of activity in forms of user questions and replies.
>>> It would really be great if the Chinese-speaking user community can
>>> actually have these discussions happen in the Apache mailing lists,
>>> so that questions / discussions / replies from developers can be indexed
>>> and searchable.
>>> Moreover, it'll give the community more insight in how active a
>>> Chinese-speaking contributor is helping with user requests,
>>> which in general is a form of contribution that the community always
>> merits
>>> a lot.
>>> 
>>> Cheers,
>>> Gordon
>>> 
>>> On Thu, Jan 24, 2019 at 12:15 PM Robert Metzger 
>>> wrote:
>>> 
 Hey all,
 
 I would like to create a new user support mailing list called "
 user...@flink.apache.org" to cater the Chinese-speaking Flink
>> community.
 
 Why?
 In the last year 24% of the traffic on flink.apache.org came from the
>>> US,
 22% from China. In the last three months, China is at 30%, the US at
>> 20%.
 An additional data point is that there's a Flink DingTalk group with
>> more
 than 5000 members, asking Flink questions.
 I believe that knowledge about Flink should be available in public
>> forums
 (our mailing list), indexable by search engines. If there's a huge
>> demand
 in a Chinese language support, we as a community should provide these
>>> users
 the tools they need, to grow our community and to allow them to follow
>>> the
 Apache way.
 
 Is it possible?
 I believe it is, because a number of other Apache projects are running
 non-English user@ mailing lists.
 Apache OpenOffice, Cocoon, OpenMeetings, CloudStack all have
>> non-English
 lists: http://mail-archives.apache.org/mod_mbox/
 One thing I want to make very clear in this discussion is that all
>>> project
 decisions, developer discussions, JIRA tickets etc. need to happen in
 English, as this is the primary language of the Apache Foundation and
>> our
 community.
 We should also clarify this on the page listing the mailing lists.
 
 How?
 If there is consensus in this discussion thread, I would request the
>> new
 mailing list next Monday.
 In case of discussions, I will start a vote on Monday or when the
 discussions have stopped.
 Then, we should put the new list on our website and start promoting it
>>> (in
 said DingTalk group and on social media).
 
 Let me know what you think about this idea :)
 
 Best,
 Robert
 
 
 PS: In case you are wondering what ZH stands for:
 https://en.wiktionary.org/wiki/ZH
 
>>> 
>> 



smime.p7s
Description: S/MIME cryptographic signature


Re: CEP - Support for multiple pattern

2018-12-16 Thread fudian.fd
Hi Jiayi,

As far as I know, there is no plan to support this feature. But I think it may 
be a very useful feature as it can eliminate the redundant network transmission 
compared to multiple operators to support multiple patterns. You can create an 
issue and we can discuss further about it on the JIRA page.  CC @Dawid

Regards,
Dian

> 在 2018年12月15日,下午5:07,bupt_ljy  写道:
> 
> Hi, all
> It’s actually very common that we construct more than one rule on the same 
> data source. And I’m developing some such kind of features for our businesses 
> and some ideas come up.
> 
> 
> Do we have any plans for supporting multiple patterns in CEP?
> 
> 
> Best,
> Jiayi Liao



smime.p7s
Description: S/MIME cryptographic signature


Re: [DISCUSS] Enhance convenience of TableEnvironment in TableAPI/SQL

2018-12-11 Thread fudian.fd
Hi Timo,

Thanks a lot for sharing the solution so quickly. I have left some comments on 
the JIRA page mainly about the backwards compatibility. Looking forward to your 
reply.

Thanks,
Dian

> 在 2018年12月11日,下午10:48,Timo Walther  写道:
> 
> Hi Dian,
> 
> I proposed a solution that should be backwards compatible and solves our 
> Maven dependency problems in the corresponding issue.
> 
> I'm happy about feedback.
> 
> Regards,
> Timo
> 
> 
> Am 11.12.18 um 11:23 schrieb fudian.fd:
>> Hi Timo,
>> 
>> Thanks a lot for your reply. I think the cause to this problem is that 
>> TableEnvironment.getTableEnvironment() returns the actual TableEnvironment 
>> implementations instead of an interface or an abstract base class. Even the 
>> porting of FLINK-11067 is done, I'm afraid that the problem may still exist. 
>> For example, for batch TableEnvironment, both java.BatchTableEnvironment and 
>> api.BatchTableEnvironment may be prompted for import. Could you share more 
>> information about what you want to do with the 7 TableEnvironments in 
>> FLINK-11067? Especially api.BatchTableEnvironment, 
>> api.StreamTableEnvironment and TableEnvironment.
>> 
>> Thanks,
>> Dian
>> 
>>> 在 2018年12月11日,下午3:41,jincheng sun  写道:
>>> 
>>> Hi Xuefu,
>>> 
>>> Thanks for your feedback, and mention the compatibility issues.
>>> You are right the change will result version incompatibility. And we my
>>> plan it's will be released in the version of 1.8.x.
>>> 
>>> To be frank, we have considered the compatibility approach, which is to
>>> retain the current TableEnvironment, and then create a new one, such as
>>> "GeneralTableEnvironment" for unified abstraction, and then Deprecated the
>>> TableEnvironment. But we feel that the code is not clean enough, and the
>>> long-term goal is that we need to make StreamTableEnvironment and
>>> BatchTableEnvironment transparent to the user, so I tend to release this
>>> change in 1.8.x, keeping the status quo in 1.7.x. What do you think? Any
>>> feedback is welcome!
>>> 
>>> Thanks,
>>> Jincheng
>>> 
>>> 
>>> Zhang, Xuefu  于2018年12月11日周二 下午1:13写道:
>>> 
>>>> Hi Jincheng,
>>>> 
>>>> Thanks for bringing this up. It seems making good sense to me. However,
>>>> one concern I have is about backward compatibility. Could you clarify
>>>> whether existing user program will break with the proposed changes?
>>>> 
>>>> The answer to the question would largely determine when this can be
>>>> introduced.
>>>> 
>>>> Thanks,
>>>> Xuefu
>>>> 
>>>> 
>>>> --
>>>> Sender:jincheng sun 
>>>> Sent at:2018 Dec 10 (Mon) 18:14
>>>> Recipient:dev 
>>>> Subject:[DISCUSS] Enhance convenience of TableEnvironment in TableAPI/SQL
>>>> 
>>>> Hi All,
>>>> 
>>>> According to the feedback from users, the design of TableEnvironment is
>>>> very inconvenient for users, and often mistakenly imported by IDE,
>>>> especially for Java users, such as:
>>>> 
>>>> ExecutionEnvironment env = ...BatchTableEnvironment tEnv =
>>>> TableEnvironment.getTableEnvironment(env);
>>>> 
>>>> The user does not know which BatchTableEnvironment should be imported,
>>>> because there are three implementations of BatchTableEnvironment, shown as
>>>> below:
>>>> 
>>>> 1. org.apache.flink.table.api.BatchTableEnvironment 2.
>>>> org.apache.flink.table.api.java.BatchTableEnvironment 3.
>>>> org.apache.flink.table.api.scala.BatchTableEnvironment
>>>> [image.png]
>>>> 
>>>> 
>>>> This brings unnecessary inconveniences to the flink user. To solve this
>>>> problem, Wei Zhong, Hequn Cheng, Dian Fu, Shaoxuan Wang and myself
>>>> discussed offline a bit and propose to change the inheritance diagram of
>>>> TableEnvironment is shown as follows:
>>>> 1. AbstractTaleEnvironment - rename current TableEnvironment to
>>>> AbstractTableEnvironment, The functionality implemented by Abstract
>>>> TableEnvironment is stream and batch shared.2. TableEnvironment - Create a
>>>> new TableEnvironment(abstract), and defined all methods in
>>>> (java/scala)StreamTableEnvironment and (java/scala)BatchTableEnvironment.
>>>> In the implementation of BatchTableEnviroment and StreamTableEnviroment,
>>>> the unsupported operations will be reported as an error.
>>>> [image.png]
>>>> Then the usage as follows:
>>>> 
>>>> ExecutionEnvironment env = …TableEnvironment tEnv =
>>>> TableEnvironment.getTableEnvironment(env)
>>>> For detailed proposals please refer to the Google doc:
>>>> https://docs.google.com/document/d/1t-AUGuaChADddyJi6e0WLsTDEnf9ZkupvvBiQ4yTTEI/edit?usp=sharing
>>>> 
>>>> Any mail feedback and Google doc comment are welcome.
>>>> 
>>>> Thanks,
>>>> Jincheng
>>>> 
>>>> 



smime.p7s
Description: S/MIME cryptographic signature


Re: [DISCUSS] Enhance convenience of TableEnvironment in TableAPI/SQL

2018-12-11 Thread fudian.fd
Hi Timo,

Thanks a lot for your reply. I think the cause to this problem is that 
TableEnvironment.getTableEnvironment() returns the actual TableEnvironment 
implementations instead of an interface or an abstract base class. Even the 
porting of FLINK-11067 is done, I'm afraid that the problem may still exist. 
For example, for batch TableEnvironment, both java.BatchTableEnvironment and 
api.BatchTableEnvironment may be prompted for import. Could you share more 
information about what you want to do with the 7 TableEnvironments in 
FLINK-11067? Especially api.BatchTableEnvironment, api.StreamTableEnvironment 
and TableEnvironment.

Thanks,
Dian

> 在 2018年12月11日,下午3:41,jincheng sun  写道:
> 
> Hi Xuefu,
> 
> Thanks for your feedback, and mention the compatibility issues.
> You are right the change will result version incompatibility. And we my
> plan it's will be released in the version of 1.8.x.
> 
> To be frank, we have considered the compatibility approach, which is to
> retain the current TableEnvironment, and then create a new one, such as
> "GeneralTableEnvironment" for unified abstraction, and then Deprecated the
> TableEnvironment. But we feel that the code is not clean enough, and the
> long-term goal is that we need to make StreamTableEnvironment and
> BatchTableEnvironment transparent to the user, so I tend to release this
> change in 1.8.x, keeping the status quo in 1.7.x. What do you think? Any
> feedback is welcome!
> 
> Thanks,
> Jincheng
> 
> 
> Zhang, Xuefu  于2018年12月11日周二 下午1:13写道:
> 
>> Hi Jincheng,
>> 
>> Thanks for bringing this up. It seems making good sense to me. However,
>> one concern I have is about backward compatibility. Could you clarify
>> whether existing user program will break with the proposed changes?
>> 
>> The answer to the question would largely determine when this can be
>> introduced.
>> 
>> Thanks,
>> Xuefu
>> 
>> 
>> --
>> Sender:jincheng sun 
>> Sent at:2018 Dec 10 (Mon) 18:14
>> Recipient:dev 
>> Subject:[DISCUSS] Enhance convenience of TableEnvironment in TableAPI/SQL
>> 
>> Hi All,
>> 
>> According to the feedback from users, the design of TableEnvironment is
>> very inconvenient for users, and often mistakenly imported by IDE,
>> especially for Java users, such as:
>> 
>> ExecutionEnvironment env = ...BatchTableEnvironment tEnv =
>> TableEnvironment.getTableEnvironment(env);
>> 
>> The user does not know which BatchTableEnvironment should be imported,
>> because there are three implementations of BatchTableEnvironment, shown as
>> below:
>> 
>> 1. org.apache.flink.table.api.BatchTableEnvironment 2.
>> org.apache.flink.table.api.java.BatchTableEnvironment 3.
>> org.apache.flink.table.api.scala.BatchTableEnvironment
>> [image.png]
>> 
>> 
>> This brings unnecessary inconveniences to the flink user. To solve this
>> problem, Wei Zhong, Hequn Cheng, Dian Fu, Shaoxuan Wang and myself
>> discussed offline a bit and propose to change the inheritance diagram of
>> TableEnvironment is shown as follows:
>> 1. AbstractTaleEnvironment - rename current TableEnvironment to
>> AbstractTableEnvironment, The functionality implemented by Abstract
>> TableEnvironment is stream and batch shared.2. TableEnvironment - Create a
>> new TableEnvironment(abstract), and defined all methods in
>> (java/scala)StreamTableEnvironment and (java/scala)BatchTableEnvironment.
>> In the implementation of BatchTableEnviroment and StreamTableEnviroment,
>> the unsupported operations will be reported as an error.
>> [image.png]
>> Then the usage as follows:
>> 
>> ExecutionEnvironment env = …TableEnvironment tEnv =
>> TableEnvironment.getTableEnvironment(env)
>> For detailed proposals please refer to the Google doc:
>> https://docs.google.com/document/d/1t-AUGuaChADddyJi6e0WLsTDEnf9ZkupvvBiQ4yTTEI/edit?usp=sharing
>> 
>> Any mail feedback and Google doc comment are welcome.
>> 
>> Thanks,
>> Jincheng
>> 
>> 



smime.p7s
Description: S/MIME cryptographic signature