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

2020-03-02 Thread shaoxun
Hi Xintong, it it a huge plan to carry on. And I get a few questions about the details. First, does "specific request" for the slots mean the requesting slot profile contains detailed information about memory and cpu? And how does a job manager determine to ask how much memory? Is it done when

Re: [DISCUSS] FLIP-108: Add GPU support in Flink

2020-03-02 Thread Becket Qin
Thanks for the FLIP Yangze. GPU resource management support is a must-have for machine learning use cases. Actually it is one of the mostly asked question from the users who are interested in using Flink for ML. Some quick comments / questions to the wiki. 1. The WebUI / REST API should probably

[jira] [Created] (FLINK-16398) Rename Protobuf Kafka Ingress / Routable Kafka Ingress type identifier strings

2020-03-02 Thread Tzu-Li (Gordon) Tai (Jira)
Tzu-Li (Gordon) Tai created FLINK-16398: --- Summary: Rename Protobuf Kafka Ingress / Routable Kafka Ingress type identifier strings Key: FLINK-16398 URL: https://issues.apache.org/jira/browse/FLINK-16398

[jira] [Created] (FLINK-16397) [metric] flink metric graphite will format double metrics to "%2.2f"

2020-03-02 Thread BlaBlabla (Jira)
BlaBlabla created FLINK-16397: - Summary: [metric] flink metric graphite will format double metrics to "%2.2f" Key: FLINK-16397 URL: https://issues.apache.org/jira/browse/FLINK-16397 Project: Flink

Re: [VOTE] FLIP-100: Add Attempt Information

2020-03-02 Thread Yadong Xie
> * Why are we duplicating the subtask index in the objects that are stored in the attempts-time-info array? I thought that all objects in the same array share the same subtask index. yes, I agree with you that subtask could be removed in the `attempts-time-info` > * Are we confident that the

[jira] [Created] (FLINK-16396) Support binding Kafka egresses with YAML-ized modules

2020-03-02 Thread Tzu-Li (Gordon) Tai (Jira)
Tzu-Li (Gordon) Tai created FLINK-16396: --- Summary: Support binding Kafka egresses with YAML-ized modules Key: FLINK-16396 URL: https://issues.apache.org/jira/browse/FLINK-16396 Project: Flink

[jira] [Created] (FLINK-16395) Mention OkHttp in NOTICE file of statefun-flink-distribution

2020-03-02 Thread Tzu-Li (Gordon) Tai (Jira)
Tzu-Li (Gordon) Tai created FLINK-16395: --- Summary: Mention OkHttp in NOTICE file of statefun-flink-distribution Key: FLINK-16395 URL: https://issues.apache.org/jira/browse/FLINK-16395 Project:

Re: Flink dev blog

2020-03-02 Thread Xintong Song
Big +1. Thanks for the idea, Arvid. I'd be excited to read such blogs. And we would also be happy to contribute some contents on the newest efforts from our team. Potential topics: - Memory configuration - Active Kubernetes integration - GPU support - Pluggable (dynamic) slot allocation Thank

[jira] [Created] (FLINK-16394) LocalFileSystemTest.testRenameFileWithNoAccess failed when Using root to execute MVN test on Linux system

2020-03-02 Thread Peidian Li (Jira)
Peidian Li created FLINK-16394: -- Summary: LocalFileSystemTest.testRenameFileWithNoAccess failed when Using root to execute MVN test on Linux system Key: FLINK-16394 URL:

Re: [DISCUSS] FLIP-108: Add GPU support in Flink

2020-03-02 Thread Xintong Song
Thanks for drafting the FLIP and kicking off the discussion, Yangze. Big +1 for this feature. Supporting using of GPU in Flink is significant, especially for the ML scenarios. I've reviewed the FLIP wiki doc and it looks good to me. I think it's a very good first step for Flink's GPU supports.

Re: Flink dev blog

2020-03-02 Thread Benchao Li
+1 for this proposal. As a contributor, it would be very helpful to have such blogs for us to understand status and future of Flink. Robert Metzger 于2020年3月3日周二 上午6:00写道: > I would be excited to read such a blog (can I request topics? :) ) > > We could start very low key by using our wiki's

[jira] [Created] (FLINK-16393) Kinesis consumer unnecessarily creates record emitter thread w/o source sync

2020-03-02 Thread Thomas Weise (Jira)
Thomas Weise created FLINK-16393: Summary: Kinesis consumer unnecessarily creates record emitter thread w/o source sync Key: FLINK-16393 URL: https://issues.apache.org/jira/browse/FLINK-16393

[jira] [Created] (FLINK-16392) oneside sorted cache in intervaljoin

2020-03-02 Thread Chen Qin (Jira)
Chen Qin created FLINK-16392: Summary: oneside sorted cache in intervaljoin Key: FLINK-16392 URL: https://issues.apache.org/jira/browse/FLINK-16392 Project: Flink Issue Type: Improvement

Re: Flink dev blog

2020-03-02 Thread Robert Metzger
I would be excited to read such a blog (can I request topics? :) ) We could start very low key by using our wiki's blog feature: https://cwiki.apache.org/confluence/pages/viewrecentblogposts.action?key=FLINK On Mon, Mar 2, 2020 at 8:26 PM Stephan Ewen wrote: > Great idea, but I also second

[jira] [Created] (FLINK-16391) Enhance the RequestReply protocol with Egresses and Delayed Messages

2020-03-02 Thread Igal Shilman (Jira)
Igal Shilman created FLINK-16391: Summary: Enhance the RequestReply protocol with Egresses and Delayed Messages Key: FLINK-16391 URL: https://issues.apache.org/jira/browse/FLINK-16391 Project: Flink

Re: Flink dev blog

2020-03-02 Thread Stephan Ewen
Great idea, but I also second Seth's comment to separate this in a clear way. It's easy to confuse new / potential users. On Mon, Mar 2, 2020 at 8:15 PM Seth Wiesman wrote: > +1 on the idea. > > My only request would be they are clearly marked as being about internals / > for advanced users to

Re: Flink dev blog

2020-03-02 Thread Seth Wiesman
+1 on the idea. My only request would be they are clearly marked as being about internals / for advanced users to not give typical users the wrong impression about how much they need to understand to use Flink. Nico's network stack blog post does this well[1]. Seth [1]

[jira] [Created] (FLINK-16390) Add `view` and `clear` methods to `PersistedTable`

2020-03-02 Thread Seth Wiesman (Jira)
Seth Wiesman created FLINK-16390: Summary: Add `view` and `clear` methods to `PersistedTable` Key: FLINK-16390 URL: https://issues.apache.org/jira/browse/FLINK-16390 Project: Flink Issue

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

2020-03-02 Thread Peter Huang
Hi Kostas, Thanks for updating the wiki. We have aligned with the implementations in the doc. But I feel it is still a little bit confusing of the naming from a user's perspective. It is well known that Flink support per job cluster and session cluster. The concept is in the layer of how a job is

Re: Flink dev blog

2020-03-02 Thread Ufuk Celebi
I'd be happy to read such a blog. Big +1 as a potential reader. ;-) – Ufuk On Mon, Mar 2, 2020 at 11:53 AM Arvid Heise wrote: > Dear devs, > > development speed of Flink has steadily increased. Lots of new concepts are > introduced and technical debt removed. However, it's hard to keep track

[jira] [Created] (FLINK-16389) Bump Kafka 0.10 to 0.10.2.2

2020-03-02 Thread Chesnay Schepler (Jira)
Chesnay Schepler created FLINK-16389: Summary: Bump Kafka 0.10 to 0.10.2.2 Key: FLINK-16389 URL: https://issues.apache.org/jira/browse/FLINK-16389 Project: Flink Issue Type: Sub-task

[jira] [Created] (FLINK-16388) Bump orc-core to 1.4.5

2020-03-02 Thread Chesnay Schepler (Jira)
Chesnay Schepler created FLINK-16388: Summary: Bump orc-core to 1.4.5 Key: FLINK-16388 URL: https://issues.apache.org/jira/browse/FLINK-16388 Project: Flink Issue Type: Sub-task

Re: [VOTE] FLIP-93: JDBC catalog and Postgres catalog

2020-03-02 Thread Leonard Xu
+1 (non-binding). Very useful feature especially for ETL, It will make connecting to existed DB systems easier. Best, Leonard > 在 2020年3月2日,21:58,Jark Wu 写道: > > +1 from my side. > > Best, > Jark > > On Mon, 2 Mar 2020 at 21:40, Kurt Young wrote: > >> +1 >> >> Best, >> Kurt >> >>

Re: Contributor permission application

2020-03-02 Thread Congxian Qiu
Hi Welcome to the Flink community You no longer need contributor permissions. You can simply create a JIRA ticket and ask to be assigned to it in order to start working. Please also take a look at the Flink's contribution guidelines [1] for more information. [1]

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

2020-03-02 Thread Kostas Kloudas
Hi Yang, The difference between per-job and application mode is that, as you described, in the per-job mode the main is executed on the client while in the application mode, the main is executed on the cluster. I do not think we have to offer "application mode" with running the main on the client

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

2020-03-02 Thread Leonard Xu
Hi Dawid, Thanks for driving this FLIP,big +1 for the proposal feature. About the connector.properties part, I suggest avoid using timestamp because timestamp is a keyword in DDL as dataType, user may feel confused, using 'timestamp.filed’ or ’source.timestamp’ will be better? ``` CREATE

[jira] [Created] (FLINK-16387) Use LinkedHashMap for deterministic order in HeapMapState.java

2020-03-02 Thread cpugputpu (Jira)
cpugputpu created FLINK-16387: - Summary: Use LinkedHashMap for deterministic order in HeapMapState.java Key: FLINK-16387 URL: https://issues.apache.org/jira/browse/FLINK-16387 Project: Flink

Re: [VOTE] FLIP-100: Add Attempt Information

2020-03-02 Thread Gary Yao
Hi Yadong, Thanks for driving this FLIP. I have a few questions/remarks: * Why are we duplicating the subtask index in the objects that are stored in the attempts-time-info array? I thought that all objects in the same array share the same subtask index. * Are we confident that the

[jira] [Created] (FLINK-16386) Use LinkedHashMap in SqlFunctionUtils.java

2020-03-02 Thread cpugputpu (Jira)
cpugputpu created FLINK-16386: - Summary: Use LinkedHashMap in SqlFunctionUtils.java Key: FLINK-16386 URL: https://issues.apache.org/jira/browse/FLINK-16386 Project: Flink Issue Type: Bug

[jira] [Created] (FLINK-16385) Use LinkedHashMap in SqlFunctionUtils.java

2020-03-02 Thread cpugputpu (Jira)
cpugputpu created FLINK-16385: - Summary: Use LinkedHashMap in SqlFunctionUtils.java Key: FLINK-16385 URL: https://issues.apache.org/jira/browse/FLINK-16385 Project: Flink Issue Type: Test

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

2020-03-02 Thread Yang Wang
Hi Kostas, Thanks a lot for your conclusion and updating the FLIP-85 WIKI. Currently, i have no more questions about motivation, approach, fault tolerance and the first phase implementation. I think the new title "Flink Application Mode" makes a lot senses to me. Especially for the containerized

[jira] [Created] (FLINK-16384) Support SHOW CREATE TABLE command in SQL Client

2020-03-02 Thread Jark Wu (Jira)
Jark Wu created FLINK-16384: --- Summary: Support SHOW CREATE TABLE command in SQL Client Key: FLINK-16384 URL: https://issues.apache.org/jira/browse/FLINK-16384 Project: Flink Issue Type: New

Re: [Discussion] Job generation / submission hooks & Atlas integration

2020-03-02 Thread Márton Balassi
Hi all, We have added the interface for registering the connectors in custom user user defined functions, like representing enrichment from an HBase table in the middle of a Flink application. We are reaching out to the Atlas community to review the implementation in the near future too, based on

Re: [DISCUSS] FLIP-106: Support Python UDF in SQL Function DDL

2020-03-02 Thread Jark Wu
Thanks for the explanation, Wei! On Mon, 2 Mar 2020 at 20:59, Wei Zhong wrote: > Hi Jark, > > Thanks for your suggestion. > > Actually, the timing of starting a Python process depends on the UDF type, > because the Python process is used to provide the necessary information to > instantiate the

Re: [VOTE] FLIP-93: JDBC catalog and Postgres catalog

2020-03-02 Thread Jark Wu
+1 from my side. Best, Jark On Mon, 2 Mar 2020 at 21:40, Kurt Young wrote: > +1 > > Best, > Kurt > > > On Mon, Mar 2, 2020 at 5:32 PM Jingsong Lee > wrote: > > > +1 from my side. > > > > Best, > > Jingsong Lee > > > > On Mon, Mar 2, 2020 at 11:06 AM Terry Wang wrote: > > > > > +1

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

2020-03-02 Thread Jark Wu
Hi Dawid, > connector properties Could we use "timestamp.field" instead of "timestamp"? This will be more consistent with "key.fields" and it can avoid to confuse users it defines a rowtime attribute (KSQL [1] use "timestamp" property to override ROWTIME information). > SYSTEM_METADATA(...) I

Re: [VOTE] FLIP-93: JDBC catalog and Postgres catalog

2020-03-02 Thread Kurt Young
+1 Best, Kurt On Mon, Mar 2, 2020 at 5:32 PM Jingsong Lee wrote: > +1 from my side. > > Best, > Jingsong Lee > > On Mon, Mar 2, 2020 at 11:06 AM Terry Wang wrote: > > > +1 (non-binding). > > With this feature, we can more easily interact traditional database in > > flink. > > > > Best, > >

Re: [DISCUSS] FLIP-106: Support Python UDF in SQL Function DDL

2020-03-02 Thread Wei Zhong
Hi Jark, Thanks for your suggestion. Actually, the timing of starting a Python process depends on the UDF type, because the Python process is used to provide the necessary information to instantiate the FunctionDefinition object of the Python UDF. For catalog function, the FunctionDefinition

[jira] [Created] (FLINK-16383) KafkaProducerExactlyOnceITCase. testExactlyOnceRegularSink fails with "The producer has already been closed"

2020-03-02 Thread Robert Metzger (Jira)
Robert Metzger created FLINK-16383: -- Summary: KafkaProducerExactlyOnceITCase. testExactlyOnceRegularSink fails with "The producer has already been closed" Key: FLINK-16383 URL:

Contributor permission application

2020-03-02 Thread 1101300123
Hi Guys, I want to contribute to Apache Flink. Would you please give me the permission as a contributor? My JIRA ID is yutaochina. 2019-03-20 博彦科技股份有限公司 --上海泓智信息科技有限公司 地址:上海市长宁区天山路8号402室 邮编:200336 手机:15501079221 邮箱:hdxg1101300...@163.com,yuta...@beyondsoft.com 发件人:"hdxg1101300123"

[jira] [Created] (FLINK-16382) SQL CLI should support command history file

2020-03-02 Thread Gyula Fora (Jira)
Gyula Fora created FLINK-16382: -- Summary: SQL CLI should support command history file Key: FLINK-16382 URL: https://issues.apache.org/jira/browse/FLINK-16382 Project: Flink Issue Type: New

[jira] [Created] (FLINK-16381) Support CREATE FUNCTION statements in SQL CLI

2020-03-02 Thread Gyula Fora (Jira)
Gyula Fora created FLINK-16381: -- Summary: Support CREATE FUNCTION statements in SQL CLI Key: FLINK-16381 URL: https://issues.apache.org/jira/browse/FLINK-16381 Project: Flink Issue Type: New

[jira] [Created] (FLINK-16380) AZP: Python test fails on jdk11 nightly test (misc profile)

2020-03-02 Thread Robert Metzger (Jira)
Robert Metzger created FLINK-16380: -- Summary: AZP: Python test fails on jdk11 nightly test (misc profile) Key: FLINK-16380 URL: https://issues.apache.org/jira/browse/FLINK-16380 Project: Flink

[jira] [Created] (FLINK-16379) Introduce fromValues in TableEnvironment

2020-03-02 Thread Dawid Wysakowicz (Jira)
Dawid Wysakowicz created FLINK-16379: Summary: Introduce fromValues in TableEnvironment Key: FLINK-16379 URL: https://issues.apache.org/jira/browse/FLINK-16379 Project: Flink Issue Type:

[jira] [Created] (FLINK-16378) Kerberized YARN on Docker test fails on jdk11 on AZP

2020-03-02 Thread Robert Metzger (Jira)
Robert Metzger created FLINK-16378: -- Summary: Kerberized YARN on Docker test fails on jdk11 on AZP Key: FLINK-16378 URL: https://issues.apache.org/jira/browse/FLINK-16378 Project: Flink

[jira] [Created] (FLINK-16377) Support inline user defined functions in expression dsl

2020-03-02 Thread Dawid Wysakowicz (Jira)
Dawid Wysakowicz created FLINK-16377: Summary: Support inline user defined functions in expression dsl Key: FLINK-16377 URL: https://issues.apache.org/jira/browse/FLINK-16377 Project: Flink

[jira] [Created] (FLINK-16376) Using consistent method to get Yarn application directory

2020-03-02 Thread Victor Wong (Jira)
Victor Wong created FLINK-16376: --- Summary: Using consistent method to get Yarn application directory Key: FLINK-16376 URL: https://issues.apache.org/jira/browse/FLINK-16376 Project: Flink

Flink dev blog

2020-03-02 Thread Arvid Heise
Dear devs, development speed of Flink has steadily increased. Lots of new concepts are introduced and technical debt removed. However, it's hard to keep track of these things if you are not directly involved. Especially for new contributors, it's often not easy to know what the best practices are

[jira] [Created] (FLINK-16375) Remove references to those methods from documentation

2020-03-02 Thread Dawid Wysakowicz (Jira)
Dawid Wysakowicz created FLINK-16375: Summary: Remove references to those methods from documentation Key: FLINK-16375 URL: https://issues.apache.org/jira/browse/FLINK-16375 Project: Flink

[jira] [Created] (FLINK-16374) StreamingKafkaITCase: IOException: error=13, Permission denied

2020-03-02 Thread Robert Metzger (Jira)
Robert Metzger created FLINK-16374: -- Summary: StreamingKafkaITCase: IOException: error=13, Permission denied Key: FLINK-16374 URL: https://issues.apache.org/jira/browse/FLINK-16374 Project: Flink

[jira] [Created] (FLINK-16373) EmbeddedLeaderService: IllegalStateException: The RPC connection is already closed

2020-03-02 Thread Robert Metzger (Jira)
Robert Metzger created FLINK-16373: -- Summary: EmbeddedLeaderService: IllegalStateException: The RPC connection is already closed Key: FLINK-16373 URL: https://issues.apache.org/jira/browse/FLINK-16373

Re: Regarding access to Jira

2020-03-02 Thread JingsongLee
Hi, Only Flink committers can assign [1] I just took a look and assigned to you. [1] https://flink.apache.org/contributing/contribute-code.html#create-jira-ticket-and-reach-consensus Best, Jingsong Lee -- From:Sivaprasanna Send

Regarding access to Jira

2020-03-02 Thread Sivaprasanna
Hi, Can you please give me access to Flink's Jira board. I created a ticket FLINK-16371 and want to assign it to myself. Thanks, Sivaprasanna

[jira] [Created] (FLINK-16372) Generate unique operator name

2020-03-02 Thread Zou (Jira)
Zou created FLINK-16372: --- Summary: Generate unique operator name Key: FLINK-16372 URL: https://issues.apache.org/jira/browse/FLINK-16372 Project: Flink Issue Type: Improvement Reporter:

[jira] [Created] (FLINK-16371) HadoopCompressionBulkWriter fails with 'java.io.NotSerializableException'

2020-03-02 Thread Sivaprasanna Sethuraman (Jira)
Sivaprasanna Sethuraman created FLINK-16371: --- Summary: HadoopCompressionBulkWriter fails with 'java.io.NotSerializableException' Key: FLINK-16371 URL: https://issues.apache.org/jira/browse/FLINK-16371

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

2020-03-02 Thread Dawid Wysakowicz
Hi Jark, Ad. 2 I added a section to discuss relation to FLIP-63 Ad. 3 Yes, I also tried to somewhat keep hierarchy of properties. Therefore you have the key.format.type. I also considered exactly what you are suggesting (prefixing with connector or kafka). I should've put that into an

[jira] [Created] (FLINK-16370) YARNHighAvailabilityITCase fails on JDK11 nightly test: NoSuchMethodError: org.apache.flink.shaded.zookeeper3.org.apache.zookeeper.server.quorum.flexible.QuorumMaj

2020-03-02 Thread Robert Metzger (Jira)
Robert Metzger created FLINK-16370: -- Summary: YARNHighAvailabilityITCase fails on JDK11 nightly test: NoSuchMethodError: org.apache.flink.shaded.zookeeper3.org.apache.zookeeper.server.quorum.flexible.QuorumMaj Key: FLINK-16370

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

2020-03-02 Thread Kostas Kloudas
Hi all, I update https://cwiki.apache.org/confluence/display/FLINK/FLIP-85+Flink+Application+Mode based on the discussion we had here: https://docs.google.com/document/d/1ji72s3FD9DYUyGuKnJoO4ApzV-nSsZa0-bceGXW7Ocw/edit# Please let me know what you think and please keep the discussion in the

Re: [VOTE] FLIP-93: JDBC catalog and Postgres catalog

2020-03-02 Thread Jingsong Lee
+1 from my side. Best, Jingsong Lee On Mon, Mar 2, 2020 at 11:06 AM Terry Wang wrote: > +1 (non-binding). > With this feature, we can more easily interact traditional database in > flink. > > Best, > Terry Wang > > > > > 2020年3月1日 18:33,zoudan 写道: > > > > +1 (non-binding) > > > > Best, > >

Re: 开发相关问题咨询Development related problems consultation

2020-03-02 Thread JingsongLee
Hi, welcome, For user side, u...@flink.apache.org is for English. user...@flink.apache.org is for Chinese. dev@flink.apache.org is for development related discussions, so please not send to it. Best, Jingsong Lee -- From:王博迪

[jira] [Created] (FLINK-16369) Allow the StreamingFileSink to restore from a previous (old) savepoint.

2020-03-02 Thread Kostas Kloudas (Jira)
Kostas Kloudas created FLINK-16369: -- Summary: Allow the StreamingFileSink to restore from a previous (old) savepoint. Key: FLINK-16369 URL: https://issues.apache.org/jira/browse/FLINK-16369 Project:

[jira] [Created] (FLINK-16367) Introduce createDmlBatch method in TableEnvironment

2020-03-02 Thread godfrey he (Jira)
godfrey he created FLINK-16367: -- Summary: Introduce createDmlBatch method in TableEnvironment Key: FLINK-16367 URL: https://issues.apache.org/jira/browse/FLINK-16367 Project: Flink Issue Type:

[jira] [Created] (FLINK-16366) Introduce executeStatement method in TableEnvironment

2020-03-02 Thread godfrey he (Jira)
godfrey he created FLINK-16366: -- Summary: Introduce executeStatement method in TableEnvironment Key: FLINK-16366 URL: https://issues.apache.org/jira/browse/FLINK-16366 Project: Flink Issue