From issues-return-193296-archive-asf-public=cust-asf.ponee.io@flink.apache.org Tue Oct 9 17:26: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 51108180668 for ; Tue, 9 Oct 2018 17:26:03 +0200 (CEST) Received: (qmail 95934 invoked by uid 500); 9 Oct 2018 15:26:02 -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 95922 invoked by uid 99); 9 Oct 2018 15:26:02 -0000 Received: from pnap-us-west-generic-nat.apache.org (HELO spamd1-us-west.apache.org) (209.188.14.142) by apache.org (qpsmtpd/0.29) with ESMTP; Tue, 09 Oct 2018 15:26:02 +0000 Received: from localhost (localhost [127.0.0.1]) by spamd1-us-west.apache.org (ASF Mail Server at spamd1-us-west.apache.org) with ESMTP id 08DBCC62DF for ; Tue, 9 Oct 2018 15:26:02 +0000 (UTC) X-Virus-Scanned: Debian amavisd-new at spamd1-us-west.apache.org X-Spam-Flag: NO X-Spam-Score: -110.3 X-Spam-Level: X-Spam-Status: No, score=-110.3 tagged_above=-999 required=6.31 tests=[ENV_AND_HDR_SPF_MATCH=-0.5, NORMAL_HTTP_TO_IP=0.001, 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 (spamd1-us-west.apache.org [10.40.0.7]) (amavisd-new, port 10024) with ESMTP id 7hglVtRZacUr for ; Tue, 9 Oct 2018 15:26: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 024145F175 for ; Tue, 9 Oct 2018 15:26: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 9556FE0D27 for ; Tue, 9 Oct 2018 15:26: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 56D25247C7 for ; Tue, 9 Oct 2018 15:26:00 +0000 (UTC) Date: Tue, 9 Oct 2018 15:26:00 +0000 (UTC) From: "Nico Kruber (JIRA)" To: issues@flink.apache.org Message-ID: In-Reply-To: References: Subject: [jira] [Updated] (FLINK-10491) Deadlock during spilling data in SpillableSubpartition 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-10491?page=3Dcom.atlassi= an.jira.plugin.system.issuetabpanels:all-tabpanel ] Nico Kruber updated FLINK-10491: -------------------------------- Fix Version/s: 1.5.5 1.6.2 1.7.0 > Deadlock during spilling data in SpillableSubpartition=20 > ------------------------------------------------------- > > Key: FLINK-10491 > URL: https://issues.apache.org/jira/browse/FLINK-10491 > Project: Flink > Issue Type: Bug > Components: Network > Affects Versions: 1.6.1, 1.5.4 > Reporter: Piotr Nowojski > Assignee: zhijiang > Priority: Critical > Labels: pull-request-available > Fix For: 1.7.0, 1.6.2, 1.5.5 > > > Originally reported here:=C2=A0[https://lists.apache.org/thread.html/472c= 8f4a2711c5e217fadd9a88f8c73670218e7432bb81ba3f5076db@%3Cuser.flink.apache.o= rg%3E] > Thread dump (from 1.5.3 version) showing two deadlocked threads, because = they are taking two locks in different order: > {noformat} > Thread-1 > "DataSink (DATA#HadoopFileOutputFormat ) (1/2)@11002" prio=3D5 tid=3D0x3e= 2 nid=3DNA waiting for monitor entry > waiting for Map (Key Extractor) (1/10)@9967 to release lock on <0x2dfb> (= a java.util.ArrayDeque) > at org.apache.flink.runtime.io.network.partition.SpillableSubpartition.re= leaseMemory(SpillableSubpartition.java:223) > at org.apache.flink.runtime.io.network.partition.ResultPartition.releaseM= emory(ResultPartition.java:373) > at org.apache.flink.runtime.io.network.buffer.LocalBufferPool.setNumBuffe= rs(LocalBufferPool.java:355) > - locked <0x2dfd> (a java.util.ArrayDeque) > at org.apache.flink.runtime.io.network.buffer.NetworkBufferPool.redistrib= uteBuffers(NetworkBufferPool.java:402) > at org.apache.flink.runtime.io.network.buffer.NetworkBufferPool.recycleMe= morySegments(NetworkBufferPool.java:203) > - locked <0x2da5> (a java.lang.Object) > at org.apache.flink.runtime.io.network.buffer.NetworkBufferPool.recycleMe= morySegments(NetworkBufferPool.java:193) > at org.apache.flink.runtime.io.network.partition.consumer.SingleInputGate= .returnExclusiveSegments(SingleInputGate.java:318) > at org.apache.flink.runtime.io.network.partition.consumer.RemoteInputChan= nel.releaseAllResources(RemoteInputChannel.java:259) > at org.apache.flink.runtime.io.network.partition.consumer.SingleInputGate= .getNextBufferOrEvent(SingleInputGate.java:578) > at org.apache.flink.runtime.io.network.partition.consumer.SingleInputGate= .pollNextBufferOrEvent(SingleInputGate.java:507) > at org.apache.flink.runtime.io.network.partition.consumer.UnionInputGate.= waitAndGetNextInputGate(UnionInputGate.java:213) > at org.apache.flink.runtime.io.network.partition.consumer.UnionInputGate.= getNextBufferOrEvent(UnionInputGate.java:163) > at org.apache.flink.runtime.io.network.api.reader.AbstractRecordReader.ge= tNextRecord(AbstractRecordReader.java:86) > at org.apache.flink.runtime.io.network.api.reader.MutableRecordReader.nex= t(MutableRecordReader.java:47) > at org.apache.flink.runtime.operators.util.ReaderIterator.next(ReaderIter= ator.java:73) > at org.apache.flink.runtime.operators.DataSinkTask.invoke(DataSinkTask.ja= va:216) > at org.apache.flink.runtime.taskmanager.Task.run(Task.java:703) > at java.lang.Thread.run(Thread.java:745) > Thread-2 > "Map (Key Extractor) (1/10)@9967" prio=3D5 tid=3D0xaab nid=3DNA waiting f= or monitor entry > java.lang.Thread.State: BLOCKED > blocks DataSink (DATA#HadoopFileOutputFormat ) (1/2)@11002 > waiting for DataSink (DATA#HadoopFileOutputFormat ) (1/2)@11002 to releas= e lock on <0x2dfd> (a java.util.ArrayDeque) > at org.apache.flink.runtime.io.network.buffer.LocalBufferPool.recycle(Loc= alBufferPool.java:261) > at org.apache.flink.runtime.io.network.buffer.NetworkBuffer.deallocate(Ne= tworkBuffer.java:171) > at org.apache.flink.shaded.netty4.io.netty.buffer.AbstractReferenceCounte= dByteBuf.release(AbstractReferenceCountedByteBuf.java:106) > at org.apache.flink.runtime.io.network.buffer.NetworkBuffer.recycleBuffer= (NetworkBuffer.java:146) > at org.apache.flink.runtime.io.network.buffer.BufferConsumer.close(Buffer= Consumer.java:110) > at org.apache.flink.runtime.io.network.partition.SpillableSubpartition.sp= illFinishedBufferConsumers(SpillableSubpartition.java:271) > at org.apache.flink.runtime.io.network.partition.SpillableSubpartition.ad= d(SpillableSubpartition.java:117) > - locked <0x2dfb> (a java.util.ArrayDeque) > at org.apache.flink.runtime.io.network.partition.SpillableSubpartition.ad= d(SpillableSubpartition.java:96) > - locked <0x2dfc> (a org.apache.flink.runtime.io.network.partition.Spilla= bleSubpartition) > at org.apache.flink.runtime.io.network.partition.ResultPartition.addBuffe= rConsumer(ResultPartition.java:255) > at org.apache.flink.runtime.io.network.api.writer.RecordWriter.requestNew= BufferBuilder(RecordWriter.java:211) > at org.apache.flink.runtime.io.network.api.writer.RecordWriter.sendToTarg= et(RecordWriter.java:142) > at org.apache.flink.runtime.io.network.api.writer.RecordWriter.emit(Recor= dWriter.java:105) > at org.apache.flink.runtime.operators.shipping.OutputCollector.collect(Ou= tputCollector.java:65) > at org.apache.flink.runtime.operators.util.metrics.CountingCollector.coll= ect(CountingCollector.java:35) > at org.apache.flink.runtime.operators.MapDriver.run(MapDriver.java:103) > at org.apache.flink.runtime.operators.BatchTask.run(BatchTask.java:503) > at org.apache.flink.runtime.operators.BatchTask.invoke(BatchTask.java:368= ) > at org.apache.flink.runtime.taskmanager.Task.run(Task.java:703) > at java.lang.Thread.run(Thread.java:745) > {noformat} > The same pattern can occur on the master (but with slightly shifted line = numbers) > CC [~NicoK] -- This message was sent by Atlassian JIRA (v7.6.3#76005)