[ 
https://issues.apache.org/jira/browse/HADOOP-15408?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16451504#comment-16451504
 ] 

Rushabh S Shah edited comment on HADOOP-15408 at 4/25/18 1:44 AM:
------------------------------------------------------------------

Thanks [~xiaochen] for the patch.
Had an offline chat with [~daryn] on the proposed fix.
Below is the summary.
Identifier for both the tokens (i.e KMS_DELEGATION_TOKEN and kms-dt) are the 
same (byte-to-byte) so we don't need to have another class 
{{KMSLegacyDelegationTokenIdentifier}} for legacy token identifier.
 Kind in Identifier doesn't mean much.

After removing {{KMSLegacyDelegationTokenIdentifier}} class and 
{{KMSLegacyDelegationTokenIdentifier}} from 
{{org.apache.hadoop.security.token.TokenIdentifier}}, 4 tests are failing in 
TestKMS because they are trying to decodeIdentifier with kind {{kms-dt}} and 
Serviceloader is not able to find any Identifier which corresponds to 
{{kms-dt}} kind.
 Since it is test-only code, we can change the test.
 Let me know if this makes sense.


was (Author: shahrs87):
Thanks [~xiaochen] for the patch.
Had an offline chat with [~daryn] on the proposed fix.
Identifier for both the tokens (i.e KMS_DELEGATION_TOKEN and kms-dt) are the 
same (byte-to-byte) so we don't need to have another class 
{{KMSLegacyDelegationTokenIdentifier}} for legacy token identifier.
Kind in Identifier doesn't mean much.

After removing {{KMSLegacyDelegationTokenIdentifier}} class and 
{{KMSLegacyDelegationTokenIdentifier}} from 
{{org.apache.hadoop.security.token.TokenIdentifier}}, 4 tests are failing in 
TestKMS because they are trying to decodeIdentifier with kind {{kms-dt}} and 
Serviceloader is not able to find any Identifier which corresponds to 
{{kms-dt}} kind.
Since it is test-only code, we can change the test.
Let me know if this makes sense.

> HADOOP-14445 broke Spark.
> -------------------------
>
>                 Key: HADOOP-15408
>                 URL: https://issues.apache.org/jira/browse/HADOOP-15408
>             Project: Hadoop Common
>          Issue Type: Bug
>    Affects Versions: 2.8.4
>            Reporter: Rushabh S Shah
>            Priority: Blocker
>         Attachments: split.patch
>
>
> Spark bundles hadoop related jars in their package.
>  Spark expects backwards compatibility between minor versions.
>  Their job failed after we deployed HADOOP-14445 in our test cluster.
> {noformat}
> 2018-04-20 21:09:53,245 INFO [main] 
> org.apache.hadoop.mapreduce.v2.app.MRAppMaster: Executing with tokens:
> 2018-04-20 21:09:53,273 ERROR [main] 
> org.apache.hadoop.mapreduce.v2.app.MRAppMaster: Error starting MRAppMaster
> java.util.ServiceConfigurationError: 
> org.apache.hadoop.security.token.TokenIdentifier: Provider 
> org.apache.hadoop.crypto.key.kms.KMSDelegationToken$
> KMSLegacyDelegationTokenIdentifier could not be instantiated
> at java.util.ServiceLoader.fail(ServiceLoader.java:232)
> at java.util.ServiceLoader.access$100(ServiceLoader.java:185)
> at java.util.ServiceLoader$LazyIterator.nextService(ServiceLoader.java:384)
> at java.util.ServiceLoader$LazyIterator.next(ServiceLoader.java:404)
> at java.util.ServiceLoader$1.next(ServiceLoader.java:480)
> at 
> org.apache.hadoop.security.token.Token.getClassForIdentifier(Token.java:117)
> at org.apache.hadoop.security.token.Token.decodeIdentifier(Token.java:138)
> at org.apache.hadoop.security.token.Token.identifierToString(Token.java:393)
> at org.apache.hadoop.security.token.Token.toString(Token.java:413)
> at java.lang.String.valueOf(String.java:2994)
> at 
> org.apache.commons.logging.impl.SLF4JLocationAwareLog.info(SLF4JLocationAwareLog.java:155)
> at 
> org.apache.hadoop.mapreduce.v2.app.MRAppMaster.initAndStartAppMaster(MRAppMaster.java:1634)
> at org.apache.hadoop.mapreduce.v2.app.MRAppMaster.main(MRAppMaster.java:1583)
> Caused by: java.lang.NoSuchFieldError: TOKEN_LEGACY_KIND
> at 
> org.apache.hadoop.crypto.key.kms.KMSDelegationToken$KMSLegacyDelegationTokenIdentifier.<init>(KMSDelegationToken.java:64)
> at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method)
> at 
> sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:62)
> at 
> sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45)
> at java.lang.reflect.Constructor.newInstance(Constructor.java:423)
> at java.lang.Class.newInstance(Class.java:442)
> at java.util.ServiceLoader$LazyIterator.nextService(ServiceLoader.java:380)
> ... 10 more
> 2018-04-20 21:09:53,278 INFO [main] org.apache.hadoop.util.ExitUtil: Exiting 
> with status 1
> {noformat}
> Their classpath looks like 
> {{\{...:hadoop-common-pre-HADOOP-14445.jar:.....:hadoop-common-with-HADOOP-14445.jar:....\}}}
> This is because the container loaded {{KMSDelegationToken}} class from an 
> older jar and {{KMSLegacyDelegationTokenIdentifier}} from new jar and it 
> fails when {{KMSLegacyDelegationTokenIdentifier}} wants to read 
> {{TOKEN_LEGACY_KIND}} from {{KMSDelegationToken}} which doesn't exist before.
>  Cc [~xiaochen]



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: common-issues-h...@hadoop.apache.org

Reply via email to