Return-Path: Delivered-To: apmail-hadoop-hdfs-issues-archive@minotaur.apache.org Received: (qmail 37295 invoked from network); 25 Jun 2010 18:09:24 -0000 Received: from unknown (HELO mail.apache.org) (140.211.11.3) by 140.211.11.9 with SMTP; 25 Jun 2010 18:09:24 -0000 Received: (qmail 31475 invoked by uid 500); 25 Jun 2010 18:09:24 -0000 Delivered-To: apmail-hadoop-hdfs-issues-archive@hadoop.apache.org Received: (qmail 31412 invoked by uid 500); 25 Jun 2010 18:09:24 -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 31404 invoked by uid 99); 25 Jun 2010 18:09:23 -0000 Received: from athena.apache.org (HELO athena.apache.org) (140.211.11.136) by apache.org (qpsmtpd/0.29) with ESMTP; Fri, 25 Jun 2010 18:09:23 +0000 X-ASF-Spam-Status: No, hits=-1545.5 required=10.0 tests=ALL_TRUSTED,AWL 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; Fri, 25 Jun 2010 18:09:23 +0000 Received: from thor (localhost [127.0.0.1]) by thor.apache.org (8.13.8+Sun/8.13.8) with ESMTP id o5PI933m021162 for ; Fri, 25 Jun 2010 18:09:03 GMT Message-ID: <973441.62631277489343319.JavaMail.jira@thor> Date: Fri, 25 Jun 2010 14:09:03 -0400 (EDT) From: "sam rash (JIRA)" To: hdfs-issues@hadoop.apache.org Subject: [jira] Commented: (HDFS-1057) Concurrent readers hit ChecksumExceptions if following a writer to very end of file In-Reply-To: <843484796.396631269216807264.JavaMail.jira@brutus.apache.org> MIME-Version: 1.0 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: 7bit X-JIRA-FingerPrint: 30527f35849b9dde25b450d4833f0394 [ https://issues.apache.org/jira/browse/HDFS-1057?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12882664#action_12882664 ] sam rash commented on HDFS-1057: -------------------------------- per out offline discussion, it seems the NN doesn't know when the pipeline is created, but the writer does, so the NN has to return the replicas for the current block in this case. I will change it so we check all DNs for a replica before using the default of 0. I need to think about if we require all DNs to have ReplicaNotFound or all have that (versus some other exception). > Concurrent readers hit ChecksumExceptions if following a writer to very end of file > ----------------------------------------------------------------------------------- > > Key: HDFS-1057 > URL: https://issues.apache.org/jira/browse/HDFS-1057 > Project: Hadoop HDFS > Issue Type: Sub-task > Components: data-node > Affects Versions: 0.20-append, 0.21.0, 0.22.0 > Reporter: Todd Lipcon > Assignee: sam rash > Priority: Blocker > Fix For: 0.20-append > > Attachments: conurrent-reader-patch-1.txt, conurrent-reader-patch-2.txt, conurrent-reader-patch-3.txt, HDFS-1057-0.20-append.patch, hdfs-1057-trunk-1.txt, hdfs-1057-trunk-2.txt, hdfs-1057-trunk-3.txt, hdfs-1057-trunk-4.txt > > > In BlockReceiver.receivePacket, it calls replicaInfo.setBytesOnDisk before calling flush(). Therefore, if there is a concurrent reader, it's possible to race here - the reader will see the new length while those bytes are still in the buffers of BlockReceiver. Thus the client will potentially see checksum errors or EOFs. Additionally, the last checksum chunk of the file is made accessible to readers even though it is not stable. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.