[jira] [Commented] (FLINK-2941) Implement a neo4j - Flink/Gelly connector

2016-03-03 Thread Martin Junghanns (JIRA)
[ https://issues.apache.org/jira/browse/FLINK-2941?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15179492#comment-15179492 ] Martin Junghanns commented on FLINK-2941: - Hi, thank you guys for the input! Forki

[GitHub] flink pull request: FLINK-3179 Combiner is not injected if Reduce ...

2016-03-03 Thread ramkrish86
Github user ramkrish86 commented on the pull request: https://github.com/apache/flink/pull/1553#issuecomment-192131350 New PR submitted @fhueske . Thanks for helping me thro this code review. It is was more of a beginner and there is a lot to learn from my side. --- If your project

[jira] [Commented] (FLINK-3179) Combiner is not injected if Reduce or GroupReduce input is explicitly partitioned

2016-03-03 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/FLINK-3179?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15179424#comment-15179424 ] ASF GitHub Bot commented on FLINK-3179: --- Github user ramkrish86 commented on the pul

[jira] [Created] (FLINK-3576) Upgrade Snappy Java to 1.1.2.1

2016-03-03 Thread Ted Yu (JIRA)
Ted Yu created FLINK-3576: - Summary: Upgrade Snappy Java to 1.1.2.1 Key: FLINK-3576 URL: https://issues.apache.org/jira/browse/FLINK-3576 Project: Flink Issue Type: Improvement Reporter:

[GitHub] flink pull request: [docs] fix javascript exception caused by disq...

2016-03-03 Thread StephanEwen
Github user StephanEwen commented on the pull request: https://github.com/apache/flink/pull/1756#issuecomment-191915950 This looks like a good fix, thanks for contributing! +1 to merge this --- If your project is set up for it, you can reply to this email and have your reply

[jira] [Commented] (FLINK-3566) Input type validation often fails on custom TypeInfo implementations

2016-03-03 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/FLINK-3566?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15178113#comment-15178113 ] ASF GitHub Bot commented on FLINK-3566: --- Github user chobeat commented on the pull r

[GitHub] flink pull request: [FLINK-3566] [FLINK-3563] Input type validatio...

2016-03-03 Thread chobeat
Github user chobeat commented on the pull request: https://github.com/apache/flink/pull/1759#issuecomment-191857083 I tested it too and it works perfectly both on an example case and in the code where the problem arose. --- If your project is set up for it, you can reply to this emai

[jira] [Commented] (FLINK-3569) Test cases fail due to Maven Shade plugin

2016-03-03 Thread Maximilian Michels (JIRA)
[ https://issues.apache.org/jira/browse/FLINK-3569?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15178079#comment-15178079 ] Maximilian Michels commented on FLINK-3569: --- I'll open a new pull request once I

[jira] [Updated] (FLINK-3380) Unstable Test: JobSubmissionFailsITCase

2016-03-03 Thread Chesnay Schepler (JIRA)
[ https://issues.apache.org/jira/browse/FLINK-3380?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Chesnay Schepler updated FLINK-3380: Labels: test-stability (was: ) > Unstable Test: JobSubmissionFailsITCase >

[jira] [Commented] (FLINK-3569) Test cases fail due to Maven Shade plugin

2016-03-03 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/FLINK-3569?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15178070#comment-15178070 ] ASF GitHub Bot commented on FLINK-3569: --- Github user mxm commented on the pull reque

[GitHub] flink pull request: [FLINK-3569] set Maven Surefire working direct...

2016-03-03 Thread mxm
Github user mxm closed the pull request at: https://github.com/apache/flink/pull/1757 --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled

[jira] [Commented] (FLINK-3569) Test cases fail due to Maven Shade plugin

2016-03-03 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/FLINK-3569?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15178071#comment-15178071 ] ASF GitHub Bot commented on FLINK-3569: --- Github user mxm closed the pull request at:

[GitHub] flink pull request: [FLINK-3569] set Maven Surefire working direct...

2016-03-03 Thread mxm
Github user mxm commented on the pull request: https://github.com/apache/flink/pull/1757#issuecomment-191842883 Setting the Surefire working directory is not possible because the Shade plugin changes the basedir beforehand. We will have to change the test cases for now if we don't wan

[jira] [Closed] (FLINK-2369) On Windows, in testFailingSortingDataSinkTask the temp file is not removed

2016-03-03 Thread Chesnay Schepler (JIRA)
[ https://issues.apache.org/jira/browse/FLINK-2369?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Chesnay Schepler closed FLINK-2369. --- Resolution: Fixed Fix Version/s: 1.0.0 > On Windows, in testFailingSortingDataSinkTask

[jira] [Closed] (FLINK-2757) DataSinkTaskTest fails on Windows

2016-03-03 Thread Chesnay Schepler (JIRA)
[ https://issues.apache.org/jira/browse/FLINK-2757?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Chesnay Schepler closed FLINK-2757. --- Resolution: Fixed Fixed via ff4fbf9bfb5107a6f4d0cbd040f36d583065aa4a > DataSinkTaskTest fails

[jira] [Assigned] (FLINK-3495) RocksDB Tests can't run on Windows

2016-03-03 Thread Chesnay Schepler (JIRA)
[ https://issues.apache.org/jira/browse/FLINK-3495?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Chesnay Schepler reassigned FLINK-3495: --- Assignee: Chesnay Schepler > RocksDB Tests can't run on Windows > ---

[jira] [Updated] (FLINK-3507) PruneEmptyRules does not prune empty node as expected.

2016-03-03 Thread Chesnay Schepler (JIRA)
[ https://issues.apache.org/jira/browse/FLINK-3507?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Chesnay Schepler updated FLINK-3507: Component/s: Table API > PruneEmptyRules does not prune empty node as expected. > -

[jira] [Resolved] (FLINK-3484) Add setSlotSharingGroup documentation

2016-03-03 Thread Aljoscha Krettek (JIRA)
[ https://issues.apache.org/jira/browse/FLINK-3484?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Aljoscha Krettek resolved FLINK-3484. - Resolution: Fixed Added here: https://github.com/apache/flink/commit/d18224befcec9c6a183

[jira] [Updated] (FLINK-3574) Implement math functions for Table API

2016-03-03 Thread Timo Walther (JIRA)
[ https://issues.apache.org/jira/browse/FLINK-3574?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Timo Walther updated FLINK-3574: Assignee: Dawid Wysakowicz > Implement math functions for Table API > --

[jira] [Resolved] (FLINK-3403) Create Section "Working with Time" in Streaming Guide

2016-03-03 Thread Aljoscha Krettek (JIRA)
[ https://issues.apache.org/jira/browse/FLINK-3403?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Aljoscha Krettek resolved FLINK-3403. - Resolution: Fixed Added > Create Section "Working with Time" in Streaming Guide > ---

[jira] [Commented] (FLINK-3575) Update Working With State Section in Doc

2016-03-03 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/FLINK-3575?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15178001#comment-15178001 ] ASF GitHub Bot commented on FLINK-3575: --- GitHub user aljoscha opened a pull request:

[GitHub] flink pull request: [FLINK-3575] Update Working With State Section...

2016-03-03 Thread aljoscha
GitHub user aljoscha opened a pull request: https://github.com/apache/flink/pull/1760 [FLINK-3575] Update Working With State Section in Doc You can merge this pull request into a Git repository by running: $ git pull https://github.com/aljoscha/flink doc-update-state Alternat

[jira] [Commented] (FLINK-3566) Input type validation often fails on custom TypeInfo implementations

2016-03-03 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/FLINK-3566?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15177982#comment-15177982 ] ASF GitHub Bot commented on FLINK-3566: --- Github user gyfora commented on the pull re

[jira] [Created] (FLINK-3575) Update Working With State Section in Doc

2016-03-03 Thread Aljoscha Krettek (JIRA)
Aljoscha Krettek created FLINK-3575: --- Summary: Update Working With State Section in Doc Key: FLINK-3575 URL: https://issues.apache.org/jira/browse/FLINK-3575 Project: Flink Issue Type: Impr

[GitHub] flink pull request: [FLINK-3566] [FLINK-3563] Input type validatio...

2016-03-03 Thread gyfora
Github user gyfora commented on the pull request: https://github.com/apache/flink/pull/1759#issuecomment-191821834 I tested it in my application where I was previously having both issues. Now it works perfectly, thanks! +1 from me :+1: --- If your project is set up

[jira] [Commented] (FLINK-3432) ZookeeperOffsetHandlerTest fails on windows

2016-03-03 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/FLINK-3432?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15177952#comment-15177952 ] ASF GitHub Bot commented on FLINK-3432: --- Github user tillrohrmann commented on the p

[jira] [Commented] (FLINK-3432) ZookeeperOffsetHandlerTest fails on windows

2016-03-03 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/FLINK-3432?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15177957#comment-15177957 ] ASF GitHub Bot commented on FLINK-3432: --- Github user asfgit closed the pull request

[jira] [Resolved] (FLINK-3432) ZookeeperOffsetHandlerTest fails on windows

2016-03-03 Thread Till Rohrmann (JIRA)
[ https://issues.apache.org/jira/browse/FLINK-3432?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Till Rohrmann resolved FLINK-3432. -- Resolution: Fixed Fixed via a52f9d8628b03cc6c223dfda33ae93d2fe16c27c > ZookeeperOffsetHandlerTe

[GitHub] flink pull request: [FLINK-3432] Fixed ZookeeperOffsethandlerTest ...

2016-03-03 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/flink/pull/1664 --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enab

[GitHub] flink pull request: [FLINK-3432] Fixed ZookeeperOffsethandlerTest ...

2016-03-03 Thread tillrohrmann
Github user tillrohrmann commented on the pull request: https://github.com/apache/flink/pull/1664#issuecomment-191814477 Will merge it. Thanks for your contribution @zentol :-) --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as we

[jira] [Commented] (FLINK-3152) Support all comparisons for Date type

2016-03-03 Thread Dawid Wysakowicz (JIRA)
[ https://issues.apache.org/jira/browse/FLINK-3152?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15177953#comment-15177953 ] Dawid Wysakowicz commented on FLINK-3152: - Great, thanks a lot. I will take it. >

[jira] [Assigned] (FLINK-3573) Implement more String functions for Table API

2016-03-03 Thread Timo Walther (JIRA)
[ https://issues.apache.org/jira/browse/FLINK-3573?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Timo Walther reassigned FLINK-3573: --- Assignee: Timo Walther > Implement more String functions for Table API >

[jira] [Commented] (FLINK-3152) Support all comparisons for Date type

2016-03-03 Thread Timo Walther (JIRA)
[ https://issues.apache.org/jira/browse/FLINK-3152?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15177949#comment-15177949 ] Timo Walther commented on FLINK-3152: - [~dawidwys] I just created [FLINK-3574]. It is

[jira] [Commented] (FLINK-2523) Make task canceling interrupt interval configurable

2016-03-03 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/FLINK-2523?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15177939#comment-15177939 ] ASF GitHub Bot commented on FLINK-2523: --- Github user tillrohrmann commented on the p

[GitHub] flink pull request: FLINK-2523: Makes the task cancellation interv...

2016-03-03 Thread tillrohrmann
Github user tillrohrmann commented on the pull request: https://github.com/apache/flink/pull/1662#issuecomment-191812912 Thanks for your contribution @kl0u. Changes for the task cancellation interval look good. Maybe it would be good to add a test case which makes sure that t

[jira] [Created] (FLINK-3574) Implement math functions for Table API

2016-03-03 Thread Timo Walther (JIRA)
Timo Walther created FLINK-3574: --- Summary: Implement math functions for Table API Key: FLINK-3574 URL: https://issues.apache.org/jira/browse/FLINK-3574 Project: Flink Issue Type: Sub-task

[jira] [Created] (FLINK-3573) Implement more String functions for Table API

2016-03-03 Thread Timo Walther (JIRA)
Timo Walther created FLINK-3573: --- Summary: Implement more String functions for Table API Key: FLINK-3573 URL: https://issues.apache.org/jira/browse/FLINK-3573 Project: Flink Issue Type: Sub-tas

[jira] [Commented] (FLINK-3467) Remove superfluous objects from DataSourceTask.invoke

2016-03-03 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/FLINK-3467?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15177934#comment-15177934 ] ASF GitHub Bot commented on FLINK-3467: --- Github user tillrohrmann commented on the p

[GitHub] flink pull request: [FLINK-3467] Remove superfluous objects from D...

2016-03-03 Thread tillrohrmann
Github user tillrohrmann commented on the pull request: https://github.com/apache/flink/pull/1691#issuecomment-191809419 Thanks for your contribution @greghogan. Changes look good to me. +1 for merging :-) --- If your project is set up for it, you can reply to this email and have you

[jira] [Assigned] (FLINK-3571) SavepointITCase.testRestoreFailure fails on Travis

2016-03-03 Thread Ufuk Celebi (JIRA)
[ https://issues.apache.org/jira/browse/FLINK-3571?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Ufuk Celebi reassigned FLINK-3571: -- Assignee: Ufuk Celebi > SavepointITCase.testRestoreFailure fails on Travis > --

[jira] [Commented] (FLINK-3454) Add test dependencies on packaged jars

2016-03-03 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/FLINK-3454?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15177929#comment-15177929 ] ASF GitHub Bot commented on FLINK-3454: --- Github user tillrohrmann commented on the p

[GitHub] flink pull request: [FLINK-3454] Add test dependencies on packaged...

2016-03-03 Thread tillrohrmann
Github user tillrohrmann commented on the pull request: https://github.com/apache/flink/pull/1698#issuecomment-191806709 I think, this does not solve the problem of the test. The test case `ScalaShellITCase.testSubmissionOfExternalLibrary` expects that `flink-ml` has been packaged, be

[jira] [Commented] (FLINK-3563) .returns() doesn't compile when using .map() with a custom MapFunction

2016-03-03 Thread Timo Walther (JIRA)
[ https://issues.apache.org/jira/browse/FLINK-3563?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15177925#comment-15177925 ] Timo Walther commented on FLINK-3563: - [~chobeat] can you check if https://github.com/

[jira] [Commented] (FLINK-3566) Input type validation often fails on custom TypeInfo implementations

2016-03-03 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/FLINK-3566?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15177921#comment-15177921 ] ASF GitHub Bot commented on FLINK-3566: --- GitHub user twalthr opened a pull request:

[GitHub] flink pull request: [FLINK-3566] [FLINK-3563] Input type validatio...

2016-03-03 Thread twalthr
GitHub user twalthr opened a pull request: https://github.com/apache/flink/pull/1759 [FLINK-3566] [FLINK-3563] Input type validation improvements This PR fixes issues with the input type validation. [FLINK-3566] Custom type information have been interpreted as basic or tup

[jira] [Commented] (FLINK-1159) Case style anonymous functions not supported by Scala API

2016-03-03 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/FLINK-1159?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15177907#comment-15177907 ] ASF GitHub Bot commented on FLINK-1159: --- Github user tillrohrmann commented on the p

[GitHub] flink pull request: [FLINK-1159] Case style anonymous functions no...

2016-03-03 Thread tillrohrmann
Github user tillrohrmann commented on the pull request: https://github.com/apache/flink/pull/1704#issuecomment-191799798 There are scalastyle violations in the code: ``` error file=/home/travis/build/apache/flink/flink-tests/src/test/scala/org/apache/flink/api/scala/extens

[jira] [Commented] (FLINK-3496) Flink-ml tests fail on Windows

2016-03-03 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/FLINK-3496?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15177903#comment-15177903 ] ASF GitHub Bot commented on FLINK-3496: --- Github user asfgit closed the pull request

[GitHub] flink pull request: [FLINK-3496] Fix ML test discovery on Windows

2016-03-03 Thread asfgit
Github user asfgit closed the pull request at: https://github.com/apache/flink/pull/1710 --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enab

[jira] [Resolved] (FLINK-3496) Flink-ml tests fail on Windows

2016-03-03 Thread Till Rohrmann (JIRA)
[ https://issues.apache.org/jira/browse/FLINK-3496?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Till Rohrmann resolved FLINK-3496. -- Resolution: Fixed Fixed via b02ee764b022683ff3ca1b739bfe8d58a298f103 > Flink-ml tests fail on W

[GitHub] flink pull request: [build] Consolidate scala checkstyle usage and...

2016-03-03 Thread mbalassi
Github user mbalassi commented on the pull request: https://github.com/apache/flink/pull/1753#issuecomment-191797058 Thanks or catching it, pushed the fix. --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project d

[jira] [Created] (FLINK-3572) SourceStreamTaskTest.testCheckpointing fails on Travis

2016-03-03 Thread Till Rohrmann (JIRA)
Till Rohrmann created FLINK-3572: Summary: SourceStreamTaskTest.testCheckpointing fails on Travis Key: FLINK-3572 URL: https://issues.apache.org/jira/browse/FLINK-3572 Project: Flink Issue Ty

[jira] [Commented] (FLINK-3496) Flink-ml tests fail on Windows

2016-03-03 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/FLINK-3496?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15177901#comment-15177901 ] ASF GitHub Bot commented on FLINK-3496: --- Github user tillrohrmann commented on the p

[GitHub] flink pull request: [FLINK-3496] Fix ML test discovery on Windows

2016-03-03 Thread tillrohrmann
Github user tillrohrmann commented on the pull request: https://github.com/apache/flink/pull/1710#issuecomment-191796890 Will merge it. --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this fe

[GitHub] flink pull request: [build] Consolidate scala checkstyle usage and...

2016-03-03 Thread tillrohrmann
Github user tillrohrmann commented on the pull request: https://github.com/apache/flink/pull/1753#issuecomment-191792667 Changes seem not to be complete. Travis test cases fail because of "Unable to find configuration file at location /home/travis/build/apache/flink/flink-yarn-tests/.

[jira] [Created] (FLINK-3571) SavepointITCase.testRestoreFailure fails on Travis

2016-03-03 Thread Till Rohrmann (JIRA)
Till Rohrmann created FLINK-3571: Summary: SavepointITCase.testRestoreFailure fails on Travis Key: FLINK-3571 URL: https://issues.apache.org/jira/browse/FLINK-3571 Project: Flink Issue Type:

[jira] [Commented] (FLINK-3563) .returns() doesn't compile when using .map() with a custom MapFunction

2016-03-03 Thread Timo Walther (JIRA)
[ https://issues.apache.org/jira/browse/FLINK-3563?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15177871#comment-15177871 ] Timo Walther commented on FLINK-3563: - I think I have identified the problem. I will o

[GitHub] flink pull request: [FLINK-3554] [streaming] Emit a MAX Watermark ...

2016-03-03 Thread StephanEwen
Github user StephanEwen commented on the pull request: https://github.com/apache/flink/pull/1750#issuecomment-191781217 So, the semantics are that at the end of a finite source, the "end of event time" is reached, which would make the emission of pending event time semantically correc

[jira] [Commented] (FLINK-3554) Bounded sources should emit a Max Watermark when they are done

2016-03-03 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/FLINK-3554?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15177858#comment-15177858 ] ASF GitHub Bot commented on FLINK-3554: --- Github user StephanEwen commented on the pu

[jira] [Assigned] (FLINK-3429) Add histogram-based timestamp/watermark extractor

2016-03-03 Thread Klou (JIRA)
[ https://issues.apache.org/jira/browse/FLINK-3429?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Klou reassigned FLINK-3429: --- Assignee: Klou > Add histogram-based timestamp/watermark extractor >

[jira] [Assigned] (FLINK-3428) Add fixed time trailing timestamp/watermark extractor

2016-03-03 Thread Klou (JIRA)
[ https://issues.apache.org/jira/browse/FLINK-3428?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Klou reassigned FLINK-3428: --- Assignee: Klou > Add fixed time trailing timestamp/watermark extractor >

[jira] [Commented] (FLINK-2941) Implement a neo4j - Flink/Gelly connector

2016-03-03 Thread Vitor Santos Vieira (JIRA)
[ https://issues.apache.org/jira/browse/FLINK-2941?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15177835#comment-15177835 ] Vitor Santos Vieira commented on FLINK-2941: I believe that this same situatio

[jira] [Commented] (FLINK-3563) .returns() doesn't compile when using .map() with a custom MapFunction

2016-03-03 Thread Simone Robutti (JIRA)
[ https://issues.apache.org/jira/browse/FLINK-3563?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15177831#comment-15177831 ] Simone Robutti commented on FLINK-3563: --- Anyway I just rewrote a good part of the li

[jira] [Commented] (FLINK-3570) Replace random NIC selection heuristic by InetAddress.getLocalHost

2016-03-03 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/FLINK-3570?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15177811#comment-15177811 ] ASF GitHub Bot commented on FLINK-3570: --- GitHub user tillrohrmann opened a pull requ

[GitHub] flink pull request: [FLINK-3570] [runtime] Use InetAddress.getLoca...

2016-03-03 Thread tillrohrmann
GitHub user tillrohrmann opened a pull request: https://github.com/apache/flink/pull/1758 [FLINK-3570] [runtime] Use InetAddress.getLocalHost() as heuristic to find local address The ConnectionUtils.findAddressUsingStrategy method tries to find out the local address which is ac

[jira] [Commented] (FLINK-3568) Hadoop's Bzip decompression is not thread safe

2016-03-03 Thread Chesnay Schepler (JIRA)
[ https://issues.apache.org/jira/browse/FLINK-3568?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15177785#comment-15177785 ] Chesnay Schepler commented on FLINK-3568: - according to HADOOP-10614 this was alre

[jira] [Commented] (FLINK-3569) Test cases fail due to Maven Shade plugin

2016-03-03 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/FLINK-3569?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15177782#comment-15177782 ] ASF GitHub Bot commented on FLINK-3569: --- Github user mxm commented on the pull reque

[GitHub] flink pull request: [FLINK-3569] set Maven Surefire working direct...

2016-03-03 Thread mxm
Github user mxm commented on the pull request: https://github.com/apache/flink/pull/1757#issuecomment-191752822 Sorry, still failing (wasn't failing in my machine). I'll look into it. --- If your project is set up for it, you can reply to this email and have your reply appear on GitHu

[jira] [Created] (FLINK-3570) Replace random NIC selection heuristic by InetAddress.getLocalHost

2016-03-03 Thread Till Rohrmann (JIRA)
Till Rohrmann created FLINK-3570: Summary: Replace random NIC selection heuristic by InetAddress.getLocalHost Key: FLINK-3570 URL: https://issues.apache.org/jira/browse/FLINK-3570 Project: Flink

[jira] [Commented] (FLINK-3179) Combiner is not injected if Reduce or GroupReduce input is explicitly partitioned

2016-03-03 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/FLINK-3179?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=1519#comment-1519 ] ASF GitHub Bot commented on FLINK-3179: --- Github user fhueske commented on the pull r

[GitHub] flink pull request: FLINK-3179 Combiner is not injected if Reduce ...

2016-03-03 Thread fhueske
Github user fhueske commented on the pull request: https://github.com/apache/flink/pull/1553#issuecomment-191752181 Sorry, I forgot a `groupBy()` in my example. It should be ``` DataSet> data = ... DataSet> pData = data.partitionByHash(0); pData.map(new SomeMapF

[jira] [Commented] (FLINK-3179) Combiner is not injected if Reduce or GroupReduce input is explicitly partitioned

2016-03-03 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/FLINK-3179?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15177765#comment-15177765 ] ASF GitHub Bot commented on FLINK-3179: --- Github user ramkrish86 commented on the pul

[GitHub] flink pull request: FLINK-3179 Combiner is not injected if Reduce ...

2016-03-03 Thread ramkrish86
Github user ramkrish86 commented on the pull request: https://github.com/apache/flink/pull/1553#issuecomment-191748107 @fhueske So for doing the above example where the partioned input goes both to the map and reducer as input should this class AllGroupWithPartialPreGroupProp

[jira] [Assigned] (FLINK-3563) .returns() doesn't compile when using .map() with a custom MapFunction

2016-03-03 Thread Timo Walther (JIRA)
[ https://issues.apache.org/jira/browse/FLINK-3563?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Timo Walther reassigned FLINK-3563: --- Assignee: Timo Walther > .returns() doesn't compile when using .map() with a custom MapFuncti

[jira] [Commented] (FLINK-3568) Hadoop's Bzip decompression is not thread safe

2016-03-03 Thread Sebastian Neef (JIRA)
[ https://issues.apache.org/jira/browse/FLINK-3568?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15177740#comment-15177740 ] Sebastian Neef commented on FLINK-3568: --- That was the default priority, but maybe "C

[jira] [Updated] (FLINK-3568) Hadoop's Bzip decompression is not thread safe

2016-03-03 Thread Sebastian Neef (JIRA)
[ https://issues.apache.org/jira/browse/FLINK-3568?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Sebastian Neef updated FLINK-3568: -- Priority: Critical (was: Major) > Hadoop's Bzip decompression is not thread safe >

[jira] [Commented] (FLINK-3568) Hadoop's Bzip decompression is not thread safe

2016-03-03 Thread Moritz Schubotz (JIRA)
[ https://issues.apache.org/jira/browse/FLINK-3568?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15177733#comment-15177733 ] Moritz Schubotz commented on FLINK-3568: Thanks for your report. However, I would

[jira] [Commented] (FLINK-3569) Test cases fail due to Maven Shade plugin

2016-03-03 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/FLINK-3569?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15177717#comment-15177717 ] ASF GitHub Bot commented on FLINK-3569: --- GitHub user mxm opened a pull request:

[GitHub] flink pull request: [FLINK-3569] set Maven Surefire working direct...

2016-03-03 Thread mxm
GitHub user mxm opened a pull request: https://github.com/apache/flink/pull/1757 [FLINK-3569] set Maven Surefire working directory You can merge this pull request into a Git repository by running: $ git pull https://github.com/mxm/flink FLINK-3569 Alternatively you can review

[jira] [Created] (FLINK-3569) Test cases fail due to Maven Shade plugin

2016-03-03 Thread Maximilian Michels (JIRA)
Maximilian Michels created FLINK-3569: - Summary: Test cases fail due to Maven Shade plugin Key: FLINK-3569 URL: https://issues.apache.org/jira/browse/FLINK-3569 Project: Flink Issue Type:

[jira] [Created] (FLINK-3568) Hadoop's Bzip decompression is not thread safe

2016-03-03 Thread Sebastian Neef (JIRA)
Sebastian Neef created FLINK-3568: - Summary: Hadoop's Bzip decompression is not thread safe Key: FLINK-3568 URL: https://issues.apache.org/jira/browse/FLINK-3568 Project: Flink Issue Type: Bu

[jira] [Resolved] (FLINK-3562) Update docs in the course of EventTimeSourceFunction removal

2016-03-03 Thread Stephan Ewen (JIRA)
[ https://issues.apache.org/jira/browse/FLINK-3562?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Stephan Ewen resolved FLINK-3562. - Resolution: Fixed Assignee: Stephan Ewen Fixed in - 1.0.1 via 7e6c2ec628ad07077dfdf47fd5b8

[jira] [Closed] (FLINK-3562) Update docs in the course of EventTimeSourceFunction removal

2016-03-03 Thread Stephan Ewen (JIRA)
[ https://issues.apache.org/jira/browse/FLINK-3562?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Stephan Ewen closed FLINK-3562. --- > Update docs in the course of EventTimeSourceFunction removal > -

[GitHub] flink pull request: [FLINK-3565] add module to force execution of ...

2016-03-03 Thread mxm
Github user mxm commented on the pull request: https://github.com/apache/flink/pull/1755#issuecomment-191703337 The issue is that we use relative paths to JARs in the tests. The Shade plugin changes the working directory. Will provide a fix. --- If your project is set up for it, you

[jira] [Commented] (FLINK-3565) FlinkKafkaConsumer does not work with Scala 2.11

2016-03-03 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/FLINK-3565?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15177658#comment-15177658 ] ASF GitHub Bot commented on FLINK-3565: --- Github user mxm commented on the pull reque

[jira] [Created] (FLINK-3567) Rework selection when grouping in Table API

2016-03-03 Thread Timo Walther (JIRA)
Timo Walther created FLINK-3567: --- Summary: Rework selection when grouping in Table API Key: FLINK-3567 URL: https://issues.apache.org/jira/browse/FLINK-3567 Project: Flink Issue Type: Bug

[jira] [Commented] (FLINK-3566) Input type validation often fails on custom TypeInfo implementations

2016-03-03 Thread Gyula Fora (JIRA)
[ https://issues.apache.org/jira/browse/FLINK-3566?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15177648#comment-15177648 ] Gyula Fora commented on FLINK-3566: --- Thanks Timo :) > Input type validation often fails

[jira] [Commented] (FLINK-3566) Input type validation often fails on custom TypeInfo implementations

2016-03-03 Thread Timo Walther (JIRA)
[ https://issues.apache.org/jira/browse/FLINK-3566?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15177641#comment-15177641 ] Timo Walther commented on FLINK-3566: - Thanks for the examples. I will look into it.

[jira] [Assigned] (FLINK-3566) Input type validation often fails on custom TypeInfo implementations

2016-03-03 Thread Timo Walther (JIRA)
[ https://issues.apache.org/jira/browse/FLINK-3566?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Timo Walther reassigned FLINK-3566: --- Assignee: Timo Walther > Input type validation often fails on custom TypeInfo implementations

[jira] [Commented] (FLINK-3564) Implement distinct() for Table API

2016-03-03 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/FLINK-3564?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15177636#comment-15177636 ] ASF GitHub Bot commented on FLINK-3564: --- Github user twalthr commented on the pull r

[jira] [Commented] (FLINK-3562) Update docs in the course of EventTimeSourceFunction removal

2016-03-03 Thread Stephan Ewen (JIRA)
[ https://issues.apache.org/jira/browse/FLINK-3562?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15177635#comment-15177635 ] Stephan Ewen commented on FLINK-3562: - Fixed the javadocs for 1.0 and 1.1 > Update do

[GitHub] flink pull request: [FLINK-3564] [table] Implement distinct() for ...

2016-03-03 Thread twalthr
Github user twalthr commented on the pull request: https://github.com/apache/flink/pull/1754#issuecomment-191699364 @fhueske I corrected the tests and rebased. I will create an issue for the wrong aggregation usage. --- If your project is set up for it, you can reply to this email an

[jira] [Commented] (FLINK-3474) Partial aggregate interface design and sort-based implementation

2016-03-03 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/FLINK-3474?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15177634#comment-15177634 ] ASF GitHub Bot commented on FLINK-3474: --- Github user ChengXiangLi commented on the p

[GitHub] flink pull request: [FLINK-3474] support partial aggregate

2016-03-03 Thread ChengXiangLi
Github user ChengXiangLi commented on the pull request: https://github.com/apache/flink/pull/1746#issuecomment-191698778 Thanks, @fhueske , i've updated the PR. --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your proj

[jira] [Commented] (FLINK-3554) Bounded sources should emit a Max Watermark when they are done

2016-03-03 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/FLINK-3554?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15177619#comment-15177619 ] ASF GitHub Bot commented on FLINK-3554: --- Github user aljoscha commented on the pull

[jira] [Commented] (FLINK-3565) FlinkKafkaConsumer does not work with Scala 2.11

2016-03-03 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/FLINK-3565?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15177618#comment-15177618 ] ASF GitHub Bot commented on FLINK-3565: --- Github user rmetzger commented on the pull

[GitHub] flink pull request: [FLINK-3554] [streaming] Emit a MAX Watermark ...

2016-03-03 Thread aljoscha
Github user aljoscha commented on the pull request: https://github.com/apache/flink/pull/1750#issuecomment-191695612 Yes, right now we favor correctness over completeness. For example in a Count window, if we emitted an unfinished count window then the count of the elements in the win

[GitHub] flink pull request: [FLINK-3565] add module to force execution of ...

2016-03-03 Thread rmetzger
Github user rmetzger commented on the pull request: https://github.com/apache/flink/pull/1755#issuecomment-191695544 The change breaks the tests: https://travis-ci.org/apache/flink/builds/113179846 --- If your project is set up for it, you can reply to this email and have your reply

[GitHub] flink pull request: [FLINK-3554] [streaming] Emit a MAX Watermark ...

2016-03-03 Thread StephanEwen
Github user StephanEwen commented on the pull request: https://github.com/apache/flink/pull/1750#issuecomment-191694897 It is a matter of how we want to define the semantics. For windows that do not depend on event time, should incomplete windows be emitted or not? There are a

[jira] [Commented] (FLINK-3554) Bounded sources should emit a Max Watermark when they are done

2016-03-03 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/FLINK-3554?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15177614#comment-15177614 ] ASF GitHub Bot commented on FLINK-3554: --- Github user StephanEwen commented on the pu

  1   2   >