Return-Path: X-Original-To: apmail-hbase-commits-archive@www.apache.org Delivered-To: apmail-hbase-commits-archive@www.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id CCE84DC41 for ; Mon, 17 Dec 2012 06:42:53 +0000 (UTC) Received: (qmail 49095 invoked by uid 500); 17 Dec 2012 06:42:53 -0000 Delivered-To: apmail-hbase-commits-archive@hbase.apache.org Received: (qmail 48660 invoked by uid 500); 17 Dec 2012 06:42:41 -0000 Mailing-List: contact commits-help@hbase.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@hbase.apache.org Delivered-To: mailing list commits@hbase.apache.org Received: (qmail 48640 invoked by uid 99); 17 Dec 2012 06:42:41 -0000 Received: from athena.apache.org (HELO athena.apache.org) (140.211.11.136) by apache.org (qpsmtpd/0.29) with ESMTP; Mon, 17 Dec 2012 06:42:41 +0000 X-ASF-Spam-Status: No, hits=-2000.0 required=5.0 tests=ALL_TRUSTED X-Spam-Check-By: apache.org Received: from [140.211.11.4] (HELO eris.apache.org) (140.211.11.4) by apache.org (qpsmtpd/0.29) with ESMTP; Mon, 17 Dec 2012 06:42:39 +0000 Received: from eris.apache.org (localhost [127.0.0.1]) by eris.apache.org (Postfix) with ESMTP id C6684238899C; Mon, 17 Dec 2012 06:42:19 +0000 (UTC) Content-Type: text/plain; charset="utf-8" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit Subject: svn commit: r1422767 - /hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlock.java Date: Mon, 17 Dec 2012 06:42:19 -0000 To: commits@hbase.apache.org From: larsh@apache.org X-Mailer: svnmailer-1.0.8-patched Message-Id: <20121217064219.C6684238899C@eris.apache.org> X-Virus-Checked: Checked by ClamAV on apache.org Author: larsh Date: Mon Dec 17 06:42:18 2012 New Revision: 1422767 URL: http://svn.apache.org/viewvc?rev=1422767&view=rev Log: HBASE-7336 Revert due to OOMs on TestHFileBlock potentially caused by this. Modified: hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlock.java Modified: hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlock.java URL: http://svn.apache.org/viewvc/hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlock.java?rev=1422767&r1=1422766&r2=1422767&view=diff ============================================================================== --- hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlock.java (original) +++ hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlock.java Mon Dec 17 06:42:18 2012 @@ -29,8 +29,6 @@ import java.io.DataOutputStream; import java.io.IOException; import java.io.InputStream; import java.nio.ByteBuffer; -import java.util.concurrent.locks.Lock; -import java.util.concurrent.locks.ReentrantLock; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataOutputStream; @@ -1257,8 +1255,6 @@ public class HFileBlock extends SchemaCo /** The path (if any) where this data is coming from */ protected Path path; - private final Lock streamLock = new ReentrantLock(); - /** The default buffer size for our buffered streams */ public static final int DEFAULT_BUFFER_SIZE = 1 << 20; @@ -1333,9 +1329,23 @@ public class HFileBlock extends SchemaCo "-byte array at offset " + destOffset); } - if (!pread && streamLock.tryLock()) { + if (pread) { + // Positional read. Better for random reads. + int extraSize = peekIntoNextBlock ? hdrSize : 0; + + int ret = istream.read(fileOffset, dest, destOffset, size + extraSize); + if (ret < size) { + throw new IOException("Positional read of " + size + " bytes " + + "failed at offset " + fileOffset + " (returned " + ret + ")"); + } + + if (ret == size || ret < size + extraSize) { + // Could not read the next block's header, or did not try. + return -1; + } + } else { // Seek + read. Better for scanning. - try { + synchronized (istream) { istream.seek(fileOffset); long realOffset = istream.getPos(); @@ -1353,22 +1363,6 @@ public class HFileBlock extends SchemaCo // Try to read the next block header. if (!readWithExtra(istream, dest, destOffset, size, hdrSize)) return -1; - } finally { - streamLock.unlock(); - } - } else { - // Positional read. Better for random reads; or when the streamLock is already locked. - int extraSize = peekIntoNextBlock ? hdrSize : 0; - - int ret = istream.read(fileOffset, dest, destOffset, size + extraSize); - if (ret < size) { - throw new IOException("Positional read of " + size + " bytes " + - "failed at offset " + fileOffset + " (returned " + ret + ")"); - } - - if (ret == size || ret < size + extraSize) { - // Could not read the next block's header, or did not try. - return -1; } }