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 EE9C7200C3A for ; Fri, 31 Mar 2017 20:38:46 +0200 (CEST) Received: by cust-asf.ponee.io (Postfix) id ED656160B80; Fri, 31 Mar 2017 18:38:46 +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 1D9C5160B7C for ; Fri, 31 Mar 2017 20:38:45 +0200 (CEST) Received: (qmail 47956 invoked by uid 500); 31 Mar 2017 18:38:45 -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 47945 invoked by uid 99); 31 Mar 2017 18:38:45 -0000 Received: from pnap-us-west-generic-nat.apache.org (HELO spamd4-us-west.apache.org) (209.188.14.142) by apache.org (qpsmtpd/0.29) with ESMTP; Fri, 31 Mar 2017 18:38:45 +0000 Received: from localhost (localhost [127.0.0.1]) by spamd4-us-west.apache.org (ASF Mail Server at spamd4-us-west.apache.org) with ESMTP id CAE17C05E9 for ; Fri, 31 Mar 2017 18:38:44 +0000 (UTC) X-Virus-Scanned: Debian amavisd-new at spamd4-us-west.apache.org X-Spam-Flag: NO X-Spam-Score: -100.002 X-Spam-Level: X-Spam-Status: No, score=-100.002 tagged_above=-999 required=6.31 tests=[RP_MATCHES_RCVD=-0.001, SPF_PASS=-0.001, USER_IN_WHITELIST=-100] autolearn=disabled Received: from mx1-lw-eu.apache.org ([10.40.0.8]) by localhost (spamd4-us-west.apache.org [10.40.0.11]) (amavisd-new, port 10024) with ESMTP id ZSDIT0g6NxnE for ; Fri, 31 Mar 2017 18:38:43 +0000 (UTC) Received: from mailrelay1-us-west.apache.org (mailrelay1-us-west.apache.org [209.188.14.139]) by mx1-lw-eu.apache.org (ASF Mail Server at mx1-lw-eu.apache.org) with ESMTP id C7D185FC3D for ; Fri, 31 Mar 2017 18:38:42 +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 0F97CE06B9 for ; Fri, 31 Mar 2017 18:38:42 +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 BF63221DD9 for ; Fri, 31 Mar 2017 18:38:41 +0000 (UTC) Date: Fri, 31 Mar 2017 18:38:41 +0000 (UTC) From: "Arpit Agarwal (JIRA)" To: hdfs-issues@hadoop.apache.org Message-ID: In-Reply-To: References: Subject: [jira] [Commented] (HDFS-11608) HDFS write crashed in the case of huge block size MIME-Version: 1.0 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: 7bit X-JIRA-FingerPrint: 30527f35849b9dde25b450d4833f0394 archived-at: Fri, 31 Mar 2017 18:38:47 -0000 [ https://issues.apache.org/jira/browse/HDFS-11608?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15951472#comment-15951472 ] Arpit Agarwal commented on HDFS-11608: -------------------------------------- Thanks for reporting this [~xiaobingo]. Here are the client side exceptions from the original description (to keep the description concise). Client out-of-mem exception, {code} 17/03/30 07:13:50 WARN hdfs.DFSClient: Caught exception java.lang.InterruptedException at java.lang.Object.wait(Native Method) at java.lang.Thread.join(Thread.java:1245) at java.lang.Thread.join(Thread.java:1319) at org.apache.hadoop.hdfs.DFSOutputStream$DataStreamer.closeResponder(DFSOutputStream.java:624) at org.apache.hadoop.hdfs.DFSOutputStream$DataStreamer.closeInternal(DFSOutputStream.java:592) at org.apache.hadoop.hdfs.DFSOutputStream$DataStreamer.run(DFSOutputStream.java:588) Exception in thread "main" java.lang.OutOfMemoryError: Java heap space at org.apache.hadoop.hdfs.util.ByteArrayManager$NewByteArrayWithoutLimit.newByteArray(ByteArrayManager.java:308) at org.apache.hadoop.hdfs.DFSOutputStream.createPacket(DFSOutputStream.java:197) at org.apache.hadoop.hdfs.DFSOutputStream.writeChunkImpl(DFSOutputStream.java:1906) at org.apache.hadoop.hdfs.DFSOutputStream.writeChunk(DFSOutputStream.java:1884) at org.apache.hadoop.fs.FSOutputSummer.writeChecksumChunks(FSOutputSummer.java:206) at org.apache.hadoop.fs.FSOutputSummer.flushBuffer(FSOutputSummer.java:163) at org.apache.hadoop.fs.FSOutputSummer.flushBuffer(FSOutputSummer.java:144) at org.apache.hadoop.hdfs.DFSOutputStream.closeImpl(DFSOutputStream.java:2321) at org.apache.hadoop.hdfs.DFSOutputStream.close(DFSOutputStream.java:2303) at org.apache.hadoop.fs.FSDataOutputStream$PositionCache.close(FSDataOutputStream.java:72) at org.apache.hadoop.fs.FSDataOutputStream.close(FSDataOutputStream.java:106) at org.apache.hadoop.io.IOUtils.cleanup(IOUtils.java:244) at org.apache.hadoop.io.IOUtils.closeStream(IOUtils.java:261) at HdfsWriterOutputStream.run(HdfsWriterOutputStream.java:57) at org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:76) at org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:90) at HdfsWriterOutputStream.main(HdfsWriterOutputStream.java:77) at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.lang.reflect.Method.invoke(Method.java:498) at org.apache.hadoop.util.RunJar.run(RunJar.java:221) at org.apache.hadoop.util.RunJar.main(RunJar.java:136) {code} Client ResponseProcessor exception, {code} 17/03/30 18:20:12 WARN hdfs.DFSClient: DFSOutputStream ResponseProcessor exception for block BP-1828245847-192.168.64.101-1490851685890:blk_1073741859_1040 java.io.EOFException: Premature EOF: no length prefix available at org.apache.hadoop.hdfs.protocolPB.PBHelper.vintPrefixed(PBHelper.java:2293) at org.apache.hadoop.hdfs.protocol.datatransfer.PipelineAck.readFields(PipelineAck.java:244) at org.apache.hadoop.hdfs.DFSOutputStream$DataStreamer$ResponseProcessor.run(DFSOutputStream.java:748) 17/03/30 18:22:32 WARN hdfs.DFSClient: DataStreamer Exception java.io.IOException: Broken pipe at sun.nio.ch.FileDispatcherImpl.write0(Native Method) at sun.nio.ch.SocketDispatcher.write(SocketDispatcher.java:47) at sun.nio.ch.IOUtil.writeFromNativeBuffer(IOUtil.java:93) at sun.nio.ch.IOUtil.write(IOUtil.java:65) at sun.nio.ch.SocketChannelImpl.write(SocketChannelImpl.java:471) at org.apache.hadoop.net.SocketOutputStream$Writer.performIO(SocketOutputStream.java:63) at org.apache.hadoop.net.SocketIOWithTimeout.doIO(SocketIOWithTimeout.java:142) at org.apache.hadoop.net.SocketOutputStream.write(SocketOutputStream.java:159) at org.apache.hadoop.net.SocketOutputStream.write(SocketOutputStream.java:117) at java.io.BufferedOutputStream.write(BufferedOutputStream.java:122) at java.io.DataOutputStream.write(DataOutputStream.java:107) at org.apache.hadoop.hdfs.DFSPacket.writeTo(DFSPacket.java:176) at org.apache.hadoop.hdfs.DFSOutputStream$DataStreamer.run(DFSOutputStream.java:522) {code} > HDFS write crashed in the case of huge block size > ------------------------------------------------- > > Key: HDFS-11608 > URL: https://issues.apache.org/jira/browse/HDFS-11608 > Project: Hadoop HDFS > Issue Type: Bug > Components: hdfs-client > Affects Versions: 2.8.0 > Reporter: Xiaobing Zhou > Assignee: Xiaobing Zhou > Priority: Critical > > We've seen HDFS write crashes in the case of huge block size. For example, writing a 3G file using 3G block size, HDFS client throws out of memory exception. DataNode gives out IOException. After changing heap size limit, DFSOutputStream ResponseProcessor exception is seen followed by Broken pipe and pipeline recovery. > Give below: > DN exception, > {noformat} > 2017-03-30 16:34:33,828 ERROR datanode.DataNode (DataXceiver.java:run(278)) - c6401.ambari.apache.org:50010:DataXceiver error processing WRITE_BLOCK operation src: /192.168.64.101:47167 dst: /192.168.64.101:50010 > java.io.IOException: Incorrect value for packet payload size: 2147483128 > at org.apache.hadoop.hdfs.protocol.datatransfer.PacketReceiver.doRead(PacketReceiver.java:159) > at org.apache.hadoop.hdfs.protocol.datatransfer.PacketReceiver.receiveNextPacket(PacketReceiver.java:109) > at org.apache.hadoop.hdfs.server.datanode.BlockReceiver.receivePacket(BlockReceiver.java:502) > at org.apache.hadoop.hdfs.server.datanode.BlockReceiver.receiveBlock(BlockReceiver.java:898) > at org.apache.hadoop.hdfs.server.datanode.DataXceiver.writeBlock(DataXceiver.java:806) > at org.apache.hadoop.hdfs.protocol.datatransfer.Receiver.opWriteBlock(Receiver.java:137) > at org.apache.hadoop.hdfs.protocol.datatransfer.Receiver.processOp(Receiver.java:74) > at org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:251) > at java.lang.Thread.run(Thread.java:745) > {noformat} -- This message was sent by Atlassian JIRA (v6.3.15#6346) --------------------------------------------------------------------- To unsubscribe, e-mail: hdfs-issues-unsubscribe@hadoop.apache.org For additional commands, e-mail: hdfs-issues-help@hadoop.apache.org