Hadoop HDFS
  1. Hadoop HDFS
  2. HDFS-1605

Convert DFSInputStream synchronized sections to a ReadWrite lock

    Details

    • Type: Improvement Improvement
    • Status: Patch Available
    • Priority: Major Major
    • Resolution: Unresolved
    • Affects Version/s: None
    • Fix Version/s: None
    • Component/s: hdfs-client
    • Labels:
      None

      Description

      Hbase does concurrent preads from multiple threads to different blocks of the same hdfs file. Each of these pread calls invoke DFSInputStream.getFileLength() and DFSInputStream.getBlockAt(). These methods are "synchronized", thus causing all the concurrent threads to serialize. It would help performance to convert this to a Read/Write lock

      1. DFSClientRWlock.1.txt
        11 kB
        dhruba borthakur
      2. DFSClientRWlock.3.txt
        11 kB
        dhruba borthakur
      3. HDFS-1605.txt
        45 kB
        Liang Xie

        Issue Links

          Activity

          dhruba borthakur created issue -
          dhruba borthakur made changes -
          Field Original Value New Value
          Link This issue is related to HDFS-1599 [ HDFS-1599 ]
          Hide
          dhruba borthakur added a comment -

          Patch for hadoop 0.20 branch

          Show
          dhruba borthakur added a comment - Patch for hadoop 0.20 branch
          dhruba borthakur made changes -
          Attachment DFSClientRWlock.1.txt [ 12469867 ]
          Hide
          dhruba borthakur added a comment -

          Merged patch with latest branch.

          Show
          dhruba borthakur added a comment - Merged patch with latest branch.
          dhruba borthakur made changes -
          Attachment DFSClientRWlock.3.txt [ 12469965 ]
          Hide
          Todd Lipcon added a comment -

          Hey Dhruba. I've never seen this lock actually be a problem in practice given current performance issues in DFSInputStream. I understand from Jonathan Gray that you're also testing some other patches to improve DFSInputStream performance. Maybe it would make sense to put those upstream first?

          (or do you have some benchmarks that shows that the rwlock helps things even with an otherwise "stock" DFSInputStream)?

          Show
          Todd Lipcon added a comment - Hey Dhruba. I've never seen this lock actually be a problem in practice given current performance issues in DFSInputStream. I understand from Jonathan Gray that you're also testing some other patches to improve DFSInputStream performance. Maybe it would make sense to put those upstream first? (or do you have some benchmarks that shows that the rwlock helps things even with an otherwise "stock" DFSInputStream)?
          Hide
          Hairong Kuang added a comment -

          +1. The latest patch looks good to me.

          Show
          Hairong Kuang added a comment - +1. The latest patch looks good to me.
          Hide
          dhruba borthakur added a comment -

          Hi todd, it will take me quite a while to get performance numbers for you. But it is fine if you do not want this in the 0.20-append branch, no problems with me.

          Show
          dhruba borthakur added a comment - Hi todd, it will take me quite a while to get performance numbers for you. But it is fine if you do not want this in the 0.20-append branch, no problems with me.
          Hide
          Liang Xie added a comment -

          we observed this hotspot in our production cluster these days. Most of the waiting lock threads jsut like below:

          19205.637:"IPC Server handler 27 on 12600" daemon prio=10 tid=0x00007f82fc1e5750 nid=0x4d9b waiting for monitor entry [0x00007f821fe78000]
          19205.637- java.lang.Thread.State: BLOCKED (on object monitor)
          19205.637- at org.apache.hadoop.hdfs.DFSInputStream.getFileLength(DFSInputStream.java:242)
          19205.637- - waiting to lock <0x000000044e20d238> (a org.apache.hadoop.hdfs.DFSInputStream)
          19205.637- at org.apache.hadoop.hdfs.DFSInputStream.read(DFSInputStream.java:982)
          19205.637- at org.apache.hadoop.fs.FSDataInputStream.read(FSDataInputStream.java:73)
          19205.637- at org.apache.hadoop.hbase.io.hfile.HFileBlock$AbstractFSReader.readAtOffset(HFileBlock.java:1393)

          and the lock holder is doing the right read things that time.

          Show
          Liang Xie added a comment - we observed this hotspot in our production cluster these days. Most of the waiting lock threads jsut like below: 19205.637:"IPC Server handler 27 on 12600" daemon prio=10 tid=0x00007f82fc1e5750 nid=0x4d9b waiting for monitor entry [0x00007f821fe78000] 19205.637- java.lang.Thread.State: BLOCKED (on object monitor) 19205.637- at org.apache.hadoop.hdfs.DFSInputStream.getFileLength(DFSInputStream.java:242) 19205.637- - waiting to lock <0x000000044e20d238> (a org.apache.hadoop.hdfs.DFSInputStream) 19205.637- at org.apache.hadoop.hdfs.DFSInputStream.read(DFSInputStream.java:982) 19205.637- at org.apache.hadoop.fs.FSDataInputStream.read(FSDataInputStream.java:73) 19205.637- at org.apache.hadoop.hbase.io.hfile.HFileBlock$AbstractFSReader.readAtOffset(HFileBlock.java:1393) and the lock holder is doing the right read things that time.
          Hide
          Liang Xie added a comment -

          Attached is a patch against trunk.

          The getFileLength hotspot pattern like below:

          "IPC Server handler 88 on 12600" daemon prio=10 tid=0x00007f82fc241580 nid=0x4ddc waiting for monitor entry [0x00007f821eefb000]
          java.lang.Thread.State: BLOCKED (on object monitor)
          at org.apache.hadoop.hdfs.DFSInputStream.getFileLength(DFSInputStream.java:242)

          • waiting to lock <0x00000004404597a8> (a org.apache.hadoop.hdfs.DFSInputStream)
            at org.apache.hadoop.hdfs.DFSInputStream.read(DFSInputStream.java:982)
            at org.apache.hadoop.fs.FSDataInputStream.read(FSDataInputStream.java:73)
            at org.apache.hadoop.hbase.io.hfile.HFileBlock$AbstractFSReader.readAtOffset(HFileBlock.java:1393)
            at org.apache.hadoop.hbase.io.hfile.HFileBlock$FSReaderV2.readBlockDataInternal(HFileBlock.java:1829)
            at org.apache.hadoop.hbase.io.hfile.HFileBlock$FSReaderV2.readBlockData(HFileBlock.java:1673)
            at org.apache.hadoop.hbase.io.hfile.HFileReaderV2.readBlock(HFileReaderV2.java:341)
            at org.apache.hadoop.hbase.io.hfile.HFileBlockIndex$BlockIndexReader.loadDataBlockWithScanInfo(HFileBlockIndex.java:254)
            at org.apache.hadoop.hbase.io.hfile.HFileReaderV2$AbstractScannerV2.seekTo(HFileReaderV2.java:485)
            at org.apache.hadoop.hbase.io.hfile.HFileReaderV2$AbstractScannerV2.seekTo(HFileReaderV2.java:506)
            at org.apache.hadoop.hbase.regionserver.StoreFileScanner.seekAtOrAfter(StoreFileScanner.java:226)
            at org.apache.hadoop.hbase.regionserver.StoreFileScanner.seek(StoreFileScanner.java:146)
            at org.apache.hadoop.hbase.regionserver.StoreFileScanner.enforceSeek(StoreFileScanner.java:354)
            at org.apache.hadoop.hbase.regionserver.KeyValueHeap.pollRealKV(KeyValueHeap.java:385)
            at org.apache.hadoop.hbase.regionserver.KeyValueHeap.generalizedSeek(KeyValueHeap.java:344)
            at org.apache.hadoop.hbase.regionserver.KeyValueHeap.requestSeek(KeyValueHeap.java:304)
            at org.apache.hadoop.hbase.regionserver.StoreScanner.reseek(StoreScanner.java:584)
          • locked <0x00000004285cb478> (a org.apache.hadoop.hbase.regionserver.StoreScanner)
            at org.apache.hadoop.hbase.regionserver.StoreScanner.next(StoreScanner.java:446)
          • locked <0x00000004285cb478> (a org.apache.hadoop.hbase.regionserver.StoreScanner)
            at org.apache.hadoop.hbase.regionserver.KeyValueHeap.next(KeyValueHeap.java:164)
            at org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.nextInternal(HRegion.java:3658)
            at org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.next(HRegion.java:3590)
          • locked <0x00000004285cb310> (a org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl)
            at org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.next(HRegion.java:3615)
          • locked <0x00000004285cb310> (a org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl)
            at org.apache.hadoop.hbase.regionserver.HRegionServer.next(HRegionServer.java:2513)
            at sun.reflect.GeneratedMethodAccessor28.invoke(Unknown Source)
            at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
            at java.lang.reflect.Method.invoke(Method.java:597)
            at org.apache.hadoop.hbase.ipc.SecureRpcEngine$Server.call(SecureRpcEngine.java:460)
            at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:1457)

          "IPC Server handler 192 on 12600" daemon prio=10 tid=0x00007f82fc2dd910 nid=0x4e44 runnable [0x00007f821d493000]
          java.lang.Thread.State: RUNNABLE
          at java.io.FileInputStream.readBytes(Native Method)
          at java.io.FileInputStream.read(FileInputStream.java:220)
          at org.apache.hadoop.hdfs.BlockReaderLocal.read(BlockReaderLocal.java:568)

          • locked <0x00000004287fb350> (a org.apache.hadoop.hdfs.BlockReaderLocal)
            at org.apache.hadoop.hdfs.DFSInputStream$ByteArrayStrategy.doRead(DFSInputStream.java:542)
            at org.apache.hadoop.hdfs.DFSInputStream.readBuffer(DFSInputStream.java:594)
          • locked <0x00000004404597a8> (a org.apache.hadoop.hdfs.DFSInputStream)
            at org.apache.hadoop.hdfs.DFSInputStream.readWithStrategy(DFSInputStream.java:648)
            at org.apache.hadoop.hdfs.DFSInputStream.read(DFSInputStream.java:689)
          • locked <0x00000004404597a8> (a org.apache.hadoop.hdfs.DFSInputStream)
            at java.io.DataInputStream.read(DataInputStream.java:132)
            at org.apache.hadoop.hbase.io.hfile.HFileBlock.readWithExtra(HFileBlock.java:614)
            at org.apache.hadoop.hbase.io.hfile.HFileBlock$AbstractFSReader.readAtOffset(HFileBlock.java:1384)
            at org.apache.hadoop.hbase.io.hfile.HFileBlock$FSReaderV2.readBlockDataInternal(HFileBlock.java:1829)
            at org.apache.hadoop.hbase.io.hfile.HFileBlock$FSReaderV2.readBlockData(HFileBlock.java:1673)
            at org.apache.hadoop.hbase.io.hfile.HFileReaderV2.readBlock(HFileReaderV2.java:341)
            at org.apache.hadoop.hbase.io.hfile.HFileBlockIndex$BlockIndexReader.loadDataBlockWithScanInfo(HFileBlockIndex.java:254)
            at org.apache.hadoop.hbase.io.hfile.HFileReaderV2$AbstractScannerV2.seekTo(HFileReaderV2.java:485)
            at org.apache.hadoop.hbase.io.hfile.HFileReaderV2$AbstractScannerV2.seekTo(HFileReaderV2.java:506)
            at org.apache.hadoop.hbase.regionserver.StoreFileScanner.seekAtOrAfter(StoreFileScanner.java:226)
            at org.apache.hadoop.hbase.regionserver.StoreFileScanner.seek(StoreFileScanner.java:146)
            at org.apache.hadoop.hbase.regionserver.StoreFileScanner.enforceSeek(StoreFileScanner.java:354)
            at org.apache.hadoop.hbase.regionserver.KeyValueHeap.pollRealKV(KeyValueHeap.java:385)
            at org.apache.hadoop.hbase.regionserver.KeyValueHeap.generalizedSeek(KeyValueHeap.java:344)
            at org.apache.hadoop.hbase.regionserver.KeyValueHeap.requestSeek(KeyValueHeap.java:304)
            at org.apache.hadoop.hbase.regionserver.StoreScanner.reseek(StoreScanner.java:584)
          • locked <0x0000000428374d70> (a org.apache.hadoop.hbase.regionserver.StoreScanner)
            at org.apache.hadoop.hbase.regionserver.StoreScanner.next(StoreScanner.java:446)
          • locked <0x0000000428374d70> (a org.apache.hadoop.hbase.regionserver.StoreScanner)
            at org.apache.hadoop.hbase.regionserver.KeyValueHeap.next(KeyValueHeap.java:164)
            at org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.nextInternal(HRegion.java:3658)
            at org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.next(HRegion.java:3590)
          • locked <0x0000000428374c08> (a org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl)
            at org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.next(HRegion.java:3615)
          • locked <0x0000000428374c08> (a org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl)
            at org.apache.hadoop.hbase.regionserver.HRegionServer.next(HRegionServer.java:2513)
            at sun.reflect.GeneratedMethodAccessor28.invoke(Unknown Source)
            at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
            at java.lang.reflect.Method.invoke(Method.java:597)
            at org.apache.hadoop.hbase.ipc.SecureRpcEngine$Server.call(SecureRpcEngine.java:460)
            at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:1457)
          Show
          Liang Xie added a comment - Attached is a patch against trunk. The getFileLength hotspot pattern like below: "IPC Server handler 88 on 12600" daemon prio=10 tid=0x00007f82fc241580 nid=0x4ddc waiting for monitor entry [0x00007f821eefb000] java.lang.Thread.State: BLOCKED (on object monitor) at org.apache.hadoop.hdfs.DFSInputStream.getFileLength(DFSInputStream.java:242) waiting to lock <0x00000004404597a8> (a org.apache.hadoop.hdfs.DFSInputStream) at org.apache.hadoop.hdfs.DFSInputStream.read(DFSInputStream.java:982) at org.apache.hadoop.fs.FSDataInputStream.read(FSDataInputStream.java:73) at org.apache.hadoop.hbase.io.hfile.HFileBlock$AbstractFSReader.readAtOffset(HFileBlock.java:1393) at org.apache.hadoop.hbase.io.hfile.HFileBlock$FSReaderV2.readBlockDataInternal(HFileBlock.java:1829) at org.apache.hadoop.hbase.io.hfile.HFileBlock$FSReaderV2.readBlockData(HFileBlock.java:1673) at org.apache.hadoop.hbase.io.hfile.HFileReaderV2.readBlock(HFileReaderV2.java:341) at org.apache.hadoop.hbase.io.hfile.HFileBlockIndex$BlockIndexReader.loadDataBlockWithScanInfo(HFileBlockIndex.java:254) at org.apache.hadoop.hbase.io.hfile.HFileReaderV2$AbstractScannerV2.seekTo(HFileReaderV2.java:485) at org.apache.hadoop.hbase.io.hfile.HFileReaderV2$AbstractScannerV2.seekTo(HFileReaderV2.java:506) at org.apache.hadoop.hbase.regionserver.StoreFileScanner.seekAtOrAfter(StoreFileScanner.java:226) at org.apache.hadoop.hbase.regionserver.StoreFileScanner.seek(StoreFileScanner.java:146) at org.apache.hadoop.hbase.regionserver.StoreFileScanner.enforceSeek(StoreFileScanner.java:354) at org.apache.hadoop.hbase.regionserver.KeyValueHeap.pollRealKV(KeyValueHeap.java:385) at org.apache.hadoop.hbase.regionserver.KeyValueHeap.generalizedSeek(KeyValueHeap.java:344) at org.apache.hadoop.hbase.regionserver.KeyValueHeap.requestSeek(KeyValueHeap.java:304) at org.apache.hadoop.hbase.regionserver.StoreScanner.reseek(StoreScanner.java:584) locked <0x00000004285cb478> (a org.apache.hadoop.hbase.regionserver.StoreScanner) at org.apache.hadoop.hbase.regionserver.StoreScanner.next(StoreScanner.java:446) locked <0x00000004285cb478> (a org.apache.hadoop.hbase.regionserver.StoreScanner) at org.apache.hadoop.hbase.regionserver.KeyValueHeap.next(KeyValueHeap.java:164) at org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.nextInternal(HRegion.java:3658) at org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.next(HRegion.java:3590) locked <0x00000004285cb310> (a org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl) at org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.next(HRegion.java:3615) locked <0x00000004285cb310> (a org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl) at org.apache.hadoop.hbase.regionserver.HRegionServer.next(HRegionServer.java:2513) at sun.reflect.GeneratedMethodAccessor28.invoke(Unknown Source) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25) at java.lang.reflect.Method.invoke(Method.java:597) at org.apache.hadoop.hbase.ipc.SecureRpcEngine$Server.call(SecureRpcEngine.java:460) at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:1457) "IPC Server handler 192 on 12600" daemon prio=10 tid=0x00007f82fc2dd910 nid=0x4e44 runnable [0x00007f821d493000] java.lang.Thread.State: RUNNABLE at java.io.FileInputStream.readBytes(Native Method) at java.io.FileInputStream.read(FileInputStream.java:220) at org.apache.hadoop.hdfs.BlockReaderLocal.read(BlockReaderLocal.java:568) locked <0x00000004287fb350> (a org.apache.hadoop.hdfs.BlockReaderLocal) at org.apache.hadoop.hdfs.DFSInputStream$ByteArrayStrategy.doRead(DFSInputStream.java:542) at org.apache.hadoop.hdfs.DFSInputStream.readBuffer(DFSInputStream.java:594) locked <0x00000004404597a8> (a org.apache.hadoop.hdfs.DFSInputStream) at org.apache.hadoop.hdfs.DFSInputStream.readWithStrategy(DFSInputStream.java:648) at org.apache.hadoop.hdfs.DFSInputStream.read(DFSInputStream.java:689) locked <0x00000004404597a8> (a org.apache.hadoop.hdfs.DFSInputStream) at java.io.DataInputStream.read(DataInputStream.java:132) at org.apache.hadoop.hbase.io.hfile.HFileBlock.readWithExtra(HFileBlock.java:614) at org.apache.hadoop.hbase.io.hfile.HFileBlock$AbstractFSReader.readAtOffset(HFileBlock.java:1384) at org.apache.hadoop.hbase.io.hfile.HFileBlock$FSReaderV2.readBlockDataInternal(HFileBlock.java:1829) at org.apache.hadoop.hbase.io.hfile.HFileBlock$FSReaderV2.readBlockData(HFileBlock.java:1673) at org.apache.hadoop.hbase.io.hfile.HFileReaderV2.readBlock(HFileReaderV2.java:341) at org.apache.hadoop.hbase.io.hfile.HFileBlockIndex$BlockIndexReader.loadDataBlockWithScanInfo(HFileBlockIndex.java:254) at org.apache.hadoop.hbase.io.hfile.HFileReaderV2$AbstractScannerV2.seekTo(HFileReaderV2.java:485) at org.apache.hadoop.hbase.io.hfile.HFileReaderV2$AbstractScannerV2.seekTo(HFileReaderV2.java:506) at org.apache.hadoop.hbase.regionserver.StoreFileScanner.seekAtOrAfter(StoreFileScanner.java:226) at org.apache.hadoop.hbase.regionserver.StoreFileScanner.seek(StoreFileScanner.java:146) at org.apache.hadoop.hbase.regionserver.StoreFileScanner.enforceSeek(StoreFileScanner.java:354) at org.apache.hadoop.hbase.regionserver.KeyValueHeap.pollRealKV(KeyValueHeap.java:385) at org.apache.hadoop.hbase.regionserver.KeyValueHeap.generalizedSeek(KeyValueHeap.java:344) at org.apache.hadoop.hbase.regionserver.KeyValueHeap.requestSeek(KeyValueHeap.java:304) at org.apache.hadoop.hbase.regionserver.StoreScanner.reseek(StoreScanner.java:584) locked <0x0000000428374d70> (a org.apache.hadoop.hbase.regionserver.StoreScanner) at org.apache.hadoop.hbase.regionserver.StoreScanner.next(StoreScanner.java:446) locked <0x0000000428374d70> (a org.apache.hadoop.hbase.regionserver.StoreScanner) at org.apache.hadoop.hbase.regionserver.KeyValueHeap.next(KeyValueHeap.java:164) at org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.nextInternal(HRegion.java:3658) at org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.next(HRegion.java:3590) locked <0x0000000428374c08> (a org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl) at org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl.next(HRegion.java:3615) locked <0x0000000428374c08> (a org.apache.hadoop.hbase.regionserver.HRegion$RegionScannerImpl) at org.apache.hadoop.hbase.regionserver.HRegionServer.next(HRegionServer.java:2513) at sun.reflect.GeneratedMethodAccessor28.invoke(Unknown Source) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25) at java.lang.reflect.Method.invoke(Method.java:597) at org.apache.hadoop.hbase.ipc.SecureRpcEngine$Server.call(SecureRpcEngine.java:460) at org.apache.hadoop.hbase.ipc.HBaseServer$Handler.run(HBaseServer.java:1457)
          Liang Xie made changes -
          Attachment HDFS-1605.txt [ 12617846 ]
          Hide
          Liang Xie added a comment -

          Considering risk, i changes most of synchronized section with writeLock. only few places(like getFileLengh, some simple getter methods) use readLock in current impl, i think it's ok since both i and dhruba borthakur observed the very same hotspot: getFileLengh, in similar scenario: HBase read.

          Show
          Liang Xie added a comment - Considering risk, i changes most of synchronized section with writeLock. only few places(like getFileLengh, some simple getter methods) use readLock in current impl, i think it's ok since both i and dhruba borthakur observed the very same hotspot: getFileLengh, in similar scenario: HBase read.
          Liang Xie made changes -
          Status Open [ 1 ] Patch Available [ 10002 ]
          Hide
          Hadoop QA added a comment -

          -1 overall. Here are the results of testing the latest attachment
          http://issues.apache.org/jira/secure/attachment/12617846/HDFS-1605.txt
          against trunk revision .

          +1 @author. The patch does not contain any @author tags.

          -1 tests included. The patch doesn't appear to include any new or modified tests.
          Please justify why no new tests are needed for this patch.
          Also please list what manual steps were performed to verify this patch.

          +1 javac. The applied patch does not increase the total number of javac compiler warnings.

          +1 javadoc. The javadoc tool did not generate any warning messages.

          +1 eclipse:eclipse. The patch built with eclipse:eclipse.

          +1 findbugs. The patch does not introduce any new Findbugs (version 1.3.9) warnings.

          +1 release audit. The applied patch does not increase the total number of release audit warnings.

          +1 core tests. The patch passed unit tests in hadoop-hdfs-project/hadoop-hdfs.

          +1 contrib tests. The patch passed contrib unit tests.

          Test results: https://builds.apache.org/job/PreCommit-HDFS-Build/5684//testReport/
          Console output: https://builds.apache.org/job/PreCommit-HDFS-Build/5684//console

          This message is automatically generated.

          Show
          Hadoop QA added a comment - -1 overall . Here are the results of testing the latest attachment http://issues.apache.org/jira/secure/attachment/12617846/HDFS-1605.txt against trunk revision . +1 @author . The patch does not contain any @author tags. -1 tests included . The patch doesn't appear to include any new or modified tests. Please justify why no new tests are needed for this patch. Also please list what manual steps were performed to verify this patch. +1 javac . The applied patch does not increase the total number of javac compiler warnings. +1 javadoc . The javadoc tool did not generate any warning messages. +1 eclipse:eclipse . The patch built with eclipse:eclipse. +1 findbugs . The patch does not introduce any new Findbugs (version 1.3.9) warnings. +1 release audit . The applied patch does not increase the total number of release audit warnings. +1 core tests . The patch passed unit tests in hadoop-hdfs-project/hadoop-hdfs. +1 contrib tests . The patch passed contrib unit tests. Test results: https://builds.apache.org/job/PreCommit-HDFS-Build/5684//testReport/ Console output: https://builds.apache.org/job/PreCommit-HDFS-Build/5684//console This message is automatically generated.
          Hide
          stack added a comment -

          I like this one:

          + waitFor(4000);

          (I know you are only indenting and the above is in the indent and retraining the old code as is when you are changing something else is the way to go but... we should fix the above; could explain a few of those 99th percentiles)

          Patch looks good on a first glance. You are conservative in the changes you make and it seems like the methods are small enough they are easy to cast as methods that require read or write lock. Will check closer soon.

          Have you deployed this on your serving cluster Liang Xie ?

          We should write a test to demonstrate the improvement. I can do that unless you have one around the place?

          Show
          stack added a comment - I like this one: + waitFor(4000); (I know you are only indenting and the above is in the indent and retraining the old code as is when you are changing something else is the way to go but... we should fix the above; could explain a few of those 99th percentiles) Patch looks good on a first glance. You are conservative in the changes you make and it seems like the methods are small enough they are easy to cast as methods that require read or write lock. Will check closer soon. Have you deployed this on your serving cluster Liang Xie ? We should write a test to demonstrate the improvement. I can do that unless you have one around the place?
          Hide
          Liang Xie added a comment -

          waitFor(4000);

          totally agreed with your point, it should have a different configurable value for online app like HBase , or offline app processing app.
          I can file a minor jira for this, thanks stack.

          Show
          Liang Xie added a comment - waitFor(4000); totally agreed with your point, it should have a different configurable value for online app like HBase , or offline app processing app. I can file a minor jira for this, thanks stack .
          Hide
          Liang Xie added a comment -

          FYI, HDFS-5663 will track waitFor(4000) issue.

          Show
          Liang Xie added a comment - FYI, HDFS-5663 will track waitFor(4000) issue.
          Hide
          Liang Xie added a comment -

          Have you deployed this on your serving cluster

          Our hdfs production version upgrade has a long interval/window, this patch has not be run on a production cluster yet.

          Show
          Liang Xie added a comment - Have you deployed this on your serving cluster Our hdfs production version upgrade has a long interval/window, this patch has not be run on a production cluster yet.
          Hide
          Liang Xie added a comment -

          We should write a test to demonstrate the improvement. I can do that unless you have one around the place?

          I think it's still too early to do it. We need to fix another hotspot about BlockReaderLocal as well, just filed HDFS-5664 FYI.

          Show
          Liang Xie added a comment - We should write a test to demonstrate the improvement. I can do that unless you have one around the place? I think it's still too early to do it. We need to fix another hotspot about BlockReaderLocal as well, just filed HDFS-5664 FYI.

            People

            • Assignee:
              dhruba borthakur
              Reporter:
              dhruba borthakur
            • Votes:
              0 Vote for this issue
              Watchers:
              12 Start watching this issue

              Dates

              • Created:
                Updated:

                Development