Re: [DISCUSS] Unified Core API for Streaming and Batch

2018-12-04 Thread Guowei Ma
Hi, all Thanks to Haibo for initiating this discussion in the community. - Relationship of DataStream, DataSet, and Table API Table/DataStream/Dataset does have different aspects. For example, DataStream can access State and Table cannot. DataStream can be easily extended by users because they d

[jira] [Created] (FLINK-11073) Make serializers immutable / provide option TypeSerializerSchemaCompatibility.compatibleWithReconfiguredSerializer

2018-12-04 Thread Tzu-Li (Gordon) Tai (JIRA)
Tzu-Li (Gordon) Tai created FLINK-11073: --- Summary: Make serializers immutable / provide option TypeSerializerSchemaCompatibility.compatibleWithReconfiguredSerializer Key: FLINK-11073 URL: https://issues.apac

Re: [DISCUSS] Flink SQL DDL Design

2018-12-04 Thread Jark Wu
Hi Shaoxuan, Thanks for pointing that out. Yes, the source/sink tag on create table is the another major difference. Summarize the main differences again: *(1) watermark definition *(2) CREATE SOURCE/SINK TABLE or CREATE TABLE (3) View DDL with properties (4) Type Definition Best, Jark On Wed,

Re: [DISCUSS] Flink SQL DDL Design

2018-12-04 Thread Shaoxuan Wang
Hi Jark, Thanks for the summary. Your plan for the 1st round implementation of DDL looks good to me. Have we reached the agreement on simplifying/unifying "create [source/sink] table" to "create table"? "Watermark definition" and "create table" are the major obstacles on the way to merge two design

[jira] [Created] (FLINK-11072) KafkaITCase.testKeyValueSupport failure

2018-12-04 Thread Bowen Li (JIRA)
Bowen Li created FLINK-11072: Summary: KafkaITCase.testKeyValueSupport failure Key: FLINK-11072 URL: https://issues.apache.org/jira/browse/FLINK-11072 Project: Flink Issue Type: Bug Com

Re: [DISCUSS] Flink SQL DDL Design

2018-12-04 Thread Jark Wu
Hi Shuyi, It seems that you have reviewed the DDL doc [1] that Lin and I drafted. This doc covers all the features running in Alibaba. But some of features might be not needed in the first version of Flink SQL DDL. So my suggestion would be to focus on the MVP DDLs and reach agreement ASAP based

Re: [DISCUSS] Embracing Table API in Flink ML

2018-12-04 Thread Weihua Jiang
It has pasted a while and I think we can move forward to JIRA discussion. I will try to split the design into smaller pieces to make it more understandable. Actually, I have already implemented an initial version and ported some flink.ml algorithms using this new API. Thus, we can have a better b

Re: [DISCUSS] Unified Core API for Streaming and Batch

2018-12-04 Thread Kurt Young
Hi all, Really excited to see this discussion really happens, I also want to share my two cents here. Lets first focus on this question: “What Flink API Stack Should be for a Unified Engine". There are multiply ways to judge whether an engine is unified or not. From user's perspective, as long as

Re: [DISCUSS] Support Interactive Programming in Flink Table API

2018-12-04 Thread Jark Wu
Hi, All the recent discussions are focused on whether there is a problem if cache() not return a Table. It seems that returning a Table explicitly is more clear (and safe?). So whether there are any problems if cache() returns a Table? @Becket Best, Jark On Tue, 4 Dec 2018 at 22:27, Till Rohrm

Re: [DISCUSS] Support Higher-order functions in Flink sql

2018-12-04 Thread Jark Wu
Hi Wenhui, This is a meaningful direction to improve the functionality for Flink SQL. As Xuefu suggested, you can come up with a design doc covering the functions you'd like to support and the improvements. IMO, the main obstacle might be the syntax for the lambda function which is not supported i

[jira] [Created] (FLINK-11071) Dynamic proxy classes cannot be resolved when deserializing job graph

2018-12-04 Thread Oleg Zhukov (JIRA)
Oleg Zhukov created FLINK-11071: --- Summary: Dynamic proxy classes cannot be resolved when deserializing job graph Key: FLINK-11071 URL: https://issues.apache.org/jira/browse/FLINK-11071 Project: Flink

Re: [ANNOUNCE] Apache Flink 1.7.0 released

2018-12-04 Thread Bowen Li
Congratulations everyone! On Mon, Dec 3, 2018 at 11:55 PM Shuyi Chen wrote: > Thanks a lot for the hard work, Till! > > Shuyi > > On Sat, Dec 1, 2018 at 4:07 AM Dominik Wosiński wrote: > > > Thanks Till for being the release manager! > > Thanks Everyone and Great Job. > > > > Best Regards, > >

Proposal regarding Queryable State

2018-12-04 Thread Galen Warren
Hi all -- I added an issue for a proposed enhancement to Queryable State, to allow for a transform to be applied to the state information, on the TaskManager, before returning results to the client. More information is in the issue . I'm intereste

Re: withPartitioner() vs calling partitionCustom() beforehand

2018-12-04 Thread Giannis Evagorou
Hi Till, Thank you for your answer. Giannis From: Till Rohrmann Sent: Monday, December 3, 2018 1:46 PM To: dev@flink.apache.org Subject: Re: withPartitioner() vs calling partitionCustom() beforehand Hi Giannis, logically the resulting plans should be identical

Re: [DISCUSS] Support Interactive Programming in Flink Table API

2018-12-04 Thread Till Rohrmann
It's true that b, c, d and e will all read from the original DAG that generates a. But all subsequent operators (when running multiple queries) which reference cachedTableA should not need to reproduce `a` but directly consume the intermediate result. Conceptually one could think of cache() as int

Re: [DISCUSS] Support Interactive Programming in Flink Table API

2018-12-04 Thread Becket Qin
Hi Till, Thanks for the clarification. I am still a little confused. If cache() returns a CachedTable, the example might become: b = a.map(...) c = a.map(...) cachedTableA = a.cache() d = cachedTableA.map(...) e = a.map() In the above case, if cache() is lazily evaluated, b, c, d and e are all

[jira] [Created] (FLINK-11070) Add stream-stream non-window cross join

2018-12-04 Thread Hequn Cheng (JIRA)
Hequn Cheng created FLINK-11070: --- Summary: Add stream-stream non-window cross join Key: FLINK-11070 URL: https://issues.apache.org/jira/browse/FLINK-11070 Project: Flink Issue Type: New Feature

[jira] [Created] (FLINK-11069) Remove FutureUtil

2018-12-04 Thread Till Rohrmann (JIRA)
Till Rohrmann created FLINK-11069: - Summary: Remove FutureUtil Key: FLINK-11069 URL: https://issues.apache.org/jira/browse/FLINK-11069 Project: Flink Issue Type: Improvement Affects Versi

[jira] [Created] (FLINK-11068) Port Table class to Java

2018-12-04 Thread Timo Walther (JIRA)
Timo Walther created FLINK-11068: Summary: Port Table class to Java Key: FLINK-11068 URL: https://issues.apache.org/jira/browse/FLINK-11068 Project: Flink Issue Type: New Feature Co

[jira] [Created] (FLINK-11067) Port TableEnvironments to Java

2018-12-04 Thread Timo Walther (JIRA)
Timo Walther created FLINK-11067: Summary: Port TableEnvironments to Java Key: FLINK-11067 URL: https://issues.apache.org/jira/browse/FLINK-11067 Project: Flink Issue Type: Sub-task

Re: [DISCUSS] Support Interactive Programming in Flink Table API

2018-12-04 Thread Till Rohrmann
Yes you are right Becket that it still depends on the actual execution of the job whether a consumer reads from a cached result or not. My point was actually about the properties of a (cached vs. non-cached) and not about the execution. I would not make cache trigger the execution of the job becau

[jira] [Created] (FLINK-11066) Migrate main Table API classes to flink-table-api-base

2018-12-04 Thread Timo Walther (JIRA)
Timo Walther created FLINK-11066: Summary: Migrate main Table API classes to flink-table-api-base Key: FLINK-11066 URL: https://issues.apache.org/jira/browse/FLINK-11066 Project: Flink Issue

[jira] [Created] (FLINK-11065) Migrate flink-table runtime classes

2018-12-04 Thread Timo Walther (JIRA)
Timo Walther created FLINK-11065: Summary: Migrate flink-table runtime classes Key: FLINK-11065 URL: https://issues.apache.org/jira/browse/FLINK-11065 Project: Flink Issue Type: Sub-task

[jira] [Created] (FLINK-11064) Setup a new flink-table module structure

2018-12-04 Thread Timo Walther (JIRA)
Timo Walther created FLINK-11064: Summary: Setup a new flink-table module structure Key: FLINK-11064 URL: https://issues.apache.org/jira/browse/FLINK-11064 Project: Flink Issue Type: Sub-task

[jira] [Created] (FLINK-11063) Make flink-table Scala-free

2018-12-04 Thread Timo Walther (JIRA)
Timo Walther created FLINK-11063: Summary: Make flink-table Scala-free Key: FLINK-11063 URL: https://issues.apache.org/jira/browse/FLINK-11063 Project: Flink Issue Type: New Feature

[jira] [Created] (FLINK-11062) web ui log error

2018-12-04 Thread lining (JIRA)
lining created FLINK-11062: -- Summary: web ui log error Key: FLINK-11062 URL: https://issues.apache.org/jira/browse/FLINK-11062 Project: Flink Issue Type: Bug Components: REST, Webfrontend

[jira] [Created] (FLINK-11061) Add travis profile that would run on each commit with scala 2.12

2018-12-04 Thread Dawid Wysakowicz (JIRA)
Dawid Wysakowicz created FLINK-11061: Summary: Add travis profile that would run on each commit with scala 2.12 Key: FLINK-11061 URL: https://issues.apache.org/jira/browse/FLINK-11061 Project: Fli