Re: [DISCUSS] Flink SQL DDL Design

2018-11-26 Thread Shuyi Chen
Thanks a lot, Xuefu. I was busy for some other stuff for the last 2 weeks, but we are definitely interested in moving this forward. I think once the unified connector API design [1] is done, we can finalize the DDL design as well and start creating concrete subtasks to collaborate on the

Re: [DISCUSS] Flink SQL DDL Design

2018-11-26 Thread Shuyi Chen
Hi Wenlong, thanks a lot for the comments. 1) I agree we can infer the table type from the queries if the Flink job is static. However, for SQL client cases, the query is adhoc, dynamic, and not known beforehand. In such case, we might want to enforce the table open mode at startup time, so users

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

2018-11-26 Thread Biao Liu
Hi Kostas again, Did I misunderstand you in last response? If you mean checkpoint in the scenario that the source and the split reader are in different operators, like Aljoscha's prototype. That's indeed a problem, so I think that's would not be the final version. Aljoscha also said in FLIP doc

[jira] [Created] (FLINK-11009) Add support for FILTER clause for non-windowed aggregation in Table API and SQL

2018-11-26 Thread Dian Fu (JIRA)
Dian Fu created FLINK-11009: --- Summary: Add support for FILTER clause for non-windowed aggregation in Table API and SQL Key: FLINK-11009 URL: https://issues.apache.org/jira/browse/FLINK-11009 Project: Flink

[jira] [Created] (FLINK-11008) Speed up upload checkpoint files using multi-thread

2018-11-26 Thread Congxian Qiu (JIRA)
Congxian Qiu created FLINK-11008: Summary: Speed up upload checkpoint files using multi-thread Key: FLINK-11008 URL: https://issues.apache.org/jira/browse/FLINK-11008 Project: Flink Issue

Re: [DISCUSS] Flink SQL DDL Design

2018-11-26 Thread Zhang, Xuefu
Hi Shuyi, I'm wondering if you folks still have the bandwidth working on this. We have some dedicated resource and like to move this forward. We can collaborate. Thanks, Xuefu -- 发件人:wenlong.lwl 日 期:2018年11月05日 11:15:35

回复:[DISCUSS]Enhancing flink scheduler by implementing blacklist mechanism

2018-11-26 Thread zhijiang
Thanks yingjie for bringing this discussion. I encountered this issue during failover and also noticed other users complainting related issues in community before. So it is necessary to have this mechanism for enhancing schedule process first, and then enrich the internal rules step by step.

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

2018-11-26 Thread Biao Liu
Hi Kostas, Regarding the checkpoint of "per thread for each split mode". IMO, there are severals things source operator need to do. 1. Source operator need to record all splits in checkpoint. The unfinished splits must be recorded. I'm not sure whether we could skip recording the finished splits,

Re: [DISCUSS] Table API Enhancement Outline

2018-11-26 Thread jincheng sun
Before we have a good support for nest-table, may be forcing the use of select is good way, at least not causing compatibility issues. Fabian Hueske 于2018年11月26日周一 下午6:48写道: > I think the question is what is the data type of 'w. > > Until now, I assumed it would be a nested tuple (Row or

[jira] [Created] (FLINK-11007) Update documentation to describe new checkpoint metadata file behavior

2018-11-26 Thread Josh Lemer (JIRA)
Josh Lemer created FLINK-11007: -- Summary: Update documentation to describe new checkpoint metadata file behavior Key: FLINK-11007 URL: https://issues.apache.org/jira/browse/FLINK-11007 Project: Flink

[DISCUSS] Flink backward compatibility

2018-11-26 Thread Thomas Weise
Hi, I wanted to bring back the topic of backward compatibility with respect to all/most of the user facing aspects of Flink. Please note that isn't limited to the programming API, but also includes job submission and management. As can be seen in [1], changes in these areas cause difficulties

[jira] [Created] (FLINK-11006) Update Calcite dependency to 1.18

2018-11-26 Thread Rong Rong (JIRA)
Rong Rong created FLINK-11006: - Summary: Update Calcite dependency to 1.18 Key: FLINK-11006 URL: https://issues.apache.org/jira/browse/FLINK-11006 Project: Flink Issue Type: Improvement

[jira] [Created] (FLINK-11005) Define flink-sql-client uber-jar dependencies via artifactSet

2018-11-26 Thread Till Rohrmann (JIRA)
Till Rohrmann created FLINK-11005: - Summary: Define flink-sql-client uber-jar dependencies via artifactSet Key: FLINK-11005 URL: https://issues.apache.org/jira/browse/FLINK-11005 Project: Flink

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

2018-11-26 Thread Piotr Nowojski
Hi Becket, > Is there any extra thing user can do on a MaterializedTable that they cannot > do on a Table? Maybe not in the initial implementation, but various DBs offer different ways to “refresh” the materialised view. Hooks, triggers, timers, manually etc. Having `MaterializedTable` would

Re: Creating a slide set for a Flink intro talk

2018-11-26 Thread Fabian Hueske
Hi, It's been a while since I started this thread. I've finished a first version of the slides and also used them to give a 40min talk [1]. The slides could be improved by adding some speaker notes. Some of the slides are maybe a bit text-heavy, but I think this is fine for slides that are used

[jira] [Created] (FLINK-11004) Wrong ProcessWindowFunction.process argument in example of Incremental Window Aggregation with ReduceFunction

2018-11-26 Thread Yuanyang Wu (JIRA)
Yuanyang Wu created FLINK-11004: --- Summary: Wrong ProcessWindowFunction.process argument in example of Incremental Window Aggregation with ReduceFunction Key: FLINK-11004 URL:

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

2018-11-26 Thread Kostas Kloudas
Hi all, >From the discussion, I understand that we are leaning towards a design where the user writes a single-threaded SplitReader, which Flink executes on another thread (not the main task thread). This way the task can have multiple readers running concurrently, each one reading a different

[jira] [Created] (FLINK-11003) Document of Java Lambda Expressions has a mistake

2018-11-26 Thread Yangze Guo (JIRA)
Yangze Guo created FLINK-11003: -- Summary: Document of Java Lambda Expressions has a mistake Key: FLINK-11003 URL: https://issues.apache.org/jira/browse/FLINK-11003 Project: Flink Issue Type:

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

2018-11-26 Thread Fabian Hueske
I'm not suggesting to add support for Ignite. This was just an example. Plasma and Arrow sound interesting, too. For the sake of this proposal, it would be up to the user to implement a TableFactory and corresponding TableSource / TableSink classes to persist and read the data. Am Mo., 26. Nov.

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

2018-11-26 Thread Flavio Pompermaier
What about to add also Apache Plasma + Arrow as an alternative to Apache Ignite? [1] https://arrow.apache.org/blog/2017/08/08/plasma-in-memory-object-store/ On Mon, Nov 26, 2018 at 11:56 AM Fabian Hueske wrote: > Hi, > > Thanks for the proposal! > > To summarize, you propose a new method

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

2018-11-26 Thread Fabian Hueske
Hi, Thanks for the proposal! To summarize, you propose a new method Table.cache(): Table that will trigger a job and write the result into some temporary storage as defined by a TableFactory. The cache() call blocks while the job is running and eventually returns a Table object that represents a

Re: [DISCUSS] Table API Enhancement Outline

2018-11-26 Thread Fabian Hueske
I think the question is what is the data type of 'w. Until now, I assumed it would be a nested tuple (Row or Tuple). Accessing nested fields in Row, Tuple or Pojo is done with get, i.e., 'w.get("rowtime"). Using a Map would not help because the access would be 'w.at("rowtime"). We can of course

Re: Apply for flink contributor permission

2018-11-26 Thread Yingjie Cao
Thanks Fabian. Fabian Hueske 于2018年11月26日周一 下午6:03写道: > Hi Yingjie, > > I've granted you contributor permissions. > > Best, Fabian > > Am Sa., 24. Nov. 2018 um 07:31 Uhr schrieb Yingjie Cao < > kevin.ying...@gmail.com>: > > > Hi guys, > > > > Could anyone kindly give me the contributor

Re: Apply for flink contributor permission

2018-11-26 Thread Fabian Hueske
Hi Yingjie, I've granted you contributor permissions. Best, Fabian Am Sa., 24. Nov. 2018 um 07:31 Uhr schrieb Yingjie Cao < kevin.ying...@gmail.com>: > Hi guys, > > Could anyone kindly give me the contributor permission? My JIRA ID is > kevin.cyj. > > Thanks, > Yingjie >