Re: [DISCUSS] FLIP-156: Runtime Interfaces for Fine-Grained Resource Requirements

2021-01-19 Thread Xintong Song
Thanks for the feedback, Stephan. Actually, your proposal has also come to my mind at some point. And I have some concerns about it. 1. It does not give users the same control as the SSG-based approach. While both approaches do not require specifying for each operator, SSG-based approach suppo

[DISCUSS] Correct time-related function behavior in Flink SQL

2021-01-19 Thread Leonard Xu
Hi, all I want to start the discussion about correcting time-related function behavior in Flink SQL, this is a tricky topic but I think it’s time to address it. Currently some temporal function behaviors are wired to users. 1. When users use a PROCTIME() in SQL, the value of PROCTIME() has a t

[jira] [Created] (FLINK-21023) Task Manager uses the container dir of Job Manager when running flink job on yarn-cluster.

2021-01-19 Thread Tang Yan (Jira)
Tang Yan created FLINK-21023: Summary: Task Manager uses the container dir of Job Manager when running flink job on yarn-cluster. Key: FLINK-21023 URL: https://issues.apache.org/jira/browse/FLINK-21023 Pr

Re: [DISCUSS] Flink configuration from environment variables

2021-01-19 Thread Ingo Bürk
Hi Steven, regarding the hierarchical override, we could even expand the substitution solution to support shell syntax with default values like state.checkpoints.dir: ${CHECKPOINTS_DIR:-path1} such that if the environment variable doesn't exist, path1 will be used. Regards Ingo On Mon, Jan 1

[jira] [Created] (FLINK-21024) Dynamic properties get exposed to job's main method if user parameters are passed

2021-01-19 Thread Matthias (Jira)
Matthias created FLINK-21024: Summary: Dynamic properties get exposed to job's main method if user parameters are passed Key: FLINK-21024 URL: https://issues.apache.org/jira/browse/FLINK-21024 Project: Fl

[jira] [Created] (FLINK-21025) SQLClientHBaseITCase fails when untarring HBase

2021-01-19 Thread Dawid Wysakowicz (Jira)
Dawid Wysakowicz created FLINK-21025: Summary: SQLClientHBaseITCase fails when untarring HBase Key: FLINK-21025 URL: https://issues.apache.org/jira/browse/FLINK-21025 Project: Flink Issue

Re: [DISCUSS] Flink configuration from environment variables

2021-01-19 Thread Ingo Bürk
Hi Yang, 1. As you said I think this doesn't affect Ververica Platform, really, so I'm more than happy to hear and follow the thoughts of people more experienced with Flink than me. 2. I wasn't aware of env.java.opts, but that's definitely a candidate where a user may want to "escape" it so it doe

Re: [ANNOUNCE] Apache Flink 1.12.1 released

2021-01-19 Thread Till Rohrmann
Thanks a lot for driving this release Xintong. This was indeed a release with some obstacles to overcome and you did it very well! Cheers, Till On Tue, Jan 19, 2021 at 5:59 AM Xingbo Huang wrote: > Thanks Xintong for the great work! > > Best, > Xingbo > > Peter Huang 于2021年1月19日周二 下午12:51写道: >

[jira] [Created] (FLINK-21026) Align column list specification with Hive in INSERT statement

2021-01-19 Thread Zhenghua Gao (Jira)
Zhenghua Gao created FLINK-21026: Summary: Align column list specification with Hive in INSERT statement Key: FLINK-21026 URL: https://issues.apache.org/jira/browse/FLINK-21026 Project: Flink

Re: [DISCUSS] Flink configuration from environment variables

2021-01-19 Thread Till Rohrmann
Hi everyone, Thanks for starting this discussion Ingo. I think being able to use env variables to change Flink's configuration will be a very useful feature. Concerning the two approaches I would be in favour of the second approach ($FLINK_CONFIG_S3_ACCESS_KEY) because it does not require the use

Re: Re: [DISCUSS] Dealing with deprecated and legacy code in Flink

2021-01-19 Thread Till Rohrmann
Thanks a lot for starting this discussion Timo. I like the idea of setting more explicit guidelines for deprecating functionality. I really like the idea of adding with the @Deprecated annotation since when the function is deprecated. Based on that one can simply search for features which should b

Re: [Vote] FLIP-157 Migrate Flink Documentation from Jekyll to Hugo

2021-01-19 Thread David Anderson
+1 David On Tue, Jan 19, 2021 at 5:28 AM Forward Xu wrote: > +1 > > Dian Fu 于2021年1月19日周二 上午11:40写道: > > > +1 > > > > > 在 2021年1月19日,上午11:34,Jark Wu 写道: > > > > > > +1 > > > > > > On Tue, 19 Jan 2021 at 01:59, Till Rohrmann > > wrote: > > > > > >> +1, > > >> > > >> Cheers, > > >> Till > > >>

Re: [ANNOUNCE] Apache Flink 1.12.1 released

2021-01-19 Thread Yangze Guo
Thanks Xintong for the great work! Best, Yangze Guo On Tue, Jan 19, 2021 at 4:47 PM Till Rohrmann wrote: > > Thanks a lot for driving this release Xintong. This was indeed a release with > some obstacles to overcome and you did it very well! > > Cheers, > Till > > On Tue, Jan 19, 2021 at 5:59 A

Re: [Vote] FLIP-157 Migrate Flink Documentation from Jekyll to Hugo

2021-01-19 Thread Leonard Xu
+1 Best, Leonard > 在 2021年1月19日,17:32,David Anderson 写道: > > +1 > > David > > On Tue, Jan 19, 2021 at 5:28 AM Forward Xu wrote: > >> +1 >> >> Dian Fu 于2021年1月19日周二 上午11:40写道: >> >>> +1 >>> 在 2021年1月19日,上午11:34,Jark Wu 写道: +1 On Tue, 19 Jan 2021 at 01:59, Till

Re: [ANNOUNCE] Apache Flink 1.12.1 released

2021-01-19 Thread Guowei Ma
Thanks Xintong's effort! Best, Guowei On Tue, Jan 19, 2021 at 5:37 PM Yangze Guo wrote: > Thanks Xintong for the great work! > > Best, > Yangze Guo > > On Tue, Jan 19, 2021 at 4:47 PM Till Rohrmann > wrote: > > > > Thanks a lot for driving this release Xintong. This was indeed a release > with

Re: [DISCUSS] Correct time-related function behavior in Flink SQL

2021-01-19 Thread Leonard Xu
I found above example format may mess up in different mail client, I post a picture here[1]. Best, Leonard [1] https://github.com/leonardBang/flink-sql-etl/blob/master/etl-job/src/main/resources/pictures/CURRRENT_TIMESTAMP.png

[jira] [Created] (FLINK-21027) Add isKeyValueImmutable() method to KeyedStateBackend interface

2021-01-19 Thread Jark Wu (Jira)
Jark Wu created FLINK-21027: --- Summary: Add isKeyValueImmutable() method to KeyedStateBackend interface Key: FLINK-21027 URL: https://issues.apache.org/jira/browse/FLINK-21027 Project: Flink Issue

[jira] [Created] (FLINK-21028) Streaming application didn't stop properly

2021-01-19 Thread Theo Diefenthal (Jira)
Theo Diefenthal created FLINK-21028: --- Summary: Streaming application didn't stop properly Key: FLINK-21028 URL: https://issues.apache.org/jira/browse/FLINK-21028 Project: Flink Issue Type:

[jira] [Created] (FLINK-21029) Failure of shutdown lead to restart of (connected) pipeline

2021-01-19 Thread Theo Diefenthal (Jira)
Theo Diefenthal created FLINK-21029: --- Summary: Failure of shutdown lead to restart of (connected) pipeline Key: FLINK-21029 URL: https://issues.apache.org/jira/browse/FLINK-21029 Project: Flink

[jira] [Created] (FLINK-21030) Broken job restart for job with disjoint graph

2021-01-19 Thread Theo Diefenthal (Jira)
Theo Diefenthal created FLINK-21030: --- Summary: Broken job restart for job with disjoint graph Key: FLINK-21030 URL: https://issues.apache.org/jira/browse/FLINK-21030 Project: Flink Issue Ty

Re: [DISCUSS] Flink configuration from environment variables

2021-01-19 Thread Ufuk Celebi
Hey all, I think that approach 2 is more idiomatic for container deployments where it can be cumbersome to manually map flink-conf.yaml contents to env vars [1]. The precedence order outlined by Till would also cover Steven's hierarchical overwrite requirement. I'm really excited about this fe

Re: [ANNOUNCE] Apache Flink 1.12.1 released

2021-01-19 Thread Wei Zhong
Thanks Xintong for the great work! Best, Wei > 在 2021年1月19日,18:00,Guowei Ma 写道: > > Thanks Xintong's effort! > Best, > Guowei > > > On Tue, Jan 19, 2021 at 5:37 PM Yangze Guo > wrote: > Thanks Xintong for the great work! > > Best, > Yangze Guo > > On Tue, Jan 19,

[jira] [Created] (FLINK-21031) JobMasterStopWithSavepointIT test is not run due to wrong name

2021-01-19 Thread Chesnay Schepler (Jira)
Chesnay Schepler created FLINK-21031: Summary: JobMasterStopWithSavepointIT test is not run due to wrong name Key: FLINK-21031 URL: https://issues.apache.org/jira/browse/FLINK-21031 Project: Flink

[jira] [Created] (FLINK-21032) JsonFileCompactionITCase fails on azure

2021-01-19 Thread Dawid Wysakowicz (Jira)
Dawid Wysakowicz created FLINK-21032: Summary: JsonFileCompactionITCase fails on azure Key: FLINK-21032 URL: https://issues.apache.org/jira/browse/FLINK-21032 Project: Flink Issue Type: B

Re: [DISCUSS] Flink configuration from environment variables

2021-01-19 Thread Till Rohrmann
I think a short FLIP would be awesome. I guess this feature hasn't been implemented yet because it has not been implemented yet ;-) I agree that this feature will improve configuration ergonomics big time :-) Cheers, Till On Tue, Jan 19, 2021 at 12:28 PM Ufuk Celebi wrote: > Hey all, > > I thi

[Announce] SQL docs are now Blink only

2021-01-19 Thread Seth Wiesman
Hi Everyone, I just merged in a PR to make the SQL / Table docs Blink planner only. Going forward, you do not need to mark something as Blink only or explain divergent semantics. Simply write the docs as if Blink were the only planner. There is a Legacy planner specific page[1]. If you add a feat

[jira] [Created] (FLINK-21033) Remove PendingCheckpoint.statsCallback

2021-01-19 Thread Roman Khachatryan (Jira)
Roman Khachatryan created FLINK-21033: - Summary: Remove PendingCheckpoint.statsCallback Key: FLINK-21033 URL: https://issues.apache.org/jira/browse/FLINK-21033 Project: Flink Issue Type:

Re: [DISCUSS] FLIP-156: Runtime Interfaces for Fine-Grained Resource Requirements

2021-01-19 Thread Till Rohrmann
Thanks for the responses Xintong and Stephan, I agree that being able to define the resource requirements for a group of operators is more user friendly. However, my concern is that we are exposing thereby internal runtime strategies which might limit our flexibility to execute a given job. Moreov

Re: [Announce] SQL docs are now Blink only

2021-01-19 Thread Till Rohrmann
Awesome. Thanks a lot Seth! This will help us to keep the docs more easily up to date. Cheers, Till On Tue, Jan 19, 2021 at 4:30 PM Seth Wiesman wrote: > Hi Everyone, > > I just merged in a PR to make the SQL / Table docs Blink planner only. > Going forward, you do not need to mark something as

Re: [DISCUSS] Flink configuration from environment variables

2021-01-19 Thread Steven Wu
Ingo, regarding "state.checkpoints.dir: ${CHECKPOINTS_DIR:-path1}", it definitely can work. but now users need to know that we can use "CHECKPOINTS_DIR" env var to override "state.checkpoints.dir". That is the inconvenience that I am trying to avoid. "state.checkpoints.dir" is well documented in t

[jira] [Created] (FLINK-21034) Rework jemalloc switch to use an environment variable

2021-01-19 Thread Chesnay Schepler (Jira)
Chesnay Schepler created FLINK-21034: Summary: Rework jemalloc switch to use an environment variable Key: FLINK-21034 URL: https://issues.apache.org/jira/browse/FLINK-21034 Project: Flink

[jira] [Created] (FLINK-21035) Deduplicate copy_plugins_if_required calls

2021-01-19 Thread Chesnay Schepler (Jira)
Chesnay Schepler created FLINK-21035: Summary: Deduplicate copy_plugins_if_required calls Key: FLINK-21035 URL: https://issues.apache.org/jira/browse/FLINK-21035 Project: Flink Issue Type

[jira] [Created] (FLINK-21036) Consider removing automatic configuration fo number of slots from docker

2021-01-19 Thread Chesnay Schepler (Jira)
Chesnay Schepler created FLINK-21036: Summary: Consider removing automatic configuration fo number of slots from docker Key: FLINK-21036 URL: https://issues.apache.org/jira/browse/FLINK-21036 Proj

[jira] [Created] (FLINK-21037) Deduplicate configuration logic in docker entrypoint

2021-01-19 Thread Chesnay Schepler (Jira)
Chesnay Schepler created FLINK-21037: Summary: Deduplicate configuration logic in docker entrypoint Key: FLINK-21037 URL: https://issues.apache.org/jira/browse/FLINK-21037 Project: Flink

[jira] [Created] (FLINK-21038) jobmanager.sh may misinterpret webui-port argument

2021-01-19 Thread Chesnay Schepler (Jira)
Chesnay Schepler created FLINK-21038: Summary: jobmanager.sh may misinterpret webui-port argument Key: FLINK-21038 URL: https://issues.apache.org/jira/browse/FLINK-21038 Project: Flink Is

Re: [Announce] SQL docs are now Blink only

2021-01-19 Thread Leonard Xu
Thanks Seth for the Great job ! As the Blink planner has been the default planner and legacy planner is on the way to remove, it makes our document clearer. Best, Leonard > 在 2021年1月19日,23:48,Till Rohrmann 写道: > > Awesome. Thanks a lot Seth! This will help us to keep the docs more easily > u

[jira] [Created] (FLINK-21039) Broken links in "dev/table/legacy_planner.zh.md"

2021-01-19 Thread Huang Xingbo (Jira)
Huang Xingbo created FLINK-21039: Summary: Broken links in "dev/table/legacy_planner.zh.md" Key: FLINK-21039 URL: https://issues.apache.org/jira/browse/FLINK-21039 Project: Flink Issue Type:

[jira] [Created] (FLINK-21040) The exception of "Unknown consumerTag" may throw when RMQSource close() is called.

2021-01-19 Thread pcalpha (Jira)
pcalpha created FLINK-21040: --- Summary: The exception of "Unknown consumerTag" may throw when RMQSource close() is called. Key: FLINK-21040 URL: https://issues.apache.org/jira/browse/FLINK-21040 Project: Fli

Re: [DISCUSS] FLIP-156: Runtime Interfaces for Fine-Grained Resource Requirements

2021-01-19 Thread Xintong Song
Thanks for the feedback, Till. ## I feel that what you proposed (operator-based + default value) might be subsumed by the SSG-based approach. Thinking of op_1 -> op_2, there are the following 4 cases, categorized by whether the resource requirements are known to the users. 1. *Both known.* As

[jira] [Created] (FLINK-21041) Introduce ExecNodeGraph to wrap the ExecNode topology

2021-01-19 Thread godfrey he (Jira)
godfrey he created FLINK-21041: -- Summary: Introduce ExecNodeGraph to wrap the ExecNode topology Key: FLINK-21041 URL: https://issues.apache.org/jira/browse/FLINK-21041 Project: Flink Issue Type:

[jira] [Created] (FLINK-21042) Correct the error in the code example in page 'aggregate-functions'.

2021-01-19 Thread Roc Marshal (Jira)
Roc Marshal created FLINK-21042: --- Summary: Correct the error in the code example in page 'aggregate-functions'. Key: FLINK-21042 URL: https://issues.apache.org/jira/browse/FLINK-21042 Project: Flink

[jira] [Created] (FLINK-21044) SemanticXidGeneratorTest.testXidsUniqueAmongGenerators test failed

2021-01-19 Thread Huang Xingbo (Jira)
Huang Xingbo created FLINK-21044: Summary: SemanticXidGeneratorTest.testXidsUniqueAmongGenerators test failed Key: FLINK-21044 URL: https://issues.apache.org/jira/browse/FLINK-21044 Project: Flink

[jira] [Created] (FLINK-21043) SemanticXidGeneratorTest.testXidsUniqueAmongGenerators test failed

2021-01-19 Thread Huang Xingbo (Jira)
Huang Xingbo created FLINK-21043: Summary: SemanticXidGeneratorTest.testXidsUniqueAmongGenerators test failed Key: FLINK-21043 URL: https://issues.apache.org/jira/browse/FLINK-21043 Project: Flink

[ANNOUNCE] Welcome Guowei Ma as a new Apache Flink Committer

2021-01-19 Thread Kurt Young
Hi everyone, I'm very happy to announce that Guowei Ma has accepted the invitation to become a Flink committer. Guowei is a very long term Flink developer, he has been extremely helpful with some important runtime changes, and also been active with answering user questions as well as discussing

Re: [ANNOUNCE] Welcome Guowei Ma as a new Apache Flink Committer

2021-01-19 Thread tison
Congrats Guowei! Best, tison. Kurt Young 于2021年1月20日周三 下午1:34写道: > Hi everyone, > > I'm very happy to announce that Guowei Ma has accepted the invitation to > become a Flink committer. > > Guowei is a very long term Flink developer, he has been extremely helpful > with > some important runtime

Re: [ANNOUNCE] Welcome Guowei Ma as a new Apache Flink Committer

2021-01-19 Thread Yuan Mei
Congrats Guowei :-) Best, Yuan On Wed, Jan 20, 2021 at 1:36 PM tison wrote: > Congrats Guowei! > > Best, > tison. > > > Kurt Young 于2021年1月20日周三 下午1:34写道: > > > Hi everyone, > > > > I'm very happy to announce that Guowei Ma has accepted the invitation to > > become a Flink committer. > > > > G

Re: [ANNOUNCE] Welcome Guowei Ma as a new Apache Flink Committer

2021-01-19 Thread Dian Fu
Congratulations Guowei! Well deserved! Regards, Dian > 在 2021年1月20日,下午1:42,Yuan Mei 写道: > > Congrats Guowei :-) > > Best, > Yuan > > On Wed, Jan 20, 2021 at 1:36 PM tison wrote: > >> Congrats Guowei! >> >> Best, >> tison. >> >> >> Kurt Young 于2021年1月20日周三 下午1:34写道: >> >>> Hi everyone,

Re: [ANNOUNCE] Welcome Guowei Ma as a new Apache Flink Committer

2021-01-19 Thread Xintong Song
Congratulations, Guowei~! Thank you~ Xintong Song On Wed, Jan 20, 2021 at 1:42 PM Yuan Mei wrote: > Congrats Guowei :-) > > Best, > Yuan > > On Wed, Jan 20, 2021 at 1:36 PM tison wrote: > > > Congrats Guowei! > > > > Best, > > tison. > > > > > > Kurt Young 于2021年1月20日周三 下午1:34写道: > > > >

Re: [ANNOUNCE] Welcome Guowei Ma as a new Apache Flink Committer

2021-01-19 Thread Yu Li
Congratulations and welcome, Guowei! Best Regards, Yu On Wed, 20 Jan 2021 at 13:46, Xintong Song wrote: > Congratulations, Guowei~! > > > Thank you~ > > Xintong Song > > > > On Wed, Jan 20, 2021 at 1:42 PM Yuan Mei wrote: > > > Congrats Guowei :-) > > > > Best, > > Yuan > > > > On Wed, Jan 20

Re: [ANNOUNCE] Welcome Guowei Ma as a new Apache Flink Committer

2021-01-19 Thread Yangze Guo
Congratulations, Guowei! Well deserved. Best, Yangze Guo On Wed, Jan 20, 2021 at 1:46 PM Xintong Song wrote: > > Congratulations, Guowei~! > > > Thank you~ > > Xintong Song > > > > On Wed, Jan 20, 2021 at 1:42 PM Yuan Mei wrote: > > > Congrats Guowei :-) > > > > Best, > > Yuan > > > > On Wed,

Re: Re: [ANNOUNCE] Welcome Guowei Ma as a new Apache Flink Committer

2021-01-19 Thread Yun Gao
Congratulations Guowei! Best, Yun-- Sender:Yangze Guo Date:2021/01/20 13:48:52 Recipient:dev Theme:Re: [ANNOUNCE] Welcome Guowei Ma as a new Apache Flink Committer Congratulations, Guowei! Well deserved. Best, Yangze Guo On Wed,

Re: Re: [ANNOUNCE] Welcome Guowei Ma as a new Apache Flink Committer

2021-01-19 Thread Yang Wang
Congratulations Guowei! Best, Yang Yun Gao 于2021年1月20日周三 下午1:52写道: > Congratulations Guowei! > > Best, > Yun-- > Sender:Yangze Guo > Date:2021/01/20 13:48:52 > Recipient:dev > Theme:Re: [ANNOUNCE] Welcome Guowei Ma as a new Apach

[jira] [Created] (FLINK-21045) Support 'load module' and 'unload module' SQL syntax

2021-01-19 Thread Nicholas Jiang (Jira)
Nicholas Jiang created FLINK-21045: -- Summary: Support 'load module' and 'unload module' SQL syntax Key: FLINK-21045 URL: https://issues.apache.org/jira/browse/FLINK-21045 Project: Flink Issu

Re: [DISCUSS] FLIP-156: Runtime Interfaces for Fine-Grained Resource Requirements

2021-01-19 Thread Yangze Guo
Thanks for the responses, Till and Xintong. I second Xintong's comment that SSG-based runtime interface will give us the flexibility to achieve op/task-based approach. That's one of the most important reasons for our design choice. Some cents regarding the default operator resource: - It might be

[jira] [Created] (FLINK-21046) test_map_view and test_map_view_iterate test failed

2021-01-19 Thread Huang Xingbo (Jira)
Huang Xingbo created FLINK-21046: Summary: test_map_view and test_map_view_iterate test failed Key: FLINK-21046 URL: https://issues.apache.org/jira/browse/FLINK-21046 Project: Flink Issue Typ

Re: [ANNOUNCE] Welcome Guowei Ma as a new Apache Flink Committer

2021-01-19 Thread Leonard Xu
Congratulations Guowei! Best, Leonard

Re: Re: [ANNOUNCE] Welcome Guowei Ma as a new Apache Flink Committer

2021-01-19 Thread Yun Tang
Congratulations Guowei! Best Yun Tang From: Yang Wang Sent: Wednesday, January 20, 2021 13:59 To: dev Subject: Re: Re: [ANNOUNCE] Welcome Guowei Ma as a new Apache Flink Committer Congratulations Guowei! Best, Yang Yun Gao 于2021年1月20日周三 下午1:52写道: > Congratu

Re: Re: [ANNOUNCE] Welcome Guowei Ma as a new Apache Flink Committer

2021-01-19 Thread SHI Xiaogang
Congratulations MA! Regards, Xiaogang Yun Tang 于2021年1月20日周三 下午2:24写道: > Congratulations Guowei! > > Best > Yun Tang > > From: Yang Wang > Sent: Wednesday, January 20, 2021 13:59 > To: dev > Subject: Re: Re: [ANNOUNCE] Welcome Guowei Ma as a new Apache Flink >

[jira] [Created] (FLINK-21047) Expose the correct registered/free resources information in SlotManager

2021-01-19 Thread Yangze Guo (Jira)
Yangze Guo created FLINK-21047: -- Summary: Expose the correct registered/free resources information in SlotManager Key: FLINK-21047 URL: https://issues.apache.org/jira/browse/FLINK-21047 Project: Flink

Re: Re: [ANNOUNCE] Welcome Guowei Ma as a new Apache Flink Committer

2021-01-19 Thread Jark Wu
Congratulations Guowei! Cheers, Jark On Wed, 20 Jan 2021 at 14:36, SHI Xiaogang wrote: > Congratulations MA! > > Regards, > Xiaogang > > Yun Tang 于2021年1月20日周三 下午2:24写道: > > > Congratulations Guowei! > > > > Best > > Yun Tang > > > > From: Yang Wang > > Sent:

[jira] [Created] (FLINK-21048) Refactor documentation related to switch memory allocator

2021-01-19 Thread Yun Tang (Jira)
Yun Tang created FLINK-21048: Summary: Refactor documentation related to switch memory allocator Key: FLINK-21048 URL: https://issues.apache.org/jira/browse/FLINK-21048 Project: Flink Issue Type

Re: Re: [ANNOUNCE] Welcome Guowei Ma as a new Apache Flink Committer

2021-01-19 Thread Danny Chan
Congratulations Guowei! Best, Danny Jark Wu 于2021年1月20日周三 下午2:47写道: > Congratulations Guowei! > > Cheers, > Jark > > On Wed, 20 Jan 2021 at 14:36, SHI Xiaogang wrote: > > > Congratulations MA! > > > > Regards, > > Xiaogang > > > > Yun Tang 于2021年1月20日周三 下午2:24写道: > > > > > Congratulations Guo

Re: Re: [ANNOUNCE] Welcome Guowei Ma as a new Apache Flink Committer

2021-01-19 Thread Arvid Heise
Congratulations! On Wed, Jan 20, 2021 at 7:53 AM Danny Chan wrote: > Congratulations Guowei! > > Best, > Danny > > Jark Wu 于2021年1月20日周三 下午2:47写道: > > > Congratulations Guowei! > > > > Cheers, > > Jark > > > > On Wed, 20 Jan 2021 at 14:36, SHI Xiaogang > wrote: > > > > > Congratulations MA! >

Re: Re: [ANNOUNCE] Welcome Guowei Ma as a new Apache Flink Committer

2021-01-19 Thread Congxian Qiu
Congrats Guowei! Best, Congxian Danny Chan 于2021年1月20日周三 下午2:59写道: > Congratulations Guowei! > > Best, > Danny > > Jark Wu 于2021年1月20日周三 下午2:47写道: > > > Congratulations Guowei! > > > > Cheers, > > Jark > > > > On Wed, 20 Jan 2021 at 14:36, SHI Xiaogang > wrote: > > > > > Congratulations MA!

Re: Re: [ANNOUNCE] Welcome Guowei Ma as a new Apache Flink Committer

2021-01-19 Thread Matthias Pohl
Congrats, Guowei! On Wed, Jan 20, 2021 at 8:22 AM Congxian Qiu wrote: > Congrats Guowei! > > Best, > Congxian > > > Danny Chan 于2021年1月20日周三 下午2:59写道: > > > Congratulations Guowei! > > > > Best, > > Danny > > > > Jark Wu 于2021年1月20日周三 下午2:47写道: > > > > > Congratulations Guowei! > > > > > > Che