Btw, side questions. Could it be, that you are accessing two different Hadoop file systems (two different schemas) or even the same one from two different users (encoded in the file system URI) within the same Flink JobMaster?
If so, the answer might be this possible resource leak in Flink: https://issues.apache.org/jira/browse/FLINK-9626 <https://issues.apache.org/jira/browse/FLINK-9626> Piotrek > On 20 Jun 2018, at 13:50, Piotr Nowojski <pi...@data-artisans.com> wrote: > > Hi, > > I was looking in this more, and I have couple of suspicions, but it’s still > hard to tell which is correct. Could you for example place a breakpoint (or > add a code there to print a stack trace) in > org.apache.log4j.helpers.AppenderAttachableImpl#addAppender > And check who is calling it? Since it seems like this method is responsible > for the growing number of ConsoleAppenders consumption. > > Piotrek > >> On 20 Jun 2018, at 09:20, Fabian Wollert <fab...@zalando.de >> <mailto:fab...@zalando.de>> wrote: >> >> Hi Piotr, thx for the hints. I checked the logs of this stack where the >> previous Heap Dump was from, there are no error messages from the >> BlobServer, it seems to work properly. >> >> But I found another issue in my setup, I had the logging not set up >> properly, so everything was logging in the default console appender. I >> changed this now to: >> >> log4j.rootLogger=INFO, FILE >> log4j.logger.akka=INFO, FILE >> log4j.logger.org.apache.kafka=INFO, FILE >> log4j.logger.org.apache.hadoop=INFO, FILE >> log4j.logger.org.apache.zookeeper=INFO, FILE >> >> # Log all info in the given file >> log4j.appender.FILE=org.apache.log4j.RollingFileAppender >> log4j.appender.FILE.File=/opt/flink/log/flink.log >> log4j.appender.FILE.MaxFileSize=100MB >> log4j.appender.FILE.MaxBackupIndex=2 >> log4j.appender.FILE.layout=org.apache.log4j.PatternLayout >> log4j.appender.FILE.layout.ConversionPattern=%d{yyyy-MM-dd HH:mm:ss,SSS} >> %-5p %c:%L - %m%n >> >> # suppress the irrelevant (wrong) warnings from the netty channel handler >> log4j.logger.org.jboss.netty.channel.DefaultChannelPipeline=ERROR, FILE >> >> though I have this setup now, I still see memory increases, but this time it >> seems again like my first suspicion is valid: >> >> <image.png> >> >> >> <image.png> >> >> What I'm here mostly wondering now: Why is still a ConsoleAppender used >> although I defined RollingFileAppender? >> >> Sry for the back and forth between different parts of the code. But your >> help is highly appreciated! >> >> Cheers >> >> -- >> >> Fabian Wollert >> Zalando SE >> >> E-Mail: fabian.woll...@zalando.de >> <mailto:fabian.woll...@zalando.de> >> >> Am Di., 19. Juni 2018 um 11:55 Uhr schrieb Piotr Nowojski >> <pi...@data-artisans.com <mailto:pi...@data-artisans.com>>: >> Hi, >> >> Can you search the logs/std err/std output for log entries like: >> >> log.warn("Failed to locally delete blob “ …) ? >> >> I see in the code, that if file deletion fails for whatever the reason, >> TransientBlobCleanupTask can loop indefinitely trying to remove it over and >> over again. That might be ok, however it’s doing it without any back off >> time as fast as possible. >> >> To confirm this, could you take couple of thread dumps and check whether >> some thread is spinning in >> org.apache.flink.runtime.blob.TransientBlobCleanupTask#run ? >> >> If that’s indeed a case, the question would be why file deletion fails? >> >> Piotrek >> >>> On 18 Jun 2018, at 15:48, Fabian Wollert <fab...@zalando.de >>> <mailto:fab...@zalando.de>> wrote: >>> >>> Hi Piotrek, thx a lot for your answer and sry for the late response. I was >>> running some more tests, but i still got the same problem. I was analyzing >>> a heap dump already with VisualVM, and thats how i got to the intention >>> that it was some S3 logging, but seems like i was wrong. on the newer >>> tests, the heap dump says the following (this time i used Eclipse >>> MemoryAnalyzer): >>> >>> <image.png> >>> <image.png> >>> <image.png> >>> Are you aware of problems with the BlobServer not cleaning up properly? I >>> tried also using a bigger instance, but this never stabilizes, it just >>> keeps increasing (gave it already 10GB+ Heap) ... >>> >>> Cheers >>> >>> -- >>> >>> Fabian Wollert >>> Zalando SE >>> >>> E-Mail: fabian.woll...@zalando.de <mailto:fabian.woll...@zalando.de> >>> >>> >>> >>> Am Mo., 11. Juni 2018 um 10:46 Uhr schrieb Piotr Nowojski >>> <pi...@data-artisans.com <mailto:pi...@data-artisans.com>>: >>> Hi, >>> >>> What kind of messages are those “logs about S3 operations”? Did you try to >>> google search them? Maybe it’s a known S3 issue? >>> >>> Another approach is please use some heap space analyser from which you can >>> backtrack classes that are referencing those “memory leaks” and again try >>> to google any known memory issues. >>> >>> It also could just mean, that it’s not a memory leak, but you just need to >>> allocate more heap space for your JVM (and memory consumption will >>> stabilise at some point). >>> >>> Piotrek >>> >>>> On 8 Jun 2018, at 18:32, Fabian Wollert <fab...@zalando.de >>>> <mailto:fab...@zalando.de>> wrote: >>>> >>>> Hi, in this email thread >>>> <http://apache-flink-user-mailing-list-archive.2336050.n4.nabble.com/Flink-and-AWS-S3-integration-java-lang-NullPointerException-null-uri-host-td20413.html> >>>> here, i tried to set up S3 as a filesystem backend for checkpoints. Now >>>> everything is working (Flink V1.5.0), but the JobMaster is accumulating >>>> Heap space, with eventually killing itself with HeapSpace OOM after >>>> several hours. If I don't enable Checkpointing, then everything is fine. >>>> I'm using the Flink S3 Shaded Libs (tried both the Hadoop and the Presto >>>> lib, no difference in this regard) from the tutorial. my checkpoint >>>> settings are this (job level): >>>> >>>> env.enableCheckpointing(1000); >>>> env.getCheckpointConfig().setCheckpointingMode(CheckpointingMode.EXACTLY_ONCE); >>>> env.getCheckpointConfig().setMinPauseBetweenCheckpoints(5000); >>>> env.getCheckpointConfig().setCheckpointTimeout(60000); >>>> env.getCheckpointConfig().setMaxConcurrentCheckpoints(1); >>>> >>>> Another clue why i suspect the S3 Checkpointing is that the heapspace dump >>>> contains a lot of char[] objects with some logs about S3 operations. >>>> >>>> anyone has an idea where to look further on this? >>>> >>>> Cheers >>>> >>>> -- >>>> >>>> Fabian Wollert >>>> Zalando SE >>>> >>>> E-Mail: fabian.woll...@zalando.de >>>> <mailto:fabian.woll...@zalando.de> >>>> >>>> Tamara-Danz-Straße 1 >>>> 10243 Berlin >>>> Fax: +49 (0)30 2759 46 93 >>>> E-mail: legalnot...@zalando.co.uk <mailto:legalnot...@zalando.co.uk> >>>> Notifications of major holdings (Sec. 33, 38, 39 WpHG): +49 (0)30 >>>> 2000889349 >>>> >>>> Management Board: >>>> Robert Gentz, David Schneider, Rubin Ritter >>>> >>>> Chairman of the Supervisory Board: >>>> Lothar Lanz >>>> >>>> Person responsible for providing the contents of Zalando SE acc. to Art. >>>> 55 RStV [Interstate Broadcasting Agreement]: Rubin Ritter >>>> Registered at the Local Court Charlottenburg Berlin, HRB 158855 B >>>> VAT registration number: DE 260543043 >>> >> >