Re: [VOTE] FLIP-188 Introduce Built-in Dynamic Table Storage

2021-11-30 Thread Kurt Young
+1 from my side. Best, Kurt On Tue, Nov 30, 2021 at 5:12 PM Jingsong Li wrote: > Hi everyone, > > Many thanks to Stephan and Timo, this makes the design of FLIP much > clearer and more reliable. > > I request that you can take another look at the updated FLIP and > please respond directly if

Re: [DISCUSS] FLIP-194: Introduce the JobResultStore

2021-11-29 Thread Kurt Young
Hi, I didn't fully read the FLIP but the name somehow confused me. My first impression of seeing this is we are providing some storage for job execution results, like the one returned with accumulators in batch mode. Would a name like "JobStautsStore" be more appropriate? Best, Kurt On Mon,

Re: [DISCUSS] Shall casting functions return null or throw exceptions for invalid input

2021-11-22 Thread Kurt Young
lip the switch > before reprocessing and you won't have any issues. If you don't because you > use the hybrid source, either you upgrade your query or you flip the flag > and in both cases this shouldn't generate any issue. > Since it's a big change, I also expect to keep this flag for so

Re: [DISCUSS] Shall casting functions return null or throw exceptions for invalid input

2021-11-19 Thread Kurt Young
ssion applies with TO_TIMESTAMP, which among the others > might even be more crucial because we directly use it in our documentation > to tell users how to compute rowtime. > > FG > > [1] Note: here the naming is a fundamental part of the issue, the function > we have today is

Re: [DISCUSS] Shall casting functions return null or throw exceptions for invalid input

2021-11-18 Thread Kurt Young
t > in the plan. > > Regards, > Timo > > > > > > [1] > > https://docs.microsoft.com/en-us/sql/t-sql/functions/parse-transact-sql?view=sql-server-ver15 > [2] https://issues.apache.org/jira/browse/FLINK-24885 > > > > > > On 18.11.21 11:34, Kurt

Re: [DISCUSS] Shall casting functions return null or throw exceptions for invalid input

2021-11-18 Thread Kurt Young
Sorry I forgot to add user ML. I also would like to gather some users feedback on this thing. Since I didn't get any feedback on this topic before from users. Best, Kurt On Thu, Nov 18, 2021 at 6:33 PM Kurt Young wrote: > (added user ML to this thread) > > HI all, > > I woul

Re: [DISCUSS] Shall casting functions return null or throw exceptions for invalid input

2021-11-18 Thread Kurt Young
(added user ML to this thread) HI all, I would like to raise a different opinion about this change. I agree with Ingo that we should not just break some existing behavior, and even if we introduce an option to control the behavior, i would propose to set the default value to current behavior. I

Re: [ANNOUNCE] New Apache Flink Committer - Jing Zhang

2021-11-15 Thread Kurt Young
Congrats Jing! Best, Kurt On Tue, Nov 16, 2021 at 9:52 AM Xintong Song wrote: > Congratulations~! > > Thank you~ > > Xintong Song > > > > On Tue, Nov 16, 2021 at 9:50 AM Dian Fu wrote: > > > Congratulations! > > > > Regards, > > Dian > > > > On Tue, Nov 16, 2021 at 9:48 AM godfrey he wrote:

Re: [VOTE] FLIP-188 Introduce Built-in Dynamic Table Storage

2021-11-09 Thread Kurt Young
+1 (binding) Best, Kurt On Wed, Nov 10, 2021 at 10:52 AM Jingsong Li wrote: > Hi everyone, > > Thanks for all the feedback so far. Based on the discussion[1] we seem > to have consensus, so I would like to start a vote on FLIP-188 for > which the FLIP has now also been updated[2]. > > The

Re: [DISCUSS] FLIP-189: SQL Client Usability Improvements

2021-11-01 Thread Kurt Young
Really cool improvements @Sergey. Can't wait to see it happen. Best, Kurt On Tue, Nov 2, 2021 at 1:56 AM Martijn Visser wrote: > Hi Sergey, > > I guess you've just set a new standard ;-) I agree with Ingo, these > improvements look really good! > > Best regards, > > Martijn > > On Mon, 1 Nov

Re: [DISCUSS] FLIP-188: Introduce Built-in Dynamic Table Storage

2021-10-31 Thread Kurt Young
Hi Till, We have discussed the possibility of putting this FLIP into another repository offline with Stephan and Timo. This looks similar with another under going effort which trying to put all connectors outside the Flink core repository. >From the motivation and scope of this FLIP, it's quite

Re: [VOTE] Release 1.14.0, release candidate #1

2021-09-15 Thread Kurt Young
I noticed that a serious performance degradation has been reported [1], shall we wait for the conclusion for that issue? [1] https://issues.apache.org/jira/browse/FLINK-24300 Best, Kurt On Wed, Sep 15, 2021 at 3:29 PM Dawid Wysakowicz wrote: > Hi everyone, > Please review and vote on the

Re: One idea for Graceful Application Evolvement

2021-09-02 Thread Kurt Young
Could you explain why you need a backfill after you take v2 into production? Best, Kurt On Fri, Sep 3, 2021 at 2:02 AM zhihao wang wrote: > Hi team > > Graceful Application Evolvement is a hard and open problem to the > community. We met this problem in our production, too. To address it, we

[ANNOUNCE] New PMC member: Guowei Ma

2021-07-06 Thread Kurt Young
Hi all! I'm very happy to announce that Guowei Ma has joined the Flink PMC! Congratulations and welcome Guowei! Best, Kurt

Re: [ANNOUNCE] Criteria for merging pull requests is updated

2021-07-02 Thread Kurt Young
It seems disabling the merge button was only proposed during the release testing phase, which IMO doesn't mean we can't use it forever. Best, Kurt On Fri, Jul 2, 2021 at 3:01 PM Xintong Song wrote: > It was part of the draft proposed in this mail [1]. And before that, it was > brought up

Re: [DISCUSS] Feedback Collection Jira Bot

2021-06-30 Thread Kurt Young
+1 to Stephan's opinion, with just one minor difference. For my experience and a project as big as Flink, picking up an issue created 1-2 years ago seems normal to me. To be more specific, during the blink planner merge, I created lots of clean up & refactor issues, trying to make the code be more

Re: [Discuss] Planning Flink 1.14

2021-06-02 Thread Kurt Young
Thanks for bringing this up. I have one thought about the release period. In a short word: shall we try to extend the release period for 1 month? There are a couple of reasons why I want to bring up this proposal. 1) I observed that lots of users are actually far behind the current Flink

Re: [DISCUSS] SQL CTAS Syntax

2021-05-28 Thread Kurt Young
Hi Konstantin, >From my understanding, this syntax has 2 major benefits: 1. Just like you said, it saves the effort to specify the schema, especially when involving hundreds of fields. 2. When using CREATE TABLE xx AS TABLE yy, it gives us the possibility to enable schema evolution, and it seems

Re: Re: [ANNOUNCE] New Apache Flink Committer - Rui Li

2021-04-22 Thread Kurt Young
Congratulations Rui! Best, Kurt On Thu, Apr 22, 2021 at 2:10 PM Arvid Heise wrote: > Congrats! > > Best, > > Arvid > > On Thu, Apr 22, 2021 at 8:07 AM Xingbo Huang wrote: > > > Congratulations Rui~! > > > > Best, > > Xingbo > > > > Xintong Song 于2021年4月22日周四 下午1:58写道: > > > > > Congrats,

[jira] [Created] (FLINK-22278) Refactor sql client's DynamicResult

2021-04-14 Thread Kurt Young (Jira)
Kurt Young created FLINK-22278: -- Summary: Refactor sql client's DynamicResult Key: FLINK-22278 URL: https://issues.apache.org/jira/browse/FLINK-22278 Project: Flink Issue Type: Improvement

[jira] [Created] (FLINK-22277) Refactor sql client's DynamicResult

2021-04-14 Thread Kurt Young (Jira)
Kurt Young created FLINK-22277: -- Summary: Refactor sql client's DynamicResult Key: FLINK-22277 URL: https://issues.apache.org/jira/browse/FLINK-22277 Project: Flink Issue Type: Improvement

[jira] [Created] (FLINK-22178) Support ignore-first-line option in new csv format

2021-04-09 Thread Kurt Young (Jira)
Kurt Young created FLINK-22178: -- Summary: Support ignore-first-line option in new csv format Key: FLINK-22178 URL: https://issues.apache.org/jira/browse/FLINK-22178 Project: Flink Issue Type

Re: [DISCUSS] Feature freeze date for 1.13

2021-04-07 Thread Kurt Young
w morning. >>> >>> This is a critical fix as now predicate pushdown won't work for any >>> stream which generates a watermark and wants to push down predicates. >>> >>> On Thu, Apr 1, 2021, 10:56 Kurt Young wrote: >>> >>>> Thanks Dawid, I ha

Re: [DISCUSS] Feature freeze date for 1.13

2021-04-01 Thread Kurt Young
ate from other > components, and it's a super useful feature for Flink users. > > Best, > > Arvid > > [1] https://github.com/apache/flink/pull/15054 > > On Thu, Apr 1, 2021 at 6:21 AM Kurt Young wrote: > >> Hi Guowei and Dawid, >> >> I want to request th

Re: [DISCUSS] Feature freeze date for 1.13

2021-03-31 Thread Kurt Young
Hi Guowei and Dawid, I want to request the permission to merge this feature [1], it's a useful improvement to sql client and won't affect other components too much. We were plan to merge it yesterday but met some tricky multi-process issue which has a very high possibility hanging the tests. It

Re: [DISCUSS] FLIP-162 follow-up discussion

2021-03-08 Thread Kurt Young
Hi Leonard, Thanks for this careful consideration. Given the fallback option will eventually change the behavior twice, which means potentially break user's job twice, I would also +1 to not introduce it. Best, Kurt On Fri, Mar 5, 2021 at 3:00 PM Leonard Xu wrote: > Hi, all > > As the

Re: [VOTE] FLIP-162: Consistent Flink SQL time function behavior

2021-03-03 Thread Kurt Young
+1 (binding) Best, Kurt On Wed, Mar 3, 2021 at 3:43 PM Timo Walther wrote: > +1 (binding) > > Regards, > Timo > > On 03.03.21 04:14, Jark Wu wrote: > > +1 (binding) > > > > Best, > > Jark > > > > On Tue, 2 Mar 2021 at 10:42, Leonard Xu wrote: > > > >> Hi all, > >> > >> I would like to start

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

2021-03-01 Thread Kurt Young
+1 (binding) - We mainly checked the patch of FLINK-20663 [1] and confirmed there is no OutOfManagedMemory error anymore. [1] https://issues.apache.org/jira/browse/FLINK-20663 Best, Kurt On Tue, Mar 2, 2021 at 12:41 PM Yu Li wrote: > +1 (binding) > > - Checked the diff between 1.12.1 and

Re: [DISCUSS]FLIP-163: SQL Client Improvements

2021-03-01 Thread Kurt Young
pens asynchronously, such an option could prevent this to > happen again. > > Regards, > Timo > > On 01.03.21 05:14, Kurt Young wrote: > > I also asked some users about their opinion that if we introduce some > > config prefixed with "table" but doesn't > > have

Re: [DISCUSS] FLIP-162: Consistent Flink SQL time function behavior

2021-03-01 Thread Kurt Young
tic batch data or streaming data. > >>>>>>> > >>>>>>> From my understanding, the "semantic" means the "eventual result". > >>>>>>> And time functions are non-deterministic, so it's reasonable to get > >&

Re: [DISCUSS]FLIP-163: SQL Client Improvements

2021-02-28 Thread Kurt Young
>>> If we want to introduce an unified "table.dml-sync" option, I prefer > >>> it should be implemented on Table API and affect all the DMLs on > >>> Table API (`tEnv.executeSql`, `Table.executeInsert`, `StatementSet`), > >>> as I have mentioned

Re: [DISCUSS] Deprecation and removal of the legacy SQL planner

2021-02-25 Thread Kurt Young
Hi Timo, First of all I want to thank you for introducing this planner design back in 1.9, this is a great work that allows lots of blink features to be merged to Flink in a reasonably short time. It greatly accelerates the evolution speed of Table & SQL. Everything comes with a cost, as you

Re: [DISCUSS]FLIP-163: SQL Client Improvements

2021-02-23 Thread Kurt Young
ust to clarify the implementation: The option should be handled by the > SQL Client only, but the name can be shared accross platforms. > > Regards, > Timo > > > On 23.02.21 09:54, Kurt Young wrote: > > Sorry for the late reply, but I'm confused by `table.multi-dml-sync`.

Re: [DISCUSS]FLIP-163: SQL Client Improvements

2021-02-23 Thread Kurt Young
Sorry for the late reply, but I'm confused by `table.multi-dml-sync`. IIUC this config will take effect with 2 use cases: 1. SQL client, either interactive mode or executing multiple statements via -f. In most cases, there will be only one INSERT INTO statement but we are controlling the

Re: [VOTE] FLIP-152: Hive Query Syntax Compatibility

2021-02-07 Thread Kurt Young
+1 Best, Kurt On Sun, Feb 7, 2021 at 7:24 PM Rui Li wrote: > Hi everyone, > > I think we have reached some consensus on FLIP-152 [1] in the discussion > thread [2]. So I'd like to start the vote for this FLIP. > > The vote will be open for 72 hours, until Feb. 10 2021 01:00 PM UTC, unless >

Re: [DISCUSS] Releasing Apache Flink 1.12.2

2021-02-05 Thread Kurt Young
Thanks for being our release manager Yuan. We found a out of memory issue [1] which will affect most batch jobs thus I think it would be great if we can include this fix in 1.12.2. [1] https://issues.apache.org/jira/browse/FLINK-20663 Best, Kurt On Sat, Feb 6, 2021 at 12:36 AM Till Rohrmann

Re: [DISCUSS] FLIP-162: Consistent Flink SQL time function behavior

2021-02-02 Thread Kurt Young
could offer an option to make them happy. If it turns out that we had wrong estimation about the user's expectation, we should change the default behavior. Best, Kurt On Tue, Feb 2, 2021 at 4:46 PM Kurt Young wrote: > Hi Timo, > > I don't think batch-stream unification can deal with all

Re: [DISCUSS] FLIP-162: Consistent Flink SQL time function behavior

2021-02-02 Thread Kurt Young
lue per-record in streaming mode and > evaluates > >>>> time function value at query start in batch mode. > >>>> > >>>> Alternative1: > >>>> Introduce function like > CURRENT_TIMESTAMP2/CURRENT_TIMESTAMP_NOW > &

[jira] [Created] (FLINK-21236) Don't explicitly use HeapMemorySegment in row format serde

2021-02-01 Thread Kurt Young (Jira)
Kurt Young created FLINK-21236: -- Summary: Don't explicitly use HeapMemorySegment in row format serde Key: FLINK-21236 URL: https://issues.apache.org/jira/browse/FLINK-21236 Project: Flink Issue

Re: [DISCUSS] FLIP-162: Consistent Flink SQL time function behavior

2021-01-31 Thread Kurt Young
gt;>>>> I’ll start the vote soon if there’re no objections. > >>>>>> > >>>>>> Best, > >>>>>> Leonard > >>>>>> > >>>>>> [1] > >>>>>> > https://docs.google.co

Re: [DISCUSS] FLIP-162: Consistent Flink SQL time function behavior

2021-01-25 Thread Kurt Young
ent behavior with just showing > UTC is wrong. Also, we all agree that when calling CURRENT_TIMESTAMP or > PROCTIME a user would like to see the time in it's current time zone. > > > > As you said, "my wall clock time". > > > > However, the question is what is the

Re: about flink calcite implement

2021-01-25 Thread Kurt Young
Flink will convert all projections and filters to calc before converting to logical nodes. You can check out FlinkStreamRuleSets.LOGICAL_RULES Best, Kurt On Mon, Jan 25, 2021 at 11:03 AM laughing.sh...@qq.com < laughing.sh...@qq.com> wrote: > > In flink table planner module, I can not find the

Re: [DISCUSS] FLIP-162: Consistent Flink SQL time function behavior

2021-01-22 Thread Kurt Young
Forgot one more thing. Continue with displaying in UTC. As a user, if Flink want to display the timestamp in UTC, why don't we offer something like UTC_TIMESTAMP? Best, Kurt On Fri, Jan 22, 2021 at 4:33 PM Kurt Young wrote: > Before jumping into technique details, let's take a step b

Re: [DISCUSS] FLIP-162: Consistent Flink SQL time function behavior

2021-01-22 Thread Kurt Young
e return value changes sound reasonable to me, and keeping the return > > type unchanged will minimize the surprise to the users. > > Besides that, I think it would be better to mention how this affects the > > window behaviors, and the interoperability with DataStream. > > &

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

2021-01-20 Thread Kurt Young
cc this to user & user-zh mailing list because this will affect lots of users, and also quite a lot of users were asking questions around this topic. Let me try to understand this from user's perspective. Your proposal will affect five functions, which are: - PROCTIME() - NOW() -

[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: [DISCUSS] Backport broadcast operations in BATCH mode to Flink

2021-01-13 Thread Kurt Young
+1 Best, Kurt On Thu, Jan 14, 2021 at 12:25 AM Seth Wiesman wrote: > +1 > > I would hope this helps attract more early adopters so if there are issues > we can resolve them in time for 1.13. > > Seth > > On Wed, Jan 13, 2021 at 5:13 AM Dawid Wysakowicz > wrote: > > > Hi, > > > > Given that

Re: Support local aggregate push down for Blink batch planner

2021-01-04 Thread Kurt Young
#2 which handle all in > the physical optimization phase? > > Looking forward for the further discussion. > > > Kurt Young 于2021年1月5日周二 上午9:52写道: > >> Local aggregation is more like a physical operator rather than logical >> operator. I would suggest going with idea

Re: Support local aggregate push down for Blink batch planner

2021-01-04 Thread Kurt Young
Local aggregation is more like a physical operator rather than logical operator. I would suggest going with idea #1. Best, Kurt On Wed, Dec 30, 2020 at 8:31 PM Sebastian Liu wrote: > Hi Jark, Thx a lot for your quick reply and valuable suggestions. > For (1): Agree: Since we are in the period

Re: [DISCUSS] FLIP-152: Hive Query Syntax Compatibility

2020-12-06 Thread Kurt Young
Thanks Rui for starting this discussion. I can see the benefit that we improve hive compatibility further, as quite some users are asking for this feature in mailing lists [1][2][3] and some online chatting tools such as DingTalk. I have 3 comments regarding to the design doc: a) Could you add

Re: Hive Streaming write compaction

2020-11-18 Thread Kurt Young
We just added this feature to 1.12 [1][2], it would be great that you can download the 1.12 RC to test it out, and give us some feedback. In case you will wonder why I linked 2 jiras, it's because both FileSystem & Hive connector share the same option options and also the implementations. [1]

Re: [VOTE] NEW FLIP-104: Add More Metrics to JobManager

2020-10-28 Thread Kurt Young
+1 Best, Kurt On Wed, Oct 28, 2020 at 2:44 PM Robert Metzger wrote: > Thank you for your UI work Yadong! > > +1 > > On Tue, Oct 27, 2020 at 6:33 PM Matthias Pohl > wrote: > > > Thanks for restarting the vote, Yadong. I really like your UI proposals. > > +1 for adding the changes of FLIP-104.

Re: [VOTE] FLIP-148: Introduce Sort-Merge Based Blocking Shuffle to Flink

2020-10-25 Thread Kurt Young
+1 (binding) Best, Kurt On Mon, Oct 26, 2020 at 11:19 AM Yingjie Cao wrote: > Hi devs, > > I'd like to start a vote for FLIP-148: Introduce Sort-Merge Based Blocking > Shuffle to Flink [1] which is discussed in discussion thread [2]. > > The vote will last for at least 72 hours until a

Re: [DISCUSS] FLIP-149: Introduce the KTable Connector

2020-10-23 Thread Kurt Young
gt;>>> [8]: > > >>>>> > > >>>>> > > >>>> > > >> > > > https://docs.oracle.com/en/database/other-databases/nosql-database/18.3/sqlfornosql/adding-table-rows-using-insert-and-upsert-statements.html > > >&g

Re: [DISCUSS] FLIP-149: Introduce the KTable Connector

2020-10-22 Thread Kurt Young
apply all the constraints mentioned in the FLIP. When > > > >> `model` is set to `table`. > > > >> > > > >> What do you think? > > > >> > > > >> Regards, > > > >> Timo > > > >> > > >

Re: [DISCUSS] FLIP-149: Introduce the KTable Connector

2020-10-20 Thread Kurt Young
Hi all, I want to describe the discussion process which drove us to have such conclusion, this might make some of the design choices easier to understand and keep everyone on the same page. Back to the motivation, what functionality do we want to provide in the first place? We got a lot of

Re: [DISCUSS] Release 1.12 Feature Freeze

2020-10-19 Thread Kurt Young
Can we change the freeze date to October 30th (Friday next week)? It would be helpful for us if we have 2 more days. Best, Kurt On Mon, Oct 19, 2020 at 5:00 PM Robert Metzger wrote: > Hi all, > > Dian and I would like to discuss a few items regarding the upcoming Flink > 1.12 feature freeze:

Re: [ANNOUNCE] New PMC member: Zhu Zhu

2020-10-09 Thread Kurt Young
Congratulations, Zhu Zhu! Best, Kurt On Sat, Oct 10, 2020 at 11:03 AM Yang Wang wrote: > Congratulations! Zhu Zhu. > > Best, > Yang > > Xintong Song 于2020年10月9日周五 下午3:35写道: > > > Congratulations, Zhu~! > > > > Thank you~ > > > > Xintong Song > > > > > > > > On Fri, Oct 9, 2020 at 3:17 PM

Re: [VOTE] FLIP-136: Improve interoperability between DataStream and Table API

2020-09-24 Thread Kurt Young
+1 (binding) Best, Kurt On Thu, Sep 24, 2020 at 4:01 PM Timo Walther wrote: > Hi all, > > after the discussion in [1], I would like to open a second voting thread > for FLIP-136 [2] which covers different topic to improve the > back-and-forth communication between DataStream API and Table

Re: [DISCUSS] FLIP-146: Improve new TableSource and TableSink interfaces

2020-09-24 Thread Kurt Young
rallel > was too heavy and it was causing the queries to fail. > So we had to limit the source parallelism to 2 threads. After the fetching > of the data the other operators could use the max parallelism as usual.. > > Best, > Flavio > > On Thu, Sep 24, 2020 at 9:59 AM Kurt Young

Re: [DISCUSS] FLIP-146: Improve new TableSource and TableSink interfaces

2020-09-24 Thread Kurt Young
Thanks Jingsong for driving this, this is indeed a useful feature and lots of users are asking for it. For setting a fixed source parallelism, I'm wondering whether this is necessary. For kafka, I can imagine users would expect Flink will use the number of partitions as the parallelism. If it's

Re: [DISCUSS] FLIP-136: Improve interoperability between DataStream and Table API

2020-09-23 Thread Kurt Young
= Row.withNames(); > > row.setField("b", 2); > > row.setField("a", 1); > > row2.equals(row1) > > The row is just a container for the serializer/converter which will > ensure ordering. > > Regards, > Timo > > On 23.09.20 15:00, Kurt Young wr

Re: [DISCUSS] FLIP-136: Improve interoperability between DataStream and Table API

2020-09-23 Thread Kurt Young
t the Row instance. During > serialization/conversion we can reorder fields, throw exceptions about > unknown field names, and set remaining fields to NULL. > > If a user uses `new Row(5)` but the serializer is configured by a data > type that only supports `Row(3)`, it will also thro

Re: [DISCUSS] FLIP-136: Improve interoperability between DataStream and Table API

2020-09-23 Thread Kurt Young
Sorry for being late, I went through the design doc and here are my comments: 1. A minor one, how about moving Schema after DataStream in all affected APIs? Such as: StreamTableEnvironment.fromDataStream(Schema, DataStream): Table StreamTableEnvironment.createTemporaryView(String, Schema,

Re: Re: [ANNOUNCE] New Apache Flink Committer - Godfrey He

2020-09-16 Thread Kurt Young
Congrats and welcome, Godfrey! Best, Kurt On Wed, Sep 16, 2020 at 4:59 PM Xintong Song wrote: > Congratulations, Godfrey~! > > Thank you~ > > Xintong Song > > > > On Wed, Sep 16, 2020 at 3:28 PM Yu Li wrote: > > > Congrats and welcome, Godfrey! > > > > Best Regards, > > Yu > > > > > > On

Re: [DISCUSS] FLIP-107: Reading table columns from different parts of source records

2020-09-10 Thread Kurt Young
USING SYSTEM_METADATA("ts") >> > > > > > > timestamp INT FROM SYSTEM_METADATA("ts") >> > > > > > > timestamp INT FROM SYSTEM_METADATA("ts") PERSISTED >> > > > > > > >> > > > > &g

Re: [DISCUSS] FLIP-140: Introduce bounded style execution for keyed streams

2020-09-09 Thread Kurt Young
end" per operator. Would that suffice? > > > > Ad. 2 > > > > I still think we can just use the first X bytes of the serialized form > > as the normalized key and fallback to comparing full keys on clashes. It > > is because we are actually not interested i

Re: [DISCUSS] FLIP-107: Reading table columns from different parts of source records

2020-09-09 Thread Kurt Young
nly a > >> "slightly rejected alternative". > >> > >> Furthermore, we would need to think about how such a new design > >> influences the LIKE clause though. > >> > >> However, we should still keep the `PERSISTED` keyword as it

Re: [DISCUSS] FLIP-107: Reading table columns from different parts of source records

2020-09-09 Thread Kurt Young
s to. > >>>>> "ts_ms": 1589355504100, # 3. timestamp of the > change > >>>> happened in database system, i.e.: transaction time in database. > >>>>> "connector": "mysql", > &g

Re: [DISCUSS] FLIP-140: Introduce bounded style execution for keyed streams

2020-09-09 Thread Kurt Young
a logical order, but we > care only about the "grouping" aspect of the sorting. Therefore I think > its enough to compare only parts of the full key as the normalized key. > > Thanks again for the really nice and thorough feedback! > > Best, > > Dawid > > On 08/

Re: [DISCUSS] FLIP-140: Introduce bounded style execution for keyed streams

2020-09-08 Thread Kurt Young
key prepended we would have to deserialize the record > for every key comparison. > > Therefore if we agree that we perform binary comparison for keys (which > are always prepended), it is actually equivalent to a DataSet with > TypeComparators that support key normalization. > &

Re: [DISCUSS] FLIP-140: Introduce bounded style execution for keyed streams

2020-09-07 Thread Kurt Young
Hi Dawid, thanks for bringing this up, it's really exciting to see that batch execution is introduced in DataStream. From the flip, it seems we are sticking with sort based execution mode (at least for now), which will sort the whole input data before any *keyed* operation is executed. I have two

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

2020-08-28 Thread Kurt Young
A quick question, does network memory treated as managed memory now? Or in the future? Best, Kurt On Wed, Aug 26, 2020 at 5:32 PM Xintong Song wrote: > Hi devs, > > I'd like to bring the discussion over FLIP-141[1], which proposes how > managed memory should be shared by various use cases

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 >>> >>>

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

2020-08-24 Thread 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

Re: [DISCUSS] FLIP-134: DataStream Semantics for Bounded Input

2020-08-16 Thread Kurt Young
Hi Kostas, Thanks for starting this discussion. The first part of this FLIP: "Batch vs Streaming Scheduling" looks reasonable to me. However, there is another dimension I think we should also take into consideration, which is whether checkpointing is enabled. This option is orthogonal (but not

Re: [DISCUSS] FLIP-107: Reading table columns from different parts of source records

2020-08-11 Thread Kurt Young
The content length of FLIP-107 is relatively short but the scope and implications it will cause is actually very big. >From what I can tell now, I think there is a good chance that we can deliver part of this FLIP in 1.12, e.g. accessing the metadata field just like you mentioned. Best, Kurt On

Re: [DISCUSS] Planning Flink 1.12

2020-08-03 Thread Kurt Young
Regarding setting the feature freeze date to late September, I have some concern that it might make the development time of 1.12 too short. One reason for this is we took too much time (about 1.5 month, from mid of May to beginning of July) for testing 1.11. It's not ideal but further squeeze the

Re: [DISCUSS] Introduce SupportsParallelismReport and SupportsStatisticsReport for Hive and Filesystem

2020-07-31 Thread Kurt Young
b-class will extend from "SupportsXXX" only if it > > has the capability, > > So the unbounded table source should not extend from > > SupportsStatisticsReport, > > or just return unknown for unbounded if a table source can work for both > > bounded and unbou

Re: [DISCUSS] FLIP-131: Consolidate the user-facing Dataflow SDKs/APIs (and deprecate the DataSet API)

2020-07-30 Thread Kurt Young
+1, looking forward to the follow up FLIPs. Best, Kurt On Thu, Jul 30, 2020 at 6:40 PM Arvid Heise wrote: > +1 of getting rid of the DataSet API. Is DataStream#iterate already > superseding DataSet iterations or would that also need to be accounted for? > > In general, all surviving APIs

Re: [DISCUSS] Introduce SupportsParallelismReport and SupportsStatisticsReport for Hive and Filesystem

2020-07-30 Thread Kurt Young
Hi Jingsong, Thanks for bringing up this discussion. In general, I'm +1 to enrich the source ability by the parallelism and stats reporting, but I'm not sure whether introducing such "Supports" interface is a good idea. I will share my thoughts separately. 1) Regarding the interface

Re: Kinesis Performance Issue (was [VOTE] Release 1.11.0, release candidate #4)

2020-07-23 Thread Kurt Young
>From my experience, java profilers are sometimes not accurate enough to find out the performance regression root cause. In this case, I would suggest you try out intel vtune amplifier to watch more detailed metrics. Best, Kurt On Fri, Jul 24, 2020 at 8:51 AM Thomas Weise wrote: > The cause

Re: [DISCUSS] FLIP-36 - Support Interactive Programming in Flink Table API

2020-07-22 Thread Kurt Young
thanks for pointing out the use of not existing API used in the given > examples. I have updated the examples in the FLIP accordingly. > > Best, > Xuannan > On Jul 16, 2020, 4:15 PM +0800, Kurt Young , wrote: > > Hi Xuanna, > > > > Thanks for the detailed design doc,

Re: [DISCUSS] FLIP-36 - Support Interactive Programming in Flink Table API

2020-07-16 Thread Kurt Young
Hi Xuanna, Thanks for the detailed design doc, it described clearly how the API looks and how to interact with Flink runtime. However, the part which relates to SQL's optimizer is kind of blurry. To be more precise, I have following questions: 1. How do you identify the CachedTable? I can

Re: Improvement idea: Naming the maven modules.

2020-07-15 Thread Kurt Young
+1, I also like this idea. Best, Kurt On Wed, Jul 15, 2020 at 7:10 PM Niels Basjes wrote: > Ok, > > I'll put up a fix > https://issues.apache.org/jira/browse/FLINK-18607 > > Niels > > On Wed, Jul 15, 2020 at 11:23 AM Aljoscha Krettek > wrote: > > > Hi, > > > > I like the proposal! I remember

Re: [DISCUSS] Sql-client lack support for new features

2020-07-01 Thread Kurt Young
Thanks Jingsong for bringing this up discussion and sorry for the late reply. I'm in general +1 for #1, and want to expand the scope of #2. First of all, I think the approach Jingsong proposed in #2 can help with covering more e2e use cases of SQL, which also draws a clean line between how to

Re: [DISCUSS] [FLINK-16824] Creating Temporal Table Function via DDL

2020-06-23 Thread Kurt Young
nk this source should emit INSERT/UPDATE (UPSERT) messages instead > of > > INSERT-only messages, e.g. a Kafka compacted topic source? > > > > Best, > > Jark > > > > > > On Mon, 22 Jun 2020 at 17:04, Konstantin Knauf > wrote: > > >

Re: [DISCUSS] [FLINK-16824] Creating Temporal Table Function via DDL

2020-06-22 Thread Kurt Young
I agree with Timo, semantic about primary key needs more thought and discussion, especially after FLIP-95 and FLIP-105. Best, Kurt On Mon, Jun 22, 2020 at 4:45 PM Timo Walther wrote: > Hi Leonard, > > thanks for the summary. > > After reading all of the previous arguments and working on

[jira] [Created] (FLINK-18224) Add document about sql client's tableau result mode

2020-06-09 Thread Kurt Young (Jira)
Kurt Young created FLINK-18224: -- Summary: Add document about sql client's tableau result mode Key: FLINK-18224 URL: https://issues.apache.org/jira/browse/FLINK-18224 Project: Flink Issue Type

Re: [ANNOUNCE] New Flink Committer: Benchao Li

2020-06-09 Thread Kurt Young
Congratulations, Benchao! Best, Kurt On Tue, Jun 9, 2020 at 2:46 PM Guanghui Zhang wrote: > Congratulations, Benchao !!! > > Leonard Xu 于2020年6月9日周二 下午2:39写道: > > > Congratulations, Benchao ! > > > > Best, > > Leonard Xu > > > > > > > > > 在 2020年6月9日,14:36,Jiayi Liao 写道: > > > > > >

Re: [DISCUSS] FLIP-84 Feedback Summary

2020-05-17 Thread Kurt Young
; Hi everyone, > >>>>>> > > > >> > >>>>>> > > > >> I would like to bring up a discussion about the result type > >>>>>> of > >>>>>> > > describe > >>>>>> > > > >> state

[jira] [Created] (FLINK-17756) Drop table/view shouldn't take affect on each other

2020-05-16 Thread Kurt Young (Jira)
Kurt Young created FLINK-17756: -- Summary: Drop table/view shouldn't take affect on each other Key: FLINK-17756 URL: https://issues.apache.org/jira/browse/FLINK-17756 Project: Flink Issue Type

[jira] [Created] (FLINK-17749) Remove fromTableSource method from TableEnvironment

2020-05-15 Thread Kurt Young (Jira)
Kurt Young created FLINK-17749: -- Summary: Remove fromTableSource method from TableEnvironment Key: FLINK-17749 URL: https://issues.apache.org/jira/browse/FLINK-17749 Project: Flink Issue Type

[jira] [Created] (FLINK-17748) Remove registration of TableSource/TableSink in Table Env

2020-05-15 Thread Kurt Young (Jira)
Kurt Young created FLINK-17748: -- Summary: Remove registration of TableSource/TableSink in Table Env Key: FLINK-17748 URL: https://issues.apache.org/jira/browse/FLINK-17748 Project: Flink Issue

Re: [DISCUSS] [FLINK-16824] Creating Temporal Table Function via DDL

2020-05-13 Thread Kurt Young
; doesn't look very suitable for our purposes. > I'm sure we can come up with a better syntax for this. > > Best, Fabian > > Am Sa., 9. Mai 2020 um 03:57 Uhr schrieb Kurt Young : > >> All tables being described by Flink's DDL are dynamic tables. But dynamic >> table is mo

[jira] [Created] (FLINK-17635) Add documentation about view support

2020-05-12 Thread Kurt Young (Jira)
Kurt Young created FLINK-17635: -- Summary: Add documentation about view support Key: FLINK-17635 URL: https://issues.apache.org/jira/browse/FLINK-17635 Project: Flink Issue Type: Sub-task

[jira] [Created] (FLINK-17599) Update documents due to FLIP-84

2020-05-10 Thread Kurt Young (Jira)
Kurt Young created FLINK-17599: -- Summary: Update documents due to FLIP-84 Key: FLINK-17599 URL: https://issues.apache.org/jira/browse/FLINK-17599 Project: Flink Issue Type: Sub-task

Re: [DISCUSS] [FLINK-16824] Creating Temporal Table Function via DDL

2020-05-08 Thread Kurt Young
tream with a specific encoding. > The TEMPORAL VIEW would be a shortcut which would allow us to perform the > conversion in Flink SQL (and not within the connector) and defining the > temporal properties on the result of the view. > > Cheers, > Fabian > > > > Am Fr., 8.

Re: What's the best practice to determine whether a job has finished or not?

2020-05-08 Thread Kurt Young
+dev Best, Kurt On Fri, May 8, 2020 at 3:35 PM Caizhi Weng wrote: > Hi Jeff, > > Thanks for the response. However I'm using executeAsync so that I can run > the job asynchronously and get a JobClient to monitor the job. JobListener > only works for synchronous execute method. Is there other

Re: [DISCUSS] [FLINK-16824] Creating Temporal Table Function via DDL

2020-05-08 Thread Kurt Young
I might missed something but why we need a new "TEMPORAL TABLE" syntax? According to Fabian's first mail: > Hence, the requirements for a temporal table are: > * The temporal table has a primary key / unique attribute > * The temporal table has a time-attribute that defines the start of the >

  1   2   3   4   5   >