You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by ra...@apache.org on 2016/02/04 07:15:24 UTC

hbase git commit: HBASE-HBASE-15203 Reduce garbage created by path.toString() during Checksum verification (Ram)

Repository: hbase
Updated Branches:
  refs/heads/master 6256ce4e6 -> 2cf8af5bf


HBASE-HBASE-15203 Reduce garbage created by path.toString() during
Checksum verification (Ram)


Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/2cf8af5b
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/2cf8af5b
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/2cf8af5b

Branch: refs/heads/master
Commit: 2cf8af5bf1d501156cbb3b421cf75c1051ead7d9
Parents: 6256ce4
Author: ramkrishna <ra...@gmail.com>
Authored: Thu Feb 4 11:44:46 2016 +0530
Committer: ramkrishna <ra...@gmail.com>
Committed: Thu Feb 4 11:44:46 2016 +0530

----------------------------------------------------------------------
 .../hadoop/hbase/io/hfile/ChecksumUtil.java     |  8 +++-----
 .../hadoop/hbase/io/hfile/HFileBlock.java       | 21 ++++++++++----------
 2 files changed, 14 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hbase/blob/2cf8af5b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/ChecksumUtil.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/ChecksumUtil.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/ChecksumUtil.java
index 61862eb..402caa8 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/ChecksumUtil.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/ChecksumUtil.java
@@ -25,7 +25,6 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.fs.ChecksumException;
 import org.apache.hadoop.hbase.classification.InterfaceAudience;
-import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hbase.util.ChecksumType;
 import org.apache.hadoop.util.DataChecksum;
 
@@ -87,7 +86,7 @@ public class ChecksumUtil {
    * The header is extracted from the specified HFileBlock while the
    * data-to-be-verified is extracted from 'data'.
    */
-  static boolean validateBlockChecksum(Path path, HFileBlock block,
+  static boolean validateBlockChecksum(String pathName, HFileBlock block,
     byte[] data, int hdrSize) throws IOException {
 
     // If this is an older version of the block that does not have
@@ -120,14 +119,13 @@ public class ChecksumUtil {
       LOG.info("length of data = " + data.length
           + " OnDiskDataSizeWithHeader = " + sizeWithHeader
           + " checksum type = " + cktype.getName()
-          + " file =" + path.toString()
+          + " file =" + pathName
           + " header size = " + hdrSize
           + " bytesPerChecksum = " + bytesPerChecksum);
     }
     try {
       dataChecksum.verifyChunkedSums(ByteBuffer.wrap(data, 0, sizeWithHeader),
-          ByteBuffer.wrap(data, sizeWithHeader, data.length - sizeWithHeader),
-                          path.toString(), 0);
+          ByteBuffer.wrap(data, sizeWithHeader, data.length - sizeWithHeader), pathName, 0);
     } catch (ChecksumException e) {
       return false;
     }

http://git-wip-us.apache.org/repos/asf/hbase/blob/2cf8af5b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlock.java
----------------------------------------------------------------------
diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlock.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlock.java
index 0a25825..e0719aa 100644
--- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlock.java
+++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileBlock.java
@@ -1353,21 +1353,22 @@ public class HFileBlock implements Cacheable {
     /** The filesystem used to access data */
     protected HFileSystem hfs;
 
-    /** 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;
 
     protected HFileContext fileContext;
+    // Cache the fileName
+    protected String pathName;
 
     public FSReaderImpl(FSDataInputStreamWrapper stream, long fileSize, HFileSystem hfs, Path path,
         HFileContext fileContext) throws IOException {
       this.fileSize = fileSize;
       this.hfs = hfs;
-      this.path = path;
+      if (path != null) {
+        this.pathName = path.toString();
+      }
       this.fileContext = fileContext;
       this.hdrSize = headerSize(fileContext.isUseHBaseChecksum());
 
@@ -1508,13 +1509,13 @@ public class HFileBlock implements Cacheable {
                          doVerificationThruHBaseChecksum);
       if (blk == null) {
         HFile.LOG.warn("HBase checksum verification failed for file " +
-                       path + " at offset " +
+                       pathName + " at offset " +
                        offset + " filesize " + fileSize +
                        ". Retrying read with HDFS checksums turned on...");
 
         if (!doVerificationThruHBaseChecksum) {
           String msg = "HBase checksum verification failed for file " +
-                       path + " at offset " +
+                       pathName + " at offset " +
                        offset + " filesize " + fileSize +
                        " but this cannot happen because doVerify is " +
                        doVerificationThruHBaseChecksum;
@@ -1536,13 +1537,13 @@ public class HFileBlock implements Cacheable {
                                     doVerificationThruHBaseChecksum);
         if (blk != null) {
           HFile.LOG.warn("HDFS checksum verification suceeded for file " +
-                         path + " at offset " +
+                         pathName + " at offset " +
                          offset + " filesize " + fileSize);
         }
       }
       if (blk == null && !doVerificationThruHBaseChecksum) {
         String msg = "readBlockData failed, possibly due to " +
-                     "checksum verification failed for file " + path +
+                     "checksum verification failed for file " + pathName +
                      " at offset " + offset + " filesize " + fileSize;
         HFile.LOG.warn(msg);
         throw new IOException(msg);
@@ -1744,7 +1745,7 @@ public class HFileBlock implements Cacheable {
      */
     protected boolean validateBlockChecksum(HFileBlock block,  byte[] data, int hdrSize)
         throws IOException {
-      return ChecksumUtil.validateBlockChecksum(path, block, data, hdrSize);
+      return ChecksumUtil.validateBlockChecksum(pathName, block, data, hdrSize);
     }
 
     @Override
@@ -1754,7 +1755,7 @@ public class HFileBlock implements Cacheable {
 
     @Override
     public String toString() {
-      return "hfs=" + hfs + ", path=" + path + ", fileContext=" + fileContext;
+      return "hfs=" + hfs + ", path=" + pathName + ", fileContext=" + fileContext;
     }
   }