From issues-return-198677-archive-asf-public=cust-asf.ponee.io@flink.apache.org Thu Nov 1 10:16:04 2018 Return-Path: X-Original-To: archive-asf-public@cust-asf.ponee.io Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by mx-eu-01.ponee.io (Postfix) with SMTP id F073C180677 for ; Thu, 1 Nov 2018 10:16:03 +0100 (CET) Received: (qmail 98550 invoked by uid 500); 1 Nov 2018 09:16: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 98541 invoked by uid 99); 1 Nov 2018 09:16:03 -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; Thu, 01 Nov 2018 09:16:03 +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 9C1861A23D2 for ; Thu, 1 Nov 2018 09:16:02 +0000 (UTC) X-Virus-Scanned: Debian amavisd-new at spamd2-us-west.apache.org X-Spam-Flag: NO X-Spam-Score: -109.501 X-Spam-Level: X-Spam-Status: No, score=-109.501 tagged_above=-999 required=6.31 tests=[ENV_AND_HDR_SPF_MATCH=-0.5, KAM_ASCII_DIVIDERS=0.8, 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 (spamd2-us-west.apache.org [10.40.0.9]) (amavisd-new, port 10024) with ESMTP id SCL89Yx76-LJ for ; Thu, 1 Nov 2018 09:16: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 215FF5F1F0 for ; Thu, 1 Nov 2018 09:16: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 AE8AFE0A31 for ; Thu, 1 Nov 2018 09:16: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 721E827764 for ; Thu, 1 Nov 2018 09:16:00 +0000 (UTC) Date: Thu, 1 Nov 2018 09:16:00 +0000 (UTC) From: "Maximilian Michels (JIRA)" To: issues@flink.apache.org Message-ID: In-Reply-To: References: Subject: [jira] [Commented] (FLINK-10672) Task stuck while writing output to flink 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-10672?page=3Dcom.atlassia= n.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=3D166= 71319#comment-16671319 ]=20 Maximilian Michels commented on FLINK-10672: -------------------------------------------- Thanks for investigating the matter [~angoenka]. This seems like an issue with the deadlock detection code. It's not a block= er for us anymore because we have the {{BATCH_FORCED}} workaround. We will = continue to investigate the pipelined execution issue. The TFX pipeline makes heavy use of broadcast variables and there is a susp= icion these might not be scheduled correctly under certain conditions. > Task stuck while writing output to flink > ---------------------------------------- > > Key: FLINK-10672 > URL: https://issues.apache.org/jira/browse/FLINK-10672 > Project: Flink > Issue Type: Bug > Affects Versions: 1.5.4 > Environment: OS: Debuan rodente 4.17 > Flink version: 1.5.4 > ||Key||Value|| > |jobmanager.heap.mb|1024| > |jobmanager.rpc.address|localhost| > |jobmanager.rpc.port|6123| > |metrics.reporter.jmx.class|org.apache.flink.metrics.jmx.JMXReporter| > |metrics.reporter.jmx.port|9250-9260| > |metrics.reporters|jmx| > |parallelism.default|1| > |rest.port|8081| > |taskmanager.heap.mb|1024| > |taskmanager.numberOfTaskSlots|1| > |web.tmpdir|/tmp/flink-web-bdb73d6c-5b9e-47b5-9ebf-eed0a7c82c26| > =C2=A0 > h1. Overview > ||Data Port||All Slots||Free Slots||CPU Cores||Physical Memory||JVM Heap = Size||Flink Managed Memory|| > |43501|1|0|12|62.9 GB|922 MB|642 MB| > h1. Memory > h2. JVM (Heap/Non-Heap) > ||Type||Committed||Used||Maximum|| > |Heap|922 MB|575 MB|922 MB| > |Non-Heap|68.8 MB|64.3 MB|-1 B| > |Total|991 MB|639 MB|922 MB| > h2. Outside JVM > ||Type||Count||Used||Capacity|| > |Direct|3,292|105 MB|105 MB| > |Mapped|0|0 B|0 B| > h1. Network > h2. Memory Segments > ||Type||Count|| > |Available|3,194| > |Total|3,278| > h1. Garbage Collection > ||Collector||Count||Time|| > |G1_Young_Generation|13|336| > |G1_Old_Generation|1|21| > Reporter: Ankur Goenka > Priority: Major > Attachments: 1uruvakHxBu.png, 3aDKQ24WvKk.png, Po89UGDn58V.png, j= mx_dump.json, jmx_dump_detailed.json, jstack_129827.log, jstack_163822.log,= jstack_66985.log > > > I am running a fairly complex pipleline with 200+ task. > The pipeline works fine with small data=C2=A0(order of 10kb input) but=C2= =A0gets stuck with a slightly larger data (300kb input). > =C2=A0 > The task gets stuck while=C2=A0writing the output toFlink, more specifica= lly it gets stuck=C2=A0while=C2=A0requesting memory segment in local buffer= pool. The Task manager UI=C2=A0shows that it has enough memory and memory = segments to work with. > The relevant stack trace is=C2=A0 > {quote}"grpc-default-executor-0" #138 daemon prio=3D5 os_prio=3D0 tid=3D0= x00007fedb0163800 nid=3D0x30b7f in Object.wait() [0x00007fedb4f90000] > java.lang.Thread.State: TIMED_WAITING (on object monitor) > at (C/C++) 0x00007fef201c7dae (Unknown Source) > at (C/C++) 0x00007fef1f2aea07 (Unknown Source) > at (C/C++) 0x00007fef1f241cd3 (Unknown Source) > at java.lang.Object.wait(Native Method) > - waiting on <0x00000000f6d56450> (a java.util.ArrayDeque) > at org.apache.flink.runtime.io.network.buffer.LocalBufferPool.requestMem= orySegment(LocalBufferPool.java:247) > - locked <0x00000000f6d56450> (a java.util.ArrayDeque) > at org.apache.flink.runtime.io.network.buffer.LocalBufferPool.requestBuf= ferBuilderBlocking(LocalBufferPool.java:204) > at org.apache.flink.runtime.io.network.api.writer.RecordWriter.requestNe= wBufferBuilder(RecordWriter.java:213) > at org.apache.flink.runtime.io.network.api.writer.RecordWriter.sendToTar= get(RecordWriter.java:144) > at org.apache.flink.runtime.io.network.api.writer.RecordWriter.emit(Reco= rdWriter.java:107) > at org.apache.flink.runtime.operators.shipping.OutputCollector.collect(O= utputCollector.java:65) > at org.apache.flink.runtime.operators.util.metrics.CountingCollector.col= lect(CountingCollector.java:35) > at org.apache.beam.runners.flink.translation.functions.FlinkExecutableSt= agePruningFunction.flatMap(FlinkExecutableStagePruningFunction.java:42) > at org.apache.beam.runners.flink.translation.functions.FlinkExecutableSt= agePruningFunction.flatMap(FlinkExecutableStagePruningFunction.java:26) > at org.apache.flink.runtime.operators.chaining.ChainedFlatMapDriver.coll= ect(ChainedFlatMapDriver.java:80) > at org.apache.flink.runtime.operators.util.metrics.CountingCollector.col= lect(CountingCollector.java:35) > at org.apache.beam.runners.flink.translation.functions.FlinkExecutableSt= ageFunction$MyDataReceiver.accept(FlinkExecutableStageFunction.java:230) > - locked <0x00000000f6a60bd0> (a java.lang.Object) > at org.apache.beam.sdk.fn.data.BeamFnDataInboundObserver.accept(BeamFnDa= taInboundObserver.java:81) > at org.apache.beam.sdk.fn.data.BeamFnDataInboundObserver.accept(BeamFnDa= taInboundObserver.java:32) > at org.apache.beam.sdk.fn.data.BeamFnDataGrpcMultiplexer$InboundObserver= .onNext(BeamFnDataGrpcMultiplexer.java:139) > at org.apache.beam.sdk.fn.data.BeamFnDataGrpcMultiplexer$InboundObserver= .onNext(BeamFnDataGrpcMultiplexer.java:125) > at org.apache.beam.vendor.grpc.v1.io.grpc.stub.ServerCalls$StreamingServ= erCallHandler$StreamingServerCallListener.onMessage(ServerCalls.java:248) > at org.apache.beam.vendor.grpc.v1.io.grpc.ForwardingServerCallListener.o= nMessage(ForwardingServerCallListener.java:33) > at org.apache.beam.vendor.grpc.v1.io.grpc.Contexts$ContextualizedServerC= allListener.onMessage(Contexts.java:76) > at org.apache.beam.vendor.grpc.v1.io.grpc.internal.ServerCallImpl$Server= StreamListenerImpl.messagesAvailable(ServerCallImpl.java:263) > at org.apache.beam.vendor.grpc.v1.io.grpc.internal.ServerImpl$JumpToAppl= icationThreadServerStreamListener$1MessagesAvailable.runInContext(ServerImp= l.java:683) > at org.apache.beam.vendor.grpc.v1.io.grpc.internal.ContextRunnable.run(C= ontextRunnable.java:37) > at org.apache.beam.vendor.grpc.v1.io.grpc.internal.SerializingExecutor.r= un(SerializingExecutor.java:123) > at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.= java:1149) > at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor= .java:624) > at java.lang.Thread.run(Thread.java:748){quote} > =C2=A0 > The full stack trace and logs are attached. > Please take a look and let me know if more information is needed. -- This message was sent by Atlassian JIRA (v7.6.3#76005)