Re: [DISCUSS] FLIP-75: Flink Web UI Improvement Proposal

2020-02-18 Thread lining jing
I think we can create the PR for the GC status later if we could find an easy way to obtain it, before that the users could get GC logs from the FLIP-103. By the way, there is a similar topic 'FlameGraph In Job Vertex' in FLIP-75 in the early discussion stage

Re: [ANNOUNCE] Apache Flink-shaded 10.0 released

2020-02-18 Thread jincheng sun
Thanks a lot for the release Chesnay! And thanks to everyone who make this release possible! Best, Jincheng Chesnay Schepler 于2020年2月19日周三 上午12:45写道: > The Apache Flink community is very happy to announce the release of > Apache Flink-shaded 10.0. > > The flink-shaded project contains a

[jira] [Created] (FLINK-16159) Add simple end-to-end test for Stateful Functions

2020-02-18 Thread Tzu-Li (Gordon) Tai (Jira)
Tzu-Li (Gordon) Tai created FLINK-16159: --- Summary: Add simple end-to-end test for Stateful Functions Key: FLINK-16159 URL: https://issues.apache.org/jira/browse/FLINK-16159 Project: Flink

Re: [DISCUSS] Improvements on FLIP Process

2020-02-18 Thread jincheng sun
Hi all, Thanks for bring up the discussion @Hequn! I agree with some concerns raised above, however, I would like to give my +1 for the proposal and I would like to share my thoughts: If I understand correctly, the proposal doesn’t encourage people to discuss in the google doc, the first step

Re: [DISCUSS] Improvements on FLIP Process

2020-02-18 Thread Yuan Mei
It is difficult to draw a clear cut between small and big issues. Hence I would prefer to stick to only one way for discussion. I would try to avoid Google Docs if having other ways mainly because of two reasons: 1. Google Docs are not always accessible to everyone. 2. Discussion on Google docs

Re: [DISCUSS] Improvements on FLIP Process

2020-02-18 Thread Jingsong Li
Hi all, thanks for launching this discussion. About eliminating Google Docs. I agree with Zhijiang, I share my concern about it. If the FLIP Driver is a Flink newer or the FLIP is very big and complicated. His/Her design maybe need change many many things, in this situation, Google doc is good

Re: [DISCUSS] Kicking off the 1.11 release cycle

2020-02-18 Thread Zhijiang
Thanks for kicking off the next release and the introduction, @Stephan! It's my pleasure to become the release manager and involve in other community works. I am working together with @Piotr for a long time, so very happy to cooperate for the release manager work again. The previous release

[jira] [Created] (FLINK-16158) SqlClient showdown when executing update if not start cluster

2020-02-18 Thread hailong wang (Jira)
hailong wang created FLINK-16158: Summary: SqlClient showdown when executing update if not start cluster Key: FLINK-16158 URL: https://issues.apache.org/jira/browse/FLINK-16158 Project: Flink

Re: Total recovery time estimation after checkpoint recovery

2020-02-18 Thread Woods, Jessica Hui
Hi Till, No, I have not received any emails regarding my question. Could you please forward your response to me? Thanks From: Till Rohrmann Sent: Tuesday, February 18, 2020 4:43 PM To: dev Subject: Re: Total recovery time estimation after checkpoint

[jira] [Created] (FLINK-16157) StateFun benchmark and performance

2020-02-18 Thread Omid B (Jira)
Omid B created FLINK-16157: -- Summary: StateFun benchmark and performance Key: FLINK-16157 URL: https://issues.apache.org/jira/browse/FLINK-16157 Project: Flink Issue Type: Wish

Re: Hotfixes on the master

2020-02-18 Thread Till Rohrmann
Thanks for raising this point Robert. I think it is important to remind the community about the agreed practices once in a while. In most of the cases I had the impression that the majority of the community sticks to the agreed rules. W/o more detailed numbers (how many of the hotfix commits are

[jira] [Created] (FLINK-16156) TableAggregateITCase.testGroupByFlatAggregate failed on Travis

2020-02-18 Thread Till Rohrmann (Jira)
Till Rohrmann created FLINK-16156: - Summary: TableAggregateITCase.testGroupByFlatAggregate failed on Travis Key: FLINK-16156 URL: https://issues.apache.org/jira/browse/FLINK-16156 Project: Flink

[jira] [Created] (FLINK-16154) Translate "Operator/Join" into Chinese

2020-02-18 Thread Yun Gao (Jira)
Yun Gao created FLINK-16154: --- Summary: Translate "Operator/Join" into Chinese Key: FLINK-16154 URL: https://issues.apache.org/jira/browse/FLINK-16154 Project: Flink Issue Type: Sub-task

[jira] [Created] (FLINK-16155) Translate "Operator/Process Function" into Chinese

2020-02-18 Thread Yun Gao (Jira)
Yun Gao created FLINK-16155: --- Summary: Translate "Operator/Process Function" into Chinese Key: FLINK-16155 URL: https://issues.apache.org/jira/browse/FLINK-16155 Project: Flink Issue Type:

[jira] [Created] (FLINK-16153) Translate "Operator/windows" into Chinese

2020-02-18 Thread Yun Gao (Jira)
Yun Gao created FLINK-16153: --- Summary: Translate "Operator/windows" into Chinese Key: FLINK-16153 URL: https://issues.apache.org/jira/browse/FLINK-16153 Project: Flink Issue Type: Sub-task

[jira] [Created] (FLINK-16152) Translate "Operator/index" into Chinese

2020-02-18 Thread Yun Gao (Jira)
Yun Gao created FLINK-16152: --- Summary: Translate "Operator/index" into Chinese Key: FLINK-16152 URL: https://issues.apache.org/jira/browse/FLINK-16152 Project: Flink Issue Type: Sub-task

Re: [DISCUSS] Improvements on FLIP Process

2020-02-18 Thread Zhijiang
Thanks for launching this discussion and also agree with the opinions of Kostas, Timo and Aljoscha. The proposed reasons for eliminating google doc are very reasonable, especially the access limitation for some people in China. Besides that, another conservative option is to make google doc as

[jira] [Created] (FLINK-16151) Translate "Event-time/Pre-defined Timestamp Extractors / Watermark Emitters" into Chinese

2020-02-18 Thread Yun Gao (Jira)
Yun Gao created FLINK-16151: --- Summary: Translate "Event-time/Pre-defined Timestamp Extractors / Watermark Emitters" into Chinese Key: FLINK-16151 URL: https://issues.apache.org/jira/browse/FLINK-16151

[DISCUSS] Kicking off the 1.11 release cycle

2020-02-18 Thread Stephan Ewen
Hi all! Now that the 1.10 release is out (congrats again, everyone!), I wanted to bring up some questions about organizing the next release cycle. The most important questions at the beginning would be - Who would volunteer as Release Managers - When would be the release date. For the

[jira] [Created] (FLINK-16150) Eclipse improvements for archetypes break functionality in vscode-java

2020-02-18 Thread Tristan Tunderman (Jira)
Tristan Tunderman created FLINK-16150: - Summary: Eclipse improvements for archetypes break functionality in vscode-java Key: FLINK-16150 URL: https://issues.apache.org/jira/browse/FLINK-16150

[ANNOUNCE] Apache Flink-shaded 10.0 released

2020-02-18 Thread Chesnay Schepler
The Apache Flink community is very happy to announce the release of Apache Flink-shaded 10.0. The flink-shaded project contains a number of shaded dependencies for Apache Flink. Apache Flink® is an open-source stream processing framework for distributed, high-performing, always-available,

[jira] [Created] (FLINK-16149) Set configurations using StreamExecutionEnvironment#getConfiguration

2020-02-18 Thread Seth Wiesman (Jira)
Seth Wiesman created FLINK-16149: Summary: Set configurations using StreamExecutionEnvironment#getConfiguration Key: FLINK-16149 URL: https://issues.apache.org/jira/browse/FLINK-16149 Project: Flink

Re: Total recovery time estimation after checkpoint recovery

2020-02-18 Thread Till Rohrmann
Hi Jessica, did you receive my previous email with the explanation? Cheers, Till On Sat, Feb 15, 2020 at 11:45 PM Woods, Jessica Hui < jessica.wo...@campus.tu-berlin.de> wrote: > ??Hi, > > I am working with Apache Flink and am interested in knowing how one could > estimate the total amount of

Re: [ANNOUNCE] Apache Flink Python API(PyFlink) 1.9.2 released

2020-02-18 Thread Till Rohrmann
Thanks for updating the 1.9.2 release wrt Flink's Python API Jincheng! Cheers, Till On Thu, Feb 13, 2020 at 12:25 PM Hequn Cheng wrote: > Thanks a lot for the release, Jincheng! > Also thanks to everyone that make this release possible! > > Best, > Hequn > > On Thu, Feb 13, 2020 at 2:18 PM

Re: [DISCUSS] Improvements on FLIP Process

2020-02-18 Thread Kostas Kloudas
+1 to what Aljoscha and Timo are proposing. I would lean towards eliminating Google Docs altogether. I think they served a purpose when discussions were among 3 to 4 people but with the current size of the community and the amount of participants per discussion they become difficult to follow.

Re: [DISCUSS] Improvements on FLIP Process

2020-02-18 Thread Timo Walther
+1 to what Aljoscha said. The past has shown that discussions in Google docs do not reach all interested parties and the tracability of design decisions becomes difficult. Google services are also partially inaccessible in certain parts of world. We should actually do the opposite and not

Re: [DISCUSS] Improvements on FLIP Process

2020-02-18 Thread Aljoscha Krettek
Hi, thanks for starting this discussion! However, I have a somewhat opposing opinion to this: we should disallow using Google Docs for FLIPs and FLIP discussions and follow the already established process more strictly. My reasons for this are: - discussions on the Google Doc are not

[jira] [Created] (FLINK-16148) Update Operations Playground to Flink 1.10.0

2020-02-18 Thread Oleg Bonar (Jira)
Oleg Bonar created FLINK-16148: -- Summary: Update Operations Playground to Flink 1.10.0 Key: FLINK-16148 URL: https://issues.apache.org/jira/browse/FLINK-16148 Project: Flink Issue Type: Task

[jira] [Created] (FLINK-16147) Add watermarkExprOutputType in WatermarkSpec#toString method

2020-02-18 Thread hailong wang (Jira)
hailong wang created FLINK-16147: Summary: Add watermarkExprOutputType in WatermarkSpec#toString method Key: FLINK-16147 URL: https://issues.apache.org/jira/browse/FLINK-16147 Project: Flink

[jira] [Created] (FLINK-16146) Improve end-to-end usability of Flink Table API & SQL

2020-02-18 Thread Jark Wu (Jira)
Jark Wu created FLINK-16146: --- Summary: Improve end-to-end usability of Flink Table API & SQL Key: FLINK-16146 URL: https://issues.apache.org/jira/browse/FLINK-16146 Project: Flink Issue Type:

checkpoint total restart time

2020-02-18 Thread Woods, Jessica Hui
Hi, I am working with Apache Flink and am interested in knowing how one could estimate the total amount of time an application spends in recovery, including the input stream "catch-up" after checkpoint recovery. What I am specifically interested in is knowing the time needed for the recovery

Re: [DISCUSS] Drop connectors for Elasticsearch 2.x and 5.x

2020-02-18 Thread Chesnay Schepler
Since one of the reasons for dropping ES2 was that it blocks some critical updates for the ES5 connector I'd prefer to keep ES5 around for 1.11, and revisit this discussion for 1.12 . On 18/02/2020 13:03, Aljoscha Krettek wrote: Wouldn't removing the ES 2.x connector be enough because we can

Re: [DISCUSS] Drop connectors for Elasticsearch 2.x and 5.x

2020-02-18 Thread Aljoscha Krettek
Wouldn't removing the ES 2.x connector be enough because we can then update the ES 5.x connector? It seems there are some users that still want to use that one. Best, Aljoscha On 18.02.20 10:42, Robert Metzger wrote: The ES5 connector is causing some problems on the CI system. It would be

[jira] [Created] (FLINK-16145) ScheduledUnit toString method throw NPE when Execution is null

2020-02-18 Thread YufeiLiu (Jira)
YufeiLiu created FLINK-16145: Summary: ScheduledUnit toString method throw NPE when Execution is null Key: FLINK-16145 URL: https://issues.apache.org/jira/browse/FLINK-16145 Project: Flink

[jira] [Created] (FLINK-16144) Add client.timeout setting and use that for CLI operations

2020-02-18 Thread Aljoscha Krettek (Jira)
Aljoscha Krettek created FLINK-16144: Summary: Add client.timeout setting and use that for CLI operations Key: FLINK-16144 URL: https://issues.apache.org/jira/browse/FLINK-16144 Project: Flink

[jira] [Created] (FLINK-16143) Turn on more date time functions of blink planner

2020-02-18 Thread Zili Chen (Jira)
Zili Chen created FLINK-16143: - Summary: Turn on more date time functions of blink planner Key: FLINK-16143 URL: https://issues.apache.org/jira/browse/FLINK-16143 Project: Flink Issue Type: New

[jira] [Created] (FLINK-16142) Memory Leak causes Metaspace OOM error on repeated job submission

2020-02-18 Thread Thomas Wozniakowski (Jira)
Thomas Wozniakowski created FLINK-16142: --- Summary: Memory Leak causes Metaspace OOM error on repeated job submission Key: FLINK-16142 URL: https://issues.apache.org/jira/browse/FLINK-16142

Re: [VOTE] FLIP-75: Flink Web UI Improvement Proposal

2020-02-18 Thread Piotr Nowojski
Hi, +1 (binding) for FLIP-98 +1 (binding) for FLIP-103 as well. I second Xintong's worries about searching for something in a large log file, however I agree this probably could be expanded/improved in the future. Piotrek > On 14 Feb 2020, at 03:28, Yadong Xie wrote: > > Hi all, > > I

Re: [DISCUSS] FLIP-75: Flink Web UI Improvement Proposal

2020-02-18 Thread Piotr Nowojski
Hi, A quick question/comment about FLIP-102. Have you thought about adding GC stats? I’m not sure what’s easily do-able, but something that would allow user to see GC issues (long/frequent pauses, lots of CPU time spent in the GC) would be quite useful for analysing performance/stability

[jira] [Created] (FLINK-16141) HiveTableSourceTest unstable

2020-02-18 Thread Stephan Ewen (Jira)
Stephan Ewen created FLINK-16141: Summary: HiveTableSourceTest unstable Key: FLINK-16141 URL: https://issues.apache.org/jira/browse/FLINK-16141 Project: Flink Issue Type: Bug

[jira] [Created] (FLINK-16140) Translate "Event Processing (CEP)" page into Chinese

2020-02-18 Thread shuai.xu (Jira)
shuai.xu created FLINK-16140: Summary: Translate "Event Processing (CEP)" page into Chinese Key: FLINK-16140 URL: https://issues.apache.org/jira/browse/FLINK-16140 Project: Flink Issue Type:

[jira] [Created] (FLINK-16139) Co-location constraints are not reset on task recovery in DefaultScheduler

2020-02-18 Thread Zhu Zhu (Jira)
Zhu Zhu created FLINK-16139: --- Summary: Co-location constraints are not reset on task recovery in DefaultScheduler Key: FLINK-16139 URL: https://issues.apache.org/jira/browse/FLINK-16139 Project: Flink

[jira] [Created] (FLINK-16138) Translate "Overview" page of "DataStream API" into Chinese

2020-02-18 Thread Yun Gao (Jira)
Yun Gao created FLINK-16138: --- Summary: Translate "Overview" page of "DataStream API" into Chinese Key: FLINK-16138 URL: https://issues.apache.org/jira/browse/FLINK-16138 Project: Flink Issue Type:

[jira] [Created] (FLINK-16137) Translate all DataStream API related pages into Chinese

2020-02-18 Thread Yun Gao (Jira)
Yun Gao created FLINK-16137: --- Summary: Translate all DataStream API related pages into Chinese Key: FLINK-16137 URL: https://issues.apache.org/jira/browse/FLINK-16137 Project: Flink Issue Type:

[jira] [Created] (FLINK-16136) YarnPrioritySchedulingITCase.yarnApplication_submissionWithPriority_shouldRespectPriority() fails

2020-02-18 Thread Robert Metzger (Jira)
Robert Metzger created FLINK-16136: -- Summary: YarnPrioritySchedulingITCase.yarnApplication_submissionWithPriority_shouldRespectPriority() fails Key: FLINK-16136 URL:

Re: [DISCUSS] Drop connectors for Elasticsearch 2.x and 5.x

2020-02-18 Thread Robert Metzger
The ES5 connector is causing some problems on the CI system. It would be nice if we could make a decision here soon. I don't want to invest time into fixing it, if we are going to remove it. I'm still in favor of removing it. If we see that there's demand for the 5.x connector after the 1.11

Sharing operator subtask state using side outputs

2020-02-18 Thread vishalovercome
I am implementing a streaming application and one of the stateful operators is trying to capture a “owner has items” relationship. The state, keyed per owner consists of details about the owner and each of the items. Ownership of an item can change and I would like to be able associate each item

[jira] [Created] (FLINK-16135) Unify AutoContextClassLoader and TemporaryClassLoaderContext

2020-02-18 Thread Stephan Ewen (Jira)
Stephan Ewen created FLINK-16135: Summary: Unify AutoContextClassLoader and TemporaryClassLoaderContext Key: FLINK-16135 URL: https://issues.apache.org/jira/browse/FLINK-16135 Project: Flink

[jira] [Created] (FLINK-16134) npm ERR! 429 Too Many Requests

2020-02-18 Thread Piotr Nowojski (Jira)
Piotr Nowojski created FLINK-16134: -- Summary: npm ERR! 429 Too Many Requests Key: FLINK-16134 URL: https://issues.apache.org/jira/browse/FLINK-16134 Project: Flink Issue Type: Bug

Re: [DISCUSS] FLIP-84: Improve & Refactor execute/sqlQuery/sqlUpdate APIS of TableEnvironment

2020-02-18 Thread godfrey he
Thanks Kurt and Jark for explanation, I now also think we should make the TableEnvironment interface more statable and should not change "sqlQuery" method and "from" method. Hi Jingsong. Regarding to the "DmlBatch", I totally agree with advantages of "addBatch" method. However, there are two more

[DISCUSS] Improvements on FLIP Process

2020-02-18 Thread Hequn Cheng
Hi everyone, Currently, when we create a FLIP we follow the FLIP process in the Flink Improvement Proposals wiki[1]. The process mainly includes the following steps: 1. Create a FLIP wiki page. 2. Raise the discussion on the mailing list. 3. Once the proposal is finalized, call a vote to have the