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 D8643200D36 for ; Mon, 6 Nov 2017 18:27:05 +0100 (CET) Received: by cust-asf.ponee.io (Postfix) id D6837160BFF; Mon, 6 Nov 2017 17:27:05 +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 01F5B160BEC for ; Mon, 6 Nov 2017 18:27:04 +0100 (CET) Received: (qmail 92590 invoked by uid 500); 6 Nov 2017 17:27:04 -0000 Mailing-List: contact hdfs-issues-help@hadoop.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Delivered-To: mailing list hdfs-issues@hadoop.apache.org Received: (qmail 92578 invoked by uid 99); 6 Nov 2017 17:27:04 -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; Mon, 06 Nov 2017 17:27:04 +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 50076C9301 for ; Mon, 6 Nov 2017 17:27:03 +0000 (UTC) X-Virus-Scanned: Debian amavisd-new at spamd1-us-west.apache.org X-Spam-Flag: NO X-Spam-Score: -99.202 X-Spam-Level: X-Spam-Status: No, score=-99.202 tagged_above=-999 required=6.31 tests=[KAM_ASCII_DIVIDERS=0.8, RP_MATCHES_RCVD=-0.001, SPF_PASS=-0.001, 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 a5A5BGSzKgmO for ; Mon, 6 Nov 2017 17:27:02 +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 3917F5FDCE for ; Mon, 6 Nov 2017 17:27:02 +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 4B126E258A for ; Mon, 6 Nov 2017 17:27:01 +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 99AA3241AB for ; Mon, 6 Nov 2017 17:27:00 +0000 (UTC) Date: Mon, 6 Nov 2017 17:27:00 +0000 (UTC) From: =?utf-8?Q?=C3=8D=C3=B1igo_Goiri_=28JIRA=29?= To: hdfs-issues@hadoop.apache.org Message-ID: In-Reply-To: References: Subject: [jira] [Commented] (HDFS-7060) Avoid taking locks when sending heartbeats from the DataNode MIME-Version: 1.0 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: quoted-printable X-JIRA-FingerPrint: 30527f35849b9dde25b450d4833f0394 archived-at: Mon, 06 Nov 2017 17:27:06 -0000 [ https://issues.apache.org/jira/browse/HDFS-7060?page=3Dcom.atlassian.= jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=3D16240= 582#comment-16240582 ]=20 =C3=8D=C3=B1igo Goiri commented on HDFS-7060: ----------------------------------- Thanks [~cheersyang], the comments now make clear the synchronization. The format of the comments in [^HDFS-7060.004.patch] is a little too short = (<60 chars) and the comments could fit into 3 and 2 lines respectively. I'm OK with this once we get a clean Yetus run and the minor comment is fix= ed. > Avoid taking locks when sending heartbeats from the DataNode > ------------------------------------------------------------ > > Key: HDFS-7060 > URL: https://issues.apache.org/jira/browse/HDFS-7060 > Project: Hadoop HDFS > Issue Type: Improvement > Reporter: Haohui Mai > Assignee: Xinwei Qin=20 > Labels: BB2015-05-TBR > Attachments: HDFS-7060-002.patch, HDFS-7060.000.patch, HDFS-7060.= 001.patch, HDFS-7060.003.patch, HDFS-7060.004.patch, complete_failed_qps.pn= g, sendHeartbeat.png > > > We're seeing the heartbeat is blocked by the monitor of {{FsDatasetImpl}}= when the DN is under heavy load of writes: > {noformat} > java.lang.Thread.State: BLOCKED (on object monitor) > at org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsVolume= Impl.getDfsUsed(FsVolumeImpl.java:115) > - waiting to lock <0x0000000780304fb8> (a org.apache.hadoop.hdfs.= server.datanode.fsdataset.impl.FsDatasetImpl) > at org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsDatase= tImpl.getStorageReports(FsDatasetImpl.java:91) > - locked <0x0000000780612fd8> (a java.lang.Object) > at org.apache.hadoop.hdfs.server.datanode.BPServiceActor.sendHear= tBeat(BPServiceActor.java:563) > at org.apache.hadoop.hdfs.server.datanode.BPServiceActor.offerSer= vice(BPServiceActor.java:668) > at org.apache.hadoop.hdfs.server.datanode.BPServiceActor.run(BPSe= rviceActor.java:827) > at java.lang.Thread.run(Thread.java:744) > java.lang.Thread.State: BLOCKED (on object monitor) > at org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsDatase= tImpl.createRbw(FsDatasetImpl.java:743) > - waiting to lock <0x0000000780304fb8> (a org.apache.hadoop.hdfs.= server.datanode.fsdataset.impl.FsDatasetImpl) > at org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsDatase= tImpl.createRbw(FsDatasetImpl.java:60) > at org.apache.hadoop.hdfs.server.datanode.BlockReceiver.(Bl= ockReceiver.java:169) > at org.apache.hadoop.hdfs.server.datanode.DataXceiver.writeBlock(= DataXceiver.java:621) > at org.apache.hadoop.hdfs.protocol.datatransfer.Receiver.opWriteB= lock(Receiver.java:124) > at org.apache.hadoop.hdfs.protocol.datatransfer.Receiver.processO= p(Receiver.java:71) > at org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXce= iver.java:232) > at java.lang.Thread.run(Thread.java:744) > java.lang.Thread.State: RUNNABLE > at java.io.UnixFileSystem.createFileExclusively(Native Method) > at java.io.File.createNewFile(File.java:1006) > at org.apache.hadoop.hdfs.server.datanode.DatanodeUtil.createTmpF= ile(DatanodeUtil.java:59) > at org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.BlockPoo= lSlice.createRbwFile(BlockPoolSlice.java:244) > at org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsVolume= Impl.createRbwFile(FsVolumeImpl.java:195) > at org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsDatase= tImpl.createRbw(FsDatasetImpl.java:753) > - locked <0x0000000780304fb8> (a org.apache.hadoop.hdfs.server.da= tanode.fsdataset.impl.FsDatasetImpl) > at org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsDatase= tImpl.createRbw(FsDatasetImpl.java:60) > at org.apache.hadoop.hdfs.server.datanode.BlockReceiver.(Bl= ockReceiver.java:169) > at org.apache.hadoop.hdfs.server.datanode.DataXceiver.writeBlock(= DataXceiver.java:621) > at org.apache.hadoop.hdfs.protocol.datatransfer.Receiver.opWriteB= lock(Receiver.java:124) > at org.apache.hadoop.hdfs.protocol.datatransfer.Receiver.processO= p(Receiver.java:71) > at org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXce= iver.java:232) > at java.lang.Thread.run(Thread.java:744) > {noformat} -- This message was sent by Atlassian JIRA (v6.4.14#64029) --------------------------------------------------------------------- To unsubscribe, e-mail: hdfs-issues-unsubscribe@hadoop.apache.org For additional commands, e-mail: hdfs-issues-help@hadoop.apache.org