Details

    • Bug
    • Status: Closed
    • Minor
    • Resolution: Fixed
    • 0.16.0
    • 0.16.1
    • None
    • 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).

      Attachments

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

        Issue Links

          Activity

            rangadi 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.

            rangadi 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 .
            rangadi Raghu Angadi added a comment -

            Suggested patch.

            rangadi Raghu Angadi added a comment - Suggested patch.
            stack Michael 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?

            stack Michael 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?
            stack Michael Stack added a comment -

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

            stack Michael Stack added a comment - Or rather, why can't it be null after the test for null?
            rangadi 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.

            rangadi 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.

            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()).

            dhruba 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()).
            rangadi 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.

            rangadi 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.
            ckunz 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.

            ckunz 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.
            jimk 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.

            jimk 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.
            rangadi 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).

            rangadi 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).

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

            dhruba Dhruba Borthakur added a comment - I think we should go with this minimal patch for 0.16. +1.
            rangadi 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 .

            rangadi 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 .
            rangadi Raghu Angadi added a comment -

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

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

            +1.

            dhruba Dhruba Borthakur added a comment - +1.
            rangadi Raghu Angadi added a comment -

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

            rangadi Raghu Angadi added a comment - Thanks Dhruba. This patch applies to both 0.16 and trunk.
            hadoopqa 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.

            hadoopqa 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.
            rangadi Raghu Angadi added a comment -

            I just committed this.

            rangadi Raghu Angadi added a comment - I just committed this.
            hudson Hudson added a comment -
            hudson Hudson added a comment - Integrated in Hadoop-trunk #421 (See http://hudson.zones.apache.org/hudson/job/Hadoop-trunk/421/ )

            People

              rangadi Raghu Angadi
              stack Michael Stack
              Votes:
              0 Vote for this issue
              Watchers:
              0 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: