Re: [DISCUSS] Development of SQL OVER / Table API Row Windows for streaming tables

2017-01-24 Thread Shaoxuan Wang
Hi Hongyu, Nice to virtually meet you here and thanks for asking. You raised a good question. Your question is more about how to refine a result, "retraction". This is a another big building block for streaming we would like to address in the near future. To the best of my understanding, in the cur

Re: [DISCUSS] Development of SQL OVER / Table API Row Windows for streaming tables

2017-01-24 Thread Hongyuhong
Hi shaoxuan, Is that means there will be only one sorted state to save records, and an accumulator to save the reduce value per group. Once an record arrive, it will first clean the out-time records and restract the reduce value and then accumulate the new record? Thus is clearly with processing

[jira] [Created] (FLINK-5635) Improve Docker tooling to make it easier to build images and launch Flink via Docker tools

2017-01-24 Thread Jamie Grier (JIRA)
Jamie Grier created FLINK-5635: -- Summary: Improve Docker tooling to make it easier to build images and launch Flink via Docker tools Key: FLINK-5635 URL: https://issues.apache.org/jira/browse/FLINK-5635

[jira] [Created] (FLINK-5634) Flink should not always redirect stdout to a file.

2017-01-24 Thread Jamie Grier (JIRA)
Jamie Grier created FLINK-5634: -- Summary: Flink should not always redirect stdout to a file. Key: FLINK-5634 URL: https://issues.apache.org/jira/browse/FLINK-5634 Project: Flink Issue Type: Bug

[jira] [Created] (FLINK-5633) ClassCastException: X cannot be cast to X when re-submitting a job.

2017-01-24 Thread Giuliano Caliari (JIRA)
Giuliano Caliari created FLINK-5633: --- Summary: ClassCastException: X cannot be cast to X when re-submitting a job. Key: FLINK-5633 URL: https://issues.apache.org/jira/browse/FLINK-5633 Project: Flin

Re: [DISCUSS] Development of SQL OVER / Table API Row Windows for streaming tables

2017-01-24 Thread Shaoxuan Wang
Hi everyone, Thanks for this great discussion, and glad to see more and more people are interested on stream SQL & tableAPI. IMO, the key problems for Over window design are the SQL semantics and the runtime design. I totally agree with Fabian that we should skip the design of TumbleRows and Sess

Re: [Dev] Flink 'InputFormat' Interface execution related problem

2017-01-24 Thread Pawan Manishka Gunarathna
Hi, Thanks a lot for Fabian and Flavio.Those information really helpful. On Tue, Jan 24, 2017 at 3:36 PM, Flavio Pompermaier wrote: > If your column on which you want to perform the split is numeric you can > use the NumericBetweenParametersProvider interface that automatically > computes th

[jira] [Created] (FLINK-5632) Typo in StreamGraph

2017-01-24 Thread Wei-Che Wei (JIRA)
Wei-Che Wei created FLINK-5632: -- Summary: Typo in StreamGraph Key: FLINK-5632 URL: https://issues.apache.org/jira/browse/FLINK-5632 Project: Flink Issue Type: Improvement Components: J

[jira] [Created] (FLINK-5631) [yarn] Support downloading additional jars from non-HDFS paths

2017-01-24 Thread Haohui Mai (JIRA)
Haohui Mai created FLINK-5631: - Summary: [yarn] Support downloading additional jars from non-HDFS paths Key: FLINK-5631 URL: https://issues.apache.org/jira/browse/FLINK-5631 Project: Flink Issue

[jira] [Created] (FLINK-5630) Followups to AggregationFunction

2017-01-24 Thread Stephan Ewen (JIRA)
Stephan Ewen created FLINK-5630: --- Summary: Followups to AggregationFunction Key: FLINK-5630 URL: https://issues.apache.org/jira/browse/FLINK-5630 Project: Flink Issue Type: Improvement

[Proposal] RichFsSinkFunction

2017-01-24 Thread Seth Wiesman
Hi all, I have been working with Flink for a while at work now and in that time I have developed several extensions that I would like to contribute back. I wanted to reach out with what has been the most significant modification for and see if it is something that the community would be interes

RE: [DISCUSS] Development of SQL OVER / Table API Row Windows for streaming tables

2017-01-24 Thread Radu Tudoran
Hi Fabian, Thanks for the feedback and for clarifications with respect to the new JIRA issues. How should we proceed from here? Will you start creating this or can we also take the design of some of these issues? I am particular interested in - streaming SQL OVER RANGE for processing time if yo

Re: clean up jira

2017-01-24 Thread Chesnay Schepler
fixed link: https://issues.apache.org/jira/issues/?jql=fixVersion%20%3D%201.2.0%20AND%20project%20%3D%20FLINK%20AND%20resolution%20%3D%20Unresolved%20ORDER%20BY%20priority%20DESC Previous link lead to FLINK-5048. On 24.01.2017 16:33, Robert Metzger wrote: Hey friends of a clean JIRA, I've unm

Re: clean up jira

2017-01-24 Thread Robert Metzger
Hey friends of a clean JIRA, I've unmarked as many JIRAs as possible from "Fix Version = 1.2.0" as possible. The list of unresolved 1.2.0 issues is now 15 items short: https://issues.apache.org/jira/browse/FLINK-5048?jql=fixVersion%20%3D%201.2.0%20AND%20project%20%3D%20FLINK%20AND%20resolution%20%

Re: [DISCUSS] Development of SQL OVER / Table API Row Windows for streaming tables

2017-01-24 Thread Fabian Hueske
Hi Radu, thanks for your comments! Yes, my intention is to open new JIRA issues to structure the development process. Everybody is very welcome to pick up issues and discuss the design proposals. At the moment I see the following six issues to start with: - streaming SQL OVER ROW for processing

Re: [DISCUSS] Development of SQL OVER / Table API Row Windows for streaming tables

2017-01-24 Thread Fabian Hueske
Hi SunJincheng, thanks a lot for your comments! regarding the suitability of DataStream sliding windows: You are right that UNBOUNDED PRECEDING windows cannot be implemented as DataStream sliding windows. The same is true for OVER RANGE windows. I think the only OVER windows that could be done wi

Re: [DISCUSS] Development of SQL OVER / Table API Row Windows for streaming tables

2017-01-24 Thread Fabian Hueske
Hi Jark, let me answer your questions: Q1: Yes, that's what I am currently thinking about. Q2: You can interpret a session id as a partitioning attribute. If you have OVER (PARTITION BY a, SessionWithGap(rowtime)), "a" would be a regular partitioning attribute and "SessionWithGap(rowtime)" would

[VOTE] Release Apache Flink 1.2.0 (RC1)

2017-01-24 Thread Robert Metzger
Dear Flink community, Please vote on releasing the following candidate as Apache Flink version 1.2.0. The commit to be voted on: 1bca1208 (*http://git-wip-us.apache.org/repos/asf/flink/commit/1bca1208 *) Branch: release-1.2.0-rc1 (htt

Re: Unregistering Managed State in Operator Backend

2017-01-24 Thread Paris Carbone
Indeed, I noticed that now. Then it should be fairly simple, if you find it reasonable too. > On 24 Jan 2017, at 14:20, Aljoscha Krettek wrote: > > Just a bit of clarification, the OperatorState stuff is independent of > keyed state backends, i.e. even if you use RocksDB the operator state will

Re: Unregistering Managed State in Operator Backend

2017-01-24 Thread Aljoscha Krettek
Just a bit of clarification, the OperatorState stuff is independent of keyed state backends, i.e. even if you use RocksDB the operator state will not be stored in RocksDB, only keyed state is stored there. Right now, when an operator state (ListState) is empty we will still write some meta data ab

Re: [DISCUSS] (Not) tagging reviewers

2017-01-24 Thread Till Rohrmann
I agree with Aljoscha that tagging the PRs helps me to get notified about PRs where I could be of help. But I also think that it should not be the ultimate responsibility of the tagged person(s) to review the code. Everyone should feel empowered to do so. And also the tagging does not free oneself

Re: Need help on understanding flink runtime and window function

2017-01-24 Thread Aljoscha Krettek
Hi,depending on which version of Flink you're using the answer changes. If you use Flink 1.1 AggregatingProcessingTimeWindowOperator should be responsible for executing that. In Flink 1.2 it should be WindowOperator. For a quick overview of how scheduling works in Flink you could look at this: htt

Re: [DISCUSS] (Not) tagging reviewers

2017-01-24 Thread Theodore Vasiloudis
I was wondering how this relates to the shepherding of PRs we have discussed in the past. If I make a PR for an issue reported from a specific committer, doesn't tagging them make sense? Has the shepherding of PRs been tried out? On Tue, Jan 24, 2017 at 12:17 PM, Aljoscha Krettek wrote: > It se

Re: [DISCUSS] (Not) tagging reviewers

2017-01-24 Thread Aljoscha Krettek
It seems I'm in a bit of a minority here but I like the @R tags. There are simply to many pull request for someone to keep track of all of them and if someone things that a certain person would be good for reviewing a change then tagging them helps them notice the PR. I think the tag should not me

Re: 答复: [DISCUSS] Apache Flink 1.2.0 RC0 (Non-voting testing release candidate)

2017-01-24 Thread Robert Metzger
RC1 creation is in progress ... On Mon, Jan 23, 2017 at 10:33 AM, Robert Metzger wrote: > Hi all, > > I would like to do a proper voting RC1 early this week. > From the issues mentioned here, most of them have pull requests or were > changed to a lower priority. > Once we've merged all outstandi

Re: Unregistering Managed State in Operator Backend

2017-01-24 Thread Paris Carbone
Sure Till, I would love to also make the patch but need to prioritize some other things these days. At least I will dig and see how complex this is regarding the different backends. I also have some follow-up questions, in case anybody has thought about these things already (or is simply int

Re: Unregistering Managed State in Operator Backend

2017-01-24 Thread Paris Carbone
Sure Till, I would love to also make the patch but need to prioritize some other things these days. At least I will dig and see how complex this is regarding the different backends. I also have some follow-up questions, in case anybody has thought about these things already (or is simply interest

Re: Unregistering Managed State in Operator Backend

2017-01-24 Thread Till Rohrmann
Hi Paris, if there is no such issue open, then please open one so that we can track the issue. If you have time to work on that even better :-) Cheers, Till On Tue, Jan 24, 2017 at 10:25 AM, Paris Carbone wrote: > Any thoughts/plans? > So should I open a Jira and add this? > > Paris > > On Jan

Re: [Dev] Flink 'InputFormat' Interface execution related problem

2017-01-24 Thread Flavio Pompermaier
If your column on which you want to perform the split is numeric you can use the NumericBetweenParametersProvider interface that automatically computes the splits for you. This is an example of its usage (at windows of 1000 items at a time) taken from the test class *JDBCInputFormatTest*: final in

Re: [Dev] Flink 'InputFormat' Interface execution related problem

2017-01-24 Thread Fabian Hueske
Hi, JdbcInputFormat implements the InputFormat interface and is handled exactly like any other InputFormat. In contrast to file-based input formats, users must explicitly specify the input splits by providing an array of parameter values which are injected into a parameterized query. This is done

RE: [DISCUSS] Development of SQL OVER / Table API Row Windows for streaming tables

2017-01-24 Thread Radu Tudoran
Hi all, Thanks for starting these discussion - it is very useful. It does make sense indeed to refactor all these and coordinate a bit the efforts not to have overlapping implementations and incompatible solutions. If you close the 3 jira issues you mentioned - do you plan to redesign them and

Re: Unregistering Managed State in Operator Backend

2017-01-24 Thread Paris Carbone
Any thoughts/plans? So should I open a Jira and add this? Paris On Jan 21, 2017, at 5:17 PM, Paris Carbone mailto:par...@kth.se>> wrote: Thank you for the answer Ufuk! To elaborate a bit more, I am not using keyed state, it would be indeed tricky in that case to discard everything. I need th