Re: Towards a spec for robust streaming SQL, Part 1

2017-05-12 Thread Shaoxuan Wang
Tyler, Yes, dynamic table changes over time. You can find more details about dynamic table from this Flink blog ( https://flink.apache.org/news/2017/04/04/dynamic-tables.html). Fabian, me and Xiaowei posted it a week before the flink-forward@SF. "A dynamic table is a table that is continuously

[jira] [Created] (FLINK-6574) External catalog should support a single level catalog

2017-05-12 Thread Haohui Mai (JIRA)
Haohui Mai created FLINK-6574: - Summary: External catalog should support a single level catalog Key: FLINK-6574 URL: https://issues.apache.org/jira/browse/FLINK-6574 Project: Flink Issue Type:

[jira] [Created] (FLINK-6573) Flink MongoDB Connector

2017-05-12 Thread Nagamallikarjuna (JIRA)
Nagamallikarjuna created FLINK-6573: --- Summary: Flink MongoDB Connector Key: FLINK-6573 URL: https://issues.apache.org/jira/browse/FLINK-6573 Project: Flink Issue Type: New Feature

Re: Towards a spec for robust streaming SQL, Part 1

2017-05-12 Thread Tyler Akidau
Being able to support an EMIT config independent of the query itself sounds great for compatible use cases (which should be many :-). Shaoxuan, can you please refresh my memory what a dynamic table means in Flink? It's basically just a state table, right? The "dynamic" part of the name is to

[jira] [Created] (FLINK-6572) Make Flink configuration properties available as env vars in Docker image

2017-05-12 Thread Joshua Griffith (JIRA)
Joshua Griffith created FLINK-6572: -- Summary: Make Flink configuration properties available as env vars in Docker image Key: FLINK-6572 URL: https://issues.apache.org/jira/browse/FLINK-6572 Project:

[jira] [Created] (FLINK-6571) InfiniteSource in SourceStreamOperatorTest should deal with InterruptedExceptions

2017-05-12 Thread Chesnay Schepler (JIRA)
Chesnay Schepler created FLINK-6571: --- Summary: InfiniteSource in SourceStreamOperatorTest should deal with InterruptedExceptions Key: FLINK-6571 URL: https://issues.apache.org/jira/browse/FLINK-6571

[jira] [Created] (FLINK-6570) QueryableStateClient constructor in documentation doesn't match actual signature

2017-05-12 Thread Robert Metzger (JIRA)
Robert Metzger created FLINK-6570: - Summary: QueryableStateClient constructor in documentation doesn't match actual signature Key: FLINK-6570 URL: https://issues.apache.org/jira/browse/FLINK-6570

Operator name

2017-05-12 Thread Flavio Pompermaier
Hi to all, in many of my Flink job it is helpful to give a name to operators in order to make JobManager UI simpler to read. Currently, to give a name to operators it is necessary to specify it of the operator everytime it is used, for example: - env.readAsCsv().map().name("My map function")

[jira] [Created] (FLINK-6569) flink-table KafkaJsonTableSource example doesn't work

2017-05-12 Thread Robert Metzger (JIRA)
Robert Metzger created FLINK-6569: - Summary: flink-table KafkaJsonTableSource example doesn't work Key: FLINK-6569 URL: https://issues.apache.org/jira/browse/FLINK-6569 Project: Flink Issue

Re: [DISCUSS] Release 1.3.0 RC0 (Non voting, testing release candidate)

2017-05-12 Thread Robert Metzger
I'm considering to do another non voting RC1 on Monday with the incremental checkpointing and maven fixes in (and of course as many other things as possible). Once the other critical fixes are in (ideally Tuesday or Wednesday), I'll create the first voting one. The current RC0 is almost unusable

[jira] [Created] (FLINK-6568) flink-table doesn't work without flink-streaming-scala dependency

2017-05-12 Thread Robert Metzger (JIRA)
Robert Metzger created FLINK-6568: - Summary: flink-table doesn't work without flink-streaming-scala dependency Key: FLINK-6568 URL: https://issues.apache.org/jira/browse/FLINK-6568 Project: Flink

[jira] [Created] (FLINK-6567) ExecutionGraphMetricsTest fails on Windows CI

2017-05-12 Thread Chesnay Schepler (JIRA)
Chesnay Schepler created FLINK-6567: --- Summary: ExecutionGraphMetricsTest fails on Windows CI Key: FLINK-6567 URL: https://issues.apache.org/jira/browse/FLINK-6567 Project: Flink Issue

Re: [DISCUSS] Release 1.3.0 RC0 (Non voting, testing release candidate)

2017-05-12 Thread Greg Hogan
+1 for sticking to the code freeze deadline and building a new release candidate but since the release is still two weeks off (5/26) I think it better to delay voting to give time for additional bug fixes. > On May 11, 2017, at 10:19 AM, Robert Metzger wrote: > > It

RE: [DISCUSS] Expose State Backend Interface for UDAGG

2017-05-12 Thread Radu Tudoran
Hi, In general I believe it is a good idea to expose the state backend to the functions. You can always optimize the data processing based on the data storage. Hence, as the level of the processing (aggregation here) you would be able to control the access to data, you can implement this in a

Re: waiting on assignment for FLINK-1750

2017-05-12 Thread Alex De Castro
Thanks! Alex On 12 May 2017 at 12:34, Fabian Hueske wrote: > Done! :-) > > 2017-05-12 11:48 GMT+02:00 Alex De Castro : > > > Thanks! > > >

Re: waiting on assignment for FLINK-1750

2017-05-12 Thread Fabian Hueske
Done! :-) 2017-05-12 11:48 GMT+02:00 Alex De Castro : > Thanks! >

Re: git overwritten by checkout

2017-05-12 Thread Fabian Hueske
Sorry, that was me :-/ Thanks for sharing how you resolved the issue. 2017-05-12 12:38 GMT+02:00 Greg Hogan : > The following file in the following commit throw the following error when > rebasing to master. I agree that case insensitive filesystems are an > abomination and

Re: [DISCUSS] Expose State Backend Interface for UDAGG

2017-05-12 Thread Shaoxuan Wang
Fabian, Thanks for your quick reply. The goal of "FLINK-6544" is not to expose state backend in all UDAGG cases. It is designed to provide an interface which provides an ability for user to access state backend when it is allowed (yes, right now this is only allowed by ProcessFunction). This

git overwritten by checkout

2017-05-12 Thread Greg Hogan
The following file in the following commit throw the following error when rebasing to master. I agree that case insensitive filesystems are an abomination and hopefully never supported by APFS but just wanted to note the situation. With a clean working directory I solved this with `git fetch

Re: [DISCUSS] Expose State Backend Interface for UDAGG

2017-05-12 Thread Fabian Hueske
Hi, thanks for the proposal. I think exposing state to UDAGGs would be very difficult and a lot of work. UDAGGs are called from ProcessFunctions (stream, OVER window and non-window aggs), AggregateFunctions (stream, group-window aggs), CombineFunctions (batch) and GroupReduceFunctions (batch).

[DISCUSS] Expose State Backend Interface for UDAGG

2017-05-12 Thread Shaoxuan Wang
Hi everyone, We made some progress in the implementation of UDAGG (FLINK-5564). However, we realized that there are cases where users may want to use state backend to store the data. For instance, the built-in MaxWithRetractAggFunction currently create a hashMap to store the historical data. It

waiting on assignment for FLINK-1750

2017-05-12 Thread Alex De Castro
Thanks!

Re: [DISCUSS] Release 1.3.0 RC0 (Non voting, testing release candidate)

2017-05-12 Thread Till Rohrmann
I would like to get https://issues.apache.org/jira/browse/FLINK-6020 also in the 1.3.0 release, because multiple job submissions to the same cluster can corrupt the stored blobs. Cheers, Till On Fri, May 12, 2017 at 9:52 AM, Fabian Hueske wrote: > Hi Haohui, > > open issue

Re: Towards a spec for robust streaming SQL, Part 1

2017-05-12 Thread Shaoxuan Wang
Thanks to Tyler and Fabian for sharing your thoughts. Regarding to the early/late update control of FLINK. IMO, each dynamic table can have an EMIT config. For FLINK table-API, this can be easily implemented in different manners, case by case. For instance, in window aggregate, we could define

Re: Towards a spec for robust streaming SQL, Part 1

2017-05-12 Thread Fabian Hueske
2017-05-11 7:14 GMT+02:00 Tyler Akidau : > On Tue, May 9, 2017 at 3:06 PM Fabian Hueske wrote: > > > Hi Tyler, > > > > thank you very much for this excellent write-up and the super nice > > visualizations! > > You are discussing a lot of the things

[jira] [Created] (FLINK-6566) Narrow down interface for compatibility hook method in VersionedIOReadableWritable

2017-05-12 Thread Tzu-Li (Gordon) Tai (JIRA)
Tzu-Li (Gordon) Tai created FLINK-6566: -- Summary: Narrow down interface for compatibility hook method in VersionedIOReadableWritable Key: FLINK-6566 URL: https://issues.apache.org/jira/browse/FLINK-6566

[jira] [Created] (FLINK-6565) Improve error messages for state restore failures

2017-05-12 Thread Tzu-Li (Gordon) Tai (JIRA)
Tzu-Li (Gordon) Tai created FLINK-6565: -- Summary: Improve error messages for state restore failures Key: FLINK-6565 URL: https://issues.apache.org/jira/browse/FLINK-6565 Project: Flink

Re: [DISCUSS] Release 1.3.0 RC0 (Non voting, testing release candidate)

2017-05-12 Thread Fabian Hueske
Hi Haohui, open issue for 1.3 are tracked in JIRA. You can specify "Fix version: 1.3.0" and an appropriate priority. I updated a couple of issues to have the fix version tag. Any bug that is found while release testing should be added to this list by setting fix version, priority, and component.