Return-Path: Delivered-To: apmail-hadoop-hdfs-issues-archive@minotaur.apache.org Received: (qmail 71224 invoked from network); 24 Oct 2010 10:54:49 -0000 Received: from unknown (HELO mail.apache.org) (140.211.11.3) by 140.211.11.9 with SMTP; 24 Oct 2010 10:54:49 -0000 Received: (qmail 93959 invoked by uid 500); 24 Oct 2010 10:54:49 -0000 Delivered-To: apmail-hadoop-hdfs-issues-archive@hadoop.apache.org Received: (qmail 93819 invoked by uid 500); 24 Oct 2010 10:54:46 -0000 Mailing-List: contact hdfs-issues-help@hadoop.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: hdfs-issues@hadoop.apache.org Delivered-To: mailing list hdfs-issues@hadoop.apache.org Received: (qmail 93811 invoked by uid 99); 24 Oct 2010 10:54:45 -0000 Received: from nike.apache.org (HELO nike.apache.org) (192.87.106.230) by apache.org (qpsmtpd/0.29) with ESMTP; Sun, 24 Oct 2010 10:54:45 +0000 X-ASF-Spam-Status: No, hits=-2000.0 required=10.0 tests=ALL_TRUSTED X-Spam-Check-By: apache.org Received: from [140.211.11.22] (HELO thor.apache.org) (140.211.11.22) by apache.org (qpsmtpd/0.29) with ESMTP; Sun, 24 Oct 2010 10:54:42 +0000 Received: from thor (localhost [127.0.0.1]) by thor.apache.org (8.13.8+Sun/8.13.8) with ESMTP id o9OAsLIi016398 for ; Sun, 24 Oct 2010 10:54:21 GMT Message-ID: <13906679.47351287917661197.JavaMail.jira@thor> Date: Sun, 24 Oct 2010 06:54:21 -0400 (EDT) From: =?utf-8?Q?Hajo_Nils_Krabbenh=C3=B6ft_=28JIRA=29?= To: hdfs-issues@hadoop.apache.org Subject: [jira] Commented: (HDFS-1459) NullPointerException in DataInputStream.readInt In-Reply-To: <11552325.10581287338963866.JavaMail.jira@thor> MIME-Version: 1.0 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: quoted-printable X-JIRA-FingerPrint: 30527f35849b9dde25b450d4833f0394 X-Virus-Checked: Checked by ClamAV on apache.org [ https://issues.apache.org/jira/browse/HDFS-1459?page=3Dcom.atlassian.= jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=3D12924= 306#action_12924306 ]=20 Hajo Nils Krabbenh=C3=B6ft commented on HDFS-1459: --------------------------------------------- I found this in my datanode logs: 2010-10-20 15:31:17,154 ERROR org.apache.hadoop.hdfs.server.datanode.DataNo= de: DatanodeRegistration(10.17.5.3:50010, storageID=3DDS-266784496-78.46.65= .54-50010-1287004808819, infoPort=3D50075, ipcPort=3D50020):DataXceiver java.io.IOException: xceiverCount 257 exceeds the limit of concurrent xciev= ers 256 =09at org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.ja= va:88) =09at java.lang.Thread.run(Thread.java:619) 2010-10-20 15:31:19,115 WARN org.apache.hadoop.hdfs.server.datanode.DataNod= e: DatanodeRegistration(10.17.5.3:50010, storageID=3DDS-266784496-78.46.65.= 54-50010-1287004808819, infoPort=3D50075, ipcPort=3D50020):Got exception wh= ile serving blk_-8099607957427967059_1974 to /10.17.5.4: java.net.SocketTimeoutException: 480000 millis timeout while waiting for ch= annel to be ready for write. ch : java.nio.channels.SocketChannel[connected= local=3D/10.17.5.3:50010 remote=3D/10.17.5.4:51336] =09at org.apache.hadoop.net.SocketIOWithTimeout.waitForIO(SocketIOWithTimeo= ut.java:246) =09at org.apache.hadoop.net.SocketOutputStream.waitForWritable(SocketOutput= Stream.java:159) =09at org.apache.hadoop.net.SocketOutputStream.transferToFully(SocketOutput= Stream.java:198) =09at org.apache.hadoop.hdfs.server.datanode.BlockSender.sendChunks(BlockSe= nder.java:313) =09at org.apache.hadoop.hdfs.server.datanode.BlockSender.sendBlock(BlockSen= der.java:401) =09at org.apache.hadoop.hdfs.server.datanode.DataXceiver.readBlock(DataXcei= ver.java:180) =09at org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.ja= va:95) =09at java.lang.Thread.run(Thread.java:619) and so far using this configuration snippet seems to fix the problem: dfs.datanode.handler.count 40 The number of server threads for the datanode. dfs.namenode.handler.count 40 The number of server threads for the namenode. =20 dfs.datanode.max.xcievers =20 2048 =20 The maximum # of threads that can be connected to a data ndoe simultaneously. Default value is 256. =20 =20 So the underlying problem seems to be that when max xcievers is reached tha= t the client does not get notified and thus reports unusable error messages= . > NullPointerException in DataInputStream.readInt > ----------------------------------------------- > > Key: HDFS-1459 > URL: https://issues.apache.org/jira/browse/HDFS-1459 > Project: Hadoop HDFS > Issue Type: Bug > Affects Versions: 0.20.1 > Environment: Debian 64 bit > Cloudera Hadoop > Reporter: Hajo Nils Krabbenh=C3=B6ft > > First, here's my source code accessing the HDFS: > final FSDataInputStream indexFile =3D getFile(bucketPathStr, Integer.toSt= ring(hashTableId) + ".index"); > indexFile.seek(bucketId * 4); > int bucketStart =3D ByteSwapper.swap(indexFile.readInt()); > int bucketEnd =3D ByteSwapper.swap(indexFile.readInt()); > final FSDataInputStream dataFile =3D getFile(bucketPathStr, Integer.toStr= ing(hashTableId) + ".data"); > dataFile.seek(bucketStart * (2 + Hasher.getConfigHashLength()) * 4); > for (int hash =3D bucketStart; hash < bucketEnd; hash++) { > =09int RimageIdA =3D ByteSwapper.swap(dataFile.readInt()); > =09int RimageIdB =3D ByteSwapper.swap(dataFile.readInt()); > =09....... read hash of length Hasher.getConfigHashLength() and work with= it .... > } > As you can see, i am reading the range to be read from an X.index file an= d then read these rows from X.data. The index file is always exactly 6.710.= 888 bytes in length. > As for the data file, everything works fine with 50 different 1.35 GB (22= blocks) data files and it fails every time i tried with 50 different 2.42 = GB (39 blocks) data files. So the cause of the bug is clearly dependent on = the file size. > I checked for ulimit and for the number of network connections and they a= re both not maxed out when the error occurs. The stack trace i get is: > java.lang.NullPointerException > =09at org.apache.hadoop.hdfs.DFSClient$DFSInputStream.readBuffer(DFSClien= t.java:1703) > =09at org.apache.hadoop.hdfs.DFSClient$DFSInputStream.read(DFSClient.java= :1755) > =09at org.apache.hadoop.hdfs.DFSClient$DFSInputStream.read(DFSClient.java= :1680) > =09at java.io.DataInputStream.readInt(DataInputStream.java:370) > ... > =09at org.apache.hadoop.mapred.MapRunner.run(MapRunner.java:50) > =09at org.apache.hadoop.mapred.MapTask.runOldMapper(MapTask.java:358) > =09at org.apache.hadoop.mapred.MapTask.run(MapTask.java:307) > =09at org.apache.hadoop.mapred.Child.main(Child.java:170) > which leads me to believe that DFSClient.blockSeekTo returns with a non-n= ull chosenNode but with blockReader =3D null. > Since the exact same jar works flawlessly with small data files and fails= reliably with big data files, i'm wondering how this could possibly depend= ent on the file's size or block count (DFSClient.java line 1628+): > s =3D socketFactory.createSocket(); > NetUtils.connect(s, targetAddr, socketTimeout); > s.setSoTimeout(socketTimeout); > Block blk =3D targetBlock.getBlock(); > blockReader =3D BlockReader.newBlockReader(s, src, blk.getBlockId(),=20 > blk.getGenerationStamp(), > offsetIntoBlock, blk.getNumBytes() - offsetIntoBlock, > buffersize, verifyChecksum, clientName); > return chosenNode; --=20 This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.