You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@trafficserver.apache.org by su...@apache.org on 2014/09/24 15:40:02 UTC

git commit: [TS-3085] Large POSTs over (relatively) slower connections failing in ats5 Call ERR_get_error_line_data() via SSL_CLR_ERR_INCR_DYN_STAT to clean up the error queue after processing an SSL_ERROR_SSL. Also, added consistent wrappers to SSL I/O

Repository: trafficserver
Updated Branches:
  refs/heads/master 17bef772a -> d12327d84


[TS-3085] Large POSTs over (relatively) slower connections failing in ats5
Call ERR_get_error_line_data() via SSL_CLR_ERR_INCR_DYN_STAT to clean up
the error queue after processing an SSL_ERROR_SSL. Also, added consistent
wrappers to SSL I/O functions and removed errno based looping on SSL_Write
per James Peach's recommendations.  Note that, this change removes the apparent
logic of reattempting SSL_Write on transient error cases (such as ENOBUF), since
openSSL documentation doesn't indicate that errno is set during SSL_Write.
https://www.openssl.org/docs/ssl/SSL_write.html


Project: http://git-wip-us.apache.org/repos/asf/trafficserver/repo
Commit: http://git-wip-us.apache.org/repos/asf/trafficserver/commit/d12327d8
Tree: http://git-wip-us.apache.org/repos/asf/trafficserver/tree/d12327d8
Diff: http://git-wip-us.apache.org/repos/asf/trafficserver/diff/d12327d8

Branch: refs/heads/master
Commit: d12327d841d07bfe1fa772c42272fbc908a39324
Parents: 17bef77
Author: Sudheer Vinukonda <su...@yahoo-inc.com>
Authored: Wed Sep 24 13:32:56 2014 +0000
Committer: Sudheer Vinukonda <su...@yahoo-inc.com>
Committed: Wed Sep 24 13:32:56 2014 +0000

----------------------------------------------------------------------
 iocore/net/P_SSLUtils.h         | 12 +++++++
 iocore/net/SSLNetVConnection.cc | 69 ++++++++++++------------------------
 iocore/net/SSLUtils.cc          | 59 ++++++++++++++++++++++++++++++
 3 files changed, 93 insertions(+), 47 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/trafficserver/blob/d12327d8/iocore/net/P_SSLUtils.h
----------------------------------------------------------------------
diff --git a/iocore/net/P_SSLUtils.h b/iocore/net/P_SSLUtils.h
index 82c41b4..269f009 100644
--- a/iocore/net/P_SSLUtils.h
+++ b/iocore/net/P_SSLUtils.h
@@ -38,6 +38,8 @@ struct SSLCertLookup;
 class SSLNetVConnection;
 struct RecRawStatBlock;
 
+typedef int ssl_error_t;
+
 enum SSL_Stats
 {
   ssl_origin_server_expired_cert_stat,
@@ -113,6 +115,12 @@ void SSLInitializeStatistics();
 // Release SSL_CTX and the associated data
 void SSLReleaseContext(SSL_CTX* ctx);
 
+// Wrapper functions to SSL I/O routines
+ssl_error_t SSLWriteBuffer(SSL * ssl, const void * buf, size_t nbytes, size_t& nwritten);
+ssl_error_t SSLReadBuffer(SSL * ssl, void * buf, size_t nbytes, size_t& nread);
+ssl_error_t SSLAccept(SSL *ssl);
+ssl_error_t SSLConnect(SSL * ssl);
+
 // Log an SSL error.
 #define SSLError(fmt, ...) SSLDiagnostic(DiagsMakeLocation(), false, NULL, fmt, ##__VA_ARGS__)
 #define SSLErrorVC(vc,fmt, ...) SSLDiagnostic(DiagsMakeLocation(), false, vc, fmt, ##__VA_ARGS__)
@@ -120,6 +128,10 @@ void SSLReleaseContext(SSL_CTX* ctx);
 #define SSLDebug(fmt, ...) SSLDiagnostic(DiagsMakeLocation(), true, NULL, fmt, ##__VA_ARGS__)
 #define SSLDebugVC(vc,fmt, ...) SSLDiagnostic(DiagsMakeLocation(), true, vc, fmt, ##__VA_ARGS__)
 
+#define SSL_CLR_ERR_INCR_DYN_STAT(x, fmt, ...) \
+  SSLDiagnostic(DiagsMakeLocation(), true, NULL, fmt, ##__VA_ARGS__); \
+  RecIncrRawStat(ssl_rsb, NULL, (int) x, 1);
+
 void SSLDiagnostic(const SrcLoc& loc, bool debug, SSLNetVConnection * vc, const char * fmt, ...) TS_PRINTFLIKE(4, 5);
 
 // Return a static string name for a SSL_ERROR constant.

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/d12327d8/iocore/net/SSLNetVConnection.cc
----------------------------------------------------------------------
diff --git a/iocore/net/SSLNetVConnection.cc b/iocore/net/SSLNetVConnection.cc
index b4269e7..0f26679 100644
--- a/iocore/net/SSLNetVConnection.cc
+++ b/iocore/net/SSLNetVConnection.cc
@@ -172,24 +172,6 @@ debug_certificate_name(const char * msg, X509_NAME * name)
   BIO_free(bio);
 }
 
-static inline int
-do_SSL_write(SSL * ssl, void *buf, int size)
-{
-  int r = 0;
-  do {
-    // need to check into SSL error handling
-    // to see if this is good enough.
-    r = SSL_write(ssl, (const char *) buf, size);
-    if (r >= 0)
-      return r;
-    else
-      r = -errno;
-  } while (r == -EINTR || r == -ENOBUFS || r == -ENOMEM);
-
-  return r;
-}
-
-
 static int
 ssl_read_from_net(SSLNetVConnection * sslvc, EThread * lthread, int64_t &ret)
 {
@@ -199,7 +181,8 @@ ssl_read_from_net(SSLNetVConnection * sslvc, EThread * lthread, int64_t &ret)
   int event = SSL_READ_ERROR_NONE;
   int64_t bytes_read;
   int64_t block_write_avail;
-  int sslErr = SSL_ERROR_NONE;
+  ssl_error_t sslErr = SSL_ERROR_NONE;
+  int nread = 0;
 
   for (bytes_read = 0; (b != 0) && (sslErr == SSL_ERROR_NONE); b = b->next) {
     block_write_avail = b->write_avail();
@@ -209,23 +192,22 @@ ssl_read_from_net(SSLNetVConnection * sslvc, EThread * lthread, int64_t &ret)
     int64_t offset = 0;
     // while can be replaced with if - need to test what works faster with openssl
     while (block_write_avail > 0) {
-      int rres = SSL_read(sslvc->ssl, b->end() + offset, (int)block_write_avail);
+      sslErr = SSLReadBuffer (sslvc->ssl, b->end() + offset, (size_t)block_write_avail, (size_t&)nread);
 
-      Debug("ssl", "[SSL_NetVConnection::ssl_read_from_net] rres=%d", rres);
+      Debug("ssl", "[SSL_NetVConnection::ssl_read_from_net] nread=%d", (int)nread);
 
-      sslErr = SSL_get_error(sslvc->ssl, rres);
       switch (sslErr) {
       case SSL_ERROR_NONE:
 
 #if DEBUG
-        SSLDebugBufferPrint("ssl_buff", b->end() + offset, rres, "SSL Read");
+        SSLDebugBufferPrint("ssl_buff", b->end() + offset, nread, "SSL Read");
 #endif
 
-        ink_assert(rres);
+        ink_assert(nread);
 
-        bytes_read += rres;
-        offset += rres;
-        block_write_avail -= rres;
+        bytes_read += nread;
+        offset += nread;
+        block_write_avail -= nread;
         ink_assert(block_write_avail >= 0);
 
         continue;
@@ -247,7 +229,7 @@ ssl_read_from_net(SSLNetVConnection * sslvc, EThread * lthread, int64_t &ret)
         break;
       case SSL_ERROR_SYSCALL:
         SSL_INCREMENT_DYN_STAT(ssl_error_syscall);
-        if (rres != 0) {
+        if (nread != 0) {
           // not EOF
           event = SSL_READ_ERROR;
           ret = errno;
@@ -267,8 +249,7 @@ ssl_read_from_net(SSLNetVConnection * sslvc, EThread * lthread, int64_t &ret)
       default:
         event = SSL_READ_ERROR;
         ret = errno;
-        SSL_INCREMENT_DYN_STAT(ssl_error_ssl);
-        Debug("ssl.error", "[SSL_NetVConnection::ssl_read_from_net]");
+        SSL_CLR_ERR_INCR_DYN_STAT(ssl_error_ssl, "[SSL_NetVConnection::ssl_read_from_net]: errno=%d", errno);
         break;
       }                         // switch
       break;
@@ -636,6 +617,7 @@ SSLNetVConnection::load_buffer_and_write(int64_t towrite, int64_t &wattempted, i
   ProxyMutex *mutex = this_ethread()->mutex;
   int64_t r = 0;
   int64_t l = 0;
+  ssl_error_t err = SSL_ERROR_NONE;
 
   // XXX Rather than dealing with the block directly, we should use the IOBufferReader API.
   int64_t offset = buf.reader()->start_offset;
@@ -675,9 +657,9 @@ SSLNetVConnection::load_buffer_and_write(int64_t towrite, int64_t &wattempted, i
 
     wattempted = l;
     total_wrote += l;
-    Debug("ssl", "SSLNetVConnection::loadBufferAndCallWrite, before do_SSL_write, l=%" PRId64", towrite=%" PRId64", b=%p",
+    Debug("ssl", "SSLNetVConnection::loadBufferAndCallWrite, before SSLWriteBuffer, l=%" PRId64", towrite=%" PRId64", b=%p",
           l, towrite, b);
-    r = do_SSL_write(ssl, b->start() + offset, (int)l);
+    err = SSLWriteBuffer(ssl, b->start() + offset, (size_t)l, (size_t&)r);
     if (r == l) {
       wattempted = total_wrote;
     }
@@ -704,8 +686,6 @@ SSLNetVConnection::load_buffer_and_write(int64_t towrite, int64_t &wattempted, i
       return (total_wrote);
     }
   } else {
-    int err = SSL_get_error(ssl, (int)r);
-
     switch (err) {
     case SSL_ERROR_NONE:
       Debug("ssl", "SSL_write-SSL_ERROR_NONE");
@@ -742,8 +722,7 @@ SSLNetVConnection::load_buffer_and_write(int64_t towrite, int64_t &wattempted, i
     case SSL_ERROR_SSL:
     default:
       r = -errno;
-      SSL_INCREMENT_DYN_STAT(ssl_error_ssl);
-      Debug("ssl.error", "SSL_write-SSL_ERROR_SSL");
+      SSL_CLR_ERR_INCR_DYN_STAT(ssl_error_ssl, "SSL_write-SSL_ERROR_SSL errno=%d", errno);
       break;
     }
     return (r);
@@ -880,8 +859,6 @@ SSLNetVConnection::sslStartHandShake(int event, int &err)
 int
 SSLNetVConnection::sslServerHandShakeEvent(int &err)
 {
-  int ret;
-
   if (SSL_HOOKS_DONE != sslPreAcceptHookState) {
     // Get the first hook if we haven't started invoking yet.
     if (SSL_HOOKS_INIT == sslPreAcceptHookState) {
@@ -939,8 +916,7 @@ SSLNetVConnection::sslServerHandShakeEvent(int &err)
     }
   }
 
-  ret = SSL_accept(ssl);
-  int ssl_error = SSL_get_error(ssl, ret);
+  ssl_error_t ssl_error = SSLAccept(ssl);
 
   if (ssl_error != SSL_ERROR_NONE) {
     err = errno;
@@ -1039,9 +1015,11 @@ SSLNetVConnection::sslServerHandShakeEvent(int &err)
   case SSL_ERROR_WANT_X509_LOOKUP:
     return EVENT_CONT;
 
+  case SSL_ERROR_SSL:
+    SSL_CLR_ERR_INCR_DYN_STAT(ssl_error_ssl, "SSLNetVConnection::sslServerHandShakeEvent, SSL_ERROR_SSL errno=%d", errno);
+    // fall through
   case SSL_ERROR_ZERO_RETURN:
   case SSL_ERROR_SYSCALL:
-  case SSL_ERROR_SSL:
   default:
     return EVENT_ERROR;
   }
@@ -1052,8 +1030,6 @@ SSLNetVConnection::sslServerHandShakeEvent(int &err)
 int
 SSLNetVConnection::sslClientHandShakeEvent(int &err)
 {
-  int ret;
-
 #if TS_USE_TLS_SNI
   if (options.sni_servername) {
     if (SSL_set_tlsext_host_name(ssl, options.sni_servername)) {
@@ -1065,8 +1041,8 @@ SSLNetVConnection::sslClientHandShakeEvent(int &err)
   }
 #endif
   
-  ret = SSL_connect(ssl);
-  switch (SSL_get_error(ssl, ret)) {
+  ssl_error_t ssl_error = SSLConnect(ssl);
+  switch (ssl_error) {
   case SSL_ERROR_NONE:
     if (is_debug_tag_set("ssl")) {
       X509 * cert = SSL_get_peer_certificate(ssl);
@@ -1122,8 +1098,7 @@ SSLNetVConnection::sslClientHandShakeEvent(int &err)
   case SSL_ERROR_SSL:
   default:
     err = errno;
-    SSL_INCREMENT_DYN_STAT(ssl_error_ssl);
-    Debug("ssl.error", "SSLNetVConnection::sslClientHandShakeEvent, SSL_ERROR_SSL");
+    SSL_CLR_ERR_INCR_DYN_STAT(ssl_error_ssl, "SSLNetVConnection::sslClientHandShakeEvent, SSL_ERROR_SSL errno=%d", errno);
     return EVENT_ERROR;
     break;
 

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/d12327d8/iocore/net/SSLUtils.cc
----------------------------------------------------------------------
diff --git a/iocore/net/SSLUtils.cc b/iocore/net/SSLUtils.cc
index a57ce90..59e0b63 100644
--- a/iocore/net/SSLUtils.cc
+++ b/iocore/net/SSLUtils.cc
@@ -1660,3 +1660,62 @@ SSLReleaseContext(SSL_CTX * ctx)
 {
   SSL_CTX_free(ctx);
 }
+
+
+ssl_error_t
+SSLWriteBuffer(SSL * ssl, const void * buf, size_t nbytes, size_t& nwritten)
+{
+  nwritten = 0;
+
+  if (unlikely(nbytes == 0)) {
+    return SSL_ERROR_NONE;
+  }
+
+  int ret = SSL_write(ssl, buf, (int)nbytes);
+  if (ret > 0) {
+    nwritten = ret;
+    return SSL_ERROR_NONE;
+  }
+
+  return SSL_get_error(ssl, ret);
+}
+
+ssl_error_t
+SSLReadBuffer(SSL * ssl, void * buf, size_t nbytes, size_t& nread)
+{
+  nread = 0;
+
+  if (unlikely(nbytes == 0)) {
+    return SSL_ERROR_NONE;
+  }
+
+  int ret = SSL_read(ssl, buf, (int)nbytes);
+  if (ret > 0) {
+    nread = ret;
+    return SSL_ERROR_NONE;
+  }
+
+  return SSL_get_error(ssl, ret);
+}
+
+ssl_error_t
+SSLAccept(SSL * ssl)
+{
+  int ret = SSL_accept(ssl);
+  if (ret > 0) {
+    return SSL_ERROR_NONE;
+  }
+
+  return SSL_get_error(ssl, ret);
+}
+
+ssl_error_t
+SSLConnect(SSL * ssl)
+{
+  int ret = SSL_connect(ssl);
+  if (ret > 0) {
+    return SSL_ERROR_NONE;
+  }
+
+  return SSL_get_error(ssl, ret);
+}


Re: git commit: [TS-3085] Large POSTs over (relatively) slower connections failing in ats5 Call ERR_get_error_line_data() via SSL_CLR_ERR_INCR_DYN_STAT to clean up the error queue after processing an SSL_ERROR_SSL. Also, added consistent wrappers to SSL I/O

Posted by James Peach <jp...@apache.org>.
On Sep 24, 2014, at 6:40 AM, sudheerv@apache.org wrote:

> Repository: trafficserver
> Updated Branches:
>  refs/heads/master 17bef772a -> d12327d84
> 
> 
> [TS-3085] Large POSTs over (relatively) slower connections failing in ats5
> Call ERR_get_error_line_data() via SSL_CLR_ERR_INCR_DYN_STAT to clean up
> the error queue after processing an SSL_ERROR_SSL. Also, added consistent
> wrappers to SSL I/O functions and removed errno based looping on SSL_Write
> per James Peach's recommendations.  Note that, this change removes the apparent
> logic of reattempting SSL_Write on transient error cases (such as ENOBUF), since
> openSSL documentation doesn't indicate that errno is set during SSL_Write.
> https://www.openssl.org/docs/ssl/SSL_write.html
> 
> 
> Project: http://git-wip-us.apache.org/repos/asf/trafficserver/repo
> Commit: http://git-wip-us.apache.org/repos/asf/trafficserver/commit/d12327d8
> Tree: http://git-wip-us.apache.org/repos/asf/trafficserver/tree/d12327d8
> Diff: http://git-wip-us.apache.org/repos/asf/trafficserver/diff/d12327d8
> 
> Branch: refs/heads/master
> Commit: d12327d841d07bfe1fa772c42272fbc908a39324
> Parents: 17bef77
> Author: Sudheer Vinukonda <su...@yahoo-inc.com>
> Authored: Wed Sep 24 13:32:56 2014 +0000
> Committer: Sudheer Vinukonda <su...@yahoo-inc.com>
> Committed: Wed Sep 24 13:32:56 2014 +0000
> 
> ----------------------------------------------------------------------
> iocore/net/P_SSLUtils.h         | 12 +++++++
> iocore/net/SSLNetVConnection.cc | 69 ++++++++++++------------------------
> iocore/net/SSLUtils.cc          | 59 ++++++++++++++++++++++++++++++
> 3 files changed, 93 insertions(+), 47 deletions(-)
> ----------------------------------------------------------------------
> 
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/d12327d8/iocore/net/P_SSLUtils.h
> ----------------------------------------------------------------------
> diff --git a/iocore/net/P_SSLUtils.h b/iocore/net/P_SSLUtils.h
> index 82c41b4..269f009 100644
> --- a/iocore/net/P_SSLUtils.h
> +++ b/iocore/net/P_SSLUtils.h
> @@ -38,6 +38,8 @@ struct SSLCertLookup;
> class SSLNetVConnection;
> struct RecRawStatBlock;
> 
> +typedef int ssl_error_t;
> +
> enum SSL_Stats
> {
>   ssl_origin_server_expired_cert_stat,
> @@ -113,6 +115,12 @@ void SSLInitializeStatistics();
> // Release SSL_CTX and the associated data
> void SSLReleaseContext(SSL_CTX* ctx);
> 
> +// Wrapper functions to SSL I/O routines
> +ssl_error_t SSLWriteBuffer(SSL * ssl, const void * buf, size_t nbytes, size_t& nwritten);
> +ssl_error_t SSLReadBuffer(SSL * ssl, void * buf, size_t nbytes, size_t& nread);
> +ssl_error_t SSLAccept(SSL *ssl);
> +ssl_error_t SSLConnect(SSL * ssl);
> +
> // Log an SSL error.
> #define SSLError(fmt, ...) SSLDiagnostic(DiagsMakeLocation(), false, NULL, fmt, ##__VA_ARGS__)
> #define SSLErrorVC(vc,fmt, ...) SSLDiagnostic(DiagsMakeLocation(), false, vc, fmt, ##__VA_ARGS__)
> @@ -120,6 +128,10 @@ void SSLReleaseContext(SSL_CTX* ctx);
> #define SSLDebug(fmt, ...) SSLDiagnostic(DiagsMakeLocation(), true, NULL, fmt, ##__VA_ARGS__)
> #define SSLDebugVC(vc,fmt, ...) SSLDiagnostic(DiagsMakeLocation(), true, vc, fmt, ##__VA_ARGS__)
> 
> +#define SSL_CLR_ERR_INCR_DYN_STAT(x, fmt, ...) \
> +  SSLDiagnostic(DiagsMakeLocation(), true, NULL, fmt, ##__VA_ARGS__); \
> +  RecIncrRawStat(ssl_rsb, NULL, (int) x, 1);
> +

Pleas wrap this in a do { } while(0) ...

> void SSLDiagnostic(const SrcLoc& loc, bool debug, SSLNetVConnection * vc, const char * fmt, ...) TS_PRINTFLIKE(4, 5);
> 
> // Return a static string name for a SSL_ERROR constant.
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/d12327d8/iocore/net/SSLNetVConnection.cc
> ----------------------------------------------------------------------
> diff --git a/iocore/net/SSLNetVConnection.cc b/iocore/net/SSLNetVConnection.cc
> index b4269e7..0f26679 100644
> --- a/iocore/net/SSLNetVConnection.cc
> +++ b/iocore/net/SSLNetVConnection.cc
> @@ -172,24 +172,6 @@ debug_certificate_name(const char * msg, X509_NAME * name)
>   BIO_free(bio);
> }
> 
> -static inline int
> -do_SSL_write(SSL * ssl, void *buf, int size)
> -{
> -  int r = 0;
> -  do {
> -    // need to check into SSL error handling
> -    // to see if this is good enough.
> -    r = SSL_write(ssl, (const char *) buf, size);
> -    if (r >= 0)
> -      return r;
> -    else
> -      r = -errno;
> -  } while (r == -EINTR || r == -ENOBUFS || r == -ENOMEM);
> -
> -  return r;
> -}
> -
> -
> static int
> ssl_read_from_net(SSLNetVConnection * sslvc, EThread * lthread, int64_t &ret)
> {
> @@ -199,7 +181,8 @@ ssl_read_from_net(SSLNetVConnection * sslvc, EThread * lthread, int64_t &ret)
>   int event = SSL_READ_ERROR_NONE;
>   int64_t bytes_read;
>   int64_t block_write_avail;
> -  int sslErr = SSL_ERROR_NONE;
> +  ssl_error_t sslErr = SSL_ERROR_NONE;
> +  int nread = 0;
> 
>   for (bytes_read = 0; (b != 0) && (sslErr == SSL_ERROR_NONE); b = b->next) {
>     block_write_avail = b->write_avail();
> @@ -209,23 +192,22 @@ ssl_read_from_net(SSLNetVConnection * sslvc, EThread * lthread, int64_t &ret)
>     int64_t offset = 0;
>     // while can be replaced with if - need to test what works faster with openssl
>     while (block_write_avail > 0) {
> -      int rres = SSL_read(sslvc->ssl, b->end() + offset, (int)block_write_avail);
> +      sslErr = SSLReadBuffer (sslvc->ssl, b->end() + offset, (size_t)block_write_avail, (size_t&)nread);

Don't cast this, just use size_t.

> 
> -      Debug("ssl", "[SSL_NetVConnection::ssl_read_from_net] rres=%d", rres);
> +      Debug("ssl", "[SSL_NetVConnection::ssl_read_from_net] nread=%d", (int)nread);
> 
> -      sslErr = SSL_get_error(sslvc->ssl, rres);
>       switch (sslErr) {
>       case SSL_ERROR_NONE:
> 
> #if DEBUG
> -        SSLDebugBufferPrint("ssl_buff", b->end() + offset, rres, "SSL Read");
> +        SSLDebugBufferPrint("ssl_buff", b->end() + offset, nread, "SSL Read");
> #endif
> 
> -        ink_assert(rres);
> +        ink_assert(nread);
> 
> -        bytes_read += rres;
> -        offset += rres;
> -        block_write_avail -= rres;
> +        bytes_read += nread;
> +        offset += nread;
> +        block_write_avail -= nread;
>         ink_assert(block_write_avail >= 0);
> 
>         continue;
> @@ -247,7 +229,7 @@ ssl_read_from_net(SSLNetVConnection * sslvc, EThread * lthread, int64_t &ret)
>         break;
>       case SSL_ERROR_SYSCALL:
>         SSL_INCREMENT_DYN_STAT(ssl_error_syscall);
> -        if (rres != 0) {
> +        if (nread != 0) {
>           // not EOF
>           event = SSL_READ_ERROR;
>           ret = errno;
> @@ -267,8 +249,7 @@ ssl_read_from_net(SSLNetVConnection * sslvc, EThread * lthread, int64_t &ret)
>       default:
>         event = SSL_READ_ERROR;
>         ret = errno;
> -        SSL_INCREMENT_DYN_STAT(ssl_error_ssl);
> -        Debug("ssl.error", "[SSL_NetVConnection::ssl_read_from_net]");
> +        SSL_CLR_ERR_INCR_DYN_STAT(ssl_error_ssl, "[SSL_NetVConnection::ssl_read_from_net]: errno=%d", errno);
>         break;
>       }                         // switch
>       break;
> @@ -636,6 +617,7 @@ SSLNetVConnection::load_buffer_and_write(int64_t towrite, int64_t &wattempted, i
>   ProxyMutex *mutex = this_ethread()->mutex;
>   int64_t r = 0;
>   int64_t l = 0;
> +  ssl_error_t err = SSL_ERROR_NONE;
> 
>   // XXX Rather than dealing with the block directly, we should use the IOBufferReader API.
>   int64_t offset = buf.reader()->start_offset;
> @@ -675,9 +657,9 @@ SSLNetVConnection::load_buffer_and_write(int64_t towrite, int64_t &wattempted, i
> 
>     wattempted = l;
>     total_wrote += l;
> -    Debug("ssl", "SSLNetVConnection::loadBufferAndCallWrite, before do_SSL_write, l=%" PRId64", towrite=%" PRId64", b=%p",
> +    Debug("ssl", "SSLNetVConnection::loadBufferAndCallWrite, before SSLWriteBuffer, l=%" PRId64", towrite=%" PRId64", b=%p",
>           l, towrite, b);
> -    r = do_SSL_write(ssl, b->start() + offset, (int)l);
> +    err = SSLWriteBuffer(ssl, b->start() + offset, (size_t)l, (size_t&)r);

Again, this cast is unsafe.

>     if (r == l) {
>       wattempted = total_wrote;
>     }
> @@ -704,8 +686,6 @@ SSLNetVConnection::load_buffer_and_write(int64_t towrite, int64_t &wattempted, i
>       return (total_wrote);
>     }
>   } else {
> -    int err = SSL_get_error(ssl, (int)r);
> -
>     switch (err) {
>     case SSL_ERROR_NONE:
>       Debug("ssl", "SSL_write-SSL_ERROR_NONE");
> @@ -742,8 +722,7 @@ SSLNetVConnection::load_buffer_and_write(int64_t towrite, int64_t &wattempted, i
>     case SSL_ERROR_SSL:
>     default:
>       r = -errno;
> -      SSL_INCREMENT_DYN_STAT(ssl_error_ssl);
> -      Debug("ssl.error", "SSL_write-SSL_ERROR_SSL");
> +      SSL_CLR_ERR_INCR_DYN_STAT(ssl_error_ssl, "SSL_write-SSL_ERROR_SSL errno=%d", errno);
>       break;
>     }
>     return (r);
> @@ -880,8 +859,6 @@ SSLNetVConnection::sslStartHandShake(int event, int &err)
> int
> SSLNetVConnection::sslServerHandShakeEvent(int &err)
> {
> -  int ret;
> -
>   if (SSL_HOOKS_DONE != sslPreAcceptHookState) {
>     // Get the first hook if we haven't started invoking yet.
>     if (SSL_HOOKS_INIT == sslPreAcceptHookState) {
> @@ -939,8 +916,7 @@ SSLNetVConnection::sslServerHandShakeEvent(int &err)
>     }
>   }
> 
> -  ret = SSL_accept(ssl);
> -  int ssl_error = SSL_get_error(ssl, ret);
> +  ssl_error_t ssl_error = SSLAccept(ssl);
> 
>   if (ssl_error != SSL_ERROR_NONE) {
>     err = errno;
> @@ -1039,9 +1015,11 @@ SSLNetVConnection::sslServerHandShakeEvent(int &err)
>   case SSL_ERROR_WANT_X509_LOOKUP:
>     return EVENT_CONT;
> 
> +  case SSL_ERROR_SSL:
> +    SSL_CLR_ERR_INCR_DYN_STAT(ssl_error_ssl, "SSLNetVConnection::sslServerHandShakeEvent, SSL_ERROR_SSL errno=%d", errno);
> +    // fall through
>   case SSL_ERROR_ZERO_RETURN:
>   case SSL_ERROR_SYSCALL:
> -  case SSL_ERROR_SSL:
>   default:
>     return EVENT_ERROR;
>   }
> @@ -1052,8 +1030,6 @@ SSLNetVConnection::sslServerHandShakeEvent(int &err)
> int
> SSLNetVConnection::sslClientHandShakeEvent(int &err)
> {
> -  int ret;
> -
> #if TS_USE_TLS_SNI
>   if (options.sni_servername) {
>     if (SSL_set_tlsext_host_name(ssl, options.sni_servername)) {
> @@ -1065,8 +1041,8 @@ SSLNetVConnection::sslClientHandShakeEvent(int &err)
>   }
> #endif
> 
> -  ret = SSL_connect(ssl);
> -  switch (SSL_get_error(ssl, ret)) {
> +  ssl_error_t ssl_error = SSLConnect(ssl);
> +  switch (ssl_error) {
>   case SSL_ERROR_NONE:
>     if (is_debug_tag_set("ssl")) {
>       X509 * cert = SSL_get_peer_certificate(ssl);
> @@ -1122,8 +1098,7 @@ SSLNetVConnection::sslClientHandShakeEvent(int &err)
>   case SSL_ERROR_SSL:
>   default:
>     err = errno;
> -    SSL_INCREMENT_DYN_STAT(ssl_error_ssl);
> -    Debug("ssl.error", "SSLNetVConnection::sslClientHandShakeEvent, SSL_ERROR_SSL");
> +    SSL_CLR_ERR_INCR_DYN_STAT(ssl_error_ssl, "SSLNetVConnection::sslClientHandShakeEvent, SSL_ERROR_SSL errno=%d", errno);
>     return EVENT_ERROR;
>     break;
> 
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/d12327d8/iocore/net/SSLUtils.cc
> ----------------------------------------------------------------------
> diff --git a/iocore/net/SSLUtils.cc b/iocore/net/SSLUtils.cc
> index a57ce90..59e0b63 100644
> --- a/iocore/net/SSLUtils.cc
> +++ b/iocore/net/SSLUtils.cc
> @@ -1660,3 +1660,62 @@ SSLReleaseContext(SSL_CTX * ctx)
> {
>   SSL_CTX_free(ctx);
> }
> +
> +
> +ssl_error_t
> +SSLWriteBuffer(SSL * ssl, const void * buf, size_t nbytes, size_t& nwritten)
> +{
> +  nwritten = 0;
> +
> +  if (unlikely(nbytes == 0)) {
> +    return SSL_ERROR_NONE;
> +  }
> +
> +  int ret = SSL_write(ssl, buf, (int)nbytes);
> +  if (ret > 0) {
> +    nwritten = ret;
> +    return SSL_ERROR_NONE;
> +  }
> +
> +  return SSL_get_error(ssl, ret);
> +}
> +
> +ssl_error_t
> +SSLReadBuffer(SSL * ssl, void * buf, size_t nbytes, size_t& nread)
> +{
> +  nread = 0;
> +
> +  if (unlikely(nbytes == 0)) {
> +    return SSL_ERROR_NONE;
> +  }
> +
> +  int ret = SSL_read(ssl, buf, (int)nbytes);
> +  if (ret > 0) {
> +    nread = ret;
> +    return SSL_ERROR_NONE;
> +  }
> +
> +  return SSL_get_error(ssl, ret);
> +}
> +
> +ssl_error_t
> +SSLAccept(SSL * ssl)
> +{
> +  int ret = SSL_accept(ssl);
> +  if (ret > 0) {
> +    return SSL_ERROR_NONE;
> +  }
> +
> +  return SSL_get_error(ssl, ret);
> +}
> +
> +ssl_error_t
> +SSLConnect(SSL * ssl)
> +{
> +  int ret = SSL_connect(ssl);
> +  if (ret > 0) {
> +    return SSL_ERROR_NONE;
> +  }
> +
> +  return SSL_get_error(ssl, ret);
> +}
> 


Re: git commit: [TS-3085] Large POSTs over (relatively) slower connections failing in ats5 Call ERR_get_error_line_data() via SSL_CLR_ERR_INCR_DYN_STAT to clean up the error queue after processing an SSL_ERROR_SSL. Also, added consistent wrappers to SSL I/O

Posted by James Peach <jp...@apache.org>.
On Sep 24, 2014, at 6:40 AM, sudheerv@apache.org wrote:

> Repository: trafficserver
> Updated Branches:
>  refs/heads/master 17bef772a -> d12327d84
> 
> 
> [TS-3085] Large POSTs over (relatively) slower connections failing in ats5
> Call ERR_get_error_line_data() via SSL_CLR_ERR_INCR_DYN_STAT to clean up
> the error queue after processing an SSL_ERROR_SSL. Also, added consistent
> wrappers to SSL I/O functions and removed errno based looping on SSL_Write
> per James Peach's recommendations.  Note that, this change removes the apparent
> logic of reattempting SSL_Write on transient error cases (such as ENOBUF), since
> openSSL documentation doesn't indicate that errno is set during SSL_Write.
> https://www.openssl.org/docs/ssl/SSL_write.html
> 
> 
> Project: http://git-wip-us.apache.org/repos/asf/trafficserver/repo
> Commit: http://git-wip-us.apache.org/repos/asf/trafficserver/commit/d12327d8
> Tree: http://git-wip-us.apache.org/repos/asf/trafficserver/tree/d12327d8
> Diff: http://git-wip-us.apache.org/repos/asf/trafficserver/diff/d12327d8
> 
> Branch: refs/heads/master
> Commit: d12327d841d07bfe1fa772c42272fbc908a39324
> Parents: 17bef77
> Author: Sudheer Vinukonda <su...@yahoo-inc.com>
> Authored: Wed Sep 24 13:32:56 2014 +0000
> Committer: Sudheer Vinukonda <su...@yahoo-inc.com>
> Committed: Wed Sep 24 13:32:56 2014 +0000
> 
> ----------------------------------------------------------------------
> iocore/net/P_SSLUtils.h         | 12 +++++++
> iocore/net/SSLNetVConnection.cc | 69 ++++++++++++------------------------
> iocore/net/SSLUtils.cc          | 59 ++++++++++++++++++++++++++++++
> 3 files changed, 93 insertions(+), 47 deletions(-)
> ----------------------------------------------------------------------
> 
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/d12327d8/iocore/net/P_SSLUtils.h
> ----------------------------------------------------------------------
> diff --git a/iocore/net/P_SSLUtils.h b/iocore/net/P_SSLUtils.h
> index 82c41b4..269f009 100644
> --- a/iocore/net/P_SSLUtils.h
> +++ b/iocore/net/P_SSLUtils.h
> @@ -38,6 +38,8 @@ struct SSLCertLookup;
> class SSLNetVConnection;
> struct RecRawStatBlock;
> 
> +typedef int ssl_error_t;
> +
> enum SSL_Stats
> {
>   ssl_origin_server_expired_cert_stat,
> @@ -113,6 +115,12 @@ void SSLInitializeStatistics();
> // Release SSL_CTX and the associated data
> void SSLReleaseContext(SSL_CTX* ctx);
> 
> +// Wrapper functions to SSL I/O routines
> +ssl_error_t SSLWriteBuffer(SSL * ssl, const void * buf, size_t nbytes, size_t& nwritten);
> +ssl_error_t SSLReadBuffer(SSL * ssl, void * buf, size_t nbytes, size_t& nread);
> +ssl_error_t SSLAccept(SSL *ssl);
> +ssl_error_t SSLConnect(SSL * ssl);
> +
> // Log an SSL error.
> #define SSLError(fmt, ...) SSLDiagnostic(DiagsMakeLocation(), false, NULL, fmt, ##__VA_ARGS__)
> #define SSLErrorVC(vc,fmt, ...) SSLDiagnostic(DiagsMakeLocation(), false, vc, fmt, ##__VA_ARGS__)
> @@ -120,6 +128,10 @@ void SSLReleaseContext(SSL_CTX* ctx);
> #define SSLDebug(fmt, ...) SSLDiagnostic(DiagsMakeLocation(), true, NULL, fmt, ##__VA_ARGS__)
> #define SSLDebugVC(vc,fmt, ...) SSLDiagnostic(DiagsMakeLocation(), true, vc, fmt, ##__VA_ARGS__)
> 
> +#define SSL_CLR_ERR_INCR_DYN_STAT(x, fmt, ...) \
> +  SSLDiagnostic(DiagsMakeLocation(), true, NULL, fmt, ##__VA_ARGS__); \
> +  RecIncrRawStat(ssl_rsb, NULL, (int) x, 1);
> +

Pleas wrap this in a do { } while(0) ...

> void SSLDiagnostic(const SrcLoc& loc, bool debug, SSLNetVConnection * vc, const char * fmt, ...) TS_PRINTFLIKE(4, 5);
> 
> // Return a static string name for a SSL_ERROR constant.
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/d12327d8/iocore/net/SSLNetVConnection.cc
> ----------------------------------------------------------------------
> diff --git a/iocore/net/SSLNetVConnection.cc b/iocore/net/SSLNetVConnection.cc
> index b4269e7..0f26679 100644
> --- a/iocore/net/SSLNetVConnection.cc
> +++ b/iocore/net/SSLNetVConnection.cc
> @@ -172,24 +172,6 @@ debug_certificate_name(const char * msg, X509_NAME * name)
>   BIO_free(bio);
> }
> 
> -static inline int
> -do_SSL_write(SSL * ssl, void *buf, int size)
> -{
> -  int r = 0;
> -  do {
> -    // need to check into SSL error handling
> -    // to see if this is good enough.
> -    r = SSL_write(ssl, (const char *) buf, size);
> -    if (r >= 0)
> -      return r;
> -    else
> -      r = -errno;
> -  } while (r == -EINTR || r == -ENOBUFS || r == -ENOMEM);
> -
> -  return r;
> -}
> -
> -
> static int
> ssl_read_from_net(SSLNetVConnection * sslvc, EThread * lthread, int64_t &ret)
> {
> @@ -199,7 +181,8 @@ ssl_read_from_net(SSLNetVConnection * sslvc, EThread * lthread, int64_t &ret)
>   int event = SSL_READ_ERROR_NONE;
>   int64_t bytes_read;
>   int64_t block_write_avail;
> -  int sslErr = SSL_ERROR_NONE;
> +  ssl_error_t sslErr = SSL_ERROR_NONE;
> +  int nread = 0;
> 
>   for (bytes_read = 0; (b != 0) && (sslErr == SSL_ERROR_NONE); b = b->next) {
>     block_write_avail = b->write_avail();
> @@ -209,23 +192,22 @@ ssl_read_from_net(SSLNetVConnection * sslvc, EThread * lthread, int64_t &ret)
>     int64_t offset = 0;
>     // while can be replaced with if - need to test what works faster with openssl
>     while (block_write_avail > 0) {
> -      int rres = SSL_read(sslvc->ssl, b->end() + offset, (int)block_write_avail);
> +      sslErr = SSLReadBuffer (sslvc->ssl, b->end() + offset, (size_t)block_write_avail, (size_t&)nread);

Don't cast this, just use size_t.

> 
> -      Debug("ssl", "[SSL_NetVConnection::ssl_read_from_net] rres=%d", rres);
> +      Debug("ssl", "[SSL_NetVConnection::ssl_read_from_net] nread=%d", (int)nread);
> 
> -      sslErr = SSL_get_error(sslvc->ssl, rres);
>       switch (sslErr) {
>       case SSL_ERROR_NONE:
> 
> #if DEBUG
> -        SSLDebugBufferPrint("ssl_buff", b->end() + offset, rres, "SSL Read");
> +        SSLDebugBufferPrint("ssl_buff", b->end() + offset, nread, "SSL Read");
> #endif
> 
> -        ink_assert(rres);
> +        ink_assert(nread);
> 
> -        bytes_read += rres;
> -        offset += rres;
> -        block_write_avail -= rres;
> +        bytes_read += nread;
> +        offset += nread;
> +        block_write_avail -= nread;
>         ink_assert(block_write_avail >= 0);
> 
>         continue;
> @@ -247,7 +229,7 @@ ssl_read_from_net(SSLNetVConnection * sslvc, EThread * lthread, int64_t &ret)
>         break;
>       case SSL_ERROR_SYSCALL:
>         SSL_INCREMENT_DYN_STAT(ssl_error_syscall);
> -        if (rres != 0) {
> +        if (nread != 0) {
>           // not EOF
>           event = SSL_READ_ERROR;
>           ret = errno;
> @@ -267,8 +249,7 @@ ssl_read_from_net(SSLNetVConnection * sslvc, EThread * lthread, int64_t &ret)
>       default:
>         event = SSL_READ_ERROR;
>         ret = errno;
> -        SSL_INCREMENT_DYN_STAT(ssl_error_ssl);
> -        Debug("ssl.error", "[SSL_NetVConnection::ssl_read_from_net]");
> +        SSL_CLR_ERR_INCR_DYN_STAT(ssl_error_ssl, "[SSL_NetVConnection::ssl_read_from_net]: errno=%d", errno);
>         break;
>       }                         // switch
>       break;
> @@ -636,6 +617,7 @@ SSLNetVConnection::load_buffer_and_write(int64_t towrite, int64_t &wattempted, i
>   ProxyMutex *mutex = this_ethread()->mutex;
>   int64_t r = 0;
>   int64_t l = 0;
> +  ssl_error_t err = SSL_ERROR_NONE;
> 
>   // XXX Rather than dealing with the block directly, we should use the IOBufferReader API.
>   int64_t offset = buf.reader()->start_offset;
> @@ -675,9 +657,9 @@ SSLNetVConnection::load_buffer_and_write(int64_t towrite, int64_t &wattempted, i
> 
>     wattempted = l;
>     total_wrote += l;
> -    Debug("ssl", "SSLNetVConnection::loadBufferAndCallWrite, before do_SSL_write, l=%" PRId64", towrite=%" PRId64", b=%p",
> +    Debug("ssl", "SSLNetVConnection::loadBufferAndCallWrite, before SSLWriteBuffer, l=%" PRId64", towrite=%" PRId64", b=%p",
>           l, towrite, b);
> -    r = do_SSL_write(ssl, b->start() + offset, (int)l);
> +    err = SSLWriteBuffer(ssl, b->start() + offset, (size_t)l, (size_t&)r);

Again, this cast is unsafe.

>     if (r == l) {
>       wattempted = total_wrote;
>     }
> @@ -704,8 +686,6 @@ SSLNetVConnection::load_buffer_and_write(int64_t towrite, int64_t &wattempted, i
>       return (total_wrote);
>     }
>   } else {
> -    int err = SSL_get_error(ssl, (int)r);
> -
>     switch (err) {
>     case SSL_ERROR_NONE:
>       Debug("ssl", "SSL_write-SSL_ERROR_NONE");
> @@ -742,8 +722,7 @@ SSLNetVConnection::load_buffer_and_write(int64_t towrite, int64_t &wattempted, i
>     case SSL_ERROR_SSL:
>     default:
>       r = -errno;
> -      SSL_INCREMENT_DYN_STAT(ssl_error_ssl);
> -      Debug("ssl.error", "SSL_write-SSL_ERROR_SSL");
> +      SSL_CLR_ERR_INCR_DYN_STAT(ssl_error_ssl, "SSL_write-SSL_ERROR_SSL errno=%d", errno);
>       break;
>     }
>     return (r);
> @@ -880,8 +859,6 @@ SSLNetVConnection::sslStartHandShake(int event, int &err)
> int
> SSLNetVConnection::sslServerHandShakeEvent(int &err)
> {
> -  int ret;
> -
>   if (SSL_HOOKS_DONE != sslPreAcceptHookState) {
>     // Get the first hook if we haven't started invoking yet.
>     if (SSL_HOOKS_INIT == sslPreAcceptHookState) {
> @@ -939,8 +916,7 @@ SSLNetVConnection::sslServerHandShakeEvent(int &err)
>     }
>   }
> 
> -  ret = SSL_accept(ssl);
> -  int ssl_error = SSL_get_error(ssl, ret);
> +  ssl_error_t ssl_error = SSLAccept(ssl);
> 
>   if (ssl_error != SSL_ERROR_NONE) {
>     err = errno;
> @@ -1039,9 +1015,11 @@ SSLNetVConnection::sslServerHandShakeEvent(int &err)
>   case SSL_ERROR_WANT_X509_LOOKUP:
>     return EVENT_CONT;
> 
> +  case SSL_ERROR_SSL:
> +    SSL_CLR_ERR_INCR_DYN_STAT(ssl_error_ssl, "SSLNetVConnection::sslServerHandShakeEvent, SSL_ERROR_SSL errno=%d", errno);
> +    // fall through
>   case SSL_ERROR_ZERO_RETURN:
>   case SSL_ERROR_SYSCALL:
> -  case SSL_ERROR_SSL:
>   default:
>     return EVENT_ERROR;
>   }
> @@ -1052,8 +1030,6 @@ SSLNetVConnection::sslServerHandShakeEvent(int &err)
> int
> SSLNetVConnection::sslClientHandShakeEvent(int &err)
> {
> -  int ret;
> -
> #if TS_USE_TLS_SNI
>   if (options.sni_servername) {
>     if (SSL_set_tlsext_host_name(ssl, options.sni_servername)) {
> @@ -1065,8 +1041,8 @@ SSLNetVConnection::sslClientHandShakeEvent(int &err)
>   }
> #endif
> 
> -  ret = SSL_connect(ssl);
> -  switch (SSL_get_error(ssl, ret)) {
> +  ssl_error_t ssl_error = SSLConnect(ssl);
> +  switch (ssl_error) {
>   case SSL_ERROR_NONE:
>     if (is_debug_tag_set("ssl")) {
>       X509 * cert = SSL_get_peer_certificate(ssl);
> @@ -1122,8 +1098,7 @@ SSLNetVConnection::sslClientHandShakeEvent(int &err)
>   case SSL_ERROR_SSL:
>   default:
>     err = errno;
> -    SSL_INCREMENT_DYN_STAT(ssl_error_ssl);
> -    Debug("ssl.error", "SSLNetVConnection::sslClientHandShakeEvent, SSL_ERROR_SSL");
> +    SSL_CLR_ERR_INCR_DYN_STAT(ssl_error_ssl, "SSLNetVConnection::sslClientHandShakeEvent, SSL_ERROR_SSL errno=%d", errno);
>     return EVENT_ERROR;
>     break;
> 
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/d12327d8/iocore/net/SSLUtils.cc
> ----------------------------------------------------------------------
> diff --git a/iocore/net/SSLUtils.cc b/iocore/net/SSLUtils.cc
> index a57ce90..59e0b63 100644
> --- a/iocore/net/SSLUtils.cc
> +++ b/iocore/net/SSLUtils.cc
> @@ -1660,3 +1660,62 @@ SSLReleaseContext(SSL_CTX * ctx)
> {
>   SSL_CTX_free(ctx);
> }
> +
> +
> +ssl_error_t
> +SSLWriteBuffer(SSL * ssl, const void * buf, size_t nbytes, size_t& nwritten)
> +{
> +  nwritten = 0;
> +
> +  if (unlikely(nbytes == 0)) {
> +    return SSL_ERROR_NONE;
> +  }
> +
> +  int ret = SSL_write(ssl, buf, (int)nbytes);
> +  if (ret > 0) {
> +    nwritten = ret;
> +    return SSL_ERROR_NONE;
> +  }
> +
> +  return SSL_get_error(ssl, ret);
> +}
> +
> +ssl_error_t
> +SSLReadBuffer(SSL * ssl, void * buf, size_t nbytes, size_t& nread)
> +{
> +  nread = 0;
> +
> +  if (unlikely(nbytes == 0)) {
> +    return SSL_ERROR_NONE;
> +  }
> +
> +  int ret = SSL_read(ssl, buf, (int)nbytes);
> +  if (ret > 0) {
> +    nread = ret;
> +    return SSL_ERROR_NONE;
> +  }
> +
> +  return SSL_get_error(ssl, ret);
> +}
> +
> +ssl_error_t
> +SSLAccept(SSL * ssl)
> +{
> +  int ret = SSL_accept(ssl);
> +  if (ret > 0) {
> +    return SSL_ERROR_NONE;
> +  }
> +
> +  return SSL_get_error(ssl, ret);
> +}
> +
> +ssl_error_t
> +SSLConnect(SSL * ssl)
> +{
> +  int ret = SSL_connect(ssl);
> +  if (ret > 0) {
> +    return SSL_ERROR_NONE;
> +  }
> +
> +  return SSL_get_error(ssl, ret);
> +}
>