Return-Path: Delivered-To: apmail-lucene-hadoop-dev-archive@locus.apache.org Received: (qmail 34342 invoked from network); 7 Jun 2007 21:09:48 -0000 Received: from hermes.apache.org (HELO mail.apache.org) (140.211.11.2) by minotaur.apache.org with SMTP; 7 Jun 2007 21:09:48 -0000 Received: (qmail 23746 invoked by uid 500); 7 Jun 2007 21:09:50 -0000 Delivered-To: apmail-lucene-hadoop-dev-archive@lucene.apache.org Received: (qmail 23713 invoked by uid 500); 7 Jun 2007 21:09:50 -0000 Mailing-List: contact hadoop-dev-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-dev@lucene.apache.org Received: (qmail 23704 invoked by uid 99); 7 Jun 2007 21:09:50 -0000 Received: from herse.apache.org (HELO herse.apache.org) (140.211.11.133) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 07 Jun 2007 14:09:50 -0700 X-ASF-Spam-Status: No, hits=-100.0 required=10.0 tests=ALL_TRUSTED X-Spam-Check-By: apache.org Received: from [140.211.11.4] (HELO brutus.apache.org) (140.211.11.4) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 07 Jun 2007 14:09:46 -0700 Received: from brutus (localhost [127.0.0.1]) by brutus.apache.org (Postfix) with ESMTP id 3818E714208 for ; Thu, 7 Jun 2007 14:09:26 -0700 (PDT) Message-ID: <19279981.1181250566226.JavaMail.jira@brutus> Date: Thu, 7 Jun 2007 14:09:26 -0700 (PDT) From: "Raghu Angadi (JIRA)" To: hadoop-dev@lucene.apache.org Subject: [jira] Commented: (HADOOP-1470) Rework FSInputChecker and FSOutputSummer to support checksum code sharing between ChecksumFileSystem and block level crc dfs In-Reply-To: <7723731.1181157326809.JavaMail.jira@brutus> MIME-Version: 1.0 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: quoted-printable X-Virus-Checked: Checked by ClamAV on apache.org [ https://issues.apache.org/jira/browse/HADOOP-1470?page=3Dcom.atlassia= n.jira.plugin.system.issuetabpanels:comment-tabpanel#action_12502511 ]=20 Raghu Angadi commented on HADOOP-1470: -------------------------------------- > could you use the above?=20 _can_ I? Yeah sure, my argument has never been it _can not_ be done. We ca= n even rechecksum to hide mismatch between {{bpc}} and {{blockSize}}. Though I don't agree with 'it is good enough until it is extremely difficul= t to use', whether it is implied here or not. I surely don't think it is an= improvement over FSInputChecker which, I think, was not explicitly designe= d to be general purpose checker.=20 =20 I sure hope this is not a blocker for HADOOP-1134. > Rework FSInputChecker and FSOutputSummer to support checksum code sharing= between ChecksumFileSystem and block level crc dfs > -------------------------------------------------------------------------= --------------------------------------------------- > > Key: HADOOP-1470 > URL: https://issues.apache.org/jira/browse/HADOOP-1470 > Project: Hadoop > Issue Type: Improvement > Components: fs > Affects Versions: 0.12.3 > Reporter: Hairong Kuang > Assignee: Hairong Kuang > Fix For: 0.14.0 > > Attachments: genericChecksum.patch > > > Comment from Doug in HADOOP-1134: > I'd prefer it if the CRC code could be shared with CheckSumFileSystem. In= particular, it seems to me that FSInputChecker and FSOutputSummer could be= extended to support pluggable sources and sinks for checksums, respectivel= y, and DFSDataInputStream and DFSDataOutputStream could use these. Advantag= es of this are: (a) single implementation of checksum logic to debug and ma= intain; (b) keeps checksumming as close to possible to data generation and = use. This patch computes checksums after data has been buffered, and valida= tes them before it is buffered. We sometimes use large buffers and would li= ke to guard against in-memory errors. The current checksum code catches a l= ot of such errors. So we should compute checksums after minimal buffering (= just bytesPerChecksum, ideally) and validate them at the last possible mome= nt (e.g., through the use of a small final buffer with a larger buffer behi= nd it). I do not think this will significantly affect performance, and data= integrity is a high priority.=20 --=20 This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.