Re: [DISCUSS][CODE STYLE] Create collections always with initial capacity

2019-08-20 Thread Andrey Zagrebin
I created an umbrella issue for the code style guide effort and a subtask
for this discussion:
https://issues.apache.org/jira/browse/FLINK-13804
I will also submit a PR to flink-web based on the conclusion.

On Mon, Aug 19, 2019 at 6:15 PM Stephan Ewen  wrote:

> @Andrey Will you open a PR to add this to the code style?
>
> On Mon, Aug 19, 2019 at 11:51 AM Andrey Zagrebin 
> wrote:
>
> > Hi All,
> >
> > It looks like this proposal has an approval and we can conclude this
> > discussion.
> > Additionally, I agree with Piotr we should really force the proven good
> > reasoning for setting the capacity to avoid confusion, redundancy and
> other
> > already mentioned things while reading and maintaining the code.
> > Ideally the need of setting the capacity should be either immediately
> clear
> > (e.g. perf etc) or explained in comments if it is non-trivial.
> > Although, it can easily enter a grey zone, so I would not demand strictly
> > performance measurement proof e.g. if the size is known and it is "per
> > record" code.
> > At the end of the day it is a decision of the code developer and
> reviewer.
> >
> > The conclusion is then:
> > Set the initial capacity only if there is a good proven reason to do it.
> > Otherwise do not clutter the code with it.
> >
> > Best,
> > Andrey
> >
> > On Thu, Aug 1, 2019 at 5:10 PM Piotr Nowojski 
> wrote:
> >
> > > Hi,
> > >
> > > > - a bit more code, increases maintenance burden.
> > >
> > > I think there is even more to that. It’s almost like a code
> duplication,
> > > albeit expressed in very different way, with all of the drawbacks of
> > > duplicated code: initial capacity can drift out of sync, causing
> > confusion.
> > > Also it’s not “a bit more code”, it might be non trivial
> > > reasoning/calculation how to set the initial value. Whenever we change
> > > something/refactor the code, "maintenance burden” will mostly come from
> > > that.
> > >
> > > Also I think this just usually falls under a premature optimisation
> rule.
> > >
> > > Besides:
> > >
> > > > The conclusion is the following at the moment:
> > > > Only set the initial capacity if you have a good idea about the
> > expected
> > > size.
> > >
> > > I would add a clause to set the initial capacity “only for good proven
> > > reasons”. It’s not about whether we can set it, but whether it makes
> > sense
> > > to do so (to avoid the before mentioned "maintenance burden”).
> > >
> > > Piotrek
> > >
> > > > On 1 Aug 2019, at 14:41, Xintong Song  wrote:
> > > >
> > > > +1 on setting initial capacity only when have good expectation on the
> > > > collection size.
> > > >
> > > > Thank you~
> > > >
> > > > Xintong Song
> > > >
> > > >
> > > >
> > > > On Thu, Aug 1, 2019 at 2:32 PM Andrey Zagrebin  >
> > > wrote:
> > > >
> > > >> Hi all,
> > > >>
> > > >> As you probably already noticed, Stephan has triggered a discussion
> > > thread
> > > >> about code style guide for Flink [1]. Recently we were discussing
> > > >> internally some smaller concerns and I would like start separate
> > threads
> > > >> for them.
> > > >>
> > > >> This thread is about creating collections always with initial
> > capacity.
> > > As
> > > >> you might have seen, some parts of our code base always initialise
> > > >> collections with some non-default capacity. You can even activate a
> > > check
> > > >> in IntelliJ Idea that can monitor and highlight creation of
> collection
> > > >> without initial capacity.
> > > >>
> > > >> Pros:
> > > >> - performance gain if there is a good reasoning about initial
> capacity
> > > >> - the capacity is always deterministic and does not depend on any
> > > changes
> > > >> of its default value in Java
> > > >> - easy to follow: always initialise, has IDE support for detection
> > > >>
> > > >> Cons (for initialising w/o good reasoning):
> > > >> - We are trying to outsmart JVM. When there is no good reasoning
> about
> > > >> initial capacity, we can rely on JVM default value.
> > > >> - It is even confusing e.g. for hash maps as the real size depends
> on
> > > the
> > > >> load factor.
> > > >> - It would only add minor performance gain.
> > > >> - a bit more code, increases maintenance burden.
> > > >>
> > > >> The conclusion is the following at the moment:
> > > >> Only set the initial capacity if you have a good idea about the
> > expected
> > > >> size.
> > > >>
> > > >> 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] Usage of Java Optional

2019-08-21 Thread Andrey Zagrebin
Hi All,

It looks like we have reached a consensus regarding the last left question.

I suggest the following final summary:

   - Use @Nullable annotation where you do not use Optional for the
   nullable values
   - If you can prove that Optional usage would lead to a performance
   degradation in critical code then fallback to @Nullable
   - Always use Optional to return nullable values in the API/public
   methods except the case of a proven performance concern
   - Do not use Optional as a function argument, instead either overload
   the method or use the Builder pattern for the set of function arguments
  - Note: an Optional argument can be allowed in a *private* helper
  method if you believe that it simplifies the code, example is in [1]
  - Do not use Optional for class fields

If there are no more comments/concerns/objections I will open a PR to
reflect this in the code style guide.

Bets,
Andrey

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

On Tue, Aug 20, 2019 at 10:35 AM Yu Li  wrote:

> Thanks for the summarize Andrey!
>
> I'd also like to adjust my -1 to +0 on using Optional as parameter for
> private methods due to the existence of the very first rule - "Avoid using
> Optional in any performance critical code". I'd regard the "possible GC
> burden while using Optional as parameter" also one performance related
> factor.
>
> And besides the code convention itself, I believe it's even more important
> to make us contributors know the reason behind.
>
> Thanks.
>
> Best Regards,
> Yu
>
>
> On Tue, 20 Aug 2019 at 10:14, Stephan Ewen  wrote:
>
> > I think Dawid raised a very good point here.
> > One of the outcomes should be that we are consistent in our
> recommendations
> > and requests during PR reviews. Otherwise we'll just confuse
> contributors.
> >
> > So I would be
> >   +1 for someone to use Optional in a private method if they believe it
> is
> > helpful
> >   -1 to push contributors during reviews to do that
> >
> >
> > On Tue, Aug 20, 2019 at 9:42 AM Dawid Wysakowicz  >
> > wrote:
> >
> > > Hi Andrey,
> > >
> > > Just wanted to quickly elaborate on my opinion. I wouldn't say I am -1,
> > > just -0 for the Optionals in private methods. I am ok with not
> > > forbidding them there. I just think in all cases there is a better
> > > solution than passing the Optionals around, even in private methods. I
> > > just hope the outcome of the discussion won't be that it is no longer
> > > allowed to suggest those during review.
> > >
> > > Best,
> > >
> > > Dawid
> > >
> > > On 19/08/2019 17:53, Andrey Zagrebin wrote:
> > > > Hi all,
> > > >
> > > > Sorry for not getting back to this discussion for some time.
> > > > It looks like in general we agree on the initially suggested points:
> > > >
> > > >- Always use Optional only to return nullable values in the
> > API/public
> > > >methods
> > > >   - Only if you can prove that Optional usage would lead to a
> > > >   performance degradation in critical code then return nullable
> > value
> > > >   directly and annotate it with @Nullable
> > > >- Passing an Optional argument to a method can be allowed if it is
> > > >within a private helper method and simplifies the code
> > > >- Optional should not be used for class fields
> > > >
> > > > The first point can be also elaborated by explicitly forbiding
> > > > Optional/Nullable parameters in public methods.
> > > > In general we can always avoid Optional parameters by either
> > overloading
> > > > the method or using a pojo with a builder to pass a set of
> parameters.
> > > >
> > > > The third point does not prevent from using @Nullable/@Nonnull for
> > class
> > > > fields.
> > > > If we agree to not use Optional for fields then not sure I see any
> use
> > > case
> > > > for SerializableOptional (please comment on that if you have more
> > > details).
> > > >
> > > > @Jingsong Lee
> > > > Using Optional in Maps.
> > > > I can see this as a possible use case.
> > > > I would leave this decision to the developer/reviewer to reason about
> > it
> > > > and keep the scope of this discussion to the variables/parameters as
> it
&g

Re: [DISCUSS][CODE STYLE] Create collections always with initial capacity

2019-08-21 Thread Andrey Zagrebin
FYI the PR: https://github.com/apache/flink-web/pull/249
A review from the discussion participants would be appreciated.

On Tue, Aug 20, 2019 at 5:29 PM Andrey Zagrebin 
wrote:

> I created an umbrella issue for the code style guide effort and a subtask
> for this discussion:
> https://issues.apache.org/jira/browse/FLINK-13804
> I will also submit a PR to flink-web based on the conclusion.
>
> On Mon, Aug 19, 2019 at 6:15 PM Stephan Ewen  wrote:
>
>> @Andrey Will you open a PR to add this to the code style?
>>
>> On Mon, Aug 19, 2019 at 11:51 AM Andrey Zagrebin 
>> wrote:
>>
>> > Hi All,
>> >
>> > It looks like this proposal has an approval and we can conclude this
>> > discussion.
>> > Additionally, I agree with Piotr we should really force the proven good
>> > reasoning for setting the capacity to avoid confusion, redundancy and
>> other
>> > already mentioned things while reading and maintaining the code.
>> > Ideally the need of setting the capacity should be either immediately
>> clear
>> > (e.g. perf etc) or explained in comments if it is non-trivial.
>> > Although, it can easily enter a grey zone, so I would not demand
>> strictly
>> > performance measurement proof e.g. if the size is known and it is "per
>> > record" code.
>> > At the end of the day it is a decision of the code developer and
>> reviewer.
>> >
>> > The conclusion is then:
>> > Set the initial capacity only if there is a good proven reason to do it.
>> > Otherwise do not clutter the code with it.
>> >
>> > Best,
>> > Andrey
>> >
>> > On Thu, Aug 1, 2019 at 5:10 PM Piotr Nowojski 
>> wrote:
>> >
>> > > Hi,
>> > >
>> > > > - a bit more code, increases maintenance burden.
>> > >
>> > > I think there is even more to that. It’s almost like a code
>> duplication,
>> > > albeit expressed in very different way, with all of the drawbacks of
>> > > duplicated code: initial capacity can drift out of sync, causing
>> > confusion.
>> > > Also it’s not “a bit more code”, it might be non trivial
>> > > reasoning/calculation how to set the initial value. Whenever we change
>> > > something/refactor the code, "maintenance burden” will mostly come
>> from
>> > > that.
>> > >
>> > > Also I think this just usually falls under a premature optimisation
>> rule.
>> > >
>> > > Besides:
>> > >
>> > > > The conclusion is the following at the moment:
>> > > > Only set the initial capacity if you have a good idea about the
>> > expected
>> > > size.
>> > >
>> > > I would add a clause to set the initial capacity “only for good proven
>> > > reasons”. It’s not about whether we can set it, but whether it makes
>> > sense
>> > > to do so (to avoid the before mentioned "maintenance burden”).
>> > >
>> > > Piotrek
>> > >
>> > > > On 1 Aug 2019, at 14:41, Xintong Song 
>> wrote:
>> > > >
>> > > > +1 on setting initial capacity only when have good expectation on
>> the
>> > > > collection size.
>> > > >
>> > > > Thank you~
>> > > >
>> > > > Xintong Song
>> > > >
>> > > >
>> > > >
>> > > > On Thu, Aug 1, 2019 at 2:32 PM Andrey Zagrebin <
>> and...@ververica.com>
>> > > wrote:
>> > > >
>> > > >> Hi all,
>> > > >>
>> > > >> As you probably already noticed, Stephan has triggered a discussion
>> > > thread
>> > > >> about code style guide for Flink [1]. Recently we were discussing
>> > > >> internally some smaller concerns and I would like start separate
>> > threads
>> > > >> for them.
>> > > >>
>> > > >> This thread is about creating collections always with initial
>> > capacity.
>> > > As
>> > > >> you might have seen, some parts of our code base always initialise
>> > > >> collections with some non-default capacity. You can even activate a
>> > > check
>> > > >> in IntelliJ Idea that can monitor and highlight creation of
>> collection
>> > > >> without initial capacity.
>> > > >>
>> > > >> Pros:
>> > > >> - performance gain if there is a good reasoning about initial
>> capacity
>> > > >> - the capacity is always deterministic and does not depend on any
>> > > changes
>> > > >> of its default value in Java
>> > > >> - easy to follow: always initialise, has IDE support for detection
>> > > >>
>> > > >> Cons (for initialising w/o good reasoning):
>> > > >> - We are trying to outsmart JVM. When there is no good reasoning
>> about
>> > > >> initial capacity, we can rely on JVM default value.
>> > > >> - It is even confusing e.g. for hash maps as the real size depends
>> on
>> > > the
>> > > >> load factor.
>> > > >> - It would only add minor performance gain.
>> > > >> - a bit more code, increases maintenance burden.
>> > > >>
>> > > >> The conclusion is the following at the moment:
>> > > >> Only set the initial capacity if you have a good idea about the
>> > expected
>> > > >> size.
>> > > >>
>> > > >> 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] Usage of Java Optional

2019-08-21 Thread Andrey Zagrebin
FYI the PR: https://github.com/apache/flink-web/pull/251
A review from the discussion participants would be appreciated.

On Wed, Aug 21, 2019 at 1:23 PM Timo Walther  wrote:

> Thanks for summarizing the discussion Andrey, +1 to this style.
>
> Regards,
> Timo
>
>
> Am 21.08.19 um 11:57 schrieb Andrey Zagrebin:
> > Hi All,
> >
> > It looks like we have reached a consensus regarding the last left
> question.
> >
> > I suggest the following final summary:
> >
> > - Use @Nullable annotation where you do not use Optional for the
> > nullable values
> > - If you can prove that Optional usage would lead to a performance
> > degradation in critical code then fallback to @Nullable
> > - Always use Optional to return nullable values in the API/public
> > methods except the case of a proven performance concern
> > - Do not use Optional as a function argument, instead either overload
> > the method or use the Builder pattern for the set of function
> arguments
> >- Note: an Optional argument can be allowed in a *private* helper
> >method if you believe that it simplifies the code, example is in
> [1]
> >- Do not use Optional for class fields
> >
> > If there are no more comments/concerns/objections I will open a PR to
> > reflect this in the code style guide.
> >
> > Bets,
> > Andrey
> >
> > [1]
> >
> https://github.com/apache/flink/blob/master/flink-formats/flink-avro/src/main/java/org/apache/flink/formats/avro/typeutils/AvroFactory.java#L95
> >
> > On Tue, Aug 20, 2019 at 10:35 AM Yu Li  wrote:
> >
> >> Thanks for the summarize Andrey!
> >>
> >> I'd also like to adjust my -1 to +0 on using Optional as parameter for
> >> private methods due to the existence of the very first rule - "Avoid
> using
> >> Optional in any performance critical code". I'd regard the "possible GC
> >> burden while using Optional as parameter" also one performance related
> >> factor.
> >>
> >> And besides the code convention itself, I believe it's even more
> important
> >> to make us contributors know the reason behind.
> >>
> >> Thanks.
> >>
> >> Best Regards,
> >> Yu
> >>
> >>
> >> On Tue, 20 Aug 2019 at 10:14, Stephan Ewen  wrote:
> >>
> >>> I think Dawid raised a very good point here.
> >>> One of the outcomes should be that we are consistent in our
> >> recommendations
> >>> and requests during PR reviews. Otherwise we'll just confuse
> >> contributors.
> >>> So I would be
> >>>+1 for someone to use Optional in a private method if they believe
> it
> >> is
> >>> helpful
> >>>-1 to push contributors during reviews to do that
> >>>
> >>>
> >>> On Tue, Aug 20, 2019 at 9:42 AM Dawid Wysakowicz <
> dwysakow...@apache.org
> >>>
> >>> wrote:
> >>>
> >>>> Hi Andrey,
> >>>>
> >>>> Just wanted to quickly elaborate on my opinion. I wouldn't say I am
> -1,
> >>>> just -0 for the Optionals in private methods. I am ok with not
> >>>> forbidding them there. I just think in all cases there is a better
> >>>> solution than passing the Optionals around, even in private methods. I
> >>>> just hope the outcome of the discussion won't be that it is no longer
> >>>> allowed to suggest those during review.
> >>>>
> >>>> Best,
> >>>>
> >>>> Dawid
> >>>>
> >>>> On 19/08/2019 17:53, Andrey Zagrebin wrote:
> >>>>> Hi all,
> >>>>>
> >>>>> Sorry for not getting back to this discussion for some time.
> >>>>> It looks like in general we agree on the initially suggested points:
> >>>>>
> >>>>> - Always use Optional only to return nullable values in the
> >>> API/public
> >>>>> methods
> >>>>>- Only if you can prove that Optional usage would lead to a
> >>>>>performance degradation in critical code then return nullable
> >>> value
> >>>>>directly and annotate it with @Nullable
> >>>>> - Passing an Optional argument to a method can be allowed if it
> is
> >>>>> within a private helper method and 

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

2019-08-21 Thread Andrey Zagrebin
Hi All,

I suggest we also conclude this discussion now.

Breaking the line of too long statements (line longness is yet to be fully
defined) to improve code readability in case of

   - Long function argument lists (declaration or call): void func(type1
   arg1, type2 arg2, ...)
   - Long sequence of chained calls:
   list.stream().map(...).reduce(...).collect(...)...

Rules:

   - Break the list of arguments/calls if the line exceeds limit or earlier
   if you believe that the breaking would improve the code readability
   - If you break the line then each argument/call should have a separate
   line, including the first one
   - Each new line argument/call should have one extra indentation relative
   to the line of the parent function name or called entity
   - The opening brace always stays on the line of the parent function name
   - The closing brace of the function argument list and the possible
   thrown exception list always stay on the line of the last argument
   - The dot of a chained call is always on the line of that chained call
   proceeding the call at the beginning

Examples of breaking:

   - Function arguments

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


   - Chained method calls:

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


I suggest we spawn separate discussion threads (can do as a follow-up)
about:

   - the hard line length limit in Java, possibly to confirm it also for
   Scala (cc @Tison)
   - indentation rules for the broken list of a declared function arguments

If there are no more comments/objections/concerns, I will open a PR to
capture the discussion outcome.

Best,
Andrey



On Wed, Aug 21, 2019 at 8:57 AM Zili Chen  wrote:

> Implement question: how to apply the line length rules?
>
> If we just turn on checkstyle rule "LineLength" then a huge
> effort is required to break lines those break the rule. If
> we use an auto-formatter here then it possibly break line
> "just at the position" awfully.
>
> Is it possible we require only to fit the rule on the fly
> a pull request touch files?
>
> Best,
> tison.
>
>
> Yu Li  于2019年8月20日周二 下午5:22写道:
>
> > I second Stephan's summarize, and to be more explicit, +1 on:
> > - Set a hard line length limit
> > - Allow arguments on the same line if below length limit
> > - With consistent argument breaking when that length is exceeded
> > - Developers can break before that if they feel it helps with readability
> >
> > FWIW, hbase project also sets the line length limit to 100 [1]
> (personally
> > I don't have any tendency on this, so JFYI).
> >
> > [1]
> >
> >
> https://github.com/apache/hbase/blob/a59f7d4ffc27ea23b9822c3c26d6aeb76ccdf9aa/hbase-checkstyle/src/main/resources/hbase/checkstyle.xml#L128
> >
> > Best Regards,
> > Yu
> >
> >
> > On Mon, 19 Aug 2019 at 18:22, Stephan Ewen  wrote:
> >
> > > I personally prefer not to break lines with few parameters.
> > > It just feels unnecessarily clumsy to parse the breaks if there are
> only
> > > two or three arguments with short names.
> > >
> > > So +1
> > >   - for a hard line length limit
> > >   - allowing arguments on the same line if below that limit
> > >   - with consistent argument breaking when that length is exceeded
> > >   - developers can break before that if they feel it helps with
> > > readability.
> > >
> > > This should be similar to what we have, except for enforcing the line
> > > length limit.
> > >
> > > I think our Java guide originally suggested 120 characters line length,
> > but
> > > we can reduce that to 100 if a majority argues for that, but it is
> > separate
> > > discussion.
> > > We uses shorter lines in Scala (100 chars) because Scala code becomes
> > > harder to read faster with long lines.
> > >
> > >
> > > On Mon, Aug 19, 2019 at 10:45 AM Andrey Zagrebin  >
> > > wrote:
> > >
> > > > Hi Everybody,
> > > >
> > > > Thanks for your feedback guys and sorry for not getting back to the
> > > > discussion for some time.
> > > >
> > > > @SHI Xiaogang
> > > > About breaking lines for thrown exceptions:
> > > > Indeed that would prevent growing the throw clause indefinitely.
> > > > I am a bit concerned about putting the right parenthesis and/or throw
> > > > clause on the next line
> > > > because in general we do not it and there are a lot of variations of
> > how
> > > > and what to put to the

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

2019-08-22 Thread Andrey Zagrebin
Hi Tison,

Regarding the automatic checks.
Yes, I suggest we conclude the discussion without the automatic checks.
As soon as we have more ideas/investigation, put into automation, we can
activate it and/or reconsider.
Nonetheless, I do not see any problem if we agree on this atm and make it
part of our code style recommendations.

Regarding putting the right parenthesis on the new line.
At the moment we do not use this approach in our code base. My personal
feeling is that it is not so often used in Java.
Anyways I think, this goes more into direction of the second follow-up to
discuss this separately:

   - indentation rules for the broken list of a declared function arguments
   (atm we usually do: one and newline before body or two idents)

or maybe we should rather name it: how to separate function declaration and
body (as you already mentioned it like this).

We can change this point:

   - The closing brace of the function argument list and the possible
   thrown exception list always stay on the line of the last argument

to

   - The possible thrown exception list is never broken and stays on the
   same last line

Then we can also adjust it if needed after the discussion about how to
separate function declaration and body.

Best,
Andrey






On Thu, Aug 22, 2019 at 9:05 AM Zili Chen  wrote:

> One more question, what do you differ
>
> *public **void func(*
> *int arg1,*
> *int arg2,*
> *...)** throws E1, E2, E3 {*
> *...*
> *}*
>
> and
>
> *public **void func(*
> *int arg1,*
> *int arg2,*
> *...
> *)** throws E1, E2, E3 {*
> *...*
> *}*
>
> I prefer the latter because parentheses are aligned in a similar way,
> as well as the border between declaration and function body is clear.
>
>
> Zili Chen  于2019年8月22日周四 上午9:53写道:
>
> > Thanks Andrey for driving the discussion. Just for clarification,
> > what we conclude here are several guidelines without automatic
> > checker/tool guard them, right?
> >
> > Best,
> > tison.
> >
> >
> > Andrey Zagrebin  于2019年8月21日周三 下午8:18写道:
> >
> >> Hi All,
> >>
> >> I suggest we also conclude this discussion now.
> >>
> >> Breaking the line of too long statements (line longness is yet to be
> fully
> >> defined) to improve code readability in case of
> >>
> >>- Long function argument lists (declaration or call): void func(type1
> >>arg1, type2 arg2, ...)
> >>- Long sequence of chained calls:
> >>list.stream().map(...).reduce(...).collect(...)...
> >>
> >> Rules:
> >>
> >>- Break the list of arguments/calls if the line exceeds limit or
> >> earlier
> >>if you believe that the breaking would improve the code readability
> >>- If you break the line then each argument/call should have a
> separate
> >>line, including the first one
> >>- Each new line argument/call should have one extra indentation
> >> relative
> >>to the line of the parent function name or called entity
> >>- The opening brace always stays on the line of the parent function
> >> name
> >>- The closing brace of the function argument list and the possible
> >>thrown exception list always stay on the line of the last argument
> >>- The dot of a chained call is always on the line of that chained
> call
> >>proceeding the call at the beginning
> >>
> >> Examples of breaking:
> >>
> >>- Function arguments
> >>
> >> *public **void func(*
> >> *int arg1,*
> >> *int arg2,*
> >> *...)** throws E1, E2, E3 {*
> >> *...*
> >> *}*
> >>
> >>
> >>- Chained method calls:
> >>
> >> *values*
> >> *.stream()*
> >> *.map(*...*)*
> >> *.collect(...);*
> >>
> >>
> >> I suggest we spawn separate discussion threads (can do as a follow-up)
> >> about:
> >>
> >>- the hard line length limit in Java, possibly to confirm it also for
> >>Scala (cc @Tison)
> >>- indentation rules for the broken list of a declared function
> >> arguments
> >>
> >> If there are no more comments/objections/concerns, I will open a PR to
> >> capture the discussion outcome.
> >>
> >> Best,
> >> Andrey
> >>
> >>
> >>
> >> On Wed, Aug 21, 2019 at 8:57 AM Zili Chen  wrote:
> >>
> >> > Implement question: how to apply the line length rules?
&

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

2019-08-22 Thread Andrey Zagrebin
FYI PR: https://github.com/apache/flink-web/pull/254

On Thu, Aug 22, 2019 at 10:11 AM Andrey Zagrebin 
wrote:

> Hi Tison,
>
> Regarding the automatic checks.
> Yes, I suggest we conclude the discussion without the automatic checks.
> As soon as we have more ideas/investigation, put into automation, we can
> activate it and/or reconsider.
> Nonetheless, I do not see any problem if we agree on this atm and make it
> part of our code style recommendations.
>
> Regarding putting the right parenthesis on the new line.
> At the moment we do not use this approach in our code base. My personal
> feeling is that it is not so often used in Java.
> Anyways I think, this goes more into direction of the second follow-up to
> discuss this separately:
>
>- indentation rules for the broken list of a declared function
>arguments (atm we usually do: one and newline before body or two idents)
>
> or maybe we should rather name it: how to separate function declaration
> and body (as you already mentioned it like this).
>
> We can change this point:
>
>- The closing brace of the function argument list and the possible
>thrown exception list always stay on the line of the last argument
>
> to
>
>- The possible thrown exception list is never broken and stays on the
>same last line
>
> Then we can also adjust it if needed after the discussion about how to
> separate function declaration and body.
>
> Best,
> Andrey
>
>
>
>
>
>
> On Thu, Aug 22, 2019 at 9:05 AM Zili Chen  wrote:
>
>> One more question, what do you differ
>>
>> *public **void func(*
>> *int arg1,*
>> *int arg2,*
>> *...)** throws E1, E2, E3 {*
>> *...*
>> *}*
>>
>> and
>>
>> *public **void func(*
>> *int arg1,*
>> *int arg2,*
>> *...
>> *)** throws E1, E2, E3 {*
>> *...*
>> *}*
>>
>> I prefer the latter because parentheses are aligned in a similar way,
>> as well as the border between declaration and function body is clear.
>>
>>
>> Zili Chen  于2019年8月22日周四 上午9:53写道:
>>
>> > Thanks Andrey for driving the discussion. Just for clarification,
>> > what we conclude here are several guidelines without automatic
>> > checker/tool guard them, right?
>> >
>> > Best,
>> > tison.
>> >
>> >
>> > Andrey Zagrebin  于2019年8月21日周三 下午8:18写道:
>> >
>> >> Hi All,
>> >>
>> >> I suggest we also conclude this discussion now.
>> >>
>> >> Breaking the line of too long statements (line longness is yet to be
>> fully
>> >> defined) to improve code readability in case of
>> >>
>> >>- Long function argument lists (declaration or call): void
>> func(type1
>> >>arg1, type2 arg2, ...)
>> >>- Long sequence of chained calls:
>> >>list.stream().map(...).reduce(...).collect(...)...
>> >>
>> >> Rules:
>> >>
>> >>- Break the list of arguments/calls if the line exceeds limit or
>> >> earlier
>> >>if you believe that the breaking would improve the code readability
>> >>- If you break the line then each argument/call should have a
>> separate
>> >>line, including the first one
>> >>- Each new line argument/call should have one extra indentation
>> >> relative
>> >>to the line of the parent function name or called entity
>> >>- The opening brace always stays on the line of the parent function
>> >> name
>> >>- The closing brace of the function argument list and the possible
>> >>thrown exception list always stay on the line of the last argument
>> >>- The dot of a chained call is always on the line of that chained
>> call
>> >>proceeding the call at the beginning
>> >>
>> >> Examples of breaking:
>> >>
>> >>- Function arguments
>> >>
>> >> *public **void func(*
>> >> *int arg1,*
>> >> *int arg2,*
>> >> *...)** throws E1, E2, E3 {*
>> >> *...*
>> >> *}*
>> >>
>> >>
>> >>- Chained method calls:
>> >>
>> >> *values*
>> >> *.stream()*
>> >> *.map(*...*)*
>> >> *.collect(...);*
>> >>
>> >>
>> >> I suggest we spawn separate discussion 

Re: [DISCUSS] FLIP-49: Unified Memory Configuration for TaskExecutors

2019-09-02 Thread Andrey Zagrebin
 Hi All,

@Xitong thanks a lot for driving the discussion.

I also reviewed the FLIP and it looks quite good to me.
Here are some comments:


   - One thing I wanted to discuss is the backwards-compatibility with the
   previous user setups. We could list which options we plan to deprecate.
   From the first glance it looks possible to provide the same/similar
   behaviour for the setups relying on the deprecated options. E.g.
   setting taskmanager.memory.preallocate to true could override the
   new taskmanager.memory.managed.offheap-fraction to 1 etc. At the moment the
   FLIP just states that in some cases it may require re-configuring of
   cluster if migrated from prior versions. My suggestion is that we try to
   keep it backwards-compatible unless there is a good reason like some major
   complication for the implementation.


Also couple of smaller things:

   - I suggest we remove TaskExecutorSpecifics from the FLIP and leave some
   general wording atm, like 'data structure to store' or 'utility classes'.
   When the classes are implemented, we put the concrete class names. This way
   we can avoid confusion and stale documents.


   - As I understand, if user task uses native memory (not direct memory,
   but e.g. unsafe.allocate or from external lib), there will be no
   explicit guard against exceeding 'task off heap memory'. Then user should
   still explicitly make sure that her/his direct buffer allocation plus any
   other memory usages does not exceed value announced as 'task off heap'. I
   guess there is no so much that can be done about it except mentioning in
   docs, similar to controlling the heap state backend.


Thanks,
Andrey

On Mon, Sep 2, 2019 at 10:07 AM Yang Wang  wrote:

> I also agree that all the configuration should be calculated out of
> TaskManager.
>
> So a full configuration should be generated before TaskManager started.
>
> Override the calculated configurations through -D now seems better.
>
>
>
> Best,
>
> Yang
>
> Xintong Song  于2019年9月2日周一 上午11:39写道:
>
> > I just updated the FLIP wiki page [1], with the following changes:
> >
> >- Network memory uses JVM direct memory, and is accounted when setting
> >JVM max direct memory size parameter.
> >- Use dynamic configurations (`-Dkey=value`) to pass calculated memory
> >configs into TaskExecutors, instead of ENV variables.
> >- Remove 'supporting memory reservation' from the scope of this FLIP.
> >
> > @till @stephan, please take another look see if there are any other
> > concerns.
> >
> > Thank you~
> >
> > Xintong Song
> >
> >
> > [1]
> >
> >
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-49%3A+Unified+Memory+Configuration+for+TaskExecutors
> >
> > On Mon, Sep 2, 2019 at 11:13 AM Xintong Song 
> > wrote:
> >
> > > Sorry for the late response.
> > >
> > > - Regarding the `TaskExecutorSpecifics` naming, let's discuss the
> detail
> > > in PR.
> > > - Regarding passing parameters into the `TaskExecutor`, +1 for using
> > > dynamic configuration at the moment, given that there are more
> questions
> > to
> > > be discussed to have a general framework for overwriting configurations
> > > with ENV variables.
> > > - Regarding memory reservation, I double checked with Yu and he will
> take
> > > care of it.
> > >
> > > Thank you~
> > >
> > > Xintong Song
> > >
> > >
> > >
> > > On Thu, Aug 29, 2019 at 7:35 PM Till Rohrmann 
> > > wrote:
> > >
> > >> What I forgot to add is that we could tackle specifying the
> > configuration
> > >> fully in an incremental way and that the full specification should be
> > the
> > >> desired end state.
> > >>
> > >> On Thu, Aug 29, 2019 at 1:33 PM Till Rohrmann 
> > >> wrote:
> > >>
> > >> > I think our goal should be that the configuration is fully specified
> > >> when
> > >> > the process is started. By considering the internal calculation step
> > to
> > >> be
> > >> > rather validate existing values and calculate missing ones, these
> two
> > >> > proposal shouldn't even conflict (given determinism).
> > >> >
> > >> > Since we don't want to change an existing flink-conf.yaml,
> specifying
> > >> the
> > >> > full configuration would require to pass in the options differently.
> > >> >
> > >> > One way could be the ENV variables approach. The reason why I'm
> trying
> > >> to
> > >> > exclude this feature from the FLIP is that I believe it needs a bit
> > more
> > >> > discussion. Just some questions which come to my mind: What would be
> > the
> > >> > exact format (FLINK_KEY_NAME)? Would we support a dot separator
> which
> > is
> > >> > supported by some systems (FLINK.KEY.NAME)? If we accept the dot
> > >> > separator what would be the order of precedence if there are two ENV
> > >> > variables defined (FLINK_KEY_NAME and FLINK.KEY.NAME)? What is the
> > >> > precedence of env variable vs. dynamic configuration value specified
> > >> via -D?
> > >> >
> > >> > Another approach could be to pass in the dynamic configuration
> values
> > >> via
> > >> > 

Re: [VOTE] FLIP-49: Unified Memory Configuration for TaskExecutors

2019-09-03 Thread Andrey Zagrebin
Thanks for starting the vote Xintong

Also +1 for the proposed FLIP-49.

@Stephan regarding namings: network vs shuffle.
My understanding so far was that the network memory is what we basically
give to Shuffle implementations and default netty implementation uses it in
particular mostly for networking.
Are the network pools used for something else outside of the shuffling
scope?

best,
Andrey

On Tue, Sep 3, 2019 at 1:01 PM Stephan Ewen  wrote:

> +1 to the proposal in general
>
> A few things seems to be a bit put of sync with the latest discussions
> though.
>
> The section about JVM Parameters states that the
> -XX:MaxDirectMemorySize value is set to Task Off-heap Memory, Shuffle
> Memory and JVM Overhead.
> The way I understand the last discussion conclusion is that it is only the
> sum of shuffle memory and user-defined direct memory.
>
> I am someone neutral but unsure about is the separation between
> "taskmanager.memory.framework.heap" and "taskmanager.memory.task.heap".
> Could that be simply combined under "taskmanager.memory.javaheap"?
>
> It might be good to also expose these values somehow in the web UI so that
> users see immediately what amount of memory TMs assume to use for what.
>
> I assume config key names and default values might be adjusted over time as
> we get feedback.
>   - I would keep the network memory under the name
> "taskmanager.memory.network". Because network memory is actually used for
> more than shuffling. Also, the old config key seems good, so why change it?
>
> One thing to be aware of is that often, the Java Heap is understood as
> "managed memory" as a whole, because it is managed by the GC not explicitly
> by the user.
> So we need to make sure that we don't confuse users by speaking of managed
> heap and unmanaged heap. All heap is managed in Java. Some memory is
> explicitly managed by Flink.
>
> Best,
> Stephan
>
>
> On Mon, Sep 2, 2019 at 3:08 PM Xintong Song  wrote:
>
> > Hi everyone,
> >
> > I'm here to re-start the voting process for FLIP-49 [1], with respect to
> > consensus reached in this thread [2] regarding some new comments and
> > concerns.
> >
> > This voting will be open for at least 72 hours. I'll try to close it Sep.
> > 5, 14:00 UTC, unless there is an objection or not enough votes.
> >
> > Thank you~
> >
> > Xintong Song
> >
> >
> > [1]
> >
> >
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-49%3A+Unified+Memory+Configuration+for+TaskExecutors
> > [2]
> >
> >
> http://apache-flink-mailing-list-archive.1008284.n3.nabble.com/DISCUSS-FLIP-49-Unified-Memory-Configuration-for-TaskExecutors-td31436.html
> >
> > On Tue, Aug 27, 2019 at 9:29 PM Xintong Song 
> > wrote:
> >
> > > Alright, then let's keep the discussion in the DISCUSS mailing thread,
> > and
> > > see whether we need to restart the vote.
> > >
> > > Thank you~
> > >
> > > Xintong Song
> > >
> > >
> > >
> > > On Tue, Aug 27, 2019 at 8:12 PM Till Rohrmann 
> > > wrote:
> > >
> > >> I had a couple of comments concerning the implementation plan. I've
> > posted
> > >> them to the original discussion thread. Depending on the outcome of
> this
> > >> discussion we might need to restart the vote.
> > >>
> > >> Cheers,
> > >> Till
> > >>
> > >> On Tue, Aug 27, 2019 at 11:14 AM Xintong Song 
> > >> wrote:
> > >>
> > >> > Hi all,
> > >> >
> > >> > I would like to start the voting process for FLIP-49 [1], which is
> > >> > discussed and reached consensus in this thread [2].
> > >> >
> > >> > This voting will be open for at least 72 hours. I'll try to close it
> > >> Aug.
> > >> > 30 10:00 UTC, unless there is an objection or not enough votes.
> > >> >
> > >> > Thank you~
> > >> >
> > >> > Xintong Song
> > >> >
> > >> >
> > >> > [1]
> > >> >
> > >> >
> > >>
> >
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-49%3A+Unified+Memory+Configuration+for+TaskExecutors
> > >> > [2]
> > >> >
> > >> >
> > >>
> >
> http://apache-flink-mailing-list-archive.1008284.n3.nabble.com/DISCUSS-FLIP-49-Unified-Memory-Configuration-for-TaskExecutors-td31436.html
> > >> >
> > >>
> > >
> >
>


Re: [VOTE] FLIP-49: Unified Memory Configuration for TaskExecutors

2019-09-04 Thread Andrey Zagrebin
 doesn't work well.
> > > - Agree that it's really important to have good documentation for this.
> > See
> > > above.
> > >
> > > @Zhijiang
> > > - Thanks for the input. My understanding is that 'shuffle memory' is a
> > > portion of the task executor memory reserved for the shuffle component.
> > The
> > > way shuffle component use these memory (local buffer pool, netty
> internal
> > > memory, etc.) can be different depending on the shuffle implementation.
> > The
> > > task executor (outside of the shuffle implementation) should only know
> > the
> > > overall memory usage of the shuffle component but no need to understand
> > > more details inside the shuffle implementation.
> > >
> > > Thank you~
> > >
> > > Xintong Song
> > >
> > >
> > >
> > > On Tue, Sep 3, 2019 at 10:41 PM zhijiang  > > .invalid>
> > > wrote:
> > >
> > > > Thanks for proposing this FLIP and also +1 on my side.
> > > >
> > > > @Andrey Zagrebin For the point of "network memory is actually used
> more
> > > > than shuffling", I guess that the component of queryable state is
> also
> > > > using network/netty stack atm, which is outside of shuffling.
> > > > In addition, if we only consider the shuffle memory provided by
> shuffle
> > > > service interface, we should not only consider the memory used by
> local
> > > > buffer pool, but also consider the netty internal memory
> > > > usages as the overhead, especially we have not the zero-copy
> > improvement
> > > > on dowstream read side. This issue might be out of the vote scope,
> just
> > > > think of we have this issue in [1]. :)
> > > >
> > > > [1] https://issues.apache.org/jira/browse/FLINK-12110
> > > >
> > > > Best,
> > > > Zhijiang
> > > > --
> > > > From:Till Rohrmann 
> > > > Send Time:2019年9月3日(星期二) 15:07
> > > > To:dev 
> > > > Subject:Re: [VOTE] FLIP-49: Unified Memory Configuration for
> > > TaskExecutors
> > > >
> > > > Thanks for creating this FLIP and starting the vote Xintong.
> > > >
> > > > +1 for the proposal from my side.
> > > >
> > > > I agree with Stephan that we might wanna revisit some of the
> > > configuration
> > > > names.
> > > >
> > > > If I understood it correctly, then Task Off-heap memory represents
> the
> > > > direct memory used by the user code, right? How would users configure
> > > > native memory requirements for the user code? If it is part of Task
> Off
> > > > heap memory, then we need to split it to set -XX:MaxDirectMemorySize
> > > > correctly or to introduce another configuration option.
> > > >
> > > > Given all these configuration options, I can see that users will get
> > > > confused quite easily. Therefore, I would like to emphasise that we
> > need
> > > a
> > > > very good documentation about how to properly configure Flink
> processes
> > > and
> > > > which knobs to turn in which cases.
> > > >
> > > > Cheers,
> > > > Till
> > > >
> > > > On Tue, Sep 3, 2019 at 2:34 PM Andrey Zagrebin  >
> > > > wrote:
> > > >
> > > > > Thanks for starting the vote Xintong
> > > > >
> > > > > Also +1 for the proposed FLIP-49.
> > > > >
> > > > > @Stephan regarding namings: network vs shuffle.
> > > > > My understanding so far was that the network memory is what we
> > > basically
> > > > > give to Shuffle implementations and default netty implementation
> uses
> > > it
> > > > in
> > > > > particular mostly for networking.
> > > > > Are the network pools used for something else outside of the
> > shuffling
> > > > > scope?
> > > > >
> > > > > best,
> > > > > Andrey
> > > > >
> > > > > On Tue, Sep 3, 2019 at 1:01 PM Stephan Ewen 
> > wrote:
> > > > >
> > > > > > +1 to the proposal in general
> > > > > >
> > > > > > A few things seems to be a bit put of sync with the latest
> > > d

Re: [DISCUSS] FLIP-53: Fine Grained Resource Management

2019-09-04 Thread Andrey Zagrebin
Thanks for updating the FLIP Xintong. It looks good to me. I would be ok to
start a vote for it.

Best,
Andrey

On Wed, Sep 4, 2019 at 10:03 AM Xintong Song  wrote:

> @all
>
> The FLIP document [1] has been updated.
>
> Thank you~
>
> Xintong Song
>
>
> [1]
>
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-53%3A+Fine+Grained+Operator+Resource+Management
>
> On Tue, Sep 3, 2019 at 7:20 PM Zhu Zhu  wrote:
>
> > Thanks Xintong for the explanation.
> >
> > For question #1, I think it's good as long as DataSet job behaviors
> remains
> > the same.
> >
> > For question #2, agreed that the resource difference is small enough(at
> > most 1 edge diff) in current supported point-wise execution edge
> connection
> > patterns.
> >
> > Thanks,
> > Zhu Zhu
> >
> > Xintong Song  于2019年9月3日周二 下午6:58写道:
> >
> > >  Thanks for the comments, Zhu & Kurt.
> > >
> > > Andrey and I also had some discussions offline, and I would like to
> first
> > > post a summary of our discussion:
> > >
> > >1. The motivation of the fraction based approach is to unify
> resource
> > >management for both operators with specified and unknown resource
> > >requirements.
> > >2. The fraction based approach proposed in this FLIP should only
> > affect
> > >streaming jobs (both bounded and unbounded). For DataSet jobs, there
> > are
> > >already some fraction based approach (in TaskConfig and
> > ChainedDriver),
> > > and
> > >we do not make any change to the existing approach.
> > >3. The scope of this FLIP does not include discussion of how to set
> > >ResourceSpec for operators.
> > >   1. For blink jobs, the optimizer can set operator resources for
> the
> > >   users, according to their configurations (default: unknown)
> > >   2. For DataStream jobs, there are no method / interface to set
> > >   operator resources at the moment (1.10). We can have in the
> future.
> > >   3. For DataSet jobs, there are existing user interfaces to set
> > >   operator resources.
> > >4. The FLIP should explain more about how ResourceSpecs works
> > >   1. PhysicalTransformations (deployed with operators into the
> > >   StreamTasks) get ResourceSpec: unknown by default or known (e.g.
> > > from the
> > >   Blink planner)
> > >   2. While generating stream graph, calculate fractions and set to
> > >   StreamConfig
> > >   3. While scheduling, convert ResourceSpec to ResourceProfile
> > >   (ResourceSpec + network memory), and deploy to slots / TMs
> matching
> > > the
> > >   resources
> > >   4. While starting Task in TM, each operator gets fraction
> converted
> > >   back to the original absolute value requested by user or fair
> > > unknown share
> > >   of the slot
> > >   5. We should not set `allSourcesInSamePipelinedRegion` to `false`
> > for
> > >DataSet jobs. Behaviors of DataSet jobs should not be changed.
> > >6. The FLIP document should differentiate works planed in this FLIP
> > and
> > >the future follow-ups more clearly, by put the follow-ups in a
> > separate
> > >section
> > >7. Another limitation of the rejected alternative setting fractions
> at
> > >scheduling time is that, the scheduler implementation does not know
> > > which
> > >tasks will be deployed into the same slot in advance.
> > >
> > > Andrey, Please bring it up if there is anything I missed.
> > >
> > > Zhu, regarding your comments:
> > >
> > >1. If we do not set `allSourcesInSamePipelinedRegion` to `false` for
> > >DataSet jobs (point 5 in the discussion summary above), then there
> > >shouldn't be any regression right?
> > >2. I think it makes sense to set the max possible network memory for
> > the
> > >JobVertex. When you say parallel instances of the same JobVertex may
> > > have
> > >need different network memory, I guess you mean the rescale
> scenarios
> > > where
> > >parallelisms of upstream / downstream vertex cannot be exactly
> divided
> > > by
> > >parallelism of downstream / upstream vertex? I would say it's
> > > acceptable to
> > >have slight difference between actually needed and allocated network
> > > memory.
> > >3. Yes, by numOpsUseOnHeapManagedMemory I mean
> > >numOpsUseOnHeapManagedMemoryInTheSameSharedGroup. I'll update the
> doc.
> > >4. Yes, it should be StreamingJobGraphGenerator. Thanks for the
> > >correction.
> > >
> > >
> > > Kurt, regarding your comments:
> > >
> > >1. I think we don't have network memory in ResourceSpec, which is
> the
> > >user facing API. We only have network memory in ResourceProfile,
> which
> > > is
> > >used internally for scheduling. The reason we do not expose network
> > > memory
> > >to the user is that, currently how many network buffers each task
> > needs
> > > is
> > >decided by the topology of execution graph (how many input / output
> > >channels it has).
> > >2. In the section "Operato

Re: [VOTE] FLIP-53: Fine Grained Operator Resource Management

2019-09-06 Thread Andrey Zagrebin
Thanks for starting the vote @Xintong

+1 for the FLIP-53

Best,
Andrey

On Fri, Sep 6, 2019 at 3:53 PM Till Rohrmann  wrote:

> Hi Xintong,
>
> thanks for starting this vote. The proposal looks good and, hence, +1 for
> it.
>
> One comment I have is concerning the first implementation step. I would
> suggest to not add the flag allSourcesInSamePipelinedRegion to the
> ExecutionConfig because the ExecutionConfig is public API. Ideally we keep
> this flag internal and don't expose it to the user.
>
> Cheers,
> Till
>
> On Fri, Sep 6, 2019 at 1:47 PM Zhu Zhu  wrote:
>
> > Thanks Xintong for proposing this better resource management.
> > This helps a lot to users who want to better manage the job resources.
> And
> > would be even more useful if in the future we can have auto-tuning
> > mechanism for jobs.
> >
> > +1 (non-binding)
> >
> > Thanks,
> > Zhu Zhu
> >
> > Xintong Song  于2019年9月6日周五 上午11:17写道:
> >
> > > Hi all,
> > >
> > > I would like to start the voting process for FLIP-53 [1], which is
> > > discussed and reached consensus in this thread [2].
> > >
> > > This voting will be open for at least 72 hours (excluding weekends).
> I'll
> > > try to close it Sep. 11, 04:00 UTC, unless there is an objection or not
> > > enough votes.
> > >
> > > Thank you~
> > >
> > > Xintong Song
> > >
> > >
> > > [1]
> > >
> > >
> >
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-53%3A+Fine+Grained+Operator+Resource+Management
> > >
> > > [2]
> > >
> > >
> >
> http://apache-flink-mailing-list-archive.1008284.n3.nabble.com/DISCUSS-FLIP-53-Fine-Grained-Resource-Management-td31831.html
> > >
> >
>


Re: [DISCUSS] FLIP-56: Dynamic Slot Allocation

2019-09-16 Thread Andrey Zagrebin
Hi Xintong,

Thanks for sharing the implementation steps. I also think they makes sense
with the feature option.

I was wondering if we could order the steps in a way that each change does
not affect other components too much, always having a working system
then maybe the feature option does not always need to split the code. Here
are some thoughts.

- We could do default slot profile firstly and include it into the TM
registration. I would suggest to add
to ResourceManagerGateway#registerTaskExecutor, not sendSlotReport.
  This way RM knows about it but does not use at this point. (parts of step
4,6)

- We could try to do step 3 firstly in a way that it also supports the
current way of allocation in TaskExecutorGateway#requestSlot with the
default slot profile
  and sends reports both with available resources and with free default
slots which correspond to the available resources. We can just remove free
default slots later.
  The new way of TaskExecutorGateway#requestResource could be also
implemented here but not used yet.

- Then step 5 can use the new TaskExecutorGateway#requestResource and the
default slot profile

- Not sure, step 5 and 7 can be implemented independently without
regression of what we have. Maybe if we do step 7 firstly it will have only
default slots firstly and it will simplify step 5 later.

Best,
Andrey

On Mon, Sep 16, 2019 at 5:53 AM Xintong Song  wrote:

> Thanks for the comments, Till and Wenlong.
>
> @Wenlong
> Regarding slot sharing, the general idea is to request a slot with
> resources for tasks of the entire slot sharing group. Details can be found
> in FLIP-53 [1], regarding how to decide the slot sharing groups and how to
> manage task resources within the shared slots.
>
> Thank you~
>
> Xintong Song
>
>
>
> On Mon, Sep 16, 2019 at 10:42 AM wenlong.lwl 
> wrote:
>
> > Hi, Xintong, thanks for the great proposal. big +1 for the feature! It is
> > something like mapreduce-1.0 to mapreduce-2.0.
> >
> > I like the design on the whole. One point may need to be included in the
> > proposal:How we deal with slot share group and dynamic slot allocation?
> It
> > can be quite different with dynamic slot allocation.
> >
> > On Fri, 13 Sep 2019 at 16:42, Till Rohrmann 
> wrote:
> >
> > > Thanks for the update Xintong. From a high level perspective the
> > > implementation plan looks good to me.
> > >
> > > Cheers,
> > > Till
> > >
> > > On Thu, Sep 12, 2019 at 11:04 AM Xintong Song 
> > > wrote:
> > >
> > > > Added implementation steps for this FLIP on the wiki page [1].
> > > >
> > > >
> > > > Thank you~
> > > >
> > > > Xintong Song
> > > >
> > > >
> > > > [1]
> > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-56%3A+Dynamic+Slot+Allocation
> > > >
> > > >
> > > >
> > > > On Tue, Aug 20, 2019 at 3:43 PM Xintong Song 
> > > > wrote:
> > > >
> > > > > @Zili
> > > > >
> > > > > As far as I know, Timo is drafting a FLIP that has taken the number
> > 55.
> > > > > There is a round-up number maintained on the FLIP wiki page [1]
> shows
> > > > > which number should be used for the new FLIP, which should be
> > increased
> > > > by
> > > > > whoever takes the number for a new FLIP.
> > > > >
> > > > > Thank you~
> > > > >
> > > > > Xintong Song
> > > > >
> > > > >
> > > > > [1]
> > > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/FLINK/Flink+Improvement+Proposals
> > > > >
> > > > > On Tue, Aug 20, 2019 at 3:28 AM Zili Chen 
> > > wrote:
> > > > >
> > > > >> We suddenly skipped FLIP-55 lol.
> > > > >>
> > > > >>
> > > > >> Xintong Song  于2019年8月19日周一 下午10:23写道:
> > > > >>
> > > > >> > Hi everyone,
> > > > >> >
> > > > >> > We would like to start a discussion thread on "FLIP-56: Dynamic
> > Slot
> > > > >> > Allocation" [1]. This is originally part of the discussion
> thread
> > > for
> > > > >> > "FLIP-53: Fine Grained Resource Management" [2]. As Till
> > suggested,
> > > we
> > > > >> > would like split the original discussion into two topics, and
> > start
> > > a
> > > > >> > separate new discussion thread as well as FLIP process for this
> > one.
> > > > >> >
> > > > >> > Thank you~
> > > > >> >
> > > > >> > Xintong Song
> > > > >> >
> > > > >> >
> > > > >> > [1]
> > > > >> >
> > > > >> >
> > > > >>
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-56%3A+Dynamic+Slot+Allocation
> > > > >> >
> > > > >> > [2]
> > > > >> >
> > > > >> >
> > > > >>
> > > >
> > >
> >
> http://apache-flink-mailing-list-archive.1008284.n3.nabble.com/DISCUSS-FLIP-53-Fine-Grained-Resource-Management-td31831.html
> > > > >> >
> > > > >>
> > > > >
> > > >
> > >
> >
>


Re: [DISCUSS] FLIP-56: Dynamic Slot Allocation

2019-09-16 Thread Andrey Zagrebin
@Xintong

Thanks for the feedback.

Just to clarify step 6:
If the first point is done before step 5 (e.g. as part of 4) then it is
just keeping the info about the default slot in RM's data structure
associated the TM and no real change in the behaviour.
When this info is available, I think it can be straightforwardly used
during step 5 where we get either concrete slot requirement
or the unknown one (step 6, point 2) which simply grabs some of the
concrete default ones (btw not clear which one, seems just some random?)

For steps 5,7, true, it is not quite clear whether we can avoid some split,
e.g. after step 5 before doing step 7.
I agree that we should introduce the feature flag if we clearly see that it
would be a bigger effort without the flag.

Best,
Andrey

On Mon, Sep 16, 2019 at 3:21 PM Till Rohrmann  wrote:

> One thing which was briefly mentioned in the Flip but not in the
> implementation plan is the update of the web UI. I think it is worth
> putting an extra item for updating the web UI to properly display the
> resources a TM has still to offer with dynamic slot allocation. I guess we
> need to pull in some JavaScript help in order to implement this step.
>
> Cheers,
> Till
>
> On Mon, Sep 16, 2019 at 2:15 PM Xintong Song 
> wrote:
>
> > Thanks for the comments, Andrey.
> >
> > - I agree that instead of ResourceManagerGateway#sendSlotReport, we
> should
> > add the default slot resource profile to
> > ResourceManagerGateway#registerTaskExecutor.
> >
> > - If I understand correctly, the reason you suggest do default slot
> > resource profile first and then do step 3 in a way that support both
> > TaskExecutorGateway#requestSlot and TaskExecutorGateway#requestResource,
> is
> > to try to avoid splitting code paths with the feature option? I think we
> > can do that, but I also want to bring it up that this can only reduce the
> > code split by the feature option (which is good) but not eliminate it. We
> > still need the feature option for the fundamental differences, e.g.
> > creating new SlotIDs on allocation vs. allocate to free slots with
> existing
> > SlotIDs.
> >
> > - I don't really think we can do step 5, 6 and 7 independently. Basically
> > they are all making changes to the same component. We probably can do
> step
> > 6 and 7 independently, but I think they both depends on step 5.
> >
> > In general, I would say it's good to have as less as possible codes split
> > by the feature option, which makes the later clean-up easier. But if it
> > cannot be easily done, I would rather not to put too much efforts on
> having
> > a good abstraction and deduplication between the new code path and the
> > original one that we are removing soon.
> >
> > What do you think?
> >
> > Thank you~
> >
> > Xintong Song
> >
> >
> >
> > On Mon, Sep 16, 2019 at 5:59 PM Andrey Zagrebin 
> > wrote:
> >
> > > Hi Xintong,
> > >
> > > Thanks for sharing the implementation steps. I also think they makes
> > sense
> > > with the feature option.
> > >
> > > I was wondering if we could order the steps in a way that each change
> > does
> > > not affect other components too much, always having a working system
> > > then maybe the feature option does not always need to split the code.
> > Here
> > > are some thoughts.
> > >
> > > - We could do default slot profile firstly and include it into the TM
> > > registration. I would suggest to add
> > > to ResourceManagerGateway#registerTaskExecutor, not sendSlotReport.
> > >   This way RM knows about it but does not use at this point. (parts of
> > step
> > > 4,6)
> > >
> > > - We could try to do step 3 firstly in a way that it also supports the
> > > current way of allocation in TaskExecutorGateway#requestSlot with the
> > > default slot profile
> > >   and sends reports both with available resources and with free default
> > > slots which correspond to the available resources. We can just remove
> > free
> > > default slots later.
> > >   The new way of TaskExecutorGateway#requestResource could be also
> > > implemented here but not used yet.
> > >
> > > - Then step 5 can use the new TaskExecutorGateway#requestResource and
> the
> > > default slot profile
> > >
> > > - Not sure, step 5 and 7 can be implemented independently without
> > > regression of what we have. Maybe if we do step 7 firstly it will have
> > only
> > > default slots firstly and it will simplify step

Re: [DISCUSS] FLIP-56: Dynamic Slot Allocation

2019-09-17 Thread Andrey Zagrebin
Thanks for the update @Xintong.
I would be ok with starting the vote.

Best,
Andrey

On Tue, Sep 17, 2019 at 6:12 AM Xintong Song  wrote:

> The implementation plan [1] is updated, with the following changes:
>
>- Add default slot resource profile to
>ResourceManagerGateway#registerTaskExecutor rather than #sendSlotReport.
>- Swap 'TaskExecutor derive and register with default slot resource
>profile' and 'Extend TaskExecutor to support dynamic slot allocation'
>- Add step for updating RestAPI / Web UI
>
> Thank you~
>
> Xintong Song
>
>
> [1]
>
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-56%3A+Dynamic+Slot+Allocation
>
> On Tue, Sep 17, 2019 at 11:49 AM Xintong Song 
> wrote:
>
> > @Till
> > Thanks for the reminding. I'll add a step for updating the web ui. I'll
> > try to involve Lining to help us with this step.
> >
> > @Andrey
> > I was thinking that after we define the RM-TM interfaces in step 2, it
> > would be good to concurrently work on both RM and TM side. But yes, if we
> > finish Step 4 early, then it would make step 6 easier. We can start to
> have
> > some IT/E2E tests, with the default slot resource profiles being
> available.
> >
> > Thank you~
> >
> > Xintong Song
> >
> >
> >
> > On Mon, Sep 16, 2019 at 9:50 PM Andrey Zagrebin 
> > wrote:
> >
> >> @Xintong
> >>
> >> Thanks for the feedback.
> >>
> >> Just to clarify step 6:
> >> If the first point is done before step 5 (e.g. as part of 4) then it is
> >> just keeping the info about the default slot in RM's data structure
> >> associated the TM and no real change in the behaviour.
> >> When this info is available, I think it can be straightforwardly used
> >> during step 5 where we get either concrete slot requirement
> >> or the unknown one (step 6, point 2) which simply grabs some of the
> >> concrete default ones (btw not clear which one, seems just some random?)
> >>
> >> For steps 5,7, true, it is not quite clear whether we can avoid some
> >> split,
> >> e.g. after step 5 before doing step 7.
> >> I agree that we should introduce the feature flag if we clearly see that
> >> it
> >> would be a bigger effort without the flag.
> >>
> >> Best,
> >> Andrey
> >>
> >> On Mon, Sep 16, 2019 at 3:21 PM Till Rohrmann 
> >> wrote:
> >>
> >> > One thing which was briefly mentioned in the Flip but not in the
> >> > implementation plan is the update of the web UI. I think it is worth
> >> > putting an extra item for updating the web UI to properly display the
> >> > resources a TM has still to offer with dynamic slot allocation. I
> guess
> >> we
> >> > need to pull in some JavaScript help in order to implement this step.
> >> >
> >> > Cheers,
> >> > Till
> >> >
> >> > On Mon, Sep 16, 2019 at 2:15 PM Xintong Song 
> >> > wrote:
> >> >
> >> > > Thanks for the comments, Andrey.
> >> > >
> >> > > - I agree that instead of ResourceManagerGateway#sendSlotReport, we
> >> > should
> >> > > add the default slot resource profile to
> >> > > ResourceManagerGateway#registerTaskExecutor.
> >> > >
> >> > > - If I understand correctly, the reason you suggest do default slot
> >> > > resource profile first and then do step 3 in a way that support both
> >> > > TaskExecutorGateway#requestSlot and
> >> TaskExecutorGateway#requestResource,
> >> > is
> >> > > to try to avoid splitting code paths with the feature option? I
> think
> >> we
> >> > > can do that, but I also want to bring it up that this can only
> reduce
> >> the
> >> > > code split by the feature option (which is good) but not eliminate
> >> it. We
> >> > > still need the feature option for the fundamental differences, e.g.
> >> > > creating new SlotIDs on allocation vs. allocate to free slots with
> >> > existing
> >> > > SlotIDs.
> >> > >
> >> > > - I don't really think we can do step 5, 6 and 7 independently.
> >> Basically
> >> > > they are all making changes to the same component. We probably can
> do
> >> > step
> >> > > 6 and 7 independently, but I think they both depends on step 

Re: [VOTE] FLIP-56: Dynamic Slot Allocation

2019-09-19 Thread Andrey Zagrebin
Hi Xintong,

Thanks for starting the vote, +1 from my side.

Best,
Andrey

On Tue, Sep 17, 2019 at 4:26 PM Xintong Song  wrote:

> Hi all,
>
> I would like to start the vote for FLIP-56 [1], on which a consensus is
> reached in this discussion thread [2].
>
> The vote will be open for at least 72 hours. I'll try to close it after
> Sep. 20 15:00 UTC, unless there is an objection or not enough votes.
>
> Thank you~
>
> Xintong Song
>
>
> [1]
>
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-56%3A+Dynamic+Slot+Allocation
>
> [2]
>
> http://apache-flink-mailing-list-archive.1008284.n3.nabble.com/DISCUSS-FLIP-56-Dynamic-Slot-Allocation-td31960.html
>


Re: [VOTE] Accept Stateful Functions into Apache Flink

2019-10-29 Thread Andrey Zagrebin
+1 (binding)

On Tue, Oct 29, 2019 at 10:09 AM Igal Shilman  wrote:

> +1 (non-binding)
>
> Thanks,
> Igal Shilman
>
> On Sat, Oct 26, 2019 at 12:25 AM Ufuk Celebi  wrote:
>
> > +1 (binding)
> >
> > On Fri, Oct 25, 2019 at 6:39 PM Maximilian Michels 
> wrote:
> >
> > > +1 (binding)
> > >
> > > On 25.10.19 14:31, Congxian Qiu wrote:
> > > > +1 (non-biding)
> > > > Best,
> > > > Congxian
> > > >
> > > >
> > > > Terry Wang  于2019年10月24日周四 上午11:15写道:
> > > >
> > > >> +1 (non-biding)
> > > >>
> > > >> Best,
> > > >> Terry Wang
> > > >>
> > > >>
> > > >>
> > > >>> 2019年10月24日 10:31,Jingsong Li  写道:
> > > >>>
> > > >>> +1 (non-binding)
> > > >>>
> > > >>> Best,
> > > >>> Jingsong Lee
> > > >>>
> > > >>> On Wed, Oct 23, 2019 at 9:02 PM Yu Li  wrote:
> > > >>>
> > >  +1 (non-binding)
> > > 
> > >  Best Regards,
> > >  Yu
> > > 
> > > 
> > >  On Wed, 23 Oct 2019 at 16:56, Haibo Sun 
> wrote:
> > > 
> > > > +1 (non-binding)Best,
> > > > Haibo
> > > >
> > > >
> > > > At 2019-10-23 09:07:41, "Becket Qin" 
> wrote:
> > > >> +1 (binding)
> > > >>
> > > >> Thanks,
> > > >>
> > > >> Jiangjie (Becket) Qin
> > > >>
> > > >> On Tue, Oct 22, 2019 at 11:44 PM Tzu-Li (Gordon) Tai <
> > >  tzuli...@apache.org
> > > >>
> > > >> wrote:
> > > >>
> > > >>> +1 (binding)
> > > >>>
> > > >>> Gordon
> > > >>>
> > > >>> On Tue, Oct 22, 2019, 10:58 PM Zhijiang <
> > > wangzhijiang...@aliyun.com
> > > >>> .invalid>
> > > >>> wrote:
> > > >>>
> > >  +1 (non-binding)
> > > 
> > >  Best,
> > >  Zhijiang
> > > 
> > > 
> > > 
> > --
> > >  From:Zhu Zhu 
> > >  Send Time:2019 Oct. 22 (Tue.) 16:33
> > >  To:dev 
> > >  Subject:Re: [VOTE] Accept Stateful Functions into Apache Flink
> > > 
> > >  +1 (non-binding)
> > > 
> > >  Thanks,
> > >  Zhu Zhu
> > > 
> > >  Biao Liu  于2019年10月22日周二 上午11:06写道:
> > > 
> > > > +1 (non-binding)
> > > >
> > > > Thanks,
> > > > Biao /'bɪ.aʊ/
> > > >
> > > >
> > > >
> > > > On Tue, 22 Oct 2019 at 10:26, Jark Wu 
> > wrote:
> > > >
> > > >> +1 (non-binding)
> > > >>
> > > >> Best,
> > > >> Jark
> > > >>
> > > >> On Tue, 22 Oct 2019 at 09:38, Hequn Cheng <
> > chenghe...@gmail.com
> > > >
> > >  wrote:
> > > >>
> > > >>> +1 (non-binding)
> > > >>>
> > > >>> Best, Hequn
> > > >>>
> > > >>> On Tue, Oct 22, 2019 at 9:21 AM Dian Fu <
> > >  dian0511...@gmail.com>
> > >  wrote:
> > > >>>
> > >  +1 (non-binding)
> > > 
> > >  Regards,
> > >  Dian
> > > 
> > > > 在 2019年10月22日,上午9:10,Kurt Young  写道:
> > > >
> > > > +1 (binding)
> > > >
> > > > Best,
> > > > Kurt
> > > >
> > > >
> > > > On Tue, Oct 22, 2019 at 12:56 AM Fabian Hueske <
> > >  fhue...@gmail.com>
> > >  wrote:
> > > >
> > > >> +1 (binding)
> > > >>
> > > >> Am Mo., 21. Okt. 2019 um 16:18 Uhr schrieb Thomas Weise
> <
> > > >>> t...@apache.org
> > > > :
> > > >>
> > > >>> +1 (binding)
> > > >>>
> > > >>>
> > > >>> On Mon, Oct 21, 2019 at 7:10 AM Timo Walther <
> > >  twal...@apache.org
> > > >>
> > >  wrote:
> > > >>>
> > >  +1 (binding)
> > > 
> > >  Thanks,
> > >  Timo
> > > 
> > > 
> > >  On 21.10.19 15:59, Till Rohrmann wrote:
> > > > +1 (binding)
> > > >
> > > > Cheers,
> > > > Till
> > > >
> > > > On Mon, Oct 21, 2019 at 12:13 PM Robert Metzger <
> > > >>> rmetz...@apache.org
> > > >>>
> > >  wrote:
> > > >
> > > >> +1 (binding)
> > > >>
> > > >> On Mon, Oct 21, 2019 at 12:06 PM Stephan Ewen <
> > > > se...@apache.org
> > > >>>
> > > >>> wrote:
> > > >>
> > > >>> This is the official vote whether to accept the
> > > > Stateful
> > > >>> Functions
> > > >>> code
> > > >>> contribution to Apache Flink.
> > > >>>
> > > >>> The current Stateful Functions code, documentation,
> > > > and
> > > > website
> > > >>> can
> > > 

Re: [ANNOUNCE] Becket Qin joins the Flink PMC

2019-10-30 Thread Andrey Zagrebin
Congrats Becket!

On Tue, Oct 29, 2019 at 6:11 PM Xuannan Su  wrote:

> Congratulations Becket!  :-)
>
> Xuannan
>
> > On Oct 28, 2019, at 6:07 AM, Fabian Hueske  wrote:
> >
> > Hi everyone,
> >
> > I'm happy to announce that Becket Qin has joined the Flink PMC.
> > Let's congratulate and welcome Becket as a new member of the Flink PMC!
> >
> > Cheers,
> > Fabian
>
>


Re: [VOTE] Accept Stateful Functions into Apache Flink

2019-10-30 Thread Andrey Zagrebin
sorry, my +1 was non-binding, confused that it was not a committer vote but
PMC.

On Wed, Oct 30, 2019 at 4:43 PM Chesnay Schepler  wrote:

> +1 (binding)
>
> On 30/10/2019 15:25, Vasiliki Kalavri wrote:
> > +1 (binding) from me. I hope this is not too late :)
> >
> > Thank you for this great contribution!
> >
> > On Wed, 30 Oct 2019 at 14:45, Stephan Ewen  wrote:
> >
> >> Thank you all for voting.
> >>
> >> The voting period has passed, but only 13 PMC members have voted so far,
> >> that is less than 2/3rd of the PMCs (17 members).
> >>
> >> I will take a few days to ping other members to vote, after that we will
> >> gradually lower the threshold as per the process to account for inactive
> >> members.
> >>
> >> Best,
> >> Stephan
> >>
> >>
> >>
> >>
> >> On Tue, Oct 29, 2019 at 6:20 PM Seth Wiesman 
> wrote:
> >>
> >>> +1 (non-binding)
> >>>
> >>> Seth
> >>>
>  On Oct 23, 2019, at 9:31 PM, Jingsong Li 
> >> wrote:
>  +1 (non-binding)
> 
>  Best,
>  Jingsong Lee
> 
> > On Wed, Oct 23, 2019 at 9:02 PM Yu Li  wrote:
> >
> > +1 (non-binding)
> >
> > Best Regards,
> > Yu
> >
> >
> >> On Wed, 23 Oct 2019 at 16:56, Haibo Sun  wrote:
> >>
> >> +1 (non-binding)Best,
> >> Haibo
> >>
> >>
> >> At 2019-10-23 09:07:41, "Becket Qin"  wrote:
> >>> +1 (binding)
> >>>
> >>> Thanks,
> >>>
> >>> Jiangjie (Becket) Qin
> >>>
> >>> On Tue, Oct 22, 2019 at 11:44 PM Tzu-Li (Gordon) Tai <
> > tzuli...@apache.org
> >>> wrote:
> >>>
>  +1 (binding)
> 
>  Gordon
> 
>  On Tue, Oct 22, 2019, 10:58 PM Zhijiang <
> >> wangzhijiang...@aliyun.com
>  .invalid>
>  wrote:
> 
> > +1 (non-binding)
> >
> > Best,
> > Zhijiang
> >
> >
> >
> --
> > From:Zhu Zhu 
> > Send Time:2019 Oct. 22 (Tue.) 16:33
> > To:dev 
> > Subject:Re: [VOTE] Accept Stateful Functions into Apache Flink
> >
> > +1 (non-binding)
> >
> > Thanks,
> > Zhu Zhu
> >
> > Biao Liu  于2019年10月22日周二 上午11:06写道:
> >
> >> +1 (non-binding)
> >>
> >> Thanks,
> >> Biao /'bɪ.aʊ/
> >>
> >>
> >>
> >>> On Tue, 22 Oct 2019 at 10:26, Jark Wu 
> wrote:
> >>>
> >>> +1 (non-binding)
> >>>
> >>> Best,
> >>> Jark
> >>>
> >>> On Tue, 22 Oct 2019 at 09:38, Hequn Cheng <
> chenghe...@gmail.com
> > wrote:
>  +1 (non-binding)
> 
>  Best, Hequn
> 
>  On Tue, Oct 22, 2019 at 9:21 AM Dian Fu <
> > dian0511...@gmail.com>
> > wrote:
> > +1 (non-binding)
> >
> > Regards,
> > Dian
> >
> >> 在 2019年10月22日,上午9:10,Kurt Young  写道:
> >>
> >> +1 (binding)
> >>
> >> Best,
> >> Kurt
> >>
> >>
> >> On Tue, Oct 22, 2019 at 12:56 AM Fabian Hueske <
> > fhue...@gmail.com>
> > wrote:
> >>> +1 (binding)
> >>>
> >>> Am Mo., 21. Okt. 2019 um 16:18 Uhr schrieb Thomas Weise <
>  t...@apache.org
> >> :
>  +1 (binding)
> 
> 
>  On Mon, Oct 21, 2019 at 7:10 AM Timo Walther <
> > twal...@apache.org
> > wrote:
> > +1 (binding)
> >
> > Thanks,
> > Timo
> >
> >
> >> On 21.10.19 15:59, Till Rohrmann wrote:
> >> +1 (binding)
> >>
> >> Cheers,
> >> Till
> >>
> >> On Mon, Oct 21, 2019 at 12:13 PM Robert Metzger <
>  rmetz...@apache.org
> > wrote:
> >>> +1 (binding)
> >>>
> >>> On Mon, Oct 21, 2019 at 12:06 PM Stephan Ewen <
> >> se...@apache.org
>  wrote:
>  This is the official vote whether to accept the
> >> Stateful
>  Functions
>  code
>  contribution to Apache Flink.
> 
>  The current Stateful Functions code, documentation,
> >> and
> >> website
>  can
>  be
>  found here:
>  https://statefun.io/
>  https://github.com/ververica/stateful-functions
> 
>  This vote should capture whether the Apache Flink
>  community
> >> is
> > intereste

Re: set job level TTL

2019-11-26 Thread Andrey Zagrebin
Hi Chen,

if I understand correctly, your question is not about the state with TTL,
correct me if I am wrong.

We could consider adding this feature to Flink: run x time => save point
=> shutdown job but it seems to me that it is something quite application
specific, not sure how high priority it will get in community.

Would it work for you to schedule an automated cron script to check your
running job and do those actions over CLI or REST API?

Cheers,
Andrey

On Mon, Nov 25, 2019 at 7:27 PM Yun Tang  wrote:

> Hi Chen
>
> There existed several problems currently:
> 1. State TTL cannot be applied to Flink SQL
> 2. No  job-level state TTL
> 3. Window state should not be limited to TTL if existed a job-level state
>
> If your goal is to achieve #2, we could introduce a configuration via
> flink-conf.yaml so that we could also achieve #1 but leave window state not
> limited to TTL.
> I think this deserves an issue to track this request.
>
> Best
> Yun Tang
>
> On 11/25/19, 8:04 AM, "Chen Qin"  wrote:
>
> I am a bit confused. Job level TTL is different from state TTL despite
> share same name.
>
> What we want to achieve in Job TTL is basically "run x time => save
> point
> => shutdown job"
>
>
> On Sun, Nov 24, 2019 at 9:57 AM Yun Tang  wrote:
>
> > Hi Chen
> >
> > Currently, state TTL setting is per-state scoped, and there is no
> > job-level TTL setting so far. Do you want similar settings in
> flink-conf to
> > enable Flink SQL could also benefit from state TTL?
> >
> > Best
> > Yun Tang
> >
> > On 11/25/19, 12:16 AM, "Chen Qin"  wrote:
> >
> > Hi there,
> >
> > We have use case of allowing ad hoc user submit a Flink job
> (mostly
> > SQL)
> > and user come back and verify results after period of time (eg
> 24hours)
> > without manual intervention. The idea is to have something like
> set job
> > level TTL configuration to 24hours. Jobmanager will honor
> 24hours of
> > instrument save point and proper shutdown of flink job.
> >
> > Just want to know if we have job level TTL setting or parity
> features
> > JIRA.
> >
> >
> > Best,
> > Chen
> >
> >
> >
>
>
>


[Proposal] Shuffle resources lifecycle management

2019-03-29 Thread Andrey Zagrebin
Hi All,

While working on pluggable shuffle architecture, looking into interactive
programming and fine-grained recovery efforts, we released that lifecycle
management of intermediate result partitions needs more detailed
discussion to enable envisioned use cases.

Here I prepared a design document to address this concern. The document
proposes certain extensions to FLIP-31 (Design of Pluggable Shuffle
Service):

https://docs.google.com/document/d/13vAJJxfRXAwI4MtO8dux8hHnNMw2Biu5XRrb_hvGehA

Looking forward to your feedback.

Thanks,
Andrey


[Discuss] Semantics of event time for state TTL

2019-04-04 Thread Andrey Zagrebin
Hi All,

As you might have already seen there is an effort tracked in FLINK-12005
[1] to support event time scale for state with time-to-live (TTL) [2].
While thinking about design, we realised that there can be multiple options
for semantics of this feature, depending on use case. There is also
sometimes confusion because of event time out-of-order nature in Flink. I
am starting this thread to discuss potential use cases of this feature and
their requirements for interested users and developers. There was already
discussion thread asking about event time for TTL and it already contains
some thoughts [3].

There are two semantical cases where we use time for TTL feature at the
moment. Firstly, we store timestamp of state last access/update. Secondly,
we use this timestamp and current timestamp to check expiration and garbage
collect state at some point later.

At the moment, Flink supports *only processing time* for both timestamps:
state *last access and current timestamp*. It is basically current local
system unix epoch time.

When it comes to event time scale, we also need to define what Flink should
use for these two timestamps. Here I will list some options and their
possible pros&cons for discussion. There might be more depending on use
case.

*Last access timestamp (stored in backend with the actual state value):*

   - *Event timestamp of currently being processed record.* This seems to
   be the simplest option and it allows user-defined timestamps in state
   backend. The problem here might be instability of event time which can not
   only increase but also decrease if records come out of order. This can lead
   to rewriting the state timestamp to smaller value which is unnatural for
   the notion of time.
   - *Max event timestamp of records seen so far for this record key.* This
   option is similar to the previous one but it tries to fix the notion of
   time to make it always increasing. Maintaining this timestamp has also
   performance implications because the previous timestamp needs to be read
   out to decide whether to rewrite it.
   - *Last emitted watermark*. This is what we usually use for other
   operations to trigger some actions in Flink, like timers and windows but it
   can be unrelated to the record which actually triggers the state update.

*Current timestamp to check expiration:*

   - *Event timestamp of last processed record.* Again quite simple but
   unpredictable option for out-of-order events. It can potentially lead to
   undesirable expiration of late buffered data in state without control.
   - *Max event timestamp of records seen so far for operator backend.* Again
   similar to previous one, more stable but still user does not have too much
   control when to expire state.
   - *Last emitted watermark*. Again, this is what we usually use for other
   operations to trigger some actions in Flink, like timers and windows. It
   also gives user some control to decide when state is expired (up to which
   point in event time) by emitting certain watermark. It is more flexible but
   complicated. If some watermark emitting strategy is already used for other
   operations, it might be not optimal for TTL and delay state cleanup.
   - *Current processing time.* This option is quite simple, It would mean
   that user just decides which timestamp to store but it will expire in real
   time. For data privacy use case, it might be better because we want state
   to be unavailable in particular real moment of time since the associated
   piece of data was created in event time. For long term approximate garbage
   collection, it might be not a problem as well. For quick expiration, the
   time skew between event and processing time can lead again to premature
   deletion of late data and user cannot delay it.

We could also make this behaviour configurable. Another option is to make
time provider pluggable for users. The interface can give users context
(currently processed record, watermark etc) and ask them which timestamp to
use. This is more complicated though.

Looking forward for your feedback.

Best,
Andrey

[1] https://issues.apache.org/jira/browse/FLINK-12005
[2]
https://docs.google.com/document/d/1SI_WoXAfOd4_NKpGyk4yh3mf59g12pSGNXRtNFi-tgM
[3]
http://apache-flink-user-mailing-list-archive.2336050.n4.nabble.com/State-TTL-in-Flink-1-6-0-td22509.html


Re: [DISCUSS] A more restrictive JIRA workflow

2019-04-10 Thread Andrey Zagrebin
Hi all,

+1 for option 2.

I also do not mind option 2, after 1-2 contributions, any contributor could
just ask the committer (who merged those contributions) about contributor
permissions.

Best,
Andrey

On Wed, Apr 10, 2019 at 3:58 AM Robert Metzger  wrote:

> I'm +1 on option 1.
>
> On Tue, Apr 9, 2019 at 1:58 AM Timo Walther  wrote:
>
> > Hi everyone,
> >
> > I'd like to bring up this discussion thread again. In summary, I think
> > we all agreed on improving the JIRA workflow to move design/consensus
> > discussions from PRs to the issues first, before implementing them.
> >
> > Two options have been proposed:
> > 1. Only committers can assign people to issues. PRs of unassigned issues
> > are closed automatically.
> > 2. Committers upgrade assignable users to contributors as an
> > intermediate step towards committership.
> >
> > I would prefer option 1 as some people also mentioned that option 2
> > requires some standadized processes otherwise it would be difficult to
> > communicate why somebody is a contributor and some somebody is not.
> >
> > What do you think?
> >
> > Regards,
> > Timo
> >
> >
> > Am 18.03.19 um 14:25 schrieb Robert Metzger:
> > > @Fabian: I don't think this is a big problem. Moving away from "giving
> > > everybody contributor permissions" to "giving it to some people" is not
> > > risky.
> > > I would leave this decision to the committers who are working with a
> > person.
> > >
> > >
> > > We should bring this discussion to a conclusion and implement the
> changes
> > > to JIRA.
> > >
> > >
> > > Nobody has raised any objections to the overall idea.
> > >
> > > Points raised:
> > > 1. We need to update the contribution guide and describe the workflow.
> > > 2. I brought up changing Flinkbot so that it auto-closes PRs without
> > > somebody assigned in JIRA.
> > >
> > > Who wants to work on an update of the contribution guide?
> > > If there's no volunteers, I'm happy to take care of this.
> > >
> > >
> > > On Fri, Mar 15, 2019 at 9:20 AM Fabian Hueske 
> wrote:
> > >
> > >> Hi,
> > >>
> > >> I'm not sure about adding an additional stage.
> > >> Who's going to decide when to "promote" a user to a contributor, i.e.,
> > >> grant assigning permission?
> > >>
> > >> Best, Fabian
> > >>
> > >> Am Do., 14. März 2019 um 13:50 Uhr schrieb Timo Walther <
> > >> twal...@apache.org
> > >>> :
> > >>> Hi Robert,
> > >>>
> > >>> I also like the idea of making every Jira user an "Assignable User",
> > but
> > >>> restrict assigning a ticket to people with committer permissions.
> > >>>
> > >>> Instead of giving contributor permissions to everyone, we could have
> a
> > >>> more staged approach from user, to contributor, and finally to
> > committer.
> > >>>
> > >>> Once people worked on a couple of JIRA issues, we can make them
> > >>> contributors.
> > >>>
> > >>> What do you think?
> > >>>
> > >>> Regards,
> > >>> Timo
> > >>>
> > >>> Am 06.03.19 um 12:33 schrieb Robert Metzger:
> >  Hi Tison,
> >  I also thought about this.
> >  Making a person a "Contributor" is required for being an "Assignable
> > >>> User",
> >  so normal Jira accounts can't be assigned to a ticket.
> > 
> >  We could make every Jira user an "Assignable User", but restrict
> > >>> assigning
> >  a ticket to people with committer permissions.
> >  There are some other permissions attached to the "Contributor" role,
> > >> such
> >  as "Closing" and "Editing" (including "Transition", "Logging work",
> > >>> etc.).
> >  I think we should keep the "Contributor" role, but we could be (as
> you
> >  propose) make it more restrictive. Maybe "invite only" for people
> who
> > >> are
> >  apparently active in our Jira.
> > 
> >  Best,
> >  Robert
> > 
> > 
> > 
> >  On Wed, Mar 6, 2019 at 11:02 AM ZiLi Chen 
> > >> wrote:
> > > Hi devs,
> > >
> > > Just now I find that one not a contributor can file issue and
> > >>> participant
> > > discussion.
> > > One becomes contributor can additionally assign an issue to a
> person
> > >> and
> > > modify fields of any issues.
> > >
> > > For a more restrictive JIRA workflow, maybe we achieve it by making
> > >> it a
> > > bit more
> > > restrictive granting contributor permission?
> > >
> > > Best,
> > > tison.
> > >
> > >
> > > Robert Metzger  于2019年2月27日周三 下午9:53写道:
> > >
> > >> I like this idea and I would like to try it to see if it solves
> the
> > >> problem.
> > >>
> > >> I can also offer to add a functionality to the Flinkbot to
> > >>> automatically
> > >> close pull requests which have been opened against a unassigned
> JIRA
> > >> ticket.
> > >> Being rejected by an automated system, which just applies a rule
> is
> > >>> nicer
> > >> than being rejected by a person.
> > >>
> > >>
> > >> On Wed, Feb 27, 2019 at 1:45 PM Stephan Ewen 
> > >> wrote:
> > >>> @Chesnay - yes, this is possi

Re: [Discuss][FLINK-8297]A solution for FLINK-8297 Timebased RocksDBListState

2019-04-15 Thread Andrey Zagrebin
Hi Faxian,

Thanks for thinking on this new approach. Here are my thoughts:

- In case of event time, although, this approach changes semantics of
original list state, it could be a good fit for certain use cases. The main
advantage is that it is deterministic in event time. The list should end up
always in the same order.

- In case of processing time, the time skew might be a problem. If task
executor's clock jumps back for some reason or it fails and another TE with
shifted clock takes over, this can potentially reorder list elements. If we
rather think about the list state as a bag, reordering might be ok but
there is also a risk that different elements might end up having the same
processing time and rewrite each other.

- In general, exploding a storage size is a trade-off to achieve more
scalability for list state and should be ok if we do not degrade existing
approach.

Let's see other opinions.

Best,
Andrey

On Fri, Apr 12, 2019 at 10:34 AM Faxian Zhao  wrote:

> Refer from PR#5185, I think we can use Timebased RocksDBListState to
> resolve it.
> Timebased RocksDBListState store list entries dispersed in rocksdb like
> RocksDBMapState.
> Key pair:
> For the timebased flink inner class like StreamRecord(enable
> event/ingestion time), the rocksdb key is
> #KeyGroup#Key#Namespace#StreamRecord.getTimestamp().
> Otherwise, the key is current procssing time.
> Value pair:
> The rocksdb value is the entries which have the same
> timestamp(event/ingestion/processing time), like the original
> RocksDBListState.
>
> The ListState.get() implement like
> org.apache.flink.contrib.streaming.state.RocksDBMapState#iterator.
> Generally, it won't load all entries one time.
>
> The rocksdb store structure.
> ---Key--- Value-
> #KeyGroup#Key#Namespace #KeyGroup#Key#Namespace#ts3 (max lexicographically
> key)
> #KeyGroup#Key#Namespace#ts1value1,value2,value7
> #KeyGroup#Key#Namespace#ts2value4,value6
> #KeyGroup#Key#Namespace#ts3value3,value5
>
>
> Advantage:
> 1. Due to the rocksdb store key with lexicographically order, so the
> entries is monotonous by time. It's friendly to event time records
> processing.
> 2. We can store the max timestamp key in the rocksdb default
> key(#KeyGroup#Key#Namespace), then we can reverse iterate the stored list.
> 3. For the CountEvictor and TimeEvictor, we can stop the iteration early
> instead of read all of them into memory.
> 4. This ListState is monotonous by time, we can provide some more methods
> for event time records processing.
> 5. I think it resolve the ttl issue naturally.
>
> Disadvantage:
> 1. It will add 8 bytes cost to store extended timestamp in key part, and
> I'm not good at rocksdb, I don't know the performance affect.
> 2. For the event time StreamRecord, it will reorder the entries by event
> time. This behavior is not align with other ListState implement.
> 3. For other records, the key is useless useless overhead.
> 4. If all of the entries have the same timestamp, the store structure is
> almost same as the original RocksDBListState.
> 5. We can't easily implement remove, size method for ListState yet.
>
> Implement:
> We can abstract a new class which is the parent of Time based
> RocksDBListState and RocksDBMapState, but we should modify
> InternalLargeListState.
> I draft some code for this in PR#7675
>


Re: [DISCUSS] A more restrictive JIRA workflow

2019-04-15 Thread Andrey Zagrebin
@Robert thanks for pointing out and sorry for confusion. The correct text:

+1 for option 1.

I also do not mind option 2, after 1-2 contributions, any contributor could
just ask the committer (who merged those contributions) about contributor
permissions.

Best,
Andrey

On Mon, Apr 15, 2019 at 10:34 AM Feng LI  wrote:

> Hello there,
>
> New to the community. Just thought you might want some inputs from new
> comers too.
>
> I prefer option 2, where you need to prove the ability and commitment with
> commits  before contributor permission is assigned.
>
> Cheers,
> Feng
>
> Le lun. 15 avr. 2019 à 09:17, Robert Metzger  a
> écrit :
>
> > @Andrey: You mention "option 2" two times, I guess one of the two uses of
> > "option 2" contains a typo?
> >
> > On Wed, Apr 10, 2019 at 10:33 AM Andrey Zagrebin 
> > wrote:
> >
> > > Hi all,
> > >
> > > +1 for option 2.
> > >
> > > I also do not mind option 2, after 1-2 contributions, any contributor
> > could
> > > just ask the committer (who merged those contributions) about
> contributor
> > > permissions.
> > >
> > > Best,
> > > Andrey
> > >
> > > On Wed, Apr 10, 2019 at 3:58 AM Robert Metzger 
> > > wrote:
> > >
> > > > I'm +1 on option 1.
> > > >
> > > > On Tue, Apr 9, 2019 at 1:58 AM Timo Walther 
> > wrote:
> > > >
> > > > > Hi everyone,
> > > > >
> > > > > I'd like to bring up this discussion thread again. In summary, I
> > think
> > > > > we all agreed on improving the JIRA workflow to move
> design/consensus
> > > > > discussions from PRs to the issues first, before implementing them.
> > > > >
> > > > > Two options have been proposed:
> > > > > 1. Only committers can assign people to issues. PRs of unassigned
> > > issues
> > > > > are closed automatically.
> > > > > 2. Committers upgrade assignable users to contributors as an
> > > > > intermediate step towards committership.
> > > > >
> > > > > I would prefer option 1 as some people also mentioned that option 2
> > > > > requires some standadized processes otherwise it would be difficult
> > to
> > > > > communicate why somebody is a contributor and some somebody is not.
> > > > >
> > > > > What do you think?
> > > > >
> > > > > Regards,
> > > > > Timo
> > > > >
> > > > >
> > > > > Am 18.03.19 um 14:25 schrieb Robert Metzger:
> > > > > > @Fabian: I don't think this is a big problem. Moving away from
> > > "giving
> > > > > > everybody contributor permissions" to "giving it to some people"
> is
> > > not
> > > > > > risky.
> > > > > > I would leave this decision to the committers who are working
> with
> > a
> > > > > person.
> > > > > >
> > > > > >
> > > > > > We should bring this discussion to a conclusion and implement the
> > > > changes
> > > > > > to JIRA.
> > > > > >
> > > > > >
> > > > > > Nobody has raised any objections to the overall idea.
> > > > > >
> > > > > > Points raised:
> > > > > > 1. We need to update the contribution guide and describe the
> > > workflow.
> > > > > > 2. I brought up changing Flinkbot so that it auto-closes PRs
> > without
> > > > > > somebody assigned in JIRA.
> > > > > >
> > > > > > Who wants to work on an update of the contribution guide?
> > > > > > If there's no volunteers, I'm happy to take care of this.
> > > > > >
> > > > > >
> > > > > > On Fri, Mar 15, 2019 at 9:20 AM Fabian Hueske  >
> > > > wrote:
> > > > > >
> > > > > >> Hi,
> > > > > >>
> > > > > >> I'm not sure about adding an additional stage.
> > > > > >> Who's going to decide when to "promote" a user to a contributor,
> > > i.e.,
> > > > > >> grant assigning permission?
> > > > > >>
> > > > > >> Best, Fabian
> > > > > >>
> &g

Re: [Discuss][FLINK-8297]A solution for FLINK-8297 Timebased RocksDBListState

2019-04-16 Thread Andrey Zagrebin
Hi Faxian,

True, we can resolve timestamp conflicts putting values into the same row,
good point.
Then re-ordering in case of internal clock jump changes behaviour comparing
with the list state we have now.
In this case, it can be similar to dispersing elements by hash and we can
call it a bag, not list.

Best,
Andrey

On Tue, Apr 16, 2019 at 5:29 AM faxianzhao  wrote:

> Hi Yun
> I think whether atomic increased number or timestamp, the key point is
> disperse the elements in the different keys.
> My point is how to design a useful key.
> For the atomic increased number, it will array the elements one by one but
> I
> think the key is useless. Because the max key is not the elements count,
> when we implement the remove method.
> Currently, for the CountEvictor, TimeEvictor and TTL scenario, we should
> iteration all of the elements to find what we want. But if we use timestamp
> key, we could terminal the iteration early to save performance or start
> from
> the available timestamp to iteration the rest elements.
>
>
>
>
> --
> Sent from: http://apache-flink-mailing-list-archive.1008284.n3.nabble.com/
>


Re: Re: [ANNOUNCE] Zhijiang Wang has been added as a committer to the Flink project

2019-07-23 Thread Andrey Zagrebin
Congratulations Zhijiang! Good work!

Best,
Andrey

On Tue, Jul 23, 2019 at 1:40 PM Hequn Cheng  wrote:

> Congratulations Zhijiang!
>
> Best, Hequn
>
> On Tue, Jul 23, 2019 at 6:17 PM JingsongLee  .invalid>
> wrote:
>
> > Congratulations Zhijiang!
> >
> > Best, Jingsong Lee
> >
> >
> > --
> > From:wenlong.lwl 
> > Send Time:2019年7月23日(星期二) 17:34
> > To:dev 
> > Subject:Re: Re: [ANNOUNCE] Zhijiang Wang has been added as a committer to
> > the Flink project
> >
> > Congrats, Zhijiang!
> >
> > On Tue, 23 Jul 2019 at 15:59, Becket Qin  wrote:
> >
> > > Congrats, Zhijiang!
> > >
> > > On Tue, Jul 23, 2019 at 2:01 PM Jark Wu  wrote:
> > >
> > > > Congratulations Zhijiang!
> > > >
> > > >
> > > > On Tue, 23 Jul 2019 at 11:30, vino yang 
> wrote:
> > > >
> > > > > Congratulations Zhijiang!
> > > > >
> > > > > Haibo Sun  于2019年7月23日周二 上午10:48写道:
> > > > >
> > > > > > Congrats, Zhejiang!
> > > > > >
> > > > > >
> > > > > > Best,
> > > > > > Haibo
> > > > > > 在 2019-07-23 10:26:20,"Yun Tang"  写道:
> > > > > > >Congratulations Zhijiang, well deserved.
> > > > > > >
> > > > > > >Best
> > > > > > >
> > > > > > >From: Yingjie Cao 
> > > > > > >Sent: Tuesday, July 23, 2019 10:23
> > > > > > >To: dev@flink.apache.org 
> > > > > > >Subject: Re: [ANNOUNCE] Zhijiang Wang has been added as a
> > committer
> > > to
> > > > > > the Flink project
> > > > > > >
> > > > > > >Congratulations Zhijiang!
> > > > > > >
> > > > > > >yangtao.yt  于2019年7月23日周二
> 上午10:17写道:
> > > > > > >
> > > > > > >> Congrats, Zhejiang!
> > > > > > >>
> > > > > > >> Best,
> > > > > > >> Tao Yang
> > > > > > >>
> > > > > > >> > 在 2019年7月23日,上午9:46,boshu Zheng  写道:
> > > > > > >> >
> > > > > > >> > Congratulations Zhijiang
> > > > > > >> >
> > > > > > >> > 发自我的 iPhone
> > > > > > >> >
> > > > > > >> >> 在 2019年7月23日,上午12:55,Xuefu Z  写道:
> > > > > > >> >>
> > > > > > >> >> Congratulations, Zhijiang!
> > > > > > >> >>
> > > > > > >> >>> On Mon, Jul 22, 2019 at 7:42 AM Bo WANG <
> > > > wbeaglewatc...@gmail.com
> > > > > >
> > > > > > >> wrote:
> > > > > > >> >>>
> > > > > > >> >>> Congratulations Zhijiang!
> > > > > > >> >>>
> > > > > > >> >>>
> > > > > > >> >>> Best,
> > > > > > >> >>>
> > > > > > >> >>> Bo WANG
> > > > > > >> >>>
> > > > > > >> >>>
> > > > > > >> >>> On Mon, Jul 22, 2019 at 10:12 PM Robert Metzger <
> > > > > > rmetz...@apache.org>
> > > > > > >> >>> wrote:
> > > > > > >> >>>
> > > > > > >>  Hey all,
> > > > > > >> 
> > > > > > >>  We've added another committer to the Flink project:
> > Zhijiang
> > > > > Wang.
> > > > > > >> 
> > > > > > >>  Congratulations Zhijiang!
> > > > > > >> 
> > > > > > >>  Best,
> > > > > > >>  Robert
> > > > > > >>  (on behalf of the Flink PMC)
> > > > > > >> 
> > > > > > >> >>>
> > > > > > >> >>
> > > > > > >> >>
> > > > > > >> >> --
> > > > > > >> >> Xuefu Zhang
> > > > > > >> >>
> > > > > > >> >> "In Honey We Trust!"
> > > > > > >>
> > > > > > >>
> > > > > >
> > > > >
> > > >
> > >
> >
>


Re: [Requirement] CI report

2019-07-23 Thread Andrey Zagrebin
Hi,

@Tison I guess it was done because there were many associated comments
notified by everyone involved in PR.

I agree that it would be actually useful for a PR's author to get
notifications about failed/succeeded CI builds, not sure how easy it is
though.

Best,
Andrey


On Tue, Jul 23, 2019 at 10:15 AM Zili Chen  wrote:

> Hi,
>
> Currently, our flinkbot updates CI report on status changing.
>
> However, it updates via editing GitHub comment, which would not send
> a notification to pr creator once status updated.
>
> Said the "PENDING" status is not quite useful, is it possible that
> flinkbot updates a final status(FAILURE/SUCCESS) by adding a new
> comment? This will be like hadoop bot updates on JIRA.
>
>
> Best,
> tison.
>


[DISCUSS][CODE STYLE] Create collections always with initial capacity

2019-08-01 Thread Andrey Zagrebin
Hi all,

As you probably already noticed, Stephan has triggered a discussion thread
about code style guide for Flink [1]. Recently we were discussing
internally some smaller concerns and I would like start separate threads
for them.

This thread is about creating collections always with initial capacity. As
you might have seen, some parts of our code base always initialise
collections with some non-default capacity. You can even activate a check
in IntelliJ Idea that can monitor and highlight creation of collection
without initial capacity.

Pros:
- performance gain if there is a good reasoning about initial capacity
- the capacity is always deterministic and does not depend on any changes
of its default value in Java
- easy to follow: always initialise, has IDE support for detection

Cons (for initialising w/o good reasoning):
- We are trying to outsmart JVM. When there is no good reasoning about
initial capacity, we can rely on JVM default value.
- It is even confusing e.g. for hash maps as the real size depends on the
load factor.
- It would only add minor performance gain.
- a bit more code, increases maintenance burden.

The conclusion is the following at the moment:
Only set the initial capacity if you have a good idea about the expected
size.

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


[DISCUSS][CODE STYLE] Usage of Java Optional

2019-08-01 Thread Andrey Zagrebin
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


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

2019-08-01 Thread Andrey Zagrebin
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] Usage of Java Optional

2019-08-01 Thread Andrey Zagrebin
EDIT: for Optional in public API vs performance concerns

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:

   - Always use Optional only to return nullable values in the API/public
   methods
  - Only if you can prove that Optional usage would lead to a
  performance degradation in critical code then return nullable value
  directly and annotate it with @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

On Thu, Aug 1, 2019 at 6: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: [VOTE] Apache Flink Release 1.9.0, release candidate #2

2019-08-15 Thread Andrey Zagrebin
+1 (non-binding)

Tested in AWS EMR Yarn: 1 master and 4 worker nodes (m5.xlarge: 4 vCore, 16
GiB).

EMR runs only on Java 8. Fine-grained recovery is enabled by default.

Modified E2E test scripts can be found here (asserting output):
https://github.com/azagrebin/flink/commits/FLINK-13597

Batch SQL:

   - S3(a) filesystem over HADOOP works out-of-the-box (already on AWS
   class path) and also if put in plugins

Streaming SQL:

   - Hadoop output (s3 does not support recoverable writers)


On Thu, Aug 15, 2019 at 11:24 AM Kurt Young  wrote:

> HI,
>
> We just find a serious bug around blink planner:
> https://issues.apache.org/jira/browse/FLINK-13708
> When user reused the table environment instance, and call `execute` method
> multiple times for
> different sql, the later call will trigger the earlier ones to be
> re-executed.
>
> It's a serious bug but seems we also have a work around, which is never
> reuse the table environment
> object. I'm not sure if we should treat this one as blocker issue of 1.9.0.
>
> What's your opinion?
>
> Best,
> Kurt
>
>
> On Thu, Aug 15, 2019 at 2:01 PM Gary Yao  wrote:
>
> > +1 (non-binding)
> >
> > Jepsen test suite passed 10 times consecutively
> >
> > On Wed, Aug 14, 2019 at 5:31 PM Aljoscha Krettek 
> > wrote:
> >
> > > +1
> > >
> > > I did some testing on a Google Cloud Dataproc cluster (it gives you a
> > > managed YARN and Google Cloud Storage (GCS)):
> > >   - tried both YARN session mode and YARN per-job mode, also using
> > > bin/flink list/cancel/etc. against a YARN session cluster
> > >   - ran examples that write to GCS, both with the native Hadoop
> > FileSystem
> > > and a custom “plugin” FileSystem
> > >   - ran stateful streaming jobs that use GCS as a checkpoint backend
> > >   - tried running SQL programs on YARN using the SQL Cli: this worked
> for
> > > YARN session mode but not for YARN per-job mode. Looking at the code I
> > > don’t think per-job mode would work from seeing how it is implemented.
> > But
> > > I think it’s an OK restriction to have for now
> > >   - in all the testing I had fine-grained recovery (region failover)
> > > enabled but I didn’t simulate any failures
> > >
> > > > On 14. Aug 2019, at 15:20, Kurt Young  wrote:
> > > >
> > > > Hi,
> > > >
> > > > Thanks for preparing this release candidate. I have verified the
> > > following:
> > > >
> > > > - verified the checksums and GPG files match the corresponding
> release
> > > files
> > > > - verified that the source archives do not contains any binaries
> > > > - build the source release with Scala 2.11 successfully.
> > > > - ran `mvn verify` locally, met 2 issuses [FLINK-13687] and
> > > [FLINK-13688],
> > > > but
> > > > both are not release blockers. Other than that, all tests are passed.
> > > > - ran all e2e tests which don't need download external packages (it's
> > > very
> > > > unstable
> > > > in China and almost impossible to download them), all passed.
> > > > - started local cluster, ran some examples. Met a small website
> display
> > > > issue
> > > > [FLINK-13591], which is also not a release blocker.
> > > >
> > > > Although we have pushed some fixes around blink planner and hive
> > > > integration
> > > > after RC2, but consider these are both preview features, I'm lean to
> be
> > > ok
> > > > to release
> > > > without these fixes.
> > > >
> > > > +1 from my side. (binding)
> > > >
> > > > Best,
> > > > Kurt
> > > >
> > > >
> > > > On Wed, Aug 14, 2019 at 5:13 PM Jark Wu  wrote:
> > > >
> > > >> Hi Gordon,
> > > >>
> > > >> I have verified the following things:
> > > >>
> > > >> - build the source release with Scala 2.12 and Scala 2.11
> successfully
> > > >> - checked/verified signatures and hashes
> > > >> - checked that all POM files point to the same version
> > > >> - ran some flink table related end-to-end tests locally and
> succeeded
> > > >> (except TPC-H e2e failed which is reported in FLINK-13704)
> > > >> - started cluster for both Scala 2.11 and 2.12, ran examples,
> verified
> > > web
> > > >> ui and log output, nothing unexpected
> > > >> - started cluster, ran a SQL query to temporal join with kafka
> source
> > > and
> > > >> mysql jdbc table, and write results to kafka again. Using DDL to
> > create
> > > the
> > > >> source and sinks. looks good.
> > > >> - reviewed the release PR
> > > >>
> > > >> As FLINK-13704 is not recognized as blocker issue, so +1 from my
> side
> > > >> (non-binding).
> > > >>
> > > >> On Tue, 13 Aug 2019 at 17:07, Till Rohrmann 
> > > wrote:
> > > >>
> > > >>> Hi Richard,
> > > >>>
> > > >>> although I can see that it would be handy for users who have PubSub
> > set
> > > >> up,
> > > >>> I would rather not include examples which require an external
> > > dependency
> > > >>> into the Flink distribution. I think examples should be
> > self-contained.
> > > >> My
> > > >>> concern is that we would bloat the distribution for many users at
> the
> > > >>> benefit of a few. Instead, I think it would be better to mak

Re: [ANNOUNCE] Andrey Zagrebin becomes a Flink committer

2019-08-19 Thread Andrey Zagrebin
Hi Everybody!

Thanks a lot for the warn welcome!
I am really happy about joining Flink committer team and hope to help the
project to grow more.

Cheers,
Andrey

On Fri, Aug 16, 2019 at 11:10 AM Terry Wang  wrote:

> Congratulations Andrey!
>
> Best,
> Terry Wang
>
>
>
> 在 2019年8月15日,下午9:27,Hequn Cheng  写道:
>
> Congratulations Andrey!
>
> On Thu, Aug 15, 2019 at 3:30 PM Fabian Hueske  wrote:
>
>> Congrats Andrey!
>>
>> Am Do., 15. Aug. 2019 um 07:58 Uhr schrieb Gary Yao :
>>
>> > Congratulations Andrey, well deserved!
>> >
>> > Best,
>> > Gary
>> >
>> > On Thu, Aug 15, 2019 at 7:50 AM Bowen Li  wrote:
>> >
>> > > Congratulations Andrey!
>> > >
>> > > On Wed, Aug 14, 2019 at 10:18 PM Rong Rong 
>> wrote:
>> > >
>> > >> Congratulations Andrey!
>> > >>
>> > >> On Wed, Aug 14, 2019 at 10:14 PM chaojianok 
>> wrote:
>> > >>
>> > >> > Congratulations Andrey!
>> > >> > At 2019-08-14 21:26:37, "Till Rohrmann" 
>> wrote:
>> > >> > >Hi everyone,
>> > >> > >
>> > >> > >I'm very happy to announce that Andrey Zagrebin accepted the
>> offer of
>> > >> the
>> > >> > >Flink PMC to become a committer of the Flink project.
>> > >> > >
>> > >> > >Andrey has been an active community member for more than 15
>> months.
>> > He
>> > >> has
>> > >> > >helped shaping numerous features such as State TTL, FRocksDB
>> release,
>> > >> > >Shuffle service abstraction, FLIP-1, result partition management
>> and
>> > >> > >various fixes/improvements. He's also frequently helping out on
>> the
>> > >> > >user@f.a.o mailing lists.
>> > >> > >
>> > >> > >Congratulations Andrey!
>> > >> > >
>> > >> > >Best, Till
>> > >> > >(on behalf of the Flink PMC)
>> > >> >
>> > >>
>> > >
>> >
>>
>
>


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

2019-08-19 Thread Andrey Zagrebin
 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
> > > >
> > >
> >
>


Re: [DISCUSS][CODE STYLE] Create collections always with initial capacity

2019-08-19 Thread Andrey Zagrebin
Hi All,

It looks like this proposal has an approval and we can conclude this
discussion.
Additionally, I agree with Piotr we should really force the proven good
reasoning for setting the capacity to avoid confusion, redundancy and other
already mentioned things while reading and maintaining the code.
Ideally the need of setting the capacity should be either immediately clear
(e.g. perf etc) or explained in comments if it is non-trivial.
Although, it can easily enter a grey zone, so I would not demand strictly
performance measurement proof e.g. if the size is known and it is "per
record" code.
At the end of the day it is a decision of the code developer and reviewer.

The conclusion is then:
Set the initial capacity only if there is a good proven reason to do it.
Otherwise do not clutter the code with it.

Best,
Andrey

On Thu, Aug 1, 2019 at 5:10 PM Piotr Nowojski  wrote:

> Hi,
>
> > - a bit more code, increases maintenance burden.
>
> I think there is even more to that. It’s almost like a code duplication,
> albeit expressed in very different way, with all of the drawbacks of
> duplicated code: initial capacity can drift out of sync, causing confusion.
> Also it’s not “a bit more code”, it might be non trivial
> reasoning/calculation how to set the initial value. Whenever we change
> something/refactor the code, "maintenance burden” will mostly come from
> that.
>
> Also I think this just usually falls under a premature optimisation rule.
>
> Besides:
>
> > The conclusion is the following at the moment:
> > Only set the initial capacity if you have a good idea about the expected
> size.
>
> I would add a clause to set the initial capacity “only for good proven
> reasons”. It’s not about whether we can set it, but whether it makes sense
> to do so (to avoid the before mentioned "maintenance burden”).
>
> Piotrek
>
> > On 1 Aug 2019, at 14:41, Xintong Song  wrote:
> >
> > +1 on setting initial capacity only when have good expectation on the
> > collection size.
> >
> > Thank you~
> >
> > Xintong Song
> >
> >
> >
> > On Thu, Aug 1, 2019 at 2:32 PM Andrey Zagrebin 
> wrote:
> >
> >> Hi all,
> >>
> >> As you probably already noticed, Stephan has triggered a discussion
> thread
> >> about code style guide for Flink [1]. Recently we were discussing
> >> internally some smaller concerns and I would like start separate threads
> >> for them.
> >>
> >> This thread is about creating collections always with initial capacity.
> As
> >> you might have seen, some parts of our code base always initialise
> >> collections with some non-default capacity. You can even activate a
> check
> >> in IntelliJ Idea that can monitor and highlight creation of collection
> >> without initial capacity.
> >>
> >> Pros:
> >> - performance gain if there is a good reasoning about initial capacity
> >> - the capacity is always deterministic and does not depend on any
> changes
> >> of its default value in Java
> >> - easy to follow: always initialise, has IDE support for detection
> >>
> >> Cons (for initialising w/o good reasoning):
> >> - We are trying to outsmart JVM. When there is no good reasoning about
> >> initial capacity, we can rely on JVM default value.
> >> - It is even confusing e.g. for hash maps as the real size depends on
> the
> >> load factor.
> >> - It would only add minor performance gain.
> >> - a bit more code, increases maintenance burden.
> >>
> >> The conclusion is the following at the moment:
> >> Only set the initial capacity if you have a good idea about the expected
> >> size.
> >>
> >> 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] Usage of Java Optional

2019-08-19 Thread Andrey Zagrebin
> 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  >> .invalid>
> >>> 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写道:
> >>>>>>
> &g

Re: Why documentation always say checkpoint does not support Flink specific features like rescaling.

2018-08-27 Thread Andrey Zagrebin
As I understand, this is more about the documented semantics of checkpoint 
feature that Flink community agreed to guarantee for users so far in the last 
releases. Although technically the rescaling is currently possible for full and 
incremental checkpoints, it can change in future if checkpoints would need 
breaking optimisations or on contrary it might be fixed in the docs as 
supported from some release and on. In general, checkpoint API is currently 
designed for failure recovery of running job, not as savepoints for manual 
restart including rescaling.

> On 27 Aug 2018, at 11:23, Chesnay Schepler  wrote:
> 
> Well that's a single StateBackend. In order to say that "Checkpoints support 
> rescaling" this must apply to all StateBackends, which AFAIK is not the case.
> 
> On 27.08.2018 10:52, Yun Tang wrote:
>> Hi, all
>> 
>> >From the latest 1.6 (or all the previous verions) documentation 
>> >https://ci.apache.org/projects/flink/flink-docs-release-1.6/ops/state/checkpoints.html#difference-to-savepoints,
>> > it said "Checkpoints have a few differences from savepoints, they do not 
>> >support Flink specific features like rescaling."
>> 
>> However, if you just take a look at RocksDB-StateBackend's code, you could 
>> see clearly that RocksDBStateBackend support rescaling from previous 
>> checkpoint. Could anyone give an explanation?
>> 
>> Best
>> Yun
>> 
> 



Re: [ANNOUNCE] New committer Gary Yao

2018-09-07 Thread Andrey Zagrebin
Congratulations Gary!

> On 7 Sep 2018, at 15:45, Stefan Richter  wrote:
> 
> Congrats Gary!
> 
>> Am 07.09.2018 um 15:14 schrieb Till Rohrmann :
>> 
>> Hi everybody,
>> 
>> On behalf of the PMC I am delighted to announce Gary Yao as a new Flink
>> committer!
>> 
>> Gary started contributing to the project in June 2017. He helped with the
>> Flip-6 implementation, implemented many of the new REST handlers, fixed
>> Mesos issues and initiated the Jepsen-based distributed test suite which
>> uncovered several serious issues. Moreover, he actively helps community
>> members on the mailing list and with PR reviews.
>> 
>> Please join me in congratulating Gary for becoming a Flink committer!
>> 
>> Cheers,
>> Till
> 



Re: Question over Incremental Snapshot vs Full Snapshot in rocksDb state backend

2018-10-25 Thread Andrey Zagrebin
Hi Chandan,

> 1. Why did we took 2 different approaches using different RocksDB apis ?
> We could have used Checkpoint api of RocksDB for fullSnapshot as well .

The reason here is partially historical. Full snapshot in RocksDB backend was 
implemented before incremental and rescaling for incremental snapshot but after 
heap backend. Full snapshot in RocksDB uses approach close to heap backend 
because Flink community plans to support the unified format for savepoints. The 
unified format would make it possible to switch backends and restore from 
savepoint. The formats still differ due to backend specifics to optimise 
snapshotting and restore but it is technically possible to unify them in future.

> 2. Is there any specific reason to use Snapshot API of rocksDB  over 
> Checkpoint api of RocksDB for fullSnapshot?

I think Checkpoint API produces separate SST file list to copy them to HDFS in 
case of incremental snapshot.

Full snapshot does not need the file list, it just needs an iterator over 
snapshotted (frozen) data. Internally RocksDB just hard-links immutable already 
existing SST files and iterates their data for Snapshot API.

Best,
Andrey


> On 24 Oct 2018, at 18:40, chandan prakash  wrote:
> 
> Thanks Tzu-Li for redirecting.
> Would also like to be corrected if my any inference from the code is 
> incorrect or incomplete.
> I am sure it will help to clear doubts of more developers like me  :)
> Thanks in advance.
> 
> Regards,
> Chandan
> 
> 
> On Wed, Oct 24, 2018 at 9:19 PM Tzu-Li (Gordon) Tai  > wrote:
> Hi,
> 
> I’m forwarding this question to Stefan (cc’ed).
> He would most likely be able to answer your question, as he has done 
> substantial work in the RocksDB state backends.
> 
> Cheers,
> Gordon
> 
> 
> On 24 October 2018 at 8:47:24 PM, chandan prakash (chandanbaran...@gmail.com 
> ) wrote:
> 
>> Hi,
>> I am new to Flink.
>> Was looking into the code to understand how Flink does FullSnapshot and 
>> Incremental Snapshot using RocksDB
>> 
>> What I understood:
>> 1. For full snapshot, we call RocksDb snapshot api which basically an 
>> iterator handle to the entries in RocksDB instance. We iterate over every 
>> entry one by one and serialize that to some distributed file system. 
>> Similarly in restore for fullSnapshot, we read the file to get every entry 
>> and apply that to the rocksDb instance one by one to fully construct the db 
>> instance.
>> 
>> 2. On the other hand in for Incremental Snapshot, we rely on RocksDB 
>> Checkpoint api to copy the sst files to HDFS/S3 incrementally.
>> Similarly on restore, we copy the sst files to local directory and 
>> instantiate rocksDB instance with the path of the directory.
>> 
>> My Question is:
>> 1. Why did we took 2 different approaches using different RocksDB apis ?
>> We could have used Checkpoint api of RocksDB for fullSnapshot as well .
>> 2. Is there any specific reason to use Snapshot API of rocksDB  over 
>> Checkpoint api of RocksDB for fullSnapshot?
>> 
>> I am sure, I am missing some important point, really curious to know that.
>> Any explanation will be really great. Thanks in advance.
>> 
>> 
>> Regards,
>> Chandan
>> 
>> 
>> 
>> 
>> 
>> --
>> Chandan Prakash
>> 
> 
> 
> -- 
> Chandan Prakash
> 



Re: [DISCUSS] Proposal of external shuffle service

2018-12-09 Thread Andrey Zagrebin
Hi Zhijiang,

Thanks for sharing the document Zhijiang. 
I decided to compile my thoughts to consider here, not to overload document 
comments any more :)

I think I still have question about job level configuration for the shuffle 
service. You mentioned that we can keep several shuffle manager objects in one 
task executor for different jobs. This is valid. My concerns are:
- how do we share shuffle manager resources among different job tasks within 
one task executor process? It could be some static objects shared by all 
shuffle manager objects of some type but it might be not scalable approach. 
Example could be multiplexed netty connections (as I understand, current netty 
stack can become just custom shuffle service).
- In case of having it per job, we might need to provide compatibility check 
between shuffle service and cluster mode (e.g. yarn ext shuffle service for 
standalone mode cluster) if it is an issue.
- Having it per job feels like the same complexity as having it per operator, 
at the first glance, just changes its granularity and where objects reside.
- what is the problem to use cluster per job mode? Then shuffle manager per 
cluster and per job is the same but might simplify other issues at the 
beginning. Streaming and batch jobs with different shuffle requirements could 
be started in different clusters per job. 

As for ShuffleManager interface, I think I see your point with the 
ResultPartitionLocation. I agree that partition needs some addressing of 
underlying connection or resources in general. It can be thinked of as an 
argument of ShuffleManager factory methods.

My point is that task code might not need to be coupled to shuffle interface. 
This way we could keep task code more independent of records transfer layer. We 
can always change later how shuffle/network service is organised internally 
without any consequences for the general task code. If task code calls just 
factories provided by JM, it might not even matter for the task in future 
whether it is configured per cluster, job or operator. Internally, factory can 
hold location of concrete type if needed.

Code example could be:

Job Manager side:

interface ShuffleManager {
  ResultPartionWriterFactory createResultPartionWriterFactory(job/task/topology 
descriptors);
  // similar for input gate factory
}

class ShuffleManagerImpl implements ShuffleManager {
  private general config, services etc;
  ResultPartionWriterFactory createResultPartionWriterFactory(job/task/topology 
descriptors) {
return new ResultPartionWriterFactoryImpl(location, job, oper id, other 
specific config etc);
  }
  // similar for input gate factory
}
...
// somewhere in higher level code put ResultPartionWriterFactory into descriptor

Task executor side receives the factory inside the descriptor and calls 
factory.create(ShuffleServiceRegistry). Example of factory:

class ResultPartionWriterFactoryImpl implements ResultPartionWriterFactory {
  // all fields are lightweight and serialisable, received from JM
  private location, shuffle service id, other specific config etc;
  
  ResultPartionWriter create(ShuffleServiceRegistry registry, maybe more 
generic args) {
// get or create task local specific ShuffleServiceImpl by id in registry
// ShuffleServiceImpl object can be shared between jobs
// register with the ShuffleServiceImpl by location, id, config etc
  }
}

interface ShuffleService extends AutoClosable {
  getId();
}

ShuffleServiceImpl manages resources and decides internally whether to do it 
per task executor, task, job or operator. It can contain network stack, e,g, 
netty connections etc. In case of external service, it can hold partition 
manager, transport client etc. It is not enforced to have it per job by this 
contract or even to have it at all. ShuffleServiceImpl also does not need to 
depend on all TaskManagerServices, only create relevant inside, e.g. network.

class ShuffleServiceRegistry {
   T getShuffleService(id);
 registerShuffleService(ShuffleService, id);
  deregisterShuffleService(id); // remove and close ShuffleService
  close(); // close all
}

ShuffleServiceRegistry is just a generic container of all available 
ShuffleService’s. It could be part of TaskManagerServices instead of 
NetworkEnvironment which could go into specific ShuffleServiceImpl.

I might still miss some details, I would appreciate any feedback.

Best,
Andrey

> On 28 Nov 2018, at 08:59, zhijiang  wrote:
> 
> Hi all,
> 
> I adjusted the umbrella jira [1] and corresponding google doc [2] to narrow 
> down the scope of introducing pluggable shuffle manager architecture as the 
> first step. 
> Welcome further feedbacks and suggestions, then I would create specific 
> subtasks for it to forward.
> 
> [1] https://issues.apache.org/jira/browse/FLINK-10653
> 
> [2] 
> https://docs.google.com/document/d/1ssTu8QE8RnF31zal4JHM1VaVENow-PweUtXSRr68nGg/edit?usp=sharing
> -

Re: [DISCUSS] Proposal of external shuffle service

2018-12-19 Thread Andrey Zagrebin
s the easiest way 
> first and adjut the level if needed in future.
> 
> 2. ShuffleManager interface
> 
> I think you mentioned three sub issues in this part:
> 
> 2.1 Introduction of additional ResultPartitionWriterFactory && 
> InputGateReaderFactory
> 
> I am not against the introduction of these two factories. The original 
> introduction of pluggable ShuffleManager interface is for creating different 
> writer and reader sides. If the ShuffleManager interface is used for creating 
> factories, and then the factories are used for creating writer and reader. I 
> still think the essence is same, and only the form is different.  That is the 
> ShuffleManager concept is seen on JobManager side, and the task only sees the 
> corresponding factories from ShuffleManager. In other words, we add another 
> factory layer to distinguish between JobManager and task. The form might seem 
> a bit better to introduce corresponding factories, so I am willing to take 
> this way for implementation.
> 
> 2.2 Whether to retain getResultPartitionLocation method in ShuffleManager 
> interface
> 
> If I understand correctly, you mean to put this location as an argument in 
> InputGateReaderFacotry constructor? If to do so, I think it makes sense and 
> we can avoid have this explicit method in interface. But we also need to 
> adjust the existing related process like updatePartitionInfo for downstream 
> side. In this case, the partition location is unknown during deploying 
> downstream tasks. Based on upstream's consumable notification, the location 
> update is triggered by JobManager to downstream side.
> 
> 2.3 ShuffleService interface
> 
> My initial thought is not making it as an interface. Because for internal or 
> external shuffle cases, they can reuse the same unified netty-based shuffle 
> service if we wrap the related componenets into current shuffle service well. 
> If we want to furtherextend other implementations of shuffle service, like 
> http-based shuffle service, then we can define an interface for it, the way 
> as current RpcService interface to get ride of only akka implementations. So 
> it also makes sense on my side to keep this interface. As for 
> ShuffleServiceRegistry class, I agree with you to have this TaskManager level 
> service for managing and sharing for all the internal tasks.
> 
> In summary, I think we do not have essential conflicts for above issues, 
> almost for the implementation aspects. And I agree with the above points, 
> especially for above 2.2 you might need double check if I understand 
> correctly. 
> Wish your further feedbacks then I can adjust the docs based on it.  Also 
> welcome any other person's feedbacks!
> 
> 
> Best,
> Zhijiang
> 
> 
> --
> 发件人:Andrey Zagrebin 
> 发送时间:2018年12月10日(星期一) 05:18
> 收件人:dev ; zhijiang 
> 抄 送:Nico Kruber ; Piotr Nowojski 
> ; Stephan Ewen ; Till Rohrmann 
> 
> 主 题:Re: [DISCUSS] Proposal of external shuffle service
> 
> Hi Zhijiang,
> 
> Thanks for sharing the document Zhijiang.
> I decided to compile my thoughts to consider here, not to overload document 
> comments any more :)
> 
> I think I still have question about job level configuration for the shuffle 
> service. You mentioned that we can keep several shuffle manager objects in 
> one task executor for different jobs. This is valid. My concerns are:
> - how do we share shuffle manager resources among different job tasks within 
> one task executor process? It could be some static objects shared by all 
> shuffle manager objects of some type but it might be not scalable approach. 
> Example could be multiplexed netty connections (as I understand, current 
> netty stack can become just custom shuffle service).
> - In case of having it per job, we might need to provide compatibility check 
> between shuffle service and cluster mode (e.g. yarn ext shuffle service for 
> standalone mode cluster) if it is an issue.
> - Having it per job feels like the same complexity as having it per operator, 
> at the first glance, just changes its granularity and where objects reside.
> - what is the problem to use cluster per job mode? Then shuffle manager per 
> cluster and per job is the same but might simplify other issues at the 
> beginning. Streaming and batch jobs with different shuffle requirements could 
> be started in different clusters per job.
> 
> As for ShuffleManager interface, I think I see your point with the 
> ResultPartitionLocation. I agree that partition needs some addressing of 
> underlying connection or resources in general. It can be thinked of as an 
> argument of ShuffleManager factory methods.
> 
> My po

Re: [DISCUSS] Proposal of external shuffle service

2018-12-24 Thread Andrey Zagrebin
feremt times, it is betterto support job level configuration for
> fexibility. Certainly it may not be a strong requirements for most cases,
> then we can reach an agreement to make the cluster level as the easiest way
> first and adjut the level if needed in future.
>
> 2. ShuffleManager interface
>
> I think you mentioned three sub issues in this part:
>
> 2.1 Introduction of additional ResultPartitionWriterFactory &&
> InputGateReaderFactory
>
> I am not against the introduction of these two factories. The original
> introduction of pluggable ShuffleManager interface is for creating
> different writer and reader sides. If the ShuffleManager interface is
> used for creating factories, and then the factories are used for creating
> writer and reader. I still think the essence is same, and only the form
> is different.  That is the ShuffleManager concept is seen on JobManager
> side, and the task only sees the corresponding factories from
> ShuffleManager. In other words, we add another factory layer to distinguish
> between JobManager and task. The form might seem a bit better to introduce
> corresponding factories, so I am willing to take this way for
> implementation.
>
> 2.2 Whether to retain getResultPartitionLocation method in ShuffleManager
> interface
>
> If I understand correctly, you mean to put this location as an argument in
> InputGateReaderFacotry constructor? If to do so, I think it makes sense and
> we can avoid have this explicit method in interface. But we also need to
> adjust the existing related process like updatePartitionInfo for downstream
> side. In this case, the partition location is unknown during deploying
> downstream tasks. Based on upstream's consumable notification, the location
> update is triggered by JobManager to downstream side.
>
> 2.3 ShuffleService interface
>
> My initial thought is not making it as an interface. Because for internal
> or external shuffle cases, they can reuse the same unified netty-based
> shuffle service if we wrap the related componenets into current shuffle
> service well. If we want to furtherextend other implementations of shuffle
> service, like http-based shuffle service, then we can define an interface
> for it, the way as current RpcService interface to get ride of only akka
> implementations. So it also makes sense on my side to keep this interface.
> As for ShuffleServiceRegistry class, I agree with you to have this
> TaskManager level service for managing and sharing for all the internal
> tasks.
>
> In summary, I think we do not have essential conflicts for above issues,
> almost for the implementation aspects. And I agree with the above points,
> especially for above 2.2 you might need double check if I understand
> correctly.
> Wish your further feedbacks then I can adjust the docs based on it.  Also
> welcome any other person's feedbacks!
>
>
> Best,
> Zhijiang
>
>
> --
> 发件人:Andrey Zagrebin 
> 发送时间:2018年12月10日(星期一) 05:18
> 收件人:dev ; zhijiang 
> 抄 送:Nico Kruber ; Piotr Nowojski <
> pi...@data-artisans.com>; Stephan Ewen ; Till Rohrmann <
> trohrm...@apache.org>
> 主 题:Re: [DISCUSS] Proposal of external shuffle service
>
> Hi Zhijiang,
>
> Thanks for sharing the document Zhijiang.
> I decided to compile my thoughts to consider here, not to overload
> document comments any more :)
>
> I think I still have question about job level configuration for the
> shuffle service. You mentioned that we can keep several shuffle manager
> objects in one task executor for different jobs. This is valid. My concerns
> are:
> - how do we share shuffle manager resources among different job tasks
> within one task executor process? It could be some static objects shared by
> all shuffle manager objects of some type but it might be not scalable
> approach. Example could be multiplexed netty connections (as I understand,
> current netty stack can become just custom shuffle service).
> - In case of having it per job, we might need to provide compatibility
> check between shuffle service and cluster mode (e.g. yarn ext shuffle
> service for standalone mode cluster) if it is an issue.
> - Having it per job feels like the same complexity as having it per
> operator, at the first glance, just changes its granularity and where
> objects reside.
> - what is the problem to use cluster per job mode? Then shuffle manager
> per cluster and per job is the same but might simplify other issues at the
> beginning. Streaming and batch jobs with different shuffle requirements
> could be started in different clusters per job.
>
> As for ShuffleManager interface, I think I see your poin

[SURVEY] Custom RocksDB branch

2019-01-15 Thread Andrey Zagrebin
Dear Flink users and developers!

I start this discussion to collect feedback about maintaining a custom
RocksDB branch for Flink, if anyone sees any problems with this approach.
Are there people who already uses a custom RocksDB client build with
RocksDB state backend?

As you might already know, the community is trying currently to make some
improvements in RocksDB state backend which require a newer version of
RocksDB client and some Flink specific changes of its code base.

In particular:
- custom c++ compaction filter for background cleanup of expired state
entries with TTL [1]
- merge operator without separating commas for the list state [2]

The TTL compaction filter is too Flink specific to merge it into the
RocksDB repository.
The second one might take time to be merged and released in RocksDB.
The plug-ability of filter/merge operators in RocksDB would allow the code
to be kept on Flink side but it is currently under development and ETA is
not cleat at the moment.

To unblock releasing of related Flink features, we suggest to create and
maintain a branch of RocksDB which would include these changes, as it
happened already in the past. The RocksDB backend module can be built
against this branch. We can get back to normal RocksDB releases, once it
allows Flink code to be kept out of it.

Thanks,
Andrey

[1] https://issues.apache.org/jira/browse/FLINK-10471
[2] https://github.com/facebook/rocksdb/pull/4806


Re: [RESULT][VOTE] Clean up and reorganize the JIRA components

2019-02-26 Thread Andrey Zagrebin
Hi all,

Thank you for such useful effort, Robert!

I would also suggest a separate tag for state TTL, like queryable state.
There might be not so many issues atm but it might help in future.

Thanks,
Andrey

On Mon, Feb 25, 2019 at 8:02 PM Robert Metzger  wrote:

> The proposal has been approved with
>
> +1 votes:
> - Timo
> - Robert
> - Jincheng
> - Shengjk1
> - Chesnay
> - Till
>
> With +4 PMC votes, the vote has passed.
>
> Thanks everybody!
> I'll soon start with this effort!
>
> On Mon, Feb 25, 2019 at 1:28 PM Till Rohrmann 
> wrote:
>
> > +1
> >
> > On Mon, Feb 25, 2019 at 11:06 AM Chesnay Schepler 
> > wrote:
> >
> >> +1
> >>
> >> On 25.02.2019 09:51, Robert Metzger wrote:
> >> > You are right. I've fixed it:
> >> >
> >>
> https://cwiki.apache.org/confluence/pages/diffpagesbyversion.action?pageId=103089605&selectedPageVersions=7&selectedPageVersions=6
> >> >
> >> > On Thu, Feb 21, 2019 at 4:27 PM Chesnay Schepler  >> > > wrote:
> >> >
> >> > Then the respective target components should have a "split
> >> > Job-Submission" migration path attached to them.
> >> > None of the 3 components you just listed have this.
> >> >
> >> > On 21.02.2019 16:10, Robert Metzger wrote:
> >> > > Thank you for the minor issues you've found. I addressed them:
> >> > >
> >> >
> >>
> https://cwiki.apache.org/confluence/pages/diffpagesbyversion.action?pageId=103089605&selectedPageVersions=6&selectedPageVersions=5
> >> > >
> >> > > @Chesnay: Regarding the "Job-Submission": I quickly looked over
> >> the
> >> > > tickets. Many of them clearly fit into categories like YARN,
> >> > "Runtime / Web
> >> > > Frontend", "Deployment / Scripts". For the difficult cases,
> >> > we'll find a
> >> > > solution :)
> >> > >
> >> > >
> >> > > On Thu, Feb 21, 2019 at 5:57 AM shengjk1  >> > > wrote:
> >> > >
> >> > >> +1 for the vote
> >> > >>
> >> > >>
> >> > >> Also agree with Chesnay Schepler.
> >> > >>
> >> > >>
> >> > >> There area few minor things  I just noticed, except as
> >> > mentioned in the
> >> > >> Chesnay Schepler email:
> >> > >>
> >> > >>
> >> > >> * "Scala Shell" is to be renamed to "Scala Shell", but it
> >> > should say keep
> >> > >>
> >> > >>
> >> > >> Best,
> >> > >> Shengjk1
> >> > >>
> >> > >>
> >> > >>
> >> > >>
> >> > >> On 02/20/2019 18:31,Chesnay Schepler >> > > wrote:
> >> > >> There area few minor things I just noticed:
> >> > >>
> >> > >> * "Job-Submission" is to be merged into "Command Line Client",
> >> > but I'm
> >> > >> certain that there are issue in that category that also/only
> >> apply
> >> > >> to REST. The same applies to Web Client I suppose, which should
> >> be
> >> > >> primarily about the WebFrontend / REST API.
> >> > >> * "Quickstarts" is to be renamed to "Quickstarts", but it
> >> > should say keep
> >> > >> * Gelly/CEP should be marked as renamed
> >> > >> * there is no "Twitter" nor "Wikiedits" component that could be
> >> > moved
> >> > >> to "Examples"
> >> > >>
> >> > >> The first point is the important one, whereas the rest are
> >> > probably just
> >> > >> things that slipped through while updating the proposal.
> >> > >>
> >> > >> On 20.02.2019 11:04, Timo Walther wrote:
> >> > >> +1
> >> > >>
> >> > >> Thanks for the proposal.
> >> > >>
> >> > >> Timo
> >> > >>
> >> > >> Am 20.02.19 um 10:53 schrieb Robert Metzger:
> >> > >> Hey all,
> >> > >>
> >> > >> As discussed in the last days, I'm proposing to clean up and
> >> > >> reorganize our
> >> > >> JIRA tickets.
> >> > >> Chesnay proposed to approve the final proposal through a VOTE.
> >> > >>
> >> > >> Here is the proposal:
> >> > >>
> >> > >>
> >> > >>
> >> >
> >>
> https://cwiki.apache.org/confluence/display/FLINK/Proposal+for+new+JIRA+Components
> >> > >>
> >> > >>
> >> > >>
> >> > >> Please vote with:
> >> > >>
> >> > >> [  ] +1 to agree
> >> > >> [  ]   0 I have no strong opinion
> >> > >> [  ]  -1 to object on the following grounds: ...
> >> > >>
> >> > >> The vote will end on Monday, Feb 25th, 2019, at 11am CET.
> >> > >> The vote will be accepted if there are at least 3 PMC +1s and
> no
> >> > >> objections.
> >> > >>
> >> > >>
> >> > >>
> >> > >>
> >> > >>
> >> >
> >>
> >>
>


Re: [ANNOUNCE] Zhu Zhu becomes a Flink committer

2019-12-16 Thread Andrey Zagrebin
Congrats Zhu Zhu!

On Mon, Dec 16, 2019 at 8:10 AM Xintong Song  wrote:

> Congratulations Zhu Zhu~
>
> Thank you~
>
> Xintong Song
>
>
>
> On Mon, Dec 16, 2019 at 12:34 PM Danny Chan  wrote:
>
> > Congrats Zhu Zhu!
> >
> > Best,
> > Danny Chan
> > 在 2019年12月14日 +0800 AM12:51,dev@flink.apache.org,写道:
> > >
> > > Congrats Zhu Zhu and welcome on board!
> >
>


Re: [DISCUSS] Some feedback after trying out the new FLIP-49 memory configurations

2019-12-20 Thread Andrey Zagrebin
Hi Stephan and Xintong,

Thanks for the further FLIP-49 feedbacks.

>   - "taskmanager.memory.size" (old main config option) is replaced by 
> "taskmanager.memory.total-process.size" which has a different meaning in 
> standalone setups. The old option did not subtract metaspace and other 
> overhead, while the new option does. That means that with the default config, 
> standalone clusters get quite a bit less memory. (independent of managed 
> memory going off heap).
> I am wondering if we could interpret "taskmanager.memory.size" as the 
> deprecated key for "taskmanager.memory.total-flink.size". That would be in 
> line with the old mechanism (assuming managed memory is set to off heap).
> The effect would be that the container size on Yarn/Mesos increases, 
> because from "taskmanager.memory.total-flink.size", we need to add overhead 
> and metaspace to reach the total process size, rather than cutting off 
> memory. But if we want, we could even adjust for that in the active resource 
> manager, getting full backwards compatibility on that part.
> Curious to hear more thoughts there.
> 
> I believe you mean "taskmanager.heap.size".
> 
> I think the problem here is that the legacy "taskmanager.heap.size" was used 
> differently in standalone setups and active yarn / mesos setups, and such 
> different calculation logics and behaviors are exactly what we want to avoid 
> with FLIP-49. Therefore, I'm not in favor of treating 
> "taskmanager.memory.total-flink.size" differently for standalone and active 
> setups.
> 
> I think what we really want is probably mapping "taskmanager.heap.size" to 
> different new config options in different setups. How about we mark 
> "taskmanager.heap.size" as deprecated key for neither of 
> "taskmanager.memory.total-process.size" and 
> "taskmanager.memory.total-flink.size". Instead, we parse it (if explicitly 
> configured) in startup scripts / active resource managers, and set the value 
> to "taskmanager.memory.total-flink.size" in the scripts and 
> "taskmanager.memory.total-process.size" in active resource managers (if the 
> new config options are not configured). We can provide util methods in 
> TaskExecutorResourceUtils for such conversions, to keep all the configuration 
> logics at one place.

I agree that the problem is that the legacy option ‘taskmanager.heap.size’ has 
different semantics for standalone/container. We had it initially falling back 
to 'taskmanager.memory.total-flink.size’ but I changed that to align it with 
container cut-off. Now I see it changes standalone setup then.
+1 for supporting its backwards compatibility differently for 
standalone/container setups.

> 
>   - Mini Cluster tries to imitate exact ratio of memory pools as a standalone 
> setup. I get the idea behind that, but I am wondering if it is the right 
> approach here.
> For example: I started a relatively large JVM (large heap size of 10 GB) 
> as a test. With the current logic, the system tries to reserve an additional 
> 6GB for managed memory which is more than there is memory left. When you see 
> the error that no memory could be allocated, you need to understand the magic 
> of how this is derived.
> I am trying to think about this from the perspective of using "Flink as a 
> Library", which the MiniCluster is close to.
> When starting Flink out of a running process, we cannot assume that we 
> are the only users of that process and that we can mold the process to our 
> demands. I think a fix value for managed memory and network memory would feel 
> more natural in such a setup than a mechanism that is tailored towards 
> exclusive use of the process.
> 
> +1 on having fixed values for managed / shuffle memory.

also +1 for that, if user has not specified any main options to derive memory. 
We should also log this fixing of managed / shuffle memory.
And just noticed, we could also sanity check framework and if explicitly 
configured task heap against available JVM heap, and at least log 
inconsistencies.

>   - Some off-heap memory goes into direct memory, some does not. This 
> confused me a bit. For example "taskmanager.memory.framework.off-heap.size" 
> is counted into MaxDirectMemory while "taskmanager.memory.task.off-heap.size" 
> is counted as native memory. Maybe we should rename the keys to reflect that. 
> There is no one "off heap" memory type after all. Maybe use 
> "taskmanager.memory.task.native: XXXmb" and 
> "taskmanager.memory.framework.direct: XXXmb" instead?
> 
> I believe "taskmanager.memory.task.off-heap.size" is also accounted in the 
> max direct memory size limit. The confusion probably comes from that 
> "taskmanager.memory.framework.off-heap.size" explicitly mentioned that in its 
> description while "taskmanager.memory.task.off-heap.size" didn't. That's 
> actually because the framework off-heap memory is introduced later in a 
> separate commit. We should fix that.
> 
> For framework / task off-heap memory, we do not 

Re: [DISCUSS] Some feedback after trying out the new FLIP-49 memory configurations

2019-12-23 Thread Andrey Zagrebin
t;>>> config option keys.
>>>>>
>>>>>- It's not necessary IMO to expose the difference concepts of
>>>>>direct / native memory to the users.
>>>>>- I would avoid introducing more options for native memory if
>>>>>possible. Taking fine grained resource management and dynamic slot
>>>>>allocation into consideration, that also means introduce more fields 
>>>>> into
>>>>>ResourceSpec / ResourceProfile.
>>>>>- My gut feeling is that having a relative loose MaxDirectMemory
>>>>>should not be a big problem.
>>>>>- In most cases, the task / framework off-heap memory should be
>>>>>   mainly (if not all) direct memory, so the difference between derived
>>>>>   MaxDirectMemory and the ideal direct memory limit should not be too 
>>>>> much.
>>>>>   - We do not have a good way to know the exact size needed for
>>>>>   jvm overhead / metaspace and framework / task off-heap memory, thus 
>>>>> having
>>>>>   to conservatively reserve slightly more memory then what actually 
>>>>> needed.
>>>>>   Such reserved but not used memory can cover for the small 
>>>>> MaxDirectMemory
>>>>>   error.
>>>>>   -
>>>>>   - MaxDirectMemory is not the only way to trigger full gc. We
>>>>>   still heap activities that can also trigger the gc.
>>>>>
>>>>> Regarding the memory type config options, I've looked into the latest
>>>>> ConfigOptions changes. I think it shouldn't be too complicated to change
>>>>> the config options to use memory type, and I can handle it maybe during
>>>>> your vacations.
>>>>>
>>>>>
>>>>> Also agree with improving MemorySize logging and parsing. This should
>>>>> not be a blocker that has to be done in 1.10. I would say we finish other
>>>>> works (testability, documentation and those discussed in this thread)
>>>>> first, and get to this only if we have time.
>>>>>
>>>>>
>>>>> Thank you~
>>>>>
>>>>> Xintong Song
>>>>>
>>>>>
>>>>>
>>>>> On Fri, Dec 20, 2019 at 6:07 PM Andrey Zagrebin <
>>>>> azagrebin.apa...@gmail.com> wrote:
>>>>>
>>>>>> Hi Stephan and Xintong,
>>>>>>
>>>>>> Thanks for the further FLIP-49 feedbacks.
>>>>>>
>>>>>>   - "taskmanager.memory.size" (old main config option) is replaced by
>>>>>>> "taskmanager.memory.total-process.size" which has a different meaning in
>>>>>>> standalone setups. The old option did not subtract metaspace and other
>>>>>>> overhead, while the new option does. That means that with the default
>>>>>>> config, standalone clusters get quite a bit less memory. (independent of
>>>>>>> managed memory going off heap).
>>>>>>> I am wondering if we could interpret "taskmanager.memory.size"
>>>>>>> as the deprecated key for "taskmanager.memory.total-flink.size". That 
>>>>>>> would
>>>>>>> be in line with the old mechanism (assuming managed memory is set to off
>>>>>>> heap).
>>>>>>> The effect would be that the container size on Yarn/Mesos
>>>>>>> increases, because from "taskmanager.memory.total-flink.size", we need 
>>>>>>> to
>>>>>>> add overhead and metaspace to reach the total process size, rather than
>>>>>>> cutting off memory. But if we want, we could even adjust for that in the
>>>>>>> active resource manager, getting full backwards compatibility on that 
>>>>>>> part.
>>>>>>> Curious to hear more thoughts there.
>>>>>>
>>>>>>
>>>>>> I believe you mean "taskmanager.heap.size".
>>>>>>
>>>>>> I think the problem here is that the legacy "taskmanager.heap.size"
>>>>>> was used differently in standalone setups and active yarn / mesos setups,
>>>>>> and such different calculation logics and behaviors are exactly 

Re: [DISCUSS] Some feedback after trying out the new FLIP-49 memory configurations

2019-12-24 Thread Andrey Zagrebin
ry.process.size"
>   - Deprecated "taskmanager.heap.size"
>- What is proper keys for network / shuffle memory
>   - "taskmanager.memory.shuffle.*"
>   - "taskmanager.memory.network.*"
>   - "taskmanager.network.memory.*"
>
>
> Thank you~
>
> Xintong Song
>
>
>
> On Tue, Dec 24, 2019 at 10:19 AM Xintong Song 
> wrote:
>
>> How about putting "taskmanager.memory.flink.size" in the configuration?
>>> Then new downloaded Flink behaves similar to the previous Standalone setups.
>>> If someone upgrades the binaries, but re-uses their old configuration,
>>> then they get the compatibility as discussed previously.
>>> We used that approach previously with the fine-grained failover recovery.
>>
>>
>> I'm trying to understand why "taskmanager.memory.flink.size" rather than
>> "taskmanager.memory.process.size" in the default flink-conf.yaml. Or put it
>> another way, why do we want the new downloaded Flink behaves similar to
>> previous Standalone setups rather than previous active mode setups? Is
>> there any special reason that I overlooked, which makes backwards
>> compatibility for standalone setups more important than for active setups?
>>
>> IMO, "taskmanager.memory.process.size" is easier for the new comers. For
>> standalone setups, users can simply configure it to their machines'
>> available memory size, without needing to worry about leaving enough space
>> for JVM overehead / metaspace. For containerized setups, it's more
>> predictable how many memory the containers / Flink could use, which is more
>> friendly for users to manage their resource quota.
>>
>> Therefore, unless there is anything I overlooked, I'm in favor of putting
>> "taskmanager.memory.process.size" rather than
>> "taskmanager.memory.flink.size" in the default configuration.
>>
>>
>> Thank you~
>>
>> Xintong Song
>>
>>
>>
>> On Tue, Dec 24, 2019 at 4:27 AM Andrey Zagrebin 
>> wrote:
>>
>>> How about putting "taskmanager.memory.flink.size" in the configuration?
>>>> Then new downloaded Flink behaves similar to the previous Standalone 
>>>> setups.
>>>> If someone upgrades the binaries, but re-uses their old configuration,
>>>> then they get the compatibility as discussed previously.
>>>> We used that approach previously with the fine-grained failover
>>>> recovery.
>>>
>>> +1, this sounds like a good compromise.
>>>
>>> +1 to not have more options for off-heap memory, that would get
>>>> confusing fast. We can keep the name "off-heap" for both task and
>>>> framework, as long as they mean the same thing: native plus direct, and
>>>> fully counted into MaxDirectMemory. I would suggest to update the config
>>>> descriptions then to reflect that.
>>>>
>>> True, this should be explained in the config descriptions.
>>>
>>> looks good to me
>>>
>>> From a user's perspective I believe "taskmanager.memory.network" would
>>>> be easier to understand as not everyone knows exactly what the shuffle
>>>> service is. I see the point that it would be a bit imprecise as we can have
>>>> different shuffle implementations but I would go with the ease of
>>>> use/understanding here. Moreover, I think that we won't have many different
>>>> shuffle service implementations in the foreseeable future.
>>>
>>> I agree that if we cannot find any other convincing names for the
>>> options, we should keep what we already have and change it if the
>>> alternative is convincing enough.
>>> The question is also whether we still want to rename it because it was
>>> "taskmanager.network.*memory*.*" in 1.9 but "taskmanager.*memory*.network.*"
>>> is more aligned with other new memory option names.
>>> Or we can just 'un'-deprecate "taskmanager.network.*memory*.*".
>>>
>>> On Mon, Dec 23, 2019 at 8:42 PM Stephan Ewen  wrote:
>>>
>>>> How about putting "taskmanager.memory.flink.size" in the configuration?
>>>> Then new downloaded Flink behaves similar to the previous Standalone 
>>>> setups.
>>>>
>>>> If someone upgrades the binaries, but re-uses their old configuration,
>>>> then they get the compatibility as discussed previously.
>&g

Re: [DISCUSS] Some feedback after trying out the new FLIP-49 memory configurations

2020-01-06 Thread Andrey Zagrebin
Thank you for more explanation Stephan and feedback Jingsong,

I see the point now.
‘taskmanager.memory.flink.size’ is indeed simpler to understand for the
newcomers in the default config because it includes less types of memory to
consider.
Most of the time beginners will think about heap size and maybe state size
to tweak in the try-out jobs.
‘taskmanager.memory.flink.size’ is better scoped for those types of memory.

The process memory forces to understand the total memory consumption
which is more important for the next steps and will probably require
reading the docs in more depth anyways.
I agree if we decide for ‘flink.size’, it is worth mentioning a pointer to
‘process.size' in its comment as an alternative to it.

I am ok with ‘flink.size’ in the default config.

Best,
Andrey

On Mon, Dec 30, 2019 at 5:13 AM Jingsong Li  wrote:

> Thank you for your wonderful discussion.
>
> +1 for set "taskmanager.memory.flink.size" in the default config.
> Maybe we can write and explain "taskmanager.memory.process.size" in the
> comments.
>
> Most of the time,
> - "trying out" users is less in-depth users and use standalone mode.
> - Production users use active setups.
>
> Default config it very important to "trying out".
> In "trying out", for a novice Java user of Flink, he want to configure as
> much memory as my standalone process should be. In fact, it's hard for him
> to realize the JVM overhead. And there is no need that must let him know.
>
> > I don't think it is realistic that users set the process memory to full
> machine memory. There is a lot on the machine as well in most cases.
>
> +1, We often run Flink in a less clean environment, such as the environment
> have HDFS. Whether we are testing or producing, we will not use all the
> memory of the machine, such as always leaving some memory for PageCache.
>
> Best,
> Jingsong Lee
>
> On Sat, Dec 28, 2019 at 7:02 PM Stephan Ewen  wrote:
>
> > "taskmanager.memory.flink.size" in the default config has a few
> advantages.
> >
> >  - The value in the default config needs to be suitable for "trying out"
> > Flink, for a good "getting started" experience.
> >
> >   - For trying out Flink, standalone is the most common entry point
> (except
> > running in IDE).
> >
> >   - In standalone setup, from total process memory, we subtract quite a
> bit
> > before we arrive at the usable memory. We also subtract managed memory
> from
> > the heap now. I fear we might end up at a heap that becomes so small that
> > it makes for a bad "getting started" experience.
> >
> >   - I don't think it is realistic that users set the process memory to
> full
> > machine memory. There is a lot on the machine as well in most cases.
> >
> >   - In the JVM world, users are used to configuring the heap size and
> know
> > that there is additional memory overhead. The
> > "taskmanager.memory.flink.size" option fits well with that mindset.
> >
> >   - One you start to think about the total process memory of Yarn
> > containers, you are already past the getting-started phase and on the
> > tuning phase.
> >
> >
> > On Tue, Dec 24, 2019, 10:25 Andrey Zagrebin 
> wrote:
> >
> > > Thanks for the summary, Xintong! It makes sense to me.
> > >
> > > How about putting "taskmanager.memory.flink.size" in the configuration?
> > > > Then new downloaded Flink behaves similar to the previous Standalone
> > > setups.
> > > > If someone upgrades the binaries, but re-uses their old
> configuration,
> > > > then they get the compatibility as discussed previously.
> > > > We used that approach previously with the fine-grained failover
> > recovery.
> > >
> > >
> > >
> > > > I'm trying to understand why "taskmanager.memory.flink.size" rather
> > than
> > > > "taskmanager.memory.process.size" in the default flink-conf.yaml. Or
> > put
> > > it
> > > > another way, why do we want the new downloaded Flink behaves similar
> to
> > > > previous Standalone setups rather than previous active mode setups?
> Is
> > > > there any special reason that I overlooked, which makes backwards
> > > > compatibility for standalone setups more important than for active
> > > setups?
> > > > IMO, "taskmanager.memory.process.size" is easier for the new comers.
> > For
> > > > standalone setups, users can simply configure it to their machines'

Re: [DISCUSS] FLIP-53: Fine Grained Resource Management

2020-01-07 Thread Andrey Zagrebin
Hi Michaël,

Thanks for mentioning this, users can benefit from this for sure.

This API is not exposed (although there is some code prepared for this)
because community believes that it would require more extensive design and
discussion before the implementation, basically another FLIP.

Although this does not look as a starter task to me,
if you have time you could start drafting the design and open it for the
discussion but
I would advise to firstly find a committer to help with this effort.
At the moment, tbh, I do not have suggestion about who could mentor this
for the next release cycle,
but maybe somebody volunteers if you ask here in the mailing list.

Best,
Andrey

On Tue, Jan 7, 2020 at 2:25 PM Michaël Melchiore  wrote:

> Hello,
>
> From the scope section of this FLIP, I understand no API is currently
> available to describe resource requirements for DataStream API. Since I
> need
> it for my use case, it seems to be a good opportunity for me to start
> contributing to Flink.
>
> Is this task suitable for a new comer ? I am an experienced Java developer.
>
> Regards,
>
> Michaël
>
>
>
> --
> Sent from: http://apache-flink-mailing-list-archive.1008284.n3.nabble.com/
>


Re: [DISCUSS] Some feedback after trying out the new FLIP-49 memory configurations

2020-01-08 Thread Andrey Zagrebin
It also looks to me that we should only swap network and memory in the
option names: 'taskmanager.memory.network.*'.
There is no strong consensus towards using new 'shuffle' naming so we can
just rename it to  'taskmanager.memory.network.*' as 'shuffle' naming has
never been released.
When we have other shuffle services and start advertising more this concept
in Flink, we could revisit again the whole naming for this concept.
https://jira.apache.org/jira/browse/FLINK-15517


Re: [DISCUSS] Some feedback after trying out the new FLIP-49 memory configurations

2020-01-13 Thread Andrey Zagrebin
Hi all,

While working on changing process memory to Flink memory in default
configuration, Xintong encountered a problem.
When -tm option is used to rewrite container memory, basically process
memory, it can collide with the default Flink memory.
For legacy users it should not be a problem as we adjusted the legacy heap
size option to be interpreted differently for standalone and container
modes.

One solution could be to say in -tm docs that we rewrite both options under
the hood: process and Flink memory, basically unset Flink memory from yaml
config.
The downside is that this adds more magic.

Alternatively, we can keep process memory in default config and, as
mentioned before, increase it to maintain the user experience by matching
the previous default setting for heap (now Flink in standalone) size.
The Flink memory can be mentioned in process memory comment as a simpler
alternative which does not require accounting for JVM overhead.
The downside is again more confusion while trying out Flink and tuning
memory at the same time.
On the other hand, if memory already needs to be tuned it should
quite quickly lead to the necessity of understanding the memory model in
Flink.

Best,
Andrey

On Thu, Jan 9, 2020 at 12:27 PM Stephan Ewen  wrote:

> Great! Thanks, guys, for the continued effort on this topic!
>
> On Thu, Jan 9, 2020 at 5:27 AM Xintong Song  wrote:
>
> > Thanks all for the discussion. I believe we have get consensus on all the
> > open questions discussed in this thread.
> >
> > Since Andrey already create a jira ticket for renaming shuffle memory
> > config keys with "taskmanager.memory.network.*", I'll create ticket for
> the
> > other topic that puts flink.size in flink-conf.yaml.
> >
> > Thank you~
> >
> > Xintong Song
> >
> >
> >
> > On Wed, Jan 8, 2020 at 9:39 PM Andrey Zagrebin 
> > wrote:
> >
> > > It also looks to me that we should only swap network and memory in the
> > > option names: 'taskmanager.memory.network.*'.
> > > There is no strong consensus towards using new 'shuffle' naming so we
> can
> > > just rename it to  'taskmanager.memory.network.*' as 'shuffle' naming
> has
> > > never been released.
> > > When we have other shuffle services and start advertising more this
> > concept
> > > in Flink, we could revisit again the whole naming for this concept.
> > > https://jira.apache.org/jira/browse/FLINK-15517
> > >
> >
>


Re: [Discuss] Tuning FLIP-49 configuration default values.

2020-01-14 Thread Andrey Zagrebin
Hi all!

Great that we have already tried out new FLIP-49 with the bigger jobs.

I am also +1 for the JVM metaspace and overhead changes.

Regarding 0.3 vs 0.4 for managed memory, +1 for having more managed memory
for Rocksdb limiting case.

In general, this looks mostly to be about memory distribution between JVM
heap and managed off-heap.
Comparing to the previous default setup, the JVM heap dropped (especially
for standalone) mostly due to moving managed from heap to off-heap and then
also adding framework off-heap.
In general, this can be the most important consequence for beginners and
those who rely on the default configuration.
Especially the legacy default configuration in standalone with falling back
heap.size to flink.size but there it seems we cannot do too much now.

I prepared a spreadsheet

to play with numbers for the mentioned in the report setups.

One idea would be to set process size (or smaller flink size respectively)
to a bigger default number, like 2048.
In this case, the abs derived default JVM heap and managed memory are close
to the previous defaults, especially for managed fraction 0.3.
This should align the defaults with the previous standalone try-out
experience where the increased off-heap memory is not strictly controlled
by the environment anyways.
The consequence for container users who relied on and updated the default
configuration is that the containers will be requested with the double size.

Best,
Andrey


On Tue, Jan 14, 2020 at 11:20 AM Till Rohrmann  wrote:

> +1 for the JVM metaspace and overhead changes.
>
> On Tue, Jan 14, 2020 at 11:19 AM Till Rohrmann 
> wrote:
>
>> I guess one of the most important results of this experiment is to have a
>> good tuning guide available for users who are past the initial try-out
>> phase because the default settings will be kind of a compromise. I assume
>> that this is part of the outstanding FLIP-49 documentation task.
>>
>> If we limit RocksDB's memory consumption by default, then I believe that
>> 0.4 would give the better all-round experience as it leaves a bit more
>> memory for RocksDB. However, I'm a bit sceptical whether we should optimize
>> the default settings for a configuration where the user still needs to
>> activate the strict memory limiting for RocksDB. In this case, I would
>> expect that the user could also adapt the managed memory fraction.
>>
>> Cheers,
>> Till
>>
>> On Tue, Jan 14, 2020 at 3:39 AM Xintong Song 
>> wrote:
>>
>>> Thanks for the feedback, Stephan and Kurt.
>>>
>>> @Stephan
>>>
>>> Regarding managed memory fraction,
>>> - It makes sense to keep the default value 0.4, if we assume rocksdb
>>> memory is limited by default.
>>> - AFAIK, currently rocksdb by default does not limit its memory usage.
>>> And I'm positive to change it.
>>> - Personally, I don't like the idea that we the out-of-box experience
>>> (for which we set the default fraction) relies on that users will manually
>>> turn another switch on.
>>>
>>> Regarding framework heap memory,
>>> - The major reason we set it by default is, as you mentioned, that to
>>> have a safe net of minimal JVM heap size.
>>> - Also, considering the in progress FLIP-56 (dynamic slot allocation),
>>> we want to reserve some heap memory that will not go into the slot
>>> profiles. That's why we decide the default value according to the heap
>>> memory usage of an empty task executor.
>>>
>>> @Kurt
>>> Regarding metaspace,
>>> - This config option ("taskmanager.memory.jvm-metaspace") only takes
>>> effect on TMs. Currently we do not set metaspace size for JM.
>>> - If we have the same metaspace problem on TMs, then yes, changing it
>>> from 128M to 64M will make it worse. However, IMO 10T tpc-ds benchmark
>>> should not be considered as out-of-box experience and it makes sense to
>>> tune the configurations for it. I think the smaller metaspace size would be
>>> a better choice for the first trying-out, where a job should not be too
>>> complicated, the TM size could be relative small (e.g. 1g).
>>>
>>> Thank you~
>>>
>>> Xintong Song
>>>
>>>
>>>
>>> On Tue, Jan 14, 2020 at 9:38 AM Kurt Young  wrote:
>>>
 HI Xingtong,

 IIRC during our tpc-ds 10T benchmark, we have suffered by JM's
 metaspace size and full gc which
 caused by lots of classloadings of source input split. Could you check
 whether changing the default
 value from 128MB to 64MB will make it worse?

 Correct me if I misunderstood anything, also cc @Jingsong

 Best,
 Kurt


 On Tue, Jan 14, 2020 at 3:44 AM Stephan Ewen  wrote:

> Hi all!
>
> Thanks a lot, Xintong, for this thorough analysis. Based on your
> analysis,
> here are some thoughts:
>
> +1 to change default JVM metaspace size from 128MB to 64MB
> +1 to change default JVM overhead min size from 128MB to 196MB
>
> Concerning the managed memory fract

Re: [DISCUSS] Some feedback after trying out the new FLIP-49 memory configurations

2020-01-14 Thread Andrey Zagrebin
Hi all,

Stephan, Till and me had another offline discussion today. Here is the
outcome of our brainstorm.

We agreed to have process.size in the default settings with the explanation
of flink.size alternative in the comment.
This way we keep -tm as a shortcut to process.size only and any
inconsistencies fail fast as if configured in yaml.
I will also follow-up on the thread "[Discuss] Tuning FLIP-49 configuration
default values" with a bit more details.

If no further objections, we can conclude this last point in this
discussion.

Best,
Andrey

On Tue, Jan 14, 2020 at 4:28 PM Stephan Ewen  wrote:

> I think that problem exists anyways and is independent of the "-tm" option.
>
> You can have a combination of `task.heap.size` and `managed.size` and
> `framework.heap.size` that conflicts with `flink.size`. In that case, we
> get an exception during the startup (incompatible memory configuration)?
> That is the price for having these "shortcut" options (`flink.size` and
> `process.size`). But it is a fair price, because the shortcuts are very
> valuable to most users.
>
> That is added with the "-tm" setting is that this is an easy way to get the
> shortcut setting added, even if it was not in the config. So where a config
> worked in standalone, it can now fail in a container setup when "-tm" is
> used.
> But that is expected, I guess, when you start manually tune different
> memory types and end up defining an inconsistent combination. And it never
> conflicts with the default setup, only with manually tuned regions.
>
> But this example shows why we need to have log output for the logic that
> validates and configures the memory settings, so that users understand what
> is happening.
>
> Best,
> Stephan
>
>
> On Tue, Jan 14, 2020 at 2:54 PM Till Rohrmann 
> wrote:
>
> > Clearing the `flink.size` option and setting `process.size` could indeed
> be
> > a solution. The thing I'm wondering is what would happen if the user has
> > configured `task.heap.size` and `managed.size` instead of `flink.size`?
> > Would we also ignore these settings? If not, then we risk to run into the
> > situation that TaskExecutorResourceUtils fails because the memory does
> not
> > add up. I guess the thing which bugs me a bit is the special casing which
> > could lead easily into inconsistent behaviour if we don't cover all
> cases.
> > The consequence of using slightly different concepts (`flink.size`,
> > `process.size`) in standalone vs. container/Yarn/Mesos mode in order to
> > keep the status quo is an increased maintenance overhead which we should
> be
> > aware of.
> >
> > Cheers,
> > Till
> >
> > On Tue, Jan 14, 2020 at 3:59 AM Xintong Song 
> > wrote:
> >
> > > True, even we have "process.size" rather than "flink.size" in the
> default
> > > config file, user can still have "flink.size" in their custom config
> > file.
> > > If we consider "-tm" as a shortcut for users to override the TM memory
> > > size, then it makes less sense to require users to remove "flink.size"
> > from
> > > their config file whenever then want to use "-tm".
> > >
> > > I'm convinced that ignoring "flink.size" might not be a bad idea.
> > > And I think we should also update the description of "-tm" (in
> > > "FlinkYarnSessionCli") to explicitly mention that it would overwrite
> > > "flink.size" and "process.size".
> > >
> > > Thank you~
> > >
> > > Xintong Song
> > >
> > >
> > >
> > > On Tue, Jan 14, 2020 at 2:24 AM Stephan Ewen  wrote:
> > >
> > > > Would be good to hear the thoughts of some more Yarn users, though.
> > > >
> > > > On Mon, Jan 13, 2020 at 7:23 PM Stephan Ewen 
> wrote:
> > > >
> > > > > I think we need an interpretation of "-tm" regardless of what is in
> > the
> > > > > default configuration, because we can always have a modified
> > > > configuration
> > > > > and then a user passes the "-tm" flag.
> > > > >
> > > > > I kind of like the first proposal: Interpret "-tm" as "override
> > memory
> > > > > size config and set the Yarn TM container size." It would mean
> > exactly
> > > > > ignoring "taskmanager.memory.flink.size" and using the "-tm" value
> > as "
> > > > > "taskmanager.memory

Re: [Discuss] Tuning FLIP-49 configuration default values.

2020-01-14 Thread Andrey Zagrebin
Hi all,

Stephan, Till and me had another offline discussion today. Here is the
outcome of our brainstorm.

*managed fraction 0.4*
just confirmed what we already discussed here.

*process.size = 1536Mb (1,5Gb)*
We agreed to have process.size in the default settings with the explanation
of flink.size alternative in the comment.
The suggestion is to increase it from 1024 to 1536mb. As you can see in the
earlier provided calculation spreadsheet,
it will result in bigger JVM Heap and managed memory (both ~0.5Gb) for all
new setups.
This should provide good enough experience for trying out Flink.

*JVM overhead min 196 -> 192Mb (128 + 64)*
small correction for better power 2 alignment of sizes

*meta space at least 96Mb?*
There is still a concern about JVM metaspace being just 64Mb.
We should confirm that it is not a problem by trying to test it also with
the SQL jobs, Blink planner.
Also, by running tpc-ds e2e Flink tests with this setting. Basically, where
more classes are generated/loaded.
We can look into this tomorrow.

*sanity check of JVM overhead*
When the explicitly configured process and flink memory sizes are verified
with the JVM meta space and overhead,
JVM overhead does not have to be the exact fraction.
It can be just within its min/max range, similar to how it is now for
network/shuffle memory check after FLINK-15300.

Best,Andrey

On Tue, Jan 14, 2020 at 4:30 PM Stephan Ewen  wrote:

> I like the idea of having a larger default "flink.size" in the config.yaml.
> Maybe we don't need to double it, but something like 1280m would be okay?
>
> On Tue, Jan 14, 2020 at 3:47 PM Andrey Zagrebin 
> wrote:
>
> > Hi all!
> >
> > Great that we have already tried out new FLIP-49 with the bigger jobs.
> >
> > I am also +1 for the JVM metaspace and overhead changes.
> >
> > Regarding 0.3 vs 0.4 for managed memory, +1 for having more managed
> memory
> > for Rocksdb limiting case.
> >
> > In general, this looks mostly to be about memory distribution between JVM
> > heap and managed off-heap.
> > Comparing to the previous default setup, the JVM heap dropped (especially
> > for standalone) mostly due to moving managed from heap to off-heap and
> then
> > also adding framework off-heap.
> > In general, this can be the most important consequence for beginners and
> > those who rely on the default configuration.
> > Especially the legacy default configuration in standalone with falling
> back
> > heap.size to flink.size but there it seems we cannot do too much now.
> >
> > I prepared a spreadsheet
> > <
> >
> https://docs.google.com/spreadsheets/d/1mJaMkMPfDJJ-w6nMXALYmTc4XxiV30P5U7DzgwLkSoE
> > >
> > to play with numbers for the mentioned in the report setups.
> >
> > One idea would be to set process size (or smaller flink size
> respectively)
> > to a bigger default number, like 2048.
> > In this case, the abs derived default JVM heap and managed memory are
> close
> > to the previous defaults, especially for managed fraction 0.3.
> > This should align the defaults with the previous standalone try-out
> > experience where the increased off-heap memory is not strictly controlled
> > by the environment anyways.
> > The consequence for container users who relied on and updated the default
> > configuration is that the containers will be requested with the double
> > size.
> >
> > Best,
> > Andrey
> >
> >
> > On Tue, Jan 14, 2020 at 11:20 AM Till Rohrmann 
> > wrote:
> >
> > > +1 for the JVM metaspace and overhead changes.
> > >
> > > On Tue, Jan 14, 2020 at 11:19 AM Till Rohrmann 
> > > wrote:
> > >
> > >> I guess one of the most important results of this experiment is to
> have
> > a
> > >> good tuning guide available for users who are past the initial try-out
> > >> phase because the default settings will be kind of a compromise. I
> > assume
> > >> that this is part of the outstanding FLIP-49 documentation task.
> > >>
> > >> If we limit RocksDB's memory consumption by default, then I believe
> that
> > >> 0.4 would give the better all-round experience as it leaves a bit more
> > >> memory for RocksDB. However, I'm a bit sceptical whether we should
> > optimize
> > >> the default settings for a configuration where the user still needs to
> > >> activate the strict memory limiting for RocksDB. In this case, I would
> > >> expect that the user could also adapt the managed memory fraction.
> > >>
> > >> Cheers,
> > >> Till
> > >>
> > 

Re: [DISCUSS] decentralized scheduling strategy is needed

2020-01-15 Thread Andrey Zagrebin
HI HuWeihua,

I think your issue should resolve with 1.9.2 and 1.10 (not released but in
progress).
You can check the related Jira ticket [1].

Best,
Andrey

[1] https://jira.apache.org/jira/browse/FLINK-12122

On Wed, Jan 15, 2020 at 10:08 AM HuWeihua  wrote:

> Hi, All
> We encountered some problems during the upgrade from Flink 1.5 to Flink
> 1.9. Flink's scheduling strategy has changed. Flink 1.9 prefers centralized
> scheduling, while Flink 1.5 prefers decentralized scheduling. This change
> has caused resources imbalance and blocked our upgrade plan. We have
> thousands of jobs that need to be upgraded.
>
> For example,
> There is a job with 10 sources and 100 sinks. Each source need 1 core and
> each sink need 0.1 core.
> Try to run this job on Yarn, configure the numberOfTaskSlots is 10,
> yarn.containers.vcores is 2.
>
> When using Flink-1.5:
> Each TaskManager will run 1 source and 9 sinks, they need 1.9 cores
> totally. So the job with this configuration works very well. The schedule
> results is shown in Figure 1.
> When using Flink-1.9:
> The 10 sources will be scheduled to one TaskManager  and the 100 sinks
> will scheduled to other 10 TaskManagers.  The schedule results is shown
> in Figure 2.
> In this scenario, the TaskManager which run sources need 10 cores, other
> TaskManagers need 1 cores. But TaskManager must be configured the same, So
> we need 11 TaskManager with 10 cores.
> This situation waste (10-2)*11 = 88 cores more than Flink 1.5.
>
> In addition to the waste of resources, we also encountered other problems
> caused by centralized scheduling strategy.
>
>1. Network bandwidth. Tasks of the same type are scheduled to the one
>TaskManager, causing too much network traffic on the machine.
>
>
>1. Some jobs need to sink to the local agent. After centralized
>scheduling, the insufficient processing capacity of the single machine
>causes a backlog of consumption.
>
>
> In summary, we think a decentralized scheduling strategy is necessary.
>
>
> Figure 1. Flink 1.5 schedule results
>
> Figure 2. Flink 1.9 schedule results
>
>
>
> Best
> Weihua Hu
>
>


Re: [DISCUSS] Change default for RocksDB timers: Java Heap => in RocksDB

2020-01-16 Thread Andrey Zagrebin
Hi Stephan,

Thanks for starting this discussion. I am +1 for this change.
In general, number of timer state keys can have the same order as number of
main state keys.
So if RocksDB is used for main state for scalability, it makes sense to
have timers there as well
unless timers are used for only very limited subset of keys which fits into
memory.

Best,
Andrey

On Thu, Jan 16, 2020 at 4:27 PM Stephan Ewen  wrote:

> Hi all!
>
> I would suggest a change of the current default for timers. A bit of
> background:
>
>   - Timers (for windows, process functions, etc.) are state that is
> managed and checkpointed as well.
>   - When using the MemoryStateBackend and the FsStateBackend, timers are
> kept on the JVM heap, like regular state.
>   - When using the RocksDBStateBackend, timers can be kept in RocksDB
> (like other state) or on the JVM heap. The JVM heap is the default though!
>
> I find this a bit un-intuitive and would propose to change this to let the
> RocksDBStateBackend store all state in RocksDB by default.
> The rationale being that if there is a tradeoff (like here), safe and
> scalable should be the default and unsafe performance be an explicit choice.
>
> This sentiment seems to be shared by various users as well, see
> https://twitter.com/StephanEwen/status/1214590846168903680 and
> https://twitter.com/StephanEwen/status/1214594273565388801
> We would of course keep the switch and mention in the performance tuning
> section that this is an option.
>
> # RocksDB State Backend Timers on Heap
>   - Pro: faster
>   - Con: not memory safe, GC overhead, longer synchronous checkpoint time,
> no incremental checkpoints
>
> #  RocksDB State Backend Timers on in RocksDB
>   - Pro: safe and scalable, asynchronously and incrementally checkpointed
>   - Con: performance overhead.
>
> Please chime in and let me know what you think.
>
> Best,
> Stephan
>
>


Re: [VOTE] Release 1.10.0, release candidate #2

2020-02-06 Thread Andrey Zagrebin
Hi Benchao,

Do you observe this issue FLINK-15938 with 1.9 or 1.10?
If with 1.9, I suggest to check with 1.10.

Thanks,
Andrey

On Thu, Feb 6, 2020 at 4:07 PM Benchao Li  wrote:

> Hi all,
>
> I found another issue[1], I don't know if it should be a blocker. But it
> does affects joins without window in blink planner.
>
> [1] https://issues.apache.org/jira/browse/FLINK-15938
>
> Jeff Zhang  于2020年2月6日周四 下午5:05写道:
>
> > Hi Jingsong,
> >
> > Thanks for the suggestion. It works for running it in IDE, but for
> > downstream project like Zeppelin where I will include flink jars in
> > classpath.
> > it only works when I specify the jars one by one explicitly in classpath,
> > using * doesn't work.
> >
> > e.g.
> >
> > The following command where I use * to specify classpath doesn't work,
> > jzhang 4794 1.2 6.3 8408904 1048828 s007 S 4:30PM 0:33.90
> > /Library/Java/JavaVirtualMachines/jdk1.8.0_211.jdk/Contents/Home/bin/java
> > -Dfile.encoding=UTF-8
> >
> >
> -Dlog4j.configuration=file:///Users/jzhang/github/zeppelin/conf/log4j.properties
> >
> >
> -Dzeppelin.log.file=/Users/jzhang/github/zeppelin/logs/zeppelin-interpreter-flink-shared_process-jzhang-JeffdeMacBook-Pro.local.log
> > -Xms1024m -Xmx2048m -XX:MaxPermSize=512m -cp
> >
> >
> *:/Users/jzhang/github/flink/build-target/lib/**:/Users/jzhang/github/zeppelin/interpreter/flink/*:/Users/jzhang/github/zeppelin/zeppelin-interpreter-api/target/*:/Users/jzhang/github/zeppelin/zeppelin-zengine/target/test-classes/*::/Users/jzhang/github/zeppelin/interpreter/zeppelin-interpreter-api-0.9.0-SNAPSHOT.jar:/Users/jzhang/github/zeppelin/zeppelin-interpreter/target/classes:/Users/jzhang/github/zeppelin/zeppelin-zengine/target/test-classes:/Users/jzhang/github/flink/build-target/opt/flink-python_2.11-1.10-SNAPSHOT.jar
> > org.apache.zeppelin.interpreter.remote.RemoteInterpreterServer 0.0.0.0
> > 52395 flink-shared_process :
> >
> >
> > While this command where I specify jar one by one explicitly in classpath
> > works
> >
> > jzhang5660 205.2  6.1  8399420 1015036 s005  R 4:43PM
> > 0:24.82
> > /Library/Java/JavaVirtualMachines/jdk1.8.0_211.jdk/Contents/Home/bin/java
> > -Dfile.encoding=UTF-8
> >
> >
> -Dlog4j.configuration=file:///Users/jzhang/github/zeppelin/conf/log4j.properties
> >
> >
> -Dzeppelin.log.file=/Users/jzhang/github/zeppelin/logs/zeppelin-interpreter-flink-shared_process-jzhang-JeffdeMacBook-Pro.local.log
> > -Xms1024m -Xmx2048m -XX:MaxPermSize=512m -cp
> >
> >
> *:/Users/jzhang/github/flink/build-target/lib/slf4j-log4j12-1.7.15.jar:/Users/jzhang/github/flink/build-target/lib/flink-connector-hive_2.11-1.10-SNAPSHOT.jar:/Users/jzhang/github/flink/build-target/lib/hive-exec-2.3.4.jar:/Users/jzhang/github/flink/build-target/lib/flink-shaded-hadoop-2-uber-2.7.5-7.0.jar:/Users/jzhang/github/flink/build-target/lib/log4j-1.2.17.jar:/Users/jzhang/github/flink/build-target/lib/flink-table-blink_2.11-1.10-SNAPSHOT.jar:/Users/jzhang/github/flink/build-target/lib/flink-table_2.11-1.10-SNAPSHOT.jar:/Users/jzhang/github/flink/build-target/lib/flink-dist_2.11-1.10-SNAPSHOT.jar:/Users/jzhang/github/flink/build-target/lib/flink-python_2.11-1.10-SNAPSHOT.jar:/Users/jzhang/github/flink/build-target/lib/flink-hadoop-compatibility_2.11-1.9.1.jar*:/Users/jzhang/github/zeppelin/interpreter/flink/*:/Users/jzhang/github/zeppelin/zeppelin-interpreter-api/target/*:/Users/jzhang/github/zeppelin/zeppelin-zengine/target/test-classes/*::/Users/jzhang/github/zeppelin/interpreter/zeppelin-interpreter-api-0.9.0-SNAPSHOT.jar:/Users/jzhang/github/zeppelin/zeppelin-interpreter/target/classes:/Users/jzhang/github/zeppelin/zeppelin-zengine/target/test-classes:/Users/jzhang/github/flink/build-target/opt/flink-python_2.11-1.10-SNAPSHOT.jar
> >
> >
> /Users/jzhang/github/flink/build-target/opt/tmp/flink-python_2.11-1.10-SNAPSHOT.jar
> > org.apache.zeppelin.interpreter.remote.RemoteInterpreterServer 0.0.0.0
> > 52603 flink-shared_process :
> >
> >
> > Jingsong Li  于2020年2月6日周四 下午4:10写道:
> >
> > > Hi Jeff,
> > >
> > >
> > > For FLINK-15935 [1],
> > >
> > > I try to think of it as a non blocker. But it's really an important
> > issue.
> > >
> > >
> > > The problem is the class loading order. We want to load the class in
> the
> > > blink-planner.jar, but actually load the class in the
> flink-planner.jar.
> > >
> > >
> > > First of all, the order of class loading is based on the order of
> > > classpath.
> > >
> > >
> > > I just tried, the order of classpath of the folder is depends on the
> > order
> > > of file names.
> > >
> > > -That is to say, our order is OK now: because
> > > flink-table-blink_2.11-1.11-SNAPSHOT.jar is before the name order of
> > > flink-table_2.11-1.11-snapshot.jar.
> > >
> > > -But once I change the name of flink-table_2.11-1.11-SNAPSHOT.jar to
> > > aflink-table_2.11-1.11-SNAPSHOT.jar, an error will be reported.
> > >
> > >
> > > The order of classpaths should be influenced by the ls of Linux. By
> > default
> > > the ls command is listing the files in al

Re: [VOTE] Release 1.10.0, release candidate #2

2020-02-06 Thread Andrey Zagrebin
alright, thanks for confirming this Benchao!

On Thu, Feb 6, 2020 at 6:36 PM Benchao Li  wrote:

> Hi Andrey,
>
> I noticed that 1.10 has changed to enabling background cleanup by default
> just after I posted to this email.
> So it won't affect 1.10 any more, just affect 1.9.x. We can move to the
> Jira ticket to discuss further more.
>
> Andrey Zagrebin  于2020年2月6日周四 下午11:30写道:
>
> > Hi Benchao,
> >
> > Do you observe this issue FLINK-15938 with 1.9 or 1.10?
> > If with 1.9, I suggest to check with 1.10.
> >
> > Thanks,
> > Andrey
> >
> > On Thu, Feb 6, 2020 at 4:07 PM Benchao Li  wrote:
> >
> > > Hi all,
> > >
> > > I found another issue[1], I don't know if it should be a blocker. But
> it
> > > does affects joins without window in blink planner.
> > >
> > > [1] https://issues.apache.org/jira/browse/FLINK-15938
> > >
> > > Jeff Zhang  于2020年2月6日周四 下午5:05写道:
> > >
> > > > Hi Jingsong,
> > > >
> > > > Thanks for the suggestion. It works for running it in IDE, but for
> > > > downstream project like Zeppelin where I will include flink jars in
> > > > classpath.
> > > > it only works when I specify the jars one by one explicitly in
> > classpath,
> > > > using * doesn't work.
> > > >
> > > > e.g.
> > > >
> > > > The following command where I use * to specify classpath doesn't
> work,
> > > > jzhang 4794 1.2 6.3 8408904 1048828 s007 S 4:30PM 0:33.90
> > > >
> > /Library/Java/JavaVirtualMachines/jdk1.8.0_211.jdk/Contents/Home/bin/java
> > > > -Dfile.encoding=UTF-8
> > > >
> > > >
> > >
> >
> -Dlog4j.configuration=file:///Users/jzhang/github/zeppelin/conf/log4j.properties
> > > >
> > > >
> > >
> >
> -Dzeppelin.log.file=/Users/jzhang/github/zeppelin/logs/zeppelin-interpreter-flink-shared_process-jzhang-JeffdeMacBook-Pro.local.log
> > > > -Xms1024m -Xmx2048m -XX:MaxPermSize=512m -cp
> > > >
> > > >
> > >
> >
> *:/Users/jzhang/github/flink/build-target/lib/**:/Users/jzhang/github/zeppelin/interpreter/flink/*:/Users/jzhang/github/zeppelin/zeppelin-interpreter-api/target/*:/Users/jzhang/github/zeppelin/zeppelin-zengine/target/test-classes/*::/Users/jzhang/github/zeppelin/interpreter/zeppelin-interpreter-api-0.9.0-SNAPSHOT.jar:/Users/jzhang/github/zeppelin/zeppelin-interpreter/target/classes:/Users/jzhang/github/zeppelin/zeppelin-zengine/target/test-classes:/Users/jzhang/github/flink/build-target/opt/flink-python_2.11-1.10-SNAPSHOT.jar
> > > > org.apache.zeppelin.interpreter.remote.RemoteInterpreterServer
> 0.0.0.0
> > > > 52395 flink-shared_process :
> > > >
> > > >
> > > > While this command where I specify jar one by one explicitly in
> > classpath
> > > > works
> > > >
> > > > jzhang5660 205.2  6.1  8399420 1015036 s005  R 4:43PM
> > > > 0:24.82
> > > >
> > /Library/Java/JavaVirtualMachines/jdk1.8.0_211.jdk/Contents/Home/bin/java
> > > > -Dfile.encoding=UTF-8
> > > >
> > > >
> > >
> >
> -Dlog4j.configuration=file:///Users/jzhang/github/zeppelin/conf/log4j.properties
> > > >
> > > >
> > >
> >
> -Dzeppelin.log.file=/Users/jzhang/github/zeppelin/logs/zeppelin-interpreter-flink-shared_process-jzhang-JeffdeMacBook-Pro.local.log
> > > > -Xms1024m -Xmx2048m -XX:MaxPermSize=512m -cp
> > > >
> > > >
> > >
> >
> *:/Users/jzhang/github/flink/build-target/lib/slf4j-log4j12-1.7.15.jar:/Users/jzhang/github/flink/build-target/lib/flink-connector-hive_2.11-1.10-SNAPSHOT.jar:/Users/jzhang/github/flink/build-target/lib/hive-exec-2.3.4.jar:/Users/jzhang/github/flink/build-target/lib/flink-shaded-hadoop-2-uber-2.7.5-7.0.jar:/Users/jzhang/github/flink/build-target/lib/log4j-1.2.17.jar:/Users/jzhang/github/flink/build-target/lib/flink-table-blink_2.11-1.10-SNAPSHOT.jar:/Users/jzhang/github/flink/build-target/lib/flink-table_2.11-1.10-SNAPSHOT.jar:/Users/jzhang/github/flink/build-target/lib/flink-dist_2.11-1.10-SNAPSHOT.jar:/Users/jzhang/github/flink/build-target/lib/flink-python_2.11-1.10-SNAPSHOT.jar:/Users/jzhang/github/flink/build-target/lib/flink-hadoop-compatibility_2.11-1.9.1.jar*:/Users/jzhang/github/zeppelin/interpreter/flink/*:/Users/jzhang/github/zeppelin/zeppelin-interpreter-api/target/*:/Users/jzhang/github/zeppelin/zeppelin-zengine/target/test-classes/*::/Users/jzhang/github/zeppelin/interpreter/zeppelin-interpreter-api-0

Re: [VOTE] Release 1.10.0, release candidate #3

2020-02-11 Thread Andrey Zagrebin
Hi,

@Jingsong Lee
Regarding "OutOfMemoryError: Direct buffer memory" in
FileChannelBoundedData$FileBufferReader
I saw you created a specific issue issue:
https://issues.apache.org/jira/browse/FLINK-15981

In general, I think we could rewrap this error
in MemorySegmentFactory#allocateUnpooledOffHeapMemory,
e.g. suggesting to increase off heap memory option:
https://issues.apache.org/jira/browse/FLINK-15989
It can always happen independently from Flink if user code over-allocates
the direct memory somewhere else.

Thanks,
Andrey

On Tue, Feb 11, 2020 at 4:12 AM Yangze Guo  wrote:

> +1 (non-binding)
>
> - Build from source
> - Run mesos e2e tests(including unmerged heap state backend and rocks
> state backend case)
>
>
> Best,
> Yangze Guo
>
> On Tue, Feb 11, 2020 at 10:08 AM Yu Li  wrote:
> >
> > Thanks for the reminder Patrick! According to the release process [1] we
> > will publish the Dockerfiles *after* the RC voting passed, to finalize
> the
> > release.
> >
> > I have created FLINK-15978 [2] and prepared a PR [3] for it, will follow
> up
> > after we conclude our RC vote. Thanks.
> >
> > Best Regards,
> > Yu
> >
> > [1]
> >
> https://cwiki.apache.org/confluence/display/FLINK/Creating+a+Flink+Release
> > [2] https://issues.apache.org/jira/browse/FLINK-15978
> > [3] https://github.com/apache/flink-docker/pull/6
> >
> >
> > On Mon, 10 Feb 2020 at 20:57, Patrick Lucas 
> wrote:
> >
> > > Now that [FLINK-15828] Integrate docker-flink/docker-flink into Flink
> > > release process  is
> > > complete, the Dockerfiles for 1.10.0 can be published as part of the
> > > release process.
> > >
> > > @Gary/@Yu: please let me know if you have any questions regarding the
> > > workflow or its documentation.
> > >
> > > --
> > > Patrick
> > >
> > > On Mon, Feb 10, 2020 at 1:29 PM Benchao Li 
> wrote:
> > >
> > > > +1 (non-binding)
> > > >
> > > > - build from source
> > > > - start standalone cluster, and run some examples
> > > > - played with sql-client with some simple sql
> > > > - run tests in IDE
> > > > - run some sqls running in 1.9 internal version with 1.10.0-rc3,
> seems
> > > 1.10
> > > > behaves well.
> > > >
> > > > Xintong Song  于2020年2月10日周一 下午8:13写道:
> > > >
> > > > > +1 (non-binding)
> > > > >
> > > > > - build from source (with tests)
> > > > > - run nightly e2e tests
> > > > > - run example jobs in local/standalone/yarn setups
> > > > > - play around with memory configurations on local/standalone/yarn
> > > setups
> > > > >
> > > > > Thank you~
> > > > >
> > > > > Xintong Song
> > > > >
> > > > >
> > > > >
> > > > > On Mon, Feb 10, 2020 at 7:55 PM Jark Wu  wrote:
> > > > >
> > > > > > +1 (binding)
> > > > > >
> > > > > > - build the source release with Scala 2.12 and Scala 2.11
> > > successfully
> > > > > > - checked/verified signatures and hashes
> > > > > > - started cluster for both Scala 2.11 and 2.12, ran examples,
> > > verified
> > > > > web
> > > > > > ui and log output, nothing unexpected
> > > > > > - started cluster and run some e2e sql queries, all of them works
> > > well
> > > > > and
> > > > > > the results are as expected:
> > > > > >   - read from kafka source, aggregate, write into mysql
> > > > > >   - read from kafka source with watermark defined in ddl, window
> > > > > aggregate,
> > > > > > write into mysql
> > > > > >   - read from kafka with computed column defined in ddl, temporal
> > > join
> > > > > with
> > > > > > a mysql table, write into kafka
> > > > > >
> > > > > > Cheers,
> > > > > > Jark
> > > > > >
> > > > > >
> > > > > > On Mon, 10 Feb 2020 at 19:23, Kurt Young 
> wrote:
> > > > > >
> > > > > > > +1 (binding)
> > > > > > >
> > > > > > > - verified signatures and checksums
> > > > > > > - start local cluster, run some examples, randomly play some
> sql
> > > with
> > > > > sql
> > > > > > > client, no suspicious error/warn log found in log files
> > > > > > > - repeat above operation with both scala 2.11 and 2.12 binary
> > > > > > >
> > > > > > > Best,
> > > > > > > Kurt
> > > > > > >
> > > > > > >
> > > > > > > On Mon, Feb 10, 2020 at 6:38 PM Yang Wang <
> danrtsey...@gmail.com>
> > > > > wrote:
> > > > > > >
> > > > > > > >  +1 non-binding
> > > > > > > >
> > > > > > > >
> > > > > > > > - Building from source with all tests skipped
> > > > > > > > - Build a custom image with 1.10-rc3
> > > > > > > > - K8s tests
> > > > > > > > * Deploy a standalone session cluster on K8s and submit
> > > > multiple
> > > > > > jobs
> > > > > > > > * Deploy a standalone per-job cluster
> > > > > > > > * Deploy a native session cluster on K8s with/without HA
> > > > > > configured,
> > > > > > > > kill TM and jobs could recover successfully
> > > > > > > >
> > > > > > > >
> > > > > > > > Best,
> > > > > > > > Yang
> > > > > > > >
> > > > > > > > Jingsong Li  于2020年2月10日周一 下午4:29写道:
> > > > > > > >
> > > > > > > > > Hi,
> > > > > > > > >
> > > > > > > > >
> > > > > > > > > +1 (non-binding) Thanks for driving this, 

[DISCUSS] FLIP-111: Docker image unification

2020-03-04 Thread Andrey Zagrebin
Hi All,

If you have ever touched the docker topic in Flink, you
probably noticed that we have multiple places in docs and repos which
address its various concerns.

We have prepared a FLIP [1] to simplify the perception of docker topic in
Flink by users. It mostly advocates for an approach of extending official
Flink image from the docker hub. For convenience, it can come with a set of
bash utilities and documented examples of their usage. The utilities allow
to:

   - run the docker image in various modes (single job, session master,
   task manager etc)
   - customise the extending Dockerfile
   - and its entry point

Eventually, the FLIP suggests to remove all other user facing Dockerfiles
and building scripts from Flink repo, move all docker docs to
apache/flink-docker and adjust existing docker use cases to refer to this
new approach (mostly Kubernetes now).

The first contributed version of Flink docker integration also contained
example and docs for the integration with Bluemix in IBM cloud. We also
suggest to maintain it outside of Flink repository (cc Markus Müller).

Thanks,
Andrey

[1]
https://cwiki.apache.org/confluence/display/FLINK/FLIP-111%3A+Docker+image+unification


Re: [DISCUSS] FLIP-111: Docker image unification

2020-03-10 Thread Andrey Zagrebin
d more flexible, e.g. it makes it very easy to
> pass values of Kubernetes Secrets into the Flink configuration.

This is indeed an interesting option to pass arguments to the entry point
in general.
For the config options, the dynamic args can be a better option as
mentioned above.

With respect to logging, I would opt to keep this very basic and to only
> support logging to the console (maybe with a fix for the web user
> interface). For everything else, users can easily build their own images
> based on library/flink (provide the dependencies, change the logging
> configuration).

agree

Thanks,
Andrey

On Sun, Mar 8, 2020 at 8:55 PM Konstantin Knauf 
wrote:

> Hi Andrey,
>
> thanks a lot for this proposal. The variety of Docker files in the project
> has been causing quite some confusion.
>
> For the entrypoint, have you considered to also allow setting
> configuration via environment variables as in "docker run -e
> FLINK_REST_BIN_PORT=8081 ..."? This is quite common and more flexible, e.g.
> it makes it very easy to pass values of Kubernetes Secrets into the Flink
> configuration.
>
> With respect to logging, I would opt to keep this very basic and to only
> support logging to the console (maybe with a fix for the web user
> interface). For everything else, users can easily build their own images
> based on library/flink (provide the dependencies, change the logging
> configuration).
>
> Cheers,
>
> Konstantin
>
>
> On Thu, Mar 5, 2020 at 11:01 AM Yang Wang  wrote:
>
>> Hi Andrey,
>>
>>
>> Thanks for driving this significant FLIP. From the user ML, we could also
>> know there are
>> many users running Flink in container environment. Then the docker image
>> will be the
>> very basic requirement. Just as you say, we should provide a unified
>> place for all various
>> usage(e.g. session, job, native k8s, swarm, etc.).
>>
>>
>> > About docker utils
>>
>> I really like the idea to provide some utils for the docker file and
>> entry point. The
>> `flink_docker_utils` will help to build the image easier. I am not sure
>> about the
>> `flink_docker_utils start_jobmaster`. Do you mean when we build a docker
>> image, we
>> need to add `RUN flink_docker_utils start_jobmaster` in the docker file?
>> Why do we need this?
>>
>>
>> > About docker entry point
>>
>> I agree with you that the docker entry point could more powerful with
>> more functionality.
>> Mostly, it is about to override the config options. If we support dynamic
>> properties, i think
>> it is more convenient for users without any learning curve.
>> `docker run flink session_jobmanager -D rest.bind-port=8081`
>>
>>
>> > About the logging
>>
>> Updating the `log4j-console.properties` to support multiple appender is a
>> better option.
>> Currently, the native K8s is suggesting users to debug the logs in this
>> way[1]. However,
>> there is also some problems. The stderr and stdout of JM/TM processes
>> could not be
>> forwarded to the docker container console.
>>
>>
>> [1].
>> https://ci.apache.org/projects/flink/flink-docs-master/ops/deployment/native_kubernetes.html#log-files
>>
>>
>> Best,
>> Yang
>>
>>
>>
>>
>> Andrey Zagrebin  于2020年3月4日周三 下午5:34写道:
>>
>>> Hi All,
>>>
>>> If you have ever touched the docker topic in Flink, you
>>> probably noticed that we have multiple places in docs and repos which
>>> address its various concerns.
>>>
>>> We have prepared a FLIP [1] to simplify the perception of docker topic in
>>> Flink by users. It mostly advocates for an approach of extending official
>>> Flink image from the docker hub. For convenience, it can come with a set
>>> of
>>> bash utilities and documented examples of their usage. The utilities
>>> allow
>>> to:
>>>
>>>- run the docker image in various modes (single job, session master,
>>>task manager etc)
>>>- customise the extending Dockerfile
>>>- and its entry point
>>>
>>> Eventually, the FLIP suggests to remove all other user facing Dockerfiles
>>> and building scripts from Flink repo, move all docker docs to
>>> apache/flink-docker and adjust existing docker use cases to refer to this
>>> new approach (mostly Kubernetes now).
>>>
>>> The first contributed version of Flink docker integration also contained
>>> example and docs for the integration with Bluemix in IBM cloud. We also
>>> suggest to maintain it outside of Flink repository (cc Markus Müller).
>>>
>>> Thanks,
>>> Andrey
>>>
>>> [1]
>>>
>>> https://cwiki.apache.org/confluence/display/FLINK/FLIP-111%3A+Docker+image+unification
>>>
>>
>
> --
>
> Konstantin Knauf | Head of Product
>
> +49 160 91394525
>
>
> Follow us @VervericaData Ververica <https://www.ververica.com/>
>
>
> --
>
> Join Flink Forward <https://flink-forward.org/> - The Apache Flink
> Conference
>
> Stream Processing | Event Driven | Real Time
>
> --
>
> Ververica GmbH | Invalidenstrasse 115, 10115 Berlin, Germany
>
> --
> Ververica GmbH
> Registered at Amtsgericht Charlottenburg: HRB 158244 B
> Managing Directors: Timothy Alexander Steinert, Yip Park Tung Jason, Ji
> (Tony) Cheng
>


Re: [DISCUSS] Releasing Flink 1.10.1

2020-03-11 Thread Andrey Zagrebin
Hi Yu,

Thanks for kicking off the 1.10.1 release discussion!

Apart from
- FLINK-16406 Increase default value for JVM Metaspace to minimise its
OutOfMemoryError
which should be merged soon

I think we should also try to get in the following issues:

- [FLINK-16225] Metaspace Out Of Memory should be handled as Fatal Error in
TaskManager
This should solve the Metaspace problem even in a better way because OOM
failure should point users to the docs immediately

- [FLINK-16408] Bind user code class loader to lifetime of a slot
This should give a better protection against class loading leaks

- [FLINK-16018] Improve error reporting when submitting batch job (instead
of AskTimeoutException)
This problem has recently happened for multiple users

Best,
Andrey


On Wed, Mar 11, 2020 at 8:46 AM Jingsong Li  wrote:

> Thanks for driving. Yu. +1 for starting the 1.10.1 release.
>
> Some issues are very important, Users are looking forward to them.
>
> Best,
> Jingsong Lee
>
> On Wed, Mar 11, 2020 at 2:52 PM Yangze Guo  wrote:
>
> > Thanks for driving this release, Yu!
> >
> > +1 for starting the 1.10.1 release cycle.
> >
> > Best,
> > Yangze Guo
> >
> > On Wed, Mar 11, 2020 at 1:42 PM Xintong Song 
> > wrote:
> > >
> > > Yu,
> > > Thanks for the explanation.
> > > I've no concerns. I was just trying to get some inputs for prioritizing
> > > tasks on my side, and ~1month sounds good to me.
> > >
> > >
> > > Thank you~
> > >
> > > Xintong Song
> > >
> > >
> > >
> > > On Wed, Mar 11, 2020 at 12:15 PM Yu Li  wrote:
> > >
> > > > bq. what is the time plan for 1.10.1?
> > > >
> > > > According to the history, the first patch release of a major version
> > will
> > > > take ~1month from discussion started, depending on the speed of
> blocker
> > > > issue resolving:
> > > > * 1.8.1: started discussion on May 28th [1], released on Jul 3rd [2]
> > > > * 1.9.1: started discussion on Sep 23rd [3], released on Oct 19th [4]
> > > >
> > > > We won't rush to match the history of course, but could use it as a
> > > > reference. And please feel free to let me know if any concerns
> Xintong.
> > > > Thanks.
> > > >
> > > > Best Regards,
> > > > Yu
> > > >
> > > > [1]
> > > >
> > > >
> >
> http://apache-flink-mailing-list-archive.1008284.n3.nabble.com/DISCUSS-Releasing-Flink-1-8-1-td29154.html
> > > > [2]
> > > >
> > > >
> >
> http://apache-flink-mailing-list-archive.1008284.n3.nabble.com/ANNOUNCE-Apache-Flink-1-8-1-released-td30124.html
> > > > [3]
> > > >
> > > >
> >
> http://apache-flink-mailing-list-archive.1008284.n3.nabble.com/DISCUSS-Releasing-Flink-1-9-1-td33343.html
> > > > [4]
> > > >
> > > >
> >
> http://apache-flink-mailing-list-archive.1008284.n3.nabble.com/ANNOUNCE-Apache-Flink-1-9-1-released-td34170.html
> > > >
> > > >
> > > > On Wed, 11 Mar 2020 at 11:54, Xintong Song 
> > wrote:
> > > >
> > > > > Thanks Yu, for the kick off and volunteering to be the release
> > manager.
> > > > >
> > > > > +1 for the proposal.
> > > > >
> > > > >
> > > > > One quick question, what is the time plan for 1.10.1?
> > > > >
> > > > >
> > > > > Thank you~
> > > > >
> > > > > Xintong Song
> > > > >
> > > > >
> > > > >
> > > > > On Wed, Mar 11, 2020 at 11:51 AM Zhijiang
> > > > >  wrote:
> > > > >
> > > > > > Thanks for driving this release, Yu!
> > > > > > +1 on my side
> > > > > >
> > > > > >
> > > > > >
> > > > > >
> > > > > >
> > > > > >
> > > > > >
> > > > > > Best,
> > > > > > Zhijiang
> > > > > >
> > > > > >
> > > > > >
> --
> > > > > > From:Yu Li 
> > > > > > Send Time:2020 Mar. 10 (Tue.) 20:25
> > > > > > To:dev 
> > > > > > Subject:Re: [DISCUSS] Releasing Flink 1.10.1
> > > > > >
> > > > > > Thanks for the supplement Hequn. Yes will also keep an eye on
> these
> > > > > > existing blocker issues.
> > > > > >
> > > > > > Best Regards,
> > > > > > Yu
> > > > > >
> > > > > >
> > > > > > On Tue, 10 Mar 2020 at 19:10, Hequn Cheng 
> > wrote:
> > > > > >
> > > > > > > Hi Yu,
> > > > > > >
> > > > > > > Thanks a lot for raising the discussion and volunteer as the
> > release
> > > > > > > manager!
> > > > > > >
> > > > > > > I found there are some other issues[1] which are marked as a
> > blocker:
> > > > > > > - FLINK-16454 Update the copyright year in NOTICE files
> > > > > > > - FLINK-16262 Class loader problem with
> > > > > > > FlinkKafkaProducer.Semantic.EXACTLY_ONCE and usrlib directory
> > > > > > > - FLINK-16170 SearchTemplateRequest ClassNotFoundException when
> > use
> > > > > > > flink-sql-connector-elasticsearch7
> > > > > > > - FLINK-16018 Improve error reporting when submitting batch job
> > > > > (instead
> > > > > > of
> > > > > > > AskTimeoutException)
> > > > > > >
> > > > > > > These may also need to be resolved in 1.10.1.
> > > > > > >
> > > > > > > Best,
> > > > > > > Hequn
> > > > > > >
> > > > > > > [1]
> > https://issues.apache.org/jira/projects/FLINK/versions/12346891
> > > > > > >
> > > > > > >
> > > > > > > On Tue, Mar 10, 2020 at 6:48 PM Yu Li 
> 

[DISCUSS] FLIP 116: Unified Memory Configuration for Job Managers

2020-03-11 Thread Andrey Zagrebin
Hi All,

As you may have noticed, 1.10 release included an extensive improvements to
memory management and configuration of Task Managers, FLIP-49: [1]. The
memory configuration of Job Managers has not been touched in 1.10.

Although, Job Manager's memory model does not look so sophisticated as
for Task Managers, It makes to align Job Manager memory model and settings
with Task Managers. Therefore, we propose to reconsider it as well in 1.11
and I prepared a FLIP 116 [2] for that.

Any feedback is appreciated.

So far, there is one discussion point about how to address native
non-direct memory usage of user code. The user code can be run e.g. in
certain job submission scenarios within the JM process. For simplicity,
FLIP suggests only an option for direct memory which is translated into the
setting of the JVM direct memory limit.
Although, we documented for TM that the similar parameters can also
address native non-direct memory usage [3], this can lead to wrong
functioning of the JVM direct memory limit. The direct memory option in JM
could be also named in more general way, e.g. off-heap memory but this
naming would somewhat hide its nature of JVM direct memory limit.
On the other hand, JVM Overhead does not suffer from this problem and
affects only the container/worker memory size which is the most important
matter to address for the native non-direct memory consumption. The caveat
here is that JVM Overhead was not supposed to be used by any Flink or user
components.

Thanks,
Andrey

[1]
https://cwiki.apache.org/confluence/display/FLINK/FLIP-49%3A+Unified+Memory+Configuration+for+TaskExecutors
[2]
https://cwiki.apache.org/confluence/display/FLINK/FLIP+116%3A+Unified+Memory+Configuration+for+Job+Managers
[3]
https://ci.apache.org/projects/flink/flink-docs-release-1.10/ops/memory/mem_detail.html#overview


Re: [DISCUSS] Releasing Flink 1.10.1

2020-03-12 Thread Andrey Zagrebin


> About "FLINK-16142 Memory Leak causes Metaspace OOM error on repeated job”

My understanding that the issue is basically covered by:

- [FLINK-16225] Metaspace Out Of Memory should be handled as Fatal Error in 
TaskManager
   no full consensus there but improving error message for existing task thread 
fatal handling could be done at least

- [FLINK-16406] Increase default value for JVM Metaspace to minimise its 
OutOfMemoryError
   see further

- [FLINK-16246] Exclude "SdkMBeanRegistrySupport" from dynamically loaded AWS 
connectors
  not sure whether this is a blocker but looks close to be resolved 

> About "FLINK-16406 Increase default value for JVM Metaspace"
>  - Have we consensus that this is okay for a bugfix release? It changes
> setups, takes away memory from heap / managed memory on existing setups
> that keep their flink-conf.yaml.

My understanding was that increasing to 256m resolved the reported problems
and we decided to make the change so I have merged it today as there were no 
more concerns.
If there are concerns I can revert it.

On the other hand, I think improving the message error with reference to the 
metaspace option should help the most
because user would not have to read all docs to fix it
then maybe this change is not even needed.

Best,
Andrey

> On 12 Mar 2020, at 12:28, Stephan Ewen  wrote:
> 
> Good idea to go ahead with 1.10.1
> 
> About "FLINK-16142 Memory Leak causes Metaspace OOM error on repeated job"
>  - I don't think we have consensus on the exact solution, yet, and some of
> the changes might also have side effects that are hard to predict, so I am
> not sure we should rush this in.
> 
> About "FLINK-16406 Increase default value for JVM Metaspace"
>  - Have we consensus that this is okay for a bugfix release? It changes
> setups, takes away memory from heap / managed memory on existing setups
> that keep their flink-conf.yaml.
> 
> We may need to unblock the release form these two issues and think about
> having 1.10.2 in the near future.
> 
> On Thu, Mar 12, 2020 at 7:15 AM Yu Li  wrote:
> 
>> Thanks for the reminder Jark. Will keep an eye on these two.
>> 
>> Best Regards,
>> Yu
>> 
>> 
>> On Thu, 12 Mar 2020 at 12:32, Jark Wu  wrote:
>> 
>>> Thanks for driving this release, Yu!
>>> +1 to start 1.10.1 release cycle.
>>> 
>>> From the Table SQL module, I think we should also try to get in the
>>> following issues:
>>> - FLINK-16441: Allow users to override flink-conf parameters from SQL CLI
>>> environment
>>>  this allows users to set e.g. statebackend, watermark interval,
>>> exactly-once/at-least-once, in the SQL CLI
>>> - FLINK-16217: SQL Client crashed when any uncatched exception is thrown
>>>  this will improve much experience when using SQL CLI
>>> 
>>> Best,
>>> Jark
>>> 
>>> 
>>> On Wed, 11 Mar 2020 at 20:37, Yu Li  wrote:
>>> 
>>>> Thanks for the suggestion Andrey! I've added 1.10.1 into FLINK-16225
>> fix
>>>> versions and promoted its priority to Critical. Will also watch the
>>>> progress of FLINK-16108/FLINK-16408.
>>>> 
>>>> Best Regards,
>>>> Yu
>>>> 
>>>> 
>>>> On Wed, 11 Mar 2020 at 18:18, Andrey Zagrebin 
>>>> wrote:
>>>> 
>>>>> Hi Yu,
>>>>> 
>>>>> Thanks for kicking off the 1.10.1 release discussion!
>>>>> 
>>>>> Apart from
>>>>> - FLINK-16406 Increase default value for JVM Metaspace to minimise
>> its
>>>>> OutOfMemoryError
>>>>> which should be merged soon
>>>>> 
>>>>> I think we should also try to get in the following issues:
>>>>> 
>>>>> - [FLINK-16225] Metaspace Out Of Memory should be handled as Fatal
>>> Error
>>>> in
>>>>> TaskManager
>>>>> This should solve the Metaspace problem even in a better way because
>>> OOM
>>>>> failure should point users to the docs immediately
>>>>> 
>>>>> - [FLINK-16408] Bind user code class loader to lifetime of a slot
>>>>> This should give a better protection against class loading leaks
>>>>> 
>>>>> - [FLINK-16018] Improve error reporting when submitting batch job
>>>> (instead
>>>>> of AskTimeoutException)
>>>>> This problem has recently happened for multiple users
>>>>> 
>>>>> Best,
>>>

Re: [DISCUSS] Releasing Flink 1.10.1

2020-03-12 Thread Andrey Zagrebin
>   - For 1.10.1 I am not completely sure, because users expect to upgrade
> that without config adjustments. That might not be possible with that
> change.

Ok, makes sense, I will revert it for 1.10 and only try to improve error 
message and docs.

> On 12 Mar 2020, at 13:15, Stephan Ewen  wrote:
> 
> @Andrey about the increase in metaspace size
>   - I have no concerns for 1.11.0.
>   - For 1.10.1 I am not completely sure, because users expect to upgrade
> that without config adjustments. That might not be possible with that
> change.
> 
> On Thu, Mar 12, 2020 at 12:55 PM Andrey Zagrebin 
> wrote:
> 
>> 
>>> About "FLINK-16142 Memory Leak causes Metaspace OOM error on repeated
>> job”
>> 
>> My understanding that the issue is basically covered by:
>> 
>> - [FLINK-16225] Metaspace Out Of Memory should be handled as Fatal Error
>> in TaskManager
>>   no full consensus there but improving error message for existing task
>> thread fatal handling could be done at least
>> 
>> - [FLINK-16406] Increase default value for JVM Metaspace to minimise its
>> OutOfMemoryError
>>   see further
>> 
>> - [FLINK-16246] Exclude "SdkMBeanRegistrySupport" from dynamically loaded
>> AWS connectors
>>  not sure whether this is a blocker but looks close to be resolved
>> 
>>> About "FLINK-16406 Increase default value for JVM Metaspace"
>>> - Have we consensus that this is okay for a bugfix release? It changes
>>> setups, takes away memory from heap / managed memory on existing setups
>>> that keep their flink-conf.yaml.
>> 
>> My understanding was that increasing to 256m resolved the reported problems
>> and we decided to make the change so I have merged it today as there were
>> no more concerns.
>> If there are concerns I can revert it.
>> 
>> On the other hand, I think improving the message error with reference to
>> the metaspace option should help the most
>> because user would not have to read all docs to fix it
>> then maybe this change is not even needed.
>> 
>> Best,
>> Andrey
>> 
>>> On 12 Mar 2020, at 12:28, Stephan Ewen  wrote:
>>> 
>>> Good idea to go ahead with 1.10.1
>>> 
>>> About "FLINK-16142 Memory Leak causes Metaspace OOM error on repeated
>> job"
>>> - I don't think we have consensus on the exact solution, yet, and some
>> of
>>> the changes might also have side effects that are hard to predict, so I
>> am
>>> not sure we should rush this in.
>>> 
>>> About "FLINK-16406 Increase default value for JVM Metaspace"
>>> - Have we consensus that this is okay for a bugfix release? It changes
>>> setups, takes away memory from heap / managed memory on existing setups
>>> that keep their flink-conf.yaml.
>>> 
>>> We may need to unblock the release form these two issues and think about
>>> having 1.10.2 in the near future.
>>> 
>>> On Thu, Mar 12, 2020 at 7:15 AM Yu Li  wrote:
>>> 
>>>> Thanks for the reminder Jark. Will keep an eye on these two.
>>>> 
>>>> Best Regards,
>>>> Yu
>>>> 
>>>> 
>>>> On Thu, 12 Mar 2020 at 12:32, Jark Wu  wrote:
>>>> 
>>>>> Thanks for driving this release, Yu!
>>>>> +1 to start 1.10.1 release cycle.
>>>>> 
>>>>> From the Table SQL module, I think we should also try to get in the
>>>>> following issues:
>>>>> - FLINK-16441: Allow users to override flink-conf parameters from SQL
>> CLI
>>>>> environment
>>>>> this allows users to set e.g. statebackend, watermark interval,
>>>>> exactly-once/at-least-once, in the SQL CLI
>>>>> - FLINK-16217: SQL Client crashed when any uncatched exception is
>> thrown
>>>>> this will improve much experience when using SQL CLI
>>>>> 
>>>>> Best,
>>>>> Jark
>>>>> 
>>>>> 
>>>>> On Wed, 11 Mar 2020 at 20:37, Yu Li  wrote:
>>>>> 
>>>>>> Thanks for the suggestion Andrey! I've added 1.10.1 into FLINK-16225
>>>> fix
>>>>>> versions and promoted its priority to Critical. Will also watch the
>>>>>> progress of FLINK-16108/FLINK-16408.
>>>>>> 
>>>>>> Best Regards,
>>>>>> Yu
>>>>>> 
>>>>>> 
>>>>>> On Wed, 11 Mar 2

Re: [DISCUSS] Releasing Flink 1.10.1

2020-03-13 Thread Andrey Zagrebin
he
> metaspace
> > > >  increase is more likely to cause problem.
> > > >
> > > > So basically only people have small 'process.size' in custom config
> > file
> > > > are really affected. I'm not sure what is the proportion of such use
> > > cases
> > > > though. (From questions asked on the user ML, probably not much).
> > > >
> > > > Thank you~
> > > >
> > > > Xintong Song
> > > >
> > > >
> > > >
> > > > On Thu, Mar 12, 2020 at 10:09 PM Stephan Ewen 
> > wrote:
> > > >
> > > > > No need to revert it now - I am not saying it should not go into
> > > 1.10.1,
> > > > I
> > > > > am just saying this is not clear to me yet.
> > > > >
> > > > > We have to trade off the fact that we may break some deployments
> with
> > > the
> > > > > fact that we will "safe" a lot of new deployments.
> > > > > I simply lack the data points / insight at the moment to understand
> > how
> > > > > significant both cases are, meaning how many users would be
> affected
> > > and
> > > > > how badly.
> > > > >
> > > > > Independent of that, improving the error message is always helpful.
> > > > >
> > > > > On Thu, Mar 12, 2020 at 1:22 PM Andrey Zagrebin <
> > > > > azagrebin.apa...@gmail.com>
> > > > > wrote:
> > > > >
> > > > > > >   - For 1.10.1 I am not completely sure, because users expect
> to
> > > > > upgrade
> > > > > > > that without config adjustments. That might not be possible
> with
> > > that
> > > > > > > change.
> > > > > >
> > > > > > Ok, makes sense, I will revert it for 1.10 and only try to
> improve
> > > > error
> > > > > > message and docs.
> > > > > >
> > > > > > > On 12 Mar 2020, at 13:15, Stephan Ewen 
> wrote:
> > > > > > >
> > > > > > > @Andrey about the increase in metaspace size
> > > > > > >   - I have no concerns for 1.11.0.
> > > > > > >   - For 1.10.1 I am not completely sure, because users expect
> to
> > > > > upgrade
> > > > > > > that without config adjustments. That might not be possible
> with
> > > that
> > > > > > > change.
> > > > > > >
> > > > > > > On Thu, Mar 12, 2020 at 12:55 PM Andrey Zagrebin <
> > > > > > azagrebin.apa...@gmail.com>
> > > > > > > wrote:
> > > > > > >
> > > > > > >>
> > > > > > >>> About "FLINK-16142 Memory Leak causes Metaspace OOM error on
> > > > repeated
> > > > > > >> job”
> > > > > > >>
> > > > > > >> My understanding that the issue is basically covered by:
> > > > > > >>
> > > > > > >> - [FLINK-16225] Metaspace Out Of Memory should be handled as
> > Fatal
> > > > > Error
> > > > > > >> in TaskManager
> > > > > > >>   no full consensus there but improving error message for
> > existing
> > > > > task
> > > > > > >> thread fatal handling could be done at least
> > > > > > >>
> > > > > > >> - [FLINK-16406] Increase default value for JVM Metaspace to
> > > minimise
> > > > > its
> > > > > > >> OutOfMemoryError
> > > > > > >>   see further
> > > > > > >>
> > > > > > >> - [FLINK-16246] Exclude "SdkMBeanRegistrySupport" from
> > dynamically
> > > > > > loaded
> > > > > > >> AWS connectors
> > > > > > >>  not sure whether this is a blocker but looks close to be
> > resolved
> > > > > > >>
> > > > > > >>> About "FLINK-16406 Increase default value for JVM Metaspace"
> > > > > > >>> - Have we consensus that this is okay for a bugfix release?
> It
> > > > > changes
> > > > > > >>> setups, takes away memory from heap / man

Re: [DISCUSS] FLIP-111: Docker image unification

2020-03-16 Thread Andrey Zagrebin
. I'll take a look at it.
>
> Regarding supporting JAVA 11:
> - Not sure if it is necessary to ship JAVA. Maybe we could just change
> the base image from openjdk:8-jre to openjdk:11-jre in template docker
> file[1]. Correct me if I understand incorrectly. Also, I agree to move
> this out of the scope of this FLIP if it indeed takes much extra
> effort.
>
> Regarding the custom configuration, the mechanism that Thomas mentioned
> LGTM.
>
> [1]
> https://github.com/apache/flink-docker/blob/master/Dockerfile-debian.template
>
> Best,
> Yangze Guo
>
> On Wed, Mar 11, 2020 at 5:52 AM Thomas Weise  wrote:
> >
> > Thanks for working on improvements to the Flink Docker container images.
> This will be important as more and more users are looking to adopt
> Kubernetes and other deployment tooling that relies on Docker images.
> >
> > A generic, dynamic configuration mechanism based on environment
> variables is essential and it is already supported via envsubst and an
> environment variable that can supply a configuration fragment:
> >
> >
> https://github.com/apache/flink-docker/blob/09adf2dcd99abfb6180e1e2b5b917b288e0c01f6/docker-entrypoint.sh#L88
> >
> https://github.com/apache/flink-docker/blob/09adf2dcd99abfb6180e1e2b5b917b288e0c01f6/docker-entrypoint.sh#L85
> >
> > This gives the necessary control for infrastructure use cases that aim
> to supply deployment tooling other users. An example in this category this
> is the FlinkK8sOperator:
> >
> > https://github.com/lyft/flinkk8soperator/tree/master/examples/wordcount
> >
> > On the flip side, attempting to support a fixed subset of configuration
> options is brittle and will probably lead to compatibility issues down the
> road:
> >
> >
> https://github.com/apache/flink-docker/blob/09adf2dcd99abfb6180e1e2b5b917b288e0c01f6/docker-entrypoint.sh#L97
> >
> > Besides the configuration, it may be worthwhile to see in which other
> ways the base Docker images can provide more flexibility to incentivize
> wider adoption.
> >
> > I would second that it is desirable to support Java 11 and in general
> use a base image that allows the (straightforward) use of more recent
> versions of other software (Python etc.)
> >
> >
> https://github.com/apache/flink-docker/blob/d3416e720377e9b4c07a2d0f4591965264ac74c5/Dockerfile-debian.template#L19
> >
> > Thanks,
> > Thomas
> >
> > On Tue, Mar 10, 2020 at 12:26 PM Andrey Zagrebin 
> wrote:
> >>
> >> Hi All,
> >>
> >> Thanks a lot for the feedback!
> >>
> >> *@Yangze Guo*
> >>
> >> - Regarding the flink_docker_utils#install_flink function, I think it
> >> > should also support build from local dist and build from a
> >> > user-defined archive.
> >>
> >> I suppose you bring this up mostly for development purpose or powerful
> >> users.
> >> Most of normal users are usually interested in mainstream released
> versions
> >> of Flink.
> >> Although, you are bring a valid concern, my idea was to keep scope of
> this
> >> FLIP mostly for those normal users.
> >> The powerful users are usually capable to design a completely
> >> custom Dockerfile themselves.
> >> At the moment, we already have custom Dockerfiles e.g. for tests in
> >>
> flink-end-to-end-tests/test-scripts/docker-hadoop-secure-cluster/Dockerfile.
> >> We can add something similar for development purposes and maybe
> introduce a
> >> special maven goal. There is a maven docker plugin, afaik.
> >> I will add this to FLIP as next step.
> >>
> >> - It seems that the install_shaded_hadoop could be an option of
> >> > install_flink
> >>
> >> I woud rather think about this as a separate independent optional step.
> >>
> >> - Should we support JAVA 11? Currently, most of the docker file based on
> >> > JAVA 8.
> >>
> >> Indeed, it is a valid concern. Java version is a fundamental property of
> >> the docker image.
> >> To customise this in the current mainstream image is difficult, this
> would
> >> require to ship it w/o Java at all.
> >> Or this is a separate discussion whether we want to distribute docker
> hub
> >> images with different Java versions or just bump it to Java 11.
> >> This should be easy in a custom Dockerfile for development purposes
> though
> >> as mentioned before.
> >>
> >> - I do not understand how to set config options through
> >>
> >> "flink_docker_utils conf

Re: Question about RocksDBStateBackend Compaction Filter state cleanup

2020-03-17 Thread Andrey Zagrebin
Hi Lake,

When the Flink doc mentions a state entry in RocksDB, we mean one key/value
pair stored by user code over any keyed state API
(keyed context in keyed operators obtained e.g. from keyBy()
transformation).
In case of map or list, the doc means map key/value and list element.

- value/aggregating/folding/reducing state: key -> value
- map state: key -> map key -> value
- list state: key -> list -> element in some position

Best,
Andrey

On Tue, Mar 17, 2020 at 11:04 AM Yun Tang  wrote:

> Hi Lake
>
> Flink leverage RocksDB's background compaction mechanism to filter
> out-of-TTL entries (by comparing with current timestamp provided from
> RocksDB's time_provider) to not let them stay in newly compacted data.
>
> This would iterator over data entries with FlinkCompactionFilter::FilterV2
> [1], and the parameter 'queryTimeAfterNumEntries' in Flink indicates the
> threshold 'query_time_after_num_entries_' in FrocksDB  [2]. Once RocksDB
> iterator more than several entries .e.g 1000, it would call time_provider
> to update current timestamp to let the process of cleaning up more eagerly
> and accurately.
>
> [1]
> https://github.com/dataArtisans/frocksdb/blob/49bc897d5d768026f1eb816d960c1f2383396ef4/utilities/flink/flink_compaction_filter.cc#L107
> [2]
> https://github.com/dataArtisans/frocksdb/blob/49bc897d5d768026f1eb816d960c1f2383396ef4/utilities/flink/flink_compaction_filter.h#L140
>
> Best
> Yun Tang
>
> --
> *From:* LakeShen 
> *Sent:* Tuesday, March 17, 2020 15:30
> *To:* dev ; user-zh ;
> user 
> *Subject:* Question about RocksDBStateBackend Compaction Filter state
> cleanup
>
> Hi community ,
>
> I see the flink RocksDBStateBackend state cleanup,now the code like this :
>
> StateTtlConfig ttlConfig = StateTtlConfig
> .newBuilder(Time.seconds(1))
> .cleanupInRocksdbCompactFilter(1000)
> .build();
>
>
>
> The default background cleanup for RocksDB backend queries the current
> timestamp each time 1000 entries have been processed.
>
>
> What's the meaning of  1000 entries? 1000 different key ?
>
> Thanks to your reply.
>
> Best regards,
> LakeShen
>


Re: [DISCUSS] FLIP 116: Unified Memory Configuration for Job Managers

2020-03-18 Thread Andrey Zagrebin
Hi all,

Thanks for the feedback, Xintong and Till.

> rename jobmanager.memory.direct.size into jobmanager.memory.off-heap.size

I am ok with that to align it with TM and avoid further complications for
users.
I will adjust the FLIP.

> change the default value of JM Metaspace size to 256 MB

Indeed, no reason to assume that the user code would need less Metaspace in
JM.
I will change it unless a better argument is reported for another value.

I think all concerns has been resolved so I am starting the voting in a
separate thread.

Best,
Andrey

On Tue, Mar 17, 2020 at 6:16 PM Till Rohrmann  wrote:

> Thanks for creating this FLIP Andrey.
>
> I agree with Xintong that we should rename jobmanager.memory.direct.size
> into jobmanager.memory.off-heap.size which accounts for native and direct
> memory usage. I think it should be good enough and is easier to understand
> for the user.
>
> Concerning the default value for the metaspace size. Did we take the
> lessons learned from the TM metaspace size into account? IIRC we are about
> to change the default value to 256 MB.
>
> Feel free to start a vote once these last two questions have been resolved.
>
> Cheers,
> Till
>
> On Thu, Mar 12, 2020 at 4:25 AM Xintong Song 
> wrote:
>
> > Thanks Andrey for kicking this discussion off.
> >
> > Regarding "direct" vs. "off-heap", I'm personally in favor of renaming
> the
> > "direct" memory in the current FLIP-116[1] to "off-heap" memory, and
> making
> > it also account for user native memory usage.
> >
> > On one hand, I think it would be good that JM & TM provide consistent
> > concepts and terminologies to users. IIUC, this is exactly the purpose of
> > this FLIP. For TMs, we already have "off-heap" memory accounting for both
> > direct and native memory usages, and we did this so that users do not
> need
> > to understand the differences between the two kinds.
> >
> > On the other hand, while for TMs it is hard to tell which kind of memory
> is
> > needed mostly due to variety of applications, I believe for JM the major
> > memory consumption is heap memory in most cases. That means we probably
> can
> > rely on the heap activities to trigger GC in most cases, and the max
> direct
> > memory limit can act as a safe net. Moreover, I think the cases should be
> > very rare that we need native memory for user codes. Therefore, we
> probably
> > should not break the JM/TM consistency for potential risks in such rare
> > cases.
> >
> > WDYT?
> >
> > Thank you~
> >
> > Xintong Song
> >
> >
> > [1]
> >
> >
> https://cwiki.apache.org/confluence/display/FLINK/FLIP+116%3A+Unified+Memory+Configuration+for+Job+Managers
> >
> > On Wed, Mar 11, 2020 at 8:56 PM Andrey Zagrebin 
> > wrote:
> >
> > > Hi All,
> > >
> > > As you may have noticed, 1.10 release included an extensive
> improvements
> > to
> > > memory management and configuration of Task Managers, FLIP-49: [1]. The
> > > memory configuration of Job Managers has not been touched in 1.10.
> > >
> > > Although, Job Manager's memory model does not look so sophisticated as
> > > for Task Managers, It makes to align Job Manager memory model and
> > settings
> > > with Task Managers. Therefore, we propose to reconsider it as well in
> > 1.11
> > > and I prepared a FLIP 116 [2] for that.
> > >
> > > Any feedback is appreciated.
> > >
> > > So far, there is one discussion point about how to address native
> > > non-direct memory usage of user code. The user code can be run e.g. in
> > > certain job submission scenarios within the JM process. For simplicity,
> > > FLIP suggests only an option for direct memory which is translated into
> > the
> > > setting of the JVM direct memory limit.
> > > Although, we documented for TM that the similar parameters can also
> > > address native non-direct memory usage [3], this can lead to wrong
> > > functioning of the JVM direct memory limit. The direct memory option in
> > JM
> > > could be also named in more general way, e.g. off-heap memory but this
> > > naming would somewhat hide its nature of JVM direct memory limit.
> > > On the other hand, JVM Overhead does not suffer from this problem and
> > > affects only the container/worker memory size which is the most
> important
> > > matter to address for the native non-direct memory consumption. The
> > caveat
> > > here is that JVM Overhead was not supposed to be used by any Flink or
> > user
> > > components.
> > >
> > > Thanks,
> > > Andrey
> > >
> > > [1]
> > >
> > >
> >
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-49%3A+Unified+Memory+Configuration+for+TaskExecutors
> > > [2]
> > >
> > >
> >
> https://cwiki.apache.org/confluence/display/FLINK/FLIP+116%3A+Unified+Memory+Configuration+for+Job+Managers
> > > [3]
> > >
> > >
> >
> https://ci.apache.org/projects/flink/flink-docs-release-1.10/ops/memory/mem_detail.html#overview
> > >
> >
>


[VOTE] FLIP 116: Unified Memory Configuration for Job Managers

2020-03-18 Thread Andrey Zagrebin
Hi All,

The discussion for FLIP-116 looks to be resolved [1].
Therefore, I start the vote for it.
The vote will end at 6pm CET on Monday, 23 March.

Best,
Andrey

[1]
http://mail-archives.apache.org/mod_mbox/flink-dev/202003.mbox/%3CCAJNyZN7AJAU_RUVhnWa7r%2B%2BtXpmUqWFH%2BG0hfoLVBzgRMmAO2w%40mail.gmail.com%3E


Re: [DISCUSS] FLIP 116: Unified Memory Configuration for Job Managers

2020-03-18 Thread Andrey Zagrebin
Hi all,

One thing more thing to mention, the current calculations can lead to
arbitrary small JVM Heap, maybe even zero.
I suggest to introduce a check where we at least recommend to set the JVM
heap to e.g. 128Mb.

Additionally, we can demand some minimum value to function and fail if it
is not fulfilled.
We could experiment with what is the working minimum but It is hard to come
up with this limit because it again can depend on the job and environment.

Best,
Andrey

On Wed, Mar 18, 2020 at 5:03 PM Andrey Zagrebin 
wrote:

> Hi all,
>
> Thanks for the feedback, Xintong and Till.
>
> > rename jobmanager.memory.direct.size into jobmanager.memory.off-heap.size
>
> I am ok with that to align it with TM and avoid further complications for
> users.
> I will adjust the FLIP.
>
> > change the default value of JM Metaspace size to 256 MB
>
> Indeed, no reason to assume that the user code would need less Metaspace
> in JM.
> I will change it unless a better argument is reported for another value.
>
> I think all concerns has been resolved so I am starting the voting in a
> separate thread.
>
> Best,
> Andrey
>
> On Tue, Mar 17, 2020 at 6:16 PM Till Rohrmann 
> wrote:
>
>> Thanks for creating this FLIP Andrey.
>>
>> I agree with Xintong that we should rename jobmanager.memory.direct.size
>> into jobmanager.memory.off-heap.size which accounts for native and direct
>> memory usage. I think it should be good enough and is easier to understand
>> for the user.
>>
>> Concerning the default value for the metaspace size. Did we take the
>> lessons learned from the TM metaspace size into account? IIRC we are about
>> to change the default value to 256 MB.
>>
>> Feel free to start a vote once these last two questions have been
>> resolved.
>>
>> Cheers,
>> Till
>>
>> On Thu, Mar 12, 2020 at 4:25 AM Xintong Song 
>> wrote:
>>
>> > Thanks Andrey for kicking this discussion off.
>> >
>> > Regarding "direct" vs. "off-heap", I'm personally in favor of renaming
>> the
>> > "direct" memory in the current FLIP-116[1] to "off-heap" memory, and
>> making
>> > it also account for user native memory usage.
>> >
>> > On one hand, I think it would be good that JM & TM provide consistent
>> > concepts and terminologies to users. IIUC, this is exactly the purpose
>> of
>> > this FLIP. For TMs, we already have "off-heap" memory accounting for
>> both
>> > direct and native memory usages, and we did this so that users do not
>> need
>> > to understand the differences between the two kinds.
>> >
>> > On the other hand, while for TMs it is hard to tell which kind of
>> memory is
>> > needed mostly due to variety of applications, I believe for JM the major
>> > memory consumption is heap memory in most cases. That means we probably
>> can
>> > rely on the heap activities to trigger GC in most cases, and the max
>> direct
>> > memory limit can act as a safe net. Moreover, I think the cases should
>> be
>> > very rare that we need native memory for user codes. Therefore, we
>> probably
>> > should not break the JM/TM consistency for potential risks in such rare
>> > cases.
>> >
>> > WDYT?
>> >
>> > Thank you~
>> >
>> > Xintong Song
>> >
>> >
>> > [1]
>> >
>> >
>> https://cwiki.apache.org/confluence/display/FLINK/FLIP+116%3A+Unified+Memory+Configuration+for+Job+Managers
>> >
>> > On Wed, Mar 11, 2020 at 8:56 PM Andrey Zagrebin 
>> > wrote:
>> >
>> > > Hi All,
>> > >
>> > > As you may have noticed, 1.10 release included an extensive
>> improvements
>> > to
>> > > memory management and configuration of Task Managers, FLIP-49: [1].
>> The
>> > > memory configuration of Job Managers has not been touched in 1.10.
>> > >
>> > > Although, Job Manager's memory model does not look so sophisticated as
>> > > for Task Managers, It makes to align Job Manager memory model and
>> > settings
>> > > with Task Managers. Therefore, we propose to reconsider it as well in
>> > 1.11
>> > > and I prepared a FLIP 116 [2] for that.
>> > >
>> > > Any feedback is appreciated.
>> > >
>> > > So far, there is one discussion point about how to address native
>> > > non-direct memory usage of user code. The user code

Re: [DISCUSS] FLIP 116: Unified Memory Configuration for Job Managers

2020-03-19 Thread Andrey Zagrebin
Alright, thanks for the feedback. I also agree with it. Then this is resolved.

> On 19 Mar 2020, at 14:14, Till Rohrmann  wrote:
> 
> I agree with Xintong's proposal. If we see that many users run into this
> problem, then one could think about escalating the warning message into a
> failure.
> 
> Cheers,
> Till
> 
> On Thu, Mar 19, 2020 at 4:23 AM Xintong Song  wrote:
> 
>> I think recommend a minimum value in docs and throw a warning if the heap
>> size is too small should be good enough.
>> Not sure about failing job if the min heap is not fulfilled. As already
>> mentioned, it would be hard to determine the min heap size. And if we make
>> the min heap configurable, then in any case that users need to configure
>> the min heap, they can configure the heap size directly.
>> 
>> Thank you~
>> 
>> Xintong Song
>> 
>> 
>> 
>> On Wed, Mar 18, 2020 at 10:55 PM Andrey Zagrebin 
>> wrote:
>> 
>>> Hi all,
>>> 
>>> One thing more thing to mention, the current calculations can lead to
>>> arbitrary small JVM Heap, maybe even zero.
>>> I suggest to introduce a check where we at least recommend to set the JVM
>>> heap to e.g. 128Mb.
>>> 
>>> Additionally, we can demand some minimum value to function and fail if it
>>> is not fulfilled.
>>> We could experiment with what is the working minimum but It is hard to
>> come
>>> up with this limit because it again can depend on the job and
>> environment.
>>> 
>>> Best,
>>> Andrey
>>> 
>>> On Wed, Mar 18, 2020 at 5:03 PM Andrey Zagrebin 
>>> wrote:
>>> 
>>>> Hi all,
>>>> 
>>>> Thanks for the feedback, Xintong and Till.
>>>> 
>>>>> rename jobmanager.memory.direct.size into
>>> jobmanager.memory.off-heap.size
>>>> 
>>>> I am ok with that to align it with TM and avoid further complications
>> for
>>>> users.
>>>> I will adjust the FLIP.
>>>> 
>>>>> change the default value of JM Metaspace size to 256 MB
>>>> 
>>>> Indeed, no reason to assume that the user code would need less
>> Metaspace
>>>> in JM.
>>>> I will change it unless a better argument is reported for another
>> value.
>>>> 
>>>> I think all concerns has been resolved so I am starting the voting in a
>>>> separate thread.
>>>> 
>>>> Best,
>>>> Andrey
>>>> 
>>>> On Tue, Mar 17, 2020 at 6:16 PM Till Rohrmann 
>>>> wrote:
>>>> 
>>>>> Thanks for creating this FLIP Andrey.
>>>>> 
>>>>> I agree with Xintong that we should rename
>> jobmanager.memory.direct.size
>>>>> into jobmanager.memory.off-heap.size which accounts for native and
>>> direct
>>>>> memory usage. I think it should be good enough and is easier to
>>> understand
>>>>> for the user.
>>>>> 
>>>>> Concerning the default value for the metaspace size. Did we take the
>>>>> lessons learned from the TM metaspace size into account? IIRC we are
>>> about
>>>>> to change the default value to 256 MB.
>>>>> 
>>>>> Feel free to start a vote once these last two questions have been
>>>>> resolved.
>>>>> 
>>>>> Cheers,
>>>>> Till
>>>>> 
>>>>> On Thu, Mar 12, 2020 at 4:25 AM Xintong Song 
>>>>> wrote:
>>>>> 
>>>>>> Thanks Andrey for kicking this discussion off.
>>>>>> 
>>>>>> Regarding "direct" vs. "off-heap", I'm personally in favor of
>> renaming
>>>>> the
>>>>>> "direct" memory in the current FLIP-116[1] to "off-heap" memory, and
>>>>> making
>>>>>> it also account for user native memory usage.
>>>>>> 
>>>>>> On one hand, I think it would be good that JM & TM provide
>> consistent
>>>>>> concepts and terminologies to users. IIUC, this is exactly the
>> purpose
>>>>> of
>>>>>> this FLIP. For TMs, we already have "off-heap" memory accounting for
>>>>> both
>>>>>> direct and native memory usages, and we did this so that users do
>> not
>>>>> need
>>>>>> to u

Re: [DISCUSS] FLIP-111: Docker image unification

2020-03-22 Thread Andrey Zagrebin
uperceded by the env variables approach. Otherwise it
> will only add more maintenance burden.
>
> Long story short, with the existing configuration options (envsubts,
> FLINK_PROPERTIES) we can already configure the Flink process and Flink
> itself. Since maintaining backwards compatibility is important, we could
> rely on these mechanisms until we have proper env variable configuration
> and don't have to introduce a new way to change the configuration.
>
> # Logging & Stdout/err
>
> ## Logging
>
> I think Konstantin is right and we should provide a log4j.properties file
> which, per default, specifies the file and console appender. We could add a
> special log4j.properties file to apache/flink-docker which we include in
> the Dockerfile.
>
> This approach will give users the most flexibility w/o relying on magic
> (e.g. tailing the log files after starting the process in the background).
>
> ## Stdout/err
>
> I think for printing the stdout/err output to STDOUT/ERR and to capture it
> in a file there are solutions. For example, one could use `program &2>1 |
> tee flink-user-taskexecutor.out` to achieve this.
>
> # Java version
>
> I agree that it would be nice to also offer a Java 11 Dockerfile. For the
> sake of limiting the scope of this proposal I would suggest to do this as a
> follow up issue.
>
> # Dev version
>
> Tooling to create a Docker image from the current Flink repository is
> indeed very nice for development. As Andrey suggested, I think this would
> be a good follow up for this proposal. I don't think that Andrey's current
> proposal would block any future developments in this direction.
>
> # Scripts
>
> At the moment, I would be in favour of placing the Dockerfile scripts
> under apache/flink-docker since they belong more to the Dockerfile than to
> Flink's binary distribution. If we see that we might be able to reuse them
> for the developer Dockerfile, then we can still move them to the Flink
> repository.
>
> I would refrain from offering special commands to set individual
> configuration options (e.g., flink_docker_utils set_web_ui_port 8081). It
> should be fine enough to do it via flink_docker-utils conifgure rest.port
> 8081 if we cannot solve it via the general configuration mechanism.
>
> Cheers,
> Till
>
> On Wed, Mar 18, 2020 at 6:38 AM Yangze Guo  wrote:
>
>> I second Thomas that we can support both Java 8 and 11.
>>
>> Best,
>> Yangze Guo
>>
>> On Wed, Mar 18, 2020 at 12:12 PM Thomas Weise  wrote:
>> >
>> > -->
>> >
>> > On Mon, Mar 16, 2020 at 1:58 AM Andrey Zagrebin 
>> wrote:
>> >>
>> >> Thanks for the further feedback Thomas and Yangze.
>> >>
>> >> > A generic, dynamic configuration mechanism based on environment
>> variables
>> >> is essential and it is already supported via envsubst and an
>> environment
>> >> variable that can supply a configuration fragment
>> >>
>> >> True, we already have this. As I understand this was introduced for
>> >> flexibility to template a custom flink-conf.yaml with env vars, put it
>> into
>> >> the FLINK_PROPERTIES and merge it with the default one.
>> >> Could we achieve the same with the dynamic properties
>> (-Drpc.port=1234),
>> >> passed as image args to run it, instead of FLINK_PROPERTIES?
>> >> They could be also parametrised with env vars. This would require
>> >> jobmanager.sh to properly propagate them to
>> >> the StandaloneSessionClusterEntrypoint though:
>> >>
>> https://github.com/docker-flink/docker-flink/pull/82#issuecomment-525285552
>> >> cc @Till
>> >> This would provide a unified configuration approach.
>> >>
>> >
>> > How would that look like for the various use cases? The k8s operator
>> would need to generate the -Dabc .. -Dxyz entry point command instead of
>> setting the FLINK_PROPERTIES environment variable? Potentially that
>> introduces additional complexity for little gain. Do most deployment
>> platforms that support Docker containers handle the command line route
>> well? Backward compatibility may also be a concern.
>> >
>> >>
>> >> > On the flip side, attempting to support a fixed subset of
>> configuration
>> >> options is brittle and will probably lead to compatibility issues down
>> the
>> >> road
>> >>
>> >> I agree with it. The idea was to have just some shortcut scripted
>> functions
>> >> to set options in f

Re: [VOTE] FLIP 116: Unified Memory Configuration for Job Managers

2020-03-25 Thread Andrey Zagrebin
Thanks everybody for the voting.
I also vote
+1 (binding)

Hereby the vote is closed and the FLIP-116 is accepted

3 binding votes:
@Till Rohrmann 
@g...@apache.org 
@azagre...@apache.org  (me)

2 non-binding votes:
@Xintong Song 
@Yang Wang 

no vetos/-1s

Best,
Andrey

On Wed, Mar 25, 2020 at 6:16 PM Gary Yao  wrote:

> +1 (binding)
>
> Best,
> Gary
>
> On Wed, Mar 18, 2020 at 3:16 PM Andrey Zagrebin 
> wrote:
>
> > Hi All,
> >
> > The discussion for FLIP-116 looks to be resolved [1].
> > Therefore, I start the vote for it.
> > The vote will end at 6pm CET on Monday, 23 March.
> >
> > Best,
> > Andrey
> >
> > [1]
> >
> >
> http://mail-archives.apache.org/mod_mbox/flink-dev/202003.mbox/%3CCAJNyZN7AJAU_RUVhnWa7r%2B%2BtXpmUqWFH%2BG0hfoLVBzgRMmAO2w%40mail.gmail.com%3E
> >
>


Re: [VOTE] FLIP-102: Add More Metrics to TaskManager

2020-03-30 Thread Andrey Zagrebin
Hi All,

Thanks for this FLIP, Yadong. This is a very good improvement to the
Flink's UI.
It looks like there are still couple of things to resolve before the final
vote.

- I also find the non-heap title in configuration confusing because there
are also other non-heap types of memory. The "off-heap" concept is quite
broad.
What about "JVM specific" meaning that it is not coming directly from Flink?
or we could remove the "Non-heap" box at all and show directly JVM
Metaspace and Overhead as separate boxes,
this would also fit if we decide to keep the Metaspace metric.

- Total Process Memory Used: I agree with Xintong, it is hard to say what
is used there.
Then the size of "Total Process Memory" basically becomes part of
configuration.

- Non-Heap Used/Max/.. Not sure what committed means here. I also think we
should either exclude it or display what is known for sure.
In general, the metaspace usage would be nice to have but it should be then
exactly metaspace usage without any thing else.

- I do not know how the mapped memory works. Is it meant for the new
spilled partitions? If the mapped memory also pulls from the direct
memory limit
then this is something we do not account in our network buffers as I
understand. In this case, this metric may be useful for tuning to understand
how much the mapped memory uses from the direct memory limit to set e.g.
framework off-heap limit correctly and avoid direct OOM.
It could be something to discuss with Zhijiang. e.g. is the direct
memory used there to buffer fetched regions of partition files or what for?

- Not sure, we need an extra wrapping box "other" for the managed memory
atm. I could be just "Managed" or "Managed by Flink".

Best,
Andrey

On Fri, Mar 27, 2020 at 6:13 AM Xintong Song  wrote:

> Sorry for the late response.
>
> I have shared my suggestions with Yadong & Lining offline. I think it would
> be better to also post them here, for the public record.
>
>- I'm not sure about displaying Total Process Memory Used. Currently, we
>do not have a good way to monitor all memory footprints of the process.
>Metrics for some native memory usages (e.g., thread stack) are absent.
>Displaying a partial used memory size could be confusing for users.
>- I would suggest merge the current Mapped Memory metrics into Direct
>Memory. Actually, the metrics are retrieved from MXBeans for direct
> buffer
>pool and mapped buffer pool. Both of the two pools are accounted for in
>-XX:MaxDirectMemorySize. There's no Flink configuration that can modify
> the
>individual pool sizes. Therefore, I think displaying the total Direct
>Memory would be good enough. Moreover, in most use cases the size of
> mapped
>buffer pool is zero and users do not need to understand what is Mapped
>Memory. For expert users who do need the separated metrics for
> individual
>pools, they can subscribe the metrics on their own.
>- I would suggest to not display Non-Heap Memory. Despite the name, the
>metrics (also retrieved from MXBeans) actually accounts for metaspace,
> code
>cache, and compressed class space. It does not account for all JVM
> native
>memory overheads, e.g., thread stack. That means the metrics of Non-Heap
>Memory do not well correspond to any of the FLIP-49 memory components.
> They
>account for Flink's JVM Metaspace and part of JVM Overhead. I think this
>brings more confusion then help to users, especially primary users.
>
>
> Thank you~
>
> Xintong Song
>
>
>
> On Thu, Mar 26, 2020 at 6:34 PM Till Rohrmann 
> wrote:
>
> > Thanks for updating the FLIP Yadong.
> >
> > What is the difference between managedMemory and managedMemoryTotal
> > and networkMemory and networkMemoryTotal in the REST response? If they
> are
> > duplicates, then we might be able to remove one.
> >
> > Apart from that, the proposal looks good to me.
> >
> > Pulling also Andrey in to hear his opinion about the representation of
> the
> > memory components.
> >
> > Cheers,
> > Till
> >
> > On Thu, Mar 19, 2020 at 11:37 AM Yadong Xie  wrote:
> >
> >> Hi all
> >>
> >> I have updated the design of the metric page and FLIP doc, please let me
> >> know what you think about it
> >>
> >> FLIP-102:
> >>
> >>
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-102%3A+Add+More+Metrics+to+TaskManager
> >> POC web:
> >>
> >>
> http://101.132.122.69:8081/web/#/task-manager/8e1f1beada3859ee8e46d0960bb1da18/metrics
> >>
> >> Till Rohrmann  于2020年2月27日周四 下午10:27写道:
> >>
> >> > Thinking a bit more about the problem whether to report the aggregated
> >> > memory statistics or the individual slot statistics, I think reporting
> >> it
> >> > on a per slot basis won't work nicely together with FLIP-56 (dynamic
> >> slot
> >> > allocation). The problem is that with FLIP-56, we will no longer have
> >> > dedicated slots. The number of slots might change over the lifetime
> of a
> >> > TaskExecutor. Hence, it won't be easy to generate a metric path f

Re: Adding a new "Docker Images" component to Jira

2020-08-08 Thread Andrey Zagrebin
+1 for the consolidation

Best,
Andrey

On Fri, Aug 7, 2020 at 3:38 PM Till Rohrmann  wrote:

> +1 for unifying Deployment / Docker, Dockerfiles and Release System /
> Docker into Docker.
>
> Cheers,
> Till
>
> On Fri, Aug 7, 2020 at 12:18 PM Robert Metzger 
> wrote:
>
> > Hi all,
> >
> > we now have 3 components containing the word "docker":
> > - Deployment / Docker
> > <
> >
> https://issues.apache.org/jira/issues/?jql=project+%3D+FLINK+AND+component+%3D+%22Deployment+%2F+Docker%22
> > >
> > (63
> > issues)
> > - Dockerfiles
> > <
> >
> https://issues.apache.org/jira/issues/?jql=project+%3D+FLINK+AND+component+%3D+Dockerfiles
> > >
> > (12
> > issues)
> > - Release System / Docker
> > <
> >
> https://issues.apache.org/jira/issues/?jql=project+%3D+FLINK+AND+component+%3D+%22Release+System+%2F+Docker%22
> > >
> > (3
> > issues)
> >
> > I would suggest consolidating these three components into one, as there
> are
> > not that many tickets for this aspect of Flink.
> > Maybe we should just rename "Deployment / Docker" to "flink-docker", and
> > merge the two other components into it?
> >
> >
> > On Fri, Feb 21, 2020 at 11:47 AM Patrick Lucas 
> > wrote:
> >
> > > Thanks, Chesnay!
> > >
> > > On Fri, Feb 21, 2020 at 11:26 AM Chesnay Schepler 
> > > wrote:
> > >
> > > > I've added a "Release System / Docker" component.
> > > >
> > > > On 21/02/2020 11:19, Patrick Lucas wrote:
> > > > > Hi,
> > > > >
> > > > > Could someone with permissions add a new component to the FLINK
> > project
> > > > in
> > > > > Jira for the Docker images  >?
> > > > >
> > > > > There is already a "Deployment / Docker" component, but that's not
> > > quite
> > > > > the same as maintenance/improvements on the flink-docker images.
> > > > >
> > > > > Either top-level "Docker Images" or perhaps "Release / Docker
> Images"
> > > > would
> > > > > be fine.
> > > > >
> > > > > Thanks,
> > > > > Patrick
> > > > >
> > > >
> > > >
> > >
> >
>


Re: [VOTE] FLIP-102: Add More Metrics to TaskManager

2020-08-20 Thread Andrey Zagrebin
Hi All,

Thanks for reviving the discussion, Matthias!

This would mean that we could adapt the current proposal to replace the
> Nonheap usage pane by a pane displaying the Metaspace usage.
>
I do not know the value of having the Nonheap usage in metrics. I can see
that the metaspace metric can be interesting for the users to debug OOMs.
We had the Nonheap usage before, so as discussed, I would be a bit careful
removing. I believe it deserves a separate poll in user ML
whether the Nonheap usage is useless or not.
As a current solution, we could keep both or merge them into one box with a
slash, like Metaspace/Nonheap -> 5Mb/10Mb, if the majority agrees that this
is not confusing and clear that the metaspace is a part of Nonheap.

Btw, the "Nonheap" in the configuration box of the current FLIP-102 is
probably incorrect or confusing as it does not one-to-one correspond to the
Nonheap JVM metric.

The only issue I see is that JVM Overhead would still not be represented in
> the memory usage
> overview.

My understanding is that we do not need a usage metric for JVM Overhead as
it is a virtual unmanaged component which is more about configuring the max
total process memory.

Is there a reason for us to introduce a nested structure
> TaskManagerMetricsInfo in the response object? I would rather keep it
> consistent in a flat structure instead, i.e. having all the members of
> TaskManagerResourceInfo being members of TaskManagerMetricsInfo

I would suggest introducing a separate REST call for
TaskManagerResourceInfo.
Semantically, TaskManagerResourceInfo is more about the TM configuration
and it is not directly related to the usage metrics.
In future, I would avoid having calls with many responsibilities and maybe
consider splitting the 'TM details' call into metrics etc unless there is a
concern for having to do more calls instead of one from UI.

Alternatively, one could think of grouping the metrics collecting the
> different values (i.e. max, used, committed) per metric in a JSON object.
> But this would apply for all the other metrics of TaskManagerMetricsInfo
> as
> well.

I would personally prefer this for metrics but I am not pushing for this.

metrics.resource.managedMemory and metrics.resource.networkMemory have
> counterparts in metrics.networkMemory[Used|Total] and
> metrics.managedMemory[Used|Total]: Is this redundant data or do they have
> different semantics?

As I understand, they have different semantics. The later is about
configuration, the former is about current usage metrics.

Is metrics.resource.totalProcessMemory a basic sum over all provided
> values?

this is again about configuration, I do not think it makes sense to come up
with a usage metric for the totalProcessMemory component.

Best,
Andrey


On Thu, Aug 20, 2020 at 9:06 AM Matthias  wrote:

> Hi Jing,
> I recently joined Ververica and started looking into FLIP-102. I'm trying
> to
> figure out how we would implement the proposal on the backend side.
> I looked into the proposal for the REST API response and a few questions
> popped up:
> - Is there a reason for us to introduce a nested structure
> TaskManagerMetricsInfo in the response object? I would rather keep it
> consistent in a flat structure instead, i.e. having all the members of
> TaskManagerResourceInfo being members of TaskManagerMetricsInfo.
>   Alternatively, one could think of grouping the metrics collecting the
> different values (i.e. max, used, committed) per metric in a JSON object.
> But this would apply for all the other metrics of TaskManagerMetricsInfo as
> well.
> - metrics.resource.managedMemory and metrics.resource.networkMemory have
> counterparts in metrics.networkMemory[Used|Total] and
> metrics.managedMemory[Used|Total]: Is this redundant data or do they have
> different semantics?
> - Is metrics.resource.totalProcessMemory a basic sum over all provided
> values? I see the necessity to have this member if we decide to not provide
> the memory usage for all memory pools (e.g. providing Metaspace but leaving
> Code Cache and Compressed Class Space as Non-Heap pools out of the
> response). Otherwise, would it be worth it to remove this member from the
> response for simplicity reasons since we could sum up the memory on the
> frontend side?
>
> Best,
> Matthias
>
>
>
> --
> Sent from: http://apache-flink-mailing-list-archive.1008284.n3.nabble.com/
>


Re: How jobmanager and task manager communicates with each other ?

2020-08-25 Thread Andrey Zagrebin
Hi Sidhant,

(1) If we are not using Flink's HA services then how we can dynamically
> configure task manager nodes to connect to job manager? Any suggestions or
> best practices?

Not sure what you mean by 'dynamically'.
I think you have to restart the task manager with the new configuration
to connect to another job manager.

(2) Which and how flink's HA service can be used for the service discovery
> of job manager ?

You can check the docs for the zookeeper implementation of the HA in Flink
[1]

Best,
Andrey

[1]
https://ci.apache.org/projects/flink/flink-docs-release-1.11/ops/jobmanager_high_availability.html

On Tue, Aug 25, 2020 at 5:45 PM sidhant gupta  wrote:

> Hi Till,
>
> Thanks for the reply.
>
> (1) If we are not using Flink's HA services then how we can dynamically
> configure task manager nodes to connect to job manager? Any suggestions or
> best practices?
>
> (2) Which and how flink's HA service can be used for the service discovery
> of job manager ?
>
> Regards
> Sidhant Gupta
>
>
> On Tue, Aug 25, 2020, 11:51 AM Till Rohrmann  wrote:
>
>> Hi Sidhant,
>>
>> the cluster components use tcp to communicate with each other. If you are
>> not using Flink's HA services, then the TaskManager nodes need to be
>> configured with the JobManager's address to connect to them. If you are
>> using HA services, then the service discovery happens through the HA
>> services. One requirement for Flink to work is that the different cluster
>> nodes on which a Flink process is started can communicate with each other.
>>
>> Cheers,
>> Till
>>
>> On Mon, Aug 24, 2020 at 6:26 PM sidhant gupta 
>> wrote:
>>
>>> ++dev@flink.apache.org
>>>
>>> On Mon, Aug 24, 2020, 7:31 PM sidhant gupta  wrote:
>>>
>>> > Hi User
>>> >
>>> > How jobmanager and task manager communicates with each other ? How to
>>> set
>>> > connection between jobmanager and task manager running in
>>> different/same
>>> > ec2 instance ? Is it http or tcp ? How the service discovery works ?
>>> >
>>> > Thanks
>>> > Sidhant Gupta
>>> >
>>>
>>


Re: [VOTE] FLIP-141: Intra-Slot Managed Memory Sharing

2020-09-09 Thread Andrey Zagrebin
+1

Best,
Andrey

On Tue, Sep 8, 2020 at 2:16 PM Yu Li  wrote:

> +1
>
> Best Regards,
> Yu
>
>
> On Tue, 8 Sep 2020 at 17:03, Aljoscha Krettek  wrote:
>
> > +1
> >
> > We just need to make sure to find a good name before the release but
> > shouldn't block any work on this.
> >
> > Aljoscha
> >
> > On 08.09.20 07:59, Xintong Song wrote:
> > > Thanks for the vote, @Jincheng.
> > >
> > >
> > > Concerning the namings, the original idea was, as you suggested, to
> have
> > > separate configuration names for batch and rocksdb while only one of
> them
> > > will take effect at a time.
> > >
> > >
> > > It was then in the discussion thread [1] that @Stepahn suggested to
> > combine
> > > these two.
> > >
> > >>  We never have batch algos and RocksDB mixed, having this as
> > separate
> > >> options is confusing as it suggests this can be combined arbitrarily.
> I
> > >> also think that a slim possibility that we may ever combine this in
> the
> > >> future is not enough reason to make it more complex/confusing.
> > >>
> > >
> > > This suggestion was also supported by others in the discussion thread.
> > > That's why we are trying to come up with a name that covers both batch
> > and
> > > rocksdb memory consumers.
> > >
> > >
> > > Thank you~
> > >
> > > Xintong Song
> > >
> > >
> > > [1]
> > >
> >
> http://apache-flink-mailing-list-archive.1008284.n3.nabble.com/DISCUSS-FLIP-141-Intra-Slot-Managed-Memory-Sharing-tp44146p44253.html
> > >
> > > On Tue, Sep 8, 2020 at 1:37 PM jincheng sun 
> > > wrote:
> > >
> > >> +1 for the proposal!
> > >>
> > >> Regarding the name of `BATCH_OP/ROCKSDB`, we can reserve the
> > configuration
> > >> names for batch and rocksdb respectively, ` batch_ OP` for batch,
> > "ROCKSDB"
> > >> for roockdb. and the default value as follows:
> > >>
> > >> {
> > >>  BATCH_OP: 70,
> > >>  ROCKSDB : 70,
> > >>  PYTHON : 30
> > >> }
> > >>
> > >> Only one of `BATCH_ OP` and `ROCKSDB` will work. What do you think?
> > >>
> > >> Best,
> > >> Jincheng
> > >>
> > >>
> > >> Xintong Song  于2020年9月7日周一 下午1:46写道:
> > >>
> > >>> Thanks for the votes.
> > >>>
> > >>> Concerning the name for batch/RocksDB memory consumer, how about
> > >> "execution
> > >>> memory"?
> > >>> We can further explain in docs and config option description that
> this
> > is
> > >>> used for job execution, which is currently dedicated to rocksdb in
> > >>> streaming and batch algorithms in batch.
> > >>>
> > >>> Thank you~
> > >>>
> > >>> Xintong Song
> > >>>
> > >>>
> > >>>
> > >>> On Mon, Sep 7, 2020 at 11:43 AM Yangze Guo 
> wrote:
> > >>>
> >  +1
> > 
> >  Best,
> >  Yangze Guo
> > 
> >  On Mon, Sep 7, 2020 at 10:54 AM Zhu Zhu  wrote:
> > >
> > > +1
> > >
> > > Thanks,
> > > Zhu
> > >
> > > Dian Fu  于2020年9月7日周一 上午10:34写道:
> > >
> > >> +1
> > >>
> > >>> 在 2020年9月3日,下午8:46,Till Rohrmann  写道:
> > >>>
> > >>> Hi Xintong,
> > >>>
> > >>> thanks for starting the vote.
> > >>>
> > >>> +1 for the proposal given that we find a proper name for the
> > >>> different memory consumers (specifically the batch/RocksDB
> > >>> consumer)
> >  and
> > >>> their corresponding weights.
> > >>>
> > >>> Cheers,
> > >>> Till
> > >>>
> > >>> On Thu, Sep 3, 2020 at 12:43 PM Xintong Song <
> > >>> tonysong...@gmail.com>
> > >> wrote:
> > >>>
> >  Hi devs,
> > 
> >  I'd like to start a voting thread on FLIP-141[1], which proposes
> > >>> how
> >  managed memory should be shared by various use cases within a
> > >>> slot.
> >  The
> >  proposal has been discussed in [2].
> > 
> >  The vote will be open for at least 72h + weekends. I'll try to
> >  close it
> > >> on
> >  September 8, unless there is an objection or not enough votes.
> > 
> >  Thank you~
> > 
> >  Xintong Song
> > 
> > 
> >  [1]
> > 
> > 
> > >>
> > 
> > >>>
> > >>
> >
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-141%3A+Intra-Slot+Managed+Memory+Sharing#FLIP141:IntraSlotManagedMemorySharing-compatibility
> > 
> >  [2]
> > 
> > 
> > >>
> > 
> > >>>
> > >>
> >
> http://apache-flink-mailing-list-archive.1008284.n3.nabble.com/DISCUSS-FLIP-141-Intra-Slot-Managed-Memory-Sharing-td44146.html
> > 
> > >>
> > >>
> > 
> > >>>
> > >>
> > >
> >
> >
>


Re: [VOTE] FLIP-141: Intra-Slot Managed Memory Sharing

2020-09-09 Thread Andrey Zagrebin
For the option name, maybe:
*flink.main*
or
*flink.managed* (this may be a bit confusing for existing users as we said
that the overall managed memory is managed by Flink)

On Wed, Sep 9, 2020 at 9:56 AM Andrey Zagrebin  wrote:

> +1
>
> Best,
> Andrey
>
> On Tue, Sep 8, 2020 at 2:16 PM Yu Li  wrote:
>
>> +1
>>
>> Best Regards,
>> Yu
>>
>>
>> On Tue, 8 Sep 2020 at 17:03, Aljoscha Krettek 
>> wrote:
>>
>> > +1
>> >
>> > We just need to make sure to find a good name before the release but
>> > shouldn't block any work on this.
>> >
>> > Aljoscha
>> >
>> > On 08.09.20 07:59, Xintong Song wrote:
>> > > Thanks for the vote, @Jincheng.
>> > >
>> > >
>> > > Concerning the namings, the original idea was, as you suggested, to
>> have
>> > > separate configuration names for batch and rocksdb while only one of
>> them
>> > > will take effect at a time.
>> > >
>> > >
>> > > It was then in the discussion thread [1] that @Stepahn suggested to
>> > combine
>> > > these two.
>> > >
>> > >>  We never have batch algos and RocksDB mixed, having this as
>> > separate
>> > >> options is confusing as it suggests this can be combined
>> arbitrarily. I
>> > >> also think that a slim possibility that we may ever combine this in
>> the
>> > >> future is not enough reason to make it more complex/confusing.
>> > >>
>> > >
>> > > This suggestion was also supported by others in the discussion thread.
>> > > That's why we are trying to come up with a name that covers both batch
>> > and
>> > > rocksdb memory consumers.
>> > >
>> > >
>> > > Thank you~
>> > >
>> > > Xintong Song
>> > >
>> > >
>> > > [1]
>> > >
>> >
>> http://apache-flink-mailing-list-archive.1008284.n3.nabble.com/DISCUSS-FLIP-141-Intra-Slot-Managed-Memory-Sharing-tp44146p44253.html
>> > >
>> > > On Tue, Sep 8, 2020 at 1:37 PM jincheng sun > >
>> > > wrote:
>> > >
>> > >> +1 for the proposal!
>> > >>
>> > >> Regarding the name of `BATCH_OP/ROCKSDB`, we can reserve the
>> > configuration
>> > >> names for batch and rocksdb respectively, ` batch_ OP` for batch,
>> > "ROCKSDB"
>> > >> for roockdb. and the default value as follows:
>> > >>
>> > >> {
>> > >>  BATCH_OP: 70,
>> > >>  ROCKSDB : 70,
>> > >>  PYTHON : 30
>> > >> }
>> > >>
>> > >> Only one of `BATCH_ OP` and `ROCKSDB` will work. What do you think?
>> > >>
>> > >> Best,
>> > >> Jincheng
>> > >>
>> > >>
>> > >> Xintong Song  于2020年9月7日周一 下午1:46写道:
>> > >>
>> > >>> Thanks for the votes.
>> > >>>
>> > >>> Concerning the name for batch/RocksDB memory consumer, how about
>> > >> "execution
>> > >>> memory"?
>> > >>> We can further explain in docs and config option description that
>> this
>> > is
>> > >>> used for job execution, which is currently dedicated to rocksdb in
>> > >>> streaming and batch algorithms in batch.
>> > >>>
>> > >>> Thank you~
>> > >>>
>> > >>> Xintong Song
>> > >>>
>> > >>>
>> > >>>
>> > >>> On Mon, Sep 7, 2020 at 11:43 AM Yangze Guo 
>> wrote:
>> > >>>
>> > >>>> +1
>> > >>>>
>> > >>>> Best,
>> > >>>> Yangze Guo
>> > >>>>
>> > >>>> On Mon, Sep 7, 2020 at 10:54 AM Zhu Zhu  wrote:
>> > >>>>>
>> > >>>>> +1
>> > >>>>>
>> > >>>>> Thanks,
>> > >>>>> Zhu
>> > >>>>>
>> > >>>>> Dian Fu  于2020年9月7日周一 上午10:34写道:
>> > >>>>>
>> > >>>>>> +1
>> > >>>>>>
>> > >>>>>>> 在 2020年9月3日,下午8:46,Till Rohrmann  写道:
>> > >>>>>>>
>> > >>>>>>> Hi Xintong,
>> > >>>>>>>
>> > >>>>>>> thanks for starting the vote.
>> > >>>>>>>
>> > >>>>>>> +1 for the proposal given that we find a proper name for the
>> > >>>>>>> different memory consumers (specifically the batch/RocksDB
>> > >>> consumer)
>> > >>>> and
>> > >>>>>>> their corresponding weights.
>> > >>>>>>>
>> > >>>>>>> Cheers,
>> > >>>>>>> Till
>> > >>>>>>>
>> > >>>>>>> On Thu, Sep 3, 2020 at 12:43 PM Xintong Song <
>> > >>> tonysong...@gmail.com>
>> > >>>>>> wrote:
>> > >>>>>>>
>> > >>>>>>>> Hi devs,
>> > >>>>>>>>
>> > >>>>>>>> I'd like to start a voting thread on FLIP-141[1], which
>> proposes
>> > >>> how
>> > >>>>>>>> managed memory should be shared by various use cases within a
>> > >>> slot.
>> > >>>> The
>> > >>>>>>>> proposal has been discussed in [2].
>> > >>>>>>>>
>> > >>>>>>>> The vote will be open for at least 72h + weekends. I'll try to
>> > >>>> close it
>> > >>>>>> on
>> > >>>>>>>> September 8, unless there is an objection or not enough votes.
>> > >>>>>>>>
>> > >>>>>>>> Thank you~
>> > >>>>>>>>
>> > >>>>>>>> Xintong Song
>> > >>>>>>>>
>> > >>>>>>>>
>> > >>>>>>>> [1]
>> > >>>>>>>>
>> > >>>>>>>>
>> > >>>>>>
>> > >>>>
>> > >>>
>> > >>
>> >
>> https://cwiki.apache.org/confluence/display/FLINK/FLIP-141%3A+Intra-Slot+Managed+Memory+Sharing#FLIP141:IntraSlotManagedMemorySharing-compatibility
>> > >>>>>>>>
>> > >>>>>>>> [2]
>> > >>>>>>>>
>> > >>>>>>>>
>> > >>>>>>
>> > >>>>
>> > >>>
>> > >>
>> >
>> http://apache-flink-mailing-list-archive.1008284.n3.nabble.com/DISCUSS-FLIP-141-Intra-Slot-Managed-Memory-Sharing-td44146.html
>> > >>>>>>>>
>> > >>>>>>
>> > >>>>>>
>> > >>>>
>> > >>>
>> > >>
>> > >
>> >
>> >
>>
>


Re: [VOTE] NEW FLIP-102: Add More Metrics to TaskManager

2020-10-23 Thread Andrey Zagrebin
Thanks a lot for this nice UI guys!

+1 and for closed issues that is just because many steps have been
already done.

Best,
Andrey

On Fri, Oct 23, 2020 at 11:12 AM Till Rohrmann  wrote:

> Thanks for reviving this Flip Yadong! The changes look good to me and the
> new memory UI looks awesome :-)
>
> I think the reason why the REST issues are closed is because they are
> already done. In that sense some of the work already finished.
>
> +1 for adopting this FLIP and moving forward with updating the web UI
> accordingly.
>
> Cheers,
> Till
>
> On Fri, Oct 23, 2020 at 8:58 AM Jark Wu  wrote:
>
> > +1
> >
> > Thanks for the work.
> >
> > Best,
> > Jark
> >
> > On Fri, 23 Oct 2020 at 10:13, Xintong Song 
> wrote:
> >
> > > Thanks Yadong, Mattias and Lining for reviving this FLIP.
> > >
> > > I've seen so many users confused by the current webui page of task
> > manager
> > > metrics. This FLIP should definitely help them understand the memory
> > > footprints and tune the configurations for task managers.
> > >
> > > The design part of this proposal looks really good to me. The UI is
> clear
> > > and easy to understand. The metrics look correct to me.
> > >
> > > KIND REMINDER: I think the section `Implementation Proposal` in the
> FLIP
> > > doc needs to be updated, so that we can vote on this FLIP. Currently,
> all
> > > the tickets listed are closed.
> > >
> > > Thank you~
> > >
> > > Xintong Song
> > >
> > >
> > >
> > > On Thu, Oct 22, 2020 at 5:53 PM Yadong Xie 
> wrote:
> > >
> > > > Hi all
> > > >
> > > > I want to start a new vote for FLIP-102, which proposes to add more
> > > metrics
> > > > to the task manager in web UI.
> > > >
> > > > The new FLIP-102 was revisited and adapted following the old ML
> > > discussion
> > > > <
> > > >
> > >
> >
> http://apache-flink-mailing-list-archive.1008284.n3.nabble.com/VOTE-FLIP-102-Add-More-Metrics-to-TaskManager-td37898.html
> > > > >
> > > > .
> > > >
> > > > Thanks to Matthias and Lining's effort, more metrics are available.
> We
> > > can
> > > > match most of the effective configuration to the metrics just as
> Flink
> > > Doc
> > > > <
> > > >
> > >
> >
> https://ci.apache.org/projects/flink/flink-docs-release-1.11/ops/memory/mem_setup_tm.html#detailed-memory-model
> > > > >
> > > > describes now.
> > > >
> > > > The vote will last for at least 72 hours, following the consensus
> > voting.
> > > >
> > > >
> > > > FLIP-102 wiki:
> > > >
> > > >
> > >
> >
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-102%3A+Add+More+Metrics+to+TaskManager
> > > >
> > > >
> > > > Discussion thread:
> > > >
> > > >
> > >
> >
> http://apache-flink-mailing-list-archive.1008284.n3.nabble.com/VOTE-FLIP-102-Add-More-Metrics-to-TaskManager-td37898.html
> > > >
> > > >
> > >
> >
> http://apache-flink-mailing-list-archive.1008284.n3.nabble.com/DISCUSS-FLIP-75-Flink-Web-UI-Improvement-Proposal-td33540.html
> > > >
> > > > Thanks,
> > > >
> > > > Yadong
> > > >
> > >
> >
>


Re: [DISCUSS] FLIP-111: Docker image unification

2020-04-03 Thread Andrey Zagrebin
Hi everyone,

Patrick and Ufuk, thanks a lot for more ideas and suggestions!

I have updated the FLIP according to the current state of discussion.
Now it also contains the implementation steps and future follow-ups.
Please, review if there are any concerns.
The order of the steps aims for keeping Flink releasable at any point if
something does not have enough time to get in.

It looks that we are reaching mostly a consensus for the open questions.
There is also a list of items, which have been discussed in this thread,
and short summary below.
As soon as there are no concerns, I will create a voting thread.

I also added some thoughts for further customising logging setup. This may
be an optional follow-up
which is additional to the default logging into files for Web UI.

# FLIP scope
The focus is users of the official releases.
Create docs for how to use the official docker image.
Remove other Dockerfiles in Flink repo.
Rely on running the official docker image in different modes (JM/TM).
Customise running the official image with env vars (This should minimise
manual manipulating of local files and creation of a custom image).

# Base oficial image

## Java versions
There is a separate effort for this:
https://github.com/apache/flink-docker/pull/9

# Run image

## Entry point modes
JM session, JM job, TM

## Entry point config
We use env vars for this, e.g. FLINK_PROPERTIES and ENABLE_BUILT_IN_PLUGINS

## Flink config options
We document the existing FLINK_PROPERTIES env var to override config
options in flink-conf.yaml.
Then later, we do not need to expose and handle any other special env vars
for config options (address, port etc).
The future plan is to make Flink process configurable by env vars, e.g.
'some.yaml.option: val' -> FLINK_SOME_YAML_OPTION=val

## Extra files: jars, custom logging properties
We can provide env vars to point to custom locations, e.g. in mounted
volumes.

# Extend image

## Python/hadoop versions, activating certain libs/plugins
Users can install extra dependencies and change configs in their custom
image which extends our base image.

# Logging

## Web UI
Modify the *log4j-console.properties* to also output logs into the files
for WebUI. Limit log file size.

## Container output
Separate effort for proper split of Flink process stdout and stderr into
files and container output
(idea with tee command: `program start-foreground &2>1 | tee
flink-user-taskexecutor.out`)

# Docker bash utils
We are not going to expose it to users as an API.
They should be able either to configure and run the standard entry point
or the documentation should give short examples about how to extend and
customise the base image.

During the implementation, we will see if it makes sense to factor out
certain bash procedures
to reuse them e.g. in custom dev versions of docker image.

# Dockerfile / image for developers
We keep it on our future roadmap. This effort should help to understand
what we can reuse there.

Best,
Andrey


On Fri, Apr 3, 2020 at 12:57 PM Till Rohrmann  wrote:

> Hi everyone,
>
> just a small inline comment.
>
> On Fri, Apr 3, 2020 at 11:42 AM Ufuk Celebi  wrote:
>
> > Hey Yang,
> >
> > thanks! See inline answers.
> >
> > On Fri, Apr 3, 2020 at 5:11 AM Yang Wang  wrote:
> >
> > > Hi Ufuk,
> > >
> > > Thanks for make the conclusion and directly point out what need to be
> > done
> > > in
> > > FLIP-111. I agree with you that we should narrow down the scope and
> focus
> > > the
> > > most important and basic part about docker image unification.
> > >
> > > (1) Extend the entrypoint script in apache/flink-docker to start the
> job
> > >> cluster entry point
> > >
> > > I want to add a small requirement for the entry point script.
> Currently,
> > > for the native
> > > K8s integration, we are using the apache/flink-docker image, but with
> > > different entry
> > > point("kubernetes-entry.sh"). Generate the java cmd in KubernetesUtils
> > and
> > > run it
> > > in the entry point. I really hope it could merge to apache/flink-docker
> > > "docker-entrypoint.sh".
> > >
> >
> > The script [1] only adds the FLINK_CLASSPATH env var which seems
> generally
> > reasonable to me. But since principled classpath and entrypoint
> > configuration is somewhat related to the follow-up improvement
> proposals, I
> > could also see this being done after FLIP-111.
> >
> >
> > > (2) Extend the example log4j-console configuration
> > >> => support log retrieval from the Flink UI out of the box
> > >
> > > If you mean to update the "flink-dist/conf/log4j-console.properties" to
> > > support console and
> > > local log files. I will say "+1". But we need to find a proper way to
> > make
> > > stdout/stderr output
> > > both available for console and log files. Maybe till's proposal could
> > help
> > > to solve this.
> > > "`program &2>1 | tee flink-user-taskexecutor.out`"
> > >
> >
> > I think we can simply add a rolling file appender with a limit on the log
> > size.
> >
> > I think this won't solve 

Re: [VOTE] FLIP-102: Add More Metrics to TaskManager

2020-04-06 Thread Andrey Zagrebin
Hi guys,

Thanks for more details Zhijiang.
It also looks to me that mapped memory size is mostly driven by OS limits
and bit-ness of JVM (32/64).

Thinking more about the 'Metrics' tab layout, couple of more things have
come into my mind.

# 'Metrics' tab -> 'Memory': 'Metrics' and 'Configuration' tabs

It contains only memory specific things and the design suggests not only
metrics but configuration as well.
Moreover, there are other metrics on top which are not in the metrics tab.
Therefore, I would name it 'Memory' and then add sub-tabs: e.g. 'Metrics'
and 'Configuration' tab.
Alternatively, one could consider splitting 'Metrics' into 'Metrics' and
'Configuration' tabs.

# Metrics (a bit different structure)

I would put memory metrics into 4 groups:
- JVM Memory
- Managed
- Network
- Garbage collection

Alternatively, one could consider:
- Managed by JVM (same as JVM Memory)
- Managed by Flink (Managed Segments and Network buffers)
- Garbage collection

## Total memory (remove from metrics)

As mentioned in the discussions before, it is hard to measure the total
memory usage.
Therefore, I would put into the configuration tab, see below.

## JVM Memory

Here we can have Heap, Non-Heap, Direct and mapped because they are all
managed by JVM.
Heap and direct can stay as they are.

### Non-Heap (could stay for now)

I think it is ok to keep Non-Heap for now because we had it also before.
This metric does not correlate explicitly with FLIP-49 but it is exposed by
JVM.
Once, we find better things to show (related only to JVM, e.g. Metaspace
etc), we can reconsider this as a follow-up.

### Mapped (looks still valuable)

As I understand at the moment, this can have a value for users to monitor
spilling of batch partitions.

### Metaspace (new, sub-component of Non-Heap, follow-up)

We have never had anything for the Metaspace. The recent experience shows
that it can be useful.
I would put it on road map as a follow-up though, because it also needs
some research and preparation on server side [1].

# Configuration (see Flink user docs picture)

We already have a picture in the docs representing memory components in
Flink [2].
The layout in this picture can be also used in this FLIP to depict the
actual configuration.
This would be more clear for users to see the same as we have in docs.

The configuration can also depict size of the total process and total Flink
memory according to docs.

As mentioned above, I also suggest to put it into a separate tab.

Best,
Andrey

[1]
https://kb.novaordis.com/index.php/Memory_Monitoring_and_Management_Platform_MBeans#Metaspace
[2]
https://ci.apache.org/projects/flink/flink-docs-release-1.10/ops/memory/mem_detail.html#overview


On Wed, Apr 1, 2020 at 8:03 PM Zhijiang 
wrote:

> Thanks for the FLIP, Yadong. In general I think this work is valuable for
> users to better understand the Flink's memory usages in different
> dimensions.
>
> Sorry for not going through every detailed discussions below, and I try to
> do that later if possible. Firstly I try to answer some Andrey's concerns
> with mmap.
>
> > - I do not know how the mapped memory works. Is it meant for the new
> spilled partitions? If the mapped memory also pulls from the direct
> > memory limit then this is something we do not account in our network
> buffers as I understand. In this case, this metric may be useful for tuning
> to understand
> > how much the mapped memory uses from the direct memory limit to set e.g.
> framework off-heap limit correctly and avoid direct OOM.
> > It could be something to discuss with Zhijiang. e.g. is the direct
> memory used there to buffer fetched regions of partition files or what for?
>
> Yes, the mapped memory is used in bounded blocking partition for batch
> jobs now, but not the default mode.
>
>  AIK it is not related and limited to the setting of `MaxDirectMemory`, so
> we do not need to worry about the current direct memory setting and the
> potential OOM issue.
> It is up to the address space to determine the mapped file size, and in 64
> bit system we can regard the limitless size in theory.
>
> Regarding the size of mapped buffer pool from MXBean, it only indicates
> how much file size were already mapped before, even it is unchanged to not
> reflect the real
> physical memory use. E.g. when the file was mapped 100GB region at the
> beginning, the mapped buffer pool from MXBean would be 100GB. But how many
> physical
> memories are really consumed is up to the specific read or write
> operations in practice, and also controlled by the operator system. E.g
> some unused regions might be
> exchanged into SWAP virtual memory when physical memory is limited.
>
> From this point, I guess it is no meaningful to show the size of mapped
> buffer pool for users who may be more concerned with how many physical
> memories are really
> used.
>
> Best,
> Zhijiang
>
>
> --
> From:Andrey Zagrebin 
> Send Time:2020 Mar. 

Re: [DISCUSS] FLIP-111: Docker image unification

2020-04-07 Thread Andrey Zagrebin
there will be overlap with things you already have taken
>> > into account.
>> >
>> >1. No more 'flink:latest' docker image tag.
>> >Related to https://issues.apache.org/jira/browse/FLINK-15794
>> >What I have learned is that the 'latest' version of a docker image
>> only
>> >makes sense IFF this is an almost standalone thing.
>> >So if I have a servlet that does something in isolation (like my
>> hobby
>> >project https://hub.docker.com/r/nielsbasjes/yauaa ) then 'latest'
>> > makes
>> >sense.
>> >With Flink you have the application code and all nodes in the cluster
>> >that are depending on each other and as such must run the exact same
>> >versions of the base software.
>> >So if you run flink in a cluster (local/yarn/k8s/mesos/swarm/...)
>> where
>> >the application and the nodes inter communicate and closely depend on
>> > each
>> >other then 'latest' is a bad idea.
>> >   1. Assume I have an application built against the Flink N api and
>> the
>> >   cluster downloads the latest which is also Flink N.
>> >   Then a week later Flink N+1 is released and the API I use changes
>> >   (Deprecated)
>> >   and a while later Flink N+2 is released and the deprecated API is
>> >   removed: Then my application no longer works even though I have
>> > not changed
>> >   anything.
>> >   So I want my application to be 'pinned' to the exact version I
>> built
>> >   it with.
>> >   2. I have a running cluster with my application and cluster
>> running
>> >   Flink N.
>> >   I add some additional nodes and the new nodes pick up the Flink
>> N+1
>> >   image ... now I have a cluster with mixed versions.
>> >   3. The version of flink is really the "Flink+Scala" version pair.
>> >   If you have the right flink but the wrong scala you get really
>> nasty
>> >   errors: https://issues.apache.org/jira/browse/FLINK-16289
>> >
>> >   2. Deploy SNAPSHOT docker images (i.e. something like
>> >*flink:1.11-SNAPSHOT_2.12*) .
>> >More and more use cases will be running on the code delivered via
>> Docker
>> >images instead of bare jar files.
>> >So if a "SNAPSHOT" is released and deployed into a 'staging' maven
>> repo
>> >(which may be locally on the developers workstation) then it is my
>> > opinion
>> >that at the same moment a "SNAPSHOT" docker image should be
>> >created/deployed.
>> >Each time a "SNAPSHOT" docker image is released this will overwrite
>> the
>> >previous "SNAPSHOT".
>> >If the final version is released the SNAPSHOTs of that version
>> >can/should be removed.
>> >This will make testing in clusters a lot easier.
>> >Also building a local fix and then running it locally will work
>> without
>> >additional modifications to the code.
>> >
>> >3. Support for a 'single application cluster'
>> >I've been playing around with the S3 plugin and what I have found is
>> >that this essentially requires all nodes to have full access to the
>> >credentials needed to connect to S3.
>> >This essentially means that a multi-tenant setup is not possible in
>> >these cases.
>> >So I think the single application cluster should be a feature
>> available
>> >in all cases.
>> >
>> >4. I would like a native-kubernetes-single-application base image.
>> >I can then create a derived image where I only add the jar of my
>> >application.
>> >My desire is that I can then create a k8s yaml file for kubectl
>> >that adds the needed configs/secrets/arguments/environment variables
>> and
>> >starts the cluster and application.
>> >Because the native kubernetes support makes it automatically scale
>> based
>> >on the application this should 'just work'.
>> >
>> > Additional note:
>> >
>> >1. Job/Task attempt logging instead of task manager logging.
>> >*I realize this has nothing to do with the docker images*
>> >I found something "hard to work with" while running some tests last
>> > week.
>> >

[VOTE] FLIP-111: Docker image unification

2020-04-07 Thread Andrey Zagrebin
Hi All,

As discussed in these threads [1] and [2],
we suggest to unify the docker topic in Flink for users [3].

This mainly means refactoring of the existing code and introducing more
docs as a first step.
The effort should enable further improvements and follow-ups for the docker
integration with Flink.

The integration with docker in Flink is currently addressed in many places
which often intersect, repeat each other or apply different approaches.
This makes it really hard to follow the whole topic for users and
maintainers. This FLIP suggests how to unify this topic. It means having
one place which has the *Dockerfile*, all necessary scripts and docs
following each other in a consistent way without any repetitions or
contradictions.

The idea is to keep all docker related resources in apache/flink-docker
. It already has a detailed
Dockerfile which is well suited for common use cases or at least serves as
a good starting point. The suggestion is to make it extensible for other
concerns which are currently addressed in other places.

Best,
Andrey

[1]
http://apache-flink-mailing-list-archive.1008284.n3.nabble.com/DISCUSS-FLIP-111-Docker-image-unification-td38444.html#a39822
[2]
http://apache-flink-mailing-list-archive.1008284.n3.nabble.com/DISCUSS-FLIP-111-Docker-image-unification-td38444i20.html#a39950
[3]
https://cwiki.apache.org/confluence/display/FLINK/FLIP-111%3A+Docker+image+unification


Re: [VOTE] FLIP-111: Docker image unification

2020-04-15 Thread Andrey Zagrebin
Hi all,

My vote is +1 (binding).

Thanks for participating and giving your votes.

Hereby, the vote is closed and the FLIP is accepted with no -1/vetos.

+1's:
3 binding (Ufuk, Till, Andrey)
4 non-binding (Yang, Canbin, Ismaël, Yangze)

Thanks,
Andrey

On Fri, Apr 10, 2020 at 8:40 AM Yangze Guo  wrote:

> +1 (non-binding)
> Thanks for driving this, Andrey.
>
> Best,
> Yangze Guo
>
> On Thu, Apr 9, 2020 at 11:33 PM Ismaël Mejía  wrote:
> >
> > +1 (non-binding)
> > Great work Andrey, pretty excited about this happening!
> >
> >
> > On Wed, Apr 8, 2020 at 4:20 AM Canbin Zheng 
> wrote:
> > >
> > > Thanks for the FLIP Andrey.
> > >
> > > +1 (non-binding) from my side.
> > >
> > > Regards,
> > > Canbin Zheng
> > >
> > > Yang Wang  于2020年4月8日周三 上午9:57写道:
> > >
> > > > Thanks Andrey for the efforts of docker image unification.
> > > >
> > > > +1 (non-binding)
> > > >
> > > > Best,
> > > > Yang
> > > >
> > > > Till Rohrmann  于2020年4月7日周二 下午11:04写道:
> > > >
> > > > > Thanks for driving this effort Andrey.
> > > > >
> > > > > +1 (binding)
> > > > >
> > > > > Cheers,
> > > > > Till
> > > > >
> > > > > On Tue, Apr 7, 2020 at 4:48 PM Ufuk Celebi  wrote:
> > > > >
> > > > > > Thanks for starting this FLIP.
> > > > > >
> > > > > > +1
> > > > > >
> > > > > > On Tue, Apr 7, 2020 at 11:29 AM Andrey Zagrebin <
> azagre...@apache.org>
> > > > > > wrote:
> > > > > >
> > > > > > > Hi All,
> > > > > > >
> > > > > > > As discussed in these threads [1] and [2],
> > > > > > > we suggest to unify the docker topic in Flink for users [3].
> > > > > > >
> > > > > > > This mainly means refactoring of the existing code and
> introducing
> > > > more
> > > > > > > docs as a first step.
> > > > > > > The effort should enable further improvements and follow-ups
> for the
> > > > > > docker
> > > > > > > integration with Flink.
> > > > > > >
> > > > > > > The integration with docker in Flink is currently addressed in
> many
> > > > > > places
> > > > > > > which often intersect, repeat each other or apply different
> > > > approaches.
> > > > > > > This makes it really hard to follow the whole topic for users
> and
> > > > > > > maintainers. This FLIP suggests how to unify this topic. It
> means
> > > > > having
> > > > > > > one place which has the *Dockerfile*, all necessary scripts
> and docs
> > > > > > > following each other in a consistent way without any
> repetitions or
> > > > > > > contradictions.
> > > > > > >
> > > > > > > The idea is to keep all docker related resources in
> > > > apache/flink-docker
> > > > > > > <https://github.com/apache/flink-docker>. It already has a
> detailed
> > > > > > > Dockerfile which is well suited for common use cases or at
> least
> > > > serves
> > > > > > as
> > > > > > > a good starting point. The suggestion is to make it extensible
> for
> > > > > other
> > > > > > > concerns which are currently addressed in other places.
> > > > > > >
> > > > > > > Best,
> > > > > > > Andrey
> > > > > > >
> > > > > > > [1]
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> http://apache-flink-mailing-list-archive.1008284.n3.nabble.com/DISCUSS-FLIP-111-Docker-image-unification-td38444.html#a39822
> > > > > > > [2]
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> http://apache-flink-mailing-list-archive.1008284.n3.nabble.com/DISCUSS-FLIP-111-Docker-image-unification-td38444i20.html#a39950
> > > > > > > [3]
> > > > > > >
> > > > > > >
> > > > > >
> > > > >
> > > >
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-111%3A+Docker+image+unification
> > > > > > >
> > > > > >
> > > > >
> > > >
>


Re: Re: The use of state ttl incremental cleanup strategy in sql deduplication resulting in significant performance degradation

2020-05-04 Thread Andrey Zagrebin
Hi lsyldliu,

You can try to tune the StateTtlConfig. As the documentation suggests [1]
the TTL incremental cleanup can decrease the per record performance. This
is the price of the automatic cleanup.
If the only thing, which happens mostly in your operator, is working with
state then even checking one additional record to cleanup is two times more
actions to do.
Timer approach was discussed in TTL feature design. It needs an additional
implementation and keeps more state but performs only one cleanup action
exactly when needed so it is a performance/storage trade-off.

Anyways, 20x degradation looks indeed a lot.
As a first step, I would suggest to configure the incremental cleanup
explicitly in `StateTtlConfigUtil#createTtlConfig` with a less entries to
check, e.g. 1 because processFirstRow/processLastRow already access the
state twice and do cleanup:

.cleanupIncrementally(1, false)


Also not sure but depending on the input data, finishBundle can happen
mostly during the snapshotting which slows down taking the checkpoint.
Could this fail the checkpoint accumulating the backpressure and slowing
down the pipeline?

Not sure why to keep the deduplication data in a Java map and in Flink
state at the same time, why not to keep it only in Flink state and
deduplicate on each incoming record?

Best,
Andrey

[1] note 2 in
https://ci.apache.org/projects/flink/flink-docs-stable/dev/stream/state/state.html#incremental-cleanup

On Wed, Apr 29, 2020 at 11:53 AM 刘大龙  wrote:

>
>
>
> > -原始邮件-
> > 发件人: "Jark Wu" 
> > 发送时间: 2020-04-29 14:09:44 (星期三)
> > 收件人: dev , "Yu Li" ,
> myas...@live.com
> > 抄送: azagre...@apache.org
> > 主题: Re: The use of state ttl incremental cleanup strategy in sql
> deduplication resulting in significant performance degradation
> >
> > Hi lsyldliu,
> >
> > Thanks for investigating this.
> >
> > First of all, if you are using mini-batch deduplication, it doesn't
> support
> > state ttl in 1.9. That's why the tps looks the same with 1.11 disable
> state
> > ttl.
> > We just introduce state ttl for mini-batch deduplication recently.
> >
> > Regarding to the performance regression, it looks very surprise to me.
> The
> > performance is reduced by 19x when StateTtlConfig is enabled in 1.11.
> > I don't have much experience of the underlying of StateTtlConfig. So I
> loop
> > in @Yu Li  @YunTang in CC who may have more insights
> on
> > this.
> >
> > For more information, we use the following StateTtlConfig [1] in blink
> > planner:
> >
> > StateTtlConfig
> >   .newBuilder(Time.milliseconds(retentionTime))
> >   .setUpdateType(StateTtlConfig.UpdateType.OnCreateAndWrite)
> >   .setStateVisibility(StateTtlConfig.StateVisibility.NeverReturnExpired)
> >   .build();
> >
> >
> > Best,
> > Jark
> >
> >
> > [1]:
> >
> https://github.com/apache/flink/blob/master/flink-table/flink-table-runtime-blink/src/main/java/org/apache/flink/table/runtime/util/StateTtlConfigUtil.java#L27
> >
> >
> >
> >
> >
> > On Wed, 29 Apr 2020 at 11:53, 刘大龙  wrote:
> >
> > > Hi, all!
> > >
> > > At flink master branch, we have supported state ttl  for sql mini-batch
> > > deduplication using incremental cleanup strategy on heap backend,
> refer to
> > > FLINK-16581. Because I want to test the performance of this feature,
> so I
> > > compile master branch code and deploy the jar to production
> > > environment,then run three types of tests, respectively:
> > >
> > >
> > >
> > >
> > > flink 1.9.0 release version enable state ttl
> > > flink 1.11-snapshot version disable state ttl
> > > flink 1.11-snapshot version enable state ttl
> > >
> > >
> > >
> > >
> > > The test query sql as follows:
> > >
> > > select order_date,
> > > sum(price * amount - goods_all_fav_amt - virtual_money_amt +
> > > goods_carriage_amt) as saleP,
> > > sum(amount) as saleN,
> > > count(distinct parent_sn) as orderN,
> > > count(distinct user_id) as cusN
> > >from(
> > > select order_date, user_id,
> > > order_type, order_status, terminal, last_update_time,
> > > goods_all_fav_amt,
> > > goods_carriage_amt, virtual_money_amt, price, amount,
> > > order_quality, quality_goods_cnt, acture_goods_amt
> > > from (select *, row_number() over(partition by order_id,
> > > order_goods_id order by proctime desc) as rownum from
> dm_trd_order_goods)
> > > where rownum=1
> > > and (order_type in (1,2,3,4,5) or order_status = 70)
> > > and terminal = 'shop' and price > 0)
> > > group by order_date
> > >
> > >
> > > At runtime, this query will generate two operators which include
> > > Deduplication and GroupAgg. In the test, the configuration is same,
> > > parallelism is 20, set kafka consumer from the earliest, and disable
> > > mini-batch function, The test results as follows:
> > >
> > > flink 1.9.0 enable state ttl:this test lasted 44m, flink receive 1374w
> > > records, average tps at 5200/s, Flink UI picture link back pressure,
> > > checkpoint
> > > flink 1.11

Re: [ANNOUNCE] New Apache Flink Committer - Xintong Song

2020-06-05 Thread Andrey Zagrebin
Welcome to committers and congrats, Xintong!

Cheers,
Andrey

On Fri, Jun 5, 2020 at 4:22 PM Till Rohrmann  wrote:

> Congratulations!
>
> Cheers,
> Till
>
> On Fri, Jun 5, 2020 at 10:00 AM Dawid Wysakowicz 
> wrote:
>
> > Congratulations!
> >
> > Best,
> >
> > Dawid
> >
> > On 05/06/2020 09:10, tison wrote:
> > > Congrats, Xintong!
> > >
> > > Best,
> > > tison.
> > >
> > >
> > > Jark Wu  于2020年6月5日周五 下午3:00写道:
> > >
> > >> Congratulations Xintong!
> > >>
> > >> Best,
> > >> Jark
> > >>
> > >> On Fri, 5 Jun 2020 at 14:32, Danny Chan  wrote:
> > >>
> > >>> Congratulations Xintong !
> > >>>
> > >>> Best,
> > >>> Danny Chan
> > >>> 在 2020年6月5日 +0800 PM2:20,dev@flink.apache.org,写道:
> >  Congratulations Xintong
> >
>


[REMINDER] Use 'starter' labels for Jira issues where it makes sense

2020-07-20 Thread Andrey Zagrebin
Hi Flink Devs,

I would like to remind you that we have a 'starter' label [1] to annotate
Jira issues which need a contribution and which are not very
complicated for the new contributors. The starter issues can be a good
opportunity for the new contributors who want to learn about Flink but do
not know where to start [2].

When you open a Jira issue, please, pay attention to whether it can be a
starter task.
Let's try to help on-boarding new contributors!

Cheers,
Andrey

[1]
https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=154995514
 (Labels)
[2]
https://flink.apache.org/contributing/contribute-code.html#looking-for-what-to-contribute


Re: Increasing Disk Read Throughput and IOPS

2018-05-25 Thread Andrey Zagrebin
Hi,

I just wanted to add that if you are using EBS you could consider to switch to 
IO provisioned type of it (io1: 
https://docs.aws.amazon.com/AWSEC2/latest/UserGuide/EBSVolumeTypes.html 
) if 
it is ok from the cost prospective. There is no burst credit but a steady IOPS 
rate can be provisioned which is higher than the baseline of general purpose 
type gp2 (for 100GB: 5k IOPS of io1 vs 0.3k IOPS of gp2 baseline). It might 
speed up background compaction and improve read performance.

In general, EBS fault tolerance does not have a lot of benefit for the current 
version of Flink. I agree to consider instance ephemeral ssd storage instead 
which seems to be anyways couple of times more performant on bigger rocksdb.

Andrey

> On 25 May 2018, at 10:52, Stefan Richter  wrote:
> 
> One more thing, I am aware of one older thread that might be interesting for 
> you about RocksDB backend and EBS: 
> http://apache-flink-user-mailing-list-archive.2336050.n4.nabble.com/checkpoint-stuck-with-rocksdb-statebackend-and-s3-filesystem-td18679.html
>  
> 
> 
>> Am 25.05.2018 um 09:59 schrieb Stefan Richter > >:
>> 
>> Hi,
>> 
>> if the problem is seemingly from reads, I think incremental checkpoints are 
>> less likely to cause the problem. What Flink version are you using? Since 
>> you mentioned the use of map state, what comes to my mind as a potential 
>> cause is described in this issue 
>> https://issues.apache.org/jira/browse/FLINK-8639 
>>  . This was improved 
>> recently. Does the problem also exist for jobs without map state?
>> 
>> Best,
>> Stefan
>> 
>>> Am 24.05.2018 um 20:25 schrieb Stephan Ewen >> >:
>>> 
>>> One thing that you can always to is disable fsync, because Flink does not 
>>> rely on RocksDBs fsync for persistence.
>>> 
>>> If you disable incremental checkpoints, does that help?
>>> If yes, it could be an issue with too many small SSTable files due to 
>>> incremental checkpoints (an issue we have on the roadmap to fix).
>>> 
>>> On Thu, May 24, 2018 at 3:52 PM, Piotr Nowojski >> > wrote:
>>> Hi,
>>> 
>>> This issue might have something to do with compaction. Problems with 
>>> compaction can especially degrade reads performance (or just increase reads 
>>> IO). Have you tried to further enforce more compactions or change 
>>> CompactionStyle?
>>> 
>>> Have you taken a look on 
>>> org.apache.flink.contrib.streaming.state.PredefinedOptions?
>>> 
>>> Maybe Stefan or Andrey could share more input on this.
>>> 
>>> Piotrek
>>> 
>>> 
>>> > On 22 May 2018, at 08:12, Govindarajan Srinivasaraghavan 
>>> > mailto:govindragh...@gmail.com>> wrote:
>>> > 
>>> > Hi All,
>>> > 
>>> > We are running flink in AWS and we are observing a strange behavior. We 
>>> > are using docker containers, EBS for storage and Rocks DB state backend. 
>>> > We have a few map and value states with checkpointing every 30 seconds 
>>> > and incremental checkpointing turned on. The issue we are noticing is the 
>>> > read IOPS and read throughput gradually increases over time and keeps 
>>> > constantly growing. The write throughput and write bytes are not 
>>> > increasing as much as reads. The checkpoints are written to a durable NFS 
>>> > storage. We are not sure what is causing this constant increase in read 
>>> > throughput but due to which we are running out of EBS burst balance and 
>>> > need to restart the job every once in a while. Attached the EBS read and 
>>> > write metrics. Has anyone encountered this issue and what could be the 
>>> > possible solution.
>>> > 
>>> > We have also tried setting the below rocksdb options but didn't help.
>>> > 
>>> > DBOptions:
>>> > currentOptions.setOptimizeFiltersForHits(true)
>>> > .setWriteBufferSize(536870912)
>>> > .setMaxWriteBufferNumber(5)
>>> > .setMinWriteBufferNumberToMerge(2);
>>> > ColumnFamilyOptions:
>>> > 
>>> > currentOptions.setMaxBackgroundCompactions(4)
>>> > .setMaxManifestFileSize(1048576)
>>> > .setMaxLogFileSize(1048576);
>>> > 
>>> > 
>>> > 
>>> > Thanks.
>>> >  
>>> >  
>>> >  
>>> 
>>> 
>> 
> 



Re: [Discuss] Proposing FLIP-25 - Support User State TTL Natively in Flink

2018-06-04 Thread Andrey Zagrebin
Hi everybody,

We have been recently brainstorming ideas around state TTL in Flink 
and compiled our thoughts in the following design doc:
https://docs.google.com/document/d/1SI_WoXAfOd4_NKpGyk4yh3mf59g12pSGNXRtNFi-tgM 

Thanks to the community, many things in the doc are based on the previous 
discussions.

As there are a lot of TTL requirements related to data privacy regulations 
(quite hot topic in EU) 
and better cleanup strategies sometimes need more research and maybe POCs, 
we suggest to start with implementing TTL API itself 
and rather without major changes in current state performance.

In a nut shell, the approach requires only appending expiration timestamp bytes 
to each state value/entry.
Firstly, just block access to expired state and clean it up on explicit 
touching, 
then gradually adopt cleanup strategies with different guarantees to address 
space concerns better, 
including:
- filter out expired state during checkpointing process
- exact cleanup with timer service (though still requires double storing of 
keys in both backends)
- piggy-back rocksdb compaction using custom filter by TTL (similar to 
cassandra custom filter)
- cleanup of heap regions around randomly accessed bucket

Please, feel free to give any feedback and comments.

Thanks,
Andrey

> On 27 May 2018, at 09:46, sihua zhou  wrote:
> 
> Hi Bowen,
> 
> 
> Thanks for your clarification, I agree that we should wait for the timer on 
> RocksDB to be finished, after that we could even do some micro-benchmark 
> before start implementing.
> 
> 
> Best, Sihua
> 
> 
> 
> 
> 
> 
> On 05/27/2018 15:07,Bowen Li wrote:
> Thank you Fabian and Sihua. I explained more in the doc itself and its
> comments. I believe the bottom line of v1 are 1) it shouldn't be worse than
> how users implement TTL themselves today, 2) we should develop a generic
> architecture for TTL for all (currently two) state backends (impl can
> vary), 3) optimizations and improvements can come at v2 or later version.
> 
> For Sihua proposal, similar to the old plan we came up, I share similar
> concerns as before and wonder if you have answers:
> 
> - it requires building custom compaction for both state backends, it's a
> bit unclear in:
> - when and who and how? The 'when' might be the hardest one, because
> it really depends on user's use cases. E.g. if it's once a day, at what
> time in a day?
> - how well it will integrate with Flink's checkpoint/savepoint
> mechanism?
> - any performance regression indications in either state backends?
> - how much is the ROI if it requires very complicated implementation?
> - I'm afraid, eventually, the optimization will easily go to a tricky
> direction we may want to avoid - shall we come up with extra design to
> amortize the cost?
> - I'm afraid the custom compaction logic will have to make some quite
> different assumptions of different state backends. E.g. It's harder to
> estimate total memory required for user's app in Heap statebackend then,
> because it depends on when you trigger the compaction and how strictly you
> will stick to the schedule everyday. Any undeterministic behavior may lead
> to users allocating less memory than enough, and eventually causes user's
> apps to be unstable
> - I want to highlight that lots of users actually want the exact TTL
> feature. How users implement TTL with timers today actually implies that
> their logic depends on exact TTL for both shrinking their state size and
> expiring a key at exactly an expected time, I chatted with a few different
> Flink users recently and confirmed it. That's why I want to add exact TTL
> as a potential goal and motivation if possible, along with relaxed TTL and
> avoiding indefinitely growing state. If we don't provide that out of box,
> many users may still use the timer way themselves
> 
> To the concern of doubling keys - in Heap state backend, the key is only a
> reference so there's only one copy, that's not a problem; in rocksdb state
> backend, yes, the state size will bigger. Well, First, I believe this's a
> tradeoff for clearer architecture. Frankly, unlike memory, disk space (even
> SSD) is relatively cheap and accessible, and we don't normally take it as a
> big constraint. Second, w.r.t. to performance, I believe rocksdb timers
> will sit in a different column family than others, which may not cause
> noticeable perf issue. The rocksdb timer service is on is way, and I want
> to see how it's implemented first before asserting if there're truly any
> potential perf burden. Finally, there're also improvements we can make
> after v1, including relaxed TTL and smaller timer state size. E.g. Flink
> can approximate timers within a user configured time range (say within 5
> sec) into a single timer. I don't have concretely plan for that yet, but
> it's doable.
> 
> Stefan is adding rocksdb timer and bringing timer service more closely to
> keyed backends, whi

Re: Ordering of stream from different kafka partitions

2018-06-19 Thread Andrey Zagrebin
Hi Amol,

I think you could try (based on your stack overflow code)
org.apache.flink.streaming.api.functions.timestamps.BoundedOutOfOrdernessTimestampExtractor
like this:

DataStream streamSource = env
.addSource(kafkaConsumer)
.setParallelism(4)
.assignTimestampsAndWatermarks(
new 
BoundedOutOfOrdernessTimestampExtractor(Time.milliseconds(3500)) {
@Override
public long extractTimestamp(Event element) {
Map timeStamp = (Map) event.get("ts”);
return (long) timeStamp.get("value");
}
});

In general, if records are sorted by anything in a Kafka partition, parallel 
subtask of Flink Kafka source will consume these records and push to user 
operators in the same order. There is maximum one consuming subtask per Kafka 
partition but several partitions might be served by one subtask. It means that 
there are the same guarantees as in Kafka: ordering per partition but not 
across them, including no global ordering. 

The case of global and per window ordering is already described by Sihua. The 
global ordering might be impractical in case of distributed system.

If a subtask of your Flink operator consumes from several partitions or there 
is no ordering at all, you can try the above approach with 
BoundedOutOfOrdernessTimestampExtractor to get approximate ordering across 
these partitions per key or all records. It is similar to ordering within a 
window. It means there could still be late records coming after out of 
orderness period of time which can break the ordering. This operator buffers 
records in state to maintain the order but only for out of orderness period of 
time which also increases latency.

Cheers,
Andrey

> On 19 Jun 2018, at 14:12, sihua zhou  wrote:
> 
> 
> 
> Hi Amol,
> 
> 
> I'm not sure whether this is impossible, especially when you need to operate 
> the record in multi parallelism. 
> 
> 
> IMO, in theroy, we can only get a ordered stream when there is a single 
> partition of kafka and operate it with a single parallelism in flink. Even in 
> this case, if you only want to order the records in a window, than you need 
> to store the records in the state, and order them when the window is 
> triggered. But if you want to order the records with a single 
> `keyBy()`(non-window), I think that's maybe impossible in practice, because 
> you need to store the all the incoming records and order the all data for 
> every incoming records, also you need to send retracted message for the 
> previous result(because every incoming record might change the global order 
> of the records).
> 
> 
> Best, Sihua
> On 06/19/2018 19:19,Amol S - iProgrammer wrote:
> Hi,
> 
> I have used flink streaming API in my application where the source of
> streaming is kafka. My kafka producer will publish data in ascending order
> of time in different partitions of kafka and consumer will read data from
> these partitions. However some kafka partitions may be slow due to some
> operation and produce late results. Is there any way to maintain order in
> this stream though the data arrive out of order. I have tried
> BoundedOutOfOrdernessTimestampExtractor but it didn't served the purpose.
> While digging this problem I came across your documentation (URL:
> https://cwiki.apache.org/confluence/display/FLINK/Time+and+Order+in+Streams)
> and tried to implement this but it didnt worked. I also tried with Table
> API order by but it seems you not support orderBy in flink 1.5 version.
> Please suggest me any workaround for this.
> 
> I have raised same concern on stack overflow
> 
> https://stackoverflow.com/questions/50904615/ordering-of-streams-while-reading-data-from-multiple-kafka-partitions
> 
> Thanks,
> 
> ---
> *Amol Suryawanshi*
> Java Developer
> am...@iprogrammer.com
> 
> 
> *iProgrammer Solutions Pvt. Ltd.*
> 
> 
> 
> *Office 103, 104, 1st Floor Pride Portal,Shivaji Housing Society,
> Bahiratwadi,Near Hotel JW Marriott, Off Senapati Bapat Road, Pune - 411016,
> MH, INDIA.**Phone: +91 9689077510 | Skype: amols_iprogrammer*
> www.iprogrammer.com 
> 



Re: Ordering of stream from different kafka partitions

2018-06-19 Thread Andrey Zagrebin
Hi Amol,

I think you could try (based on your stack overflow code)
org.apache.flink.streaming.api.functions.timestamps.BoundedOutOfOrdernessTimestampExtractor
like this:

DataStream streamSource = env
   .addSource(kafkaConsumer)
   .setParallelism(4)
   .assignTimestampsAndWatermarks(
   new 
BoundedOutOfOrdernessTimestampExtractor(Time.milliseconds(3500)) {
   @Override
   public long extractTimestamp(Event element) {
   Map timeStamp = (Map) event.get("ts”);
   return (long) timeStamp.get("value");
   }
   });

In general, if records are sorted by anything in a Kafka partition, parallel 
subtask of Flink Kafka source will consume these records and push to user 
operators in the same order. There is maximum one consuming subtask per Kafka 
partition but several partitions might be served by one subtask. It means that 
there are the same guarantees as in Kafka: ordering per partition but not 
across them, including no global ordering. 

The case of global and per window ordering is already described by Sihua. The 
global ordering might be impractical in case of distributed system.

If a subtask of your Flink operator consumes from several partitions or there 
is no ordering at all, you can try the above approach with 
BoundedOutOfOrdernessTimestampExtractor to get approximate ordering across 
these partitions per key or all records. It is similar to ordering within a 
window. It means there could still be late records coming after out of 
orderness period of time which can break the ordering. This operator buffers 
records in state to maintain the order but only for out of orderness period of 
time which also increases latency.

Cheers,
Andrey

> On 19 Jun 2018, at 14:12, sihua zhou  wrote:
> 
> 
> 
> Hi Amol,
> 
> 
> I'm not sure whether this is impossible, especially when you need to operate 
> the record in multi parallelism. 
> 
> 
> IMO, in theroy, we can only get a ordered stream when there is a single 
> partition of kafka and operate it with a single parallelism in flink. Even in 
> this case, if you only want to order the records in a window, than you need 
> to store the records in the state, and order them when the window is 
> triggered. But if you want to order the records with a single 
> `keyBy()`(non-window), I think that's maybe impossible in practice, because 
> you need to store the all the incoming records and order the all data for 
> every incoming records, also you need to send retracted message for the 
> previous result(because every incoming record might change the global order 
> of the records).
> 
> 
> Best, Sihua
> On 06/19/2018 19:19,Amol S - iProgrammer wrote:
> Hi,
> 
> I have used flink streaming API in my application where the source of
> streaming is kafka. My kafka producer will publish data in ascending order
> of time in different partitions of kafka and consumer will read data from
> these partitions. However some kafka partitions may be slow due to some
> operation and produce late results. Is there any way to maintain order in
> this stream though the data arrive out of order. I have tried
> BoundedOutOfOrdernessTimestampExtractor but it didn't served the purpose.
> While digging this problem I came across your documentation (URL:
> https://cwiki.apache.org/confluence/display/FLINK/Time+and+Order+in+Streams)
> and tried to implement this but it didnt worked. I also tried with Table
> API order by but it seems you not support orderBy in flink 1.5 version.
> Please suggest me any workaround for this.
> 
> I have raised same concern on stack overflow
> 
> https://stackoverflow.com/questions/50904615/ordering-of-streams-while-reading-data-from-multiple-kafka-partitions
> 
> Thanks,
> 
> ---
> *Amol Suryawanshi*
> Java Developer
> am...@iprogrammer.com
> 
> 
> *iProgrammer Solutions Pvt. Ltd.*
> 
> 
> 
> *Office 103, 104, 1st Floor Pride Portal,Shivaji Housing Society,
> Bahiratwadi,Near Hotel JW Marriott, Off Senapati Bapat Road, Pune - 411016,
> MH, INDIA.**Phone: +91 9689077510 | Skype: amols_iprogrammer*
> www.iprogrammer.com 
> 



Re: Ordering of stream from different kafka partitions

2018-06-20 Thread Andrey Zagrebin
Hi,

Good point, sorry for confusion, BoundedOutOfOrdernessTimestampExtractor of 
course does not buffer records, you need to apply windowing (e.g. 
TumblingEventTimeWindows) for that and then sort the window output by time and 
emit records in sorted order.

You can also use windowAll which already does keyBy((record) -> 0) and makes 
the stream non-parallel:

 sourceStream
.setParallelism(4)
.assignTimestampsAndWatermarks(new 
BoundedOutOfOrdernessTimestampExtractor<>(…) {…})
.windowAll(TumblingEventTimeWindows.of(Time...))
.process(new OrderTheRecords()))

Cheers,
Andrey

> On 20 Jun 2018, at 10:14, sihua zhou  wrote:
> 
> 
> 
> Hi,
> 
> I think a global ordering is a bit impractical on production, but in theroy, 
> you still can do that. You need to
> 
> - Firstly fix the operate's parallelism to 1(except the source node).
> - If you want to sort the records within a bouned time, then you can keyBy() 
> a constant and window it, buffer the records in the state and sort the 
> records when the window is triggered, the code maybe as follows.
> {code}
> sourceStream
> .setParallelism(4)
> .assignTimestampsAndWatermarks(
> new 
> BoundedOutOfOrdernessTimestampExtractor(Time.milliseconds(3500))
> {
> @Override
> public long extractTimestamp(Event element) {
> Map timeStamp = (Map) event.get("ts”);
> return (long) timeStamp.get("value");
> }
>  })
> .keyBy((record) -> 0)// keyby the constant value
> .window(...)
> .process(new OrderTheRecords()))
> .setParallelism(1);
> {code}
> 
> - If you want to sort the records truly globally(non-window), then you could 
> keyBy a constant, store the records in the state and sort the records in the 
> process() function for every incoming record. And if you want a perfect 
> correct output, then maybe you need to do retraction (because every incoming 
> records may change the globally order), the code maybe as follows
> {code}
> sourceStream
> .setParallelism(4)
> .keyBy((record) -> 0) // keyby the constant value
> .process(new OrderTheRecords()))
> .setParallelism(1);
> {code}
> 
> 
> In all the case, you need to fix the parallelism of the OrderTheRecord 
> operate to 1, which makes your job non-scale-able and becomes the bottleneck. 
> So a global ordering maybe not practical on production (but if the source's 
> TPS is very low, then maybe practical).
> 
> Best, Sihua
> 
> On 06/20/2018 15:36,Amol S - iProgrammer 
> <mailto:am...@iprogrammer.com> wrote: 
> Hello Andrey,
> 
> Thanks for your quick response. I have tried with your above code but it
> didn't suit's my requirement. I need global ordering of my records by using
> multiple kafka partitions. Please suggest me any workaround for this. as
> mentioned in this
> <https://cwiki.apache.org/confluence/display/FLINK/Time+and+Order+in+Streams 
> <https://cwiki.apache.org/confluence/display/FLINK/Time+and+Order+in+Streams>>
> link is it possible to buffer data for some amount of time and then perform
> sort on this or any other way out there?
> 
> ---
> *Amol Suryawanshi*
> Java Developer
> am...@iprogrammer.com
> 
> 
> *iProgrammer Solutions Pvt. Ltd.*
> 
> 
> 
> *Office 103, 104, 1st Floor Pride Portal,Shivaji Housing Society,
> Bahiratwadi,Near Hotel JW Marriott, Off Senapati Bapat Road, Pune - 411016,
> MH, INDIA.**Phone: +91 9689077510 | Skype: amols_iprogrammer*
> www.iprogrammer.com <http://www.iprogrammer.com/>  <mailto:sac...@iprogrammer.com>>
> 
> 
> On Tue, Jun 19, 2018 at 10:19 PM, Andrey Zagrebin  <mailto:and...@data-artisans.com>>
> wrote:
> 
> Hi Amol,
> 
> I think you could try (based on your stack overflow code)
> org.apache.flink.streaming.api.functions.timestamps.
> BoundedOutOfOrdernessTimestampExtractor
> like this:
> 
> DataStream streamSource = env
> .addSource(kafkaConsumer)
> .setParallelism(4)
> .assignTimestampsAndWatermarks(
> new BoundedOutOfOrdernessTimestampExtractor(Time.milliseconds(3500))
> {
> @Override
> public long extractTimestamp(Event element) {
> Map timeStamp = (Map) event.get("ts”);
> return (long) timeStamp.get("value");
> }
> });
> 
> In general, if records are sorted by anything in a Kafka partition,
> parallel subta

Re: Ordering of stream from different kafka partitions

2018-06-20 Thread Andrey Zagrebin
Hi Amol,

> In above code also it will sort the records in specific time window only.

All windows will be emitted as watermark passes the end of the window. The 
watermark only increases. So the non-overlapping windows should be also sorted 
by time and as a consequence the records across windows either, if this is the 
concern about sorting records only in a specific time window.

> 1. How should I create N consumers dynamically based on partition count?

sourceStream.setParallelism(N), each Flink consumer parallel subtask will serve 
one Kafka partition.

> 2. Is number of consumers dynamically grows as number of partition
> increased in middle of execution?

Dynamically added Kafka partitions will be eventually discovered by Flink 
consumers (flink.partition-discovery.interval-millis) and picked up by some 
consumer. Flink job has be rescaled separately.

Currently parallelism of Flink operator cannot be changed while the job is 
running. The way to go now is to use savepoint/checkpoint, stop the job and 
start the new one with changed parallelism from the previous 
savepoint/checkpoint (see Flink docs). New job will pick up from partition 
offsets of previous job.

> 3. How to create partition specific kafka consumer in flink?

The partition-consumer assignment is now implementation specific for Flink.
There is an open issue for custom assignment 
https://issues.apache.org/jira/browse/FLINK-8570 
<https://issues.apache.org/jira/browse/FLINK-8570> e.g. if you need specific 
locality of keys/consumers.

I would simply suggest to assign some key to each record and let all records 
for particular key to go into the same Kafka partition. On the Flink side if a 
corresponding keyBy() is applied to the Kafka source, all the records for this 
particular key will go to the same parallel subtask of subsequent operator, 
sorted by time if they were originally sorted in its Kafka partition. This is 
more scalable approach than total global ordering.

Cheers,
Andrey

> On 20 Jun 2018, at 13:17, Amol S - iProgrammer  wrote:
> 
> Hello Andrey,
> 
> In above code also it will sort the records in specific time window only.
> Anyways we agreed to create N number of partitions with N number of
> consumers based on some key as order is maintained per kafka partition.
> 
> I have some questions about this.
> 
> 1. How should I create N consumers dynamically based on partition count?
> 2. Is number of consumers dynamically grows as number of partition
> increased in middle of execution?
> 3. How to create partition specific kafka consumer in flink?
> 
> ---
> *Amol Suryawanshi*
> Java Developer
> am...@iprogrammer.com <mailto:am...@iprogrammer.com>
> 
> 
> *iProgrammer Solutions Pvt. Ltd.*
> 
> 
> 
> *Office 103, 104, 1st Floor Pride Portal,Shivaji Housing Society,
> Bahiratwadi,Near Hotel JW Marriott, Off Senapati Bapat Road, Pune - 411016,
> MH, INDIA.**Phone: +91 9689077510 | Skype: amols_iprogrammer*
> www.iprogrammer.com <http://www.iprogrammer.com/>  <mailto:sac...@iprogrammer.com>>
> 
> 
> On Wed, Jun 20, 2018 at 2:38 PM, Andrey Zagrebin  <mailto:and...@data-artisans.com>>
> wrote:
> 
>> Hi,
>> 
>> Good point, sorry for confusion, BoundedOutOfOrdernessTimestampExtractor
>> of course does not buffer records, you need to apply windowing
>> (e.g. TumblingEventTimeWindows) for that and then sort the window output
>> by time and emit records in sorted order.
>> 
>> You can also use windowAll which already does keyBy((record) -> 0) and
>> makes the stream non-parallel:
>> 
>> sourceStream
>>.setParallelism(4)
>>.assignTimestampsAndWatermarks(new
>> BoundedOutOfOrdernessTimestampExtractor<>(…) {…})
>>.windowAll(TumblingEventTimeWindows.of(Time...))
>>.process(new OrderTheRecords()))
>> 
>> Cheers,
>> Andrey
>> 
>> On 20 Jun 2018, at 10:14, sihua zhou  wrote:
>> 
>> 
>> 
>> Hi,
>> 
>> I think a global ordering is a bit impractical on production, but in
>> theroy, you still can do that. You need to
>> 
>> - Firstly fix the operate's parallelism to 1(except the source node).
>> - If you want to sort the records within a bouned time, then you can
>> keyBy() a constant and window it, buffer the records in the state and sort
>> the records when the window is triggered, the code maybe as follows.
>>{code}
>>sourceStream
>>.setParallelism(4)
>>.assignTimestampsAndWatermarks(
>>new BoundedOutOfOrdernessTimestamp
>> Extractor(Time.milliseconds(3500))
>&g

Re: [VOTE] Release 1.6.0, release candidate #3

2018-08-07 Thread Andrey Zagrebin
Hi everyone,

After careful consideration, I think that it makes sense to change the order of 
serialisation for state values with TTL. It will simplify further optimisations 
and feature improvements.

If it is still possible, I would ask to include this PR to the release 
candidate:
https://github.com/apache/flink/pull/6510 

This change will help to avoid compatibility issues in future.

Sorry for the short notice.

Thanks,
Andrey

> On 7 Aug 2018, at 10:01, Till Rohrmann  wrote:
> 
> Hi everyone,
> Please review and vote on the release candidate #3 for the version 1.6.0,
> as follows:
> [ ] +1, Approve the release
> [ ] -1, Do not approve the release (please provide specific comments)
> 
> 
> The complete staging area is available for your review, which includes:
> * JIRA release notes [1],
> * the official Apache source release and binary convenience releases to be
> deployed to dist.apache.org [2], which are signed with the key with
> fingerprint 1F302569A96CFFD5 [3],
> * all artifacts to be deployed to the Maven Central Repository [4],
> * source code tag "release-1.6.0-rc3" [5],
> 
> Please use this document for coordinating testing efforts: [6]
> 
> The vote will be shortened since we only added minor fixes on top of the RC
> 2. It will close on Wednesday 6:30pm CET. It is adopted by majority
> approval, with at least 3 PMC affirmative votes.
> 
> Thanks,
> Your friendly Release Manager
> 
> [1]
> https://issues.apache.org/jira/secure/ReleaseNote.jspa?projectId=12315522&version=12342760
> [2] https://dist.apache.org/repos/dist/dev/flink/flink-1.6.0/
> [3] https://dist.apache.org/repos/dist/release/flink/KEYS
> [4] https://repository.apache.org/content/repositories/orgapacheflink-1177
> [5] https://github.com/apache/flink/tree/release-1.6.0-rc3
> [6]
> https://docs.google.com/document/d/18rsgMQdT2OhpwdQOjb7WTuJlWdCi58N-lzdhELLBBsE/edit?usp=sharing
> 
> Pro-tip: you can create a settings.xml file with these contents:
> 
> 
> 
>  flink-1.6.0
> 
> 
>  
>flink-1.6.0
>
>  
>flink-1.6.0
>
> 
> https://repository.apache.org/content/repositories/orgapacheflink-1177/
>
>  
>  
>archetype
>
> 
> https://repository.apache.org/content/repositories/orgapacheflink-1177/
>
>  
>
>  
> 
> 
> 
> And reference that in you maven commands via --settings
> path/to/settings.xml. This is useful for creating a quickstart based on the
> staged release and for building against the staged jars.



[jira] [Created] (FLINK-13802) Flink code style guide

2019-08-20 Thread Andrey Zagrebin (Jira)
Andrey Zagrebin created FLINK-13802:
---

 Summary: Flink code style guide
 Key: FLINK-13802
 URL: https://issues.apache.org/jira/browse/FLINK-13802
 Project: Flink
  Issue Type: Task
  Components: Documentation, Project Website
Reporter: Andrey Zagrebin


This is an umbrella issue to introduce and improve Flink code style guide.



--
This message was sent by Atlassian Jira
(v8.3.2#803003)


[jira] [Created] (FLINK-13804) Collections initial capacity

2019-08-20 Thread Andrey Zagrebin (Jira)
Andrey Zagrebin created FLINK-13804:
---

 Summary: Collections initial capacity
 Key: FLINK-13804
 URL: https://issues.apache.org/jira/browse/FLINK-13804
 Project: Flink
  Issue Type: Sub-task
  Components: Documentation, Project Website
Reporter: Andrey Zagrebin
Assignee: Andrey Zagrebin


The code style conclusion to add to web site:
 
Set the initial capacity only if there is a good proven reason to do it. 
Otherwise do not clutter the code with it.



--
This message was sent by Atlassian Jira
(v8.3.2#803003)


  1   2   3   >