Re: [Discuss] FLIP-43: Savepoint Connector

2019-05-31 Thread Konstantin Knauf
Hi Seth, big +1, happy to see this moving forward :) I have seen plenty of users, who refrained using managed state for some of their data/use cases due to the lack of something like this. I am not sure about the name "Savepoint Connector", but for a different reason. While it is technically a "co

[jira] [Created] (FLINK-12689) Building flink-dist fails because flink-azure-fs-hadoop jar cannot be added to /opt

2019-05-31 Thread Gary Yao (JIRA)
Gary Yao created FLINK-12689: Summary: Building flink-dist fails because flink-azure-fs-hadoop jar cannot be added to /opt Key: FLINK-12689 URL: https://issues.apache.org/jira/browse/FLINK-12689 Project:

Re: [DISCUSS] FLIP-33: Standardize connector metrics

2019-05-31 Thread Piotr Nowojski
Hey Becket, Re 1a) and 1b) +1 from my side. I’ve discussed this issue: >>> >>> 2. It would be nice to have metrics, that allow us to check the cause of >>> back pressure: >>> a) for sources, length of input queue (in bytes? Or boolean >>> hasSomethingl/isEmpty) >>> b) for sinks, length of outpu

Re: [Discuss] FLIP-43: Savepoint Connector

2019-05-31 Thread Piotr Nowojski
I was long awaiting this feature! I can not help much with reviewing, but big +1 from my side :) One thing that would be great for analyzing the state and possible smaller modifications, would be to hook this in with Flink SQL :) Especially if it could be done in a way that would work out of th

Re: [Discuss] FLIP-43: Savepoint Connector

2019-05-31 Thread Tzu-Li (Gordon) Tai
@Piotr Yes, we're aiming this for the 1.9 release. This was also mentioned in the recent 1.9 feature discussion thread [1]. [1] http://apache-flink-mailing-list-archive.1008284.n3.nabble.com/DISCUSS-Features-for-Apache-Flink-1-9-0-td28701.html On Fri, May 31, 2019 at 4:34 PM Piotr Nowojski wrote

Re: [Discuss] FLIP-43: Savepoint Connector

2019-05-31 Thread Jan Lukavský
Hi, this is awesome, and really useful feature. If I might ask for one thing to consider - would it be possible to make the Savepoint manipulation API (at least writing the Savepoint) less dependent on other parts of Flink internals (e.g. |KeyedStateBootstrapFunction|) and provide something m

[jira] [Created] (FLINK-12690) Introduce Planner interface

2019-05-31 Thread Dawid Wysakowicz (JIRA)
Dawid Wysakowicz created FLINK-12690: Summary: Introduce Planner interface Key: FLINK-12690 URL: https://issues.apache.org/jira/browse/FLINK-12690 Project: Flink Issue Type: Improvement

Re: [Discuss] FLIP-43: Savepoint Connector

2019-05-31 Thread Seth Wiesman
@Piotr I definitely would like to see this have sql integrations at some point. The reason for holding off is that to do so would require savepoint format, it is not currently possible to discover states and schemas without state descriptors in a robust way. I think it’s best to keep this in

[jira] [Created] (FLINK-12691) Make Queue Capacity and Timeout of AsyncWaitOperator changeable during runtime

2019-05-31 Thread Konstantin Knauf (JIRA)
Konstantin Knauf created FLINK-12691: Summary: Make Queue Capacity and Timeout of AsyncWaitOperator changeable during runtime Key: FLINK-12691 URL: https://issues.apache.org/jira/browse/FLINK-12691

Re: [Discuss] FLIP-43: Savepoint Connector

2019-05-31 Thread Seth Wiesman
@Konstantin agreed, that was a large impotence for this feature. Also I am happy to change the name to something that better describes the feature set. @Lan Savepoints depend heavily on a number of flink internal components that may change between versions: state backends internals, type seria

Re: [Discuss] FLIP-43: Savepoint Connector

2019-05-31 Thread Seth Wiesman
And I can definitely imagine a “savepoint catalog” at some point in the future. Seth > On May 31, 2019, at 4:39 AM, Tzu-Li (Gordon) Tai wrote: > > @Piotr > Yes, we're aiming this for the 1.9 release. This was also mentioned in the > recent 1.9 feature discussion thread [1]. > > [1] > http://a

Re: [Discuss] FLIP-43: Savepoint Connector

2019-05-31 Thread Jan Lukavský
Hi Seth, that sounds reasonable. What I was asking for was not to reverse engineer binary format, but to make the savepoint write API a little more reusable, so that it could be wrapped into some other technology. I don't know the details enough to propose a solution, but it seems to me, that

[jira] [Created] (FLINK-12692) Support disk spilling in HeapKeyedStateBackend

2019-05-31 Thread Yu Li (JIRA)
Yu Li created FLINK-12692: - Summary: Support disk spilling in HeapKeyedStateBackend Key: FLINK-12692 URL: https://issues.apache.org/jira/browse/FLINK-12692 Project: Flink Issue Type: New Feature

Join in Apache Flink community

2019-05-31 Thread Pengfei Li
Hi Guys, I want to contribute to Apache Flink. Would you please give me the permission as a contributor? Thanks. My JIRA ID is *PengFei Li* (with email address: lpengfei2...@gmail.com) Best Regards, PengFei Li

[jira] [Created] (FLINK-12693) Store state per key-group in CopyOnWriteStateTable

2019-05-31 Thread Yu Li (JIRA)
Yu Li created FLINK-12693: - Summary: Store state per key-group in CopyOnWriteStateTable Key: FLINK-12693 URL: https://issues.apache.org/jira/browse/FLINK-12693 Project: Flink Issue Type: Sub-task

[jira] [Created] (FLINK-12694) Implement a HeapAccountingManager

2019-05-31 Thread Yu Li (JIRA)
Yu Li created FLINK-12694: - Summary: Implement a HeapAccountingManager Key: FLINK-12694 URL: https://issues.apache.org/jira/browse/FLINK-12694 Project: Flink Issue Type: Sub-task Components

[jira] [Created] (FLINK-12695) Implement a HeapStatusMonitor

2019-05-31 Thread Yu Li (JIRA)
Yu Li created FLINK-12695: - Summary: Implement a HeapStatusMonitor Key: FLINK-12695 URL: https://issues.apache.org/jira/browse/FLINK-12695 Project: Flink Issue Type: Sub-task Components: Ru

[jira] [Created] (FLINK-12696) Implement a MmapManager

2019-05-31 Thread Yu Li (JIRA)
Yu Li created FLINK-12696: - Summary: Implement a MmapManager Key: FLINK-12696 URL: https://issues.apache.org/jira/browse/FLINK-12696 Project: Flink Issue Type: Sub-task Components: Runtime

[jira] [Created] (FLINK-12697) Support on-disk state storage for HeapKeyedStateBackend

2019-05-31 Thread Yu Li (JIRA)
Yu Li created FLINK-12697: - Summary: Support on-disk state storage for HeapKeyedStateBackend Key: FLINK-12697 URL: https://issues.apache.org/jira/browse/FLINK-12697 Project: Flink Issue Type: Sub-tas

[jira] [Created] (FLINK-12698) Implement a SpillLoadManager

2019-05-31 Thread Yu Li (JIRA)
Yu Li created FLINK-12698: - Summary: Implement a SpillLoadManager Key: FLINK-12698 URL: https://issues.apache.org/jira/browse/FLINK-12698 Project: Flink Issue Type: Sub-task Components: Run

[jira] [Created] (FLINK-12699) Reduce CPU consumption when snapshot/restore the spilled key-group

2019-05-31 Thread Yu Li (JIRA)
Yu Li created FLINK-12699: - Summary: Reduce CPU consumption when snapshot/restore the spilled key-group Key: FLINK-12699 URL: https://issues.apache.org/jira/browse/FLINK-12699 Project: Flink Issue T

[jira] [Created] (FLINK-12700) Nodejs installation suggestions

2019-05-31 Thread xiezhiqiang (JIRA)
xiezhiqiang created FLINK-12700: --- Summary: Nodejs installation suggestions Key: FLINK-12700 URL: https://issues.apache.org/jira/browse/FLINK-12700 Project: Flink Issue Type: Improvement

Re: [Discuss] FLIP-43: Savepoint Connector

2019-05-31 Thread Piotr Nowojski
> I think it’s best to keep this initial implementation focused and add those > changes if there is adoption and interest in the community. I agree. I didn’t mean to hold the implementation/acceptance of this until someone solve the SQL story :) Piotrek > On 31 May 2019, at 13:18, Seth Wiesma

[jira] [Created] (FLINK-12701) Column name alias causes exception when used with where and group-by

2019-05-31 Thread Josh Bradt (JIRA)
Josh Bradt created FLINK-12701: -- Summary: Column name alias causes exception when used with where and group-by Key: FLINK-12701 URL: https://issues.apache.org/jira/browse/FLINK-12701 Project: Flink

Re: [Discuss] FLIP-43: Savepoint Connector

2019-05-31 Thread Seth Wiesman
@Jan Gotcha, So in reusing components it explicitly is not a writer. This is not a savepoint output format in the way we have a parquet output format. The reason for the Transform api is to hide the underlying details, it does not simply append a output writer to the end of a dataset. This get

Re: [Discuss] FLIP-43: Savepoint Connector

2019-05-31 Thread Jan Lukavský
Hi Seth, yes, that helped! :-) What I was looking for is essentially `org.apache.flink.connectors.savepoint.output.SavepointOutputFormat`. It would be great if this could be written in a way, that would enable reusing it in different engine (as I mentioned - Apache Spark). There seem to be s

Re: [Discuss] FLIP-43: Savepoint Connector

2019-05-31 Thread Seth Wiesman
The SavepointOutputFormat only writes out the savepoint metadata file and should be mostly ignored. The actual state is written out by stream operators and tied into the flink runtime[1, 2, 3]. This is the most important part and the piece that I don’t think can be reasonably extracted. Seth

Re: [ANNOUNCE] Apache Flink-shaded 7.0 released

2019-05-31 Thread Bowen Li
Thanks Jincheng for driving this release! On Thu, May 30, 2019 at 11:40 PM Terry Wang wrote: > Wow~ Glad to see this! > Thanks Jincheng and Chesnay for your effort! > > > 在 2019年5月31日,下午1:53,jincheng sun 写道: > > > > Hi all, > > > > The Apache Flink community is very happy to announce the releas

[jira] [Created] (FLINK-12702) support 'properties' in catalog entries in SQL CLI yaml file

2019-05-31 Thread Bowen Li (JIRA)
Bowen Li created FLINK-12702: Summary: support 'properties' in catalog entries in SQL CLI yaml file Key: FLINK-12702 URL: https://issues.apache.org/jira/browse/FLINK-12702 Project: Flink Issue T