We have also noticed such behaviour when running on Yarn, had to restart
the session for the changes in the jar to be picked up.
On Mon, 31 Jul 2017 at 17:13, Ufuk Celebi wrote:
> Hey Mike!
>
> Thanks for the detailed information about your setup. I'm also puzzled
> by this...
>
> (1) Which vers
Vishnu Viswanath created FLINK-7299:
---
Summary: Write GenericRecord using AvroOutputFormat
Key: FLINK-7299
URL: https://issues.apache.org/jira/browse/FLINK-7299
Project: Flink
Issue Type
etreff: Re: AVRO Union type support in Flink
> Datum: Wed, 19 Jul 2017 10:26:24 -0400
> Von: Vishnu Viswanath
>
> An: Timo Walther
>
>
> Hi Timo,
>
> Thanks for checking that. I did not try yet. My current application uses
> Cascading and it has the limitati
vishnu viswanath created FLINK-6726:
---
Summary: Allow setting Timers in ProcessWindowFunction
Key: FLINK-6726
URL: https://issues.apache.org/jira/browse/FLINK-6726
Project: Flink
Issue Type
vishnu viswanath created FLINK-6372:
---
Summary: change-scala-version.sh does not change version for
flink-gelly-examples
Key: FLINK-6372
URL: https://issues.apache.org/jira/browse/FLINK-6372
Project
Hi All,
Regarding the scale up and down feature mentioned in this talk:
https://www.youtube.com/watch?v=IHSMnlWXkZ4&list=PLDX4T_cnKjD2UC6wJr_wRbIvtlMtkc-n2&index=18
Can this only be done when restarting the job from a save point? Is there a
case where the state is transferred over from one node t
ting-in-
> Flink-td15931.html
> > @Stefan Richter
> > <https://issues.apache.org/jira/secure/ViewProfile.jspa?name=srichter>
> > and @Xiaogang
> > Shi
> > <https://issues.apache.org/jira/secure/ViewProfile.jspa?
> name=xiaogang.shi>
> > might
Hi,
Can someone point me to the branch where the ongoing work for incremental
checkpoint is going on, I would like to try it out even if the work is not
complete.
I have a use case where the state size increase about ~1gb every 5 minutes.
Thanks,
Vishnu
Works for me also.
On Fri, Nov 18, 2016 at 12:35 PM, Stephan Ewen wrote:
> Just checked it, the link works for me.
>
>
> On Fri, Nov 18, 2016 at 7:20 PM, amir bahmanyari
> wrote:
>
>> [image: Inline image]
>>
>>
>> --
>> *From:* Kostas Tzoumas
>> *To:* "dev@flink.ap
nal class that should not really be user
> facing, that's my motivation for replacing it.
>
> Cheers,
> Aljoscha
>
> On Mon, 17 Oct 2016 at 19:23 Vishnu Viswanath <
> vishnu.viswanat...@gmail.com>
> wrote:
>
> > Hi Aljoscha,
> >
> > Thanks for t
rely on tuples because they can be confusing for
> people who write Scala code because they are not Scala tuples. That's not
> strictly necessary, though, you can spin it however you like.
>
> Cheers,
> Aljoscha
>
> On Fri, 7 Oct 2016 at 18:46 Vishnu Viswanath >
>
is captured in this interface. Basically, will
> the typical remove() method from Iterators be available?
>
> Best regards,
>
>
> -----Original Message-
> From: Vishnu Viswanath [mailto:vishnu.viswanat...@gmail.com]
> Sent: Friday, October 07, 2016 8:29 AM
> To: Dev
&
Record. In case timestamp is not set,
TimeEvictor can capture this by checking the Tuple2.f0 (which will be
Long.MIN_VALUE) and ignore the eviction.
If you think this is fine, I will make the changes and also edit the FLIP.
Regards,
Vishnu
On Wed, Oct 5, 2016 at 9:49 PM, Vishnu Viswanath &l
meEvictor())
> .apply(...)
>
> With this, we would just have to find a good way of passing the timestamps
> in the Evictor interface and a good way of implementing the
> EvictingWindowOperator.
>
> Cheers,
> Aljoscha
>
>
> On Sun, 18 Sep 2016 at 18:14 Vishnu Viswanat
from the type of the Iterable that is
> passed to the evictor to avoid code duplication in EvictingWindowOperator)
> I'm open for suggestions there since I didn't come up with a better
> solution yet. :-)
>
> Cheers,
> Aljoscha
>
>
>
> On Sat, 30 Jul 2016 at
erflow.com/documentation/apache-flink
Regards,
Vishnu Viswanath
code for the EvictingWindowOperator any more.
> > It could be a very thin subclass of WindowOperator.
> >
> > Cheers,
> > Aljoscha
> >
> > On Wed, 27 Jul 2016 at 03:56 Vishnu Viswanath <
> > vishnu.viswanat...@gmail.com> wrote:
>
> > This is very costly but the only way I see of doing this right now with
> > every state backend.
> >
> > Cheers,
> > Aljoscha
> >
> > On Mon, 25 Jul 2016 at 09:46 Radu Tudoran
> wrote:
> >
> > > Hi,
> > >
> > > Tha
dissemination) by persons other than the intended recipient(s) is
> prohibited. If you receive this e-mail in error, please notify the sender
> by phone or email immediately and delete it!
>
>
> -Original Message-
> From: Vishnu Viswanath [mailto:vishnu.viswanat..
Hi,
I have created a FLIP page for this enhancement
https://cwiki.apache.org/confluence/display/FLINK/FLIP-4+%3A+Enhance+Window+Evictor
Thanks,
Vishnu
On Thu, Jul 21, 2016 at 6:53 AM, Vishnu Viswanath <
vishnu.viswanat...@gmail.com> wrote:
> Thanks Aljoscha.
>
> On Thu, Jul 21,
e elements to the window function.
>
> This is very inefficient but the only way I see of doing it right now.
>
> Cheers,
> Aljoscha
>
>
> On Thu, 21 Jul 2016 at 01:32 Vishnu Viswanath <
> vishnu.viswanat...@gmail.com>
> wrote:
>
> > Hi,
> >
> >
;
> How are you starting your cluster? My guess is that you are running
> the cluster in local mode, which is not starting up the network
> components. Is that the case?
>
> – Ufuk
>
>
> On Thu, Jul 21, 2016 at 1:56 AM, Vishnu Viswanath
> wrote:
> > Hi,
> >
>
lse in the NetworkEnvironment.java. I am not sure how to proceed from
here, do I need to set some configs for QueryableState to work?
PS : I know this feature is not merged yet, but I was trying this out as
part of my POC, any help is appreciated.
Thanks,
Vishnu Viswanath
d add() methods
since it is an AppendingState, but that causes the evicted elements to come
back when the trigger is fired next time. (It works fine when I use
MemoryStateBackend)
Is this expected behavior or am I missing something.
Thanks,
Vishnu
On Mon, Jul 18, 2016 at 7:15 AM, Vishnu Visw
Hi Aljoscha,
Thanks! Yes, I have the create page option now in wiki.
Regards,
Vishnu Viswanath,
On Mon, Jul 18, 2016 at 6:34 AM, Aljoscha Krettek
wrote:
> @Radu, addition of more window types and sorting should be part of another
> design proposal. This is interesting stuff but I th
016 at 10:22 PM, Vishnu Viswanath <
vishnu.viswanat...@gmail.com> wrote:
> Hi Aljoscha,
>
> I agree, the user will know exactly that they are creating an EventTime
> based evictor or ProcessingTime based evictor looking at the code.
> So do you think it will be ok to have multipl
e is very
> important, in my opinion.
>
> On Wed, 13 Jul 2016 at 16:28 Vishnu Viswanath <
> vishnu.viswanat...@gmail.com>
> wrote:
>
> > Hi,
> >
> > I was hoping to use the isEventTime method in the WindowAssigner to set
> > that information in the Evic
Hi,
I was hoping to use the isEventTime method in the WindowAssigner to set
that information in the EvictorContext.
What do you think?.
Thanks and Regards,
Vishnu Viswanath,
On Wed, Jul 13, 2016 at 10:09 AM, Aljoscha Krettek
wrote:
> Hi,
> I think the way to go here is to add b
override the open and close in
EvitingWindowOperator to make the reference of EvictorContext null.
Thanks and Regards,
Vishnu Viswanath,
On Fri, Jul 8, 2016 at 7:40 PM, Vishnu Viswanath <
vishnu.viswanat...@gmail.com> wrote:
My thought process when asking if we can use state backend in
My thought process when asking if we can use state backend in window
function was : can we add the elements to be evicted into some state and
allow the evictAfter to read it from some context and remove it from the
window?
On Fri, Jul 8, 2016 at 7:30 PM, Vishnu Viswanath <
vishnu.viswa
7;t know how excatly the internals of these
work, so this might be a wrong question)
Thanks and Regards,
Vishnu Viswanath,
On Fri, Jul 8, 2016 at 10:20 AM, Aljoscha Krettek
wrote:
> Hi Vishnu,
> how long would these patterns be? The Trigger would not have to sort the
> elements for ever
the elements in the window everytime it sees an element. (I was
planning to do this sorting in the window, which will be less often - only
when the trigger fires)
Thanks and Regards,
Vishnu Viswanath,
On Fri, Jul 8, 2016 at 6:04 AM, Aljoscha Krettek
wrote:
Hi,
> come to think of it, th
for haven't arrived
yet) wait and try again when the trigger gets fired next time.
Thanks and Regards,
Vishnu Viswanath,
On Thu, Jul 7, 2016 at 9:19 AM, Radu Tudoran
wrote:
> Hi,
>
> @Aljoscha - I can understand the reason why you are hesitant to introduce
> "slower&quo
vishnu viswanath created FLINK-4174:
---
Summary: Enhance Window Evictor
Key: FLINK-4174
URL: https://issues.apache.org/jira/browse/FLINK-4174
Project: Flink
Issue Type: Sub-task
ld be good to have the window sorted
based on EventTime.
Thanks and Regards,
Vishnu Viswanath,
On Wed, Jul 6, 2016 at 3:55 PM, Maxim wrote:
> Actually for such evictor to be useful the window should be sorted by some
> field, usually event time. What do you think about adding so
apply function. Currently it
is before the apply function, but I have a use case where I need to first
apply the function and evict afterward.*
I would be interested in contributing to the code base. Please let me know
the steps.
Thanks and Regards,
Vishnu Viswanath
On Wed, Jul 6, 2016 at 11:
ld, but I would appreciate any suggestion on
whether these are viable changes or will there any performance issue if
these are done. Also any pointer on where to start(e.g, do I create a new
class similar to EvictingWindowOperator that extends WindowOperator?)
Thanks and Regards,
Vishnu Viswanath,
On W
>
> Best, Fabian
>
> 2016-06-16 13:18 GMT+02:00 Vishnu Viswanath >:
>
> > Hi,
> >
> > Will queryable state available in 1.1.0?
> >
> > Thanks,
> > Vishnu
> >
> > On Thursday, 16 June 2016, Aljoscha Krettek wrote:
> >
> &
>> - Kinesis Connector
> > >> - Persistent file sources for streaming
> > >>
> > >> Half merged:
> > >> - Resource manager refactoring
> > >>
> > >> Unmerged features:
> > >> - Cassandra connector
> > >
39 matches
Mail list logo