Re: [DISCUSS] FLIP-138: Declarative Resource management

2020-08-27 Thread Zhu Zhu
Thanks for the response!

>> The scheduler doesn't have to wait for one stage to finish
Does it mean we will declare resources and decide the parallelism for a
stage which is partially
schedulable, i.e. when input data are ready just for part of the execution
vertices?

>> This will get more complicated once we allow the scheduler to change the
parallelism while the job is running
Agreed. Looks to me it's a problem for batch jobs only and can be avoided
for streaming jobs.
Will this FLIP limit its scope to streaming jobs, and improvements for
batch jobs are to be done later?

Thanks,
Zhu

Chesnay Schepler  于2020年8月28日周五 上午2:27写道:

> The scheduler doesn't have to wait for one stage to finish. It is still
> aware that the upstream execution vertex has finished, and can request/use
> slots accordingly to schedule the consumer.
>
> This will get more complicated once we allow the scheduler to change the
> parallelism while the job is running, for which we will need some
> enhancements to the network stack to allow the producer to run without
> knowing the consumer parallelism ahead of time. I'm not too clear on the
> details, but we'll some form of keygroup-like approach for sub partitions
> (maxParallelism and all that).
>
> On 27/08/2020 20:05, Zhu Zhu wrote:
>
> Thanks Chesnay for proposing this improvement.
> It's of good value to allow jobs to make best use of available resources
> adaptively. Not
> to mention it further supports reactive mode.
> So big +1 for it.
>
> I have a minor concern about possible regression in certain cases due to
> the proposed
> JobVertex-wise scheduling which replaces current ExecutionVertex-wise
> scheduling.
> In the proposal, looks to me it requires a stage to finish before its
> consumer stage can be
> scheduled. This limitation, however, does not exist in current scheduler.
> In the case that there
> exists a POINTWISE BLOCKING edge, the downstream execution region can be
> scheduled
> right after its connected upstream execution vertices finishes, even
> before the whole upstream
> stage finishes. This allows the region to be launched earlier and make use
> of available resources.
> Do we need to let the new scheduler retain this property?
>
> Thanks,
> Zhu
>
> Xintong Song  于2020年8月26日周三 下午6:59写道:
>
>> Thanks for the quick response.
>>
>> *Job prioritization, Allocation IDs, Minimum resource
>> requirements, SlotManager Implementation Plan:* Sounds good to me.
>>
>> *FLIP-56*
>> Good point about the trade-off. I believe maximum resource utilization and
>> quick deployment are desired in different scenarios. E.g., a long running
>> streaming job deserves some deployment latency to improve the resource
>> utilization, which benefits the entire lifecycle of the job. On the other
>> hand, short batch queries may prefer quick deployment, otherwise the time
>> for resource allocation might significantly increase the response time.
>> It would be good enough for me to bring these questions to attention.
>> Nothing that I'm aware of should block this FLIP.
>>
>> Thank you~
>>
>> Xintong Song
>>
>>
>>
>> On Wed, Aug 26, 2020 at 5:14 PM Chesnay Schepler 
>> wrote:
>>
>> > Thank you Xintong for your questions!
>> > Job prioritization
>> > Yes, the job which declares it's initial requirements first is
>> prioritized.
>> > This is very much for simplicity; for example this avoids the nasty case
>> > where all jobs get some resources, but none get enough to actually run
>> the
>> > job.
>> > Minimum resource requirements
>> >
>> > My bad; at some point we want to allow the JobMaster to declare a range
>> of
>> > resources it could use to run a job, for example min=1, target=10,
>> > max=+inf.
>> >
>> > With this model, the RM would then try to balance the resources such
>> that
>> > as many jobs as possible are as close to the target state as possible.
>> >
>> > Currently, the minimum/target/maximum resources are all the same. So the
>> > notification is sent whenever the current requirements cannot be met.
>> > Allocation IDs
>> > We do intend to, at the very least, remove AllocationIDs on the
>> > SlotManager side, as they are just not required there.
>> >
>> > On the slotpool side we have to keep them around at least until the
>> > existing Slotpool implementations are removed (not sure whether we'll
>> fully
>> > commit to this in 1.12), since the interfaces use AllocationIDs, which
>> also
>> > bleed into the JobMaster.
>> > The TaskExecutor is in a similar position.
>> > But in the long-term, yes they will be removed, and most usages will
>> > probably be replaced by the SlotID.
>> > FLIP-56
>> >
>> > Dynamic slot allocations are indeed quite interesting and raise a few
>> > questions; for example, the main purpose of it is to ensure maximum
>> > resource utilization. In that case, should the JobMaster be allowed to
>> > re-use a slot it if the task requires less resources than the slot
>> > provides, or should it always request a new slot that exactly matches?
>> >
>> > There 

[jira] [Created] (FLINK-19082) Add docs for temporal table and temporal table join

2020-08-27 Thread Leonard Xu (Jira)
Leonard Xu created FLINK-19082:
--

 Summary: Add docs for temporal table and temporal table join
 Key: FLINK-19082
 URL: https://issues.apache.org/jira/browse/FLINK-19082
 Project: Flink
  Issue Type: Sub-task
Reporter: Leonard Xu






--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Created] (FLINK-19081) Deprecate TemporalTableFunction and Table$createTemporalTableFunction()

2020-08-27 Thread Leonard Xu (Jira)
Leonard Xu created FLINK-19081:
--

 Summary: Deprecate TemporalTableFunction and 
Table$createTemporalTableFunction()
 Key: FLINK-19081
 URL: https://issues.apache.org/jira/browse/FLINK-19081
 Project: Flink
  Issue Type: Sub-task
Reporter: Leonard Xu






--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Created] (FLINK-19080) Materialize timeindicator data type in the right input of temporal join

2020-08-27 Thread Leonard Xu (Jira)
Leonard Xu created FLINK-19080:
--

 Summary: Materialize timeindicator data type in the right input of 
temporal join
 Key: FLINK-19080
 URL: https://issues.apache.org/jira/browse/FLINK-19080
 Project: Flink
  Issue Type: Sub-task
Reporter: Leonard Xu






--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Created] (FLINK-19079) Support row time deduplicate operator

2020-08-27 Thread Leonard Xu (Jira)
Leonard Xu created FLINK-19079:
--

 Summary: Support row time deduplicate operator
 Key: FLINK-19079
 URL: https://issues.apache.org/jira/browse/FLINK-19079
 Project: Flink
  Issue Type: Sub-task
Reporter: Leonard Xu






--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Created] (FLINK-19078) Import rowtime join temporal operator

2020-08-27 Thread Leonard Xu (Jira)
Leonard Xu created FLINK-19078:
--

 Summary: Import rowtime join temporal operator
 Key: FLINK-19078
 URL: https://issues.apache.org/jira/browse/FLINK-19078
 Project: Flink
  Issue Type: Sub-task
Reporter: Leonard Xu






--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Created] (FLINK-19076) Import rule to deal Temporal Join condition

2020-08-27 Thread Leonard Xu (Jira)
Leonard Xu created FLINK-19076:
--

 Summary: Import rule to deal Temporal Join condition
 Key: FLINK-19076
 URL: https://issues.apache.org/jira/browse/FLINK-19076
 Project: Flink
  Issue Type: Sub-task
Reporter: Leonard Xu






--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Created] (FLINK-19077) Improve process time temporal join operator

2020-08-27 Thread Leonard Xu (Jira)
Leonard Xu created FLINK-19077:
--

 Summary: Improve process time temporal join operator
 Key: FLINK-19077
 URL: https://issues.apache.org/jira/browse/FLINK-19077
 Project: Flink
  Issue Type: Sub-task
Reporter: Leonard Xu






--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Created] (FLINK-19075) Infer changelog trait for temporal join node

2020-08-27 Thread Leonard Xu (Jira)
Leonard Xu created FLINK-19075:
--

 Summary: Infer  changelog trait for temporal join node
 Key: FLINK-19075
 URL: https://issues.apache.org/jira/browse/FLINK-19075
 Project: Flink
  Issue Type: Bug
Reporter: Leonard Xu






--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Created] (FLINK-19074) Materialize timeindicator in the right input of temporal join

2020-08-27 Thread Leonard Xu (Jira)
Leonard Xu created FLINK-19074:
--

 Summary: Materialize timeindicator in the right input of temporal 
join
 Key: FLINK-19074
 URL: https://issues.apache.org/jira/browse/FLINK-19074
 Project: Flink
  Issue Type: Bug
Reporter: Leonard Xu






--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Created] (FLINK-19073) Improve streamExecTemporalJoinRule

2020-08-27 Thread Leonard Xu (Jira)
Leonard Xu created FLINK-19073:
--

 Summary: Improve streamExecTemporalJoinRule
 Key: FLINK-19073
 URL: https://issues.apache.org/jira/browse/FLINK-19073
 Project: Flink
  Issue Type: Sub-task
Reporter: Leonard Xu






--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Created] (FLINK-19072) Import Temporal Table join rule

2020-08-27 Thread Leonard Xu (Jira)
Leonard Xu created FLINK-19072:
--

 Summary: Import Temporal Table join rule
 Key: FLINK-19072
 URL: https://issues.apache.org/jira/browse/FLINK-19072
 Project: Flink
  Issue Type: Sub-task
Reporter: Leonard Xu






--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Created] (FLINK-19070) Hive connector should throw a meaningful exception if user reads/writes ACID tables

2020-08-27 Thread Rui Li (Jira)
Rui Li created FLINK-19070:
--

 Summary: Hive connector should throw a meaningful exception if 
user reads/writes ACID tables
 Key: FLINK-19070
 URL: https://issues.apache.org/jira/browse/FLINK-19070
 Project: Flink
  Issue Type: Improvement
  Components: Connectors / Hive
Reporter: Rui Li
 Fix For: 1.12.0






--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Created] (FLINK-19071) Some Hive window functions are not supported

2020-08-27 Thread Rui Li (Jira)
Rui Li created FLINK-19071:
--

 Summary: Some Hive window functions are not supported
 Key: FLINK-19071
 URL: https://issues.apache.org/jira/browse/FLINK-19071
 Project: Flink
  Issue Type: Task
  Components: Connectors / Hive
Reporter: Rui Li






--
This message was sent by Atlassian Jira
(v8.3.4#803005)


Re: [ANNOUNCE] New PMC member: Dian Fu

2020-08-27 Thread Yun Tang
Congratulations , Dian!

Best
Yun Tang

From: Yang Wang 
Sent: Friday, August 28, 2020 10:28
To: Arvid Heise 
Cc: Benchao Li ; dev ; user-zh 
; Dian Fu ; user 

Subject: Re: [ANNOUNCE] New PMC member: Dian Fu

Congratulations Dian !


Best,
Yang

Arvid Heise mailto:ar...@ververica.com>> 于2020年8月28日周五 
上午1:39写道:
Congrats Dian :)

On Thu, Aug 27, 2020 at 5:01 PM Benchao Li 
mailto:libenc...@apache.org>> wrote:
Congratulations Dian!

Cranmer, Danny  于2020年8月27日周四 下午10:55写道:
Congratulations Dian! :D

On 27/08/2020, 15:25, "Robert Metzger" 
mailto:rmetz...@apache.org>> wrote:

CAUTION: This email originated from outside of the organization. Do not 
click links or open attachments unless you can confirm the sender and know the 
content is safe.



Congratulations Dian!

On Thu, Aug 27, 2020 at 3:09 PM Congxian Qiu 
mailto:qcx978132...@gmail.com>> wrote:

> Congratulations Dian
> Best,
> Congxian
>
>
> Xintong Song mailto:tonysong...@gmail.com>> 
于2020年8月27日周四 下午7:50写道:
>
> > Congratulations Dian~!
> >
> > Thank you~
> >
> > Xintong Song
> >
> >
> >
> > On Thu, Aug 27, 2020 at 7:42 PM Jark Wu 
mailto:imj...@gmail.com>> wrote:
> >
> > > Congratulations Dian!
> > >
> > > Best,
> > > Jark
> > >
> > > On Thu, 27 Aug 2020 at 19:37, Leonard Xu 
mailto:xbjt...@gmail.com>> wrote:
> > >
> > > > Congrats, Dian!  Well deserved.
> > > >
> > > > Best
> > > > Leonard
> > > >
> > > > > 在 2020年8月27日,19:34,Kurt Young 
mailto:ykt...@gmail.com>> 写道:
> > > > >
> > > > > Congratulations Dian!
> > > > >
> > > > > Best,
> > > > > Kurt
> > > > >
> > > > >
> > > > > On Thu, Aug 27, 2020 at 7:28 PM Rui Li 
mailto:lirui.fu...@gmail.com>>
> > wrote:
> > > > >
> > > > >> Congratulations Dian!
> > > > >>
> > > > >> On Thu, Aug 27, 2020 at 5:39 PM Yuan Mei 
mailto:yuanmei.w...@gmail.com>>
> > > > wrote:
> > > > >>
> > > > >>> Congrats!
> > > > >>>
> > > > >>> On Thu, Aug 27, 2020 at 5:38 PM Xingbo Huang 
mailto:hxbks...@gmail.com>
> >
> > > > wrote:
> > > > >>>
> > > >  Congratulations Dian!
> > > > 
> > > >  Best,
> > > >  Xingbo
> > > > 
> > > >  jincheng sun 
mailto:sunjincheng...@gmail.com>> 于2020年8月27日周四 
下午5:24写道:
> > > > 
> > > > > Hi all,
> > > > >
> > > > > On behalf of the Flink PMC, I'm happy to announce that Dian Fu
> is
> > > now
> > > > > part of the Apache Flink Project Management Committee (PMC).
> > > > >
> > > > > Dian Fu has been very active on PyFlink component, working on
> > > various
> > > > > important features, such as the Python UDF and Pandas
> > integration,
> > > > and
> > > > > keeps checking and voting for our releases, and also has
> > > successfully
> > > > > produced two releases(1.9.3&1.11.1) as RM, currently working 
as
> > RM
> > > > to push
> > > > > forward the release of Flink 1.12.
> > > > >
> > > > > Please join me in congratulating Dian Fu for becoming a Flink
> PMC
> > > > > Member!
> > > > >
> > > > > Best,
> > > > > Jincheng(on behalf of the Flink PMC)
> > > > >
> > > > 
> > > > >>
> > > > >> --
> > > > >> Best regards!
> > > > >> Rui Li
> > > > >>
> > > >
> > > >
> > >
> >
>



--

Best,
Benchao Li


--

Arvid Heise | Senior Java Developer

[https://lh5.googleusercontent.com/ODbO0aq1IqKMfuoy_pw2YH8r6dqDRTq37rg3ytg11FCGJx12jJ1ff_SANPBxTHzSJTUQY9JLuoXq4NB7Om7j6Vq1lg6jIOKz8S5g2VKDGwicbj5fbY09PVb6mD5TdRuWEUvEMZTG]


Follow us @VervericaData

--

Join Flink Forward - 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 (Toni) 
Cheng


[jira] [Created] (FLINK-19069) finalizeOnMaster takes too much time and client timeouts

2020-08-27 Thread Jiayi Liao (Jira)
Jiayi Liao created FLINK-19069:
--

 Summary: finalizeOnMaster takes too much time and client timeouts
 Key: FLINK-19069
 URL: https://issues.apache.org/jira/browse/FLINK-19069
 Project: Flink
  Issue Type: Improvement
  Components: Runtime / Task
Affects Versions: 1.9.0
Reporter: Jiayi Liao


Currently we execute {{finalizeOnMaster}} in JM's main thread, which may stuck 
the JM for a very long time and client timeouts eventually. 

For example, we'd like to write data to HDFS  and commit files on JM, which 
takes more than ten minutes to commit tens of thousands files.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Created] (FLINK-19068) Filter verbose pod events for KubernetesResourceManagerDriver

2020-08-27 Thread Xintong Song (Jira)
Xintong Song created FLINK-19068:


 Summary: Filter verbose pod events for 
KubernetesResourceManagerDriver
 Key: FLINK-19068
 URL: https://issues.apache.org/jira/browse/FLINK-19068
 Project: Flink
  Issue Type: Improvement
  Components: Deployment / Kubernetes
Reporter: Xintong Song


A status of a Kubernetes pod consists of many detailed fields. Currently, Flink 
receives pod {{MODIFIED}} events from the {{KubernetesPodsWatcher}} on every 
single change to these fields, many of which Flink does not care.

The verbose events will not affect the functionality of Flink, but will pollute 
the logs with repeated messages, because Flink only looks into the fields it 
interested in and those fields are identical.

E.g., when a task manager is stopped due to idle timeout, Flink receives 3 
events:
* MODIFIED: container terminated
* MODIFIED: {{deletionGracePeriodSeconds}} changes from 30 to 0, which is a 
Kubernetes internal status change after containers are gracefully terminated
* DELETED: Flink removes metadata of the terminated pod

Among the 3 messages, Flink is only interested in the 1st MODIFIED message, but 
will try to process all of them because the container status is terminated.

I propose to Filter the verbose events in 
{{KubernetesResourceManagerDriver.PodCallbackHandlerImpl}}, to only process the 
status changes interested by Flink. This probably requires recording the status 
of all living pods, to compare with the incoming events for detecting status 
changes.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


Re: [ANNOUNCE] New PMC member: Dian Fu

2020-08-27 Thread Yang Wang
Congratulations Dian !


Best,
Yang

Arvid Heise  于2020年8月28日周五 上午1:39写道:

> Congrats Dian :)
>
> On Thu, Aug 27, 2020 at 5:01 PM Benchao Li  wrote:
>
>> Congratulations Dian!
>>
>> Cranmer, Danny  于2020年8月27日周四 下午10:55写道:
>>
>>> Congratulations Dian! :D
>>>
>>> On 27/08/2020, 15:25, "Robert Metzger"  wrote:
>>>
>>> CAUTION: This email originated from outside of the organization. Do
>>> not click links or open attachments unless you can confirm the sender and
>>> know the content is safe.
>>>
>>>
>>>
>>> Congratulations Dian!
>>>
>>> On Thu, Aug 27, 2020 at 3:09 PM Congxian Qiu 
>>> wrote:
>>>
>>> > Congratulations Dian
>>> > Best,
>>> > Congxian
>>> >
>>> >
>>> > Xintong Song  于2020年8月27日周四 下午7:50写道:
>>> >
>>> > > Congratulations Dian~!
>>> > >
>>> > > Thank you~
>>> > >
>>> > > Xintong Song
>>> > >
>>> > >
>>> > >
>>> > > On Thu, Aug 27, 2020 at 7:42 PM Jark Wu 
>>> wrote:
>>> > >
>>> > > > Congratulations Dian!
>>> > > >
>>> > > > Best,
>>> > > > Jark
>>> > > >
>>> > > > On Thu, 27 Aug 2020 at 19:37, Leonard Xu 
>>> wrote:
>>> > > >
>>> > > > > Congrats, Dian!  Well deserved.
>>> > > > >
>>> > > > > Best
>>> > > > > Leonard
>>> > > > >
>>> > > > > > 在 2020年8月27日,19:34,Kurt Young  写道:
>>> > > > > >
>>> > > > > > Congratulations Dian!
>>> > > > > >
>>> > > > > > Best,
>>> > > > > > Kurt
>>> > > > > >
>>> > > > > >
>>> > > > > > On Thu, Aug 27, 2020 at 7:28 PM Rui Li <
>>> lirui.fu...@gmail.com>
>>> > > wrote:
>>> > > > > >
>>> > > > > >> Congratulations Dian!
>>> > > > > >>
>>> > > > > >> On Thu, Aug 27, 2020 at 5:39 PM Yuan Mei <
>>> yuanmei.w...@gmail.com>
>>> > > > > wrote:
>>> > > > > >>
>>> > > > > >>> Congrats!
>>> > > > > >>>
>>> > > > > >>> On Thu, Aug 27, 2020 at 5:38 PM Xingbo Huang <
>>> hxbks...@gmail.com
>>> > >
>>> > > > > wrote:
>>> > > > > >>>
>>> > > > >  Congratulations Dian!
>>> > > > > 
>>> > > > >  Best,
>>> > > > >  Xingbo
>>> > > > > 
>>> > > > >  jincheng sun  于2020年8月27日周四
>>> 下午5:24写道:
>>> > > > > 
>>> > > > > > Hi all,
>>> > > > > >
>>> > > > > > On behalf of the Flink PMC, I'm happy to announce that
>>> Dian Fu
>>> > is
>>> > > > now
>>> > > > > > part of the Apache Flink Project Management Committee
>>> (PMC).
>>> > > > > >
>>> > > > > > Dian Fu has been very active on PyFlink component,
>>> working on
>>> > > > various
>>> > > > > > important features, such as the Python UDF and Pandas
>>> > > integration,
>>> > > > > and
>>> > > > > > keeps checking and voting for our releases, and also
>>> has
>>> > > > successfully
>>> > > > > > produced two releases(1.9.3&1.11.1) as RM, currently
>>> working as
>>> > > RM
>>> > > > > to push
>>> > > > > > forward the release of Flink 1.12.
>>> > > > > >
>>> > > > > > Please join me in congratulating Dian Fu for becoming
>>> a Flink
>>> > PMC
>>> > > > > > Member!
>>> > > > > >
>>> > > > > > Best,
>>> > > > > > Jincheng(on behalf of the Flink PMC)
>>> > > > > >
>>> > > > > 
>>> > > > > >>
>>> > > > > >> --
>>> > > > > >> Best regards!
>>> > > > > >> Rui Li
>>> > > > > >>
>>> > > > >
>>> > > > >
>>> > > >
>>> > >
>>> >
>>>
>>>
>>
>> --
>>
>> Best,
>> Benchao Li
>>
>
>
> --
>
> Arvid Heise | Senior Java Developer
>
> 
>
> Follow us @VervericaData
>
> --
>
> Join Flink Forward  - 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
> (Toni) Cheng
>


Re: [VOTE] FLIP-132: Temporal Table DDL and Temporal Table Join

2020-08-27 Thread Leonard Xu
Hi all,

The voting time for FLIP-132 has passed. I'm closing the vote now.

There were 4 + 1 votes, 3 of which are binding:
- Fabian (binding)
- Jark (binding)
- Rui Li (non-binding)
- Kurt (binding) 

There were no -1 votes. Hereby, FLIP-132 has been accepted.

Thanks everyone for participating the discussion and giving valuable feedback!

Best,
Leonard

> 在 2020年8月25日,12:44,Kurt Young  写道:
> 
> +1, making concepts clear and understandable to all the developers is a very 
> important thing.
> Thanks Leonard for driving this. 
> 
> Best,
> Kurt
> 
> On Tue, Aug 25, 2020 at 10:47 AM Rui Li  > wrote:
> +1. Thanks Leonard for driving this.
> 
> On Tue, Aug 25, 2020 at 10:10 AM Jark Wu  > wrote:
> 
> > Thanks Leonard!
> >
> > +1 to the FLIP.
> >
> > Best,
> > Jark
> >
> > On Tue, 25 Aug 2020 at 01:41, Fabian Hueske  > > wrote:
> >
> >> Leonard, Thanks for updating the FLIP!
> >>
> >> +1 to the current version.
> >>
> >> Thanks, Fabian
> >>
> >> Am Mo., 24. Aug. 2020 um 17:56 Uhr schrieb Leonard Xu  >> 
> >> >:
> >>
> >>> Hi all,
> >>>
> >>> I would like to start the vote for FLIP-132 [1], which has been
> >>> discussed and
> >>> reached a consensus in the discussion thread [2].
> >>>
> >>> The vote will be open until 27th August (72h), unless there is an
> >>> objection or not enough votes.
> >>>
> >>> Best,
> >>> Leonard
> >>> [1]
> >>> https://cwiki.apache.org/confluence/display/FLINK/FLIP-132+Temporal+Table+DDL+and+Temporal+Table+Join
> >>>  
> >>> 
> >>>
> >>> [2]
> >>> http://apache-flink-mailing-list-archive.1008284.n3.nabble.com/DISCUSS-FLIP-132-Temporal-Table-DDL-td43483.html
> >>>  
> >>> 
> >>>
> >>>
> >>>
> 
> -- 
> Best regards!
> Rui Li



[jira] [Created] (FLINK-19067) FileNotFoundException when run flink examples on standby JobManager

2020-08-27 Thread JieFang.He (Jira)
JieFang.He created FLINK-19067:
--

 Summary: FileNotFoundException when run flink examples on standby 
JobManager
 Key: FLINK-19067
 URL: https://issues.apache.org/jira/browse/FLINK-19067
 Project: Flink
  Issue Type: Bug
Affects Versions: 1.11.1
Reporter: JieFang.He


1、When run examples/batch/WordCount.jar on standby JobManager,it will fail with 
the exception:

Caused by: java.io.FileNotFoundException: 
/data2/storageDir/default/blob/job_d29414828f614d5466e239be4d3889ac/blob_p-a2ebe1c5aa160595f214b4bd0f39d80e42ee2e93-f458f1c12dc023e78d25f191de1d7c4b
 (No such file or directory)
 at java.io.FileInputStream.open0(Native Method)
 at java.io.FileInputStream.open(FileInputStream.java:195)
 at java.io.FileInputStream.(FileInputStream.java:138)
 at 
org.apache.flink.core.fs.local.LocalDataInputStream.(LocalDataInputStream.java:50)
 at 
org.apache.flink.core.fs.local.LocalFileSystem.open(LocalFileSystem.java:143)
 at 
org.apache.flink.runtime.blob.FileSystemBlobStore.get(FileSystemBlobStore.java:105)
 at 
org.apache.flink.runtime.blob.FileSystemBlobStore.get(FileSystemBlobStore.java:87)
 at 
org.apache.flink.runtime.blob.BlobServer.getFileInternal(BlobServer.java:501)
 at 
org.apache.flink.runtime.blob.BlobServerConnection.get(BlobServerConnection.java:231)
 at 
org.apache.flink.runtime.blob.BlobServerConnection.run(BlobServerConnection.java:117)

 

2、Run examples success on other nodes

3、After run success on the other node, it can run success on the Standby 
JobManager. But run again will fail

 



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Created] (FLINK-19066) InnerJoinITCase.testBigForSpill failed with "java.lang.ClassCastException: org.apache.flink.table.runtime.util.ResettableExternalBuffer$BufferIterator cannot be cast to

2020-08-27 Thread Dian Fu (Jira)
Dian Fu created FLINK-19066:
---

 Summary: InnerJoinITCase.testBigForSpill failed with 
"java.lang.ClassCastException: 
org.apache.flink.table.runtime.util.ResettableExternalBuffer$BufferIterator 
cannot be cast to org.apache.flink.table.data.binary.BinaryRowData" 
 Key: FLINK-19066
 URL: https://issues.apache.org/jira/browse/FLINK-19066
 Project: Flink
  Issue Type: Bug
  Components: Table SQL / Planner
Affects Versions: 1.12.0
Reporter: Dian Fu
 Fix For: 1.12.0


[https://dev.azure.com/apache-flink/apache-flink/_build/results?buildId=5942=logs=904e5037-64c0-5f69-f6d5-e21b89cf6484=39857031-7f0c-5fd5-d730-a19c5794f839]

{code}
Caused by: java.lang.ClassCastException: 
org.apache.flink.table.runtime.util.ResettableExternalBuffer$BufferIterator 
cannot be cast to org.apache.flink.table.data.binary.BinaryRowData
at 
org.apache.flink.table.runtime.util.ResettableExternalBuffer$InMemoryBuffer$InMemoryBufferIterator.next(ResettableExternalBuffer.java:678)
at 
org.apache.flink.table.runtime.util.ResettableExternalBuffer$InMemoryBuffer$InMemoryBufferIterator.next(ResettableExternalBuffer.java:650)
{code}



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


Re: [ANNOUNCE] Introducing the GSoD 2020 Participants.

2020-08-27 Thread Arvid Heise
Welcome. I'm grateful for having you two on board. I'll guess that the
documentation will be top-notched in 3 months judging from your background.

I hope we can learn from the improvements and also apply them to the other
parts of the documentation (or avoid the fixed issues in the future).

On Thu, Aug 27, 2020 at 6:05 AM Jark Wu  wrote:

> Welcome Kartik and Muhammad! Thanks in advance for helping improve Flink
> documentation.
>
> Best,
> Jark
>
> On Thu, 27 Aug 2020 at 03:59, Till Rohrmann  wrote:
>
> > Welcome Muhammad and Kartik! Thanks a lot for helping us with improving
> > Flink's documentation.
> >
> > Cheers,
> > Till
> >
> > On Wed, Aug 26, 2020 at 7:32 PM Konstantin Knauf 
> > wrote:
> >
> > > Welcome Kartik & Muhammad! Looking very much forward to your
> > contributions
> > > :)
> > >
> > > On Wed, Aug 26, 2020 at 5:52 PM Kartik Khare 
> > > wrote:
> > >
> > > > Hi all,
> > > > It's a great opportunity to get to work with you guys. I have always
> > > > admired Flink's performance and simplicity and have been looking
> > forward
> > > to
> > > > contribute more.
> > > >
> > > > Looking forward to exciting next 3 months.
> > > >
> > > > Regards,
> > > > Kartik
> > > >
> > > > On Wed, 26 Aug 2020, 14:42 Marta Paes Moreira, 
> > > > wrote:
> > > >
> > > > > Hi, Everyone!
> > > > >
> > > > > I'd like to officially welcome the applicants that were selected to
> > > work
> > > > > with the Flink community for this year's Google Season of Docs
> (GSoD)
> > > > [1]: *Kartik
> > > > > Khare* and *Muhammad Haseeb Asif*!
> > > > >
> > > > >- Kartik [2] is a software engineer at Walmart Labs and a
> regular
> > > > >contributor to multiple Apache projects. He is also a prolific
> > > writer
> > > > on
> > > > >Medium and has previously published on the Flink blog. Last
> year,
> > he
> > > > >contributed to Apache Airflow as part of GSoD and he's currently
> > > > revamping
> > > > >the Apache Pinot documentation.
> > > > >
> > > > >
> > > > >- Muhammad [3] is a dual degree master student at KTH and TU
> > Berlin,
> > > > >focusing on distributed systems and data intensive processing
> (in
> > > > >particular, performance optimization of state backends). He
> writes
> > > > >frequently about Flink on Medium and you can catch him and his
> > > > colleague
> > > > >Sruthi this Friday at Beam Summit [4]!
> > > > >
> > > > > They will be working to improve the Table API/SQL documentation
> over
> > a
> > > > > 3-month period, with the support of Aljoscha and Seth as mentors.
> > > > >
> > > > > Please give them a warm welcome to the Flink developer community!
> > > > >
> > > > > Marta
> > > > >
> > > > > [1] https://developers.google.com/season-of-docs/docs/participants
> > > > > [2] https://github.com/KKcorps
> > > > > [3] https://www.linkedin.com/in/haseebasif/
> > > > > [4] https://2020.beamsummit.org/sessions/nexmark-beam-flinkndb/
> > > > >
> > > >
> > >
> > >
> > > --
> > >
> > > Konstantin Knauf
> > >
> > > https://twitter.com/snntrable
> > >
> > > https://github.com/knaufk
> > >
> >
>


-- 

Arvid Heise | Senior Java Developer



Follow us @VervericaData

--

Join Flink Forward  - 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
(Toni) Cheng


Re: [DISCUSS] FLIP-138: Declarative Resource management

2020-08-27 Thread Chesnay Schepler
The scheduler doesn't have to wait for one stage to finish. It is still 
aware that the upstream execution vertex has finished, and can 
request/use slots accordingly to schedule the consumer.


This will get more complicated once we allow the scheduler to change the 
parallelism while the job is running, for which we will need some 
enhancements to the network stack to allow the producer to run without 
knowing the consumer parallelism ahead of time. I'm not too clear on the 
details, but we'll some form of keygroup-like approach for sub 
partitions (maxParallelism and all that).



On 27/08/2020 20:05, Zhu Zhu wrote:

Thanks Chesnay for proposing this improvement.
It's of good value to allow jobs to make best use of available 
resources adaptively. Not

to mention it further supports reactive mode.
So big +1 for it.

I have a minor concern about possible regression in certain cases due 
to the proposed
JobVertex-wise scheduling which replaces current ExecutionVertex-wise 
scheduling.
In the proposal, looks to me it requires a stage to finish before its 
consumer stage can be
scheduled. This limitation, however, does not exist in current 
scheduler. In the case that there
exists a POINTWISE BLOCKING edge, the downstream execution region can 
be scheduled
right after its connected upstream execution vertices finishes, even 
before the whole upstream
stage finishes. This allows the region to be launched earlier and make 
use of available resources.

Do we need to let the new scheduler retain this property?

Thanks,
Zhu

Xintong Song mailto:tonysong...@gmail.com>> 
于2020年8月26日周三 下午6:59写道:


Thanks for the quick response.

*Job prioritization, Allocation IDs, Minimum resource
requirements, SlotManager Implementation Plan:* Sounds good to me.

*FLIP-56*
Good point about the trade-off. I believe maximum resource
utilization and
quick deployment are desired in different scenarios. E.g., a long
running
streaming job deserves some deployment latency to improve the resource
utilization, which benefits the entire lifecycle of the job. On
the other
hand, short batch queries may prefer quick deployment, otherwise
the time
for resource allocation might significantly increase the response
time.
It would be good enough for me to bring these questions to attention.
Nothing that I'm aware of should block this FLIP.

Thank you~

Xintong Song



On Wed, Aug 26, 2020 at 5:14 PM Chesnay Schepler
mailto:ches...@apache.org>> wrote:

> Thank you Xintong for your questions!
> Job prioritization
> Yes, the job which declares it's initial requirements first is
prioritized.
> This is very much for simplicity; for example this avoids the
nasty case
> where all jobs get some resources, but none get enough to
actually run the
> job.
> Minimum resource requirements
>
> My bad; at some point we want to allow the JobMaster to declare
a range of
> resources it could use to run a job, for example min=1, target=10,
> max=+inf.
>
> With this model, the RM would then try to balance the resources
such that
> as many jobs as possible are as close to the target state as
possible.
>
> Currently, the minimum/target/maximum resources are all the
same. So the
> notification is sent whenever the current requirements cannot be
met.
> Allocation IDs
> We do intend to, at the very least, remove AllocationIDs on the
> SlotManager side, as they are just not required there.
>
> On the slotpool side we have to keep them around at least until the
> existing Slotpool implementations are removed (not sure whether
we'll fully
> commit to this in 1.12), since the interfaces use AllocationIDs,
which also
> bleed into the JobMaster.
> The TaskExecutor is in a similar position.
> But in the long-term, yes they will be removed, and most usages will
> probably be replaced by the SlotID.
> FLIP-56
>
> Dynamic slot allocations are indeed quite interesting and raise
a few
> questions; for example, the main purpose of it is to ensure maximum
> resource utilization. In that case, should the JobMaster be
allowed to
> re-use a slot it if the task requires less resources than the slot
> provides, or should it always request a new slot that exactly
matches?
>
> There is a trade-off to be made between maximum resource utilization
> (request exactly matching slots, and only re-use exact matches)
and quicker
> job deployment (re-use slot even if they don't exactly match, skip
> round-trip to RM).
>
> As for how to handle the lack of a preemptively known SlotIDs,
that should
> be fine in and of itself; we already handle a similar case when
we request
> a new TaskExecutor to be started. So long as there is some way
to know how
> many resources the 

Re: [DISCUSS] FLIP-138: Declarative Resource management

2020-08-27 Thread Zhu Zhu
Thanks Chesnay for proposing this improvement.
It's of good value to allow jobs to make best use of available resources
adaptively. Not
to mention it further supports reactive mode.
So big +1 for it.

I have a minor concern about possible regression in certain cases due to
the proposed
JobVertex-wise scheduling which replaces current ExecutionVertex-wise
scheduling.
In the proposal, looks to me it requires a stage to finish before its
consumer stage can be
scheduled. This limitation, however, does not exist in current scheduler.
In the case that there
exists a POINTWISE BLOCKING edge, the downstream execution region can be
scheduled
right after its connected upstream execution vertices finishes, even before
the whole upstream
stage finishes. This allows the region to be launched earlier and make use
of available resources.
Do we need to let the new scheduler retain this property?

Thanks,
Zhu

Xintong Song  于2020年8月26日周三 下午6:59写道:

> Thanks for the quick response.
>
> *Job prioritization, Allocation IDs, Minimum resource
> requirements, SlotManager Implementation Plan:* Sounds good to me.
>
> *FLIP-56*
> Good point about the trade-off. I believe maximum resource utilization and
> quick deployment are desired in different scenarios. E.g., a long running
> streaming job deserves some deployment latency to improve the resource
> utilization, which benefits the entire lifecycle of the job. On the other
> hand, short batch queries may prefer quick deployment, otherwise the time
> for resource allocation might significantly increase the response time.
> It would be good enough for me to bring these questions to attention.
> Nothing that I'm aware of should block this FLIP.
>
> Thank you~
>
> Xintong Song
>
>
>
> On Wed, Aug 26, 2020 at 5:14 PM Chesnay Schepler 
> wrote:
>
> > Thank you Xintong for your questions!
> > Job prioritization
> > Yes, the job which declares it's initial requirements first is
> prioritized.
> > This is very much for simplicity; for example this avoids the nasty case
> > where all jobs get some resources, but none get enough to actually run
> the
> > job.
> > Minimum resource requirements
> >
> > My bad; at some point we want to allow the JobMaster to declare a range
> of
> > resources it could use to run a job, for example min=1, target=10,
> > max=+inf.
> >
> > With this model, the RM would then try to balance the resources such that
> > as many jobs as possible are as close to the target state as possible.
> >
> > Currently, the minimum/target/maximum resources are all the same. So the
> > notification is sent whenever the current requirements cannot be met.
> > Allocation IDs
> > We do intend to, at the very least, remove AllocationIDs on the
> > SlotManager side, as they are just not required there.
> >
> > On the slotpool side we have to keep them around at least until the
> > existing Slotpool implementations are removed (not sure whether we'll
> fully
> > commit to this in 1.12), since the interfaces use AllocationIDs, which
> also
> > bleed into the JobMaster.
> > The TaskExecutor is in a similar position.
> > But in the long-term, yes they will be removed, and most usages will
> > probably be replaced by the SlotID.
> > FLIP-56
> >
> > Dynamic slot allocations are indeed quite interesting and raise a few
> > questions; for example, the main purpose of it is to ensure maximum
> > resource utilization. In that case, should the JobMaster be allowed to
> > re-use a slot it if the task requires less resources than the slot
> > provides, or should it always request a new slot that exactly matches?
> >
> > There is a trade-off to be made between maximum resource utilization
> > (request exactly matching slots, and only re-use exact matches) and
> quicker
> > job deployment (re-use slot even if they don't exactly match, skip
> > round-trip to RM).
> >
> > As for how to handle the lack of a preemptively known SlotIDs, that
> should
> > be fine in and of itself; we already handle a similar case when we
> request
> > a new TaskExecutor to be started. So long as there is some way to know
> how
> > many resources the TaskExecutor has in total I do not see a problem at
> the
> > moment. We will get the SlotID eventually by virtue of the heartbeat
> > SlotReport.
> > Implementation plan (SlotManager)
> > You are on the right track. The SlotManager tracks the declared resource
> > requirements, and if the requirements increased it creates a SlotRequest,
> > which then goes through similar code paths as we have at the moment (try
> to
> > find a free slot, if found tell the TM, otherwise try to request new TM).
> > The SlotManager changes are not that substantial to get a working
> version;
> > we have a PoC and most of the work went into refactoring the SlotManager
> > into a more manageable state. (split into several components, stricter
> and
> > simplified Slot life-cycle, ...).
> > Offer/free slots between JM/TM
> > Gotta run, but that's a good question and I'll think about. But I think
> 

Re: [ANNOUNCE] New PMC member: Dian Fu

2020-08-27 Thread Arvid Heise
Congrats Dian :)

On Thu, Aug 27, 2020 at 5:01 PM Benchao Li  wrote:

> Congratulations Dian!
>
> Cranmer, Danny  于2020年8月27日周四 下午10:55写道:
>
>> Congratulations Dian! :D
>>
>> On 27/08/2020, 15:25, "Robert Metzger"  wrote:
>>
>> CAUTION: This email originated from outside of the organization. Do
>> not click links or open attachments unless you can confirm the sender and
>> know the content is safe.
>>
>>
>>
>> Congratulations Dian!
>>
>> On Thu, Aug 27, 2020 at 3:09 PM Congxian Qiu 
>> wrote:
>>
>> > Congratulations Dian
>> > Best,
>> > Congxian
>> >
>> >
>> > Xintong Song  于2020年8月27日周四 下午7:50写道:
>> >
>> > > Congratulations Dian~!
>> > >
>> > > Thank you~
>> > >
>> > > Xintong Song
>> > >
>> > >
>> > >
>> > > On Thu, Aug 27, 2020 at 7:42 PM Jark Wu  wrote:
>> > >
>> > > > Congratulations Dian!
>> > > >
>> > > > Best,
>> > > > Jark
>> > > >
>> > > > On Thu, 27 Aug 2020 at 19:37, Leonard Xu 
>> wrote:
>> > > >
>> > > > > Congrats, Dian!  Well deserved.
>> > > > >
>> > > > > Best
>> > > > > Leonard
>> > > > >
>> > > > > > 在 2020年8月27日,19:34,Kurt Young  写道:
>> > > > > >
>> > > > > > Congratulations Dian!
>> > > > > >
>> > > > > > Best,
>> > > > > > Kurt
>> > > > > >
>> > > > > >
>> > > > > > On Thu, Aug 27, 2020 at 7:28 PM Rui Li <
>> lirui.fu...@gmail.com>
>> > > wrote:
>> > > > > >
>> > > > > >> Congratulations Dian!
>> > > > > >>
>> > > > > >> On Thu, Aug 27, 2020 at 5:39 PM Yuan Mei <
>> yuanmei.w...@gmail.com>
>> > > > > wrote:
>> > > > > >>
>> > > > > >>> Congrats!
>> > > > > >>>
>> > > > > >>> On Thu, Aug 27, 2020 at 5:38 PM Xingbo Huang <
>> hxbks...@gmail.com
>> > >
>> > > > > wrote:
>> > > > > >>>
>> > > > >  Congratulations Dian!
>> > > > > 
>> > > > >  Best,
>> > > > >  Xingbo
>> > > > > 
>> > > > >  jincheng sun  于2020年8月27日周四
>> 下午5:24写道:
>> > > > > 
>> > > > > > Hi all,
>> > > > > >
>> > > > > > On behalf of the Flink PMC, I'm happy to announce that
>> Dian Fu
>> > is
>> > > > now
>> > > > > > part of the Apache Flink Project Management Committee
>> (PMC).
>> > > > > >
>> > > > > > Dian Fu has been very active on PyFlink component,
>> working on
>> > > > various
>> > > > > > important features, such as the Python UDF and Pandas
>> > > integration,
>> > > > > and
>> > > > > > keeps checking and voting for our releases, and also has
>> > > > successfully
>> > > > > > produced two releases(1.9.3&1.11.1) as RM, currently
>> working as
>> > > RM
>> > > > > to push
>> > > > > > forward the release of Flink 1.12.
>> > > > > >
>> > > > > > Please join me in congratulating Dian Fu for becoming a
>> Flink
>> > PMC
>> > > > > > Member!
>> > > > > >
>> > > > > > Best,
>> > > > > > Jincheng(on behalf of the Flink PMC)
>> > > > > >
>> > > > > 
>> > > > > >>
>> > > > > >> --
>> > > > > >> Best regards!
>> > > > > >> Rui Li
>> > > > > >>
>> > > > >
>> > > > >
>> > > >
>> > >
>> >
>>
>>
>
> --
>
> Best,
> Benchao Li
>


-- 

Arvid Heise | Senior Java Developer



Follow us @VervericaData

--

Join Flink Forward  - 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
(Toni) Cheng


Re: [ANNOUNCE] New PMC member: Dian Fu

2020-08-27 Thread Benchao Li
Congratulations Dian!

Cranmer, Danny  于2020年8月27日周四 下午10:55写道:

> Congratulations Dian! :D
>
> On 27/08/2020, 15:25, "Robert Metzger"  wrote:
>
> CAUTION: This email originated from outside of the organization. Do
> not click links or open attachments unless you can confirm the sender and
> know the content is safe.
>
>
>
> Congratulations Dian!
>
> On Thu, Aug 27, 2020 at 3:09 PM Congxian Qiu 
> wrote:
>
> > Congratulations Dian
> > Best,
> > Congxian
> >
> >
> > Xintong Song  于2020年8月27日周四 下午7:50写道:
> >
> > > Congratulations Dian~!
> > >
> > > Thank you~
> > >
> > > Xintong Song
> > >
> > >
> > >
> > > On Thu, Aug 27, 2020 at 7:42 PM Jark Wu  wrote:
> > >
> > > > Congratulations Dian!
> > > >
> > > > Best,
> > > > Jark
> > > >
> > > > On Thu, 27 Aug 2020 at 19:37, Leonard Xu 
> wrote:
> > > >
> > > > > Congrats, Dian!  Well deserved.
> > > > >
> > > > > Best
> > > > > Leonard
> > > > >
> > > > > > 在 2020年8月27日,19:34,Kurt Young  写道:
> > > > > >
> > > > > > Congratulations Dian!
> > > > > >
> > > > > > Best,
> > > > > > Kurt
> > > > > >
> > > > > >
> > > > > > On Thu, Aug 27, 2020 at 7:28 PM Rui Li <
> lirui.fu...@gmail.com>
> > > wrote:
> > > > > >
> > > > > >> Congratulations Dian!
> > > > > >>
> > > > > >> On Thu, Aug 27, 2020 at 5:39 PM Yuan Mei <
> yuanmei.w...@gmail.com>
> > > > > wrote:
> > > > > >>
> > > > > >>> Congrats!
> > > > > >>>
> > > > > >>> On Thu, Aug 27, 2020 at 5:38 PM Xingbo Huang <
> hxbks...@gmail.com
> > >
> > > > > wrote:
> > > > > >>>
> > > > >  Congratulations Dian!
> > > > > 
> > > > >  Best,
> > > > >  Xingbo
> > > > > 
> > > > >  jincheng sun  于2020年8月27日周四
> 下午5:24写道:
> > > > > 
> > > > > > Hi all,
> > > > > >
> > > > > > On behalf of the Flink PMC, I'm happy to announce that
> Dian Fu
> > is
> > > > now
> > > > > > part of the Apache Flink Project Management Committee
> (PMC).
> > > > > >
> > > > > > Dian Fu has been very active on PyFlink component,
> working on
> > > > various
> > > > > > important features, such as the Python UDF and Pandas
> > > integration,
> > > > > and
> > > > > > keeps checking and voting for our releases, and also has
> > > > successfully
> > > > > > produced two releases(1.9.3&1.11.1) as RM, currently
> working as
> > > RM
> > > > > to push
> > > > > > forward the release of Flink 1.12.
> > > > > >
> > > > > > Please join me in congratulating Dian Fu for becoming a
> Flink
> > PMC
> > > > > > Member!
> > > > > >
> > > > > > Best,
> > > > > > Jincheng(on behalf of the Flink PMC)
> > > > > >
> > > > > 
> > > > > >>
> > > > > >> --
> > > > > >> Best regards!
> > > > > >> Rui Li
> > > > > >>
> > > > >
> > > > >
> > > >
> > >
> >
>
>

-- 

Best,
Benchao Li


Re: [ANNOUNCE] New PMC member: Dian Fu

2020-08-27 Thread Cranmer, Danny
Congratulations Dian! :D

On 27/08/2020, 15:25, "Robert Metzger"  wrote:

CAUTION: This email originated from outside of the organization. Do not 
click links or open attachments unless you can confirm the sender and know the 
content is safe.



Congratulations Dian!

On Thu, Aug 27, 2020 at 3:09 PM Congxian Qiu  wrote:

> Congratulations Dian
> Best,
> Congxian
>
>
> Xintong Song  于2020年8月27日周四 下午7:50写道:
>
> > Congratulations Dian~!
> >
> > Thank you~
> >
> > Xintong Song
> >
> >
> >
> > On Thu, Aug 27, 2020 at 7:42 PM Jark Wu  wrote:
> >
> > > Congratulations Dian!
> > >
> > > Best,
> > > Jark
> > >
> > > On Thu, 27 Aug 2020 at 19:37, Leonard Xu  wrote:
> > >
> > > > Congrats, Dian!  Well deserved.
> > > >
> > > > Best
> > > > Leonard
> > > >
> > > > > 在 2020年8月27日,19:34,Kurt Young  写道:
> > > > >
> > > > > Congratulations Dian!
> > > > >
> > > > > Best,
> > > > > Kurt
> > > > >
> > > > >
> > > > > On Thu, Aug 27, 2020 at 7:28 PM Rui Li 
> > wrote:
> > > > >
> > > > >> Congratulations Dian!
> > > > >>
> > > > >> On Thu, Aug 27, 2020 at 5:39 PM Yuan Mei 
> > > > wrote:
> > > > >>
> > > > >>> Congrats!
> > > > >>>
> > > > >>> On Thu, Aug 27, 2020 at 5:38 PM Xingbo Huang  >
> > > > wrote:
> > > > >>>
> > > >  Congratulations Dian!
> > > > 
> > > >  Best,
> > > >  Xingbo
> > > > 
> > > >  jincheng sun  于2020年8月27日周四 下午5:24写道:
> > > > 
> > > > > Hi all,
> > > > >
> > > > > On behalf of the Flink PMC, I'm happy to announce that Dian Fu
> is
> > > now
> > > > > part of the Apache Flink Project Management Committee (PMC).
> > > > >
> > > > > Dian Fu has been very active on PyFlink component, working on
> > > various
> > > > > important features, such as the Python UDF and Pandas
> > integration,
> > > > and
> > > > > keeps checking and voting for our releases, and also has
> > > successfully
> > > > > produced two releases(1.9.3&1.11.1) as RM, currently working 
as
> > RM
> > > > to push
> > > > > forward the release of Flink 1.12.
> > > > >
> > > > > Please join me in congratulating Dian Fu for becoming a Flink
> PMC
> > > > > Member!
> > > > >
> > > > > Best,
> > > > > Jincheng(on behalf of the Flink PMC)
> > > > >
> > > > 
> > > > >>
> > > > >> --
> > > > >> Best regards!
> > > > >> Rui Li
> > > > >>
> > > >
> > > >
> > >
> >
>



Re: [ANNOUNCE] New PMC member: Dian Fu

2020-08-27 Thread Robert Metzger
Congratulations Dian!

On Thu, Aug 27, 2020 at 3:09 PM Congxian Qiu  wrote:

> Congratulations Dian
> Best,
> Congxian
>
>
> Xintong Song  于2020年8月27日周四 下午7:50写道:
>
> > Congratulations Dian~!
> >
> > Thank you~
> >
> > Xintong Song
> >
> >
> >
> > On Thu, Aug 27, 2020 at 7:42 PM Jark Wu  wrote:
> >
> > > Congratulations Dian!
> > >
> > > Best,
> > > Jark
> > >
> > > On Thu, 27 Aug 2020 at 19:37, Leonard Xu  wrote:
> > >
> > > > Congrats, Dian!  Well deserved.
> > > >
> > > > Best
> > > > Leonard
> > > >
> > > > > 在 2020年8月27日,19:34,Kurt Young  写道:
> > > > >
> > > > > Congratulations Dian!
> > > > >
> > > > > Best,
> > > > > Kurt
> > > > >
> > > > >
> > > > > On Thu, Aug 27, 2020 at 7:28 PM Rui Li 
> > wrote:
> > > > >
> > > > >> Congratulations Dian!
> > > > >>
> > > > >> On Thu, Aug 27, 2020 at 5:39 PM Yuan Mei 
> > > > wrote:
> > > > >>
> > > > >>> Congrats!
> > > > >>>
> > > > >>> On Thu, Aug 27, 2020 at 5:38 PM Xingbo Huang  >
> > > > wrote:
> > > > >>>
> > > >  Congratulations Dian!
> > > > 
> > > >  Best,
> > > >  Xingbo
> > > > 
> > > >  jincheng sun  于2020年8月27日周四 下午5:24写道:
> > > > 
> > > > > Hi all,
> > > > >
> > > > > On behalf of the Flink PMC, I'm happy to announce that Dian Fu
> is
> > > now
> > > > > part of the Apache Flink Project Management Committee (PMC).
> > > > >
> > > > > Dian Fu has been very active on PyFlink component, working on
> > > various
> > > > > important features, such as the Python UDF and Pandas
> > integration,
> > > > and
> > > > > keeps checking and voting for our releases, and also has
> > > successfully
> > > > > produced two releases(1.9.3&1.11.1) as RM, currently working as
> > RM
> > > > to push
> > > > > forward the release of Flink 1.12.
> > > > >
> > > > > Please join me in congratulating Dian Fu for becoming a Flink
> PMC
> > > > > Member!
> > > > >
> > > > > Best,
> > > > > Jincheng(on behalf of the Flink PMC)
> > > > >
> > > > 
> > > > >>
> > > > >> --
> > > > >> Best regards!
> > > > >> Rui Li
> > > > >>
> > > >
> > > >
> > >
> >
>


[jira] [Created] (FLINK-19065) java.lang.IllegalStateException: Auto generated UIDs have been disabled on join

2020-08-27 Thread Maris (Jira)
Maris created FLINK-19065:
-

 Summary: java.lang.IllegalStateException: Auto generated UIDs have 
been disabled on join
 Key: FLINK-19065
 URL: https://issues.apache.org/jira/browse/FLINK-19065
 Project: Flink
  Issue Type: Bug
Affects Versions: 1.11.1, 1.11.0
Reporter: Maris


Join operation with AutoGeneratedUID disabled leads to 
{code:java}
java.lang.IllegalStateException: Auto generated UIDs have been disabled but no 
UID or hash has been assigned to operator Map
{code}

code to reproduce
{code:java}
class JoinSpec extends AnyFlatSpec with Matchers with Serializable {
  it should "be able to join streams" in {
val env = StreamExecutionEnvironment.getExecutionEnvironment
env.getConfig.disableAutoGeneratedUIDs()

val a = env.fromCollection(List("1", "2", "3")).name("a").uid("source-uid")
val b = env.fromCollection(List("1", "2", "3")).name("b").uid("source-uid2")
val c = a
  .join(b)
  .where(identity)
  .equalTo(identity)
  .window(TumblingProcessingTimeWindows.of(Time.seconds(30)))((a, b) => a+b)
  .uid("joined").name("joined")

c.addSink(s => println(s))
  .name("ab")
  .uid("ab")

println(env.getExecutionPlan)
env.execute
succeed
  }
}
{code}



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Created] (FLINK-19064) HBaseRowDataInputFormat is leaking resources

2020-08-27 Thread Robert Metzger (Jira)
Robert Metzger created FLINK-19064:
--

 Summary: HBaseRowDataInputFormat is leaking resources
 Key: FLINK-19064
 URL: https://issues.apache.org/jira/browse/FLINK-19064
 Project: Flink
  Issue Type: Bug
  Components: Connectors / HBase
Affects Versions: 1.12.0
Reporter: Robert Metzger


{{HBaseRowDataInputFormat.configure()}} is calling {{connectToTable()}}, which 
creates a connection to HBase that is not closed again.

A user reported this problem on the user@ list: 
https://lists.apache.org/thread.html/ra04f6996eb50ee83aabd2ad0d50bec9afb6a924bfbb48ada3269c6d8%40%3Cuser.flink.apache.org%3E



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


Re: [ANNOUNCE] New PMC member: Dian Fu

2020-08-27 Thread Congxian Qiu
Congratulations Dian
Best,
Congxian


Xintong Song  于2020年8月27日周四 下午7:50写道:

> Congratulations Dian~!
>
> Thank you~
>
> Xintong Song
>
>
>
> On Thu, Aug 27, 2020 at 7:42 PM Jark Wu  wrote:
>
> > Congratulations Dian!
> >
> > Best,
> > Jark
> >
> > On Thu, 27 Aug 2020 at 19:37, Leonard Xu  wrote:
> >
> > > Congrats, Dian!  Well deserved.
> > >
> > > Best
> > > Leonard
> > >
> > > > 在 2020年8月27日,19:34,Kurt Young  写道:
> > > >
> > > > Congratulations Dian!
> > > >
> > > > Best,
> > > > Kurt
> > > >
> > > >
> > > > On Thu, Aug 27, 2020 at 7:28 PM Rui Li 
> wrote:
> > > >
> > > >> Congratulations Dian!
> > > >>
> > > >> On Thu, Aug 27, 2020 at 5:39 PM Yuan Mei 
> > > wrote:
> > > >>
> > > >>> Congrats!
> > > >>>
> > > >>> On Thu, Aug 27, 2020 at 5:38 PM Xingbo Huang 
> > > wrote:
> > > >>>
> > >  Congratulations Dian!
> > > 
> > >  Best,
> > >  Xingbo
> > > 
> > >  jincheng sun  于2020年8月27日周四 下午5:24写道:
> > > 
> > > > Hi all,
> > > >
> > > > On behalf of the Flink PMC, I'm happy to announce that Dian Fu is
> > now
> > > > part of the Apache Flink Project Management Committee (PMC).
> > > >
> > > > Dian Fu has been very active on PyFlink component, working on
> > various
> > > > important features, such as the Python UDF and Pandas
> integration,
> > > and
> > > > keeps checking and voting for our releases, and also has
> > successfully
> > > > produced two releases(1.9.3&1.11.1) as RM, currently working as
> RM
> > > to push
> > > > forward the release of Flink 1.12.
> > > >
> > > > Please join me in congratulating Dian Fu for becoming a Flink PMC
> > > > Member!
> > > >
> > > > Best,
> > > > Jincheng(on behalf of the Flink PMC)
> > > >
> > > 
> > > >>
> > > >> --
> > > >> Best regards!
> > > >> Rui Li
> > > >>
> > >
> > >
> >
>


Re: [ANNOUNCE] Weekly Community Update 2020/31-34

2020-08-27 Thread Robert Metzger
Thanks a lot for doing these updates!

On Tue, Aug 25, 2020 at 10:23 PM Konstantin Knauf  wrote:

> Dear community,
>
> The "weekly" community update is back after a short summer break! This time
> I've tried to cover most of what happened during the last four weeks, but I
> might pick up some older topics in the next weeks' updates, too.
>
> Activity on the dev@ mailing list has picked up quite a bit as feature
> development & design for the next releases of Apache Flink and Apache Flink
> Stateful Functions is going at full steam. In detail:
>
> Flink Development
> ==
>
> * [releases] [Flink 1.12] The work on Flink 1.12 is well underway with
> feature freeze planned for end of October [1]. Our release managers Robert
> & Dian are periodically reminding the developer community of current
> blockers to reduce time during release testing for this release [2].
>
> * [releases] [Stateful Functions 2.2] Igal has started a discussion
> releasing Stateful Functions 2.2. soon (proposed feature freeze:
> September 10). The most notable feature is maybe the option to embed a
> stateful functions module in a DataStream program via DataStream
> Ingress/Egress. Checkout [3] for a full list of the planned features.
>
> * [releases] [Flink 1.10] Flink 1.10.2 was released. [4]
>
> * [apis] Besides the Stateful Functions API, Flink currently has three
> top-level APIs: DataStream (streaming), DataSet (batch) and TableAPI/SQL
> (unified). A major step towards the goal of a truly unified batch and
> stream processing engine is the unification of the DataStream/DataSet APIs.
> This is one of the main topics of the upcoming release(s), specifically:
> * Aljoscha has published FLIP-131 [5] proposing to deprecate and
> eventually drop the DataSet API. In order to still support the same breadth
> of use cases, we need to make sure that all its use cases are covered by
> the two remaining APIs: a unified DataStream API and the Table API. These
> changes are not part of FLIP-131 itself, but are covered in other FLIPs,
> which already exist (like FLIP-27 [6] or FLIP-129 [7]) or will be published
> over the next few weeks like FLIP-134 (see below). [8]
> * Most importantly, FLIP-134 [9] discusses how the DataStream API could
> be used to efficiently execute batch workloads in the future. In essence
> the FLIP proposes to introduce a BATCH and a STREAMING execution mode for
> DataStream programs. The STREAMING mode corresponds to the current
> behavior, while the BATCH mode adjusts the behavior in various areas to fit
> the requirements of batch processing, e.g. pipelined scheduling with region
> failover, blocking shuffles, no checkpointing, no watermarks, ... [10]
>
> * [apis] Time proposes FLIP-136 to improve the interoperability between the
> Data Stream and Table API. The FLIP covers the conversion between
> DataStream <-> Table (incl. cnangelong streams, watermarks, etc.) as well
> as more additional support for working with the Row type in the DataStream
> API. [11]
>
> * [datastream api] Dawid proposes to remove a set of deprecated methods
> from the DataStream API. [12]
>
> * [runtime] Yuan Mei has started a discussion on FLIP-135 to introduce
> task-local recovery. The FLIP is about the introduction of a new
> failover/recovery strategy for Flink Jobs, that trades consistency for
> availability. Specifically, in the case of approximate task-local recovery
> the failure of some tasks would not trigger a restart of the rest of the
> job, but in turn you can expect data loss or duplication. [13]
>
> * [python] Xingbo Huang proposes to extend the support of Pandas/vectorized
> functions from scalar functions to aggregate functions. For more details on
> Pandas support on PyFlink see the blog post linked below. [14]
>
> * [connectors] Aljoscha has started a discussion on dropping support for
> Kafka 0.10/0.11 in Flink 1.12+. [15]
>
> * [connectors] Robert has revived the discussion on adding support for
> Hbase 2.3.x. There is a consensus to add the HBase 2.x connector Apache
> Flink, but no consensus yet on whether to move the existing HBase 1.x from
> the Flink project to Apache Bahir, too. [16]
> 
> [1]
>
> http://apache-flink-mailing-list-archive.1008284.n3.nabble.com/DISCUSS-Planning-Flink-1-12-tp43348.html
> [2]
>
> http://apache-flink-mailing-list-archive.1008284.n3.nabble.com/DISCUSS-Release-1-12-Stale-blockers-and-build-instabilities-tp43477.html
> [3]
>
> http://apache-flink-mailing-list-archive.1008284.n3.nabble.com/Next-Stateful-Functions-Release-tp44063.html
> [4] https://flink.apache.org/news/2020/08/25/release-1.10.2.html
> [5]
> https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=158866741
> [6]
>
> https://cwiki.apache.org/confluence/display/FLINK/FLIP-27%3A+Refactor+Source+Interface?src=contextnavpagetreemode
>
> [7]
>
> 

[jira] [Created] (FLINK-19063) Support join late event from dimension table side in temporal table join

2020-08-27 Thread Leonard Xu (Jira)
Leonard Xu created FLINK-19063:
--

 Summary: Support join late event from dimension table side in 
temporal table join 
 Key: FLINK-19063
 URL: https://issues.apache.org/jira/browse/FLINK-19063
 Project: Flink
  Issue Type: Improvement
  Components: Table SQL / API
Reporter: Leonard Xu


To join late event from dimension table side in temporal table join is a common 
user case

from user-zh mail list[1][3].

And another similar user case is how to enable the faster stream to wait the 
slower stream  in regular stream join[3]. 

I think we can discuss how to support these user cases.   

 

 

[1][http://apache-flink.147419.n8.nabble.com/Flink-join-td6563.html]

[2][http://apache-flink.147419.n8.nabble.com/flinksql-mysql-td3584.html#a3585]

[3][http://apache-flink.147419.n8.nabble.com/Flink-sql-td4435.html#a4436]



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


Re: Next Stateful Functions Release

2020-08-27 Thread Marta Paes Moreira
+1

The feature set also seems quite in line with user feedback, which is
always great to see.

Thanks, Igal!

On Thu, Aug 27, 2020 at 1:34 PM Robert Metzger  wrote:

> +1 to release Statefun 2.2
>
> On Tue, Aug 25, 2020 at 2:16 PM David Anderson 
> wrote:
>
> > Igal,
> >
> > The feature set you propose sounds great to me -- as a user I see
> > plenty there to get excited about. As for the feature freeze date, I
> > don't really have an informed opinion.
> >
> > David
> >
> > On Mon, Aug 24, 2020 at 10:15 AM Igal Shilman 
> wrote:
> > >
> > > Hi Flink devs,
> > >
> > > We have a few upcoming / implemented features for Stateful Functions on
> > the
> > > radar, and would like to give a heads up on what to expect for the next
> > > release:
> > >
> > > 1. Upgrade support for Flink 1.11.x. [FLINK-18812]
> > > 2. Fine grained control on remote state configuration, such as state
> TTL.
> > > [FLINK-17954]
> > > 3. New state construct for dynamic state registration [FLINK-18316]
> > > 4. Add a DataStream API to StateFun [FLINK-19001]
> > > 5. Support async handlers for the Python SDK [FLINK-18518]
> > > 6. Add more metrics around async operations and backpressure
> > [FLINK-19020]
> > > 7. Out-of-box support for common storage systems in flink-statefun
> Docker
> > > image [FLINK-19019]
> > >
> > > With these we think the project will be in a good spot for the next
> > release.
> > > What do you think about aiming at 10.9.2020 for a feature freeze for
> > > StateFun 2.2?
> > >
> > > Kind regards,
> > > Igal.
> >
>


Re: [ANNOUNCE] New PMC member: Dian Fu

2020-08-27 Thread Xintong Song
Congratulations Dian~!

Thank you~

Xintong Song



On Thu, Aug 27, 2020 at 7:42 PM Jark Wu  wrote:

> Congratulations Dian!
>
> Best,
> Jark
>
> On Thu, 27 Aug 2020 at 19:37, Leonard Xu  wrote:
>
> > Congrats, Dian!  Well deserved.
> >
> > Best
> > Leonard
> >
> > > 在 2020年8月27日,19:34,Kurt Young  写道:
> > >
> > > Congratulations Dian!
> > >
> > > Best,
> > > Kurt
> > >
> > >
> > > On Thu, Aug 27, 2020 at 7:28 PM Rui Li  wrote:
> > >
> > >> Congratulations Dian!
> > >>
> > >> On Thu, Aug 27, 2020 at 5:39 PM Yuan Mei 
> > wrote:
> > >>
> > >>> Congrats!
> > >>>
> > >>> On Thu, Aug 27, 2020 at 5:38 PM Xingbo Huang 
> > wrote:
> > >>>
> >  Congratulations Dian!
> > 
> >  Best,
> >  Xingbo
> > 
> >  jincheng sun  于2020年8月27日周四 下午5:24写道:
> > 
> > > Hi all,
> > >
> > > On behalf of the Flink PMC, I'm happy to announce that Dian Fu is
> now
> > > part of the Apache Flink Project Management Committee (PMC).
> > >
> > > Dian Fu has been very active on PyFlink component, working on
> various
> > > important features, such as the Python UDF and Pandas integration,
> > and
> > > keeps checking and voting for our releases, and also has
> successfully
> > > produced two releases(1.9.3&1.11.1) as RM, currently working as RM
> > to push
> > > forward the release of Flink 1.12.
> > >
> > > Please join me in congratulating Dian Fu for becoming a Flink PMC
> > > Member!
> > >
> > > Best,
> > > Jincheng(on behalf of the Flink PMC)
> > >
> > 
> > >>
> > >> --
> > >> Best regards!
> > >> Rui Li
> > >>
> >
> >
>


Re: [ANNOUNCE] New PMC member: Dian Fu

2020-08-27 Thread Jark Wu
Congratulations Dian!

Best,
Jark

On Thu, 27 Aug 2020 at 19:37, Leonard Xu  wrote:

> Congrats, Dian!  Well deserved.
>
> Best
> Leonard
>
> > 在 2020年8月27日,19:34,Kurt Young  写道:
> >
> > Congratulations Dian!
> >
> > Best,
> > Kurt
> >
> >
> > On Thu, Aug 27, 2020 at 7:28 PM Rui Li  wrote:
> >
> >> Congratulations Dian!
> >>
> >> On Thu, Aug 27, 2020 at 5:39 PM Yuan Mei 
> wrote:
> >>
> >>> Congrats!
> >>>
> >>> On Thu, Aug 27, 2020 at 5:38 PM Xingbo Huang 
> wrote:
> >>>
>  Congratulations Dian!
> 
>  Best,
>  Xingbo
> 
>  jincheng sun  于2020年8月27日周四 下午5:24写道:
> 
> > Hi all,
> >
> > On behalf of the Flink PMC, I'm happy to announce that Dian Fu is now
> > part of the Apache Flink Project Management Committee (PMC).
> >
> > Dian Fu has been very active on PyFlink component, working on various
> > important features, such as the Python UDF and Pandas integration,
> and
> > keeps checking and voting for our releases, and also has successfully
> > produced two releases(1.9.3&1.11.1) as RM, currently working as RM
> to push
> > forward the release of Flink 1.12.
> >
> > Please join me in congratulating Dian Fu for becoming a Flink PMC
> > Member!
> >
> > Best,
> > Jincheng(on behalf of the Flink PMC)
> >
> 
> >>
> >> --
> >> Best regards!
> >> Rui Li
> >>
>
>


Re: [ANNOUNCE] New PMC member: Dian Fu

2020-08-27 Thread Leonard Xu
Congrats, Dian!  Well deserved.

Best
Leonard

> 在 2020年8月27日,19:34,Kurt Young  写道:
> 
> Congratulations Dian!
> 
> Best,
> Kurt
> 
> 
> On Thu, Aug 27, 2020 at 7:28 PM Rui Li  wrote:
> 
>> Congratulations Dian!
>> 
>> On Thu, Aug 27, 2020 at 5:39 PM Yuan Mei  wrote:
>> 
>>> Congrats!
>>> 
>>> On Thu, Aug 27, 2020 at 5:38 PM Xingbo Huang  wrote:
>>> 
 Congratulations Dian!
 
 Best,
 Xingbo
 
 jincheng sun  于2020年8月27日周四 下午5:24写道:
 
> Hi all,
> 
> On behalf of the Flink PMC, I'm happy to announce that Dian Fu is now
> part of the Apache Flink Project Management Committee (PMC).
> 
> Dian Fu has been very active on PyFlink component, working on various
> important features, such as the Python UDF and Pandas integration, and
> keeps checking and voting for our releases, and also has successfully
> produced two releases(1.9.3&1.11.1) as RM, currently working as RM to push
> forward the release of Flink 1.12.
> 
> Please join me in congratulating Dian Fu for becoming a Flink PMC
> Member!
> 
> Best,
> Jincheng(on behalf of the Flink PMC)
> 
 
>> 
>> --
>> Best regards!
>> Rui Li
>> 



Re: Next Stateful Functions Release

2020-08-27 Thread Robert Metzger
+1 to release Statefun 2.2

On Tue, Aug 25, 2020 at 2:16 PM David Anderson 
wrote:

> Igal,
>
> The feature set you propose sounds great to me -- as a user I see
> plenty there to get excited about. As for the feature freeze date, I
> don't really have an informed opinion.
>
> David
>
> On Mon, Aug 24, 2020 at 10:15 AM Igal Shilman  wrote:
> >
> > Hi Flink devs,
> >
> > We have a few upcoming / implemented features for Stateful Functions on
> the
> > radar, and would like to give a heads up on what to expect for the next
> > release:
> >
> > 1. Upgrade support for Flink 1.11.x. [FLINK-18812]
> > 2. Fine grained control on remote state configuration, such as state TTL.
> > [FLINK-17954]
> > 3. New state construct for dynamic state registration [FLINK-18316]
> > 4. Add a DataStream API to StateFun [FLINK-19001]
> > 5. Support async handlers for the Python SDK [FLINK-18518]
> > 6. Add more metrics around async operations and backpressure
> [FLINK-19020]
> > 7. Out-of-box support for common storage systems in flink-statefun Docker
> > image [FLINK-19019]
> >
> > With these we think the project will be in a good spot for the next
> release.
> > What do you think about aiming at 10.9.2020 for a feature freeze for
> > StateFun 2.2?
> >
> > Kind regards,
> > Igal.
>


Re: [ANNOUNCE] New PMC member: Dian Fu

2020-08-27 Thread Kurt Young
Congratulations Dian!

Best,
Kurt


On Thu, Aug 27, 2020 at 7:28 PM Rui Li  wrote:

> Congratulations Dian!
>
> On Thu, Aug 27, 2020 at 5:39 PM Yuan Mei  wrote:
>
>> Congrats!
>>
>> On Thu, Aug 27, 2020 at 5:38 PM Xingbo Huang  wrote:
>>
>>> Congratulations Dian!
>>>
>>> Best,
>>> Xingbo
>>>
>>> jincheng sun  于2020年8月27日周四 下午5:24写道:
>>>
 Hi all,

 On behalf of the Flink PMC, I'm happy to announce that Dian Fu is now
 part of the Apache Flink Project Management Committee (PMC).

 Dian Fu has been very active on PyFlink component, working on various
 important features, such as the Python UDF and Pandas integration, and
 keeps checking and voting for our releases, and also has successfully
 produced two releases(1.9.3&1.11.1) as RM, currently working as RM to push
 forward the release of Flink 1.12.

 Please join me in congratulating Dian Fu for becoming a Flink PMC
 Member!

 Best,
 Jincheng(on behalf of the Flink PMC)

>>>
>
> --
> Best regards!
> Rui Li
>


Re: [ANNOUNCE] New PMC member: Dian Fu

2020-08-27 Thread Rui Li
Congratulations Dian!

On Thu, Aug 27, 2020 at 5:39 PM Yuan Mei  wrote:

> Congrats!
>
> On Thu, Aug 27, 2020 at 5:38 PM Xingbo Huang  wrote:
>
>> Congratulations Dian!
>>
>> Best,
>> Xingbo
>>
>> jincheng sun  于2020年8月27日周四 下午5:24写道:
>>
>>> Hi all,
>>>
>>> On behalf of the Flink PMC, I'm happy to announce that Dian Fu is now
>>> part of the Apache Flink Project Management Committee (PMC).
>>>
>>> Dian Fu has been very active on PyFlink component, working on various
>>> important features, such as the Python UDF and Pandas integration, and
>>> keeps checking and voting for our releases, and also has successfully
>>> produced two releases(1.9.3&1.11.1) as RM, currently working as RM to push
>>> forward the release of Flink 1.12.
>>>
>>> Please join me in congratulating Dian Fu for becoming a Flink PMC Member!
>>>
>>> Best,
>>> Jincheng(on behalf of the Flink PMC)
>>>
>>

-- 
Best regards!
Rui Li


[jira] [Created] (FLINK-19062) Improve alter table DDL

2020-08-27 Thread Jingsong Lee (Jira)
Jingsong Lee created FLINK-19062:


 Summary: Improve alter table DDL
 Key: FLINK-19062
 URL: https://issues.apache.org/jira/browse/FLINK-19062
 Project: Flink
  Issue Type: New Feature
  Components: Table SQL / API
Reporter: Jingsong Lee


Support:
 * ALTER TABLE table_name UNSET TBLPROPERTIES (key)
 * ALTER TABLE table_name CHANGE [COLUMN] col_old_name col_new_name column_type 
[COMMENT col_comment]
 * ALTER TABLE table_name add `column` ...
 * ALTER TABLE table_name replace `column` ...



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


Re: Re: [ANNOUNCE] New PMC member: Dian Fu

2020-08-27 Thread Yangze Guo
Congrats Dian!

Best,
Yangze Guo

On Thu, Aug 27, 2020 at 6:26 PM Zhu Zhu  wrote:
>
> Congratulations Dian!
>
> Thanks,
> Zhu
>
> Zhijiang  于2020年8月27日周四 下午6:04写道:
>
> > Congrats, Dian!
> >
> > --
> > From:Yun Gao 
> > Send Time:2020年8月27日(星期四) 17:44
> > To:dev ; Dian Fu ; user <
> > u...@flink.apache.org>; user-zh 
> > Subject:Re: Re: [ANNOUNCE] New PMC member: Dian Fu
> >
> > Congratulations Dian !
> >
> >  Best
> >  Yun
> >
> >
> > --
> > Sender:Marta Paes Moreira
> > Date:2020/08/27 17:42:34
> > Recipient:Yuan Mei
> > Cc:Xingbo Huang; jincheng sun > >; dev; Dian Fu; user<
> > u...@flink.apache.org>; user-zh
> > Theme:Re: [ANNOUNCE] New PMC member: Dian Fu
> >
> > Congrats, Dian!
> > On Thu, Aug 27, 2020 at 11:39 AM Yuan Mei  wrote:
> >
> > Congrats!
> > On Thu, Aug 27, 2020 at 5:38 PM Xingbo Huang  wrote:
> >
> > Congratulations Dian!
> >
> > Best,
> > Xingbo
> > jincheng sun  于2020年8月27日周四 下午5:24写道:
> >
> > Hi all,
> >
> >
> > On behalf of the Flink PMC, I'm happy to announce that Dian Fu is now part 
> > of the Apache Flink Project Management Committee (PMC).
> >
> >
> > Dian Fu has been very active on PyFlink component, working on various 
> > important features, such as the Python UDF and Pandas integration, and 
> > keeps checking and voting for our releases, and also has successfully 
> > produced two releases(1.9.3&1.11.1) as RM, currently working as RM to push 
> > forward the release of Flink 1.12.
> >
> > Please join me in congratulating Dian Fu for becoming a Flink PMC Member!
> >
> > Best,
> > Jincheng(on behalf of the Flink PMC)
> >
> >
> >


Re: Re: [ANNOUNCE] New PMC member: Dian Fu

2020-08-27 Thread Zhu Zhu
Congratulations Dian!

Thanks,
Zhu

Zhijiang  于2020年8月27日周四 下午6:04写道:

> Congrats, Dian!
>
> --
> From:Yun Gao 
> Send Time:2020年8月27日(星期四) 17:44
> To:dev ; Dian Fu ; user <
> u...@flink.apache.org>; user-zh 
> Subject:Re: Re: [ANNOUNCE] New PMC member: Dian Fu
>
> Congratulations Dian !
>
>  Best
>  Yun
>
>
> --
> Sender:Marta Paes Moreira
> Date:2020/08/27 17:42:34
> Recipient:Yuan Mei
> Cc:Xingbo Huang; jincheng sun >; dev; Dian Fu; user<
> u...@flink.apache.org>; user-zh
> Theme:Re: [ANNOUNCE] New PMC member: Dian Fu
>
> Congrats, Dian!
> On Thu, Aug 27, 2020 at 11:39 AM Yuan Mei  wrote:
>
> Congrats!
> On Thu, Aug 27, 2020 at 5:38 PM Xingbo Huang  wrote:
>
> Congratulations Dian!
>
> Best,
> Xingbo
> jincheng sun  于2020年8月27日周四 下午5:24写道:
>
> Hi all,
>
>
> On behalf of the Flink PMC, I'm happy to announce that Dian Fu is now part of 
> the Apache Flink Project Management Committee (PMC).
>
>
> Dian Fu has been very active on PyFlink component, working on various 
> important features, such as the Python UDF and Pandas integration, and keeps 
> checking and voting for our releases, and also has successfully produced two 
> releases(1.9.3&1.11.1) as RM, currently working as RM to push forward the 
> release of Flink 1.12.
>
> Please join me in congratulating Dian Fu for becoming a Flink PMC Member!
>
> Best,
> Jincheng(on behalf of the Flink PMC)
>
>
>


Re: Re: [ANNOUNCE] New PMC member: Dian Fu

2020-08-27 Thread Zhijiang
Congrats, Dian!


--
From:Yun Gao 
Send Time:2020年8月27日(星期四) 17:44
To:dev ; Dian Fu ; user 
; user-zh 
Subject:Re: Re: [ANNOUNCE] New PMC member: Dian Fu

Congratulations Dian !

 Best
 Yun


--
Sender:Marta Paes Moreira
Date:2020/08/27 17:42:34
Recipient:Yuan Mei
Cc:Xingbo Huang; jincheng sun; 
dev; Dian Fu; 
user; user-zh
Theme:Re: [ANNOUNCE] New PMC member: Dian Fu

Congrats, Dian!
On Thu, Aug 27, 2020 at 11:39 AM Yuan Mei  wrote:

Congrats!
On Thu, Aug 27, 2020 at 5:38 PM Xingbo Huang  wrote:

Congratulations Dian!

Best,
Xingbo
jincheng sun  于2020年8月27日周四 下午5:24写道:

Hi all,

On behalf of the Flink PMC, I'm happy to announce that Dian Fu is now part of 
the Apache Flink Project Management Committee (PMC).

Dian Fu has been very active on PyFlink component, working on various important 
features, such as the Python UDF and Pandas integration, and keeps checking and 
voting for our releases, and also has successfully produced two 
releases(1.9.3&1.11.1) as RM, currently working as RM to push forward the 
release of Flink 1.12.

Please join me in congratulating Dian Fu for becoming a Flink PMC Member!

Best,
Jincheng(on behalf of the Flink PMC)



Re: [ANNOUNCE] New PMC member: Dian Fu

2020-08-27 Thread Paul Lam
Congrats, Dian!

Best,
Paul Lam

> 2020年8月27日 17:42,Marta Paes Moreira  写道:
> 
> Congrats, Dian!
> 
> On Thu, Aug 27, 2020 at 11:39 AM Yuan Mei  > wrote:
> Congrats!
> 
> On Thu, Aug 27, 2020 at 5:38 PM Xingbo Huang  > wrote:
> Congratulations Dian!
> 
> Best,
> Xingbo
> 
> jincheng sun mailto:sunjincheng...@gmail.com>> 
> 于2020年8月27日周四 下午5:24写道:
> Hi all,
> 
> On behalf of the Flink PMC, I'm happy to announce that Dian Fu is now part of 
> the Apache Flink Project Management Committee (PMC).
> 
> Dian Fu has been very active on PyFlink component, working on various 
> important features, such as the Python UDF and Pandas integration, and keeps 
> checking and voting for our releases, and also has successfully produced two 
> releases(1.9.3&1.11.1) as RM, currently working as RM to push forward the 
> release of Flink 1.12.
> 
> Please join me in congratulating Dian Fu for becoming a Flink PMC Member!
> 
> Best,
> Jincheng(on behalf of the Flink PMC)



Re: Re: [ANNOUNCE] New PMC member: Dian Fu

2020-08-27 Thread Yun Gao
Congratulations Dian !

 Best
 Yun


--
Sender:Marta Paes Moreira
Date:2020/08/27 17:42:34
Recipient:Yuan Mei
Cc:Xingbo Huang; jincheng sun; 
dev; Dian Fu; 
user; user-zh
Theme:Re: [ANNOUNCE] New PMC member: Dian Fu

Congrats, Dian!
On Thu, Aug 27, 2020 at 11:39 AM Yuan Mei  wrote:

Congrats!
On Thu, Aug 27, 2020 at 5:38 PM Xingbo Huang  wrote:

Congratulations Dian!

Best,
Xingbo
jincheng sun  于2020年8月27日周四 下午5:24写道:

Hi all,

On behalf of the Flink PMC, I'm happy to announce that Dian Fu is now part of 
the Apache Flink Project Management Committee (PMC).

Dian Fu has been very active on PyFlink component, working on various important 
features, such as the Python UDF and Pandas integration, and keeps checking and 
voting for our releases, and also has successfully produced two 
releases(1.9.3&1.11.1) as RM, currently working as RM to push forward the 
release of Flink 1.12.

Please join me in congratulating Dian Fu for becoming a Flink PMC Member!

Best,
Jincheng(on behalf of the Flink PMC)


Re: [ANNOUNCE] New PMC member: Dian Fu

2020-08-27 Thread Marta Paes Moreira
Congrats, Dian!

On Thu, Aug 27, 2020 at 11:39 AM Yuan Mei  wrote:

> Congrats!
>
> On Thu, Aug 27, 2020 at 5:38 PM Xingbo Huang  wrote:
>
>> Congratulations Dian!
>>
>> Best,
>> Xingbo
>>
>> jincheng sun  于2020年8月27日周四 下午5:24写道:
>>
>>> Hi all,
>>>
>>> On behalf of the Flink PMC, I'm happy to announce that Dian Fu is now
>>> part of the Apache Flink Project Management Committee (PMC).
>>>
>>> Dian Fu has been very active on PyFlink component, working on various
>>> important features, such as the Python UDF and Pandas integration, and
>>> keeps checking and voting for our releases, and also has successfully
>>> produced two releases(1.9.3&1.11.1) as RM, currently working as RM to push
>>> forward the release of Flink 1.12.
>>>
>>> Please join me in congratulating Dian Fu for becoming a Flink PMC Member!
>>>
>>> Best,
>>> Jincheng(on behalf of the Flink PMC)
>>>
>>


Re: [DISCUSS] Remove Kafka 0.10.x connector (and possibly 0.11.x)

2020-08-27 Thread Paul Lam
Hi,

I think it’s okay, given that we can either migrate to the universal connector 
or still use the compatible 0.10/0.11 connector of 1.11 release as Chesnay 
mentioned when upgrading to 1.12.

IIUC, the migration process to the universal connector would be (please correct 
me if I’m wrong):
1. Stop the job with a savepoint, committing the offset to Kafka brokers.
2. Modify user code, migrate to he universal connector, and change the source 
operator id to discard the old connector states.
3. Start the job with the savepoint, and read Kafka from group offsets.

Best,
Paul Lam

> 2020年8月27日 16:27,Aljoscha Krettek  写道:
> 
> @Konstantin: Yes, I'm talking about dropping those modules. We don't have any 
> special code for supporting Kafka 0.10/0.11 in the "modern" connector, that 
> comes from the Kafka Consumer/Producer code we're using.
> 
> @Paul: The modern Kafka connector works with Kafka brokers as far back as 
> 0.10, would that be enough or do you still think we should have the actual 
> Kafka 0.10 Consumer code in Flink as well.
> 
> Best,
> Aljoscha
> 
> On 25.08.20 23:15, Chesnay Schepler wrote:
>> +1 to remove both the 1.10 and 1.11 connectors.
>> The connectors have not been actively developed for some time. They are 
>> basically just sitting around causing noise by causing test instabilities 
>> and eating CI time.
>> It would  also allow us to really simplify the module structure of the Kafka 
>> connectors.
>> Users may continue to use the 1.11 version of the connectors with future 
>> Flink versions, and we may even provide critical bug fixes in a 1.11 bugfix 
>> release (albeit unlikely).
>> While ultimately this is a separate topic I would also be in favor of 
>> removing any migration paths we have from 0.11 to the universal connector;
>> as these are already present in 1.11 users may migrate to the universal 
>> connector before jumping to Flink 1.12+.
>> On 25/08/2020 18:49, Konstantin Knauf wrote:
>>> Hi Aljoscha,
>>> 
>>> I am assuming you're asking about dropping the 
>>> flink-connector-kafka-0.10/0.11 modules, right? Or are you talking about 
>>> removing support for Kafka 0.10/0.11 from the universal connector?
>>> 
>>> I am in favor of removing flink-connector-kafka-0.10/0.11 in the next 
>>> release. These modules would still be available in Flink 1.11- as a 
>>> reference, and could be used with Flink 1.12+ with small or no 
>>> modifications. To my knowledge, you also use the universal Kafka connector 
>>> with 0.10 brokers, but there might be a performance penalty if I remember 
>>> correctly. In general, I find it important to continuously reduce baggage 
>>> that accumulates over time and this seems like a good opportunity.
>>> 
>>> Best,
>>> 
>>> Konstantin
>>> 
>>> On Tue, Aug 25, 2020 at 4:59 AM Paul Lam >>  >> >> wrote:
>>> 
>>> Hi Aljoscha,
>>> 
>>> I'm lightly leaning towards keeping the 0.10 connector, for Kafka
>>> 0.10 still has a steady user base in my observation.
>>> 
>>> But if we drop 0.10 connector, can we ensure the users would be
>>> able to smoothly migrate to 0.11 connector/universal connector?
>>> 
>>> If I remember correctly, the universal connector is compatible
>>> with 0.10 brokers, but I want to double check that.
>>> 
>>> Best,
>>> Paul Lam
>>> 
 2020年8月24日 22:46,Aljoscha Krettek >>> 
 >> 写道:
 
 Hi all,
 
 this thought came up on FLINK-17260 [1] but I think it would be a
 good idea in general. The issue reminded us that Kafka didn't
 have an idempotent/fault-tolerant Producer before Kafka 0.11.0.
 By now we have had the "modern" Kafka connector that roughly
 follows new Kafka releases for a while and this one supports
 Kafka cluster versions as far back as 0.10.2.0 (I believe).
 
 What are your thoughts on removing support for older Kafka
 versions? And yes, I know that we had multiple discussions like
 this in the past but I'm trying to gauge the current sentiment.
 
 I'm cross-posting to the user-ml since this is important for both
 users and developers.
 
 Best,
 Aljoscha
 
 [1] https://issues.apache.org/jira/browse/FLINK-17260
>>> 
>>> 
>>> 
>>> -- 
>>> 
>>> Konstantin Knauf
>>> 
>>> https://twitter.com/snntrable
>>> 
>>> https://github.com/knaufk



Re: [ANNOUNCE] New PMC member: Dian Fu

2020-08-27 Thread Yuan Mei
Congrats!

On Thu, Aug 27, 2020 at 5:38 PM Xingbo Huang  wrote:

> Congratulations Dian!
>
> Best,
> Xingbo
>
> jincheng sun  于2020年8月27日周四 下午5:24写道:
>
>> Hi all,
>>
>> On behalf of the Flink PMC, I'm happy to announce that Dian Fu is now
>> part of the Apache Flink Project Management Committee (PMC).
>>
>> Dian Fu has been very active on PyFlink component, working on various
>> important features, such as the Python UDF and Pandas integration, and
>> keeps checking and voting for our releases, and also has successfully
>> produced two releases(1.9.3&1.11.1) as RM, currently working as RM to push
>> forward the release of Flink 1.12.
>>
>> Please join me in congratulating Dian Fu for becoming a Flink PMC Member!
>>
>> Best,
>> Jincheng(on behalf of the Flink PMC)
>>
>


Re: [ANNOUNCE] New PMC member: Dian Fu

2020-08-27 Thread Xingbo Huang
Congratulations Dian!

Best,
Xingbo

jincheng sun  于2020年8月27日周四 下午5:24写道:

> Hi all,
>
> On behalf of the Flink PMC, I'm happy to announce that Dian Fu is now part
> of the Apache Flink Project Management Committee (PMC).
>
> Dian Fu has been very active on PyFlink component, working on various
> important features, such as the Python UDF and Pandas integration, and
> keeps checking and voting for our releases, and also has successfully
> produced two releases(1.9.3&1.11.1) as RM, currently working as RM to push
> forward the release of Flink 1.12.
>
> Please join me in congratulating Dian Fu for becoming a Flink PMC Member!
>
> Best,
> Jincheng(on behalf of the Flink PMC)
>


[ANNOUNCE] New PMC member: Dian Fu

2020-08-27 Thread jincheng sun
Hi all,

On behalf of the Flink PMC, I'm happy to announce that Dian Fu is now part
of the Apache Flink Project Management Committee (PMC).

Dian Fu has been very active on PyFlink component, working on various
important features, such as the Python UDF and Pandas integration, and
keeps checking and voting for our releases, and also has successfully
produced two releases(1.9.3&1.11.1) as RM, currently working as RM to push
forward the release of Flink 1.12.

Please join me in congratulating Dian Fu for becoming a Flink PMC Member!

Best,
Jincheng(on behalf of the Flink PMC)


[jira] [Created] (FLINK-19061) HiveCatalog fails to get partition column stats if partition value contains special characters

2020-08-27 Thread Rui Li (Jira)
Rui Li created FLINK-19061:
--

 Summary: HiveCatalog fails to get partition column stats if 
partition value contains special characters
 Key: FLINK-19061
 URL: https://issues.apache.org/jira/browse/FLINK-19061
 Project: Flink
  Issue Type: Bug
  Components: Connectors / Hive
Reporter: Rui Li
 Fix For: 1.12.0






--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Created] (FLINK-19060) Checkpoint not triggered when use broadcast stream

2020-08-27 Thread henvealf (Jira)
henvealf created FLINK-19060:


 Summary: Checkpoint not triggered when use broadcast stream
 Key: FLINK-19060
 URL: https://issues.apache.org/jira/browse/FLINK-19060
 Project: Flink
  Issue Type: Bug
  Components: API / DataStream
Affects Versions: 1.11.1
Reporter: henvealf
 Attachments: image-2020-08-27-16-41-23-699.png, 
image-2020-08-27-16-44-37-442.png, image-2020-08-27-16-45-28-134.png

Code:

!image-2020-08-27-16-43-30-536.png!

KafkaSourceConfig:

 consumer.setStartFromGroupOffsets()



Web UI:
   !image-2020-08-27-16-45-28-134.png!

Checkpoint always doesn't happen. Did I write something wrong?

Thanks!



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


Re: [DISCUSS] Removing deprecated methods from DataStream API

2020-08-27 Thread Aljoscha Krettek
Did you consider DataStream.project() yet? In general I think we should 
remove most of the relational-ish methods from DataStream. More 
candidates in this set of methods would be the tuple index/expression 
methods for aggregations like min/max etc...


Aljoscha

On 25.08.20 20:52, Konstantin Knauf wrote:

I would argue that the guarantees of @Public methods that became
ineffective were broken when they became ineffective (and were deprecated).

- ExecutionConfig#disable/enableSysoutLogging (deprecated in 1.10)
- ExecutionConfig#set/isFailTaskOnCheckpointError (deprecated in 1.9)

Removing these methods seems like the better of two evils to me as it shows
users that they have been using no-ops for some time.

On Thu, Aug 20, 2020 at 10:50 AM Stephan Ewen  wrote:


We have removed some public methods in the past, after a careful
deprecation period, if they were not well working any more.

The sentiment I got from users is that careful cleanup is in fact
appreciated, otherwise things get confusing over time (the deprecated
methods cause noise in the API).
Still, we need to be very careful here.

I would suggest to
   - start with the non-public breaking methods
   - remove fold() (very long deprecated)
   - remove split() buggy

Removing the env.socketStream() and env.fileStream() methods would
probably be good, too. They are very long deprecated and they don't work
well (with checkpoints) and the sources are the first thing a user needs to
understand when starting with Flink, so removing noise there is super
helpful.


On Thu, Aug 20, 2020 at 8:53 AM Dawid Wysakowicz 
wrote:


Hey Till,

You've got a good point here. Removing some of the methods would cause
breaking the stability guarantees. I do understand if we decide not to
remove them for that reason, let me explain though why I am thinking it
might make sense to remove them already. First of all I am a bit afraid it
might take a long time before we arrive at the 2.0 version. We have not
ever discussed that in the community. At the same time a lot of the methods
already don't work or are buggy, and we do not fix them any more.

Methods which removing would not break the Public guarantees:

- ExecutionConfig#set/getCodeAnalysisMode (deprecated in 1.9)
- RuntimeContext#getAllAccumulators (deprecated in 0.10)
- ExecutionConfig#isLatencyTrackingEnabled (deprecated in 1.7)
- 
StreamExecutionEnvironment#setNumberOfExecutionRetries/getNumberOfExecutionRetries
(not the equivalent in the ExecutionConfig)
- StreamExecutionEnvironment#setStateBackend(AbstractStateBackend)
(deprecated in 1.5)

Methods which removing would break the Public guarantees:

which have no effect:

- ExecutionConfig#disable/enableSysoutLogging (deprecated in 1.10)
- ExecutionConfig#set/isFailTaskOnCheckpointError (deprecated in 1.9)

which are buggy or discouraged and thus we do not support fixing them:

- DataStream#split (deprecated in 1.8)
- DataStream#fold and all related classes and methods such as
FoldFunction, FoldingState, FoldingStateDescriptor ... (deprecated in
1.3/1.4)

The methods like:

- 
StreamExecutionEnvironment#readFile,readFileStream(...),socketTextStream(...),socketTextStream(...),

- methods in (Connected)DataStream that specify keys as either
indices or field names
-
ExecutionConfig#setNumberOfExecutionRetries/getNumberOfExecutionRetries

should be working just fine and I feel the least eager to remove those.

I'd suggest I will open PRs for removing the methods that will not cause
breakage of the Public guarantees as the general feedback was rather
positive. For the rest I do understand the resentment to do so and will not
do it in the 1.x branch. Still I think it is valuable to have the
discussion.

Best,

Dawid


On 18/08/2020 09:26, Till Rohrmann wrote:

Having looked at the proposed set of methods to remove I've noticed that
some of them are actually annotated with @Public. According to our
stability guarantees, only major releases (1.0, 2.0, etc.) can break APIs
with this annotation. Hence, I believe that we cannot simply remove them
unless the community decides to change the stability guarantees we give or
by making the next release a major release (Flink 2.0).

Cheers,
Till

On Tue, Aug 18, 2020 at 5:57 AM Yun Gao  wrote:


+1 for removing the methods that are deprecated for a while & have
alternative methods.

One specific thing is that if we remove the DataStream#split, do we
consider enabling side-output in more operators in the future ? Currently
it should be only available in ProcessFunctions, but not available to other
commonly used UDF like Source or AsyncFunction[1].

One temporary solution occurs to me is to add a ProcessFunction after
the operators want to use side-output. But I think the solution is not very
direct to come up with and if it really works we might add it to the
document of side-output.

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

Best,
  Yun


Re: [DISCUSS] Remove Kafka 0.10.x connector (and possibly 0.11.x)

2020-08-27 Thread Aljoscha Krettek
@Konstantin: Yes, I'm talking about dropping those modules. We don't 
have any special code for supporting Kafka 0.10/0.11 in the "modern" 
connector, that comes from the Kafka Consumer/Producer code we're using.


@Paul: The modern Kafka connector works with Kafka brokers as far back 
as 0.10, would that be enough or do you still think we should have the 
actual Kafka 0.10 Consumer code in Flink as well.


Best,
Aljoscha

On 25.08.20 23:15, Chesnay Schepler wrote:

+1 to remove both the 1.10 and 1.11 connectors.

The connectors have not been actively developed for some time. They are 
basically just sitting around causing noise by causing test 
instabilities and eating CI time.
It would  also allow us to really simplify the module structure of the 
Kafka connectors.


Users may continue to use the 1.11 version of the connectors with future 
Flink versions, and we may even provide critical bug fixes in a 1.11 
bugfix release (albeit unlikely).


While ultimately this is a separate topic I would also be in favor of 
removing any migration paths we have from 0.11 to the universal connector;
as these are already present in 1.11 users may migrate to the universal 
connector before jumping to Flink 1.12+.


On 25/08/2020 18:49, Konstantin Knauf wrote:

Hi Aljoscha,

I am assuming you're asking about dropping the 
flink-connector-kafka-0.10/0.11 modules, right? Or are you talking 
about removing support for Kafka 0.10/0.11 from the universal connector?


I am in favor of removing flink-connector-kafka-0.10/0.11 in the next 
release. These modules would still be available in Flink 1.11- as a 
reference, and could be used with Flink 1.12+ with small or no 
modifications. To my knowledge, you also use the universal Kafka 
connector with 0.10 brokers, but there might be a performance 
penalty if I remember correctly. In general, I find it important 
to continuously reduce baggage that accumulates over time and this 
seems like a good opportunity.


Best,

Konstantin

On Tue, Aug 25, 2020 at 4:59 AM Paul Lam > wrote:


    Hi Aljoscha,

    I'm lightly leaning towards keeping the 0.10 connector, for Kafka
    0.10 still has a steady user base in my observation.

    But if we drop 0.10 connector, can we ensure the users would be
    able to smoothly migrate to 0.11 connector/universal connector?

    If I remember correctly, the universal connector is compatible
    with 0.10 brokers, but I want to double check that.

    Best,
    Paul Lam


    2020年8月24日 22:46,Aljoscha Krettek mailto:aljos...@apache.org>> 写道:

    Hi all,

    this thought came up on FLINK-17260 [1] but I think it would be a
    good idea in general. The issue reminded us that Kafka didn't
    have an idempotent/fault-tolerant Producer before Kafka 0.11.0.
    By now we have had the "modern" Kafka connector that roughly
    follows new Kafka releases for a while and this one supports
    Kafka cluster versions as far back as 0.10.2.0 (I believe).

    What are your thoughts on removing support for older Kafka
    versions? And yes, I know that we had multiple discussions like
    this in the past but I'm trying to gauge the current sentiment.

    I'm cross-posting to the user-ml since this is important for both
    users and developers.

    Best,
    Aljoscha

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




--

Konstantin Knauf

https://twitter.com/snntrable

https://github.com/knaufk








Re: [ANNOUNCE] Apache Flink 1.10.2 released

2020-08-27 Thread Zhijiang
Congrats, thanks for the release manager work Zhu Zhu and everyone involved in!

Best,
Zhijiang
--
From:liupengcheng 
Send Time:2020年8月26日(星期三) 19:37
To:dev ; Xingbo Huang 
Cc:Guowei Ma ; user-zh ; Yangze 
Guo ; Dian Fu ; Zhu Zhu 
; user 
Subject:Re: [ANNOUNCE] Apache Flink 1.10.2 released

Thanks ZhuZhu for managing this release and everyone who contributed to this.

Best,
Pengcheng

 在 2020/8/26 下午7:06,“Congxian Qiu” 写入:

Thanks ZhuZhu for managing this release and everyone else who contributed
to this release!

Best,
Congxian


Xingbo Huang  于2020年8月26日周三 下午1:53写道:

> Thanks Zhu for the great work and everyone who contributed to this 
release!
>
> Best,
> Xingbo
>
> Guowei Ma  于2020年8月26日周三 下午12:43写道:
>
>> Hi,
>>
>> Thanks a lot for being the release manager Zhu Zhu!
>> Thanks everyone contributed to this!
>>
>> Best,
>> Guowei
>>
>>
>> On Wed, Aug 26, 2020 at 11:18 AM Yun Tang  wrote:
>>
>>> Thanks for Zhu's work to manage this release and everyone who
>>> contributed to this!
>>>
>>> Best,
>>> Yun Tang
>>> 
>>> From: Yangze Guo 
>>> Sent: Tuesday, August 25, 2020 14:47
>>> To: Dian Fu 
>>> Cc: Zhu Zhu ; dev ; user <
>>> u...@flink.apache.org>; user-zh 
>>> Subject: Re: [ANNOUNCE] Apache Flink 1.10.2 released
>>>
>>> Thanks a lot for being the release manager Zhu Zhu!
>>> Congrats to all others who have contributed to the release!
>>>
>>> Best,
>>> Yangze Guo
>>>
>>> On Tue, Aug 25, 2020 at 2:42 PM Dian Fu  wrote:
>>> >
>>> > Thanks ZhuZhu for managing this release and everyone else who
>>> contributed to this release!
>>> >
>>> > Regards,
>>> > Dian
>>> >
>>> > 在 2020年8月25日,下午2:22,Till Rohrmann  写道:
>>> >
>>> > Great news. Thanks a lot for being our release manager Zhu Zhu and to
>>> all others who have contributed to the release!
>>> >
>>> > Cheers,
>>> > Till
>>> >
>>> > On Tue, Aug 25, 2020 at 5:37 AM Zhu Zhu  wrote:
>>> >>
>>> >> The Apache Flink community is very happy to announce the release of
>>> Apache Flink 1.10.2, which is the first bugfix release for the Apache 
Flink
>>> 1.10 series.
>>> >>
>>> >> Apache Flink(r) is an open-source stream processing framework for
>>> distributed, high-performing, always-available, and accurate data 
streaming
>>> applications.
>>> >>
>>> >> The release is available for download at:
>>> >> https://flink.apache.org/downloads.html
>>> >>
>>> >> Please check out the release blog post for an overview of the
>>> improvements for this bugfix release:
>>> >> https://flink.apache.org/news/2020/08/25/release-1.10.2.html
>>> >>
>>> >> The full release notes are available in Jira:
>>> >>
>>> 
https://issues.apache.org/jira/secure/ReleaseNote.jspa?projectId=12315522=12347791
>>> >>
>>> >> We would like to thank all contributors of the Apache Flink community
>>> who made this release possible!
>>> >>
>>> >> Thanks,
>>> >> Zhu
>>> >
>>> >
>>>
>>



[jira] [Created] (FLINK-19059) Support to consume retractions for OVER WINDOW operator

2020-08-27 Thread Jark Wu (Jira)
Jark Wu created FLINK-19059:
---

 Summary: Support to consume retractions for OVER WINDOW operator
 Key: FLINK-19059
 URL: https://issues.apache.org/jira/browse/FLINK-19059
 Project: Flink
  Issue Type: New Feature
  Components: Table SQL / Planner
Reporter: Jark Wu


Currently, OVER WINDOW in streaming SQL only support to consume insert-only 
stream and doesn't support to consume retractions (i.e. contains UPDATE/DELETE 
messages). This results in OVER WINDOW can't be used to follow after unbounded 
group by or streaming join. 



--
This message was sent by Atlassian Jira
(v8.3.4#803005)