Re: [DISCUSS] FLIP-27: Refactor Source Interface

2020-01-15 Thread Becket Qin
Hi Steven, Unfortunately we were behind schedule and did not get this into 1.10... So it will be in 1.11 instead. Thanks, Jiangjie (Becket) Qin On Thu, Jan 16, 2020 at 10:39 AM Steven Wu wrote: > Becket, is FLIP-27 still on track to be released in 1.10? > > On Tue, Jan 7, 2020 at 7:04 PM Beck

[jira] [Created] (FLINK-15611) KafkaITCase.testOneToOneSources fails on Travis

2020-01-15 Thread Yangze Guo (Jira)
Yangze Guo created FLINK-15611: -- Summary: KafkaITCase.testOneToOneSources fails on Travis Key: FLINK-15611 URL: https://issues.apache.org/jira/browse/FLINK-15611 Project: Flink Issue Type: Bug

[jira] [Created] (FLINK-15610) How to achieve the udf that the number of return column is uncertain

2020-01-15 Thread hehuiyuan (Jira)
hehuiyuan created FLINK-15610: - Summary: How to achieve the udf that the number of return column is uncertain Key: FLINK-15610 URL: https://issues.apache.org/jira/browse/FLINK-15610 Project: Flink

Re: Flink Sql Join, how to clear the sql join state?

2020-01-15 Thread Benchao Li
Hi LakeShen, Maybe "Idle State Retention Time"[1] may help in your case. [1] https://ci.apache.org/projects/flink/flink-docs-master/dev/table/streaming/query_configuration.html#idle-state-retention-time LakeShen 于2020年1月16日周四 上午10:15写道: > Hi community,now I am use flink sql inner join in my co

[jira] [Created] (FLINK-15609) Add blink built-in functions from FlinkSqlOperatorTable to BuiltInFunctionDefinitions

2020-01-15 Thread Jingsong Lee (Jira)
Jingsong Lee created FLINK-15609: Summary: Add blink built-in functions from FlinkSqlOperatorTable to BuiltInFunctionDefinitions Key: FLINK-15609 URL: https://issues.apache.org/jira/browse/FLINK-15609

[jira] [Created] (FLINK-15608) State Processor api Write new state will get NullPointerException

2020-01-15 Thread HunterHunter (Jira)
HunterHunter created FLINK-15608: Summary: State Processor api Write new state will get NullPointerException Key: FLINK-15608 URL: https://issues.apache.org/jira/browse/FLINK-15608 Project: Flink

Re: [DISCUSS] Make AppendingState#add refuse to add null element

2020-01-15 Thread Yun Tang
+1 for unifying the behavior of AppendingState#add . However, I have concern for the usage of window reducing function [1], I'm not sure whether user would rely on processing StreamRecord(null) to clear state. As you can see, user could not see the reducing window state directly, and the only w

Re: Please give me the permission as a contributor

2020-01-15 Thread Congxian Qiu
Welcome to the Flink community! You no longer need contributor permissions to open JIRA tickets. You can simply open a JIRA ticket and ask a committer to assign you to it and start working on it. Please check the Flink's contribution guidelines [1] for more details. [1] https://flink.apache.org/co

Re: [DISCUSS] FLIP-85: Delayed Job Graph Generation

2020-01-15 Thread Yang Wang
Hi all, Thanks a lot for the feedback from @Kostas Kloudas. Your all concerns are on point. The FLIP-85 is mainly focused on supporting cluster mode for per-job. Since it is more urgent and have much more use cases both in Yarn and Kubernetes deployment. For session cluster, we could have more dis

Re: [DISCUSS] FLIP-27: Refactor Source Interface

2020-01-15 Thread Steven Wu
Becket, is FLIP-27 still on track to be released in 1.10? On Tue, Jan 7, 2020 at 7:04 PM Becket Qin wrote: > Hi folks, > > Happy new year! > > Stephan and I chatted offline yesterday. After reading the email thread > again, I found that I have misunderstood Dawid's original proposal > regarding

Flink Sql Join, how to clear the sql join state?

2020-01-15 Thread LakeShen
Hi community,now I am use flink sql inner join in my code,I saw the flink document, the flink sql inner join will keep both sides of the join input in Flink’s state forever. As result , the hdfs files size are so big , is there any way to clear the sql join state? Thanks to your reply.

Re: [DISCUSS] Improve TableFactory

2020-01-15 Thread Bowen Li
Hi Jingsong, The 1st and 2nd pain points you described are very valid, as I'm more familiar with them. I agree these are shortcomings of the current Flink SQL design. A couple comments on your 1st proposal: 1. is it better to have explicit APIs like "createBatchTableSource(...)" and "createStrea

[jira] [Created] (FLINK-15607) throw exception when users trying to use Hive aggregate functions in streaming mode

2020-01-15 Thread Bowen Li (Jira)
Bowen Li created FLINK-15607: Summary: throw exception when users trying to use Hive aggregate functions in streaming mode Key: FLINK-15607 URL: https://issues.apache.org/jira/browse/FLINK-15607 Project:

Re: [DISCUSS] FLIP-85: Delayed Job Graph Generation

2020-01-15 Thread Peter Huang
Hi Kostas, Thanks for this feedback. I can't agree more about the opinion. The cluster mode should be added first in per job cluster. 1) For job cluster implementation 1. Job graph recovery from configuration or store as static job graph as session cluster. I think the static one will be better f

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

2020-01-15 Thread Till Rohrmann
I'd be fine with these changes. Thanks for the summary Xintong. Cheers, Till On Wed, Jan 15, 2020 at 11:09 AM Xintong Song wrote: > Thank you all for the well discussion. > > If there's no further concerns or objections, I would like to conclude this > thread into the following action items. >

[jira] [Created] (FLINK-15606) Deprecate enable default background cleanup of state with TTL

2020-01-15 Thread Andrey Zagrebin (Jira)
Andrey Zagrebin created FLINK-15606: --- Summary: Deprecate enable default background cleanup of state with TTL Key: FLINK-15606 URL: https://issues.apache.org/jira/browse/FLINK-15606 Project: Flink

Re: [DISCUSS] FLIP-85: Delayed Job Graph Generation

2020-01-15 Thread Kostas Kloudas
Hi all, I am writing here as the discussion on the Google Doc seems to be a bit difficult to follow. I think that in order to be able to make progress, it would be helpful to focus on per-job mode for now. The reason is that: 1) making the (unique) JobSubmitHandler responsible for creating the j

[jira] [Created] (FLINK-15605) Remove deprecated in 1.9 StateTtlConfig.TimeCharacteristic

2020-01-15 Thread Andrey Zagrebin (Jira)
Andrey Zagrebin created FLINK-15605: --- Summary: Remove deprecated in 1.9 StateTtlConfig.TimeCharacteristic Key: FLINK-15605 URL: https://issues.apache.org/jira/browse/FLINK-15605 Project: Flink

[jira] [Created] (FLINK-15604) Error for 504 returns

2020-01-15 Thread LCID Fire (Jira)
LCID Fire created FLINK-15604: - Summary: Error for 504 returns Key: FLINK-15604 URL: https://issues.apache.org/jira/browse/FLINK-15604 Project: Flink Issue Type: Bug Reporter: LCID Fi

Let Flink SQL PlannerExpressionParserImpl#FieldRefrence use Unicode as its default charset

2020-01-15 Thread ??????
Hi all,  the related issue:https://issues.apache.org/jira/browse/FLINK-15573   As the title tells, what I do want to do is let the `FieldRefrence` use Unicode as its default charset (or maybe as an optional  charset which can be configured). According to the  `PlannerExpressionParserImpl`, cur

Let Flink SQL PlannerExpressionParserImpl#FieldRefrence use Unicode as its default charset

2020-01-15 Thread ??????
Hi all,  the related issue:https://issues.apache.org/jira/browse/FLINK-15573   As the title tells, what I do want to do is let the `FieldRefrence` use Unicode as its default charset (or maybe as an optional  charset which can be configured). According to the  `PlannerExpressionParserImpl`, cur

[jira] [Created] (FLINK-15603) Show "barrier lag" in checkpoint statistics

2020-01-15 Thread Stephan Ewen (Jira)
Stephan Ewen created FLINK-15603: Summary: Show "barrier lag" in checkpoint statistics Key: FLINK-15603 URL: https://issues.apache.org/jira/browse/FLINK-15603 Project: Flink Issue Type: Impro

[jira] [Created] (FLINK-15602) Blink planner does not respect the precision when casting timestamp to varchar

2020-01-15 Thread Dawid Wysakowicz (Jira)
Dawid Wysakowicz created FLINK-15602: Summary: Blink planner does not respect the precision when casting timestamp to varchar Key: FLINK-15602 URL: https://issues.apache.org/jira/browse/FLINK-15602

[jira] [Created] (FLINK-15601) Remove useless constant field NUM_STOP_CALL_TRIES in Execution

2020-01-15 Thread vinoyang (Jira)
vinoyang created FLINK-15601: Summary: Remove useless constant field NUM_STOP_CALL_TRIES in Execution Key: FLINK-15601 URL: https://issues.apache.org/jira/browse/FLINK-15601 Project: Flink Issue

[jira] [Created] (FLINK-15600) Further relax the UDF constraints for Java classes

2020-01-15 Thread Timo Walther (Jira)
Timo Walther created FLINK-15600: Summary: Further relax the UDF constraints for Java classes Key: FLINK-15600 URL: https://issues.apache.org/jira/browse/FLINK-15600 Project: Flink Issue Type

Re: [DISCUSS] decentralized scheduling strategy is needed

2020-01-15 Thread HuWeihua
Hi, Andrey Thanks for your response. I have checked this Jira ticket and I think it can work in standalone mode which TaskManager has been started before scheduling tasks. But we are currently running flink on yarn in per-job cluster mode. I noticed that this issue has already been raised. I wi

[jira] [Created] (FLINK-15599) SQL client requires both legacy and blink planner to be on the classpath

2020-01-15 Thread Dawid Wysakowicz (Jira)
Dawid Wysakowicz created FLINK-15599: Summary: SQL client requires both legacy and blink planner to be on the classpath Key: FLINK-15599 URL: https://issues.apache.org/jira/browse/FLINK-15599 Proj

[jira] [Created] (FLINK-15598) Memory accuracy loss in YarnClusterDescriptor may lead to deployment failure.

2020-01-15 Thread Xintong Song (Jira)
Xintong Song created FLINK-15598: Summary: Memory accuracy loss in YarnClusterDescriptor may lead to deployment failure. Key: FLINK-15598 URL: https://issues.apache.org/jira/browse/FLINK-15598 Project

Re: Please give me the permission as a contributor

2020-01-15 Thread Zhu Zhu
Welcome to the Flink community! You no longer need contributor permissions to open JIRA tickets. You can simply open a JIRA ticket and ask a committer to assign you to it and start working on it. Please check the Flink's contribution guidelines [1] for more details. [1] https://flink.apache.org/co

Re: [DISCUSS] decentralized scheduling strategy is needed

2020-01-15 Thread Chesnay Schepler
This is a known issue that's will be fixed in 1.9.2/1.10.0; see https://issues.apache.org/jira/browse/FLINK-12122 . On 15/01/2020 10:07, HuWeihua wrote: Hi, All We encountered some problems during the upgrade from Flink 1.5 to Flink 1.9. Flink's scheduling strategy has changed. Flink 1.9 prefe

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

2020-01-15 Thread Xintong Song
Thank you all for the well discussion. If there's no further concerns or objections, I would like to conclude this thread into the following action items. - Change default value of "taskmanager.memory.jvm-overhead.min" to 192MB. - Change default value of "taskmanager.memory.jvm-metaspace.si

Re: [DISCUSS] decentralized scheduling strategy is needed

2020-01-15 Thread Andrey Zagrebin
HI HuWeihua, I think your issue should resolve with 1.9.2 and 1.10 (not released but in progress). You can check the related Jira ticket [1]. Best, Andrey [1] https://jira.apache.org/jira/browse/FLINK-12122 On Wed, Jan 15, 2020 at 10:08 AM HuWeihua wrote: > Hi, All > We encountered some probl

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

2020-01-15 Thread Xintong Song
There's more idea from offline discussion with Andrey. If we decide to make metaspace 96MB, we can also make process.size 1568MB (1.5G + 32MB). According to the spreadsheet , 1.5GB process size and 64MB

[jira] [Created] (FLINK-15597) Relax sanity check of JVM memory overhead to be within its min/max

2020-01-15 Thread Andrey Zagrebin (Jira)
Andrey Zagrebin created FLINK-15597: --- Summary: Relax sanity check of JVM memory overhead to be within its min/max Key: FLINK-15597 URL: https://issues.apache.org/jira/browse/FLINK-15597 Project: Fli

Please give me the permission as a contributor

2020-01-15 Thread m...@lishiyu.cn
Hi Guys, I want to contribute to Apache Flink. Would you please give me the permission as a contributor? My JIRA ID is lishiyu. m...@lishiyu.cn