Return-Path: X-Original-To: apmail-flink-user-archive@minotaur.apache.org Delivered-To: apmail-flink-user-archive@minotaur.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id B259619679 for ; Thu, 7 Apr 2016 08:45:25 +0000 (UTC) Received: (qmail 51564 invoked by uid 500); 7 Apr 2016 08:45:25 -0000 Delivered-To: apmail-flink-user-archive@flink.apache.org Received: (qmail 51458 invoked by uid 500); 7 Apr 2016 08:45:25 -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 51449 invoked by uid 99); 7 Apr 2016 08:45:25 -0000 Received: from mail-relay.apache.org (HELO mail-relay.apache.org) (140.211.11.15) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 07 Apr 2016 08:45:25 +0000 Received: from mail-lb0-f170.google.com (mail-lb0-f170.google.com [209.85.217.170]) by mail-relay.apache.org (ASF Mail Server at mail-relay.apache.org) with ESMTPSA id B460D1A01D7 for ; Thu, 7 Apr 2016 08:45:24 +0000 (UTC) Received: by mail-lb0-f170.google.com with SMTP id vo2so45444322lbb.1 for ; Thu, 07 Apr 2016 01:45:24 -0700 (PDT) X-Gm-Message-State: AD7BkJJzg4gwHda4Eg2qVUsumq+0FtNMPAEgz5b4sbPdDsfXLFPJxeve/bh+SAgzfEIsDvgEFcdESzYZjd1k+Q== X-Received: by 10.112.160.35 with SMTP id xh3mr844941lbb.86.1460018723195; Thu, 07 Apr 2016 01:45:23 -0700 (PDT) MIME-Version: 1.0 References: In-Reply-To: From: Aljoscha Krettek Date: Thu, 07 Apr 2016 08:45:13 +0000 X-Gmail-Original-Message-ID: Message-ID: Subject: Re: Handling large state (incremental snapshot?) To: user@flink.apache.org Content-Type: multipart/alternative; boundary=001a11c33f105c9297052fe11947 --001a11c33f105c9297052fe11947 Content-Type: text/plain; charset=UTF-8 Ah yes, you're right. With the non-keyed stream it doesn't make a big difference because it's only one big state value. The throughput still seems quite low. Have you ever tried looking at the "back pressure" tab on the Flink dashboard. For this I would suggest to disable chaining, so that every operator is run in an isolated task: env.disableOperatorChaining(); On Thu, 7 Apr 2016 at 05:11 Hironori Ogibayashi wrote: > I tried RocksDB, but the result was almost the same. > > I used the following code and put 2.6million distinct records into Kafka. > After processing all records, the state on the HDFS become about 250MB > and time needed for > the checkpoint was almost 5sec. Processing throughput was > FsStateBackend-> 8000msg/sec, RocksDBSsateBackend -> 9000msg/sec > > --- > env.setStateBackend(new > RocksDBStateBackend("hdfs://:8020/apps/flink/checkpoints")); > > val stream = env > .addSource(new FlinkKafkaConsumer09[String]("kafka.json2", new > SimpleStringSchema(), properties)) > .map(parseJson(_)) > .timeWindowAll(Time.of(10, TimeUnit.DAYS)) > .trigger(MyContinuousProcessingTimeTrigger.of(Time.seconds(5))) > // count distinct values > .fold(Set[String]()){(r,i) => { r + i}} > .map{x => (System.currentTimeMillis(), x.size)} > .addSink(new ElasticsearchSink(config, transports, new > IndexRequestBuilder[Tuple2[Long, Int]] { > override def createIndexRequest(element: Tuple2[Long, Int], > ctx: RuntimeContext): IndexRequest = { > val json = new HashMap[String, AnyRef] > json.put("@timestamp", new Timestamp(element._1)) > json.put("count", element._2: java.lang.Integer) > > Requests.indexRequest.index("dummy3").`type`("my-type").source(json) > } > })) > --- > > I guess this is because I used non-keyed stream, so I had one state > record with a big value (all distinct value). > I think copying all 250MB(or more) file to HDFS in every checkpoint > will be heavy, so I will try storing the distinct values > in the external datastore (e.g. redis). > Also, when incremental snapshot get implemented, I want to try. > > Regards, > Hironori > > 2016-04-05 21:40 GMT+09:00 Hironori Ogibayashi : > > Aljoscha, > > > > Thank you for your quick response. > > Yes, I am using FsStateBackend, so I will try RocksDB backend. > > > > Regards, > > Hironori > > > > 2016-04-05 21:23 GMT+09:00 Aljoscha Krettek : > >> Hi, > >> I guess you are using the FsStateBackend, is that correct? You could try > >> using the RocksDB state backend: > >> > https://ci.apache.org/projects/flink/flink-docs-master/apis/streaming/state_backends.html#the-rocksdbstatebackend > >> > >> With this, throughput will be lower but the overhead per checkpoint > could be > >> lower. Also, with this most of the file copying necessary for the > checkpoint > >> will be done while data processing keeps running (asynchronous > snapshot). > >> > >> As to incremental snapshots. I'm afraid this feature is not yet > implemented > >> but we're working on it. > >> > >> Cheers, > >> Aljoscha > >> > >> On Tue, 5 Apr 2016 at 14:06 Hironori Ogibayashi > >> wrote: > >>> > >>> Hello, > >>> > >>> I am trying to implement windowed distinct count on a stream. In this > >>> case, the state > >>> have to hold all distinct value in the window, so can be large. > >>> > >>> In my test, if the state size become about 400MB, checkpointing takes > >>> 40sec and spends most of Taskmanager's CPU. > >>> Are there any good way to handle this situation? > >>> > >>> Flink document mentions about incremental snapshot, and I am > interested in > >>> it, > >>> but could not find how to enable it. (not implemented yet?) > >>> > >>> > https://ci.apache.org/projects/flink/flink-docs-release-1.0/internals/stream_checkpointing.html > >>> > >>> Regards, > >>> Hironori > --001a11c33f105c9297052fe11947 Content-Type: text/html; charset=UTF-8 Content-Transfer-Encoding: quoted-printable
Ah yes, you're right. With the non-keyed stream it doe= sn't make a big difference because it's only one big state value.
The throughput still seems quite low. Have you ever tried= looking at the "back pressure" tab on the Flink dashboard. For t= his I would suggest to disable chaining, so that every operator is run in a= n isolated task:

env.disableOperatorChaining();

On Thu, 7 Apr 2016= at 05:11 Hironori Ogibayashi <o= gibayashi@gmail.com> wrote:
= I tried RocksDB, but the result was almost the same.

I used the following code and put 2.6million distinct records into Kafka. After processing all records, the state on the HDFS become about 250MB
and time needed for
the checkpoint was almost 5sec. Processing throughput was
FsStateBackend-> 8000msg/sec, RocksDBSsateBackend -> 9000msg/sec

---
=C2=A0 =C2=A0 env.setStateBackend(new
RocksDBStateBackend("hdfs://<hdfs_host>:8020/apps/flink/checkpoi= nts"));

=C2=A0 =C2=A0 val stream =3D env
=C2=A0 =C2=A0 =C2=A0 .addSource(new FlinkKafkaConsumer09[String]("kafk= a.json2", new
SimpleStringSchema(), properties))
=C2=A0 =C2=A0 =C2=A0 .map(parseJson(_))
=C2=A0 =C2=A0 =C2=A0 .timeWindowAll(Time.of(10, TimeUnit.DAYS))
=C2=A0 =C2=A0 =C2=A0 .trigger(MyContinuousProcessingTimeTrigger.of(Time.sec= onds(5)))
=C2=A0 =C2=A0 =C2=A0 // count distinct values
=C2=A0 =C2=A0 =C2=A0 .fold(Set[String]()){(r,i) =3D> { r + i}}
=C2=A0 =C2=A0 =C2=A0 .map{x =3D> (System.currentTimeMillis(), x.size)} =C2=A0 =C2=A0 =C2=A0 .addSink(new ElasticsearchSink(config, transports, new=
IndexRequestBuilder[Tuple2[Long, Int]]=C2=A0 {
=C2=A0 =C2=A0 =C2=A0 =C2=A0 override def createIndexRequest(element: Tuple2= [Long, Int],
ctx: RuntimeContext): IndexRequest =3D {
=C2=A0 =C2=A0 =C2=A0 =C2=A0 =C2=A0 val json =3D new HashMap[String, AnyRef]=
=C2=A0 =C2=A0 =C2=A0 =C2=A0 =C2=A0 json.put("@timestamp", new Tim= estamp(element._1))
=C2=A0 =C2=A0 =C2=A0 =C2=A0 =C2=A0 json.put("count", element._2: = java.lang.Integer)
=C2=A0 =C2=A0 =C2=A0 =C2=A0 =C2=A0 Requests.indexRequest.index("dummy3= ").`type`("my-type").source(json)
=C2=A0 =C2=A0 =C2=A0 =C2=A0 }
=C2=A0 =C2=A0 =C2=A0 }))
---

I guess this is because I used non-keyed stream, so I had one state
record with a big value (all distinct value).
I think copying all 250MB(or more) file to HDFS in every checkpoint
will be heavy, so I will try storing the distinct values
in the external datastore (e.g. redis).
Also, when incremental snapshot get implemented, I want to try.

Regards,
Hironori

2016-04-05 21:40 GMT+09:00 Hironori Ogibayashi <ogibayashi@gmail.com>:
> Aljoscha,
>
> Thank you for your quick response.
> Yes, I am using FsStateBackend, so I will try RocksDB backend.
>
> Regards,
> Hironori
>
> 2016-04-05 21:23 GMT+09:00 Aljoscha Krettek <aljoscha@apache.org>:
>> Hi,
>> I guess you are using the FsStateBackend, is that correct? You cou= ld try
>> using the RocksDB state backend:
>> https://ci.apache.org/projects/flink/flink-docs-maste= r/apis/streaming/state_backends.html#the-rocksdbstatebackend
>>
>> With this, throughput will be lower but the overhead per checkpoin= t could be
>> lower. Also, with this most of the file copying necessary for the = checkpoint
>> will be done while data processing keeps running (asynchronous sna= pshot).
>>
>> As to incremental snapshots. I'm afraid this feature is not ye= t implemented
>> but we're working on it.
>>
>> Cheers,
>> Aljoscha
>>
>> On Tue, 5 Apr 2016 at 14:06 Hironori Ogibayashi <ogibayashi@gmail.com> >> wrote:
>>>
>>> Hello,
>>>
>>> I am trying to implement windowed distinct count on a stream. = In this
>>> case, the state
>>> have to hold all distinct value in the window, so can be large= .
>>>
>>> In my test, if the state size become about 400MB, checkpointin= g takes
>>> 40sec and spends most of Taskmanager's CPU.
>>> Are there any good way to handle this situation?
>>>
>>> Flink document mentions about incremental snapshot, and I am i= nterested in
>>> it,
>>> but could not find how to enable it. (not implemented yet?) >>>
>>> https://ci.apache.org/projects/flink/flink-docs-release-1.0/interna= ls/stream_checkpointing.html
>>>
>>> Regards,
>>> Hironori
--001a11c33f105c9297052fe11947--