You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@trafficserver.apache.org by ma...@apache.org on 2018/09/10 08:41:08 UTC

[trafficserver] branch quic-latest updated: Reduce dependency for SSL library

This is an automated email from the ASF dual-hosted git repository.

maskit pushed a commit to branch quic-latest
in repository https://gitbox.apache.org/repos/asf/trafficserver.git


The following commit(s) were added to refs/heads/quic-latest by this push:
     new 41f043e  Reduce dependency for SSL library
41f043e is described below

commit 41f043e53d02d782cd24d3dbc74b21369c58f1c6
Author: Masakazu Kitajo <ma...@apache.org>
AuthorDate: Thu Aug 30 14:00:45 2018 +0900

    Reduce dependency for SSL library
---
 iocore/net/quic/QUICGlobals.cc             |  8 +++---
 iocore/net/quic/QUICGlobals.h              |  2 +-
 iocore/net/quic/QUICHandshake.cc           | 44 ++++++++++++++++++++----------
 iocore/net/quic/QUICHandshake.h            | 16 ++++++-----
 iocore/net/quic/QUICHandshakeProtocol.h    | 24 ++++++++++------
 iocore/net/quic/QUICTLS.cc                 | 24 ++++++++++++++++
 iocore/net/quic/QUICTLS.h                  | 10 +++++++
 iocore/net/quic/QUICTLS_openssl.cc         |  2 ++
 iocore/net/quic/QUICTransportParameters.cc | 13 +++++----
 9 files changed, 102 insertions(+), 41 deletions(-)

diff --git a/iocore/net/quic/QUICGlobals.cc b/iocore/net/quic/QUICGlobals.cc
index ceb4c95..c318feb 100644
--- a/iocore/net/quic/QUICGlobals.cc
+++ b/iocore/net/quic/QUICGlobals.cc
@@ -31,15 +31,15 @@
 
 RecRawStatBlock *quic_rsb;
 
-int QUIC::ssl_quic_qc_index = -1;
-int QUIC::ssl_quic_hs_index = -1;
+int QUIC::ssl_quic_qc_index  = -1;
+int QUIC::ssl_quic_tls_index = -1;
 
 void
 QUIC::init()
 {
   QUIC::_register_stats();
-  ssl_quic_qc_index = SSL_get_ex_new_index(0, (void *)"QUICConnection index", nullptr, nullptr, nullptr);
-  ssl_quic_hs_index = SSL_get_ex_new_index(0, (void *)"QUICHandshake index", nullptr, nullptr, nullptr);
+  ssl_quic_qc_index  = SSL_get_ex_new_index(0, (void *)"QUICConnection index", nullptr, nullptr, nullptr);
+  ssl_quic_tls_index = SSL_get_ex_new_index(0, (void *)"QUICTLS index", nullptr, nullptr, nullptr);
 }
 
 int
diff --git a/iocore/net/quic/QUICGlobals.h b/iocore/net/quic/QUICGlobals.h
index 379fa18..0e1c1eb 100644
--- a/iocore/net/quic/QUICGlobals.h
+++ b/iocore/net/quic/QUICGlobals.h
@@ -35,7 +35,7 @@ public:
                                       unsigned inlen, void *);
 
   static int ssl_quic_qc_index;
-  static int ssl_quic_hs_index;
+  static int ssl_quic_tls_index;
 
 private:
   static void _register_stats();
diff --git a/iocore/net/quic/QUICHandshake.cc b/iocore/net/quic/QUICHandshake.cc
index a949011..645ef6e 100644
--- a/iocore/net/quic/QUICHandshake.cc
+++ b/iocore/net/quic/QUICHandshake.cc
@@ -97,7 +97,6 @@ QUICHandshake::QUICHandshake(QUICConnection *qc, QUICHandshakeProtocol *hsp, QUI
   if (dynamic_cast<QUICTLS *>(hsp)) {
     SSL *ssl = static_cast<QUICTLS *>(hsp)->ssl_handle();
     SSL_set_ex_data(ssl, QUIC::ssl_quic_qc_index, qc);
-    SSL_set_ex_data(ssl, QUIC::ssl_quic_hs_index, this);
   }
 
   this->_hs_protocol->initialize_key_materials(this->_qc->original_connection_id());
@@ -223,14 +222,26 @@ QUICHandshake::negotiated_application_name(const uint8_t **name, unsigned int *l
   this->_hs_protocol->negotiated_application_name(name, len);
 }
 
-void
-QUICHandshake::set_transport_parameters(std::shared_ptr<QUICTransportParametersInClientHello> tp)
+bool
+QUICHandshake::check_remote_transport_parameters()
+{
+  auto tp = this->_hs_protocol->remote_transport_parameters();
+  if (std::dynamic_pointer_cast<const QUICTransportParametersInClientHello>(tp)) {
+    return this->_check_remote_transport_parameters(std::static_pointer_cast<const QUICTransportParametersInClientHello>(tp));
+  } else {
+    return this->_check_remote_transport_parameters(
+      std::static_pointer_cast<const QUICTransportParametersInEncryptedExtensions>(tp));
+  }
+}
+
+bool
+QUICHandshake::_check_remote_transport_parameters(std::shared_ptr<const QUICTransportParametersInClientHello> tp)
 {
   // An endpoint MUST treat receipt of duplicate transport parameters as a connection error of type TRANSPORT_PARAMETER_ERROR.
   if (!tp->is_valid()) {
     QUICHSDebug("Transport parameter is not valid");
     this->_abort_handshake(QUICTransErrorCode::TRANSPORT_PARAMETER_ERROR);
-    return;
+    return false;
   }
 
   this->_remote_transport_parameters = tp;
@@ -239,21 +250,21 @@ QUICHandshake::set_transport_parameters(std::shared_ptr<QUICTransportParametersI
   if (this->_version_negotiator->validate(tp.get()) != QUICVersionNegotiationStatus::VALIDATED) {
     QUICHSDebug("Version revalidation failed");
     this->_abort_handshake(QUICTransErrorCode::VERSION_NEGOTIATION_ERROR);
-    return;
+    return false;
   }
 
   QUICHSDebug("Version negotiation validated: %x", tp->initial_version());
-  return;
+  return true;
 }
 
-void
-QUICHandshake::set_transport_parameters(std::shared_ptr<QUICTransportParametersInEncryptedExtensions> tp)
+bool
+QUICHandshake::_check_remote_transport_parameters(std::shared_ptr<const QUICTransportParametersInEncryptedExtensions> tp)
 {
   // An endpoint MUST treat receipt of duplicate transport parameters as a connection error of type TRANSPORT_PARAMETER_ERROR.
   if (!tp->is_valid()) {
     QUICHSDebug("Transport parameter is not valid");
     this->_abort_handshake(QUICTransErrorCode::TRANSPORT_PARAMETER_ERROR);
-    return;
+    return false;
   }
 
   this->_remote_transport_parameters = tp;
@@ -262,10 +273,10 @@ QUICHandshake::set_transport_parameters(std::shared_ptr<QUICTransportParametersI
   if (this->_version_negotiator->validate(tp.get()) != QUICVersionNegotiationStatus::VALIDATED) {
     QUICHSDebug("Version revalidation failed");
     this->_abort_handshake(QUICTransErrorCode::VERSION_NEGOTIATION_ERROR);
-    return;
+    return false;
   }
 
-  return;
+  return true;
 }
 
 std::shared_ptr<const QUICTransportParameters>
@@ -365,9 +376,9 @@ QUICHandshake::_load_local_server_transport_parameters(QUICVersion negotiated_ve
   // MAYs
   tp->set(QUICTransportParameterId::INITIAL_MAX_BIDI_STREAMS, params->initial_max_bidi_streams_in());
   tp->set(QUICTransportParameterId::INITIAL_MAX_UNI_STREAMS, params->initial_max_uni_streams_in());
-  // this->_local_transport_parameters.add(QUICTransportParameterId::MAX_PACKET_SIZE, {{0x00, 0x00}, 2});
+  this->_local_transport_parameters = std::shared_ptr<QUICTransportParameters>(tp);
 
-  this->_local_transport_parameters = std::unique_ptr<QUICTransportParameters>(tp);
+  this->_hs_protocol->set_local_transport_parameters(this->_local_transport_parameters);
 }
 
 void
@@ -386,7 +397,7 @@ QUICHandshake::_load_local_client_transport_parameters(QUICVersion initial_versi
   tp->set(QUICTransportParameterId::INITIAL_MAX_BIDI_STREAMS, params->initial_max_bidi_streams_out());
   tp->set(QUICTransportParameterId::INITIAL_MAX_UNI_STREAMS, params->initial_max_uni_streams_out());
 
-  this->_local_transport_parameters = std::unique_ptr<QUICTransportParameters>(tp);
+  this->_hs_protocol->set_local_transport_parameters(std::unique_ptr<QUICTransportParameters>(tp));
 }
 
 QUICErrorUPtr
@@ -421,6 +432,11 @@ QUICHandshake::do_handshake()
   out.max_buf_len                        = MAX_HANDSHAKE_MSG_LEN;
 
   int result = this->_hs_protocol->handshake(&out, &in);
+  if (this->_remote_transport_parameters == nullptr) {
+    if (!this->check_remote_transport_parameters()) {
+      result = 0;
+    }
+  }
 
   if (result == 1) {
     for (auto level : QUIC_ENCRYPTION_LEVELS) {
diff --git a/iocore/net/quic/QUICHandshake.h b/iocore/net/quic/QUICHandshake.h
index fb5cc21..ce016bf 100644
--- a/iocore/net/quic/QUICHandshake.h
+++ b/iocore/net/quic/QUICHandshake.h
@@ -63,6 +63,8 @@ public:
 
   QUICErrorUPtr do_handshake();
 
+  bool check_remote_transport_parameters();
+
   // Getters
   QUICVersion negotiated_version();
   const char *negotiated_cipher_suite();
@@ -75,14 +77,9 @@ public:
   bool is_stateless_retry_enabled() const;
   bool has_remote_tp() const;
 
-  void set_transport_parameters(std::shared_ptr<QUICTransportParametersInClientHello> tp);
-  void set_transport_parameters(std::shared_ptr<QUICTransportParametersInEncryptedExtensions> tp);
-
 private:
-  QUICConnection *_qc                                                   = nullptr;
-  QUICHandshakeProtocol *_hs_protocol                                   = nullptr;
-  std::shared_ptr<QUICTransportParameters> _local_transport_parameters  = nullptr;
-  std::shared_ptr<QUICTransportParameters> _remote_transport_parameters = nullptr;
+  QUICConnection *_qc                 = nullptr;
+  QUICHandshakeProtocol *_hs_protocol = nullptr;
 
   QUICVersionNegotiator *_version_negotiator = nullptr;
   QUICStatelessResetToken _reset_token;
@@ -103,5 +100,10 @@ private:
   }
   void _load_local_server_transport_parameters(QUICVersion negotiated_version);
   void _load_local_client_transport_parameters(QUICVersion initial_version);
+  bool _check_remote_transport_parameters(std::shared_ptr<const QUICTransportParametersInClientHello> tp);
+  bool _check_remote_transport_parameters(std::shared_ptr<const QUICTransportParametersInEncryptedExtensions> tp);
+  std::shared_ptr<const QUICTransportParameters> _local_transport_parameters  = nullptr;
+  std::shared_ptr<const QUICTransportParameters> _remote_transport_parameters = nullptr;
+
   void _abort_handshake(QUICTransErrorCode code);
 };
diff --git a/iocore/net/quic/QUICHandshakeProtocol.h b/iocore/net/quic/QUICHandshakeProtocol.h
index f489e30..21eef39 100644
--- a/iocore/net/quic/QUICHandshakeProtocol.h
+++ b/iocore/net/quic/QUICHandshakeProtocol.h
@@ -25,6 +25,7 @@
 
 #include "QUICKeyGenerator.h"
 #include "QUICTypes.h"
+#include "QUICTransportParameters.h"
 
 class QUICHandshakeProtocol;
 
@@ -71,15 +72,20 @@ public:
   QUICHandshakeProtocol(){};
   virtual ~QUICHandshakeProtocol(){};
 
-  virtual int handshake(QUICHandshakeMsgs *out, const QUICHandshakeMsgs *in)                           = 0;
-  virtual void reset()                                                                                 = 0;
-  virtual bool is_handshake_finished() const                                                           = 0;
-  virtual bool is_ready_to_derive() const                                                              = 0;
-  virtual bool is_key_derived(QUICKeyPhase key_phase, bool for_encryption) const                       = 0;
-  virtual int initialize_key_materials(QUICConnectionId cid)                                           = 0;
-  virtual int update_key_materials()                                                                   = 0;
-  virtual const char *negotiated_cipher_suite() const                                                  = 0;
-  virtual void negotiated_application_name(const uint8_t **name, unsigned int *len) const              = 0;
+  virtual int handshake(QUICHandshakeMsgs *out, const QUICHandshakeMsgs *in)                      = 0;
+  virtual void reset()                                                                            = 0;
+  virtual bool is_handshake_finished() const                                                      = 0;
+  virtual bool is_ready_to_derive() const                                                         = 0;
+  virtual bool is_key_derived(QUICKeyPhase key_phase, bool for_encryption) const                  = 0;
+  virtual int initialize_key_materials(QUICConnectionId cid)                                      = 0;
+  virtual int update_key_materials()                                                              = 0;
+  virtual const char *negotiated_cipher_suite() const                                             = 0;
+  virtual void negotiated_application_name(const uint8_t **name, unsigned int *len) const         = 0;
+  virtual std::shared_ptr<const QUICTransportParameters> local_transport_parameters()             = 0;
+  virtual std::shared_ptr<const QUICTransportParameters> remote_transport_parameters()            = 0;
+  virtual void set_local_transport_parameters(std::shared_ptr<const QUICTransportParameters> tp)  = 0;
+  virtual void set_remote_transport_parameters(std::shared_ptr<const QUICTransportParameters> tp) = 0;
+
   virtual bool encrypt(uint8_t *cipher, size_t &cipher_len, size_t max_cipher_len, const uint8_t *plain, size_t plain_len,
                        uint64_t pkt_num, const uint8_t *ad, size_t ad_len, QUICKeyPhase phase) const   = 0;
   virtual bool decrypt(uint8_t *plain, size_t &plain_len, size_t max_plain_len, const uint8_t *cipher, size_t cipher_len,
diff --git a/iocore/net/quic/QUICTLS.cc b/iocore/net/quic/QUICTLS.cc
index 23dbfab..363fca2 100644
--- a/iocore/net/quic/QUICTLS.cc
+++ b/iocore/net/quic/QUICTLS.cc
@@ -37,6 +37,30 @@ QUICTLS::ssl_handle()
   return this->_ssl;
 }
 
+std::shared_ptr<const QUICTransportParameters>
+QUICTLS::local_transport_parameters()
+{
+  return this->_local_transport_parameters;
+}
+
+std::shared_ptr<const QUICTransportParameters>
+QUICTLS::remote_transport_parameters()
+{
+  return this->_remote_transport_parameters;
+}
+
+void
+QUICTLS::set_local_transport_parameters(std::shared_ptr<const QUICTransportParameters> tp)
+{
+  this->_local_transport_parameters = tp;
+}
+
+void
+QUICTLS::set_remote_transport_parameters(std::shared_ptr<const QUICTransportParameters> tp)
+{
+  this->_remote_transport_parameters = tp;
+}
+
 QUICTLS::~QUICTLS()
 {
   SSL_free(this->_ssl);
diff --git a/iocore/net/quic/QUICTLS.h b/iocore/net/quic/QUICTLS.h
index d7f2ec1..9ac6cc1 100644
--- a/iocore/net/quic/QUICTLS.h
+++ b/iocore/net/quic/QUICTLS.h
@@ -51,9 +51,16 @@ public:
   static QUICEncryptionLevel get_encryption_level(int msg_type);
   static uint16_t convert_to_quic_trans_error_code(uint8_t alert);
 
+  std::shared_ptr<const QUICTransportParameters> local_transport_parameters() override;
+  std::shared_ptr<const QUICTransportParameters> remote_transport_parameters() override;
+  void set_local_transport_parameters(std::shared_ptr<const QUICTransportParameters> tp) override;
+  void set_remote_transport_parameters(std::shared_ptr<const QUICTransportParameters> tp) override;
+
   // FIXME Should not exist
   SSL *ssl_handle();
 
+  // QUICHandshakeProtocol
+
   int handshake(QUICHandshakeMsgs *out, const QUICHandshakeMsgs *in) override;
   void reset() override;
   bool is_handshake_finished() const override;
@@ -105,4 +112,7 @@ private:
   bool _early_data                       = true;
   QUICEncryptionLevel _current_level     = QUICEncryptionLevel::INITIAL;
   HandshakeState _state                  = HandshakeState::PROCESSING;
+
+  std::shared_ptr<const QUICTransportParameters> _local_transport_parameters  = nullptr;
+  std::shared_ptr<const QUICTransportParameters> _remote_transport_parameters = nullptr;
 };
diff --git a/iocore/net/quic/QUICTLS_openssl.cc b/iocore/net/quic/QUICTLS_openssl.cc
index bdeda68..ba7d278 100644
--- a/iocore/net/quic/QUICTLS_openssl.cc
+++ b/iocore/net/quic/QUICTLS_openssl.cc
@@ -20,6 +20,7 @@
  *  See the License for the specific language governing permissions and
  *  limitations under the License.
  */
+#include "QUICGlobals.h"
 #include "QUICTLS.h"
 
 #include <openssl/err.h>
@@ -228,6 +229,7 @@ QUICTLS::QUICTLS(SSL_CTX *ssl_ctx, NetVConnectionContext_t nvc_ctx)
   this->_client_pp = new QUICPacketProtection();
   this->_server_pp = new QUICPacketProtection();
 
+  SSL_set_ex_data(this->_ssl, QUIC::ssl_quic_tls_index, this);
   SSL_set_key_callback(this->_ssl, key_cb, this);
 }
 
diff --git a/iocore/net/quic/QUICTransportParameters.cc b/iocore/net/quic/QUICTransportParameters.cc
index a1db90a..de7382f 100644
--- a/iocore/net/quic/QUICTransportParameters.cc
+++ b/iocore/net/quic/QUICTransportParameters.cc
@@ -29,6 +29,7 @@
 #include "QUICConnection.h"
 #include "QUICHandshake.h"
 #include "QUICDebugNames.h"
+#include "QUICTLS.h"
 
 static constexpr int TRANSPORT_PARAMETERS_MAXIMUM_SIZE = 65535;
 static constexpr char tag[]                            = "quic_handshake";
@@ -481,9 +482,9 @@ int
 QUICTransportParametersHandler::add(SSL *s, unsigned int ext_type, unsigned int context, const unsigned char **out, size_t *outlen,
                                     X509 *x, size_t chainidx, int *al, void *add_arg)
 {
-  QUICHandshake *hs = static_cast<QUICHandshake *>(SSL_get_ex_data(s, QUIC::ssl_quic_hs_index));
-  *out              = reinterpret_cast<const unsigned char *>(ats_malloc(TRANSPORT_PARAMETERS_MAXIMUM_SIZE));
-  hs->local_transport_parameters()->store(const_cast<uint8_t *>(*out), reinterpret_cast<uint16_t *>(outlen));
+  QUICTLS *qtls = static_cast<QUICTLS *>(SSL_get_ex_data(s, QUIC::ssl_quic_tls_index));
+  *out          = reinterpret_cast<const unsigned char *>(ats_malloc(TRANSPORT_PARAMETERS_MAXIMUM_SIZE));
+  qtls->local_transport_parameters()->store(const_cast<uint8_t *>(*out), reinterpret_cast<uint16_t *>(outlen));
 
   return 1;
 }
@@ -498,14 +499,14 @@ int
 QUICTransportParametersHandler::parse(SSL *s, unsigned int ext_type, unsigned int context, const unsigned char *in, size_t inlen,
                                       X509 *x, size_t chainidx, int *al, void *parse_arg)
 {
-  QUICHandshake *hs = static_cast<QUICHandshake *>(SSL_get_ex_data(s, QUIC::ssl_quic_hs_index));
+  QUICTLS *qtls = static_cast<QUICTLS *>(SSL_get_ex_data(s, QUIC::ssl_quic_tls_index));
 
   switch (context) {
   case SSL_EXT_CLIENT_HELLO:
-    hs->set_transport_parameters(std::make_shared<QUICTransportParametersInClientHello>(in, inlen));
+    qtls->set_remote_transport_parameters(std::make_shared<QUICTransportParametersInClientHello>(in, inlen));
     break;
   case SSL_EXT_TLS1_3_ENCRYPTED_EXTENSIONS:
-    hs->set_transport_parameters(std::make_shared<QUICTransportParametersInEncryptedExtensions>(in, inlen));
+    qtls->set_remote_transport_parameters(std::make_shared<QUICTransportParametersInEncryptedExtensions>(in, inlen));
     break;
   default:
     // Do nothing