[GitHub] flink pull request #3502: [FLINK-4565] Support for SQL IN operator

2017-03-10 Thread DmytroShkvyra
Github user DmytroShkvyra closed the pull request at: https://github.com/apache/flink/pull/3502 --- 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

[GitHub] flink pull request #3502: [FLINK-4565] Support for SQL IN operator

2017-03-10 Thread DmytroShkvyra
GitHub user DmytroShkvyra reopened a pull request: https://github.com/apache/flink/pull/3502 [FLINK-4565] Support for SQL IN operator [FLINK-4565] Support for SQL IN operator This PR is a part of work on SQL IN operator in Table API, which implements IN for literals. Two cas

[jira] [Created] (FLINK-6019) Some log4j messages do not have a loglevel field set, so they can't be suppressed

2017-03-10 Thread Luke Hutchison (JIRA)
Luke Hutchison created FLINK-6019: - Summary: Some log4j messages do not have a loglevel field set, so they can't be suppressed Key: FLINK-6019 URL: https://issues.apache.org/jira/browse/FLINK-6019 Pro

[jira] [Assigned] (FLINK-6000) Can not start HA cluster with start-cluster.sh

2017-03-10 Thread Dawid Wysakowicz (JIRA)
[ https://issues.apache.org/jira/browse/FLINK-6000?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Dawid Wysakowicz reassigned FLINK-6000: --- Assignee: Dawid Wysakowicz > Can not start HA cluster with start-cluster.sh > ---

[jira] [Commented] (FLINK-6019) Some log4j messages do not have a loglevel field set, so they can't be suppressed

2017-03-10 Thread Luke Hutchison (JIRA)
[ https://issues.apache.org/jira/browse/FLINK-6019?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15904680#comment-15904680 ] Luke Hutchison commented on FLINK-6019: --- Also, these messages do not respond to sett

[jira] [Commented] (FLINK-6000) Can not start HA cluster with start-cluster.sh

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

[GitHub] flink pull request #3506: [FLINK-6000] Fix starting HA cluster with start-cl...

2017-03-10 Thread dawidwys
GitHub user dawidwys opened a pull request: https://github.com/apache/flink/pull/3506 [FLINK-6000] Fix starting HA cluster with start-cluster.sh You can merge this pull request into a Git repository by running: $ git pull https://github.com/dawidwys/flink flink6000 Alternativ

[GitHub] flink issue #3461: [FLINK-5954] Always assign names to the window in the Str...

2017-03-10 Thread fhueske
Github user fhueske commented on the issue: https://github.com/apache/flink/pull/3461 Thanks for the PR @haohui. Looks good to me. +1 to merge --- 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 h

[jira] [Commented] (FLINK-5954) Always assign names to the window in the Stream SQL API

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

[jira] [Created] (FLINK-6020) Blob Server cannot hanlde multiple job sumits(with same content) parallelly

2017-03-10 Thread Tao Wang (JIRA)
Tao Wang created FLINK-6020: --- Summary: Blob Server cannot hanlde multiple job sumits(with same content) parallelly Key: FLINK-6020 URL: https://issues.apache.org/jira/browse/FLINK-6020 Project: Flink

[GitHub] flink pull request #3507: [FLINK-6006] [kafka] Always use complete restored ...

2017-03-10 Thread tzulitai
GitHub user tzulitai opened a pull request: https://github.com/apache/flink/pull/3507 [FLINK-6006] [kafka] Always use complete restored state in FlinkKafkaConsumer (This PR is the fix of FLINK-6006 for Flink 1.1) Previously, the Kafka Consumer performs partition list queryi

[jira] [Commented] (FLINK-6006) Kafka Consumer can lose state if queried partition list is incomplete on restore

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

[GitHub] flink issue #3406: [flink-5568] [Table API & SQL]Introduce interface for cat...

2017-03-10 Thread fhueske
Github user fhueske commented on the issue: https://github.com/apache/flink/pull/3406 Hi @KurtYoung, you are right. Only `requiredProperties()` would be required to verify properties. I thought that the other two methods would be a good way to define the parameters of the co

[jira] [Commented] (FLINK-5859) support partition pruning on Table API & SQL

2017-03-10 Thread Fabian Hueske (JIRA)
[ https://issues.apache.org/jira/browse/FLINK-5859?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15904724#comment-15904724 ] Fabian Hueske commented on FLINK-5859: -- Yes, the first changes should be attributed t

[jira] [Commented] (FLINK-5985) Flink treats every task as stateful (making topology changes impossible)

2017-03-10 Thread Ufuk Celebi (JIRA)
[ https://issues.apache.org/jira/browse/FLINK-5985?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15904779#comment-15904779 ] Ufuk Celebi commented on FLINK-5985: Thanks for taking the time to create the example,

[GitHub] flink pull request #3406: [flink-5568] [Table API & SQL]Introduce interface ...

2017-03-10 Thread beyond1920
Github user beyond1920 commented on a diff in the pull request: https://github.com/apache/flink/pull/3406#discussion_r105362292 --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/catalog/ExternalCatalog.scala --- @@ -0,0 +1,114 @@ +/* + * Licensed

[jira] [Commented] (FLINK-5890) GatherSumApply broken when object reuse enabled

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

[jira] [Commented] (FLINK-5654) Add processing time OVER RANGE BETWEEN x PRECEDING aggregation to SQL

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

[jira] [Commented] (FLINK-5653) Add processing time OVER ROWS BETWEEN x PRECEDING aggregation to SQL

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

[GitHub] flink issue #3504: [FLINK-6010] Documentation: correct IntelliJ IDEA Plugins...

2017-03-10 Thread StephanEwen
Github user StephanEwen commented on the issue: https://github.com/apache/flink/pull/3504 Looks good, merging this... --- 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

[jira] [Commented] (FLINK-6010) Documentation: correct IntelliJ IDEA Plugins path in 'Installing the Scala plugin' section

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

[GitHub] flink pull request #3406: [flink-5568] [Table API & SQL]Introduce interface ...

2017-03-10 Thread beyond1920
Github user beyond1920 commented on a diff in the pull request: https://github.com/apache/flink/pull/3406#discussion_r105362031 --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/catalog/CatalogTableHelper.scala --- @@ -0,0 +1,138 @@ +/* + * Licens

[GitHub] flink pull request #3406: [flink-5568] [Table API & SQL]Introduce interface ...

2017-03-10 Thread beyond1920
Github user beyond1920 commented on a diff in the pull request: https://github.com/apache/flink/pull/3406#discussion_r105363598 --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/catalog/ExternalCatalog.scala --- @@ -0,0 +1,114 @@ +/* + * Licensed

[GitHub] flink issue #3459: [FLINK-5654] Add processing time OVER RANGE BETWEEN x PRE...

2017-03-10 Thread fhueske
Github user fhueske commented on the issue: https://github.com/apache/flink/pull/3459 Hi @huawei-flink, most Maven modules in Flink are implemented in either Java or Scala. We have a few modules (like flink-runtime) which are mixed but didn't make good experience with this d

[GitHub] flink issue #3402: [FLINK-5890] [gelly] GatherSumApply broken when object re...

2017-03-10 Thread StephanEwen
Github user StephanEwen commented on the issue: https://github.com/apache/flink/pull/3402 @greghogan you are right, I overlooked the adjusted tests! This looks like a good fix, please merge! --- If your project is set up for it, you can reply to this email and have your reply

[GitHub] flink pull request #3406: [flink-5568] [Table API & SQL]Introduce interface ...

2017-03-10 Thread beyond1920
Github user beyond1920 commented on a diff in the pull request: https://github.com/apache/flink/pull/3406#discussion_r105363822 --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/catalog/ExternalCatalog.scala --- @@ -0,0 +1,114 @@ +/* + * Licensed

[jira] [Commented] (FLINK-5995) Get a Exception when creating the ListStateDescriptor with a TypeInformation

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

[GitHub] flink issue #3431: [FLINK-5910] [gelly] Framework for Gelly examples

2017-03-10 Thread vasia
Github user vasia commented on the issue: https://github.com/apache/flink/pull/3431 Thanks! Then, it's good to go from my side. --- 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 en

[jira] [Commented] (FLINK-6005) unit test ArrayList initializations without initial size

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

[jira] [Commented] (FLINK-5910) Framework for Gelly examples

2017-03-10 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/FLINK-5910?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15904820#comment-15904820 ] ASF GitHub Bot commented on FLINK-5910: --- Github user vasia commented on the issue:

[GitHub] flink issue #3503: [FLINK-5995][checkpoints] fix Get a Exception when creati...

2017-03-10 Thread StephanEwen
Github user StephanEwen commented on the issue: https://github.com/apache/flink/pull/3503 Thanks for finding and fixing this, important fix! It would be great if you could add a test for this, to make sure the bug is not accidentally re-introduced later by someone. You can fol

[GitHub] flink issue #3487: [FLINK-5980] Expose max-parallelism value in RuntimeConte...

2017-03-10 Thread StephanEwen
Github user StephanEwen commented on the issue: https://github.com/apache/flink/pull/3487 I think this looks good now, merging this... --- 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 fea

[jira] [Commented] (FLINK-5980) Expose max-parallelism value in RuntimeContext

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

[GitHub] flink issue #3499: [FLINK-6005] fix some ArrayList initializations without i...

2017-03-10 Thread StephanEwen
Github user StephanEwen commented on the issue: https://github.com/apache/flink/pull/3499 Looks good, merging... --- 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 wishe

[GitHub] flink issue #3477: [Flink-3318] Add support for quantifiers to CEP's pattern...

2017-03-10 Thread dawidwys
Github user dawidwys commented on the issue: https://github.com/apache/flink/pull/3477 @kl0u could you have a look at this 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 project does not have this feature ena

[jira] [Commented] (FLINK-5998) Un-fat Hadoop from Flink fat jar

2017-03-10 Thread Robert Metzger (JIRA)
[ https://issues.apache.org/jira/browse/FLINK-5998?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15904837#comment-15904837 ] Robert Metzger commented on FLINK-5998: --- [~wheat9] Thanks a lot for assigning that i

[GitHub] flink issue #3443: [FLINK-5653] Add processing time OVER ROWS BETWEEN x PREC...

2017-03-10 Thread fhueske
Github user fhueske commented on the issue: https://github.com/apache/flink/pull/3443 Hi, I commented on your other PR (#3459) about mixing Java and Scala and how to squash the commits. Thanks, Fabian --- If your project is set up for it, you can reply to this email and have

[GitHub] flink pull request #3484: [FLINK-4460] Side Outputs in Flink

2017-03-10 Thread aljoscha
Github user aljoscha commented on a diff in the pull request: https://github.com/apache/flink/pull/3484#discussion_r105368360 --- Diff: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperator.java --- @@ -419,6 +435,14 @@ public voi

[jira] [Commented] (FLINK-5991) Expose Broadcast Operator State through public APIs

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

[jira] [Commented] (FLINK-4460) Side Outputs in Flink

2017-03-10 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/FLINK-4460?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15904842#comment-15904842 ] ASF GitHub Bot commented on FLINK-4460: --- Github user aljoscha commented on a diff in

[GitHub] flink issue #3508: [FLINK-5991] [state-backend, streaming] Expose Broadcast ...

2017-03-10 Thread tzulitai
Github user tzulitai commented on the issue: https://github.com/apache/flink/pull/3508 R: @aljoscha @StefanRRichter --- 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 wi

[GitHub] flink pull request #3508: [FLINK-5991] [state-backend, streaming] Expose Bro...

2017-03-10 Thread tzulitai
GitHub user tzulitai opened a pull request: https://github.com/apache/flink/pull/3508 [FLINK-5991] [state-backend, streaming] Expose Broadcast Operator State This PR exposes broadcast operator state through the `CheckpointedFunction` interface, by adding broadcast state access metho

[jira] [Commented] (FLINK-5991) Expose Broadcast Operator State through public APIs

2017-03-10 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/FLINK-5991?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15904846#comment-15904846 ] ASF GitHub Bot commented on FLINK-5991: --- Github user tzulitai commented on the issue

[GitHub] flink issue #3485: FLINK-5976 [tests] Deduplicate Tokenizer in tests

2017-03-10 Thread StephanEwen
Github user StephanEwen commented on the issue: https://github.com/apache/flink/pull/3485 Looks good, thanks! Merging this... --- 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 enab

[jira] [Commented] (FLINK-5976) Refactoring duplicate Tokenizer in flink-test

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

[jira] [Created] (FLINK-6021) Downloads page references "Hadoop 1 version" which isn't an option

2017-03-10 Thread Patrick Lucas (JIRA)
Patrick Lucas created FLINK-6021: Summary: Downloads page references "Hadoop 1 version" which isn't an option Key: FLINK-6021 URL: https://issues.apache.org/jira/browse/FLINK-6021 Project: Flink

[GitHub] flink issue #2905: [FLINK-5190] [runtime] fix ZooKeeperLeaderRetrievalServic...

2017-03-10 Thread tillrohrmann
Github user tillrohrmann commented on the issue: https://github.com/apache/flink/pull/2905 I fear that we cannot merge the PR as it is, because it would leave the `CuratorFramework` client open for some of the components used in the master. For example, the `WebMonitor` uses the same

[jira] [Commented] (FLINK-5190) ZooKeeperLeaderRetrievalService should not close the zk client when stop

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

[jira] [Commented] (FLINK-5985) Flink treats every task as stateful (making topology changes impossible)

2017-03-10 Thread Ufuk Celebi (JIRA)
[ https://issues.apache.org/jira/browse/FLINK-5985?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15904869#comment-15904869 ] Ufuk Celebi commented on FLINK-5985: Thanks again for reporting this. You are right, w

[jira] [Commented] (FLINK-5981) SSL version and ciper suites cannot be constrained as configured

2017-03-10 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/FLINK-5981?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15904873#comment-15904873 ] ASF GitHub Bot commented on FLINK-5981: --- Github user StephanEwen commented on a diff

[jira] [Commented] (FLINK-5981) SSL version and ciper suites cannot be constrained as configured

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

[GitHub] flink pull request #3484: [FLINK-4460] Side Outputs in Flink

2017-03-10 Thread aljoscha
Github user aljoscha commented on a diff in the pull request: https://github.com/apache/flink/pull/3484#discussion_r105372676 --- Diff: flink-streaming-java/src/main/java/org/apache/flink/streaming/runtime/operators/windowing/WindowOperator.java --- @@ -419,6 +435,14 @@ public voi

[jira] [Commented] (FLINK-4460) Side Outputs in Flink

2017-03-10 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/FLINK-4460?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15904878#comment-15904878 ] ASF GitHub Bot commented on FLINK-4460: --- Github user aljoscha commented on a diff in

[GitHub] flink pull request #3486: [FLINK-5981][SECURITY]make ssl version and cipher ...

2017-03-10 Thread StephanEwen
Github user StephanEwen commented on a diff in the pull request: https://github.com/apache/flink/pull/3486#discussion_r105371699 --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/net/SSLUtils.java --- @@ -55,6 +58,42 @@ public static boolean getSSLEnabled(Configuration

[GitHub] flink issue #3486: [FLINK-5981][SECURITY]make ssl version and cipher suites ...

2017-03-10 Thread StephanEwen
Github user StephanEwen commented on the issue: https://github.com/apache/flink/pull/3486 I think this is a good fix. Is it possible to add some form of test that makes sure the config is properly applied in all cases? What do you think about adding to the `SSLUtilsTest` class and

[jira] [Created] (FLINK-6022) Improve support for Avro GenericRecord

2017-03-10 Thread Robert Metzger (JIRA)
Robert Metzger created FLINK-6022: - Summary: Improve support for Avro GenericRecord Key: FLINK-6022 URL: https://issues.apache.org/jira/browse/FLINK-6022 Project: Flink Issue Type: Improvemen

[GitHub] flink issue #3508: [FLINK-5991] [state-backend, streaming] Expose Broadcast ...

2017-03-10 Thread StefanRRichter
Github user StefanRRichter commented on the issue: https://github.com/apache/flink/pull/3508 The PR re-introduces methods that have been removed from the public interface before the release. Reason the remove the methods was that we had too little time to make a final decision on the

[jira] [Commented] (FLINK-5991) Expose Broadcast Operator State through public APIs

2017-03-10 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/FLINK-5991?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15904882#comment-15904882 ] ASF GitHub Bot commented on FLINK-5991: --- Github user StefanRRichter commented on the

[GitHub] flink issue #3477: [Flink-3318] Add support for quantifiers to CEP's pattern...

2017-03-10 Thread kl0u
Github user kl0u commented on the issue: https://github.com/apache/flink/pull/3477 Hi @dawidwys , I will do this in the beginning of next week at the latest. One thing that I saw is that in the `Pattern` class you do not check for conflicting configurations e.g. oneToMany and

[jira] [Commented] (FLINK-5985) Flink treats every task as stateful (making topology changes impossible)

2017-03-10 Thread Stefan Richter (JIRA)
[ https://issues.apache.org/jira/browse/FLINK-5985?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15904895#comment-15904895 ] Stefan Richter commented on FLINK-5985: --- I think the proper solution is based on pre

[jira] [Commented] (FLINK-6022) Improve support for Avro GenericRecord

2017-03-10 Thread Flavio Pompermaier (JIRA)
[ https://issues.apache.org/jira/browse/FLINK-6022?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15904899#comment-15904899 ] Flavio Pompermaier commented on FLINK-6022: --- HI [~rmetzger], does this apply als

[GitHub] flink pull request #3501: [FLINK-5874] Restrict key types in the DataStream ...

2017-03-10 Thread kl0u
Github user kl0u commented on a diff in the pull request: https://github.com/apache/flink/pull/3501#discussion_r105377617 --- Diff: flink-tests/src/test/java/org/apache/flink/test/checkpointing/SavepointITCase.java --- @@ -736,7 +736,7 @@ public void restoreState(List state) throw

[jira] [Commented] (FLINK-5874) Reject arrays as keys in DataStream API to avoid inconsistent hashing

2017-03-10 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/FLINK-5874?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15904915#comment-15904915 ] ASF GitHub Bot commented on FLINK-5874: --- Github user kl0u commented on a diff in the

[jira] [Commented] (FLINK-5874) Reject arrays as keys in DataStream API to avoid inconsistent hashing

2017-03-10 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/FLINK-5874?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15904917#comment-15904917 ] ASF GitHub Bot commented on FLINK-5874: --- Github user kl0u commented on a diff in the

[GitHub] flink pull request #3501: [FLINK-5874] Restrict key types in the DataStream ...

2017-03-10 Thread kl0u
Github user kl0u commented on a diff in the pull request: https://github.com/apache/flink/pull/3501#discussion_r105377853 --- Diff: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/KeyedStream.java --- @@ -114,9 +121,53 @@ public KeyedStream(DataStream

[GitHub] flink pull request #3501: [FLINK-5874] Restrict key types in the DataStream ...

2017-03-10 Thread kl0u
Github user kl0u commented on a diff in the pull request: https://github.com/apache/flink/pull/3501#discussion_r105377789 --- Diff: flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/KeyedStream.java --- @@ -114,9 +121,53 @@ public KeyedStream(DataStream

[jira] [Commented] (FLINK-5874) Reject arrays as keys in DataStream API to avoid inconsistent hashing

2017-03-10 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/FLINK-5874?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15904918#comment-15904918 ] ASF GitHub Bot commented on FLINK-5874: --- Github user kl0u commented on a diff in the

[jira] [Commented] (FLINK-6022) Improve support for Avro GenericRecord

2017-03-10 Thread Robert Metzger (JIRA)
[ https://issues.apache.org/jira/browse/FLINK-6022?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15904920#comment-15904920 ] Robert Metzger commented on FLINK-6022: --- I think protobuf always works with generate

[GitHub] flink pull request #3501: [FLINK-5874] Restrict key types in the DataStream ...

2017-03-10 Thread kl0u
Github user kl0u commented on a diff in the pull request: https://github.com/apache/flink/pull/3501#discussion_r105378808 --- Diff: flink-streaming-java/src/test/java/org/apache/flink/streaming/api/DataStreamTest.java --- @@ -906,6 +919,256 @@ public void testChannelSelectors() {

[jira] [Commented] (FLINK-5874) Reject arrays as keys in DataStream API to avoid inconsistent hashing

2017-03-10 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/FLINK-5874?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15904928#comment-15904928 ] ASF GitHub Bot commented on FLINK-5874: --- Github user kl0u commented on a diff in the

[jira] [Commented] (FLINK-4545) Flink automatically manages TM network buffer

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

[GitHub] flink issue #3480: [FLINK-4545] use size-restricted LocalBufferPool instance...

2017-03-10 Thread StephanEwen
Github user StephanEwen commented on the issue: https://github.com/apache/flink/pull/3480 Did another review - looks good to me! Merging... --- 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

[GitHub] flink pull request #3501: [FLINK-5874] Restrict key types in the DataStream ...

2017-03-10 Thread kl0u
Github user kl0u commented on a diff in the pull request: https://github.com/apache/flink/pull/3501#discussion_r105378932 --- Diff: flink-streaming-java/src/test/java/org/apache/flink/streaming/api/DataStreamTest.java --- @@ -906,6 +919,256 @@ public void testChannelSelectors() {

[jira] [Commented] (FLINK-5874) Reject arrays as keys in DataStream API to avoid inconsistent hashing

2017-03-10 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/FLINK-5874?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15904931#comment-15904931 ] ASF GitHub Bot commented on FLINK-5874: --- Github user kl0u commented on a diff in the

[jira] [Resolved] (FLINK-5980) Expose max-parallelism value in RuntimeContext

2017-03-10 Thread Biao Liu (JIRA)
[ https://issues.apache.org/jira/browse/FLINK-5980?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Biao Liu resolved FLINK-5980. - Resolution: Fixed > Expose max-parallelism value in RuntimeContext > -

[jira] [Commented] (FLINK-5874) Reject arrays as keys in DataStream API to avoid inconsistent hashing

2017-03-10 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/FLINK-5874?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15904945#comment-15904945 ] ASF GitHub Bot commented on FLINK-5874: --- Github user kl0u commented on a diff in the

[GitHub] flink pull request #3501: [FLINK-5874] Restrict key types in the DataStream ...

2017-03-10 Thread kl0u
Github user kl0u commented on a diff in the pull request: https://github.com/apache/flink/pull/3501#discussion_r105380399 --- Diff: flink-streaming-java/src/test/java/org/apache/flink/streaming/api/DataStreamTest.java --- @@ -906,6 +919,256 @@ public void testChannelSelectors() {

[GitHub] flink issue #3218: [FLINK-5642][query] fix a race condition with HeadListSta...

2017-03-10 Thread NicoK
Github user NicoK commented on the issue: https://github.com/apache/flink/pull/3218 ok, let's close this PR as the issue is actually deeper than originally though and can only be fixed with a new heap state backend or by locking for queryable state queries as well --- If your projec

[jira] [Commented] (FLINK-5642) queryable state: race condition with HeadListState

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

[GitHub] flink pull request #3218: [FLINK-5642][query] fix a race condition with Head...

2017-03-10 Thread NicoK
Github user NicoK closed the pull request at: https://github.com/apache/flink/pull/3218 --- 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 enabl

[jira] [Commented] (FLINK-5642) queryable state: race condition with HeadListState

2017-03-10 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/FLINK-5642?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15904952#comment-15904952 ] ASF GitHub Bot commented on FLINK-5642: --- Github user NicoK commented on the issue:

[GitHub] flink pull request #3406: [flink-5568] [Table API & SQL]Introduce interface ...

2017-03-10 Thread fhueske
Github user fhueske commented on a diff in the pull request: https://github.com/apache/flink/pull/3406#discussion_r105382341 --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/catalog/CatalogTableHelper.scala --- @@ -0,0 +1,138 @@ +/* + * Licensed

[jira] [Created] (FLINK-6023) Fix Scala snippet into Process Function (Low-level Operations) Doc

2017-03-10 Thread Mauro Cortellazzi (JIRA)
Mauro Cortellazzi created FLINK-6023: Summary: Fix Scala snippet into Process Function (Low-level Operations) Doc Key: FLINK-6023 URL: https://issues.apache.org/jira/browse/FLINK-6023 Project: Fli

[jira] [Assigned] (FLINK-6023) Fix Scala snippet into Process Function (Low-level Operations) Doc

2017-03-10 Thread Andrea Sella (JIRA)
[ https://issues.apache.org/jira/browse/FLINK-6023?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Andrea Sella reassigned FLINK-6023: --- Assignee: Andrea Sella > Fix Scala snippet into Process Function (Low-level Operations) Doc >

[GitHub] flink pull request #3406: [flink-5568] [Table API & SQL]Introduce interface ...

2017-03-10 Thread beyond1920
Github user beyond1920 commented on a diff in the pull request: https://github.com/apache/flink/pull/3406#discussion_r105384191 --- Diff: flink-libraries/flink-table/src/main/scala/org/apache/flink/table/catalog/CatalogTableHelper.scala --- @@ -0,0 +1,138 @@ +/* + * Licens

[GitHub] flink pull request #3466: [FLINK-5715] Asynchronous snapshots for heap-based...

2017-03-10 Thread StephanEwen
Github user StephanEwen commented on a diff in the pull request: https://github.com/apache/flink/pull/3466#discussion_r105389424 --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FsStateBackend.java --- @@ -97,6 +100,27 @@ public FsStateBackend(String

[jira] [Commented] (FLINK-5715) Asynchronous snapshotting for HeapKeyedStateBackend

2017-03-10 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/FLINK-5715?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15905007#comment-15905007 ] ASF GitHub Bot commented on FLINK-5715: --- Github user StephanEwen commented on a diff

[jira] [Commented] (FLINK-5715) Asynchronous snapshotting for HeapKeyedStateBackend

2017-03-10 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/FLINK-5715?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15905015#comment-15905015 ] ASF GitHub Bot commented on FLINK-5715: --- Github user StefanRRichter commented on a d

[GitHub] flink pull request #3466: [FLINK-5715] Asynchronous snapshots for heap-based...

2017-03-10 Thread StefanRRichter
Github user StefanRRichter commented on a diff in the pull request: https://github.com/apache/flink/pull/3466#discussion_r105390074 --- Diff: flink-runtime/src/main/java/org/apache/flink/runtime/state/filesystem/FsStateBackend.java --- @@ -97,6 +100,27 @@ public FsStateBackend(Str

[GitHub] flink issue #3501: [FLINK-5874] Restrict key types in the DataStream API.

2017-03-10 Thread kl0u
Github user kl0u commented on the issue: https://github.com/apache/flink/pull/3501 Thanks for the thorough review @zentol . I integrated the comments that I agree with and commented on the rest. If you agree with the changes, I can merge it. --- If your project is set up for it, you

[jira] [Commented] (FLINK-5874) Reject arrays as keys in DataStream API to avoid inconsistent hashing

2017-03-10 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/FLINK-5874?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15905022#comment-15905022 ] ASF GitHub Bot commented on FLINK-5874: --- Github user kl0u commented on the issue:

[GitHub] flink issue #1668: [FLINK-3257] Add Exactly-Once Processing Guarantees for I...

2017-03-10 Thread senorcarbone
Github user senorcarbone commented on the issue: https://github.com/apache/flink/pull/1668 Hey @addisonj. Sure! You could perhaps review the changes and maybe see how to discard empty operator states if you are motivated. This is the only pending issue for this PR. thanks! ---

[jira] [Commented] (FLINK-3257) Add Exactly-Once Processing Guarantees in Iterative DataStream Jobs

2017-03-10 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/FLINK-3257?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15905044#comment-15905044 ] ASF GitHub Bot commented on FLINK-3257: --- Github user senorcarbone commented on the i

[jira] [Commented] (FLINK-3123) Allow setting custom start-offsets for the Kafka consumer

2017-03-10 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/FLINK-3123?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15905047#comment-15905047 ] ASF GitHub Bot commented on FLINK-3123: --- Github user tzulitai commented on the issue

[GitHub] flink issue #2687: [FLINK-3123] [kafka] Allow custom specific start offsets ...

2017-03-10 Thread tzulitai
Github user tzulitai commented on the issue: https://github.com/apache/flink/pull/2687 Rebased on the latest Kafka consumer changes in `master`. --- 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 hav

[jira] [Assigned] (FLINK-6023) Fix Scala snippet into Process Function (Low-level Operations) Doc

2017-03-10 Thread Andrea Sella (JIRA)
[ https://issues.apache.org/jira/browse/FLINK-6023?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ] Andrea Sella reassigned FLINK-6023: --- Assignee: (was: Andrea Sella) > Fix Scala snippet into Process Function (Low-level Operat

[GitHub] flink issue #3402: [FLINK-5890] [gelly] GatherSumApply broken when object re...

2017-03-10 Thread greghogan
Github user greghogan commented on the issue: https://github.com/apache/flink/pull/3402 @StephanEwen my last comment was ambiguous, I had originally modified a test and then with yesterday's commit reverted that change and added as a new test. Will merge. --- If your projec

[jira] [Commented] (FLINK-5890) GatherSumApply broken when object reuse enabled

2017-03-10 Thread ASF GitHub Bot (JIRA)
[ https://issues.apache.org/jira/browse/FLINK-5890?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15905072#comment-15905072 ] ASF GitHub Bot commented on FLINK-5890: --- Github user greghogan commented on the issu

[jira] [Commented] (FLINK-5808) Missing verification for setParallelism and setMaxParallelism

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

[GitHub] flink pull request #3509: [FLINK-5808] Fix Missing verification for setParal...

2017-03-10 Thread aljoscha
GitHub user aljoscha opened a pull request: https://github.com/apache/flink/pull/3509 [FLINK-5808] Fix Missing verification for setParallelism and setMaxParallelism You can merge this pull request into a Git repository by running: $ git pull https://github.com/aljoscha/flink

  1   2   3   >