From 3e2289f5fda5fc95bd74ab6e629b466edf3ddbef Mon Sep 17 00:00:00 2001 From: Ashish Singhi Date: Wed, 20 Jan 2016 18:47:25 +0530 Subject: [PATCH] HBASE-9393 Hbase does not closing a closed socket resulting in many CLOSE_WAIT --- .../org/apache/hadoop/hbase/io/hfile/HFile.java | 7 +- .../apache/hadoop/hbase/io/hfile/HFileBlock.java | 105 +++++++++++---------- 2 files changed, 58 insertions(+), 54 deletions(-) diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java index 669ad92..e22c0cd 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFile.java @@ -491,10 +491,11 @@ public class HFile { private static Reader pickReaderVersion(Path path, FSDataInputStreamWrapper fsdis, long size, CacheConfig cacheConf, HFileSystem hfs, Configuration conf) throws IOException { FixedFileTrailer trailer = null; + boolean isHBaseChecksum = fsdis.shouldUseHBaseChecksum(); + FSDataInputStream stream = fsdis.getStream(isHBaseChecksum); try { - boolean isHBaseChecksum = fsdis.shouldUseHBaseChecksum(); assert !isHBaseChecksum; // Initially we must read with FS checksum. - trailer = FixedFileTrailer.readFromStream(fsdis.getStream(isHBaseChecksum), size); + trailer = FixedFileTrailer.readFromStream(stream, size); switch (trailer.getMajorVersion()) { case 2: LOG.debug("Opening HFile v2 with v3 reader"); @@ -511,6 +512,8 @@ public class HFile { LOG.warn("Error closing fsdis FSDataInputStreamWrapper", t2); } throw new CorruptHFileException("Problem reading HFile Trailer from file " + path, t); + } finally { + stream.unbuffer(); } } 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 6916ba0..9fcb3b8 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 @@ -1501,61 +1501,62 @@ public class HFileBlock implements Cacheable { // guaranteed to use hdfs checksum verification. boolean doVerificationThruHBaseChecksum = streamWrapper.shouldUseHBaseChecksum(); FSDataInputStream is = streamWrapper.getStream(doVerificationThruHBaseChecksum); - - HFileBlock blk = readBlockDataInternal(is, offset, - onDiskSizeWithHeaderL, - uncompressedSize, pread, - doVerificationThruHBaseChecksum); - if (blk == null) { - HFile.LOG.warn("HBase checksum verification failed for file " + - path + " 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 " + - offset + " filesize " + fileSize + - " but this cannot happen because doVerify is " + - doVerificationThruHBaseChecksum; - HFile.LOG.warn(msg); - throw new IOException(msg); // cannot happen case here + HFileBlock blk = null; + try { + + blk = + readBlockDataInternal(is, offset, onDiskSizeWithHeaderL, uncompressedSize, pread, + doVerificationThruHBaseChecksum); + if (blk == null) { + HFile.LOG.warn("HBase checksum verification failed for file " + path + " 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 " + offset + + " filesize " + fileSize + " but this cannot happen because doVerify is " + + doVerificationThruHBaseChecksum; + HFile.LOG.warn(msg); + throw new IOException(msg); // cannot happen case here + } + HFile.checksumFailures.incrementAndGet(); // update metrics + + // If we have a checksum failure, we fall back into a mode where + // the next few reads use HDFS level checksums. We aim to make the + // next CHECKSUM_VERIFICATION_NUM_IO_THRESHOLD reads avoid + // hbase checksum verification, but since this value is set without + // holding any locks, it can so happen that we might actually do + // a few more than precisely this number. + is = this.streamWrapper.fallbackToFsChecksum(CHECKSUM_VERIFICATION_NUM_IO_THRESHOLD); + doVerificationThruHBaseChecksum = false; + blk = + readBlockDataInternal(is, offset, onDiskSizeWithHeaderL, uncompressedSize, pread, + doVerificationThruHBaseChecksum); + if (blk != null) { + HFile.LOG.warn("HDFS checksum verification suceeded for file " + path + " at offset " + + offset + " filesize " + fileSize); + } } - HFile.checksumFailures.incrementAndGet(); // update metrics - - // If we have a checksum failure, we fall back into a mode where - // the next few reads use HDFS level checksums. We aim to make the - // next CHECKSUM_VERIFICATION_NUM_IO_THRESHOLD reads avoid - // hbase checksum verification, but since this value is set without - // holding any locks, it can so happen that we might actually do - // a few more than precisely this number. - is = this.streamWrapper.fallbackToFsChecksum(CHECKSUM_VERIFICATION_NUM_IO_THRESHOLD); - doVerificationThruHBaseChecksum = false; - blk = readBlockDataInternal(is, offset, onDiskSizeWithHeaderL, - uncompressedSize, pread, - doVerificationThruHBaseChecksum); - if (blk != null) { - HFile.LOG.warn("HDFS checksum verification suceeded for file " + - path + " at offset " + - offset + " filesize " + fileSize); + if (blk == null && !doVerificationThruHBaseChecksum) { + String msg = + "readBlockData failed, possibly due to " + "checksum verification failed for file " + + path + " at offset " + offset + " filesize " + fileSize; + HFile.LOG.warn(msg); + throw new IOException(msg); } - } - if (blk == null && !doVerificationThruHBaseChecksum) { - String msg = "readBlockData failed, possibly due to " + - "checksum verification failed for file " + path + - " at offset " + offset + " filesize " + fileSize; - HFile.LOG.warn(msg); - throw new IOException(msg); - } - // If there is a checksum mismatch earlier, then retry with - // HBase checksums switched off and use HDFS checksum verification. - // This triggers HDFS to detect and fix corrupt replicas. The - // next checksumOffCount read requests will use HDFS checksums. - // The decrementing of this.checksumOffCount is not thread-safe, - // but it is harmless because eventually checksumOffCount will be - // a negative number. - streamWrapper.checksumOk(); + // If there is a checksum mismatch earlier, then retry with + // HBase checksums switched off and use HDFS checksum verification. + // This triggers HDFS to detect and fix corrupt replicas. The + // next checksumOffCount read requests will use HDFS checksums. + // The decrementing of this.checksumOffCount is not thread-safe, + // but it is harmless because eventually checksumOffCount will be + // a negative number. + streamWrapper.checksumOk(); + } finally { + is.unbuffer(); + } return blk; } -- 1.9.2.msysgit.0