Re: Status of Flink-Calcite integration

2016-12-20 Thread Fabian Hueske
Hi Haohui, the Flink community started about a year ago to port its Table API on top of Apache Calcite and finalized the integration in June. Now, Calcite is the central component of Flink's relational APIs. Flink features SQL and the Table API, a language-integrated query (LINQ) API. Both APIs ar

Re: [VOTE] Release Apache Flink 1.1.4 (RC4)

2016-12-20 Thread Gyula Fóra
I have built and ran the tests for the latest RC, so I carry my +1 vote from the previous thread Gyula Ufuk Celebi ezt írta (időpont: 2016. dec. 19., H, 16:16): > Dear Flink community, > > Please vote on releasing the following candidate as Apache Flink version > 1.1.4. > > The commit to be vot

Re: [DISCUSS] Add Side Input/Broadcast Set For Streaming API

2016-12-20 Thread Gábor Gévay
Hello, I am also interested in this feature for a paper that I'm writing. I have the "slowly evolving side input" case with a complicated custom "update precondition" that would be expressible by a stateful UDF that makes its decisions from looking at the elements of the main stream. Best, Gábor

InputGate consumedSubpartitionIndex in ALL_TO_ALL distribution pattern

2016-12-20 Thread Luis Alves
Hi! Can someone confirm that the following statement is true: If we have a task consuming from two other tasks (both using ALL_TO_ALL distribution pattern). Then, this task will have two InputGates consuming from the same index (consumedSubpartitionIndex). This means that each task would always

[jira] [Created] (FLINK-5371) Add documentation for async I/O

2016-12-20 Thread Till Rohrmann (JIRA)
Till Rohrmann created FLINK-5371: Summary: Add documentation for async I/O Key: FLINK-5371 URL: https://issues.apache.org/jira/browse/FLINK-5371 Project: Flink Issue Type: Improvement

Re: InputGate consumedSubpartitionIndex in ALL_TO_ALL distribution pattern

2016-12-20 Thread Stephan Ewen
Yes, that is true. On Tue, Dec 20, 2016 at 12:46 PM, Luis Alves wrote: > Hi! > > Can someone confirm that the following statement is true: If we have a > task consuming from two other tasks (both using ALL_TO_ALL distribution > pattern). Then, this task will have two InputGates consuming from th

Flink gives incorrect result when event time windowing used

2016-12-20 Thread Jaromir Vanek
Hi, I am using Flink 1.1.3 and following example doesn't work for me as expected. I've got three input elements with similar timestamp (equaling to window maxTimestamp). I'm using /event time/ notion of time with /TumblingEventTimeWindows/. I would expect all three elements to be processed in th

Re: [DISCUSS] Schedule and Scope for Flink 1.2

2016-12-20 Thread Robert Metzger
Quick update here: I talked to Aljoscha offline, and the backwards compatibility is still being tested (there were some bugs identified while writing the tests). Also, Stephan made some fixes to the build infrastructure ( https://github.com/apache/flink/pull/3029) that would be good to be included

Re: Flink gives incorrect result when event time windowing used

2016-12-20 Thread Fabian Hueske
Hi Jaromir, thank you very much for reporting this issue. The behavior you are describing is not in line with the documentation of watermarks [1] which clearly states that a watermark of time t tells the system that no more events with a timestamp < t will occur (otherwise they would be considered

Re: [DISCUSS] Schedule and Scope for Flink 1.2

2016-12-20 Thread Aljoscha Krettek
I just merged the most important backwards compatibility changes, with tests. I think this one is still a blocker: https://issues.apache.org/jira/browse/FLINK-5320 (WindowedStream.fold() cannot be used). And this one is a potential blocker for some users: https://issues.apache.org/jira/browse/FLIN

Re: [DISCUSS] Schedule and Scope for Flink 1.2

2016-12-20 Thread Robert Metzger
Hi Aljoscha, thanks a lot for resolving this last big release blocker! I'll definitively fork off in the release branch in the next few hours. Stephan asked me to wait for a few more minutes for his maven changes. The JIRAs you've mentioned seem to be pretty isolated, so its fine to merge them af

[jira] [Created] (FLINK-5372) Fix RocksDBAsyncSnapshotTest.testCancelFullyAsyncCheckpoints()

2016-12-20 Thread Aljoscha Krettek (JIRA)
Aljoscha Krettek created FLINK-5372: --- Summary: Fix RocksDBAsyncSnapshotTest.testCancelFullyAsyncCheckpoints() Key: FLINK-5372 URL: https://issues.apache.org/jira/browse/FLINK-5372 Project: Flink

Re: Flink gives incorrect result when event time windowing used

2016-12-20 Thread Aljoscha Krettek
I'm afraid the doc is wrong here. The JavaDoc on Watermark says this about watermarks: "A Watermark tells operators that receive it that no elements with a timestamp older or equal to the watermark timestamp should arrive at the operator." The system also relies on this fact, as visible in how ti

Re: [DISCUSS] Schedule and Scope for Flink 1.2

2016-12-20 Thread Robert Metzger
Okay, I forked off a release-1.2 branch and updated the version in master to 1.3-SNAPSHOT. I'll create the first RC (non voting) in the next few days. On Tue, Dec 20, 2016 at 4:39 PM, Robert Metzger wrote: > Hi Aljoscha, > thanks a lot for resolving this last big release blocker! > > I'll defini

[jira] [Created] (FLINK-5373) Extend Unit Tests for StateAssignmentOperation

2016-12-20 Thread Aljoscha Krettek (JIRA)
Aljoscha Krettek created FLINK-5373: --- Summary: Extend Unit Tests for StateAssignmentOperation Key: FLINK-5373 URL: https://issues.apache.org/jira/browse/FLINK-5373 Project: Flink Issue Type

[jira] [Created] (FLINK-5374) Extend Unit Tests for RegisteredBackendStateMetaInfo

2016-12-20 Thread Aljoscha Krettek (JIRA)
Aljoscha Krettek created FLINK-5374: --- Summary: Extend Unit Tests for RegisteredBackendStateMetaInfo Key: FLINK-5374 URL: https://issues.apache.org/jira/browse/FLINK-5374 Project: Flink Issu

[jira] [Created] (FLINK-5375) Fix watermark documentation

2016-12-20 Thread Fabian Hueske (JIRA)
Fabian Hueske created FLINK-5375: Summary: Fix watermark documentation Key: FLINK-5375 URL: https://issues.apache.org/jira/browse/FLINK-5375 Project: Flink Issue Type: Bug Component

Re: Flink gives incorrect result when event time windowing used

2016-12-20 Thread Fabian Hueske
Thanks for the clarification Aljoscha. I added https://issues.apache.org/jira/browse/FLINK-5375 to fix this issue. Best, Fabian 2016-12-20 17:58 GMT+01:00 Aljoscha Krettek : > I'm afraid the doc is wrong here. The JavaDoc on Watermark says this about > watermarks: > > "A Watermark tells operato

[jira] [Created] (FLINK-5376) Misleading log statements in UnorderedStreamElementQueue

2016-12-20 Thread Ted Yu (JIRA)
Ted Yu created FLINK-5376: - Summary: Misleading log statements in UnorderedStreamElementQueue Key: FLINK-5376 URL: https://issues.apache.org/jira/browse/FLINK-5376 Project: Flink Issue Type: Bug

Re: Status of Flink-Calcite integration

2016-12-20 Thread Haohui Mai
Hi Fabian, Thanks for the reply. The design document is quite interesting down the road. For our business needs we are interested in adding supports like windowing and joins in the streaming SQL part. The question is that how does it fit in the long-term road map? I quickly skimmed through the co

Re: [DISCUSS] Schedule and Scope for Flink 1.2

2016-12-20 Thread Stephan Ewen
Very nice, thanks Robert! On Tue, Dec 20, 2016 at 6:02 PM, Robert Metzger wrote: > Okay, I forked off a release-1.2 branch and updated the version in master > to 1.3-SNAPSHOT. > I'll create the first RC (non voting) in the next few days. > > On Tue, Dec 20, 2016 at 4:39 PM, Robert Metzger > wro

Re: [VOTE] Release Apache Flink 1.1.4 (RC4)

2016-12-20 Thread Stephan Ewen
+1 from my side - No relevant changes to dependencies (other than minor version bumps) - No updates to LICENSE and NOTICE necessary - Ran a local build on Linux with Hadoop 2.4 and Scala 2.10, all tests pass - Started standalone cluster and ran several examples - log output looks good On