From issues-return-177016-apmail-flink-issues-archive=flink.apache.org@flink.apache.org Thu Jul 12 13:15:03 2018 Return-Path: X-Original-To: apmail-flink-issues-archive@minotaur.apache.org Delivered-To: apmail-flink-issues-archive@minotaur.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id 31BAC18991 for ; Thu, 12 Jul 2018 13:15:03 +0000 (UTC) Received: (qmail 34172 invoked by uid 500); 12 Jul 2018 13:15:03 -0000 Delivered-To: apmail-flink-issues-archive@flink.apache.org Received: (qmail 34130 invoked by uid 500); 12 Jul 2018 13:15:03 -0000 Mailing-List: contact issues-help@flink.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@flink.apache.org Delivered-To: mailing list issues@flink.apache.org Received: (qmail 34118 invoked by uid 99); 12 Jul 2018 13:15:03 -0000 Received: from pnap-us-west-generic-nat.apache.org (HELO spamd3-us-west.apache.org) (209.188.14.142) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 12 Jul 2018 13:15:03 +0000 Received: from localhost (localhost [127.0.0.1]) by spamd3-us-west.apache.org (ASF Mail Server at spamd3-us-west.apache.org) with ESMTP id 9F59D1805D3 for ; Thu, 12 Jul 2018 13:15:02 +0000 (UTC) X-Virus-Scanned: Debian amavisd-new at spamd3-us-west.apache.org X-Spam-Flag: NO X-Spam-Score: -110.301 X-Spam-Level: X-Spam-Status: No, score=-110.301 tagged_above=-999 required=6.31 tests=[ENV_AND_HDR_SPF_MATCH=-0.5, RCVD_IN_DNSWL_MED=-2.3, SPF_PASS=-0.001, USER_IN_DEF_SPF_WL=-7.5, USER_IN_WHITELIST=-100] autolearn=disabled Received: from mx1-lw-us.apache.org ([10.40.0.8]) by localhost (spamd3-us-west.apache.org [10.40.0.10]) (amavisd-new, port 10024) with ESMTP id tpCji6NKK2hh for ; Thu, 12 Jul 2018 13:15:01 +0000 (UTC) Received: from mailrelay1-us-west.apache.org (mailrelay1-us-west.apache.org [209.188.14.139]) by mx1-lw-us.apache.org (ASF Mail Server at mx1-lw-us.apache.org) with ESMTP id 352325F43B for ; Thu, 12 Jul 2018 13:15:01 +0000 (UTC) Received: from jira-lw-us.apache.org (unknown [207.244.88.139]) by mailrelay1-us-west.apache.org (ASF Mail Server at mailrelay1-us-west.apache.org) with ESMTP id 9A0E1E104C for ; Thu, 12 Jul 2018 13:15:00 +0000 (UTC) Received: from jira-lw-us.apache.org (localhost [127.0.0.1]) by jira-lw-us.apache.org (ASF Mail Server at jira-lw-us.apache.org) with ESMTP id 1DEEF23F98 for ; Thu, 12 Jul 2018 13:15:00 +0000 (UTC) Date: Thu, 12 Jul 2018 13:15:00 +0000 (UTC) From: "Sayat Satybaldiyev (JIRA)" To: issues@flink.apache.org Message-ID: In-Reply-To: References: Subject: [jira] [Commented] (FLINK-9831) Too many open files for RocksDB MIME-Version: 1.0 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: quoted-printable X-JIRA-FingerPrint: 30527f35849b9dde25b450d4833f0394 [ https://issues.apache.org/jira/browse/FLINK-9831?page=3Dcom.atlassian= .jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=3D1654= 1610#comment-16541610 ]=20 Sayat Satybaldiyev commented on FLINK-9831: ------------------------------------------- [~srichter] I think I found an issue. I did {code:java} lsof=C2=A0-p FLINK_TM_PID | wc=C2=A0-l{code} and it give=C2=A08240 open file descriptors. Then I've looked in into the l= imits of the FLINK_TM_PID in the /proc directory it gives the following out= put: =C2=A0 =C2=A0 {code:java} Limit Soft Limit Hard Limit Units=20 Max open files 8192 8192 files {code} =C2=A0 My guess, that Flink TM has overridden=C2=A0_max_open_file_ during startup = and set the custom one.=C2=A0 This might=C2=A0be related to this rocksdb=C2= =A0issue [https://github.com/facebook/rocksdb/issues/457.] Question: how can I find which value Flinks uses by default with RocksDB? T= he second on, how can(or should) I override this limit? =C2=A0 > Too many open files for RocksDB > ------------------------------- > > Key: FLINK-9831 > URL: https://issues.apache.org/jira/browse/FLINK-9831 > Project: Flink > Issue Type: Bug > Components: State Backends, Checkpointing > Affects Versions: 1.5.0 > Reporter: Sayat Satybaldiyev > Priority: Major > Attachments: flink_open_files.txt > > > While running only one Flink job, which is backed by RocksDB with checkpo= ining to HDFS we encounter an exception that TM cannot access the SST file = because the process=C2=A0has too many open files. However, we have already = increased the file soft/hard limit on the machine. > Number open files for TM on the machine: > =C2=A0 > {code:java} > lsof -p 23301|wc -l > 8241{code} > =C2=A0 > Instance=C2=A0limits > =C2=A0 > {code:java} > ulimit -a > core file size (blocks, -c) 0 > data seg size (kbytes, -d) unlimited > scheduling priority (-e) 0 > file size (blocks, -f) unlimited > pending signals (-i) 256726 > max locked memory (kbytes, -l) 64 > max memory size (kbytes, -m) unlimited > open files (-n) 1048576 > pipe size (512 bytes, -p) 8 > POSIX message queues (bytes, -q) 819200 > real-time priority (-r) 0 > stack size (kbytes, -s) 8192 > cpu time (seconds, -t) unlimited > max user processes (-u) 128000 > virtual memory (kbytes, -v) unlimited > file locks (-x) unlimited > =C2=A0 > {code} > =C2=A0 > [^flink_open_files.txt] > java.lang.Exception: Exception while creating StreamOperatorStateContext. > =09at org.apache.flink.streaming.api.operators.StreamTaskStateInitializer= Impl.streamOperatorStateContext(StreamTaskStateInitializerImpl.java:191) > =09at org.apache.flink.streaming.api.operators.AbstractStreamOperator.ini= tializeState(AbstractStreamOperator.java:227) > =09at org.apache.flink.streaming.runtime.tasks.StreamTask.initializeState= (StreamTask.java:730) > =09at org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTa= sk.java:295) > =09at org.apache.flink.runtime.taskmanager.Task.run(Task.java:703) > =09at java.lang.Thread.run(Thread.java:748) > Caused by: org.apache.flink.util.FlinkException: Could not restore keyed = state backend for KeyedCoProcessOperator_98a16ed3228ec4a08acd8d78420516a1_(= 1/1) from any of the 1 provided restore options. > =09at org.apache.flink.streaming.api.operators.BackendRestorerProcedure.c= reateAndRestore(BackendRestorerProcedure.java:137) > =09at org.apache.flink.streaming.api.operators.StreamTaskStateInitializer= Impl.keyedStatedBackend(StreamTaskStateInitializerImpl.java:276) > =09at org.apache.flink.streaming.api.operators.StreamTaskStateInitializer= Impl.streamOperatorStateContext(StreamTaskStateInitializerImpl.java:132) > =09... 5 more > Caused by: java.io.FileNotFoundException: /tmp/flink-io-3da06c9e-f619-44c= 9-b95f-54ee9b1a084a/job_b3ecbdc0eb2dc2dfbf5532ec1fcef9da_op_KeyedCoProcessO= perator_98a16ed3228ec4a08acd8d78420516a1__1_1__uuid_c4b82a7e-8a04-4704-9e0b= -393c3243cef2/3701639a-bacd-4861-99d8-5f3d112e88d6/000016.sst (Too many ope= n files) > =09at java.io.FileOutputStream.open0(Native Method) > =09at java.io.FileOutputStream.open(FileOutputStream.java:270) > =09at java.io.FileOutputStream.(FileOutputStream.java:213) > =09at java.io.FileOutputStream.(FileOutputStream.java:162) > =09at org.apache.flink.core.fs.local.LocalDataOutputStream.(LocalDa= taOutputStream.java:47) > =09at org.apache.flink.core.fs.local.LocalFileSystem.create(LocalFileSyst= em.java:275) > =09at org.apache.flink.core.fs.SafetyNetWrapperFileSystem.create(SafetyNe= tWrapperFileSystem.java:121) > =09at org.apache.flink.contrib.streaming.state.RocksDBKeyedStateBackend$R= ocksDBIncrementalRestoreOperation.copyStateDataHandleData(RocksDBKeyedState= Backend.java:1008) > =09at org.apache.flink.contrib.streaming.state.RocksDBKeyedStateBackend$R= ocksDBIncrementalRestoreOperation.transferAllDataFromStateHandles(RocksDBKe= yedStateBackend.java:988) > =09at org.apache.flink.contrib.streaming.state.RocksDBKeyedStateBackend$R= ocksDBIncrementalRestoreOperation.transferAllStateDataToDirectory(RocksDBKe= yedStateBackend.java:973) > =09at org.apache.flink.contrib.streaming.state.RocksDBKeyedStateBackend$R= ocksDBIncrementalRestoreOperation.restoreInstance(RocksDBKeyedStateBackend.= java:758) > =09at org.apache.flink.contrib.streaming.state.RocksDBKeyedStateBackend$R= ocksDBIncrementalRestoreOperation.restore(RocksDBKeyedStateBackend.java:732= ) > =09at org.apache.flink.contrib.streaming.state.RocksDBKeyedStateBackend.r= estore(RocksDBKeyedStateBackend.java:443) > =09at org.apache.flink.contrib.streaming.state.RocksDBKeyedStateBackend.r= estore(RocksDBKeyedStateBackend.java:149) > =09at org.apache.flink.streaming.api.operators.BackendRestorerProcedure.a= ttemptCreateAndRestore(BackendRestorerProcedure.java:151) > =09at org.apache.flink.streaming.api.operators.BackendRestorerProcedure.c= reateAndRestore(BackendRestorerProcedure.java:123) > =09... 7 more -- This message was sent by Atlassian JIRA (v7.6.3#76005)