Hadoop Common
  1. Hadoop Common
  2. HADOOP-2341

Datanode active connections never returns to 0

    Details

    • Type: Bug Bug
    • Status: Closed
    • Priority: Major Major
    • Resolution: Incomplete
    • Affects Version/s: 0.16.0
    • Fix Version/s: None
    • Component/s: None
    • Labels:
      None

      Description

      On trunk i continue to see the following in my data node logs:

      2007-12-03 15:46:47,696 DEBUG dfs.DataNode - XX.XX.XX.XXX:50010:Number of active connections is: 42
      2007-12-03 15:46:48,135 DEBUG dfs.DataNode - XX.XX.XX.XXX:50010:Number of active connections is: 41
      2007-12-03 15:46:48,439 DEBUG dfs.DataNode - XX.XX.XX.XXX:50010:Number of active connections is: 40
      2007-12-03 15:46:48,479 DEBUG dfs.DataNode - XX.XX.XX.XXX:50010:Number of active connections is: 39
      2007-12-03 15:46:48,611 DEBUG dfs.DataNode - XX.XX.XX.XXX:50010:Number of active connections is: 38
      2007-12-03 15:46:48,898 DEBUG dfs.DataNode - XX.XX.XX.XXX:50010:Number of active connections is: 37
      2007-12-03 15:46:48,989 DEBUG dfs.DataNode - XX.XX.XX.XXX:50010:Number of active connections is: 36
      2007-12-03 15:46:51,010 DEBUG dfs.DataNode - XX.XX.XX.XXX:50010:Number of active connections is: 35
      2007-12-03 15:46:51,758 DEBUG dfs.DataNode - XX.XX.XX.XXX:50010:Number of active connections is: 34
      2007-12-03 15:46:52,148 DEBUG dfs.DataNode - XX.XX.XX.XXX:50010:Number of active connections is: 33

      This number never returns to 0, even after many hours of no new data being manipulated or added into the DFS.

      Looking at netstat -tn i see significant amount of data in the send-q that never goes away:

      tcp 0 34240 ::ffff:XX.XX.XX.XXX:50010 ::ffff:YY.YY.YY.YY:55792 ESTABLISHED
      tcp 0 38968 ::ffff:XX.XX.XX.XXX:50010 ::ffff:YY.YY.YY.YY:38169 ESTABLISHED
      tcp 0 38456 ::ffff:XX.XX.XX.XXX:50010 ::ffff:YY.YY.YY.YY:35456 ESTABLISHED
      tcp 0 29640 ::ffff:XX.XX.XX.XXX:50010 ::ffff:YY.YY.YY.YY:59845 ESTABLISHED
      tcp 0 50168 ::ffff:XX.XX.XX.XXX:50010 ::ffff:YY.YY.YY.YY:44584 ESTABLISHED

      When sniffing the network I see that the remote side (YY.YY.YY.YY) is returning a window size of 0
      16:11:41.760474 IP XX.XX.XX.XXX.50010 > YY.YY.YY.YY.44584: . ack 3339984123 win 46 <nop,nop,timestamp 1786247180 885681789>
      16:11:41.761597 IP YY.YY.YY.YY.44584 > XX.XX.XX.XXX.50010: . ack 1 win 0 <nop,nop,timestamp 885801786 1775711351>

      Then we look at the stack traces on each datanode, I will have tons of threads that never go away in the following trace:

      Thread 6516 (org.apache.hadoop.dfs.DataNode$DataXceiver@166068b6):
        State: RUNNABLE
        Blocked count: 0
        Waited count: 0
        Stack:
          java.net.SocketOutputStream.socketWrite0(Native Method)
          java.net.SocketOutputStream.socketWrite(SocketOutputStream.java:92)
          java.net.SocketOutputStream.write(SocketOutputStream.java:136)
          java.io.BufferedOutputStream.flushBuffer(BufferedOutputStream.java:65)
          java.io.BufferedOutputStream.write(BufferedOutputStream.java:109)
          java.io.DataOutputStream.write(DataOutputStream.java:90)
          org.apache.hadoop.dfs.DataNode$BlockSender.sendChunk(DataNode.java:1400)
          org.apache.hadoop.dfs.DataNode$BlockSender.sendBlock(DataNode.java:1433)
          org.apache.hadoop.dfs.DataNode$DataXceiver.readBlock(DataNode.java:904)
          org.apache.hadoop.dfs.DataNode$DataXceiver.run(DataNode.java:849)
          java.lang.Thread.run(Thread.java:619)
      

      Unfortunately there's very little in the logs with exceptions that could point to this. I have some exceptions the following, but nothing that points to problems between XX and YY:

      2007-12-02 11:19:47,889 WARN  dfs.DataNode - Unexpected error trying to delete block blk_4515246476002110310. Block not found in blockMap. 
      2007-12-02 11:19:47,922 WARN  dfs.DataNode - java.io.IOException: Error in deleting blocks.
              at org.apache.hadoop.dfs.FSDataset.invalidate(FSDataset.java:750)
              at org.apache.hadoop.dfs.DataNode.processCommand(DataNode.java:675)
              at org.apache.hadoop.dfs.DataNode.offerService(DataNode.java:569)
              at org.apache.hadoop.dfs.DataNode.run(DataNode.java:1720)
              at java.lang.Thread.run(Thread.java:619)
      
      1. stacks-XX.XX.XX.XXX.txt
        30 kB
        Paul Saab
      2. stacks-YY.YY.YY.YY.txt
        41 kB
        Paul Saab
      3. hregionserver-stack.txt
        10 kB
        Paul Saab
      4. dfsclient.patch
        2 kB
        stack

        Issue Links

          Activity

          Paul Saab created issue -
          Hide
          dhruba borthakur added a comment -

          It appears that there are HDFS clients running on machines marked as yy.yy.yy.yy. They are trying to read files from the datanodes.

          Show
          dhruba borthakur added a comment - It appears that there are HDFS clients running on machines marked as yy.yy.yy.yy. They are trying to read files from the datanodes.
          Hide
          Paul Saab added a comment -

          but data is never read. Both sides of the connection stack traces show each
          machine in write.

          Show
          Paul Saab added a comment - but data is never read. Both sides of the connection stack traces show each machine in write.
          Hide
          Michael Bieniosek added a comment -

          I am seeing this too. If I look at datanode:50075/stacks, I see 70 threads stuck with:

          Thread 3023977 (org.apache.hadoop.dfs.DataNode$DataXceiver@28075c96):
          State: RUNNABLE
          Blocked count: 0
          Waited count: 0
          Stack:
          java.net.SocketOutputStream.socketWrite0(Native Method)
          java.net.SocketOutputStream.socketWrite(Unknown Source)
          java.net.SocketOutputStream.write(Unknown Source)
          java.io.BufferedOutputStream.flushBuffer(Unknown Source)
          java.io.BufferedOutputStream.write(Unknown Source)
          java.io.DataOutputStream.write(Unknown Source)
          org.apache.hadoop.dfs.DataNode$BlockSender.sendChunk(DataNode.java:1175)
          org.apache.hadoop.dfs.DataNode$BlockSender.sendBlock(DataNode.java:1208)
          org.apache.hadoop.dfs.DataNode$DataXceiver.readBlock(DataNode.java:850)
          org.apache.hadoop.dfs.DataNode$DataXceiver.run(DataNode.java:801)
          java.lang.Thread.run(Unknown Source)

          Show
          Michael Bieniosek added a comment - I am seeing this too. If I look at datanode:50075/stacks, I see 70 threads stuck with: Thread 3023977 (org.apache.hadoop.dfs.DataNode$DataXceiver@28075c96): State: RUNNABLE Blocked count: 0 Waited count: 0 Stack: java.net.SocketOutputStream.socketWrite0(Native Method) java.net.SocketOutputStream.socketWrite(Unknown Source) java.net.SocketOutputStream.write(Unknown Source) java.io.BufferedOutputStream.flushBuffer(Unknown Source) java.io.BufferedOutputStream.write(Unknown Source) java.io.DataOutputStream.write(Unknown Source) org.apache.hadoop.dfs.DataNode$BlockSender.sendChunk(DataNode.java:1175) org.apache.hadoop.dfs.DataNode$BlockSender.sendBlock(DataNode.java:1208) org.apache.hadoop.dfs.DataNode$DataXceiver.readBlock(DataNode.java:850) org.apache.hadoop.dfs.DataNode$DataXceiver.run(DataNode.java:801) java.lang.Thread.run(Unknown Source)
          Hide
          Michael Bieniosek added a comment -

          In DataNode.BlockSender.sendBlock, we have

                  while (endOffset > offset) {
                    // Write one data chunk per loop.
                    long len = sendChunk();
                    offset += len;
                    totalRead += len + checksumSize;
                  }
          

          In the BlockSender constructor, we have

                  if (length >= 0) {
                    // Make sure endOffset points to end of a checksumed chunk.
                    long tmpLen = startOffset + length + (startOffset - offset);
                    if (tmpLen % bytesPerChecksum != 0) {
                      tmpLen += (bytesPerChecksum - tmpLen % bytesPerChecksum);
                    }
                    if (tmpLen < endOffset) {
                      endOffset = tmpLen;
                    }
                  }
          

          So in some cases, endOffset can include extra bytes for checksums in the constructor. However, checksum bytes are never added to offset when it is compared to endOffset in the sendBlock method. I believe this may be the problem.

          Show
          Michael Bieniosek added a comment - In DataNode.BlockSender.sendBlock, we have while (endOffset > offset) { // Write one data chunk per loop. long len = sendChunk(); offset += len; totalRead += len + checksumSize; } In the BlockSender constructor, we have if (length >= 0) { // Make sure endOffset points to end of a checksumed chunk. long tmpLen = startOffset + length + (startOffset - offset); if (tmpLen % bytesPerChecksum != 0) { tmpLen += (bytesPerChecksum - tmpLen % bytesPerChecksum); } if (tmpLen < endOffset) { endOffset = tmpLen; } } So in some cases, endOffset can include extra bytes for checksums in the constructor. However, checksum bytes are never added to offset when it is compared to endOffset in the sendBlock method. I believe this may be the problem.
          Hide
          Raghu Angadi added a comment -

          tcp 0 50168 ::ffff:XX.XX.XX.XXX:50010 ::ffff:YY.YY.YY.YY:44584 ESTABLISHED

          When sniffing the network I see that the remote side (YY.YY.YY.YY) is returning a window size of 0
          16:11:41.760474 IP XX.XX.XX.XXX.50010 > YY.YY.YY.YY.44584: . ack 3339984123 win 46 <nop,nop,timestamp 1786247180 885681789>
          16:11:41.761597 IP YY.YY.YY.YY.44584 > XX.XX.XX.XXX.50010: . ack 1 win 0 <nop,nop,timestamp 885801786 1775711351>

          This is pretty odd. If I looked at just netstat, I would think client is reading a block. But if I look at the tcpdump, I would think client is writing data (note DataNode is acking 3Mb and client is acking 1 byte). Are you sure these two correspond to the same TCP connection?

          Please include client side stack if you can. Also, if this is on Linux, with strace and lsof, you might be able to match stacktrace and the tcp connection involved on the Datanode.

          Show
          Raghu Angadi added a comment - tcp 0 50168 ::ffff:XX.XX.XX.XXX:50010 ::ffff:YY.YY.YY.YY:44584 ESTABLISHED When sniffing the network I see that the remote side (YY.YY.YY.YY) is returning a window size of 0 16:11:41.760474 IP XX.XX.XX.XXX.50010 > YY.YY.YY.YY.44584: . ack 3339984123 win 46 <nop,nop,timestamp 1786247180 885681789> 16:11:41.761597 IP YY.YY.YY.YY.44584 > XX.XX.XX.XXX.50010: . ack 1 win 0 <nop,nop,timestamp 885801786 1775711351> This is pretty odd. If I looked at just netstat, I would think client is reading a block. But if I look at the tcpdump, I would think client is writing data (note DataNode is acking 3Mb and client is acking 1 byte). Are you sure these two correspond to the same TCP connection? Please include client side stack if you can. Also, if this is on Linux, with strace and lsof, you might be able to match stacktrace and the tcp connection involved on the Datanode.
          Hide
          Paul Saab added a comment -

          The connections are still sitting like this for the last few hours.. Here's the trace from both sides at the same time.

          20:17:40.669439 IP XX.XX.XX.XXX.50010 > YY.YY.YY.YY.44584: . ack 3339984123 win 46 <nop,nop,timestamp 1801007426 900441594>
          20:17:40.670752 IP YY.YY.YY.YY.44584 > XX.XX.XX.XXX.50010: . ack 1 win 0 <nop,nop,timestamp 900561591 1775711351>

          20:17:40.683665 IP XX.XX.XX.XXX.50010 > YY.YY.YY.YY.44584: . ack 1 win 46 <nop,nop,timestamp 1801007426 900441594>
          20:17:40.683685 IP YY.YY.YY.YY.44584 > XX.XX.XX.XXX.50010: . ack 1 win 0 <nop,nop,timestamp 900561591 1775711351>

          lsof
          java 11091 user 363u IPv6 1044460194 TCP YY.YY.YY.YY:44584->XX.XX.XX.XXX:50010 (ESTABLISHED)

          java 8853 user 53u IPv6 335510414 TCP XX.XX.XX.XXX:50010->YY.YY.YY.YY:44584 (ESTABLISHED)

          strace shows nothing even when tcp retransmits,

          strace -p 11091
          Process 11091 attached - interrupt to quit
          futex(0x4022b9e0, FUTEX_WAIT, 11117, NULL^[[A^[[A <unfinished ...>
          Process 11091 detached

          Show
          Paul Saab added a comment - The connections are still sitting like this for the last few hours.. Here's the trace from both sides at the same time. 20:17:40.669439 IP XX.XX.XX.XXX.50010 > YY.YY.YY.YY.44584: . ack 3339984123 win 46 <nop,nop,timestamp 1801007426 900441594> 20:17:40.670752 IP YY.YY.YY.YY.44584 > XX.XX.XX.XXX.50010: . ack 1 win 0 <nop,nop,timestamp 900561591 1775711351> – 20:17:40.683665 IP XX.XX.XX.XXX.50010 > YY.YY.YY.YY.44584: . ack 1 win 46 <nop,nop,timestamp 1801007426 900441594> 20:17:40.683685 IP YY.YY.YY.YY.44584 > XX.XX.XX.XXX.50010: . ack 1 win 0 <nop,nop,timestamp 900561591 1775711351> lsof java 11091 user 363u IPv6 1044460194 TCP YY.YY.YY.YY:44584->XX.XX.XX.XXX:50010 (ESTABLISHED) java 8853 user 53u IPv6 335510414 TCP XX.XX.XX.XXX:50010->YY.YY.YY.YY:44584 (ESTABLISHED) strace shows nothing even when tcp retransmits, strace -p 11091 Process 11091 attached - interrupt to quit futex(0x4022b9e0, FUTEX_WAIT, 11117, NULL^[[A^[[A <unfinished ...> Process 11091 detached
          Hide
          Paul Saab added a comment -

          stacktrace from XX.XX.XX.XXX host

          Show
          Paul Saab added a comment - stacktrace from XX.XX.XX.XXX host
          Paul Saab made changes -
          Field Original Value New Value
          Attachment jstack-pid8853.txt [ 12370906 ]
          Hide
          Paul Saab added a comment -

          jstack from YY.YY.YY.YY

          Show
          Paul Saab added a comment - jstack from YY.YY.YY.YY
          Paul Saab made changes -
          Attachment jstack-pid11091.txt [ 12370907 ]
          Paul Saab made changes -
          Attachment jstack-pid11091.txt [ 12370907 ]
          Paul Saab made changes -
          Attachment jstack-pid8853.txt [ 12370906 ]
          Paul Saab made changes -
          Attachment stacks-XX.XX.XX.XXX.txt [ 12370909 ]
          Paul Saab made changes -
          Attachment stacks-YY.YY.YY.YY.txt [ 12370910 ]
          Hide
          Raghu Angadi added a comment -

          Thanks Paul, will look at the traces.
          The pid in lsof might not be the pid of the thread that is trying to write. For a given stack, jstack
          lists the pid as a hex, then you need to strace to see which fd it is trying to write to.

          Show
          Raghu Angadi added a comment - Thanks Paul, will look at the traces. The pid in lsof might not be the pid of the thread that is trying to write. For a given stack, jstack lists the pid as a hex, then you need to strace to see which fd it is trying to write to.
          Hide
          Paul Saab added a comment -

          This is related to hbase. The question is, what's going on to keep all that data in the send and receive queues of each machine and sockets being blocked forever.

          Show
          Paul Saab added a comment - This is related to hbase. The question is, what's going on to keep all that data in the send and receive queues of each machine and sockets being blocked forever.
          Hide
          Raghu Angadi added a comment -

          The stacks on both sides are DataNodes. So these data transfer (attempts) are between two datanodes?

          Show
          Raghu Angadi added a comment - The stacks on both sides are DataNodes. So these data transfer (attempts) are between two datanodes?
          Paul Saab made changes -
          Attachment hregionserver-stack.txt [ 12370955 ]
          Hide
          Paul Saab added a comment -

          I thought it was, but after futher investigation it is between a datanode
          and a HRegionServer after I started digging a bit more and reran a job to
          verify. I'll be attaching the trace from a HRegionServer. I cannot see
          which thread would have that file open though through jstack or lsof. I
          straced every thread and none seemed to be stuck in read or reading any data
          from the socket between the two hosts.

          Show
          Paul Saab added a comment - I thought it was, but after futher investigation it is between a datanode and a HRegionServer after I started digging a bit more and reran a job to verify. I'll be attaching the trace from a HRegionServer. I cannot see which thread would have that file open though through jstack or lsof. I straced every thread and none seemed to be stuck in read or reading any data from the socket between the two hosts.
          Hide
          Michael Bieniosek added a comment -

          It seems like these socket reads/writes should time out eventually. But the 70 threads on my datanode are still waiting on write.

          Show
          Michael Bieniosek added a comment - It seems like these socket reads/writes should time out eventually. But the 70 threads on my datanode are still waiting on write.
          Hide
          Raghu Angadi added a comment -

          > none seemed to be stuck in read or reading any data from the socket between the two hosts.
          That could be the actual problem. If a client opens a HDFS file and stops reading in the middle, DataNode could be stuck forever. DataNode sets read timeout but not write timeout. I think it should (I will file a jira for it).

          I think we need to look for cases where HRegionServer might not be closing a DFS inputstream. You could /proc/HRegionServerPid/fds to see if it looks like there is an FD leak.

          Show
          Raghu Angadi added a comment - > none seemed to be stuck in read or reading any data from the socket between the two hosts. That could be the actual problem. If a client opens a HDFS file and stops reading in the middle, DataNode could be stuck forever. DataNode sets read timeout but not write timeout. I think it should (I will file a jira for it). I think we need to look for cases where HRegionServer might not be closing a DFS inputstream. You could /proc/HRegionServerPid/fds to see if it looks like there is an FD leak.
          Hide
          stack added a comment -

          I did a review of hbase. Found a leak, HADOOP-2362, but it doesn't seem to fix the problem.

          But on review, I'm now thinking that it could the way that Hbase is using hdfs that is making for all the occupied threads over in the datanode. Below I explain why I think this. Would appreciate any input or alternate suggestions for Hbase usage of hdfs.

          After doing an upload, I let the cluster alone till it hit a steady state. Here's what netstat looks like if I only look for the ESTABLISHED connections:

          [stack@aa0-000-13 hbase]$ netstat -tn|grep ESTABLISHED|sort > /tmp/ns.txt
          ..
          tcp        0  95488 ::ffff:XX.XX.XX.140:50010  ::ffff:XX.XX.XX.140:32853  ESTABLISHED
          tcp        0  95488 ::ffff:XX.XX.XX.140:50010  ::ffff:XX.XX.XX.140:32860  ESTABLISHED
          tcp        0  95488 ::ffff:XX.XX.XX.140:50010  ::ffff:XX.XX.XX.140:32866  ESTABLISHED
          tcp        0  95488 ::ffff:XX.XX.XX.140:50010  ::ffff:XX.XX.XX.140:32872  ESTABLISHED
          ...
          tcp    71976      0 ::ffff:XX.XX.XX.140:60511  ::ffff:XX.XX.XX.140:50010  ESTABLISHED 
          tcp    71976      0 ::ffff:XX.XX.XX.140:60513  ::ffff:XX.XX.XX.140:50010  ESTABLISHED 
          tcp    71976      0 ::ffff:XX.XX.XX.140:60711  ::ffff:XX.XX.XX.140:50010  ESTABLISHED 
          tcp    71976      0 ::ffff:XX.XX.XX.140:60713  ::ffff:XX.XX.XX.140:50010  ESTABLISHED 
          tcp    71976      0 ::ffff:XX.XX.XX.140:60999  ::ffff:XX.XX.XX.140:50010  ESTABLISHED
          

          The number remained constant.

          Counting the number of open connections, I was able to make rough correlation with count of open MapFiles in hbase.

          I wrote a little program:

             public static void main(String[] args) throws Exception {
              HBaseConfiguration conf = new HBaseConfiguration();
              MapFile.Reader reader =
                new MapFile.Reader(FileSystem.get(conf), args[0], conf);
              Thread.sleep(1000 * 1000);
            }
          

          Everytime I ran my program on XX.XX.XX.140, the ESTABLISHED count went up by one as did the number of occupied threads in the datanode.

          Hbase keeps open MapFile.Readers. The number of concurrent open Readers goes up as a server deploys more and more regions.

          Is there something wrong w/ this pattern – should hbase not be doing this – or is there something wrong w/ datanode such that a thread is completely occupied while a client has a MapFile.Reader open or is this just the way its supposed to work?

          Show
          stack added a comment - I did a review of hbase. Found a leak, HADOOP-2362 , but it doesn't seem to fix the problem. But on review, I'm now thinking that it could the way that Hbase is using hdfs that is making for all the occupied threads over in the datanode. Below I explain why I think this. Would appreciate any input or alternate suggestions for Hbase usage of hdfs. After doing an upload, I let the cluster alone till it hit a steady state. Here's what netstat looks like if I only look for the ESTABLISHED connections: [stack@aa0-000-13 hbase]$ netstat -tn|grep ESTABLISHED|sort > /tmp/ns.txt .. tcp 0 95488 ::ffff:XX.XX.XX.140:50010 ::ffff:XX.XX.XX.140:32853 ESTABLISHED tcp 0 95488 ::ffff:XX.XX.XX.140:50010 ::ffff:XX.XX.XX.140:32860 ESTABLISHED tcp 0 95488 ::ffff:XX.XX.XX.140:50010 ::ffff:XX.XX.XX.140:32866 ESTABLISHED tcp 0 95488 ::ffff:XX.XX.XX.140:50010 ::ffff:XX.XX.XX.140:32872 ESTABLISHED ... tcp 71976 0 ::ffff:XX.XX.XX.140:60511 ::ffff:XX.XX.XX.140:50010 ESTABLISHED tcp 71976 0 ::ffff:XX.XX.XX.140:60513 ::ffff:XX.XX.XX.140:50010 ESTABLISHED tcp 71976 0 ::ffff:XX.XX.XX.140:60711 ::ffff:XX.XX.XX.140:50010 ESTABLISHED tcp 71976 0 ::ffff:XX.XX.XX.140:60713 ::ffff:XX.XX.XX.140:50010 ESTABLISHED tcp 71976 0 ::ffff:XX.XX.XX.140:60999 ::ffff:XX.XX.XX.140:50010 ESTABLISHED The number remained constant. Counting the number of open connections, I was able to make rough correlation with count of open MapFiles in hbase. I wrote a little program: public static void main( String [] args) throws Exception { HBaseConfiguration conf = new HBaseConfiguration(); MapFile.Reader reader = new MapFile.Reader(FileSystem.get(conf), args[0], conf); Thread .sleep(1000 * 1000); } Everytime I ran my program on XX.XX.XX.140, the ESTABLISHED count went up by one as did the number of occupied threads in the datanode. Hbase keeps open MapFile.Readers. The number of concurrent open Readers goes up as a server deploys more and more regions. Is there something wrong w/ this pattern – should hbase not be doing this – or is there something wrong w/ datanode such that a thread is completely occupied while a client has a MapFile.Reader open or is this just the way its supposed to work?
          Hide
          stack added a comment -

          I stopped the cluster. The connections went away. I restarted the cluster. They all came back.

          Show
          stack added a comment - I stopped the cluster. The connections went away. I restarted the cluster. They all came back.
          Hide
          Raghu Angadi added a comment -

          > Hbase keeps open MapFile.Readers. The number of concurrent open Readers goes up as a server deploys more and more regions.

          As mentioned earlier, if you open a DFS file you do take up one fd on client and one fd (and a thread) on datanode. What did you expect instead? Unlike normal file open, MapFile.Reader() seem to seek inside the constructor.. that pretty much assures client opens a socket to the datanode. Also see HADOOP-2346.

          Show
          Raghu Angadi added a comment - > Hbase keeps open MapFile.Readers. The number of concurrent open Readers goes up as a server deploys more and more regions. As mentioned earlier, if you open a DFS file you do take up one fd on client and one fd (and a thread) on datanode. What did you expect instead? Unlike normal file open, MapFile.Reader() seem to seek inside the constructor.. that pretty much assures client opens a socket to the datanode. Also see HADOOP-2346 .
          Hide
          Raghu Angadi added a comment -

          What is the desired bevaviour w.r.t Hbase with this pattern?

          Show
          Raghu Angadi added a comment - What is the desired bevaviour w.r.t Hbase with this pattern?
          Hide
          Michael Bieniosek added a comment -

          I've noticed that the regionserver is using select for its io, which doesn't tie up threads, whereas the datanode seems to have one thread open/blocked for each write. Is this intentional? Shouldn't they be using the same mechanism?

          Show
          Michael Bieniosek added a comment - I've noticed that the regionserver is using select for its io, which doesn't tie up threads, whereas the datanode seems to have one thread open/blocked for each write. Is this intentional? Shouldn't they be using the same mechanism?
          Hide
          Raghu Angadi added a comment -

          Thread is not the only cost. We pay for buffers, tcp connections and more. Why does the regionserver want to keep the connections open for so long?

          Show
          Raghu Angadi added a comment - Thread is not the only cost. We pay for buffers, tcp connections and more. Why does the regionserver want to keep the connections open for so long?
          Hide
          stack added a comment -

          Raghu: Thanks for responding.

          Now we've done the tracing, I suppose it makes sense that an open hdfs file descriptor consumes a socket, thread, buffers, etc. in the data node. That the cost is so high was a bit of a surprise going by the client side with its having only one connection to the datanode (Are there plans to do as Michael B asks above making server side do selects instead of a socket-per file?).

          I found some leaks after instrumenting SequenceFile and I upped the size of the files we keep over hdfs so we have less open files at any one time. This helps some. There still seems to be an issue w/ our not closing down everything as I see outstanding CLOSE_WAITS after I'm done with an upload. I'm now trying to instrument DFSClient to see if I can figure whats going on.

          In hbase, a table is made of regions; regions are made of stores and stores are made of mapfiles where the key is row/column/timestamp and the value is the cell data. hbase keeps open all mapfiles so access to a 'row' is 'fast'. Open mapfiles will have their index in memory, the socket connection up, etc. The alternative of opening a file on every row access, I'd imagine, would be slower.

          Desire is consuming as few resources as possible in hdfs while keeping access as fast as possible. Any suggestions on how we might improve our usage would be appreciated.

          Show
          stack added a comment - Raghu: Thanks for responding. Now we've done the tracing, I suppose it makes sense that an open hdfs file descriptor consumes a socket, thread, buffers, etc. in the data node. That the cost is so high was a bit of a surprise going by the client side with its having only one connection to the datanode (Are there plans to do as Michael B asks above making server side do selects instead of a socket-per file?). I found some leaks after instrumenting SequenceFile and I upped the size of the files we keep over hdfs so we have less open files at any one time. This helps some. There still seems to be an issue w/ our not closing down everything as I see outstanding CLOSE_WAITS after I'm done with an upload. I'm now trying to instrument DFSClient to see if I can figure whats going on. In hbase, a table is made of regions; regions are made of stores and stores are made of mapfiles where the key is row/column/timestamp and the value is the cell data. hbase keeps open all mapfiles so access to a 'row' is 'fast'. Open mapfiles will have their index in memory, the socket connection up, etc. The alternative of opening a file on every row access, I'd imagine, would be slower. Desire is consuming as few resources as possible in hdfs while keeping access as fast as possible. Any suggestions on how we might improve our usage would be appreciated.
          Hide
          Raghu Angadi added a comment -

          > hbase keeps open all mapfiles so access to a 'row' is 'fast'.

          I guess what you are trying to avoid is talking to Namenode for file information each time. Does pread() (position read) suite you better? These files seem idle for most of the time. When you actually read, how much data do you read? I think there are going to be more users of pread().. for now it at least as fast as regular read.. there might some more optimizations in for this. Depending on amount of data you are reading, cost and latency of connection establishment might be negligible.

          Show
          Raghu Angadi added a comment - > hbase keeps open all mapfiles so access to a 'row' is 'fast'. I guess what you are trying to avoid is talking to Namenode for file information each time. Does pread() (position read) suite you better? These files seem idle for most of the time. When you actually read, how much data do you read? I think there are going to be more users of pread().. for now it at least as fast as regular read.. there might some more optimizations in for this. Depending on amount of data you are reading, cost and latency of connection establishment might be negligible.
          Hide
          stack added a comment -

          Ok. So, now I think we're back around to Pauls' original complaint; that active connections never goes to zero.

          I instrumented DFSClient (See attached patch – maybe I should go ahead and make an lsof-like tool for hdfs cients keeping a map of open files adding file on open and removing from the map on close?). Every open during a test run has a corresponding close. After the test run, I see this in datanode log:

          ...
          2007-12-08 00:56:06,429 INFO org.apache.hadoop.dfs.DataNode: XX.XX.XX.140:50010Served block blk_-107599750641758201 to /XX.XX.XX.140
          2007-12-08 00:56:06,429 DEBUG org.apache.hadoop.dfs.DataNode: XX.XX.XX.140:50010:Number of active connections is: 10
          2007-12-08 00:56:06,431 DEBUG org.apache.hadoop.dfs.DataNode: Number of active connections is: 10
          2007-12-08 00:56:06,431 INFO org.apache.hadoop.dfs.DataNode: XX.XX.XX.140:50010Served block blk_8573562922806881814 to /XX.XX.XX.140
          2007-12-08 00:56:06,431 DEBUG org.apache.hadoop.dfs.DataNode: XX.XX.XX.140:50010:Number of active connections is: 10
          ..
          

          ... and this when I look w/ netstat:

          [stack@aa0-000-13 ~]$ netstat -tn|grep -v ESTABLISHED|grep CLOSE_WAIT
          tcp       38      0 XX.XX.XX.140:51141         208.76.46.110:389           CLOSE_WAIT   
          tcp       38      0 XX.XX.XX.140:55612         208.76.46.110:389           CLOSE_WAIT   
          tcp       38      0 fdfb:c1e8:7640:2:2e0::53696 fdfb:c1e8:7640:6:2e0:81:389 CLOSE_WAIT  
          tcp        1      0 ::ffff:XX.XX.XX.140:56087  ::ffff:XX.XX.XX.140:50010  CLOSE_WAIT    
          tcp        1      0 ::ffff:XX.XX.XX.140:56088  ::ffff:XX.XX.XX.140:50010  CLOSE_WAIT    
          tcp        1      0 ::ffff:XX.XX.XX.140:53432  ::ffff:XX.XX.XX.140:50010  CLOSE_WAIT    
          tcp    37585      0 ::ffff:XX.XX.XX.140:53428  ::ffff:XX.XX.XX.140:50010  CLOSE_WAIT    
          tcp     1893      0 ::ffff:XX.XX.XX.140:53430  ::ffff:XX.XX.XX.140:50010  CLOSE_WAIT    
          tcp    28174      0 ::ffff:XX.XX.XX.140:53684  ::ffff:XX.XX.XX.140:50010  CLOSE_WAIT    
          tcp     1078      0 ::ffff:XX.XX.XX.140:53686  ::ffff:XX.XX.XX.140:50010  CLOSE_WAIT    
          tcp        1      0 ::ffff:XX.XX.XX.140:39643  ::ffff:XX.XX.XX.140:50010  CLOSE_WAIT    
          tcp    28230      0 ::ffff:XX.XX.XX.140:39777  ::ffff:XX.XX.XX.140:50010  CLOSE_WAIT    
          tcp    28230      0 ::ffff:XX.XX.XX.140:39782  ::ffff:XX.XX.XX.140:50010  CLOSE_WAIT    
          tcp        1      0 ::ffff:XX.XX.XX.140:39831  ::ffff:XX.XX.XX.140:50010  CLOSE_WAIT    
          tcp        1      0 ::ffff:XX.XX.XX.140:39832  ::ffff:XX.XX.XX.140:50010  CLOSE_WAIT    
          tcp        0      0 ::ffff:XX.XX.XX.140:40871  ::ffff:XX.XX.XX.140:50010  CLOSE_WAIT    
          tcp        1      0 ::ffff:XX.XX.XX.140:40872  ::ffff:XX.XX.XX.140:50010  CLOSE_WAIT    
          tcp        1      0 ::ffff:XX.XX.XX.140:41448  ::ffff:XX.XX.XX.140:50010  CLOSE_WAIT    
          tcp    34056      0 ::ffff:XX.XX.XX.140:41444  ::ffff:XX.XX.XX.140:50010  CLOSE_WAIT    
          tcp     1848      0 ::ffff:XX.XX.XX.140:41446  ::ffff:XX.XX.XX.140:50010  CLOSE_WAIT
          

          As far as I can tell, the application – an hbase regionserver – is calling close but the close is not being run properly inside in DFSClient. Anyone have suggestions on how I might figure whats up w/ DFSClient or, if its hbase, how else I might go about looking for an improper close?

          Show
          stack added a comment - Ok. So, now I think we're back around to Pauls' original complaint; that active connections never goes to zero. I instrumented DFSClient (See attached patch – maybe I should go ahead and make an lsof-like tool for hdfs cients keeping a map of open files adding file on open and removing from the map on close?). Every open during a test run has a corresponding close. After the test run, I see this in datanode log: ... 2007-12-08 00:56:06,429 INFO org.apache.hadoop.dfs.DataNode: XX.XX.XX.140:50010Served block blk_-107599750641758201 to /XX.XX.XX.140 2007-12-08 00:56:06,429 DEBUG org.apache.hadoop.dfs.DataNode: XX.XX.XX.140:50010: Number of active connections is: 10 2007-12-08 00:56:06,431 DEBUG org.apache.hadoop.dfs.DataNode: Number of active connections is: 10 2007-12-08 00:56:06,431 INFO org.apache.hadoop.dfs.DataNode: XX.XX.XX.140:50010Served block blk_8573562922806881814 to /XX.XX.XX.140 2007-12-08 00:56:06,431 DEBUG org.apache.hadoop.dfs.DataNode: XX.XX.XX.140:50010: Number of active connections is: 10 .. ... and this when I look w/ netstat: [stack@aa0-000-13 ~]$ netstat -tn|grep -v ESTABLISHED|grep CLOSE_WAIT tcp 38 0 XX.XX.XX.140:51141 208.76.46.110:389 CLOSE_WAIT tcp 38 0 XX.XX.XX.140:55612 208.76.46.110:389 CLOSE_WAIT tcp 38 0 fdfb:c1e8:7640:2:2e0::53696 fdfb:c1e8:7640:6:2e0:81:389 CLOSE_WAIT tcp 1 0 ::ffff:XX.XX.XX.140:56087 ::ffff:XX.XX.XX.140:50010 CLOSE_WAIT tcp 1 0 ::ffff:XX.XX.XX.140:56088 ::ffff:XX.XX.XX.140:50010 CLOSE_WAIT tcp 1 0 ::ffff:XX.XX.XX.140:53432 ::ffff:XX.XX.XX.140:50010 CLOSE_WAIT tcp 37585 0 ::ffff:XX.XX.XX.140:53428 ::ffff:XX.XX.XX.140:50010 CLOSE_WAIT tcp 1893 0 ::ffff:XX.XX.XX.140:53430 ::ffff:XX.XX.XX.140:50010 CLOSE_WAIT tcp 28174 0 ::ffff:XX.XX.XX.140:53684 ::ffff:XX.XX.XX.140:50010 CLOSE_WAIT tcp 1078 0 ::ffff:XX.XX.XX.140:53686 ::ffff:XX.XX.XX.140:50010 CLOSE_WAIT tcp 1 0 ::ffff:XX.XX.XX.140:39643 ::ffff:XX.XX.XX.140:50010 CLOSE_WAIT tcp 28230 0 ::ffff:XX.XX.XX.140:39777 ::ffff:XX.XX.XX.140:50010 CLOSE_WAIT tcp 28230 0 ::ffff:XX.XX.XX.140:39782 ::ffff:XX.XX.XX.140:50010 CLOSE_WAIT tcp 1 0 ::ffff:XX.XX.XX.140:39831 ::ffff:XX.XX.XX.140:50010 CLOSE_WAIT tcp 1 0 ::ffff:XX.XX.XX.140:39832 ::ffff:XX.XX.XX.140:50010 CLOSE_WAIT tcp 0 0 ::ffff:XX.XX.XX.140:40871 ::ffff:XX.XX.XX.140:50010 CLOSE_WAIT tcp 1 0 ::ffff:XX.XX.XX.140:40872 ::ffff:XX.XX.XX.140:50010 CLOSE_WAIT tcp 1 0 ::ffff:XX.XX.XX.140:41448 ::ffff:XX.XX.XX.140:50010 CLOSE_WAIT tcp 34056 0 ::ffff:XX.XX.XX.140:41444 ::ffff:XX.XX.XX.140:50010 CLOSE_WAIT tcp 1848 0 ::ffff:XX.XX.XX.140:41446 ::ffff:XX.XX.XX.140:50010 CLOSE_WAIT As far as I can tell, the application – an hbase regionserver – is calling close but the close is not being run properly inside in DFSClient. Anyone have suggestions on how I might figure whats up w/ DFSClient or, if its hbase, how else I might go about looking for an improper close?
          stack made changes -
          Attachment dfsclient.patch [ 12371274 ]
          Hide
          Raghu Angadi added a comment -

          I briefly looked at DFSClient and could not find anything suspicious. Please let us know if you do or if there is a way to reproduce this.

          Show
          Raghu Angadi added a comment - I briefly looked at DFSClient and could not find anything suspicious. Please let us know if you do or if there is a way to reproduce this.
          Hide
          Raghu Angadi added a comment -

          You can try printing file name and local address:port each time a socket is opened and closed in DFSInputStream.
          This will match filenames and the sockets from netstat.

          Show
          Raghu Angadi added a comment - You can try printing file name and local address:port each time a socket is opened and closed in DFSInputStream. This will match filenames and the sockets from netstat.
          Hide
          stack added a comment -

          Thanks for the suggestion Raghu. Let me try it.

          Show
          stack added a comment - Thanks for the suggestion Raghu. Let me try it.
          Hide
          stack added a comment - - edited

          On how to reproduce, first apply this to MapFile:

          Index: src/java/org/apache/hadoop/io/MapFile.java
          ===================================================================
          --- src/java/org/apache/hadoop/io/MapFile.java  (revision 603416)
          +++ src/java/org/apache/hadoop/io/MapFile.java  (working copy)
          @@ -563,8 +563,12 @@
               String out = args[1];
           
               Configuration conf = new Configuration();
          -    FileSystem fs = FileSystem.getLocal(conf);
          +    // FileSystem fs = FileSystem.getLocal(conf);
          +    FileSystem fs = FileSystem.get(conf);
               MapFile.Reader reader = new MapFile.Reader(fs, in, conf);
          +    LOG.info("READER opened -- go do a netstat -tn");
          +    Thread.sleep(60*1000);
          +    LOG.info("READER moving on to invoke next");
               MapFile.Writer writer =
                 new MapFile.Writer(conf, fs, out, reader.getKeyClass(), reader.getValueClass());
           
          @@ -574,8 +578,14 @@
           
               while (reader.next(key, value))               // copy all entries
                 writer.append(key, value);
          +    LOG.info("READER past append -- check netstat before close");
          +    Thread.sleep(60*1000);
           
               writer.close();
          +
          +    reader.close();
          +    LOG.info("READER closed -- waiting around 60seconds so can see netstat");
          +    Thread.sleep(60*1000);
             }
           
           }
          

          It adds a pause after opening a small map file in dfs and after running through append – just before close – as well as after close, just so you can do a netsat -tn to see the CLOSE_WAITs showing up.

          Build and then do:

          % netstat -tn|grep CLOSE_WAIT|wc
          

          Remember the count.

          Then do the following where the file is any smallish MapFile up in hdfs:

          ./bin/hadoop org.apache.hadoop.io.MapFile /hbase123/hregion_-70236052/info/mapfiles/3026597445031962455 /x
          

          Put it in background and do netstat every time it logs.

          On open, you'll see two new CLOSE_WAITs, one for the data file and the other for the index. They stick around through the append. They go away on close of the map file.

          Show
          stack added a comment - - edited On how to reproduce, first apply this to MapFile: Index: src/java/org/apache/hadoop/io/MapFile.java =================================================================== --- src/java/org/apache/hadoop/io/MapFile.java (revision 603416) +++ src/java/org/apache/hadoop/io/MapFile.java (working copy) @@ -563,8 +563,12 @@ String out = args[1]; Configuration conf = new Configuration(); - FileSystem fs = FileSystem.getLocal(conf); + // FileSystem fs = FileSystem.getLocal(conf); + FileSystem fs = FileSystem.get(conf); MapFile.Reader reader = new MapFile.Reader(fs, in, conf); + LOG.info( "READER opened -- go do a netstat -tn" ); + Thread .sleep(60*1000); + LOG.info( "READER moving on to invoke next" ); MapFile.Writer writer = new MapFile.Writer(conf, fs, out, reader.getKeyClass(), reader.getValueClass()); @@ -574,8 +578,14 @@ while (reader.next(key, value)) // copy all entries writer.append(key, value); + LOG.info( "READER past append -- check netstat before close" ); + Thread .sleep(60*1000); writer.close(); + + reader.close(); + LOG.info( "READER closed -- waiting around 60seconds so can see netstat" ); + Thread .sleep(60*1000); } } It adds a pause after opening a small map file in dfs and after running through append – just before close – as well as after close, just so you can do a netsat -tn to see the CLOSE_WAITs showing up. Build and then do: % netstat -tn|grep CLOSE_WAIT|wc Remember the count. Then do the following where the file is any smallish MapFile up in hdfs: ./bin/hadoop org.apache.hadoop.io.MapFile /hbase123/hregion_-70236052/info/mapfiles/3026597445031962455 /x Put it in background and do netstat every time it logs. On open, you'll see two new CLOSE_WAITs, one for the data file and the other for the index. They stick around through the append. They go away on close of the map file.
          Hide
          Raghu Angadi added a comment -

          > They go away on close of the map file.

          This seems alright. I thought you meant you had connections in CLOSE_WAIT even after closing the file. Could you describe what the bug here is or what you expected to happen.

          Here is what I think happened and it is consistent with both your experiment and my understing of DFS:
          say 'smallish' is 4k.
          MapFile.Reader() opens two DFS files and seek to some posision.
          So DFSClient makes to connections for these files and asks the datanode(s) for data.
          Datanodes flush the data and all of it fits in receiver buffer, even if there is no read from the
          app. Datanode closes the connection resulting in CLOSE_WAIT on the DFSClient side.

          When MapFile reads, I guess it does not actually read past the EOF of either file so DFSClient does
          not actually know that it reached EOF and so does not close the socket yet.

          The socket is closed when the InputStream is closed.

          Show
          Raghu Angadi added a comment - > They go away on close of the map file. This seems alright. I thought you meant you had connections in CLOSE_WAIT even after closing the file. Could you describe what the bug here is or what you expected to happen. Here is what I think happened and it is consistent with both your experiment and my understing of DFS: say 'smallish' is 4k. MapFile.Reader() opens two DFS files and seek to some posision. So DFSClient makes to connections for these files and asks the datanode(s) for data. Datanodes flush the data and all of it fits in receiver buffer, even if there is no read from the app. Datanode closes the connection resulting in CLOSE_WAIT on the DFSClient side. When MapFile reads, I guess it does not actually read past the EOF of either file so DFSClient does not actually know that it reached EOF and so does not close the socket yet. The socket is closed when the InputStream is closed.
          Hide
          stack added a comment -

          I was just looking at it. I agree w/ your description.

          Here's the problem I see in the current behavior. See if you agree with me.

          Studying these CLOSE_WAITs over last few days, the client buffer shows 1 or 0 bytes in the queue. At a minimum, I would expect that when client has read all of a block – netstat shows queues of size 0 – then the client should close its socket and free up datanode-side resources. Queues with 1 byte only in them are suspicious. It seems to be the most common socket stasis (You'll see this state if you run the above patch). If I were to guess, the block math is either wrong or, since the state is common, could the client recognize this condition and pull in the lonely byte (closing socket afterward)?

          These outstanding CLOSE_WAITs are an issue in hbase. Our usage of hdfs is not like MRs where tasks march through MapFiles opening, next'ing and then closing moving on to the next file in the partition. hbase holds open hundreds/thousands of MapFiles concurrently ready to field any random access the wiley client might throw at us. Last night, Paul Saab did a billion plus upload into hbase and at one point on one data node, 300plus sockets were consumed showing the CLOSE_WAIT state. HADOOP-2407 cuts that number in half but it seems reasonable that we can cut the number of consumed resources even further.

          Show
          stack added a comment - I was just looking at it. I agree w/ your description. Here's the problem I see in the current behavior. See if you agree with me. Studying these CLOSE_WAITs over last few days, the client buffer shows 1 or 0 bytes in the queue. At a minimum, I would expect that when client has read all of a block – netstat shows queues of size 0 – then the client should close its socket and free up datanode-side resources. Queues with 1 byte only in them are suspicious. It seems to be the most common socket stasis (You'll see this state if you run the above patch). If I were to guess, the block math is either wrong or, since the state is common, could the client recognize this condition and pull in the lonely byte (closing socket afterward)? These outstanding CLOSE_WAITs are an issue in hbase. Our usage of hdfs is not like MRs where tasks march through MapFiles opening, next'ing and then closing moving on to the next file in the partition. hbase holds open hundreds/thousands of MapFiles concurrently ready to field any random access the wiley client might throw at us. Last night, Paul Saab did a billion plus upload into hbase and at one point on one data node, 300plus sockets were consumed showing the CLOSE_WAIT state. HADOOP-2407 cuts that number in half but it seems reasonable that we can cut the number of consumed resources even further.
          Hide
          Raghu Angadi added a comment -

          >Studying these CLOSE_WAITs over last few days, the client buffer shows 1 or 0 bytes in the queue. At a minimum, I would expect that when client has read all of a block - netstat shows queues of size 0 - then the client should close its socket and free up datanode-side resources.

          This is ok. There are a few extra bytes in the stream at the end that indicate a proper end of stream to the client. A client would read that only when the client tries to read more. There is a BufferedInputStream between the socket and DFSClient to confuse the things more.

          One could argue that DFSClient should try to read a few a bytes more that the user wants to read and close the socket... Thats a different issue. If we want to do this I think it should be done with a non-blocking read since we don't want user to wait more than what is required. Even if we did that I don't think it solves your problem, since this helps only if the random access is accessing at the end of a block. If you are accessing some other place, you would just be holding 64k (or 128k?) of precious kernel memory on both sides, only to throw away in next random access.

          I don't think streaming read is meant for random reads. I don't have much knowledge of access pattern or average size of reads in Hbase, but did you look into using pread?

          > These outstanding CLOSE_WAITs are an issue in hbase.

          How about idle connections? Is that ok?

          Show
          Raghu Angadi added a comment - >Studying these CLOSE_WAITs over last few days, the client buffer shows 1 or 0 bytes in the queue. At a minimum, I would expect that when client has read all of a block - netstat shows queues of size 0 - then the client should close its socket and free up datanode-side resources. This is ok. There are a few extra bytes in the stream at the end that indicate a proper end of stream to the client. A client would read that only when the client tries to read more. There is a BufferedInputStream between the socket and DFSClient to confuse the things more. One could argue that DFSClient should try to read a few a bytes more that the user wants to read and close the socket... Thats a different issue. If we want to do this I think it should be done with a non-blocking read since we don't want user to wait more than what is required. Even if we did that I don't think it solves your problem, since this helps only if the random access is accessing at the end of a block. If you are accessing some other place, you would just be holding 64k (or 128k?) of precious kernel memory on both sides, only to throw away in next random access. I don't think streaming read is meant for random reads. I don't have much knowledge of access pattern or average size of reads in Hbase, but did you look into using pread? > These outstanding CLOSE_WAITs are an issue in hbase. How about idle connections? Is that ok?
          Hide
          Michael Bieniosek added a comment -

          > How about idle connections? Is that ok?

          Because datanode does not use select for io, each idle connection seems to consume a thread. This gets expensive, not necessarily the connections themselves.

          Show
          Michael Bieniosek added a comment - > How about idle connections? Is that ok? Because datanode does not use select for io, each idle connection seems to consume a thread. This gets expensive, not necessarily the connections themselves.
          Hide
          Raghu Angadi added a comment -

          So an idle connection is more expensive than a connection in CLOSE_WAIT... I agree. Doesn't Hbase on DFS have both issues? Do you think at a given time HBase has a lot more connections in CLOSE_WAIT than idle?

          Show
          Raghu Angadi added a comment - So an idle connection is more expensive than a connection in CLOSE_WAIT... I agree. Doesn't Hbase on DFS have both issues? Do you think at a given time HBase has a lot more connections in CLOSE_WAIT than idle?
          Hide
          Michael Bieniosek added a comment -

          Well, hbase does use select, so it doesn't consume threads to have idle connections.

          Show
          Michael Bieniosek added a comment - Well, hbase does use select, so it doesn't consume threads to have idle connections.
          Hide
          Raghu Angadi added a comment -

          Great. I would have thought tcp connections and their filled buffers would take more real memory than a thread.
          CLOSE_WAIT connection don't consume any resources on Datanode.

          Do we still want to keep this jira open?

          Show
          Raghu Angadi added a comment - Great. I would have thought tcp connections and their filled buffers would take more real memory than a thread. CLOSE_WAIT connection don't consume any resources on Datanode. Do we still want to keep this jira open?
          Hide
          stack added a comment -

          Pardon me Raghu, I don't follow what you mean by pread? I presume you're not referring to the clib call and instead mean some java equiv.

          MapFile as-is works for hbase 'cos index eases random accesses.

          At any given time, CLOSE_WAIT in client are minority; more usual is an idle open connection and yes, hbase has both issues.

          Above I misspoke when I said an uploading last night was occupying 300+ connections all in CLOSE_WAIT. It actually went over 900 open connections in the datanode. I don't know how many were in CLOSE_WAIT (and yes, these CLOSE_WAITs are different from the client-side CLOSE_WAITS I've been going on about above).

          Is there an issue to move the datanode to use select that you know of?

          Lets keep the issue open another while if you don't mind. It would be good to have more confidence that we can account for all open sockets in the datanode before closing. For example: the 900+ open connections in the datanode cited above seems excessive when I try counting how many open MapFiles there should have been present at the time. Also, am having trouble matching blocked reads in client to blocked writes cited in thread dumps attached above.

          Show
          stack added a comment - Pardon me Raghu, I don't follow what you mean by pread? I presume you're not referring to the clib call and instead mean some java equiv. MapFile as-is works for hbase 'cos index eases random accesses. At any given time, CLOSE_WAIT in client are minority; more usual is an idle open connection and yes, hbase has both issues. Above I misspoke when I said an uploading last night was occupying 300+ connections all in CLOSE_WAIT. It actually went over 900 open connections in the datanode. I don't know how many were in CLOSE_WAIT (and yes, these CLOSE_WAITs are different from the client-side CLOSE_WAITS I've been going on about above). Is there an issue to move the datanode to use select that you know of? Lets keep the issue open another while if you don't mind. It would be good to have more confidence that we can account for all open sockets in the datanode before closing. For example: the 900+ open connections in the datanode cited above seems excessive when I try counting how many open MapFiles there should have been present at the time. Also, am having trouble matching blocked reads in client to blocked writes cited in thread dumps attached above.
          Hide
          Raghu Angadi added a comment -

          Sure. Please keep the Jira open if you think there is a bug.

          pread I mentioned is FSInputStrea.read(long, buf, offset, len).

          > Is there an issue to move the datanode to use select that you know of?

          What does that fix? What is the problem it is causing now? It surely won't fix 900 (unaccounted) connections. IMHO reducing threads on Datanode just fixes just one of the symptoms and does not address the real issue of using streaming api to do sparse random reads.

          Show
          Raghu Angadi added a comment - Sure. Please keep the Jira open if you think there is a bug. pread I mentioned is FSInputStrea.read(long, buf, offset, len). > Is there an issue to move the datanode to use select that you know of? What does that fix? What is the problem it is causing now? It surely won't fix 900 (unaccounted) connections. IMHO reducing threads on Datanode just fixes just one of the symptoms and does not address the real issue of using streaming api to do sparse random reads.
          Hide
          Raghu Angadi added a comment -

          >>Is there an issue to move the datanode to use select that you know of?
          >What does that fix?

          I hope my comment not misunderstood. Of course using as few threads as required is better than using more. We should probably file a different jira if you want the feature any way. I am not sure if that is a fix for this jira since we don't know what the bug is yet. Also it needs async disk i/o along with async network i/o.

          Show
          Raghu Angadi added a comment - >>Is there an issue to move the datanode to use select that you know of? >What does that fix? I hope my comment not misunderstood. Of course using as few threads as required is better than using more. We should probably file a different jira if you want the feature any way. I am not sure if that is a fix for this jira since we don't know what the bug is yet. Also it needs async disk i/o along with async network i/o.
          Hide
          Doug Cutting added a comment -

          > pread I mentioned is FSInputStream.read(long, buf, offset, len).

          Michael is calling MapFile#get(). That uses a buffered stream, since calls might sometimes be random, and they might sometimes be sequential, when enumerating a range of values.

          It would certainly be nice if we could support keeping lots of HDFS files open without paying a large penalty on either the client or on the datanode. Perhaps idle connections to datanodes should time out. I think the existing client code would re-establish connections if the datanode were simply to close idle connections after a time. Could that work?

          Show
          Doug Cutting added a comment - > pread I mentioned is FSInputStream.read(long, buf, offset, len). Michael is calling MapFile#get(). That uses a buffered stream, since calls might sometimes be random, and they might sometimes be sequential, when enumerating a range of values. It would certainly be nice if we could support keeping lots of HDFS files open without paying a large penalty on either the client or on the datanode. Perhaps idle connections to datanodes should time out. I think the existing client code would re-establish connections if the datanode were simply to close idle connections after a time. Could that work?
          Hide
          Raghu Angadi added a comment -

          > Perhaps idle connections to datanodes should time out.
          I already filed HADOOP-2346.

          Show
          Raghu Angadi added a comment - > Perhaps idle connections to datanodes should time out. I already filed HADOOP-2346 .
          Hide
          stack added a comment -

          Resolving as incomplete.

          Last few uploads and internal use here have less open MapFiles because of changes made in hbase prompted by investigations around this issue.

          Regards datanode using select instead of a socket per, seems like folks are more leaning toward timing out idle connections as means of resource conservation. That'd work for hbase too.

          Regards the 900+ open mapfiles, let me find the illegitimate if any, and will open new issue if I find any.

          Show
          stack added a comment - Resolving as incomplete. Last few uploads and internal use here have less open MapFiles because of changes made in hbase prompted by investigations around this issue. Regards datanode using select instead of a socket per, seems like folks are more leaning toward timing out idle connections as means of resource conservation. That'd work for hbase too. Regards the 900+ open mapfiles, let me find the illegitimate if any, and will open new issue if I find any.
          stack made changes -
          Status Open [ 1 ] Resolved [ 5 ]
          Resolution Incomplete [ 4 ]
          Doug Cutting made changes -
          Status Resolved [ 5 ] Closed [ 6 ]
          Raghu Angadi made changes -
          Link This issue is related to HADOOP-2346 [ HADOOP-2346 ]
          Owen O'Malley made changes -
          Component/s dfs [ 12310710 ]

            People

            • Assignee:
              Unassigned
              Reporter:
              Paul Saab
            • Votes:
              0 Vote for this issue
              Watchers:
              2 Start watching this issue

              Dates

              • Created:
                Updated:
                Resolved:

                Development