Return-Path: Delivered-To: apmail-lucene-hadoop-commits-archive@locus.apache.org Received: (qmail 42338 invoked from network); 14 May 2007 18:29:18 -0000 Received: from hermes.apache.org (HELO mail.apache.org) (140.211.11.2) by minotaur.apache.org with SMTP; 14 May 2007 18:29:18 -0000 Received: (qmail 76250 invoked by uid 500); 14 May 2007 18:29:24 -0000 Delivered-To: apmail-lucene-hadoop-commits-archive@lucene.apache.org Received: (qmail 76228 invoked by uid 500); 14 May 2007 18:29:24 -0000 Mailing-List: contact hadoop-commits-help@lucene.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: hadoop-dev@lucene.apache.org Delivered-To: mailing list hadoop-commits@lucene.apache.org Received: (qmail 76219 invoked by uid 99); 14 May 2007 18:29:24 -0000 Received: from herse.apache.org (HELO herse.apache.org) (140.211.11.133) by apache.org (qpsmtpd/0.29) with ESMTP; Mon, 14 May 2007 11:29:24 -0700 X-ASF-Spam-Status: No, hits=-99.5 required=10.0 tests=ALL_TRUSTED,NO_REAL_NAME X-Spam-Check-By: apache.org Received: from [140.211.11.3] (HELO eris.apache.org) (140.211.11.3) by apache.org (qpsmtpd/0.29) with ESMTP; Mon, 14 May 2007 11:29:17 -0700 Received: by eris.apache.org (Postfix, from userid 65534) id 380AE1A9838; Mon, 14 May 2007 11:28:57 -0700 (PDT) Content-Type: text/plain; charset="utf-8" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit Subject: svn commit: r537941 - in /lucene/hadoop/branches/branch-0.13: CHANGES.txt src/java/org/apache/hadoop/fs/ChecksumFileSystem.java Date: Mon, 14 May 2007 18:28:57 -0000 To: hadoop-commits@lucene.apache.org From: cutting@apache.org X-Mailer: svnmailer-1.1.0 Message-Id: <20070514182857.380AE1A9838@eris.apache.org> X-Virus-Checked: Checked by ClamAV on apache.org Author: cutting Date: Mon May 14 11:28:56 2007 New Revision: 537941 URL: http://svn.apache.org/viewvc?view=rev&rev=537941 Log: Merge -r 537938:537939 from trunk to 0.13 branch. Fixes: HADOOP-1345. Modified: lucene/hadoop/branches/branch-0.13/CHANGES.txt lucene/hadoop/branches/branch-0.13/src/java/org/apache/hadoop/fs/ChecksumFileSystem.java Modified: lucene/hadoop/branches/branch-0.13/CHANGES.txt URL: http://svn.apache.org/viewvc/lucene/hadoop/branches/branch-0.13/CHANGES.txt?view=diff&rev=537941&r1=537940&r2=537941 ============================================================================== --- lucene/hadoop/branches/branch-0.13/CHANGES.txt (original) +++ lucene/hadoop/branches/branch-0.13/CHANGES.txt Mon May 14 11:28:56 2007 @@ -375,6 +375,9 @@ 111. HADOOP-1350. Fix shuffle performance problem caused by forcing chunked encoding of map outputs. (Devaraj Das via cutting) +112. HADOOP-1345. Fix HDFS to correctly retry another replica when a + checksum error is encountered. (Hairong Kuang via cutting) + Release 0.12.3 - 2007-04-06 Modified: lucene/hadoop/branches/branch-0.13/src/java/org/apache/hadoop/fs/ChecksumFileSystem.java URL: http://svn.apache.org/viewvc/lucene/hadoop/branches/branch-0.13/src/java/org/apache/hadoop/fs/ChecksumFileSystem.java?view=diff&rev=537941&r1=537940&r2=537941 ============================================================================== --- lucene/hadoop/branches/branch-0.13/src/java/org/apache/hadoop/fs/ChecksumFileSystem.java (original) +++ lucene/hadoop/branches/branch-0.13/src/java/org/apache/hadoop/fs/ChecksumFileSystem.java Mon May 14 11:28:56 2007 @@ -134,13 +134,17 @@ } } + private long getChecksumFilePos( long dataPos ) { + return HEADER_LENGTH + 4*(dataPos/bytesPerSum); + } + public void seek(long desired) throws IOException { // seek to a checksum boundary long checksumBoundary = desired/bytesPerSum*bytesPerSum; if (checksumBoundary != getPos()) { datas.seek(checksumBoundary); if (sums != null) { - sums.seek(HEADER_LENGTH + 4*(checksumBoundary/bytesPerSum)); + sums.seek(getChecksumFilePos(checksumBoundary)); } } @@ -226,13 +230,11 @@ throw ce; } - sums.seek(oldSumsPos); - datas.seek(oldPos); - if (seekToNewSource(oldPos)) { // Since at least one of the sources is different, // the read might succeed, so we'll retry. retry = true; + seek(oldPos); //make sure Checksum sum's value gets restored } else { // Neither the data stream nor the checksum stream are being read // from different sources, meaning we'll still get a checksum error @@ -320,7 +322,7 @@ @Override public boolean seekToNewSource(long targetPos) throws IOException { boolean newDataSource = datas.seekToNewSource(targetPos); - return sums.seekToNewSource(targetPos/bytesPerSum) || newDataSource; + return sums.seekToNewSource(getChecksumFilePos(targetPos)) || newDataSource; } }