Details

    • Type: Sub-task Sub-task
    • Status: Closed
    • Priority: Major Major
    • Resolution: Fixed
    • Affects Version/s: 0.22.0
    • Fix Version/s: 0.21.0
    • Component/s: jobtracker
    • Labels:
      None
    • Release Note:
      mapreduce.job.complete.cancel.delegation.tokens - if false - don't cancel delegation token renewal when the job is complete, because it may be used by some other job.

      Description

      JobTracker should automatically renew delegation tokens for the jobs it is currently running.

      1. 1430-bp20-bugfix.patch
        1 kB
        Devaraj Das
      2. 1430-BP20-Add.patch
        6 kB
        Boris Shkolnik
      3. 1430-dd4-BP20.patch
        31 kB
        Boris Shkolnik
      4. 1430-dd4.patch
        28 kB
        Devaraj Das
      5. 1430-dd3.patch
        29 kB
        Devaraj Das
      6. 1430-dd2.patch
        30 kB
        Devaraj Das
      7. 1430-dd1.patch
        30 kB
        Devaraj Das
      8. MAPREDUCE-1430-14.patch
        29 kB
        Boris Shkolnik
      9. MAPREDUCE-1430-13.patch
        28 kB
        Boris Shkolnik
      10. MAPREDUCE-1430-12.patch
        27 kB
        Boris Shkolnik
      11. MAPREDUCE-1430-8.patch
        31 kB
        Boris Shkolnik
      12. MAPREDUCE-1430-6.patch
        19 kB
        Boris Shkolnik
      13. MAPREDUCE-1430-5.patch
        17 kB
        Boris Shkolnik

        Activity

        Hide
        Boris Shkolnik added a comment -

        We may need to add an expiration date to the o.a.h.security.Token class
        Created HADOOP-6523 for this.

        Show
        Boris Shkolnik added a comment - We may need to add an expiration date to the o.a.h.security.Token class Created HADOOP-6523 for this.
        Hide
        Devaraj Das added a comment -

        Comments:
        1) The class DelegationTokenRenewal could be named better
        2) I am having trouble understanding what you have done in DelegationTokenRenewal to do with resetting of timers. When the JobTracker first renews the token, it can check when the token is supposed to be renewed again, and create a corresponding TimerTask. Then cancel the TimerTask when the job is done. The Timer object needs to be created only once, right?
        3) You should have the cancellation of tokens based on a boolean configuration like "mapreduce.job.cancel.delegation.tokens".
        4) The KIND defined in DelegationTokenRenewal should come from the DelegationTokenIdentifier in HDFS
        5) The place where the delegation tokens are registered for renewal is not correct. Note that the JobInProgress constructor uses a token to copy the jobconf file from the hdfs and by then the token for accessing the jobconf should have got renewed. The renewal should happen before the JobInProgress is constructed in submitJob.
        6) Also, now that we have MapReduce delegation tokens, it will be nice to have the class DelegationTokenRenewal not assume only hdfs delegation tokens.
        7) The Timer should be cancelled when the JobTracker is stopped.

        Show
        Devaraj Das added a comment - Comments: 1) The class DelegationTokenRenewal could be named better 2) I am having trouble understanding what you have done in DelegationTokenRenewal to do with resetting of timers. When the JobTracker first renews the token, it can check when the token is supposed to be renewed again, and create a corresponding TimerTask. Then cancel the TimerTask when the job is done. The Timer object needs to be created only once, right? 3) You should have the cancellation of tokens based on a boolean configuration like "mapreduce.job.cancel.delegation.tokens". 4) The KIND defined in DelegationTokenRenewal should come from the DelegationTokenIdentifier in HDFS 5) The place where the delegation tokens are registered for renewal is not correct. Note that the JobInProgress constructor uses a token to copy the jobconf file from the hdfs and by then the token for accessing the jobconf should have got renewed. The renewal should happen before the JobInProgress is constructed in submitJob. 6) Also, now that we have MapReduce delegation tokens, it will be nice to have the class DelegationTokenRenewal not assume only hdfs delegation tokens. 7) The Timer should be cancelled when the JobTracker is stopped.
        Hide
        Boris Shkolnik added a comment -

        1) The class DelegationTokenRenewal could be named better

        It seems logical to me. If you have better suggestions please share.

        2) I am having trouble understanding what you have done in DelegationTokenRenewal to do with resetting of timers. When the JobTracker first renews the token, it can check when the token is supposed to be renewed again, and create a corresponding TimerTask. Then cancel the TimerTask when the job is done. The Timer object needs to be created only once, right?

        This way you will create multiple timer objects. In the current design only one timer object is created - for the next expiring token.

        3) You should have the cancellation of tokens based on a boolean configuration like "mapreduce.job.cancel.delegation.tokens".

        this is a new requirement. I will add it. I assume is the value is 'false' - we don't cancel any tokens for the job? (not registering them up for renewal).

        4) The KIND defined in DelegationTokenRenewal should come from the DelegationTokenIdentifier in HDFS

        it is package private. One would need to modify HDFS for it.

        5) The place where the delegation tokens are registered for renewal is not correct. Note that the JobInProgress constructor uses a token to copy the jobconf file from the hdfs and by then the token for accessing the jobconf should have got renewed. The renewal should happen before the JobInProgress is constructed in submitJob.

        I don't think it is critical. Tokens don't really need to be renewed immediately. So the token is perfectly valid at this point.

        6) Also, now that we have MapReduce delegation tokens, it will be nice to have the class DelegationTokenRenewal not assume only hdfs delegation tokens.

        Well, we talked about it - and I understood that we DON'T want to renew MapReduce tokens in JobTracker.

        7) The Timer should be cancelled when the JobTracker is stopped.

        Ok. Will do.

        Show
        Boris Shkolnik added a comment - 1) The class DelegationTokenRenewal could be named better It seems logical to me. If you have better suggestions please share. 2) I am having trouble understanding what you have done in DelegationTokenRenewal to do with resetting of timers. When the JobTracker first renews the token, it can check when the token is supposed to be renewed again, and create a corresponding TimerTask. Then cancel the TimerTask when the job is done. The Timer object needs to be created only once, right? This way you will create multiple timer objects. In the current design only one timer object is created - for the next expiring token. 3) You should have the cancellation of tokens based on a boolean configuration like "mapreduce.job.cancel.delegation.tokens". this is a new requirement. I will add it. I assume is the value is 'false' - we don't cancel any tokens for the job? (not registering them up for renewal). 4) The KIND defined in DelegationTokenRenewal should come from the DelegationTokenIdentifier in HDFS it is package private. One would need to modify HDFS for it. 5) The place where the delegation tokens are registered for renewal is not correct. Note that the JobInProgress constructor uses a token to copy the jobconf file from the hdfs and by then the token for accessing the jobconf should have got renewed. The renewal should happen before the JobInProgress is constructed in submitJob. I don't think it is critical. Tokens don't really need to be renewed immediately. So the token is perfectly valid at this point. 6) Also, now that we have MapReduce delegation tokens, it will be nice to have the class DelegationTokenRenewal not assume only hdfs delegation tokens. Well, we talked about it - and I understood that we DON'T want to renew MapReduce tokens in JobTracker. 7) The Timer should be cancelled when the JobTracker is stopped. Ok. Will do.
        Hide
        Boris Shkolnik added a comment -

        fixed 2,3,5,6,7 from Devaraj's comments

        Show
        Boris Shkolnik added a comment - fixed 2,3,5,6,7 from Devaraj's comments
        Hide
        Devaraj Das added a comment -

        1) For (3), it should be that, cancellation is done for tokens owned by jobs that have mapreduce.job.cancel.delegation.tokens set to true, when the job is done. The idea is to immediately cancel the tokens once the job is done unless the user sets mapreduce.job.cancel.delegation.tokens to false (in which case it probably means that the user launched jobs from tasks of the job in question, and these new jobs would be using the same tokens for talking to the hdfs). For cancelling the token you should invoke the appropriate RPC. Thinking about it, the config could be named to something like "mapreduce.job.complete.cancel.delegation.tokens"

        2) Please make the DelegationTokenRenewal class package private.

        3) renewalTimer should be a daemon thread.

        4) Methods like getDfsForToken could be declared as private.

        5) The logic for setting the renewal to 60 minutes on an exception should be within the renewDelegationToken method (newExpirationDate could be initialized to that).

        6) You don't need synchronization on delegationTokens in the setTimerForTokenRenewal method.

        7) The delegationTokens list could be a HashSet instead

        Otherwise, looks good. I am still unhappy about the fact that you declare a string for "HDFS_DELEGATION_TOKEN" within your class..

        Show
        Devaraj Das added a comment - 1) For (3), it should be that, cancellation is done for tokens owned by jobs that have mapreduce.job.cancel.delegation.tokens set to true, when the job is done. The idea is to immediately cancel the tokens once the job is done unless the user sets mapreduce.job.cancel.delegation.tokens to false (in which case it probably means that the user launched jobs from tasks of the job in question, and these new jobs would be using the same tokens for talking to the hdfs). For cancelling the token you should invoke the appropriate RPC. Thinking about it, the config could be named to something like "mapreduce.job.complete.cancel.delegation.tokens" 2) Please make the DelegationTokenRenewal class package private. 3) renewalTimer should be a daemon thread. 4) Methods like getDfsForToken could be declared as private. 5) The logic for setting the renewal to 60 minutes on an exception should be within the renewDelegationToken method (newExpirationDate could be initialized to that). 6) You don't need synchronization on delegationTokens in the setTimerForTokenRenewal method. 7) The delegationTokens list could be a HashSet instead Otherwise, looks good. I am still unhappy about the fact that you declare a string for "HDFS_DELEGATION_TOKEN" within your class..
        Hide
        Devaraj Das added a comment -

        One more thing - the submitJob API where you set up the delegation tokens for renewal should be within a "try-catch" block. If there is any exception during job submission then the token should be removed from the delegation token set so that it never gets renewed. Also, you should cancel the token if the mapreduce.job.complete.cancel.delegation.tokens is true. BTW by default mapreduce.job.complete.cancel.delegation.tokens should be true.

        Show
        Devaraj Das added a comment - One more thing - the submitJob API where you set up the delegation tokens for renewal should be within a "try-catch" block. If there is any exception during job submission then the token should be removed from the delegation token set so that it never gets renewed. Also, you should cancel the token if the mapreduce.job.complete.cancel.delegation.tokens is true. BTW by default mapreduce.job.complete.cancel.delegation.tokens should be true.
        Hide
        Boris Shkolnik added a comment -

        1) For (3), it should be that, cancellation is done for tokens owned by jobs that have mapreduce.job.cancel.delegation.tokens set to true, when the job is done. The idea is to immediately cancel the tokens once the job is done unless the user sets mapreduce.job.cancel.delegation.tokens to false (in which case it probably means that the user launched jobs from tasks of the job in question, and these new jobs would be using the same tokens for talking to the hdfs). For cancelling the token you should invoke the appropriate RPC. Thinking about it, the config could be named to something like "mapreduce.job.complete.cancel.delegation.tokens"

        Added cancellation for completed jobs (unless "mapreduce.job.complete.cancel.delegation.tokens" is set to false (default is true)

        2) Please make the DelegationTokenRenewal class package private.

        It is used from JobTracker so it must remain public. Added @InterfaceAudience.Private.

        3) renewalTimer should be a daemon thread.

        Done.

        4) Methods like getDfsForToken could be declared as private.

        Done.

        5) The logic for setting the renewal to 60 minutes on an exception should be within the renewDelegationToken method (newExpirationDate could be initialized to that).

        Done.

        6) You don't need synchronization on delegationTokens in the setTimerForTokenRenewal method.

        Done.

        7) The delegationTokens list could be a HashSet instead

        Actually, if we want direct access it should be a MAP keyed by jobId. And that will point to a list. So I changed it to Map<String, List<DTToRenew>>

        I've also
        8) modified Test to verify cancellation of the token.
        9) moved registration for renewal to job initialization and cancelation to garbageCollect()
        10) specified default for ""mapreduce.job.complete.cancel.delegation.tokens" to true in mapred-default.xml
        11) change all Boolean renew/cancelToken to long

        and more..

        Show
        Boris Shkolnik added a comment - 1) For (3), it should be that, cancellation is done for tokens owned by jobs that have mapreduce.job.cancel.delegation.tokens set to true, when the job is done. The idea is to immediately cancel the tokens once the job is done unless the user sets mapreduce.job.cancel.delegation.tokens to false (in which case it probably means that the user launched jobs from tasks of the job in question, and these new jobs would be using the same tokens for talking to the hdfs). For cancelling the token you should invoke the appropriate RPC. Thinking about it, the config could be named to something like "mapreduce.job.complete.cancel.delegation.tokens" Added cancellation for completed jobs (unless "mapreduce.job.complete.cancel.delegation.tokens" is set to false (default is true) 2) Please make the DelegationTokenRenewal class package private. It is used from JobTracker so it must remain public. Added @InterfaceAudience.Private. 3) renewalTimer should be a daemon thread. Done. 4) Methods like getDfsForToken could be declared as private. Done. 5) The logic for setting the renewal to 60 minutes on an exception should be within the renewDelegationToken method (newExpirationDate could be initialized to that). Done. 6) You don't need synchronization on delegationTokens in the setTimerForTokenRenewal method. Done. 7) The delegationTokens list could be a HashSet instead Actually, if we want direct access it should be a MAP keyed by jobId. And that will point to a list. So I changed it to Map<String, List<DTToRenew>> I've also 8) modified Test to verify cancellation of the token. 9) moved registration for renewal to job initialization and cancelation to garbageCollect() 10) specified default for ""mapreduce.job.complete.cancel.delegation.tokens" to true in mapred-default.xml 11) change all Boolean renew/cancelToken to long and more..
        Hide
        Boris Shkolnik added a comment -

        merged with m1503

        Show
        Boris Shkolnik added a comment - merged with m1503
        Hide
        Devaraj Das added a comment -

        Looks close now. Some things to be fixed:
        1) The hashmap for delegation tokens could use the JobID object as the key
        2) I don't see a good motivation for having equals and hashcode implementations in the private class DelegationTokenToRenew. The implementations can be improved as well but I don't see a strong reason for introducing them in this patch.
        3) The initial value of newExpirationDate could be 60 minutes. I don't see the need for initializing it to -1 and then setting it to some other value based on that.
        4) In removeDelegationTokenRenewal, the checks for the jobid for the tokens is redundant. The hashmap already provides you with that list.
        5) The DelegationTokenToRenew class doesn't need to store the jobID at all. Everywhere the jobID could be passed as argument to the methods where it is required.
        6) When would alreadyInMap ever return true? If it never does, i suggest we remove this check.
        7) You haven't synchronized on the accesses to delegationTokens object everywhere. Maybe, a better approach would be to just define the object as a synchronized.

        Show
        Devaraj Das added a comment - Looks close now. Some things to be fixed: 1) The hashmap for delegation tokens could use the JobID object as the key 2) I don't see a good motivation for having equals and hashcode implementations in the private class DelegationTokenToRenew. The implementations can be improved as well but I don't see a strong reason for introducing them in this patch. 3) The initial value of newExpirationDate could be 60 minutes. I don't see the need for initializing it to -1 and then setting it to some other value based on that. 4) In removeDelegationTokenRenewal, the checks for the jobid for the tokens is redundant. The hashmap already provides you with that list. 5) The DelegationTokenToRenew class doesn't need to store the jobID at all. Everywhere the jobID could be passed as argument to the methods where it is required. 6) When would alreadyInMap ever return true? If it never does, i suggest we remove this check. 7) You haven't synchronized on the accesses to delegationTokens object everywhere. Maybe, a better approach would be to just define the object as a synchronized.
        Hide
        Boris Shkolnik added a comment -

        1) The hashmap for delegation tokens could use the JobID object as the key
        yes it could. it could use jobid.toString() too.

        2) I don't see a good motivation for having equals and hashcode implementations in the private class DelegationTokenToRenew. The implementations can be improved as well but I don't see a strong reason for introducing them in this patch.
        Equals is needed to for contains() to work correctly (comparing actual Tokens instead of DelegationTokenToRenew.

        3) The initial value of newExpirationDate could be 60 minutes. I don't see the need for initializing it to -1 and then setting it to some other value based on that.
        This setting is protecting from unexpected/erroneous returns from dfs.renewDelegationToken().

        4) In removeDelegationTokenRenewal, the checks for the jobid for the tokens is redundant. The hashmap already provides you with that list.
        Removed.

        5) The DelegationTokenToRenew class doesn't need to store the jobID at all. Everywhere the jobID could be passed as argument to the methods where it is required.
        This is how it is passed around.

        6) When would alreadyInMap ever return true? If it never does, i suggest we remove this check.
        This is to protect from erroneous calls, to avoid same DelegationToken to be added twice.

        7) You haven't synchronized on the accesses to delegationTokens object everywhere. Maybe, a better approach would be to just define the object as a synchronized.
        Well, the only non-protected write access is with clear() method in close() which is called when JT is shutting down.
        But, just to make the things safer, I will make the map synchronized.

        Show
        Boris Shkolnik added a comment - 1) The hashmap for delegation tokens could use the JobID object as the key yes it could. it could use jobid.toString() too. 2) I don't see a good motivation for having equals and hashcode implementations in the private class DelegationTokenToRenew. The implementations can be improved as well but I don't see a strong reason for introducing them in this patch. Equals is needed to for contains() to work correctly (comparing actual Tokens instead of DelegationTokenToRenew. 3) The initial value of newExpirationDate could be 60 minutes. I don't see the need for initializing it to -1 and then setting it to some other value based on that. This setting is protecting from unexpected/erroneous returns from dfs.renewDelegationToken(). 4) In removeDelegationTokenRenewal, the checks for the jobid for the tokens is redundant. The hashmap already provides you with that list. Removed. 5) The DelegationTokenToRenew class doesn't need to store the jobID at all. Everywhere the jobID could be passed as argument to the methods where it is required. This is how it is passed around. 6) When would alreadyInMap ever return true? If it never does, i suggest we remove this check. This is to protect from erroneous calls, to avoid same DelegationToken to be added twice. 7) You haven't synchronized on the accesses to delegationTokens object everywhere. Maybe, a better approach would be to just define the object as a synchronized. Well, the only non-protected write access is with clear() method in close() which is called when JT is shutting down. But, just to make the things safer, I will make the map synchronized.
        Hide
        Boris Shkolnik added a comment -

        addressed Devaraj's review.

        Show
        Boris Shkolnik added a comment - addressed Devaraj's review.
        Hide
        Devaraj Das added a comment -

        Looks good. But I must admit that I missed a point in my earlier review. The removal of the token when a renewal fails should only be removing that token. Now you remove all the tokens. This may be too aggressive. So i'd suggest add a method removeDelegationTokenRenewal(JobID, Token) and do an exact match of the token and remove that (and that method gets called from renewDelegationToken on an exception). In this implementation, you needn't call cancelToken since the token is invalid already.
        The current implementation of removeDelegationTokenRenewal(JobID) should only be called from JobInProgress then.

        Show
        Devaraj Das added a comment - Looks good. But I must admit that I missed a point in my earlier review. The removal of the token when a renewal fails should only be removing that token. Now you remove all the tokens. This may be too aggressive. So i'd suggest add a method removeDelegationTokenRenewal(JobID, Token) and do an exact match of the token and remove that (and that method gets called from renewDelegationToken on an exception). In this implementation, you needn't call cancelToken since the token is invalid already. The current implementation of removeDelegationTokenRenewal(JobID) should only be called from JobInProgress then.
        Hide
        Boris Shkolnik added a comment -

        fixed (now removes only the failing token)

        Show
        Boris Shkolnik added a comment - fixed (now removes only the failing token)
        Hide
        Hadoop QA added a comment -

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

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

        +1 tests included. The patch appears to include 5 new or modified tests.

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

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

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

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

        -1 core tests. The patch failed core unit tests.

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

        Test results: http://hudson.zones.apache.org/hudson/job/Mapreduce-Patch-h6.grid.sp2.yahoo.net/466/testReport/
        Findbugs warnings: http://hudson.zones.apache.org/hudson/job/Mapreduce-Patch-h6.grid.sp2.yahoo.net/466/artifact/trunk/build/test/findbugs/newPatchFindbugsWarnings.html
        Checkstyle results: http://hudson.zones.apache.org/hudson/job/Mapreduce-Patch-h6.grid.sp2.yahoo.net/466/artifact/trunk/build/test/checkstyle-errors.html
        Console output: http://hudson.zones.apache.org/hudson/job/Mapreduce-Patch-h6.grid.sp2.yahoo.net/466/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/12436382/MAPREDUCE-1430-14.patch against trunk revision 911745. +1 @author. The patch does not contain any @author tags. +1 tests included. The patch appears to include 5 new or modified tests. +1 javadoc. The javadoc tool did not generate any warning messages. +1 javac. The applied patch does not increase the total number of javac compiler warnings. +1 findbugs. The patch does not introduce any new Findbugs warnings. +1 release audit. The applied patch does not increase the total number of release audit warnings. -1 core tests. The patch failed core unit tests. +1 contrib tests. The patch passed contrib unit tests. Test results: http://hudson.zones.apache.org/hudson/job/Mapreduce-Patch-h6.grid.sp2.yahoo.net/466/testReport/ Findbugs warnings: http://hudson.zones.apache.org/hudson/job/Mapreduce-Patch-h6.grid.sp2.yahoo.net/466/artifact/trunk/build/test/findbugs/newPatchFindbugsWarnings.html Checkstyle results: http://hudson.zones.apache.org/hudson/job/Mapreduce-Patch-h6.grid.sp2.yahoo.net/466/artifact/trunk/build/test/checkstyle-errors.html Console output: http://hudson.zones.apache.org/hudson/job/Mapreduce-Patch-h6.grid.sp2.yahoo.net/466/console This message is automatically generated.
        Hide
        Devaraj Das added a comment -

        Sigh.. I know we talked about this offline but i forgot to document on jira. Anyway, I have fixed these minor things in the attached patch.
        1) LOG.info changed to LOG.debug in DelegationTokenRenewal.java.
        2) Defined the config for controlling whether delegation tokens should be cancelled (upon job completion) in JobContext.java.
        3) Defined an API in Job.java to allow users to programmatically set the value of the config that would control the cancellation of the delegation tokens upon job completion.
        4) Moved the call to registerDelegationTokensForRenewal just before the delegation tokens is first used.

        I intend to commit this patch with the above minor changes soon.

        Show
        Devaraj Das added a comment - Sigh.. I know we talked about this offline but i forgot to document on jira. Anyway, I have fixed these minor things in the attached patch. 1) LOG.info changed to LOG.debug in DelegationTokenRenewal.java. 2) Defined the config for controlling whether delegation tokens should be cancelled (upon job completion) in JobContext.java. 3) Defined an API in Job.java to allow users to programmatically set the value of the config that would control the cancellation of the delegation tokens upon job completion. 4) Moved the call to registerDelegationTokensForRenewal just before the delegation tokens is first used. I intend to commit this patch with the above minor changes soon.
        Hide
        Devaraj Das added a comment -

        Another thing fixed - removeDelegationTokenRenewalForJob wasn't removing the map entry from the hashmap (and that could lead to a memory leak). This patch has that fix.

        Show
        Devaraj Das added a comment - Another thing fixed - removeDelegationTokenRenewalForJob wasn't removing the map entry from the hashmap (and that could lead to a memory leak). This patch has that fix.
        Hide
        Boris Shkolnik added a comment -

        1) LOG.info changed to LOG.debug in DelegationTokenRenewal.java.
        I think we should leave at least one LOG.info in the registration. Otherwise we won't be tell what the problem was if it happened.

        otherwise it is good.
        +1

        Show
        Boris Shkolnik added a comment - 1) LOG.info changed to LOG.debug in DelegationTokenRenewal.java. I think we should leave at least one LOG.info in the registration. Otherwise we won't be tell what the problem was if it happened. otherwise it is good. +1
        Hide
        Devaraj Das added a comment -

        Puts back the LOG.info for the delegation token renewal registration. I also removed the call to DelegationTokenRenewal.close from JobTracker.close. After thinking on this aspect it seemed to make sense to not cancel the tokens that the JobTracker didn't own in the first place. Also, when the JobTracker restarts, it can continue to run the jobs that were running earlier, if the delegation tokens are not cancelled. The tokens will die their natural death if no one renews them...

        Show
        Devaraj Das added a comment - Puts back the LOG.info for the delegation token renewal registration. I also removed the call to DelegationTokenRenewal.close from JobTracker.close. After thinking on this aspect it seemed to make sense to not cancel the tokens that the JobTracker didn't own in the first place. Also, when the JobTracker restarts, it can continue to run the jobs that were running earlier, if the delegation tokens are not cancelled. The tokens will die their natural death if no one renews them...
        Hide
        Devaraj Das added a comment -

        Ouch, the token renewal registration should happen after the jobconf is localized. The attached patch fixes this.

        Show
        Devaraj Das added a comment - Ouch, the token renewal registration should happen after the jobconf is localized. The attached patch fixes this.
        Hide
        Boris Shkolnik added a comment -

        test-patch for 1430-dd4.patch
        [exec] +1 overall.
        [exec]
        [exec] +1 @author. The patch does not contain any @author tags.
        [exec]
        [exec] +1 tests included. The patch appears to include 6 new or modified tests.
        [exec]
        [exec] +1 javadoc. The javadoc tool did not generate any warning messages.
        [exec]
        [exec] +1 javac. The applied patch does not increase the total number of javac compiler warnings.
        [exec]
        [exec] +1 findbugs. The patch does not introduce any new Findbugs warnings.
        [exec]
        [exec] +1 release audit. The applied patch does not increase the total number of release audit warnings.

        Show
        Boris Shkolnik added a comment - test-patch for 1430-dd4.patch [exec] +1 overall. [exec] [exec] +1 @author. The patch does not contain any @author tags. [exec] [exec] +1 tests included. The patch appears to include 6 new or modified tests. [exec] [exec] +1 javadoc. The javadoc tool did not generate any warning messages. [exec] [exec] +1 javac. The applied patch does not increase the total number of javac compiler warnings. [exec] [exec] +1 findbugs. The patch does not introduce any new Findbugs warnings. [exec] [exec] +1 release audit. The applied patch does not increase the total number of release audit warnings.
        Hide
        Boris Shkolnik added a comment -

        1430-dd4-BP20.patch - for previous HADOOP version, not for commit

        Show
        Boris Shkolnik added a comment - 1430-dd4-BP20.patch - for previous HADOOP version, not for commit
        Hide
        Devaraj Das added a comment -

        I just committed this. Thanks, Boris!

        Show
        Devaraj Das added a comment - I just committed this. Thanks, Boris!
        Hide
        Boris Shkolnik added a comment -

        1430-BP20-Add.patch - for previous hadoop version
        Takes care of TestQueueManager
        Running tests now.

        Show
        Boris Shkolnik added a comment - 1430-BP20-Add.patch - for previous hadoop version Takes care of TestQueueManager Running tests now.
        Hide
        Hadoop QA added a comment -

        -1 overall. Here are the results of testing the latest attachment
        http://issues.apache.org/jira/secure/attachment/12436453/1430-dd1.patch
        against trunk revision 912196.

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

        +1 tests included. The patch appears to include 6 new or modified tests.

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

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

        -1 findbugs. The patch appears to introduce 1 new Findbugs warnings.

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

        -1 core tests. The patch failed core unit tests.

        -1 contrib tests. The patch failed contrib unit tests.

        Test results: http://hudson.zones.apache.org/hudson/job/Mapreduce-Patch-h6.grid.sp2.yahoo.net/467/testReport/
        Findbugs warnings: http://hudson.zones.apache.org/hudson/job/Mapreduce-Patch-h6.grid.sp2.yahoo.net/467/artifact/trunk/build/test/findbugs/newPatchFindbugsWarnings.html
        Checkstyle results: http://hudson.zones.apache.org/hudson/job/Mapreduce-Patch-h6.grid.sp2.yahoo.net/467/artifact/trunk/build/test/checkstyle-errors.html
        Console output: http://hudson.zones.apache.org/hudson/job/Mapreduce-Patch-h6.grid.sp2.yahoo.net/467/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/12436453/1430-dd1.patch against trunk revision 912196. +1 @author. The patch does not contain any @author tags. +1 tests included. The patch appears to include 6 new or modified tests. +1 javadoc. The javadoc tool did not generate any warning messages. +1 javac. The applied patch does not increase the total number of javac compiler warnings. -1 findbugs. The patch appears to introduce 1 new Findbugs warnings. +1 release audit. The applied patch does not increase the total number of release audit warnings. -1 core tests. The patch failed core unit tests. -1 contrib tests. The patch failed contrib unit tests. Test results: http://hudson.zones.apache.org/hudson/job/Mapreduce-Patch-h6.grid.sp2.yahoo.net/467/testReport/ Findbugs warnings: http://hudson.zones.apache.org/hudson/job/Mapreduce-Patch-h6.grid.sp2.yahoo.net/467/artifact/trunk/build/test/findbugs/newPatchFindbugsWarnings.html Checkstyle results: http://hudson.zones.apache.org/hudson/job/Mapreduce-Patch-h6.grid.sp2.yahoo.net/467/artifact/trunk/build/test/checkstyle-errors.html Console output: http://hudson.zones.apache.org/hudson/job/Mapreduce-Patch-h6.grid.sp2.yahoo.net/467/console This message is automatically generated.
        Hide
        Hudson added a comment -

        Integrated in Hadoop-Mapreduce-trunk-Commit #252 (See http://hudson.zones.apache.org/hudson/job/Hadoop-Mapreduce-trunk-Commit/252/)

        Show
        Hudson added a comment - Integrated in Hadoop-Mapreduce-trunk-Commit #252 (See http://hudson.zones.apache.org/hudson/job/Hadoop-Mapreduce-trunk-Commit/252/ )
        Hide
        Devaraj Das added a comment -

        A bugfix on top of the earlier Y20 patch. Not for commit.

        Show
        Devaraj Das added a comment - A bugfix on top of the earlier Y20 patch. Not for commit.

          People

          • Assignee:
            Boris Shkolnik
            Reporter:
            Devaraj Das
          • Votes:
            0 Vote for this issue
            Watchers:
            0 Start watching this issue

            Dates

            • Created:
              Updated:
              Resolved:

              Development