You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-issues@hadoop.apache.org by "Rushabh S Shah (JIRA)" <ji...@apache.org> on 2018/05/02 12:46:00 UTC

[jira] [Commented] (HADOOP-14841) Let KMS Client retry 'No content to map' EOFExceptions

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

Rushabh S Shah commented on HADOOP-14841:
-----------------------------------------

We (Daryn and I) were able to reproduce the issue and found the root cause also.
Here is the test program that we used to reproduce.
{code:title=TestStressKMS.java|borderStyle=solid}
public class TestStressKMS extends Thread {
  public static final Log LOG = LogFactory.getLog(TestStressKMS.class);
  public void run() {
  final Path path = new Path("/EZ1/test.pig");
  final Configuration conf = new Configuration();

  try {
    FileSystem fileSystem = path.getFileSystem(conf);
    InputStream inStream = fileSystem.open(path)
    inStream.read();
  } catch (Throwable ioe) {
    System.out.println(ioe);
  }
}


  public static void main(String[] args) {
    int numThreads = Integer.parseInt(args[0]);
    Thread[] threads = new Thread[numThreads];
    for(int i = 0; i < numThreads; i++) {
      threads[i] = new TestStressKMS();
      threads[i].setName("TestStressKMS " + i);
      System.out.println(threads[i].getName());
      threads[i].start();
    }

    try {
      for(int i = 0; i < numThreads; i++) {
        threads[i].join();
      }
    } catch (InterruptedException ie) {
    }
  }
}
{code}
We ran the above progeam with 128 threads and multiple such processes from different nodes.

+What we saw on server:+
After few minutes, the server logged following exception.
{noformat}
2018-04-27 03:25:26,321 WARN KMSExceptionsProvider - User:'hadoopqa (auth:KERBEROS)' Method:POST URL:http://<kms-server>:4443/kms/v1/keyversion/key1%400/_eek?eek_op=decrypt Response:Internal Server Error-No content to map to Object due to end of input
java.io.EOFException: No content to map to Object due to end of input
at org.codehaus.jackson.map.ObjectMapper._initForReading(ObjectMapper.java:2775)
at org.codehaus.jackson.map.ObjectMapper._readMapAndClose(ObjectMapper.java:2718)
at org.codehaus.jackson.map.ObjectMapper.readValue(ObjectMapper.java:1863)
at org.apache.hadoop.crypto.key.kms.server.KMSJSONReader.readFrom(KMSJSONReader.java:57)
at org.apache.hadoop.crypto.key.kms.server.KMSJSONReader.readFrom(KMSJSONReader.java:35)
at com.sun.jersey.spi.container.ContainerRequest.getEntity(ContainerRequest.java:474)
at com.sun.jersey.server.impl.model.method.dispatch.EntityParamDispatchProvider$EntityInjectable.getValue(EntityParamDispatchProvider.java:123)
at com.sun.jersey.server.impl.inject.InjectableValuesProvider.getInjectableValues(InjectableValuesProvider.java:46)
at com.sun.jersey.server.impl.model.method.dispatch.AbstractResourceMethodDispatchProvider$EntityParamInInvoker.getParams(AbstractResourceMethodDi
spatchProvider.java:153)
at com.sun.jersey.server.impl.model.method.dispatch.AbstractResourceMethodDispatchProvider$ResponseOutInvoker._dispatch(AbstractResourceMethodDisp
atchProvider.java:203)
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:790)
at org.eclipse.jetty.servlet.ServletHolder.handle(ServletHolder.java:845)
at org.eclipse.jetty.servlet.ServletHandler$CachedChain.doFilter(ServletHandler.java:1772)
at org.eclipse.jetty.websocket.server.WebSocketUpgradeFilter.doFilter(WebSocketUpgradeFilter.java:193)
at org.eclipse.jetty.servlet.ServletHandler$CachedChain.doFilter(ServletHandler.java:1759)
at org.apache.hadoop.crypto.key.kms.server.KMSMDCFilter.doFilter(KMSMDCFilter.java:84)
at org.eclipse.jetty.servlet.ServletHandler$CachedChain.doFilter(ServletHandler.java:1759)
at org.apache.hadoop.security.authentication.server.AuthenticationFilter.doFilter(AuthenticationFilter.java:636)
at org.apache.hadoop.security.token.delegation.web.DelegationTokenAuthenticationFilter.doFilter(DelegationTokenAuthenticationFilter.java:294)
at org.apache.hadoop.security.authentication.server.AuthenticationFilter.doFilter(AuthenticationFilter.java:588)
at org.apache.hadoop.crypto.key.kms.server.KMSAuthenticationFilter.doFilter(KMSAuthenticationFilter.java:130)
at org.eclipse.jetty.servlet.ServletHandler$CachedChain.doFilter(ServletHandler.java:1751)
at yjava.servlet.filter.YHdrsFilter.doFilter(YHdrsFilter.java:73)
at org.eclipse.jetty.servlet.ServletHandler$CachedChain.doFilter(ServletHandler.java:1759)
at yjava.cookie.CookieDataFilter.doFilter(CookieDataFilter.java:109)
at org.eclipse.jetty.servlet.ServletHandler$CachedChain.doFilter(ServletHandler.java:1759)
at yjava.servlet.filter.DoNotTrackFilter.doFilter(DoNotTrackFilter.java:118)
at org.eclipse.jetty.servlet.ServletHandler$CachedChain.doFilter(ServletHandler.java:1759)
at yjava.remote.ip.RemoteIPFilter.doFilter(RemoteIPFilter.java:179)
at org.eclipse.jetty.servlet.ServletHandler$CachedChain.doFilter(ServletHandler.java:1759)
at yjava.security.yiv.servlet.InputValidationFilter.doFilter(InputValidationFilter.java:225)
at org.eclipse.jetty.servlet.ServletHandler$CachedChain.doFilter(ServletHandler.java:1759)
at org.eclipse.jetty.servlet.ServletHandler.doHandle(ServletHandler.java:582)
at org.eclipse.jetty.server.handler.ScopedHandler.handle(ScopedHandler.java:143)
at org.eclipse.jetty.security.SecurityHandler.handle(SecurityHandler.java:548)
at org.eclipse.jetty.server.session.SessionHandler.doHandle(SessionHandler.java:226)
at org.eclipse.jetty.server.handler.ContextHandler.doHandle(ContextHandler.java:1180)
at org.eclipse.jetty.servlet.ServletHandler.doScope(ServletHandler.java:512)
at org.eclipse.jetty.server.session.SessionHandler.doScope(SessionHandler.java:185)
at org.eclipse.jetty.server.handler.ContextHandler.doScope(ContextHandler.java:1112)
at org.eclipse.jetty.server.handler.ScopedHandler.handle(ScopedHandler.java:141)
at org.eclipse.jetty.server.handler.ContextHandlerCollection.handle(ContextHandlerCollection.java:213)
at org.eclipse.jetty.server.handler.HandlerCollection.handle(HandlerCollection.java:119)
at org.eclipse.jetty.server.handler.gzip.GzipHandler.handle(GzipHandler.java:395)
at org.eclipse.jetty.server.handler.StatisticsHandler.handle(StatisticsHandler.java:169)
at org.eclipse.jetty.server.handler.HandlerWrapper.handle(HandlerWrapper.java:134)
at org.eclipse.jetty.server.Server.handle(Server.java:534)
at org.eclipse.jetty.server.HttpChannel.handle(HttpChannel.java:320)
at org.eclipse.jetty.server.HttpConnection.onFillable(HttpConnection.java:251)
at org.eclipse.jetty.io.AbstractConnection$ReadCallback.succeeded(AbstractConnection.java:283)
at org.eclipse.jetty.io.FillInterest.fillable(FillInterest.java:110)
at org.eclipse.jetty.io.SelectChannelEndPoint$2.run(SelectChannelEndPoint.java:93)
at org.eclipse.jetty.util.thread.QueuedThreadPool.runJob(QueuedThreadPool.java:671)
at org.eclipse.jetty.util.thread.QueuedThreadPool$2.run(QueuedThreadPool.java:589)
at java.lang.Thread.run(Thread.java:748)
{noformat}
We added debug logs to see what was the payload for such bad request and we found that the payload was empty.
Then we dumped the network traffic to see what the request looked like for such bad request.
We found that client was not sending {{Content-Length}} in the request header.
Sample request header for one bad request.
{noformat}
FINE: sun.net.www.MessageHeader@6756ff7710 pairs: {POST /kms/v1/keyversion/key1%400/_eek?eek_op=decrypt&uuid=9c5d8593-87dd-41ff-882e-10e33a0759b2 HTTP/1.1: null}{Content-Type: application/json}{tName: TestStressKMS 25}{Cache-Control: no-cache}{Pragma: no-cache}{User-Agent: Java/1.8.0_60}{Host: <kms-server>:4443}{Accept: text/html, image/gif, image/jpeg, *; q=.2, */*; q=.2}{Connection: keep-alive}{Cookie: hadoop.auth="u=hadoopqa&p=hadoopqa@<realm>&t=kerberos-dt&e=??"}
{noformat}
We added one new header {{tName}} and one param {{uuid}} to querystring for debugging purposes.

+What we saw on client+
Since we were blasting kms with muli-threaded custom program and with multiple such processes, after a minute or so client ran out of ephemeral ports and were not able to create any sockets.
The client logged the following exception repeatedly.
{noformat}
18/04/27 20:42:26 WARN kms.LoadBalancingKMSClientProvider: KMS provider at <kms-host> threw an IOException: 
java.net.ConnectException: Cannot assign requested address
at java.net.PlainSocketImpl.socketConnect(Native Method)
at java.net.AbstractPlainSocketImpl.doConnect(AbstractPlainSocketImpl.java:350)
at java.net.AbstractPlainSocketImpl.connectToAddress(AbstractPlainSocketImpl.java:206)
at java.net.AbstractPlainSocketImpl.connect(AbstractPlainSocketImpl.java:188)
at java.net.SocksSocketImpl.connect(SocksSocketImpl.java:392)
at java.net.Socket.connect(Socket.java:589)
at sun.net.NetworkClient.doConnect(NetworkClient.java:175)
at sun.net.www.http.HttpClient.openServer(HttpClient.java:432)
at sun.net.www.http.HttpClient.openServer(HttpClient.java:527)
at sun.net.www.http.HttpClient.<init>(HttpClient.java:211)
at sun.net.www.http.HttpClient.New(HttpClient.java:308)
at sun.net.www.http.HttpClient.New(HttpClient.java:326)
at sun.net.www.protocol.http.HttpURLConnection.getNewHttpClient(HttpURLConnection.java:1168)
at sun.net.www.protocol.http.HttpURLConnection.plainConnect0(HttpURLConnection.java:1104)
at sun.net.www.protocol.http.HttpURLConnection.plainConnect(HttpURLConnection.java:998)
at sun.net.www.protocol.http.HttpURLConnection.connect(HttpURLConnection.java:932)
at sun.net.www.protocol.http.HttpURLConnection.getInputStream0(HttpURLConnection.java:1512)
at sun.net.www.protocol.http.HttpURLConnection.getInputStream(HttpURLConnection.java:1440)
at org.apache.hadoop.crypto.key.kms.KMSClientProvider.call(KMSClientProvider.java:528) // The corresponding code: IOUtils.closeStream(conn.getInputStream()); 
at org.apache.hadoop.crypto.key.kms.KMSClientProvider.call(KMSClientProvider.java:516)
at org.apache.hadoop.crypto.key.kms.KMSClientProvider.decryptEncryptedKey(KMSClientProvider.java:784)
at org.apache.hadoop.crypto.key.kms.LoadBalancingKMSClientProvider$5.call(LoadBalancingKMSClientProvider.java:279)
at org.apache.hadoop.crypto.key.kms.LoadBalancingKMSClientProvider$5.call(LoadBalancingKMSClientProvider.java:275)
at org.apache.hadoop.crypto.key.kms.LoadBalancingKMSClientProvider.doOp(LoadBalancingKMSClientProvider.java:123)
at org.apache.hadoop.crypto.key.kms.LoadBalancingKMSClientProvider.decryptEncryptedKey(LoadBalancingKMSClientProvider.java:275)
at org.apache.hadoop.crypto.key.KeyProviderCryptoExtension.decryptEncryptedKey(KeyProviderCryptoExtension.java:388)
at org.apache.hadoop.hdfs.HdfsKMSUtil.decryptEncryptedDataEncryptionKey(HdfsKMSUtil.java:226)
at org.apache.hadoop.hdfs.HdfsKMSUtil.createWrappedInputStream(HdfsKMSUtil.java:206)
at org.apache.hadoop.hdfs.DFSClient.createWrappedInputStream(DFSClient.java:989)
at org.apache.hadoop.hdfs.DistributedFileSystem$4.doCall(DistributedFileSystem.java:330)
at org.apache.hadoop.hdfs.DistributedFileSystem$4.doCall(DistributedFileSystem.java:325)
at org.apache.hadoop.fs.FileSystemLinkResolver.resolve(FileSystemLinkResolver.java:81)
at org.apache.hadoop.hdfs.DistributedFileSystem.open(DistributedFileSystem.java:337)
at org.apache.hadoop.fs.FileSystem.open(FileSystem.java:796)
at TestStressKMS.run(TestStressKMS.java:37)
{noformat}

Below is the relevant piece of code.
{code:title=KMSClientProvider.java|borderStyle=solid}

private <T> T call(HttpURLConnection conn, Map jsonOutput, int expectedResponse, Class<T> klass, int authRetryCount)
throws IOException {
  T ret = null;
  try {
    if (jsonOutput != null) {
      writeJson(jsonOutput, conn.getOutputStream());
    }
  } catch (IOException ex) {
    IOUtils.closeStream(conn.getInputStream()); // This is line#528 in the below stack trace.
    throw ex;
  }
...
...
{code}
It was throwing exception in the catch block when it is trying to close the connection's {{inputStream}}.
This is because it never created the connection and when we try to retrieve inputStream from _not-yet-open_ connection, it will try to create connection.
Since there were no available socket, the catch block blew up and masked the original failure.

But for some connections, it *was able* to create connection in the catch block while closing the {{inputStream}} and it just sent the request header to server without json payload.
And since server didn't receive payload, {{ObjectMapper}} was unable to read any payload and blew up.

So the root cause was: client failed to get the OutputStream from connection object to write json payload and the catch block masked the original failure.

> Let KMS Client retry 'No content to map' EOFExceptions
> ------------------------------------------------------
>
>                 Key: HADOOP-14841
>                 URL: https://issues.apache.org/jira/browse/HADOOP-14841
>             Project: Hadoop Common
>          Issue Type: Improvement
>          Components: kms
>    Affects Versions: 2.6.0
>            Reporter: Xiao Chen
>            Assignee: Xiao Chen
>            Priority: Major
>         Attachments: HADOOP-14841.01.patch, HADOOP-14841.02.patch
>
>
> We have seen quite some occurrences when the KMS server is stressed, some of the requests would end up getting a 500 return code, with this in the server log:
> {noformat}
> 2017-08-31 06:45:33,021 WARN org.apache.hadoop.crypto.key.kms.server.KMS: User impala/HOSTNAME@REALM (auth:KERBEROS) request POST https://HOSTNAME:16000/kms/v1/keyversion/MNHDKEdWtZWM4vPb0p2bw544vdSRB2gy7APAQURcZns/_eek?eek_op=decrypt caused exception.
> java.io.EOFException: No content to map to Object due to end of input
>         at org.codehaus.jackson.map.ObjectMapper._initForReading(ObjectMapper.java:2444)
>         at org.codehaus.jackson.map.ObjectMapper._readMapAndClose(ObjectMapper.java:2396)
>         at org.codehaus.jackson.map.ObjectMapper.readValue(ObjectMapper.java:1648)
>         at org.apache.hadoop.crypto.key.kms.server.KMSJSONReader.readFrom(KMSJSONReader.java:54)
>         at com.sun.jersey.spi.container.ContainerRequest.getEntity(ContainerRequest.java:474)
>         at com.sun.jersey.server.impl.model.method.dispatch.EntityParamDispatchProvider$EntityInjectable.getValue(EntityParamDispatchProvider.java:123)
>         at com.sun.jersey.server.impl.inject.InjectableValuesProvider.getInjectableValues(InjectableValuesProvider.java:46)
>         at com.sun.jersey.server.impl.model.method.dispatch.AbstractResourceMethodDispatchProvider$EntityParamInInvoker.getParams(AbstractResourceMethodDispatchProvider.java:153)
>         at com.sun.jersey.server.impl.model.method.dispatch.AbstractResourceMethodDispatchProvider$ResponseOutInvoker._dispatch(AbstractResourceMethodDispatchProvider.java:203)
>         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 org.apache.catalina.valves.ErrorReportValve.invoke(ErrorReportValve.java:103)
>         at org.apache.catalina.core.StandardEngineValve.invoke(StandardEngineValve.java:109)
>         at org.apache.catalina.connector.CoyoteAdapter.service(CoyoteAdapter.java:293)
>         at org.apache.coyote.http11.Http11Processor.process(Http11Processor.java:859)
>         at org.apache.coyote.http11.Http11Protocol$Http11ConnectionHandler.process(Http11Protocol.java:610)
>         at org.apache.tomcat.util.net.JIoEndpoint$Worker.run(JIoEndpoint.java:503)
>         at java.lang.Thread.run(Thread.java:745)
> {noformat}



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

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