Re: [DISCUSS] CPU flame graph for a job vertex in web UI.

2019-08-02 Thread boshu Zheng
Big +1 for this helpful feature :)


On 08/02/2019 13:54, Jark Wu wrote:
Hi David,

The demo looks charming! I think it will definitely help a lot when
performance tuning.
A big +1 for this.

I cc-ed Yadong who's one of the main contributors of the new Web UI.
Maybe he can give some help on the front end.

Regards,
Jark

On Fri, 2 Aug 2019 at 04:26, David Morávek  wrote:

> Hi Till, thanks for the feedback! These endpoints are only called when the
> vertex is selected in the UI, so there should be any heavy RPC load. For
> back-pressure, we only sample top 3 calls of the stack (depth = 3). For the
> flame-graph, we want to sample the whole stack trace and we need different
> sampling rate (longer period, more samples). Those are the main reasons to
> split these in two "trackers", but I may be missing something.
>
> I've prepared a little demo, so others can have a better idea of what I
> have in mind.
>
> https://youtu.be/GUNDehj9z9o
>
> Please note that this is a proof of concept and I'm not frontend person, so
> it may look little clumsy :)
>
> D.
>
> On Thu, Aug 1, 2019 at 11:40 AM Till Rohrmann 
> wrote:
>
> > Hi David,
> >
> > thanks for starting this discussion. I like the idea of improving
> insights
> > into Flink's execution and I believe that a flame graph could be helpful.
> >
> > I quickly glanced over your changes and I think they go in a good
> > direction. One idea could be to share the `StackTraceSample` produced by
> > the `StackTraceSampleCoordinator` between the different
> > `StackTraceOperatorTracker` so that we don't send multiple requests for
> the
> > same operators. That way we would decrease a bit the RPC load.
> >
> > Apart from that, I think the next steps would be to find a committer who
> > could shepherd this effort and help you with merging it.
> >
> > Cheers,
> > Till
> >
> > On Wed, Jul 31, 2019 at 7:05 PM David Morávek  wrote:
> >
> > > Hello,
> > >
> > > While looking into Flink internals, I've noticed that there is already
> a
> > > mechanism for stack-trace sampling of a particular job vertex.
> > >
> > > I think it may be really useful to allow user to easily render a cpu
> > > flamegraph  in a new UI
> > for
> > > a
> > > selected vertex (new tab next to back pressure) of a running job. Back
> > > pressure tab already provides a good idea of which vertex causes
> trouble,
> > > but it's hard to say what's actually going on.
> > >
> > > I've tried to implement a basic REST endpoint
> > > <
> > >
> >
> https://github.com/dmvk/flink/commit/716231822d2fe99004895cdd0a365560479445b9
> > > >,
> > > that prepares data for the flame graph rendering and it seems to be
> > > providing good insight.
> > >
> > > It should be straightforward to render data from the endpoint in new UI
> > > using existing  javascript
> > > libraries.
> > >
> > > WDYT? Is this worth pushing forward?
> > >
> > > D.
> > >
> >
>


Re: [DISCUSS] CPU flame graph for a job vertex in web UI.

2019-08-02 Thread Paul Lam
Hi David,

Thanks for the new feature! I think the flame graph would be a useful tool to 
understand the state of job executions, and it looks good too. +1 for this.

And a minor question: do we plan to support multiple kinds of flame graphs? It 
would be great if we have both on-cpu and off-cpu flame graphs.

Best,
Paul Lam

> 在 2019年8月2日,04:24,David Morávek  写道:
> 
> Hi Till, thanks for the feedback! These endpoints are only called when the
> vertex is selected in the UI, so there should be any heavy RPC load. For
> back-pressure, we only sample top 3 calls of the stack (depth = 3). For the
> flame-graph, we want to sample the whole stack trace and we need different
> sampling rate (longer period, more samples). Those are the main reasons to
> split these in two "trackers", but I may be missing something.
> 
> I've prepared a little demo, so others can have a better idea of what I
> have in mind.
> 
> https://youtu.be/GUNDehj9z9o
> 
> Please note that this is a proof of concept and I'm not frontend person, so
> it may look little clumsy :)
> 
> D.
> 
> On Thu, Aug 1, 2019 at 11:40 AM Till Rohrmann  wrote:
> 
>> Hi David,
>> 
>> thanks for starting this discussion. I like the idea of improving insights
>> into Flink's execution and I believe that a flame graph could be helpful.
>> 
>> I quickly glanced over your changes and I think they go in a good
>> direction. One idea could be to share the `StackTraceSample` produced by
>> the `StackTraceSampleCoordinator` between the different
>> `StackTraceOperatorTracker` so that we don't send multiple requests for the
>> same operators. That way we would decrease a bit the RPC load.
>> 
>> Apart from that, I think the next steps would be to find a committer who
>> could shepherd this effort and help you with merging it.
>> 
>> Cheers,
>> Till
>> 
>> On Wed, Jul 31, 2019 at 7:05 PM David Morávek  wrote:
>> 
>>> Hello,
>>> 
>>> While looking into Flink internals, I've noticed that there is already a
>>> mechanism for stack-trace sampling of a particular job vertex.
>>> 
>>> I think it may be really useful to allow user to easily render a cpu
>>> flamegraph  in a new UI
>> for
>>> a
>>> selected vertex (new tab next to back pressure) of a running job. Back
>>> pressure tab already provides a good idea of which vertex causes trouble,
>>> but it's hard to say what's actually going on.
>>> 
>>> I've tried to implement a basic REST endpoint
>>> <
>>> 
>> https://github.com/dmvk/flink/commit/716231822d2fe99004895cdd0a365560479445b9
 ,
>>> that prepares data for the flame graph rendering and it seems to be
>>> providing good insight.
>>> 
>>> It should be straightforward to render data from the endpoint in new UI
>>> using existing  javascript
>>> libraries.
>>> 
>>> WDYT? Is this worth pushing forward?
>>> 
>>> D.
>>> 
>> 



[jira] [Created] (FLINK-13548) Support priority of the Flink YARN application

2019-08-02 Thread thy (JIRA)
thy created FLINK-13548:
---

 Summary: Support priority of the Flink YARN application
 Key: FLINK-13548
 URL: https://issues.apache.org/jira/browse/FLINK-13548
 Project: Flink
  Issue Type: Improvement
  Components: Deployment / YARN
Reporter: thy


Currently, Flink 1.9 does not support yarn priority submission. The default 
priority of yarn submission jobs is 1.

Based on this, we can provide a ConfigOption in YarnConfigOptions. The 
submission of the priority range from 1 to 9(from low to high priority). When 
the user does not configure the parameters, the priority is still 1. 



--
This message was sent by Atlassian JIRA
(v7.6.14#76016)


Re: [DISCUSS][CODE STYLE] Breaking long function argument lists and chained method calls

2019-08-02 Thread Chesnay Schepler

Just so everyone remembers:

Any suggested code-style should be
a) configurable in the IDE (otherwise we'll never be able to auto-format)
b) be verifiable via checkstyle (otherwise we'll end up manually 
checking for code-style again)


On 02/08/2019 03:20, SHI Xiaogang wrote:

Hi Andrey,

Thanks for bringing this. Personally, I prefer to the following style which
(1) puts the right parenthese in the next line
(2) a new line for each exception if exceptions can not be put in the same
line

That way, parentheses are aligned in a similar way to braces and exceptions
can be well aligned.

*public **void func(*
*int arg1,*
*int arg2,*
*...
*) throws E1, E2, E3 {*
*...
*}*

or

*public **void func(*
*int arg1,*
*int arg2,*
*...
*) throws
*E1,
*E2,
*E3 {*
*...
*}*

Regards,
Xiaogang

Andrey Zagrebin  于2019年8月1日周四 下午11:19写道:


Hi all,

This is one more small suggestion for the recent thread about code style
guide in Flink [1].

We already have a note about using a new line for each chained call in
Scala, e.g. either:

*values**.stream()**.map(...)**,collect(...);*

or

*values*
*.stream()*
*.map(*...*)*
*.collect(...)*

if it would result in a too long line by keeping all chained calls in one
line.

The suggestion is to have it for Java as well and add the same rule for a
long list of function arguments. So it is either:

*public void func(int arg1, int arg2, ...) throws E1, E2, E3 {*
*...*
*}*

or

*public **void func(*
*int arg1,*
*int arg2,*
*...)** throws E1, E2, E3 {*
*...*
*}*

but thrown exceptions stay on the same last line.

Please, feel free to share you thoughts.

Best,
Andrey

[1]

http://mail-archives.apache.org/mod_mbox/flink-dev/201906.mbox/%3ced91df4b-7cab-4547-a430-85bc710fd...@apache.org%3E





Re: [DISCUSS][CODE STYLE] Breaking long function argument lists and chained method calls

2019-08-02 Thread Biao Liu
Hi Andrey,

Thank you for bringing us this discussion.

I would like to make some details clear. Correct me if I am wrong.

The guide draft [1] says the line length is limited in 100 characters. From
my understanding, this discussion suggests if there is more than 100
characters in one line (both Scala and Java), we should start a new line
(or lines).

*Question 1*: If a line does not exceed 100 characters, should we break the
chained calls into lines? Currently the chained calls always been broken
into lines even it's not too long. Does it just a suggestion or a
limitation?
I prefer it's a limitation which must be respected. And we should always
break the chained calls no matter how long the line is.

For a chained method calls, the new line should be started with the dot.

*Question 2:* The indent of new line should be 1 tab or 2 tabs? Currently
there exists these two different styles. This rule should be also applied
to function arguments.

BTW, big +1 to options from Chesnay. We should make auto-format possible on
our project.

1.
https://docs.google.com/document/d/1owKfK1DwXA-w6qnx3R7t2D_o0BsFkkukGlRhvl3XXjQ/edit#

Thanks,
Biao /'bɪ.aʊ/



On Fri, Aug 2, 2019 at 9:20 AM SHI Xiaogang  wrote:

> Hi Andrey,
>
> Thanks for bringing this. Personally, I prefer to the following style which
> (1) puts the right parenthese in the next line
> (2) a new line for each exception if exceptions can not be put in the same
> line
>
> That way, parentheses are aligned in a similar way to braces and exceptions
> can be well aligned.
>
> *public **void func(*
> *int arg1,*
> *int arg2,*
> *...
> *) throws E1, E2, E3 {*
> *...
> *}*
>
> or
>
> *public **void func(*
> *int arg1,*
> *int arg2,*
> *...
> *) throws
> *E1,
> *E2,
> *E3 {*
> *...
> *}*
>
> Regards,
> Xiaogang
>
> Andrey Zagrebin  于2019年8月1日周四 下午11:19写道:
>
> > Hi all,
> >
> > This is one more small suggestion for the recent thread about code style
> > guide in Flink [1].
> >
> > We already have a note about using a new line for each chained call in
> > Scala, e.g. either:
> >
> > *values**.stream()**.map(...)**,collect(...);*
> >
> > or
> >
> > *values*
> > *.stream()*
> > *.map(*...*)*
> > *.collect(...)*
> >
> > if it would result in a too long line by keeping all chained calls in one
> > line.
> >
> > The suggestion is to have it for Java as well and add the same rule for a
> > long list of function arguments. So it is either:
> >
> > *public void func(int arg1, int arg2, ...) throws E1, E2, E3 {*
> > *...*
> > *}*
> >
> > or
> >
> > *public **void func(*
> > *int arg1,*
> > *int arg2,*
> > *...)** throws E1, E2, E3 {*
> > *...*
> > *}*
> >
> > but thrown exceptions stay on the same last line.
> >
> > Please, feel free to share you thoughts.
> >
> > Best,
> > Andrey
> >
> > [1]
> >
> >
> http://mail-archives.apache.org/mod_mbox/flink-dev/201906.mbox/%3ced91df4b-7cab-4547-a430-85bc710fd...@apache.org%3E
> >
>


[jira] [Created] (FLINK-13549) Unable to query Hive table with char or varchar columns

2019-08-02 Thread Rui Li (JIRA)
Rui Li created FLINK-13549:
--

 Summary: Unable to query Hive table with char or varchar columns
 Key: FLINK-13549
 URL: https://issues.apache.org/jira/browse/FLINK-13549
 Project: Flink
  Issue Type: Bug
  Components: Connectors / Hive
Reporter: Rui Li






--
This message was sent by Atlassian JIRA
(v7.6.14#76016)


[jira] [Created] (FLINK-13550) Support for CPU FlameGraphs in new web UI

2019-08-02 Thread David Moravek (JIRA)
David Moravek created FLINK-13550:
-

 Summary: Support for CPU FlameGraphs in new web UI
 Key: FLINK-13550
 URL: https://issues.apache.org/jira/browse/FLINK-13550
 Project: Flink
  Issue Type: New Feature
  Components: Runtime / Web Frontend
Reporter: David Moravek


For a better insight into a running job, it would be useful to have ability to 
render a CPU flame graph for a particular job vertex.

Flink already has a stack-trace sampling mechanism in-place, so it should be 
straightforward to implement.

This should be done by implementing a new endpoint in REST API, which would 
sample the stack-trace the same way as current BackPressureTracker does, only 
with a different sampling rate and length of sampling.

[Here|https://www.youtube.com/watch?v=GUNDehj9z9o] is a little demo of the 
feature.



--
This message was sent by Atlassian JIRA
(v7.6.14#76016)


[jira] [Created] (FLINK-13551) Add vertex FlameGraph REST endpoint

2019-08-02 Thread David Moravek (JIRA)
David Moravek created FLINK-13551:
-

 Summary: Add vertex FlameGraph REST endpoint
 Key: FLINK-13551
 URL: https://issues.apache.org/jira/browse/FLINK-13551
 Project: Flink
  Issue Type: Sub-task
  Components: Runtime / REST
Reporter: David Moravek


Add a new endpoint that returns data for flame graph rendering.



--
This message was sent by Atlassian JIRA
(v7.6.14#76016)


[jira] [Created] (FLINK-13552) Render vertex FlameGraph in web UI

2019-08-02 Thread David Moravek (JIRA)
David Moravek created FLINK-13552:
-

 Summary: Render vertex FlameGraph in web UI
 Key: FLINK-13552
 URL: https://issues.apache.org/jira/browse/FLINK-13552
 Project: Flink
  Issue Type: Sub-task
  Components: Runtime / Web Frontend
Reporter: David Moravek


Add a new FlameGraph tab in "vertex detail" page, that will actively poll flame 
graph endpoint and render it using d3 library.



--
This message was sent by Atlassian JIRA
(v7.6.14#76016)


Re: instable checkpointing after migration to flink 1.8

2019-08-02 Thread Bekir Oguz
Forgot to add the checkpoint details after it was complete. This is for that 
long running checkpoint with id 95632.



> Op 2 aug. 2019, om 11:18 heeft Bekir Oguz  het 
> volgende geschreven:
> 
> Hi Congxian,
> I was able to fetch the logs of the task manager (attached) and the 
> screenshots of the latest long checkpoint. I will get the logs of the job 
> manager for the next long running checkpoint. And also I will try to get a 
> jstack during the long running checkpoint.
> 
> Note: Since at the Subtasks tab we do not have the subtask numbers, and at 
> the Details tab of the checkpoint, we have the subtask numbers but not the 
> task manager hosts, it is difficult to match those. We’re assuming they have 
> the same order, so seeing that 3rd subtask is failing, I am getting the 3rd 
> line at the Subtasks tab which leads to the task manager host 
> flink-taskmanager-84ccd5bddf-2cbxn. ***It would be a great feature if you 
> guys also include the subtask-id’s to the Subtasks view.*** 
> 
> Note: timestamps in the task manager log are in UTC and I am at the moment at 
> zone UTC+3, so the time 10:30 at the screenshot matches the time 7:30 in the 
> log.
> 
> 
> Kind regards,
> Bekir
> 
> 
> 
> 
> 
> 
> 
> 
> 
>> Op 2 aug. 2019, om 07:23 heeft Congxian Qiu > > het volgende geschreven:
>> 
>> Hi Bekir
>> I’ll first summary the problem here(please correct me if I’m wrong)
>> 1. The same program runs on 1.6 never encounter such problems
>> 2. Some checkpoints completed too long (15+ min), but other normal 
>> checkpoints complete less than 1 min
>> 3. Some  bad checkpoint will have a large sync time, async time seems ok
>> 4. Some bad checkpoint, the e2e duration will much bigger than (sync_time + 
>> async_time) 
>> First, answer the last question, the e2e duration is ack_time - 
>> trigger_time, so it always bigger than (sync_time + async_time), but we have 
>> a big gap here, this may be problematic.
>> According to all the information, maybe the problem is some task start to do 
>> checkpoint too late and the sync checkpoint part took some time too long, 
>> Could you please share some more information such below:
>> - A Screenshot of summary for one bad checkpoint(we call it A here)
>> - The detailed information of checkpoint A(includes all the problematic 
>> subtasks)
>> - Jobmanager.log and the taskmanager.log for the problematic task and a 
>> health task
>> - Share the screenshot of subtasks for the problematic task(includes the 
>> `Bytes received`, `Records received`, `Bytes sent`, `Records sent` column), 
>> here wants to compare the problematic parallelism and good parallelism’s 
>> information, please also share the information is there has a data skew 
>> among the parallelisms,
>> - could you please share some jstacks of the problematic parallelism — here 
>> wants to check whether the task is too busy to handle the barrier. (flame 
>> graph or other things is always welcome here)
>> 
>> Best,
>> Congxian
>> 
>> 
>> Congxian Qiu mailto:qcx978132...@gmail.com>> 
>> 于2019年8月1日周四 下午8:26写道:
>> Hi Bekir
>> 
>> I'll first comb through all the information here, and try to find out the 
>> reason with you, maybe need you to share some more information :)
>> 
>> Best,
>> Congxian
>> 
>> 
>> Bekir Oguz mailto:bekir.o...@persgroep.net>> 
>> 于2019年8月1日周四 下午5:00写道:
>> Hi Fabian,
>> Thanks for sharing this with us, but we’re already on version 1.8.1.
>> 
>> What I don’t understand is which mechanism in Flink adds 15 minutes to the 
>> checkpoint duration occasionally. Can you maybe give us some hints on where 
>> to look at? Is there a default timeout of 15 minutes defined somewhere in 
>> Flink? I couldn’t find one.
>> 
>> In our pipeline, most of the checkpoints complete in less than a minute and 
>> some of them completed in 15 minutes+(less than a minute).
>> There’s definitely something which adds 15 minutes. This is happening in one 
>> or more subtasks during checkpointing.
>> 
>> Please see the screenshot below:
>> 
>> 
>> Regards,
>> Bekir
>> 
>> 
>> 
>>> Op 23 jul. 2019, om 16:37 heeft Fabian Hueske >> > het volgende geschreven:
>>> 
>>> Hi Bekir,
>>> 
>>> Another user reported checkpointing issues with Flink 1.8.0 [1].
>>> These seem to be resolved with Flink 1.8.1.
>>> 
>>> Hope this helps,
>>> Fabian
>>> 
>>> [1]
>>> https://lists.apache.org/thread.html/991fe3b09fd6a052ff52e5f7d9cdd9418545e68b02e23493097d9bc4@%3Cuser.flink.apache.org%3E
>>>  
>>> 
>>> 
>>> Am Mi., 17. Juli 2019 um 09:16 Uhr schrieb Congxian Qiu <
>>> qcx978132...@gmail.com >:
>>> 
 Hi Bekir
 
 First of all, I think there is something wrong.  the state size is almost
 the same,  but the duration is different so much.
 
 The checkpoint for RocksDBStatebackend is dump sst files, then copy the
>>>

Re: [DISCUSS] CPU flame graph for a job vertex in web UI.

2019-08-02 Thread David Morávek
Hi Paul, for now I only plan to add the one based on java stack traces.

On Fri, Aug 2, 2019 at 9:34 AM Paul Lam  wrote:

> Hi David,
>
> Thanks for the new feature! I think the flame graph would be a useful tool
> to understand the state of job executions, and it looks good too. +1 for
> this.
>
> And a minor question: do we plan to support multiple kinds of flame
> graphs? It would be great if we have both on-cpu and off-cpu flame graphs.
>
> Best,
> Paul Lam
>
> > 在 2019年8月2日,04:24,David Morávek  写道:
> >
> > Hi Till, thanks for the feedback! These endpoints are only called when
> the
> > vertex is selected in the UI, so there should be any heavy RPC load. For
> > back-pressure, we only sample top 3 calls of the stack (depth = 3). For
> the
> > flame-graph, we want to sample the whole stack trace and we need
> different
> > sampling rate (longer period, more samples). Those are the main reasons
> to
> > split these in two "trackers", but I may be missing something.
> >
> > I've prepared a little demo, so others can have a better idea of what I
> > have in mind.
> >
> > https://youtu.be/GUNDehj9z9o
> >
> > Please note that this is a proof of concept and I'm not frontend person,
> so
> > it may look little clumsy :)
> >
> > D.
> >
> > On Thu, Aug 1, 2019 at 11:40 AM Till Rohrmann 
> wrote:
> >
> >> Hi David,
> >>
> >> thanks for starting this discussion. I like the idea of improving
> insights
> >> into Flink's execution and I believe that a flame graph could be
> helpful.
> >>
> >> I quickly glanced over your changes and I think they go in a good
> >> direction. One idea could be to share the `StackTraceSample` produced by
> >> the `StackTraceSampleCoordinator` between the different
> >> `StackTraceOperatorTracker` so that we don't send multiple requests for
> the
> >> same operators. That way we would decrease a bit the RPC load.
> >>
> >> Apart from that, I think the next steps would be to find a committer who
> >> could shepherd this effort and help you with merging it.
> >>
> >> Cheers,
> >> Till
> >>
> >> On Wed, Jul 31, 2019 at 7:05 PM David Morávek  wrote:
> >>
> >>> Hello,
> >>>
> >>> While looking into Flink internals, I've noticed that there is already
> a
> >>> mechanism for stack-trace sampling of a particular job vertex.
> >>>
> >>> I think it may be really useful to allow user to easily render a cpu
> >>> flamegraph  in a new UI
> >> for
> >>> a
> >>> selected vertex (new tab next to back pressure) of a running job. Back
> >>> pressure tab already provides a good idea of which vertex causes
> trouble,
> >>> but it's hard to say what's actually going on.
> >>>
> >>> I've tried to implement a basic REST endpoint
> >>> <
> >>>
> >>
> https://github.com/dmvk/flink/commit/716231822d2fe99004895cdd0a365560479445b9
>  ,
> >>> that prepares data for the flame graph rendering and it seems to be
> >>> providing good insight.
> >>>
> >>> It should be straightforward to render data from the endpoint in new UI
> >>> using existing  javascript
> >>> libraries.
> >>>
> >>> WDYT? Is this worth pushing forward?
> >>>
> >>> D.
> >>>
> >>
>
>


Customize StreamingFileSink: Enable extending StreamingFileSink class

2019-08-02 Thread Kailash Dayanand
Hello,

There were a few things which we missing in
https://issues.apache.org/jira/browse/FLINK-12539, due to which it was not
possible to extend the StreamingFileSink. To fix this I have created a
follow-up PR but I am not sure if this is the best way to make these
changes:
https://github.com/apache/flink/compare/master...kailashhd:FLINK-12539.

In case this is the best approach to do this, I will do ahead and create
another JIRA and PR for this. Appreciate the help.

Thanks
Kailash


Re: [DISCUSS] CPU flame graph for a job vertex in web UI.

2019-08-02 Thread David Morávek
I've created FLINK-13550 
to track the issue.

Is there any committer who'd be willing to "shepherd this effort"? :)

Thanks,
D.

On Fri, Aug 2, 2019 at 10:22 AM David Morávek  wrote:

> Hi Paul, for now I only plan to add the one based on java stack traces.
>
> On Fri, Aug 2, 2019 at 9:34 AM Paul Lam  wrote:
>
>> Hi David,
>>
>> Thanks for the new feature! I think the flame graph would be a useful
>> tool to understand the state of job executions, and it looks good too. +1
>> for this.
>>
>> And a minor question: do we plan to support multiple kinds of flame
>> graphs? It would be great if we have both on-cpu and off-cpu flame graphs.
>>
>> Best,
>> Paul Lam
>>
>> > 在 2019年8月2日,04:24,David Morávek  写道:
>> >
>> > Hi Till, thanks for the feedback! These endpoints are only called when
>> the
>> > vertex is selected in the UI, so there should be any heavy RPC load. For
>> > back-pressure, we only sample top 3 calls of the stack (depth = 3). For
>> the
>> > flame-graph, we want to sample the whole stack trace and we need
>> different
>> > sampling rate (longer period, more samples). Those are the main reasons
>> to
>> > split these in two "trackers", but I may be missing something.
>> >
>> > I've prepared a little demo, so others can have a better idea of what I
>> > have in mind.
>> >
>> > https://youtu.be/GUNDehj9z9o
>> >
>> > Please note that this is a proof of concept and I'm not frontend
>> person, so
>> > it may look little clumsy :)
>> >
>> > D.
>> >
>> > On Thu, Aug 1, 2019 at 11:40 AM Till Rohrmann 
>> wrote:
>> >
>> >> Hi David,
>> >>
>> >> thanks for starting this discussion. I like the idea of improving
>> insights
>> >> into Flink's execution and I believe that a flame graph could be
>> helpful.
>> >>
>> >> I quickly glanced over your changes and I think they go in a good
>> >> direction. One idea could be to share the `StackTraceSample` produced
>> by
>> >> the `StackTraceSampleCoordinator` between the different
>> >> `StackTraceOperatorTracker` so that we don't send multiple requests
>> for the
>> >> same operators. That way we would decrease a bit the RPC load.
>> >>
>> >> Apart from that, I think the next steps would be to find a committer
>> who
>> >> could shepherd this effort and help you with merging it.
>> >>
>> >> Cheers,
>> >> Till
>> >>
>> >> On Wed, Jul 31, 2019 at 7:05 PM David Morávek  wrote:
>> >>
>> >>> Hello,
>> >>>
>> >>> While looking into Flink internals, I've noticed that there is
>> already a
>> >>> mechanism for stack-trace sampling of a particular job vertex.
>> >>>
>> >>> I think it may be really useful to allow user to easily render a cpu
>> >>> flamegraph  in a new UI
>> >> for
>> >>> a
>> >>> selected vertex (new tab next to back pressure) of a running job. Back
>> >>> pressure tab already provides a good idea of which vertex causes
>> trouble,
>> >>> but it's hard to say what's actually going on.
>> >>>
>> >>> I've tried to implement a basic REST endpoint
>> >>> <
>> >>>
>> >>
>> https://github.com/dmvk/flink/commit/716231822d2fe99004895cdd0a365560479445b9
>>  ,
>> >>> that prepares data for the flame graph rendering and it seems to be
>> >>> providing good insight.
>> >>>
>> >>> It should be straightforward to render data from the endpoint in new
>> UI
>> >>> using existing 
>> javascript
>> >>> libraries.
>> >>>
>> >>> WDYT? Is this worth pushing forward?
>> >>>
>> >>> D.
>> >>>
>> >>
>>
>>


Re: [DISCUSS][CODE STYLE] Usage of Java Optional

2019-08-02 Thread Biao Liu
Hi Andrey,

Thanks for working on this.

+1 it's clear and acceptable for me.

To Qi,

IMO the most performance critical codes are "per record" code path. We
should definitely avoid Optional there. For your concern, it's "per buffer"
code path which seems to be acceptable with Optional.

Just one more question, is there any other code paths which are also
critical? I think we'd better note that clearly.

Thanks,
Biao /'bɪ.aʊ/



On Fri, Aug 2, 2019 at 11:08 AM qi luo  wrote:

> Agree that using Optional will improve code robustness. However we’re
> hesitating to use Optional in data intensive operations.
>
> For example, SingleInputGate is already creating Optional for every
> BufferOrEvent in getNextBufferOrEvent(). How much performance gain would we
> get if it’s replaced by null check?
>
> Regards,
> Qi
>
> > On Aug 1, 2019, at 11:00 PM, Andrey Zagrebin 
> wrote:
> >
> > Hi all,
> >
> > This is the next follow up discussion about suggestions for the recent
> > thread about code style guide in Flink [1].
> >
> > In general, one could argue that any variable, which is nullable, can be
> > replaced by wrapping it with Optional to explicitly show that it can be
> > null. Examples are:
> >
> >   - returned values to force user to check not null
> >   - optional function arguments, e.g. with implicit default values
> >   - even class fields as e.g. optional config options with implicit
> >   default values
> >
> >
> > At the same time, we also have @Nullable annotation to express this
> > intention.
> >
> > Also, when the class Optional was introduced, Oracle posted a guideline
> > about its usage [2]. Basically, it suggests to use it mostly in APIs for
> > returned values to inform and force users to check the returned value
> > instead of returning null and avoid NullPointerException.
> >
> > Wrapping with Optional also comes with the performance overhead.
> >
> > Following the Oracle's guide in general, the suggestion is:
> >
> >   - Avoid using Optional in any performance critical code
> >   - Use Optional only to return nullable values in the API/public methods
> >   unless it is performance critical then rather use @Nullable
> >   - Passing an Optional argument to a method can be allowed if it is
> >   within a private helper method and simplifies the code, example is in
> [3]
> >   - Optional should not be used for class fields
> >
> >
> > Please, feel free to share you thoughts.
> >
> > Best,
> > Andrey
> >
> > [1]
> >
> http://mail-archives.apache.org/mod_mbox/flink-dev/201906.mbox/%3ced91df4b-7cab-4547-a430-85bc710fd...@apache.org%3E
> > [2]
> >
> https://www.oracle.com/technetwork/articles/java/java8-optional-2175753.html
> > [3]
> >
> https://github.com/apache/flink/blob/master/flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/typeutils/AvroFactory.java#L95
>
>


Re: [DISCUSS][CODE STYLE] Usage of Java Optional

2019-08-02 Thread Jark Wu
Hi Andrey,

I have some concern on point (3) "even class fields as e.g. optional config
options with implicit default values".

Regarding to the Oracle's guide (4) "Optional should not be used for class
fields".
And IntelliJ IDEA also report warnings if a class field is Optional,
because Optional is not serializable.


Do we allow Optional as class field only if the class is not serializable
or forbid this totally?

Thanks,
Jark

On Fri, 2 Aug 2019 at 16:30, Biao Liu  wrote:

> Hi Andrey,
>
> Thanks for working on this.
>
> +1 it's clear and acceptable for me.
>
> To Qi,
>
> IMO the most performance critical codes are "per record" code path. We
> should definitely avoid Optional there. For your concern, it's "per buffer"
> code path which seems to be acceptable with Optional.
>
> Just one more question, is there any other code paths which are also
> critical? I think we'd better note that clearly.
>
> Thanks,
> Biao /'bɪ.aʊ/
>
>
>
> On Fri, Aug 2, 2019 at 11:08 AM qi luo  wrote:
>
> > Agree that using Optional will improve code robustness. However we’re
> > hesitating to use Optional in data intensive operations.
> >
> > For example, SingleInputGate is already creating Optional for every
> > BufferOrEvent in getNextBufferOrEvent(). How much performance gain would
> we
> > get if it’s replaced by null check?
> >
> > Regards,
> > Qi
> >
> > > On Aug 1, 2019, at 11:00 PM, Andrey Zagrebin 
> > wrote:
> > >
> > > Hi all,
> > >
> > > This is the next follow up discussion about suggestions for the recent
> > > thread about code style guide in Flink [1].
> > >
> > > In general, one could argue that any variable, which is nullable, can
> be
> > > replaced by wrapping it with Optional to explicitly show that it can be
> > > null. Examples are:
> > >
> > >   - returned values to force user to check not null
> > >   - optional function arguments, e.g. with implicit default values
> > >   - even class fields as e.g. optional config options with implicit
> > >   default values
> > >
> > >
> > > At the same time, we also have @Nullable annotation to express this
> > > intention.
> > >
> > > Also, when the class Optional was introduced, Oracle posted a guideline
> > > about its usage [2]. Basically, it suggests to use it mostly in APIs
> for
> > > returned values to inform and force users to check the returned value
> > > instead of returning null and avoid NullPointerException.
> > >
> > > Wrapping with Optional also comes with the performance overhead.
> > >
> > > Following the Oracle's guide in general, the suggestion is:
> > >
> > >   - Avoid using Optional in any performance critical code
> > >   - Use Optional only to return nullable values in the API/public
> methods
> > >   unless it is performance critical then rather use @Nullable
> > >   - Passing an Optional argument to a method can be allowed if it is
> > >   within a private helper method and simplifies the code, example is in
> > [3]
> > >   - Optional should not be used for class fields
> > >
> > >
> > > Please, feel free to share you thoughts.
> > >
> > > Best,
> > > Andrey
> > >
> > > [1]
> > >
> >
> http://mail-archives.apache.org/mod_mbox/flink-dev/201906.mbox/%3ced91df4b-7cab-4547-a430-85bc710fd...@apache.org%3E
> > > [2]
> > >
> >
> https://www.oracle.com/technetwork/articles/java/java8-optional-2175753.html
> > > [3]
> > >
> >
> https://github.com/apache/flink/blob/master/flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/typeutils/AvroFactory.java#L95
> >
> >
>


Re: [DISCUSS][CODE STYLE] Usage of Java Optional

2019-08-02 Thread Zili Chen
Hi Jark,

Follow your opinion, for class field, we can make
use of @Nullable/@Nonnull annotation or Flink's
SerializableOptional. It would be sufficient.

Best,
tison.


Jark Wu  于2019年8月2日周五 下午4:57写道:

> Hi Andrey,
>
> I have some concern on point (3) "even class fields as e.g. optional config
> options with implicit default values".
>
> Regarding to the Oracle's guide (4) "Optional should not be used for class
> fields".
> And IntelliJ IDEA also report warnings if a class field is Optional,
> because Optional is not serializable.
>
>
> Do we allow Optional as class field only if the class is not serializable
> or forbid this totally?
>
> Thanks,
> Jark
>
> On Fri, 2 Aug 2019 at 16:30, Biao Liu  wrote:
>
> > Hi Andrey,
> >
> > Thanks for working on this.
> >
> > +1 it's clear and acceptable for me.
> >
> > To Qi,
> >
> > IMO the most performance critical codes are "per record" code path. We
> > should definitely avoid Optional there. For your concern, it's "per
> buffer"
> > code path which seems to be acceptable with Optional.
> >
> > Just one more question, is there any other code paths which are also
> > critical? I think we'd better note that clearly.
> >
> > Thanks,
> > Biao /'bɪ.aʊ/
> >
> >
> >
> > On Fri, Aug 2, 2019 at 11:08 AM qi luo  wrote:
> >
> > > Agree that using Optional will improve code robustness. However we’re
> > > hesitating to use Optional in data intensive operations.
> > >
> > > For example, SingleInputGate is already creating Optional for every
> > > BufferOrEvent in getNextBufferOrEvent(). How much performance gain
> would
> > we
> > > get if it’s replaced by null check?
> > >
> > > Regards,
> > > Qi
> > >
> > > > On Aug 1, 2019, at 11:00 PM, Andrey Zagrebin 
> > > wrote:
> > > >
> > > > Hi all,
> > > >
> > > > This is the next follow up discussion about suggestions for the
> recent
> > > > thread about code style guide in Flink [1].
> > > >
> > > > In general, one could argue that any variable, which is nullable, can
> > be
> > > > replaced by wrapping it with Optional to explicitly show that it can
> be
> > > > null. Examples are:
> > > >
> > > >   - returned values to force user to check not null
> > > >   - optional function arguments, e.g. with implicit default values
> > > >   - even class fields as e.g. optional config options with implicit
> > > >   default values
> > > >
> > > >
> > > > At the same time, we also have @Nullable annotation to express this
> > > > intention.
> > > >
> > > > Also, when the class Optional was introduced, Oracle posted a
> guideline
> > > > about its usage [2]. Basically, it suggests to use it mostly in APIs
> > for
> > > > returned values to inform and force users to check the returned value
> > > > instead of returning null and avoid NullPointerException.
> > > >
> > > > Wrapping with Optional also comes with the performance overhead.
> > > >
> > > > Following the Oracle's guide in general, the suggestion is:
> > > >
> > > >   - Avoid using Optional in any performance critical code
> > > >   - Use Optional only to return nullable values in the API/public
> > methods
> > > >   unless it is performance critical then rather use @Nullable
> > > >   - Passing an Optional argument to a method can be allowed if it is
> > > >   within a private helper method and simplifies the code, example is
> in
> > > [3]
> > > >   - Optional should not be used for class fields
> > > >
> > > >
> > > > Please, feel free to share you thoughts.
> > > >
> > > > Best,
> > > > Andrey
> > > >
> > > > [1]
> > > >
> > >
> >
> http://mail-archives.apache.org/mod_mbox/flink-dev/201906.mbox/%3ced91df4b-7cab-4547-a430-85bc710fd...@apache.org%3E
> > > > [2]
> > > >
> > >
> >
> https://www.oracle.com/technetwork/articles/java/java8-optional-2175753.html
> > > > [3]
> > > >
> > >
> >
> https://github.com/apache/flink/blob/master/flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/typeutils/AvroFactory.java#L95
> > >
> > >
> >
>


Re: [DISCUSS][CODE STYLE] Usage of Java Optional

2019-08-02 Thread Jark Wu
Hi Zili,

Yes. I agree to use @Nullable/@Nonnull/SerializableOptional as the class
field instead of Optional.



On Fri, 2 Aug 2019 at 17:00, Zili Chen  wrote:

> Hi Jark,
>
> Follow your opinion, for class field, we can make
> use of @Nullable/@Nonnull annotation or Flink's
> SerializableOptional. It would be sufficient.
>
> Best,
> tison.
>
>
> Jark Wu  于2019年8月2日周五 下午4:57写道:
>
> > Hi Andrey,
> >
> > I have some concern on point (3) "even class fields as e.g. optional
> config
> > options with implicit default values".
> >
> > Regarding to the Oracle's guide (4) "Optional should not be used for
> class
> > fields".
> > And IntelliJ IDEA also report warnings if a class field is Optional,
> > because Optional is not serializable.
> >
> >
> > Do we allow Optional as class field only if the class is not serializable
> > or forbid this totally?
> >
> > Thanks,
> > Jark
> >
> > On Fri, 2 Aug 2019 at 16:30, Biao Liu  wrote:
> >
> > > Hi Andrey,
> > >
> > > Thanks for working on this.
> > >
> > > +1 it's clear and acceptable for me.
> > >
> > > To Qi,
> > >
> > > IMO the most performance critical codes are "per record" code path. We
> > > should definitely avoid Optional there. For your concern, it's "per
> > buffer"
> > > code path which seems to be acceptable with Optional.
> > >
> > > Just one more question, is there any other code paths which are also
> > > critical? I think we'd better note that clearly.
> > >
> > > Thanks,
> > > Biao /'bɪ.aʊ/
> > >
> > >
> > >
> > > On Fri, Aug 2, 2019 at 11:08 AM qi luo  wrote:
> > >
> > > > Agree that using Optional will improve code robustness. However we’re
> > > > hesitating to use Optional in data intensive operations.
> > > >
> > > > For example, SingleInputGate is already creating Optional for every
> > > > BufferOrEvent in getNextBufferOrEvent(). How much performance gain
> > would
> > > we
> > > > get if it’s replaced by null check?
> > > >
> > > > Regards,
> > > > Qi
> > > >
> > > > > On Aug 1, 2019, at 11:00 PM, Andrey Zagrebin  >
> > > > wrote:
> > > > >
> > > > > Hi all,
> > > > >
> > > > > This is the next follow up discussion about suggestions for the
> > recent
> > > > > thread about code style guide in Flink [1].
> > > > >
> > > > > In general, one could argue that any variable, which is nullable,
> can
> > > be
> > > > > replaced by wrapping it with Optional to explicitly show that it
> can
> > be
> > > > > null. Examples are:
> > > > >
> > > > >   - returned values to force user to check not null
> > > > >   - optional function arguments, e.g. with implicit default values
> > > > >   - even class fields as e.g. optional config options with implicit
> > > > >   default values
> > > > >
> > > > >
> > > > > At the same time, we also have @Nullable annotation to express this
> > > > > intention.
> > > > >
> > > > > Also, when the class Optional was introduced, Oracle posted a
> > guideline
> > > > > about its usage [2]. Basically, it suggests to use it mostly in
> APIs
> > > for
> > > > > returned values to inform and force users to check the returned
> value
> > > > > instead of returning null and avoid NullPointerException.
> > > > >
> > > > > Wrapping with Optional also comes with the performance overhead.
> > > > >
> > > > > Following the Oracle's guide in general, the suggestion is:
> > > > >
> > > > >   - Avoid using Optional in any performance critical code
> > > > >   - Use Optional only to return nullable values in the API/public
> > > methods
> > > > >   unless it is performance critical then rather use @Nullable
> > > > >   - Passing an Optional argument to a method can be allowed if it
> is
> > > > >   within a private helper method and simplifies the code, example
> is
> > in
> > > > [3]
> > > > >   - Optional should not be used for class fields
> > > > >
> > > > >
> > > > > Please, feel free to share you thoughts.
> > > > >
> > > > > Best,
> > > > > Andrey
> > > > >
> > > > > [1]
> > > > >
> > > >
> > >
> >
> http://mail-archives.apache.org/mod_mbox/flink-dev/201906.mbox/%3ced91df4b-7cab-4547-a430-85bc710fd...@apache.org%3E
> > > > > [2]
> > > > >
> > > >
> > >
> >
> https://www.oracle.com/technetwork/articles/java/java8-optional-2175753.html
> > > > > [3]
> > > > >
> > > >
> > >
> >
> https://github.com/apache/flink/blob/master/flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/typeutils/AvroFactory.java#L95
> > > >
> > > >
> > >
> >
>


Re: [DISCUSS][CODE STYLE] Usage of Java Optional

2019-08-02 Thread Biao Liu
Hi Jark & Zili,

I thought it means "Optional should not be used for class fields". However
now I get a bit confused about the edited version.

Anyway +1 to "Optional should not be used for class fields"

Thanks,
Biao /'bɪ.aʊ/



On Fri, Aug 2, 2019 at 5:00 PM Zili Chen  wrote:

> Hi Jark,
>
> Follow your opinion, for class field, we can make
> use of @Nullable/@Nonnull annotation or Flink's
> SerializableOptional. It would be sufficient.
>
> Best,
> tison.
>
>
> Jark Wu  于2019年8月2日周五 下午4:57写道:
>
> > Hi Andrey,
> >
> > I have some concern on point (3) "even class fields as e.g. optional
> config
> > options with implicit default values".
> >
> > Regarding to the Oracle's guide (4) "Optional should not be used for
> class
> > fields".
> > And IntelliJ IDEA also report warnings if a class field is Optional,
> > because Optional is not serializable.
> >
> >
> > Do we allow Optional as class field only if the class is not serializable
> > or forbid this totally?
> >
> > Thanks,
> > Jark
> >
> > On Fri, 2 Aug 2019 at 16:30, Biao Liu  wrote:
> >
> > > Hi Andrey,
> > >
> > > Thanks for working on this.
> > >
> > > +1 it's clear and acceptable for me.
> > >
> > > To Qi,
> > >
> > > IMO the most performance critical codes are "per record" code path. We
> > > should definitely avoid Optional there. For your concern, it's "per
> > buffer"
> > > code path which seems to be acceptable with Optional.
> > >
> > > Just one more question, is there any other code paths which are also
> > > critical? I think we'd better note that clearly.
> > >
> > > Thanks,
> > > Biao /'bɪ.aʊ/
> > >
> > >
> > >
> > > On Fri, Aug 2, 2019 at 11:08 AM qi luo  wrote:
> > >
> > > > Agree that using Optional will improve code robustness. However we’re
> > > > hesitating to use Optional in data intensive operations.
> > > >
> > > > For example, SingleInputGate is already creating Optional for every
> > > > BufferOrEvent in getNextBufferOrEvent(). How much performance gain
> > would
> > > we
> > > > get if it’s replaced by null check?
> > > >
> > > > Regards,
> > > > Qi
> > > >
> > > > > On Aug 1, 2019, at 11:00 PM, Andrey Zagrebin  >
> > > > wrote:
> > > > >
> > > > > Hi all,
> > > > >
> > > > > This is the next follow up discussion about suggestions for the
> > recent
> > > > > thread about code style guide in Flink [1].
> > > > >
> > > > > In general, one could argue that any variable, which is nullable,
> can
> > > be
> > > > > replaced by wrapping it with Optional to explicitly show that it
> can
> > be
> > > > > null. Examples are:
> > > > >
> > > > >   - returned values to force user to check not null
> > > > >   - optional function arguments, e.g. with implicit default values
> > > > >   - even class fields as e.g. optional config options with implicit
> > > > >   default values
> > > > >
> > > > >
> > > > > At the same time, we also have @Nullable annotation to express this
> > > > > intention.
> > > > >
> > > > > Also, when the class Optional was introduced, Oracle posted a
> > guideline
> > > > > about its usage [2]. Basically, it suggests to use it mostly in
> APIs
> > > for
> > > > > returned values to inform and force users to check the returned
> value
> > > > > instead of returning null and avoid NullPointerException.
> > > > >
> > > > > Wrapping with Optional also comes with the performance overhead.
> > > > >
> > > > > Following the Oracle's guide in general, the suggestion is:
> > > > >
> > > > >   - Avoid using Optional in any performance critical code
> > > > >   - Use Optional only to return nullable values in the API/public
> > > methods
> > > > >   unless it is performance critical then rather use @Nullable
> > > > >   - Passing an Optional argument to a method can be allowed if it
> is
> > > > >   within a private helper method and simplifies the code, example
> is
> > in
> > > > [3]
> > > > >   - Optional should not be used for class fields
> > > > >
> > > > >
> > > > > Please, feel free to share you thoughts.
> > > > >
> > > > > Best,
> > > > > Andrey
> > > > >
> > > > > [1]
> > > > >
> > > >
> > >
> >
> http://mail-archives.apache.org/mod_mbox/flink-dev/201906.mbox/%3ced91df4b-7cab-4547-a430-85bc710fd...@apache.org%3E
> > > > > [2]
> > > > >
> > > >
> > >
> >
> https://www.oracle.com/technetwork/articles/java/java8-optional-2175753.html
> > > > > [3]
> > > > >
> > > >
> > >
> >
> https://github.com/apache/flink/blob/master/flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/typeutils/AvroFactory.java#L95
> > > >
> > > >
> > >
> >
>


[jira] [Created] (FLINK-13553) KvStateServerHandlerTest.readInboundBlocking unstable on Travis

2019-08-02 Thread Till Rohrmann (JIRA)
Till Rohrmann created FLINK-13553:
-

 Summary: KvStateServerHandlerTest.readInboundBlocking unstable on 
Travis
 Key: FLINK-13553
 URL: https://issues.apache.org/jira/browse/FLINK-13553
 Project: Flink
  Issue Type: Bug
  Components: Runtime / Queryable State
Affects Versions: 1.10.0
Reporter: Till Rohrmann
 Fix For: 1.10.0


The {{KvStateServerHandlerTest.readInboundBlocking}} and 
{{KvStateServerHandlerTest.testQueryExecutorShutDown}} fail on Travis with a 
{{TimeoutException}}.

https://api.travis-ci.org/v3/job/566420641/log.txt



--
This message was sent by Atlassian JIRA
(v7.6.14#76016)


[jira] [Created] (FLINK-13554) ResourceManager should have a timeout on starting new TaskExecutors.

2019-08-02 Thread Xintong Song (JIRA)
Xintong Song created FLINK-13554:


 Summary: ResourceManager should have a timeout on starting new 
TaskExecutors.
 Key: FLINK-13554
 URL: https://issues.apache.org/jira/browse/FLINK-13554
 Project: Flink
  Issue Type: Improvement
  Components: Runtime / Coordination
Affects Versions: 1.9.0
Reporter: Xintong Song


Recently, we encountered a case that one TaskExecutor get stuck during 
launching on Yarn (without fail), causing that job cannot recover from 
continuous failovers.

The reason the TaskExecutor gets stuck is due to our environment problem. The 
TaskExecutor gets stuck somewhere after the ResourceManager starts the 
TaskExecutor and waiting for the TaskExecutor to be brought up and register. 
Later when the slot request timeouts, the job fails over and requests slots 
from ResourceManager again, the ResourceManager still see a TaskExecutor (the 
stuck one) is being started and will not request new container from Yarn. 
Therefore, the job can not recover from failure.

I think to avoid such unrecoverable status, the ResourceManager need to have a 
timeout on starting new TaskExecutor. If the starting of TaskExecutor takes too 
long, it should just fail the TaskExecutor and starts a new one.



--
This message was sent by Atlassian JIRA
(v7.6.14#76016)


[jira] [Created] (FLINK-13555) Failures of slot requests requiring unfulfillable managed memory should not be ignored.

2019-08-02 Thread Xintong Song (JIRA)
Xintong Song created FLINK-13555:


 Summary: Failures of slot requests requiring unfulfillable managed 
memory should not be ignored.
 Key: FLINK-13555
 URL: https://issues.apache.org/jira/browse/FLINK-13555
 Project: Flink
  Issue Type: Improvement
  Components: Runtime / Coordination
Affects Versions: 1.9.0
Reporter: Xintong Song
 Fix For: 1.9.0
 Attachments: flink-unk-standalonesession-0-u-home.log, 
flink-unk-taskexecutor-0-u-home.log

Currently, SlotPool ignores failures of requesting slots from ResourceManager 
for all batch slot requests. The idea behind this is to allow batch slot 
requests pending at SlotPool and waiting for other tasks to finish and release 
slots. A slot request will be failed only if it is not fulfilled in its timeout.

However, there could be two kinds of request slots from RM failures.
 # RM does not have available slots. All slots are in use at the moment. But 
they might become available later when the currently running tasks finish.
 # The slot request requires too many resources that can not be fulfilled by 
any slot (available or not) in the cluster. The request is also not likely to 
be fulfilled later.

For the 2nd kinds of failures, it doesn't make sense to wait for the timeout. 
We should fail the job immediately, with proper error messages describing the 
problem and suggesting the user to tune job or cluster configurations.



--
This message was sent by Atlassian JIRA
(v7.6.14#76016)


Re: [RESULT][VOTE] Migrate to sponsored Travis account

2019-08-02 Thread Chesnay Schepler
I'm currently modifying the cibot to do this automatically; should be 
finished until Monday.


On 02/08/2019 07:41, Jark Wu wrote:

Hi Chesnay,

Can we assign Flink Committers the permission of flink-ci/flink repo?
Several times, when I pushed some new commits, the old build jobs are still
in pending and not canceled.
Before we fix that, we can manually cancel some old jobs to save build
resource.

Best,
Jark


On Wed, 10 Jul 2019 at 16:17, Chesnay Schepler  wrote:


Your best bet would be to check the first commit in the PR and check the
parent commit.

To re-run things, you will have to rebase the PR on the latest master.

On 10/07/2019 03:32, Kurt Young wrote:

Thanks for all your efforts Chesnay, it indeed improves a lot for our
develop experience. BTW, do you know how to find the master branch
information which the CI runs with?

For example, like this one:
https://travis-ci.com/flink-ci/flink/jobs/214542568
It shows pass with the commits, which rebased on the master when the CI
is triggered. But it's both possible that the master branch CI runs on is
the
same or different with current master. If it's the same, I can simply

rely

on the
passed information to push commits, but if it's not, I think i should

find

another
way to re-trigger tests based on the newest master.

Do you know where can I get such information?

Best,
Kurt


On Tue, Jul 9, 2019 at 3:27 AM Chesnay Schepler 

wrote:

The kinks have been worked out; the bot is running again and pr builds
are yet again no longer running on ASF resources.

PRs are mirrored to: https://github.com/flink-ci/flink
Bot source: https://github.com/flink-ci/ci-bot

On 08/07/2019 17:14, Chesnay Schepler wrote:

I have temporarily re-enabled running PR builds on the ASF account;
migrating to the Travis subscription caused some issues in the bot
that I have to fix first.

On 07/07/2019 23:01, Chesnay Schepler wrote:

The vote has passed unanimously in favor of migrating to a separate
Travis account.

I will now set things up such that no PullRequest is no longer run on
the ASF servers.
This is a major setup in reducing our usage of ASF resources.
For the time being we'll use free Travis plan for flink-ci (i.e. 5
workers, which is the same the ASF gives us). Over the course of the
next week we'll setup the Ververica subscription to increase this

limit.

  From now now, a bot will mirror all new and updated PullRequests to a
mirror repository (https://github.com/flink-ci/flink-ci) and write an
update into the PR once the build is complete.
I have ran the bots for the past 3 days in parallel to our existing
Travis and it was working without major issues.

The biggest change that contributors will see is that there's no
longer a icon next to each commit. We may revisit this in the future.

I'll setup a repo with the source of the bot later.

On 04/07/2019 10:46, Chesnay Schepler wrote:

I've raised a JIRA
with INFRA to
inquire whether it would be possible to switch to a different Travis
account, and if so what steps would need to be taken.
We need a proper confirmation from INFRA since we are not in full
control of the flink repository (for example, we cannot access the
settings page).

If this is indeed possible, Ververica is willing sponsor a Travis
account for the Flink project.
This would provide us with more than enough resources than we need.

Since this makes the project more reliant on resources provided by
external companies I would like to vote on this.

Please vote on this proposal, as follows:
[ ] +1, Approve the migration to a Ververica-sponsored Travis
account, provided that INFRA approves
[ ] -1, Do not approach the migration to a Ververica-sponsored
Travis account

The vote will be open for at least 24h, and until we have
confirmation from INFRA. The voting period may be shorter than the
usual 3 days since our current is effectively not working.

On 04/07/2019 06:51, Bowen Li wrote:

Re: > Are they using their own Travis CI pool, or did the switch to
an entirely different CI service?

I reached out to Wes and Krisztián from Apache Arrow PMC. They are
currently moving away from ASF's Travis to their own in-house metal
machines at [1] with custom CI application at [2]. They've seen
significant improvement w.r.t both much higher performance and
basically no resource waiting time, "night-and-day" difference
quoting Wes.

Re: > If we can just switch to our own Travis pool, just for our
project, then this might be something we can do fairly quickly?

I believe so, according to [3] and [4]


[1] https://ci.ursalabs.org/ 
[2] https://github.com/ursa-labs/ursabot
[3]


https://docs.travis-ci.com/user/migrate/open-source-repository-migration

[4]


https://docs.travis-ci.com/user/migrate/open-source-on-travis-ci-com



On Wed, Jul 3, 2019 at 12:01 AM Chesnay Schepler
mailto:ches...@apache.org>> wrote:

  Are they using their own Travis CI pool, or did the switch to

an


Re: [Question] What is the difference between Embedded and SingleLeaderElectionService?

2019-08-02 Thread Till Rohrmann
I think at the moment SingleLeaderElectionService is not really used since
it is part of the YarnIntraNonHaMasterServices. The idea of these ha
services was to offer a different ha implementation which slightly
different guarantees. Concretely, the YarnIntraNonHaMasterServices handle
operator and TaskManager faults but not master faults.

The EmbeddedHaServices are only used by the MiniCluster.

I'm not so sure whether the refactoring is so easy because
EmbeddedLeaderElectionService is an inner class of EmbeddedLeaderService
and calls methods of this class. Why do you wanna change anything there?

Cheers,
Till

On Fri, Aug 2, 2019 at 6:33 AM Zili Chen  wrote:

> Hi devs,
>
> I found that these two classes are quite similar except
> SingleLeaderElectionService has a pre-config leader id.
>
> However, I don't see use points of that leader id. Also
> a random UUID would work as a DEFAULT_LEADER_ID(0).
> I consider whether we could replace SingleLeaderElectionService
> with EmbeddedLeaderElectionService, or merge.
>
> Best,
> tison.
>


Re: [RESULT][VOTE] Migrate to sponsored Travis account

2019-08-02 Thread Jark Wu
Wow. That's great! Thanks Chesnay.

On Fri, 2 Aug 2019 at 17:50, Chesnay Schepler  wrote:

> I'm currently modifying the cibot to do this automatically; should be
> finished until Monday.
>
> On 02/08/2019 07:41, Jark Wu wrote:
> > Hi Chesnay,
> >
> > Can we assign Flink Committers the permission of flink-ci/flink repo?
> > Several times, when I pushed some new commits, the old build jobs are
> still
> > in pending and not canceled.
> > Before we fix that, we can manually cancel some old jobs to save build
> > resource.
> >
> > Best,
> > Jark
> >
> >
> > On Wed, 10 Jul 2019 at 16:17, Chesnay Schepler 
> wrote:
> >
> >> Your best bet would be to check the first commit in the PR and check the
> >> parent commit.
> >>
> >> To re-run things, you will have to rebase the PR on the latest master.
> >>
> >> On 10/07/2019 03:32, Kurt Young wrote:
> >>> Thanks for all your efforts Chesnay, it indeed improves a lot for our
> >>> develop experience. BTW, do you know how to find the master branch
> >>> information which the CI runs with?
> >>>
> >>> For example, like this one:
> >>> https://travis-ci.com/flink-ci/flink/jobs/214542568
> >>> It shows pass with the commits, which rebased on the master when the CI
> >>> is triggered. But it's both possible that the master branch CI runs on
> is
> >>> the
> >>> same or different with current master. If it's the same, I can simply
> >> rely
> >>> on the
> >>> passed information to push commits, but if it's not, I think i should
> >> find
> >>> another
> >>> way to re-trigger tests based on the newest master.
> >>>
> >>> Do you know where can I get such information?
> >>>
> >>> Best,
> >>> Kurt
> >>>
> >>>
> >>> On Tue, Jul 9, 2019 at 3:27 AM Chesnay Schepler 
> >> wrote:
>  The kinks have been worked out; the bot is running again and pr builds
>  are yet again no longer running on ASF resources.
> 
>  PRs are mirrored to: https://github.com/flink-ci/flink
>  Bot source: https://github.com/flink-ci/ci-bot
> 
>  On 08/07/2019 17:14, Chesnay Schepler wrote:
> > I have temporarily re-enabled running PR builds on the ASF account;
> > migrating to the Travis subscription caused some issues in the bot
> > that I have to fix first.
> >
> > On 07/07/2019 23:01, Chesnay Schepler wrote:
> >> The vote has passed unanimously in favor of migrating to a separate
> >> Travis account.
> >>
> >> I will now set things up such that no PullRequest is no longer run
> on
> >> the ASF servers.
> >> This is a major setup in reducing our usage of ASF resources.
> >> For the time being we'll use free Travis plan for flink-ci (i.e. 5
> >> workers, which is the same the ASF gives us). Over the course of the
> >> next week we'll setup the Ververica subscription to increase this
> >> limit.
> >>   From now now, a bot will mirror all new and updated PullRequests
> to a
> >> mirror repository (https://github.com/flink-ci/flink-ci) and write
> an
> >> update into the PR once the build is complete.
> >> I have ran the bots for the past 3 days in parallel to our existing
> >> Travis and it was working without major issues.
> >>
> >> The biggest change that contributors will see is that there's no
> >> longer a icon next to each commit. We may revisit this in the
> future.
> >>
> >> I'll setup a repo with the source of the bot later.
> >>
> >> On 04/07/2019 10:46, Chesnay Schepler wrote:
> >>> I've raised a JIRA
> >>> with INFRA to
> >>> inquire whether it would be possible to switch to a different
> Travis
> >>> account, and if so what steps would need to be taken.
> >>> We need a proper confirmation from INFRA since we are not in full
> >>> control of the flink repository (for example, we cannot access the
> >>> settings page).
> >>>
> >>> If this is indeed possible, Ververica is willing sponsor a Travis
> >>> account for the Flink project.
> >>> This would provide us with more than enough resources than we need.
> >>>
> >>> Since this makes the project more reliant on resources provided by
> >>> external companies I would like to vote on this.
> >>>
> >>> Please vote on this proposal, as follows:
> >>> [ ] +1, Approve the migration to a Ververica-sponsored Travis
> >>> account, provided that INFRA approves
> >>> [ ] -1, Do not approach the migration to a Ververica-sponsored
> >>> Travis account
> >>>
> >>> The vote will be open for at least 24h, and until we have
> >>> confirmation from INFRA. The voting period may be shorter than the
> >>> usual 3 days since our current is effectively not working.
> >>>
> >>> On 04/07/2019 06:51, Bowen Li wrote:
>  Re: > Are they using their own Travis CI pool, or did the switch
> to
>  an entirely different CI service?
> 
>  I reached out to Wes and Krisztián 

[jira] [Created] (FLINK-13556) Python profile failed on Travis with setup problem

2019-08-02 Thread Till Rohrmann (JIRA)
Till Rohrmann created FLINK-13556:
-

 Summary: Python profile failed on Travis with setup problem
 Key: FLINK-13556
 URL: https://issues.apache.org/jira/browse/FLINK-13556
 Project: Flink
  Issue Type: Bug
  Components: API / Python
Affects Versions: 1.9.0, 1.10.0
Reporter: Till Rohrmann
 Fix For: 1.9.0


The Python profile failed on Travis because conda install 3.5 failed with the 
following error {{Error([('SSL routines', 'ssl3_get_record', 'decryption failed 
or bad record mac')])}}.

https://api.travis-ci.org/v3/job/566835602/log.txt



--
This message was sent by Atlassian JIRA
(v7.6.14#76016)


Re: [DISCUSS][CODE STYLE] Usage of Java Optional

2019-08-02 Thread Timo Walther

Hi everyone,

I would vote for using Optional only as method return type for 
non-performance critical code. Nothing more. No fields, no method 
parameters. Method parameters can be overloaded and internally a class 
can work with nulls and @Nullable. Optional is meant for API method 
return types and I think we should not abuse it and spam the code with 
`@SuppressWarnings("OptionalUsedAsFieldOrParameterType")`.


Regards,

Timo



Am 02.08.19 um 11:08 schrieb Biao Liu:

Hi Jark & Zili,

I thought it means "Optional should not be used for class fields". However
now I get a bit confused about the edited version.

Anyway +1 to "Optional should not be used for class fields"

Thanks,
Biao /'bɪ.aʊ/



On Fri, Aug 2, 2019 at 5:00 PM Zili Chen  wrote:


Hi Jark,

Follow your opinion, for class field, we can make
use of @Nullable/@Nonnull annotation or Flink's
SerializableOptional. It would be sufficient.

Best,
tison.


Jark Wu  于2019年8月2日周五 下午4:57写道:


Hi Andrey,

I have some concern on point (3) "even class fields as e.g. optional

config

options with implicit default values".

Regarding to the Oracle's guide (4) "Optional should not be used for

class

fields".
And IntelliJ IDEA also report warnings if a class field is Optional,
because Optional is not serializable.


Do we allow Optional as class field only if the class is not serializable
or forbid this totally?

Thanks,
Jark

On Fri, 2 Aug 2019 at 16:30, Biao Liu  wrote:


Hi Andrey,

Thanks for working on this.

+1 it's clear and acceptable for me.

To Qi,

IMO the most performance critical codes are "per record" code path. We
should definitely avoid Optional there. For your concern, it's "per

buffer"

code path which seems to be acceptable with Optional.

Just one more question, is there any other code paths which are also
critical? I think we'd better note that clearly.

Thanks,
Biao /'bɪ.aʊ/



On Fri, Aug 2, 2019 at 11:08 AM qi luo  wrote:


Agree that using Optional will improve code robustness. However we’re
hesitating to use Optional in data intensive operations.

For example, SingleInputGate is already creating Optional for every
BufferOrEvent in getNextBufferOrEvent(). How much performance gain

would

we

get if it’s replaced by null check?

Regards,
Qi


On Aug 1, 2019, at 11:00 PM, Andrey Zagrebin 
wrote:

Hi all,

This is the next follow up discussion about suggestions for the

recent

thread about code style guide in Flink [1].

In general, one could argue that any variable, which is nullable,

can

be

replaced by wrapping it with Optional to explicitly show that it

can

be

null. Examples are:

   - returned values to force user to check not null
   - optional function arguments, e.g. with implicit default values
   - even class fields as e.g. optional config options with implicit
   default values


At the same time, we also have @Nullable annotation to express this
intention.

Also, when the class Optional was introduced, Oracle posted a

guideline

about its usage [2]. Basically, it suggests to use it mostly in

APIs

for

returned values to inform and force users to check the returned

value

instead of returning null and avoid NullPointerException.

Wrapping with Optional also comes with the performance overhead.

Following the Oracle's guide in general, the suggestion is:

   - Avoid using Optional in any performance critical code
   - Use Optional only to return nullable values in the API/public

methods

   unless it is performance critical then rather use @Nullable
   - Passing an Optional argument to a method can be allowed if it

is

   within a private helper method and simplifies the code, example

is

in

[3]

   - Optional should not be used for class fields


Please, feel free to share you thoughts.

Best,
Andrey

[1]


http://mail-archives.apache.org/mod_mbox/flink-dev/201906.mbox/%3ced91df4b-7cab-4547-a430-85bc710fd...@apache.org%3E

[2]


https://www.oracle.com/technetwork/articles/java/java8-optional-2175753.html

[3]


https://github.com/apache/flink/blob/master/flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/typeutils/AvroFactory.java#L95






[jira] [Created] (FLINK-13557) Improve failover strategy documentation with explanation of concepts

2019-08-02 Thread Till Rohrmann (JIRA)
Till Rohrmann created FLINK-13557:
-

 Summary: Improve failover strategy documentation with explanation 
of concepts
 Key: FLINK-13557
 URL: https://issues.apache.org/jira/browse/FLINK-13557
 Project: Flink
  Issue Type: Improvement
  Components: Documentation
Affects Versions: 1.9.0, 1.10.0
Reporter: Till Rohrmann


The current failover strategy configuration documentation 
(https://ci.apache.org/projects/flink/flink-docs-master/dev/task_failure_recovery.html#failover-strategies)
 could be improved by explaining the concept of pipelined regions in more 
detail. It could benefit from some figures visualizing the concept. We should 
even consider adding the explanation of pipelined regions to Flink's concept 
page because it is quite central to how Flink (will) work.



--
This message was sent by Atlassian JIRA
(v7.6.14#76016)


Re: [DISCUSS][CODE STYLE] Usage of Java Optional

2019-08-02 Thread JingsongLee
Hi, 
First, Optional is just a wrapper, just like boxed value. So as long as it's
 not a field level operation, I think it is OK to performance.
I think guava optional has a good summary to the uses. [1]
> As a method return type, as an alternative to returning null to indicate 
that no value was available
> To distinguish between "unknown" (for example, not present in a map) 
and "known to have no value"
> To wrap nullable references for storage in a collection that does not support
The latter two points seem reasonable, but they have few scenes.

[1] 
https://github.com/google/guava/blob/master/guava/src/com/google/common/base/Optional.java

Best,
Jingsong Lee


--
From:Timo Walther 
Send Time:2019年8月2日(星期五) 14:12
To:dev 
Subject:Re: [DISCUSS][CODE STYLE] Usage of Java Optional

Hi everyone,

I would vote for using Optional only as method return type for 
non-performance critical code. Nothing more. No fields, no method 
parameters. Method parameters can be overloaded and internally a class 
can work with nulls and @Nullable. Optional is meant for API method 
return types and I think we should not abuse it and spam the code with 
`@SuppressWarnings("OptionalUsedAsFieldOrParameterType")`.

Regards,

Timo



Am 02.08.19 um 11:08 schrieb Biao Liu:
> Hi Jark & Zili,
>
> I thought it means "Optional should not be used for class fields". However
> now I get a bit confused about the edited version.
>
> Anyway +1 to "Optional should not be used for class fields"
>
> Thanks,
> Biao /'bɪ.aʊ/
>
>
>
> On Fri, Aug 2, 2019 at 5:00 PM Zili Chen  wrote:
>
>> Hi Jark,
>>
>> Follow your opinion, for class field, we can make
>> use of @Nullable/@Nonnull annotation or Flink's
>> SerializableOptional. It would be sufficient.
>>
>> Best,
>> tison.
>>
>>
>> Jark Wu  于2019年8月2日周五 下午4:57写道:
>>
>>> Hi Andrey,
>>>
>>> I have some concern on point (3) "even class fields as e.g. optional
>> config
>>> options with implicit default values".
>>>
>>> Regarding to the Oracle's guide (4) "Optional should not be used for
>> class
>>> fields".
>>> And IntelliJ IDEA also report warnings if a class field is Optional,
>>> because Optional is not serializable.
>>>
>>>
>>> Do we allow Optional as class field only if the class is not serializable
>>> or forbid this totally?
>>>
>>> Thanks,
>>> Jark
>>>
>>> On Fri, 2 Aug 2019 at 16:30, Biao Liu  wrote:
>>>
 Hi Andrey,

 Thanks for working on this.

 +1 it's clear and acceptable for me.

 To Qi,

 IMO the most performance critical codes are "per record" code path. We
 should definitely avoid Optional there. For your concern, it's "per
>>> buffer"
 code path which seems to be acceptable with Optional.

 Just one more question, is there any other code paths which are also
 critical? I think we'd better note that clearly.

 Thanks,
 Biao /'bɪ.aʊ/



 On Fri, Aug 2, 2019 at 11:08 AM qi luo  wrote:

> Agree that using Optional will improve code robustness. However we’re
> hesitating to use Optional in data intensive operations.
>
> For example, SingleInputGate is already creating Optional for every
> BufferOrEvent in getNextBufferOrEvent(). How much performance gain
>>> would
 we
> get if it’s replaced by null check?
>
> Regards,
> Qi
>
>> On Aug 1, 2019, at 11:00 PM, Andrey Zagrebin  wrote:
>> Hi all,
>>
>> This is the next follow up discussion about suggestions for the
>>> recent
>> thread about code style guide in Flink [1].
>>
>> In general, one could argue that any variable, which is nullable,
>> can
 be
>> replaced by wrapping it with Optional to explicitly show that it
>> can
>>> be
>> null. Examples are:
>>
>>- returned values to force user to check not null
>>- optional function arguments, e.g. with implicit default values
>>- even class fields as e.g. optional config options with implicit
>>default values
>>
>>
>> At the same time, we also have @Nullable annotation to express this
>> intention.
>>
>> Also, when the class Optional was introduced, Oracle posted a
>>> guideline
>> about its usage [2]. Basically, it suggests to use it mostly in
>> APIs
 for
>> returned values to inform and force users to check the returned
>> value
>> instead of returning null and avoid NullPointerException.
>>
>> Wrapping with Optional also comes with the performance overhead.
>>
>> Following the Oracle's guide in general, the suggestion is:
>>
>>- Avoid using Optional in any performance critical code
>>- Use Optional only to return nullable values in the API/public
 methods
>>unless it is performance critical then rather use @Nullable
>>- Passing an Optional argument to a method can be allowed if it
>> is
>>within a private helper method 

Re: [RESULT][VOTE] Migrate to sponsored Travis account

2019-08-02 Thread Chesnay Schepler

Update: Implemented and deployed.

On 02/08/2019 12:11, Jark Wu wrote:

Wow. That's great! Thanks Chesnay.

On Fri, 2 Aug 2019 at 17:50, Chesnay Schepler > wrote:


I'm currently modifying the cibot to do this automatically; should be
finished until Monday.

On 02/08/2019 07:41, Jark Wu wrote:
> Hi Chesnay,
>
> Can we assign Flink Committers the permission of flink-ci/flink
repo?
> Several times, when I pushed some new commits, the old build
jobs are still
> in pending and not canceled.
> Before we fix that, we can manually cancel some old jobs to save
build
> resource.
>
> Best,
> Jark
>
>
> On Wed, 10 Jul 2019 at 16:17, Chesnay Schepler
mailto:ches...@apache.org>> wrote:
>
>> Your best bet would be to check the first commit in the PR and
check the
>> parent commit.
>>
>> To re-run things, you will have to rebase the PR on the latest
master.
>>
>> On 10/07/2019 03:32, Kurt Young wrote:
>>> Thanks for all your efforts Chesnay, it indeed improves a lot
for our
>>> develop experience. BTW, do you know how to find the master branch
>>> information which the CI runs with?
>>>
>>> For example, like this one:
>>> https://travis-ci.com/flink-ci/flink/jobs/214542568
>>> It shows pass with the commits, which rebased on the master
when the CI
>>> is triggered. But it's both possible that the master branch CI
runs on is
>>> the
>>> same or different with current master. If it's the same, I can
simply
>> rely
>>> on the
>>> passed information to push commits, but if it's not, I think i
should
>> find
>>> another
>>> way to re-trigger tests based on the newest master.
>>>
>>> Do you know where can I get such information?
>>>
>>> Best,
>>> Kurt
>>>
>>>
>>> On Tue, Jul 9, 2019 at 3:27 AM Chesnay Schepler
mailto:ches...@apache.org>>
>> wrote:
 The kinks have been worked out; the bot is running again and
pr builds
 are yet again no longer running on ASF resources.

 PRs are mirrored to: https://github.com/flink-ci/flink
 Bot source: https://github.com/flink-ci/ci-bot

 On 08/07/2019 17:14, Chesnay Schepler wrote:
> I have temporarily re-enabled running PR builds on the ASF
account;
> migrating to the Travis subscription caused some issues in
the bot
> that I have to fix first.
>
> On 07/07/2019 23:01, Chesnay Schepler wrote:
>> The vote has passed unanimously in favor of migrating to a
separate
>> Travis account.
>>
>> I will now set things up such that no PullRequest is no
longer run on
>> the ASF servers.
>> This is a major setup in reducing our usage of ASF resources.
>> For the time being we'll use free Travis plan for flink-ci
(i.e. 5
>> workers, which is the same the ASF gives us). Over the
course of the
>> next week we'll setup the Ververica subscription to
increase this
>> limit.
>>   From now now, a bot will mirror all new and updated
PullRequests to a
>> mirror repository (https://github.com/flink-ci/flink-ci)
and write an
>> update into the PR once the build is complete.
>> I have ran the bots for the past 3 days in parallel to our
existing
>> Travis and it was working without major issues.
>>
>> The biggest change that contributors will see is that
there's no
>> longer a icon next to each commit. We may revisit this in
the future.
>>
>> I'll setup a repo with the source of the bot later.
>>
>> On 04/07/2019 10:46, Chesnay Schepler wrote:
>>> I've raised a JIRA
>>> with
INFRA to
>>> inquire whether it would be possible to switch to a
different Travis
>>> account, and if so what steps would need to be taken.
>>> We need a proper confirmation from INFRA since we are not
in full
>>> control of the flink repository (for example, we cannot
access the
>>> settings page).
>>>
>>> If this is indeed possible, Ververica is willing sponsor a
Travis
>>> account for the Flink project.
>>> This would provide us with more than enough resources than
we need.
>>>
>>> Since this makes the project more reliant on resources
provided by
>>> external companies I would like to vote on this.
>>>
>>> Please vote on this proposal, as follows:
>>> [ ] +1, Approve the migration to a Ververica-sponsored Travis
>>> account, provided that INFRA approves
>>> [ ] -1, Do not approach the migration to a Ververica-sponsored
>>> Travis a

[jira] [Created] (FLINK-13558) Include table examples in flink-dist

2019-08-02 Thread Dawid Wysakowicz (JIRA)
Dawid Wysakowicz created FLINK-13558:


 Summary: Include table examples in flink-dist
 Key: FLINK-13558
 URL: https://issues.apache.org/jira/browse/FLINK-13558
 Project: Flink
  Issue Type: Improvement
  Components: Examples, Table SQL / API
Affects Versions: 1.9.0
Reporter: Dawid Wysakowicz
Assignee: Dawid Wysakowicz
 Fix For: 1.9.0


We want to treat the table api as first-class API. We already included in the 
lib directory flink.
We should also include some examples of the table api in the distribution.

Before that we should strip all the dependency and just include the classes 
from  example module.



--
This message was sent by Atlassian JIRA
(v7.6.14#76016)


Re: [DISCUSS][CODE STYLE] Usage of Java Optional

2019-08-02 Thread Yu Li
TL; DR: I second Timo that we should use Optional only as method return
type for non-performance critical code.

>From the example given on our AvroFactory [1] I also noticed that Jetbrains
marks the OptionalUsedAsFieldOrParameterType inspection as a warning. It's
relatively easy to understand why it's not suggested to use (java.util)
Optional as a field since it's not serializable. What made me feel curious
is that why we shouldn't use it as a parameter type, so I did some
investigation and here is what I found:

There's a JB blog talking about java8 top tips [2] where we could find the
advice around Optional, there I found another blog telling about the
pragmatic approach of using Optional [3]. Reading further we could see the
reason why we shouldn't use Optional as parameter type, please allow me to
quote here:

It is often the case that domain objects hang about in memory for a fair
while, as processing in the application occurs, making each optional
instance rather long-lived (tied to the lifetime of the domain object). By
contrast, the Optionalinstance returned from the getter is likely to be
very short-lived. The caller will call the getter, interpret the result,
and then move on. If you know anything about garbage collection you'll know
that the JVM handles these short-lived objects well. In addition, there is
more potential for hotspot to remove the costs of the Optional instance
when it is short lived. While it is easy to claim this is "premature
optimization", as engineers it is our responsibility to know the limits and
capabilities of the system we work with and to choose carefully the point
where it should be stressed.

And there's another JB blog about code smell on Null [4], which I'd also
suggest to read(smile).

[1]
https://github.com/apache/flink/blob/master/flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/typeutils/AvroFactory.java#L95
[2] https://blog.jetbrains.com/idea/2016/07/java-8-top-tips/
[3] https://blog.joda.org/2015/08/java-se-8-optional-pragmatic-approach.html
[4] https://blog.jetbrains.com/idea/2017/08/code-smells-null/

Best Regards,
Yu


On Fri, 2 Aug 2019 at 14:54, JingsongLee 
wrote:

> Hi,
> First, Optional is just a wrapper, just like boxed value. So as long as
> it's
>  not a field level operation, I think it is OK to performance.
> I think guava optional has a good summary to the uses. [1]
> > As a method return type, as an alternative to returning null to indicate
> that no value was available
> > To distinguish between "unknown" (for example, not present in a map)
> and "known to have no value"
> > To wrap nullable references for storage in a collection that does not
> support
> The latter two points seem reasonable, but they have few scenes.
>
> [1]
> https://github.com/google/guava/blob/master/guava/src/com/google/common/base/Optional.java
>
> Best,
> Jingsong Lee
>
>
> --
> From:Timo Walther 
> Send Time:2019年8月2日(星期五) 14:12
> To:dev 
> Subject:Re: [DISCUSS][CODE STYLE] Usage of Java Optional
>
> Hi everyone,
>
> I would vote for using Optional only as method return type for
> non-performance critical code. Nothing more. No fields, no method
> parameters. Method parameters can be overloaded and internally a class
> can work with nulls and @Nullable. Optional is meant for API method
> return types and I think we should not abuse it and spam the code with
> `@SuppressWarnings("OptionalUsedAsFieldOrParameterType")`.
>
> Regards,
>
> Timo
>
>
>
> Am 02.08.19 um 11:08 schrieb Biao Liu:
> > Hi Jark & Zili,
> >
> > I thought it means "Optional should not be used for class fields".
> However
> > now I get a bit confused about the edited version.
> >
> > Anyway +1 to "Optional should not be used for class fields"
> >
> > Thanks,
> > Biao /'bɪ.aʊ/
> >
> >
> >
> > On Fri, Aug 2, 2019 at 5:00 PM Zili Chen  wrote:
> >
> >> Hi Jark,
> >>
> >> Follow your opinion, for class field, we can make
> >> use of @Nullable/@Nonnull annotation or Flink's
> >> SerializableOptional. It would be sufficient.
> >>
> >> Best,
> >> tison.
> >>
> >>
> >> Jark Wu  于2019年8月2日周五 下午4:57写道:
> >>
> >>> Hi Andrey,
> >>>
> >>> I have some concern on point (3) "even class fields as e.g. optional
> >> config
> >>> options with implicit default values".
> >>>
> >>> Regarding to the Oracle's guide (4) "Optional should not be used for
> >> class
> >>> fields".
> >>> And IntelliJ IDEA also report warnings if a class field is Optional,
> >>> because Optional is not serializable.
> >>>
> >>>
> >>> Do we allow Optional as class field only if the class is not
> serializable
> >>> or forbid this totally?
> >>>
> >>> Thanks,
> >>> Jark
> >>>
> >>> On Fri, 2 Aug 2019 at 16:30, Biao Liu  wrote:
> >>>
>  Hi Andrey,
> 
>  Thanks for working on this.
> 
>  +1 it's clear and acceptable for me.
> 
>  To Qi,
> 
>  IMO the most performance critical codes are "per record" code path. We
>  should definit

Re: instable checkpointing after migration to flink 1.8

2019-08-02 Thread Congxian Qiu
Hi Bekir

Cloud you please also share the below information:
- jobmanager.log
- taskmanager.log(with debug info enabled) for the problematic subtask.
- the DAG of your program (if can provide the skeleton program is better --
can send to me privately)

For the subIndex, maybe we can use the deploy log message in jobmanager log
to identify which subtask we want. For example in JM log, we'll have
something like "2019-08-02 11:38:47,291 INFO
org.apache.flink.runtime.executiongraph.ExecutionGraph - Deploying Source:
Custom Source (2/2) (attempt #0) to
container_e62_1551952890130_2071_01_02 @ aa.bb.cc.dd.ee
(dataPort=39488)" then we know "Custum Source (2/2)" was deplyed to "
aa.bb.cc.dd.ee" with port 39488. Sadly, there maybe still more than one
subtasks in one contain :(

Best,
Congxian


Bekir Oguz  于2019年8月2日周五 下午4:22写道:

> Forgot to add the checkpoint details after it was complete. This is for
> that long running checkpoint with id 95632.
>
>
>
> Op 2 aug. 2019, om 11:18 heeft Bekir Oguz  het
> volgende geschreven:
>
> Hi Congxian,
> I was able to fetch the logs of the task manager (attached) and the
> screenshots of the latest long checkpoint. I will get the logs of the job
> manager for the next long running checkpoint. And also I will try to get a
> jstack during the long running checkpoint.
>
> Note: Since at the Subtasks tab we do not have the subtask numbers, and at
> the Details tab of the checkpoint, we have the subtask numbers but not the
> task manager hosts, it is difficult to match those. We’re assuming they
> have the same order, so seeing that 3rd subtask is failing, I am getting
> the 3rd line at the Subtasks tab which leads to the task manager
> host flink-taskmanager-84ccd5bddf-2cbxn. ***It would be a great feature if
> you guys also include the subtask-id’s to the Subtasks view.***
>
> Note: timestamps in the task manager log are in UTC and I am at the moment
> at zone UTC+3, so the time 10:30 at the screenshot matches the time 7:30 in
> the log.
>
>
> Kind regards,
> Bekir
>
> 
>
> 
> 
> 
>
>
>
> Op 2 aug. 2019, om 07:23 heeft Congxian Qiu  het
> volgende geschreven:
>
> Hi Bekir
> I’ll first summary the problem here(please correct me if I’m wrong)
> 1. The same program runs on 1.6 never encounter such problems
> 2. Some checkpoints completed too long (15+ min), but other normal
> checkpoints complete less than 1 min
> 3. Some  bad checkpoint will have a large sync time, async time seems ok
> 4. Some bad checkpoint, the e2e duration will much bigger than (sync_time
> + async_time)
> First, answer the last question, the e2e duration is ack_time -
> trigger_time, so it always bigger than (sync_time + async_time), but we
> have a big gap here, this may be problematic.
> According to all the information, maybe the problem is some task start to
> do checkpoint too late and the sync checkpoint part took some time too
> long, Could you please share some more information such below:
> - A Screenshot of summary for one bad checkpoint(we call it A here)
> - The detailed information of checkpoint A(includes all the problematic
> subtasks)
> - Jobmanager.log and the taskmanager.log for the problematic task and a
> health task
> - Share the screenshot of subtasks for the problematic task(includes the
> `Bytes received`, `Records received`, `Bytes sent`, `Records sent` column),
> here wants to compare the problematic parallelism and good parallelism’s
> information, please also share the information is there has a data skew
> among the parallelisms,
> - could you please share some jstacks of the problematic parallelism —
> here wants to check whether the task is too busy to handle the barrier.
> (flame graph or other things is always welcome here)
>
> Best,
> Congxian
>
>
> Congxian Qiu  于2019年8月1日周四 下午8:26写道:
>
>> Hi Bekir
>>
>> I'll first comb through all the information here, and try to find out the
>> reason with you, maybe need you to share some more information :)
>>
>> Best,
>> Congxian
>>
>>
>> Bekir Oguz  于2019年8月1日周四 下午5:00写道:
>>
>>> Hi Fabian,
>>> Thanks for sharing this with us, but we’re already on version 1.8.1.
>>>
>>> What I don’t understand is which mechanism in Flink adds 15 minutes to
>>> the checkpoint duration occasionally. Can you maybe give us some hints on
>>> where to look at? Is there a default timeout of 15 minutes defined
>>> somewhere in Flink? I couldn’t find one.
>>>
>>> In our pipeline, most of the checkpoints complete in less than a minute
>>> and some of them completed in 15 minutes+(less than a minute).
>>> There’s definitely something which adds 15 minutes. This is happening in
>>> one or more subtasks during checkpointing.
>>>
>>> Please see the screenshot below:
>>>
>>> Regards,
>>> Bekir
>>>
>>>
>>>
>>> Op 23 jul. 2019, om 16:37 heeft Fabian Hueske  het
>>> volgende geschreven:
>>>
>>> Hi Bekir,
>>>
>>> Another user reported checkpointing issues with Flink 1.8.0 [1].
>>> These seem to be resolved with Flink 1.8.1.
>>>
>>> Hope this helps,
>>> Fabian
>>>
>>> [

[jira] [Created] (FLINK-13559) Expose YARN Dynamic configuration option flag for flink CLI run

2019-08-02 Thread Gyula Fora (JIRA)
Gyula Fora created FLINK-13559:
--

 Summary: Expose YARN Dynamic configuration option flag for flink 
CLI run
 Key: FLINK-13559
 URL: https://issues.apache.org/jira/browse/FLINK-13559
 Project: Flink
  Issue Type: New Feature
  Components: Command Line Client
Reporter: Gyula Fora


When running flink on yarn, the yarn-session component understand a dynamic 
option  parameter which allows us to define/overwrite flink conf params from 
the deploy command.

This is however not accessible when we use the flink run command to start a 
single job yarn cluster as the command line client fails on this unknown 
parameter.

Simply adding this extra option to the flink run-options would let flink 
forward this parameter to the yarn session thus enabling this already existing 
feature.

In addition I suggest we add the extra "conf" long option name for this 
parameter:

flink run {{-m yarn-cluster}} --conf key=value ...
or
flink run {{-m yarn-cluster}} --Dkey=value ...



--
This message was sent by Atlassian JIRA
(v7.6.14#76016)


[jira] [Created] (FLINK-13560) Expose YARN ship files option flag for flink CLI run

2019-08-02 Thread Gyula Fora (JIRA)
Gyula Fora created FLINK-13560:
--

 Summary: Expose YARN ship files option flag for flink CLI run
 Key: FLINK-13560
 URL: https://issues.apache.org/jira/browse/FLINK-13560
 Project: Flink
  Issue Type: New Feature
  Components: Command Line Client
Reporter: Gyula Fora


When running flink on yarn, the yarn-session component allows the user to ship 
extra libraries to the cluster useing the --ship / -t parameter.

This is however not accessible when we use the flink run command to start a 
single job yarn cluster as the command line client fails on this unknown 
parameter.

We should expose this on the flink run command to enable this already existing 
feature for single job yarn apps.

flink run {{-m yarn-cluster}} --ship shipFiles



--
This message was sent by Atlassian JIRA
(v7.6.14#76016)


[jira] [Created] (FLINK-13561) Verify and correct builtin time function's semantic for Blink planner

2019-08-02 Thread Jingsong Lee (JIRA)
Jingsong Lee created FLINK-13561:


 Summary: Verify and correct builtin time function's semantic for 
Blink planner
 Key: FLINK-13561
 URL: https://issues.apache.org/jira/browse/FLINK-13561
 Project: Flink
  Issue Type: Sub-task
  Components: Table SQL / Planner
Reporter: Jingsong Lee
 Fix For: 1.9.0, 1.10.0


Some time function should be corrected:

toTimestamp('2016-03-31') not support in blink.

unix_timestamp and from_unixtime should care about time zone.



--
This message was sent by Atlassian JIRA
(v7.6.14#76016)


Re: REST API / JarRunHandler: More flexibility for launching jobs

2019-08-02 Thread Thomas Weise
Thanks for looking into this.

I see the "Jar run handler" as function that takes few parameters and
returns a job ID. I think it would be nice if the handler doesn't hard code
the function. Perhaps this could be accomplished by pushing the code into
something like "ExtractJobGraphAndSubmitToDispatcherEnvironment" that the
main method could also bypass if it has an alternative way to provide the
jobId via a context variable?

Zili: I looked at the client API proposal and left a few comments. I think
it is important to improve programmatic job submission. But it also seems
orthogonal to how the jar run handler operates (i.e. these issues could be
addressed independently).

Chesnay: You are right that the Beam job sever could be hacked to extract
job graph and other ingredients. This isn't desirable though because these
Flink internals should not be exposed downstream. But even if we went down
that route we would still need a way to let the jar run handler know to
just return the ID of an already submitted job vs. trying to submit one
from OptimizerPlanEnvironment.

The intended sequence would be:

REST client provides a launcher jar
REST client "runs jar"
REST handler calls main()
main launches Beam job server, runs Beam pipeline construction code against
that job server
job server uses RemoteEnvironment to submit real job
main "returns job id"
REST handler returns job id

Thomas


On Wed, Jul 31, 2019 at 4:33 AM Zili Chen  wrote:

> By the way, currently Dispatcher implements RestfulGateway
> and delegate resource request to ResourceManager. If we can,
> semantically, let WebMonitor implement RestfulGateway,
> and delegate job request to Dispatcher, resource request to
> ResourceManager, it seems reasonable that when WebMonitor
> receives a JarRun request, it spawns a process and run
> the main method of the main class of that jar.
>
> Best,
> tison.
>
>
> Zili Chen  于2019年7月31日周三 下午7:10写道:
>
>> I don't think the `Program` interface could solve the problem.
>>
>> The launcher launches the job server which creates the job graph,
>> submits it and keeps monitoring. Even if user program implement
>> `Program` Flink still extracts the JobGraph from `getPlan` and
>> submits it, instead of really execute codes in main method of
>> user program, so that the launcher is not started.
>>
>> @Thomas,
>>
>> Here is an ongoing discussion on client refactoring[1] as Till
>> mentioned. However, I'm afraid that with current jar run semantic,
>> i.e., extract the job graph and submit it to the Dispatcher, it cannot
>> fits your requirement. The problem is that REST API directly
>> communicates with Dispatcher and thus it's strange to tell the
>> Dispatcher "just run a program in a process".
>>
>> As you mentioned in the document, with CLI in session mode the
>> whole program would be executed sequentially. I'll appreciate it
>> if you can participant the thread on client refactor[1]. In the
>> design document[2], we propose to provide rich interfaces for
>> downstream projects integration. You can customize your CLI for
>> executing your program arbitrarily. Any requirement or advise
>> would be help.
>>
>> Best,
>> tison.
>>
>> [1]
>> https://lists.apache.org/thread.html/ce99cba4a10b9dc40eb729d39910f315ae41d80ec74f09a356c73938@%3Cdev.flink.apache.org%3E
>> [2]
>> https://docs.google.com/document/d/1UWJE7eYWiMuZewBKS0YmdVO2LUTqXPd6-pbOCof9ddY/edit
>>
>>
>>
>>
>> Till Rohrmann  于2019年7月31日周三 下午4:50写道:
>>
>>> Are you looking for something similar to the `Program` interface? This
>>> interface, even though it is a bit outdated and might get removed in the
>>> future, offers a `getPlan` method which is called in order to generate
>>> the
>>> `JobGraph`. In the client refactoring discussion thread it is currently
>>> being discussed what to do with this interface.
>>>
>>> Cheers,
>>> Till
>>>
>>> On Wed, Jul 31, 2019 at 10:41 AM Chesnay Schepler 
>>> wrote:
>>>
>>> > Couldn't the beam job server use the same work-around we're using in
>>> the
>>> > JarRunHandler to get access to the JobGraph?
>>> >
>>> > On 26/07/2019 17:38, Thomas Weise wrote:
>>> > > Hi Till,
>>> > >
>>> > > Thanks for taking a look!
>>> > >
>>> > > The Beam job server does not currently have the ability to just
>>> output
>>> > the
>>> > > job graph (and related artifacts) that could then be used with the
>>> > > JobSubmitHandler. It is itself using StreamExecutionEnvironment,
>>> which in
>>> > > turn will lead to a REST API submission.
>>> > >
>>> > > Here I'm looking at what happens before the Beam job server gets
>>> > involved:
>>> > > the interaction of the k8s operator with the Flink deployment. The
>>> jar
>>> > run
>>> > > endpoint (ignoring the current handler implementation) is generic and
>>> > > pretty much exactly matches what we would need for a uniform entry
>>> point.
>>> > > It's just that in the Beam case the jar file would itself be a
>>> "launcher"
>>> > > that doesn't provide the job graph itself, but the dependencies and
>>> > > mechan

[jira] [Created] (FLINK-13562) throws exception when FlinkRelMdColumnInterval meets two stage stream group aggregate

2019-08-02 Thread godfrey he (JIRA)
godfrey he created FLINK-13562:
--

 Summary: throws exception when FlinkRelMdColumnInterval meets two 
stage stream group aggregate
 Key: FLINK-13562
 URL: https://issues.apache.org/jira/browse/FLINK-13562
 Project: Flink
  Issue Type: Bug
  Components: Table SQL / Planner
Reporter: godfrey he
 Fix For: 1.9.0, 1.10.0


test case:

{code:scala}
  @Test
  def testTwoDistinctAggregateWithNonDistinctAgg(): Unit = {
util.addTableSource[(Int, Long, String)]("MyTable", 'a, 'b, 'c)
util.verifyPlan("SELECT c, SUM(DISTINCT a), SUM(a), COUNT(DISTINCT b) FROM 
MyTable GROUP BY c")
  }
{code}



org.apache.flink.table.api.TableException: Sum aggregate function does not 
support type: ''VARCHAR''.
Please re-check the data type.

at 
org.apache.flink.table.planner.plan.utils.AggFunctionFactory.createSumAggFunction(AggFunctionFactory.scala:191)
at 
org.apache.flink.table.planner.plan.utils.AggFunctionFactory.createAggFunction(AggFunctionFactory.scala:74)
at 
org.apache.flink.table.planner.plan.utils.AggregateUtil$$anonfun$9.apply(AggregateUtil.scala:285)
at 
org.apache.flink.table.planner.plan.utils.AggregateUtil$$anonfun$9.apply(AggregateUtil.scala:279)
at 
scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
at 
scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
at 
scala.collection.mutable.ResizableArray$class.foreach(ResizableArray.scala:59)
at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:48)
at scala.collection.TraversableLike$class.map(TraversableLike.scala:234)
at scala.collection.AbstractTraversable.map(Traversable.scala:104)
at 
org.apache.flink.table.planner.plan.utils.AggregateUtil$.transformToAggregateInfoList(AggregateUtil.scala:279)
at 
org.apache.flink.table.planner.plan.utils.AggregateUtil$.getOutputIndexToAggCallIndexMap(AggregateUtil.scala:154)
at 
org.apache.flink.table.planner.plan.metadata.FlinkRelMdColumnInterval.getAggCallIndexInLocalAgg$1(FlinkRelMdColumnInterval.scala:504)
at 
org.apache.flink.table.planner.plan.metadata.FlinkRelMdColumnInterval.estimateColumnIntervalOfAggregate(FlinkRelMdColumnInterval.scala:526)
at 
org.apache.flink.table.planner.plan.metadata.FlinkRelMdColumnInterval.getColumnInterval(FlinkRelMdColumnInterval.scala:417)
at GeneratedMetadataHandler_ColumnInterval.getColumnInterval_$(Unknown 
Source)
at GeneratedMetadataHandler_ColumnInterval.getColumnInterval(Unknown 
Source)
at 
org.apache.flink.table.planner.plan.metadata.FlinkRelMetadataQuery.getColumnInterval(FlinkRelMetadataQuery.java:122)



--
This message was sent by Atlassian JIRA
(v7.6.14#76016)


[jira] [Created] (FLINK-13563) TumblingGroupWindow should implement toString method

2019-08-02 Thread godfrey he (JIRA)
godfrey he created FLINK-13563:
--

 Summary: TumblingGroupWindow should implement toString method
 Key: FLINK-13563
 URL: https://issues.apache.org/jira/browse/FLINK-13563
 Project: Flink
  Issue Type: Bug
  Components: Table SQL / Planner
Affects Versions: 1.9.0, 1.10.0
Reporter: godfrey he
 Fix For: 1.9.0, 1.10.0


{code:scala}
  @Test
  def testAllEventTimeTumblingGroupWindowOverTime(): Unit = {
val util = streamTestUtil()
val table = util.addDataStream[(Long, Int, String)](
  "T1", 'long, 'int, 'string, 'rowtime.rowtime)

val windowedTable = table
  .window(Tumble over 5.millis on 'rowtime as 'w)
  .groupBy('w)
  .select('int.count)
util.verifyPlan(windowedTable)
  }
{code}

currently, it's physical plan is 

{code:java}
HashWindowAggregate(window=[TumblingGroupWindow], select=[Final_COUNT(count$0) 
AS EXPR$0])
+- Exchange(distribution=[single])
   +- LocalHashWindowAggregate(window=[TumblingGroupWindow], 
select=[Partial_COUNT(int) AS count$0])
  +- TableSourceScan(table=[[default_catalog, default_database, Table1, 
source: [TestTableSource(long, int, string)]]], fields=[long, int, string])
{code}

we know nothing about the TumblingGroupWindow except its name




--
This message was sent by Atlassian JIRA
(v7.6.14#76016)


Re: [DISCUSS][CODE STYLE] Breaking long function argument lists and chained method calls

2019-08-02 Thread SHI Xiaogang
Hi Chesnay,

Thanks a lot for your reminder.

For Intellij settings, the style i proposed can be configured as below
* Method declaration parameters: chop down if long
* align when multiple: YES
* new line after '(': YES
* place ')' on new line: YES
* Method call arguments: chop down if long
* align when multiple: YES
* take priority over call chain wrapping: YES
* new line after '(': YES
* place ')' on new line: YES
* Throws list: chop down if long
* align when multiline: YES

As far as i know, there does not exist standard checks for the alignment of
method parameters or arguments. It needs further investigation to see
whether we can validate these styles via customized checks.


Biao Liu  于2019年8月2日周五 下午4:00写道:

> Hi Andrey,
>
> Thank you for bringing us this discussion.
>
> I would like to make some details clear. Correct me if I am wrong.
>
> The guide draft [1] says the line length is limited in 100 characters. From
> my understanding, this discussion suggests if there is more than 100
> characters in one line (both Scala and Java), we should start a new line
> (or lines).
>
> *Question 1*: If a line does not exceed 100 characters, should we break the
> chained calls into lines? Currently the chained calls always been broken
> into lines even it's not too long. Does it just a suggestion or a
> limitation?
> I prefer it's a limitation which must be respected. And we should always
> break the chained calls no matter how long the line is.
>
> For a chained method calls, the new line should be started with the dot.
>
> *Question 2:* The indent of new line should be 1 tab or 2 tabs? Currently
> there exists these two different styles. This rule should be also applied
> to function arguments.
>
> BTW, big +1 to options from Chesnay. We should make auto-format possible on
> our project.
>
> 1.
>
> https://docs.google.com/document/d/1owKfK1DwXA-w6qnx3R7t2D_o0BsFkkukGlRhvl3XXjQ/edit#
>
> Thanks,
> Biao /'bɪ.aʊ/
>
>
>
> On Fri, Aug 2, 2019 at 9:20 AM SHI Xiaogang 
> wrote:
>
> > Hi Andrey,
> >
> > Thanks for bringing this. Personally, I prefer to the following style
> which
> > (1) puts the right parenthese in the next line
> > (2) a new line for each exception if exceptions can not be put in the
> same
> > line
> >
> > That way, parentheses are aligned in a similar way to braces and
> exceptions
> > can be well aligned.
> >
> > *public **void func(*
> > *int arg1,*
> > *int arg2,*
> > *...
> > *) throws E1, E2, E3 {*
> > *...
> > *}*
> >
> > or
> >
> > *public **void func(*
> > *int arg1,*
> > *int arg2,*
> > *...
> > *) throws
> > *E1,
> > *E2,
> > *E3 {*
> > *...
> > *}*
> >
> > Regards,
> > Xiaogang
> >
> > Andrey Zagrebin  于2019年8月1日周四 下午11:19写道:
> >
> > > Hi all,
> > >
> > > This is one more small suggestion for the recent thread about code
> style
> > > guide in Flink [1].
> > >
> > > We already have a note about using a new line for each chained call in
> > > Scala, e.g. either:
> > >
> > > *values**.stream()**.map(...)**,collect(...);*
> > >
> > > or
> > >
> > > *values*
> > > *.stream()*
> > > *.map(*...*)*
> > > *.collect(...)*
> > >
> > > if it would result in a too long line by keeping all chained calls in
> one
> > > line.
> > >
> > > The suggestion is to have it for Java as well and add the same rule
> for a
> > > long list of function arguments. So it is either:
> > >
> > > *public void func(int arg1, int arg2, ...) throws E1, E2, E3 {*
> > > *...*
> > > *}*
> > >
> > > or
> > >
> > > *public **void func(*
> > > *int arg1,*
> > > *int arg2,*
> > > *...)** throws E1, E2, E3 {*
> > > *...*
> > > *}*
> > >
> > > but thrown exceptions stay on the same last line.
> > >
> > > Please, feel free to share you thoughts.
> > >
> > > Best,
> > > Andrey
> > >
> > > [1]
> > >
> > >
> >
> http://mail-archives.apache.org/mod_mbox/flink-dev/201906.mbox/%3ced91df4b-7cab-4547-a430-85bc710fd...@apache.org%3E
> > >
> >
>