Details

    • Type: Bug Bug
    • Status: Closed
    • Priority: Minor Minor
    • Resolution: Fixed
    • Affects Version/s: 0.16.0
    • Fix Version/s: 0.16.1
    • Component/s: None
    • Labels:
      None

      Description

      Saw this in logs:

      2008-01-31 18:55:02,128 ERROR org.apache.hadoop.hbase.HRegionServer: Compaction failed for region TestTable,0009438931,1201805282651
      java.lang.NullPointerException
              at org.apache.hadoop.dfs.DFSClient$DFSOutputStream.close(DFSClient.java:2262)
              at org.apache.hadoop.fs.FSDataOutputStream$PositionCache.close(FSDataOutputStream.java:51)
              at org.apache.hadoop.fs.FSDataOutputStream.close(FSDataOutputStream.java:67)
              at org.apache.hadoop.hbase.HStoreFile.writeInfo(HStoreFile.java:365)
              at org.apache.hadoop.hbase.HStore.compact(HStore.java:1236) 
              at org.apache.hadoop.hbase.HRegion.compactStores(HRegion.java:775)
              at org.apache.hadoop.hbase.HRegion.compactIfNeeded(HRegion.java:707)
              at org.apache.hadoop.hbase.HRegionServer$CompactSplitThread.run(HRegionServer.java:253)
      

      Look to see if the response data method needs to be made volatile (There's a test for null just before we use it on line #2262).

      1. HADOOP-2756.patch
        0.7 kB
        Raghu Angadi
      2. HADOOP-2756.patch
        1 kB
        Raghu Angadi

        Issue Links

          Activity

          Hide
          Hudson added a comment -
          Show
          Hudson added a comment - Integrated in Hadoop-trunk #421 (See http://hudson.zones.apache.org/hudson/job/Hadoop-trunk/421/ )
          Hide
          Raghu Angadi added a comment -

          I just committed this.

          Show
          Raghu Angadi added a comment - I just committed this.
          Hide
          Hadoop QA added a comment -

          -1 overall. Here are the results of testing the latest attachment
          http://issues.apache.org/jira/secure/attachment/12377133/HADOOP-2756.patch
          against trunk revision 619744.

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

          tests included -1. The patch doesn't appear to include any new or modified tests.
          Please justify why no tests are needed for this patch.

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

          javac +1. The applied patch does not generate any new javac compiler warnings.

          release audit +1. The applied patch does not generate any new release audit warnings.

          findbugs +1. The patch does not introduce any new Findbugs warnings.

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

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

          Test results: http://hudson.zones.apache.org/hudson/job/Hadoop-Patch/1896/testReport/
          Findbugs warnings: http://hudson.zones.apache.org/hudson/job/Hadoop-Patch/1896/artifact/trunk/build/test/findbugs/newPatchFindbugsWarnings.html
          Checkstyle results: http://hudson.zones.apache.org/hudson/job/Hadoop-Patch/1896/artifact/trunk/build/test/checkstyle-errors.html
          Console output: http://hudson.zones.apache.org/hudson/job/Hadoop-Patch/1896/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/12377133/HADOOP-2756.patch against trunk revision 619744. @author +1. The patch does not contain any @author tags. tests included -1. The patch doesn't appear to include any new or modified tests. Please justify why no tests are needed for this patch. javadoc +1. The javadoc tool did not generate any warning messages. javac +1. The applied patch does not generate any new javac compiler warnings. release audit +1. The applied patch does not generate any new release audit warnings. findbugs +1. The patch does not introduce any new Findbugs warnings. core tests +1. The patch passed core unit tests. contrib tests +1. The patch passed contrib unit tests. Test results: http://hudson.zones.apache.org/hudson/job/Hadoop-Patch/1896/testReport/ Findbugs warnings: http://hudson.zones.apache.org/hudson/job/Hadoop-Patch/1896/artifact/trunk/build/test/findbugs/newPatchFindbugsWarnings.html Checkstyle results: http://hudson.zones.apache.org/hudson/job/Hadoop-Patch/1896/artifact/trunk/build/test/checkstyle-errors.html Console output: http://hudson.zones.apache.org/hudson/job/Hadoop-Patch/1896/console This message is automatically generated.
          Hide
          Raghu Angadi added a comment -

          Thanks Dhruba. This patch applies to both 0.16 and trunk.

          Show
          Raghu Angadi added a comment - Thanks Dhruba. This patch applies to both 0.16 and trunk.
          Hide
          dhruba borthakur added a comment -

          +1.

          Show
          dhruba borthakur added a comment - +1.
          Hide
          Raghu Angadi added a comment -

          If we prefer minimalist patch, then the first patch is fine too.

          Show
          Raghu Angadi added a comment - If we prefer minimalist patch, then the first patch is fine too.
          Hide
          Raghu Angadi added a comment -

          Dhruba,

          Could you review the updated patch?

          This does not include a unit test since it is correlated more with sun spots .

          Show
          Raghu Angadi added a comment - Dhruba, Could you review the updated patch? This does not include a unit test since it is correlated more with sun spots .
          Hide
          dhruba borthakur added a comment -

          I think we should go with this minimal patch for 0.16. +1.

          Show
          dhruba borthakur added a comment - I think we should go with this minimal patch for 0.16. +1.
          Hide
          Raghu Angadi added a comment -

          > The streamer thread might not exit (hence the streamer.join() might block indefinitely) if the reponseder thread is not closed (via response.close()).

          I don't see dataStreamer waiting for repsonse to close(). If it is not closed, it closes it.

          I think attached patch is fine. I checked the peripheral code again and I don't see it introducing any new problem and fixes this bug. Does anyone see why it is wrong? If this fix is ok, I will submit a slightly reformatted patch (i.e. bring all the references to 'response' inside closeInternal() to one place).

          Show
          Raghu Angadi added a comment - > The streamer thread might not exit (hence the streamer.join() might block indefinitely) if the reponseder thread is not closed (via response.close()). I don't see dataStreamer waiting for repsonse to close(). If it is not closed, it closes it. I think attached patch is fine. I checked the peripheral code again and I don't see it introducing any new problem and fixes this bug. Does anyone see why it is wrong? If this fix is ok, I will submit a slightly reformatted patch (i.e. bring all the references to 'response' inside closeInternal() to one place).
          Hide
          Jim Kellerman added a comment -

          +1 on this being a blocker for 0.16.1

          We have seen this fairly often, but it is not reproducable. Seems to be related to sunspots or something.

          Show
          Jim Kellerman added a comment - +1 on this being a blocker for 0.16.1 We have seen this fairly often, but it is not reproducable. Seems to be related to sunspots or something.
          Hide
          Christian Kunz added a comment -

          +1 blocker for 0.16.

          Some of our tasks fail with this exception, not necessarily under load. And when trying to run distcp on the target cluster with about 20,000 mappers there were more than 400 such failures.

          Show
          Christian Kunz added a comment - +1 blocker for 0.16. Some of our tasks fail with this exception, not necessarily under load. And when trying to run distcp on the target cluster with about 20,000 mappers there were more than 400 such failures.
          Hide
          Raghu Angadi added a comment - - edited

          we should probably be able to know that looking at the code. Any way, please feel free to change the 'fix version'. Personally I don't think we should have NPEs.

          Show
          Raghu Angadi added a comment - - edited we should probably be able to know that looking at the code. Any way, please feel free to change the 'fix version'. Personally I don't think we should have NPEs.
          Hide
          dhruba borthakur added a comment -

          The code looks correct but there could be an issue that needs more testing. The streamer thread might not exit (hence the streamer.join() might block indefinitely) if the reponseder thread is not closed (via response.close()).

          Show
          dhruba borthakur added a comment - The code looks correct but there could be an issue that needs more testing. The streamer thread might not exit (hence the streamer.join() might block indefinitely) if the reponseder thread is not closed (via response.close()).
          Hide
          Raghu Angadi added a comment -

          If streamer can be null when we go join at line #2276 (before this patch is applied), why can't it be null just after the call to streamer.close?

          NPE is with "response" object. I think streamer is the one that sets it to null asynchronously. I am just extending the fix in HADOOP-2717. This is as correct as the patch there.

          Show
          Raghu Angadi added a comment - If streamer can be null when we go join at line #2276 (before this patch is applied), why can't it be null just after the call to streamer.close? NPE is with "response" object. I think streamer is the one that sets it to null asynchronously. I am just extending the fix in HADOOP-2717 . This is as correct as the patch there.
          Hide
          stack added a comment -

          Or rather, why can't it be null after the test for null?

          Show
          stack added a comment - Or rather, why can't it be null after the test for null?
          Hide
          stack added a comment -

          Will this fix the issue? It just moves the join closer to the streamer.close. If streamer can be null when we go join at line #2276 (before this patch is applied), why can't it be null just after the call to streamer.close?

          Show
          stack added a comment - Will this fix the issue? It just moves the join closer to the streamer.close. If streamer can be null when we go join at line #2276 (before this patch is applied), why can't it be null just after the call to streamer.close?
          Hide
          Raghu Angadi added a comment -

          Suggested patch.

          Show
          Raghu Angadi added a comment - Suggested patch.
          Hide
          Raghu Angadi added a comment -

          This is probably a blocker for 0.16. Changed component to dfs. The fix might be similar to the fix in HADOOP-2717.

          Show
          Raghu Angadi added a comment - This is probably a blocker for 0.16. Changed component to dfs. The fix might be similar to the fix in HADOOP-2717 .

            People

            • Assignee:
              Raghu Angadi
              Reporter:
              stack
            • Votes:
              0 Vote for this issue
              Watchers:
              0 Start watching this issue

              Dates

              • Created:
                Updated:
                Resolved:

                Development