Re: [DISCUSS ] HybridSouce Table & Sql api timeline

2022-05-09 Thread Ran Tao
Hi, Becket. Thanks for your suggestions. My id is: Ran Tao And i will draft this flip in a few days. thanks~ Becket Qin 于2022年5月10日周二 12:40写道: > Hi Ran, > > The FLIP process can be found here[1]. > > You don't need to pass the vote, in fact the vote is based on the FLIP > wiki. So drafting the

Re: [DISCUSS ] HybridSouce Table & Sql api timeline

2022-05-09 Thread Becket Qin
Hi Ran, The FLIP process can be found here[1]. You don't need to pass the vote, in fact the vote is based on the FLIP wiki. So drafting the FLIP wiki would be the first step. After that you may start a discussion thread in the mailing list so people can have the discussion about the feature

Re:Re: Re: Re: [DISCUSS] FLIP-229: Introduces Join Hint for Flink SQL Batch Job

2022-05-09 Thread Xuyang
Hi Cao. If i guess right, what you mean is the scope that the join hint takes affect. In your example 1, the expected result is that the hint will not takes affect both on (t1 join t2) and (t3 join t1) if we have the sql following, "SELECT /*+ SHUFFLE_HASH(test1) */ t1.a FROM (select test1.*

Re: [DISCUSS] FLIP-91: Support SQL Client Gateway

2022-05-09 Thread Shengkai Fang
Hi, everyone. Thanks for all the inputs. Hope my feedback can address most of questions. After the long discussion, I think we have reached the consensus about the design. If the discussion doesn't get more response, I think we can start the vote tomorrow. Best, Shengkai Shengkai Fang

Re: [DISCUSS ] HybridSouce Table & Sql api timeline

2022-05-09 Thread Ran Tao
Hi, Martijn, Jacky. Thanks for your responding. It indeed need a designed doc or FLIP to illustrate some details and concrete implementation. And i'm glad to work on this issue. I wonder whether i can create a FLIP under discussion firstly to write the draft design of the implementation about

Re: [DISCUSS] FLIP-224: Blacklist Mechanism

2022-05-09 Thread Becket Qin
Hi Lijie, Thanks for updating the FLIP. It looks like the public interface section did not fully reflect all the user sensible behavior and API. Can you put everything that users may be aware of there? That would include the REST API, metrics, configurations, public java Interfaces or pluggables

[jira] [Created] (FLINK-27560) Refactor SimpleStateRequestHandler for PyFlink state

2022-05-09 Thread Juntao Hu (Jira)
Juntao Hu created FLINK-27560: - Summary: Refactor SimpleStateRequestHandler for PyFlink state Key: FLINK-27560 URL: https://issues.apache.org/jira/browse/FLINK-27560 Project: Flink Issue Type:

[jira] [Created] (FLINK-27559) Some question about flink operator state

2022-05-09 Thread Underwood (Jira)
Underwood created FLINK-27559: - Summary: Some question about flink operator state Key: FLINK-27559 URL: https://issues.apache.org/jira/browse/FLINK-27559 Project: Flink Issue Type: New Feature

Re: [DISCUSS] FLIP-226: Introduce Schema Evolution on Table Store

2022-05-09 Thread Jingsong Li
Hi Jark, Thanks for your feedback. > 1) Does table-store support evolve schemas multiple times during a checkpoint? In this case this checkpoint is split into multiple commits, e.g.: - commit1: write 1 million rows - commit1: write 1 million rows - commit2: evolve mode 1 - commit3: write 1

Re: Contributing to Flink

2022-05-09 Thread Atri Sharma
Hi Martijn, I am happy to start wherever you recommend, but typically like moving to complex tasks soon. My interest and expertise lies in runtime and query processing. Atri On Mon, May 9, 2022 at 8:13 PM Martijn Visser wrote: > > Hi Atri, thanks for reaching out! How familiar are you with

[jira] [Created] (FLINK-27558) Introduce a new optional option for TableStoreFactory to represent planned manifest entries

2022-05-09 Thread Jane Chan (Jira)
Jane Chan created FLINK-27558: - Summary: Introduce a new optional option for TableStoreFactory to represent planned manifest entries Key: FLINK-27558 URL: https://issues.apache.org/jira/browse/FLINK-27558

Re: Contributing to Flink

2022-05-09 Thread Martijn Visser
Hi Atri, thanks for reaching out! How familiar are you with Flink? I could imagine that it might make sense to start with addressing a test stability to understand the Flink codebase, but you could of course also help out with a review of a PR. For example,

Re: Contributing to Flink

2022-05-09 Thread Atri Sharma
Gentle reminder, please On Mon, 9 May 2022, 01:15 Atri Sharma, wrote: > Hi Folks, > > I have recently started using Flink and am excited to contribute to > the same. I am looking at two subcomponents: > > >

Re: [Discuss] Creating an Apache Flink slack workspace

2022-05-09 Thread Robert Metzger
Thanks a lot for your answer. The onboarding experience to the ASF Slack is indeed not ideal: https://apisix.apache.org/docs/general/join#join-the-slack-channel I'll see if we can improve it On Mon, May 9, 2022 at 3:38 PM Martijn Visser wrote: > As far as I recall you can't sign up for the ASF

Re: [Discuss] Creating an Apache Flink slack workspace

2022-05-09 Thread Martijn Visser
As far as I recall you can't sign up for the ASF instance of Slack, you can only get there if you're a committer or if you're invited by a committer. On Mon, 9 May 2022 at 15:15, Robert Metzger wrote: > Sorry for joining this discussion late, and thanks for the summary Xintong! > > Why are we

[jira] [Created] (FLINK-27557) Create the empty writer for 'ALTER TABLE ... COMPACT'

2022-05-09 Thread Jane Chan (Jira)
Jane Chan created FLINK-27557: - Summary: Create the empty writer for 'ALTER TABLE ... COMPACT' Key: FLINK-27557 URL: https://issues.apache.org/jira/browse/FLINK-27557 Project: Flink Issue Type:

Re: [Discuss] Creating an Apache Flink slack workspace

2022-05-09 Thread Robert Metzger
Sorry for joining this discussion late, and thanks for the summary Xintong! Why are we considering a separate slack instance instead of using the ASF Slack instance? The ASF instance is paid, so all messages are retained forever, and quite a few people are already on that Slack instance. There is

Re: Edit Permissions for Flink Connector Template

2022-05-09 Thread Ber, Jeremy
Attached e-mail thread with Martijn. - Jeremy On 5/6/22, 9:21 PM, "Xintong Song" 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. Hi Jeremy, Could

Re: [DISCUSS ] HybridSouce Table & Sql api timeline

2022-05-09 Thread Martijn Visser
Hi, I would welcome any contribution that gets the HybridSource to become available for Table API and SQL use cases. I don't think there's anyone working on this right now. I do think it requires a FLIP so that we can think how such a connector could be made available. Do you want to work on

Re: [DISCUSS] FLIP-226: Introduce Schema Evolution on Table Store

2022-05-09 Thread Jark Wu
Thanks for proposing this exciting feature, Jingsong! I only have a few questions: 1) Does table-store support evolve schemas multiple times during a checkpoint? For example, cp1 -> write 1M rows (may flush file store) -> evolve schema1 -> write 1M rows (may flush file store again) -> evolve

Re: [DISCUSS] Add scala-2.13 support for implementing completed jdk11 or higher flink version

2022-05-09 Thread Ran Tao
Thanks Martijn, got it. If we upgrade jdk8 to jdk11 and we want to keep the compatibility, now we can't upgrade scala version indeed. I will stay focused for this problem, thanks a lot ~ Martijn Visser 于2022年5月9日周一 19:55写道: > Hi Ran, > > Wouldn't a Scala developer who upgrades from Java 8 to

Re: [DISCUSS] Add scala-2.13 support for implementing completed jdk11 or higher flink version

2022-05-09 Thread Martijn Visser
Hi Ran, Wouldn't a Scala developer who upgrades from Java 8 to Java 11 run into the problem that his savepoint isn't compatible if we would make such a change? Best regards, Martijn On Mon, 9 May 2022 at 13:31, Ran Tao wrote: > Hi, Martijn. Thanks for responding, i have read these issues,

Re: [VOTE] FLIP-225: Implement standalone mode support in the kubernetes operator

2022-05-09 Thread Jassat, Usamah
Hi everyone, The vote finished with 3 binding votes and no disapproving votes. I will continue with the work. Thanks, Usamah On 05/05/2022, 08:43, "Yang Wang" wrote: CAUTION: This email originated from outside of the organization. Do not click links or open attachments unless you can

Re: [DISCUSS] Planning Flink 1.16 with the community

2022-05-09 Thread Johannes Moser
Thanks Chesnay, Godfrey, Xingbo and Martijn for volunteering as release managers. I would also recommend to have an eye on the "Release Notes:" fields in JIRA for related issues early, as this is used to assemble the final release notes. I'd also like to take the chance to ask contributors to

Re: [DISCUSS] Add scala-2.13 support for implementing completed jdk11 or higher flink version

2022-05-09 Thread Ran Tao
Hi, Martijn. Thanks for responding, i have read these issues, and the issues u mentioned are about the compatibility of current released version(which use jdk8), so we are stuck with 2.12.7. But my concern is more about optional jdk11 build, when we use jdk11, we must enable jdk11-target

[RESULT] [VOTE] Apache Flink Table Store 0.1.0, release candidate #2

2022-05-09 Thread Jingsong Li
Hi everyone, I am pleased to announce that we have unanimously approved this release candidate: There are 8 approving votes, 4 of which are binding: - Caizhi Weng (non-binding) - Konstantin Knauf (binding) - Jane Chan (non-binding) - Nicholas Jiang (non-binding) - Zheng Hu (non-binding) - Becket

Re: Re: Re: [DISCUSS] FLIP-229: Introduces Join Hint for Flink SQL Batch Job

2022-05-09 Thread cao zou
Hi Xuyang, thanks for your explanation. For more information about inheritPath and the logic of propagating, you can see `RelOptUtil`. As the doc says, The hint will be propagated from the root node to the children nodes. if we have an AST as follows: LogicalProjectHint1[] > +- LogicalJoin

[DISCUSS] Planning Flink 1.16 with the community

2022-05-09 Thread Martijn Visser
Hi everyone, With Flink 1.15 released last week, we're starting with the next release cycle for what will become Flink 1.16. As previously discussed [1] Chesnay, Godfrey, Xingbo and myself have volunteered as release managers. We're aiming to cut the release branch on the 25th of July. If needed,

Re: [DISCUSS] FLIP-224: Blacklist Mechanism

2022-05-09 Thread Lijie Wang
Hi everyone, Based on the discussion in the mailing list, I updated the FLIP doc, the changes include: 1. Changed the description of the motivation section to more clearly describe the problem this FLIP is trying to solve. 2. Only *Manually* is supported. 3. Adopted some suggestions, such as

Re: [DISCUSS] FLIP-226: Introduce Schema Evolution on Table Store

2022-05-09 Thread Caizhi Weng
Hi all! +1 for this FLIP. By adding schema information into data files we can not only support schema evolution, which is a very useful feature for data storages, but also make it easier for table store to integrate with other systems. For example timestamp type in Hive does not support

[jira] [Created] (FLINK-27556) Performance regression in checkpointSingleInput.UNALIGNED on 29.04.2022

2022-05-09 Thread Roman Khachatryan (Jira)
Roman Khachatryan created FLINK-27556: - Summary: Performance regression in checkpointSingleInput.UNALIGNED on 29.04.2022 Key: FLINK-27556 URL: https://issues.apache.org/jira/browse/FLINK-27556

[jira] [Created] (FLINK-27555) Performance regression in schedulingDownstreamTasks on 02.05.2022

2022-05-09 Thread Roman Khachatryan (Jira)
Roman Khachatryan created FLINK-27555: - Summary: Performance regression in schedulingDownstreamTasks on 02.05.2022 Key: FLINK-27555 URL: https://issues.apache.org/jira/browse/FLINK-27555 Project:

Re: Re: 【Could we support distribute by For FlinkSql】

2022-05-09 Thread Jark Wu
We will start a FLIP discussion in the dev mailing list, so please watch on the ML. I also find that you opened FLINK-27541, we will also update FLINK-27541 once we have an initial FLIP. Best, Jark On Mon, 9 May 2022 at 15:18, lpengdr...@163.com wrote: > Yeah! That's great. Thank you! Where

Re: [DISCUSS] Add scala-2.13 support for implementing completed jdk11 or higher flink version

2022-05-09 Thread Martijn Visser
Hi Ran, Thanks for opening the discussion. There's a Flink ticket already for adding Scala 2.13 support [1] and also for upgrading to the latest Scala 2.12 version [2]. The problem for upgrading to a later version of Scala is outlined in [3]. We couldn't upgrade to later versions of Scala because

[DISCUSS] Add scala-2.13 support for implementing completed jdk11 or higher flink version

2022-05-09 Thread Ran Tao
Hi, guys. I have found out that current flink project build with jdk11 wasn't implemented completed or pure jdk11 version. (higher version has same problem) when enable jdk11, it has some scenarios. 1. Using jdk11 but source 1.8 target 1.8 -> it's ok. 2. Using jdk11 and with jdk11-target so

Re: Re: 【Could we support distribute by For FlinkSql】

2022-05-09 Thread lpengdr...@163.com
Yeah! That's great. Thank you! Where can i get more information about that? lpengdr...@163.com 发件人: Jark Wu 发送时间: 2022-05-09 14:12 收件人: dev 抄送: 贺小令 主题: Re: Re: 【Could we support distribute by For FlinkSql】 I got what you want, maybe something like DISTRIBUTED BY in Hive SQL. The

Re:Re: Re: [DISCUSS] FLIP-229: Introduces Join Hint for Flink SQL Batch Job

2022-05-09 Thread Xuyang
Thanks for your attention, Li. I agree with you and Cao that in theory join hint should support alias just like other popular computing engines and dbs. But the main difficult is that currently, calcite will ignore the alias when converting SqlNode to RelNode, so the information about alias

[jira] [Created] (FLINK-27554) The asf-site does not build on Apple Silicon

2022-05-09 Thread Jiangjie Qin (Jira)
Jiangjie Qin created FLINK-27554: Summary: The asf-site does not build on Apple Silicon Key: FLINK-27554 URL: https://issues.apache.org/jira/browse/FLINK-27554 Project: Flink Issue Type:

Re: [Question][Kafka SQL Connector] When will decoding format emits multiple keys for a Kafka key; why do a cartesian product.

2022-05-09 Thread Yufei Zhang
Hi Jark, Thanks for the explanation, it answered my question well ~ Only one thing, if the keyRow count is N and value rowCount is M (N, M > 1), the cartesian product might not cover all use cases. But I think we don't need to worry about it for now, since this case is rare and we can discuss it

[jira] [Created] (FLINK-27553) Clarify the semantic of RecordWriter interface.

2022-05-09 Thread Zheng Hu (Jira)
Zheng Hu created FLINK-27553: Summary: Clarify the semantic of RecordWriter interface. Key: FLINK-27553 URL: https://issues.apache.org/jira/browse/FLINK-27553 Project: Flink Issue Type: Sub-task

[jira] [Created] (FLINK-27552) Prometheus metrics

2022-05-09 Thread Jira
João Boto created FLINK-27552: - Summary: Prometheus metrics Key: FLINK-27552 URL: https://issues.apache.org/jira/browse/FLINK-27552 Project: Flink Issue Type: Bug Components: Runtime /

Re: [Question][Kafka SQL Connector] When will decoding format emits multiple keys for a Kafka key; why do a cartesian product.

2022-05-09 Thread Jark Wu
Hi Affe, Regarding the implementation, from the interface of `DeserializationSchema#deserialize(byte[], Collector)`, it might emit multiple rows. So this is just a more generic implementation instead of hard-code dropping rows. Even though, currently, there is no built-in key format that will

[jira] [Created] (FLINK-27551) Consider implementing our own status update logic

2022-05-09 Thread Gyula Fora (Jira)
Gyula Fora created FLINK-27551: -- Summary: Consider implementing our own status update logic Key: FLINK-27551 URL: https://issues.apache.org/jira/browse/FLINK-27551 Project: Flink Issue Type:

Re: [DISCUSS] FLIP-91: Support SQL Client Gateway

2022-05-09 Thread Shengkai Fang
Hi JingSong. Thanks for your feedback. > reorganize the FLIP, what Pluggable Endpoint Discovery is, and how users to add new Endpoint, before introducing SQLGatewayService. I update the FLIP: reorganize the order and add more details. > Then I have some doubts about the name

Re:Re: [DISCUSS] FLIP-229: Introduces Join Hint for Flink SQL Batch Job

2022-05-09 Thread Xuyang
Hi, Cao, thanks for your feedback. —— "Firstly, table alias could be supported in the hint?" I think table alias couldn't be supported in the hint just like what you say and only table name and view name are supported in the hint temporarily. Currently, the alias will be ignored when the

Re: Re: 【Could we support distribute by For FlinkSql】

2022-05-09 Thread Jark Wu
I got what you want, maybe something like DISTRIBUTED BY in Hive SQL. The community is planning to support this feature but has not started yet. @Godfrey will drive this work. Best, Jark On Mon, 9 May 2022 at 13:45, lpengdr...@163.com wrote: > Hi > Thanks for your reply. > The way I

[jira] [Created] (FLINK-27550) Remove checking yarn queues before submitting job to Yarn

2022-05-09 Thread Junfan Zhang (Jira)
Junfan Zhang created FLINK-27550: Summary: Remove checking yarn queues before submitting job to Yarn Key: FLINK-27550 URL: https://issues.apache.org/jira/browse/FLINK-27550 Project: Flink