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

2020-02-03 Thread Piotr Nowojski
+1 (binding) Piotrek > On 4 Feb 2020, at 05:39, Zhijiang wrote: > > +1 (binding), we are waiting too long for it. :) > > Best, > Zhijiang > > > -- > From:Guowei Ma > Send Time:2020 Feb. 4 (Tue.) 12:34 > To:dev > Subject:Re:

[jira] [Created] (FLINK-15896) Stop using TableSource::getTableSchema

2020-02-03 Thread Kurt Young (Jira)
Kurt Young created FLINK-15896: -- Summary: Stop using TableSource::getTableSchema Key: FLINK-15896 URL: https://issues.apache.org/jira/browse/FLINK-15896 Project: Flink Issue Type: Sub-task

[jira] [Created] (FLINK-15895) Stop using TableSource::getReturnType except for compatibility purpose

2020-02-03 Thread Kurt Young (Jira)
Kurt Young created FLINK-15895: -- Summary: Stop using TableSource::getReturnType except for compatibility purpose Key: FLINK-15895 URL: https://issues.apache.org/jira/browse/FLINK-15895 Project: Flink

[jira] [Created] (FLINK-15894) Stop overriding TableSource::getTableSchema in flink walkthrough

2020-02-03 Thread Kurt Young (Jira)
Kurt Young created FLINK-15894: -- Summary: Stop overriding TableSource::getTableSchema in flink walkthrough Key: FLINK-15894 URL: https://issues.apache.org/jira/browse/FLINK-15894 Project: Flink

[jira] [Created] (FLINK-15893) Stop overriding TableSource::getTableSchema in python table source

2020-02-03 Thread Kurt Young (Jira)
Kurt Young created FLINK-15893: -- Summary: Stop overriding TableSource::getTableSchema in python table source Key: FLINK-15893 URL: https://issues.apache.org/jira/browse/FLINK-15893 Project: Flink

[jira] [Created] (FLINK-15892) Stop overriding TableSource::getTableSchema in csv table source

2020-02-03 Thread Kurt Young (Jira)
Kurt Young created FLINK-15892: -- Summary: Stop overriding TableSource::getTableSchema in csv table source Key: FLINK-15892 URL: https://issues.apache.org/jira/browse/FLINK-15892 Project: Flink

[jira] [Created] (FLINK-15890) Stop overriding TableSource::getTableSchema in orc table source

2020-02-03 Thread Kurt Young (Jira)
Kurt Young created FLINK-15890: -- Summary: Stop overriding TableSource::getTableSchema in orc table source Key: FLINK-15890 URL: https://issues.apache.org/jira/browse/FLINK-15890 Project: Flink

[jira] [Created] (FLINK-15891) Stop overriding TableSource::getTableSchema in parquet table source

2020-02-03 Thread Kurt Young (Jira)
Kurt Young created FLINK-15891: -- Summary: Stop overriding TableSource::getTableSchema in parquet table source Key: FLINK-15891 URL: https://issues.apache.org/jira/browse/FLINK-15891 Project: Flink

[jira] [Created] (FLINK-15889) Stop overriding TableSource::getTableSchema in jdbc connector

2020-02-03 Thread Kurt Young (Jira)
Kurt Young created FLINK-15889: -- Summary: Stop overriding TableSource::getTableSchema in jdbc connector Key: FLINK-15889 URL: https://issues.apache.org/jira/browse/FLINK-15889 Project: Flink

[jira] [Created] (FLINK-15888) Stop overriding TableSource::getTableSchema in hbase

2020-02-03 Thread Kurt Young (Jira)
Kurt Young created FLINK-15888: -- Summary: Stop overriding TableSource::getTableSchema in hbase Key: FLINK-15888 URL: https://issues.apache.org/jira/browse/FLINK-15888 Project: Flink Issue Type:

[jira] [Created] (FLINK-15887) Stop overriding TableSource::getTableSchema in kafka

2020-02-03 Thread Kurt Young (Jira)
Kurt Young created FLINK-15887: -- Summary: Stop overriding TableSource::getTableSchema in kafka Key: FLINK-15887 URL: https://issues.apache.org/jira/browse/FLINK-15887 Project: Flink Issue Type:

[jira] [Created] (FLINK-15886) Stop overriding TableSource::getTableSchema in hive table source

2020-02-03 Thread Kurt Young (Jira)
Kurt Young created FLINK-15886: -- Summary: Stop overriding TableSource::getTableSchema in hive table source Key: FLINK-15886 URL: https://issues.apache.org/jira/browse/FLINK-15886 Project: Flink

[jira] [Created] (FLINK-15885) Stop overriding TableSource::getTableSchema in tests

2020-02-03 Thread Kurt Young (Jira)
Kurt Young created FLINK-15885: -- Summary: Stop overriding TableSource::getTableSchema in tests Key: FLINK-15885 URL: https://issues.apache.org/jira/browse/FLINK-15885 Project: Flink Issue Type:

[jira] [Created] (FLINK-15884) Stop overriding TableSource::getReturnType in python table source

2020-02-03 Thread Kurt Young (Jira)
Kurt Young created FLINK-15884: -- Summary: Stop overriding TableSource::getReturnType in python table source Key: FLINK-15884 URL: https://issues.apache.org/jira/browse/FLINK-15884 Project: Flink

[jira] [Created] (FLINK-15882) Stop overriding TableSource::getReturnType in orc table source

2020-02-03 Thread Kurt Young (Jira)
Kurt Young created FLINK-15882: -- Summary: Stop overriding TableSource::getReturnType in orc table source Key: FLINK-15882 URL: https://issues.apache.org/jira/browse/FLINK-15882 Project: Flink

[jira] [Created] (FLINK-15883) Stop overriding TableSource::getReturnType in parquet table source

2020-02-03 Thread Kurt Young (Jira)
Kurt Young created FLINK-15883: -- Summary: Stop overriding TableSource::getReturnType in parquet table source Key: FLINK-15883 URL: https://issues.apache.org/jira/browse/FLINK-15883 Project: Flink

[jira] [Created] (FLINK-15881) Stop overriding TableSource::getReturnType in jdbc connector

2020-02-03 Thread Kurt Young (Jira)
Kurt Young created FLINK-15881: -- Summary: Stop overriding TableSource::getReturnType in jdbc connector Key: FLINK-15881 URL: https://issues.apache.org/jira/browse/FLINK-15881 Project: Flink

[jira] [Created] (FLINK-15879) Stop overriding TableSource::getReturnType in kafka connector

2020-02-03 Thread Kurt Young (Jira)
Kurt Young created FLINK-15879: -- Summary: Stop overriding TableSource::getReturnType in kafka connector Key: FLINK-15879 URL: https://issues.apache.org/jira/browse/FLINK-15879 Project: Flink

[jira] [Created] (FLINK-15880) Stop overriding TableSource::getReturnType in hbase connector

2020-02-03 Thread Kurt Young (Jira)
Kurt Young created FLINK-15880: -- Summary: Stop overriding TableSource::getReturnType in hbase connector Key: FLINK-15880 URL: https://issues.apache.org/jira/browse/FLINK-15880 Project: Flink

[jira] [Created] (FLINK-15878) Stop overriding TableSource::getReturnType in tests

2020-02-03 Thread Kurt Young (Jira)
Kurt Young created FLINK-15878: -- Summary: Stop overriding TableSource::getReturnType in tests Key: FLINK-15878 URL: https://issues.apache.org/jira/browse/FLINK-15878 Project: Flink Issue Type:

[jira] [Created] (FLINK-15877) Stop using deprecated methods from TableSource interface

2020-02-03 Thread Kurt Young (Jira)
Kurt Young created FLINK-15877: -- Summary: Stop using deprecated methods from TableSource interface Key: FLINK-15877 URL: https://issues.apache.org/jira/browse/FLINK-15877 Project: Flink Issue

Re: [DISCUSS] Include flink-ml-api and flink-ml-lib in opt

2020-02-03 Thread Hequn Cheng
Hi Till, Thanks a lot for your suggestion. It's a good idea to offer the flink-ml libraries as optional dependencies on the download page which can make the dist smaller. But I also have some concerns for it, e.g., the download page now only includes the latest 3 releases. We may need to find

Re: [DISCUSS] Upload the Flink Python API 1.9.x to PyPI for user convenience.

2020-02-03 Thread Hequn Cheng
Hi Jincheng, +1 for this proposal. >From the perspective of users, I think it would nice to have PyFlink on PyPI which makes it much easier to install PyFlink. Best, Hequn On Tue, Feb 4, 2020 at 1:09 PM Jeff Zhang wrote: > +1 > > > Xingbo Huang 于2020年2月4日周二 下午1:07写道: > >> Hi Jincheng, >> >>

Re: RocksDB Compaction filter to clean up state with TTL

2020-02-03 Thread Yu Li
Hi Abhilasha, We were conservative about enabling this by default to prevent any unexpected problems. However, since there has been no reported issues so far, we will enable this by default in 1.10.0 release. Please refer to FLINK-14898 [1], FLINK-15506 [2] and (currently drafted) 1.10.0 release

[jira] [Created] (FLINK-15876) The alternative code for GroupedProcessingTimeWindowExample don't compile pass current version

2020-02-03 Thread Wong (Jira)
Wong created FLINK-15876: Summary: The alternative code for GroupedProcessingTimeWindowExample don't compile pass current version Key: FLINK-15876 URL: https://issues.apache.org/jira/browse/FLINK-15876

[VOTE] Improve TableFactory to add Context

2020-02-03 Thread Jingsong Li
Hi all, I would like to start the vote for the improve of TableFactory, which is discussed and reached a consensus in the discussion thread[2]. The vote will be open for at least 72 hours. I'll try to close it unless there is an objection or not enough votes. [1]

Re: [DISCUSS] Improve TableFactory

2020-02-03 Thread Jingsong Li
So the interface will be: public interface TableSourceFactory extends TableFactory { .. /** * Creates and configures a {@link TableSource} based on the given {@link Context}. * * @param context context of this table source. * @return the configured table source. */

Re: [DISCUSS] Improve TableFactory

2020-02-03 Thread Jingsong Li
Hi all, After rethinking and discussion with Kurt, I'd like to remove "isBounded". We can delay this is bounded message to TableSink. With TableSink refactor, we need consider "consumeDataStream" and "consumeBoundedStream". Best, Jingsong Lee On Mon, Feb 3, 2020 at 4:17 PM Jingsong Li wrote:

Re: [DISCUSS] Upload the Flink Python API 1.9.x to PyPI for user convenience.

2020-02-03 Thread Jeff Zhang
+1 Xingbo Huang 于2020年2月4日周二 下午1:07写道: > Hi Jincheng, > > Thanks for driving this. > +1 for this proposal. > > Compared to building from source, downloading directly from PyPI will > greatly save the development cost of Python users. > > Best, > Xingbo > > > > Wei Zhong 于2020年2月4日周二

Re: [DISCUSS] Upload the Flink Python API 1.9.x to PyPI for user convenience.

2020-02-03 Thread Xingbo Huang
Hi Jincheng, Thanks for driving this. +1 for this proposal. Compared to building from source, downloading directly from PyPI will greatly save the development cost of Python users. Best, Xingbo Wei Zhong 于2020年2月4日周二 下午12:43写道: > Hi Jincheng, > > Thanks for bring up this discussion! > > +1

Re: [DISCUSS] Upload the Flink Python API 1.9.x to PyPI for user convenience.

2020-02-03 Thread Wei Zhong
Hi Jincheng, Thanks for bring up this discussion! +1 for this proposal. Building from source takes long time and requires a good network environment. Some users may not have such an environment. Uploading to PyPI will greatly improve the user experience. Best, Wei jincheng sun 于2020年2月4日周二

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

2020-02-03 Thread Zhijiang
+1 (binding), we are waiting too long for it. :) Best, Zhijiang -- From:Guowei Ma Send Time:2020 Feb. 4 (Tue.) 12:34 To:dev Subject:Re: [VOTE] FLIP-27 - Refactor Source Interface +1 (non-binding), thanks for driving. Best,

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

2020-02-03 Thread Guowei Ma
+1 (non-binding), thanks for driving. Best, Guowei Jingsong Li 于2020年2月4日周二 上午11:20写道: > +1 (non-binding), thanks for driving. > FLIP-27 is the basis of a lot of follow-up work. > > Best, > Jingsong Lee > > On Tue, Feb 4, 2020 at 10:26 AM Jark Wu wrote: > > > Thanks for driving this Becket!

[jira] [Created] (FLINK-15875) Bump Beam to 2.19.0

2020-02-03 Thread Dian Fu (Jira)
Dian Fu created FLINK-15875: --- Summary: Bump Beam to 2.19.0 Key: FLINK-15875 URL: https://issues.apache.org/jira/browse/FLINK-15875 Project: Flink Issue Type: Improvement Components: API

[jira] [Created] (FLINK-15874) Setup Travis cron job for Stateful Functions documentation build

2020-02-03 Thread Tzu-Li (Gordon) Tai (Jira)
Tzu-Li (Gordon) Tai created FLINK-15874: --- Summary: Setup Travis cron job for Stateful Functions documentation build Key: FLINK-15874 URL: https://issues.apache.org/jira/browse/FLINK-15874

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

2020-02-03 Thread Wei Zhong
+1 (non-binding), thanks for driving this! Best, Wei > 在 2020年2月4日,11:47,Danny Chan 写道: > > +1 (non-binding), thanks for the great work ! > > Best, > Danny Chan > 在 2020年2月4日 +0800 AM11:20,dev@flink.apache.org,写道: >> >> +1 (non-binding), thanks for driving. >> FLIP-27 is the basis of a lot

[jira] [Created] (FLINK-15873) Matched result may not be output if existing earlier partial matches

2020-02-03 Thread shuai.xu (Jira)
shuai.xu created FLINK-15873: Summary: Matched result may not be output if existing earlier partial matches Key: FLINK-15873 URL: https://issues.apache.org/jira/browse/FLINK-15873 Project: Flink

[DISCUSS] Upload the Flink Python API 1.9.x to PyPI for user convenience.

2020-02-03 Thread jincheng sun
Hi folks, I am very happy to receive some user inquiries about the use of Flink Python API (PyFlink) recently. One of the more common questions is whether it is possible to install PyFlink without using source code build. The most convenient and natural way for users is to use `pip install

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

2020-02-03 Thread Danny Chan
+1 (non-binding), thanks for the great work ! Best, Danny Chan 在 2020年2月4日 +0800 AM11:20,dev@flink.apache.org,写道: > > +1 (non-binding), thanks for driving. > FLIP-27 is the basis of a lot of follow-up work.

[jira] [Created] (FLINK-15872) Remove unnecessary code in InputFormat

2020-02-03 Thread Paul Lin (Jira)
Paul Lin created FLINK-15872: Summary: Remove unnecessary code in InputFormat Key: FLINK-15872 URL: https://issues.apache.org/jira/browse/FLINK-15872 Project: Flink Issue Type: Improvement

Re: [VOTE] Release 1.10.0, release candidate #1

2020-02-03 Thread Jingsong Li
Another critical issue is FLINK-15858[1]. It is indeed a regression. But we don''t want to block release. Will try our best to fix it. [1] https://issues.apache.org/jira/browse/FLINK-15858 Best, Jingsong Lee On Tue, Feb 4, 2020 at 9:56 AM Thomas Weise wrote: > I opened a PR for FLINK-15868 >

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

2020-02-03 Thread Jingsong Li
+1 (non-binding), thanks for driving. FLIP-27 is the basis of a lot of follow-up work. Best, Jingsong Lee On Tue, Feb 4, 2020 at 10:26 AM Jark Wu wrote: > Thanks for driving this Becket! > > +1 from my side. > > Cheers, > Jark > > On Mon, 3 Feb 2020 at 18:06, Yu Li wrote: > > > +1, thanks for

Re: [DISCUSS] Update UpsertStreamTableSink and RetractStreamTableSink to new type system

2020-02-03 Thread Zhenghua Gao
Hi Jark, thanks for your comments. >>>IIUC, the framework will only recognize getRecordDataType and >>>ignore getConsumedDataType for UpsertStreamTableSink, is that right? Your are right. >>>getRecordDataType is little confused as UpsertStreamTableSink already has >>>three getXXXType(). the

Re: [DISCUSS] Update UpsertStreamTableSink and RetractStreamTableSink to new type system

2020-02-03 Thread Zhenghua Gao
Should we distinguish *record data type* and *consumed data type*? Currently the design of UpsertStreamTableSink and RetractStreamTableSink DO distinguish them. In my proposal the framework will ignore *getConsumedDataType*, so it's ok to use *getConsumedDataType* to do the job if we don't

[jira] [Created] (FLINK-15871) Support to start sidecar container

2020-02-03 Thread Yang Wang (Jira)
Yang Wang created FLINK-15871: - Summary: Support to start sidecar container Key: FLINK-15871 URL: https://issues.apache.org/jira/browse/FLINK-15871 Project: Flink Issue Type: Sub-task

[jira] [Created] (FLINK-15870) Wait the job's terminal state through REST interface

2020-02-03 Thread Yangze Guo (Jira)
Yangze Guo created FLINK-15870: -- Summary: Wait the job's terminal state through REST interface Key: FLINK-15870 URL: https://issues.apache.org/jira/browse/FLINK-15870 Project: Flink Issue Type:

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

2020-02-03 Thread Jark Wu
Thanks for driving this Becket! +1 from my side. Cheers, Jark On Mon, 3 Feb 2020 at 18:06, Yu Li wrote: > +1, thanks for the efforts Becket! > > Best Regards, > Yu > > > On Mon, 3 Feb 2020 at 17:52, Becket Qin wrote: > > > Bump up the thread. > > > > On Tue, Jan 21, 2020 at 10:43 AM Becket

Re: Large intervaljoin related question

2020-02-03 Thread Jark Wu
Hi Chen, AFAIK, DataStream doesn't have too much operator level optimization for large window and interval join. The suggested best practices are - please do not slide a window with very small step - use rocksdb statebackend on SSD for large state. - increase parallelism for the operators of

Re: [VOTE] Release 1.10.0, release candidate #1

2020-02-03 Thread Thomas Weise
I opened a PR for FLINK-15868 : https://github.com/apache/flink/pull/11006 With that change, I was able to run an application that consumes from Kinesis. I should have data tomorrow regarding the performance. Two questions/observations: 1) Is

Re: [ANNOUNCE] Yu Li became a Flink committer

2020-02-03 Thread Danny Chan
Congratulations Yu! Best, Danny Chan 在 2020年2月4日 +0800 AM1:46,dev@flink.apache.org,写道: > > Congratulations Yu!

[jira] [Created] (FLINK-15869) Suppoort to read and write meta information of a table

2020-02-03 Thread Jark Wu (Jira)
Jark Wu created FLINK-15869: --- Summary: Suppoort to read and write meta information of a table Key: FLINK-15869 URL: https://issues.apache.org/jira/browse/FLINK-15869 Project: Flink Issue Type: New

RocksDB Compaction filter to clean up state with TTL

2020-02-03 Thread Seth, Abhilasha
Hello, Flink 1.8 introduces the config ‘state.backend.rocksdb.ttl.compaction.filter.enabled’ to enable or disable the compaction filter to cleanup state with TTL. I was curious why its disabled by default. Are there any performance implications of turning it ON by default? Thanks, Abhilasha

Re: [VOTE] Release 1.10.0, release candidate #1

2020-02-03 Thread Thomas Weise
I found another issue with the Kinesis connector: https://issues.apache.org/jira/browse/FLINK-15868 On Mon, Feb 3, 2020 at 3:35 AM Gary Yao wrote: > Hi everyone, > > I am hereby canceling the vote due to: > > FLINK-15837 > FLINK-15840 > > Another RC will be created later today. > >

[jira] [Created] (FLINK-15868) Kinesis consumer fails due to jackson-cbor conflict in 1.10 RC1

2020-02-03 Thread Thomas Weise (Jira)
Thomas Weise created FLINK-15868: Summary: Kinesis consumer fails due to jackson-cbor conflict in 1.10 RC1 Key: FLINK-15868 URL: https://issues.apache.org/jira/browse/FLINK-15868 Project: Flink

Re: [ANNOUNCE] Yu Li became a Flink committer

2020-02-03 Thread Henry Saputra
Belated congrats to Yu Li - Henry On Thu, Jan 23, 2020 at 12:59 AM Stephan Ewen wrote: > Hi all! > > We are announcing that Yu Li has joined the rank of Flink committers. > > Yu joined already in late December, but the announcement got lost because > of the Christmas and New Years season, so

Re: [DISCUSS] have separate Flink distributions with built-in Hive dependencies

2020-02-03 Thread Stephan Ewen
We have had much trouble in the past from "too deep too custom" integrations that everyone got out of the box, i.e., Hadoop. Flink has has such a broad spectrum of use cases, if we have custom build for every other framework in that spectrum, we'll be in trouble. So I would also be -1 for custom

[jira] [Created] (FLINK-15867) LAST_VALUE aggregate function does not support time-related types

2020-02-03 Thread Jira
Benoît Paris created FLINK-15867: Summary: LAST_VALUE aggregate function does not support time-related types Key: FLINK-15867 URL: https://issues.apache.org/jira/browse/FLINK-15867 Project: Flink

[jira] [Created] (FLINK-15866) ClosureCleaner#getSuperClassOrInterfaceName throw NPE

2020-02-03 Thread Aven Wu (Jira)
Aven Wu created FLINK-15866: --- Summary: ClosureCleaner#getSuperClassOrInterfaceName throw NPE Key: FLINK-15866 URL: https://issues.apache.org/jira/browse/FLINK-15866 Project: Flink Issue Type: Bug

Re: [DISCUSS] Update UpsertStreamTableSink and RetractStreamTableSink to new type system

2020-02-03 Thread Jark Wu
Thanks Zhenghua for starting this discussion. Currently, all the UpsertStreamTableSinks can't upgrade to the new type system which affects usability a lot. I hope we can fix that in 1.11. I'm find with *getRecordDataType* for a temporary solution. IIUC, the framework will only recognize

[jira] [Created] (FLINK-15865) When to add .uid() call: inconsistent definition of operators in Flink docs

2020-02-03 Thread Jun Qin (Jira)
Jun Qin created FLINK-15865: --- Summary: When to add .uid() call: inconsistent definition of operators in Flink docs Key: FLINK-15865 URL: https://issues.apache.org/jira/browse/FLINK-15865 Project: Flink

Re: [VOTE] Release 1.10.0, release candidate #1

2020-02-03 Thread Gary Yao
Hi everyone, I am hereby canceling the vote due to: FLINK-15837 FLINK-15840 Another RC will be created later today. Best, Gary On Mon, Jan 27, 2020 at 10:06 PM Gary Yao wrote: > Hi everyone, > Please review and vote on the release candidate #1 for the version 1.10.0, > as follows: >

[jira] [Created] (FLINK-15864) Upgrade jackson-databind dependency to 2.10.1 for security reasons

2020-02-03 Thread Till Rohrmann (Jira)
Till Rohrmann created FLINK-15864: - Summary: Upgrade jackson-databind dependency to 2.10.1 for security reasons Key: FLINK-15864 URL: https://issues.apache.org/jira/browse/FLINK-15864 Project: Flink

[jira] [Created] (FLINK-15863) Fix docs stating that savepoints are relocatable

2020-02-03 Thread Nico Kruber (Jira)
Nico Kruber created FLINK-15863: --- Summary: Fix docs stating that savepoints are relocatable Key: FLINK-15863 URL: https://issues.apache.org/jira/browse/FLINK-15863 Project: Flink Issue Type:

Re: [DISCUSS] FLIP-75: Flink Web UI Improvement Proposal

2020-02-03 Thread Till Rohrmann
I think there is no such description because we never did it before. I just figured that FLIP-75 could actually be a good candidate to start this practice. We would need a community discussion first, though. Cheers, Till On Mon, Feb 3, 2020 at 10:28 AM Yadong Xie wrote: > Hi Till > I didn’t

Re: [DISCUSS] Include flink-ml-api and flink-ml-lib in opt

2020-02-03 Thread Till Rohrmann
An alternative solution would be to offer the flink-ml libraries as optional dependencies on the download page. Similar to how we offer the different SQL formats and Hadoop releases [1]. [1] https://flink.apache.org/downloads.html Cheers, Till On Mon, Feb 3, 2020 at 10:19 AM Hequn Cheng wrote:

Re: Quick Introduction of myself - Kartheek.

2020-02-03 Thread Till Rohrmann
Hi Kartheek, welcome to the Flink community. The best way to get started is to read the contribution guidelines [1]. [1] https://flink.apache.org/contributing/how-to-contribute.html Cheers, Till On Sun, Feb 2, 2020 at 6:52 AM Kartheek kark wrote: > Good Morning everyone, > I am kartheek,

Re: REST Monitoring Savepoint failed

2020-02-03 Thread Till Rohrmann
At the moment this is the case Ramya. We plan to add the auto scaling feature back again in one of the future Flink versions, though. Cheers, Till On Mon, Feb 3, 2020 at 5:27 AM Ramya Ramamurthy wrote: > Thanks Till Rohrmann for the update. > > So even if we upgrade to the newer version of

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

2020-02-03 Thread Yu Li
+1, thanks for the efforts Becket! Best Regards, Yu On Mon, 3 Feb 2020 at 17:52, Becket Qin wrote: > Bump up the thread. > > On Tue, Jan 21, 2020 at 10:43 AM Becket Qin wrote: > > > Hi Folks, > > > > I'd like to resume the voting thread for FlIP-27. > > > > Please note that the FLIP wiki has

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

2020-02-03 Thread Becket Qin
Bump up the thread. On Tue, Jan 21, 2020 at 10:43 AM Becket Qin wrote: > Hi Folks, > > I'd like to resume the voting thread for FlIP-27. > > Please note that the FLIP wiki has been updated to reflect the latest > discussions in the discussion thread. > > To avoid confusion, I'll only count the

Re: [VOTE] Release 1.10.0, release candidate #1

2020-02-03 Thread Kostas Kloudas
+1 (binding) - Built Flink locally - Tested quickstart by writing simple, WordCount-like jobs - Submitted them to Yarn both "per-job" and "session" mode For Thomas' comment, I agree that in this release we change how some of the execution options are propagated through the stack. This was done

Re: [DISCUSS] Update UpsertStreamTableSink and RetractStreamTableSink to new type system

2020-02-03 Thread Zhenghua Gao
Hi Jingsong, For now, only UpsertStreamTableSink and RetractStreamTableSink consumes JTuple2 So the 'getConsumedDataType' interface is not necessary in validate & codegen phase. See https://github.com/apache/flink/pull/10880/files#diff-137d090bd719f3a99ae8ba6603ed81ebR52 and

Re: [DISCUSS] FLIP-75: Flink Web UI Improvement Proposal

2020-02-03 Thread Yadong Xie
Hi Till I didn’t find how to create of sub flip at cwiki.apache.org do you mean to create 9 more FLIPS instead of FLIP-75? Till Rohrmann 于2020年1月30日周四 下午11:12写道: > Would it be easier if FLIP-75 would be the umbrella FLIP and we would vote > on the individual improvements as sub FLIPs?

[DISCUSS] FLIP-94 Rework 2-phase commit abstractions

2020-02-03 Thread Roman Khachatryan
Hi everyone, I'd like to kick off the discussion on the redesign of TwoPhaseCommitSinkFunction [1]. The primary motivation is to provide a solution that suits the needs of both Kafka Sink and JDBC exactly once sink. Other possible scenarios include File Sink, WAL and batch jobs. Current

Re: connection timeout during shuffle initialization

2020-02-03 Thread Piotr Nowojski
Hi, Coming back to this idea: >> Removing synchronization *did solve* the problem for me, because it >> allows flink to leverage the whole netty event loop pool and it's ok to >> have a single thread blocked for a little while (we still can accept >> connections with other threads). Assignment

Re: [DISCUSS] Include flink-ml-api and flink-ml-lib in opt

2020-02-03 Thread Hequn Cheng
Thank you all for your feedback and suggestions! Best, Hequn On Mon, Feb 3, 2020 at 5:07 PM Becket Qin wrote: > Thanks for bringing up the discussion, Hequn. > > +1 on adding `flink-ml-api` and `flink-ml-lib` into opt. This would make > it much easier for the users to try out some simple ml

Re: [DISCUSS] Include flink-ml-api and flink-ml-lib in opt

2020-02-03 Thread Becket Qin
Thanks for bringing up the discussion, Hequn. +1 on adding `flink-ml-api` and `flink-ml-lib` into opt. This would make it much easier for the users to try out some simple ml tasks. Thanks, Jiangjie (Becket) Qin On Mon, Feb 3, 2020 at 4:34 PM jincheng sun wrote: > Thank you for pushing

Re: [DISCUSS] Update UpsertStreamTableSink and RetractStreamTableSink to new type system

2020-02-03 Thread Jingsong Li
Hi Zhenghua, The *getRecordDataType* looks good to me. But the main problem is how to represent the tuple type in DataType. I understand that it is necessary to use StructuredType, but at present, planner does not support StructuredType, so the other way is to support StructuredType. Best,

Re: [DISCUSS] Update UpsertStreamTableSink and RetractStreamTableSink to new type system

2020-02-03 Thread Kurt Young
Would overriding `getConsumedDataType` do the job? Best, Kurt On Mon, Feb 3, 2020 at 3:52 PM Zhenghua Gao wrote: > Hi all, > > FLINK-12254[1] [2] updated TableSink and related interfaces to new type > system which > allows connectors use the new type system based on DataTypes. > > But

[jira] [Created] (FLINK-15862) Remove deprecated class KafkaPartitioner and all its usages

2020-02-03 Thread Dawid Wysakowicz (Jira)
Dawid Wysakowicz created FLINK-15862: Summary: Remove deprecated class KafkaPartitioner and all its usages Key: FLINK-15862 URL: https://issues.apache.org/jira/browse/FLINK-15862 Project: Flink

[jira] [Created] (FLINK-15861) Change TableEnvironmentImpl to StreamTableaEnvironmentImpl in python blink batch mode

2020-02-03 Thread Huang Xingbo (Jira)
Huang Xingbo created FLINK-15861: Summary: Change TableEnvironmentImpl to StreamTableaEnvironmentImpl in python blink batch mode Key: FLINK-15861 URL: https://issues.apache.org/jira/browse/FLINK-15861

Re: [DISCUSS] Include flink-ml-api and flink-ml-lib in opt

2020-02-03 Thread jincheng sun
Thank you for pushing forward @Hequn Cheng ! Hi @Becket Qin , Do you have any concerns on this ? Best, Jincheng Hequn Cheng 于2020年2月3日周一 下午2:09写道: > Hi everyone, > > Thanks for the feedback. As there are no objections, I've opened a JIRA > issue(FLINK-15847[1]) to address this issue. > The

Re: [DISCUSS] Improve TableFactory

2020-02-03 Thread Jingsong Li
Hi Jark, Thanks involving, yes, it's hard to understand to add isBounded on the source. I recommend adding only to sink at present, because sink has upstream. Its upstream is either bounded or unbounded. Hi all, Let me summarize with your suggestions. public interface TableSourceFactory

[jira] [Created] (FLINK-15860) Store temporary functions as CatalogFunctions in FunctionCatalog

2020-02-03 Thread Dawid Wysakowicz (Jira)
Dawid Wysakowicz created FLINK-15860: Summary: Store temporary functions as CatalogFunctions in FunctionCatalog Key: FLINK-15860 URL: https://issues.apache.org/jira/browse/FLINK-15860 Project:

[jira] [Created] (FLINK-15859) Unify identifiers in the interface methods of CatalogManager

2020-02-03 Thread Dawid Wysakowicz (Jira)
Dawid Wysakowicz created FLINK-15859: Summary: Unify identifiers in the interface methods of CatalogManager Key: FLINK-15859 URL: https://issues.apache.org/jira/browse/FLINK-15859 Project: Flink