Re: [DISCUSS] Not marking Jira issues as resolved in 1.5.0 as resolved in 1.6.0

2018-03-14 Thread Chesnay Schepler
+1 to mark bugs as fixed in 1.5.0 only. On 15.03.2018 01:40, Aljoscha Krettek wrote: Hi, We currently have some issues that are marked as resolved for both 1.5.0 and 1.6.0 [1]. The reason is that we have the release-1.5 branch and the master branch, which will eventually become the branch for

Re: Flip 6 mesos support

2018-03-14 Thread Renjie Liu
Thanks for the clarification On Thu, Mar 15, 2018 at 2:30 PM 周思华 wrote: > Hi Renjie, > if I am not misunderstand, you just need to start the cluster as normal as > before. The dispatcher and resourcemanager are spawned by ClusterEntryPoint > (you can have a look at yarn-session.sh & FlinkYarnSes

Re: Flip 6 mesos support

2018-03-14 Thread 周思华
Hi Renjie, if I am not misunderstand, you just need to start the cluster as normal as before. The dispatcher and resourcemanager are spawned by ClusterEntryPoint (you can have a look at yarn-session.sh & FlinkYarnSessionCli & YarnSessionClusterEntrypoint), and the TM are spawned by ResourceManag

[jira] [Created] (FLINK-8946) TaskManager stop sending metrics after JobManager failover

2018-03-14 Thread Truong Duc Kien (JIRA)
Truong Duc Kien created FLINK-8946: -- Summary: TaskManager stop sending metrics after JobManager failover Key: FLINK-8946 URL: https://issues.apache.org/jira/browse/FLINK-8946 Project: Flink

Re: Flip 6 mesos support

2018-03-14 Thread Renjie Liu
Hi, Till: In fact I'm asking how to deploy other components such as dispatcher, etc. Till Rohrmann 于 2018年3月15日周四 上午12:17写道: > Hi Renjie, > > in the current master and release-1.5 branch flip-6 is activated by > default. If you want to turn it off you have to add `mode: old` to your > flink-conf

Re: Correct way to reference Hadoop dependencies in Flink 1.4.0

2018-03-14 Thread lrao
Hi, I am running this on a Hadoop-free cluster (i.e. no YARN etc.). I have the following dependencies packaged in my user application JAR: aws-java-sdk 1.7.4 flink-hadoop-fs 1.4.0 flink-shaded-hadoop2 1.4.0 flink-connector-filesystem_2.11 1.4.0 hadoop-common 2.7.4 hadoop-aws 2.7.4 I have also t

[DISCUSS] Not marking Jira issues as resolved in 1.5.0 as resolved in 1.6.0

2018-03-14 Thread Aljoscha Krettek
Hi, We currently have some issues that are marked as resolved for both 1.5.0 and 1.6.0 [1]. The reason is that we have the release-1.5 branch and the master branch, which will eventually become the branch for 1.6.0. I think this can lead to confusion because the release notes are created based

Re: Flip 6 mesos support

2018-03-14 Thread Shuyi Chen
Hi Till, have we tested the YARN kerberos integration in flip6? AFAI remember, YARNSessionFIFOSecuredITCase is not functioning (FLINK-8562 ), do we have similar integration test for flip6? Also, Flink yarn kerberos integration in the old deployment

Re: [VOTE] Release 1.3.3, release candidate #2

2018-03-14 Thread Ted Yu
+1 Ran the following command - passed: mvn clean package -Pjdk8 On Wed, Mar 14, 2018 at 3:26 AM, Tzu-Li (Gordon) Tai wrote: > Hi everyone, > > Please review and vote on release candidate #2 for Flink 1.3.3, as > follows: > [ ] +1, Approve the release > [ ] -1, Do not approve the release (pleas

Re: [Proposal] CEP library changes - review request

2018-03-14 Thread Aljoscha Krettek
Hi, I think this should have been sent to the dev mailing list because in the user mailing list it might disappear among a lot of other mail. Forwarding... Best, Aljoscha > On 14. Mar 2018, at 06:20, Shailesh Jain wrote: > > Hi, > > We've been facing issues* w.r.t watermarks not supported p

Re: Correct way to reference Hadoop dependencies in Flink 1.4.0

2018-03-14 Thread Francesco Ciuci
Hi, You do not just need the hadoop dependencies in the jar but you need to have the hadoop file system running in your machine/cluster. Regards On 14 March 2018 at 18:38, l...@lyft.com wrote: > I'm trying to use a BucketingSink to write files to S3 in my Flink job. > > I have the Hadoop depen

Correct way to reference Hadoop dependencies in Flink 1.4.0

2018-03-14 Thread lrao
I'm trying to use a BucketingSink to write files to S3 in my Flink job. I have the Hadoop dependencies I need packaged in my user application jar. However, on running the job I get the following error (from the taskmanager): java.lang.RuntimeException: Error while creating FileSystem when initia

Re: Flip 6 mesos support

2018-03-14 Thread Till Rohrmann
Hi Renjie, in the current master and release-1.5 branch flip-6 is activated by default. If you want to turn it off you have to add `mode: old` to your flink-conf.yaml. I'm really happy that you want to test it out :-) Cheers, Till On Wed, Mar 14, 2018 at 3:03 PM, Renjie Liu wrote: > Hi Till: >

[jira] [Created] (FLINK-8945) Allow customization of the KinesisProxy Interface

2018-03-14 Thread Kailash Hassan Dayanand (JIRA)
Kailash Hassan Dayanand created FLINK-8945: -- Summary: Allow customization of the KinesisProxy Interface Key: FLINK-8945 URL: https://issues.apache.org/jira/browse/FLINK-8945 Project: Flink

[jira] [Created] (FLINK-8944) Use ListShards for shard discovery in the flink kinesis connector

2018-03-14 Thread Kailash Hassan Dayanand (JIRA)
Kailash Hassan Dayanand created FLINK-8944: -- Summary: Use ListShards for shard discovery in the flink kinesis connector Key: FLINK-8944 URL: https://issues.apache.org/jira/browse/FLINK-8944 P

[jira] [Created] (FLINK-8943) Jobs will not recover if DFS is temporarily unavailable

2018-03-14 Thread Gary Yao (JIRA)
Gary Yao created FLINK-8943: --- Summary: Jobs will not recover if DFS is temporarily unavailable Key: FLINK-8943 URL: https://issues.apache.org/jira/browse/FLINK-8943 Project: Flink Issue Type: Bug

Re: Flip 6 mesos support

2018-03-14 Thread Renjie Liu
Hi Till: Is there any doc on deploying flink in flip6 mode? We want to help testing it. Till Rohrmann 于 2018年3月14日周三 下午7:08写道: > Hi Renjie, > > in order to make Mesos work, we only needed to implement a Mesos specific > ResourceManager. Look at MesosResourceManager for more details. As > dispatc

[jira] [Created] (FLINK-8942) Pass targer ResourceID to HeartbeatListener#retrievePayload

2018-03-14 Thread Chesnay Schepler (JIRA)
Chesnay Schepler created FLINK-8942: --- Summary: Pass targer ResourceID to HeartbeatListener#retrievePayload Key: FLINK-8942 URL: https://issues.apache.org/jira/browse/FLINK-8942 Project: Flink

[jira] [Created] (FLINK-8941) SpanningRecordSerializationTest fails on Travis

2018-03-14 Thread Chesnay Schepler (JIRA)
Chesnay Schepler created FLINK-8941: --- Summary: SpanningRecordSerializationTest fails on Travis Key: FLINK-8941 URL: https://issues.apache.org/jira/browse/FLINK-8941 Project: Flink Issue Typ

[jira] [Created] (FLINK-8940) Implement JobMaster#disposeSavepoint

2018-03-14 Thread Chesnay Schepler (JIRA)
Chesnay Schepler created FLINK-8940: --- Summary: Implement JobMaster#disposeSavepoint Key: FLINK-8940 URL: https://issues.apache.org/jira/browse/FLINK-8940 Project: Flink Issue Type: Improvem

Re: flink web ui authentication

2018-03-14 Thread Diego Reiriz Cores
Hi Sampath, You can use nginx to protect the flink dashboard as David Anderson answered you on stackoverflow: https://stackoverflow.com/questions/49254307/flink-web-ui-authentication Here you got a guide from the guys of DigitalOcean about protecting an url with basic authentication: https://www.

Re: flink web ui authentication

2018-03-14 Thread Till Rohrmann
Hi Sampath, at the moment, Flink does not support such a feature. Cheers, Till On Tue, Mar 13, 2018 at 11:17 AM, Sampath Bhat wrote: > Hello > > I would like to know if flink supports any user level authentication like > username/password for flink web ui. > > Regards > Sampath S >

Re: Flip 6 mesos support

2018-03-14 Thread Till Rohrmann
Hi Renjie, in order to make Mesos work, we only needed to implement a Mesos specific ResourceManager. Look at MesosResourceManager for more details. As dispatcher, we use the StandaloneDispatcher which is spawned by the MesosSessionClusterEntrypoint. Cheers, Till On Wed, Mar 14, 2018 at 9:32 AM,

[VOTE] Release 1.3.3, release candidate #2

2018-03-14 Thread Tzu-Li (Gordon) Tai
Hi everyone, Please review and vote on release candidate #2 for Flink 1.3.3, as follows:   [ ] +1, Approve the release   [ ] -1, Do not approve the release (please provide specific comments) The complete staging area is available for your review, which includes:   * JIRA release notes [1],   * th

Re: Starting open source development with Flink

2018-03-14 Thread Fabian Hueske
Hi Deepak, You can open JIRAs for bugs you discovered or minor improvements. Larger features should be discussed on the dev mailing list first. I'd suggest to start contributing by fixing a bug. Best, Fabian 2018-03-13 3:39 GMT+01:00 Deepak Sharma : > Hi Flink team! > > I've been using Flink at

Flip 6 mesos support

2018-03-14 Thread Renjie Liu
Hi all: I'm reading the source code and it seems that flip6 does not support mesos? According to the design, client send job graph to dispatcher and dispatcher spawn job mananger and resource manager for job execution. But I can't find dispatcher implementation for mesos. -- Liu, Renjie Software E

[jira] [Created] (FLINK-8939) Provide better support for saving streaming data to s3

2018-03-14 Thread chris snow (JIRA)
chris snow created FLINK-8939: - Summary: Provide better support for saving streaming data to s3 Key: FLINK-8939 URL: https://issues.apache.org/jira/browse/FLINK-8939 Project: Flink Issue Type: Im