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 BA2BE200C68 for ; Wed, 3 May 2017 18:05:24 +0200 (CEST) Received: by cust-asf.ponee.io (Postfix) id B8D69160BB5; Wed, 3 May 2017 16:05:24 +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 AE128160BAA for ; Wed, 3 May 2017 18:05:23 +0200 (CEST) Received: (qmail 4825 invoked by uid 500); 3 May 2017 16:05:22 -0000 Mailing-List: contact user-help@flink.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Delivered-To: mailing list user@flink.apache.org Received: (qmail 4815 invoked by uid 99); 3 May 2017 16:05:22 -0000 Received: from pnap-us-west-generic-nat.apache.org (HELO spamd2-us-west.apache.org) (209.188.14.142) by apache.org (qpsmtpd/0.29) with ESMTP; Wed, 03 May 2017 16:05:22 +0000 Received: from localhost (localhost [127.0.0.1]) by spamd2-us-west.apache.org (ASF Mail Server at spamd2-us-west.apache.org) with ESMTP id 5293E1A06F8 for ; Wed, 3 May 2017 16:05:22 +0000 (UTC) X-Virus-Scanned: Debian amavisd-new at spamd2-us-west.apache.org X-Spam-Flag: NO X-Spam-Score: 2.481 X-Spam-Level: ** X-Spam-Status: No, score=2.481 tagged_above=-999 required=6.31 tests=[DKIM_SIGNED=0.1, DKIM_VALID=-0.1, HTML_MESSAGE=2, RCVD_IN_DNSWL_NONE=-0.0001, RCVD_IN_MSPIKE_H3=-0.01, RCVD_IN_MSPIKE_WL=-0.01, RCVD_IN_SORBS_SPAM=0.5, URIBL_BLOCKED=0.001] autolearn=disabled Authentication-Results: spamd2-us-west.apache.org (amavisd-new); dkim=pass (2048-bit key) header.d=data-artisans-com.20150623.gappssmtp.com Received: from mx1-lw-eu.apache.org ([10.40.0.8]) by localhost (spamd2-us-west.apache.org [10.40.0.9]) (amavisd-new, port 10024) with ESMTP id yixzt7wJsmNZ for ; Wed, 3 May 2017 16:05:20 +0000 (UTC) Received: from mail-wm0-f49.google.com (mail-wm0-f49.google.com [74.125.82.49]) by mx1-lw-eu.apache.org (ASF Mail Server at mx1-lw-eu.apache.org) with ESMTPS id D4F6E5F5CA for ; Wed, 3 May 2017 16:05:14 +0000 (UTC) Received: by mail-wm0-f49.google.com with SMTP id w64so152063819wma.0 for ; Wed, 03 May 2017 09:05:14 -0700 (PDT) DKIM-Signature: v=1; a=rsa-sha256; c=relaxed/relaxed; d=data-artisans-com.20150623.gappssmtp.com; s=20150623; h=from:message-id:mime-version:subject:date:in-reply-to:cc:to :references; bh=PKGli8axThLeD+bUqy6koryih6L/vhuriACQh5moWYc=; b=VPyElULRXB7iuJ4EVBrpUEIyvjYXqjPvv0bhXS+DnREG7HR2yaOM+kCmyyIr1tQGuD 5caAuXM6q+kko2vua0Ifkz7DCkJBHEtg5bGu7WPmR8CkEe9gxPZoK3ejuBIMxTVw13Kr 3bexWTQnRlXiPupY5ML9TR9zTWzpXyQZVfVXxhRPnzj/4YwgLyzOSBp6D4kC2rb/c2ZK WmSrtEG7zifR4Df/nfk2EGi9AxZppxD4lrcfaDa2yhT2QvdqyOXFgY/nD1vCAXzsMK/Q M0r5ABTRLsJtIunRVAbU3Yv4keS/jKAZAJtpVl8fHYfrKUQmY2cvY8awaT9nHf9yssB6 QfEg== X-Google-DKIM-Signature: v=1; a=rsa-sha256; c=relaxed/relaxed; d=1e100.net; s=20161025; h=x-gm-message-state:from:message-id:mime-version:subject:date :in-reply-to:cc:to:references; bh=PKGli8axThLeD+bUqy6koryih6L/vhuriACQh5moWYc=; b=FCN8jkpTMi7hFHA78+BcKqRY0TboSuHw/ARPDex5S1Iw1E9CSpl/Xw+h9AlW6+p8kM n02cqHodZ0LQcJBizPCw4iUciy/vxA2ahc/BLl+dZ61/Rh4R8eSOtWfP/+uRoAtwLZQs PZ9zU6VsZu4AeeIlcpabbsRggyrd/niZSbYAV1wrSkcgtgnJnKagFp1vTP6lym2D6PSd dydCuMSABqqsZxXdRJ92s898MFiO144SfMFD+FVmPxxVAFa47nwnE105fUdyo3FjBz5m q0KUQVyJs10vCiAd1Y4oBfqnu0QUBcoix4I3LIFVhiIAjW7BqT115LojLz9X/Alh8X5r x4Ug== X-Gm-Message-State: AN3rC/7A5IW0XB5E96cne6x9R3Dk8gDiR9wBhld8UBuiXE7Ln8nRZLhB 2gfIAgWydIj6zEtl X-Received: by 10.28.128.202 with SMTP id b193mr2798411wmd.53.1493827514341; Wed, 03 May 2017 09:05:14 -0700 (PDT) Received: from skynet.fritz.box (dslb-084-059-068-070.084.059.pools.vodafone-ip.de. [84.59.68.70]) by smtp.gmail.com with ESMTPSA id f64sm3866073wmg.2.2017.05.03.09.05.13 (version=TLS1_2 cipher=ECDHE-RSA-AES128-GCM-SHA256 bits=128/128); Wed, 03 May 2017 09:05:13 -0700 (PDT) From: Stefan Richter Message-Id: Content-Type: multipart/alternative; boundary="Apple-Mail=_0495BEC6-CEEA-4000-843D-2EA496F7FA4C" Mime-Version: 1.0 (Mac OS X Mail 10.3 \(3273\)) Subject: Re: Tuning RocksDB Date: Wed, 3 May 2017 18:05:11 +0200 In-Reply-To: <08BD9A55-F373-4741-AEB8-32A7B55A74A5@data-artisans.com> Cc: user@flink.apache.org To: Jason Brelloch References: <08BD9A55-F373-4741-AEB8-32A7B55A74A5@data-artisans.com> X-Mailer: Apple Mail (2.3273) archived-at: Wed, 03 May 2017 16:05:24 -0000 --Apple-Mail=_0495BEC6-CEEA-4000-843D-2EA496F7FA4C Content-Transfer-Encoding: quoted-printable Content-Type: text/plain; charset=utf-8 Sorry, just saw that your question was actually mainly about = checkpointing, but it can still be related to my previous answer. I = assume the checkpointing time is the time that is reported in the web = interface? This would be the end-to-end runtime of the checkpoint which = does not really tell you how much time is spend on writing the state = itself, but you can find this exact detail in the logging; you can grep = for lines that start with "Asynchronous RocksDB snapshot=E2=80=9C. The = background is that end-to-end also includes the time the checkpoint = barrier needs to travel to the operator. If there is a lot of = backpressure and a lot of network buffers, this can take a while. Still, = the reason for the backpressure could still be in the way you access = RocksDB, as it seems you are de/serializing every time you update an = ever-growing value under a single key. I can see that accesses under = this conditions could become very slow eventually, but could remain fast = on the FSBackend for the reason from my first answer. > Am 03.05.2017 um 17:54 schrieb Stefan Richter = : >=20 > Hi, >=20 > typically, I would expect that the bottleneck with the RocksDB backend = is not RocksDB itself, but your TypeSerializers. I suggest to first run = a profiler/sampling attached to the process and check if the problematic = methods are in serialization or the actual accesses to RocksDB. The = RocksDB backend has to go through de/serialize roundtrips on every = single state access, while the FSBackend works on heap objects = immediately. For checkpoints, the RocksDB backend can write bytes = directly whereas the FSBackend has to use the serializers to get from = objects to bytes, so their actions w.r.t. how serializers are used are = kind of inverted between operation and checkpointing. For Flink 1.3 we = also will introduce incremental checkpoints on RocksDB that piggyback on = the SST files. Flink 1.2 is writing checkpoints and savepoints fully and = in a custom format. >=20 > Best, > Stefan >=20 >> Am 03.05.2017 um 16:46 schrieb Jason Brelloch >: >>=20 >> Hey all, >>=20 >> I am looking for some advice on tuning rocksDB for better performance = in Flink 1.2. I created a pretty simple job with a single kafka source = and one flatmap function that just stores 50000 events in a single key = of managed keyed state and then drops everything else, to test = checkpoint performance. Using a basic FsStateBackend configured as: >>=20 >> val backend =3D new = FsStateBackend("file:///home/jason/flink/checkpoint = ") >> env.setStateBackend(backend) >>=20 >> With about 30MB of state we see the checkpoints completing in 151ms. = Using a RocksDBStateBackend configured as: >>=20 >> val backend =3D new = RocksDBStateBackend("file:///home/jason/flink/checkpoint = ") >> backend.setDbStoragePath("file:///home/jason/flink/rocksdb = ") >> backend.setPredefinedOptions(PredefinedOptions.FLASH_SSD_OPTIMIZED) >> env.setStateBackend(backend) >>=20 >> Running the same test the checkpoint takes 3 minutes 42 seconds. >>=20 >> I expect it to be slower, but that seems excessive. I am also a = little confused as to when rocksDB and flink decide to write to disk, = because watching the database the .sst file wasn't created until = significantly after the checkpoint was completed, and the state had not = changed. Is there anything I can do to increase the speed of the = checkpoints, or anywhere I can look to debug the issue? (Nothing seems = out of the ordinary in the flink logs or rocksDB logs) >>=20 >> Thanks! >>=20 >> --=20 >> Jason Brelloch | Product Developer >> 3405 Piedmont Rd. NE, Suite 325, Atlanta, GA 30305=20 >> >> Subscribe to the BetterCloud Monitor = - Get IT delivered = to your inbox >=20 --Apple-Mail=_0495BEC6-CEEA-4000-843D-2EA496F7FA4C Content-Transfer-Encoding: quoted-printable Content-Type: text/html; charset=utf-8
Sorry, just saw that your question was = actually mainly about checkpointing, but it can still be related to my = previous answer. I assume the checkpointing time is the time that is = reported in the web interface? This would be the end-to-end runtime of = the checkpoint which does not really tell you how much time is spend on = writing the state itself, but you can find this exact detail in the = logging; you can grep for lines that start with "Asynchronous RocksDB = snapshot=E2=80=9C. The background is that end-to-end also includes the = time the checkpoint barrier needs to travel to the operator. If there is = a lot of backpressure and a lot of network buffers, this can take a = while. Still, the reason for the backpressure could still be in the way = you access RocksDB, as it seems you are de/serializing every time you = update an ever-growing value under a single key. I can see that accesses = under this conditions could become very slow eventually, but could = remain fast on the FSBackend for the reason from my first = answer.

Am 03.05.2017 um 17:54 schrieb Stefan Richter <s.richter@data-artisans.com>:

Hi,

typically, I would = expect that the bottleneck with the RocksDB backend is not RocksDB = itself, but your TypeSerializers. I suggest to first run a = profiler/sampling attached to the process and check if the problematic = methods are in serialization or the actual accesses to RocksDB. The = RocksDB backend has to go through de/serialize roundtrips on every = single state access, while the FSBackend works on heap objects = immediately. For checkpoints, the RocksDB backend can write bytes = directly whereas the FSBackend has to use the serializers to get from = objects to bytes, so their actions w.r.t. how serializers are used are = kind of inverted between operation and checkpointing. For Flink 1.3 we = also will introduce incremental checkpoints on RocksDB that piggyback on = the SST files. Flink 1.2 is writing checkpoints and savepoints fully and = in a custom format.

Best,
Stefan

Am 03.05.2017 um 16:46 schrieb Jason Brelloch <jb.bc.flk@gmail.com>:

Hey all,

I = am looking for some advice on tuning rocksDB for better performance in = Flink 1.2.  I created a pretty simple job with a single kafka = source and one flatmap function that just stores 50000 events in a = single key of managed keyed state and then drops everything else, to = test checkpoint performance.  Using a basic FsStateBackend = configured as:

val backend =3D new FsStateBackend("file:///home/jason/flink/checkpoint")
env.setStateBackend(backend)

With about 30MB of state = we see the checkpoints completing in 151ms.  Using a = RocksDBStateBackend configured as:

val backend =3D new = RocksDBStateBackend("file:///home/jason/flink/checkpoint")
backend.setDbStoragePath("file:///home/jason/flink/rocksdb")
backend.setPredefinedOptions(PredefinedOptions.FLASH_SSD_OPTIMI= ZED)
env.setStateBackend(backend)

Running the same test = the checkpoint takes 3 minutes 42 seconds.

I expect it to be slower, but that = seems excessive.  I am also a little confused as to when rocksDB = and flink decide to write to disk, because watching the database the = .sst file wasn't created until significantly after the checkpoint was = completed, and the state had not changed.  Is there anything I can = do to increase the speed of the checkpoints, or anywhere I can look to = debug the issue?  (Nothing seems out of the ordinary in the flink = logs or rocksDB logs)

Thanks!

--
Jason = Brelloch | = Product Developer
3405 Piedmont Rd. NE, Suite 325, = Atlanta, GA 30305 
3D""
Subscribe to the BetterCloud Monitor - Get IT = delivered to your inbox


= --Apple-Mail=_0495BEC6-CEEA-4000-843D-2EA496F7FA4C--