Hadoop Map/Reduce
  1. Hadoop Map/Reduce
  2. MAPREDUCE-5154

staging directory deletion fails because delegation tokens have been cancelled

    Details

    • Type: Bug Bug
    • Status: Closed
    • Priority: Major Major
    • Resolution: Fixed
    • Affects Version/s: 1.2.0
    • Fix Version/s: 1.2.0
    • Component/s: jobtracker
    • Labels:
      None

      Description

      In a secure setup, the jobtracker needs the job's delegation tokens to delete the staging directory. MAPREDUCE-4850 made it so that job cleanup staging directory deletion occurs asynchronously, so that it could order it with system directory deletion. This introduced the issue that a job's delegation tokens could be cancelled before the cleanup thread got around to deleting it, causing the deletion to fail.

      1. MAPREDUCE-5154.patch
        8 kB
        Sandy Ryza
      2. MAPREDUCE-5154.patch
        3 kB
        Arun C Murthy
      3. MAPREDUCE-5154.patch
        4 kB
        Arun C Murthy

        Activity

        Hide
        Matt Foley added a comment -

        Closed upon release of Hadoop 1.2.0.

        Show
        Matt Foley added a comment - Closed upon release of Hadoop 1.2.0.
        Hide
        Sandy Ryza added a comment -

        Thanks Arun!

        Show
        Sandy Ryza added a comment - Thanks Arun!
        Hide
        Arun C Murthy added a comment -

        I just committed this. Thanks Sandy!

        Show
        Arun C Murthy added a comment - I just committed this. Thanks Sandy!
        Hide
        Arun C Murthy added a comment -

        Thanks for the quick check Sandy Ryza. Here is the update with better docs.

        Show
        Arun C Murthy added a comment - Thanks for the quick check Sandy Ryza . Here is the update with better docs.
        Hide
        Hadoop QA added a comment -

        -1 overall. Here are the results of testing the latest attachment
        http://issues.apache.org/jira/secure/attachment/12581283/MAPREDUCE-5154.patch
        against trunk revision .

        -1 patch. The patch command could not apply the patch.

        Console output: https://builds.apache.org/job/PreCommit-MAPREDUCE-Build/3566//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/12581283/MAPREDUCE-5154.patch against trunk revision . -1 patch . The patch command could not apply the patch. Console output: https://builds.apache.org/job/PreCommit-MAPREDUCE-Build/3566//console This message is automatically generated.
        Hide
        Sandy Ryza added a comment -

        I considered that, and my thought for the separate async call was that I didn't see the token cancellation as related to deletions in a general sense. But your approach seems reasonable to me too, and has the advantage of stirring up less code, so +1.

        Nit: it might be good to add a comment to the new PathDeletionContext that explains what it's used for and makes it clear that the token cancellation comes after the deletion.

        Show
        Sandy Ryza added a comment - I considered that, and my thought for the separate async call was that I didn't see the token cancellation as related to deletions in a general sense. But your approach seems reasonable to me too, and has the advantage of stirring up less code, so +1. Nit: it might be good to add a comment to the new PathDeletionContext that explains what it's used for and makes it clear that the token cancellation comes after the deletion.
        Hide
        Arun C Murthy added a comment -

        Sandy Ryza This is the fix I had in mind. LMK what you think. Thanks.

        Show
        Arun C Murthy added a comment - Sandy Ryza This is the fix I had in mind. LMK what you think. Thanks.
        Hide
        Arun C Murthy added a comment -

        Sandy Ryza Wouldn't it be simpler to just pass in the delegation token to the PathCleanupContext and get it to (optionally) cancel the token inline i.e. after the delete?

        Show
        Arun C Murthy added a comment - Sandy Ryza Wouldn't it be simpler to just pass in the delegation token to the PathCleanupContext and get it to (optionally) cancel the token inline i.e. after the delete?
        Hide
        Sandy Ryza added a comment -

        Verified it on a secure cluster

        Show
        Sandy Ryza added a comment - Verified it on a secure cluster
        Hide
        Alejandro Abdelnur added a comment -

        LGTM, any chance to have a testcase? Else ensure it works in a secure setup as expected?

        Show
        Alejandro Abdelnur added a comment - LGTM, any chance to have a testcase? Else ensure it works in a secure setup as expected?
        Hide
        Hadoop QA added a comment -

        -1 overall. Here are the results of testing the latest attachment
        http://issues.apache.org/jira/secure/attachment/12579073/MAPREDUCE-5154.patch
        against trunk revision .

        -1 patch. The patch command could not apply the patch.

        Console output: https://builds.apache.org/job/PreCommit-MAPREDUCE-Build/3530//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/12579073/MAPREDUCE-5154.patch against trunk revision . -1 patch . The patch command could not apply the patch. Console output: https://builds.apache.org/job/PreCommit-MAPREDUCE-Build/3530//console This message is automatically generated.
        Hide
        Sandy Ryza added a comment -

        Uploading a patch that cancels the delegation tokens asynchronously as well. This required modifying CleanupQueue to accept delegation tokens to cancel in addition to files to delete.

        Both TestJobRecovery and TestJobCleanup pass.

        Show
        Sandy Ryza added a comment - Uploading a patch that cancels the delegation tokens asynchronously as well. This required modifying CleanupQueue to accept delegation tokens to cancel in addition to files to delete. Both TestJobRecovery and TestJobCleanup pass.

          People

          • Assignee:
            Sandy Ryza
            Reporter:
            Sandy Ryza
          • Votes:
            0 Vote for this issue
            Watchers:
            6 Start watching this issue

            Dates

            • Created:
              Updated:
              Resolved:

              Development