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 2017/08/25 02:16:33 UTC

[trafficserver] branch quic-latest updated: Refactor handshake

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 1acd290  Refactor handshake
1acd290 is described below

commit 1acd290179f3ad0f963ba6410e6cf8d50b271f62
Author: Masakazu Kitajo <ma...@apache.org>
AuthorDate: Fri Aug 25 11:14:30 2017 +0900

    Refactor handshake
    
    As per 7.2 Cryptographic and Transport Handshake, crypto module, transport
    parameter, negotiated version, negotiated application are provided from
    Handshake.
---
 iocore/net/P_QUICNetVConnection.h          |  10 +--
 iocore/net/QUICNetVConnection.cc           | 126 +++++----------------------
 iocore/net/quic/Mock.h                     |   7 --
 iocore/net/quic/QUICApplication.h          |   1 +
 iocore/net/quic/QUICConnection.h           |  14 +--
 iocore/net/quic/QUICCrypto.cc              |   5 +-
 iocore/net/quic/QUICCrypto.h               |   5 +-
 iocore/net/quic/QUICFrame.h                |   1 -
 iocore/net/quic/QUICGlobals.cc             |   9 +-
 iocore/net/quic/QUICGlobals.h              |   6 +-
 iocore/net/quic/QUICHandshake.cc           | 131 ++++++++++++++++++++++++++++-
 iocore/net/quic/QUICHandshake.h            |  32 ++++++-
 iocore/net/quic/QUICStream.h               |   1 -
 iocore/net/quic/QUICStreamManager.cc       |  26 +++---
 iocore/net/quic/QUICStreamManager.h        |  13 +--
 iocore/net/quic/QUICTransportParameters.cc |  14 ++-
 iocore/net/quic/QUICTransportParameters.h  |   2 -
 iocore/net/quic/QUICTypes.h                |   4 -
 iocore/net/quic/QUICVersionNegotiator.cc   |  19 +++--
 iocore/net/quic/QUICVersionNegotiator.h    |   8 +-
 20 files changed, 241 insertions(+), 193 deletions(-)

diff --git a/iocore/net/P_QUICNetVConnection.h b/iocore/net/P_QUICNetVConnection.h
index 40a9a40..785f30b 100644
--- a/iocore/net/P_QUICNetVConnection.h
+++ b/iocore/net/P_QUICNetVConnection.h
@@ -166,7 +166,7 @@ public:
   virtual void net_read_io(NetHandler *nh, EThread *lthread) override;
   virtual int64_t load_buffer_and_write(int64_t towrite, MIOBufferAccessor &buf, int64_t &total_written, int &needs) override;
 
-  SSLNextProtocolSet *next_protocol_set();
+  // QUICNetVConnection
   void registerNextProtocolSet(SSLNextProtocolSet *s);
 
   // QUICConnection
@@ -174,9 +174,8 @@ public:
   uint32_t minimum_quic_packet_size() override;
   uint32_t maximum_stream_frame_data_size() override;
   uint32_t pmtu() override;
-  void set_transport_parameters(std::unique_ptr<QUICTransportParameters> tp) override;
-  const QUICTransportParameters &local_transport_parameters() override;
-  const QUICTransportParameters &remote_transport_parameters() override;
+  NetVConnectionContext_t direction() override;
+  SSLNextProtocolSet *next_protocol_set() override;
   void close(QUICError error) override;
 
   // QUICConnection (QUICPacketTransmitter)
@@ -204,9 +203,6 @@ private:
 
   SSLNextProtocolSet *_next_protocol_set = nullptr;
 
-  std::unique_ptr<QUICTransportParameters> _local_transport_parameters  = nullptr;
-  std::unique_ptr<QUICTransportParameters> _remote_transport_parameters = nullptr;
-
   // TODO: use custom allocator and make them std::unique_ptr or std::shared_ptr
   // or make them just member variables.
   QUICVersionNegotiator *_version_negotiator       = nullptr;
diff --git a/iocore/net/QUICNetVConnection.cc b/iocore/net/QUICNetVConnection.cc
index 2588579..75b16f9 100644
--- a/iocore/net/QUICNetVConnection.cc
+++ b/iocore/net/QUICNetVConnection.cc
@@ -94,14 +94,16 @@ QUICNetVConnection::startEvent(int /*event ATS_UNUSED */, Event *e)
 void
 QUICNetVConnection::start(SSL_CTX *ssl_ctx)
 {
-  this->_version_negotiator = new QUICVersionNegotiator(&this->_packet_factory, this);
-  this->_crypto             = new QUICCrypto(ssl_ctx, this);
-  this->_frame_dispatcher   = new QUICFrameDispatcher();
+  // Version 0x00000001 uses stream 0 for cryptographic handshake with TLS 1.3, but newer version may not
+  this->_handshake_handler = new QUICHandshake(this, ssl_ctx);
+  this->_application_map.set(STREAM_ID_FOR_HANDSHAKE, this->_handshake_handler);
+
+  this->_crypto           = this->_handshake_handler->crypto_module();
+  this->_frame_dispatcher = new QUICFrameDispatcher();
   this->_packet_factory.set_crypto_module(this->_crypto);
 
   // Create frame handlers
-  this->_stream_manager = new QUICStreamManager();
-  this->_stream_manager->init(this, this, &this->_application_map);
+  this->_stream_manager        = new QUICStreamManager(this, &this->_application_map);
   this->_congestion_controller = new QUICCongestionController();
   this->_loss_detector         = new QUICLossDetector(this);
 
@@ -109,31 +111,6 @@ QUICNetVConnection::start(SSL_CTX *ssl_ctx)
   this->_frame_dispatcher->add_handler(this->_stream_manager);
   this->_frame_dispatcher->add_handler(this->_congestion_controller);
   this->_frame_dispatcher->add_handler(this->_loss_detector);
-
-  QUICConfig::scoped_config params;
-
-  // MUSTs
-  QUICTransportParametersInEncryptedExtensions *tp = new QUICTransportParametersInEncryptedExtensions();
-
-  tp->add(QUICTransportParameterId::INITIAL_MAX_STREAM_DATA,
-          std::unique_ptr<QUICTransportParameterValue>(
-            new QUICTransportParameterValue(params->initial_max_stream_data(), sizeof(params->initial_max_stream_data()))));
-
-  tp->add(QUICTransportParameterId::INITIAL_MAX_DATA, std::unique_ptr<QUICTransportParameterValue>(new QUICTransportParameterValue(
-                                                        params->initial_max_data(), sizeof(params->initial_max_data()))));
-
-  tp->add(QUICTransportParameterId::INITIAL_MAX_STREAM_ID,
-          std::unique_ptr<QUICTransportParameterValue>(
-            new QUICTransportParameterValue(params->initial_max_stream_id(), sizeof(params->initial_max_stream_id()))));
-
-  tp->add(QUICTransportParameterId::IDLE_TIMEOUT, std::unique_ptr<QUICTransportParameterValue>(new QUICTransportParameterValue(
-                                                    params->no_activity_timeout_in(), sizeof(uint16_t))));
-
-  tp->add_version(QUIC_SUPPORTED_VERSIONS[0]);
-  // MAYs
-  // this->_local_transport_parameters.add(QUICTransportParameterId::TRUNCATE_CONNECTION_ID, {});
-  // this->_local_transport_parameters.add(QUICTransportParameterId::MAX_PACKET_SIZE, {{0x00, 0x00}, 2});
-  this->_local_transport_parameters = std::unique_ptr<QUICTransportParameters>(tp);
 }
 
 void
@@ -174,53 +151,10 @@ QUICNetVConnection::pmtu()
   return this->_pmtu;
 }
 
-void
-QUICNetVConnection::set_transport_parameters(std::unique_ptr<QUICTransportParameters> tp)
-{
-  this->_remote_transport_parameters = std::move(tp);
-  this->_stream_manager->init_flow_control_params(*this->_local_transport_parameters, *this->_remote_transport_parameters);
-
-  const QUICTransportParametersInClientHello *tp_in_ch =
-    dynamic_cast<QUICTransportParametersInClientHello *>(this->_remote_transport_parameters.get());
-  if (tp_in_ch) {
-    // Version revalidation
-    QUICVersion version = tp_in_ch->negotiated_version();
-    if (this->_version_negotiator->revalidate(version) != QUICVersionNegotiationStatus::REVALIDATED) {
-      this->close({QUICErrorClass::QUIC_TRANSPORT, QUICErrorCode::QUIC_VERSION_NEGOTIATION_MISMATCH});
-      return;
-    }
-    if (tp_in_ch->negotiated_version() != tp_in_ch->initial_version()) {
-      // FIXME Check initial_version
-      /* If the initial version is different from the negotiated_version, a
-       * stateless server MUST check that it would have sent a version
-       * negotiation packet if it had received a packet with the indicated
-       * initial_version. (Draft-04 7.3.4. Version Negotiation Validation)
-       */
-      this->close({QUICErrorClass::QUIC_TRANSPORT, QUICErrorCode::QUIC_VERSION_NEGOTIATION_MISMATCH});
-      return;
-    }
-    DebugQUICCon("Version negotiation revalidated: %x", tp_in_ch->negotiated_version());
-    return;
-  }
-
-  const QUICTransportParametersInEncryptedExtensions *tp_in_ee =
-    dynamic_cast<QUICTransportParametersInEncryptedExtensions *>(this->_remote_transport_parameters.get());
-  if (tp_in_ee) {
-    // TODO Add client side implementation
-    return;
-  }
-}
-
-const QUICTransportParameters &
-QUICNetVConnection::local_transport_parameters()
+NetVConnectionContext_t
+QUICNetVConnection::direction()
 {
-  return *this->_local_transport_parameters;
-}
-
-const QUICTransportParameters &
-QUICNetVConnection::remote_transport_parameters()
-{
-  return *this->_remote_transport_parameters;
+  return this->netvc_context;
 }
 
 uint32_t
@@ -419,8 +353,10 @@ QUICNetVConnection::state_handshake(int event, Event *data)
   }
 
   if (this->_handshake_handler && this->_handshake_handler->is_completed()) {
-    DebugQUICCon("setup quic application");
     this->_application_map.set_default(this->_create_application());
+    this->_stream_manager->init_flow_control_params(this->_handshake_handler->local_transport_parameters(),
+                                                    this->_handshake_handler->remote_transport_parameters());
+
     DebugQUICCon("Enter state_connection_established");
     SET_HANDLER((NetVConnHandler)&QUICNetVConnection::state_connection_established);
   }
@@ -557,42 +493,22 @@ QUICNetVConnection::_state_handshake_process_initial_client_packet(std::unique_p
 {
   if (packet->size() < this->minimum_quic_packet_size()) {
     DebugQUICCon("%" PRId32 ", %" PRId32, packet->size(), this->minimum_quic_packet_size());
-
     return QUICError(QUICErrorClass::QUIC_TRANSPORT, QUICErrorCode::QUIC_INTERNAL_ERROR);
   }
 
-  // Negotiate version
-  if (this->_version_negotiator->status() == QUICVersionNegotiationStatus::NOT_NEGOTIATED) {
-    if (packet->type() != QUICPacketType::CLIENT_INITIAL) {
-      return QUICError(QUICErrorClass::QUIC_TRANSPORT, QUICErrorCode::QUIC_INTERNAL_ERROR);
-    }
-
-    if (!packet->version()) {
-      return QUICError(QUICErrorClass::QUIC_TRANSPORT, QUICErrorCode::QUIC_INTERNAL_ERROR);
-    }
-
-    if (this->_version_negotiator->negotiate(packet.get()) != QUICVersionNegotiationStatus::NEGOTIATED) {
-      DebugQUICCon("Version negotiation failed: %x", packet->version());
-      return QUICError(QUICErrorClass::QUIC_TRANSPORT, QUICErrorCode::QUIC_VERSION_NEGOTIATION_MISMATCH);
-    }
-
-    DebugQUICCon("Version negotiation succeeded: %x", packet->version());
-    this->_packet_factory.set_version(packet->version());
+  // Start handshake
+  QUICError error = this->_handshake_handler->start(packet.get(), &this->_packet_factory);
+  if (this->_handshake_handler->is_version_negotiated()) {
     // Check integrity (QUIC-TLS-04: 6.1. Integrity Check Processing)
-    if (!packet->has_valid_fnv1a_hash()) {
+    if (packet->has_valid_fnv1a_hash()) {
+      bool should_send_ack;
+      this->_frame_dispatcher->receive_frames(packet->payload(), packet->payload_size(), should_send_ack);
+    } else {
       DebugQUICCon("Invalid FNV-1a hash value");
       return QUICError(QUICErrorClass::QUIC_TRANSPORT, QUICErrorCode::CRYPTOGRAPHIC_ERROR);
     }
-
-    // Version 0x00000001 uses stream 0 for cryptographic handshake with TLS 1.3, but newer version may not
-    this->_handshake_handler = new QUICHandshake(this, this->_crypto);
-    this->_application_map.set(STREAM_ID_FOR_HANDSHAKE, this->_handshake_handler);
-    bool should_send_ack;
-
-    return this->_frame_dispatcher->receive_frames(packet->payload(), packet->payload_size(), should_send_ack);
   }
-
-  return QUICError(QUICErrorClass::NONE);
+  return error;
 }
 
 QUICError
diff --git a/iocore/net/quic/Mock.h b/iocore/net/quic/Mock.h
index 6e72a74..842a6c7 100644
--- a/iocore/net/quic/Mock.h
+++ b/iocore/net/quic/Mock.h
@@ -96,7 +96,6 @@ class MockQUICConnection : public QUICConnection
 {
 public:
   MockQUICConnection() : QUICConnection() { this->_mutex = new_ProxyMutex(); };
-
   void
   transmit_packet(std::unique_ptr<const QUICPacket> packet) override
   {
@@ -200,7 +199,6 @@ class MockQUICPacketTransmitter : public QUICPacketTransmitter
 {
 public:
   MockQUICPacketTransmitter() : QUICPacketTransmitter() { this->_mutex = new_ProxyMutex(); };
-
   void
   transmit_packet(std::unique_ptr<const QUICPacket> packet) override
   {
@@ -242,7 +240,6 @@ class MockQUICLossDetector : public QUICLossDetector
 {
 public:
   MockQUICLossDetector() : QUICLossDetector(new MockQUICPacketTransmitter()) {}
-
   void
   rcv_frame(std::shared_ptr<const QUICFrame>)
   {
@@ -258,7 +255,6 @@ class MockQUICStreamManager : public QUICStreamManager
 {
 public:
   MockQUICStreamManager() : QUICStreamManager() {}
-
   // Override
   virtual QUICError
   handle_frame(std::shared_ptr<const QUICFrame> f) override
@@ -295,9 +291,7 @@ public:
   }
 
   bool is_recv_avail_more_than(uint64_t /* size */) override { return true; }
-
   void send_frame(std::unique_ptr<QUICFrame, QUICFrameDeleterFunc> /* frame */) override { return; }
-
 private:
   int _totalFrameCount = 0;
   int _frameCount[256] = {0};
@@ -307,7 +301,6 @@ class MockQUICCongestionController : public QUICCongestionController
 {
 public:
   MockQUICCongestionController() : QUICCongestionController() {}
-
   // Override
   virtual QUICError
   handle_frame(std::shared_ptr<const QUICFrame> f) override
diff --git a/iocore/net/quic/QUICApplication.h b/iocore/net/quic/QUICApplication.h
index 3e2426e..39c1974 100644
--- a/iocore/net/quic/QUICApplication.h
+++ b/iocore/net/quic/QUICApplication.h
@@ -64,6 +64,7 @@ class QUICApplication : public Continuation
 {
 public:
   QUICApplication(QUICConnection *qc);
+  virtual ~QUICApplication(){};
 
   void set_stream(QUICStream *stream);
   bool is_stream_set(QUICStream *stream);
diff --git a/iocore/net/quic/QUICConnection.h b/iocore/net/quic/QUICConnection.h
index 9aad687..0e571ed 100644
--- a/iocore/net/quic/QUICConnection.h
+++ b/iocore/net/quic/QUICConnection.h
@@ -29,15 +29,15 @@
 #include "QUICTransportParameters.h"
 
 class QUICApplication;
+class SSLNextProtocolSet;
 
 class QUICConnection : public QUICPacketTransmitter, public QUICFrameTransmitter, public QUICFrameHandler
 {
 public:
-  virtual uint32_t maximum_quic_packet_size()                                        = 0;
-  virtual uint32_t minimum_quic_packet_size()                                        = 0;
-  virtual uint32_t pmtu()                                                            = 0;
-  virtual void set_transport_parameters(std::unique_ptr<QUICTransportParameters> tp) = 0;
-  virtual const QUICTransportParameters &local_transport_parameters()                = 0;
-  virtual const QUICTransportParameters &remote_transport_parameters()               = 0;
-  virtual void close(QUICError error)                                                = 0;
+  virtual uint32_t maximum_quic_packet_size()     = 0;
+  virtual uint32_t minimum_quic_packet_size()     = 0;
+  virtual uint32_t pmtu()                         = 0;
+  virtual NetVConnectionContext_t direction()     = 0;
+  virtual SSLNextProtocolSet *next_protocol_set() = 0;
+  virtual void close(QUICError error)             = 0;
 };
diff --git a/iocore/net/quic/QUICCrypto.cc b/iocore/net/quic/QUICCrypto.cc
index 53b2799..02872e7 100644
--- a/iocore/net/quic/QUICCrypto.cc
+++ b/iocore/net/quic/QUICCrypto.cc
@@ -83,10 +83,8 @@ QUICPacketProtection::key_phase() const
 //
 // QUICCrypto
 //
-QUICCrypto::QUICCrypto(SSL_CTX *ssl_ctx, NetVConnection *vc) : _netvc_context(vc->get_context())
+QUICCrypto::QUICCrypto(SSL *ssl, NetVConnectionContext_t nvc_ctx) : _ssl(ssl), _netvc_context(nvc_ctx)
 {
-  this->_ssl = SSL_new(ssl_ctx);
-  SSL_set_ex_data(this->_ssl, QUIC::ssl_quic_vc_index, vc);
   if (this->_netvc_context == NET_VCONNECTION_IN) {
     SSL_set_accept_state(this->_ssl);
   } else if (this->_netvc_context == NET_VCONNECTION_OUT) {
@@ -101,7 +99,6 @@ QUICCrypto::QUICCrypto(SSL_CTX *ssl_ctx, NetVConnection *vc) : _netvc_context(vc
 
 QUICCrypto::~QUICCrypto()
 {
-  SSL_free(this->_ssl);
   delete this->_client_pp;
   delete this->_server_pp;
 }
diff --git a/iocore/net/quic/QUICCrypto.h b/iocore/net/quic/QUICCrypto.h
index 5617e61..1eb24ac 100644
--- a/iocore/net/quic/QUICCrypto.h
+++ b/iocore/net/quic/QUICCrypto.h
@@ -38,7 +38,6 @@
 
 struct KeyMaterial {
   KeyMaterial(size_t secret_len, size_t key_len, size_t iv_len) : secret_len(secret_len), key_len(key_len), iv_len(iv_len) {}
-
   uint8_t secret[EVP_MAX_MD_SIZE] = {0};
   uint8_t key[EVP_MAX_KEY_LENGTH] = {0};
   uint8_t iv[EVP_MAX_IV_LENGTH]   = {0};
@@ -65,8 +64,8 @@ private:
 class QUICCrypto
 {
 public:
-  QUICCrypto(SSL_CTX *, NetVConnection *);
-  ~QUICCrypto();
+  QUICCrypto(SSL *, NetVConnectionContext_t);
+  virtual ~QUICCrypto();
 
   bool handshake(uint8_t *out, size_t &out_len, size_t max_out_len, const uint8_t *in, size_t in_len);
   bool is_handshake_finished() const;
diff --git a/iocore/net/quic/QUICFrame.h b/iocore/net/quic/QUICFrame.h
index 8be22bf..79e21b4 100644
--- a/iocore/net/quic/QUICFrame.h
+++ b/iocore/net/quic/QUICFrame.h
@@ -116,7 +116,6 @@ public:
 
       const QUICAckFrame::AckBlock &operator*() const { return this->_current_block; };
       const QUICAckFrame::AckBlock *operator->() const { return &this->_current_block; };
-
       const QUICAckFrame::AckBlock &operator++()
       {
         ++(this->_index);
diff --git a/iocore/net/quic/QUICGlobals.cc b/iocore/net/quic/QUICGlobals.cc
index b8dd77d..cdad89b 100644
--- a/iocore/net/quic/QUICGlobals.cc
+++ b/iocore/net/quic/QUICGlobals.cc
@@ -26,17 +26,18 @@
 #include "P_QUICNetVConnection.h"
 #include "P_SSLNextProtocolSet.h"
 
-int QUIC::ssl_quic_vc_index = -1;
+int QUIC::ssl_quic_qc_index = -1;
+int QUIC::ssl_quic_hs_index = -1;
 
 int
 QUIC::ssl_select_next_protocol(SSL *ssl, const unsigned char **out, unsigned char *outlen, const unsigned char *in, unsigned inlen,
                                void *)
 {
   const unsigned char *npn;
-  unsigned npnsz           = 0;
-  QUICNetVConnection *qnvc = static_cast<QUICNetVConnection *>(SSL_get_ex_data(ssl, QUIC::ssl_quic_vc_index));
+  unsigned npnsz     = 0;
+  QUICConnection *qc = static_cast<QUICConnection *>(SSL_get_ex_data(ssl, QUIC::ssl_quic_qc_index));
 
-  qnvc->next_protocol_set()->advertiseProtocols(&npn, &npnsz);
+  qc->next_protocol_set()->advertiseProtocols(&npn, &npnsz);
   if (SSL_select_next_proto((unsigned char **)out, outlen, npn, npnsz, in, inlen) == OPENSSL_NPN_NEGOTIATED) {
     return SSL_TLSEXT_ERR_OK;
   }
diff --git a/iocore/net/quic/QUICGlobals.h b/iocore/net/quic/QUICGlobals.h
index e1d2b4a..1af0f73 100644
--- a/iocore/net/quic/QUICGlobals.h
+++ b/iocore/net/quic/QUICGlobals.h
@@ -31,9 +31,11 @@ public:
   static void
   init()
   {
-    ssl_quic_vc_index = SSL_get_ex_new_index(0, (void *)"NetVC index", nullptr, nullptr, nullptr);
+    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);
   }
-  static int ssl_quic_vc_index;
+  static int ssl_quic_qc_index;
+  static int ssl_quic_hs_index;
 
   // SSL callbacks
   static int ssl_select_next_protocol(SSL *ssl, const unsigned char **out, unsigned char *outlen, const unsigned char *in,
diff --git a/iocore/net/quic/QUICHandshake.cc b/iocore/net/quic/QUICHandshake.cc
index 8426138..9e8fdfd 100644
--- a/iocore/net/quic/QUICHandshake.cc
+++ b/iocore/net/quic/QUICHandshake.cc
@@ -21,7 +21,11 @@
  *  limitations under the License.
  */
 
+#include "QUICGlobals.h"
 #include "QUICHandshake.h"
+#include "QUICVersionNegotiator.h"
+#include "QUICConfig.h"
+#include "P_SSLNextProtocolSet.h"
 
 #define I_WANNA_DUMP_THIS_BUF(buf, len)                                                                                           \
   {                                                                                                                               \
@@ -48,16 +52,69 @@ const static int UDP_MAXIMUM_PAYLOAD_SIZE = 65527;
 // TODO: fix size
 const static int MAX_HANDSHAKE_MSG_LEN = 65527;
 
-QUICHandshake::QUICHandshake(QUICConnection *qc, QUICCrypto *c) : QUICApplication(qc), _crypto(c)
+QUICHandshake::QUICHandshake(QUICConnection *qc, SSL_CTX *ssl_ctx) : QUICApplication(qc)
 {
+  this->_ssl = SSL_new(ssl_ctx);
+  SSL_set_ex_data(this->_ssl, QUIC::ssl_quic_qc_index, qc);
+  SSL_set_ex_data(this->_ssl, QUIC::ssl_quic_hs_index, this);
+  this->_crypto             = new QUICCrypto(this->_ssl, qc->direction());
+  this->_version_negotiator = new QUICVersionNegotiator();
+
+  this->_load_local_transport_parameters();
+
   SET_HANDLER(&QUICHandshake::state_read_client_hello);
 }
 
+QUICHandshake::~QUICHandshake()
+{
+  SSL_free(this->_ssl);
+}
+
+QUICError
+QUICHandshake::start(const QUICPacket *initial_packet, QUICPacketFactory *packet_factory)
+{
+  // Negotiate version
+  if (this->_version_negotiator->status() == QUICVersionNegotiationStatus::NOT_NEGOTIATED) {
+    if (initial_packet->type() != QUICPacketType::CLIENT_INITIAL) {
+      return QUICError(QUICErrorClass::QUIC_TRANSPORT, QUICErrorCode::QUIC_INTERNAL_ERROR);
+    }
+    if (initial_packet->version()) {
+      if (this->_version_negotiator->negotiate(initial_packet) == QUICVersionNegotiationStatus::NEGOTIATED) {
+        Debug(tag, "Version negotiation succeeded: %x", initial_packet->version());
+        packet_factory->set_version(this->_version_negotiator->negotiated_version());
+      } else {
+        this->_client_qc->transmit_packet(packet_factory->create_version_negotiation_packet(initial_packet));
+        Debug(tag, "Version negotiation failed: %x", initial_packet->version());
+      }
+    } else {
+      return QUICError(QUICErrorClass::QUIC_TRANSPORT, QUICErrorCode::QUIC_INTERNAL_ERROR);
+    }
+  }
+  return QUICError(QUICErrorClass::NONE);
+}
+
+bool
+QUICHandshake::is_version_negotiated()
+{
+  return (this->_version_negotiator->status() == QUICVersionNegotiationStatus::NEGOTIATED);
+}
+
 bool
 QUICHandshake::is_completed()
 {
-  QUICCrypto *crypto = this->_crypto;
-  return crypto->is_handshake_finished();
+  return this->_crypto->is_handshake_finished();
+}
+
+QUICVersion
+QUICHandshake::negotiated_version()
+{
+  return this->_version_negotiator->negotiated_version();
+}
+
+QUICCrypto *
+QUICHandshake::crypto_module()
+{
+  return this->_crypto;
 }
 
 void
@@ -66,6 +123,45 @@ QUICHandshake::negotiated_application_name(const uint8_t **name, unsigned int *l
   SSL_get0_alpn_selected(this->_crypto->ssl_handle(), name, len);
 }
 
+void
+QUICHandshake::set_transport_parameters(std::shared_ptr<QUICTransportParameters> tp)
+{
+  this->_remote_transport_parameters = tp;
+
+  const QUICTransportParametersInClientHello *tp_in_ch =
+    dynamic_cast<const QUICTransportParametersInClientHello *>(this->_remote_transport_parameters.get());
+  if (tp_in_ch) {
+    // Version revalidation
+    if (this->_version_negotiator->revalidate(tp_in_ch) != QUICVersionNegotiationStatus::REVALIDATED) {
+      this->_client_qc->close({QUICErrorClass::QUIC_TRANSPORT, QUICErrorCode::QUIC_VERSION_NEGOTIATION_MISMATCH});
+      Debug(tag, "Enter state_closed");
+      SET_HANDLER(&QUICHandshake::state_closed);
+      return;
+    }
+    Debug(tag, "Version negotiation revalidated: %x", tp_in_ch->negotiated_version());
+    return;
+  }
+
+  const QUICTransportParametersInEncryptedExtensions *tp_in_ee =
+    dynamic_cast<const QUICTransportParametersInEncryptedExtensions *>(this->_remote_transport_parameters.get());
+  if (tp_in_ee) {
+    // TODO Add client side implementation
+    return;
+  }
+}
+
+std::shared_ptr<const QUICTransportParameters>
+QUICHandshake::local_transport_parameters()
+{
+  return this->_local_transport_parameters;
+}
+
+std::shared_ptr<const QUICTransportParameters>
+QUICHandshake::remote_transport_parameters()
+{
+  return this->_remote_transport_parameters;
+}
+
 int
 QUICHandshake::state_read_client_hello(int event, Event *data)
 {
@@ -136,6 +232,35 @@ QUICHandshake::state_closed(int event, void *data)
   return EVENT_CONT;
 }
 
+void
+QUICHandshake::_load_local_transport_parameters()
+{
+  QUICConfig::scoped_config params;
+
+  // MUSTs
+  QUICTransportParametersInEncryptedExtensions *tp = new QUICTransportParametersInEncryptedExtensions();
+
+  tp->add(QUICTransportParameterId::INITIAL_MAX_STREAM_DATA,
+          std::unique_ptr<QUICTransportParameterValue>(
+            new QUICTransportParameterValue(params->initial_max_stream_data(), sizeof(params->initial_max_stream_data()))));
+
+  tp->add(QUICTransportParameterId::INITIAL_MAX_DATA, std::unique_ptr<QUICTransportParameterValue>(new QUICTransportParameterValue(
+                                                        params->initial_max_data(), sizeof(params->initial_max_data()))));
+
+  tp->add(QUICTransportParameterId::INITIAL_MAX_STREAM_ID,
+          std::unique_ptr<QUICTransportParameterValue>(
+            new QUICTransportParameterValue(params->initial_max_stream_id(), sizeof(params->initial_max_stream_id()))));
+
+  tp->add(QUICTransportParameterId::IDLE_TIMEOUT, std::unique_ptr<QUICTransportParameterValue>(new QUICTransportParameterValue(
+                                                    params->no_activity_timeout_in(), sizeof(uint16_t))));
+
+  tp->add_version(QUIC_SUPPORTED_VERSIONS[0]);
+  // MAYs
+  // this->_local_transport_parameters.add(QUICTransportParameterId::TRUNCATE_CONNECTION_ID, {});
+  // this->_local_transport_parameters.add(QUICTransportParameterId::MAX_PACKET_SIZE, {{0x00, 0x00}, 2});
+  this->_local_transport_parameters = std::unique_ptr<QUICTransportParameters>(tp);
+}
+
 QUICError
 QUICHandshake::_process_client_hello()
 {
diff --git a/iocore/net/quic/QUICHandshake.h b/iocore/net/quic/QUICHandshake.h
index 3a501f0..61baa4a 100644
--- a/iocore/net/quic/QUICHandshake.h
+++ b/iocore/net/quic/QUICHandshake.h
@@ -43,21 +43,47 @@
  *  v
  * state_closed()
  */
+
+class QUICVersionNegotiator;
+class SSLNextProtocolSet;
+
 class QUICHandshake : public QUICApplication
 {
 public:
-  QUICHandshake(QUICConnection *qc, QUICCrypto *c);
+  QUICHandshake(QUICConnection *qc, SSL_CTX *ssl_ctx);
+  ~QUICHandshake();
+
+  QUICError start(const QUICPacket *initial_packet, QUICPacketFactory *packet_factory);
 
+  // States
   int state_read_client_hello(int event, Event *data);
   int state_read_client_finished(int event, Event *data);
   int state_address_validation(int event, void *data);
   int state_complete(int event, void *data);
   int state_closed(int event, void *data);
-  bool is_completed();
+
+  // Getters
+  QUICCrypto *crypto_module();
+  QUICVersion negotiated_version();
   void negotiated_application_name(const uint8_t **name, unsigned int *len);
+  std::shared_ptr<const QUICTransportParameters> local_transport_parameters();
+  std::shared_ptr<const QUICTransportParameters> remote_transport_parameters();
+
+  bool is_version_negotiated();
+  bool is_completed();
+
+  void set_transport_parameters(std::shared_ptr<QUICTransportParameters> tp);
 
 private:
-  QUICCrypto *_crypto = nullptr;
+  SSL *_ssl                                                             = nullptr;
+  QUICCrypto *_crypto                                                   = nullptr;
+  std::shared_ptr<QUICTransportParameters> _local_transport_parameters  = nullptr;
+  std::shared_ptr<QUICTransportParameters> _remote_transport_parameters = nullptr;
+
+  QUICVersionNegotiator *_version_negotiator = nullptr;
+
+  void _load_local_transport_parameters();
+
   QUICError _process_client_hello();
   QUICError _process_client_finished();
   QUICError _process_handshake_complete();
diff --git a/iocore/net/quic/QUICStream.h b/iocore/net/quic/QUICStream.h
index 6d35e6d..dfcb2c6 100644
--- a/iocore/net/quic/QUICStream.h
+++ b/iocore/net/quic/QUICStream.h
@@ -44,7 +44,6 @@ class QUICStream : public VConnection
 public:
   QUICStream() : VConnection(nullptr) {}
   ~QUICStream() {}
-
   void init(QUICStreamManager *manager, QUICFrameTransmitter *tx, uint32_t id, uint64_t recv_max_stream_data = 0,
             uint64_t send_max_stream_data = 0);
   void start();
diff --git a/iocore/net/quic/QUICStreamManager.cc b/iocore/net/quic/QUICStreamManager.cc
index 4ac3b94..0fdb968 100644
--- a/iocore/net/quic/QUICStreamManager.cc
+++ b/iocore/net/quic/QUICStreamManager.cc
@@ -32,14 +32,8 @@ const static char *tag = "quic_stream_manager";
 ClassAllocator<QUICStreamManager> quicStreamManagerAllocator("quicStreamManagerAllocator");
 ClassAllocator<QUICStream> quicStreamAllocator("quicStreamAllocator");
 
-int
-QUICStreamManager::init(QUICFrameTransmitter *tx, QUICConnection *qc, QUICApplicationMap *app_map)
+QUICStreamManager::QUICStreamManager(QUICFrameTransmitter *tx, QUICApplicationMap *app_map) : _tx(tx), _app_map(app_map)
 {
-  this->_tx      = tx;
-  this->_qc      = qc;
-  this->_app_map = app_map;
-
-  return 0;
 }
 
 std::vector<QUICFrameType>
@@ -50,15 +44,19 @@ QUICStreamManager::interests()
 }
 
 void
-QUICStreamManager::init_flow_control_params(const QUICTransportParameters &local_tp, const QUICTransportParameters &remote_tp)
+QUICStreamManager::init_flow_control_params(std::shared_ptr<const QUICTransportParameters> local_tp,
+                                            std::shared_ptr<const QUICTransportParameters> remote_tp)
 {
+  this->_local_tp  = local_tp;
+  this->_remote_tp = remote_tp;
+
   // Connection level
-  this->_recv_max_data = QUICMaximumData(local_tp.initial_max_data());
-  this->_send_max_data = QUICMaximumData(remote_tp.initial_max_data());
+  this->_recv_max_data = QUICMaximumData(local_tp->initial_max_data());
+  this->_send_max_data = QUICMaximumData(remote_tp->initial_max_data());
 
   // Setup a stream for Handshake
   QUICStream *stream = this->_find_stream(STREAM_ID_FOR_HANDSHAKE);
-  stream->init_flow_control_params(local_tp.initial_max_stream_data(), remote_tp.initial_max_stream_data());
+  stream->init_flow_control_params(local_tp->initial_max_stream_data(), remote_tp->initial_max_stream_data());
 }
 
 QUICError
@@ -106,7 +104,7 @@ void
 QUICStreamManager::slide_recv_max_data()
 {
   // TODO: How much should this be increased?
-  this->_recv_max_data += this->_qc->local_transport_parameters().initial_max_data();
+  this->_recv_max_data += this->_local_tp->initial_max_data();
   this->send_frame(QUICFrameFactory::create_max_data_frame(this->_recv_max_data));
 }
 
@@ -221,8 +219,8 @@ QUICStreamManager::_find_or_create_stream(QUICStreamId stream_id)
       // XXX rece/send max_stream_data are going to be set by init_flow_control_params()
       stream->init(this, this->_tx, stream_id);
     } else {
-      const QUICTransportParameters &local_tp  = this->_qc->local_transport_parameters();
-      const QUICTransportParameters &remote_tp = this->_qc->remote_transport_parameters();
+      const QUICTransportParameters &local_tp  = *this->_local_tp;
+      const QUICTransportParameters &remote_tp = *this->_remote_tp;
 
       // TODO: check local_tp and remote_tp is initialized
       stream->init(this, this->_tx, stream_id, local_tp.initial_max_stream_data(), remote_tp.initial_max_stream_data());
diff --git a/iocore/net/quic/QUICStreamManager.h b/iocore/net/quic/QUICStreamManager.h
index fbfc6e9..17b44b4 100644
--- a/iocore/net/quic/QUICStreamManager.h
+++ b/iocore/net/quic/QUICStreamManager.h
@@ -36,8 +36,7 @@ class QUICStreamManager : public QUICFrameHandler
 {
 public:
   QUICStreamManager(){};
-
-  int init(QUICFrameTransmitter *tx, QUICConnection *qc, QUICApplicationMap *app_map);
+  QUICStreamManager(QUICFrameTransmitter *tx, QUICApplicationMap *app_map);
   virtual std::vector<QUICFrameType> interests() override;
   virtual QUICError handle_frame(std::shared_ptr<const QUICFrame>) override;
   virtual void send_frame(std::unique_ptr<QUICFrame, QUICFrameDeleterFunc> frame);
@@ -46,7 +45,8 @@ public:
   virtual bool is_recv_avail_more_than(uint64_t size);
   void add_recv_total_offset(uint64_t delta);
   void slide_recv_max_data();
-  void init_flow_control_params(const QUICTransportParameters &local_tp, const QUICTransportParameters &remote_tp);
+  void init_flow_control_params(std::shared_ptr<const QUICTransportParameters> local_tp,
+                                std::shared_ptr<const QUICTransportParameters> remote_tp);
   uint64_t recv_max_data() const;
   uint64_t send_max_data() const;
   uint64_t recv_total_offset() const;
@@ -62,9 +62,10 @@ private:
   QUICError _handle_frame(std::shared_ptr<const QUICMaxStreamDataFrame>);
   QUICError _handle_frame(std::shared_ptr<const QUICStreamBlockedFrame>);
 
-  QUICApplicationMap *_app_map = nullptr;
-  QUICFrameTransmitter *_tx    = nullptr;
-  QUICConnection *_qc          = nullptr;
+  QUICFrameTransmitter *_tx                                 = nullptr;
+  QUICApplicationMap *_app_map                              = nullptr;
+  std::shared_ptr<const QUICTransportParameters> _local_tp  = nullptr;
+  std::shared_ptr<const QUICTransportParameters> _remote_tp = nullptr;
 
   QUICMaximumData _recv_max_data = {0};
   QUICMaximumData _send_max_data = {0};
diff --git a/iocore/net/quic/QUICTransportParameters.cc b/iocore/net/quic/QUICTransportParameters.cc
index 7365084..53894c8 100644
--- a/iocore/net/quic/QUICTransportParameters.cc
+++ b/iocore/net/quic/QUICTransportParameters.cc
@@ -235,10 +235,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)
 {
-  QUICConnection *qc =
-    static_cast<QUICConnection *>(static_cast<QUICNetVConnection *>(SSL_get_ex_data(s, QUIC::ssl_quic_vc_index)));
-  *out = reinterpret_cast<const unsigned char *>(ats_malloc(TRANSPORT_PARAMETERS_MAXIMUM_SIZE));
-  qc->local_transport_parameters().store(const_cast<uint8_t *>(*out), reinterpret_cast<uint16_t *>(outlen));
+  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));
 
   return 1;
 }
@@ -253,11 +252,8 @@ 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)
 {
-  QUICConnection *qc =
-    static_cast<QUICConnection *>(static_cast<QUICNetVConnection *>(SSL_get_ex_data(s, QUIC::ssl_quic_vc_index)));
-  QUICTransportParametersInClientHello *tp     = new QUICTransportParametersInClientHello(in, inlen);
-  std::unique_ptr<QUICTransportParameters> utp = std::unique_ptr<QUICTransportParameters>(tp);
-  qc->set_transport_parameters(std::move(utp));
+  QUICHandshake *hs = static_cast<QUICHandshake *>(SSL_get_ex_data(s, QUIC::ssl_quic_hs_index));
+  hs->set_transport_parameters(std::make_shared<QUICTransportParametersInClientHello>(in, inlen));
 
   return 1;
 }
diff --git a/iocore/net/quic/QUICTransportParameters.h b/iocore/net/quic/QUICTransportParameters.h
index a32261f..a84aba8 100644
--- a/iocore/net/quic/QUICTransportParameters.h
+++ b/iocore/net/quic/QUICTransportParameters.h
@@ -43,7 +43,6 @@ public:
   };
 
   explicit operator bool() const { return true; }
-
   bool
   operator==(const QUICTransportParameterId &x) const
   {
@@ -57,7 +56,6 @@ public:
   }
 
   operator uint16_t() const { return _id; };
-
   QUICTransportParameterId() : _id(0){};
   QUICTransportParameterId(uint16_t id) : _id(id){};
 
diff --git a/iocore/net/quic/QUICTypes.h b/iocore/net/quic/QUICTypes.h
index 83a0500..aabf1d9 100644
--- a/iocore/net/quic/QUICTypes.h
+++ b/iocore/net/quic/QUICTypes.h
@@ -153,9 +153,7 @@ class QUICConnectionId
 {
 public:
   explicit operator bool() const { return true; }
-
   operator uint64_t() const { return _id; };
-
   QUICConnectionId() { this->randomize(); };
   QUICConnectionId(uint64_t id) : _id(id){};
 
@@ -174,7 +172,6 @@ class QUICMaximumData
 {
 public:
   QUICMaximumData(uint64_t d) : _data(d) {}
-
   bool
   operator>(uint64_t r) const
   {
@@ -220,7 +217,6 @@ public:
   }
 
   operator uint64_t() const { return _data; }
-
 private:
   uint64_t _data = 0; // in units of 1024 octets
 };
diff --git a/iocore/net/quic/QUICVersionNegotiator.cc b/iocore/net/quic/QUICVersionNegotiator.cc
index d08ee1b..aa91c0c 100644
--- a/iocore/net/quic/QUICVersionNegotiator.cc
+++ b/iocore/net/quic/QUICVersionNegotiator.cc
@@ -22,9 +22,7 @@
  */
 
 #include "QUICVersionNegotiator.h"
-
-QUICVersionNegotiator::QUICVersionNegotiator(QUICPacketFactory *packet_factory, QUICPacketTransmitter *tx)
-  : _packet_factory(packet_factory), _tx(tx){};
+#include "QUICTransportParameters.h"
 
 QUICVersionNegotiationStatus
 QUICVersionNegotiator::status()
@@ -38,16 +36,25 @@ QUICVersionNegotiator::negotiate(const QUICPacket *initial_packet)
   if (this->_is_supported(initial_packet->version())) {
     this->_status             = QUICVersionNegotiationStatus::NEGOTIATED;
     this->_negotiated_version = initial_packet->version();
-  } else {
-    this->_tx->transmit_packet(this->_packet_factory->create_version_negotiation_packet(initial_packet));
   }
   return this->_status;
 }
 
 QUICVersionNegotiationStatus
-QUICVersionNegotiator::revalidate(QUICVersion version)
+QUICVersionNegotiator::revalidate(const QUICTransportParametersInClientHello *tp)
 {
+  QUICVersion version = tp->negotiated_version();
   if (this->_negotiated_version == version) {
+    if (tp->negotiated_version() != tp->initial_version()) {
+      // FIXME Check initial_version
+      /* If the initial version is different from the negotiated_version, a
+       * stateless server MUST check that it would have sent a version
+       * negotiation packet if it had received a packet with the indicated
+       * initial_version. (Draft-04 7.3.4. Version Negotiation Validation)
+       */
+      this->_status             = QUICVersionNegotiationStatus::FAILED;
+      this->_negotiated_version = 0;
+    }
     this->_status = QUICVersionNegotiationStatus::REVALIDATED;
   } else {
     this->_status             = QUICVersionNegotiationStatus::FAILED;
diff --git a/iocore/net/quic/QUICVersionNegotiator.h b/iocore/net/quic/QUICVersionNegotiator.h
index fd37693..6b7e075 100644
--- a/iocore/net/quic/QUICVersionNegotiator.h
+++ b/iocore/net/quic/QUICVersionNegotiator.h
@@ -24,7 +24,8 @@
 #pragma once
 
 #include "QUICTypes.h"
-#include "QUICPacketTransmitter.h"
+#include "QUICPacket.h"
+#include "QUICTransportParameters.h"
 
 /**
  * @brief Abstruct QUIC Application Class
@@ -33,16 +34,13 @@
 class QUICVersionNegotiator
 {
 public:
-  QUICVersionNegotiator(QUICPacketFactory *packet_factory, QUICPacketTransmitter *tx);
   QUICVersionNegotiationStatus status();
   QUICVersionNegotiationStatus negotiate(const QUICPacket *initial_packet);
-  QUICVersionNegotiationStatus revalidate(QUICVersion version);
+  QUICVersionNegotiationStatus revalidate(const QUICTransportParametersInClientHello *tp);
   QUICVersion negotiated_version();
 
 private:
   QUICVersion _negotiated_version      = 0;
-  QUICPacketFactory *_packet_factory   = nullptr;
-  QUICPacketTransmitter *_tx           = nullptr;
   QUICVersionNegotiationStatus _status = QUICVersionNegotiationStatus::NOT_NEGOTIATED;
 
   bool _is_supported(QUICVersion version);

-- 
To stop receiving notification emails like this one, please contact
['"commits@trafficserver.apache.org" <co...@trafficserver.apache.org>'].