[jira] [Created] (FLINK-14114) Shift down ClusterClient#timeout to RestClusterClient

2019-09-17 Thread TisonKun (Jira)
TisonKun created FLINK-14114: Summary: Shift down ClusterClient#timeout to RestClusterClient Key: FLINK-14114 URL: https://issues.apache.org/jira/browse/FLINK-14114 Project: Flink Issue Type: Sub

Re: [PROPOSAL] Merge NewClusterClient into ClusterClient

2019-09-17 Thread Zili Chen
Hi Jeff, Thanks for your reply. The ongoing client API enhancement thread[1] is mainly aimed at dealing with issues of our client API, as you mentioned, current client API is no so clean. Because client API naturally becomes public & user-facing inteface it is expected that we start a series of

Re: [PROPOSAL] Merge NewClusterClient into ClusterClient

2019-09-17 Thread Jeff Zhang
Thanks for raising this discussion. Overall +1 to merge NewClusterClient into ClusterClient. 1. I think it is OK to break the backward compatibility. This current client api is no so clean which already cause issue for downstream project and flink itself. In flink scala shell, I notice this kind o

[VOTE] FLIP-63: Rework table partition support

2019-09-17 Thread JingsongLee
Hi Flink devs, I would like to start the voting for FLIP-63 Rework table partition support. FLIP wiki: https://cwiki.apache.org/confluence/display/FLINK/FLIP-63%3A+Rework+table+partition+support Discussion thread: http://apache-flink-mailing-list-archive.1008284.n3.nabble.com/DISCUSS-FLIP-6

Re: [DISCUSS] FLIP-56: Dynamic Slot Allocation

2019-09-17 Thread Xintong Song
@tao I think we cannot limit the cpu usage of a slot, nor isolate the usages between slots. We do have cpu limits for the task executor in some scenarios, such as on yarn with strict cgroup mode. The purpose of bookkeep and dynamic allocation of cpu cores is to prevent scheduling tasks with too m

[PROPOSAL] Merge NewClusterClient into ClusterClient

2019-09-17 Thread Zili Chen
Hi devs, FLINK-14096[1] was created yesterday. It is aimed at merge the bridge class NewClusterClient into ClusterClient because with the effort under FLINK-10392 this bridge class is no longer necessary. Technically in current codebase all implementation of interface NewClusterClient is subclass

Re: [DISCUSS] FLIP-63: Rework table partition support

2019-09-17 Thread JingsongLee
Thanks for your reply and google doc comments. It has been discussed for two weeks now. I will start a vote thread. Best, Jingsong Lee -- From:Kurt Young Send Time:2019年9月16日(星期一) 15:55 To:dev Cc:JingsongLee Subject:Re: [DISCUSS

[jira] [Created] (FLINK-14113) Remove class JobWithJars

2019-09-17 Thread TisonKun (Jira)
TisonKun created FLINK-14113: Summary: Remove class JobWithJars Key: FLINK-14113 URL: https://issues.apache.org/jira/browse/FLINK-14113 Project: Flink Issue Type: Sub-task Components: C

Re: [DISCUSS] FLIP-57 - Rework FunctionCatalog

2019-09-17 Thread Timo Walther
Hi everyone, @Xuefu: I would like to avoid adding too many things incrementally. Users should be able to override all catalog objects consistently according to FLIP-64 (Support for Temporary Objects in Table module). If functions are treated completely different, we need more code and special

[jira] [Created] (FLINK-14112) Removing zookeeper state should cause the task manager and job managers to restart

2019-09-17 Thread Aaron Levin (Jira)
Aaron Levin created FLINK-14112: --- Summary: Removing zookeeper state should cause the task manager and job managers to restart Key: FLINK-14112 URL: https://issues.apache.org/jira/browse/FLINK-14112 Proj

[jira] [Created] (FLINK-14111) Flink should be robust to a non-leader Zookeeper host going down

2019-09-17 Thread Aaron Levin (Jira)
Aaron Levin created FLINK-14111: --- Summary: Flink should be robust to a non-leader Zookeeper host going down Key: FLINK-14111 URL: https://issues.apache.org/jira/browse/FLINK-14111 Project: Flink

[jira] [Created] (FLINK-14110) Deleting state.backend.rocksdb.localdir causes silent failure

2019-09-17 Thread Aaron Levin (Jira)
Aaron Levin created FLINK-14110: --- Summary: Deleting state.backend.rocksdb.localdir causes silent failure Key: FLINK-14110 URL: https://issues.apache.org/jira/browse/FLINK-14110 Project: Flink

[jira] [Created] (FLINK-14109) Improve javadocs and tests for high-availability backend

2019-09-17 Thread Brendan MacDonell (Jira)
Brendan MacDonell created FLINK-14109: - Summary: Improve javadocs and tests for high-availability backend Key: FLINK-14109 URL: https://issues.apache.org/jira/browse/FLINK-14109 Project: Flink

[DISCUSS] FLIP-68: Extend Core Table System with Modular Plugins

2019-09-17 Thread Bowen Li
Hi devs, We'd like to kick off a conversation on "FLIP-68: Extend Core Table System with Modular Plugins" [1]. The modular approach was raised in discussion of how to support Hive built-in functions in FLIP-57 [2]. As we discussed and looked deeper, we think it’s a good opportunity to broaden th

[jira] [Created] (FLINK-14108) Support for Confluent Kafka schema registry for Avro serialisation

2019-09-17 Thread Lasse Nedergaard (Jira)
Lasse Nedergaard created FLINK-14108: - Summary: Support for Confluent Kafka schema registry for Avro serialisation Key: FLINK-14108 URL: https://issues.apache.org/jira/browse/FLINK-14108 Project

[jira] [Created] (FLINK-14107) Kinesis consumer record emitter may deadlock under event time alignment

2019-09-17 Thread Thomas Weise (Jira)
Thomas Weise created FLINK-14107: Summary: Kinesis consumer record emitter may deadlock under event time alignment Key: FLINK-14107 URL: https://issues.apache.org/jira/browse/FLINK-14107 Project: Flin

Re: [DISCUSS] FLIP-56: Dynamic Slot Allocation

2019-09-17 Thread tao xiao
Sorry if I ask a question that has been addressed before. please point me to the reference. How do we limit the cpu usage to a slot? Does the thread that executes the slot get paused when it uses CPU cycles more than it requests? On Tue, Sep 17, 2019 at 10:23 PM Xintong Song wrote: > Thanks fo

[VOTE] FLIP-56: Dynamic Slot Allocation

2019-09-17 Thread Xintong Song
Hi all, I would like to start the vote for FLIP-56 [1], on which a consensus is reached in this discussion thread [2]. The vote will be open for at least 72 hours. I'll try to close it after Sep. 20 15:00 UTC, unless there is an objection or not enough votes. Thank you~ Xintong Song [1] https

Re: [DISCUSS] FLIP-56: Dynamic Slot Allocation

2019-09-17 Thread Xintong Song
Thanks for the feedback, Andrey. I'll start the vote. Thank you~ Xintong Song On Tue, Sep 17, 2019 at 10:09 PM Andrey Zagrebin wrote: > Thanks for the update @Xintong. > I would be ok with starting the vote. > > Best, > Andrey > > On Tue, Sep 17, 2019 at 6:12 AM Xintong Song > wrote: > > >

[jira] [Created] (FLINK-14106) Make SlotManager pluggable

2019-09-17 Thread Xintong Song (Jira)
Xintong Song created FLINK-14106: Summary: Make SlotManager pluggable Key: FLINK-14106 URL: https://issues.apache.org/jira/browse/FLINK-14106 Project: Flink Issue Type: Improvement

Re: [DISCUSS] FLIP-56: Dynamic Slot Allocation

2019-09-17 Thread Andrey Zagrebin
Thanks for the update @Xintong. I would be ok with starting the vote. Best, Andrey On Tue, Sep 17, 2019 at 6:12 AM Xintong Song wrote: > The implementation plan [1] is updated, with the following changes: > >- Add default slot resource profile to >ResourceManagerGateway#registerTaskExec

Re: [DISCUSS] FLIP-70 - Support Computed Column for Flink SQL

2019-09-17 Thread Qi Luo
Fantastic! We're also very interested in this feature. +Boxiu On Tue, Sep 17, 2019 at 11:31 AM Danny Chan wrote: > In umbrella task FLINK-10232 we have introduced CREATE TABLE grammar in > our new module flink-sql-parser. And we proposed to use computed column to > describe the time attribute o

[jira] [Created] (FLINK-14105) Make flink-runtime scala-free

2019-09-17 Thread TisonKun (Jira)
TisonKun created FLINK-14105: Summary: Make flink-runtime scala-free Key: FLINK-14105 URL: https://issues.apache.org/jira/browse/FLINK-14105 Project: Flink Issue Type: Task Components:

[jira] [Created] (FLINK-14104) Bump Jackson to 2.9.9.3

2019-09-17 Thread Nico Kruber (Jira)
Nico Kruber created FLINK-14104: --- Summary: Bump Jackson to 2.9.9.3 Key: FLINK-14104 URL: https://issues.apache.org/jira/browse/FLINK-14104 Project: Flink Issue Type: Bug Components: B

[jira] [Created] (FLINK-14103) StreamTask refactoring: refine and improve exceptions passed to failExternally call

2019-09-17 Thread Alex (Jira)
Alex created FLINK-14103: Summary: StreamTask refactoring: refine and improve exceptions passed to failExternally call Key: FLINK-14103 URL: https://issues.apache.org/jira/browse/FLINK-14103 Project: Flink

[jira] [Created] (FLINK-14102) Introduce DB2Dialect

2019-09-17 Thread Canbin Zheng (Jira)
Canbin Zheng created FLINK-14102: Summary: Introduce DB2Dialect Key: FLINK-14102 URL: https://issues.apache.org/jira/browse/FLINK-14102 Project: Flink Issue Type: Sub-task Reporte

[jira] [Created] (FLINK-14101) Introduce SqlServerDialect

2019-09-17 Thread Canbin Zheng (Jira)
Canbin Zheng created FLINK-14101: Summary: Introduce SqlServerDialect Key: FLINK-14101 URL: https://issues.apache.org/jira/browse/FLINK-14101 Project: Flink Issue Type: Sub-task R

[jira] [Created] (FLINK-14099) SQL supports timestamp in Long

2019-09-17 Thread Zijie Lu (Jira)
Zijie Lu created FLINK-14099: Summary: SQL supports timestamp in Long Key: FLINK-14099 URL: https://issues.apache.org/jira/browse/FLINK-14099 Project: Flink Issue Type: New Feature Re

[jira] [Created] (FLINK-14100) Introduce OracleDialect

2019-09-17 Thread Canbin Zheng (Jira)
Canbin Zheng created FLINK-14100: Summary: Introduce OracleDialect Key: FLINK-14100 URL: https://issues.apache.org/jira/browse/FLINK-14100 Project: Flink Issue Type: Sub-task Repo

[jira] [Created] (FLINK-14098) Support multiple sql statements splitting by semicolon for TableEnvironment

2019-09-17 Thread Canbin Zheng (Jira)
Canbin Zheng created FLINK-14098: Summary: Support multiple sql statements splitting by semicolon for TableEnvironment Key: FLINK-14098 URL: https://issues.apache.org/jira/browse/FLINK-14098 Project:

[jira] [Created] (FLINK-14097) Optimize AsyncWaitOperator to process more than one element per mailbox letter

2019-09-17 Thread Alex (Jira)
Alex created FLINK-14097: Summary: Optimize AsyncWaitOperator to process more than one element per mailbox letter Key: FLINK-14097 URL: https://issues.apache.org/jira/browse/FLINK-14097 Project: Flink

[jira] [Created] (FLINK-14096) Merge NewClusterClient into ClusterClient

2019-09-17 Thread TisonKun (Jira)
TisonKun created FLINK-14096: Summary: Merge NewClusterClient into ClusterClient Key: FLINK-14096 URL: https://issues.apache.org/jira/browse/FLINK-14096 Project: Flink Issue Type: Sub-task

[jira] [Created] (FLINK-14095) Add Pravega in Flink ecosystem page

2019-09-17 Thread Yumin Zhou (Jira)
Yumin Zhou created FLINK-14095: -- Summary: Add Pravega in Flink ecosystem page Key: FLINK-14095 URL: https://issues.apache.org/jira/browse/FLINK-14095 Project: Flink Issue Type: Improvement

Re: [DISCUSS] FLIP-67: Global partitions lifecycle

2019-09-17 Thread zhijiang
Thanks Chesnay for this FLIP and sorry for touching it a bit delay on my side. I also have some similar concerns which Till already proposed before. 1. The consistent terminology in different components. On JM side, PartitionTracker#getPersistedBlockingPartitions is defined for getting global

Re: Is Idle state retention time in SQL client possible?

2019-09-17 Thread Fabian Hueske
I'm not sure what you are asking for. "execution: min/max-idle-state-retention" are the entries in the configuration file for the SQL CLI client to configure "idle state retention time" as described in the link that you posted. Btw. user question should be directed to the u...@flink.apache.org mai

Re: Is Idle state retention time in SQL client possible?

2019-09-17 Thread srikanth flink
Hey, I came across the configuration under 'exection' in .yaml file. Any other info is appreciated. Thanks Srikanth On Tue, Sep 17, 2019 at 2:31 PM srikanth flink wrote: > Hi there, > > I've come across this link >

[jira] [Created] (FLINK-14094) Fix OperatorIOMetricGroup repeat register problem

2019-09-17 Thread xymaqingxiang (Jira)
xymaqingxiang created FLINK-14094: - Summary: Fix OperatorIOMetricGroup repeat register problem Key: FLINK-14094 URL: https://issues.apache.org/jira/browse/FLINK-14094 Project: Flink Issue Typ

Re: Is Idle state retention time in SQL client possible?

2019-09-17 Thread Fabian Hueske
Hi, This can be set via the environment file. Please have a look at the documentation [1] (see "execution: min-idle-state-retention: " and "execution: max-idle-retention: " keys). Fabian [1] https://ci.apache.org/projects/flink/flink-docs-release-1.9/dev/table/sqlClient.html#environment-files A

Is Idle state retention time in SQL client possible?

2019-09-17 Thread srikanth flink
Hi there, I've come across this link for Idle state retention time. Would like to know if I could configure this for SQL client and how? Thanks Srikanth

[jira] [Created] (FLINK-14093) Java8 lambdas and exceptions lead to compile error

2019-09-17 Thread zzsmdfj (Jira)
zzsmdfj created FLINK-14093: --- Summary: Java8 lambdas and exceptions lead to compile error Key: FLINK-14093 URL: https://issues.apache.org/jira/browse/FLINK-14093 Project: Flink Issue Type: Bug