From 3c006bc835c0207d77305fb0aa02c25bf107cd5b Mon Sep 17 00:00:00 2001 From: Ashish Singhi Date: Mon, 25 Jan 2016 18:11:33 +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 | 26 ++++++++++++++++++++-- .../apache/hadoop/hbase/io/hfile/HFileBlock.java | 20 ++++++++++++++++- .../hadoop/hbase/io/hfile/HFileReaderImpl.java | 9 ++++++++ 3 files changed, 52 insertions(+), 3 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 1e1835f..e6886a0 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 @@ -36,11 +36,10 @@ import java.util.SortedMap; import java.util.TreeMap; import java.util.concurrent.atomic.AtomicLong; -import org.apache.hadoop.hbase.util.ByteStringer; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.CanUnbuffer; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileStatus; @@ -50,6 +49,7 @@ import org.apache.hadoop.fs.PathFilter; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.CellComparator; import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.fs.HFileSystem; import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper; import org.apache.hadoop.hbase.io.compress.Compression; @@ -60,6 +60,7 @@ import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos; import org.apache.hadoop.hbase.protobuf.generated.HBaseProtos.BytesBytesPair; import org.apache.hadoop.hbase.protobuf.generated.HFileProtos; import org.apache.hadoop.hbase.util.BloomFilterWriter; +import org.apache.hadoop.hbase.util.ByteStringer; import org.apache.hadoop.hbase.util.Bytes; import org.apache.hadoop.hbase.util.FSUtils; import org.apache.hadoop.io.Writable; @@ -474,6 +475,11 @@ public class HFile { @VisibleForTesting boolean prefetchComplete(); + + /** + * To close only the stream's socket. HBASE-9393 + */ + void unbufferStream(); } /** @@ -513,6 +519,22 @@ public class HFile { LOG.warn("Error closing fsdis FSDataInputStreamWrapper", t2); } throw new CorruptHFileException("Problem reading HFile Trailer from file " + path, t); + } finally { + unbufferStream(fsdis); + } + } + + static void unbufferStream(FSDataInputStreamWrapper fsdis) { + boolean useHBaseChecksum = fsdis.shouldUseHBaseChecksum(); + final FSDataInputStream stream = fsdis.getStream(useHBaseChecksum); + if (stream != null && stream.getWrappedStream() instanceof CanUnbuffer) { + // Enclosing unbuffer() in try-catch just to be on defensive side. + try { + stream.unbuffer(); + } catch (Throwable e) { + LOG.error("Failed to unbuffer the stream so possibly there may be a TCP socket connection " + + "left open in CLOSE_WAIT state.", e); + } } } 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..b4b5f81 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 @@ -33,10 +33,10 @@ import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.HConstants; import org.apache.hadoop.hbase.classification.InterfaceAudience; import org.apache.hadoop.hbase.fs.HFileSystem; -import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper; import org.apache.hadoop.hbase.io.ByteArrayOutputStream; import org.apache.hadoop.hbase.io.ByteBuffInputStream; import org.apache.hadoop.hbase.io.ByteBufferSupportDataOutputStream; +import org.apache.hadoop.hbase.io.FSDataInputStreamWrapper; import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding; import org.apache.hadoop.hbase.io.encoding.HFileBlockDecodingContext; import org.apache.hadoop.hbase.io.encoding.HFileBlockDefaultDecodingContext; @@ -1311,6 +1311,11 @@ public class HFileBlock implements Cacheable { void setIncludesMemstoreTS(boolean includesMemstoreTS); void setDataBlockEncoder(HFileDataBlockEncoder encoder); + + /** + * To close only the stream's socket. HBASE-9393 + */ + void unbufferStream(); } /** @@ -1756,6 +1761,19 @@ public class HFileBlock implements Cacheable { public String toString() { return "hfs=" + hfs + ", path=" + path + ", fileContext=" + fileContext; } + + @Override + public void unbufferStream() { + // To handle concurrent reads, ensure that no other client is accessing the streams while we + // unbuffer it. + if (streamLock.tryLock()) { + try { + HFile.unbufferStream(this.streamWrapper); + } finally { + streamLock.unlock(); + } + } + } } @Override diff --git a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderImpl.java b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderImpl.java index 4db26d1..8fe5ed1 100644 --- a/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderImpl.java +++ b/hbase-server/src/main/java/org/apache/hadoop/hbase/io/hfile/HFileReaderImpl.java @@ -570,6 +570,10 @@ public class HFileReaderImpl implements HFile.Reader, Configurable { @Override public void close() { + if (!pread) { + // For seek + pread stream socket should be closed when the scanner is closed. HBASE-9393 + reader.unbufferStream(); + } this.returnBlocks(true); } @@ -1891,4 +1895,9 @@ public class HFileReaderImpl implements HFile.Reader, Configurable { public int getMajorVersion() { return 3; } + + @Override + public void unbufferStream() { + fsBlockReader.unbufferStream(); + } } -- 1.9.2.msysgit.0