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

Antony Jay commented on HDFS-13603:
-----------------------------------

Exception stack trace

 

[2018-05-23 14:20:44,952] WARN [http-16000-11] (server.KMS) -hdfs- User 
hdfs/node0032.hostname.com@domain (auth:KERBEROS) request GET 
https://node0038.hostname.com:16000/kms/v1/key/encrypt_keyname/_eek?num_keys=150&eek_op=generate
 caused exception.
java.io.IOException: 
com.google.common.util.concurrent.UncheckedExecutionException: 
java.lang.NullPointerException: No KeyVersion exists for key 'encrypt_keyname'
 at 
org.apache.hadoop.crypto.key.kms.server.KMS.generateEncryptedKeys(KMS.java:517)
 at sun.reflect.GeneratedMethodAccessor45.invoke(Unknown Source)
 at 
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
 at java.lang.reflect.Method.invoke(Method.java:498)
 at 
com.sun.jersey.spi.container.JavaMethodInvokerFactory$1.invoke(JavaMethodInvokerFactory.java:60)
 at 
com.sun.jersey.server.impl.model.method.dispatch.AbstractResourceMethodDispatchProvider$ResponseOutInvoker._dispatch(AbstractResourceMethodDispatchProvider.java:205)
 at 
com.sun.jersey.server.impl.model.method.dispatch.ResourceJavaMethodDispatcher.dispatch(ResourceJavaMethodDispatcher.java:75)
 at 
com.sun.jersey.server.impl.uri.rules.HttpMethodRule.accept(HttpMethodRule.java:288)
 at 
com.sun.jersey.server.impl.uri.rules.RightHandPathRule.accept(RightHandPathRule.java:147)
 at 
com.sun.jersey.server.impl.uri.rules.ResourceClassRule.accept(ResourceClassRule.java:108)
 at 
com.sun.jersey.server.impl.uri.rules.RightHandPathRule.accept(RightHandPathRule.java:147)
 at 
com.sun.jersey.server.impl.uri.rules.RootResourceClassesRule.accept(RootResourceClassesRule.java:84)
 at 
com.sun.jersey.server.impl.application.WebApplicationImpl._handleRequest(WebApplicationImpl.java:1469)
 at 
com.sun.jersey.server.impl.application.WebApplicationImpl._handleRequest(WebApplicationImpl.java:1400)
 at 
com.sun.jersey.server.impl.application.WebApplicationImpl.handleRequest(WebApplicationImpl.java:1349)
 at 
com.sun.jersey.server.impl.application.WebApplicationImpl.handleRequest(WebApplicationImpl.java:1339)
 at 
com.sun.jersey.spi.container.servlet.WebComponent.service(WebComponent.java:416)
 at 
com.sun.jersey.spi.container.servlet.ServletContainer.service(ServletContainer.java:537)
 at 
com.sun.jersey.spi.container.servlet.ServletContainer.service(ServletContainer.java:699)
 at javax.servlet.http.HttpServlet.service(HttpServlet.java:723)
 at 
org.apache.catalina.core.ApplicationFilterChain.internalDoFilter(ApplicationFilterChain.java:290)
 at 
org.apache.catalina.core.ApplicationFilterChain.doFilter(ApplicationFilterChain.java:206)
 at 
org.apache.hadoop.crypto.key.kms.server.KMSMDCFilter.doFilter(KMSMDCFilter.java:84)
 at 
org.apache.catalina.core.ApplicationFilterChain.internalDoFilter(ApplicationFilterChain.java:235)
 at 
org.apache.catalina.core.ApplicationFilterChain.doFilter(ApplicationFilterChain.java:206)
 at 
org.apache.hadoop.security.authentication.server.AuthenticationFilter.doFilter(AuthenticationFilter.java:631)
 at 
org.apache.hadoop.security.token.delegation.web.DelegationTokenAuthenticationFilter.doFilter(DelegationTokenAuthenticationFilter.java:301)
 at 
org.apache.hadoop.security.authentication.server.AuthenticationFilter.doFilter(AuthenticationFilter.java:579)
 at 
org.apache.hadoop.crypto.key.kms.server.KMSAuthenticationFilter.doFilter(KMSAuthenticationFilter.java:130)
 at 
org.apache.catalina.core.ApplicationFilterChain.internalDoFilter(ApplicationFilterChain.java:235)
 at 
org.apache.catalina.core.ApplicationFilterChain.doFilter(ApplicationFilterChain.java:206)
 at 
org.apache.catalina.core.StandardWrapperValve.invoke(StandardWrapperValve.java:233)
 at 
org.apache.catalina.core.StandardContextValve.invoke(StandardContextValve.java:191)
 at 
org.apache.catalina.core.StandardHostValve.invoke(StandardHostValve.java:127)
 at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
 at java.lang.Thread.run(Thread.java:748)
Caused by: java.util.concurrent.ExecutionException: java.io.IOException: 
com.google.common.util.concurrent.UncheckedExecutionException: 
java.lang.NullPointerException: No KeyVersion exists for key 'encrypt_keyname'
 at 
com.google.common.util.concurrent.AbstractFuture$Sync.getValue(AbstractFuture.java:289)
 at 
com.google.common.util.concurrent.AbstractFuture$Sync.get(AbstractFuture.java:276)
 at 
com.google.common.util.concurrent.AbstractFuture.get(AbstractFuture.java:111)
 at 
com.google.common.util.concurrent.Uninterruptibles.getUninterruptibly(Uninterruptibles.java:132)
 at 
com.google.common.cache.LocalCache$Segment.getAndRecordStats(LocalCache.java:2381)
 at com.google.common.cache.LocalCache$Segment.loadSync(LocalCache.java:2351)
 at 
com.google.common.cache.LocalCache$Segment.lockedGetOrLoad(LocalCache.java:2313)
 at com.google.common.cache.LocalCache$Segment.get(LocalCache.java:2228)
 at com.google.common.cache.LocalCache.get(LocalCache.java:3965)
 at com.google.common.cache.LocalCache.getOrLoad(LocalCache.java:3969)
 at 
com.google.common.cache.LocalCache$LocalManualCache.get(LocalCache.java:4829)
 at 
org.apache.hadoop.crypto.key.kms.ValueQueue.initializeQueuesForKeys(ValueQueue.java:274)
 at 
org.apache.hadoop.crypto.key.kms.KMSClientProvider.warmUpEncryptedKeys(KMSClientProvider.java:886)
 ... 5 more
Caused by: java.io.IOException: 
com.google.common.util.concurrent.UncheckedExecutionException: 
java.lang.NullPointerException: No KeyVersion exists for key 'encrypt_keyname'
 at sun.reflect.GeneratedConstructorAccessor81.newInstance(Unknown Source)
 at 
sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45)
 at java.lang.reflect.Constructor.newInstance(Constructor.java:423)
 at 
org.apache.hadoop.util.HttpExceptionUtils.validateResponse(HttpExceptionUtils.java:157)
 at 
org.apache.hadoop.crypto.key.kms.KMSClientProvider.call(KMSClientProvider.java:527)
 at 
org.apache.hadoop.crypto.key.kms.KMSClientProvider.call(KMSClientProvider.java:488)
 at 
org.apache.hadoop.crypto.key.kms.KMSClientProvider.access$200(KMSClientProvider.java:94)
 at 
org.apache.hadoop.crypto.key.kms.KMSClientProvider$EncryptedQueueRefiller.fillQueueForKey(KMSClientProvider.java:149)
 at org.apache.hadoop.crypto.key.kms.ValueQueue$1.load(ValueQueue.java:246)
 at org.apache.hadoop.crypto.key.kms.ValueQueue$1.load(ValueQueue.java:240)
 at 
com.google.common.cache.LocalCache$LoadingValueReference.loadFuture(LocalCache.java:3568)
 at com.google.common.cache.LocalCache$Segment.loadSync(LocalCache.java:2350)
 ... 12 more

> Warmup NameNode EDEK thread retries continuously if there's an invalid key 
> ---------------------------------------------------------------------------
>
>                 Key: HDFS-13603
>                 URL: https://issues.apache.org/jira/browse/HDFS-13603
>             Project: Hadoop HDFS
>          Issue Type: Bug
>          Components: encryption, namenode
>    Affects Versions: 2.8.0
>            Reporter: Antony Jay
>            Priority: Major
>
> https://issues.apache.org/jira/browse/HDFS-9405 adds a background thread to 
> pre-warm EDEK cache. 
> However this fails and retries continuously if key retrieval fails for one 
> encryption zone. In our usecase, we have temporarily removed keys for certain 
> encryption zones.  Currently namenode and kms log is filled up with errors 
> related to background thread retrying warmup for ever .
> The pre-warm thread should
>  * Continue to refresh other encryption zones even if it fails for one
>  * Should retry only if it fails for all encryption zones, which will be the 
> case when kms is down.
>  



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

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

Reply via email to