[ 
https://issues.apache.org/jira/browse/STORM-3577?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Ethan Li updated STORM-3577:
----------------------------
    Description: 
*Background*

Worker uses WorkerToken to connect to Nimbus/Supervisor, (e.g. in 
Worker.doHeartBeat method). If WorkerToken is not in place, it will fall back 
to Kerberos.

 

*Issue:*

Users can submit topology and the topology is running fine.

But error shows up in worker log if "storm upload-credentials" is executed 
(with AutoTGT being used). (2.2.0.y is our internal version of apache-storm 
master branch)

 
{code:java}
2020-02-04 00:12:57.975 o.a.s.d.w.Worker heartbeat-timer [WARN] Exception when 
send heartbeat to local supervisor
2020-02-04 00:12:57.984 o.a.s.s.a.k.ClientCallbackHandler heartbeat-timer 
[WARN] Could not login: the client is being asked for a password, but the  
client code does not currently support obtaining a password from the user. Make 
sure that the client is configured to use a ticket cache (using the JAAS 
configuration setting 'useTicketCache=true)' and restart the client. If you 
still get this message after that, the TGT in the ticket cache has expired and 
must be manually refreshed. To do so, first determine if you are using a 
password or a keytab. If the former, run kinit in a Unix shell in the 
environment of the user who is running this client using the command 'kinit 
<princ>' (where <princ> is the name of the client's Kerberos principal). If the 
latter, do 'kinit -k -t <keytab> <princ>' (where <princ> is the name of the 
Kerberos principal, and <keytab> is the location of the keytab file). After 
manually refreshing your cache, restart this client. If you continue to see 
this message after manually refreshing your cache, ensure that your KDC host's 
clock is in sync with this host's clock.
2020-02-04 00:12:57.984 o.a.s.s.a.k.KerberosSaslTransportPlugin heartbeat-timer 
[ERROR] Server failed to login in 
principal:javax.security.auth.login.LoginException: No password provided
javax.security.auth.login.LoginException: No password provided
        at 
com.sun.security.auth.module.Krb5LoginModule.promptForPass(Krb5LoginModule.java:919)
 ~[?:1.8.0_181]
        at 
com.sun.security.auth.module.Krb5LoginModule.attemptAuthentication(Krb5LoginModule.java:760)
 ~[?:1.8.0_181]
        at 
com.sun.security.auth.module.Krb5LoginModule.login(Krb5LoginModule.java:617) 
~[?:1.8.0_181]
        at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) 
~[?:1.8.0_181]
        at 
sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) 
~[?:1.8.0_181]
        at 
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
 ~[?:1.8.0_181]
        at java.lang.reflect.Method.invoke(Method.java:498) ~[?:1.8.0_181]
        at javax.security.auth.login.LoginContext.invoke(LoginContext.java:755) 
~[?:1.8.0_181]
        at 
javax.security.auth.login.LoginContext.access$000(LoginContext.java:195) 
~[?:1.8.0_181]
        at javax.security.auth.login.LoginContext$4.run(LoginContext.java:682) 
~[?:1.8.0_181]
        at javax.security.auth.login.LoginContext$4.run(LoginContext.java:680) 
~[?:1.8.0_181]
        at java.security.AccessController.doPrivileged(Native Method) 
~[?:1.8.0_181]
        at 
javax.security.auth.login.LoginContext.invokePriv(LoginContext.java:680) 
~[?:1.8.0_181]
        at javax.security.auth.login.LoginContext.login(LoginContext.java:587) 
~[?:1.8.0_181]
        at org.apache.storm.messaging.netty.Login.login(Login.java:300) 
~[storm-client-2.2.0.y.jar:2.2.0.y]
        at org.apache.storm.messaging.netty.Login.<init>(Login.java:84) 
~[storm-client-2.2.0.y.jar:2.2.0.y]
        at 
org.apache.storm.security.auth.kerberos.KerberosSaslTransportPlugin.mkLogin(KerberosSaslTransportPlugin.java:112)
 ~[storm-client-2.2.0.y.jar:2.2.0.y]
        at 
org.apache.storm.security.auth.kerberos.KerberosSaslTransportPlugin.kerberosConnect(KerberosSaslTransportPlugin.java:171)
 ~[storm-client-2.2.0.y.jar:2.2.0.y]
        at 
org.apache.storm.security.auth.kerberos.KerberosSaslTransportPlugin.connect(KerberosSaslTransportPlugin.java:138)
 ~[storm-client-2.2.0.y.jar:2.2.0.y]
        at 
org.apache.storm.security.auth.TBackoffConnect.doConnectWithRetry(TBackoffConnect.java:48)
 ~[storm-client-2.2.0.y.jar:2.2.0.y]
        at 
org.apache.storm.security.auth.ThriftClient.reconnect(ThriftClient.java:98) 
~[storm-client-2.2.0.y.jar:2.2.0.y]
        at 
org.apache.storm.security.auth.ThriftClient.<init>(ThriftClient.java:69) 
~[storm-client-2.2.0.y.jar:2.2.0.y]
        at org.apache.storm.utils.NimbusClient.<init>(NimbusClient.java:80) 
~[storm-client-2.2.0.y.jar:2.2.0.y]
        at 
org.apache.storm.utils.NimbusClient.getConfiguredClientAs(NimbusClient.java:221)
 ~[storm-client-2.2.0.y.jar:2.2.0.y]
        at 
org.apache.storm.utils.NimbusClient.getConfiguredClientAs(NimbusClient.java:179)
 ~[storm-client-2.2.0.y.jar:2.2.0.y]
        at 
org.apache.storm.utils.NimbusClient.getConfiguredClient(NimbusClient.java:138) 
~[storm-client-2.2.0.y.jar:2.2.0.y]
        at 
org.apache.storm.daemon.worker.Worker.heartbeatToMasterIfLocalbeatFail(Worker.java:456)
 ~[storm-client-2.2.0.y.jar:2.2.0.y]
        at org.apache.storm.daemon.worker.Worker.doHeartBeat(Worker.java:361) 
~[storm-client-2.2.0.y.jar:2.2.0.y]
        at 
org.apache.storm.daemon.worker.Worker.lambda$loadWorker$2(Worker.java:209) 
~[storm-client-2.2.0.y.jar:2.2.0.y]
        at org.apache.storm.StormTimer$1.run(StormTimer.java:110) 
[storm-client-2.2.0.y.jar:2.2.0.y]
        at org.apache.storm.StormTimer$StormTimerTask.run(StormTimer.java:226) 
[storm-client-2.2.0.y.jar:2.2.0.y]
2020-02-04 00:12:57.985 o.a.s.u.NimbusClient heartbeat-timer [WARN] Ignoring 
exception while trying to get leader nimbus info from 
quadiumtan-ni.tan.ygrid.yahoo.com. will retry with a different seed host.
java.lang.RuntimeException: java.lang.RuntimeException: 
javax.security.auth.login.LoginException: No password provided
        at 
org.apache.storm.security.auth.ThriftClient.reconnect(ThriftClient.java:108) 
~[storm-client-2.2.0.y.jar:2.2.0.y]
        at 
org.apache.storm.security.auth.ThriftClient.<init>(ThriftClient.java:69) 
~[storm-client-2.2.0.y.jar:2.2.0.y]
        at org.apache.storm.utils.NimbusClient.<init>(NimbusClient.java:80) 
~[storm-client-2.2.0.y.jar:2.2.0.y]
        at 
org.apache.storm.utils.NimbusClient.getConfiguredClientAs(NimbusClient.java:221)
 ~[storm-client-2.2.0.y.jar:2.2.0.y]
        at 
org.apache.storm.utils.NimbusClient.getConfiguredClientAs(NimbusClient.java:179)
 ~[storm-client-2.2.0.y.jar:2.2.0.y]
        at 
org.apache.storm.utils.NimbusClient.getConfiguredClient(NimbusClient.java:138) 
~[storm-client-2.2.0.y.jar:2.2.0.y]
        at 
org.apache.storm.daemon.worker.Worker.heartbeatToMasterIfLocalbeatFail(Worker.java:456)
 ~[storm-client-2.2.0.y.jar:2.2.0.y]
        at org.apache.storm.daemon.worker.Worker.doHeartBeat(Worker.java:361) 
~[storm-client-2.2.0.y.jar:2.2.0.y]
        at 
org.apache.storm.daemon.worker.Worker.lambda$loadWorker$2(Worker.java:209) 
~[storm-client-2.2.0.y.jar:2.2.0.y]
        at org.apache.storm.StormTimer$1.run(StormTimer.java:110) 
[storm-client-2.2.0.y.jar:2.2.0.y]
        at org.apache.storm.StormTimer$StormTimerTask.run(StormTimer.java:226) 
[storm-client-2.2.0.y.jar:2.2.0.y]
Caused by: java.lang.RuntimeException: 
javax.security.auth.login.LoginException: No password provided
        at 
org.apache.storm.security.auth.kerberos.KerberosSaslTransportPlugin.mkLogin(KerberosSaslTransportPlugin.java:117)
 ~[storm-client-2.2.0.y.jar:2.2.0.y]
        at 
org.apache.storm.security.auth.kerberos.KerberosSaslTransportPlugin.kerberosConnect(KerberosSaslTransportPlugin.java:171)
 ~[storm-client-2.2.0.y.jar:2.2.0.y]
        at 
org.apache.storm.security.auth.kerberos.KerberosSaslTransportPlugin.connect(KerberosSaslTransportPlugin.java:138)
 ~[storm-client-2.2.0.y.jar:2.2.0.y]
        at 
org.apache.storm.security.auth.TBackoffConnect.doConnectWithRetry(TBackoffConnect.java:48)
 ~[storm-client-2.2.0.y.jar:2.2.0.y]
        at 
org.apache.storm.security.auth.ThriftClient.reconnect(ThriftClient.java:98) 
~[storm-client-2.2.0.y.jar:2.2.0.y]
        ... 10 more
Caused by: javax.security.auth.login.LoginException: No password provided
        at 
com.sun.security.auth.module.Krb5LoginModule.promptForPass(Krb5LoginModule.java:919)
 ~[?:1.8.0_181]
        at 
com.sun.security.auth.module.Krb5LoginModule.attemptAuthentication(Krb5LoginModule.java:760)
 ~[?:1.8.0_181]
        at 
com.sun.security.auth.module.Krb5LoginModule.login(Krb5LoginModule.java:617) 
~[?:1.8.0_181]
        at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) 
~[?:1.8.0_181]
        at 
sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) 
~[?:1.8.0_181]
        at 
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
 ~[?:1.8.0_181]
        at java.lang.reflect.Method.invoke(Method.java:498) ~[?:1.8.0_181]
        at javax.security.auth.login.LoginContext.invoke(LoginContext.java:755) 
~[?:1.8.0_181]
        at 
javax.security.auth.login.LoginContext.access$000(LoginContext.java:195) 
~[?:1.8.0_181]
        at javax.security.auth.login.LoginContext$4.run(LoginContext.java:682) 
~[?:1.8.0_181]
        at javax.security.auth.login.LoginContext$4.run(LoginContext.java:680) 
~[?:1.8.0_181]
        at java.security.AccessController.doPrivileged(Native Method) 
~[?:1.8.0_181]
        at 
javax.security.auth.login.LoginContext.invokePriv(LoginContext.java:680) 
~[?:1.8.0_181]
        at javax.security.auth.login.LoginContext.login(LoginContext.java:587) 
~[?:1.8.0_181]
        at org.apache.storm.messaging.netty.Login.login(Login.java:300) 
~[storm-client-2.2.0.y.jar:2.2.0.y]
        at org.apache.storm.messaging.netty.Login.<init>(Login.java:84) 
~[storm-client-2.2.0.y.jar:2.2.0.y]
        at 
org.apache.storm.security.auth.kerberos.KerberosSaslTransportPlugin.mkLogin(KerberosSaslTransportPlugin.java:112)
 ~[storm-client-2.2.0.y.jar:2.2.0.y]
        at 
org.apache.storm.security.auth.kerberos.KerberosSaslTransportPlugin.kerberosConnect(KerberosSaslTransportPlugin.java:171)
 ~[storm-client-2.2.0.y.jar:2.2.0.y]
        at 
org.apache.storm.security.auth.kerberos.KerberosSaslTransportPlugin.connect(KerberosSaslTransportPlugin.java:138)
 ~[storm-client-2.2.0.y.jar:2.2.0.y]
        at 
org.apache.storm.security.auth.TBackoffConnect.doConnectWithRetry(TBackoffConnect.java:48)
 ~[storm-client-2.2.0.y.jar:2.2.0.y]
        at 
org.apache.storm.security.auth.ThriftClient.reconnect(ThriftClient.java:98) 
~[storm-client-2.2.0.y.jar:2.2.0.y]
        ... 10 more
{code}
It can be reproduced by
{code:java}
/storm jar /home/y/lib64/jars/storm-starter.jar  
org.apache.storm.starter.WordCountTopology wc -c topology.debug=false

kinit -R # refresh TGT. This is must-have. So upload-credentials will do 
something and trigger the bug

storm upload-credentials wc

## Errors will show up in worker log in up to 30s (credential refresh period)
{code}
 

*BUGS*

 

*BUG1* When new credentials got uploaded, Worker will try to update 
credentials. But while it does it, it will also try to replace WorkerToken if 
it changes. But it has a bug in the code:

[https://github.com/apache/storm/blob/master/storm-client/src/jvm/org/apache/storm/security/auth/ClientAuthUtils.java#L411-L416]

 

Here in the code, "token" could equal to "previous" if tokens didn't change 
because WorkerToken.equals() method only cares about the content of 
WorkerToken. The result of this function is the tokens got removed completely.

So in this case, because tokens are not present, Worker will fall back to use 
kerberos to connect to Nimbus/Supervisor. 
[https://github.com/apache/storm/blob/master/storm-client/src/jvm/org/apache/storm/security/auth/kerberos/KerberosSaslTransportPlugin.java#L122-L139]

And here comes the second bug

*BUG2*. Kerberos connection from Worker to Nimbus/Supervisor is not working 
properly, hence the error logs above. 

  was:
*Background*

Worker uses WorkerToken to connect to Nimbus/Supervisor, (e.g. in 
Worker.doHeartBeat method). If WorkerToken is not in place, it will fall back 
to Kerberos.

 

*Issue:*

Users can submit topology and the topology is running fine.

But error shows up in worker log if "storm upload-credentials" is executed. 
(2.2.0.y is our internal version of apache-storm master branch)

 
{code:java}
2020-02-04 00:12:57.975 o.a.s.d.w.Worker heartbeat-timer [WARN] Exception when 
send heartbeat to local supervisor
2020-02-04 00:12:57.984 o.a.s.s.a.k.ClientCallbackHandler heartbeat-timer 
[WARN] Could not login: the client is being asked for a password, but the  
client code does not currently support obtaining a password from the user. Make 
sure that the client is configured to use a ticket cache (using the JAAS 
configuration setting 'useTicketCache=true)' and restart the client. If you 
still get this message after that, the TGT in the ticket cache has expired and 
must be manually refreshed. To do so, first determine if you are using a 
password or a keytab. If the former, run kinit in a Unix shell in the 
environment of the user who is running this client using the command 'kinit 
<princ>' (where <princ> is the name of the client's Kerberos principal). If the 
latter, do 'kinit -k -t <keytab> <princ>' (where <princ> is the name of the 
Kerberos principal, and <keytab> is the location of the keytab file). After 
manually refreshing your cache, restart this client. If you continue to see 
this message after manually refreshing your cache, ensure that your KDC host's 
clock is in sync with this host's clock.
2020-02-04 00:12:57.984 o.a.s.s.a.k.KerberosSaslTransportPlugin heartbeat-timer 
[ERROR] Server failed to login in 
principal:javax.security.auth.login.LoginException: No password provided
javax.security.auth.login.LoginException: No password provided
        at 
com.sun.security.auth.module.Krb5LoginModule.promptForPass(Krb5LoginModule.java:919)
 ~[?:1.8.0_181]
        at 
com.sun.security.auth.module.Krb5LoginModule.attemptAuthentication(Krb5LoginModule.java:760)
 ~[?:1.8.0_181]
        at 
com.sun.security.auth.module.Krb5LoginModule.login(Krb5LoginModule.java:617) 
~[?:1.8.0_181]
        at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) 
~[?:1.8.0_181]
        at 
sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) 
~[?:1.8.0_181]
        at 
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
 ~[?:1.8.0_181]
        at java.lang.reflect.Method.invoke(Method.java:498) ~[?:1.8.0_181]
        at javax.security.auth.login.LoginContext.invoke(LoginContext.java:755) 
~[?:1.8.0_181]
        at 
javax.security.auth.login.LoginContext.access$000(LoginContext.java:195) 
~[?:1.8.0_181]
        at javax.security.auth.login.LoginContext$4.run(LoginContext.java:682) 
~[?:1.8.0_181]
        at javax.security.auth.login.LoginContext$4.run(LoginContext.java:680) 
~[?:1.8.0_181]
        at java.security.AccessController.doPrivileged(Native Method) 
~[?:1.8.0_181]
        at 
javax.security.auth.login.LoginContext.invokePriv(LoginContext.java:680) 
~[?:1.8.0_181]
        at javax.security.auth.login.LoginContext.login(LoginContext.java:587) 
~[?:1.8.0_181]
        at org.apache.storm.messaging.netty.Login.login(Login.java:300) 
~[storm-client-2.2.0.y.jar:2.2.0.y]
        at org.apache.storm.messaging.netty.Login.<init>(Login.java:84) 
~[storm-client-2.2.0.y.jar:2.2.0.y]
        at 
org.apache.storm.security.auth.kerberos.KerberosSaslTransportPlugin.mkLogin(KerberosSaslTransportPlugin.java:112)
 ~[storm-client-2.2.0.y.jar:2.2.0.y]
        at 
org.apache.storm.security.auth.kerberos.KerberosSaslTransportPlugin.kerberosConnect(KerberosSaslTransportPlugin.java:171)
 ~[storm-client-2.2.0.y.jar:2.2.0.y]
        at 
org.apache.storm.security.auth.kerberos.KerberosSaslTransportPlugin.connect(KerberosSaslTransportPlugin.java:138)
 ~[storm-client-2.2.0.y.jar:2.2.0.y]
        at 
org.apache.storm.security.auth.TBackoffConnect.doConnectWithRetry(TBackoffConnect.java:48)
 ~[storm-client-2.2.0.y.jar:2.2.0.y]
        at 
org.apache.storm.security.auth.ThriftClient.reconnect(ThriftClient.java:98) 
~[storm-client-2.2.0.y.jar:2.2.0.y]
        at 
org.apache.storm.security.auth.ThriftClient.<init>(ThriftClient.java:69) 
~[storm-client-2.2.0.y.jar:2.2.0.y]
        at org.apache.storm.utils.NimbusClient.<init>(NimbusClient.java:80) 
~[storm-client-2.2.0.y.jar:2.2.0.y]
        at 
org.apache.storm.utils.NimbusClient.getConfiguredClientAs(NimbusClient.java:221)
 ~[storm-client-2.2.0.y.jar:2.2.0.y]
        at 
org.apache.storm.utils.NimbusClient.getConfiguredClientAs(NimbusClient.java:179)
 ~[storm-client-2.2.0.y.jar:2.2.0.y]
        at 
org.apache.storm.utils.NimbusClient.getConfiguredClient(NimbusClient.java:138) 
~[storm-client-2.2.0.y.jar:2.2.0.y]
        at 
org.apache.storm.daemon.worker.Worker.heartbeatToMasterIfLocalbeatFail(Worker.java:456)
 ~[storm-client-2.2.0.y.jar:2.2.0.y]
        at org.apache.storm.daemon.worker.Worker.doHeartBeat(Worker.java:361) 
~[storm-client-2.2.0.y.jar:2.2.0.y]
        at 
org.apache.storm.daemon.worker.Worker.lambda$loadWorker$2(Worker.java:209) 
~[storm-client-2.2.0.y.jar:2.2.0.y]
        at org.apache.storm.StormTimer$1.run(StormTimer.java:110) 
[storm-client-2.2.0.y.jar:2.2.0.y]
        at org.apache.storm.StormTimer$StormTimerTask.run(StormTimer.java:226) 
[storm-client-2.2.0.y.jar:2.2.0.y]
2020-02-04 00:12:57.985 o.a.s.u.NimbusClient heartbeat-timer [WARN] Ignoring 
exception while trying to get leader nimbus info from 
quadiumtan-ni.tan.ygrid.yahoo.com. will retry with a different seed host.
java.lang.RuntimeException: java.lang.RuntimeException: 
javax.security.auth.login.LoginException: No password provided
        at 
org.apache.storm.security.auth.ThriftClient.reconnect(ThriftClient.java:108) 
~[storm-client-2.2.0.y.jar:2.2.0.y]
        at 
org.apache.storm.security.auth.ThriftClient.<init>(ThriftClient.java:69) 
~[storm-client-2.2.0.y.jar:2.2.0.y]
        at org.apache.storm.utils.NimbusClient.<init>(NimbusClient.java:80) 
~[storm-client-2.2.0.y.jar:2.2.0.y]
        at 
org.apache.storm.utils.NimbusClient.getConfiguredClientAs(NimbusClient.java:221)
 ~[storm-client-2.2.0.y.jar:2.2.0.y]
        at 
org.apache.storm.utils.NimbusClient.getConfiguredClientAs(NimbusClient.java:179)
 ~[storm-client-2.2.0.y.jar:2.2.0.y]
        at 
org.apache.storm.utils.NimbusClient.getConfiguredClient(NimbusClient.java:138) 
~[storm-client-2.2.0.y.jar:2.2.0.y]
        at 
org.apache.storm.daemon.worker.Worker.heartbeatToMasterIfLocalbeatFail(Worker.java:456)
 ~[storm-client-2.2.0.y.jar:2.2.0.y]
        at org.apache.storm.daemon.worker.Worker.doHeartBeat(Worker.java:361) 
~[storm-client-2.2.0.y.jar:2.2.0.y]
        at 
org.apache.storm.daemon.worker.Worker.lambda$loadWorker$2(Worker.java:209) 
~[storm-client-2.2.0.y.jar:2.2.0.y]
        at org.apache.storm.StormTimer$1.run(StormTimer.java:110) 
[storm-client-2.2.0.y.jar:2.2.0.y]
        at org.apache.storm.StormTimer$StormTimerTask.run(StormTimer.java:226) 
[storm-client-2.2.0.y.jar:2.2.0.y]
Caused by: java.lang.RuntimeException: 
javax.security.auth.login.LoginException: No password provided
        at 
org.apache.storm.security.auth.kerberos.KerberosSaslTransportPlugin.mkLogin(KerberosSaslTransportPlugin.java:117)
 ~[storm-client-2.2.0.y.jar:2.2.0.y]
        at 
org.apache.storm.security.auth.kerberos.KerberosSaslTransportPlugin.kerberosConnect(KerberosSaslTransportPlugin.java:171)
 ~[storm-client-2.2.0.y.jar:2.2.0.y]
        at 
org.apache.storm.security.auth.kerberos.KerberosSaslTransportPlugin.connect(KerberosSaslTransportPlugin.java:138)
 ~[storm-client-2.2.0.y.jar:2.2.0.y]
        at 
org.apache.storm.security.auth.TBackoffConnect.doConnectWithRetry(TBackoffConnect.java:48)
 ~[storm-client-2.2.0.y.jar:2.2.0.y]
        at 
org.apache.storm.security.auth.ThriftClient.reconnect(ThriftClient.java:98) 
~[storm-client-2.2.0.y.jar:2.2.0.y]
        ... 10 more
Caused by: javax.security.auth.login.LoginException: No password provided
        at 
com.sun.security.auth.module.Krb5LoginModule.promptForPass(Krb5LoginModule.java:919)
 ~[?:1.8.0_181]
        at 
com.sun.security.auth.module.Krb5LoginModule.attemptAuthentication(Krb5LoginModule.java:760)
 ~[?:1.8.0_181]
        at 
com.sun.security.auth.module.Krb5LoginModule.login(Krb5LoginModule.java:617) 
~[?:1.8.0_181]
        at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) 
~[?:1.8.0_181]
        at 
sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) 
~[?:1.8.0_181]
        at 
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
 ~[?:1.8.0_181]
        at java.lang.reflect.Method.invoke(Method.java:498) ~[?:1.8.0_181]
        at javax.security.auth.login.LoginContext.invoke(LoginContext.java:755) 
~[?:1.8.0_181]
        at 
javax.security.auth.login.LoginContext.access$000(LoginContext.java:195) 
~[?:1.8.0_181]
        at javax.security.auth.login.LoginContext$4.run(LoginContext.java:682) 
~[?:1.8.0_181]
        at javax.security.auth.login.LoginContext$4.run(LoginContext.java:680) 
~[?:1.8.0_181]
        at java.security.AccessController.doPrivileged(Native Method) 
~[?:1.8.0_181]
        at 
javax.security.auth.login.LoginContext.invokePriv(LoginContext.java:680) 
~[?:1.8.0_181]
        at javax.security.auth.login.LoginContext.login(LoginContext.java:587) 
~[?:1.8.0_181]
        at org.apache.storm.messaging.netty.Login.login(Login.java:300) 
~[storm-client-2.2.0.y.jar:2.2.0.y]
        at org.apache.storm.messaging.netty.Login.<init>(Login.java:84) 
~[storm-client-2.2.0.y.jar:2.2.0.y]
        at 
org.apache.storm.security.auth.kerberos.KerberosSaslTransportPlugin.mkLogin(KerberosSaslTransportPlugin.java:112)
 ~[storm-client-2.2.0.y.jar:2.2.0.y]
        at 
org.apache.storm.security.auth.kerberos.KerberosSaslTransportPlugin.kerberosConnect(KerberosSaslTransportPlugin.java:171)
 ~[storm-client-2.2.0.y.jar:2.2.0.y]
        at 
org.apache.storm.security.auth.kerberos.KerberosSaslTransportPlugin.connect(KerberosSaslTransportPlugin.java:138)
 ~[storm-client-2.2.0.y.jar:2.2.0.y]
        at 
org.apache.storm.security.auth.TBackoffConnect.doConnectWithRetry(TBackoffConnect.java:48)
 ~[storm-client-2.2.0.y.jar:2.2.0.y]
        at 
org.apache.storm.security.auth.ThriftClient.reconnect(ThriftClient.java:98) 
~[storm-client-2.2.0.y.jar:2.2.0.y]
        ... 10 more
{code}
It can be reproduced by
{code:java}
/storm jar /home/y/lib64/jars/storm-starter.jar  
org.apache.storm.starter.WordCountTopology wc -c topology.debug=false

kinit -R # refresh TGT. This is must-have. So upload-credentials will do 
something and trigger the bug

storm upload-credentials wc

## Errors will show up in worker log in up to 30s (credential refresh period)
{code}
 

*BUGS*

 

*BUG1* When new credentials got uploaded, Worker will try to update 
credentials. But while it does it, it will also try to replace WorkerToken if 
it changes. But it has a bug in the code:

[https://github.com/apache/storm/blob/master/storm-client/src/jvm/org/apache/storm/security/auth/ClientAuthUtils.java#L411-L416]

 

Here in the code, "token" could equal to "previous" if tokens didn't change 
because WorkerToken.equals() method only cares about the content of 
WorkerToken. The result of this function is the tokens got removed completely.

So in this case, because tokens are not present, Worker will fall back to use 
kerberos to connect to Nimbus/Supervisor. 
[https://github.com/apache/storm/blob/master/storm-client/src/jvm/org/apache/storm/security/auth/kerberos/KerberosSaslTransportPlugin.java#L122-L139]

And here comes the second bug

*BUG2*. Kerberos connection from Worker to Nimbus/Supervisor is not working 
properly, hence the error logs above. 


> upload-credentials Breaks Topology in secure cluster
> ----------------------------------------------------
>
>                 Key: STORM-3577
>                 URL: https://issues.apache.org/jira/browse/STORM-3577
>             Project: Apache Storm
>          Issue Type: Bug
>    Affects Versions: 2.0.0, 2.1.0
>            Reporter: Ethan Li
>            Priority: Critical
>              Labels: pull-request-available
>          Time Spent: 10m
>  Remaining Estimate: 0h
>
> *Background*
> Worker uses WorkerToken to connect to Nimbus/Supervisor, (e.g. in 
> Worker.doHeartBeat method). If WorkerToken is not in place, it will fall back 
> to Kerberos.
>  
> *Issue:*
> Users can submit topology and the topology is running fine.
> But error shows up in worker log if "storm upload-credentials" is executed 
> (with AutoTGT being used). (2.2.0.y is our internal version of apache-storm 
> master branch)
>  
> {code:java}
> 2020-02-04 00:12:57.975 o.a.s.d.w.Worker heartbeat-timer [WARN] Exception 
> when send heartbeat to local supervisor
> 2020-02-04 00:12:57.984 o.a.s.s.a.k.ClientCallbackHandler heartbeat-timer 
> [WARN] Could not login: the client is being asked for a password, but the  
> client code does not currently support obtaining a password from the user. 
> Make sure that the client is configured to use a ticket cache (using the JAAS 
> configuration setting 'useTicketCache=true)' and restart the client. If you 
> still get this message after that, the TGT in the ticket cache has expired 
> and must be manually refreshed. To do so, first determine if you are using a 
> password or a keytab. If the former, run kinit in a Unix shell in the 
> environment of the user who is running this client using the command 'kinit 
> <princ>' (where <princ> is the name of the client's Kerberos principal). If 
> the latter, do 'kinit -k -t <keytab> <princ>' (where <princ> is the name of 
> the Kerberos principal, and <keytab> is the location of the keytab file). 
> After manually refreshing your cache, restart this client. If you continue to 
> see this message after manually refreshing your cache, ensure that your KDC 
> host's clock is in sync with this host's clock.
> 2020-02-04 00:12:57.984 o.a.s.s.a.k.KerberosSaslTransportPlugin 
> heartbeat-timer [ERROR] Server failed to login in 
> principal:javax.security.auth.login.LoginException: No password provided
> javax.security.auth.login.LoginException: No password provided
>       at 
> com.sun.security.auth.module.Krb5LoginModule.promptForPass(Krb5LoginModule.java:919)
>  ~[?:1.8.0_181]
>       at 
> com.sun.security.auth.module.Krb5LoginModule.attemptAuthentication(Krb5LoginModule.java:760)
>  ~[?:1.8.0_181]
>       at 
> com.sun.security.auth.module.Krb5LoginModule.login(Krb5LoginModule.java:617) 
> ~[?:1.8.0_181]
>       at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) 
> ~[?:1.8.0_181]
>       at 
> sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) 
> ~[?:1.8.0_181]
>       at 
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
>  ~[?:1.8.0_181]
>       at java.lang.reflect.Method.invoke(Method.java:498) ~[?:1.8.0_181]
>       at javax.security.auth.login.LoginContext.invoke(LoginContext.java:755) 
> ~[?:1.8.0_181]
>       at 
> javax.security.auth.login.LoginContext.access$000(LoginContext.java:195) 
> ~[?:1.8.0_181]
>       at javax.security.auth.login.LoginContext$4.run(LoginContext.java:682) 
> ~[?:1.8.0_181]
>       at javax.security.auth.login.LoginContext$4.run(LoginContext.java:680) 
> ~[?:1.8.0_181]
>       at java.security.AccessController.doPrivileged(Native Method) 
> ~[?:1.8.0_181]
>       at 
> javax.security.auth.login.LoginContext.invokePriv(LoginContext.java:680) 
> ~[?:1.8.0_181]
>       at javax.security.auth.login.LoginContext.login(LoginContext.java:587) 
> ~[?:1.8.0_181]
>       at org.apache.storm.messaging.netty.Login.login(Login.java:300) 
> ~[storm-client-2.2.0.y.jar:2.2.0.y]
>       at org.apache.storm.messaging.netty.Login.<init>(Login.java:84) 
> ~[storm-client-2.2.0.y.jar:2.2.0.y]
>       at 
> org.apache.storm.security.auth.kerberos.KerberosSaslTransportPlugin.mkLogin(KerberosSaslTransportPlugin.java:112)
>  ~[storm-client-2.2.0.y.jar:2.2.0.y]
>       at 
> org.apache.storm.security.auth.kerberos.KerberosSaslTransportPlugin.kerberosConnect(KerberosSaslTransportPlugin.java:171)
>  ~[storm-client-2.2.0.y.jar:2.2.0.y]
>       at 
> org.apache.storm.security.auth.kerberos.KerberosSaslTransportPlugin.connect(KerberosSaslTransportPlugin.java:138)
>  ~[storm-client-2.2.0.y.jar:2.2.0.y]
>       at 
> org.apache.storm.security.auth.TBackoffConnect.doConnectWithRetry(TBackoffConnect.java:48)
>  ~[storm-client-2.2.0.y.jar:2.2.0.y]
>       at 
> org.apache.storm.security.auth.ThriftClient.reconnect(ThriftClient.java:98) 
> ~[storm-client-2.2.0.y.jar:2.2.0.y]
>       at 
> org.apache.storm.security.auth.ThriftClient.<init>(ThriftClient.java:69) 
> ~[storm-client-2.2.0.y.jar:2.2.0.y]
>       at org.apache.storm.utils.NimbusClient.<init>(NimbusClient.java:80) 
> ~[storm-client-2.2.0.y.jar:2.2.0.y]
>       at 
> org.apache.storm.utils.NimbusClient.getConfiguredClientAs(NimbusClient.java:221)
>  ~[storm-client-2.2.0.y.jar:2.2.0.y]
>       at 
> org.apache.storm.utils.NimbusClient.getConfiguredClientAs(NimbusClient.java:179)
>  ~[storm-client-2.2.0.y.jar:2.2.0.y]
>       at 
> org.apache.storm.utils.NimbusClient.getConfiguredClient(NimbusClient.java:138)
>  ~[storm-client-2.2.0.y.jar:2.2.0.y]
>       at 
> org.apache.storm.daemon.worker.Worker.heartbeatToMasterIfLocalbeatFail(Worker.java:456)
>  ~[storm-client-2.2.0.y.jar:2.2.0.y]
>       at org.apache.storm.daemon.worker.Worker.doHeartBeat(Worker.java:361) 
> ~[storm-client-2.2.0.y.jar:2.2.0.y]
>       at 
> org.apache.storm.daemon.worker.Worker.lambda$loadWorker$2(Worker.java:209) 
> ~[storm-client-2.2.0.y.jar:2.2.0.y]
>       at org.apache.storm.StormTimer$1.run(StormTimer.java:110) 
> [storm-client-2.2.0.y.jar:2.2.0.y]
>       at org.apache.storm.StormTimer$StormTimerTask.run(StormTimer.java:226) 
> [storm-client-2.2.0.y.jar:2.2.0.y]
> 2020-02-04 00:12:57.985 o.a.s.u.NimbusClient heartbeat-timer [WARN] Ignoring 
> exception while trying to get leader nimbus info from 
> quadiumtan-ni.tan.ygrid.yahoo.com. will retry with a different seed host.
> java.lang.RuntimeException: java.lang.RuntimeException: 
> javax.security.auth.login.LoginException: No password provided
>       at 
> org.apache.storm.security.auth.ThriftClient.reconnect(ThriftClient.java:108) 
> ~[storm-client-2.2.0.y.jar:2.2.0.y]
>       at 
> org.apache.storm.security.auth.ThriftClient.<init>(ThriftClient.java:69) 
> ~[storm-client-2.2.0.y.jar:2.2.0.y]
>       at org.apache.storm.utils.NimbusClient.<init>(NimbusClient.java:80) 
> ~[storm-client-2.2.0.y.jar:2.2.0.y]
>       at 
> org.apache.storm.utils.NimbusClient.getConfiguredClientAs(NimbusClient.java:221)
>  ~[storm-client-2.2.0.y.jar:2.2.0.y]
>       at 
> org.apache.storm.utils.NimbusClient.getConfiguredClientAs(NimbusClient.java:179)
>  ~[storm-client-2.2.0.y.jar:2.2.0.y]
>       at 
> org.apache.storm.utils.NimbusClient.getConfiguredClient(NimbusClient.java:138)
>  ~[storm-client-2.2.0.y.jar:2.2.0.y]
>       at 
> org.apache.storm.daemon.worker.Worker.heartbeatToMasterIfLocalbeatFail(Worker.java:456)
>  ~[storm-client-2.2.0.y.jar:2.2.0.y]
>       at org.apache.storm.daemon.worker.Worker.doHeartBeat(Worker.java:361) 
> ~[storm-client-2.2.0.y.jar:2.2.0.y]
>       at 
> org.apache.storm.daemon.worker.Worker.lambda$loadWorker$2(Worker.java:209) 
> ~[storm-client-2.2.0.y.jar:2.2.0.y]
>       at org.apache.storm.StormTimer$1.run(StormTimer.java:110) 
> [storm-client-2.2.0.y.jar:2.2.0.y]
>       at org.apache.storm.StormTimer$StormTimerTask.run(StormTimer.java:226) 
> [storm-client-2.2.0.y.jar:2.2.0.y]
> Caused by: java.lang.RuntimeException: 
> javax.security.auth.login.LoginException: No password provided
>       at 
> org.apache.storm.security.auth.kerberos.KerberosSaslTransportPlugin.mkLogin(KerberosSaslTransportPlugin.java:117)
>  ~[storm-client-2.2.0.y.jar:2.2.0.y]
>       at 
> org.apache.storm.security.auth.kerberos.KerberosSaslTransportPlugin.kerberosConnect(KerberosSaslTransportPlugin.java:171)
>  ~[storm-client-2.2.0.y.jar:2.2.0.y]
>       at 
> org.apache.storm.security.auth.kerberos.KerberosSaslTransportPlugin.connect(KerberosSaslTransportPlugin.java:138)
>  ~[storm-client-2.2.0.y.jar:2.2.0.y]
>       at 
> org.apache.storm.security.auth.TBackoffConnect.doConnectWithRetry(TBackoffConnect.java:48)
>  ~[storm-client-2.2.0.y.jar:2.2.0.y]
>       at 
> org.apache.storm.security.auth.ThriftClient.reconnect(ThriftClient.java:98) 
> ~[storm-client-2.2.0.y.jar:2.2.0.y]
>       ... 10 more
> Caused by: javax.security.auth.login.LoginException: No password provided
>       at 
> com.sun.security.auth.module.Krb5LoginModule.promptForPass(Krb5LoginModule.java:919)
>  ~[?:1.8.0_181]
>       at 
> com.sun.security.auth.module.Krb5LoginModule.attemptAuthentication(Krb5LoginModule.java:760)
>  ~[?:1.8.0_181]
>       at 
> com.sun.security.auth.module.Krb5LoginModule.login(Krb5LoginModule.java:617) 
> ~[?:1.8.0_181]
>       at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) 
> ~[?:1.8.0_181]
>       at 
> sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) 
> ~[?:1.8.0_181]
>       at 
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
>  ~[?:1.8.0_181]
>       at java.lang.reflect.Method.invoke(Method.java:498) ~[?:1.8.0_181]
>       at javax.security.auth.login.LoginContext.invoke(LoginContext.java:755) 
> ~[?:1.8.0_181]
>       at 
> javax.security.auth.login.LoginContext.access$000(LoginContext.java:195) 
> ~[?:1.8.0_181]
>       at javax.security.auth.login.LoginContext$4.run(LoginContext.java:682) 
> ~[?:1.8.0_181]
>       at javax.security.auth.login.LoginContext$4.run(LoginContext.java:680) 
> ~[?:1.8.0_181]
>       at java.security.AccessController.doPrivileged(Native Method) 
> ~[?:1.8.0_181]
>       at 
> javax.security.auth.login.LoginContext.invokePriv(LoginContext.java:680) 
> ~[?:1.8.0_181]
>       at javax.security.auth.login.LoginContext.login(LoginContext.java:587) 
> ~[?:1.8.0_181]
>       at org.apache.storm.messaging.netty.Login.login(Login.java:300) 
> ~[storm-client-2.2.0.y.jar:2.2.0.y]
>       at org.apache.storm.messaging.netty.Login.<init>(Login.java:84) 
> ~[storm-client-2.2.0.y.jar:2.2.0.y]
>       at 
> org.apache.storm.security.auth.kerberos.KerberosSaslTransportPlugin.mkLogin(KerberosSaslTransportPlugin.java:112)
>  ~[storm-client-2.2.0.y.jar:2.2.0.y]
>       at 
> org.apache.storm.security.auth.kerberos.KerberosSaslTransportPlugin.kerberosConnect(KerberosSaslTransportPlugin.java:171)
>  ~[storm-client-2.2.0.y.jar:2.2.0.y]
>       at 
> org.apache.storm.security.auth.kerberos.KerberosSaslTransportPlugin.connect(KerberosSaslTransportPlugin.java:138)
>  ~[storm-client-2.2.0.y.jar:2.2.0.y]
>       at 
> org.apache.storm.security.auth.TBackoffConnect.doConnectWithRetry(TBackoffConnect.java:48)
>  ~[storm-client-2.2.0.y.jar:2.2.0.y]
>       at 
> org.apache.storm.security.auth.ThriftClient.reconnect(ThriftClient.java:98) 
> ~[storm-client-2.2.0.y.jar:2.2.0.y]
>       ... 10 more
> {code}
> It can be reproduced by
> {code:java}
> /storm jar /home/y/lib64/jars/storm-starter.jar  
> org.apache.storm.starter.WordCountTopology wc -c topology.debug=false
> kinit -R # refresh TGT. This is must-have. So upload-credentials will do 
> something and trigger the bug
> storm upload-credentials wc
> ## Errors will show up in worker log in up to 30s (credential refresh period)
> {code}
>  
> *BUGS*
>  
> *BUG1* When new credentials got uploaded, Worker will try to update 
> credentials. But while it does it, it will also try to replace WorkerToken if 
> it changes. But it has a bug in the code:
> [https://github.com/apache/storm/blob/master/storm-client/src/jvm/org/apache/storm/security/auth/ClientAuthUtils.java#L411-L416]
>  
> Here in the code, "token" could equal to "previous" if tokens didn't change 
> because WorkerToken.equals() method only cares about the content of 
> WorkerToken. The result of this function is the tokens got removed completely.
> So in this case, because tokens are not present, Worker will fall back to use 
> kerberos to connect to Nimbus/Supervisor. 
> [https://github.com/apache/storm/blob/master/storm-client/src/jvm/org/apache/storm/security/auth/kerberos/KerberosSaslTransportPlugin.java#L122-L139]
> And here comes the second bug
> *BUG2*. Kerberos connection from Worker to Nimbus/Supervisor is not working 
> properly, hence the error logs above. 



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to