Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id C85032009E8 for ; Mon, 30 May 2016 11:22:48 +0200 (CEST) Received: by cust-asf.ponee.io (Postfix) id C6EEC160A19; Mon, 30 May 2016 09:22:48 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id 984B0160A16 for ; Mon, 30 May 2016 11:22:47 +0200 (CEST) Received: (qmail 86223 invoked by uid 500); 30 May 2016 09:22:46 -0000 Mailing-List: contact user-help@flink.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: user@flink.apache.org Delivered-To: mailing list user@flink.apache.org Received: (qmail 86214 invoked by uid 99); 30 May 2016 09:22:46 -0000 Received: from mail-relay.apache.org (HELO mail-relay.apache.org) (140.211.11.15) by apache.org (qpsmtpd/0.29) with ESMTP; Mon, 30 May 2016 09:22:46 +0000 Received: from mail-ig0-f171.google.com (mail-ig0-f171.google.com [209.85.213.171]) by mail-relay.apache.org (ASF Mail Server at mail-relay.apache.org) with ESMTPSA id 57E1E1A0182 for ; Mon, 30 May 2016 09:22:46 +0000 (UTC) Received: by mail-ig0-f171.google.com with SMTP id ph1so9356571igb.1 for ; Mon, 30 May 2016 02:22:46 -0700 (PDT) X-Gm-Message-State: ALyK8tI6q0ON11ewDx+p0X90Wz+2W2ixleUjWcUCkqC9wz0Zp2PP8UXPzac+QRWckkLS/DK1jDu5S6KHT3na+g== X-Received: by 10.50.167.34 with SMTP id zl2mr7126867igb.49.1464600165192; Mon, 30 May 2016 02:22:45 -0700 (PDT) MIME-Version: 1.0 References: <20882204.1634631464161178776.JavaMail.root@correo> <9408737.1634771464161339122.JavaMail.root@correo> In-Reply-To: From: Aljoscha Krettek Date: Mon, 30 May 2016 09:22:35 +0000 X-Gmail-Original-Message-ID: Message-ID: Subject: Re: Incremental updates To: user@flink.apache.org Content-Type: multipart/alternative; boundary=485b393aad8795e17105340bcc29 archived-at: Mon, 30 May 2016 09:22:49 -0000 --485b393aad8795e17105340bcc29 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: quoted-printable Hi, the state will be kept indefinitely but we are planning to introduce a setting that would allow setting a time-to-live on state. I think this is exactly what you would need. As an alternative, maybe you could implement your program using windows? In this way you would also bound how long state is kept. As for using ValueState, here's a short example: DataStream stream =3D ...; KeyedStream keyedStream =3D stream.keyBy("id"); keyedStream.map(new RichMapFunction>() { private ValueState count; public void open(Configuration cfg) { state =3D getRuntimeContext().getState( new ValueStateDescriptor("count", LongSerializer.INSTANCE, 0L)); } public Tuple2 map(MyType value) { long count =3D state.value() + 1; state.update(value); return new Tuple2<>(value, count); } }); Best, Aljoscha On Fri, 27 May 2016 at 18:59 Malgorzata Kudelska wrote: > Hi, > If I specify the userId as the key variable as you suggested, will the > state variables be kept for every observed value of the key? I have a > situation where I have a lot of userIds and many of them occure just once= , > so I don't want to keep the state for them for ever. I need the possibili= ty > to set a timeout to forget the data regarding users that don't produce an= y > events for a certain amount if time. Is that possible with flink? > In order to add some custom information for every userId to the > checkpointed state, do you suggest to make a ValueState variable for a > stream keyed by userId. If yes, could you give some example? > > Cheers, > Gosia > Hi, > newly added nodes would sit idle, yes. Only when we finish the rescaling > work mentioned in the link will we be able to dynamically adapt. > > The internal implementation of this will in fact hash keys to a larger > number of partitions than the number of individual partitions and use the= se > "key groups" to allows scaling to differing numbers of partitions. Once > this is in it will also work on Yarn. Right now, running on Yarn does not > allow a job to dynamically pick up new computing resources. > > Cheers, > Aljoscha > > On Thu, 26 May 2016 at 15:50 Malgorzata Kudelska > wrote: > >> Hi, >> So is there any possibility to utilize an extra node that joins the >> cluster or will it remain idle? >> What if I use a custom key function that matches the key variable to a >> number of keys bigger than the initial number of nodes (following the id= ea >> from your link)? >> What about running flink on yarn, would that solve anything? >> >> Cheers, >> Gosia >> 25 maj 2016 22:54 "Aljoscha Krettek" napisa=C5=82(= a): >> >>> Hi, >>> first question: are you manually keying by "userId % >>> numberOfPartitions"? Flink internally does roughly "key.hash() % >>> numPartitions" so it is enough to specify the userId as your key. >>> >>> Now, for you questions: >>> >>> 1. What Flink guarantees is that the state for a key k is always >>> available when an element with key k is being processed. Internally, th= is >>> means that elements with the same key will be processed by the same >>> partition, though there would be other ways of achieving those guarante= es. >>> >>> 2. Right now, when a node disappears the job will fail. Then recovery >>> will kick in and restore from the latest checkpoint on a (possibly) new= set >>> of nodes. The system will make sure that the partitions and the state a= re >>> correctly matched. >>> >>> 3. Also answered by the above, I hope at least :-) >>> >>> 4. This does currently not work but the ongoing work in this is tracked >>> by https://issues.apache.org/jira/browse/FLINK-3755. >>> >>> Cheers, >>> Aljoscha >>> >>> On Wed, 25 May 2016 at 21:09 Malgorzata Kudelska >>> wrote: >>> >>>> Hi, >>>> I have the following situation. >>>> - a keyed stream with a key defined as: userId % numberOfPartitions >>>> - a custom flatMap transformation where I use a StateValue variable to >>>> keep the state of some calculations for each userId >>>> - my questions are: >>>> 1. Does flink guarantee that the users with a given key will be always >>>> processed by the same partition assuming that the number of nodes is >>>> constant? >>>> 2. What will happen when one node disapears or a new one joins? How >>>> will flink redistribute the users that were processed by the one that >>>> disapeared? >>>> 3. Will flink restore the state variables of these users from the last >>>> checkpoint and redistribute them to the new processing nodes? >>>> 4. How will flink redistribute the worload when a new node joins? >>>> >>>> Cheers, >>>> Gosia >>>> Hi, >>>> right now, this does not work but we're is also actively working on >>>> that. This is the design doc for part one of the necessary changes: >>>> https://docs.google.com/document/d/1G1OS1z3xEBOrYD4wSu-LuBCyPUWyFd9l3T= 9WyssQ63w/edit?usp=3Dsharing >>>> >>>> Cheers, >>>> Aljoscha >>>> >>>> On Wed, 25 May 2016 at 13:32 Malgorzata Kudelska < >>>> m.j.kudelska@gmail.com> wrote: >>>> >>>>> Hi, >>>>> Thanks for your reply. >>>>> >>>>> Is Flink able to detect that an additional server joined and rebalanc= e >>>>> the processing? How is it done if I have a keyed stream and some cust= om >>>>> ValueState variables? >>>>> >>>>> Cheers, >>>>> Gosia >>>>> >>>>> 2016-05-25 11:32 GMT+02:00 Aljoscha Krettek : >>>>> >>>>>> Hi Gosia, >>>>>> right now, Flink is not doing incremental checkpoints. Every >>>>>> checkpoint is fully valid in isolation. Incremental checkpointing ca= me up >>>>>> several times on ML discussions and we a planning to work on it once >>>>>> someone finds some free time. >>>>>> >>>>>> Cheers, >>>>>> Aljoscha >>>>>> >>>>>> On Wed, 25 May 2016 at 09:29 Rub=C3=A9n Casado >>>>>> wrote: >>>>>> >>>>>>> Hi Gosia >>>>>>> >>>>>>> You can have a look to the PROTEUS project we are doing [1]. We are >>>>>>> implementing incremental version of analytics operations. For examp= le you >>>>>>> can see in [2] the implementation of the incremental AVG. Maybe the= code >>>>>>> can give you some ideas :-) >>>>>>> >>>>>>> >>>>>>> [1] >>>>>>> https://github.com/proteus-h2020/proteus-backend/tree/development >>>>>>> [2] >>>>>>> https://github.com/proteus-h2020/proteus-backend/blob/development/s= rc/main/java/com/treelogic/proteus/flink/incops/IncrementalAverage.java >>>>>>> >>>>>>> ______________________________________ >>>>>>> >>>>>>> *Dr. Rub=C3=A9n Casado* >>>>>>> Head of Big Data >>>>>>> Treelogic >>>>>>> >>>>>>> *ruben.casado.treelogic* >>>>>>> >>>>>>> +34 902 286 386 - +34 607 18 28 06 >>>>>>> Parque Tecnol=C3=B3gico de Asturias =C2=B7 Parcela 30 >>>>>>> E33428 Llanera =C2=B7 Asturias [Spain] >>>>>>> www.treelogic.com >>>>>>> ______________________________________ >>>>>>> >>>>>>> >>>>>>> ----- Mensaje original ----- >>>>>>> De: "Malgorzata Kudelska" >>>>>>> Para: user@flink.apache.org >>>>>>> Enviados: Martes, 24 de Mayo 2016 22:01:28 GMT +01:00 Amsterdam / >>>>>>> Berl=C3=ADn / Berna / Roma / Estocolmo / Viena >>>>>>> Asunto: Incremental updates >>>>>>> >>>>>>> >>>>>>> Hi, >>>>>>> I have the following question. Does Flink support incremental >>>>>>> updates? >>>>>>> >>>>>>> In particular, I have a custom StateValue object and during the >>>>>>> checkpoints I would like to save only the fields that changed since= the >>>>>>> previous checkpoint. Is that possible? >>>>>>> >>>>>>> Regards, >>>>>>> Gosia >>>>>>> >>>>>> >>>>> --485b393aad8795e17105340bcc29 Content-Type: text/html; charset=UTF-8 Content-Transfer-Encoding: quoted-printable
Hi,
the state will be kept indefinitely but we are pla= nning to introduce a setting that would allow setting a time-to-live on sta= te. I think this is exactly what you would need. As an alternative, maybe y= ou could implement your program using windows? In this way you would also b= ound how long state is kept.

As for using ValueSta= te, here's a short example:

DataStream<= ;MyType> stream =3D ...;
KeyedStream<MyType> keyedStream= =3D stream.keyBy("id");

keyedStream.map= (new RichMapFunction<MyType, Tuple2<MyType, Long>>() {

=C2=A0 =C2=A0 private ValueState<Long> count;
<= div>
=C2=A0 =C2=A0 public void open(Configuration cfg) {
=C2=A0 =C2=A0 =C2=A0 =C2=A0 state =3D getRuntimeContext().getState(
=C2=A0 =C2=A0 =C2=A0 =C2=A0 =C2=A0 =C2=A0 =C2=A0 =C2=A0 new ValueS= tateDescriptor<Long>("count", LongSerializer.INSTANCE, 0L))= ;
=C2=A0 =C2=A0 }

=C2=A0 =C2=A0 public T= uple2<MyType, Long> map(MyType value) {
=C2=A0 =C2=A0 =C2= =A0 =C2=A0 long count =3D state.value() + 1;
=C2=A0 =C2=A0 =C2=A0= =C2=A0 state.update(value);
=C2=A0 =C2=A0 =C2=A0 =C2=A0 return n= ew Tuple2<>(value, count);
=C2=A0 =C2=A0 }
});

Best,
Aljoscha

On Fri, 27 May 2016 at 18:59 Malgorza= ta Kudelska <m.j.kudelska@gmai= l.com> wrote:

Hi,
If I specify the userId as the key variable as you suggested, will the stat= e variables be kept for every observed value of the key? I have a situation= where I have a lot of userIds and many of them occure just once, so I don&= #39;t want to keep the state for them for ever. I need the possibility to s= et a timeout to forget the data regarding users that don't produce any = events for a certain amount if time. Is that possible with flink?
In order to add some custom information for every userId to the checkpointe= d state, do you suggest to make a ValueState variable for a stream keyed by= userId. If yes, could you give some example?

Cheers,
Gosia

Hi,
newly added nodes would sit idle, yes. Only when= we finish the rescaling work mentioned in the link will we be able to dyna= mically adapt.

The internal implementation of this= will in fact hash keys to a larger number of partitions than the number of= individual partitions and use these "key groups" to allows scali= ng to differing numbers of partitions. Once this is in it will also work on= Yarn. Right now, running on Yarn does not allow a job to dynamically pick = up new computing resources.

Cheers,
Aljo= scha

On Thu, 26 = May 2016 at 15:50 Malgorzata Kudelska <m.j.kudelska@gmail.com> wrote:
<= blockquote class=3D"gmail_quote" style=3D"margin:0 0 0 .8ex;border-left:1px= #ccc solid;padding-left:1ex">

Hi,
So is there any possibility to utilize an extra node that joins the cluster= or will it remain idle?
What if I use a custom key function that matches the key variable to a numb= er of keys bigger than the initial number of nodes (following the idea from= your link)?
What about running flink on yarn, would that solve anything?

Cheers,
Gosia

25 maj 2016 22:54 "Aljoscha Krettek" &= lt;aljoscha@apache= .org> napisa=C5=82(a):
Hi,
first question: are you manually keying = by "userId % numberOfPartitions"? Flink internally does roughly &= quot;key.hash() % numPartitions" so it is enough to specify the userId= as your key.

Now, for you questions:
1. What Flink guarantees is that the state for a key k is alwa= ys available when an element with key k is being processed. Internally, thi= s means that elements with the same key will be processed by the same parti= tion, though there would be other ways of achieving those guarantees.
=

2. Right now, when a node disappears the job will fail.= Then recovery will kick in and restore from the latest checkpoint on a (po= ssibly) new set of nodes. The system will make sure that the partitions and= the state are correctly matched.

3. Also answered= by the above, I hope at least :-)

4. This does cu= rrently not work but the ongoing work in this is tracked by=C2=A0https:= //issues.apache.org/jira/browse/FLINK-3755.

Ch= eers,
Aljoscha

On Wed, 25 May 2016 at 21:09 Malgorzata Kudelska <m.j.kudelska@gmail.com&g= t; wrote:

Hi,
I have the following situation.
- a keyed stream with a key defined as: userId % numberOfPartitions
- a custom flatMap transformation where I use a StateValue variable to keep= the state of some calculations for each userId
- my questions are:
1. Does flink guarantee that the users with a given key will be always proc= essed by the same partition assuming that the number of nodes is constant?<= br> 2. What will happen when one node disapears or a new one joins?=C2=A0 How w= ill flink redistribute the users that were processed by the one that disape= ared?
3. Will flink restore the state variables of these users from the last chec= kpoint and redistribute them to the new processing nodes?
4. How will flink redistribute the worload when a new node joins?

Cheers,
Gosia

Hi,
right now, this does not work but we're is a= lso actively working on that. This is the design doc for part one of the ne= cessary changes:=C2=A0https://docs.google.com/document/d/1G1OS1z3xEBOrYD4wSu-LuBCyPUWyFd9l3T9Wys= sQ63w/edit?usp=3Dsharing

Cheers,
Alj= oscha

On Wed, 25 May 2= 016 at 13:32 Malgorzata Kudelska <m.j.kudelska@gmail.com> wrote:
Hi,
Thanks for your reply.

Is Flink able to detect that an additional server jo= ined and rebalance the processing? How is it done if I have a keyed stream = and some custom ValueState variables?

Cheers,=C2= =A0
Gosia

2016-05-25 11:32 GMT+02:00 Aljoscha Krettek <aljoscha@apac= he.org>:
H= i Gosia,
right now, Flink is not doing incremental checkpoints. Every c= heckpoint is fully valid in isolation. Incremental checkpointing came up se= veral times on ML discussions and we a planning to work on it once someone = finds some free time.

Cheers,
Aljoscha

On Wed,= 25 May 2016 at 09:29 Rub=C3=A9n Casado <ruben.casado@treelogic.com> wrote:<= br>
Hi Gosia

You can have a look to the = PROTEUS project we are doing [1]. We are implementing incremental version o= f analytics operations. For example you can see in [2] the implementation o= f the incremental AVG. Maybe the code can give you some ideas :-)

[1] https://github.com/proteus-h2020/proteus-backend= /tree/development
[2] https://github.com/proteus-= h2020/proteus-backend/blob/development/src/main/java/com/treelogic/proteus/= flink/incops/IncrementalAverage.java

= ______________________________________
Dr. Rub=C3=A9n Casado
Head of Big Da= ta
Treelogic
<= font face=3D"arial, helvetica, sans-serif" color=3D"#3366FF">ruben.casado.treelogic

+34 902 28= 6 386 - +34 607 18 28 06

Parque Tecnol=C3=B3gico de Asturias =C2=B7 Parcela 30
E33428 Llanera =C2=B7 Asturias [Spain]
www.treelogic.com____= __________________________________
=


----- Mensaje original -----
De: "M= algorzata Kudelska" <m.j.kudelska@gmail.com>
Para: user@flink.apache.org
Enviados= : Martes, 24 de Mayo 2016 22:01:28 GMT +01:00 Amsterdam / Berl=C3=ADn / Ber= na / Roma / Estocolmo / Viena
Asunto: Incremental updates


Hi,
I have the following question. Does Flink support incremental updates?

In particular, I have a custom StateValue object and during = the checkpoints I would like to save only the fields that changed since the= previous checkpoint. Is that possible?

Regards,
Gosia


--485b393aad8795e17105340bcc29--