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/11/21 06:21:04 UTC

[trafficserver] branch quic-latest updated: Fix cleartext packet protection

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 0173ca9  Fix cleartext packet protection
0173ca9 is described below

commit 0173ca9d97b49f61c87a17b9bf57d40500798074
Author: Masakazu Kitajo <ma...@apache.org>
AuthorDate: Tue Nov 21 15:17:30 2017 +0900

    Fix cleartext packet protection
---
 iocore/net/P_QUICNetVConnection.h                  |   4 +-
 iocore/net/QUICNetVConnection.cc                   |  19 +++-
 iocore/net/QUICPacketHandler.cc                    |   2 +-
 iocore/net/quic/Mock.h                             |  52 +++++++++
 iocore/net/quic/QUICAckFrameCreator.cc             |   1 +
 iocore/net/quic/QUICCongestionController.cc        |   1 +
 iocore/net/quic/QUICConnection.h                   |   5 +-
 iocore/net/quic/QUICCrypto.cc                      |  26 ++---
 iocore/net/quic/QUICCrypto.h                       |  68 ++----------
 iocore/net/quic/{QUICCrypto.h => QUICCryptoTls.h}  |  39 ++-----
 iocore/net/quic/QUICCrypto_boringssl.cc            |  28 ++---
 iocore/net/quic/QUICCrypto_openssl.cc              |  18 +--
 iocore/net/quic/QUICHandshake.cc                   |  11 +-
 iocore/net/quic/QUICPacket.cc                      | 121 ++++++++++++---------
 iocore/net/quic/QUICPacket.h                       |  17 ++-
 iocore/net/quic/QUICPacketTransmitter.h            |   1 +
 iocore/net/quic/QUICTypes.h                        |   2 +-
 iocore/net/quic/test/test_QUICCrypto.cc            |  10 +-
 iocore/net/quic/test/test_QUICLossDetector.cc      |   7 +-
 iocore/net/quic/test/test_QUICPacketFactory.cc     |   9 +-
 iocore/net/quic/test/test_QUICVersionNegotiator.cc |   2 +
 21 files changed, 242 insertions(+), 201 deletions(-)

diff --git a/iocore/net/P_QUICNetVConnection.h b/iocore/net/P_QUICNetVConnection.h
index b59bc03..bba69ad 100644
--- a/iocore/net/P_QUICNetVConnection.h
+++ b/iocore/net/P_QUICNetVConnection.h
@@ -134,7 +134,7 @@ class QUICNetVConnection : public UnixNetVConnection, public QUICConnection
 
 public:
   QUICNetVConnection() {}
-  void init(UDPConnection *, QUICPacketHandler *);
+  void init(QUICConnectionId cid, UDPConnection *, QUICPacketHandler *);
 
   // UnixNetVConnection
   void reenable(VIO *vio) override;
@@ -163,6 +163,7 @@ public:
   bool is_closed();
 
   // QUICConnection
+  QUICConnectionId original_connection_id() override;
   QUICConnectionId connection_id() override;
   uint32_t maximum_quic_packet_size() override;
   uint32_t minimum_quic_packet_size() override;
@@ -189,6 +190,7 @@ public:
 
 private:
   std::random_device _rnd;
+  QUICConnectionId _original_quic_connection_id;
   QUICConnectionId _quic_connection_id;
   QUICPacketNumber _largest_received_packet_number = 0;
   UDPConnection *_udp_con                          = nullptr;
diff --git a/iocore/net/QUICNetVConnection.cc b/iocore/net/QUICNetVConnection.cc
index 7c7db3a..cc226fa 100644
--- a/iocore/net/QUICNetVConnection.cc
+++ b/iocore/net/QUICNetVConnection.cc
@@ -46,7 +46,7 @@
 #define DebugQUICCon(fmt, ...) \
   Debug("quic_net", "[%" PRIx64 "] " fmt, static_cast<uint64_t>(this->_quic_connection_id), ##__VA_ARGS__)
 
-static constexpr uint32_t MAX_PACKET_OVERHEAD                = 25; // Max long header len(17) + FNV-1a hash len(8)
+static constexpr uint32_t MAX_PACKET_OVERHEAD                = 17; // Max long header len(17)
 static constexpr uint32_t MAX_STREAM_FRAME_OVERHEAD          = 15;
 static constexpr uint32_t MINIMUM_INITIAL_CLIENT_PACKET_SIZE = 1200;
 
@@ -54,13 +54,14 @@ ClassAllocator<QUICNetVConnection> quicNetVCAllocator("quicNetVCAllocator");
 
 // XXX This might be called on ET_UDP thread
 void
-QUICNetVConnection::init(UDPConnection *udp_con, QUICPacketHandler *packet_handler)
+QUICNetVConnection::init(QUICConnectionId original_cid, UDPConnection *udp_con, QUICPacketHandler *packet_handler)
 {
   SET_HANDLER((NetVConnHandler)&QUICNetVConnection::state_pre_handshake);
-  this->_packet_transmitter_mutex = new_ProxyMutex();
-  this->_frame_transmitter_mutex  = new_ProxyMutex();
-  this->_udp_con                  = udp_con;
-  this->_packet_handler           = packet_handler;
+  this->_packet_transmitter_mutex    = new_ProxyMutex();
+  this->_frame_transmitter_mutex     = new_ProxyMutex();
+  this->_udp_con                     = udp_con;
+  this->_packet_handler              = packet_handler;
+  this->_original_quic_connection_id = original_cid;
   this->_quic_connection_id.randomize();
 }
 
@@ -152,6 +153,12 @@ QUICNetVConnection::reenable(VIO *vio)
 }
 
 QUICConnectionId
+QUICNetVConnection::original_connection_id()
+{
+  return this->_original_quic_connection_id;
+}
+
+QUICConnectionId
 QUICNetVConnection::connection_id()
 {
   return this->_quic_connection_id;
diff --git a/iocore/net/QUICPacketHandler.cc b/iocore/net/QUICPacketHandler.cc
index aff1f87..8dc8104 100644
--- a/iocore/net/QUICPacketHandler.cc
+++ b/iocore/net/QUICPacketHandler.cc
@@ -149,7 +149,7 @@ QUICPacketHandler::_recv_packet(int event, UDPPacket *udpPacket)
     // Create a new NetVConnection
     vc =
       static_cast<QUICNetVConnection *>(getNetProcessor()->allocate_vc(((UnixUDPConnection *)udpPacket->getConnection())->ethread));
-    vc->init(udpPacket->getConnection(), this);
+    vc->init(cid, udpPacket->getConnection(), this);
     vc->id = net_next_connection_number();
     vc->con.move(con);
     vc->submit_time = Thread::get_hrtime();
diff --git a/iocore/net/quic/Mock.h b/iocore/net/quic/Mock.h
index 22ba21d..3050a43 100644
--- a/iocore/net/quic/Mock.h
+++ b/iocore/net/quic/Mock.h
@@ -151,6 +151,12 @@ public:
     return 0;
   }
 
+  QUICConnectionId
+  original_connection_id() override
+  {
+    return 0;
+  }
+
   void
   transmit_packet(QUICPacketUPtr packet) override
   {
@@ -393,6 +399,52 @@ public:
   }
 };
 
+class MockQUICCrypto : public QUICCrypto
+{
+public:
+  MockQUICCrypto() : QUICCrypto() {}
+
+  bool
+  handshake(uint8_t *out, size_t &out_len, size_t max_out_len, const uint8_t *in, size_t in_len) override
+  {
+    return true;
+  }
+
+  bool
+  is_handshake_finished() const override
+  {
+    return true;
+  }
+
+  int
+  initialize_key_materials(QUICConnectionId cid) override
+  {
+    return 0;
+  }
+
+  int
+  update_key_materials() override
+  {
+    return 0;
+  }
+
+  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 override
+  {
+    memcpy(cipher, plain, plain_len);
+    return true;
+  }
+
+  bool
+  decrypt(uint8_t *plain, size_t &plain_len, size_t max_plain_len, const uint8_t *cipher, size_t cipher_len, uint64_t pkt_num,
+          const uint8_t *ad, size_t ad_len, QUICKeyPhase phase) const override
+  {
+    memcpy(plain, cipher, cipher_len);
+    return true;
+  }
+};
+
 void NetVConnection::cancel_OOB(){};
 Action *
 NetVConnection::send_OOB(Continuation *, char *, int)
diff --git a/iocore/net/quic/QUICAckFrameCreator.cc b/iocore/net/quic/QUICAckFrameCreator.cc
index e3988f7..f352d83 100644
--- a/iocore/net/quic/QUICAckFrameCreator.cc
+++ b/iocore/net/quic/QUICAckFrameCreator.cc
@@ -21,6 +21,7 @@
  *  limitations under the License.
  */
 
+#include "I_EventSystem.h"
 #include "QUICAckFrameCreator.h"
 #include <algorithm>
 
diff --git a/iocore/net/quic/QUICCongestionController.cc b/iocore/net/quic/QUICCongestionController.cc
index 4a0ea79..d06902b 100644
--- a/iocore/net/quic/QUICCongestionController.cc
+++ b/iocore/net/quic/QUICCongestionController.cc
@@ -21,6 +21,7 @@
  *  limitations under the License.
  */
 
+#include <ts/Diags.h>
 #include <QUICCongestionController.h>
 
 static constexpr char tag[] = "quic_congestion_controller";
diff --git a/iocore/net/quic/QUICConnection.h b/iocore/net/quic/QUICConnection.h
index 93634ad..e236c82 100644
--- a/iocore/net/quic/QUICConnection.h
+++ b/iocore/net/quic/QUICConnection.h
@@ -23,6 +23,8 @@
 
 #pragma once
 
+#include "I_EventSystem.h"
+#include "I_NetVConnection.h"
 #include "QUICPacketTransmitter.h"
 #include "QUICFrameTransmitter.h"
 #include "QUICFrameHandler.h"
@@ -35,7 +37,8 @@ class SSLNextProtocolSet;
 class QUICConnection : public QUICPacketTransmitter, public QUICFrameTransmitter, public QUICFrameHandler
 {
 public:
-  virtual QUICConnectionId connection_id() = 0;
+  virtual QUICConnectionId original_connection_id() = 0;
+  virtual QUICConnectionId connection_id()          = 0;
 
   /*
    * Retruns the maximum packet size at the time called
diff --git a/iocore/net/quic/QUICCrypto.cc b/iocore/net/quic/QUICCrypto.cc
index cfe9d4b..9041ad5 100644
--- a/iocore/net/quic/QUICCrypto.cc
+++ b/iocore/net/quic/QUICCrypto.cc
@@ -21,7 +21,7 @@
  *  limitations under the License.
  */
 #include "QUICGlobals.h"
-#include "QUICCrypto.h"
+#include "QUICCryptoTls.h"
 
 #include <openssl/err.h>
 #include <openssl/ssl.h>
@@ -84,7 +84,7 @@ QUICPacketProtection::key_phase() const
 //
 // QUICCrypto
 //
-QUICCrypto::QUICCrypto(SSL *ssl, NetVConnectionContext_t nvc_ctx) : _ssl(ssl), _netvc_context(nvc_ctx)
+QUICCryptoTls::QUICCryptoTls(SSL *ssl, NetVConnectionContext_t nvc_ctx) : QUICCrypto(), _ssl(ssl), _netvc_context(nvc_ctx)
 {
   if (this->_netvc_context == NET_VCONNECTION_IN) {
     SSL_set_accept_state(this->_ssl);
@@ -98,14 +98,14 @@ QUICCrypto::QUICCrypto(SSL *ssl, NetVConnectionContext_t nvc_ctx) : _ssl(ssl), _
   this->_server_pp = new QUICPacketProtection();
 }
 
-QUICCrypto::~QUICCrypto()
+QUICCryptoTls::~QUICCryptoTls()
 {
   delete this->_client_pp;
   delete this->_server_pp;
 }
 
 bool
-QUICCrypto::handshake(uint8_t *out, size_t &out_len, size_t max_out_len, const uint8_t *in, size_t in_len)
+QUICCryptoTls::handshake(uint8_t *out, size_t &out_len, size_t max_out_len, const uint8_t *in, size_t in_len)
 {
   ink_assert(this->_ssl != nullptr);
 
@@ -153,13 +153,13 @@ QUICCrypto::handshake(uint8_t *out, size_t &out_len, size_t max_out_len, const u
 }
 
 bool
-QUICCrypto::is_handshake_finished() const
+QUICCryptoTls::is_handshake_finished() const
 {
   return SSL_is_init_finished(this->_ssl);
 }
 
 int
-QUICCrypto::initialize_key_materials(QUICConnectionId cid)
+QUICCryptoTls::initialize_key_materials(QUICConnectionId cid)
 {
   // Generate keys
   std::unique_ptr<KeyMaterial> km;
@@ -175,7 +175,7 @@ QUICCrypto::initialize_key_materials(QUICConnectionId cid)
 }
 
 int
-QUICCrypto::update_key_materials()
+QUICCryptoTls::update_key_materials()
 {
   ink_assert(this->is_handshake_finished());
   // Switch key phase
@@ -206,14 +206,14 @@ QUICCrypto::update_key_materials()
 }
 
 SSL *
-QUICCrypto::ssl_handle()
+QUICCryptoTls::ssl_handle()
 {
   return this->_ssl;
 }
 
 bool
-QUICCrypto::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
+QUICCryptoTls::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
 {
   QUICPacketProtection *pp = nullptr;
 
@@ -236,8 +236,8 @@ QUICCrypto::encrypt(uint8_t *cipher, size_t &cipher_len, size_t max_cipher_len,
 }
 
 bool
-QUICCrypto::decrypt(uint8_t *plain, size_t &plain_len, size_t max_plain_len, const uint8_t *cipher, size_t cipher_len,
-                    uint64_t pkt_num, const uint8_t *ad, size_t ad_len, QUICKeyPhase phase) const
+QUICCryptoTls::decrypt(uint8_t *plain, size_t &plain_len, size_t max_plain_len, const uint8_t *cipher, size_t cipher_len,
+                       uint64_t pkt_num, const uint8_t *ad, size_t ad_len, QUICKeyPhase phase) const
 {
   QUICPacketProtection *pp = nullptr;
 
@@ -274,7 +274,7 @@ QUICCrypto::decrypt(uint8_t *plain, size_t &plain_len, size_t max_plain_len, con
  *
  */
 void
-QUICCrypto::_gen_nonce(uint8_t *nonce, size_t &nonce_len, uint64_t pkt_num, const uint8_t *iv, size_t iv_len) const
+QUICCryptoTls::_gen_nonce(uint8_t *nonce, size_t &nonce_len, uint64_t pkt_num, const uint8_t *iv, size_t iv_len) const
 {
   nonce_len = iv_len;
   memcpy(nonce, iv, iv_len);
diff --git a/iocore/net/quic/QUICCrypto.h b/iocore/net/quic/QUICCrypto.h
index 62f6db8..5c57083 100644
--- a/iocore/net/quic/QUICCrypto.h
+++ b/iocore/net/quic/QUICCrypto.h
@@ -24,17 +24,6 @@
 #pragma once
 
 #include "QUICKeyGenerator.h"
-#include <openssl/ssl.h>
-
-#ifdef OPENSSL_IS_BORINGSSL
-#include <openssl/digest.h>
-#include <openssl/cipher.h>
-#else
-#include <openssl/evp.h>
-#endif
-
-#include "I_EventSystem.h"
-#include "I_NetVConnection.h"
 #include "QUICTypes.h"
 
 class QUICPacketProtection
@@ -56,50 +45,15 @@ private:
 class QUICCrypto
 {
 public:
-  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;
-  int initialize_key_materials(QUICConnectionId cid);
-  int update_key_materials();
-  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;
-  bool decrypt(uint8_t *plain, size_t &plain_len, size_t max_plain_len, const uint8_t *cipher, size_t cipher_len, uint64_t pkt_num,
-               const uint8_t *ad, size_t ad_len, QUICKeyPhase phase) const;
-  int update_client_keymaterial();
-  int update_server_keymaterial();
-
-  // FIXME SSL handle should not be exported
-  SSL *ssl_handle();
-
-private:
-  QUICKeyGenerator _keygen_for_client = QUICKeyGenerator(QUICKeyGenerator::Context::CLIENT);
-  QUICKeyGenerator _keygen_for_server = QUICKeyGenerator(QUICKeyGenerator::Context::SERVER);
-  void _gen_nonce(uint8_t *nonce, size_t &nonce_len, uint64_t pkt_num, const uint8_t *iv, size_t iv_len) const;
-#ifdef OPENSSL_IS_BORINGSSL
-  const EVP_AEAD *_get_evp_aead() const;
-  size_t _get_aead_key_len(const EVP_AEAD *aead) const;
-  size_t _get_aead_nonce_len(const EVP_AEAD *aead) const;
-#else
-  const EVP_CIPHER *_get_evp_aead() const;
-  size_t _get_aead_key_len(const EVP_CIPHER *aead) const;
-  size_t _get_aead_nonce_len(const EVP_CIPHER *aead) const;
-#endif // OPENSSL_IS_BORINGSSL
-  size_t _get_aead_tag_len() const;
-
-  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, const KeyMaterial &km, size_t tag_len) const;
-  bool _decrypt(uint8_t *plain, size_t &plain_len, size_t max_plain_len, const uint8_t *cipher, size_t cipher_len, uint64_t pkt_num,
-                const uint8_t *ad, size_t ad_len, const KeyMaterial &km, size_t tag_len) const;
-
-  SSL *_ssl = nullptr;
-#ifdef OPENSSL_IS_BORINGSSL
-  const EVP_AEAD *_aead = nullptr;
-#else
-  const EVP_CIPHER *_aead = nullptr;
-#endif // OPENSSL_IS_BORINGSSL
-  QUICPacketProtection *_client_pp       = nullptr;
-  QUICPacketProtection *_server_pp       = nullptr;
-  NetVConnectionContext_t _netvc_context = NET_VCONNECTION_UNSET;
+  QUICCrypto(){};
+  virtual ~QUICCrypto(){};
+
+  virtual bool handshake(uint8_t *out, size_t &out_len, size_t max_out_len, const uint8_t *in, size_t in_len) = 0;
+  virtual bool is_handshake_finished() const                 = 0;
+  virtual int initialize_key_materials(QUICConnectionId cid) = 0;
+  virtual int update_key_materials()                         = 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,
+                       uint64_t pkt_num, const uint8_t *ad, size_t ad_len, QUICKeyPhase phase) const = 0;
 };
diff --git a/iocore/net/quic/QUICCrypto.h b/iocore/net/quic/QUICCryptoTls.h
similarity index 77%
copy from iocore/net/quic/QUICCrypto.h
copy to iocore/net/quic/QUICCryptoTls.h
index 62f6db8..0e41793 100644
--- a/iocore/net/quic/QUICCrypto.h
+++ b/iocore/net/quic/QUICCryptoTls.h
@@ -23,7 +23,6 @@
 
 #pragma once
 
-#include "QUICKeyGenerator.h"
 #include <openssl/ssl.h>
 
 #ifdef OPENSSL_IS_BORINGSSL
@@ -35,40 +34,22 @@
 
 #include "I_EventSystem.h"
 #include "I_NetVConnection.h"
-#include "QUICTypes.h"
+#include "QUICCrypto.h"
 
-class QUICPacketProtection
+class QUICCryptoTls : public QUICCrypto
 {
 public:
-  QUICPacketProtection(){};
-  ~QUICPacketProtection();
-  void set_key(std::unique_ptr<KeyMaterial> km, QUICKeyPhase phase);
-  const KeyMaterial &get_key(QUICKeyPhase phase) const;
-  QUICKeyPhase key_phase() const;
+  QUICCryptoTls(SSL *ssl, NetVConnectionContext_t nvc_ctx);
+  ~QUICCryptoTls();
 
-private:
-  std::unique_ptr<KeyMaterial> _cleartext_key = nullptr;
-  std::unique_ptr<KeyMaterial> _phase_0_key   = nullptr;
-  std::unique_ptr<KeyMaterial> _phase_1_key   = nullptr;
-  QUICKeyPhase _key_phase                     = QUICKeyPhase::CLEARTEXT;
-};
-
-class QUICCrypto
-{
-public:
-  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;
-  int initialize_key_materials(QUICConnectionId cid);
-  int update_key_materials();
+  bool handshake(uint8_t *out, size_t &out_len, size_t max_out_len, const uint8_t *in, size_t in_len) override;
+  bool is_handshake_finished() const override;
+  int initialize_key_materials(QUICConnectionId cid) override;
+  int update_key_materials() override;
   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;
+               const uint8_t *ad, size_t ad_len, QUICKeyPhase phase) const override;
   bool decrypt(uint8_t *plain, size_t &plain_len, size_t max_plain_len, const uint8_t *cipher, size_t cipher_len, uint64_t pkt_num,
-               const uint8_t *ad, size_t ad_len, QUICKeyPhase phase) const;
-  int update_client_keymaterial();
-  int update_server_keymaterial();
+               const uint8_t *ad, size_t ad_len, QUICKeyPhase phase) const override;
 
   // FIXME SSL handle should not be exported
   SSL *ssl_handle();
diff --git a/iocore/net/quic/QUICCrypto_boringssl.cc b/iocore/net/quic/QUICCrypto_boringssl.cc
index 6b50b6b..0d9ff5b 100644
--- a/iocore/net/quic/QUICCrypto_boringssl.cc
+++ b/iocore/net/quic/QUICCrypto_boringssl.cc
@@ -20,7 +20,7 @@
  *  See the License for the specific language governing permissions and
  *  limitations under the License.
  */
-#include "QUICCrypto.h"
+#include "QUICCryptoTls.h"
 
 #include <openssl/base.h>
 #include <openssl/err.h>
@@ -32,7 +32,7 @@
 static constexpr char tag[] = "quic_crypto";
 
 const EVP_AEAD *
-QUICCrypto::_get_evp_aead(const SSL_CIPHER *cipher) const
+QUICCryptoTls::_get_evp_aead(const SSL_CIPHER *cipher) const
 {
   ink_assert(SSL_CIPHER_is_AEAD(cipher));
 
@@ -51,7 +51,7 @@ QUICCrypto::_get_evp_aead(const SSL_CIPHER *cipher) const
 
 // SSL_HANDSHAKE_MAC_SHA256, SSL_HANDSHAKE_MAC_SHA384 are defind in `ssl/internal.h` of BoringSSL
 const EVP_MD *
-QUICCrypto::_get_handshake_digest(const SSL_CIPHER *cipher) const
+QUICCryptoTls::_get_handshake_digest(const SSL_CIPHER *cipher) const
 {
   switch (cipher->algorithm_prf) {
   case 0x2:
@@ -66,26 +66,26 @@ QUICCrypto::_get_handshake_digest(const SSL_CIPHER *cipher) const
 }
 
 size_t
-QUICCrypto::_get_aead_tag_len(const SSL_CIPHER * /* cipher */) const
+QUICCryptoTls::_get_aead_tag_len(const SSL_CIPHER * /* cipher */) const
 {
   return EVP_AEAD_DEFAULT_TAG_LENGTH;
 }
 
 size_t
-QUICCrypto::_get_aead_key_len(const EVP_AEAD *aead) const
+QUICCryptoTls::_get_aead_key_len(const EVP_AEAD *aead) const
 {
   return EVP_AEAD_key_length(aead);
 }
 
 size_t
-QUICCrypto::_get_aead_nonce_len(const EVP_AEAD *aead) const
+QUICCryptoTls::_get_aead_nonce_len(const EVP_AEAD *aead) const
 {
   return EVP_AEAD_nonce_length(aead);
 }
 
 int
-QUICCrypto::_hkdf_expand_label(uint8_t *dst, size_t dst_len, const uint8_t *secret, size_t secret_len, const char *label,
-                               size_t label_len, const EVP_MD *digest) const
+QUICCryptoTls::_hkdf_expand_label(uint8_t *dst, size_t dst_len, const uint8_t *secret, size_t secret_len, const char *label,
+                                  size_t label_len, const EVP_MD *digest) const
 {
   uint8_t info[256] = {0};
   size_t info_len   = 0;
@@ -94,9 +94,9 @@ QUICCrypto::_hkdf_expand_label(uint8_t *dst, size_t dst_len, const uint8_t *secr
 }
 
 bool
-QUICCrypto::_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, const uint8_t *key, size_t key_len, const uint8_t *iv,
-                     size_t iv_len, size_t tag_len) const
+QUICCryptoTls::_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, const uint8_t *key, size_t key_len, const uint8_t *iv,
+                        size_t iv_len, size_t tag_len) const
 {
   uint8_t nonce[EVP_MAX_IV_LENGTH] = {0};
   size_t nonce_len                 = 0;
@@ -119,9 +119,9 @@ QUICCrypto::_encrypt(uint8_t *cipher, size_t &cipher_len, size_t max_cipher_len,
 }
 
 bool
-QUICCrypto::_decrypt(uint8_t *plain, size_t &plain_len, size_t max_plain_len, const uint8_t *cipher, size_t cipher_len,
-                     uint64_t pkt_num, const uint8_t *ad, size_t ad_len, const uint8_t *key, size_t key_len, const uint8_t *iv,
-                     size_t iv_len, size_t tag_len) const
+QUICCryptoTls::_decrypt(uint8_t *plain, size_t &plain_len, size_t max_plain_len, const uint8_t *cipher, size_t cipher_len,
+                        uint64_t pkt_num, const uint8_t *ad, size_t ad_len, const uint8_t *key, size_t key_len, const uint8_t *iv,
+                        size_t iv_len, size_t tag_len) const
 {
   uint8_t nonce[EVP_MAX_IV_LENGTH] = {0};
   size_t nonce_len                 = 0;
diff --git a/iocore/net/quic/QUICCrypto_openssl.cc b/iocore/net/quic/QUICCrypto_openssl.cc
index e62a5ef..b216ae1 100644
--- a/iocore/net/quic/QUICCrypto_openssl.cc
+++ b/iocore/net/quic/QUICCrypto_openssl.cc
@@ -20,7 +20,7 @@
  *  See the License for the specific language governing permissions and
  *  limitations under the License.
  */
-#include "QUICCrypto.h"
+#include "QUICCryptoTls.h"
 
 #include <openssl/err.h>
 #include <openssl/ssl.h>
@@ -31,7 +31,7 @@
 static constexpr char tag[] = "quic_crypto";
 
 const EVP_CIPHER *
-QUICCrypto::_get_evp_aead() const
+QUICCryptoTls::_get_evp_aead() const
 {
   if (this->is_handshake_finished()) {
     switch (SSL_CIPHER_get_id(SSL_get_current_cipher(this->_ssl))) {
@@ -54,7 +54,7 @@ QUICCrypto::_get_evp_aead() const
 }
 
 size_t
-QUICCrypto::_get_aead_tag_len() const
+QUICCryptoTls::_get_aead_tag_len() const
 {
   if (this->is_handshake_finished()) {
     switch (SSL_CIPHER_get_id(SSL_get_current_cipher(this->_ssl))) {
@@ -77,20 +77,20 @@ QUICCrypto::_get_aead_tag_len() const
 }
 
 size_t
-QUICCrypto::_get_aead_key_len(const EVP_CIPHER *aead) const
+QUICCryptoTls::_get_aead_key_len(const EVP_CIPHER *aead) const
 {
   return EVP_CIPHER_key_length(aead);
 }
 
 size_t
-QUICCrypto::_get_aead_nonce_len(const EVP_CIPHER *aead) const
+QUICCryptoTls::_get_aead_nonce_len(const EVP_CIPHER *aead) const
 {
   return EVP_CIPHER_iv_length(aead);
 }
 
 bool
-QUICCrypto::_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, const KeyMaterial &km, size_t tag_len) const
+QUICCryptoTls::_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, const KeyMaterial &km, size_t tag_len) const
 {
   uint8_t nonce[EVP_MAX_IV_LENGTH] = {0};
   size_t nonce_len                 = 0;
@@ -138,8 +138,8 @@ QUICCrypto::_encrypt(uint8_t *cipher, size_t &cipher_len, size_t max_cipher_len,
 }
 
 bool
-QUICCrypto::_decrypt(uint8_t *plain, size_t &plain_len, size_t max_plain_len, const uint8_t *cipher, size_t cipher_len,
-                     uint64_t pkt_num, const uint8_t *ad, size_t ad_len, const KeyMaterial &km, size_t tag_len) const
+QUICCryptoTls::_decrypt(uint8_t *plain, size_t &plain_len, size_t max_plain_len, const uint8_t *cipher, size_t cipher_len,
+                        uint64_t pkt_num, const uint8_t *ad, size_t ad_len, const KeyMaterial &km, size_t tag_len) const
 {
   uint8_t nonce[EVP_MAX_IV_LENGTH] = {0};
   size_t nonce_len                 = 0;
diff --git a/iocore/net/quic/QUICHandshake.cc b/iocore/net/quic/QUICHandshake.cc
index c33f3b6..faf2c1c 100644
--- a/iocore/net/quic/QUICHandshake.cc
+++ b/iocore/net/quic/QUICHandshake.cc
@@ -23,6 +23,7 @@
 
 #include "QUICGlobals.h"
 #include "QUICHandshake.h"
+#include "QUICCryptoTls.h"
 
 #include <utility>
 #include "QUICVersionNegotiator.h"
@@ -86,10 +87,10 @@ QUICHandshake::QUICHandshake(QUICConnection *qc, SSL_CTX *ssl_ctx, QUICStateless
   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->_crypto             = new QUICCryptoTls(this->_ssl, qc->direction());
   this->_version_negotiator = new QUICVersionNegotiator();
 
-  this->_crypto->initialize_key_materials(this->_client_qc->connection_id());
+  this->_crypto->initialize_key_materials(this->_client_qc->original_connection_id());
   this->_load_local_transport_parameters();
 
   SET_HANDLER(&QUICHandshake::state_read_client_hello);
@@ -151,7 +152,11 @@ QUICHandshake::crypto_module()
 void
 QUICHandshake::negotiated_application_name(const uint8_t **name, unsigned int *len)
 {
-  SSL_get0_alpn_selected(this->_crypto->ssl_handle(), name, len);
+  // FIXME Generalize and remove dynamic_cast
+  QUICCryptoTls *crypto_tls = dynamic_cast<QUICCryptoTls *>(this->_crypto);
+  if (crypto_tls) {
+    SSL_get0_alpn_selected(crypto_tls->ssl_handle(), name, len);
+  }
 }
 
 void
diff --git a/iocore/net/quic/QUICPacket.cc b/iocore/net/quic/QUICPacket.cc
index 2f62cc1..35e55d4 100644
--- a/iocore/net/quic/QUICPacket.cc
+++ b/iocore/net/quic/QUICPacket.cc
@@ -22,6 +22,7 @@
  */
 
 #include <ts/ink_assert.h>
+#include <ts/Diags.h>
 #include "QUICPacket.h"
 
 ClassAllocator<QUICPacket> quicPacketAllocator("quicPacketAllocator");
@@ -33,7 +34,6 @@ static constexpr int OFFSET_PACKET_NUMBER = 9;
 static constexpr int OFFSET_VERSION       = 13;
 static constexpr int OFFSET_PAYLOAD       = 17;
 static constexpr int LONGHEADER_LENGTH    = 17;
-static constexpr int FNV1A_HASH_LEN       = 8;
 
 //
 // QUICPacketHeader
@@ -41,7 +41,14 @@ static constexpr int FNV1A_HASH_LEN       = 8;
 const uint8_t *
 QUICPacketHeader::buf()
 {
-  return this->_buf;
+  if (this->_buf) {
+    return this->_buf;
+  } else {
+    // TODO Reuse serialzied data if nothing has changed
+    size_t dummy;
+    this->store(this->_serialized, &dummy);
+    return this->_serialized;
+  }
 }
 
 QUICPacketHeader *
@@ -199,7 +206,8 @@ QUICPacketLongHeader::has_key_phase() const
 QUICKeyPhase
 QUICPacketLongHeader::key_phase() const
 {
-  return QUICKeyPhase::PHASE_0;
+  // TODO LongHeader will also be used for 0-RTT packets
+  return QUICKeyPhase::CLEARTEXT;
 }
 
 uint16_t
@@ -458,36 +466,25 @@ QUICPacketShortHeader::store(uint8_t *buf, size_t *len) const
 // QUICPacket
 //
 
-QUICPacket::QUICPacket(QUICPacketHeader *header, ats_unique_buf payload, size_t payload_len, QUICPacketNumber base_packet_number)
+QUICPacket::QUICPacket(QUICPacketHeader *header, ats_unique_buf payload, size_t payload_len)
 {
   this->_header       = header;
-  this->_size         = this->_header->length() + payload_len;
   this->_payload      = std::move(payload);
   this->_payload_size = payload_len;
 }
 
-QUICPacket::QUICPacket(QUICPacketType type, QUICConnectionId connection_id, QUICPacketNumber packet_number,
-                       QUICPacketNumber base_packet_number, QUICVersion version, ats_unique_buf payload, size_t len,
-                       bool retransmittable)
+QUICPacket::QUICPacket(QUICPacketHeader *header, ats_unique_buf payload, size_t payload_len, bool retransmittable)
 {
-  this->_header = QUICPacketHeader::build(type, connection_id, packet_number, base_packet_number, version, std::move(payload), len);
-  this->_size   = this->_header->length() + len;
-  if (type != QUICPacketType::ZERO_RTT_PROTECTED && type != QUICPacketType::ONE_RTT_PROTECTED_KEY_PHASE_0 &&
-      type != QUICPacketType::ONE_RTT_PROTECTED_KEY_PHASE_1) {
-    this->_size += FNV1A_HASH_LEN;
-  }
+  this->_header             = header;
+  this->_payload            = std::move(payload);
+  this->_payload_size       = payload_len;
   this->_is_retransmittable = retransmittable;
 }
 
 QUICPacket::QUICPacket(QUICPacketType type, QUICPacketNumber packet_number, QUICPacketNumber base_packet_number,
                        ats_unique_buf payload, size_t len, bool retransmittable)
 {
-  this->_header = QUICPacketHeader::build(type, packet_number, base_packet_number, std::move(payload), len);
-  this->_size   = this->_header->length() + len;
-  if (type != QUICPacketType::ZERO_RTT_PROTECTED && type != QUICPacketType::ONE_RTT_PROTECTED_KEY_PHASE_0 &&
-      type != QUICPacketType::ONE_RTT_PROTECTED_KEY_PHASE_1) {
-    this->_size += FNV1A_HASH_LEN;
-  }
+  this->_header             = QUICPacketHeader::build(type, packet_number, base_packet_number, std::move(payload), len);
   this->_is_retransmittable = retransmittable;
 }
 
@@ -495,11 +492,6 @@ QUICPacket::QUICPacket(QUICPacketType type, QUICConnectionId connection_id, QUIC
                        QUICPacketNumber base_packet_number, ats_unique_buf payload, size_t len, bool retransmittable)
 {
   this->_header = QUICPacketHeader::build(type, connection_id, packet_number, base_packet_number, std::move(payload), len);
-  this->_size   = this->_header->length() + len;
-  if (type != QUICPacketType::ZERO_RTT_PROTECTED && type != QUICPacketType::ONE_RTT_PROTECTED_KEY_PHASE_0 &&
-      type != QUICPacketType::ONE_RTT_PROTECTED_KEY_PHASE_1) {
-    this->_size += FNV1A_HASH_LEN;
-  }
   this->_is_retransmittable = retransmittable;
 }
 
@@ -515,10 +507,9 @@ QUICPacket::QUICPacket(QUICPacketType type, QUICConnectionId connection_id, QUIC
   for (int i = 15; i < 23; ++i) {
     fake_payload.get()[i] = rnd() & 0xFF;
   }
-
+  // TODO stateless packet format changed
   this->_header =
     QUICPacketHeader::build(type, connection_id, fake_packet_number, fake_base_packet_number, std::move(fake_payload), 15 + 8);
-  this->_size               = this->_header->length() + 15 + 8;
   this->_is_retransmittable = false;
 }
 
@@ -562,7 +553,7 @@ QUICPacket::header() const
 const uint8_t *
 QUICPacket::payload() const
 {
-  return this->_header->payload();
+  return this->_payload.get();
 }
 
 QUICVersion
@@ -580,7 +571,7 @@ QUICPacket::is_retransmittable() const
 uint16_t
 QUICPacket::size() const
 {
-  return this->_size;
+  return this->header_size() + this->payload_size();
 }
 
 uint16_t
@@ -592,14 +583,7 @@ QUICPacket::header_size() const
 uint16_t
 QUICPacket::payload_size() const
 {
-  if (this->type() == QUICPacketType::STATELESS_RESET) {
-    return this->_size - this->_header->length();
-  } else if (this->type() != QUICPacketType::ZERO_RTT_PROTECTED && this->type() != QUICPacketType::ONE_RTT_PROTECTED_KEY_PHASE_0 &&
-             this->type() != QUICPacketType::ONE_RTT_PROTECTED_KEY_PHASE_1) {
-    return this->_size - this->_header->length() - FNV1A_HASH_LEN;
-  } else {
-    return this->_size - this->_header->length();
-  }
+  return this->_payload_size;
 }
 
 QUICKeyPhase
@@ -611,8 +595,9 @@ QUICPacket::key_phase() const
 void
 QUICPacket::store(uint8_t *buf, size_t *len) const
 {
-  memcpy(buf, this->payload(), this->payload_size());
-  *len = this->payload_size();
+  memcpy(buf, this->_header->buf(), this->_header->length());
+  memcpy(buf + this->_header->length(), this->payload(), this->payload_size());
+  *len = this->_header->length() + this->payload_size();
 }
 
 uint8_t
@@ -706,6 +691,7 @@ QUICPacketFactory::create(ats_unique_buf buf, size_t len, QUICPacketNumber base_
     } else {
       result = QUICPacketCreationResult::FAILED;
     }
+    break;
   default:
     result = QUICPacketCreationResult::FAILED;
     break;
@@ -714,7 +700,7 @@ QUICPacketFactory::create(ats_unique_buf buf, size_t len, QUICPacketNumber base_
   QUICPacket *packet = nullptr;
   if (result == QUICPacketCreationResult::SUCCESS) {
     packet = quicPacketAllocator.alloc();
-    new (packet) QUICPacket(header, std::move(plain_txt), plain_txt_len, base_packet_number);
+    new (packet) QUICPacket(header, std::move(plain_txt), plain_txt_len);
   }
 
   return QUICPacketUPtr(packet, &QUICPacketDeleter::delete_packet);
@@ -723,6 +709,10 @@ QUICPacketFactory::create(ats_unique_buf buf, size_t len, QUICPacketNumber base_
 QUICPacketUPtr
 QUICPacketFactory::create_version_negotiation_packet(const QUICPacket *packet_sent_by_client, QUICPacketNumber base_packet_number)
 {
+  size_t max_cipher_txt_len = 2048;
+  ats_unique_buf cipher_txt = ats_unique_malloc(max_cipher_txt_len);
+  size_t cipher_txt_len     = 0;
+
   size_t len = sizeof(QUICVersion) * countof(QUIC_SUPPORTED_VERSIONS);
   ats_unique_buf versions(reinterpret_cast<uint8_t *>(ats_malloc(len)), [](void *p) { ats_free(p); });
   uint8_t *p = versions.get();
@@ -733,10 +723,16 @@ QUICPacketFactory::create_version_negotiation_packet(const QUICPacket *packet_se
     p += n;
   }
 
-  QUICPacket *packet = quicPacketAllocator.alloc();
-  new (packet)
-    QUICPacket(QUICPacketType::VERSION_NEGOTIATION, packet_sent_by_client->connection_id(), packet_sent_by_client->packet_number(),
-               base_packet_number, packet_sent_by_client->version(), std::move(versions), len, false);
+  QUICPacket *packet       = nullptr;
+  QUICPacketHeader *header = QUICPacketHeader::build(QUICPacketType::VERSION_NEGOTIATION, packet_sent_by_client->connection_id(),
+                                                     packet_sent_by_client->packet_number(), base_packet_number,
+                                                     packet_sent_by_client->version(), std::move(versions), len);
+  if (this->_crypto->encrypt(cipher_txt.get(), cipher_txt_len, max_cipher_txt_len, header->payload(), header->payload_size(),
+                             header->packet_number(), header->buf(), header->length(), header->key_phase())) {
+    packet = quicPacketAllocator.alloc();
+    new (packet) QUICPacket(header, std::move(cipher_txt), cipher_txt_len, false);
+  }
+
   return QUICPacketUPtr(packet, QUICPacketDeleter::delete_packet);
 }
 
@@ -744,10 +740,22 @@ QUICPacketUPtr
 QUICPacketFactory::create_server_cleartext_packet(QUICConnectionId connection_id, QUICPacketNumber base_packet_number,
                                                   ats_unique_buf payload, size_t len, bool retransmittable)
 {
-  QUICPacket *p = quicPacketAllocator.alloc();
-  new (p) QUICPacket(QUICPacketType::SERVER_CLEARTEXT, connection_id, this->_packet_number_generator.next(), base_packet_number,
-                     this->_version, std::move(payload), len, retransmittable);
-  return QUICPacketUPtr(p, &QUICPacketDeleter::delete_packet);
+  size_t max_cipher_txt_len = 2048;
+  ats_unique_buf cipher_txt = ats_unique_malloc(max_cipher_txt_len);
+  size_t cipher_txt_len     = 0;
+
+  QUICPacket *packet = nullptr;
+  QUICPacketHeader *header =
+    QUICPacketHeader::build(QUICPacketType::SERVER_CLEARTEXT, connection_id, this->_packet_number_generator.next(),
+                            base_packet_number, this->_version, std::move(payload), len);
+
+  if (this->_crypto->encrypt(cipher_txt.get(), cipher_txt_len, max_cipher_txt_len, header->payload(), header->payload_size(),
+                             header->packet_number(), header->buf(), header->length(), header->key_phase())) {
+    packet = quicPacketAllocator.alloc();
+    new (packet) QUICPacket(header, std::move(cipher_txt), cipher_txt_len, retransmittable);
+  }
+
+  return QUICPacketUPtr(packet, &QUICPacketDeleter::delete_packet);
 }
 
 QUICPacketUPtr
@@ -789,9 +797,20 @@ QUICPacketUPtr
 QUICPacketFactory::create_client_initial_packet(QUICConnectionId connection_id, QUICPacketNumber base_packet_number,
                                                 QUICVersion version, ats_unique_buf payload, size_t len)
 {
-  QUICPacket *packet = quicPacketAllocator.alloc();
-  new (packet) QUICPacket(QUICPacketType::CLIENT_INITIAL, connection_id, this->_packet_number_generator.next(), base_packet_number,
-                          version, std::move(payload), len, true);
+  size_t max_cipher_txt_len = 2048;
+  ats_unique_buf cipher_txt = ats_unique_malloc(max_cipher_txt_len);
+  size_t cipher_txt_len     = 0;
+
+  QUICPacket *packet = nullptr;
+  QUICPacketHeader *header =
+    QUICPacketHeader::build(QUICPacketType::CLIENT_INITIAL, connection_id, this->_packet_number_generator.next(),
+                            base_packet_number, version, std::move(payload), len);
+  if (this->_crypto->encrypt(cipher_txt.get(), cipher_txt_len, max_cipher_txt_len, header->payload(), header->payload_size(),
+                             header->packet_number(), header->buf(), header->length(), header->key_phase())) {
+    packet = quicPacketAllocator.alloc();
+    new (packet) QUICPacket(header, std::move(cipher_txt), cipher_txt_len, false);
+  }
+
   return QUICPacketUPtr(packet, &QUICPacketDeleter::delete_packet);
 }
 
diff --git a/iocore/net/quic/QUICPacket.h b/iocore/net/quic/QUICPacket.h
index b51d234..b7173ed 100644
--- a/iocore/net/quic/QUICPacket.h
+++ b/iocore/net/quic/QUICPacket.h
@@ -119,6 +119,8 @@ protected:
   const uint8_t *_buf = nullptr;
   size_t _buf_len     = 0;
 
+  // These are used only if the instance was created without a buffer
+  uint8_t _serialized[16];
   ats_unique_buf _payload              = ats_unique_buf(nullptr, [](void *p) { ats_free(p); });
   QUICPacketType _type                 = QUICPacketType::UNINITIALIZED;
   QUICKeyPhase _key_phase              = QUICKeyPhase::CLEARTEXT;
@@ -187,15 +189,19 @@ public:
   /*
    * Creates a QUICPacket with a QUICPacketHeader and a buffer that contains payload
    *
-   * QUICPacket class doesn't care about whether the payload is protected (encrypted) or not.
+   * This will be used for receiving packets. Therefore, it is expected that payload is already decrypted.
+   * However,  QUICPacket class itself doesn't care about whether the payload is protected (encrypted) or not.
    */
-  QUICPacket(QUICPacketHeader *header, ats_unique_buf payload, size_t payload_len, QUICPacketNumber base_packet_number);
+  QUICPacket(QUICPacketHeader *header, ats_unique_buf payload, size_t payload_len);
 
   /*
-   * Creates a QUICPacket that has a Long Header
+   * Creates a QUICPacket with a QUICPacketHeader, a buffer that contains payload and a flag that indicates whether the packet can
+   * retransmittable
+   *
+   * This will be used for sending packets. Therefore, it is expected that payload is already encrypted.
+   * However, QUICPacket class itself doesn't care about whether the payload is protected (encrypted) or not.
    */
-  QUICPacket(QUICPacketType type, QUICConnectionId connection_id, QUICPacketNumber packet_number,
-             QUICPacketNumber base_packet_number, QUICVersion version, ats_unique_buf payload, size_t len, bool retransmittable);
+  QUICPacket(QUICPacketHeader *header, ats_unique_buf payload, size_t payload_len, bool retransmittable);
 
   /*
    * Creates a QUICPacket that has a Short Header
@@ -240,7 +246,6 @@ private:
   QUICPacketHeader *_header;
   ats_unique_buf _payload  = ats_unique_buf(nullptr, [](void *p) { ats_free(p); });
   size_t _payload_size     = 0;
-  size_t _size             = 0;
   bool _is_retransmittable = false;
 };
 
diff --git a/iocore/net/quic/QUICPacketTransmitter.h b/iocore/net/quic/QUICPacketTransmitter.h
index 323c98d..5396a6a 100644
--- a/iocore/net/quic/QUICPacketTransmitter.h
+++ b/iocore/net/quic/QUICPacketTransmitter.h
@@ -24,6 +24,7 @@
 #pragma once
 
 #include "QUICPacket.h"
+#include "I_EventSystem.h"
 
 class QUICPacketTransmitter
 {
diff --git a/iocore/net/quic/QUICTypes.h b/iocore/net/quic/QUICTypes.h
index ee8dbbe..e4fd5ad 100644
--- a/iocore/net/quic/QUICTypes.h
+++ b/iocore/net/quic/QUICTypes.h
@@ -58,7 +58,7 @@ using QUICOffset       = uint64_t;
 // Note: You also need to update tests for VersionNegotiationPacket, if you change the number of versions
 // Prefix for drafts (0xff000000) + draft number
 constexpr QUICVersion QUIC_SUPPORTED_VERSIONS[] = {
-  0xff000005,
+  0xff000007,
 };
 constexpr QUICStreamId STREAM_ID_FOR_HANDSHAKE = 0;
 
diff --git a/iocore/net/quic/test/test_QUICCrypto.cc b/iocore/net/quic/test/test_QUICCrypto.cc
index 35479e1..056874f 100644
--- a/iocore/net/quic/test/test_QUICCrypto.cc
+++ b/iocore/net/quic/test/test_QUICCrypto.cc
@@ -34,7 +34,7 @@
 #include <openssl/ssl.h>
 
 #include "Mock.h"
-#include "QUICCrypto.h"
+#include "QUICCryptoTls.h"
 
 static constexpr uint32_t MAX_HANDSHAKE_MSG_LEN = 2048;
 
@@ -109,7 +109,7 @@ TEST_CASE("QUICCrypto Cleartext", "[quic]")
   SSL_CTX *client_ssl_ctx = SSL_CTX_new(TLS_method());
   SSL_CTX_set_min_proto_version(client_ssl_ctx, TLS1_3_VERSION);
   SSL_CTX_set_max_proto_version(client_ssl_ctx, TLS1_3_VERSION);
-  QUICCrypto *client = new QUICCrypto(SSL_new(client_ssl_ctx), NET_VCONNECTION_OUT);
+  QUICCrypto *client = new QUICCryptoTls(SSL_new(client_ssl_ctx), NET_VCONNECTION_OUT);
 
   // Server
   SSL_CTX *server_ssl_ctx = SSL_CTX_new(TLS_method());
@@ -119,7 +119,7 @@ TEST_CASE("QUICCrypto Cleartext", "[quic]")
   SSL_CTX_use_certificate(server_ssl_ctx, PEM_read_bio_X509(crt_bio, nullptr, nullptr, nullptr));
   BIO *key_bio(BIO_new_mem_buf(server_key, sizeof(server_key)));
   SSL_CTX_use_PrivateKey(server_ssl_ctx, PEM_read_bio_PrivateKey(key_bio, nullptr, nullptr, nullptr));
-  QUICCrypto *server = new QUICCrypto(SSL_new(server_ssl_ctx), NET_VCONNECTION_IN);
+  QUICCrypto *server = new QUICCryptoTls(SSL_new(server_ssl_ctx), NET_VCONNECTION_IN);
 
   // Client Hello
   uint8_t client_hello[MAX_HANDSHAKE_MSG_LEN] = {0};
@@ -195,7 +195,7 @@ TEST_CASE("QUICCrypto 1-RTT", "[quic]")
   SSL_CTX *client_ssl_ctx = SSL_CTX_new(TLS_method());
   SSL_CTX_set_min_proto_version(client_ssl_ctx, TLS1_3_VERSION);
   SSL_CTX_set_max_proto_version(client_ssl_ctx, TLS1_3_VERSION);
-  QUICCrypto *client = new QUICCrypto(SSL_new(client_ssl_ctx), NET_VCONNECTION_OUT);
+  QUICCrypto *client = new QUICCryptoTls(SSL_new(client_ssl_ctx), NET_VCONNECTION_OUT);
 
   // Server
   SSL_CTX *server_ssl_ctx = SSL_CTX_new(TLS_method());
@@ -205,7 +205,7 @@ TEST_CASE("QUICCrypto 1-RTT", "[quic]")
   SSL_CTX_use_certificate(server_ssl_ctx, PEM_read_bio_X509(crt_bio, nullptr, nullptr, nullptr));
   BIO *key_bio(BIO_new_mem_buf(server_key, sizeof(server_key)));
   SSL_CTX_use_PrivateKey(server_ssl_ctx, PEM_read_bio_PrivateKey(key_bio, nullptr, nullptr, nullptr));
-  QUICCrypto *server = new QUICCrypto(SSL_new(server_ssl_ctx), NET_VCONNECTION_IN);
+  QUICCrypto *server = new QUICCryptoTls(SSL_new(server_ssl_ctx), NET_VCONNECTION_IN);
 
   // Client Hello
   uint8_t client_hello[MAX_HANDSHAKE_MSG_LEN] = {0};
diff --git a/iocore/net/quic/test/test_QUICLossDetector.cc b/iocore/net/quic/test/test_QUICLossDetector.cc
index 543249b..4284182 100644
--- a/iocore/net/quic/test/test_QUICLossDetector.cc
+++ b/iocore/net/quic/test/test_QUICLossDetector.cc
@@ -40,9 +40,10 @@ TEST_CASE("QUICLossDetector_Loss_in_Handshake", "[quic]")
   ats_unique_buf payload = ats_unique_malloc(sizeof(raw));
   memcpy(payload.get(), raw, sizeof(raw));
 
-  QUICPacketUPtr packet = QUICPacketUPtr(new QUICPacket(QUICPacketType::SERVER_CLEARTEXT, 0xffddbb9977553311ULL, 0x00000001, 0,
-                                                        0x00112233, std::move(payload), sizeof(raw), true),
-                                         [](QUICPacket *p) { delete p; });
+  QUICPacketHeader *header = QUICPacketHeader::build(QUICPacketType::SERVER_CLEARTEXT, 0xffddbb9977553311ULL, 0x00000001, 0,
+                                                     0x00112233, std::move(payload), sizeof(raw));
+  QUICPacketUPtr packet =
+    QUICPacketUPtr(new QUICPacket(header, std::move(payload), sizeof(raw), true), [](QUICPacket *p) { delete p; });
   detector.on_packet_sent(std::move(packet));
   ink_hrtime_sleep(HRTIME_MSECONDS(1000));
   CHECK(tx->_retransmit_count > 0);
diff --git a/iocore/net/quic/test/test_QUICPacketFactory.cc b/iocore/net/quic/test/test_QUICPacketFactory.cc
index 7a63fca..4a5f2f9 100644
--- a/iocore/net/quic/test/test_QUICPacketFactory.cc
+++ b/iocore/net/quic/test/test_QUICPacketFactory.cc
@@ -24,10 +24,13 @@
 #include "catch.hpp"
 
 #include "quic/QUICPacket.h"
+#include "quic/Mock.h"
 
 TEST_CASE("QUICPacketFactory_Create_VersionNegotiationPacket", "[quic]")
 {
   QUICPacketFactory factory;
+  MockQUICCrypto crypto;
+  factory.set_crypto_module(&crypto);
 
   uint8_t client_initial_packet_header[] = {
     0x82,                                           // Type
@@ -47,12 +50,14 @@ TEST_CASE("QUICPacketFactory_Create_VersionNegotiationPacket", "[quic]")
   CHECK(packet->type() == QUICPacketType::VERSION_NEGOTIATION);
   CHECK(packet->connection_id() == client_initial_packet.connection_id());
   CHECK(packet->packet_number() == client_initial_packet.packet_number());
-  CHECK(memcmp(packet->payload(), "\xff\x00\x00\x05", 4) == 0);
+  CHECK(memcmp(packet->payload(), "\xff\x00\x00\x07", 4) == 0);
 }
 
 TEST_CASE("QUICPacketFactory_Create_ServerCleartextPacket", "[quic]")
 {
   QUICPacketFactory factory;
+  MockQUICCrypto crypto;
+  factory.set_crypto_module(&crypto);
   factory.set_version(0x11223344);
 
   uint8_t raw[]          = {0xaa, 0xbb, 0xcc, 0xdd};
@@ -70,6 +75,8 @@ TEST_CASE("QUICPacketFactory_Create_ServerCleartextPacket", "[quic]")
 TEST_CASE("QUICPacketFactory_Create_StatelessResetPacket", "[quic]")
 {
   QUICPacketFactory factory;
+  MockQUICCrypto crypto;
+  factory.set_crypto_module(&crypto);
   QUICStatelessToken token;
   token.gen_token(12345);
   uint8_t expected_output[] = {
diff --git a/iocore/net/quic/test/test_QUICVersionNegotiator.cc b/iocore/net/quic/test/test_QUICVersionNegotiator.cc
index e2971d5..c8c0a93 100644
--- a/iocore/net/quic/test/test_QUICVersionNegotiator.cc
+++ b/iocore/net/quic/test/test_QUICVersionNegotiator.cc
@@ -29,6 +29,8 @@
 TEST_CASE("QUICVersionNegotiator_Normal", "[quic]")
 {
   QUICPacketFactory packet_factory;
+  MockQUICCrypto crypto;
+  packet_factory.set_crypto_module(&crypto);
   QUICVersionNegotiator vn;
 
   // Check initial state

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