Re: Custom TimestampExtractor and FlinkKafkaConsumer082

2015-12-11 Thread Aljoscha Krettek
The PRs are merged. :D > On 11 Dec 2015, at 17:28, Stephan Ewen wrote: > > A solution for that is in these two pull requests: > > https://github.com/apache/flink/pull/1447 > > https://github.com/apache/flink/pull/1448 > > On Fri, Dec 4, 2015 at 10:21 PM, Robert Metzger wrote: > I think we nee

Re: Custom TimestampExtractor and FlinkKafkaConsumer082

2015-12-11 Thread Stephan Ewen
A solution for that is in these two pull requests: https://github.com/apache/flink/pull/1447 https://github.com/apache/flink/pull/1448 On Fri, Dec 4, 2015 at 10:21 PM, Robert Metzger wrote: > I think we need to find a solution for this problem soon. > Another user is most likely affected: > ht

streaming state

2015-12-11 Thread Alex Rovner
Hello all, I was wondering if someone would be kind enough to enlighten me on a few topics. We are trying to join two streams of data on a key. We were thinking of partitioning topics in Kafka by the key, however I also saw that Flink is able to partition on its own and I was wondering whether Fli

S3 Input/Output with temporary credentials (IAM Roles)

2015-12-11 Thread Vladimir Stoyak
Our setup involves AWS IAM roles when with permanent access_key and access_secret we need to assume specific role (ie getting temporary credentials to use AWS resources). I was wondering what would be the best way handling this, ie how to set  fs.s3n.awsAccessKeyId and fs.s3n.awsSecretAccessKey p

Re: Getting two types of events from a Window (Trigger)?

2015-12-11 Thread Stephan Ewen
You can strip the parts that accumulate the data for writing the files, then it becomes a very slim example to build on top of... On Fri, Dec 11, 2015 at 3:20 PM, Niels Basjes wrote: > I'll have an other look at the example code you sent me. > Thanks. > > On Fri, Dec 11, 2015 at 3:00 PM, Stephan

Re: Getting two types of events from a Window (Trigger)?

2015-12-11 Thread Niels Basjes
I'll have an other look at the example code you sent me. Thanks. On Fri, Dec 11, 2015 at 3:00 PM, Stephan Ewen wrote: > Hi Niels! > > Quick question (I am probably overlooking something here) - if you simply > want to emit each element (Trigger onElement) in addition to a special data > stream,

Re: Getting two types of events from a Window (Trigger)?

2015-12-11 Thread Niels Basjes
I sort of expected this to be the case. Thanks, for confirming. Niels On Fri, Dec 11, 2015 at 1:59 PM, Aljoscha Krettek wrote: > Hi Niels, > I’m afraid this will not work. (If I understood correctly what you are > trying to do.) When the trigger is being serialized/deserialized each > parallel

Re: Getting two types of events from a Window (Trigger)?

2015-12-11 Thread Stephan Ewen
Hi Niels! Quick question (I am probably overlooking something here) - if you simply want to emit each element (Trigger onElement) in addition to a special data stream, can you not simply have two operators that consume the original data stream: The window operator and the additional source. If yo

Re: Getting two types of events from a Window (Trigger)?

2015-12-11 Thread Aljoscha Krettek
Hi Niels, I’m afraid this will not work. (If I understood correctly what you are trying to do.) When the trigger is being serialized/deserialized each parallel instance of the trigger has their own copy of the QueueSource object. Plus, a separate instance of the QueueSource itself will be runnin

Re: Getting two types of events from a Window (Trigger)?

2015-12-11 Thread Niels Basjes
Hi, Just to let you know: I tried passing a SourceFunction but I haven't been able to get that to work (yet). I passed an instance of this (see code below) into my Trigger and stored it there as: private QueueSource output; and then I called from the onElement something like: output.put("F

Re: Hello a question about metrics

2015-12-11 Thread Philip Lee
Thanks for your suggestion! I will try it later! My group member want to use linux application like cpustat, memstat, vmstat. The point is that running on spark and flink is on JVM, right? FYI, cpustat and memstat capture the hardware resource, not virtual mahcine. Do you think it could be approp

Re: Getting two types of events from a Window (Trigger)?

2015-12-11 Thread Niels Basjes
Thanks. The way I solved it now is by creating a class that persists data into something external (right now HBase and/or Kafka) and use that from the trigger to output the data. I have two followup questions: 1) Is it possible to pass an instance of 'SourceFunction' as such a parameter (without