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/08 01:46:03 UTC

[trafficserver] branch quic-latest updated: Add basic support of IETF QUIC (draft-04)

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

masaori 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 14ebf42  Add basic support of IETF QUIC (draft-04)
14ebf42 is described below

commit 14ebf42f3f408c784a73f476587f9e39facfa97a
Author: Masaori Koshiba <ma...@apache.org>
AuthorDate: Tue Aug 8 10:26:35 2017 +0900

    Add basic support of IETF QUIC (draft-04)
    
    Some features listed on the First Implementation is supported.
    - Version negotiation
    - Basic packetization and reliability
    - Basic STREAM sending and receiving
    - Integration with TLS 1.3 handshake (1-RTT)
    - Authentication for cleartext
    - PADDING frames
    - Connection ID
    - Packet protection
    - Frame parsing for all frames
    
    This commit also has
    - Simple echo QUIC application
    
    But this commit doesn't support
    - Connection Close
    
    Many features which are out of scope of First Implementation are
    also not supported.
    - The entire HTTP mapping
    - Congestion control
    ...etc
    
    Co-authored-by: Masakazu Kitajo <ma...@apache.org>
---
 .gitignore                                         |   13 +
 configure.ac                                       |   29 +
 iocore/eventsystem/I_Event.h                       |    2 +
 iocore/eventsystem/I_Thread.h                      |    5 +
 iocore/net/I_NetProcessor.h                        |    2 +
 iocore/net/Makefile.am                             |    8 +
 iocore/net/P_Net.h                                 |    5 +
 iocore/net/P_QUICNetProcessor.h                    |   73 ++
 iocore/net/P_QUICNetVConnection.h                  |  205 +++
 iocore/net/P_QUICPacketHandler.h                   |   47 +
 iocore/net/P_SSLUtils.h                            |    1 +
 iocore/net/QUICNetProcessor.cc                     |  144 +++
 iocore/net/QUICNetVConnection.cc                   |  530 ++++++++
 iocore/net/QUICPacketHandler.cc                    |  148 +++
 iocore/net/SSLUtils.cc                             |   75 ++
 iocore/net/quic/Makefile.am                        |   66 +
 iocore/net/quic/Mock.h                             |  233 ++++
 iocore/net/quic/QUICAckFrameCreator.cc             |  102 ++
 iocore/net/quic/QUICAckFrameCreator.h              |   69 +
 iocore/net/quic/QUICApplication.cc                 |  139 +++
 iocore/net/quic/QUICApplication.h                  |   79 ++
 iocore/net/quic/QUICCongestionController.cc        |   40 +
 iocore/net/quic/QUICCongestionController.h         |   36 +
 iocore/net/quic/QUICConnectionManager.cc           |   42 +
 iocore/net/quic/QUICConnectionManager.h            |   40 +
 iocore/net/quic/QUICCrypto.cc                      |  405 ++++++
 iocore/net/quic/QUICCrypto.h                       |  121 ++
 iocore/net/quic/QUICCrypto_boringssl.cc            |  144 +++
 iocore/net/quic/QUICCrypto_openssl.cc              |  236 ++++
 iocore/net/quic/QUICDebugNames.cc                  |  118 ++
 iocore/net/quic/QUICDebugNames.h                   |   35 +
 iocore/net/quic/QUICEchoApp.cc                     |   85 ++
 iocore/net/quic/QUICEchoApp.h                      |   42 +
 iocore/net/quic/QUICEvents.h                       |   31 +
 iocore/net/quic/QUICFlowController.cc              |   42 +
 iocore/net/quic/QUICFlowController.h               |   36 +
 iocore/net/quic/QUICFrame.cc                       | 1315 ++++++++++++++++++++
 iocore/net/quic/QUICFrame.h                        |  616 +++++++++
 iocore/net/quic/QUICFrameDispatcher.cc             |  148 +++
 iocore/net/quic/QUICFrameDispatcher.h              |   53 +
 iocore/net/quic/QUICFrameHandler.h                 |   31 +
 iocore/net/quic/QUICFrameTransmitter.h             |   37 +
 iocore/net/quic/QUICHandshake.cc                   |  249 ++++
 iocore/net/quic/QUICHandshake.h                    |   68 +
 iocore/net/quic/QUICLossDetector.cc                |  325 +++++
 iocore/net/quic/QUICLossDetector.h                 |  108 ++
 iocore/net/quic/QUICPacket.cc                      |  673 ++++++++++
 iocore/net/quic/QUICPacket.h                       |  191 +++
 iocore/net/quic/QUICPacketTransmitter.h            |   50 +
 iocore/net/quic/QUICStream.cc                      |  331 +++++
 iocore/net/quic/QUICStream.h                       |   98 ++
 iocore/net/quic/QUICStreamManager.cc               |  114 ++
 iocore/net/quic/QUICStreamManager.h                |   55 +
 iocore/net/quic/QUICStreamState.cc                 |   58 +
 iocore/net/quic/QUICStreamState.h                  |   54 +
 iocore/net/quic/QUICTransportParameters.cc         |  132 ++
 iocore/net/quic/QUICTransportParameters.h          |  120 ++
 iocore/net/quic/QUICTypes.cc                       |  140 +++
 iocore/net/quic/QUICTypes.h                        |  179 +++
 iocore/net/quic/QUICVersionNegotiator.cc           |   63 +
 iocore/net/quic/QUICVersionNegotiator.h            |   47 +
 iocore/net/quic/test/Makefile.am                   |  368 ++++++
 iocore/net/quic/test/event_processor_main.cc       |   53 +
 iocore/net/quic/test/main.cc                       |   27 +
 iocore/net/quic/test/test_QUICAckFrameCreator.cc   |   69 +
 iocore/net/quic/test/test_QUICCrypto.cc            |  189 +++
 iocore/net/quic/test/test_QUICFrame.cc             |  607 +++++++++
 iocore/net/quic/test/test_QUICFrameDispatcher.cc   |   74 ++
 iocore/net/quic/test/test_QUICLossDetector.cc      |   57 +
 iocore/net/quic/test/test_QUICPacket.cc            |   96 ++
 iocore/net/quic/test/test_QUICPacketFactory.cc     |   69 +
 iocore/net/quic/test/test_QUICStream.cc            |  140 +++
 iocore/net/quic/test/test_QUICStreamState.cc       |   47 +
 .../net/quic/test/test_QUICTransportParameters.cc  |  164 +++
 iocore/net/quic/test/test_QUICTypeUtil.cc          |   69 +
 iocore/net/quic/test/test_QUICVersionNegotiator.cc |   47 +
 lib/records/I_RecHttp.h                            |   26 +-
 lib/records/RecHttp.cc                             |   17 +
 proxy/Main.cc                                      |    5 +
 proxy/Makefile.am                                  |    5 +-
 proxy/hq/HQ.cc                                     |   32 +
 proxy/hq/HQ.h                                      |   44 +
 proxy/hq/HQSessionAccept.cc                        |   85 ++
 proxy/hq/HQSessionAccept.h                         |   56 +
 proxy/hq/Makefile.am                               |   43 +
 proxy/http/HttpProxyServerMain.cc                  |   11 +-
 proxy/http/Makefile.am                             |    3 +-
 87 files changed, 11082 insertions(+), 4 deletions(-)

diff --git a/.gitignore b/.gitignore
index 6c62df4..a5a618f 100644
--- a/.gitignore
+++ b/.gitignore
@@ -87,6 +87,19 @@ lib/perl/lib/Apache/TS.pm
 
 iocore/net/test_certlookup
 iocore/net/test_UDPNet
+iocore/net/quic/test/test_QUICFrame
+iocore/net/quic/test/test_QUICFrameDispatcher
+iocore/net/quic/test/test_QUICPacket
+iocore/net/quic/test/test_QUICPacketFactory
+iocore/net/quic/test/test_QUICStream
+iocore/net/quic/test/test_QUICStreamState
+iocore/net/quic/test/test_QUICTransportParameters
+iocore/net/quic/test/test_QUICCrypto
+iocore/net/quic/test/test_QUICLossDetector
+iocore/net/quic/test/test_QUICTypeUtil
+iocore/net/quic/test/test_QUICAckFrameCreator
+iocore/net/quic/test/test_QUICVersionNegotiator
+iocore/net/quic/ts_quic_client
 iocore/aio/test_AIO
 iocore/eventsystem/test_Buffer
 iocore/eventsystem/test_Event
diff --git a/configure.ac b/configure.ac
index 33ccce8..4e1692d 100644
--- a/configure.ac
+++ b/configure.ac
@@ -1174,6 +1174,31 @@ AC_CHECK_FUNC([EVP_MD_CTX_reset], [],
 AC_CHECK_FUNC([EVP_MD_CTX_free], [],
               [AC_DEFINE([EVP_MD_CTX_free], [EVP_MD_CTX_destroy], [Renamed in OpenSSL 1.1])])
 
+AC_MSG_CHECKING([for TLS 1.3 is supported])
+AC_COMPILE_IFELSE([AC_LANG_PROGRAM([[#include <openssl/ssl.h>]],
+                                   [[
+                                     #ifndef TLS1_3_VERSION
+                                     # error no TLS1_3 support
+                                     #endif
+                                   ]])
+                  ],
+                  [AC_MSG_RESULT([yes])],
+                  [AC_ERROR(OpenSSL 1.1.1+ or BoringSSL is required);
+                   AC_MSG_RESULT([no])])
+
+
+AC_MSG_CHECKING([for OpenSSL is BoringSSL])
+AC_COMPILE_IFELSE([AC_LANG_PROGRAM([[#include <openssl/base.h>]],
+                                   [[
+                                     #ifndef OPENSSL_IS_BORINGSSL
+                                     # error not boringssl
+                                     #endif
+                                   ]])
+                  ],
+                  [AC_MSG_RESULT([yes]); openssl_is_boringssl=1],
+                  [AC_MSG_RESULT([no])])
+AM_CONDITIONAL([OPENSSL_IS_BORINGSSL], [test -n "$openssl_is_boringssl"])
+
 LIBS="$saved_LIBS"
 
 #
@@ -1875,6 +1900,7 @@ AC_SUBST([default_stack_size], [$with_default_stack_size])
 iocore_include_dirs="\
 -I\$(abs_top_srcdir)/iocore/eventsystem \
 -I\$(abs_top_srcdir)/iocore/net \
+-I\$(abs_top_srcdir)/iocore/net/quic \
 -I\$(abs_top_srcdir)/iocore/aio \
 -I\$(abs_top_srcdir)/iocore/hostdb \
 -I\$(abs_top_srcdir)/iocore/cache \
@@ -1932,6 +1958,8 @@ AC_CONFIG_FILES([
   iocore/eventsystem/Makefile
   iocore/hostdb/Makefile
   iocore/net/Makefile
+  iocore/net/quic/Makefile
+  iocore/net/quic/test/Makefile
   iocore/utils/Makefile
   lib/Makefile
   lib/cppapi/Makefile
@@ -1961,6 +1989,7 @@ AC_CONFIG_FILES([
   proxy/http/Makefile
   proxy/http/remap/Makefile
   proxy/http2/Makefile
+  proxy/hq/Makefile
   proxy/logging/Makefile
   proxy/shared/Makefile
   rc/Makefile
diff --git a/iocore/eventsystem/I_Event.h b/iocore/eventsystem/I_Event.h
index cfb99ba..cfc2045 100644
--- a/iocore/eventsystem/I_Event.h
+++ b/iocore/eventsystem/I_Event.h
@@ -74,6 +74,8 @@
 #define HTTP2_SESSION_EVENTS_START 2250
 #define HTTP_TUNNEL_EVENTS_START 2300
 #define HTTP_SCH_UPDATE_EVENTS_START 2400
+#define QUIC_EVENT_EVENTS_START 2500
+#define HQ_SESSION_EVENTS_START 2600
 #define NT_ASYNC_CONNECT_EVENT_EVENTS_START 3000
 #define NT_ASYNC_IO_EVENT_EVENTS_START 3100
 #define RAFT_EVENT_EVENTS_START 3200
diff --git a/iocore/eventsystem/I_Thread.h b/iocore/eventsystem/I_Thread.h
index f053dec..a4926b8 100644
--- a/iocore/eventsystem/I_Thread.h
+++ b/iocore/eventsystem/I_Thread.h
@@ -129,9 +129,14 @@ public:
   ProxyAllocator eventAllocator;
   ProxyAllocator netVCAllocator;
   ProxyAllocator sslNetVCAllocator;
+  ProxyAllocator quicNetVCAllocator;
   ProxyAllocator http1ClientSessionAllocator;
   ProxyAllocator http2ClientSessionAllocator;
   ProxyAllocator http2StreamAllocator;
+  ProxyAllocator quicClientSessionAllocator;
+  ProxyAllocator quicHandshakeAllocator;
+  ProxyAllocator quicStreamAllocator;
+  ProxyAllocator quicStreamManagerAllocator;
   ProxyAllocator httpServerSessionAllocator;
   ProxyAllocator hdrHeapAllocator;
   ProxyAllocator strHeapAllocator;
diff --git a/iocore/net/I_NetProcessor.h b/iocore/net/I_NetProcessor.h
index 75ecf06..3e52528 100644
--- a/iocore/net/I_NetProcessor.h
+++ b/iocore/net/I_NetProcessor.h
@@ -269,4 +269,6 @@ extern inkcoreapi NetProcessor &netProcessor;
 */
 extern inkcoreapi NetProcessor &sslNetProcessor;
 
+extern inkcoreapi NetProcessor &quicNetProcessor;
+
 #endif
diff --git a/iocore/net/Makefile.am b/iocore/net/Makefile.am
index f994b54..252a4c3 100644
--- a/iocore/net/Makefile.am
+++ b/iocore/net/Makefile.am
@@ -16,6 +16,8 @@
 #  See the License for the specific language governing permissions and
 #  limitations under the License.
 
+SUBDIRS = quic
+
 AM_CPPFLAGS += \
   $(iocore_include_dirs) \
   -I$(abs_top_srcdir)/lib \
@@ -112,6 +114,9 @@ libinknet_a_SOURCES = \
   P_UDPIOEvent.h \
   P_UDPNet.h \
   P_UDPPacket.h \
+  P_QUICPacketHandler.h \
+  P_QUICNetProcessor.h \
+  P_QUICNetVConnection.h \
   P_UnixCompletionUtil.h \
   P_UnixNet.h \
   P_UnixNetProcessor.h \
@@ -134,6 +139,9 @@ libinknet_a_SOURCES = \
   OCSPStapling.cc \
   Socks.cc \
   UDPIOEvent.cc \
+  QUICPacketHandler.cc \
+  QUICNetProcessor.cc \
+  QUICNetVConnection.cc \
   UnixConnection.cc \
   UnixNet.cc \
   UnixNetAccept.cc \
diff --git a/iocore/net/P_Net.h b/iocore/net/P_Net.h
index 9fecf19..87456c4 100644
--- a/iocore/net/P_Net.h
+++ b/iocore/net/P_Net.h
@@ -109,6 +109,11 @@ extern RecRawStatBlock *net_rsb;
 #include "P_SSLNetAccept.h"
 #include "P_SSLCertLookup.h"
 
+#include "P_QUICNetVConnection.h"
+#include "P_QUICNetProcessor.h"
+#include "P_QUICPacketHandler.h"
+// #include "P_QUICCertLookup.h"
+
 #undef NET_SYSTEM_MODULE_VERSION
 #define NET_SYSTEM_MODULE_VERSION \
   makeModuleVersion(NET_SYSTEM_MODULE_MAJOR_VERSION, NET_SYSTEM_MODULE_MINOR_VERSION, PRIVATE_MODULE_HEADER)
diff --git a/iocore/net/P_QUICNetProcessor.h b/iocore/net/P_QUICNetProcessor.h
new file mode 100644
index 0000000..461afdd
--- /dev/null
+++ b/iocore/net/P_QUICNetProcessor.h
@@ -0,0 +1,73 @@
+/** @file
+
+  A brief file description
+
+  @section license License
+
+  Licensed to the Apache Software Foundation (ASF) under one
+  or more contributor license agreements.  See the NOTICE file
+  distributed with this work for additional information
+  regarding copyright ownership.  The ASF licenses this file
+  to you under the Apache License, Version 2.0 (the
+  "License"); you may not use this file except in compliance
+  with the License.  You may obtain a copy of the License at
+
+      http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+
+ */
+
+/****************************************************************************
+
+  P_QUICNetProcessor.h
+
+  The QUIC version of the UnixNetProcessor class.  The majority of the logic
+  is in UnixNetProcessor.  The QUICNetProcessor provides the following:
+
+  * QUIC library initialization through the start() method.
+  * Allocation of a QUICNetVConnection through the allocate_vc virtual method.
+
+  Possibly another pass through could simplify the allocate_vc logic too, but
+  I think I will stop here for now.
+
+ ****************************************************************************/
+#pragma once
+
+#include "ts/ink_platform.h"
+#include "P_Net.h"
+// #include "P_QUICConfig.h"
+#include "ts/Map.h"
+
+class UnixNetVConnection;
+struct NetAccept;
+
+//////////////////////////////////////////////////////////////////
+//
+//  class QUICNetProcessor
+//
+//////////////////////////////////////////////////////////////////
+struct QUICNetProcessor : public UnixNetProcessor {
+public:
+  QUICNetProcessor();
+  virtual ~QUICNetProcessor();
+
+  virtual int start(int, size_t stacksize) override;
+  void cleanup();
+
+  virtual NetAccept *createNetAccept(const NetProcessor::AcceptOptions &opt) override;
+  virtual NetVConnection *allocate_vc(EThread *t) override;
+
+  Action *main_accept(Continuation *cont, SOCKET fd, AcceptOptions const &opt) override;
+
+private:
+  QUICNetProcessor(const QUICNetProcessor &);
+  QUICNetProcessor &operator=(const QUICNetProcessor &);
+
+  SSL_CTX *_ssl_ctx;
+};
+
+extern QUICNetProcessor quic_NetProcessor;
diff --git a/iocore/net/P_QUICNetVConnection.h b/iocore/net/P_QUICNetVConnection.h
new file mode 100644
index 0000000..e0bd7cc
--- /dev/null
+++ b/iocore/net/P_QUICNetVConnection.h
@@ -0,0 +1,205 @@
+/** @file
+
+  A brief file description
+
+  @section license License
+
+  Licensed to the Apache Software Foundation (ASF) under one
+  or more contributor license agreements.  See the NOTICE file
+  distributed with this work for additional information
+  regarding copyright ownership.  The ASF licenses this file
+  to you under the Apache License, Version 2.0 (the
+  "License"); you may not use this file except in compliance
+  with the License.  You may obtain a copy of the License at
+
+      http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+ */
+
+/****************************************************************************
+
+  QUICNetVConnection.h
+
+  This file implements an I/O Processor for network I/O.
+
+
+ ****************************************************************************/
+#pragma once
+
+#include <queue>
+
+#include "ts/ink_platform.h"
+#include "P_Net.h"
+#include "P_EventSystem.h"
+#include "P_UnixNetVConnection.h"
+#include "P_UnixNet.h"
+#include "P_UDPNet.h"
+#include "ts/apidefs.h"
+#include "ts/List.h"
+
+#include "quic/QUICVersionNegotiator.h"
+#include "quic/QUICPacket.h"
+#include "quic/QUICFrame.h"
+#include "quic/QUICFrameDispatcher.h"
+#include "quic/QUICHandshake.h"
+#include "quic/QUICApplication.h"
+#include "quic/QUICStream.h"
+#include "quic/QUICCrypto.h"
+#include "quic/QUICAckFrameCreator.h"
+#include "quic/QUICLossDetector.h"
+#include "quic/QUICPacketTransmitter.h"
+#include "quic/QUICFrameTransmitter.h"
+#include "quic/QUICStreamManager.h"
+#include "quic/QUICConnectionManager.h"
+#include "quic/QUICFlowController.h"
+#include "quic/QUICCongestionController.h"
+
+// These are included here because older OpenQUIC libraries don't have them.
+// Don't copy these defines, or use their values directly, they are merely
+// here to avoid compiler errors.
+#ifndef QUIC_TLSEXT_ERR_OK
+#define QUIC_TLSEXT_ERR_OK 0
+#endif
+
+#ifndef QUIC_TLSEXT_ERR_NOACK
+#define QUIC_TLSEXT_ERR_NOACK 3
+#endif
+
+#define QUIC_OP_HANDSHAKE 0x16
+
+// TS-2503: dynamic TLS record sizing
+// For smaller records, we should also reserve space for various TCP options
+// (timestamps, SACKs.. up to 40 bytes [1]), and account for TLS record overhead
+// (another 20-60 bytes on average, depending on the negotiated ciphersuite [2]).
+// All in all: 1500 - 40 (IP) - 20 (TCP) - 40 (TCP options) - TLS overhead (60-100)
+// For larger records, the size is determined by TLS protocol record size
+#define QUIC_DEF_TLS_RECORD_SIZE 1300  // 1500 - 40 (IP) - 20 (TCP) - 40 (TCP options) - TLS overhead (60-100)
+#define QUIC_MAX_TLS_RECORD_SIZE 16383 // 2^14 - 1
+#define QUIC_DEF_TLS_RECORD_BYTE_THRESHOLD 1000000
+#define QUIC_DEF_TLS_RECORD_MSEC_THRESHOLD 1000
+
+// class QUICNextProtocolSet;
+// struct QUICCertLookup;
+
+typedef enum {
+  QUIC_HOOK_OP_DEFAULT,                      ///< Null / initialization value. Do normal processing.
+  QUIC_HOOK_OP_TUNNEL,                       ///< Switch to blind tunnel
+  QUIC_HOOK_OP_TERMINATE,                    ///< Termination connection / transaction.
+  QUIC_HOOK_OP_LAST = QUIC_HOOK_OP_TERMINATE ///< End marker value.
+} QuicVConnOp;
+
+//////////////////////////////////////////////////////////////////
+//
+//  class NetVConnection
+//
+//  A VConnection for a network socket.
+//
+//////////////////////////////////////////////////////////////////
+
+typedef std::unique_ptr<uint8_t> ats_uint8_t_unique_ptr;
+
+struct QUICPacketHandler;
+class QUICLossDetector;
+
+/**
+ * @class QUICNetVConnection
+ * @brief A NetVConnection for a QUIC network socket
+ * @detail
+ *
+ * state_handshake()
+ *  | READ:
+ *  |  _state_handshake_process_initial_client_packet()
+ *  |  _state_handshake_process_client_cleartext_packet()
+ *  |  _state_handshake_process_zero_rtt_protected_packet()
+ *  | WRITE:
+ *  |  _state_common_send_packet()
+ *  v
+ * state_connection_established()
+ *  | READ:
+ *  |  _state_connection_established_process_packet()
+ *  | WRITE:
+ *  |  _state_common_send_packet()
+ *  v
+ *  X
+ *
+ **/
+class QUICNetVConnection : public UnixNetVConnection, public QUICPacketTransmitter, public QUICFrameTransmitter
+{
+  typedef UnixNetVConnection super; ///< Parent type.
+
+public:
+  QUICNetVConnection();
+
+  void init(UDPConnection *, QUICPacketHandler *);
+
+  void reenable(VIO *vio) override;
+  VIO *do_io_read(Continuation *c, int64_t nbytes, MIOBuffer *buf) override;
+  VIO *do_io_write(Continuation *c, int64_t nbytes, IOBufferReader *buf, bool owner = false) override;
+  int startEvent(int event, Event *e);
+  int state_handshake(int event, Event *data);
+  int state_connection_established(int event, Event *data);
+  void start(SSL_CTX *);
+  uint32_t maximum_quic_packet_size();
+  uint32_t minimum_quic_packet_size();
+  virtual void transmit_packet(std::unique_ptr<const QUICPacket> packet) override;
+  virtual void retransmit_packet(const QUICPacket &packet) override;
+  virtual Ptr<ProxyMutex> get_transmitter_mutex() override;
+  void push_packet(std::unique_ptr<const QUICPacket> packet);
+  virtual void transmit_frame(std::unique_ptr<QUICFrame, QUICFrameDeleterFunc> frame) override;
+  void close(QUICError error);
+  void free(EThread *t) override;
+
+  UDPConnection *get_udp_con();
+  QUICApplication *get_application(QUICStreamId stream_id);
+  QUICCrypto *get_crypto();
+
+  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;
+
+private:
+  QUICConnectionId _quic_connection_id;
+  UDPConnection *_udp_con            = nullptr;
+  QUICPacketHandler *_packet_handler = nullptr;
+  QUICPacketFactory _packet_factory;
+  QUICFrameFactory _frame_factory;
+  QUICAckFrameCreator _ack_frame_creator;
+
+  uint32_t _pmtu = 1280;
+
+  // TODO: use custom allocator and make them std::unique_ptr or std::shared_ptr
+  // or make them just member variables.
+  QUICVersionNegotiator *_version_negotiator         = nullptr;
+  QUICHandshake *_handshake_handler                  = nullptr;
+  QUICApplication *_application                      = nullptr;
+  QUICCrypto *_crypto                                = nullptr;
+  std::shared_ptr<QUICLossDetector> _loss_detector   = nullptr;
+  std::shared_ptr<QUICStreamManager> _stream_manager = nullptr;
+  QUICFrameDispatcher *_frame_dispatcher             = nullptr;
+
+  Queue<QUICPacket> _packet_recv_queue;
+  Queue<QUICPacket> _packet_send_queue;
+  std::queue<std::unique_ptr<QUICFrame, QUICFrameDeleterFunc>> _frame_buffer;
+
+  void _packetize_frames();
+  std::unique_ptr<QUICPacket> _build_packet(ats_unique_buf buf, size_t len, bool retransmittable,
+                                            QUICPacketType type = QUICPacketType::UNINITIALIZED);
+
+  void _recv_and_ack(const uint8_t *payload, uint16_t size, QUICPacketNumber packet_num);
+
+  QUICError _state_handshake_process_initial_client_packet(std::unique_ptr<const QUICPacket> packet);
+  QUICError _state_handshake_process_client_cleartext_packet(std::unique_ptr<const QUICPacket> packet);
+  QUICError _state_handshake_process_zero_rtt_protected_packet(std::unique_ptr<const QUICPacket> packet);
+  QUICError _state_connection_established_process_packet(std::unique_ptr<const QUICPacket> packet);
+  QUICError _state_common_send_packet();
+
+  Ptr<ProxyMutex> _transmitter_mutex;
+};
+
+typedef int (QUICNetVConnection::*QUICNetVConnHandler)(int, void *);
+
+extern ClassAllocator<QUICNetVConnection> quicNetVCAllocator;
diff --git a/iocore/net/P_QUICPacketHandler.h b/iocore/net/P_QUICPacketHandler.h
new file mode 100644
index 0000000..4d21060
--- /dev/null
+++ b/iocore/net/P_QUICPacketHandler.h
@@ -0,0 +1,47 @@
+/** @file
+
+  A brief file description
+
+  @section license License
+
+  Licensed to the Apache Software Foundation (ASF) under one
+  or more contributor license agreements.  See the NOTICE file
+  distributed with this work for additional information
+  regarding copyright ownership.  The ASF licenses this file
+  to you under the Apache License, Version 2.0 (the
+  "License"); you may not use this file except in compliance
+  with the License.  You may obtain a copy of the License at
+
+      http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+ */
+
+#pragma once
+
+#include "ts/ink_platform.h"
+#include "P_Connection.h"
+#include "P_NetAccept.h"
+#include "P_QUICNetVConnection.h"
+
+struct QUICPacketHandler : public NetAccept {
+public:
+  QUICPacketHandler(const NetProcessor::AcceptOptions &opt, SSL_CTX *);
+  virtual ~QUICPacketHandler();
+
+  virtual NetProcessor *getNetProcessor() const override;
+  virtual NetAccept *clone() const override;
+  virtual int acceptEvent(int event, void *e) override;
+  void init_accept(EThread *t) override;
+  void send_packet(const QUICPacket &packet, QUICNetVConnection *vc);
+
+private:
+  void _recv_packet(int event, UDPPacket *udpPacket);
+
+  Map<int64_t, QUICNetVConnection *> _connections;
+  SSL_CTX *_ssl_ctx;
+};
diff --git a/iocore/net/P_SSLUtils.h b/iocore/net/P_SSLUtils.h
index a3d530d..01a6298 100644
--- a/iocore/net/P_SSLUtils.h
+++ b/iocore/net/P_SSLUtils.h
@@ -161,6 +161,7 @@ void SSLDebugBufferPrint(const char *tag, const char *buffer, unsigned buflen, c
 
 // Load the SSL certificate configuration.
 bool SSLParseCertificateConfiguration(const SSLConfigParams *params, SSLCertLookup *lookup);
+bool SSLParseCertificateConfiguration(const SSLConfigParams *params, SSL_CTX *ssl_ctx);
 
 // Attach a SSL NetVC back pointer to a SSL session.
 void SSLNetVCAttach(SSL *ssl, SSLNetVConnection *vc);
diff --git a/iocore/net/QUICNetProcessor.cc b/iocore/net/QUICNetProcessor.cc
new file mode 100644
index 0000000..72fb4e1
--- /dev/null
+++ b/iocore/net/QUICNetProcessor.cc
@@ -0,0 +1,144 @@
+/** @file
+
+  @section license License
+
+  Licensed to the Apache Software Foundation (ASF) under one
+  or more contributor license agreements.  See the NOTICE file
+  distributed with this work for additional information
+  regarding copyright ownership.  The ASF licenses this file
+  to you under the Apache License, Version 2.0 (the
+  "License"); you may not use this file except in compliance
+  with the License.  You may obtain a copy of the License at
+
+      http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+ */
+
+#include "ts/ink_config.h"
+
+#include "P_Net.h"
+#include "ts/I_Layout.h"
+#include "I_RecHttp.h"
+// #include "P_QUICUtils.h"
+
+//
+// Global Data
+//
+
+QUICNetProcessor quic_NetProcessor;
+
+QUICNetProcessor::QUICNetProcessor()
+{
+}
+
+QUICNetProcessor::~QUICNetProcessor()
+{
+  cleanup();
+}
+
+void
+QUICNetProcessor::cleanup()
+{
+  SSL_CTX_free(this->_ssl_ctx);
+}
+
+int
+QUICNetProcessor::start(int, size_t stacksize)
+{
+  // This initialization order matters ...
+  // QUICInitializeLibrary();
+  // QUICConfig::startup();
+
+  // if (!QUICCertificateConfig::startup())
+  //   return -1;
+
+  // Acquire a QUICConfigParams instance *after* we start QUIC up.
+  // QUICConfig::scoped_config params;
+
+  // Initialize QUIC statistics. This depends on an initial set of certificates being loaded above.
+  // QUICInitializeStatistics();
+
+  // TODO: load certs from SSLConfig
+  this->_ssl_ctx = SSL_CTX_new(TLS_method());
+  SSL_CTX_set_min_proto_version(this->_ssl_ctx, TLS1_3_VERSION);
+  SSL_CTX_set_max_proto_version(this->_ssl_ctx, TLS1_3_VERSION);
+
+  SSLConfig::scoped_config params;
+  SSLParseCertificateConfiguration(params, this->_ssl_ctx);
+
+  if (SSL_CTX_check_private_key(this->_ssl_ctx) != 1) {
+    Error("check private key failed");
+    ink_assert(false);
+  }
+
+  return 0;
+}
+
+NetAccept *
+QUICNetProcessor::createNetAccept(const NetProcessor::AcceptOptions &opt)
+{
+  return (NetAccept *)new QUICPacketHandler(opt, this->_ssl_ctx);
+}
+
+NetVConnection *
+QUICNetProcessor::allocate_vc(EThread *t)
+{
+  QUICNetVConnection *vc;
+
+  if (t) {
+    vc = THREAD_ALLOC_INIT(quicNetVCAllocator, t);
+  } else {
+    if (likely(vc = quicNetVCAllocator.alloc())) {
+      vc->from_accept_thread = true;
+    }
+  }
+
+  return vc;
+}
+
+Action *
+QUICNetProcessor::main_accept(Continuation *cont, SOCKET fd, AcceptOptions const &opt)
+{
+  // UnixNetProcessor *this_unp = static_cast<UnixNetProcessor *>(this);
+  Debug("iocore_net_processor", "NetProcessor::main_accept - port %d,recv_bufsize %d, send_bufsize %d, sockopt 0x%0x",
+        opt.local_port, opt.recv_bufsize, opt.send_bufsize, opt.sockopt_flags);
+
+  ProxyMutex *mutex  = this_ethread()->mutex.get();
+  int accept_threads = opt.accept_threads; // might be changed.
+  IpEndpoint accept_ip;                    // local binding address.
+  // char thr_name[MAX_THREAD_NAME_LENGTH];
+
+  NetAccept *na = createNetAccept(opt);
+  if (opt.accept_threads < 0) {
+    REC_ReadConfigInteger(accept_threads, "proxy.config.accept_threads");
+  }
+  NET_INCREMENT_DYN_STAT(net_accepts_currently_open_stat);
+
+  if (opt.localhost_only) {
+    accept_ip.setToLoopback(opt.ip_family);
+  } else if (opt.local_ip.isValid()) {
+    accept_ip.assign(opt.local_ip);
+  } else {
+    accept_ip.setToAnyAddr(opt.ip_family);
+  }
+  ink_assert(0 < opt.local_port && opt.local_port < 65536);
+  accept_ip.port() = htons(opt.local_port);
+
+  na->accept_fn = net_accept;
+  na->server.fd = fd;
+  ats_ip_copy(&na->server.accept_addr, &accept_ip);
+
+  na->action_         = new NetAcceptAction();
+  *na->action_        = cont;
+  na->action_->server = &na->server;
+  na->init_accept();
+
+  udpNet.UDPBind((Continuation *)na, &na->server.accept_addr.sa, 1024000, 1024000);
+
+  return na->action_.get();
+}
diff --git a/iocore/net/QUICNetVConnection.cc b/iocore/net/QUICNetVConnection.cc
new file mode 100644
index 0000000..926568f
--- /dev/null
+++ b/iocore/net/QUICNetVConnection.cc
@@ -0,0 +1,530 @@
+/** @file
+
+  A brief file description
+
+  @section license License
+
+  Licensed to the Apache Software Foundation (ASF) under one
+  or more contributor license agreements.  See the NOTICE file
+  distributed with this work for additional information
+  regarding copyright ownership.  The ASF licenses this file
+  to you under the Apache License, Version 2.0 (the
+  "License"); you may not use this file except in compliance
+  with the License.  You may obtain a copy of the License at
+
+      http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+ */
+#include <climits>
+#include <string>
+
+#include "ts/ink_config.h"
+#include "ts/EventNotify.h"
+#include "records/I_RecHttp.h"
+#include "ts/Diags.h"
+
+#include "P_Net.h"
+#include "InkAPIInternal.h" // Added to include the quic_hook definitions
+#include "BIO_fastopen.h"
+#include "Log.h"
+
+#include "QUICEchoApp.h"
+#include "QUICDebugNames.h"
+#include "QUICEvents.h"
+
+#define STATE_FROM_VIO(_x) ((NetState *)(((char *)(_x)) - STATE_VIO_OFFSET))
+#define STATE_VIO_OFFSET ((uintptr_t) & ((NetState *)0)->vio)
+
+const static char *tag = "quic_net";
+
+const static uint32_t MINIMUM_MTU         = 1280;
+const static uint32_t MAX_PACKET_OVERHEAD = 25; // Max long header len(17) + FNV-1a hash len(8)
+
+ClassAllocator<QUICNetVConnection> quicNetVCAllocator("quicNetVCAllocator");
+
+QUICNetVConnection::QUICNetVConnection() : UnixNetVConnection()
+{
+  SET_HANDLER((NetVConnHandler)&QUICNetVConnection::state_handshake);
+}
+
+void
+QUICNetVConnection::init(UDPConnection *udp_con, QUICPacketHandler *packet_handler)
+{
+  this->_transmitter_mutex = new_ProxyMutex();
+  this->_udp_con           = udp_con;
+  this->_transmitter_mutex = new_ProxyMutex();
+  this->_packet_handler    = packet_handler;
+  this->_quic_connection_id.randomize();
+}
+
+VIO *
+QUICNetVConnection::do_io_read(Continuation *c, int64_t nbytes, MIOBuffer *buf)
+{
+  return super::do_io_read(c, nbytes, buf);
+}
+
+VIO *
+QUICNetVConnection::do_io_write(Continuation *c, int64_t nbytes, IOBufferReader *buf, bool owner)
+{
+  return super::do_io_write(c, nbytes, buf, owner);
+}
+
+int
+QUICNetVConnection::startEvent(int /*event ATS_UNUSED */, Event *e)
+{
+  return EVENT_DONE;
+}
+
+// XXX This might be called on ET_UDP thread
+void
+QUICNetVConnection::start(SSL_CTX *ssl_ctx)
+{
+  this->_version_negotiator = new QUICVersionNegotiator(&this->_packet_factory, this);
+  this->_crypto             = new QUICCrypto(ssl_ctx, this->netvc_context);
+  this->_packet_factory.set_crypto_module(this->_crypto);
+
+  // FIXME Should these have to be shared_ptr?
+  this->_loss_detector  = std::make_shared<QUICLossDetector>(this);
+  this->_stream_manager = std::make_shared<QUICStreamManager>();
+  this->_stream_manager->init(this);
+  this->_stream_manager->set_connection(this); // FIXME Want to remove;
+
+  std::shared_ptr<QUICConnectionManager> connectionManager       = std::make_shared<QUICConnectionManager>(this);
+  std::shared_ptr<QUICFlowController> flowController             = std::make_shared<QUICFlowController>();
+  std::shared_ptr<QUICCongestionController> congestionController = std::make_shared<QUICCongestionController>();
+  this->_frame_dispatcher =
+    new QUICFrameDispatcher(connectionManager, this->_stream_manager, flowController, congestionController, this->_loss_detector);
+
+  // TODO set timeout from conf
+  this->set_active_timeout(0);
+  this->set_inactivity_timeout(2);
+}
+
+// TODO: call free when close connection
+void
+QUICNetVConnection::free(EThread *t)
+{
+  this->_udp_con        = nullptr;
+  this->_packet_handler = nullptr;
+
+  delete this->_version_negotiator;
+  delete this->_handshake_handler;
+  delete this->_application;
+  delete this->_crypto;
+  delete this->_frame_dispatcher;
+  // XXX _loss_detector and _stream_manager are std::shared_ptr
+
+  // TODO: clear member variables like `UnixNetVConnection::free(EThread *t)`
+  this->mutex.clear();
+
+  if (from_accept_thread) {
+    quicNetVCAllocator.free(this);
+  } else {
+    THREAD_FREE(this, quicNetVCAllocator, t);
+  }
+}
+
+void
+QUICNetVConnection::reenable(VIO *vio)
+{
+  return;
+}
+
+uint32_t
+QUICNetVConnection::minimum_quic_packet_size()
+{
+  if (this->options.ip_family == PF_INET6) {
+    return MINIMUM_MTU - 48;
+  } else {
+    return MINIMUM_MTU - 28;
+  }
+}
+
+uint32_t
+QUICNetVConnection::maximum_quic_packet_size()
+{
+  if (this->options.ip_family == PF_INET6) {
+    return this->_pmtu - 48;
+  } else {
+    return this->_pmtu - 28;
+  }
+}
+
+void
+QUICNetVConnection::transmit_packet(std::unique_ptr<const QUICPacket> packet)
+{
+  // TODO Remove const_cast
+  this->_packet_send_queue.enqueue(const_cast<QUICPacket *>(packet.release()));
+  eventProcessor.schedule_imm(this, ET_CALL, QUIC_EVENT_PACKET_WRITE_READY, nullptr);
+}
+
+void
+QUICNetVConnection::retransmit_packet(const QUICPacket &packet)
+{
+  uint16_t size          = packet.payload_size();
+  const uint8_t *payload = packet.payload();
+
+  std::unique_ptr<QUICFrame, QUICFrameDeleterFunc> frame(nullptr, &QUICFrameDeleter::delete_null_frame);
+  uint16_t cursor = 0;
+
+  while (cursor < size) {
+    frame = QUICFrameFactory::create(payload + cursor, size - cursor);
+    cursor += frame->size();
+
+    switch (frame->type()) {
+    case QUICFrameType::PADDING:
+    case QUICFrameType::ACK:
+      break;
+    default:
+      frame = QUICFrameFactory::create_retransmission_frame(std::move(frame), packet);
+      this->transmit_frame(std::move(frame));
+      break;
+    }
+  }
+}
+
+Ptr<ProxyMutex>
+QUICNetVConnection::get_transmitter_mutex()
+{
+  return this->_transmitter_mutex;
+}
+
+void
+QUICNetVConnection::push_packet(std::unique_ptr<QUICPacket const> packet)
+{
+  Debug(tag, "Type=%s Size=%u", QUICDebugNames::packet_type(packet->type()), packet->size());
+  this->_packet_recv_queue.enqueue(const_cast<QUICPacket *>(packet.release()));
+}
+
+void
+QUICNetVConnection::transmit_frame(std::unique_ptr<QUICFrame, QUICFrameDeleterFunc> frame)
+{
+  Debug(tag, "Type=%s Size=%zu", QUICDebugNames::frame_type(frame->type()), frame->size());
+  this->_frame_buffer.push(std::move(frame));
+  eventProcessor.schedule_imm(this, ET_CALL, QUIC_EVENT_PACKET_WRITE_READY, nullptr);
+}
+
+void
+QUICNetVConnection::close(QUICError error)
+{
+  this->transmit_frame(QUICFrameFactory::create_connection_close_frame(error.code, 0, ""));
+}
+
+int
+QUICNetVConnection::state_handshake(int event, Event *data)
+{
+  QUICError error;
+
+  if (!thread) {
+    thread = this_ethread();
+  }
+
+  if (!nh) {
+    nh = get_NetHandler(this_ethread());
+  }
+
+  switch (event) {
+  case QUIC_EVENT_PACKET_READ_READY: {
+    std::unique_ptr<const QUICPacket> p = std::unique_ptr<const QUICPacket>(this->_packet_recv_queue.dequeue());
+    switch (p->type()) {
+    case QUICPacketType::CLIENT_INITIAL:
+      error = this->_state_handshake_process_initial_client_packet(std::move(p));
+      break;
+    case QUICPacketType::CLIENT_CLEARTEXT:
+      error = this->_state_handshake_process_client_cleartext_packet(std::move(p));
+      break;
+    case QUICPacketType::ZERO_RTT_PROTECTED:
+      error = this->_state_handshake_process_zero_rtt_protected_packet(std::move(p));
+      break;
+    default:
+      error = QUICError(QUICErrorClass::QUIC_TRANSPORT, QUICErrorCode::QUIC_INTERNAL_ERROR);
+      break;
+    }
+
+    break;
+  }
+  case QUIC_EVENT_PACKET_WRITE_READY: {
+    error = this->_state_common_send_packet();
+    break;
+  }
+  default:
+    Debug(tag, "Unexpected event: %u", event);
+  }
+
+  if (error.cls != QUICErrorClass::NONE) {
+    // TODO: Send error if needed
+    Debug(tag, "QUICError: cls=%u, code=0x%x", error.cls, error.code);
+  }
+
+  if (this->_handshake_handler && this->_handshake_handler->is_completed()) {
+    Debug(tag, "Enter state_connection_established");
+    SET_HANDLER((NetVConnHandler)&QUICNetVConnection::state_connection_established);
+  }
+
+  return EVENT_CONT;
+}
+
+int
+QUICNetVConnection::state_connection_established(int event, Event *data)
+{
+  QUICError error;
+  switch (event) {
+  case QUIC_EVENT_PACKET_READ_READY: {
+    std::unique_ptr<const QUICPacket> p = std::unique_ptr<const QUICPacket>(this->_packet_recv_queue.dequeue());
+    switch (p->type()) {
+    case QUICPacketType::ONE_RTT_PROTECTED_KEY_PHASE_0:
+    case QUICPacketType::ONE_RTT_PROTECTED_KEY_PHASE_1:
+      error = this->_state_connection_established_process_packet(std::move(p));
+      break;
+    default:
+      error = QUICError(QUICErrorClass::QUIC_TRANSPORT, QUICErrorCode::QUIC_INTERNAL_ERROR);
+      break;
+    }
+    break;
+  }
+  case QUIC_EVENT_PACKET_WRITE_READY: {
+    error = this->_state_common_send_packet();
+    break;
+  }
+  }
+
+  if (error.cls != QUICErrorClass::NONE) {
+    // TODO: Send error if needed
+    Debug(tag, "QUICError: cls=%u, code=0x%x", error.cls, error.code);
+  }
+
+  return EVENT_CONT;
+}
+
+UDPConnection *
+QUICNetVConnection::get_udp_con()
+{
+  return this->_udp_con;
+}
+
+QUICApplication *
+QUICNetVConnection::get_application(QUICStreamId stream_id)
+{
+  if (stream_id == STREAM_ID_FOR_HANDSHAKE) {
+    return static_cast<QUICApplication *>(this->_handshake_handler);
+  } else {
+    if (!this->_application) {
+      Debug(tag, "setup quic application");
+      // TODO: Instantiate negotiated application
+      const uint8_t *application = this->_handshake_handler->negotiated_application_name();
+      if (memcmp(application, "hq", 2) == 0) {
+        QUICEchoApp *echo_app = new QUICEchoApp(new_ProxyMutex(), this);
+        this->_application    = echo_app;
+      }
+    }
+  }
+  return this->_application;
+}
+
+QUICCrypto *
+QUICNetVConnection::get_crypto()
+{
+  return this->_crypto;
+}
+
+void
+QUICNetVConnection::net_read_io(NetHandler *nh, EThread *lthread)
+{
+  ink_assert(false);
+
+  return;
+}
+
+int64_t
+QUICNetVConnection::load_buffer_and_write(int64_t towrite, MIOBufferAccessor &buf, int64_t &total_written, int &needs)
+{
+  ink_assert(false);
+
+  return 0;
+}
+
+QUICError
+QUICNetVConnection::_state_handshake_process_initial_client_packet(std::unique_ptr<const QUICPacket> packet)
+{
+  if (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()) {
+      if (this->_version_negotiator->negotiate(packet.get()) == QUICVersionNegotiationStatus::NEGOTIATED) {
+        Debug(tag, "Version negotiation succeeded: %x", packet->version());
+        this->_packet_factory.set_version(packet->version());
+        // Check integrity (QUIC-TLS-04: 6.1. Integrity Check Processing)
+        if (packet->has_valid_fnv1a_hash()) {
+          this->_handshake_handler = new QUICHandshake(new_ProxyMutex(), this);
+          this->_frame_dispatcher->receive_frames(packet->payload(), packet->payload_size());
+        } else {
+          Debug(tag, "Invalid FNV-1a hash value");
+        }
+      } else {
+        Debug(tag, "Version negotiation failed: %x", packet->version());
+      }
+    } else {
+      return QUICError(QUICErrorClass::QUIC_TRANSPORT, QUICErrorCode::QUIC_INTERNAL_ERROR);
+    }
+  }
+
+  return QUICError(QUICErrorClass::NONE);
+}
+
+QUICError
+QUICNetVConnection::_state_handshake_process_client_cleartext_packet(std::unique_ptr<const QUICPacket> packet)
+{
+  // The payload of this packet contains STREAM frames and could contain PADDING and ACK frames
+  if (packet->has_valid_fnv1a_hash()) {
+    this->_recv_and_ack(packet->payload(), packet->payload_size(), packet->packet_number());
+  } else {
+    Debug(tag, "Invalid FNV-1a hash value");
+  }
+  return QUICError(QUICErrorClass::NONE);
+}
+
+QUICError
+QUICNetVConnection::_state_handshake_process_zero_rtt_protected_packet(std::unique_ptr<const QUICPacket> packet)
+{
+  // TODO: Decrypt the packet
+  // decrypt(payload, p);
+  // TODO: Not sure what we have to do
+  return QUICError(QUICErrorClass::NONE);
+}
+
+QUICError
+QUICNetVConnection::_state_connection_established_process_packet(std::unique_ptr<const QUICPacket> packet)
+{
+  // TODO: fix size
+  size_t max_plain_txt_len = 2048;
+  ats_unique_buf plain_txt = ats_unique_malloc(max_plain_txt_len);
+  size_t plain_txt_len     = 0;
+
+  if (this->_crypto->decrypt(plain_txt.get(), plain_txt_len, max_plain_txt_len, packet->payload(), packet->payload_size(),
+                             packet->packet_number(), packet->header(), packet->header_size(), packet->key_phase())) {
+    Debug(tag, "Decrypt Packet, pkt_num: %llu, header_len: %hu, payload_len: %zu", packet->packet_number(), packet->header_size(),
+          plain_txt_len);
+
+    this->_recv_and_ack(plain_txt.get(), plain_txt_len, packet->packet_number());
+
+    return QUICError(QUICErrorClass::NONE);
+  } else {
+    Debug(tag, "CRYPTOGRAPHIC Error");
+
+    return QUICError(QUICErrorClass::CRYPTOGRAPHIC);
+  }
+}
+
+QUICError
+QUICNetVConnection::_state_common_send_packet()
+{
+  this->_packetize_frames();
+
+  const QUICPacket *packet;
+  while ((packet = this->_packet_send_queue.dequeue()) != nullptr) {
+    this->_packet_handler->send_packet(*packet, this);
+    this->_loss_detector->on_packet_sent(std::unique_ptr<const QUICPacket>(packet));
+  }
+
+  return QUICError(QUICErrorClass::NONE);
+}
+
+void
+QUICNetVConnection::_packetize_frames()
+{
+  uint32_t max_size = this->maximum_quic_packet_size();
+  uint32_t min_size = this->minimum_quic_packet_size();
+  ats_unique_buf buf(nullptr, [](void *p) { ats_free(p); });
+  size_t len = 0;
+
+  // Put frames into buf as many as possible
+  std::unique_ptr<QUICFrame, QUICFrameDeleterFunc> frame(nullptr, nullptr);
+  bool retransmittable                = false;
+  QUICPacketType previous_packet_type = QUICPacketType::UNINITIALIZED;
+  QUICPacketType current_packet_type  = QUICPacketType::UNINITIALIZED;
+
+  while (this->_frame_buffer.size() > 0) {
+    frame = std::move(this->_frame_buffer.front());
+    this->_frame_buffer.pop();
+    QUICRetransmissionFrame *rf = dynamic_cast<QUICRetransmissionFrame *>(frame.get());
+    previous_packet_type        = current_packet_type;
+    if (rf) {
+      current_packet_type = rf->packet_type();
+    } else {
+      current_packet_type = QUICPacketType::UNINITIALIZED;
+    }
+    if (len + frame->size() + MAX_PACKET_OVERHEAD > max_size || (previous_packet_type != current_packet_type && len > 0)) {
+      ink_assert(len > 0);
+      SCOPED_MUTEX_LOCK(transmitter_lock, this->get_transmitter_mutex().get(), this_ethread());
+      this->transmit_packet(this->_build_packet(std::move(buf), len, retransmittable, previous_packet_type));
+      len = 0;
+    }
+    retransmittable = retransmittable || (frame->type() != QUICFrameType::ACK && frame->type() != QUICFrameType::PADDING);
+
+    if (buf == nullptr) {
+      buf = ats_unique_malloc(max_size);
+    }
+    size_t l = 0;
+    Debug(tag, "type=%s", QUICDebugNames::frame_type(frame->type()));
+    frame->store(buf.get() + len, &l);
+    len += l;
+  }
+
+  if (len != 0) {
+    // Pad with PADDING frames
+    if (min_size > len) {
+      // FIXME QUICNetVConnection should not know the actual type value of PADDING frame
+      memset(buf.get() + len, 0, min_size - len);
+      len += min_size - len;
+    }
+    SCOPED_MUTEX_LOCK(transmitter_lock, this->get_transmitter_mutex().get(), this_ethread());
+    this->transmit_packet(this->_build_packet(std::move(buf), len, retransmittable, current_packet_type));
+  }
+}
+
+void
+QUICNetVConnection::_recv_and_ack(const uint8_t *payload, uint16_t size, QUICPacketNumber packet_num)
+{
+  bool should_send_ack = this->_frame_dispatcher->receive_frames(payload, size);
+  this->_ack_frame_creator.update(packet_num, should_send_ack);
+  std::unique_ptr<QUICFrame, QUICFrameDeleterFunc> ack_frame = this->_ack_frame_creator.create_if_needed();
+  if (ack_frame != nullptr) {
+    this->transmit_frame(std::move(ack_frame));
+    eventProcessor.schedule_imm(this, ET_CALL, QUIC_EVENT_PACKET_WRITE_READY, nullptr);
+  }
+}
+
+std::unique_ptr<QUICPacket>
+QUICNetVConnection::_build_packet(ats_unique_buf buf, size_t len, bool retransmittable, QUICPacketType type)
+{
+  std::unique_ptr<QUICPacket> packet;
+  Debug(tag, "retransmittable %u", retransmittable);
+
+  switch (type) {
+  case QUICPacketType::SERVER_CLEARTEXT:
+    packet = this->_packet_factory.create_server_cleartext_packet(this->_quic_connection_id, std::move(buf), len, retransmittable);
+    break;
+  default:
+    if (this->_handshake_handler && this->_handshake_handler->is_completed()) {
+      packet =
+        this->_packet_factory.create_server_protected_packet(this->_quic_connection_id, std::move(buf), len, retransmittable);
+    } else {
+      packet =
+        this->_packet_factory.create_server_cleartext_packet(this->_quic_connection_id, std::move(buf), len, retransmittable);
+    }
+    break;
+  }
+
+  return packet;
+}
diff --git a/iocore/net/QUICPacketHandler.cc b/iocore/net/QUICPacketHandler.cc
new file mode 100644
index 0000000..8847429
--- /dev/null
+++ b/iocore/net/QUICPacketHandler.cc
@@ -0,0 +1,148 @@
+/** @file
+
+  @section license License
+
+  Licensed to the Apache Software Foundation (ASF) under one
+  or more contributor license agreements.  See the NOTICE file
+  distributed with this work for additional information
+  regarding copyright ownership.  The ASF licenses this file
+  to you under the Apache License, Version 2.0 (the
+  "License"); you may not use this file except in compliance
+  with the License.  You may obtain a copy of the License at
+
+      http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+ */
+
+#include "ts/ink_config.h"
+#include "P_Net.h"
+
+#include "QUICPacket.h"
+#include "QUICDebugNames.h"
+#include "QUICEvents.h"
+
+QUICPacketHandler::QUICPacketHandler(const NetProcessor::AcceptOptions &opt, SSL_CTX *ctx) : NetAccept(opt), _ssl_ctx(ctx)
+{
+  this->mutex = new_ProxyMutex();
+}
+
+QUICPacketHandler::~QUICPacketHandler()
+{
+}
+
+NetProcessor *
+QUICPacketHandler::getNetProcessor() const
+{
+  return &quic_NetProcessor;
+}
+
+NetAccept *
+QUICPacketHandler::clone() const
+{
+  NetAccept *na;
+  na  = new QUICPacketHandler(opt, this->_ssl_ctx);
+  *na = *this;
+  return na;
+}
+
+int
+QUICPacketHandler::acceptEvent(int event, void *data)
+{
+  // NetVConnection *netvc;
+  ink_release_assert(event == NET_EVENT_DATAGRAM_OPEN || event == NET_EVENT_DATAGRAM_READ_READY ||
+                     event == NET_EVENT_DATAGRAM_ERROR);
+  ink_release_assert((event == NET_EVENT_DATAGRAM_OPEN) ? (data != nullptr) : (1));
+  ink_release_assert((event == NET_EVENT_DATAGRAM_READ_READY) ? (data != nullptr) : (1));
+
+  if (event == NET_EVENT_DATAGRAM_OPEN) {
+    // Nothing to do.
+    return EVENT_CONT;
+  } else if (event == NET_EVENT_DATAGRAM_READ_READY) {
+    Queue<UDPPacket> *queue = (Queue<UDPPacket> *)data;
+    UDPPacket *packet_r;
+    ip_port_text_buffer ipb;
+    while ((packet_r = queue->dequeue())) {
+      Debug("quic_sec", "received packet from %s, size=%lld", ats_ip_nptop(&packet_r->from.sa, ipb, sizeof(ipb)),
+            packet_r->getPktLength());
+      this->_recv_packet(event, packet_r);
+    }
+    return EVENT_CONT;
+  }
+
+  /////////////////
+  // EVENT_ERROR //
+  /////////////////
+  if (((long)data) == -ECONNABORTED) {
+  }
+
+  ink_abort("QUIC accept received fatal error: errno = %d", -((int)(intptr_t)data));
+  return EVENT_CONT;
+  return 0;
+}
+
+void
+QUICPacketHandler::init_accept(EThread *t = nullptr)
+{
+  SET_HANDLER(&QUICPacketHandler::acceptEvent);
+}
+
+void
+QUICPacketHandler::_recv_packet(int event, UDPPacket *udpPacket)
+{
+  IOBufferBlock *block = udpPacket->getIOBlockChain();
+
+  std::unique_ptr<QUICPacket> qPkt = std::unique_ptr<QUICPacket>(QUICPacketFactory::create(block));
+  QUICNetVConnection *vc           = this->_connections.get(qPkt->connection_id());
+
+  if (!vc) {
+    // Unknown Connection ID
+    Connection con;
+    con.setRemote(&udpPacket->from.sa);
+    vc =
+      static_cast<QUICNetVConnection *>(getNetProcessor()->allocate_vc(((UnixUDPConnection *)udpPacket->getConnection())->ethread));
+    vc->init(udpPacket->getConnection(), this);
+    vc->id = net_next_connection_number();
+    vc->con.move(con);
+    vc->submit_time = Thread::get_hrtime();
+    vc->mutex       = this->mutex;
+    vc->action_     = *this->action_;
+    vc->set_is_transparent(this->opt.f_inbound_transparent);
+    vc->set_context(NET_VCONNECTION_IN);
+    vc->read.triggered = 1;
+    vc->start(this->_ssl_ctx);
+    // TODO: Handle Connection ID of Client Cleartext / Non-Final Server Cleartext Packet
+    this->_connections.put(qPkt->connection_id(), vc);
+  }
+
+  vc->push_packet(std::move(qPkt));
+
+  // send to EThread
+  eventProcessor.schedule_imm(vc, ET_CALL, QUIC_EVENT_PACKET_READ_READY, nullptr);
+}
+
+// TODO: Should be called via eventProcessor?
+void
+QUICPacketHandler::send_packet(const QUICPacket &packet, QUICNetVConnection *vc)
+{
+  // TODO: remove a connection which is created by Client Initial
+  //       or update key to new one
+  if (!this->_connections.get(packet.connection_id())) {
+    this->_connections.put(packet.connection_id(), vc);
+  }
+
+  uint8_t udp_payload[65536];
+  size_t udp_len;
+  packet.store(udp_payload, &udp_len);
+  UDPPacket *udpPkt = new_UDPPacket(vc->con.addr, 0, reinterpret_cast<char *>(udp_payload), udp_len);
+
+  // NOTE: p will be enqueued to udpOutQueue of UDPNetHandler
+  ip_port_text_buffer ipb;
+  Debug("quic_sec", "send %s packet to %s, size=%lld", QUICDebugNames::packet_type(packet.type()),
+        ats_ip_nptop(&udpPkt->to.sa, ipb, sizeof(ipb)), udpPkt->getPktLength());
+  vc->get_udp_con()->send(this, udpPkt);
+}
diff --git a/iocore/net/SSLUtils.cc b/iocore/net/SSLUtils.cc
index 49cb5e3..a01510e 100644
--- a/iocore/net/SSLUtils.cc
+++ b/iocore/net/SSLUtils.cc
@@ -1965,6 +1965,81 @@ ssl_extract_certificate(const matcher_line *line_info, ssl_user_config &sslMultC
   return true;
 }
 
+// TODO: remove this function and setup SSL_CTX for QUIC somehow
+bool
+SSLParseCertificateConfiguration(const SSLConfigParams *params, SSL_CTX *ssl_ctx)
+{
+  char *tok_state = nullptr;
+  char *line      = nullptr;
+  ats_scoped_str file_buf;
+  unsigned line_num = 0;
+  matcher_line line_info;
+
+  const matcher_tags sslCertTags = {nullptr, nullptr, nullptr, nullptr, nullptr, nullptr, false};
+
+  Note("loading SSL certificate configuration from %s", params->configFilePath);
+
+  if (params->configFilePath) {
+    file_buf = readIntoBuffer(params->configFilePath, __func__, nullptr);
+  }
+
+  if (!file_buf) {
+    Error("failed to read SSL certificate configuration from %s", params->configFilePath);
+    return false;
+  }
+
+  // Optionally elevate/allow file access to read root-only
+  // certificates. The destructor will drop privilege for us.
+  uint32_t elevate_setting = 0;
+  REC_ReadConfigInteger(elevate_setting, "proxy.config.ssl.cert.load_elevated");
+  ElevateAccess elevate_access(elevate_setting ? ElevateAccess::FILE_PRIVILEGE : 0);
+
+  line = tokLine(file_buf, &tok_state);
+  while (line != nullptr) {
+    line_num++;
+
+    // Skip all blank spaces at beginning of line.
+    while (*line && isspace(*line)) {
+      line++;
+    }
+
+    if (*line != '\0' && *line != '#') {
+      ssl_user_config sslMultiCertSettings;
+      const char *errPtr;
+
+      errPtr = parseConfigLine(line, &line_info, &sslCertTags);
+
+      if (errPtr != nullptr) {
+        RecSignalWarning(REC_SIGNAL_CONFIG_ERROR, "%s: discarding %s entry at line %d: %s", __func__, params->configFilePath,
+                         line_num, errPtr);
+      } else {
+        if (ssl_extract_certificate(&line_info, sslMultiCertSettings)) {
+          // There must be a certificate specified unless the tunnel action is set
+          if (sslMultiCertSettings.cert || sslMultiCertSettings.opt != SSLCertContext::OPT_TUNNEL) {
+            if (SSL_CTX_use_PrivateKey_file(ssl_ctx, sslMultiCertSettings.key.get(), SSL_FILETYPE_PEM) != 1) {
+              Error("Couldn't load private_key: %s", sslMultiCertSettings.key.get());
+              return false;
+            }
+
+            if (SSL_CTX_use_certificate_chain_file(ssl_ctx, sslMultiCertSettings.cert.get()) != 1) {
+              Error("Couldn't load cert: %s", sslMultiCertSettings.cert.get());
+              return false;
+            }
+
+            return true;
+
+          } else {
+            Warning("No ssl_cert_name specified and no tunnel action set");
+          }
+        }
+      }
+    }
+
+    line = tokLine(nullptr, &tok_state);
+  }
+  return true;
+}
+
 bool
 SSLParseCertificateConfiguration(const SSLConfigParams *params, SSLCertLookup *lookup)
 {
diff --git a/iocore/net/quic/Makefile.am b/iocore/net/quic/Makefile.am
new file mode 100644
index 0000000..b0b799a
--- /dev/null
+++ b/iocore/net/quic/Makefile.am
@@ -0,0 +1,66 @@
+# Makefile.am for the traffic/iocore/net hierarchy
+#
+#  Licensed to the Apache Software Foundation (ASF) under one
+#  or more contributor license agreements.  See the NOTICE file
+#  distributed with this work for additional information
+#  regarding copyright ownership.  The ASF licenses this file
+#  to you under the Apache License, Version 2.0 (the
+#  "License"); you may not use this file except in compliance
+#  with the License.  You may obtain a copy of the License at
+#
+#      http://www.apache.org/licenses/LICENSE-2.0
+#
+#  Unless required by applicable law or agreed to in writing, software
+#  distributed under the License is distributed on an "AS IS" BASIS,
+#  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+#  See the License for the specific language governing permissions and
+#  limitations under the License.
+
+SUBDIRS = test
+
+AM_CPPFLAGS += \
+  $(iocore_include_dirs) \
+  -I$(abs_top_srcdir)/lib \
+  -I$(abs_top_srcdir)/lib/records \
+  -I$(abs_top_srcdir)/proxy \
+  -I$(abs_top_srcdir)/proxy/hdrs \
+  -I$(abs_top_srcdir)/proxy/shared \
+  -I$(abs_top_srcdir)/proxy/logging \
+  -I$(abs_top_srcdir)/mgmt \
+  -I$(abs_top_srcdir)/mgmt/utils \
+  -I$(abs_top_srcdir)/proxy/http \
+  @OPENSSL_INCLUDES@
+
+noinst_LIBRARIES = libquic.a
+
+if OPENSSL_IS_BORINGSSL
+QUICCrypto_impl = QUICCrypto_boringssl.cc
+else
+QUICCrypto_impl = QUICCrypto_openssl.cc
+endif
+
+libquic_a_SOURCES = \
+  QUICTypes.cc \
+  QUICPacket.cc \
+  QUICFrame.cc \
+  QUICFrameDispatcher.cc \
+  QUICVersionNegotiator.cc \
+  QUICConnectionManager.cc \
+  QUICLossDetector.cc \
+  QUICStreamManager.cc \
+  QUICFlowController.cc \
+  QUICCongestionController.cc \
+  QUICStreamState.cc \
+  QUICStream.cc \
+  QUICHandshake.cc \
+  QUICCrypto.cc \
+  $(QUICCrypto_impl) \
+  QUICAckFrameCreator.cc \
+  QUICDebugNames.cc \
+  QUICApplication.cc \
+  QUICEchoApp.cc
+
+include $(top_srcdir)/build/tidy.mk
+
+tidy-local: $(DIST_SOURCES)
+	$(CXX_Clang_Tidy)
diff --git a/iocore/net/quic/Mock.h b/iocore/net/quic/Mock.h
new file mode 100644
index 0000000..094bce3
--- /dev/null
+++ b/iocore/net/quic/Mock.h
@@ -0,0 +1,233 @@
+#include "QUICConnectionManager.h"
+#include "QUICStreamManager.h"
+#include "QUICFlowController.h"
+#include "QUICCongestionController.h"
+#include "QUICLossDetector.h"
+#include "QUICEvents.h"
+#include "QUICPacketTransmitter.h"
+
+class MockQUICPacketTransmitter : public QUICPacketTransmitter
+{
+public:
+  MockQUICPacketTransmitter() : QUICPacketTransmitter() {
+    this->_mutex= new_ProxyMutex();
+  };
+
+  void
+  transmit_packet(std::unique_ptr<const QUICPacket> packet) override
+  {
+    ++_transmit_count;
+  }
+
+  void
+  retransmit_packet(const QUICPacket &packet) override
+  {
+    ++_retransmit_count;
+  }
+
+  Ptr<ProxyMutex>
+  get_transmitter_mutex() override
+  {
+    return this->_mutex;
+  }
+
+  int _transmit_count   = 0;
+  int _retransmit_count = 0;
+  Ptr<ProxyMutex> _mutex;
+};
+
+class MockQUICFrameTransmitter : public QUICFrameTransmitter
+{
+  void
+  transmit_frame(std::unique_ptr<QUICFrame, QUICFrameDeleterFunc> frame)
+  {
+  }
+};
+
+class MockQUICLossDetector : public QUICLossDetector
+{
+public:
+  MockQUICLossDetector() : QUICLossDetector(new MockQUICPacketTransmitter()) {}
+
+  void
+  rcv_frame(std::shared_ptr<const QUICFrame>)
+  {
+  }
+
+  void
+  on_packet_sent(std::unique_ptr<const QUICPacket> packet)
+  {
+  }
+};
+
+class MockQUICConnectionManager : public QUICConnectionManager
+{
+public:
+  MockQUICConnectionManager() : QUICConnectionManager(new MockQUICFrameTransmitter()) {}
+
+  // Override
+  virtual void
+  handle_frame(std::shared_ptr<const QUICFrame> f) override
+  {
+    ++_frameCount[static_cast<int>(f->type())];
+    ++_totalFrameCount;
+  }
+
+  // for Test
+  int
+  getStreamFrameCount()
+  {
+    return _frameCount[static_cast<int>(QUICFrameType::STREAM)];
+  }
+
+  int
+  getAckFrameCount()
+  {
+    return _frameCount[static_cast<int>(QUICFrameType::ACK)];
+  }
+
+  int
+  getPingFrameCount()
+  {
+    return _frameCount[static_cast<int>(QUICFrameType::PING)];
+  }
+
+  int
+  getTotalFrameCount()
+  {
+    return _totalFrameCount;
+  }
+
+private:
+  int _totalFrameCount = 0;
+  int _frameCount[256] = {0};
+};
+
+class MockQUICStreamManager : public QUICStreamManager
+{
+public:
+  MockQUICStreamManager() : QUICStreamManager() {}
+
+  // Override
+  virtual void
+  handle_frame(std::shared_ptr<const QUICFrame> f) override
+  {
+    ++_frameCount[static_cast<int>(f->type())];
+    ++_totalFrameCount;
+  }
+
+  // for Test
+  int
+  getStreamFrameCount()
+  {
+    return _frameCount[static_cast<int>(QUICFrameType::STREAM)];
+  }
+
+  int
+  getAckFrameCount()
+  {
+    return _frameCount[static_cast<int>(QUICFrameType::ACK)];
+  }
+
+  int
+  getPingFrameCount()
+  {
+    return _frameCount[static_cast<int>(QUICFrameType::PING)];
+  }
+
+  int
+  getTotalFrameCount()
+  {
+    return _totalFrameCount;
+  }
+
+private:
+  int _totalFrameCount = 0;
+  int _frameCount[256] = {0};
+};
+
+class MockQUICCongestionController : public QUICCongestionController
+{
+public:
+  MockQUICCongestionController() : QUICCongestionController() {}
+
+  // Override
+  virtual void
+  handle_frame(std::shared_ptr<const QUICFrame> f) override
+  {
+    ++_frameCount[static_cast<int>(f->type())];
+    ++_totalFrameCount;
+  }
+
+  // for Test
+  int
+  getStreamFrameCount()
+  {
+    return _frameCount[static_cast<int>(QUICFrameType::STREAM)];
+  }
+
+  int
+  getAckFrameCount()
+  {
+    return _frameCount[static_cast<int>(QUICFrameType::ACK)];
+  }
+
+  int
+  getPingFrameCount()
+  {
+    return _frameCount[static_cast<int>(QUICFrameType::PING)];
+  }
+
+  int
+  getTotalFrameCount()
+  {
+    return _totalFrameCount;
+  }
+
+private:
+  int _totalFrameCount = 0;
+  int _frameCount[256] = {0};
+};
+
+class MockQUICFlowController : public QUICFlowController
+{
+public:
+  MockQUICFlowController() {}
+
+  // Override
+  virtual void
+  handle_frame(std::shared_ptr<const QUICFrame> f) override
+  {
+    ++_frameCount[static_cast<int>(f->type())];
+    ++_totalFrameCount;
+  }
+
+  // for Test
+  int
+  getStreamFrameCount()
+  {
+    return _frameCount[static_cast<int>(QUICFrameType::STREAM)];
+  }
+
+  int
+  getAckFrameCount()
+  {
+    return _frameCount[static_cast<int>(QUICFrameType::ACK)];
+  }
+
+  int
+  getPingFrameCount()
+  {
+    return _frameCount[static_cast<int>(QUICFrameType::PING)];
+  }
+
+  int
+  getTotalFrameCount()
+  {
+    return _totalFrameCount;
+  }
+
+private:
+  int _totalFrameCount = 0;
+  int _frameCount[256] = {0};
+};
diff --git a/iocore/net/quic/QUICAckFrameCreator.cc b/iocore/net/quic/QUICAckFrameCreator.cc
new file mode 100644
index 0000000..26c8199
--- /dev/null
+++ b/iocore/net/quic/QUICAckFrameCreator.cc
@@ -0,0 +1,102 @@
+/** @file
+ *
+ *  A brief file description
+ *
+ *  @section license License
+ *
+ *  Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+#include "QUICAckFrameCreator.h"
+#include <algorithm>
+
+int
+QUICAckFrameCreator::update(QUICPacketNumber packet_number, bool acknowledgable)
+{
+  if (this->_packet_count == MAXIMUM_PACKET_COUNT) {
+    return -1;
+  }
+  if (packet_number > this->_largest_ack_number) {
+    this->_largest_ack_number        = packet_number;
+    this->_largest_ack_received_time = Thread::get_hrtime();
+  }
+  this->_packet_numbers[this->_packet_count++] = packet_number - this->_last_ack_number;
+  if (acknowledgable && !this->_can_send) {
+    this->_can_send = true;
+  }
+
+  return 0;
+}
+
+std::unique_ptr<QUICAckFrame, QUICFrameDeleterFunc>
+QUICAckFrameCreator::create()
+{
+  std::unique_ptr<QUICAckFrame, QUICFrameDeleterFunc> ack_frame = {nullptr, QUICFrameDeleter::delete_null_frame};
+  if (this->_can_send) {
+    ack_frame              = this->_create_ack_frame();
+    this->_last_ack_number = this->_largest_ack_number;
+    this->_can_send        = false;
+    this->_packet_count    = 0;
+  }
+  return ack_frame;
+}
+
+std::unique_ptr<QUICAckFrame, QUICFrameDeleterFunc>
+QUICAckFrameCreator::create_if_needed()
+{
+  // TODO What would be criteria?
+  return this->create();
+}
+
+void
+QUICAckFrameCreator::_sort_packet_numbers()
+{
+  // TODO Find more smart way
+  std::sort(this->_packet_numbers, this->_packet_numbers + this->_packet_count);
+}
+
+std::unique_ptr<QUICAckFrame, QUICFrameDeleterFunc>
+QUICAckFrameCreator::_create_ack_frame()
+{
+  std::unique_ptr<QUICAckFrame, QUICFrameDeleterFunc> ack_frame = {nullptr, QUICFrameDeleter::delete_null_frame};
+  this->_sort_packet_numbers();
+  uint16_t start = this->_packet_numbers[0];
+  uint8_t gap;
+  int i;
+  uint64_t length = 0;
+  for (i = 0, length = 0; i < this->_packet_count; ++i, ++length) {
+    if (this->_packet_numbers[i] == start + length) {
+      continue;
+    }
+    if (ack_frame) {
+      ack_frame->ack_block_section()->add_ack_block({gap, length});
+    } else {
+      uint16_t delay = (Thread::get_hrtime() - this->_largest_ack_received_time) / 1000; // TODO Milliseconds?
+      ack_frame      = QUICFrameFactory::create_ack_frame(this->_largest_ack_number, delay, length);
+    }
+    gap    = this->_packet_numbers[i] - this->_packet_numbers[i - 1] - 1;
+    start  = this->_packet_numbers[i];
+    length = 0;
+  }
+  if (ack_frame) {
+    ack_frame->ack_block_section()->add_ack_block({gap, length});
+  } else {
+    uint16_t delay = (Thread::get_hrtime() - this->_largest_ack_received_time) / 1000; // TODO Milliseconds?
+    ack_frame      = QUICFrameFactory::create_ack_frame(this->_largest_ack_number, delay, length);
+  }
+  return ack_frame;
+}
diff --git a/iocore/net/quic/QUICAckFrameCreator.h b/iocore/net/quic/QUICAckFrameCreator.h
new file mode 100644
index 0000000..8b940a4
--- /dev/null
+++ b/iocore/net/quic/QUICAckFrameCreator.h
@@ -0,0 +1,69 @@
+/** @file
+ *
+ *  A brief file description
+ *
+ *  @section license License
+ *
+ *  Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+#pragma once
+
+#include "ts/ink_hrtime.h"
+#include "QUICTypes.h"
+#include "QUICFrame.h"
+
+class QUICAckFrameCreator
+{
+public:
+  static const int MAXIMUM_PACKET_COUNT = 256;
+  QUICAckFrameCreator(){};
+
+  /*
+   * All packet numbers ATS received need to be passed to this method.
+   * Returns 0 if updated successfully.
+   */
+  int update(QUICPacketNumber packet_number, bool acknowledgable);
+
+  /*
+   * Returns QUICAckFrame only if ACK frame is able to be sent.
+   * Caller must send the ACK frame to the peer if it was returned.
+   * Usually you should use create_if_needed() instead, but you may want to
+   * call this when ATS receives PING frame.
+   */
+  std::unique_ptr<QUICAckFrame, QUICFrameDeleterFunc> create();
+
+  /*
+   * Returns QUICAckFrame only if ACK frame need to be sent,
+   * because sending an ACK frame per incoming ACK-able packet isn't sufficient.
+   * Caller must send the ACK frame to the peer if it was returned.
+   */
+  std::unique_ptr<QUICAckFrame, QUICFrameDeleterFunc> create_if_needed();
+
+private:
+  bool _can_send = false;
+
+  QUICPacketNumber _largest_ack_number  = 0;
+  QUICPacketNumber _last_ack_number     = 0;
+  ink_hrtime _largest_ack_received_time = 0;
+
+  uint16_t _packet_numbers[MAXIMUM_PACKET_COUNT];
+  uint16_t _packet_count = 0;
+
+  void _sort_packet_numbers();
+  std::unique_ptr<QUICAckFrame, QUICFrameDeleterFunc> _create_ack_frame();
+};
diff --git a/iocore/net/quic/QUICApplication.cc b/iocore/net/quic/QUICApplication.cc
new file mode 100644
index 0000000..343a6be
--- /dev/null
+++ b/iocore/net/quic/QUICApplication.cc
@@ -0,0 +1,139 @@
+/** @file
+ *
+ *  A brief file description
+ *
+ *  @section license License
+ *
+ *  Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+#include "QUICApplication.h"
+
+#include "ts/MemView.h"
+#include "QUICStream.h"
+
+const static char *tag = "quic_app";
+
+//
+// QUICStreamIO
+//
+QUICStreamIO::QUICStreamIO(QUICApplication *app, QUICStream *stream)
+{
+  this->_read_buffer  = new_MIOBuffer(BUFFER_SIZE_INDEX_4K);
+  this->_write_buffer = new_MIOBuffer(BUFFER_SIZE_INDEX_4K);
+
+  this->_read_buffer_reader  = _read_buffer->alloc_reader();
+  this->_write_buffer_reader = _write_buffer->alloc_reader();
+
+  this->_read_vio  = stream->do_io_read(app, 0, _read_buffer);
+  this->_write_vio = stream->do_io_write(app, 0, _write_buffer_reader);
+}
+
+int64_t
+QUICStreamIO::read_avail()
+{
+  return this->_read_buffer_reader->read_avail();
+}
+
+int64_t
+QUICStreamIO::read(uint8_t *buf, int64_t len)
+{
+  return this->_read_buffer_reader->read(const_cast<uint8_t *>(buf), len);
+}
+
+int64_t
+QUICStreamIO::write(uint8_t *buf, int64_t len)
+{
+  SCOPED_MUTEX_LOCK(lock, this->_write_vio->mutex, this_ethread());
+
+  int64_t bytes_add = this->_write_buffer->write(buf, len);
+  this->_write_vio->nbytes += bytes_add;
+
+  return bytes_add;
+}
+
+void
+QUICStreamIO::read_reenable()
+{
+  return this->_read_vio->reenable();
+}
+
+void
+QUICStreamIO::write_reenable()
+{
+  return this->_write_vio->reenable();
+}
+
+//
+// QUICApplication
+//
+QUICApplication::QUICApplication(ProxyMutex *m, QUICNetVConnection *vc) : Continuation(m)
+{
+  this->_client_vc = vc;
+}
+
+// @brief Bind stream and application
+void
+QUICApplication::set_stream(QUICStream *stream)
+{
+  QUICStreamIO *stream_io = new QUICStreamIO(this, stream);
+  this->_stream_map.insert(std::make_pair(stream->id(), stream_io));
+}
+
+bool
+QUICApplication::is_stream_set(QUICStream *stream)
+{
+  auto result = this->_stream_map.find(stream->id());
+
+  return result != this->_stream_map.end();
+}
+
+void
+QUICApplication::reenable(QUICStream *stream)
+{
+  QUICStreamIO *stream_io = this->_find_stream_io(stream->id());
+  if (stream_io) {
+    stream_io->read_reenable();
+    stream_io->write_reenable();
+  } else {
+    Debug(tag, "Unknown Stream, id: %d", stream->id());
+  }
+
+  return;
+}
+
+void
+QUICApplication::unset_stream(QUICStream *stream)
+{
+  QUICStreamIO *stream_io = this->_find_stream_io(stream->id());
+  if (stream_io) {
+    this->_stream_map.erase(stream->id());
+    delete stream_io;
+  }
+}
+
+QUICStreamIO *
+QUICApplication::_find_stream_io(QUICStreamId id)
+{
+  auto result = this->_stream_map.find(id);
+
+  if (result == this->_stream_map.end()) {
+    return nullptr;
+  } else {
+    return result->second;
+  }
+}
diff --git a/iocore/net/quic/QUICApplication.h b/iocore/net/quic/QUICApplication.h
new file mode 100644
index 0000000..d876e6a
--- /dev/null
+++ b/iocore/net/quic/QUICApplication.h
@@ -0,0 +1,79 @@
+/** @file
+ *
+ *  A brief file description
+ *
+ *  @section license License
+ *
+ *  Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+#pragma once
+
+#include "I_VConnection.h"
+#include "QUICTypes.h"
+
+class QUICNetVConnection;
+class QUICStream;
+class QUICApplication;
+
+/**
+ * @brief QUICStream I/O interface from QUIC Application
+ */
+class QUICStreamIO
+{
+public:
+  QUICStreamIO(QUICApplication *app, QUICStream *stream);
+
+  int64_t read_avail();
+  int64_t read(uint8_t *buf, int64_t len);
+  int64_t write(uint8_t *buf, int64_t len);
+  void read_reenable();
+  void write_reenable();
+
+private:
+  MIOBuffer *_read_buffer  = nullptr;
+  MIOBuffer *_write_buffer = nullptr;
+
+  IOBufferReader *_read_buffer_reader  = nullptr;
+  IOBufferReader *_write_buffer_reader = nullptr;
+
+  VIO *_read_vio  = nullptr;
+  VIO *_write_vio = nullptr;
+};
+
+/**
+ * @brief Abstruct QUIC Application Class
+ * @detail Every quic application must inherits this class
+ */
+class QUICApplication : public Continuation
+{
+public:
+  QUICApplication(ProxyMutex *m, QUICNetVConnection *vc);
+
+  void set_stream(QUICStream *stream);
+  bool is_stream_set(QUICStream *stream);
+  void reenable(QUICStream *stream);
+  void unset_stream(QUICStream *stream);
+
+protected:
+  QUICStreamIO *_find_stream_io(QUICStreamId id);
+
+  QUICNetVConnection *_client_vc = nullptr;
+
+private:
+  std::map<QUICStreamId, QUICStreamIO *> _stream_map;
+};
diff --git a/iocore/net/quic/QUICCongestionController.cc b/iocore/net/quic/QUICCongestionController.cc
new file mode 100644
index 0000000..79df5d8
--- /dev/null
+++ b/iocore/net/quic/QUICCongestionController.cc
@@ -0,0 +1,40 @@
+/** @file
+ *
+ *  A brief file description
+ *
+ *  @section license License
+ *
+ *  Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+#include <QUICCongestionController.h>
+
+const static char *tag = "quic_congestion_controller";
+
+void
+QUICCongestionController::handle_frame(std::shared_ptr<const QUICFrame> frame)
+{
+  switch (frame->type()) {
+  case QUICFrameType::STREAM:
+  case QUICFrameType::ACK:
+    break;
+  default:
+    Debug(tag, "Unexpected frame type: %02x", frame->type());
+    ink_assert(false);
+    break;
+  }
+}
diff --git a/iocore/net/quic/QUICCongestionController.h b/iocore/net/quic/QUICCongestionController.h
new file mode 100644
index 0000000..a579a48
--- /dev/null
+++ b/iocore/net/quic/QUICCongestionController.h
@@ -0,0 +1,36 @@
+/** @file
+ *
+ *  A brief file description
+ *
+ *  @section license License
+ *
+ *  Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+#pragma once
+
+#include <QUICFrameHandler.h>
+
+// TODO Implement congestion controll.
+// Congestion controller will be required after the 2nd implementation draft.
+class QUICCongestionController : public QUICFrameHandler
+{
+public:
+  virtual void handle_frame(std::shared_ptr<const QUICFrame>) override;
+
+private:
+};
diff --git a/iocore/net/quic/QUICConnectionManager.cc b/iocore/net/quic/QUICConnectionManager.cc
new file mode 100644
index 0000000..fc1240b
--- /dev/null
+++ b/iocore/net/quic/QUICConnectionManager.cc
@@ -0,0 +1,42 @@
+/** @file
+ *
+ *  A brief file description
+ *
+ *  @section license License
+ *
+ *  Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+#include <QUICConnectionManager.h>
+
+const static char *tag = "quic_connection_manager";
+
+void
+QUICConnectionManager::handle_frame(std::shared_ptr<const QUICFrame> frame)
+{
+  switch (frame->type()) {
+  case QUICFrameType::CONNECTION_CLOSE:
+  case QUICFrameType::STREAM:
+  case QUICFrameType::GOAWAY:
+  case QUICFrameType::PING:
+    break;
+  default:
+    Debug(tag, "Unexpected frame type: %02x", frame->type());
+    ink_assert(false);
+    break;
+  }
+}
diff --git a/iocore/net/quic/QUICConnectionManager.h b/iocore/net/quic/QUICConnectionManager.h
new file mode 100644
index 0000000..3a67734
--- /dev/null
+++ b/iocore/net/quic/QUICConnectionManager.h
@@ -0,0 +1,40 @@
+/** @file
+ *
+ *  A brief file description
+ *
+ *  @section license License
+ *
+ *  Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+#pragma once
+
+#include <QUICFrameHandler.h>
+#include <QUICFrame.h>
+#include "QUICFrameTransmitter.h"
+
+class QUICConnectionManager : public QUICFrameHandler
+{
+public:
+  QUICConnectionManager(QUICFrameTransmitter *tx) : _tx(tx){};
+  virtual void handle_frame(std::shared_ptr<const QUICFrame> frame) override;
+
+private:
+  QUICFrameTransmitter *_tx = nullptr;
+
+  void _handle_ping_frame(const QUICPingFrame *);
+};
diff --git a/iocore/net/quic/QUICCrypto.cc b/iocore/net/quic/QUICCrypto.cc
new file mode 100644
index 0000000..2707dfa
--- /dev/null
+++ b/iocore/net/quic/QUICCrypto.cc
@@ -0,0 +1,405 @@
+/** @file
+ *
+ *  QUIC Crypto (TLS to Secure QUIC)
+ *
+ *  @section license License
+ *
+ *  Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+#include "QUICCrypto.h"
+
+#include <openssl/err.h>
+#include <openssl/ssl.h>
+#include <openssl/bio.h>
+
+#include "ts/Diags.h"
+#include "ts/MemView.h"
+#include "QUICTypes.h"
+
+constexpr static char tag[] = "quic_crypto";
+
+// constexpr static ts::StringView _exporter_label_0_rtt("EXPORTER-QUIC 0-RTT Secret", ts::StringView::literal);
+constexpr static ts::StringView exporter_label_client_1_rtt("EXPORTER-QUIC client 1-RTT Secret", ts::StringView::literal);
+constexpr static ts::StringView exporter_label_server_1_rtt("EXPORTER-QUIC server 1-RTT Secret", ts::StringView::literal);
+
+// [quic-tls draft-04] "TLS 1.3, " + Label
+// constexpr static ts::StringView expand_label_client_1_rtt("TLS 1.3, QUIC client 1-RTT secret", ts::StringView::literal);
+// constexpr static ts::StringView expand_label_server_1_rtt("TLS 1.3, QUIC server 1-RTT secret", ts::StringView::literal);
+constexpr static ts::StringView expand_label_key("TLS 1.3, key", ts::StringView::literal);
+constexpr static ts::StringView expand_label_iv("TLS 1.3, iv", ts::StringView::literal);
+
+//
+// QUICPacketProtection
+//
+
+QUICPacketProtection::~QUICPacketProtection()
+{
+  delete this->_phase_0_key;
+  delete this->_phase_1_key;
+}
+
+void
+QUICPacketProtection::set_key(KeyMaterial *km, QUICKeyPhase phase)
+{
+  this->_key_phase = phase;
+  if (phase == QUICKeyPhase::PHASE_0) {
+    this->_phase_0_key = km;
+  } else {
+    this->_phase_1_key = km;
+  }
+}
+
+const KeyMaterial *
+QUICPacketProtection::get_key(QUICKeyPhase phase) const
+{
+  if (phase == QUICKeyPhase::PHASE_0) {
+    return this->_phase_0_key;
+  } else {
+    return this->_phase_1_key;
+  }
+}
+
+QUICKeyPhase
+QUICPacketProtection::key_phase() const
+{
+  return this->_key_phase;
+}
+
+//
+// QUICCrypto
+//
+QUICCrypto::QUICCrypto(SSL_CTX *ssl_ctx, NetVConnectionContext_t c) : _netvc_context(c)
+{
+  this->_ssl = SSL_new(ssl_ctx);
+  if (this->_netvc_context == NET_VCONNECTION_IN) {
+    SSL_set_accept_state(this->_ssl);
+  } else if (this->_netvc_context == NET_VCONNECTION_OUT) {
+    SSL_set_connect_state(this->_ssl);
+  } else {
+    ink_assert(false);
+  }
+
+  this->_client_pp = new QUICPacketProtection();
+  this->_server_pp = new QUICPacketProtection();
+}
+
+QUICCrypto::~QUICCrypto()
+{
+  SSL_free(this->_ssl);
+  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)
+{
+  ink_assert(this->_ssl != nullptr);
+
+  BIO *rbio = BIO_new(BIO_s_mem());
+  BIO *wbio = BIO_new(BIO_s_mem());
+  if (in != nullptr || in_len != 0) {
+    BIO_write(rbio, in, in_len);
+  }
+  SSL_set_bio(this->_ssl, rbio, wbio);
+
+  if (!SSL_is_init_finished(this->_ssl)) {
+    ERR_clear_error();
+    int ret = SSL_do_handshake(this->_ssl);
+    if (ret <= 0) {
+      int err = SSL_get_error(this->_ssl, ret);
+
+      switch (err) {
+      case SSL_ERROR_WANT_READ:
+      case SSL_ERROR_WANT_WRITE:
+        break;
+      default:
+        char err_buf[32] = {0};
+        ERR_error_string_n(err, err_buf, sizeof(err_buf));
+        Debug(tag, "Handshake error: %s (%d)", err_buf, err);
+        return false;
+      }
+    }
+  }
+
+  // OpenSSL doesn't have BIO_mem_contents
+  // const uint8_t *buf;
+  // if (!BIO_mem_contents(wbio, &buf, &out_len)) {
+  //   return false;
+  // }
+  // if (out_len <= 0) {
+  //   return false;
+  // }
+
+  out_len = BIO_read(wbio, out, max_out_len);
+  if (out_len <= 0) {
+    return false;
+  }
+
+  return true;
+}
+
+bool
+QUICCrypto::is_handshake_finished() const
+{
+  return SSL_is_init_finished(this->_ssl);
+}
+
+int
+QUICCrypto::setup_session()
+{
+  const SSL_CIPHER *cipher = SSL_get_current_cipher(this->_ssl);
+  this->_digest            = _get_handshake_digest(cipher);
+  this->_aead              = _get_evp_aead(cipher);
+
+  size_t secret_len = EVP_MD_size(this->_digest);
+  size_t key_len    = _get_aead_key_len(this->_aead);
+  size_t iv_len     = max(static_cast<size_t>(8), _get_aead_nonce_len(this->_aead));
+
+  int r = 0;
+
+  r = _export_client_keymaterial(secret_len, key_len, iv_len);
+  if (r != 1) {
+    return r;
+  }
+
+  r = _export_server_keymaterial(secret_len, key_len, iv_len);
+  if (r != 1) {
+    return r;
+  }
+
+  Debug(tag, "Negotiated ciper: %s, secret_len: %zu, key_len: %zu, iv_len: %zu", SSL_CIPHER_get_name(cipher), secret_len, key_len,
+        iv_len);
+  return 1;
+}
+
+/**
+ * update client_pp_secret_<N+1> and keying material
+ */
+int
+QUICCrypto::update_client_keymaterial()
+{
+  return 0;
+  // KeyMaterial *km_n   = nullptr;
+  // KeyMaterial *km_n_1 = new KeyMaterial(km_n->secret_len, km_n->key_len, km_n->iv_len);
+  // uint8_t secret[256] = {0};
+  // int r               = 0;
+
+  // r = _hkdf_expand_label(secret, km_n->secret_len, km_n->secret, km_n->secret_len, _expand_label_client_1_rtt,
+  //                       sizeof(_expand_label_client_1_rtt), this->_digest);
+  // if (r != 1) {
+  //   return r;
+  // }
+
+  // r = km_n_1->init(this->_aead, this->_digest, secret);
+  // if (r != 1) {
+  //   return r;
+  // }
+  // this->_server_pp->set_key(km_n_1, new_key_phase);
+
+  // return 1;
+}
+
+/**
+ * update server_pp_secret_<N+1> and keying material
+ */
+int
+QUICCrypto::update_server_keymaterial()
+{
+  return 0;
+  // KeyMaterial *km_n   = nullptr;
+  // KeyMaterial *km_n_1 = new KeyMaterial(km_n->secret_len, km_n->key_len, km_n->iv_len);
+  // uint8_t secret[256] = {0};
+  // int r               = 0;
+
+  // r = _hkdf_expand_label(secret, km_n->secret_len, km_n->secret, km_n->secret_len, _expand_label_server_1_rtt,
+  //                       sizeof(_expand_label_server_1_rtt), this->_digest);
+  // if (r != 1) {
+  //   return r;
+  // }
+
+  // r = km_n_1->init(this->_aead, this->_digest, secret);
+  // if (r != 1) {
+  //   return r;
+  // }
+  // this->_server_pp->set_key(km_n_1, new_key_phase);
+
+  // return 1;
+}
+
+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
+{
+  const KeyMaterial *km = nullptr;
+
+  switch (this->_netvc_context) {
+  case NET_VCONNECTION_IN: {
+    km = this->_server_pp->get_key(phase);
+    break;
+  }
+  case NET_VCONNECTION_OUT: {
+    km = this->_client_pp->get_key(phase);
+    break;
+  }
+  default:
+    ink_assert(false);
+    return false;
+  }
+
+  size_t tag_len = _get_aead_tag_len(SSL_get_current_cipher(this->_ssl));
+  return _encrypt(cipher, cipher_len, max_cipher_len, plain, plain_len, pkt_num, ad, ad_len, km->key, km->key_len, km->iv,
+                  km->iv_len, tag_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
+{
+  const KeyMaterial *km = nullptr;
+
+  switch (this->_netvc_context) {
+  case NET_VCONNECTION_IN: {
+    km = this->_client_pp->get_key(phase);
+    break;
+  }
+  case NET_VCONNECTION_OUT: {
+    km = this->_server_pp->get_key(phase);
+    break;
+  }
+  default:
+    ink_assert(false);
+    return false;
+  }
+
+  size_t tag_len = _get_aead_tag_len(SSL_get_current_cipher(this->_ssl));
+  return _decrypt(plain, plain_len, max_plain_len, cipher, cipher_len, pkt_num, ad, ad_len, km->key, km->key_len, km->iv,
+                  km->iv_len, tag_len);
+}
+
+int
+QUICCrypto::_export_secret(uint8_t *dst, size_t dst_len, const char *label, size_t label_len) const
+{
+  return SSL_export_keying_material(this->_ssl, dst, dst_len, label, label_len, reinterpret_cast<const uint8_t *>(""), 0, 1);
+}
+
+/**
+ * export client_pp_secret_0 and keying material
+ */
+int
+QUICCrypto::_export_client_keymaterial(size_t secret_len, size_t key_len, size_t iv_len)
+{
+  KeyMaterial *km = new KeyMaterial(secret_len, key_len, iv_len);
+  int r           = 0;
+
+  r = _export_secret(km->secret, secret_len, exporter_label_client_1_rtt.ptr(), exporter_label_client_1_rtt.size());
+  if (r != 1) {
+    Debug(tag, "Failed to export secret");
+    return r;
+  }
+
+  r = _hkdf_expand_label(km->key, key_len, km->secret, secret_len, expand_label_key.ptr(), expand_label_key.size(), this->_digest);
+  if (r != 1) {
+    Debug(tag, "Failed to expand label for key");
+    return r;
+  }
+
+  r = _hkdf_expand_label(km->iv, iv_len, km->secret, secret_len, expand_label_iv.ptr(), expand_label_iv.size(), this->_digest);
+  if (r != 1) {
+    Debug(tag, "Failed to expand label for iv");
+    return r;
+  }
+
+  this->_client_pp->set_key(km, QUICKeyPhase::PHASE_0);
+
+  return 1;
+}
+
+/**
+ * export server_pp_secret_0 and keying material
+ */
+int
+QUICCrypto::_export_server_keymaterial(size_t secret_len, size_t key_len, size_t iv_len)
+{
+  KeyMaterial *km = new KeyMaterial(secret_len, key_len, iv_len);
+  int r           = 0;
+
+  r = _export_secret(km->secret, secret_len, exporter_label_server_1_rtt.ptr(), exporter_label_server_1_rtt.size());
+  if (r != 1) {
+    return r;
+  }
+
+  r = _hkdf_expand_label(km->key, key_len, km->secret, secret_len, expand_label_key.ptr(), expand_label_key.size(), this->_digest);
+  if (r != 1) {
+    Debug(tag, "Failed to expand label for key");
+    return r;
+  }
+
+  r = _hkdf_expand_label(km->iv, iv_len, km->secret, secret_len, expand_label_iv.ptr(), expand_label_iv.size(), this->_digest);
+  if (r != 1) {
+    Debug(tag, "Failed to expand label for iv");
+    return r;
+  }
+
+  this->_server_pp->set_key(km, QUICKeyPhase::PHASE_0);
+
+  return 1;
+}
+
+/**
+ * Example iv_len = 12
+ *
+ *   0                   1
+ *   0 1 2 3 4 5 6 7 8 9 0 1 2  (byte)
+ *  +-+-+-+-+-+-+-+-+-+-+-+-+-+
+ *  |           iv            |    // IV
+ *  +-+-+-+-+-+-+-+-+-+-+-+-+-+
+ *  |0|0|0|0|    pkt num      |    // network byte order & left-padded with zeros
+ *  +-+-+-+-+-+-+-+-+-+-+-+-+-+
+ *  |          nonce          |    // nonce = iv xor pkt_num
+ *  +-+-+-+-+-+-+-+-+-+-+-+-+-+
+ *
+ */
+void
+QUICCrypto::_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);
+
+  pkt_num    = htobe64(pkt_num);
+  uint8_t *p = reinterpret_cast<uint8_t *>(&pkt_num);
+
+  for (size_t i = 0; i < 8; ++i) {
+    nonce[iv_len - 8 + i] ^= p[i];
+  }
+}
+
+bool
+QUICCrypto::_gen_info(uint8_t *info, size_t &info_len, const char *label, size_t label_len, size_t length) const
+{
+  info[0] = length / 256;
+  info[1] = length % 256;
+  info[2] = label_len;
+  info_len += 3;
+
+  memcpy(info + info_len, label, label_len);
+  info_len += label_len;
+
+  info[info_len] = 0x00;
+  ++info_len;
+
+  return true;
+}
diff --git a/iocore/net/quic/QUICCrypto.h b/iocore/net/quic/QUICCrypto.h
new file mode 100644
index 0000000..e018680
--- /dev/null
+++ b/iocore/net/quic/QUICCrypto.h
@@ -0,0 +1,121 @@
+/** @file
+ *
+ *  QUIC TLS
+ *
+ *  @section license License
+ *
+ *  Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+#ifndef __QUIC_CRYPTO__
+#define __QUIC_CRYPTO__
+
+#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"
+
+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};
+  size_t secret_len               = 0;
+  size_t key_len                  = 0;
+  size_t iv_len                   = 0;
+};
+
+class QUICPacketProtection
+{
+public:
+  QUICPacketProtection(){};
+  ~QUICPacketProtection();
+  void set_key(KeyMaterial *km, QUICKeyPhase phase);
+  const KeyMaterial *get_key(QUICKeyPhase phase) const;
+  QUICKeyPhase key_phase() const;
+
+private:
+  KeyMaterial *_phase_0_key = nullptr;
+  KeyMaterial *_phase_1_key = nullptr;
+  QUICKeyPhase _key_phase   = QUICKeyPhase::PHASE_UNINITIALIZED;
+};
+
+class QUICCrypto
+{
+public:
+  QUICCrypto(SSL_CTX *, NetVConnectionContext_t);
+  ~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 setup_session();
+  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();
+
+private:
+  int _export_secret(uint8_t *dst, size_t dst_len, const char *label, size_t label_len) const;
+  int _export_client_keymaterial(size_t secret_len, size_t key_len, size_t iv_len);
+  int _export_server_keymaterial(size_t secret_len, size_t key_len, size_t iv_len);
+  void _gen_nonce(uint8_t *nonce, size_t &nonce_len, uint64_t pkt_num, const uint8_t *iv, size_t iv_len) const;
+  bool _gen_info(uint8_t *info, size_t &info_len, const char *label, size_t label_len, size_t length) const;
+  int _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;
+#ifdef OPENSSL_IS_BORINGSSL
+  const EVP_AEAD *_get_evp_aead(const SSL_CIPHER *cipher) 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 SSL_CIPHER *cipher) 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
+  const EVP_MD *_get_handshake_digest(const SSL_CIPHER *cipher) const;
+  size_t _get_aead_tag_len(const SSL_CIPHER *cipher) 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 uint8_t *key, size_t key_len, const uint8_t *iv,
+                size_t iv_len, 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 uint8_t *key, size_t key_len, const uint8_t *iv, size_t iv_len,
+                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
+  const EVP_MD *_digest                  = nullptr;
+  QUICPacketProtection *_client_pp       = nullptr;
+  QUICPacketProtection *_server_pp       = nullptr;
+  NetVConnectionContext_t _netvc_context = NET_VCONNECTION_UNSET;
+};
+
+#endif // __QUIC_CRYPTO__
diff --git a/iocore/net/quic/QUICCrypto_boringssl.cc b/iocore/net/quic/QUICCrypto_boringssl.cc
new file mode 100644
index 0000000..bd4a0c0
--- /dev/null
+++ b/iocore/net/quic/QUICCrypto_boringssl.cc
@@ -0,0 +1,144 @@
+/** @file
+ *
+ *  QUIC Crypto (TLS to Secure QUIC) using BoringSSL
+ *
+ *  @section license License
+ *
+ *  Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+#include "QUICCrypto.h"
+
+#include <openssl/base.h>
+#include <openssl/err.h>
+#include <openssl/ssl.h>
+#include <openssl/bio.h>
+#include <openssl/hkdf.h>
+#include <openssl/aead.h>
+
+const static char tag[] = "quic_crypto";
+
+const EVP_AEAD *
+QUICCrypto::_get_evp_aead(const SSL_CIPHER *cipher) const
+{
+  ink_assert(SSL_CIPHER_is_AEAD(cipher));
+
+  if (SSL_CIPHER_is_AES128GCM(cipher)) {
+    return EVP_aead_aes_128_gcm();
+  } else if ((cipher->algorithm_enc & 0x00000010L) != 0) {
+    // SSL_AES256GCM is 0x00000010L ( defined in `ssl/internal.h` ).
+    // There're no `SSL_CIPHER_is_AES256GCM(const SSL_CIPHER *cipher)`.
+    return EVP_aead_aes_256_gcm();
+  } else if (SSL_CIPHER_is_CHACHA20POLY1305(cipher)) {
+    return EVP_aead_chacha20_poly1305();
+  } else {
+    return nullptr;
+  }
+}
+
+// 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
+{
+  switch (cipher->algorithm_prf) {
+  case 0x2:
+    // SSL_HANDSHAKE_MAC_SHA256:
+    return EVP_sha256();
+  case 0x4:
+    // SSL_HANDSHAKE_MAC_SHA384:
+    return EVP_sha384();
+  default:
+    return nullptr;
+  }
+}
+
+size_t
+QUICCrypto::_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
+{
+  return EVP_AEAD_key_length(aead);
+}
+
+size_t
+QUICCrypto::_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
+{
+  uint8_t info[256] = {0};
+  size_t info_len   = 0;
+  _gen_info(info, info_len, label, label_len, dst_len);
+  return HKDF(dst, dst_len, digest, secret, secret_len, nullptr, 0, info, info_len);
+}
+
+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
+{
+  uint8_t nonce[EVP_MAX_IV_LENGTH] = {0};
+  size_t nonce_len                 = 0;
+  _gen_nonce(nonce, nonce_len, pkt_num, iv, iv_len);
+
+  EVP_AEAD_CTX *aead_ctx = EVP_AEAD_CTX_new(this->_aead, key, key_len, tag_len);
+  if (!aead_ctx) {
+    Debug(tag, "Failed to create EVP_AEAD_CTX");
+    return false;
+  }
+
+  if (!EVP_AEAD_CTX_seal(aead_ctx, cipher, &cipher_len, max_cipher_len, nonce, nonce_len, plain, plain_len, ad, ad_len)) {
+    Debug(tag, "Failed to encrypt");
+    return false;
+  }
+
+  EVP_AEAD_CTX_free(aead_ctx);
+
+  return true;
+}
+
+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
+{
+  uint8_t nonce[EVP_MAX_IV_LENGTH] = {0};
+  size_t nonce_len                 = 0;
+  _gen_nonce(nonce, nonce_len, pkt_num, iv, iv_len);
+
+  EVP_AEAD_CTX *aead_ctx = EVP_AEAD_CTX_new(this->_aead, key, key_len, tag_len);
+  if (!aead_ctx) {
+    Debug(tag, "Failed to create EVP_AEAD_CTX");
+    return false;
+  }
+
+  if (!EVP_AEAD_CTX_open(aead_ctx, plain, &plain_len, max_plain_len, nonce, nonce_len, cipher, cipher_len, ad, ad_len)) {
+    Debug(tag, "Failed to decrypt");
+    return false;
+  }
+
+  EVP_AEAD_CTX_free(aead_ctx);
+
+  return true;
+}
diff --git a/iocore/net/quic/QUICCrypto_openssl.cc b/iocore/net/quic/QUICCrypto_openssl.cc
new file mode 100644
index 0000000..d0a6f2d
--- /dev/null
+++ b/iocore/net/quic/QUICCrypto_openssl.cc
@@ -0,0 +1,236 @@
+/** @file
+ *
+ *  QUIC Crypto (TLS to Secure QUIC) using OpenSSL
+ *
+ *  @section license License
+ *
+ *  Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+#include "QUICCrypto.h"
+
+#include <openssl/err.h>
+#include <openssl/ssl.h>
+#include <openssl/bio.h>
+#include <openssl/kdf.h>
+#include <openssl/evp.h>
+
+const static char tag[] = "quic_crypto";
+
+const EVP_CIPHER *
+QUICCrypto::_get_evp_aead(const SSL_CIPHER *cipher) const
+{
+  switch (SSL_CIPHER_get_id(cipher)) {
+  case TLS1_3_CK_AES_128_GCM_SHA256:
+    return EVP_aes_128_gcm();
+  case TLS1_3_CK_AES_256_GCM_SHA384:
+    return EVP_aes_256_gcm();
+  case TLS1_3_CK_CHACHA20_POLY1305_SHA256:
+    return EVP_chacha20_poly1305();
+  case TLS1_3_CK_AES_128_CCM_SHA256:
+  case TLS1_3_CK_AES_128_CCM_8_SHA256:
+    return EVP_aes_128_ccm();
+  default:
+    ink_assert(false);
+    return nullptr;
+  }
+}
+
+const EVP_MD *
+QUICCrypto::_get_handshake_digest(const SSL_CIPHER *cipher) const
+{
+  switch (SSL_CIPHER_get_id(cipher)) {
+  case TLS1_3_CK_AES_128_GCM_SHA256:
+  case TLS1_3_CK_CHACHA20_POLY1305_SHA256:
+  case TLS1_3_CK_AES_128_CCM_SHA256:
+  case TLS1_3_CK_AES_128_CCM_8_SHA256:
+    return EVP_sha256();
+  case TLS1_3_CK_AES_256_GCM_SHA384:
+    return EVP_sha384();
+  default:
+    ink_assert(false);
+    return nullptr;
+  }
+}
+
+size_t
+QUICCrypto::_get_aead_tag_len(const SSL_CIPHER *cipher) const
+{
+  switch (SSL_CIPHER_get_id(cipher)) {
+  case TLS1_3_CK_AES_128_GCM_SHA256:
+  case TLS1_3_CK_AES_256_GCM_SHA384:
+    return EVP_GCM_TLS_TAG_LEN;
+  case TLS1_3_CK_CHACHA20_POLY1305_SHA256:
+    return EVP_CHACHAPOLY_TLS_TAG_LEN;
+  case TLS1_3_CK_AES_128_CCM_SHA256:
+    return EVP_CCM_TLS_TAG_LEN;
+  case TLS1_3_CK_AES_128_CCM_8_SHA256:
+    return EVP_CCM8_TLS_TAG_LEN;
+  default:
+    ink_assert(false);
+    return -1;
+  }
+}
+
+size_t
+QUICCrypto::_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
+{
+  return EVP_CIPHER_iv_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
+{
+  uint8_t info[256] = {0};
+  size_t info_len   = 0;
+  _gen_info(info, info_len, label, label_len, dst_len);
+
+  EVP_PKEY_CTX *pctx = EVP_PKEY_CTX_new_id(EVP_PKEY_HKDF, nullptr);
+  if (!EVP_PKEY_derive_init(pctx)) {
+    return -1;
+  }
+  if (!EVP_PKEY_CTX_hkdf_mode(pctx, EVP_PKEY_HKDEF_MODE_EXPAND_ONLY)) {
+    return -1;
+  }
+  if (!EVP_PKEY_CTX_set_hkdf_md(pctx, digest)) {
+    return -1;
+  }
+  if (!EVP_PKEY_CTX_set1_hkdf_salt(pctx, "", 0)) {
+    return -1;
+  }
+  if (!EVP_PKEY_CTX_set1_hkdf_key(pctx, secret, secret_len)) {
+    return -1;
+  }
+  if (!EVP_PKEY_CTX_add1_hkdf_info(pctx, info, info_len)) {
+    return -1;
+  }
+  if (!EVP_PKEY_derive(pctx, dst, &dst_len)) {
+    return -1;
+  }
+
+  return 1;
+}
+
+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
+{
+  uint8_t nonce[EVP_MAX_IV_LENGTH] = {0};
+  size_t nonce_len                 = 0;
+  _gen_nonce(nonce, nonce_len, pkt_num, iv, iv_len);
+
+  EVP_CIPHER_CTX *aead_ctx;
+  int len;
+
+  if (!(aead_ctx = EVP_CIPHER_CTX_new())) {
+    return false;
+  }
+  if (!EVP_EncryptInit_ex(aead_ctx, this->_aead, nullptr, nullptr, nullptr)) {
+    return false;
+  }
+  if (!EVP_CIPHER_CTX_ctrl(aead_ctx, EVP_CTRL_AEAD_SET_IVLEN, nonce_len, nullptr)) {
+    return false;
+  }
+  if (!EVP_EncryptInit_ex(aead_ctx, nullptr, nullptr, key, nonce)) {
+    return false;
+  }
+  if (!EVP_EncryptUpdate(aead_ctx, nullptr, &len, ad, ad_len)) {
+    return false;
+  }
+  if (!EVP_EncryptUpdate(aead_ctx, cipher, &len, plain, plain_len)) {
+    return false;
+  }
+  cipher_len = len;
+
+  if (!EVP_EncryptFinal_ex(aead_ctx, cipher + len, &len)) {
+    return false;
+  }
+  cipher_len += len;
+
+  if (max_cipher_len < cipher_len + tag_len) {
+    return false;
+  }
+  if (!EVP_CIPHER_CTX_ctrl(aead_ctx, EVP_CTRL_AEAD_GET_TAG, tag_len, cipher + cipher_len)) {
+    return false;
+  }
+  cipher_len += tag_len;
+
+  EVP_CIPHER_CTX_free(aead_ctx);
+
+  return true;
+}
+
+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
+{
+  uint8_t nonce[EVP_MAX_IV_LENGTH] = {0};
+  size_t nonce_len                 = 0;
+  _gen_nonce(nonce, nonce_len, pkt_num, iv, iv_len);
+
+  EVP_CIPHER_CTX *aead_ctx;
+  int len;
+
+  if (!(aead_ctx = EVP_CIPHER_CTX_new())) {
+    return false;
+  }
+  if (!EVP_DecryptInit_ex(aead_ctx, this->_aead, nullptr, nullptr, nullptr)) {
+    return false;
+  }
+  if (!EVP_CIPHER_CTX_ctrl(aead_ctx, EVP_CTRL_AEAD_SET_IVLEN, nonce_len, nullptr)) {
+    return false;
+  }
+  if (!EVP_DecryptInit_ex(aead_ctx, nullptr, nullptr, key, nonce)) {
+    return false;
+  }
+  if (!EVP_DecryptUpdate(aead_ctx, nullptr, &len, ad, ad_len)) {
+    return false;
+  }
+
+  if (cipher_len < tag_len) {
+    return false;
+  }
+  cipher_len -= tag_len;
+  if (!EVP_DecryptUpdate(aead_ctx, plain, &len, cipher, cipher_len)) {
+    return false;
+  }
+  plain_len = len;
+
+  if (!EVP_CIPHER_CTX_ctrl(aead_ctx, EVP_CTRL_AEAD_SET_TAG, tag_len, const_cast<uint8_t *>(cipher + cipher_len))) {
+    return false;
+  }
+
+  int ret = EVP_DecryptFinal_ex(aead_ctx, plain + len, &len);
+
+  EVP_CIPHER_CTX_free(aead_ctx);
+
+  if (ret > 0) {
+    plain_len += len;
+    return true;
+  } else {
+    Debug(tag, "Failed to decrypt");
+    return false;
+  }
+}
diff --git a/iocore/net/quic/QUICDebugNames.cc b/iocore/net/quic/QUICDebugNames.cc
new file mode 100644
index 0000000..74f7a2a
--- /dev/null
+++ b/iocore/net/quic/QUICDebugNames.cc
@@ -0,0 +1,118 @@
+/** @file
+ *
+ *  A brief file description
+ *
+ *  @section license License
+ *
+ *  Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+#include "QUICDebugNames.h"
+#include "I_VConnection.h"
+
+const char *
+QUICDebugNames::packet_type(QUICPacketType type)
+{
+  switch (type) {
+  case QUICPacketType::VERSION_NEGOTIATION:
+    return "VERSION_NEGOTIATION";
+  case QUICPacketType::CLIENT_INITIAL:
+    return "CLIENT_INITIAL";
+  case QUICPacketType::SERVER_STATELESS_RETRY:
+    return "SERVER_STATELESS_RETRY";
+  case QUICPacketType::SERVER_CLEARTEXT:
+    return "SERVER_CLEARTEXT";
+  case QUICPacketType::CLIENT_CLEARTEXT:
+    return "CLIENT_CLEARTEXT";
+  case QUICPacketType::ZERO_RTT_PROTECTED:
+    return "ZERO_RTT_PROTECTED";
+  case QUICPacketType::ONE_RTT_PROTECTED_KEY_PHASE_0:
+    return "ONE_RTT_PROTECTED_KEY_PHASE_0";
+  case QUICPacketType::ONE_RTT_PROTECTED_KEY_PHASE_1:
+    return "ONE_RTT_PROTECTED_KEY_PHASE_1";
+  case QUICPacketType::PUBLIC_RESET:
+    return "PUBLIC_RESET";
+  case QUICPacketType::UNINITIALIZED:
+  default:
+    return "UNKNOWN";
+  }
+}
+
+const char *
+QUICDebugNames::frame_type(QUICFrameType type)
+{
+  switch (type) {
+  case QUICFrameType::PADDING:
+    return "PADDING";
+  case QUICFrameType::RST_STREAM:
+    return "RST_STREAM";
+  case QUICFrameType::CONNECTION_CLOSE:
+    return "CONNECTION_CLOSE";
+  case QUICFrameType::GOAWAY:
+    return "GOAWAY";
+  case QUICFrameType::MAX_DATA:
+    return "MAX_DATA";
+  case QUICFrameType::MAX_STREAM_DATA:
+    return "MAX_STREAM_DATA";
+  case QUICFrameType::MAX_STREAM_ID:
+    return "MAX_STREAM_ID";
+  case QUICFrameType::PING:
+    return "PING";
+  case QUICFrameType::BLOCKED:
+    return "BLOCKED";
+  case QUICFrameType::STREAM_BLOCKED:
+    return "STREAM_BLOCKED";
+  case QUICFrameType::STREAM_ID_NEEDED:
+    return "STREAM_ID_NEEDED";
+  case QUICFrameType::NEW_CONNECTION_ID:
+    return "NEW_CONNECTION_ID";
+  case QUICFrameType::ACK:
+    return "ACK";
+  case QUICFrameType::STREAM:
+    return "STREAM";
+  case QUICFrameType::UNKNOWN:
+  default:
+    return "UNKNOWN";
+  }
+}
+
+const char *
+QUICDebugNames::vc_event(int event)
+{
+  switch (event) {
+  case VC_EVENT_READ_READY:
+    return "VC_EVENT_READ_READY";
+  case VC_EVENT_READ_COMPLETE:
+    return "VC_EVENT_READ_COMPLETE";
+  case VC_EVENT_WRITE_READY:
+    return "VC_EVENT_WRITE_READY";
+  case VC_EVENT_WRITE_COMPLETE:
+    return "VC_EVENT_WRITE_COMPLETE";
+  case VC_EVENT_EOS:
+    return "VC_EVENT_EOS";
+  case VC_EVENT_ERROR:
+    return "VC_EVENT_ERROR";
+  case VC_EVENT_INACTIVITY_TIMEOUT:
+    return "VC_EVENT_INACTIVITY_TIMEOUT";
+  case VC_EVENT_ACTIVE_TIMEOUT:
+    return "VC_EVENT_ACTIVE_TIMEOUT";
+  case VC_EVENT_OOB_COMPLETE:
+    return "VC_EVENT_OOB_COMPLETE";
+  default:
+    return "UNKNOWN";
+  }
+}
diff --git a/iocore/net/quic/QUICDebugNames.h b/iocore/net/quic/QUICDebugNames.h
new file mode 100644
index 0000000..a6fe715
--- /dev/null
+++ b/iocore/net/quic/QUICDebugNames.h
@@ -0,0 +1,35 @@
+/** @file
+ *
+ *  A brief file description
+ *
+ *  @section license License
+ *
+ *  Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+#pragma once
+
+#include "QUICTypes.h"
+
+class QUICDebugNames
+{
+public:
+  static const char *packet_type(QUICPacketType type);
+  static const char *frame_type(QUICFrameType type);
+  // TODO: move to somewhere
+  static const char *vc_event(int event);
+};
diff --git a/iocore/net/quic/QUICEchoApp.cc b/iocore/net/quic/QUICEchoApp.cc
new file mode 100644
index 0000000..45bb4f9
--- /dev/null
+++ b/iocore/net/quic/QUICEchoApp.cc
@@ -0,0 +1,85 @@
+/** @file
+ *
+ *  A brief file description
+ *
+ *  @section license License
+ *
+ *  Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+#include "QUICEchoApp.h"
+
+#include "P_Net.h"
+#include "P_QUICNetVConnection.h"
+#include "QUICDebugNames.h"
+
+const static char *tag = "quic_echo_app";
+
+QUICEchoApp::QUICEchoApp(ProxyMutex *m, QUICNetVConnection *vc) : QUICApplication(m, vc)
+{
+  SET_HANDLER(&QUICEchoApp::main_event_handler);
+}
+
+int
+QUICEchoApp::main_event_handler(int event, Event *data)
+{
+  Debug(tag, "%s", QUICDebugNames::vc_event(event));
+
+  QUICStream *stream      = reinterpret_cast<QUICStream *>(data->cookie);
+  QUICStreamIO *stream_io = this->_find_stream_io(stream->id());
+  if (stream_io == nullptr) {
+    Debug(tag, "Unknown Stream, id: %d", stream->id());
+    return -1;
+  }
+
+  switch (event) {
+  case VC_EVENT_READ_READY:
+  case VC_EVENT_READ_COMPLETE: {
+    uint8_t msg[1024] = {0};
+    int64_t msg_len   = 1024;
+
+    int64_t read_len = stream_io->read(msg, msg_len);
+
+    if (read_len) {
+      Debug(tag, "msg: %s, len: %" PRId64, msg, read_len);
+
+      stream_io->write(msg, read_len);
+      stream_io->write_reenable();
+      stream_io->read_reenable();
+    } else {
+      Debug(tag, "No MSG");
+    }
+    break;
+  }
+  case VC_EVENT_WRITE_READY:
+  case VC_EVENT_WRITE_COMPLETE: {
+    // do nothing
+    break;
+  }
+  case VC_EVENT_EOS:
+  case VC_EVENT_ERROR:
+  case VC_EVENT_INACTIVITY_TIMEOUT:
+  case VC_EVENT_ACTIVE_TIMEOUT: {
+    ink_assert(false);
+    break;
+  }
+  default:
+    break;
+  }
+
+  return EVENT_CONT;
+}
diff --git a/iocore/net/quic/QUICEchoApp.h b/iocore/net/quic/QUICEchoApp.h
new file mode 100644
index 0000000..a8310b7
--- /dev/null
+++ b/iocore/net/quic/QUICEchoApp.h
@@ -0,0 +1,42 @@
+/** @file
+ *
+ *  A brief file description
+ *
+ *  @section license License
+ *
+ *  Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+#ifndef __QUIC_ECHOAPP__
+#define __QUIC_ECHOAPP__
+
+#include "I_VConnection.h"
+#include "QUICApplication.h"
+
+/**
+ * @brief Echo over QUIC
+ * @detail An example application over QUIC.
+ *         Receive DATA of STREAM Frame and echo it.
+ */
+class QUICEchoApp : public QUICApplication
+{
+public:
+  QUICEchoApp(ProxyMutex *m, QUICNetVConnection *vc);
+
+  int main_event_handler(int event, Event *data);
+};
+#endif // __QUIC_HANDSHAKE__
diff --git a/iocore/net/quic/QUICEvents.h b/iocore/net/quic/QUICEvents.h
new file mode 100644
index 0000000..1a8a9b2
--- /dev/null
+++ b/iocore/net/quic/QUICEvents.h
@@ -0,0 +1,31 @@
+/** @file
+ *
+ *  A brief file description
+ *
+ *  @section license License
+ *
+ *  Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+#pragma once
+
+#include "I_Event.h"
+
+enum {
+  QUIC_EVENT_PACKET_READ_READY = QUIC_EVENT_EVENTS_START,
+  QUIC_EVENT_PACKET_WRITE_READY,
+};
diff --git a/iocore/net/quic/QUICFlowController.cc b/iocore/net/quic/QUICFlowController.cc
new file mode 100644
index 0000000..b746aa2
--- /dev/null
+++ b/iocore/net/quic/QUICFlowController.cc
@@ -0,0 +1,42 @@
+/** @file
+ *
+ *  A brief file description
+ *
+ *  @section license License
+ *
+ *  Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+#include <QUICFlowController.h>
+
+const static char *tag = "quic_flow_controller";
+
+void
+QUICFlowController::handle_frame(std::shared_ptr<const QUICFrame> frame)
+{
+  switch (frame->type()) {
+  case QUICFrameType::MAX_DATA:
+  case QUICFrameType::MAX_STREAM_DATA:
+  case QUICFrameType::BLOCKED:
+  case QUICFrameType::STREAM:
+    break;
+  default:
+    Debug(tag, "Unexpected frame type: %02x", frame->type());
+    ink_assert(false);
+    break;
+  }
+}
diff --git a/iocore/net/quic/QUICFlowController.h b/iocore/net/quic/QUICFlowController.h
new file mode 100644
index 0000000..6009d2c
--- /dev/null
+++ b/iocore/net/quic/QUICFlowController.h
@@ -0,0 +1,36 @@
+/** @file
+ *
+ *  A brief file description
+ *
+ *  @section license License
+ *
+ *  Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+#pragma once
+
+#include <QUICFrameHandler.h>
+
+// TODO Implement flow controll
+// Flow controll will be required for the 2nd implementation draft
+class QUICFlowController : QUICFrameHandler
+{
+public:
+  virtual void handle_frame(std::shared_ptr<const QUICFrame>) override;
+
+private:
+};
diff --git a/iocore/net/quic/QUICFrame.cc b/iocore/net/quic/QUICFrame.cc
new file mode 100644
index 0000000..8ca581d
--- /dev/null
+++ b/iocore/net/quic/QUICFrame.cc
@@ -0,0 +1,1315 @@
+/** @file
+ *
+ *  A brief file description
+ *
+ *  @section license License
+ *
+ *  Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+#include "QUICFrame.h"
+
+ClassAllocator<QUICStreamFrame> quicStreamFrameAllocator("quicStreamFrameAllocator");
+ClassAllocator<QUICAckFrame> quicAckFrameAllocator("quicAckFrameAllocator");
+ClassAllocator<QUICPaddingFrame> quicPaddingFrameAllocator("quicPaddingFrameAllocator");
+ClassAllocator<QUICRstStreamFrame> quicRstStreamFrameAllocator("quicRstStreamFrameAllocator");
+ClassAllocator<QUICConnectionCloseFrame> quicConnectionCloseFrameAllocator("quicConnectionCloseFrameAllocator");
+ClassAllocator<QUICGoawayFrame> quicGoawayFrameAllocator("quicGoawayAllocator");
+ClassAllocator<QUICMaxDataFrame> quicMaxDataFrameAllocator("quicMaxDataFrameAllocator");
+ClassAllocator<QUICMaxStreamDataFrame> quicMaxStreamDataFrameAllocator("quicMaxStreamDataFrameAllocator");
+ClassAllocator<QUICMaxStreamIdFrame> quicMaxStreamIdFrameAllocator("quicMaxStreamDataIdAllocator");
+ClassAllocator<QUICPingFrame> quicPingFrameAllocator("quicPingFrameAllocator");
+ClassAllocator<QUICBlockedFrame> quicBlockedFrameAllocator("quicBlockedFrameAllocator");
+ClassAllocator<QUICStreamBlockedFrame> quicStreamBlockedFrameAllocator("quicStreamBlockedFrameAllocator");
+ClassAllocator<QUICStreamIdNeededFrame> quicStreamIdNeededFrameAllocator("quicStreamIdNeededFrameAllocator");
+ClassAllocator<QUICNewConnectionIdFrame> quicNewConnectionIdFrameAllocator("quicNewConnectionIdFrameAllocator");
+ClassAllocator<QUICRetransmissionFrame> quicRetransmissionFrameAllocator("quicRetransmissionFrameAllocator");
+
+QUICFrameType
+QUICFrame::type() const
+{
+  return QUICFrame::type(this->_buf);
+}
+
+QUICFrameType
+QUICFrame::type(const uint8_t *buf)
+{
+  if (buf[0] >= static_cast<uint8_t>(QUICFrameType::STREAM)) {
+    return QUICFrameType::STREAM;
+  } else if (buf[0] >= static_cast<uint8_t>(QUICFrameType::ACK)) {
+    return QUICFrameType::ACK;
+  } else if (static_cast<uint8_t>(QUICFrameType::NEW_CONNECTION_ID) < buf[0] && buf[0] < static_cast<uint8_t>(QUICFrameType::ACK)) {
+    return QUICFrameType::UNKNOWN;
+  } else {
+    return static_cast<QUICFrameType>(buf[0]);
+  }
+}
+
+void
+QUICFrame::reset(const uint8_t *buf, size_t len)
+{
+  this->_buf = buf;
+  this->_len = len;
+}
+
+//
+// STREAM Frame
+//
+
+QUICStreamFrame::QUICStreamFrame(const uint8_t *data, size_t data_len, QUICStreamId stream_id, QUICOffset offset)
+{
+  this->_data      = data;
+  this->_data_len  = data_len;
+  this->_stream_id = stream_id;
+  this->_offset    = offset;
+}
+
+QUICFrameType
+QUICStreamFrame::type() const
+{
+  return QUICFrameType::STREAM;
+}
+
+size_t
+QUICStreamFrame::size() const
+{
+  if (this->_buf) {
+    return this->_get_data_offset() + this->data_length();
+  } else {
+    return 1 + 4 + 8 + 2 + this->data_length();
+  }
+}
+
+void
+QUICStreamFrame::store(uint8_t *buf, size_t *len) const
+{
+  this->store(buf, len, true);
+}
+
+void
+QUICStreamFrame::store(uint8_t *buf, size_t *len, bool include_length_field) const
+{
+  size_t n;
+  // Build Frame Type: "11FSSOOD"
+  buf[0] = static_cast<uint8_t>(QUICFrameType::STREAM);
+  *len   = 1;
+
+  // "SS" of "11FSSOOD"
+  // use 32 bit length for now
+  buf[0] += (0x03 << 3);
+  QUICTypeUtil::write_QUICStreamId(this->stream_id(), 4, buf + *len, &n);
+  *len += n;
+
+  // "OO" of "11FSSOOD"
+  // use 64 bit length for now
+  buf[0] += (0x03 << 1);
+  QUICTypeUtil::write_QUICOffset(this->offset(), 8, buf + *len, &n);
+  *len += n;
+
+  // "D" of "11FSSOOD"
+  if (include_length_field) {
+    buf[0] += 0x01;
+    QUICTypeUtil::write_uint_as_nbytes(this->data_length(), 2, buf + *len, &n);
+    *len += n;
+  }
+
+  memcpy(buf + *len, this->data(), this->data_length());
+  *len += this->data_length();
+}
+
+QUICStreamId
+QUICStreamFrame::stream_id() const
+{
+  if (this->_buf) {
+    return QUICTypeUtil::read_QUICStreamId(this->_buf + this->_get_stream_id_offset(), this->_get_stream_id_len());
+  } else {
+    return this->_stream_id;
+  }
+}
+
+QUICOffset
+QUICStreamFrame::offset() const
+{
+  if (this->_buf) {
+    return QUICTypeUtil::read_QUICOffset(this->_buf + this->_get_offset_offset(), this->_get_offset_len());
+  } else {
+    return this->_offset;
+  }
+}
+
+const uint8_t *
+QUICStreamFrame::data() const
+{
+  if (this->_buf) {
+    return this->_buf + this->_get_data_offset();
+  } else {
+    return this->_data;
+  }
+}
+
+size_t
+QUICStreamFrame::data_length() const
+{
+  if (this->_buf) {
+    if (this->has_data_length_field()) {
+      return QUICTypeUtil::read_nbytes_as_uint(this->_buf + this->_get_offset_offset() + this->_get_offset_len(), 2);
+    } else {
+      return this->_len - this->_get_data_offset();
+    }
+  } else {
+    return this->_data_len;
+  }
+}
+
+/**
+ * "D" of "11FSSOOD"
+ */
+bool
+QUICStreamFrame::has_data_length_field() const
+{
+  return (this->_buf[0] & 0x01) != 0;
+}
+
+/**
+ * "F" of "11FSSOOD"
+ */
+bool
+QUICStreamFrame::has_fin_flag() const
+{
+  if (this->_buf) {
+    return (this->_buf[0] & 0x20) != 0;
+  } else {
+    return this->_fin;
+  }
+}
+
+size_t
+QUICStreamFrame::_get_stream_id_offset() const
+{
+  return 1;
+}
+
+size_t
+QUICStreamFrame::_get_offset_offset() const
+{
+  return this->_get_stream_id_offset() + this->_get_stream_id_len();
+}
+
+size_t
+QUICStreamFrame::_get_data_offset() const
+{
+  if (this->_buf) {
+    if (this->has_data_length_field()) {
+      return this->_get_offset_offset() + this->_get_offset_len() + 2;
+    } else {
+      return this->_get_offset_offset() + this->_get_offset_len();
+    }
+  } else {
+    return 0;
+  }
+}
+
+/**
+ * "SS" of "11FSSOOD"
+ * The value 00, 01, 02, and 03 indicate lengths of 8, 16, 24, and 32 bits long respectively.
+ */
+size_t
+QUICStreamFrame::_get_stream_id_len() const
+{
+  return ((this->_buf[0] & 0x18) >> 3) + 1;
+}
+
+/**
+ * "OO" of "11FSSOOD"
+ * The values 00, 01, 02, and 03 indicate lengths of 0, 16, 32, and 64 bits long respectively.
+ */
+size_t
+QUICStreamFrame::_get_offset_len() const
+{
+  int OO_bits = (this->_buf[0] & 0x06) >> 1;
+  if (OO_bits == 0) {
+    return 0;
+  } else {
+    return 0x01 << OO_bits;
+  }
+}
+
+//
+// ACK frame
+//
+
+QUICAckFrame::QUICAckFrame(const uint8_t *buf, size_t len) : QUICFrame(buf, len)
+{
+  this->reset(buf, len);
+}
+
+QUICAckFrame::QUICAckFrame(QUICPacketNumber largest_acknowledged, uint16_t ack_delay, uint64_t first_ack_block_length)
+{
+  this->_largest_acknowledged = largest_acknowledged;
+  this->_ack_delay            = ack_delay;
+  this->_ack_block_section    = new AckBlockSection(first_ack_block_length);
+}
+
+QUICAckFrame::~QUICAckFrame()
+{
+  if (this->_ack_block_section) {
+    delete this->_ack_block_section;
+    this->_ack_block_section = nullptr;
+  }
+}
+
+void
+QUICAckFrame::reset(const uint8_t *buf, size_t len)
+{
+  QUICFrame::reset(buf, len);
+  this->_ack_block_section =
+    new AckBlockSection(buf + this->_get_ack_block_section_offset(), this->num_blocks(), this->_get_ack_block_length());
+}
+
+QUICFrameType
+QUICAckFrame::type() const
+{
+  return QUICFrameType::ACK;
+}
+
+size_t
+QUICAckFrame::size() const
+{
+  if (this->_buf) {
+    return this->_get_timestamp_section_offset() + this->timestamp_section()->size();
+  } else {
+    return 0;
+  }
+}
+
+void
+QUICAckFrame::store(uint8_t *buf, size_t *len) const
+{
+  size_t n;
+  uint8_t *p = buf;
+
+  // Build Frame Type: "101NLLMM"
+  buf[0] = static_cast<uint8_t>(QUICFrameType::ACK);
+  p += 1;
+
+  // "N" of "101NLLMM"
+  if (this->_ack_block_section->count() > 0) {
+    buf[0] += 0x10;
+    *p = this->_ack_block_section->count();
+    p += 1;
+  }
+
+  *p = this->_timestamp_section->count();
+  p += 1;
+
+  // "LL" of "101NLLMM"
+  if (this->_largest_acknowledged <= 0xff) {
+    QUICTypeUtil::write_uint_as_nbytes(this->_largest_acknowledged, 1, p, &n);
+  } else if (this->_largest_acknowledged <= 0xffff) {
+    buf[0] += 0x01 << 2;
+    QUICTypeUtil::write_uint_as_nbytes(this->_largest_acknowledged, 2, p, &n);
+  } else if (this->_largest_acknowledged <= 0xffffffff) {
+    buf[0] += 0x02 << 2;
+    QUICTypeUtil::write_uint_as_nbytes(this->_largest_acknowledged, 4, p, &n);
+  } else {
+    buf[0] += 0x03 << 2;
+    QUICTypeUtil::write_uint_as_nbytes(this->_largest_acknowledged, 6, p, &n);
+  }
+  p += n;
+
+  QUICTypeUtil::write_uint_as_nbytes(this->_ack_delay, 2, p, &n);
+  p += n;
+
+  // "MM" of "101NLLMM"
+  // use 32 bit length for now
+  // TODO The length should be returned by ackBlockSection
+  buf[0] += 0x02;
+  this->_ack_block_section->store(p, &n);
+  p += n;
+
+  this->_timestamp_section->store(p, &n);
+  p += n;
+
+  *len = p - buf;
+}
+
+uint8_t
+QUICAckFrame::num_blocks() const
+{
+  if (this->has_ack_blocks()) {
+    if (this->_buf) {
+      return this->_buf[1];
+    } else {
+      return this->_ack_block_section->count();
+    }
+  } else {
+    return 0;
+  }
+}
+
+uint8_t
+QUICAckFrame::num_timestamps() const
+{
+  return this->_buf[this->_get_num_timestamp_offset()];
+}
+
+QUICPacketNumber
+QUICAckFrame::largest_acknowledged() const
+{
+  if (this->_buf) {
+    return QUICTypeUtil::read_QUICPacketNumber(this->_buf + this->_get_largest_acknowledged_offset(),
+                                               this->_get_largest_acknowledged_length());
+  } else {
+    return this->_largest_acknowledged;
+  }
+}
+
+uint16_t
+QUICAckFrame::ack_delay() const
+{
+  if (this->_buf) {
+    return QUICTypeUtil::read_nbytes_as_uint(this->_buf + this->_get_ack_delay_offset(), 2);
+  } else {
+    return this->_ack_delay;
+  }
+}
+
+/**
+ * N of 101NLLMM
+ */
+bool
+QUICAckFrame::has_ack_blocks() const
+{
+  if (this->_buf) {
+    return (this->_buf[0] & 0x10) != 0;
+  } else {
+    return this->_ack_block_section->count() != 0;
+  }
+}
+
+QUICAckFrame::AckBlockSection *
+QUICAckFrame::ack_block_section()
+{
+  return this->_ack_block_section;
+}
+
+const QUICAckFrame::AckBlockSection *
+QUICAckFrame::ack_block_section() const
+{
+  return this->_ack_block_section;
+}
+
+const QUICAckFrame::TimestampSection *
+QUICAckFrame::timestamp_section() const
+{
+  return this->_timestamp_section;
+}
+
+size_t
+QUICAckFrame::_get_num_timestamp_offset() const
+{
+  if (this->has_ack_blocks()) {
+    return 2;
+  } else {
+    return 1;
+  }
+}
+
+/**
+ * LL of 101NLLMM
+ */
+size_t
+QUICAckFrame::_get_largest_acknowledged_length() const
+{
+  /*
+   * 0 -> 1 byte
+   * 1 -> 2 byte
+   * 2 -> 4 byte
+   * 3 -> 6 byte
+  */
+  int n = (this->_buf[0] & 0x0c) >> 2;
+  if (n == 0) {
+    return 1;
+  } else {
+    return n * 2;
+  }
+}
+
+size_t
+QUICAckFrame::_get_largest_acknowledged_offset() const
+{
+  return this->_get_num_timestamp_offset() + 1;
+}
+
+/**
+ * MM of 101NLLMM
+ */
+size_t
+QUICAckFrame::_get_ack_block_length() const
+{
+  /*
+   * 0 -> 1 byte
+   * 1 -> 2 byte
+   * 2 -> 4 byte
+   * 3 -> 6 byte
+  */
+  int n = this->_buf[0] & 0x03;
+  if (n == 0) {
+    return 1;
+  } else {
+    return n * 2;
+  }
+}
+
+size_t
+QUICAckFrame::_get_ack_delay_offset() const
+{
+  return this->_get_largest_acknowledged_offset() + this->_get_largest_acknowledged_length();
+}
+
+size_t
+QUICAckFrame::_get_ack_block_section_offset() const
+{
+  return this->_get_ack_delay_offset() + 2;
+}
+
+size_t
+QUICAckFrame::_get_timestamp_section_offset() const
+{
+  return this->_get_ack_block_section_offset() + this->ack_block_section()->size();
+}
+
+QUICAckFrame::AckBlockSection::AckBlockSection(const uint8_t *buf, uint8_t num_blocks, uint8_t ack_block_length)
+{
+  this->_buf              = buf;
+  this->_num_blocks       = num_blocks;
+  this->_ack_block_length = ack_block_length;
+}
+
+QUICAckFrame::AckBlockSection::AckBlockSection(uint64_t first_ack_block_length)
+{
+  this->_first_ack_block_length = first_ack_block_length;
+}
+
+QUICAckFrame::AckBlock::AckBlock(const uint8_t *buf, uint8_t ack_block_length)
+{
+  uint8_t gap     = buf[0];
+  uint64_t length = QUICTypeUtil::read_nbytes_as_uint(buf + 1, ack_block_length);
+  this->_data     = (static_cast<uint64_t>(gap) << 56) + length;
+}
+
+QUICAckFrame::AckBlock::AckBlock(uint8_t gap, uint64_t length)
+{
+  this->_data = (static_cast<uint64_t>(gap) << 56) + length;
+}
+
+uint8_t
+QUICAckFrame::AckBlock::gap() const
+{
+  return this->_data >> 56;
+}
+
+uint64_t
+QUICAckFrame::AckBlock::length() const
+{
+  return this->_data & 0x0000FFFFFFFFFFFF;
+}
+
+uint8_t
+QUICAckFrame::AckBlockSection::count() const
+{
+  if (this->_buf) {
+    return this->_num_blocks;
+  } else {
+    return this->_ack_blocks.size();
+  }
+}
+
+size_t
+QUICAckFrame::AckBlockSection::size() const
+{
+  if (this->_buf) {
+    return this->_ack_block_length + (this->_ack_block_length + 1) * this->_num_blocks;
+  } else {
+    // TODO Which block length should we use?
+    return 48 + (48 + 1) * this->_ack_blocks.size();
+  }
+}
+
+void
+QUICAckFrame::AckBlockSection::store(uint8_t *buf, size_t *len) const
+{
+  uint8_t *p = buf;
+  size_t dummy;
+  QUICTypeUtil::write_uint_as_nbytes(this->_first_ack_block_length, 4, buf, &dummy);
+  p += 4;
+  for (auto &&block : *this) {
+    p[0] = block.gap();
+    p += 1;
+    QUICTypeUtil::write_uint_as_nbytes(block.length(), 4, buf, &dummy);
+    p += 4;
+  }
+  *len = p - buf;
+}
+
+uint64_t
+QUICAckFrame::AckBlockSection::first_ack_block_length() const
+{
+  return this->_first_ack_block_length;
+}
+
+void
+QUICAckFrame::AckBlockSection::add_ack_block(AckBlock block)
+{
+  this->_ack_blocks.push_back(block);
+}
+
+QUICAckFrame::AckBlockSection::const_iterator
+QUICAckFrame::AckBlockSection::begin() const
+{
+  if (this->_buf) {
+    return const_iterator(0, this->_buf, this->_num_blocks, this->_ack_block_length);
+  } else {
+    return const_iterator(0, &this->_ack_blocks);
+  }
+}
+
+QUICAckFrame::AckBlockSection::const_iterator
+QUICAckFrame::AckBlockSection::end() const
+{
+  if (this->_buf) {
+    return const_iterator(this->_num_blocks, this->_buf, this->_num_blocks, this->_ack_block_length);
+  } else {
+    return const_iterator(this->_ack_blocks.size(), &this->_ack_blocks);
+  }
+}
+
+QUICAckFrame::AckBlockSection::const_iterator::const_iterator(uint8_t index, const uint8_t *buf, uint8_t num_blocks,
+                                                              uint8_t ack_block_length)
+{
+  this->_index         = index;
+  this->_buf           = buf;
+  this->_current_block = AckBlock(buf, ack_block_length);
+}
+
+QUICAckFrame::AckBlockSection::const_iterator::const_iterator(uint8_t index, const std::vector<QUICAckFrame::AckBlock> *ack_block)
+{
+  this->_index      = index;
+  this->_ack_blocks = ack_block;
+  if (this->_ack_blocks->size()) {
+    if (this->_ack_blocks->size() == this->_index) {
+      this->_current_block = {0, 0ULL};
+    } else {
+      this->_current_block = this->_ack_blocks->at(this->_index);
+    }
+  }
+}
+
+uint8_t
+QUICAckFrame::TimestampSection::count() const
+{
+  return 0;
+}
+
+size_t
+QUICAckFrame::TimestampSection::size() const
+{
+  return 0;
+}
+
+void
+QUICAckFrame::TimestampSection::store(uint8_t *buf, size_t *len) const
+{
+  if (this->count() == 0) {
+    *len = 0;
+    return;
+  }
+
+  // TODO: Store timestamp data
+}
+
+//
+// RST_STREAM frame
+//
+
+QUICRstStreamFrame::QUICRstStreamFrame(QUICErrorCode error_code, QUICStreamId stream_id, QUICOffset final_offset)
+  : _error_code(error_code), _stream_id(stream_id), _final_offset(final_offset)
+{
+}
+
+QUICFrameType
+QUICRstStreamFrame::type() const
+{
+  return QUICFrameType::RST_STREAM;
+}
+
+size_t
+QUICRstStreamFrame::size() const
+{
+  return 17;
+}
+
+void
+QUICRstStreamFrame::store(uint8_t *buf, size_t *len) const
+{
+  size_t n;
+  uint8_t *p = buf;
+  *p         = 0x01;
+  ++p;
+  QUICTypeUtil::write_QUICErrorCode(this->_error_code, p, &n);
+  p += n;
+  QUICTypeUtil::write_QUICStreamId(this->_stream_id, 4, p, &n);
+  p += n;
+  QUICTypeUtil::write_QUICOffset(this->_final_offset, 8, p, &n);
+  p += n;
+
+  *len = p - buf;
+}
+
+QUICErrorCode
+QUICRstStreamFrame::error_code() const
+{
+  return QUICTypeUtil::read_QUICErrorCode(this->_buf + 1);
+}
+
+QUICStreamId
+QUICRstStreamFrame::stream_id() const
+{
+  return QUICTypeUtil::read_QUICStreamId(this->_buf + 5, 4);
+}
+
+QUICOffset
+QUICRstStreamFrame::final_offset() const
+{
+  return QUICTypeUtil::read_QUICOffset(this->_buf + 9, 8);
+}
+
+//
+// PING frame
+//
+QUICFrameType
+QUICPingFrame::type() const
+{
+  return QUICFrameType::PING;
+}
+
+size_t
+QUICPingFrame::size() const
+{
+  return 1;
+}
+
+void
+QUICPingFrame::store(uint8_t *buf, size_t *len) const
+{
+  buf[0] = 0x07;
+  *len   = 1;
+}
+
+//
+// PADDING frame
+//
+QUICFrameType
+QUICPaddingFrame::type() const
+{
+  return QUICFrameType::PADDING;
+}
+
+size_t
+QUICPaddingFrame::size() const
+{
+  return 1;
+}
+
+void
+QUICPaddingFrame::store(uint8_t *buf, size_t *len) const
+{
+  buf[0] = 0x00;
+  *len   = 1;
+}
+
+//
+// GOAWAY frame
+//
+
+QUICGoawayFrame::QUICGoawayFrame(QUICStreamId client_stream_id, QUICStreamId server_stream_id)
+{
+  this->_client_stream_id = client_stream_id;
+  this->_server_stream_id = server_stream_id;
+}
+
+QUICFrameType
+QUICGoawayFrame::type() const
+{
+  return QUICFrameType::GOAWAY;
+}
+
+size_t
+QUICGoawayFrame::size() const
+{
+  return 9;
+}
+
+void
+QUICGoawayFrame::store(uint8_t *buf, size_t *len) const
+{
+  size_t n;
+  uint8_t *p = buf;
+  *p         = 0x03;
+  ++p;
+  QUICTypeUtil::write_QUICStreamId(this->_client_stream_id, 4, p, &n);
+  p += n;
+  QUICTypeUtil::write_QUICStreamId(this->_server_stream_id, 4, p, &n);
+  p += n;
+  *len = p - buf;
+}
+
+QUICStreamId
+QUICGoawayFrame::client_stream_id() const
+{
+  if (this->_buf) {
+    return QUICTypeUtil::read_QUICStreamId(this->_buf + 1, 4);
+  } else {
+    return this->_client_stream_id;
+  }
+}
+
+QUICStreamId
+QUICGoawayFrame::server_stream_id() const
+{
+  if (this->_buf) {
+    return QUICTypeUtil::read_QUICStreamId(this->_buf + 5, 4);
+  } else {
+    return this->_server_stream_id;
+  }
+}
+
+//
+// CONNECTION_CLOSE frame
+//
+
+QUICConnectionCloseFrame::QUICConnectionCloseFrame(QUICErrorCode error_code, uint16_t reason_phrase_length,
+                                                   const char *reason_phrase)
+{
+  this->_error_code           = error_code;
+  this->_reason_phrase_length = reason_phrase_length;
+  this->_reason_phrase        = reason_phrase;
+}
+
+QUICFrameType
+QUICConnectionCloseFrame::type() const
+{
+  return QUICFrameType::CONNECTION_CLOSE;
+}
+
+size_t
+QUICConnectionCloseFrame::size() const
+{
+  return 7 + this->reason_phrase_length();
+}
+
+void
+QUICConnectionCloseFrame::store(uint8_t *buf, size_t *len) const
+{
+  size_t n;
+  uint8_t *p = buf;
+  *p         = 0x02;
+  ++p;
+  QUICTypeUtil::write_QUICErrorCode(this->_error_code, p, &n);
+  p += n;
+  QUICTypeUtil::write_uint_as_nbytes(this->_reason_phrase_length, 2, p, &n);
+  p += n;
+  memcpy(p, this->_reason_phrase, this->_reason_phrase_length);
+  p += this->_reason_phrase_length;
+
+  *len = p - buf;
+}
+
+QUICErrorCode
+QUICConnectionCloseFrame::error_code() const
+{
+  if (this->_buf) {
+    return QUICTypeUtil::read_QUICErrorCode(this->_buf + 1);
+  } else {
+    return this->_error_code;
+  }
+}
+
+uint16_t
+QUICConnectionCloseFrame::reason_phrase_length() const
+{
+  if (this->_buf) {
+    return QUICTypeUtil::read_nbytes_as_uint(this->_buf + 5, 2);
+  } else {
+    return this->_reason_phrase_length;
+  }
+}
+
+const char *
+QUICConnectionCloseFrame::reason_phrase() const
+{
+  if (this->_buf) {
+    return reinterpret_cast<const char *>(this->_buf + 7);
+  } else {
+    return this->_reason_phrase;
+  }
+}
+
+//
+// MAX_DATA frame
+//
+QUICMaxDataFrame::QUICMaxDataFrame(uint64_t maximum_data)
+{
+  this->_maximum_data = maximum_data;
+}
+
+QUICFrameType
+QUICMaxDataFrame::type() const
+{
+  return QUICFrameType::MAX_DATA;
+}
+
+size_t
+QUICMaxDataFrame::size() const
+{
+  return 9;
+}
+
+void
+QUICMaxDataFrame::store(uint8_t *buf, size_t *len) const
+{
+  size_t n;
+  uint8_t *p = buf;
+  *p         = 0x04;
+  ++p;
+  QUICTypeUtil::write_uint_as_nbytes(this->_maximum_data, 8, p, &n);
+  p += n;
+
+  *len = p - buf;
+}
+
+uint64_t
+QUICMaxDataFrame::maximum_data() const
+{
+  if (this->_buf) {
+    return QUICTypeUtil::read_nbytes_as_uint(this->_buf + 1, 8);
+  } else {
+    return this->_maximum_data;
+  }
+}
+
+//
+// MAX_STREAM_DATA
+//
+QUICMaxStreamDataFrame::QUICMaxStreamDataFrame(QUICStreamId stream_id, uint64_t maximum_stream_data)
+{
+  this->_stream_id           = stream_id;
+  this->_maximum_stream_data = maximum_stream_data;
+}
+
+QUICFrameType
+QUICMaxStreamDataFrame::type() const
+{
+  return QUICFrameType::MAX_STREAM_DATA;
+}
+
+size_t
+QUICMaxStreamDataFrame::size() const
+{
+  return 13;
+}
+
+void
+QUICMaxStreamDataFrame::store(uint8_t *buf, size_t *len) const
+{
+  size_t n;
+  uint8_t *p = buf;
+  *p         = 0x05;
+  ++p;
+  QUICTypeUtil::write_uint_as_nbytes(this->_stream_id, 4, p, &n);
+  p += n;
+  QUICTypeUtil::write_uint_as_nbytes(this->_maximum_stream_data, 8, p, &n);
+  p += n;
+
+  *len = p - buf;
+}
+
+QUICStreamId
+QUICMaxStreamDataFrame::stream_id() const
+{
+  if (this->_buf) {
+    return QUICTypeUtil::read_nbytes_as_uint(this->_buf + 1, 4);
+  } else {
+    return this->_stream_id;
+  }
+}
+
+uint64_t
+QUICMaxStreamDataFrame::maximum_stream_data() const
+{
+  if (this->_buf) {
+    return QUICTypeUtil::read_nbytes_as_uint(this->_buf + 5, 8);
+  } else {
+    return this->_maximum_stream_data;
+  }
+}
+
+//
+// MAX_STREAM_ID
+//
+QUICMaxStreamIdFrame::QUICMaxStreamIdFrame(QUICStreamId maximum_stream_id)
+{
+  this->_maximum_stream_id = maximum_stream_id;
+}
+
+QUICFrameType
+QUICMaxStreamIdFrame::type() const
+{
+  return QUICFrameType::MAX_STREAM_ID;
+}
+
+size_t
+QUICMaxStreamIdFrame::size() const
+{
+  return 5;
+}
+
+void
+QUICMaxStreamIdFrame::store(uint8_t *buf, size_t *len) const
+{
+  size_t n;
+  uint8_t *p = buf;
+  *p         = 0x06;
+  ++p;
+  QUICTypeUtil::write_uint_as_nbytes(this->_maximum_stream_id, 4, p, &n);
+  p += n;
+
+  *len = p - buf;
+}
+
+QUICStreamId
+QUICMaxStreamIdFrame::maximum_stream_id() const
+{
+  if (this->_buf) {
+    return QUICTypeUtil::read_nbytes_as_uint(this->_buf + 1, 4);
+  } else {
+    return this->_maximum_stream_id;
+  }
+}
+
+//
+// BLOCKED frame
+//
+QUICFrameType
+QUICBlockedFrame::type() const
+{
+  return QUICFrameType::BLOCKED;
+}
+
+size_t
+QUICBlockedFrame::size() const
+{
+  return 1;
+}
+
+void
+QUICBlockedFrame::store(uint8_t *buf, size_t *len) const
+{
+  buf[0] = 0x08;
+  *len   = 1;
+}
+
+//
+// STREAM_BLOCKED frame
+//
+QUICStreamBlockedFrame::QUICStreamBlockedFrame(QUICStreamId stream_id)
+{
+  this->_stream_id = stream_id;
+}
+
+QUICFrameType
+QUICStreamBlockedFrame::type() const
+{
+  return QUICFrameType::STREAM_BLOCKED;
+}
+
+size_t
+QUICStreamBlockedFrame::size() const
+{
+  return 5;
+}
+
+void
+QUICStreamBlockedFrame::store(uint8_t *buf, size_t *len) const
+{
+  size_t n;
+  uint8_t *p = buf;
+  *p         = 0x09;
+  ++p;
+  QUICTypeUtil::write_uint_as_nbytes(this->_stream_id, 4, p, &n);
+  p += n;
+
+  *len = p - buf;
+}
+
+QUICStreamId
+QUICStreamBlockedFrame::stream_id() const
+{
+  if (this->_buf) {
+    return QUICTypeUtil::read_QUICStreamId(this->_buf + 1, 4);
+  } else {
+    return this->_stream_id;
+  }
+}
+
+//
+// STREAM_ID_NEEDED frame
+//
+QUICFrameType
+QUICStreamIdNeededFrame::type() const
+{
+  return QUICFrameType::STREAM_ID_NEEDED;
+}
+
+size_t
+QUICStreamIdNeededFrame::size() const
+{
+  return 1;
+}
+
+void
+QUICStreamIdNeededFrame::store(uint8_t *buf, size_t *len) const
+{
+  buf[0] = 0x0a;
+  *len   = 1;
+}
+
+//
+// NEW_CONNECTION_ID frame
+//
+
+QUICNewConnectionIdFrame::QUICNewConnectionIdFrame(uint16_t sequence, QUICConnectionId connection_id)
+{
+  this->_sequence      = sequence;
+  this->_connection_id = connection_id;
+}
+
+QUICFrameType
+QUICNewConnectionIdFrame::type() const
+{
+  return QUICFrameType::NEW_CONNECTION_ID;
+}
+
+size_t
+QUICNewConnectionIdFrame::size() const
+{
+  return 11;
+}
+
+void
+QUICNewConnectionIdFrame::store(uint8_t *buf, size_t *len) const
+{
+  size_t n;
+  uint8_t *p = buf;
+  *p         = 0x0b;
+  ++p;
+  QUICTypeUtil::write_uint_as_nbytes(this->_sequence, 2, p, &n);
+  p += n;
+  QUICTypeUtil::write_QUICConnectionId(this->_connection_id, 8, p, &n);
+  p += n;
+
+  *len = p - buf;
+}
+
+uint16_t
+QUICNewConnectionIdFrame::sequence() const
+{
+  if (this->_buf) {
+    return QUICTypeUtil::read_nbytes_as_uint(this->_buf + 1, 2);
+  } else {
+    return this->_sequence;
+  }
+}
+
+QUICConnectionId
+QUICNewConnectionIdFrame::connection_id() const
+{
+  if (this->_buf) {
+    return QUICTypeUtil::read_QUICConnectionId(this->_buf + 3, 8);
+  } else {
+    return this->_connection_id;
+  }
+}
+
+//
+// QUICRetransmissionFrame
+//
+QUICRetransmissionFrame::QUICRetransmissionFrame(std::unique_ptr<QUICFrame, QUICFrameDeleterFunc> original_frame,
+                                                 const QUICPacket &original_packet)
+  : QUICFrame(), _packet_type(original_packet.type())
+{
+  size_t dummy;
+  this->_size = original_frame->size();
+  this->_data = ats_unique_malloc(this->_size);
+  this->_buf  = this->_data.get();
+  original_frame->store(this->_data.get(), &dummy);
+}
+
+size_t
+QUICRetransmissionFrame::size() const
+{
+  return this->_size;
+}
+
+void
+QUICRetransmissionFrame::store(uint8_t *buf, size_t *len) const
+{
+  memcpy(buf, this->_data.get(), this->_size);
+  *len = this->_size;
+}
+
+QUICPacketType
+QUICRetransmissionFrame::packet_type() const
+{
+  return this->_packet_type;
+}
+
+//
+// QUICFrameFactory
+//
+
+std::unique_ptr<QUICFrame, QUICFrameDeleterFunc>
+QUICFrameFactory::create(const uint8_t *buf, size_t len)
+{
+  QUICFrame *frame;
+
+  switch (QUICFrame::type(buf)) {
+  case QUICFrameType::STREAM:
+    frame = quicStreamFrameAllocator.alloc();
+    new (frame) QUICStreamFrame(buf, len);
+    return std::unique_ptr<QUICFrame, QUICFrameDeleterFunc>(frame, &QUICFrameDeleter::delete_stream_frame);
+  case QUICFrameType::ACK:
+    frame = quicAckFrameAllocator.alloc();
+    new (frame) QUICAckFrame(buf, len);
+    return std::unique_ptr<QUICFrame, QUICFrameDeleterFunc>(frame, &QUICFrameDeleter::delete_ack_frame);
+  case QUICFrameType::PADDING:
+    frame = quicPaddingFrameAllocator.alloc();
+    new (frame) QUICPaddingFrame(buf, len);
+    return std::unique_ptr<QUICFrame, QUICFrameDeleterFunc>(frame, &QUICFrameDeleter::delete_padding_frame);
+  case QUICFrameType::RST_STREAM:
+    frame = quicRstStreamFrameAllocator.alloc();
+    new (frame) QUICRstStreamFrame(buf, len);
+    return std::unique_ptr<QUICFrame, QUICFrameDeleterFunc>(frame, &QUICFrameDeleter::delete_rst_stream_frame);
+  case QUICFrameType::CONNECTION_CLOSE:
+    frame = quicConnectionCloseFrameAllocator.alloc();
+    new (frame) QUICConnectionCloseFrame(buf, len);
+    return std::unique_ptr<QUICFrame, QUICFrameDeleterFunc>(frame, &QUICFrameDeleter::delete_connection_close_frame);
+  case QUICFrameType::GOAWAY:
+    frame = quicGoawayFrameAllocator.alloc();
+    new (frame) QUICGoawayFrame(buf, len);
+    return std::unique_ptr<QUICFrame, QUICFrameDeleterFunc>(frame, &QUICFrameDeleter::delete_goaway_frame);
+  case QUICFrameType::MAX_DATA:
+    frame = quicMaxDataFrameAllocator.alloc();
+    new (frame) QUICMaxDataFrame(buf, len);
+    return std::unique_ptr<QUICFrame, QUICFrameDeleterFunc>(frame, &QUICFrameDeleter::delete_max_data_frame);
+  case QUICFrameType::MAX_STREAM_DATA:
+    frame = quicMaxStreamDataFrameAllocator.alloc();
+    new (frame) QUICMaxStreamDataFrame(buf, len);
+    return std::unique_ptr<QUICFrame, QUICFrameDeleterFunc>(frame, &QUICFrameDeleter::delete_max_stream_data_frame);
+  case QUICFrameType::MAX_STREAM_ID:
+    frame = quicMaxStreamIdFrameAllocator.alloc();
+    new (frame) QUICMaxStreamIdFrame(buf, len);
+    return std::unique_ptr<QUICFrame, QUICFrameDeleterFunc>(frame, &QUICFrameDeleter::delete_max_stream_id_frame);
+  case QUICFrameType::PING:
+    frame = quicPingFrameAllocator.alloc();
+    new (frame) QUICPingFrame(buf, len);
+    return std::unique_ptr<QUICFrame, QUICFrameDeleterFunc>(frame, &QUICFrameDeleter::delete_ping_frame);
+  case QUICFrameType::BLOCKED:
+    frame = quicBlockedFrameAllocator.alloc();
+    new (frame) QUICBlockedFrame(buf, len);
+    return std::unique_ptr<QUICFrame, QUICFrameDeleterFunc>(frame, &QUICFrameDeleter::delete_blocked_frame);
+  case QUICFrameType::STREAM_BLOCKED:
+    frame = quicStreamBlockedFrameAllocator.alloc();
+    new (frame) QUICStreamBlockedFrame(buf, len);
+    return std::unique_ptr<QUICFrame, QUICFrameDeleterFunc>(frame, &QUICFrameDeleter::delete_stream_blocked_frame);
+  case QUICFrameType::STREAM_ID_NEEDED:
+    frame = quicStreamIdNeededFrameAllocator.alloc();
+    new (frame) QUICStreamIdNeededFrame(buf, len);
+    return std::unique_ptr<QUICFrame, QUICFrameDeleterFunc>(frame, &QUICFrameDeleter::delete_stream_id_needed_frame);
+  case QUICFrameType::NEW_CONNECTION_ID:
+    frame = quicNewConnectionIdFrameAllocator.alloc();
+    new (frame) QUICNewConnectionIdFrame(buf, len);
+    return std::unique_ptr<QUICFrame, QUICFrameDeleterFunc>(frame, &QUICFrameDeleter::delete_new_connection_id_frame);
+  default:
+    // Unknown frame
+    return std::unique_ptr<QUICFrame, QUICFrameDeleterFunc>(nullptr, &QUICFrameDeleter::delete_null_frame);
+  }
+}
+
+std::shared_ptr<const QUICFrame>
+QUICFrameFactory::fast_create(const uint8_t *buf, size_t len)
+{
+  if (QUICFrame::type(buf) == QUICFrameType::UNKNOWN) {
+    return nullptr;
+  }
+
+  std::shared_ptr<QUICFrame> frame = this->_reusable_frames[static_cast<ptrdiff_t>(QUICFrame::type(buf))];
+
+  if (frame == nullptr) {
+    frame = QUICFrameFactory::create(buf, len);
+    if (frame != nullptr) {
+      this->_reusable_frames[static_cast<ptrdiff_t>(QUICFrame::type(buf))] = frame;
+    }
+  } else {
+    frame->reset(buf, len);
+  }
+
+  return frame;
+}
+
+std::unique_ptr<QUICStreamFrame, QUICFrameDeleterFunc>
+QUICFrameFactory::create_stream_frame(const uint8_t *data, size_t data_len, QUICStreamId stream_id, QUICOffset offset)
+{
+  QUICStreamFrame *frame = quicStreamFrameAllocator.alloc();
+  new (frame) QUICStreamFrame(data, data_len, stream_id, offset);
+  return std::unique_ptr<QUICStreamFrame, QUICFrameDeleterFunc>(frame, &QUICFrameDeleter::delete_stream_frame);
+}
+
+std::unique_ptr<QUICAckFrame, QUICFrameDeleterFunc>
+QUICFrameFactory::create_ack_frame(QUICPacketNumber largest_acknowledged, uint16_t ack_delay, uint64_t first_ack_block_length)
+{
+  QUICAckFrame *frame = quicAckFrameAllocator.alloc();
+  new (frame) QUICAckFrame(largest_acknowledged, ack_delay, first_ack_block_length);
+  return std::unique_ptr<QUICAckFrame, QUICFrameDeleterFunc>(frame, &QUICFrameDeleter::delete_ack_frame);
+}
+
+std::unique_ptr<QUICConnectionCloseFrame, QUICFrameDeleterFunc>
+QUICFrameFactory::create_connection_close_frame(QUICErrorCode error_code, uint16_t reason_phrase_length, const char *reason_phrase)
+{
+  QUICConnectionCloseFrame *frame = quicConnectionCloseFrameAllocator.alloc();
+  new (frame) QUICConnectionCloseFrame(error_code, reason_phrase_length, reason_phrase);
+  return std::unique_ptr<QUICConnectionCloseFrame, QUICFrameDeleterFunc>(frame, &QUICFrameDeleter::delete_connection_close_frame);
+}
+
+std::unique_ptr<QUICRetransmissionFrame, QUICFrameDeleterFunc>
+QUICFrameFactory::create_retransmission_frame(std::unique_ptr<QUICFrame, QUICFrameDeleterFunc> original_frame,
+                                              const QUICPacket &original_packet)
+{
+  QUICRetransmissionFrame *frame = quicRetransmissionFrameAllocator.alloc();
+  new (frame) QUICRetransmissionFrame(std::move(original_frame), original_packet);
+  return std::unique_ptr<QUICRetransmissionFrame, QUICFrameDeleterFunc>(frame, &QUICFrameDeleter::delete_retransmission_frame);
+}
diff --git a/iocore/net/quic/QUICFrame.h b/iocore/net/quic/QUICFrame.h
new file mode 100644
index 0000000..a424780
--- /dev/null
+++ b/iocore/net/quic/QUICFrame.h
@@ -0,0 +1,616 @@
+/** @file
+ *
+ *  A brief file description
+ *
+ *  @section license License
+ *
+ *  Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+#pragma once
+
+#include <memory>
+#include "ts/List.h"
+#include <vector>
+#include <iterator>
+
+#include "QUICTypes.h"
+#include "QUICPacket.h"
+
+class QUICFrame
+{
+public:
+  QUICFrame(const uint8_t *buf, size_t len) : _buf(buf), _len(len){};
+  virtual QUICFrameType type() const;
+  virtual size_t size() const = 0;
+  virtual void store(uint8_t *buf, size_t *len) const = 0;
+  virtual void reset(const uint8_t *buf, size_t len);
+  static QUICFrameType type(const uint8_t *buf);
+
+  LINK(QUICFrame, link);
+
+protected:
+  QUICFrame() {}
+  const uint8_t *_buf = nullptr;
+  size_t _len         = 0;
+};
+
+//
+// STREAM Frame
+//
+
+class QUICStreamFrame : public QUICFrame
+{
+public:
+  QUICStreamFrame() : QUICFrame() {}
+  QUICStreamFrame(const uint8_t *buf, size_t len) : QUICFrame(buf, len) {}
+  QUICStreamFrame(const uint8_t *buf, size_t len, QUICStreamId streamid, QUICOffset offset);
+  virtual QUICFrameType type() const override;
+  virtual size_t size() const override;
+  virtual void store(uint8_t *buf, size_t *len) const override;
+  void store(uint8_t *buf, size_t *len, bool include_length_field) const;
+  QUICStreamId stream_id() const;
+  QUICOffset offset() const;
+  const uint8_t *data() const;
+  size_t data_length() const;
+  bool has_data_length_field() const;
+  bool has_fin_flag() const;
+
+  LINK(QUICStreamFrame, link);
+
+private:
+  const uint8_t *_data    = nullptr;
+  size_t _data_len        = 0;
+  QUICStreamId _stream_id = 0;
+  QUICOffset _offset      = 0;
+  bool _fin               = false;
+  size_t _get_data_offset() const;
+  size_t _get_stream_id_offset() const;
+  size_t _get_offset_offset() const;
+  size_t _get_stream_id_len() const;
+  size_t _get_offset_len() const;
+};
+
+//
+// ACK Frame
+//
+
+class QUICAckFrame : public QUICFrame
+{
+public:
+  class AckBlock
+  {
+  public:
+    AckBlock(const uint8_t *buf, uint8_t ack_block_length);
+    AckBlock(uint8_t gap, uint64_t length);
+    uint8_t gap() const;
+    uint64_t length() const;
+    LINK(QUICAckFrame::AckBlock, link);
+
+  private:
+    uint64_t _data = 0;
+  };
+
+  class AckBlockSection
+  {
+  public:
+    class const_iterator : public std::iterator<std::input_iterator_tag, QUICAckFrame::AckBlock>
+    {
+    public:
+      const_iterator(uint8_t index, const uint8_t *buf, uint8_t num_blocks, uint8_t ack_block_length);
+      const_iterator(uint8_t index, const std::vector<QUICAckFrame::AckBlock> *ack_blocks);
+
+      const QUICAckFrame::AckBlock &operator*() const { return this->_current_block; };
+      const QUICAckFrame::AckBlock *operator->() const { return &this->_current_block; };
+
+      const QUICAckFrame::AckBlock &operator++()
+      {
+        ++(this->_index);
+
+        if (this->_buf) {
+          // TODO Parse Ack Block
+        } else {
+          if (this->_ack_blocks->size() == this->_index) {
+            this->_current_block = {0, 0ULL};
+          } else {
+            this->_current_block = this->_ack_blocks->at(this->_index);
+          }
+        }
+
+        return this->_current_block;
+      };
+
+      const bool
+      operator!=(const const_iterator &ite) const
+      {
+        return this->_index != ite._index;
+      };
+
+      const bool
+      operator==(const const_iterator &ite) const
+      {
+        return this->_index == ite._index;
+      };
+
+    private:
+      uint8_t _index;
+      const uint8_t *_buf;
+      const std::vector<QUICAckFrame::AckBlock> *_ack_blocks = nullptr;
+      QUICAckFrame::AckBlock _current_block                  = {0, 0ULL};
+    };
+
+    AckBlockSection(uint64_t first_ack_block_length);
+    AckBlockSection(const uint8_t *buf, uint8_t num_blocks, uint8_t ack_block_length);
+    uint8_t count() const;
+    size_t size() const;
+    void store(uint8_t *buf, size_t *len) const;
+    uint64_t first_ack_block_length() const;
+    void add_ack_block(const AckBlock block);
+    const_iterator begin() const;
+    const_iterator end() const;
+
+  private:
+    const uint8_t *_buf              = nullptr;
+    uint64_t _first_ack_block_length = 0;
+    uint8_t _num_blocks              = 0;
+    uint8_t _ack_block_length        = 0;
+    std::vector<QUICAckFrame::AckBlock> _ack_blocks;
+  };
+
+  class TimestampSection
+  {
+  public:
+    uint8_t count() const;
+    size_t size() const;
+    void store(uint8_t *buf, size_t *len) const;
+    void add_timestamp();
+  };
+
+  QUICAckFrame() : QUICFrame() {}
+  QUICAckFrame(const uint8_t *buf, size_t len);
+  QUICAckFrame(QUICPacketNumber largest_acknowledged, uint16_t ack_delay, uint64_t first_ack_block_length);
+  ~QUICAckFrame();
+  virtual void reset(const uint8_t *buf, size_t len) override;
+  virtual QUICFrameType type() const override;
+  virtual size_t size() const override;
+  virtual void store(uint8_t *buf, size_t *len) const override;
+  uint8_t num_blocks() const;
+  uint8_t num_timestamps() const;
+  QUICPacketNumber largest_acknowledged() const;
+  uint16_t ack_delay() const;
+  const AckBlockSection *ack_block_section() const;
+  AckBlockSection *ack_block_section();
+  const TimestampSection *timestamp_section() const;
+  bool has_ack_blocks() const;
+
+private:
+  size_t _get_num_timestamp_offset() const;
+  size_t _get_largest_acknowledged_offset() const;
+  size_t _get_largest_acknowledged_length() const;
+  size_t _get_ack_block_length() const;
+  size_t _get_ack_delay_offset() const;
+  size_t _get_ack_block_section_offset() const;
+  size_t _get_timestamp_section_offset() const;
+  QUICPacketNumber _largest_acknowledged = 0;
+  uint16_t _ack_delay                    = 0;
+  AckBlockSection *_ack_block_section    = nullptr;
+  TimestampSection *_timestamp_section   = nullptr;
+};
+
+//
+// RST_STREAM
+//
+
+class QUICRstStreamFrame : public QUICFrame
+{
+public:
+  QUICRstStreamFrame() : QUICFrame() {}
+  QUICRstStreamFrame(const uint8_t *buf, size_t len) : QUICFrame(buf, len) {}
+  QUICRstStreamFrame(QUICErrorCode error_code, QUICStreamId stream_id, QUICOffset final_offset);
+  virtual QUICFrameType type() const override;
+  virtual size_t size() const override;
+  virtual void store(uint8_t *buf, size_t *len) const override;
+  QUICErrorCode error_code() const;
+  QUICStreamId stream_id() const;
+  QUICOffset final_offset() const;
+
+private:
+  QUICErrorCode _error_code;
+  QUICStreamId _stream_id  = 0;
+  QUICOffset _final_offset = 0;
+};
+
+//
+// PING
+//
+
+class QUICPingFrame : public QUICFrame
+{
+public:
+  QUICPingFrame() : QUICFrame() {}
+  QUICPingFrame(const uint8_t *buf, size_t len) : QUICFrame(buf, len) {}
+  virtual QUICFrameType type() const override;
+  virtual size_t size() const override;
+  virtual void store(uint8_t *buf, size_t *len) const override;
+};
+
+// PADDING
+
+class QUICPaddingFrame : public QUICFrame
+{
+public:
+  QUICPaddingFrame() : QUICFrame() {}
+  QUICPaddingFrame(const uint8_t *buf, size_t len) : QUICFrame(buf, len) {}
+  virtual QUICFrameType type() const override;
+  virtual size_t size() const override;
+  virtual void store(uint8_t *buf, size_t *len) const override;
+};
+
+//
+// GOAWAY
+//
+
+class QUICGoawayFrame : public QUICFrame
+{
+public:
+  QUICGoawayFrame() : QUICFrame() {}
+  QUICGoawayFrame(const uint8_t *buf, size_t len) : QUICFrame(buf, len) {}
+  QUICGoawayFrame(QUICStreamId client_stream_id, QUICStreamId server_stream_id);
+  virtual QUICFrameType type() const override;
+  virtual size_t size() const override;
+  virtual void store(uint8_t *buf, size_t *len) const override;
+  QUICStreamId client_stream_id() const;
+  QUICStreamId server_stream_id() const;
+
+private:
+  QUICStreamId _client_stream_id = 0;
+  QUICStreamId _server_stream_id = 0;
+};
+
+//
+// CONNECTION_CLOSE
+//
+
+class QUICConnectionCloseFrame : public QUICFrame
+{
+public:
+  QUICConnectionCloseFrame() : QUICFrame() {}
+  QUICConnectionCloseFrame(const uint8_t *buf, size_t len) : QUICFrame(buf, len) {}
+  QUICConnectionCloseFrame(QUICErrorCode error_code, uint16_t reason_phrase_length, const char *reason_phrase);
+  virtual QUICFrameType type() const override;
+  virtual size_t size() const override;
+  virtual void store(uint8_t *buf, size_t *len) const override;
+  QUICErrorCode error_code() const;
+  uint16_t reason_phrase_length() const;
+  const char *reason_phrase() const;
+
+private:
+  QUICErrorCode _error_code;
+  uint16_t _reason_phrase_length = 0;
+  const char *_reason_phrase     = nullptr;
+};
+
+//
+// MAX_DATA
+//
+
+class QUICMaxDataFrame : public QUICFrame
+{
+public:
+  QUICMaxDataFrame() : QUICFrame() {}
+  QUICMaxDataFrame(const uint8_t *buf, size_t len) : QUICFrame(buf, len) {}
+  QUICMaxDataFrame(uint64_t maximum_data);
+  virtual QUICFrameType type() const override;
+  virtual size_t size() const override;
+  virtual void store(uint8_t *buf, size_t *len) const override;
+  uint64_t maximum_data() const;
+
+private:
+  uint64_t _maximum_data = 0;
+};
+
+//
+// MAX_STREAM_DATA
+//
+
+class QUICMaxStreamDataFrame : public QUICFrame
+{
+public:
+  QUICMaxStreamDataFrame() : QUICFrame() {}
+  QUICMaxStreamDataFrame(const uint8_t *buf, size_t len) : QUICFrame(buf, len) {}
+  QUICMaxStreamDataFrame(QUICStreamId stream_id, uint64_t maximum_stream_data);
+  virtual QUICFrameType type() const override;
+  virtual size_t size() const override;
+  virtual void store(uint8_t *buf, size_t *len) const override;
+  QUICStreamId stream_id() const;
+  uint64_t maximum_stream_data() const;
+
+private:
+  QUICStreamId _stream_id       = 0;
+  uint64_t _maximum_stream_data = 0;
+};
+
+//
+// MAX_STREAM_ID
+//
+
+class QUICMaxStreamIdFrame : public QUICFrame
+{
+public:
+  QUICMaxStreamIdFrame() : QUICFrame() {}
+  QUICMaxStreamIdFrame(const uint8_t *buf, size_t len) : QUICFrame(buf, len) {}
+  QUICMaxStreamIdFrame(QUICStreamId maximum_stream_id);
+  virtual QUICFrameType type() const override;
+  virtual size_t size() const override;
+  virtual void store(uint8_t *buf, size_t *len) const override;
+  QUICStreamId maximum_stream_id() const;
+
+private:
+  QUICStreamId _maximum_stream_id = 0;
+};
+
+//
+// BLOCKED
+//
+class QUICBlockedFrame : public QUICFrame
+{
+public:
+  QUICBlockedFrame() : QUICFrame() {}
+  QUICBlockedFrame(const uint8_t *buf, size_t len) : QUICFrame(buf, len) {}
+  virtual QUICFrameType type() const override;
+  virtual size_t size() const override;
+  virtual void store(uint8_t *buf, size_t *len) const override;
+};
+
+//
+// STREAM_BLOCKED
+//
+
+class QUICStreamBlockedFrame : public QUICFrame
+{
+public:
+  QUICStreamBlockedFrame() : QUICFrame() {}
+  QUICStreamBlockedFrame(const uint8_t *buf, size_t len) : QUICFrame(buf, len) {}
+  QUICStreamBlockedFrame(QUICStreamId stream_id);
+  virtual QUICFrameType type() const override;
+  virtual size_t size() const override;
+  virtual void store(uint8_t *buf, size_t *len) const override;
+  QUICStreamId stream_id() const;
+
+private:
+  QUICStreamId _stream_id;
+};
+
+//
+// STREAM_ID_NEEDED
+//
+class QUICStreamIdNeededFrame : public QUICFrame
+{
+public:
+  QUICStreamIdNeededFrame() : QUICFrame() {}
+  QUICStreamIdNeededFrame(const uint8_t *buf, size_t len) : QUICFrame(buf, len) {}
+  virtual QUICFrameType type() const override;
+  virtual size_t size() const override;
+  virtual void store(uint8_t *buf, size_t *len) const override;
+};
+
+//
+// NEW_CONNECTION_ID
+//
+
+class QUICNewConnectionIdFrame : public QUICFrame
+{
+public:
+  QUICNewConnectionIdFrame() : QUICFrame() {}
+  QUICNewConnectionIdFrame(const uint8_t *buf, size_t len) : QUICFrame(buf, len) {}
+  QUICNewConnectionIdFrame(uint16_t sequence, QUICConnectionId connection_id);
+  virtual QUICFrameType type() const override;
+  virtual size_t size() const override;
+  virtual void store(uint8_t *buf, size_t *len) const override;
+  uint16_t sequence() const;
+  QUICConnectionId connection_id() const;
+
+private:
+  uint16_t _sequence              = 0;
+  QUICConnectionId _connection_id = 0;
+};
+
+typedef void (*QUICFrameDeleterFunc)(QUICFrame *p);
+
+//
+// Retransmission Frame
+//
+
+class QUICRetransmissionFrame : public QUICFrame
+{
+public:
+  QUICRetransmissionFrame() : QUICFrame() {}
+  QUICRetransmissionFrame(std::unique_ptr<QUICFrame, QUICFrameDeleterFunc> original_frame, const QUICPacket &original_packet);
+  virtual size_t size() const override;
+  virtual void store(uint8_t *buf, size_t *len) const override;
+  QUICPacketType packet_type() const;
+
+private:
+  std::unique_ptr<QUICFrame, QUICFrameDeleterFunc> _frame = std::unique_ptr<QUICFrame, QUICFrameDeleterFunc>(nullptr, nullptr);
+  ats_unique_buf _data = ats_unique_buf(nullptr, [](void *p) { ats_free(p); });
+  size_t _size;
+  QUICPacketType _packet_type;
+};
+
+extern ClassAllocator<QUICStreamFrame> quicStreamFrameAllocator;
+extern ClassAllocator<QUICAckFrame> quicAckFrameAllocator;
+extern ClassAllocator<QUICPaddingFrame> quicPaddingFrameAllocator;
+extern ClassAllocator<QUICRstStreamFrame> quicRstStreamFrameAllocator;
+extern ClassAllocator<QUICConnectionCloseFrame> quicConnectionCloseFrameAllocator;
+extern ClassAllocator<QUICGoawayFrame> quicGoawayFrameAllocator;
+extern ClassAllocator<QUICMaxDataFrame> quicMaxDataFrameAllocator;
+extern ClassAllocator<QUICMaxStreamDataFrame> quicMaxStreamDataFrameAllocator;
+extern ClassAllocator<QUICMaxStreamIdFrame> quicMaxStreamIdFrameAllocator;
+extern ClassAllocator<QUICPingFrame> quicPingFrameAllocator;
+extern ClassAllocator<QUICBlockedFrame> quicBlockedFrameAllocator;
+extern ClassAllocator<QUICStreamBlockedFrame> quicStreamBlockedFrameAllocator;
+extern ClassAllocator<QUICStreamIdNeededFrame> quicStreamIdNeededFrameAllocator;
+extern ClassAllocator<QUICNewConnectionIdFrame> quicNewConnectionIdFrameAllocator;
+extern ClassAllocator<QUICRetransmissionFrame> quicRetransmissionFrameAllocator;
+
+class QUICFrameDeleter
+{
+public:
+  // TODO Probably these methods should call destructor
+  static void
+  delete_null_frame(QUICFrame *frame)
+  {
+  }
+
+  static void
+  delete_stream_frame(QUICFrame *frame)
+  {
+    quicStreamFrameAllocator.free(static_cast<QUICStreamFrame *>(frame));
+  }
+
+  static void
+  delete_ack_frame(QUICFrame *frame)
+  {
+    quicAckFrameAllocator.free(static_cast<QUICAckFrame *>(frame));
+  }
+
+  static void
+  delete_padding_frame(QUICFrame *frame)
+  {
+    quicPaddingFrameAllocator.free(static_cast<QUICPaddingFrame *>(frame));
+  }
+
+  static void
+  delete_rst_stream_frame(QUICFrame *frame)
+  {
+    quicRstStreamFrameAllocator.free(static_cast<QUICRstStreamFrame *>(frame));
+  }
+
+  static void
+  delete_connection_close_frame(QUICFrame *frame)
+  {
+    quicConnectionCloseFrameAllocator.free(static_cast<QUICConnectionCloseFrame *>(frame));
+  }
+
+  static void
+  delete_goaway_frame(QUICFrame *frame)
+  {
+    quicGoawayFrameAllocator.free(static_cast<QUICGoawayFrame *>(frame));
+  }
+
+  static void
+  delete_max_data_frame(QUICFrame *frame)
+  {
+    quicMaxDataFrameAllocator.free(static_cast<QUICMaxDataFrame *>(frame));
+  }
+
+  static void
+  delete_max_stream_data_frame(QUICFrame *frame)
+  {
+    quicMaxStreamDataFrameAllocator.free(static_cast<QUICMaxStreamDataFrame *>(frame));
+  }
+
+  static void
+  delete_max_stream_id_frame(QUICFrame *frame)
+  {
+    quicMaxStreamIdFrameAllocator.free(static_cast<QUICMaxStreamIdFrame *>(frame));
+  }
+
+  static void
+  delete_ping_frame(QUICFrame *frame)
+  {
+    quicPingFrameAllocator.free(static_cast<QUICPingFrame *>(frame));
+  }
+
+  static void
+  delete_blocked_frame(QUICFrame *frame)
+  {
+    quicBlockedFrameAllocator.free(static_cast<QUICBlockedFrame *>(frame));
+  }
+
+  static void
+  delete_stream_blocked_frame(QUICFrame *frame)
+  {
+    quicStreamBlockedFrameAllocator.free(static_cast<QUICStreamBlockedFrame *>(frame));
+  }
+
+  static void
+  delete_stream_id_needed_frame(QUICFrame *frame)
+  {
+    quicStreamIdNeededFrameAllocator.free(static_cast<QUICStreamIdNeededFrame *>(frame));
+  }
+
+  static void
+  delete_new_connection_id_frame(QUICFrame *frame)
+  {
+    quicNewConnectionIdFrameAllocator.free(static_cast<QUICNewConnectionIdFrame *>(frame));
+  }
+
+  static void
+  delete_retransmission_frame(QUICFrame *frame)
+  {
+    quicRetransmissionFrameAllocator.free(static_cast<QUICRetransmissionFrame *>(frame));
+  }
+};
+
+//
+// QUICFrameFactory
+//
+class QUICFrameFactory
+{
+public:
+  /*
+   * This is used for creating a QUICFrame object based on received data.
+   */
+  static std::unique_ptr<QUICFrame, QUICFrameDeleterFunc> create(const uint8_t *buf, size_t len);
+
+  /*
+   * This works almost the same as create() but it reuses created objects for performance.
+   * If you create a frame object which has the same frame type that you created before, the object will be reset by new data.
+   */
+  std::shared_ptr<const QUICFrame> fast_create(const uint8_t *buf, size_t len);
+
+  /*
+   * Creates a STREAM frame.
+   * You have to make sure that the data size won't exceed the maximum size of QUIC packet.
+   */
+  static std::unique_ptr<QUICStreamFrame, QUICFrameDeleterFunc> create_stream_frame(const uint8_t *data, size_t data_len,
+                                                                                    QUICStreamId stream_id, QUICOffset offset);
+  /*
+   * Creates a ACK frame.
+   * You shouldn't call this directly but through QUICAckFrameCreator because QUICAckFrameCreator manages packet numbers that we
+   * need to ack.
+   */
+  static std::unique_ptr<QUICAckFrame, QUICFrameDeleterFunc> create_ack_frame(QUICPacketNumber largest_acknowledged,
+                                                                              uint16_t ack_delay, uint64_t first_ack_block_length);
+  /*
+   * Creates a CONNECTION_CLOSE frame.
+   */
+  static std::unique_ptr<QUICConnectionCloseFrame, QUICFrameDeleterFunc> create_connection_close_frame(
+    QUICErrorCode error_code, uint16_t reason_phrase_length, const char *reason_phrase);
+
+  /*
+   * Creates a retransmission frame, which is very special.
+   * This retransmission frame will be used only for retransmission and it's not a standard frame type.
+   */
+  static std::unique_ptr<QUICRetransmissionFrame, QUICFrameDeleterFunc> create_retransmission_frame(
+    std::unique_ptr<QUICFrame, QUICFrameDeleterFunc> original_frame, const QUICPacket &original_packet);
+
+private:
+  // FIXME Actual number of frame types is several but some of the values are not sequential.
+  std::shared_ptr<QUICFrame> _reusable_frames[256] = {nullptr};
+};
diff --git a/iocore/net/quic/QUICFrameDispatcher.cc b/iocore/net/quic/QUICFrameDispatcher.cc
new file mode 100644
index 0000000..2a79560
--- /dev/null
+++ b/iocore/net/quic/QUICFrameDispatcher.cc
@@ -0,0 +1,148 @@
+/** @file
+ *
+ *  A brief file description
+ *
+ *  @section license License
+ *
+ *  Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+#include "QUICFrameDispatcher.h"
+#include "QUICConnectionManager.h"
+#include "QUICStreamManager.h"
+#include "QUICFlowController.h"
+#include "QUICCongestionController.h"
+#include "QUICLossDetector.h"
+#include "QUICEvents.h"
+
+const static char *tag = "quic_frame_handler";
+
+//
+// Frame Dispatcher
+//
+QUICFrameDispatcher::QUICFrameDispatcher(const std::shared_ptr<QUICConnectionManager> cmgr,
+                                         const std::shared_ptr<QUICStreamManager> smgr,
+                                         const std::shared_ptr<QUICFlowController> fctlr,
+                                         const std::shared_ptr<QUICCongestionController> cctlr,
+                                         const std::shared_ptr<QUICLossDetector> ld)
+{
+  connectionManager    = cmgr;
+  streamManager        = smgr;
+  flowController       = fctlr;
+  congestionController = cctlr;
+  lossDetector         = ld;
+}
+
+bool
+QUICFrameDispatcher::receive_frames(const uint8_t *payload, uint16_t size)
+{
+  std::shared_ptr<const QUICFrame> frame(nullptr);
+  uint16_t cursor      = 0;
+  bool should_send_ack = false;
+
+  while (cursor < size) {
+    frame = this->_frame_factory.fast_create(payload + cursor, size - cursor);
+    if (frame == nullptr) {
+      Debug(tag, "Failed to create a frame (%u bytes skipped)", size - cursor);
+      break;
+    }
+    cursor += frame->size();
+
+    // TODO: check debug build
+    if (frame->type() != QUICFrameType::PADDING) {
+      Debug(tag, "frame type %d, size %zu", frame->type(), frame->size());
+    }
+
+    // FIXME We should probably use a mapping table. All the objects has the common interface (QUICFrameHandler).
+    switch (frame->type()) {
+    case QUICFrameType::PADDING: {
+      // NOTE: do nothing
+      break;
+    }
+    case QUICFrameType::RST_STREAM: {
+      streamManager->handle_frame(frame);
+      should_send_ack = true;
+      break;
+    }
+    case QUICFrameType::CONNECTION_CLOSE: {
+      connectionManager->handle_frame(frame);
+      should_send_ack = true;
+      break;
+    }
+    case QUICFrameType::GOAWAY: {
+      connectionManager->handle_frame(frame);
+      should_send_ack = true;
+      break;
+    }
+    case QUICFrameType::MAX_DATA: {
+      flowController->handle_frame(frame);
+      should_send_ack = true;
+      break;
+    }
+    case QUICFrameType::MAX_STREAM_DATA: {
+      flowController->handle_frame(frame);
+      should_send_ack = true;
+      break;
+    }
+    case QUICFrameType::MAX_STREAM_ID: {
+      should_send_ack = true;
+      break;
+    }
+    case QUICFrameType::PING: {
+      connectionManager->handle_frame(frame);
+      should_send_ack = true;
+      break;
+    }
+    case QUICFrameType::BLOCKED: {
+      flowController->handle_frame(frame);
+      should_send_ack = true;
+      break;
+    }
+    case QUICFrameType::STREAM_BLOCKED: {
+      should_send_ack = true;
+      break;
+    }
+    case QUICFrameType::STREAM_ID_NEEDED: {
+      should_send_ack = true;
+      break;
+    }
+    case QUICFrameType::NEW_CONNECTION_ID: {
+      should_send_ack = true;
+      break;
+    }
+    case QUICFrameType::ACK: {
+      congestionController->handle_frame(frame);
+      this->lossDetector->handle_frame(frame);
+      break;
+    }
+    case QUICFrameType::STREAM: {
+      streamManager->handle_frame(frame);
+      flowController->handle_frame(frame);
+      congestionController->handle_frame(frame);
+      should_send_ack = true;
+      break;
+    }
+    default:
+      // Unknown frame
+      Debug(tag, "Unknown frame type: %02x", frame->type());
+      ink_assert(false);
+
+      break;
+    }
+  }
+  return should_send_ack;
+}
diff --git a/iocore/net/quic/QUICFrameDispatcher.h b/iocore/net/quic/QUICFrameDispatcher.h
new file mode 100644
index 0000000..c16316b
--- /dev/null
+++ b/iocore/net/quic/QUICFrameDispatcher.h
@@ -0,0 +1,53 @@
+/** @file
+ *
+ *  A brief file description
+ *
+ *  @section license License
+ *
+ *  Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+#pragma once
+
+#include "QUICFrame.h"
+
+class QUICStreamManager;
+class QUICFlowController;
+class QUICConnectionManager;
+class QUICCongestionController;
+class QUICLossDetector;
+
+class QUICFrameDispatcher
+{
+public:
+  QUICFrameDispatcher(const std::shared_ptr<QUICConnectionManager> cmgr, const std::shared_ptr<QUICStreamManager> smgr,
+                      const std::shared_ptr<QUICFlowController> fctlr, const std::shared_ptr<QUICCongestionController> cctlr,
+                      const std::shared_ptr<QUICLossDetector> ld);
+  /*
+   * Returns true if ACK frame should be sent
+   */
+  bool receive_frames(const uint8_t *payload, uint16_t size);
+
+  std::shared_ptr<QUICConnectionManager> connectionManager       = nullptr;
+  std::shared_ptr<QUICStreamManager> streamManager               = nullptr;
+  std::shared_ptr<QUICFlowController> flowController             = nullptr;
+  std::shared_ptr<QUICCongestionController> congestionController = nullptr;
+  std::shared_ptr<QUICLossDetector> lossDetector                 = nullptr;
+
+private:
+  QUICFrameFactory _frame_factory;
+};
diff --git a/iocore/net/quic/QUICFrameHandler.h b/iocore/net/quic/QUICFrameHandler.h
new file mode 100644
index 0000000..6a99bac
--- /dev/null
+++ b/iocore/net/quic/QUICFrameHandler.h
@@ -0,0 +1,31 @@
+/** @file
+ *
+ *  A brief file description
+ *
+ *  @section license License
+ *
+ *  Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+#pragma once
+
+#include <QUICFrame.h>
+
+class QUICFrameHandler
+{
+  virtual void handle_frame(std::shared_ptr<const QUICFrame> frame) = 0;
+};
diff --git a/iocore/net/quic/QUICFrameTransmitter.h b/iocore/net/quic/QUICFrameTransmitter.h
new file mode 100644
index 0000000..8612daf
--- /dev/null
+++ b/iocore/net/quic/QUICFrameTransmitter.h
@@ -0,0 +1,37 @@
+/** @file
+ *
+ *  A brief file description
+ *
+ *  @section license License
+ *
+ *  Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+#pragma once
+
+#include "QUICFrame.h"
+
+class QUICFrameTransmitter
+{
+public:
+  /*
+   * Enqueue a frame for transmission
+   *
+   * This sends QUIC_PACKET_WRITE_READY event.
+   */
+  virtual void transmit_frame(std::unique_ptr<QUICFrame, QUICFrameDeleterFunc> frame) = 0;
+};
diff --git a/iocore/net/quic/QUICHandshake.cc b/iocore/net/quic/QUICHandshake.cc
new file mode 100644
index 0000000..e4221ae
--- /dev/null
+++ b/iocore/net/quic/QUICHandshake.cc
@@ -0,0 +1,249 @@
+/** @file
+ *
+ *  A brief file description
+ *
+ *  @section license License
+ *
+ *  Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+#include "QUICHandshake.h"
+
+#include "P_Net.h"
+#include "P_QUICNetVConnection.h"
+#include "QUICApplication.h"
+
+#define I_WANNA_DUMP_THIS_BUF(buf, len)                                                                                           \
+  {                                                                                                                               \
+    int i, j;                                                                                                                     \
+    fprintf(stderr, "len=%lld\n", len);                                                                                           \
+    for (i = 0; i < len / 8; i++) {                                                                                               \
+      fprintf(stderr, "%02x %02x %02x %02x %02x %02x %02x %02x ", buf[i * 8 + 0], buf[i * 8 + 1], buf[i * 8 + 2], buf[i * 8 + 3], \
+              buf[i * 8 + 4], buf[i * 8 + 5], buf[i * 8 + 6], buf[i * 8 + 7]);                                                    \
+      if ((i + 1) % 4 == 0 || (len % 8 == 0 && i + 1 == len / 8)) {                                                               \
+        fprintf(stderr, "\n");                                                                                                    \
+      }                                                                                                                           \
+    }                                                                                                                             \
+    if (len % 8 != 0) {                                                                                                           \
+      fprintf(stderr, "%0x", buf[i * 8 + 0]);                                                                                     \
+      for (j = 1; j < len % 8; j++) {                                                                                             \
+        fprintf(stderr, " %02x", buf[i * 8 + j]);                                                                                 \
+      }                                                                                                                           \
+      fprintf(stderr, "\n");                                                                                                      \
+    }                                                                                                                             \
+  }
+
+const static char *tag                    = "quic_handshake";
+const static int UDP_MAXIMUM_PAYLOAD_SIZE = 65527;
+// TODO: fix size
+const static int MAX_HANDSHAKE_MSG_LEN = 65527;
+
+QUICHandshake::QUICHandshake(ProxyMutex *m, QUICNetVConnection *vc) : QUICApplication(m, vc)
+{
+  SET_HANDLER(&QUICHandshake::state_read_client_hello);
+}
+
+bool
+QUICHandshake::is_completed()
+{
+  QUICCrypto *crypto = this->_client_vc->get_crypto();
+  return crypto->is_handshake_finished();
+}
+
+const uint8_t *
+QUICHandshake::negotiated_application_name()
+{
+  // TODO Use the protocol name negotiated on ALPN
+  return reinterpret_cast<const uint8_t *>("hq");
+}
+
+int
+QUICHandshake::state_read_client_hello(int event, Event *data)
+{
+  QUICError error;
+  switch (event) {
+  case VC_EVENT_READ_READY:
+  case VC_EVENT_READ_COMPLETE: {
+    error = this->_process_client_hello();
+    break;
+  }
+  default:
+    Debug(tag, "event: %d", event);
+    break;
+  }
+
+  if (error.cls != QUICErrorClass::NONE) {
+    this->_client_vc->close(error);
+    Debug(tag, "Enter state_closed");
+    SET_HANDLER(&QUICHandshake::state_closed);
+  }
+
+  return EVENT_CONT;
+}
+
+int
+QUICHandshake::state_read_client_finished(int event, Event *data)
+{
+  QUICError error;
+  switch (event) {
+  case VC_EVENT_READ_READY:
+  case VC_EVENT_READ_COMPLETE: {
+    error = this->_process_client_finished();
+    break;
+  }
+  default:
+    Debug(tag, "event: %d", event);
+    break;
+  }
+
+  if (error.cls != QUICErrorClass::NONE) {
+    this->_client_vc->close(error);
+    Debug(tag, "Enter state_closed");
+    SET_HANDLER(&QUICHandshake::state_closed);
+  }
+
+  return EVENT_CONT;
+}
+
+int
+QUICHandshake::state_address_validation(int event, void *data)
+{
+  // TODO Address validation should be implemented for the 2nd implementation draft
+  return EVENT_CONT;
+}
+
+int
+QUICHandshake::state_complete(int event, void *data)
+{
+  Debug(tag, "event: %d", event);
+  Debug(tag, "Got an event on complete state. Ignoring it for now.");
+
+  return EVENT_CONT;
+}
+
+int
+QUICHandshake::state_closed(int event, void *data)
+{
+  return EVENT_CONT;
+}
+
+QUICError
+QUICHandshake::_process_client_hello()
+{
+  QUICStreamIO *stream_io = this->_find_stream_io(STREAM_ID_FOR_HANDSHAKE);
+
+  // Complete message should fit in a packet and be able to read
+  uint8_t msg[UDP_MAXIMUM_PAYLOAD_SIZE] = {0};
+  int64_t msg_len                       = stream_io->read_avail();
+  stream_io->read(msg, msg_len);
+
+  if (msg_len <= 0) {
+    Debug(tag, "No message");
+    return QUICError(QUICErrorClass::NONE);
+  }
+
+  // ----- DEBUG ----->
+  I_WANNA_DUMP_THIS_BUF(msg, msg_len);
+  // <----- DEBUG -----
+
+  QUICCrypto *crypto = this->_client_vc->get_crypto();
+
+  uint8_t server_hello[MAX_HANDSHAKE_MSG_LEN] = {0};
+  size_t server_hello_len                     = 0;
+  bool result                                 = false;
+  result = crypto->handshake(server_hello, server_hello_len, MAX_HANDSHAKE_MSG_LEN, msg, msg_len);
+
+  if (result) {
+    // ----- DEBUG ----->
+    I_WANNA_DUMP_THIS_BUF(server_hello, static_cast<int64_t>(server_hello_len));
+    // <----- DEBUG -----
+
+    Debug(tag, "Enter state_read_client_finished");
+    SET_HANDLER(&QUICHandshake::state_read_client_finished);
+
+    stream_io->write(server_hello, server_hello_len);
+    stream_io->write_reenable();
+    stream_io->read_reenable();
+
+    return QUICError(QUICErrorClass::NONE);
+  } else {
+    return QUICError(QUICErrorClass::CRYPTOGRAPHIC, QUICErrorCode::TLS_HANDSHAKE_FAILED);
+  }
+}
+
+QUICError
+QUICHandshake::_process_client_finished()
+{
+  QUICStreamIO *stream_io = this->_find_stream_io(STREAM_ID_FOR_HANDSHAKE);
+
+  // Complete message should fit in a packet and be able to read
+  uint8_t msg[UDP_MAXIMUM_PAYLOAD_SIZE] = {0};
+  int64_t msg_len                       = stream_io->read_avail();
+  stream_io->read(msg, msg_len);
+
+  if (msg_len <= 0) {
+    Debug(tag, "No message");
+    return QUICError(QUICErrorClass::NONE);
+  }
+
+  // ----- DEBUG ----->
+  I_WANNA_DUMP_THIS_BUF(msg, msg_len);
+  // <----- DEBUG -----
+
+  QUICCrypto *crypto = this->_client_vc->get_crypto();
+
+  uint8_t out[MAX_HANDSHAKE_MSG_LEN] = {0};
+  size_t out_len                     = 0;
+  bool result                        = false;
+  result                             = crypto->handshake(out, out_len, MAX_HANDSHAKE_MSG_LEN, msg, msg_len);
+
+  if (result) {
+    // ----- DEBUG ----->
+    I_WANNA_DUMP_THIS_BUF(out, static_cast<int64_t>(out_len));
+    // <----- DEBUG -----
+
+    ink_assert(this->is_completed());
+    Debug(tag, "Handshake is completed");
+
+    Debug(tag, "Enter state_complete");
+    SET_HANDLER(&QUICHandshake::state_complete);
+    _process_handshake_complete();
+
+    stream_io->write(out, out_len);
+    stream_io->write_reenable();
+    stream_io->read_reenable();
+
+    return QUICError(QUICErrorClass::NONE);
+  } else {
+    return QUICError(QUICErrorClass::CRYPTOGRAPHIC, QUICErrorCode::TLS_HANDSHAKE_FAILED);
+  }
+}
+
+QUICError
+QUICHandshake::_process_handshake_complete()
+{
+  QUICCrypto *crypto = this->_client_vc->get_crypto();
+  int r              = crypto->setup_session();
+
+  if (r) {
+    Debug(tag, "Keying Materials are exported");
+  } else {
+    Debug(tag, "Failed to export Keying Materials");
+  }
+
+  return QUICError(QUICErrorClass::NONE);
+}
diff --git a/iocore/net/quic/QUICHandshake.h b/iocore/net/quic/QUICHandshake.h
new file mode 100644
index 0000000..28854ec
--- /dev/null
+++ b/iocore/net/quic/QUICHandshake.h
@@ -0,0 +1,68 @@
+/** @file
+ *
+ *  A brief file description
+ *
+ *  @section license License
+ *
+ *  Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+#ifndef __QUIC_HANDSHAKE__
+#define __QUIC_HANDSHAKE__
+
+#include "I_VConnection.h"
+#include "QUICApplication.h"
+
+class QUICNetVConnection;
+
+/**
+ * @class QUICHandshake
+ * @brief Do handshake as a QUIC application
+ * @detail
+ *
+ * state_read_client_hello()
+ *  |  | _process_client_hello()
+ *  |  v
+ *  | state_read_client_finished()
+ *  |  |  | _process_client_finished()
+ *  | /  | _process_handshake_complete()
+ *  |     v
+ *  |    state_complete()
+ *  |
+ *  v
+ * state_closed()
+ */
+class QUICHandshake : public QUICApplication
+{
+public:
+  QUICHandshake(ProxyMutex *m, QUICNetVConnection *vc);
+
+  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();
+  const uint8_t *negotiated_application_name();
+
+private:
+  QUICError _process_client_hello();
+  QUICError _process_client_finished();
+  QUICError _process_handshake_complete();
+};
+
+#endif // __QUIC_HANDSHAKE__
diff --git a/iocore/net/quic/QUICLossDetector.cc b/iocore/net/quic/QUICLossDetector.cc
new file mode 100644
index 0000000..5ffc58f
--- /dev/null
+++ b/iocore/net/quic/QUICLossDetector.cc
@@ -0,0 +1,325 @@
+/** @file
+ *
+ *  A brief file description
+ *
+ *  @section license License
+ *
+ *  Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+#include "QUICLossDetector.h"
+#include "QUICEvents.h"
+#include "ts/ink_assert.h"
+
+QUICLossDetector::QUICLossDetector(QUICPacketTransmitter *transmitter) : _transmitter(transmitter)
+{
+  this->mutex = new_ProxyMutex();
+
+  this->_loss_detection_alarm = nullptr;
+  this->_handshake_count      = 0;
+  this->_tlp_count            = 0;
+  this->_rto_count            = 0;
+  if (this->_time_loss_detection) {
+    this->_reordering_threshold     = INFINITY;
+    this->_time_reordering_fraction = this->_TIME_REORDERING_FRACTION;
+  } else {
+    this->_reordering_threshold     = this->_REORDERING_THRESHOLD;
+    this->_time_reordering_fraction = INFINITY;
+  }
+  this->_loss_time                = 0;
+  this->_smoothed_rtt             = 0;
+  this->_rttvar                   = 0;
+  this->_largest_sent_before_rto  = 0;
+  this->_time_of_last_sent_packet = 0;
+
+  SET_HANDLER(&QUICLossDetector::event_handler);
+}
+
+int
+QUICLossDetector::event_handler(int event, Event *edata)
+{
+  switch (event) {
+  case EVENT_INTERVAL:
+    this->_on_loss_detection_alarm();
+    break;
+  default:
+    break;
+  }
+  return EVENT_CONT;
+}
+
+void
+QUICLossDetector::handle_frame(std::shared_ptr<const QUICFrame> frame)
+{
+  switch (frame->type()) {
+  case QUICFrameType::ACK:
+    this->_on_ack_received(std::dynamic_pointer_cast<const QUICAckFrame>(frame));
+    break;
+  default:
+    Debug("quic_loss_detector", "Unexpected frame type: %02x", frame->type());
+    ink_assert(false);
+    break;
+  }
+}
+
+void
+QUICLossDetector::_detect_lost_packets(QUICPacketNumber largest_acked_packet_number)
+{
+  SCOPED_MUTEX_LOCK(lock, this->mutex, this_ethread());
+  this->_loss_time = 0;
+  std::set<QUICPacketNumber> lost_packets;
+  uint32_t delay_until_lost = UINT32_MAX;
+
+  if (this->_time_reordering_fraction != INFINITY) {
+    delay_until_lost = (1 + this->_time_reordering_fraction) * max(this->_latest_rtt, this->_smoothed_rtt);
+  } else if (largest_acked_packet_number == this->_largest_sent_packet) {
+    // Early retransmit alarm.
+    delay_until_lost = 9 / 8 * max(this->_latest_rtt, this->_smoothed_rtt);
+  }
+  for (auto &unacked : this->_sent_packets) {
+    if (unacked.first >= largest_acked_packet_number) {
+      break;
+    }
+    ink_hrtime time_since_sent = Thread::get_hrtime() - unacked.second->time;
+    uint64_t packet_delta      = largest_acked_packet_number - unacked.second->packet_number;
+    if (time_since_sent > delay_until_lost) {
+      lost_packets.insert(unacked.first);
+    } else if (packet_delta > this->_reordering_threshold) {
+      lost_packets.insert(unacked.first);
+    } else if (this->_loss_time == 0 && delay_until_lost != INFINITY) {
+      this->_loss_time = Thread::get_hrtime() + delay_until_lost - time_since_sent;
+    }
+  }
+
+  // Inform the congestion controller of lost packets and
+  // lets it decide whether to retransmit immediately.
+  if (!lost_packets.empty()) {
+    // TODO cc->on_packets_lost(lost_packets);
+    for (auto packet_number : lost_packets) {
+      this->_decrement_packet_count(packet_number);
+      this->_sent_packets.erase(packet_number);
+    }
+  }
+}
+
+void
+QUICLossDetector::on_packet_sent(std::unique_ptr<const QUICPacket> packet)
+{
+  bool is_handshake   = false;
+  QUICPacketType type = packet->type();
+  if (type != QUICPacketType::ZERO_RTT_PROTECTED || type != QUICPacketType::ONE_RTT_PROTECTED_KEY_PHASE_0 ||
+      type != QUICPacketType::ONE_RTT_PROTECTED_KEY_PHASE_1) {
+    is_handshake = true;
+  }
+  return this->_on_packet_sent(packet->packet_number(), packet->is_retransmittable(), is_handshake, packet->size(),
+                               std::move(packet));
+}
+
+void
+QUICLossDetector::_on_packet_sent(QUICPacketNumber packet_number, bool is_retransmittable, bool is_handshake, size_t sent_bytes,
+                                  std::unique_ptr<const QUICPacket> packet)
+{
+  SCOPED_MUTEX_LOCK(lock, this->mutex, this_ethread());
+  this->_largest_sent_packet      = packet_number;
+  this->_time_of_last_sent_packet = Thread::get_hrtime();
+  // FIXME Should we really keep actual packet object?
+
+  std::unique_ptr<PacketInfo> packet_info(new PacketInfo(
+    {packet_number, this->_time_of_last_sent_packet, is_retransmittable, is_handshake, sent_bytes, std::move(packet)}));
+  this->_sent_packets.insert(std::pair<QUICPacketNumber, std::unique_ptr<PacketInfo>>(packet_number, std::move(packet_info)));
+  if (is_handshake) {
+    ++this->_handshake_outstanding;
+  }
+  if (is_retransmittable) {
+    ++this->_retransmittable_outstanding;
+    this->_set_loss_detection_alarm();
+  }
+}
+
+void
+QUICLossDetector::_on_ack_received(std::shared_ptr<const QUICAckFrame> ack_frame)
+{
+  SCOPED_MUTEX_LOCK(lock, this->mutex, this_ethread());
+  // If the largest acked is newly acked, update the RTT.
+  auto pi = this->_sent_packets.find(ack_frame->largest_acknowledged());
+  if (pi != this->_sent_packets.end()) {
+    this->_latest_rtt = Thread::get_hrtime() - pi->second->time;
+    if (this->_latest_rtt > ack_frame->ack_delay()) {
+      this->_latest_rtt -= ack_frame->ack_delay();
+    }
+    this->_update_rtt(this->_latest_rtt);
+  }
+  // Find all newly acked packets.
+  for (auto acked_packet_number : this->_determine_newly_acked_packets(*ack_frame)) {
+    this->_on_packet_acked(acked_packet_number);
+  }
+
+  this->_detect_lost_packets(ack_frame->largest_acknowledged());
+  Debug("quic_loss_detector", "Unacked handshake pkt %u, retransmittable pkt %u", this->_handshake_outstanding,
+        this->_retransmittable_outstanding);
+  this->_set_loss_detection_alarm();
+}
+
+void
+QUICLossDetector::_on_packet_acked(QUICPacketNumber acked_packet_number)
+{
+  SCOPED_MUTEX_LOCK(lock, this->mutex, this_ethread());
+  Debug("quic_loss_detector", "Packet number %llu has been acked", acked_packet_number);
+  this->_largest_acked_packet = acked_packet_number;
+  // If a packet sent prior to RTO was acked, then the RTO
+  // was spurious.  Otherwise, inform congestion control.
+  if (this->_rto_count > 0 && acked_packet_number > this->_largest_sent_before_rto) {
+    // TODO cc->on_retransmission_timeout_verified();
+  }
+  this->_handshake_count = 0;
+  this->_tlp_count       = 0;
+  this->_rto_count       = 0;
+  this->_decrement_packet_count(acked_packet_number);
+  this->_sent_packets.erase(acked_packet_number);
+}
+
+void
+QUICLossDetector::_decrement_packet_count(QUICPacketNumber packet_number)
+{
+  SCOPED_MUTEX_LOCK(lock, this->mutex, this_ethread());
+  auto ite = this->_sent_packets.find(packet_number);
+  if (ite != this->_sent_packets.end()) {
+    if (ite->second->handshake) {
+      --this->_handshake_outstanding;
+    }
+    if (ite->second->retransmittable) {
+      --this->_retransmittable_outstanding;
+    }
+  }
+}
+
+void
+QUICLossDetector::_on_loss_detection_alarm()
+{
+  if (this->_handshake_outstanding) {
+    // Handshake retransmission alarm.
+    this->_retransmit_handshake_packets();
+    this->_handshake_count++;
+  } else if (this->_loss_time != 0) {
+    // Early retransmit or Time Loss Detection
+    this->_detect_lost_packets(this->_largest_acked_packet);
+  } else if (this->_tlp_count < this->_MAX_TLPS) {
+    // Tail Loss Probe.
+    // eventProcessor.schedule_imm(this->_handler, ET_CALL, QUIC_EVENT_RETRANSMIT_ONE_PACKET);
+    this->_tlp_count++;
+  } else {
+    // RTO.
+    if (this->_rto_count == 0) {
+      this->_largest_sent_before_rto = this->_largest_sent_packet;
+    }
+    // eventProcessor.schedule_imm(this->_handler, ET_CALL, QUIC_EVENT_RETRANSMIT_TWO_PACKET);
+    this->_rto_count++;
+  }
+  Debug("quic_loss_detector", "Unacked handshake pkt %u, retransmittable pkt %u", this->_handshake_outstanding,
+        this->_retransmittable_outstanding);
+  this->_set_loss_detection_alarm();
+}
+
+void
+QUICLossDetector::_update_rtt(uint32_t latest_rtt)
+{
+  // Based on {{RFC6298}}.
+  if (this->_smoothed_rtt == 0) {
+    this->_smoothed_rtt = latest_rtt;
+    this->_rttvar       = latest_rtt / 2;
+  } else {
+    this->_rttvar       = 3 / 4 * this->_rttvar + 1 / 4 * (this->_smoothed_rtt - latest_rtt);
+    this->_smoothed_rtt = 7 / 8 * this->_smoothed_rtt + 1 / 8 * latest_rtt;
+  }
+}
+
+void
+QUICLossDetector::_set_loss_detection_alarm()
+{
+  uint32_t alarm_duration;
+  if (!this->_retransmittable_outstanding && this->_loss_detection_alarm) {
+    this->_loss_detection_alarm->cancel();
+    this->_loss_detection_alarm = nullptr;
+    Debug("quic_loss_detection", "Loss detection alarm has been unset");
+    return;
+  }
+  if (this->_handshake_outstanding) {
+    // Handshake retransmission alarm.
+    if (this->_smoothed_rtt == 0) {
+      alarm_duration = 2 * this->_DEFAULT_INITIAL_RTT;
+    } else {
+      alarm_duration = 2 * this->_smoothed_rtt;
+    }
+    alarm_duration = max(alarm_duration, this->_MIN_TLP_TIMEOUT);
+    alarm_duration = alarm_duration * (2 ^ this->_handshake_count);
+    Debug("quic_loss_detection", "Handshake retransmission alarm will be set");
+  } else if (this->_loss_time != 0) {
+    // Early retransmit timer or time loss detection.
+    alarm_duration = this->_loss_time - Thread::get_hrtime();
+    Debug("quic_loss_detection", "Early retransmit timer or time loss detection will be set");
+  } else if (this->_tlp_count < this->_MAX_TLPS) {
+    // Tail Loss Probe
+    if (this->_retransmittable_outstanding) {
+      alarm_duration = 1.5 * this->_smoothed_rtt + this->_DELAYED_ACK_TIMEOUT;
+    } else {
+      alarm_duration = this->_MIN_TLP_TIMEOUT;
+    }
+    alarm_duration = max(alarm_duration, 2 * this->_smoothed_rtt);
+    Debug("quic_loss_detection", "TLP alarm will be set");
+  } else {
+    // RTO alarm
+    alarm_duration = this->_smoothed_rtt + 4 * this->_rttvar;
+    alarm_duration = max(alarm_duration, this->_MIN_RTO_TIMEOUT);
+    alarm_duration = alarm_duration * (2 ^ this->_rto_count);
+    Debug("quic_loss_detection", "RTO alarm will be set");
+  }
+
+  if (this->_loss_detection_alarm) {
+    this->_loss_detection_alarm->cancel();
+  }
+  this->_loss_detection_alarm = eventProcessor.schedule_in(this, alarm_duration);
+  Debug("quic_loss_detection", "Loss detection alarm has been set to %u", alarm_duration);
+}
+
+std::set<QUICPacketNumber>
+QUICLossDetector::_determine_newly_acked_packets(const QUICAckFrame &ack_frame)
+{
+  std::set<QUICPacketNumber> packets;
+  QUICPacketNumber x = ack_frame.largest_acknowledged();
+  packets.insert(x);
+  for (auto &&block : *(ack_frame.ack_block_section())) {
+    for (int i = 0; i < block.gap(); ++i) {
+      packets.insert(++x);
+    }
+    x += block.length();
+  }
+
+  return packets;
+}
+
+void
+QUICLossDetector::_retransmit_handshake_packets()
+{
+  SCOPED_MUTEX_LOCK(transmitter_lock, this->_transmitter->get_transmitter_mutex().get(), this_ethread());
+  SCOPED_MUTEX_LOCK(lock, this->mutex, this_ethread());
+  for (auto &info : this->_sent_packets) {
+    if (!info.second->handshake) {
+      break;
+    }
+    this->_transmitter->retransmit_packet(*info.second->packet);
+  }
+}
diff --git a/iocore/net/quic/QUICLossDetector.h b/iocore/net/quic/QUICLossDetector.h
new file mode 100644
index 0000000..0a2b71c
--- /dev/null
+++ b/iocore/net/quic/QUICLossDetector.h
@@ -0,0 +1,108 @@
+/** @file
+ *
+ *  A brief file description
+ *
+ *  @section license License
+ *
+ *  Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+#pragma once
+
+// TODO Using STL Map because ts/Map lacks remove method
+#include <map>
+#include <set>
+
+#include "../../eventsystem/I_EventSystem.h"
+#include "../../eventsystem/I_Action.h"
+#include "ts/ink_hrtime.h"
+#include "I_VConnection.h"
+#include "P_Net.h"
+#include "QUICTypes.h"
+#include "QUICPacket.h"
+#include "QUICFrame.h"
+#include "QUICFrameHandler.h"
+#include "QUICPacketTransmitter.h"
+
+class QUICLossDetector : public Continuation, public QUICFrameHandler
+{
+public:
+  QUICLossDetector(QUICPacketTransmitter *transmitter);
+
+  int event_handler(int event, Event *edata);
+
+  virtual void handle_frame(std::shared_ptr<const QUICFrame>) override;
+  void on_packet_sent(std::unique_ptr<const QUICPacket> packet);
+
+private:
+  struct PacketInfo {
+    QUICPacketNumber packet_number;
+    ink_hrtime time;
+    bool retransmittable;
+    bool handshake;
+    size_t bytes;
+    std::unique_ptr<const QUICPacket> packet;
+  };
+
+  bool _time_loss_detection = false;
+
+  // TODO QUICCongestionController *cc = nullptr;
+
+  // 3.2.1.  Constants of interest
+  uint32_t _MAX_TLPS               = 2;
+  uint32_t _REORDERING_THRESHOLD   = 3;
+  double _TIME_REORDERING_FRACTION = 1 / 8;
+  uint32_t _MIN_TLP_TIMEOUT        = HRTIME_MSECONDS(10);
+  uint32_t _MIN_RTO_TIMEOUT        = HRTIME_MSECONDS(200);
+  uint32_t _DELAYED_ACK_TIMEOUT    = HRTIME_MSECONDS(25);
+  uint32_t _DEFAULT_INITIAL_RTT    = HRTIME_MSECONDS(100);
+
+  // 3.2.2.  Variables of interest
+  Action *_loss_detection_alarm;
+  uint32_t _handshake_count = 0;
+  uint32_t _tlp_count       = 0;
+  uint32_t _rto_count       = 0;
+  uint32_t _largest_sent_before_rto;
+  uint32_t _largest_sent_packet;
+  uint32_t _largest_acked_packet;
+  uint32_t _time_of_last_sent_packet;
+  uint32_t _latest_rtt;
+  uint32_t _smoothed_rtt;
+  uint32_t _rttvar;
+  uint32_t _reordering_threshold;
+  double _time_reordering_fraction;
+  uint32_t _loss_time;
+  std::map<QUICPacketNumber, std::unique_ptr<PacketInfo>> _sent_packets;
+
+  uint32_t _handshake_outstanding       = 0;
+  uint32_t _retransmittable_outstanding = 0;
+  void _decrement_packet_count(QUICPacketNumber packet_number);
+
+  void _on_packet_sent(QUICPacketNumber packet_number, bool is_retransmittable, bool is_handshake, size_t sent_bytes,
+                       std::unique_ptr<const QUICPacket> packet);
+  void _on_ack_received(std::shared_ptr<const QUICAckFrame> ack_frame);
+  void _on_packet_acked(QUICPacketNumber acked_packet_number);
+  void _update_rtt(uint32_t latest_rtt);
+  void _detect_lost_packets(QUICPacketNumber largest_acked);
+  void _set_loss_detection_alarm();
+  void _on_loss_detection_alarm();
+  std::set<QUICPacketNumber> _determine_newly_acked_packets(const QUICAckFrame &ack_frame);
+
+  void _retransmit_handshake_packets();
+
+  QUICPacketTransmitter *_transmitter = nullptr;
+};
diff --git a/iocore/net/quic/QUICPacket.cc b/iocore/net/quic/QUICPacket.cc
new file mode 100644
index 0000000..135dd85
--- /dev/null
+++ b/iocore/net/quic/QUICPacket.cc
@@ -0,0 +1,673 @@
+/** @file
+ *
+ *  A brief file description
+ *
+ *  @section license License
+ *
+ *  Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+#include <ts/ink_assert.h>
+#include "QUICPacket.h"
+
+static const int OFFSET_CONNECTION_ID = 1;
+static const int OFFSET_PACKET_NUMBER = 9;
+static const int OFFSET_VERSION       = 13;
+static const int OFFSET_PAYLOAD       = 17;
+static const int LONGHEADER_LENGTH    = 17;
+static const int FNV1A_HASH_LEN       = 8;
+
+ats_unique_buf
+ats_unique_malloc(size_t size)
+{
+  return ats_unique_buf(reinterpret_cast<uint8_t *>(ats_malloc(size)), [](void *p) { ats_free(p); });
+}
+
+const uint8_t *
+QUICPacketHeader::buf()
+{
+  return this->_buf;
+}
+
+QUICPacketHeader *
+QUICPacketHeader::load(const uint8_t *buf, size_t len)
+{
+  if (QUICTypeUtil::hasLongHeader(buf)) {
+    return new QUICPacketLongHeader(buf, len);
+  } else {
+    return new QUICPacketShortHeader(buf, len);
+  }
+}
+
+QUICPacketHeader *
+QUICPacketHeader::build(QUICPacketType type, QUICConnectionId connection_id, QUICPacketNumber packet_number, QUICVersion version,
+                        ats_unique_buf payload, size_t len)
+{
+  return new QUICPacketLongHeader(type, connection_id, packet_number, version, std::move(payload), len);
+}
+
+QUICPacketHeader *
+QUICPacketHeader::build(QUICPacketType type, QUICPacketNumber packet_number, ats_unique_buf payload, size_t len)
+{
+  return new QUICPacketShortHeader(type, packet_number, std::move(payload), len);
+}
+
+QUICPacketHeader *
+QUICPacketHeader::build(QUICPacketType type, QUICConnectionId connection_id, QUICPacketNumber packet_number, ats_unique_buf payload,
+                        size_t len)
+{
+  return new QUICPacketShortHeader(type, connection_id, packet_number, std::move(payload), len);
+}
+
+// QUICPacketLongHeader
+
+QUICPacketLongHeader::QUICPacketLongHeader(QUICPacketType type, QUICConnectionId connection_id, QUICPacketNumber packet_number,
+                                           QUICVersion version, ats_unique_buf buf, size_t len)
+{
+  this->_type              = type;
+  this->_has_connection_id = true;
+  this->_connection_id     = connection_id;
+  this->_packet_number     = packet_number;
+  this->_has_version       = true;
+  this->_version           = version;
+  this->_payload           = std::move(buf);
+  this->_payload_len       = len;
+}
+
+QUICPacketType
+QUICPacketLongHeader::type() const
+{
+  if (this->_buf) {
+    int type = this->_buf[0] & 0x7F;
+    if (type < static_cast<int>(QUICPacketType::UNINITIALIZED)) {
+      return static_cast<QUICPacketType>(type);
+    } else {
+      return QUICPacketType::UNINITIALIZED;
+    }
+  } else {
+    return this->_type;
+  }
+}
+
+QUICConnectionId
+QUICPacketLongHeader::connection_id() const
+{
+  if (this->_buf) {
+    return QUICTypeUtil::read_QUICConnectionId(this->_buf + OFFSET_CONNECTION_ID, 8);
+  } else {
+    return this->_connection_id;
+  }
+}
+
+QUICPacketNumber
+QUICPacketLongHeader::packet_number() const
+{
+  if (this->_buf) {
+    return QUICTypeUtil::read_QUICPacketNumber(this->_buf + OFFSET_PACKET_NUMBER, 4);
+  } else {
+    return this->_packet_number;
+  }
+}
+
+bool
+QUICPacketLongHeader::has_version() const
+{
+  return true;
+}
+
+QUICVersion
+QUICPacketLongHeader::version() const
+{
+  if (this->_buf) {
+    return QUICTypeUtil::read_QUICVersion(this->_buf + OFFSET_VERSION);
+  } else {
+    return this->_version;
+  }
+}
+
+bool
+QUICPacketLongHeader::has_connection_id() const
+{
+  return true;
+}
+
+const uint8_t *
+QUICPacketLongHeader::payload() const
+{
+  if (this->_buf) {
+    return this->_buf + OFFSET_PAYLOAD;
+  } else {
+    return this->_payload.get();
+  }
+}
+
+bool
+QUICPacketLongHeader::has_key_phase() const
+{
+  return false;
+}
+
+QUICKeyPhase
+QUICPacketLongHeader::key_phase() const
+{
+  return QUICKeyPhase::PHASE_0;
+}
+
+uint16_t
+QUICPacketLongHeader::length() const
+{
+  return LONGHEADER_LENGTH;
+}
+
+void
+QUICPacketLongHeader::store(uint8_t *buf, size_t *len) const
+{
+  size_t n;
+  *len   = 0;
+  buf[0] = 0x80;
+  buf[0] += static_cast<uint8_t>(this->_type);
+  *len += 1;
+  QUICTypeUtil::write_QUICConnectionId(this->_connection_id, 8, buf + *len, &n);
+  *len += n;
+  QUICTypeUtil::write_QUICPacketNumber(this->_packet_number, 4, buf + *len, &n);
+  *len += n;
+  QUICTypeUtil::write_QUICVersion(this->_version, buf + *len, &n);
+  *len += n;
+}
+
+// QUICPacketShortHeader
+
+QUICPacketShortHeader::QUICPacketShortHeader(QUICPacketType type, QUICPacketNumber packet_number, ats_unique_buf buf, size_t len)
+{
+  this->_type          = type;
+  this->_has_key_phase = true;
+  this->_packet_number = packet_number;
+  this->_payload       = std::move(buf);
+  this->_payload_len   = len;
+
+  if (type == QUICPacketType::ONE_RTT_PROTECTED_KEY_PHASE_0) {
+    this->_key_phase = QUICKeyPhase::PHASE_0;
+  } else if (type == QUICPacketType::ONE_RTT_PROTECTED_KEY_PHASE_1) {
+    this->_key_phase = QUICKeyPhase::PHASE_1;
+  } else {
+    ink_assert(false);
+    this->_key_phase = QUICKeyPhase::PHASE_UNINITIALIZED;
+  }
+
+  if (packet_number <= 0xFF) {
+    this->_packet_number_type = QUICPacketShortHeaderType::ONE;
+  } else if (packet_number <= 0xFFFF) {
+    this->_packet_number_type = QUICPacketShortHeaderType::TWO;
+  } else {
+    this->_packet_number_type = QUICPacketShortHeaderType::THREE;
+  }
+}
+
+QUICPacketShortHeader::QUICPacketShortHeader(QUICPacketType type, QUICConnectionId connection_id, QUICPacketNumber packet_number,
+                                             ats_unique_buf buf, size_t len)
+{
+  this->_type              = type;
+  this->_has_key_phase     = true;
+  this->_has_connection_id = true;
+  this->_connection_id     = connection_id;
+  this->_packet_number     = packet_number;
+  this->_payload           = std::move(buf);
+  this->_payload_len       = len;
+
+  if (type == QUICPacketType::ONE_RTT_PROTECTED_KEY_PHASE_0) {
+    this->_key_phase = QUICKeyPhase::PHASE_0;
+  } else if (type == QUICPacketType::ONE_RTT_PROTECTED_KEY_PHASE_1) {
+    this->_key_phase = QUICKeyPhase::PHASE_1;
+  } else {
+    ink_assert(false);
+    this->_key_phase = QUICKeyPhase::PHASE_UNINITIALIZED;
+  }
+
+  if (packet_number <= 0xFF) {
+    this->_packet_number_type = QUICPacketShortHeaderType::ONE;
+  } else if (packet_number <= 0xFFFF) {
+    this->_packet_number_type = QUICPacketShortHeaderType::TWO;
+  } else {
+    this->_packet_number_type = QUICPacketShortHeaderType::THREE;
+  }
+}
+
+QUICPacketType
+QUICPacketShortHeader::type() const
+{
+  QUICKeyPhase key_phase = this->key_phase();
+
+  switch (key_phase) {
+  case QUICKeyPhase::PHASE_0: {
+    return QUICPacketType::ONE_RTT_PROTECTED_KEY_PHASE_0;
+  }
+  case QUICKeyPhase::PHASE_1: {
+    return QUICPacketType::ONE_RTT_PROTECTED_KEY_PHASE_1;
+  }
+  default:
+    ink_assert(false);
+    return QUICPacketType::UNINITIALIZED;
+  }
+}
+
+QUICConnectionId
+QUICPacketShortHeader::connection_id() const
+{
+  if (this->_buf) {
+    ink_release_assert(this->has_connection_id());
+    return QUICTypeUtil::read_QUICConnectionId(this->_buf + OFFSET_CONNECTION_ID, 8);
+  } else {
+    return _connection_id;
+  }
+}
+
+QUICPacketNumber
+QUICPacketShortHeader::packet_number() const
+{
+  if (this->_buf) {
+    int n      = this->_packet_numberLen();
+    int offset = 1;
+    if (this->has_connection_id()) {
+      offset = OFFSET_PACKET_NUMBER;
+    }
+
+    return QUICTypeUtil::read_QUICPacketNumber(this->_buf + offset, n);
+  } else {
+    return this->_packet_number;
+  }
+}
+
+bool
+QUICPacketShortHeader::has_version() const
+{
+  return false;
+}
+
+QUICVersion
+QUICPacketShortHeader::version() const
+{
+  return 0;
+}
+
+int
+QUICPacketShortHeader::_packet_numberLen() const
+{
+  QUICPacketShortHeaderType type;
+  if (this->_buf) {
+    type = static_cast<QUICPacketShortHeaderType>(this->_buf[0] & 0x1F);
+  } else {
+    type = this->_packet_number_type;
+  }
+
+  switch (type) {
+  case QUICPacketShortHeaderType::ONE:
+    return 1;
+  case QUICPacketShortHeaderType::TWO:
+    return 2;
+  case QUICPacketShortHeaderType::THREE:
+    return 4;
+  default:
+    ink_assert(false);
+    return 0;
+  }
+}
+
+bool
+QUICPacketShortHeader::has_connection_id() const
+{
+  if (this->_buf) {
+    return (this->_buf[0] & 0x40) != 0;
+  } else {
+    return this->_has_connection_id;
+  }
+}
+
+const uint8_t *
+QUICPacketShortHeader::payload() const
+{
+  if (this->_buf) {
+    return this->_buf + length();
+  } else {
+    return this->_payload.get();
+  }
+}
+
+bool
+QUICPacketShortHeader::has_key_phase() const
+{
+  return true;
+}
+
+QUICKeyPhase
+QUICPacketShortHeader::key_phase() const
+{
+  if (this->_buf) {
+    if (this->_buf[0] & 0x20) {
+      return QUICKeyPhase::PHASE_1;
+    } else {
+      return QUICKeyPhase::PHASE_0;
+    }
+  } else {
+    return _key_phase;
+  }
+}
+
+/**
+ * Header Length (doesn't include payload length)
+ */
+uint16_t
+QUICPacketShortHeader::length() const
+{
+  uint16_t len = 1;
+
+  if (this->has_connection_id()) {
+    len += 8;
+  }
+  len += this->_packet_numberLen();
+
+  return len;
+}
+
+void
+QUICPacketShortHeader::store(uint8_t *buf, size_t *len) const
+{
+  size_t n;
+  *len   = 0;
+  buf[0] = 0x00;
+  if (this->_has_connection_id) {
+    buf[0] += 0x40;
+  }
+  if (this->_key_phase == QUICKeyPhase::PHASE_1) {
+    buf[0] += 0x20;
+  }
+  buf[0] += static_cast<uint8_t>(this->_packet_number_type);
+  *len += 1;
+  if (this->_has_connection_id) {
+    QUICTypeUtil::write_QUICConnectionId(this->_connection_id, 8, buf + *len, &n);
+    *len += n;
+  }
+  switch (this->_packet_number_type) {
+  case QUICPacketShortHeaderType::ONE:
+    QUICTypeUtil::write_QUICPacketNumber(this->_packet_number, 1, buf + *len, &n);
+    break;
+  case QUICPacketShortHeaderType::TWO:
+    QUICTypeUtil::write_QUICPacketNumber(this->_packet_number, 2, buf + *len, &n);
+    break;
+  case QUICPacketShortHeaderType::THREE:
+    QUICTypeUtil::write_QUICPacketNumber(this->_packet_number, 4, buf + *len, &n);
+    break;
+  default:
+    ink_release_assert(0);
+  }
+  *len += n;
+}
+
+// QUICPacket
+
+QUICPacket::QUICPacket(IOBufferBlock *block) : _block(block)
+{
+  this->_size   = block->size();
+  this->_header = QUICPacketHeader::load(reinterpret_cast<const uint8_t *>(this->_block->buf()), this->_block->size());
+}
+
+QUICPacket::QUICPacket(QUICPacketType type, QUICConnectionId connection_id, QUICPacketNumber packet_number, QUICVersion version,
+                       ats_unique_buf payload, size_t len, bool retransmittable)
+{
+  this->_header = QUICPacketHeader::build(type, connection_id, 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->_is_retransmittable = retransmittable;
+}
+
+QUICPacket::QUICPacket(QUICPacketType type, QUICPacketNumber packet_number, ats_unique_buf payload, size_t len,
+                       bool retransmittable)
+{
+  this->_header = QUICPacketHeader::build(type, 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;
+}
+
+QUICPacket::QUICPacket(QUICPacketType type, QUICConnectionId connection_id, QUICPacketNumber packet_number, ats_unique_buf payload,
+                       size_t len, bool retransmittable)
+{
+  this->_header = QUICPacketHeader::build(type, connection_id, 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;
+}
+
+/**
+ * When packet is "Short Header Packet", QUICPacket::type() will return 1-RTT Protected (key phase 0)
+ * or 1-RTT Protected (key phase 1)
+ */
+QUICPacketType
+QUICPacket::type() const
+{
+  return this->_header->type();
+}
+
+QUICConnectionId
+QUICPacket::connection_id() const
+{
+  return this->_header->connection_id();
+}
+
+QUICPacketNumber
+QUICPacket::packet_number() const
+{
+  return this->_header->packet_number();
+}
+
+const uint8_t *
+QUICPacket::header() const
+{
+  return this->_header->buf();
+}
+
+const uint8_t *
+QUICPacket::payload() const
+{
+  return this->_header->payload();
+}
+
+QUICVersion
+QUICPacket::version() const
+{
+  return this->_header->version();
+}
+
+bool
+QUICPacket::is_retransmittable() const
+{
+  return this->_is_retransmittable;
+}
+
+uint16_t
+QUICPacket::size() const
+{
+  return this->_size;
+}
+
+uint16_t
+QUICPacket::header_size() const
+{
+  return this->_header->length();
+}
+
+uint16_t
+QUICPacket::payload_size() const
+{
+  // FIXME Protected packets may / may not contain something at the end
+  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();
+  }
+}
+
+QUICKeyPhase
+QUICPacket::key_phase() const
+{
+  return this->_header->key_phase();
+}
+
+void
+QUICPacket::store(uint8_t *buf, size_t *len) const
+{
+  this->_header->store(buf, len);
+  ink_assert(this->size() >= *len);
+
+  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) {
+    memcpy(buf + *len, this->payload(), this->payload_size());
+    *len += this->payload_size();
+
+    fnv1a(buf, *len, buf + *len);
+    *len += FNV1A_HASH_LEN;
+  } else {
+    ink_assert(this->_protected_payload);
+    memcpy(buf + *len, this->_protected_payload.get(), this->_protected_payload_size);
+    *len += this->_protected_payload_size;
+  }
+}
+
+void
+QUICPacket::store_header(uint8_t *buf, size_t *len) const
+{
+  this->_header->store(buf, len);
+}
+
+bool
+QUICPacket::has_valid_fnv1a_hash() const
+{
+  uint8_t hash[FNV1A_HASH_LEN];
+  fnv1a(reinterpret_cast<const uint8_t *>(this->_block->buf()), this->_block->size() - FNV1A_HASH_LEN, hash);
+  return memcmp(this->_block->buf() + this->_block->size() - FNV1A_HASH_LEN, hash, 8) == 0;
+}
+
+void
+QUICPacket::set_protected_payload(ats_unique_buf cipher_txt, size_t cipher_txt_len)
+{
+  this->_protected_payload      = std::move(cipher_txt);
+  this->_protected_payload_size = cipher_txt_len;
+}
+
+QUICPacket *
+QUICPacketFactory::create(IOBufferBlock *block)
+{
+  // TODO: Use custom allocator
+  return new QUICPacket(block);
+}
+
+std::unique_ptr<QUICPacket>
+QUICPacketFactory::create_version_negotiation_packet(const QUICPacket *packet_sent_by_client)
+{
+  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();
+
+  size_t n;
+  for (auto v : QUIC_SUPPORTED_VERSIONS) {
+    QUICTypeUtil::write_QUICVersion(v, p, &n);
+    p += n;
+  }
+
+  // TODO: Use custom allocator
+  return std::unique_ptr<QUICPacket>(new QUICPacket(QUICPacketType::VERSION_NEGOTIATION, packet_sent_by_client->connection_id(),
+                                                    packet_sent_by_client->packet_number(), packet_sent_by_client->version(),
+                                                    std::move(versions), len, false));
+}
+
+std::unique_ptr<QUICPacket>
+QUICPacketFactory::create_server_cleartext_packet(QUICConnectionId connection_id, ats_unique_buf payload, size_t len,
+                                                  bool retransmittable)
+{
+  // TODO: Use custom allocator
+  return std::unique_ptr<QUICPacket>(new QUICPacket(QUICPacketType::SERVER_CLEARTEXT, connection_id,
+                                                    this->_packet_number_generator.next(), this->_version, std::move(payload), len,
+                                                    retransmittable));
+}
+
+std::unique_ptr<QUICPacket>
+QUICPacketFactory::create_server_protected_packet(QUICConnectionId connection_id, ats_unique_buf payload, size_t len,
+                                                  bool retransmittable)
+{
+  // TODO Key phase should be picked up from QUICCrypto, probably
+  // TODO Use class allocator
+  auto packet =
+    std::unique_ptr<QUICPacket>(new QUICPacket(QUICPacketType::ONE_RTT_PROTECTED_KEY_PHASE_0, connection_id,
+                                               this->_packet_number_generator.next(), std::move(payload), len, retransmittable));
+
+  // TODO: use pmtu of UnixNetVConnection
+  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;
+
+  // TODO: do not dump header twice
+  uint8_t ad[17] = {0};
+  size_t ad_len  = 0;
+  packet->store_header(ad, &ad_len);
+
+  if (this->_crypto->encrypt(cipher_txt.get(), cipher_txt_len, max_cipher_txt_len, packet->payload(), packet->payload_size(),
+                             packet->packet_number(), ad, ad_len, packet->key_phase())) {
+    packet->set_protected_payload(std::move(cipher_txt), cipher_txt_len);
+    Debug("quic_packet_factory", "Encrypt Packet, pkt_num: %llu, header_len: %zu payload: %zu", packet->packet_number(), ad_len,
+          cipher_txt_len);
+    return packet;
+  } else {
+    Debug("quic_packet_factory", "CRYPTOGRAPHIC Error");
+    return nullptr;
+  }
+}
+
+std::unique_ptr<QUICPacket>
+QUICPacketFactory::create_client_initial_packet(QUICConnectionId connection_id, QUICVersion version, ats_unique_buf payload,
+                                                size_t len)
+{
+  return std::unique_ptr<QUICPacket>(new QUICPacket(QUICPacketType::CLIENT_INITIAL, connection_id,
+                                                    this->_packet_number_generator.next(), version, std::move(payload), len, true));
+}
+
+void
+QUICPacketFactory::set_version(QUICVersion negotiated_version)
+{
+  ink_assert(this->_version == 0);
+  this->_version = negotiated_version;
+}
+
+void
+QUICPacketFactory::set_crypto_module(QUICCrypto *crypto)
+{
+  this->_crypto = crypto;
+}
+
+QUICPacketNumber
+QUICPacketNumberGenerator::next()
+{
+  return this->_current++;
+}
diff --git a/iocore/net/quic/QUICPacket.h b/iocore/net/quic/QUICPacket.h
new file mode 100644
index 0000000..e719236
--- /dev/null
+++ b/iocore/net/quic/QUICPacket.h
@@ -0,0 +1,191 @@
+/** @file
+ *
+ *  A brief file description
+ *
+ *  @section license License
+ *
+ *  Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+#pragma once
+
+#include <memory>
+#include <cstddef>
+
+#include "ts/List.h"
+#include "I_IOBuffer.h"
+
+#include "QUICTypes.h"
+#include "QUICCrypto.h"
+
+#define QUIC_FIELD_OFFSET_CONNECTION_ID 1
+#define QUIC_FIELD_OFFSET_PACKET_NUMBER 4
+#define QUIC_FIELD_OFFSET_PAYLOAD 5
+
+// TODO: move to lib/ts/ink_memory.h?
+using ats_unique_buf = std::unique_ptr<uint8_t, decltype(&ats_free)>;
+ats_unique_buf ats_unique_malloc(size_t size);
+
+class QUICPacketHeader
+{
+public:
+  QUICPacketHeader(const uint8_t *buf, size_t len) : _buf(buf) {}
+  const uint8_t *buf();
+  virtual QUICPacketType type() const            = 0;
+  virtual QUICConnectionId connection_id() const = 0;
+  virtual QUICPacketNumber packet_number() const = 0;
+  virtual QUICVersion version() const            = 0;
+  virtual const uint8_t *payload() const         = 0;
+  virtual QUICKeyPhase key_phase() const         = 0;
+  virtual uint16_t length() const                = 0;
+  virtual void store(uint8_t *buf, size_t *len) const = 0;
+  static QUICPacketHeader *load(const uint8_t *buf, size_t len);
+  static QUICPacketHeader *build(QUICPacketType type, QUICConnectionId connection_id, QUICPacketNumber packet_number,
+                                 QUICVersion version, ats_unique_buf payload, size_t len);
+  static QUICPacketHeader *build(QUICPacketType type, QUICPacketNumber packet_number, ats_unique_buf payload, size_t len);
+  static QUICPacketHeader *build(QUICPacketType type, QUICConnectionId connection_id, QUICPacketNumber packet_number,
+                                 ats_unique_buf payload, size_t len);
+  virtual bool has_key_phase() const     = 0;
+  virtual bool has_connection_id() const = 0;
+  virtual bool has_version() const       = 0;
+
+protected:
+  QUICPacketHeader(){};
+
+  const uint8_t *_buf             = nullptr;
+  ats_unique_buf _payload         = ats_unique_buf(nullptr, [](void *p) { ats_free(p); });
+  QUICPacketType _type            = QUICPacketType::UNINITIALIZED;
+  QUICKeyPhase _key_phase         = QUICKeyPhase::PHASE_UNINITIALIZED;
+  QUICConnectionId _connection_id = 0;
+  QUICPacketNumber _packet_number = 0;
+  QUICVersion _version            = 0;
+  size_t _payload_len             = 0;
+  bool _has_key_phase             = false;
+  bool _has_connection_id         = false;
+  bool _has_version               = false;
+};
+
+class QUICPacketLongHeader : public QUICPacketHeader
+{
+public:
+  QUICPacketLongHeader(const uint8_t *buf, size_t len) : QUICPacketHeader(buf, len) {}
+  QUICPacketLongHeader(QUICPacketType type, QUICConnectionId connection_id, QUICPacketNumber packet_number, QUICVersion version,
+                       ats_unique_buf buf, size_t len);
+  QUICPacketType type() const;
+  QUICConnectionId connection_id() const;
+  QUICPacketNumber packet_number() const;
+  bool has_version() const;
+  QUICVersion version() const;
+  const uint8_t *payload() const;
+  bool has_connection_id() const;
+  QUICKeyPhase key_phase() const;
+  bool has_key_phase() const;
+  uint16_t length() const;
+  void store(uint8_t *buf, size_t *len) const;
+};
+
+class QUICPacketShortHeader : public QUICPacketHeader
+{
+public:
+  QUICPacketShortHeader(const uint8_t *buf, size_t len) : QUICPacketHeader(buf, len) {}
+  QUICPacketShortHeader(QUICPacketType type, QUICPacketNumber packet_number, ats_unique_buf buf, size_t len);
+  QUICPacketShortHeader(QUICPacketType type, QUICConnectionId connection_id, QUICPacketNumber packet_number, ats_unique_buf buf,
+                        size_t len);
+  QUICPacketType type() const;
+  QUICConnectionId connection_id() const;
+  QUICPacketNumber packet_number() const;
+  bool has_version() const;
+  QUICVersion version() const;
+  const uint8_t *payload() const;
+  bool has_connection_id() const;
+  QUICKeyPhase key_phase() const;
+  bool has_key_phase() const;
+  uint16_t length() const;
+  void store(uint8_t *buf, size_t *len) const;
+
+private:
+  int _packet_numberLen() const;
+  QUICPacketShortHeaderType _packet_number_type = QUICPacketShortHeaderType::UNINITIALIZED;
+};
+
+class QUICPacket
+{
+public:
+  QUICPacket(IOBufferBlock *block);
+  QUICPacket(QUICPacketType type, QUICConnectionId connection_id, QUICPacketNumber packet_number, QUICVersion version,
+             ats_unique_buf payload, size_t len, bool retransmittable);
+  QUICPacket(QUICPacketType type, QUICPacketNumber packet_number, ats_unique_buf payload, size_t len, bool retransmittable);
+  QUICPacket(QUICPacketType type, QUICConnectionId connection_id, QUICPacketNumber packet_number, ats_unique_buf payload,
+             size_t len, bool retransmittabl);
+
+  void set_protected_payload(ats_unique_buf cipher_txt, size_t cipher_txt_len);
+  QUICPacketType type() const;
+  QUICConnectionId connection_id() const;
+  QUICPacketNumber packet_number() const;
+  QUICVersion version() const;
+  const uint8_t *header() const;
+  const uint8_t *payload() const;
+  bool is_retransmittable() const;
+  uint16_t size() const;
+  uint16_t header_size() const;
+  uint16_t payload_size() const;
+  void store(uint8_t *buf, size_t *len) const;
+  void store_header(uint8_t *buf, size_t *len) const;
+  bool has_valid_fnv1a_hash() const;
+  QUICKeyPhase key_phase() const;
+  LINK(QUICPacket, link);
+
+private:
+  IOBufferBlock *_block             = nullptr;
+  ats_unique_buf _protected_payload = ats_unique_buf(nullptr, [](void *p) { ats_free(p); });
+  size_t _size                      = 0;
+  size_t _protected_payload_size    = 0;
+  QUICPacketHeader *_header;
+  bool _is_retransmittable = false;
+};
+
+class QUICPacketNumberGenerator
+{
+public:
+  QUICPacketNumberGenerator(QUICPacketNumber initial_number = 0) : _current(initial_number){};
+  QUICPacketNumber next();
+
+private:
+  QUICPacketNumber _current = 0;
+};
+
+class QUICPacketFactory
+{
+public:
+  static QUICPacket *create(IOBufferBlock *block);
+  std::unique_ptr<QUICPacket> create_version_negotiation_packet(const QUICPacket *packet_sent_by_client);
+  std::unique_ptr<QUICPacket> create_server_cleartext_packet(QUICConnectionId connection_id, ats_unique_buf payload, size_t len,
+                                                             bool retransmittable);
+  std::unique_ptr<QUICPacket> create_server_protected_packet(QUICConnectionId connection_id, ats_unique_buf payload, size_t len,
+                                                             bool retransmittable);
+  std::unique_ptr<QUICPacket> create_client_initial_packet(QUICConnectionId connection_id, QUICVersion version,
+                                                           ats_unique_buf payload, size_t len);
+  void set_version(QUICVersion negotiated_version);
+  void set_crypto_module(QUICCrypto *crypto);
+
+private:
+  QUICVersion _version = 0;
+  QUICCrypto *_crypto  = nullptr;
+  QUICPacketNumberGenerator _packet_number_generator;
+};
+
+void fnv1a(const uint8_t *data, size_t len, uint8_t *hash);
diff --git a/iocore/net/quic/QUICPacketTransmitter.h b/iocore/net/quic/QUICPacketTransmitter.h
new file mode 100644
index 0000000..d8b8a41
--- /dev/null
+++ b/iocore/net/quic/QUICPacketTransmitter.h
@@ -0,0 +1,50 @@
+/** @file
+ *
+ *  A brief file description
+ *
+ *  @section license License
+ *
+ *  Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+#pragma once
+
+#include "QUICPacket.h"
+
+class QUICPacketTransmitter
+{
+public:
+  /*
+   * Enqueue a packetfor transmission
+   *
+   * This sends QUIC_PACKET_WRITE_READY event.
+   */
+  virtual void transmit_packet(std::unique_ptr<const QUICPacket> packet) = 0;
+
+  /*
+   * Enqueue a packet for retransmission
+   * All frames except ACK and PADDING frames in the original packet will be retransmitted on a new packet.
+   * This sends QUIC_PACKET_WRITE_READY event.
+   */
+  virtual void retransmit_packet(const QUICPacket &packet) = 0;
+
+  /*
+   * Returns a mutex for transmitter interfaces.
+   * You have to acquire a lock with this mutex before calling any methods provieded by QUICPacketTransmitter
+   */
+  virtual Ptr<ProxyMutex> get_transmitter_mutex() = 0;
+};
diff --git a/iocore/net/quic/QUICStream.cc b/iocore/net/quic/QUICStream.cc
new file mode 100644
index 0000000..1e82095
--- /dev/null
+++ b/iocore/net/quic/QUICStream.cc
@@ -0,0 +1,331 @@
+/** @file
+ *
+ *  A brief file description
+ *
+ *  @section license License
+ *
+ *  Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+#include "QUICStream.h"
+
+#include "I_Event.h"
+#include "QUICStreamManager.h"
+#include "QUICDebugNames.h"
+
+const static char *tag = "quic_stream";
+
+void
+QUICStream::init(QUICStreamManager *manager, QUICStreamId id)
+{
+  this->_streamManager = manager;
+
+  this->_id = id;
+
+  this->mutex = new_ProxyMutex();
+}
+
+void
+QUICStream::start()
+{
+  SET_HANDLER(&QUICStream::main_event_handler);
+}
+
+uint32_t
+QUICStream::id()
+{
+  return this->_id;
+}
+
+int
+QUICStream::main_event_handler(int event, void *data)
+{
+  Debug(tag, "%s", QUICDebugNames::vc_event(event));
+
+  switch (event) {
+  case VC_EVENT_READ_READY:
+  case VC_EVENT_READ_COMPLETE: {
+    this->_signal_read_event(true);
+    this->_read_event = nullptr;
+
+    break;
+  }
+  case VC_EVENT_WRITE_READY:
+  case VC_EVENT_WRITE_COMPLETE: {
+    this->_send();
+    this->_signal_write_event(true);
+    this->_write_event = nullptr;
+
+    break;
+  }
+  case VC_EVENT_EOS:
+  case VC_EVENT_ERROR:
+  case VC_EVENT_INACTIVITY_TIMEOUT:
+  case VC_EVENT_ACTIVE_TIMEOUT: {
+    ink_assert(false);
+    break;
+  }
+  default:
+    Debug(tag, "unknown event");
+    ink_assert(false);
+  }
+
+  return EVENT_CONT;
+}
+
+VIO *
+QUICStream::do_io_read(Continuation *c, int64_t nbytes, MIOBuffer *buf)
+{
+  if (buf) {
+    this->_read_vio.buffer.writer_for(buf);
+  } else {
+    this->_read_vio.buffer.clear();
+  }
+
+  this->_read_vio.mutex     = c ? c->mutex : this->mutex;
+  this->_read_vio._cont     = c;
+  this->_read_vio.nbytes    = nbytes;
+  this->_read_vio.ndone     = 0;
+  this->_read_vio.vc_server = this;
+  this->_read_vio.op        = VIO::READ;
+
+  // TODO: If read function is added, call reenable here
+  this->_read_vio.reenable();
+
+  return &this->_read_vio;
+}
+
+VIO *
+QUICStream::do_io_write(Continuation *c, int64_t nbytes, IOBufferReader *buf, bool owner)
+{
+  if (buf) {
+    this->_write_vio.buffer.reader_for(buf);
+  } else {
+    this->_write_vio.buffer.clear();
+  }
+
+  this->_write_vio.mutex     = c ? c->mutex : this->mutex;
+  this->_write_vio._cont     = c;
+  this->_write_vio.nbytes    = nbytes;
+  this->_write_vio.ndone     = 0;
+  this->_write_vio.vc_server = this;
+  this->_write_vio.op        = VIO::WRITE;
+
+  this->_write_vio.reenable();
+
+  return &this->_write_vio;
+}
+
+void
+QUICStream::do_io_close(int lerrno)
+{
+  this->_read_vio.buffer.clear();
+  this->_read_vio.nbytes = 0;
+  this->_read_vio.op     = VIO::NONE;
+  this->_read_vio._cont  = nullptr;
+
+  this->_write_vio.buffer.clear();
+  this->_write_vio.nbytes = 0;
+  this->_write_vio.op     = VIO::NONE;
+  this->_write_vio._cont  = nullptr;
+}
+
+void
+QUICStream::do_io_shutdown(ShutdownHowTo_t howto)
+{
+  ink_assert(false); // unimplemented yet
+}
+
+void
+QUICStream::reenable(VIO *vio)
+{
+  if (vio->op == VIO::READ) {
+    SCOPED_MUTEX_LOCK(lock, this->_read_vio.mutex, this_ethread());
+
+    if (this->_read_vio.nbytes > 0) {
+      int event = (this->_read_vio.ntodo() == 0) ? VC_EVENT_READ_COMPLETE : VC_EVENT_READ_READY;
+
+      if (this->_read_event == nullptr) {
+        this->_read_event = this_ethread()->schedule_imm_local(this, event);
+      }
+    }
+  } else if (vio->op == VIO::WRITE) {
+    SCOPED_MUTEX_LOCK(lock, this->_write_vio.mutex, this_ethread());
+
+    if (this->_write_vio.nbytes > 0) {
+      int event = (this->_write_vio.ntodo() == 0) ? VC_EVENT_WRITE_COMPLETE : VC_EVENT_WRITE_READY;
+
+      if (this->_write_event == nullptr) {
+        this->_write_event = this_ethread()->schedule_imm_local(this, event);
+      }
+    }
+  }
+}
+
+/**
+ * @brief Signal event to this->_read_vio._cont
+ * @param (call_update)  If true, safe to call vio handler directly.
+ *   Or called from do_io_read. Still setting things up. Send event to handle this after the dust settles
+ */
+void
+QUICStream::_signal_read_event(bool direct)
+{
+  int event          = (this->_read_vio.ntodo() == 0) ? VC_EVENT_READ_COMPLETE : VC_EVENT_READ_READY;
+  Continuation *cont = this->_read_vio._cont;
+
+  if (direct) {
+    Event *e          = eventAllocator.alloc();
+    e->callback_event = event;
+    e->cookie         = this;
+    e->init(cont, 0, 0);
+
+    cont->handleEvent(event, e);
+  } else {
+    this_ethread()->schedule_imm(cont, event, this);
+  }
+}
+
+/**
+ * @brief Signal event to this->_write_vio._cont
+ * @param (call_update)  If true, safe to call vio handler directly.
+ *   Or called from do_io_write. Still setting things up. Send event to handle this after the dust settles
+ */
+void
+QUICStream::_signal_write_event(bool direct)
+{
+  int event          = (this->_write_vio.ntodo() == 0) ? VC_EVENT_WRITE_COMPLETE : VC_EVENT_WRITE_READY;
+  Continuation *cont = this->_write_vio._cont;
+
+  if (direct) {
+    Event *e          = eventAllocator.alloc();
+    e->callback_event = event;
+    e->cookie         = this;
+    e->init(cont, 0, 0);
+
+    cont->handleEvent(event, e);
+  } else {
+    this_ethread()->schedule_imm(cont, event, this);
+  }
+}
+
+void
+QUICStream::_write_to_read_vio(std::shared_ptr<const QUICStreamFrame> frame)
+{
+  SCOPED_MUTEX_LOCK(lock, this->_read_vio.mutex, this_ethread());
+
+  int bytes_added = this->_read_vio.buffer.writer()->write(frame->data(), frame->data_length());
+  this->_read_vio.nbytes += bytes_added;
+  this->_request_buffer_offset += frame->data_length();
+}
+
+void
+QUICStream::_reorder_data()
+{
+  while (auto frame = _request_stream_frame_buffer[this->_request_buffer_offset]) {
+    this->_request_stream_frame_buffer.erase(this->_request_buffer_offset);
+    this->_write_to_read_vio(frame);
+  }
+}
+
+/**
+ * @brief Receive STREAM frame
+ * @detail When receive STREAM frame, reorder frames and write to buffer of read_vio.
+ * If the reordering or writting operation is heavy, split out them to read function,
+ * which is called by application via do_io_read() or reenable().
+ */
+void
+QUICStream::recv(std::shared_ptr<const QUICStreamFrame> frame)
+{
+  ink_assert(_id == frame->stream_id());
+  ink_assert(this->_read_vio.op == VIO::READ);
+
+  if (!this->_state.is_allowed_to_receive(*frame)) {
+    this->reset();
+    return;
+  }
+  this->_state.update_with_received_frame(*frame);
+
+  if (this->_request_buffer_offset > frame->offset()) {
+    // Do nothing. Just ignore STREAM frame.
+    return;
+  } else if (this->_request_buffer_offset == frame->offset()) {
+    this->_write_to_read_vio(frame);
+    this->_reorder_data();
+  } else {
+    // NOTE: push fragments in _request_stream_frame_buffer temporally.
+    // They will be reordered when missing data is filled and offset is matched.
+    this->_request_stream_frame_buffer.insert(std::make_pair(frame->offset(), frame));
+  }
+
+  return;
+}
+
+/**
+ * @brief Send STREAM DATA from _response_buffer
+ */
+void
+QUICStream::_send()
+{
+  SCOPED_MUTEX_LOCK(lock, this->_write_vio.mutex, this_ethread());
+
+  IOBufferReader *reader = this->_write_vio.get_reader();
+  int64_t bytes_avail    = reader->read_avail();
+  int64_t total_len      = 0;
+  // TODO: refer maximum_quic_packet_size
+  // uint32_t max_size = this->client_vc->maximum_quic_packet_size() - MAX_STREAM_FRAME_HEADER_LEN(15) - MAX_PACKET_OVERHEAD(25);
+  uint32_t max_size = 1212;
+
+  while (total_len < bytes_avail) {
+    int64_t data_len = reader->block_read_avail();
+    size_t len       = 0;
+
+    if (data_len > max_size) {
+      len = max_size;
+    } else {
+      len = data_len;
+    }
+
+    std::unique_ptr<QUICStreamFrame, QUICFrameDeleterFunc> frame = QUICFrameFactory::create_stream_frame(
+      reinterpret_cast<const uint8_t *>(reader->start()), len, this->_id, this->_response_buffer_offset);
+
+    this->_response_buffer_offset += len;
+    reader->consume(len);
+    this->_write_vio.ndone += len;
+    total_len += len;
+
+    if (!this->_state.is_allowed_to_send(*frame)) {
+      // FIXME: What should we do?
+      break;
+    }
+    this->_state.update_with_sent_frame(*frame);
+    this->_streamManager->send_frame(std::move(frame));
+  }
+
+  return;
+}
+
+void
+QUICStream::reset()
+{
+  // TODO: Create a RST_STREAM frame and pass it to Stream Manager
+}
+
+bool
+QUICStream::is_read_ready()
+{
+  return this->_read_vio.nbytes > 0;
+}
diff --git a/iocore/net/quic/QUICStream.h b/iocore/net/quic/QUICStream.h
new file mode 100644
index 0000000..7cae47f
--- /dev/null
+++ b/iocore/net/quic/QUICStream.h
@@ -0,0 +1,98 @@
+/** @file
+ *
+ *  A brief file description
+ *
+ *  @section license License
+ *
+ *  Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+#pragma once
+
+#include "ts/List.h"
+#include "ts/PriorityQueue.h"
+
+#include "I_VConnection.h"
+
+#include "QUICFrame.h"
+#include "QUICStreamState.h"
+
+class QUICStreamState;
+class QUICStreamManager;
+
+/**
+ * @brief QUIC Stream
+ * TODO: This is similar to Http2Stream. Need to think some integration.
+ */
+class QUICStream : public VConnection
+{
+public:
+  QUICStream() : VConnection(nullptr) {}
+  ~QUICStream() {}
+
+  void init(QUICStreamManager *manager, uint32_t id);
+  void start();
+  int main_event_handler(int event, void *data);
+
+  uint32_t id();
+
+  // Implement VConnection interface.
+  VIO *do_io_read(Continuation *c, int64_t nbytes = INT64_MAX, MIOBuffer *buf = nullptr) override;
+  VIO *do_io_write(Continuation *c = nullptr, int64_t nbytes = INT64_MAX, IOBufferReader *buf = 0, bool owner = false) override;
+  void do_io_close(int lerrno = -1) override;
+  void do_io_shutdown(ShutdownHowTo_t howto) override;
+  void reenable(VIO *vio) override;
+
+  void recv(std::shared_ptr<const QUICStreamFrame> frame);
+  void reset();
+
+  bool is_read_ready();
+
+  LINK(QUICStream, link);
+
+private:
+  QUICStreamState _state;
+
+  void _send();
+
+  void _write_to_read_vio(std::shared_ptr<const QUICStreamFrame>);
+  void _reorder_data();
+  // NOTE: Those are called update_read_request/update_write_request in Http2Stream
+  // void _read_from_net(uint64_t read_len, bool direct);
+  // void _write_to_net(IOBufferReader *buf_reader, int64_t write_len, bool direct);
+
+  void _signal_read_event(bool call_update);
+  void _signal_write_event(bool call_update);
+
+  Event *_send_tracked_event(Event *event, int send_event, VIO *vio);
+
+  QUICStreamId _id                   = 0;
+  QUICOffset _request_buffer_offset  = 0;
+  QUICOffset _response_buffer_offset = 0;
+
+  VIO _read_vio;
+  VIO _write_vio;
+
+  Event *_read_event;
+  Event *_write_event;
+
+  // Fragments of received STREAM frame (offset is unmatched)
+  // TODO: Consider to replace with ts/RbTree.h or other data structure
+  std::map<QUICOffset, std::shared_ptr<const QUICStreamFrame>> _request_stream_frame_buffer;
+
+  QUICStreamManager *_streamManager;
+};
diff --git a/iocore/net/quic/QUICStreamManager.cc b/iocore/net/quic/QUICStreamManager.cc
new file mode 100644
index 0000000..c5bb46e
--- /dev/null
+++ b/iocore/net/quic/QUICStreamManager.cc
@@ -0,0 +1,114 @@
+/** @file
+ *
+ *  A brief file description
+ *
+ *  @section license License
+ *
+ *  Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+#include <QUICStreamManager.h>
+
+#include <QUICApplication.h>
+#include <P_QUICNetVConnection.h>
+
+const static char *tag = "quic_stream_manager";
+
+ClassAllocator<QUICStreamManager> quicStreamManagerAllocator("quicStreamManagerAllocator");
+ClassAllocator<QUICStream> quicStreamAllocator("quicStreamAllocator");
+
+int
+QUICStreamManager::init(QUICFrameTransmitter *tx)
+{
+  this->_tx = tx;
+  return 0;
+}
+
+void
+QUICStreamManager::set_connection(QUICNetVConnection *vc)
+{
+  this->_vc = vc;
+}
+
+void
+QUICStreamManager::handle_frame(std::shared_ptr<const QUICFrame> frame)
+{
+  switch (frame->type()) {
+  case QUICFrameType::STREAM:
+    this->_handle_stream_frame(std::dynamic_pointer_cast<const QUICStreamFrame>(frame));
+    break;
+  default:
+    Debug(tag, "Unexpected frame type: %02x", frame->type());
+    ink_assert(false);
+    break;
+  }
+}
+
+void
+QUICStreamManager::_handle_stream_frame(std::shared_ptr<const QUICStreamFrame> frame)
+{
+  QUICStream *stream           = this->_find_or_create_stream(frame->stream_id());
+  QUICApplication *application = this->_vc->get_application(frame->stream_id());
+
+  if (!application->is_stream_set(stream)) {
+    application->set_stream(stream);
+  }
+
+  stream->recv(frame);
+  // FIXME: schedule VC_EVENT_READ_READY to application every single frame?
+  // If application reading buffer continuously, do not schedule event.
+  this_ethread()->schedule_imm(application, VC_EVENT_READ_READY, stream);
+
+  return;
+}
+
+/**
+ * @brief Send stream frame
+ */
+void
+QUICStreamManager::send_frame(std::unique_ptr<QUICFrame, QUICFrameDeleterFunc> frame)
+{
+  this->_tx->transmit_frame(std::move(frame));
+
+  return;
+}
+
+QUICStream *
+QUICStreamManager::_find_stream(QUICStreamId id)
+{
+  for (QUICStream *s = this->stream_list.head; s; s = s->link.next) {
+    if (s->id() == id) {
+      return s;
+    }
+  }
+  return nullptr;
+}
+
+QUICStream *
+QUICStreamManager::_find_or_create_stream(QUICStreamId stream_id)
+{
+  QUICStream *stream = this->_find_stream(stream_id);
+  if (!stream) {
+    // TODO Free the stream somewhere
+    stream = THREAD_ALLOC_INIT(quicStreamAllocator, this_ethread());
+    stream->init(this, stream_id);
+    stream->start();
+
+    this->stream_list.push(stream);
+  }
+  return stream;
+}
diff --git a/iocore/net/quic/QUICStreamManager.h b/iocore/net/quic/QUICStreamManager.h
new file mode 100644
index 0000000..3daff59
--- /dev/null
+++ b/iocore/net/quic/QUICStreamManager.h
@@ -0,0 +1,55 @@
+/** @file
+ *
+ *  A brief file description
+ *
+ *  @section license License
+ *
+ *  Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+#pragma once
+
+#include "QUICTypes.h"
+#include "QUICStream.h"
+#include "QUICFrameHandler.h"
+#include "QUICFrame.h"
+#include "QUICFrameTransmitter.h"
+
+class QUICNetVConnection;
+
+class QUICStreamManager : public QUICFrameHandler
+{
+public:
+  QUICStreamManager(){};
+
+  int init(QUICFrameTransmitter *tx);
+  void set_connection(QUICNetVConnection *vc); // FIXME Want to remove.
+  virtual void handle_frame(std::shared_ptr<const QUICFrame>) override;
+  void send_frame(std::unique_ptr<QUICFrame, QUICFrameDeleterFunc> frame);
+
+  DLL<QUICStream> stream_list;
+
+private:
+  QUICStream *_find_or_create_stream(QUICStreamId stream_id);
+  QUICStream *_find_stream(QUICStreamId id);
+
+  QUICNetVConnection *_vc   = nullptr;
+  QUICFrameTransmitter *_tx = nullptr;
+
+private:
+  void _handle_stream_frame(std::shared_ptr<const QUICStreamFrame>);
+};
diff --git a/iocore/net/quic/QUICStreamState.cc b/iocore/net/quic/QUICStreamState.cc
new file mode 100644
index 0000000..dfe9f4d
--- /dev/null
+++ b/iocore/net/quic/QUICStreamState.cc
@@ -0,0 +1,58 @@
+#include "QUICStreamState.h"
+#include "ts/ink_assert.h"
+
+const QUICStreamState::State
+QUICStreamState::get() const
+{
+  return this->_state;
+}
+
+bool
+QUICStreamState::is_allowed_to_send(const QUICFrame &frame) const
+{
+  return true;
+}
+
+bool
+QUICStreamState::is_allowed_to_receive(const QUICFrame &frame) const
+{
+  return true;
+}
+
+void
+QUICStreamState::update_with_received_frame(const QUICFrame &frame)
+{
+  switch (this->_state) {
+  case State::idle:
+    this->_set_state(State::open);
+  // fall through
+  case State::open: {
+    if (frame.type() == QUICFrameType::STREAM && dynamic_cast<const QUICStreamFrame &>(frame).has_fin_flag()) {
+      this->_set_state(State::half_closed_remote);
+    } else if (frame.type() == QUICFrameType::RST_STREAM) {
+      this->_set_state(State::closed);
+    }
+  } break;
+  case State::half_closed_local:
+  case State::half_closed_remote:
+  case State::closed:
+  case State::illegal:
+    // Once we get illegal state, no way to recover it
+    break;
+  default:
+    break;
+  }
+}
+
+void
+QUICStreamState::update_with_sent_frame(const QUICFrame &frame)
+{
+}
+
+void
+QUICStreamState::_set_state(State s)
+{
+  ink_assert(s != State::idle);
+  ink_assert(s != State::illegal);
+  this->_state = s;
+}
diff --git a/iocore/net/quic/QUICStreamState.h b/iocore/net/quic/QUICStreamState.h
new file mode 100644
index 0000000..0479308
--- /dev/null
+++ b/iocore/net/quic/QUICStreamState.h
@@ -0,0 +1,54 @@
+/** @file
+ *
+ *  A brief file description
+ *
+ *  @section license License
+ *
+ *  Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+#pragma once
+
+#include "QUICFrame.h"
+
+class QUICStreamState
+{
+public:
+  // 10.1.  Life of a Stream
+  enum class State {
+    idle,
+    open,
+    half_closed_remote,
+    half_closed_local,
+    closed,
+    illegal // not on the specification, just for internal use
+  };
+  const State get() const;
+  bool is_allowed_to_send(const QUICFrame &frame) const;
+  bool is_allowed_to_receive(const QUICFrame &frame) const;
+
+  /*
+   * Updates its internal state
+   * Internal state will be "illegal" state if inappropriate frame was passed
+   */
+  void update_with_received_frame(const QUICFrame &frame);
+  void update_with_sent_frame(const QUICFrame &frame);
+
+private:
+  void _set_state(State s);
+  State _state = State::idle;
+};
diff --git a/iocore/net/quic/QUICTransportParameters.cc b/iocore/net/quic/QUICTransportParameters.cc
new file mode 100644
index 0000000..4b483a2
--- /dev/null
+++ b/iocore/net/quic/QUICTransportParameters.cc
@@ -0,0 +1,132 @@
+#include <cstdlib>
+#include "QUICTransportParameters.h"
+
+QUICTransportParameterValue
+QUICTransportParameters::get(QUICTransportParameterId tpid) const
+{
+  QUICTransportParameterValue value;
+  const uint8_t *p = this->_buf + this->_parameters_offset();
+
+  uint16_t n = (p[0] << 8) + p[1];
+  p += 2;
+  for (; n > 0; --n) {
+    uint16_t _id = (p[0] << 8) + p[1];
+    p += 2;
+    uint16_t _value_len = (p[0] << 8) + p[1];
+    p += 2;
+    if (tpid == _id) {
+      value.data = p;
+      value.len  = _value_len;
+      return value;
+    }
+    p += _value_len;
+  }
+  value.data = nullptr;
+  value.len  = 0;
+  return value;
+}
+
+void
+QUICTransportParameters::add(QUICTransportParameterId id, QUICTransportParameterValue value)
+{
+  _parameters.put(id, value);
+}
+
+void
+QUICTransportParameters::store(uint8_t *buf, uint16_t *len) const
+{
+  uint8_t *p = buf;
+
+  // Why Map::get() doesn't have const??
+  QUICTransportParameters *me = const_cast<QUICTransportParameters *>(this);
+
+  // Write QUIC versions
+  this->_store(p, len);
+  p += *len;
+
+  // Write parameters
+  Vec<QUICTransportParameterId> keys;
+  me->_parameters.get_keys(keys);
+  unsigned int n = keys.length();
+  p[0]           = (n & 0xff00) >> 8;
+  p[1]           = n & 0xff;
+  p += 2;
+  for (unsigned int i = 0; i < n; ++i) {
+    QUICTransportParameterValue value;
+    p[0] = (keys[i] & 0xff00) >> 8;
+    p[1] = keys[i] & 0xff;
+    p += 2;
+    value = me->_parameters.get(keys[i]);
+    p[0]  = (value.len & 0xff00) >> 8;
+    p[1]  = value.len & 0xff;
+    p += 2;
+    memcpy(p, value.data, value.len);
+    p += value.len;
+  }
+  *len = (p - buf);
+}
+
+void
+QUICTransportParametersInClientHello::_store(uint8_t *buf, uint16_t *len) const
+{
+  size_t l;
+  *len = 0;
+  QUICTypeUtil::write_QUICVersion(this->_negotiated_version, buf, &l);
+  buf += l;
+  *len += l;
+  QUICTypeUtil::write_QUICVersion(this->_initial_version, buf, &l);
+  *len += l;
+}
+
+std::ptrdiff_t
+QUICTransportParametersInClientHello::_parameters_offset() const
+{
+  return 8; // sizeof(QUICVersion) + sizeof(QUICVersion)
+}
+
+QUICVersion
+QUICTransportParametersInClientHello::negotiated_version() const
+{
+  return QUICTypeUtil::read_QUICVersion(this->_buf);
+}
+
+QUICVersion
+QUICTransportParametersInClientHello::initial_version() const
+{
+  return QUICTypeUtil::read_QUICVersion(this->_buf + sizeof(QUICVersion));
+}
+
+void
+QUICTransportParametersInEncryptedExtensions::_store(uint8_t *buf, uint16_t *len) const
+{
+  uint8_t *p = buf;
+  size_t l;
+
+  p[0] = (this->_n_versions & 0xff00) >> 8;
+  p[1] = this->_n_versions & 0xff;
+  p += 2;
+  for (int i = 0; i < this->_n_versions; ++i) {
+    QUICTypeUtil::write_QUICVersion(this->_versions[i], p, &l);
+    p += l;
+  }
+  *len = p - buf;
+}
+
+const uint8_t *
+QUICTransportParametersInEncryptedExtensions::supported_versions(uint16_t *n) const
+{
+  *n = (this->_buf[0] << 8) + this->_buf[1];
+  return this->_buf + 2;
+}
+
+void
+QUICTransportParametersInEncryptedExtensions::add_version(QUICVersion version)
+{
+  this->_versions[this->_n_versions++] = version;
+}
+
+std::ptrdiff_t
+QUICTransportParametersInEncryptedExtensions::_parameters_offset() const
+{
+  return 2 + 4 * ((this->_buf[0] << 8) + this->_buf[1]);
+}
diff --git a/iocore/net/quic/QUICTransportParameters.h b/iocore/net/quic/QUICTransportParameters.h
new file mode 100644
index 0000000..244b33e
--- /dev/null
+++ b/iocore/net/quic/QUICTransportParameters.h
@@ -0,0 +1,120 @@
+/** @file
+ *
+ *  A brief file description
+ *
+ *  @section license License
+ *
+ *  Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+#pragma once
+
+#include "QUICTypes.h"
+#include "ts/Map.h"
+#include <cstddef>
+
+class QUICTransportParameterId
+{
+public:
+  enum {
+    INITIAL_MAX_STREAM_DATA = 0,
+    INITIAL_MAX_DATA,
+    INITIAL_MAX_STREAM_ID,
+    IDLE_TIMEOUT,
+    TRUNCATE_CONNECTION_ID,
+    MAX_PACKET_SIZE,
+  };
+
+  explicit operator bool() const { return true; }
+
+  bool
+  operator==(const QUICTransportParameterId &x) const
+  {
+    return this->_id == x._id;
+  }
+
+  bool
+  operator==(uint16_t &x) const
+  {
+    return this->_id == x;
+  }
+
+  operator uint16_t() const { return _id; };
+
+  QUICTransportParameterId() : _id(0){};
+  QUICTransportParameterId(uint16_t id) : _id(id){};
+
+private:
+  uint16_t _id = 0;
+};
+
+typedef struct _QUICTransportParameterValue {
+  _QUICTransportParameterValue(){};
+  _QUICTransportParameterValue(const uint8_t *str) : data(str), len(str ? strlen(reinterpret_cast<const char *>(str)) : 0){};
+  _QUICTransportParameterValue(const uint8_t *_data, uint16_t _len) : data(_data), len(_len){};
+  const uint8_t *data = nullptr;
+  uint16_t len        = 0;
+} QUICTransportParameterValue;
+
+class QUICTransportParameters
+{
+public:
+  QUICTransportParameters(const uint8_t *buf) : _buf(buf){};
+  QUICTransportParameterValue get(QUICTransportParameterId id) const;
+  void add(QUICTransportParameterId id, QUICTransportParameterValue value);
+  void store(uint8_t *buf, uint16_t *len) const;
+
+protected:
+  virtual std::ptrdiff_t _parameters_offset() const = 0;
+  virtual void _store(uint8_t *buf, uint16_t *len) const = 0;
+  const uint8_t *_buf;
+  Map<QUICTransportParameterId, QUICTransportParameterValue> _parameters;
+};
+
+class QUICTransportParametersInClientHello : public QUICTransportParameters
+{
+public:
+  QUICTransportParametersInClientHello(QUICVersion negotiated_version, QUICVersion initial_version)
+    : QUICTransportParameters(nullptr), _negotiated_version(negotiated_version), _initial_version(initial_version){};
+  QUICTransportParametersInClientHello(const uint8_t *buf) : QUICTransportParameters(buf){};
+  QUICVersion negotiated_version() const;
+  QUICVersion initial_version() const;
+
+protected:
+  std::ptrdiff_t _parameters_offset() const override;
+  void _store(uint8_t *buf, uint16_t *len) const override;
+
+private:
+  QUICVersion _negotiated_version = 0;
+  QUICVersion _initial_version    = 0;
+};
+
+class QUICTransportParametersInEncryptedExtensions : public QUICTransportParameters
+{
+public:
+  QUICTransportParametersInEncryptedExtensions() : QUICTransportParameters(nullptr){};
+  QUICTransportParametersInEncryptedExtensions(const uint8_t *buf) : QUICTransportParameters(buf){};
+  const uint8_t *supported_versions(uint16_t *n) const;
+  void add_version(QUICVersion version);
+
+protected:
+  std::ptrdiff_t _parameters_offset() const override;
+  void _store(uint8_t *buf, uint16_t *len) const override;
+
+  uint8_t _n_versions        = 0;
+  QUICVersion _versions[256] = {};
+};
diff --git a/iocore/net/quic/QUICTypes.cc b/iocore/net/quic/QUICTypes.cc
new file mode 100644
index 0000000..0be3d00
--- /dev/null
+++ b/iocore/net/quic/QUICTypes.cc
@@ -0,0 +1,140 @@
+/** @file
+ *
+ *  A brief file description
+ *
+ *  @section license License
+ *
+ *  Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+#include "QUICTypes.h"
+
+// TODO: Update version number
+// Note: You need to update QUICTypes.h if you change the number of versions
+const QUICVersion QUIC_SUPPORTED_VERSIONS[] = {
+  0xff000004, // Prefix for drafts (0xff000000) + draft number
+};
+
+const QUICStreamId STREAM_ID_FOR_HANDSHAKE = 0;
+
+bool
+QUICTypeUtil::hasLongHeader(const uint8_t *buf)
+{
+  return (buf[0] & 0x80) != 0;
+}
+
+QUICConnectionId
+QUICTypeUtil::read_QUICConnectionId(const uint8_t *buf, uint8_t len)
+{
+  return static_cast<QUICPacketNumber>(read_nbytes_as_uint(buf, len));
+}
+
+QUICPacketNumber
+QUICTypeUtil::read_QUICPacketNumber(const uint8_t *buf, uint8_t len)
+{
+  return static_cast<QUICPacketNumber>(read_nbytes_as_uint(buf, len));
+}
+
+QUICVersion
+QUICTypeUtil::read_QUICVersion(const uint8_t *buf)
+{
+  return static_cast<QUICVersion>(read_nbytes_as_uint(buf, 4));
+}
+
+QUICStreamId
+QUICTypeUtil::read_QUICStreamId(const uint8_t *buf, uint8_t len)
+{
+  return static_cast<QUICStreamId>(read_nbytes_as_uint(buf, len));
+}
+
+QUICOffset
+QUICTypeUtil::read_QUICOffset(const uint8_t *buf, uint8_t len)
+{
+  return static_cast<QUICOffset>(read_nbytes_as_uint(buf, len));
+}
+
+QUICErrorCode
+QUICTypeUtil::read_QUICErrorCode(const uint8_t *buf)
+{
+  return static_cast<QUICErrorCode>(read_nbytes_as_uint(buf, 4));
+}
+
+uint64_t
+QUICTypeUtil::read_nbytes_as_uint(const uint8_t *buf, uint8_t n)
+{
+  uint64_t value = 0;
+  memcpy(&value, buf, n);
+  return be64toh(value << (64 - n * 8));
+}
+
+void
+QUICTypeUtil::write_QUICConnectionId(QUICConnectionId connection_id, uint8_t n, uint8_t *buf, size_t *len)
+{
+  write_uint_as_nbytes(static_cast<uint64_t>(connection_id), n, buf, len);
+}
+
+void
+QUICTypeUtil::write_QUICPacketNumber(QUICPacketNumber packet_number, uint8_t n, uint8_t *buf, size_t *len)
+{
+  write_uint_as_nbytes(static_cast<uint64_t>(packet_number), n, buf, len);
+}
+
+void
+QUICTypeUtil::write_QUICVersion(QUICVersion version, uint8_t *buf, size_t *len)
+{
+  write_uint_as_nbytes(static_cast<uint64_t>(version), 4, buf, len);
+}
+
+void
+QUICTypeUtil::write_QUICStreamId(QUICStreamId stream_id, uint8_t n, uint8_t *buf, size_t *len)
+{
+  write_uint_as_nbytes(static_cast<uint64_t>(stream_id), n, buf, len);
+}
+
+void
+QUICTypeUtil::write_QUICOffset(QUICOffset offset, uint8_t n, uint8_t *buf, size_t *len)
+{
+  write_uint_as_nbytes(static_cast<uint64_t>(offset), n, buf, len);
+}
+
+void
+QUICTypeUtil::write_QUICErrorCode(QUICErrorCode error_code, uint8_t *buf, size_t *len)
+{
+  write_uint_as_nbytes(static_cast<uint64_t>(error_code), 4, buf, len);
+}
+
+void
+QUICTypeUtil::write_uint_as_nbytes(uint64_t value, uint8_t n, uint8_t *buf, size_t *len)
+{
+  value = htobe64(value) >> (64 - n * 8);
+  memcpy(buf, reinterpret_cast<uint8_t *>(&value), n);
+  *len = n;
+}
+
+void
+fnv1a(const uint8_t *data, size_t len, uint8_t *hash)
+{
+  uint64_t h     = 0xcbf29ce484222325ULL;
+  uint64_t prime = 0x100000001b3ULL;
+  size_t n;
+
+  for (size_t i = 0; i < len; ++i) {
+    h ^= data[i];
+    h *= prime;
+  }
+  return QUICTypeUtil::write_uint_as_nbytes(h, 8, hash, &n);
+}
diff --git a/iocore/net/quic/QUICTypes.h b/iocore/net/quic/QUICTypes.h
new file mode 100644
index 0000000..d1f613b
--- /dev/null
+++ b/iocore/net/quic/QUICTypes.h
@@ -0,0 +1,179 @@
+/** @file
+ *
+ *  A brief file description
+ *
+ *  @section license License
+ *
+ *  Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+#pragma once
+
+#ifdef __APPLE__
+#include <libkern/OSByteOrder.h>
+#define be64toh(x) OSSwapBigToHostInt64(x)
+#define htobe64(x) OSSwapHostToBigInt64(x)
+#else
+#include <endian.h>
+#endif
+#include <cstring>
+
+#include <random>
+#include <cstdint>
+
+// These magical defines should be removed when we implement seriously
+#define MAGIC_NUMBER_0 0
+#define MAGIC_NUMBER_1 1
+#define MAGIC_NUMBER_TRUE true
+
+typedef uint64_t QUICPacketNumber;
+typedef uint32_t QUICVersion;
+typedef uint32_t QUICStreamId;
+typedef uint64_t QUICOffset;
+
+extern const QUICVersion QUIC_SUPPORTED_VERSIONS[1];
+extern const QUICStreamId STREAM_ID_FOR_HANDSHAKE;
+
+enum class QUICPacketType : int {
+  VERSION_NEGOTIATION = 1,
+  CLIENT_INITIAL,
+  SERVER_STATELESS_RETRY,
+  SERVER_CLEARTEXT,
+  CLIENT_CLEARTEXT,
+  ZERO_RTT_PROTECTED,
+  ONE_RTT_PROTECTED_KEY_PHASE_0,
+  ONE_RTT_PROTECTED_KEY_PHASE_1,
+  PUBLIC_RESET,
+  UNINITIALIZED,
+};
+
+// To detect length of Packet Number
+enum class QUICPacketShortHeaderType : int {
+  ONE = 1,
+  TWO,
+  THREE,
+  UNINITIALIZED,
+};
+
+enum class QUICFrameType : int {
+  PADDING = 0x00,
+  RST_STREAM,
+  CONNECTION_CLOSE,
+  GOAWAY,
+  MAX_DATA,
+  MAX_STREAM_DATA,
+  MAX_STREAM_ID,
+  PING,
+  BLOCKED,
+  STREAM_BLOCKED,
+  STREAM_ID_NEEDED,
+  NEW_CONNECTION_ID,
+  ACK     = 0xA0,
+  STREAM  = 0xC0,
+  UNKNOWN = 0x100,
+};
+
+enum class QUICVersionNegotiationStatus {
+  NOT_NEGOTIATED, // Haven't negotiated yet
+  NEGOTIATED,     // Negotiated
+  REVALIDATED,    // Revalidated in cryptographic handshake
+  FAILED,         // Negotiation failed
+};
+
+enum class QUICKeyPhase : int {
+  PHASE_0 = 0,
+  PHASE_1,
+  PHASE_UNINITIALIZED,
+};
+
+enum class QUICErrorClass {
+  NONE,
+  AQPPLICATION_SPECIFIC,
+  HOST_LOCAL,
+  QUIC_TRANSPORT,
+  CRYPTOGRAPHIC,
+};
+
+enum class QUICErrorCode : uint32_t {
+  APPLICATION_SPECIFIC_ERROR = 0,
+  HOST_LOCAL_ERROR           = 0x40000000,
+  QUIC_TRANSPORT_ERROR       = 0x80000000,
+  QUIC_INTERNAL_ERROR        = 0x80000001,
+  CRYPTOGRAPHIC_ERROR        = 0xC0000000,
+  TLS_HANDSHAKE_FAILED       = 0xC000001C,
+  // TODO Add error codes
+};
+
+struct QUICError {
+  QUICError(const QUICErrorClass error_class = QUICErrorClass::NONE,
+            const QUICErrorCode error_code = QUICErrorCode::APPLICATION_SPECIFIC_ERROR, const char *err_msg = nullptr)
+  {
+    cls  = error_class;
+    code = error_code;
+    msg  = err_msg;
+  };
+
+  QUICErrorClass cls;
+  QUICErrorCode code;
+  const char *msg;
+};
+
+class QUICConnectionId
+{
+public:
+  explicit operator bool() const { return true; }
+
+  operator uint64_t() const { return _id; };
+
+  QUICConnectionId() { this->randomize(); };
+  QUICConnectionId(uint64_t id) : _id(id){};
+
+  void
+  randomize()
+  {
+    std::random_device rnd;
+    this->_id = (static_cast<uint64_t>(rnd()) << 32) + rnd();
+  };
+
+private:
+  uint64_t _id;
+};
+
+class QUICTypeUtil
+{
+public:
+  static bool hasLongHeader(const uint8_t *buf);
+
+  static QUICConnectionId read_QUICConnectionId(const uint8_t *buf, uint8_t n);
+  static QUICPacketNumber read_QUICPacketNumber(const uint8_t *buf, uint8_t n);
+  static QUICVersion read_QUICVersion(const uint8_t *buf);
+  static QUICStreamId read_QUICStreamId(const uint8_t *buf, uint8_t n);
+  static QUICOffset read_QUICOffset(const uint8_t *buf, uint8_t n);
+  static QUICErrorCode read_QUICErrorCode(const uint8_t *buf);
+
+  static void write_QUICConnectionId(QUICConnectionId connection_id, uint8_t n, uint8_t *buf, size_t *len);
+  static void write_QUICPacketNumber(QUICPacketNumber packet_number, uint8_t n, uint8_t *buf, size_t *len);
+  static void write_QUICVersion(QUICVersion version, uint8_t *buf, size_t *len);
+  static void write_QUICStreamId(QUICStreamId stream_id, uint8_t n, uint8_t *buf, size_t *len);
+  static void write_QUICOffset(QUICOffset offset, uint8_t n, uint8_t *buf, size_t *len);
+  static void write_QUICErrorCode(QUICErrorCode error_code, uint8_t *buf, size_t *len);
+
+  static uint64_t read_nbytes_as_uint(const uint8_t *buf, uint8_t n);
+  static void write_uint_as_nbytes(uint64_t value, uint8_t n, uint8_t *buf, size_t *len);
+
+private:
+};
diff --git a/iocore/net/quic/QUICVersionNegotiator.cc b/iocore/net/quic/QUICVersionNegotiator.cc
new file mode 100644
index 0000000..6e68563
--- /dev/null
+++ b/iocore/net/quic/QUICVersionNegotiator.cc
@@ -0,0 +1,63 @@
+/** @file
+ *
+ *  A brief file description
+ *
+ *  @section license License
+ *
+ *  Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+#include "QUICVersionNegotiator.h"
+
+QUICVersionNegotiator::QUICVersionNegotiator(QUICPacketFactory *packet_factory, QUICPacketTransmitter *tx)
+  : _packet_factory(packet_factory), _tx(tx){};
+
+QUICVersionNegotiationStatus
+QUICVersionNegotiator::status()
+{
+  return this->_status;
+}
+
+QUICVersionNegotiationStatus
+QUICVersionNegotiator::negotiate(const QUICPacket *initial_packet)
+{
+  if (this->_is_supported(initial_packet->version())) {
+    this->_status = QUICVersionNegotiationStatus::NEGOTIATED;
+  } else {
+    this->_tx->transmit_packet(this->_packet_factory->create_version_negotiation_packet(initial_packet));
+  }
+  return this->_status;
+}
+
+QUICVersionNegotiationStatus
+QUICVersionNegotiator::revalidate(QUICVersion version)
+{
+  // TDOO revalidate the version
+  this->_status = QUICVersionNegotiationStatus::FAILED;
+  return _status;
+}
+
+bool
+QUICVersionNegotiator::_is_supported(QUICVersion version)
+{
+  for (auto v : QUIC_SUPPORTED_VERSIONS) {
+    if (v == version) {
+      return true;
+    }
+  }
+  return false;
+}
diff --git a/iocore/net/quic/QUICVersionNegotiator.h b/iocore/net/quic/QUICVersionNegotiator.h
new file mode 100644
index 0000000..8410d75
--- /dev/null
+++ b/iocore/net/quic/QUICVersionNegotiator.h
@@ -0,0 +1,47 @@
+/** @file
+ *
+ *  A brief file description
+ *
+ *  @section license License
+ *
+ *  Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+#pragma once
+
+#include "QUICTypes.h"
+#include "QUICPacketTransmitter.h"
+
+/**
+ * @brief Abstruct QUIC Application Class
+ * @detail Every quic application must inherits this class
+ */
+class QUICVersionNegotiator
+{
+public:
+  QUICVersionNegotiator(QUICPacketFactory *packet_factory, QUICPacketTransmitter *tx);
+  QUICVersionNegotiationStatus status();
+  QUICVersionNegotiationStatus negotiate(const QUICPacket *initial_packet);
+  QUICVersionNegotiationStatus revalidate(QUICVersion version);
+
+private:
+  QUICPacketFactory *_packet_factory   = nullptr;
+  QUICPacketTransmitter *_tx           = nullptr;
+  QUICVersionNegotiationStatus _status = QUICVersionNegotiationStatus::NOT_NEGOTIATED;
+
+  bool _is_supported(QUICVersion version);
+};
diff --git a/iocore/net/quic/test/Makefile.am b/iocore/net/quic/test/Makefile.am
new file mode 100644
index 0000000..1216cda
--- /dev/null
+++ b/iocore/net/quic/test/Makefile.am
@@ -0,0 +1,368 @@
+# Makefile.am for the traffic/iocore/net hierarchy
+#
+#  Licensed to the Apache Software Foundation (ASF) under one
+#  or more contributor license agreements.  See the NOTICE file
+#  distributed with this work for additional information
+#  regarding copyright ownership.  The ASF licenses this file
+#  to you under the Apache License, Version 2.0 (the
+#  "License"); you may not use this file except in compliance
+#  with the License.  You may obtain a copy of the License at
+#
+#      http://www.apache.org/licenses/LICENSE-2.0
+#
+#  Unless required by applicable law or agreed to in writing, software
+#  distributed under the License is distributed on an "AS IS" BASIS,
+#  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+#  See the License for the specific language governing permissions and
+#  limitations under the License.
+
+TESTS = $(check_PROGRAMS)
+check_PROGRAMS = \
+  test_QUICPacket \
+  test_QUICPacketFactory \
+  test_QUICFrame \
+  test_QUICFrameDispatcher \
+  test_QUICStreamState \
+  test_QUICStream \
+  test_QUICTransportParameters \
+  test_QUICCrypto \
+  test_QUICLossDetector \
+  test_QUICTypeUtil \
+  test_QUICAckFrameCreator \
+  test_QUICVersionNegotiator
+
+
+AM_CPPFLAGS += \
+  $(iocore_include_dirs) \
+  -I$(abs_top_srcdir)/lib \
+  -I$(abs_top_srcdir)/lib/records \
+  -I$(abs_top_srcdir)/mgmt \
+  -I$(abs_top_srcdir)/mgmt/utils \
+  -I$(abs_top_srcdir)/proxy \
+  -I$(abs_top_srcdir)/proxy/hdrs \
+  -I$(abs_top_srcdir)/proxy/http \
+  -I$(abs_top_srcdir)/proxy/logging \
+  -I$(abs_top_srcdir)/proxy/shared \
+  -I$(abs_top_srcdir)/tests/include \
+  @OPENSSL_INCLUDES@
+
+if OPENSSL_IS_BORINGSSL
+QUICCrypto_impl = ../QUICCrypto_boringssl.cc
+else
+QUICCrypto_impl = ../QUICCrypto_openssl.cc
+endif
+
+#
+# test_QUICPacket
+#
+test_QUICPacket_CPPFLAGS = \
+  $(AM_CPPFLAGS)
+
+test_QUICPacket_LDFLAGS = \
+  @AM_LDFLAGS@
+
+test_QUICPacket_SOURCES = \
+  event_processor_main.cc \
+  test_QUICPacket.cc \
+  ../QUICPacket.cc \
+  ../QUICCrypto.cc \
+  $(QUICCrypto_impl) \
+  ../QUICTypes.cc
+
+test_QUICPacket_LDADD = \
+  $(top_builddir)/lib/ts/libtsutil.la \
+  $(top_builddir)/iocore/eventsystem/libinkevent.a \
+  $(top_builddir)/iocore/net/quic/libquic.a \
+  $(top_builddir)/lib/records/librecords_p.a \
+  $(top_builddir)/mgmt/libmgmt_p.la \
+  $(top_builddir)/lib/ts/libtsutil.la \
+  $(top_builddir)/proxy/shared/libUglyLogStubs.a \
+  @LIBTCL@ \
+  @HWLOC_LIBS@
+
+#
+# test_QUICPacketFactory
+#
+test_QUICPacketFactory_CPPFLAGS = \
+  $(AM_CPPFLAGS)
+
+test_QUICPacketFactory_LDFLAGS = \
+  @AM_LDFLAGS@
+
+test_QUICPacketFactory_SOURCES = \
+  event_processor_main.cc \
+  test_QUICPacketFactory.cc \
+  ../QUICPacket.cc \
+  ../QUICCrypto.cc \
+  $(QUICCrypto_impl) \
+  ../QUICTypes.cc
+
+test_QUICPacketFactory_LDADD = \
+  $(top_builddir)/lib/ts/libtsutil.la \
+  $(top_builddir)/iocore/eventsystem/libinkevent.a \
+  $(top_builddir)/iocore/net/quic/libquic.a \
+  $(top_builddir)/lib/records/librecords_p.a \
+  $(top_builddir)/mgmt/libmgmt_p.la \
+  $(top_builddir)/lib/ts/libtsutil.la \
+  $(top_builddir)/proxy/shared/libUglyLogStubs.a \
+  @LIBTCL@ \
+  @HWLOC_LIBS@
+
+#
+# test_QUICFrame
+#
+test_QUICFrame_CPPFLAGS = \
+  $(AM_CPPFLAGS)
+
+test_QUICFrame_LDFLAGS = \
+  @AM_LDFLAGS@
+
+test_QUICFrame_SOURCES = \
+  main.cc \
+  test_QUICFrame.cc \
+  ../QUICFrame.cc \
+  ../QUICPacket.cc \
+  ../QUICCrypto.cc \
+  $(QUICCrypto_impl) \
+  ../QUICTypes.cc
+
+test_QUICFrame_LDADD = \
+  $(top_builddir)/lib/ts/libtsutil.la \
+  $(top_builddir)/iocore/eventsystem/libinkevent.a
+
+#
+# test_QUICFrameDispatcher
+#
+test_QUICFrameDispatcher_CPPFLAGS = \
+  $(AM_CPPFLAGS)
+
+test_QUICFrameDispatcher_LDFLAGS = \
+  @AM_LDFLAGS@
+
+test_QUICFrameDispatcher_SOURCES = \
+  main.cc \
+  test_QUICFrameDispatcher.cc \
+  ../QUICFrameDispatcher.cc \
+  ../QUICConnectionManager.cc \
+  ../QUICStreamManager.cc \
+  ../QUICFlowController.cc \
+  ../QUICCongestionController.cc \
+  ../QUICLossDetector.cc \
+  ../QUICFrame.cc \
+  ../QUICPacket.cc \
+  ../QUICStream.cc \
+  ../QUICStreamState.cc \
+  ../QUICApplication.cc \
+  ../QUICHandshake.cc \
+  ../QUICTypes.cc \
+  ../QUICEchoApp.cc \
+  ../QUICDebugNames.cc \
+  ../QUICCrypto.cc \
+  $(QUICCrypto_impl)
+
+test_QUICFrameDispatcher_LDADD = \
+  $(top_builddir)/lib/ts/libtsutil.la \
+  $(top_builddir)/iocore/eventsystem/libinkevent.a \
+  $(top_builddir)/lib/records/librecords_p.a \
+  $(top_builddir)/mgmt/libmgmt_p.la \
+  $(top_builddir)/lib/ts/libtsutil.la \
+  $(top_builddir)/proxy/shared/libUglyLogStubs.a \
+  @LIBTCL@ \
+  @HWLOC_LIBS@
+
+
+#
+# test_QUICStreamState
+#
+test_QUICStreamState_CPPFLAGS = \
+  $(AM_CPPFLAGS)
+
+test_QUICStreamState_LDFLAGS = \
+  @AM_LDFLAGS@
+
+test_QUICStreamState_SOURCES = \
+  main.cc \
+  test_QUICStreamState.cc \
+  ../QUICStreamState.cc
+
+test_QUICStreamState_LDADD = \
+  $(top_builddir)/lib/ts/libtsutil.la \
+  $(top_builddir)/iocore/eventsystem/libinkevent.a \
+  $(top_builddir)/iocore/net/quic/libquic.a \
+  $(top_builddir)/lib/records/librecords_p.a \
+  $(top_builddir)/mgmt/libmgmt_p.la \
+  $(top_builddir)/lib/ts/libtsutil.la \
+  $(top_builddir)/proxy/shared/libUglyLogStubs.a \
+  @LIBTCL@ \
+  @HWLOC_LIBS@
+
+#
+# test_QUICStream
+#
+test_QUICStream_CPPFLAGS = \
+  $(AM_CPPFLAGS)
+
+test_QUICStream_LDFLAGS = \
+  @AM_LDFLAGS@
+
+test_QUICStream_SOURCES = \
+  event_processor_main.cc \
+  test_QUICStream.cc \
+  ../QUICStream.cc \
+  ../QUICFrameDispatcher.cc \
+  ../QUICConnectionManager.cc \
+  ../QUICStreamManager.cc \
+  ../QUICFlowController.cc \
+  ../QUICCongestionController.cc
+
+test_QUICStream_LDADD = \
+  $(top_builddir)/lib/ts/libtsutil.la \
+  $(top_builddir)/iocore/eventsystem/libinkevent.a \
+  $(top_builddir)/iocore/net/quic/libquic.a \
+  $(top_builddir)/lib/records/librecords_p.a \
+  $(top_builddir)/mgmt/libmgmt_p.la \
+  $(top_builddir)/lib/ts/libtsutil.la \
+  $(top_builddir)/proxy/shared/libUglyLogStubs.a \
+  @LIBTCL@ \
+  @HWLOC_LIBS@
+
+#
+# test_QUICTransportParameters
+#
+test_QUICTransportParameters_CPPFLAGS = \
+  $(AM_CPPFLAGS)
+
+test_QUICTransportParameters_LDFLAGS = \
+  @AM_LDFLAGS@
+
+test_QUICTransportParameters_SOURCES = \
+  main.cc \
+  test_QUICTransportParameters.cc \
+  ../QUICTransportParameters.cc \
+  ../QUICTypes.cc
+
+test_QUICTransportParameters_LDADD = \
+  $(top_builddir)/lib/ts/libtsutil.la \
+  $(top_builddir)/iocore/eventsystem/libinkevent.a
+
+#
+# test_QUICCrypto
+#
+test_QUICCrypto_CPPFLAGS = \
+  $(AM_CPPFLAGS)
+
+test_QUICCrypto_LDFLAGS = \
+  @AM_LDFLAGS@ \
+  @OPENSSL_LDFLAGS@
+
+test_QUICCrypto_LDADD = \
+  @OPENSSL_LIBS@ \
+  $(top_builddir)/lib/ts/libtsutil.la \
+  $(top_builddir)/iocore/eventsystem/libinkevent.a
+
+test_QUICCrypto_SOURCES = \
+  main.cc \
+  test_QUICCrypto.cc \
+  ../QUICCrypto.cc \
+  $(QUICCrypto_impl) \
+  ../QUICCrypto.h
+
+#
+# test_QUICLossDetector
+#
+test_QUICLossDetector_CPPFLAGS = \
+  $(AM_CPPFLAGS)
+
+test_QUICLossDetector_LDFLAGS = \
+  @AM_LDFLAGS@
+
+test_QUICLossDetector_LDADD = \
+  $(top_builddir)/lib/ts/libtsutil.la \
+  $(top_builddir)/iocore/eventsystem/libinkevent.a \
+  $(top_builddir)/iocore/net/quic/libquic.a \
+  $(top_builddir)/lib/records/librecords_p.a \
+  $(top_builddir)/mgmt/libmgmt_p.la \
+  $(top_builddir)/lib/ts/libtsutil.la \
+  $(top_builddir)/proxy/shared/libUglyLogStubs.a \
+  @LIBTCL@ \
+  @HWLOC_LIBS@
+
+test_QUICLossDetector_SOURCES = \
+  event_processor_main.cc \
+  test_QUICLossDetector.cc \
+  ../QUICLossDetector.cc \
+  ../QUICTypes.cc \
+  ../QUICPacket.cc \
+  ../QUICCrypto.cc \
+  $(QUICCrypto_impl) \
+  ../QUICFrame.cc
+
+#
+# test_QUICTypeUtil
+#
+test_QUICTypeUtil_CPPFLAGS = \
+  $(AM_CPPFLAGS)
+
+test_QUICTypeUtil_LDFLAGS = \
+  @AM_LDFLAGS@
+
+test_QUICTypeUtil_LDADD = \
+  $(top_builddir)/lib/ts/libtsutil.la
+
+test_QUICTypeUtil_SOURCES = \
+  main.cc \
+  test_QUICTypeUtil.cc \
+  ../QUICTypes.cc
+
+#
+# test_QUICAckFrameCreator
+#
+test_QUICAckFrameCreator_CPPFLAGS = \
+  $(AM_CPPFLAGS)
+
+test_QUICAckFrameCreator_LDFLAGS = \
+  @AM_LDFLAGS@
+
+test_QUICAckFrameCreator_LDADD = \
+  $(top_builddir)/iocore/eventsystem/libinkevent.a \
+  $(top_builddir)/lib/ts/libtsutil.la
+
+test_QUICAckFrameCreator_SOURCES = \
+  main.cc \
+  test_QUICAckFrameCreator.cc \
+  ../QUICAckFrameCreator.cc \
+  ../QUICTypes.cc \
+  ../QUICFrame.cc \
+  ../QUICPacket.cc \
+  ../QUICCrypto.cc \
+  $(QUICCrypto_impl)
+
+#
+# test_QUICTypeUtil
+#
+test_QUICVersionNegotiator_CPPFLAGS = \
+  $(AM_CPPFLAGS)
+
+test_QUICVersionNegotiator_LDFLAGS = \
+  @AM_LDFLAGS@
+
+test_QUICVersionNegotiator_LDADD = \
+  $(top_builddir)/lib/ts/libtsutil.la \
+  $(top_builddir)/iocore/eventsystem/libinkevent.a \
+  $(top_builddir)/proxy/shared/libUglyLogStubs.a \
+  @LIBTCL@ \
+  @HWLOC_LIBS@
+
+test_QUICVersionNegotiator_SOURCES = \
+  main.cc \
+  test_QUICVersionNegotiator.cc \
+  ../QUICTypes.cc \
+  ../QUICPacket.cc \
+  ../QUICCrypto.cc \
+  $(QUICCrypto_impl) \
+  ../QUICVersionNegotiator.cc
+
+include $(top_srcdir)/build/tidy.mk
+
+tidy-local: $(DIST_SOURCES)
+	$(CXX_Clang_Tidy)
diff --git a/iocore/net/quic/test/event_processor_main.cc b/iocore/net/quic/test/event_processor_main.cc
new file mode 100644
index 0000000..fbfead6
--- /dev/null
+++ b/iocore/net/quic/test/event_processor_main.cc
@@ -0,0 +1,53 @@
+/** @file
+ *
+ *  A brief file description
+ *
+ *  @section license License
+ *
+ *  Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+// To make compile faster
+// https://github.com/philsquared/Catch/blob/master/docs/slow-compiles.md
+#define CATCH_CONFIG_MAIN
+#include "catch.hpp"
+
+#include "I_EventSystem.h"
+#include "ts/ink_string.h"
+#include "ts/I_Layout.h"
+#include "diags.i"
+
+#define TEST_THREADS 1
+
+struct EventProcessorListener : Catch::TestEventListenerBase {
+  using TestEventListenerBase::TestEventListenerBase; // inherit constructor
+
+  virtual void
+  testRunStarting(Catch::TestRunInfo const &testRunInfo) override
+  {
+    Layout::create();
+    init_diags("", nullptr);
+    RecProcessInit(RECM_STAND_ALONE);
+
+    ink_event_system_init(EVENT_SYSTEM_MODULE_VERSION);
+    eventProcessor.start(TEST_THREADS);
+
+    Thread *main_thread = new EThread;
+    main_thread->set_specific();
+  }
+};
+CATCH_REGISTER_LISTENER(EventProcessorListener);
diff --git a/iocore/net/quic/test/main.cc b/iocore/net/quic/test/main.cc
new file mode 100644
index 0000000..213dec8
--- /dev/null
+++ b/iocore/net/quic/test/main.cc
@@ -0,0 +1,27 @@
+/** @file
+ *
+ *  A brief file description
+ *
+ *  @section license License
+ *
+ *  Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+// To make compile faster
+// https://github.com/philsquared/Catch/blob/master/docs/slow-compiles.md
+#define CATCH_CONFIG_MAIN
+#include "catch.hpp"
diff --git a/iocore/net/quic/test/test_QUICAckFrameCreator.cc b/iocore/net/quic/test/test_QUICAckFrameCreator.cc
new file mode 100644
index 0000000..a519687
--- /dev/null
+++ b/iocore/net/quic/test/test_QUICAckFrameCreator.cc
@@ -0,0 +1,69 @@
+/** @file
+ *
+ *  A brief file description
+ *
+ *  @section license License
+ *
+ *  Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+#include "catch.hpp"
+
+#include "quic/QUICAckFrameCreator.h"
+
+TEST_CASE("QUICAckFrameCreator", "[quic]")
+{
+  QUICAckFrameCreator creator;
+  std::unique_ptr<QUICAckFrame, QUICFrameDeleterFunc> frame = {nullptr, nullptr};
+
+  // Initial state
+  frame = creator.create();
+  CHECK(frame == nullptr);
+
+  // One packet
+  creator.update(1, true);
+  frame = creator.create();
+  CHECK(frame != nullptr);
+  CHECK(frame->num_blocks() == 0);
+  CHECK(frame->largest_acknowledged() == 1);
+  CHECK(frame->ack_block_section()->first_ack_block_length() == 1);
+
+  frame = creator.create();
+  CHECK(frame == nullptr);
+
+  // Not sequential
+  creator.update(2, true);
+  creator.update(5, true);
+  creator.update(3, true);
+  creator.update(4, true);
+  frame = creator.create();
+  CHECK(frame != nullptr);
+  CHECK(frame->num_blocks() == 0);
+  CHECK(frame->largest_acknowledged() == 5);
+  CHECK(frame->ack_block_section()->first_ack_block_length() == 4);
+
+  // Loss
+  creator.update(6, true);
+  creator.update(7, true);
+  creator.update(10, true);
+  frame = creator.create();
+  CHECK(frame != nullptr);
+  CHECK(frame->num_blocks() == 1);
+  CHECK(frame->largest_acknowledged() == 10);
+  CHECK(frame->ack_block_section()->first_ack_block_length() == 2);
+  CHECK(frame->ack_block_section()->begin()->gap() == 2);
+}
diff --git a/iocore/net/quic/test/test_QUICCrypto.cc b/iocore/net/quic/test/test_QUICCrypto.cc
new file mode 100644
index 0000000..f3c13c2
--- /dev/null
+++ b/iocore/net/quic/test/test_QUICCrypto.cc
@@ -0,0 +1,189 @@
+/** @file
+ *
+ *  A brief file description
+ *
+ *  @section license License
+ *
+ *  Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+#include "catch.hpp"
+
+#include <iostream>
+#include <fstream>
+#include <iomanip>
+
+#ifdef OPENSSL_IS_BORINGSSL
+#include <openssl/base.h>
+#endif
+
+#include <openssl/ssl.h>
+
+#include "QUICCrypto.h"
+
+const static uint32_t MAX_HANDSHAKE_MSG_LEN = 2048;
+
+static const char server_crt[] = "-----BEGIN CERTIFICATE-----\n"
+                                 "MIIDRjCCAi4CCQDoLSBwQxmcJTANBgkqhkiG9w0BAQsFADBlMQswCQYDVQQGEwJK\n"
+                                 "UDEOMAwGA1UECBMFVG9reW8xDzANBgNVBAcTBk1pbmF0bzEhMB8GA1UEChMYSW50\n"
+                                 "ZXJuZXQgV2lkZ2l0cyBQdHkgTHRkMRIwEAYDVQQDEwlsb2NhbGhvc3QwHhcNMTcw\n"
+                                 "MTE4MDEyMzA3WhcNMjcwMTE2MDEyMzA3WjBlMQswCQYDVQQGEwJKUDEOMAwGA1UE\n"
+                                 "CBMFVG9reW8xDzANBgNVBAcTBk1pbmF0bzEhMB8GA1UEChMYSW50ZXJuZXQgV2lk\n"
+                                 "Z2l0cyBQdHkgTHRkMRIwEAYDVQQDEwlsb2NhbGhvc3QwggEiMA0GCSqGSIb3DQEB\n"
+                                 "AQUAA4IBDwAwggEKAoIBAQC70j62KOWkuqNsDhl+7uqKFS6TMcJYLdYrH1YInwlY\n"
+                                 "htOMSMWx2hPSYYBKzVQpLvhe2LPbhLwcVJdq4aqQNjNpxrpxW/YIY5zxCRVgQsgf\n"
+                                 "KXiKgUR0G+F3MQHsm1YIqxQU2OeJldIZUBM2YMDp8h1CXTAvGaAZaXsqO9UvR2Zw\n"
+                                 "JZJ+GElYNlNwhdStqIM8v1JNFjfO3gWkVqTv+QM4fmpror2pp8CaDrueg4PrSY3Y\n"
+                                 "D/WG75rkmlrW26t0Q8fjkn+s/UiQ3V/IkP1+MfrJWH6RL2DGjBv2KfNAik42xWUi\n"
+                                 "KXzaNcDFN4hjqVG59O9bPnUDn1wPypY/TXB4iqSAlxupAgMBAAEwDQYJKoZIhvcN\n"
+                                 "AQELBQADggEBAKLc+P5YfusNYIkX3YE+gHBVpo95xnoVUcsGr/h1zanCkmsyKkNU\n"
+                                 "e2w9xsVnRLgpRfwrnwiaNP/k6cPYt5ePPCJjUfkO7Ql7DCcjLgEp8lrvxMmRIdSg\n"
+                                 "LPq+NdityxXYhfaZdGdXjnLLiq3zYL/8aYjjZ8YAZTuu6pBgfGvjcqYLV1ohimrP\n"
+                                 "8BW0BbnvedqTyL7tdKjdiWnHE5ObrxnphL2evoStskBr5CLYR4vX7+qp0oVSz2Ol\n"
+                                 "nBMV3wXyhHBY1tuT1SK7ajC/ZHrciZosACRV5PC6nKXi3shWOxt76SZV3HcMmFwX\n"
+                                 "NQYYTBOlb5U080adFSmP5/6NRzrKwZ3mD2s=\n"
+                                 "-----END CERTIFICATE-----\n";
+
+static const char server_key[] = "-----BEGIN RSA PRIVATE KEY-----\n"
+                                 "MIIEpAIBAAKCAQEAu9I+tijlpLqjbA4Zfu7qihUukzHCWC3WKx9WCJ8JWIbTjEjF\n"
+                                 "sdoT0mGASs1UKS74Xtiz24S8HFSXauGqkDYzaca6cVv2CGOc8QkVYELIHyl4ioFE\n"
+                                 "dBvhdzEB7JtWCKsUFNjniZXSGVATNmDA6fIdQl0wLxmgGWl7KjvVL0dmcCWSfhhJ\n"
+                                 "WDZTcIXUraiDPL9STRY3zt4FpFak7/kDOH5qa6K9qafAmg67noOD60mN2A/1hu+a\n"
+                                 "5Jpa1turdEPH45J/rP1IkN1fyJD9fjH6yVh+kS9gxowb9inzQIpONsVlIil82jXA\n"
+                                 "xTeIY6lRufTvWz51A59cD8qWP01weIqkgJcbqQIDAQABAoIBADI3ShEF6jAavmq7\n"
+                                 "clGfqxF0DFnKaf2Nc79fx27SpnsGwTS2mDSu67HJ47UcJK5GIp2pLp04ZdrlOv6W\n"
+                                 "izW3aBOV0G9SePtRNrqzBQYRlNPQEKxnV1f7xFJLxgnulhgHNX1FaNI+PkgKQri9\n"
+                                 "MZba5rvBkoplPYrNyuJF0P+tBVRiISWDY00PlZ57pQDyOvXzUckAkxmjNzo+86ld\n"
+                                 "/NyO+nR45vVKSeIBT5tT67D8wRisZgO/7QKP5sbKYwa7AR4sTEYFwBaFi4Mr6v1T\n"
+                                 "kp0KxOFBI+MioFwyK7ZjkoKClrY/K0IPsKfn2vmi6jLpfkA+qCl1JsVhrfVO3KJc\n"
+                                 "PXXF4QECgYEA9339GQS2AWSuA/9ZgHFqTTOEEHujCkh9D4mKO4LRi5hKPN9NQKUU\n"
+                                 "KgaBXWTbr8FwOTXw6HMl0SaIOdc6VxdzViNvPCpu2Wn8hyTC5Mjs/BtXkXNcBQqs\n"
+                                 "tPm0JxgC6fpQAb+gU+zZ+QQNlUWH/CEiQFxxGNzBn9E3Xq2j0StdhPECgYEAwkci\n"
+                                 "GiQuM4KMDdwbs4RDlEZyvXxWwgHKPoXv/Uq7HXtuT1FGb/+Rf3BGimMf2Qqmppp8\n"
+                                 "MAZ+xk+eXhtqKZHsV2ifhUfuVZ6NPhT2WRyn6MozuHh3MK4l2KtOhxulcoX/2sDk\n"
+                                 "dLYclxhXZFuXvbLz2KpgMmPMGyzEQNHQaoTkojkCgYEAxb/wVGY0OybD+EO2su9s\n"
+                                 "PaVU94qielvzOU/vmJ9taTnUz5Co/Gcqlm2+Pe6RrnxEfCICjOk8pUJBhN3ZKq99\n"
+                                 "I62Keqt5CNUrxpvz8bQtzz7VmE1xkEG4P55pePcxlNzBwrPnmkdc3yCC7euxvR6I\n"
+                                 "bJ6wa2owd89Gi6r4gvBAeDECgYBpdiPU/P73h05v16RR9uKYgwWWRwDxn/chqaN1\n"
+                                 "ZDPe9ToUZJJQCfP5sgEY7mZDc7yzg/kWOPBoxp+5hjhDCKu7Z1fxCfMfF0qlAMwZ\n"
+                                 "46xieiFJaluJWX/B9nxSa3eMi6EwJrXdhV5Pxy7pk67zk0k7vIEr2XDa75o5dawl\n"
+                                 "pq5WQQKBgQC9xsRLtQjnDEdNEgCicTupa7BXmvc9tRb1mA5SeqjwzYuulrTyvn5Y\n"
+                                 "QOXYdz8aeZ+ZQ/cDeGA3jA6lekWnExkp9enHeqadyDWM7rvXi800E6gB/vrO7r/c\n"
+                                 "iE+fpXud6cwNw2XYsk6RBSQ8qhJoCpa+koPXfSJOZ9Y89NMbtq0w3Q==\n"
+                                 "-----END RSA PRIVATE KEY-----\n";
+
+void
+print_hex(const uint8_t *v, size_t len)
+{
+  for (size_t i = 0; i < len; i++) {
+    std::cout << std::setw(2) << std::setfill('0') << std::hex << static_cast<uint32_t>(v[i]) << " ";
+
+    if (i != 0 && (i + 1) % 32 == 0 && i != len - 1) {
+      std::cout << std::endl;
+    }
+  }
+
+  std::cout << std::endl;
+
+  return;
+}
+
+TEST_CASE("QUICCrypto 1-RTT", "[quic]")
+{
+  // Client
+  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(client_ssl_ctx, NET_VCONNECTION_OUT);
+
+  // Server
+  SSL_CTX *server_ssl_ctx = SSL_CTX_new(TLS_method());
+  SSL_CTX_set_min_proto_version(server_ssl_ctx, TLS1_3_VERSION);
+  SSL_CTX_set_max_proto_version(server_ssl_ctx, TLS1_3_VERSION);
+  BIO *crt_bio(BIO_new_mem_buf(server_crt, sizeof(server_crt)));
+  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(server_ssl_ctx, NET_VCONNECTION_IN);
+
+  // Client Hello
+  uint8_t client_hello[MAX_HANDSHAKE_MSG_LEN] = {0};
+  size_t client_hello_len                     = 0;
+  CHECK(client->handshake(client_hello, client_hello_len, MAX_HANDSHAKE_MSG_LEN, nullptr, 0));
+  std::cout << "Client Hello" << std::endl;
+  print_hex(client_hello, client_hello_len);
+
+  // Server Hello
+  uint8_t server_hello[MAX_HANDSHAKE_MSG_LEN] = {0};
+  size_t server_hello_len                     = 0;
+  CHECK(server->handshake(server_hello, server_hello_len, MAX_HANDSHAKE_MSG_LEN, client_hello, client_hello_len));
+  std::cout << "Server Hello" << std::endl;
+  print_hex(server_hello, server_hello_len);
+
+  // Client Fnished
+  uint8_t client_finished[MAX_HANDSHAKE_MSG_LEN] = {0};
+  size_t client_finished_len                     = 0;
+  CHECK(client->handshake(client_finished, client_finished_len, MAX_HANDSHAKE_MSG_LEN, server_hello, server_hello_len));
+  std::cout << "Client Finished" << std::endl;
+  print_hex(client_finished, client_finished_len);
+
+  // Post Handshake Msg
+  uint8_t post_handshake_msg[MAX_HANDSHAKE_MSG_LEN] = {0};
+  size_t post_handshake_msg_len                     = 0;
+  CHECK(server->handshake(post_handshake_msg, post_handshake_msg_len, MAX_HANDSHAKE_MSG_LEN, client_finished, client_finished_len));
+  std::cout << "Post Handshake Message" << std::endl;
+  print_hex(post_handshake_msg, post_handshake_msg_len);
+
+  CHECK(client->setup_session());
+  CHECK(server->setup_session());
+
+  // encrypt - decrypt
+  uint8_t original[] = {
+    0x41, 0x70, 0x61, 0x63, 0x68, 0x65, 0x20, 0x54, 0x72, 0x61, 0x66, 0x66, 0x69, 0x63, 0x20, 0x53,
+    0x65, 0x72, 0x76, 0x65, 0x72, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00,
+    0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00,
+    0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00,
+  };
+  uint64_t pkt_num = 0x123456789;
+  uint8_t ad[]     = {0x01, 0x02, 0x03, 0x04, 0x05, 0x06, 0x07, 0x08, 0x09, 0x0a, 0x0b, 0x0c, 0x0d, 0x0e, 0x0f};
+
+  // client (encrypt) - server (decrypt)
+  std::cout << "Original Text" << std::endl;
+  print_hex(original, sizeof(original));
+
+  uint8_t cipher[128] = {0}; // >= original len + EVP_AEAD_max_overhead
+  size_t cipher_len   = 0;
+  CHECK(client->encrypt(cipher, cipher_len, sizeof(cipher), original, sizeof(original), pkt_num, ad, sizeof(ad),
+                        QUICKeyPhase::PHASE_0));
+
+  std::cout << "Encrypted Text" << std::endl;
+  print_hex(cipher, cipher_len);
+
+  uint8_t plain[128] = {0};
+  size_t plain_len   = 0;
+  CHECK(server->decrypt(plain, plain_len, sizeof(plain), cipher, cipher_len, pkt_num, ad, sizeof(ad), QUICKeyPhase::PHASE_0));
+
+  std::cout << "Decrypted Text" << std::endl;
+  print_hex(plain, plain_len);
+
+  CHECK(sizeof(original) == (plain_len));
+  CHECK(memcmp(original, plain, plain_len) == 0);
+
+  // Teardown
+  delete client;
+  delete server;
+}
diff --git a/iocore/net/quic/test/test_QUICFrame.cc b/iocore/net/quic/test/test_QUICFrame.cc
new file mode 100644
index 0000000..2fbc6a9
--- /dev/null
+++ b/iocore/net/quic/test/test_QUICFrame.cc
@@ -0,0 +1,607 @@
+/** @file
+ *
+ *  A brief file description
+ *
+ *  @section license License
+ *
+ *  Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+#include "catch.hpp"
+
+#include "quic/QUICFrame.h"
+
+TEST_CASE("QUICFrame Type", "[quic]")
+{
+  CHECK(QUICFrame::type(reinterpret_cast<const uint8_t *>("\x00")) == QUICFrameType::PADDING);
+  CHECK(QUICFrame::type(reinterpret_cast<const uint8_t *>("\x01")) == QUICFrameType::RST_STREAM);
+  CHECK(QUICFrame::type(reinterpret_cast<const uint8_t *>("\x02")) == QUICFrameType::CONNECTION_CLOSE);
+  CHECK(QUICFrame::type(reinterpret_cast<const uint8_t *>("\x03")) == QUICFrameType::GOAWAY);
+  CHECK(QUICFrame::type(reinterpret_cast<const uint8_t *>("\x04")) == QUICFrameType::MAX_DATA);
+  CHECK(QUICFrame::type(reinterpret_cast<const uint8_t *>("\x05")) == QUICFrameType::MAX_STREAM_DATA);
+  CHECK(QUICFrame::type(reinterpret_cast<const uint8_t *>("\x06")) == QUICFrameType::MAX_STREAM_ID);
+  CHECK(QUICFrame::type(reinterpret_cast<const uint8_t *>("\x07")) == QUICFrameType::PING);
+  CHECK(QUICFrame::type(reinterpret_cast<const uint8_t *>("\x08")) == QUICFrameType::BLOCKED);
+  CHECK(QUICFrame::type(reinterpret_cast<const uint8_t *>("\x09")) == QUICFrameType::STREAM_BLOCKED);
+  CHECK(QUICFrame::type(reinterpret_cast<const uint8_t *>("\x0a")) == QUICFrameType::STREAM_ID_NEEDED);
+  CHECK(QUICFrame::type(reinterpret_cast<const uint8_t *>("\x0b")) == QUICFrameType::NEW_CONNECTION_ID);
+  // Undefined ragne
+  CHECK(QUICFrame::type(reinterpret_cast<const uint8_t *>("\x0c")) == QUICFrameType::UNKNOWN);
+  CHECK(QUICFrame::type(reinterpret_cast<const uint8_t *>("\x9f")) == QUICFrameType::UNKNOWN);
+  // Range of ACK
+  CHECK(QUICFrame::type(reinterpret_cast<const uint8_t *>("\xa0")) == QUICFrameType::ACK);
+  CHECK(QUICFrame::type(reinterpret_cast<const uint8_t *>("\xbf")) == QUICFrameType::ACK);
+  // Range of STREAM
+  CHECK(QUICFrame::type(reinterpret_cast<const uint8_t *>("\xc0")) == QUICFrameType::STREAM);
+  CHECK(QUICFrame::type(reinterpret_cast<const uint8_t *>("\xff")) == QUICFrameType::STREAM);
+}
+
+TEST_CASE("Construct QUICFrame", "[quic]")
+{
+  uint8_t payload[] = "foo";
+  uint8_t buf[65536];
+  size_t len;
+
+  QUICStreamFrame frame1(payload, sizeof(payload), 0xffcc9966, 0xffddbb9977553311);
+  frame1.store(buf, &len);
+  CHECK(frame1.type() == QUICFrameType::STREAM);
+  CHECK(frame1.size() == 19);
+  CHECK(frame1.stream_id() == 0xffcc9966);
+  CHECK(frame1.offset() == 0xffddbb9977553311);
+  CHECK(frame1.data_length() == 4);
+  CHECK(memcmp(frame1.data(), "foo\0", 4) == 0);
+  CHECK(frame1.has_fin_flag() == false);
+}
+
+TEST_CASE("Load STREAM Frame 1", "[quic]")
+{
+  uint8_t buf1[] = {
+    0xC0,                   // 11FSSOOD
+    0x01,                   // Stream ID
+    0x01, 0x02, 0x03, 0x04, // Stream Data
+  };
+  std::shared_ptr<const QUICFrame> frame1 = QUICFrameFactory::create(buf1, sizeof(buf1));
+  CHECK(frame1->type() == QUICFrameType::STREAM);
+  CHECK(frame1->size() == 6);
+  std::shared_ptr<const QUICStreamFrame> streamFrame1 = std::dynamic_pointer_cast<const QUICStreamFrame>(frame1);
+  CHECK(streamFrame1->stream_id() == 0x01);
+  CHECK(streamFrame1->offset() == 0x00);
+  CHECK(streamFrame1->data_length() == 4);
+  CHECK(memcmp(streamFrame1->data(), "\x01\x02\x03\x04", 4) == 0);
+  CHECK(streamFrame1->has_fin_flag() == false);
+}
+
+TEST_CASE("Load STREAM Frame 2", "[quic]")
+{
+  uint8_t buf1[] = {
+    0xC1,                         // 11FSSOOD
+    0x01,                         // Stream ID
+    0x00, 0x05,                   // Data Length
+    0x01, 0x02, 0x03, 0x04, 0x05, // Stream Data
+  };
+  std::shared_ptr<const QUICFrame> frame1 = QUICFrameFactory::create(buf1, sizeof(buf1));
+  CHECK(frame1->type() == QUICFrameType::STREAM);
+  CHECK(frame1->size() == 9);
+  std::shared_ptr<const QUICStreamFrame> streamFrame1 = std::dynamic_pointer_cast<const QUICStreamFrame>(frame1);
+  CHECK(streamFrame1->stream_id() == 0x01);
+  CHECK(streamFrame1->offset() == 0x00);
+  CHECK(streamFrame1->data_length() == 5);
+  CHECK(memcmp(streamFrame1->data(), "\x01\x02\x03\x04\x05", 5) == 0);
+  CHECK(streamFrame1->has_fin_flag() == false);
+}
+
+TEST_CASE("Store STREAM Frame", "[quic]")
+{
+  uint8_t buf[65535];
+  size_t len;
+
+  // 32bit stream id, 64bit offset
+  uint8_t expected[] = {
+    0xDF,                                           // 11FSSOOD
+    0x00, 0x00, 0x00, 0x01,                         // Stream ID
+    0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, // Offset
+    0x00, 0x05,                                     // Data Length
+    0x01, 0x02, 0x03, 0x04, 0x05,                   // Stream Data
+  };
+  QUICStreamFrame streamFrame(reinterpret_cast<const uint8_t *>("\x01\x02\x03\x04\x05"), 5, 0x01, 0x00);
+  streamFrame.store(buf, &len);
+  CHECK(len == 20);
+  CHECK(memcmp(buf, expected, len) == 0);
+}
+
+TEST_CASE("Load Ack Frame 1", "[quic]")
+{
+  // 0 Ack Block, 0 Timestamp, 8 bit packet number length, 8 bit block length
+  uint8_t buf1[] = {
+    0xA0,       // 101NLLMM
+    0x00,       // NumTS
+    0x12,       // Largest Acknowledged
+    0x34, 0x56, // Ack Delay
+    0x00,       // Ack Block Section
+  };
+  std::shared_ptr<const QUICFrame> frame1 = QUICFrameFactory::create(buf1, sizeof(buf1));
+  CHECK(frame1->type() == QUICFrameType::ACK);
+  CHECK(frame1->size() == 6);
+  std::shared_ptr<const QUICAckFrame> ackFrame1 = std::dynamic_pointer_cast<const QUICAckFrame>(frame1);
+  CHECK(ackFrame1 != nullptr);
+  CHECK(ackFrame1->has_ack_blocks() == false);
+  CHECK(ackFrame1->num_timestamps() == 0);
+  CHECK(ackFrame1->largest_acknowledged() == 0x12);
+  CHECK(ackFrame1->ack_delay() == 0x3456);
+
+  // TODO: 1 Ack Block, 0 Timestamp
+  // TODO: 1 Ack Block, 1 Timestamp
+}
+
+TEST_CASE("Load Ack Frame 2", "[quic]")
+{
+  // 0 Ack Block, 0 Timestamp, 8 bit packet number length, 8 bit block length
+  uint8_t buf1[] = {
+    0xAA,                   // 101NLLMM '0b10101010' { N: 0, LL: 10, MM:10 }
+    0x00,                   // NumTS
+    0x00, 0x00, 0x00, 0x01, // Largest Acknowledged
+    0x01, 0x71,             // Ack Delay
+    0x00, 0x00, 0x00, 0x01, // ACK Block
+  };
+  std::shared_ptr<const QUICFrame> frame1 = QUICFrameFactory::create(buf1, sizeof(buf1));
+  CHECK(frame1->type() == QUICFrameType::ACK);
+  CHECK(frame1->size() == 12);
+  std::shared_ptr<const QUICAckFrame> ackFrame1 = std::dynamic_pointer_cast<const QUICAckFrame>(frame1);
+  CHECK(ackFrame1 != nullptr);
+  CHECK(ackFrame1->has_ack_blocks() == false);
+  CHECK(ackFrame1->num_timestamps() == 0);
+  CHECK(ackFrame1->largest_acknowledged() == 0x01);
+  CHECK(ackFrame1->ack_delay() == 0x0171);
+
+  // TODO: 1 Ack Block, 0 Timestamp
+  // TODO: 1 Ack Block, 1 Timestamp
+}
+
+TEST_CASE("Store Ack Frame", "[quic]")
+{
+  uint8_t buf[65535];
+  size_t len;
+
+  // 0 Ack Block, 0 Timestamp, 8 bit packet number length, 8 bit block length
+  uint8_t expected[] = {
+    0xA2,                   // 101NLLMM
+    0x00,                   // NumTS
+    0x12,                   // Largest Acknowledged
+    0x34, 0x56,             // Ack Delay
+    0x00, 0x00, 0x00, 0x00, // Ack Block Section
+  };
+  QUICAckFrame ackFrame(0x12, 0x3456, 0);
+  ackFrame.store(buf, &len);
+  CHECK(len == 9);
+  CHECK(memcmp(buf, expected, len) == 0);
+
+  // TODO: Add ack blocks and timestamps
+}
+
+TEST_CASE("Load RST_STREAM Frame", "[quic]")
+{
+  uint8_t buf1[] = {
+    0x01,                                          // Type
+    0x80, 0x00, 0x00, 0x00,                        // Error Code
+    0x12, 0x34, 0x56, 0x78,                        // Stream ID
+    0x11, 0x22, 0x33, 0x44, 0x55, 0x66, 0x77, 0x88 // Final Offset
+  };
+  std::shared_ptr<const QUICFrame> frame1 = QUICFrameFactory::create(buf1, sizeof(buf1));
+  CHECK(frame1->type() == QUICFrameType::RST_STREAM);
+  CHECK(frame1->size() == 17);
+  std::shared_ptr<const QUICRstStreamFrame> rstStreamFrame1 = std::dynamic_pointer_cast<const QUICRstStreamFrame>(frame1);
+  CHECK(rstStreamFrame1 != nullptr);
+  CHECK(rstStreamFrame1->error_code() == QUICErrorCode::QUIC_TRANSPORT_ERROR);
+  CHECK(rstStreamFrame1->stream_id() == 0x12345678);
+  CHECK(rstStreamFrame1->final_offset() == 0x1122334455667788);
+}
+
+TEST_CASE("Store RST_STREAM Frame", "[quic]")
+{
+  uint8_t buf[65535];
+  size_t len;
+
+  uint8_t expected[] = {
+    0x01,                                          // Type
+    0x80, 0x00, 0x00, 0x00,                        // Error Code
+    0x12, 0x34, 0x56, 0x78,                        // Stream ID
+    0x11, 0x22, 0x33, 0x44, 0x55, 0x66, 0x77, 0x88 // Final Offset
+  };
+  QUICRstStreamFrame rstStreamFrame(QUICErrorCode::QUIC_TRANSPORT_ERROR, 0x12345678, 0x1122334455667788);
+  rstStreamFrame.store(buf, &len);
+  CHECK(len == 17);
+  CHECK(memcmp(buf, expected, len) == 0);
+}
+
+TEST_CASE("Load Ping Frame", "[quic]")
+{
+  uint8_t buf1[] = {
+    0x07, // Type
+  };
+  std::shared_ptr<const QUICFrame> frame1 = QUICFrameFactory::create(buf1, sizeof(buf1));
+  CHECK(frame1->type() == QUICFrameType::PING);
+  CHECK(frame1->size() == 1);
+  std::shared_ptr<const QUICPingFrame> pingStreamFrame1 = std::dynamic_pointer_cast<const QUICPingFrame>(frame1);
+  CHECK(pingStreamFrame1 != nullptr);
+}
+
+TEST_CASE("Store Ping Frame", "[quic]")
+{
+  uint8_t buf[65535];
+  size_t len;
+
+  uint8_t expected[] = {
+    0x07, // Type
+  };
+  QUICPingFrame pingStreamFrame;
+  pingStreamFrame.store(buf, &len);
+  CHECK(len == 1);
+  CHECK(memcmp(buf, expected, len) == 0);
+}
+
+TEST_CASE("Load Padding Frame", "[quic]")
+{
+  uint8_t buf1[] = {
+    0x00, // Type
+  };
+  std::shared_ptr<const QUICFrame> frame1 = QUICFrameFactory::create(buf1, sizeof(buf1));
+  CHECK(frame1->type() == QUICFrameType::PADDING);
+  CHECK(frame1->size() == 1);
+  std::shared_ptr<const QUICPaddingFrame> paddingFrame1 = std::dynamic_pointer_cast<const QUICPaddingFrame>(frame1);
+  CHECK(paddingFrame1 != nullptr);
+}
+
+TEST_CASE("Store Padding Frame", "[quic]")
+{
+  uint8_t buf[65535];
+  size_t len;
+
+  uint8_t expected[] = {
+    0x00, // Type
+  };
+  QUICPaddingFrame paddingFrame;
+  paddingFrame.store(buf, &len);
+  CHECK(len == 1);
+  CHECK(memcmp(buf, expected, len) == 0);
+}
+
+TEST_CASE("Load Goaway Frame", "[quic]")
+{
+  uint8_t buf1[] = {
+    0x03,                   // Type
+    0x12, 0x34, 0x56, 0x78, // Latest Client Stream ID
+    0xAA, 0xBB, 0xCC, 0xDD, // Latest Server Stream ID
+  };
+  std::shared_ptr<const QUICFrame> frame1 = QUICFrameFactory::create(buf1, sizeof(buf1));
+  CHECK(frame1->type() == QUICFrameType::GOAWAY);
+  CHECK(frame1->size() == 9);
+  std::shared_ptr<const QUICGoawayFrame> goawayFrame1 = std::dynamic_pointer_cast<const QUICGoawayFrame>(frame1);
+  CHECK(goawayFrame1 != nullptr);
+  CHECK(goawayFrame1->client_stream_id() == 0x12345678);
+  CHECK(goawayFrame1->server_stream_id() == 0xAABBCCDD);
+}
+
+TEST_CASE("Store Goaway Frame", "[quic]")
+{
+  uint8_t buf[65535];
+  size_t len;
+
+  uint8_t expected[] = {
+    0x03,                   // Type
+    0x12, 0x34, 0x56, 0x78, // Latest Client Stream ID
+    0xAA, 0xBB, 0xCC, 0xDD, // Latest Server Stream ID
+  };
+  QUICGoawayFrame goawayFrame(0x12345678, 0xAABBCCDD);
+  goawayFrame.store(buf, &len);
+  CHECK(len == 9);
+  CHECK(memcmp(buf, expected, len) == 0);
+}
+
+TEST_CASE("Load ConnectionClose Frame", "[quic]")
+{
+  uint8_t buf1[] = {
+    0x02,                        // Type
+    0x80, 0x00, 0x00, 0x00,      // Error Code
+    0x00, 0x05,                  // Reason Phrase Length
+    0x41, 0x42, 0x43, 0x44, 0x45 // Reason Phrase ("ABCDE");
+  };
+  std::shared_ptr<const QUICFrame> frame1 = QUICFrameFactory::create(buf1, sizeof(buf1));
+  CHECK(frame1->type() == QUICFrameType::CONNECTION_CLOSE);
+  CHECK(frame1->size() == 12);
+  std::shared_ptr<const QUICConnectionCloseFrame> connectionCloseFrame1 =
+    std::dynamic_pointer_cast<const QUICConnectionCloseFrame>(frame1);
+  CHECK(connectionCloseFrame1 != nullptr);
+  CHECK(connectionCloseFrame1->error_code() == QUICErrorCode::QUIC_TRANSPORT_ERROR);
+  CHECK(connectionCloseFrame1->reason_phrase_length() == 5);
+  CHECK(memcmp(connectionCloseFrame1->reason_phrase(), buf1 + 7, 5) == 0);
+}
+
+TEST_CASE("Store ConnectionClose Frame", "[quic]")
+{
+  uint8_t buf[65535];
+  size_t len;
+
+  uint8_t expected[] = {
+    0x02,                        // Type
+    0x80, 0x00, 0x00, 0x00,      // Error Code
+    0x00, 0x05,                  // Reason Phrase Length
+    0x41, 0x42, 0x43, 0x44, 0x45 // Reason Phrase ("ABCDE");
+  };
+  QUICConnectionCloseFrame connectionCloseFrame(QUICErrorCode::QUIC_TRANSPORT_ERROR, 5, "ABCDE");
+  connectionCloseFrame.store(buf, &len);
+  CHECK(len == 12);
+  CHECK(memcmp(buf, expected, len) == 0);
+}
+
+TEST_CASE("Load MaxData Frame", "[quic]")
+{
+  uint8_t buf1[] = {
+    0x04,                                          // Type
+    0x11, 0x22, 0x33, 0x44, 0x55, 0x66, 0x77, 0x88 // Maximum Data
+  };
+  std::shared_ptr<const QUICFrame> frame1 = QUICFrameFactory::create(buf1, sizeof(buf1));
+  CHECK(frame1->type() == QUICFrameType::MAX_DATA);
+  CHECK(frame1->size() == 9);
+  std::shared_ptr<const QUICMaxDataFrame> maxDataFrame1 = std::dynamic_pointer_cast<const QUICMaxDataFrame>(frame1);
+  CHECK(maxDataFrame1 != nullptr);
+  CHECK(maxDataFrame1->maximum_data() == 0x1122334455667788ULL);
+}
+
+TEST_CASE("Store MaxData Frame", "[quic]")
+{
+  uint8_t buf[65535];
+  size_t len;
+
+  uint8_t expected[] = {
+    0x04,                                          // Type
+    0x11, 0x22, 0x33, 0x44, 0x55, 0x66, 0x77, 0x88 // Maximum Data
+  };
+  QUICMaxDataFrame maxDataFrame(0x1122334455667788);
+  maxDataFrame.store(buf, &len);
+  CHECK(len == 9);
+  CHECK(memcmp(buf, expected, len) == 0);
+}
+
+TEST_CASE("Load MaxStreamData Frame", "[quic]")
+{
+  uint8_t buf1[] = {
+    0x05,                                          // Type
+    0x01, 0x02, 0x03, 0x04,                        // Stream ID
+    0x11, 0x22, 0x33, 0x44, 0x55, 0x66, 0x77, 0x88 // Maximum Stream Data
+  };
+  std::shared_ptr<const QUICFrame> frame1 = QUICFrameFactory::create(buf1, sizeof(buf1));
+  CHECK(frame1->type() == QUICFrameType::MAX_STREAM_DATA);
+  CHECK(frame1->size() == 13);
+  std::shared_ptr<const QUICMaxStreamDataFrame> maxStreamDataFrame1 =
+    std::dynamic_pointer_cast<const QUICMaxStreamDataFrame>(frame1);
+  CHECK(maxStreamDataFrame1 != nullptr);
+  CHECK(maxStreamDataFrame1->stream_id() == 0x01020304);
+  CHECK(maxStreamDataFrame1->maximum_stream_data() == 0x1122334455667788ULL);
+}
+
+TEST_CASE("Store MaxStreamData Frame", "[quic]")
+{
+  uint8_t buf[65535];
+  size_t len;
+
+  uint8_t expected[] = {
+    0x05,                                          // Type
+    0x01, 0x02, 0x03, 0x04,                        // Stream ID
+    0x11, 0x22, 0x33, 0x44, 0x55, 0x66, 0x77, 0x88 // Maximum Stream Data
+  };
+  QUICMaxStreamDataFrame maxStreamDataFrame(0x01020304, 0x1122334455667788ULL);
+  maxStreamDataFrame.store(buf, &len);
+  CHECK(len == 13);
+  CHECK(memcmp(buf, expected, len) == 0);
+}
+
+TEST_CASE("Load MaxStreamId Frame", "[quic]")
+{
+  uint8_t buf1[] = {
+    0x06,                   // Type
+    0x01, 0x02, 0x03, 0x04, // Stream ID
+  };
+  std::shared_ptr<const QUICFrame> frame1 = QUICFrameFactory::create(buf1, sizeof(buf1));
+  CHECK(frame1->type() == QUICFrameType::MAX_STREAM_ID);
+  CHECK(frame1->size() == 5);
+  std::shared_ptr<const QUICMaxStreamIdFrame> maxStreamIdFrame1 = std::dynamic_pointer_cast<const QUICMaxStreamIdFrame>(frame1);
+  CHECK(maxStreamIdFrame1 != nullptr);
+  CHECK(maxStreamIdFrame1->maximum_stream_id() == 0x01020304);
+}
+
+TEST_CASE("Store MaxStreamId Frame", "[quic]")
+{
+  uint8_t buf[65535];
+  size_t len;
+
+  uint8_t expected[] = {
+    0x06,                   // Type
+    0x01, 0x02, 0x03, 0x04, // Stream ID
+  };
+  QUICMaxStreamIdFrame maxStreamIdFrame(0x01020304);
+  maxStreamIdFrame.store(buf, &len);
+  CHECK(len == 5);
+  CHECK(memcmp(buf, expected, len) == 0);
+}
+
+TEST_CASE("Load Blocked Frame", "[quic]")
+{
+  uint8_t buf1[] = {
+    0x08, // Type
+  };
+  std::shared_ptr<const QUICFrame> frame1 = QUICFrameFactory::create(buf1, sizeof(buf1));
+  CHECK(frame1->type() == QUICFrameType::BLOCKED);
+  CHECK(frame1->size() == 1);
+  std::shared_ptr<const QUICBlockedFrame> blockedStreamFrame1 = std::dynamic_pointer_cast<const QUICBlockedFrame>(frame1);
+  CHECK(blockedStreamFrame1 != nullptr);
+}
+
+TEST_CASE("Store Blocked Frame", "[quic]")
+{
+  uint8_t buf[65535];
+  size_t len;
+
+  uint8_t expected[] = {
+    0x08, // Type
+  };
+  QUICBlockedFrame blockedStreamFrame;
+  blockedStreamFrame.store(buf, &len);
+  CHECK(len == 1);
+  CHECK(memcmp(buf, expected, len) == 0);
+}
+
+TEST_CASE("Load StreamBlocked Frame", "[quic]")
+{
+  uint8_t buf1[] = {
+    0x09,                   // Type
+    0x01, 0x02, 0x03, 0x04, // Stream ID
+  };
+  std::shared_ptr<const QUICFrame> frame1 = QUICFrameFactory::create(buf1, sizeof(buf1));
+  CHECK(frame1->type() == QUICFrameType::STREAM_BLOCKED);
+  CHECK(frame1->size() == 5);
+  std::shared_ptr<const QUICStreamBlockedFrame> streamBlockedFrame1 =
+    std::dynamic_pointer_cast<const QUICStreamBlockedFrame>(frame1);
+  CHECK(streamBlockedFrame1 != nullptr);
+  CHECK(streamBlockedFrame1->stream_id() == 0x01020304);
+}
+
+TEST_CASE("Store StreamBlocked Frame", "[quic]")
+{
+  uint8_t buf[65535];
+  size_t len;
+
+  uint8_t expected[] = {
+    0x09,                   // Type
+    0x01, 0x02, 0x03, 0x04, // Stream ID
+  };
+  QUICStreamBlockedFrame streamBlockedFrame(0x01020304);
+  streamBlockedFrame.store(buf, &len);
+  CHECK(len == 5);
+  CHECK(memcmp(buf, expected, len) == 0);
+}
+
+TEST_CASE("Load StreamIdNeeded Frame", "[quic]")
+{
+  uint8_t buf1[] = {
+    0x0a, // Type
+  };
+  std::shared_ptr<const QUICFrame> frame1 = QUICFrameFactory::create(buf1, sizeof(buf1));
+  CHECK(frame1->type() == QUICFrameType::STREAM_ID_NEEDED);
+  CHECK(frame1->size() == 1);
+  std::shared_ptr<const QUICStreamIdNeededFrame> streamIdNeededFrame1 =
+    std::dynamic_pointer_cast<const QUICStreamIdNeededFrame>(frame1);
+  CHECK(streamIdNeededFrame1 != nullptr);
+}
+
+TEST_CASE("Store StreamIdNeeded Frame", "[quic]")
+{
+  uint8_t buf[65535];
+  size_t len;
+
+  uint8_t expected[] = {
+    0x0a, // Type
+  };
+  QUICStreamIdNeededFrame streamIdNeededStreamFrame;
+  streamIdNeededStreamFrame.store(buf, &len);
+  CHECK(len == 1);
+  CHECK(memcmp(buf, expected, len) == 0);
+}
+
+TEST_CASE("Load NewConnectionId Frame", "[quic]")
+{
+  uint8_t buf1[] = {
+    0x0b,                                          // Type
+    0x01, 0x02,                                    // Sequence
+    0x11, 0x22, 0x33, 0x44, 0x55, 0x66, 0x77, 0x88 // Connection ID
+  };
+  std::shared_ptr<const QUICFrame> frame1 = QUICFrameFactory::create(buf1, sizeof(buf1));
+  CHECK(frame1->type() == QUICFrameType::NEW_CONNECTION_ID);
+  CHECK(frame1->size() == 11);
+  std::shared_ptr<const QUICNewConnectionIdFrame> newConnectionIdFrame1 =
+    std::dynamic_pointer_cast<const QUICNewConnectionIdFrame>(frame1);
+  CHECK(newConnectionIdFrame1 != nullptr);
+  CHECK(newConnectionIdFrame1->sequence() == 0x0102);
+  CHECK(newConnectionIdFrame1->connection_id() == 0x1122334455667788ULL);
+}
+
+TEST_CASE("Store NewConnectionId Frame", "[quic]")
+{
+  uint8_t buf[65535];
+  size_t len;
+
+  uint8_t expected[] = {
+    0x0b,                                          // Type
+    0x01, 0x02,                                    // Sequence
+    0x11, 0x22, 0x33, 0x44, 0x55, 0x66, 0x77, 0x88 // Connection ID
+  };
+  QUICNewConnectionIdFrame newConnectionIdFrame(0x0102, 0x1122334455667788ULL);
+  newConnectionIdFrame.store(buf, &len);
+  CHECK(len == 11);
+  CHECK(memcmp(buf, expected, len) == 0);
+}
+
+TEST_CASE("QUICFrameFactory Create Unknown Frame", "[quic]")
+{
+  uint8_t buf1[] = {
+    0x0f, // Type
+  };
+  std::shared_ptr<const QUICFrame> frame1 = QUICFrameFactory::create(buf1, sizeof(buf1));
+  CHECK(frame1 == nullptr);
+}
+
+TEST_CASE("QUICFrameFactory Fast Create Frame", "[quic]")
+{
+  QUICFrameFactory factory;
+
+  uint8_t buf1[] = {
+    0x03,                   // Type
+    0x12, 0x34, 0x56, 0x78, // Latest Client Stream ID
+    0xAA, 0xBB, 0xCC, 0xDD, // Latest Server Stream ID
+  };
+  uint8_t buf2[] = {
+    0x03,                   // Type
+    0x11, 0x22, 0x33, 0x44, // Latest Client Stream ID
+    0x0A, 0x0B, 0x0C, 0x0D, // Latest Server Stream ID
+  };
+  std::shared_ptr<const QUICFrame> frame1 = factory.fast_create(buf1, sizeof(buf1));
+  CHECK(frame1 != nullptr);
+
+  std::shared_ptr<const QUICGoawayFrame> goawayFrame1 = std::dynamic_pointer_cast<const QUICGoawayFrame>(frame1);
+  CHECK(goawayFrame1 != nullptr);
+  CHECK(goawayFrame1->client_stream_id() == 0x12345678);
+  CHECK(goawayFrame1->server_stream_id() == 0xAABBCCDD);
+
+  std::shared_ptr<const QUICFrame> frame2 = factory.fast_create(buf2, sizeof(buf2));
+  CHECK(frame2 != nullptr);
+  std::shared_ptr<const QUICGoawayFrame> goawayFrame2 = std::dynamic_pointer_cast<const QUICGoawayFrame>(frame2);
+  CHECK(goawayFrame2 != nullptr);
+  CHECK(goawayFrame2->client_stream_id() == 0x11223344);
+  CHECK(goawayFrame2->server_stream_id() == 0x0A0B0C0D);
+
+  CHECK(frame1 == frame2);
+}
+
+TEST_CASE("QUICFrameFactory Fast Create Unknown Frame", "[quic]")
+{
+  QUICFrameFactory factory;
+
+  uint8_t buf1[] = {
+    0x0f, // Type
+  };
+  std::shared_ptr<const QUICFrame> frame1 = factory.fast_create(buf1, sizeof(buf1));
+  CHECK(frame1 == nullptr);
+}
diff --git a/iocore/net/quic/test/test_QUICFrameDispatcher.cc b/iocore/net/quic/test/test_QUICFrameDispatcher.cc
new file mode 100644
index 0000000..a406a45
--- /dev/null
+++ b/iocore/net/quic/test/test_QUICFrameDispatcher.cc
@@ -0,0 +1,74 @@
+/** @file
+ *
+ *  A brief file description
+ *
+ *  @section license License
+ *
+ *  Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+#include "catch.hpp"
+
+#include "quic/QUICFrameDispatcher.h"
+#include "quic/Mock.h"
+#include <memory>
+
+TEST_CASE("QUICFrameHandler", "[quic]")
+{
+  uint8_t payload[] = {0x01};
+  QUICStreamFrame streamFrame(payload, 1, 0x03, 0);
+
+  auto connectionManager    = std::make_shared<MockQUICConnectionManager>();
+  auto streamManager        = std::make_shared<MockQUICStreamManager>();
+  auto flowController       = std::make_shared<MockQUICFlowController>();
+  auto congestionController = std::make_shared<MockQUICCongestionController>();
+  auto lossDetector         = std::make_shared<MockQUICLossDetector>();
+  QUICFrameDispatcher quicFrameDispatcher(connectionManager, streamManager, flowController, congestionController, lossDetector);
+
+  // Initial state
+  CHECK(connectionManager->getTotalFrameCount() == 0);
+  CHECK(streamManager->getTotalFrameCount() == 0);
+  CHECK(flowController->getTotalFrameCount() == 0);
+  CHECK(congestionController->getTotalFrameCount() == 0);
+
+  // STREAM frame
+  uint8_t buf[4096] = {0};
+  size_t len        = 0;
+  streamFrame.store(buf, &len);
+  quicFrameDispatcher.receive_frames(buf, len);
+  CHECK(connectionManager->getTotalFrameCount() == 0);
+  CHECK(streamManager->getTotalFrameCount() == 1);
+  CHECK(flowController->getTotalFrameCount() == 1);
+  CHECK(congestionController->getTotalFrameCount() == 1);
+}
+
+// Stubs
+QUICApplication *QUICNetVConnection::get_application(QUICStreamId)
+{
+  return nullptr;
+}
+
+QUICCrypto *
+QUICNetVConnection::get_crypto()
+{
+  return nullptr;
+}
+
+void QUICNetVConnection::close(QUICError)
+{
+  return;
+}
diff --git a/iocore/net/quic/test/test_QUICLossDetector.cc b/iocore/net/quic/test/test_QUICLossDetector.cc
new file mode 100644
index 0000000..09b4800
--- /dev/null
+++ b/iocore/net/quic/test/test_QUICLossDetector.cc
@@ -0,0 +1,57 @@
+/** @file
+ *
+ *  A brief file description
+ *
+ *  @section license License
+ *
+ *  Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+#include "catch.hpp"
+
+#include "QUICLossDetector.h"
+#include "QUICEvents.h"
+#include "Mock.h"
+
+TEST_CASE("QUICLossDetector_Loss_in_Handshake", "[quic]")
+{
+  MockQUICPacketTransmitter *tx = new MockQUICPacketTransmitter();
+  QUICLossDetector detector(tx);
+
+  // Check initial state
+  CHECK(tx->_retransmit_count == 0);
+
+  // Send SERVER_CLEARTEXT (Handshake message)
+  uint8_t raw[4]         = {0};
+  ats_unique_buf payload = ats_unique_malloc(sizeof(raw));
+  memcpy(payload.get(), raw, sizeof(raw));
+
+  std::unique_ptr<const QUICPacket> packet = std::unique_ptr<const QUICPacket>(new QUICPacket(
+    QUICPacketType::SERVER_CLEARTEXT, 0xffddbb9977553311ULL, 0x00000001, 0x00112233, std::move(payload), sizeof(raw), true));
+  detector.on_packet_sent(std::move(packet));
+  ink_hrtime_sleep(HRTIME_MSECONDS(1000));
+  CHECK(tx->_retransmit_count > 0);
+
+  // Receive ACK
+  std::shared_ptr<QUICAckFrame> frame = std::make_shared<QUICAckFrame>(0x01, 20, 0);
+  frame->ack_block_section()->add_ack_block({0, 1ULL});
+  detector.handle_frame(frame);
+  ink_hrtime_sleep(HRTIME_MSECONDS(1500));
+  int retransmit_count = tx->_retransmit_count;
+  ink_hrtime_sleep(HRTIME_MSECONDS(1500));
+  CHECK(tx->_retransmit_count == retransmit_count);
+}
diff --git a/iocore/net/quic/test/test_QUICPacket.cc b/iocore/net/quic/test/test_QUICPacket.cc
new file mode 100644
index 0000000..1dc3b70
--- /dev/null
+++ b/iocore/net/quic/test/test_QUICPacket.cc
@@ -0,0 +1,96 @@
+/** @file
+ *
+ *  A brief file description
+ *
+ *  @section license License
+ *
+ *  Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+#include "catch.hpp"
+
+#include "quic/QUICPacket.h"
+
+TEST_CASE("Loading Long Header Packet", "[quic]")
+{
+  uint8_t raw[]          = {0x01, 0x02, 0x03, 0x04};
+  ats_unique_buf payload = ats_unique_malloc(sizeof(raw));
+  memcpy(payload.get(), raw, sizeof(raw));
+
+  // Cleartext packet with a long header
+  QUICPacket packet1(QUICPacketType::CLIENT_CLEARTEXT, 0xffddbb9977553311ULL, 0xffcc9966, 0x00112233, std::move(payload),
+                     sizeof(raw), true);
+
+  uint8_t buf[65536];
+  size_t len;
+  packet1.store(buf, &len);
+
+  IOBufferBlock *block = new_IOBufferBlock();
+  block->alloc(iobuffer_size_to_index(len));
+  memcpy(block->end(), buf, len);
+  block->fill(len);
+
+  const QUICPacket packet2(block);
+
+  CHECK(packet2.type() == QUICPacketType::CLIENT_CLEARTEXT);
+  CHECK(packet2.connection_id() == 0xffddbb9977553311ULL);
+  CHECK(packet2.packet_number() == 0xffcc9966);
+  CHECK(packet2.version() == 0x00112233);
+  CHECK(packet2.size() == 29);
+  CHECK(packet2.payload_size() == sizeof(raw));
+  CHECK(memcmp(packet2.payload(), raw, sizeof(raw)) == 0);
+}
+
+TEST_CASE("Loading Short Header Packet", "[quic]")
+{
+  uint8_t raw[]          = {0x01, 0x02, 0x03, 0x04};
+  ats_unique_buf payload = ats_unique_malloc(sizeof(raw));
+  memcpy(payload.get(), raw, sizeof(raw));
+
+  uint8_t protected_raw[]          = {0x04, 0x03, 0x02, 0x01, 0x00};
+  ats_unique_buf protected_payload = ats_unique_malloc(sizeof(protected_raw));
+  memcpy(protected_payload.get(), protected_raw, sizeof(protected_raw));
+
+  // Cleartext packet with a long header
+  QUICPacket packet1(QUICPacketType::ONE_RTT_PROTECTED_KEY_PHASE_0, 0xffcc9966, std::move(payload), sizeof(raw), true);
+  packet1.set_protected_payload(std::move(protected_payload), sizeof(protected_raw));
+
+  uint8_t buf[65536];
+  size_t len;
+  packet1.store(buf, &len);
+
+  IOBufferBlock *block = new_IOBufferBlock();
+  block->alloc(iobuffer_size_to_index(len));
+  memcpy(block->end(), buf, len);
+  block->fill(len);
+
+  const QUICPacket packet2(block);
+
+  CHECK(packet2.type() == QUICPacketType::ONE_RTT_PROTECTED_KEY_PHASE_0);
+  CHECK(packet2.packet_number() == 0xffcc9966);
+  CHECK(packet2.size() == 10);
+  CHECK(packet2.payload_size() == sizeof(protected_raw));
+  CHECK(memcmp(packet2.payload(), protected_raw, sizeof(protected_raw)) == 0);
+}
+
+TEST_CASE("Loading Unknown Packet", "[quic]")
+{
+  const uint8_t buf[]      = {0xff};
+  QUICPacketHeader *header = QUICPacketHeader::load(buf, sizeof(buf));
+
+  CHECK(header->type() == QUICPacketType::UNINITIALIZED);
+}
diff --git a/iocore/net/quic/test/test_QUICPacketFactory.cc b/iocore/net/quic/test/test_QUICPacketFactory.cc
new file mode 100644
index 0000000..48f6c5d
--- /dev/null
+++ b/iocore/net/quic/test/test_QUICPacketFactory.cc
@@ -0,0 +1,69 @@
+/** @file
+ *
+ *  A brief file description
+ *
+ *  @section license License
+ *
+ *  Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+#include "catch.hpp"
+
+#include "quic/QUICPacket.h"
+
+TEST_CASE("QUICPacketFactory_Create_VersionNegotiationPacket", "[quic]")
+{
+  QUICPacketFactory factory;
+
+  const uint8_t client_initial_packet_data[] = {
+    0x82,                                           // Type
+    0x01, 0x02, 0x03, 0x04, 0x05, 0x06, 0x07, 0x08, // Connection id
+    0x00, 0x00, 0x00, 0x00,                         // Packet number
+    0xaa, 0xbb, 0xcc, 0xdd,                         // Version
+    0x00                                            // Payload
+  };
+
+  IOBufferBlock *block = new_IOBufferBlock();
+  block->alloc(iobuffer_size_to_index(sizeof(client_initial_packet_data)));
+  memcpy(block->end(), client_initial_packet_data, sizeof(client_initial_packet_data));
+  block->fill(sizeof(client_initial_packet_data));
+
+  QUICPacket client_initial_packet(block);
+
+  std::unique_ptr<QUICPacket> packet = factory.create_version_negotiation_packet(&client_initial_packet);
+  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\x04", 4) == 0);
+}
+
+TEST_CASE("QUICPacketFactory_Create_ServerCleartextPacket", "[quic]")
+{
+  QUICPacketFactory factory;
+  factory.set_version(0x11223344);
+
+  uint8_t raw[]          = {0xaa, 0xbb, 0xcc, 0xdd};
+  ats_unique_buf payload = ats_unique_malloc(sizeof(raw));
+  memcpy(payload.get(), raw, sizeof(raw));
+
+  std::unique_ptr<QUICPacket> packet = factory.create_server_cleartext_packet(0x01020304, std::move(payload), sizeof(raw), true);
+  CHECK(packet->type() == QUICPacketType::SERVER_CLEARTEXT);
+  CHECK(packet->connection_id() == 0x01020304);
+  CHECK(memcmp(packet->payload(), raw, sizeof(raw)) == 0);
+  CHECK(packet->packet_number() == 0);
+  CHECK(packet->version() == 0x11223344);
+}
diff --git a/iocore/net/quic/test/test_QUICStream.cc b/iocore/net/quic/test/test_QUICStream.cc
new file mode 100644
index 0000000..042f39d
--- /dev/null
+++ b/iocore/net/quic/test/test_QUICStream.cc
@@ -0,0 +1,140 @@
+/** @file
+ *
+ *  A brief file description
+ *
+ *  @section license License
+ *
+ *  Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+#include "catch.hpp"
+
+#include "quic/QUICStream.h"
+#include "quic/Mock.h"
+
+namespace
+{
+// Test Data
+uint8_t payload[]  = {0x01, 0x02, 0x03, 0x04, 0x05, 0x06, 0x07, 0x08, 0x09, 0x0a, 0x0b, 0x0c, 0x0d, 0x0e, 0x0f, 0x10};
+uint32_t stream_id = 0x03;
+
+std::shared_ptr<QUICStreamFrame> frame_1 = std::make_shared<QUICStreamFrame>(payload, 2, stream_id, 0);
+std::shared_ptr<QUICStreamFrame> frame_2 = std::make_shared<QUICStreamFrame>(payload + 2, 2, stream_id, 2);
+std::shared_ptr<QUICStreamFrame> frame_3 = std::make_shared<QUICStreamFrame>(payload + 4, 2, stream_id, 4);
+std::shared_ptr<QUICStreamFrame> frame_4 = std::make_shared<QUICStreamFrame>(payload + 6, 2, stream_id, 6);
+std::shared_ptr<QUICStreamFrame> frame_5 = std::make_shared<QUICStreamFrame>(payload + 8, 2, stream_id, 8);
+std::shared_ptr<QUICStreamFrame> frame_6 = std::make_shared<QUICStreamFrame>(payload + 10, 2, stream_id, 10);
+std::shared_ptr<QUICStreamFrame> frame_7 = std::make_shared<QUICStreamFrame>(payload + 12, 2, stream_id, 12);
+std::shared_ptr<QUICStreamFrame> frame_8 = std::make_shared<QUICStreamFrame>(payload + 14, 2, stream_id, 14);
+
+TEST_CASE("QUICStream_assembling_byte_stream_1", "[quic]")
+{
+  MIOBuffer *read_buffer = new_MIOBuffer(BUFFER_SIZE_INDEX_4K);
+  IOBufferReader *reader = read_buffer->alloc_reader();
+
+  std::unique_ptr<QUICStream> stream(new QUICStream());
+  stream->init(nullptr, stream_id);
+  stream->do_io_read(nullptr, 0, read_buffer);
+
+  stream->recv(frame_1);
+  stream->recv(frame_2);
+  stream->recv(frame_3);
+  stream->recv(frame_4);
+  stream->recv(frame_5);
+  stream->recv(frame_6);
+  stream->recv(frame_7);
+  stream->recv(frame_8);
+
+  uint8_t buf[32];
+  int64_t len = reader->read_avail();
+  reader->read(buf, len);
+
+  CHECK(len == 16);
+  CHECK(memcmp(buf, payload, len) == 0);
+}
+
+TEST_CASE("QUICStream_assembling_byte_stream_2", "[quic]")
+{
+  MIOBuffer *read_buffer = new_MIOBuffer(BUFFER_SIZE_INDEX_4K);
+  IOBufferReader *reader = read_buffer->alloc_reader();
+
+  std::unique_ptr<QUICStream> stream(new QUICStream());
+  stream->init(nullptr, stream_id);
+  stream->do_io_read(nullptr, 0, read_buffer);
+
+  stream->recv(frame_8);
+  stream->recv(frame_7);
+  stream->recv(frame_6);
+  stream->recv(frame_5);
+  stream->recv(frame_4);
+  stream->recv(frame_3);
+  stream->recv(frame_2);
+  stream->recv(frame_1);
+
+  uint8_t buf[32];
+  int64_t len = reader->read_avail();
+  reader->read(buf, len);
+
+  CHECK(len == 16);
+  CHECK(memcmp(buf, payload, len) == 0);
+}
+
+TEST_CASE("QUICStream_assembling_byte_stream_3", "[quic]")
+{
+  MIOBuffer *read_buffer = new_MIOBuffer(BUFFER_SIZE_INDEX_4K);
+  IOBufferReader *reader = read_buffer->alloc_reader();
+
+  std::unique_ptr<QUICStream> stream(new QUICStream());
+  stream->init(nullptr, stream_id);
+  stream->do_io_read(nullptr, 0, read_buffer);
+
+  stream->recv(frame_8);
+  stream->recv(frame_7);
+  stream->recv(frame_6);
+  stream->recv(frame_7); // duplicated frame
+  stream->recv(frame_5);
+  stream->recv(frame_3);
+  stream->recv(frame_1);
+  stream->recv(frame_2);
+  stream->recv(frame_4);
+  stream->recv(frame_5); // duplicated frame
+
+  uint8_t buf[32];
+  int64_t len = reader->read_avail();
+  reader->read(buf, len);
+
+  CHECK(len == 16);
+  CHECK(memcmp(buf, payload, len) == 0);
+}
+}
+
+// Stubs
+QUICApplication *QUICNetVConnection::get_application(QUICStreamId)
+{
+  return nullptr;
+}
+
+QUICCrypto *
+QUICNetVConnection::get_crypto()
+{
+  return nullptr;
+}
+
+void QUICNetVConnection::close(QUICError)
+{
+  return;
+}
diff --git a/iocore/net/quic/test/test_QUICStreamState.cc b/iocore/net/quic/test/test_QUICStreamState.cc
new file mode 100644
index 0000000..353cffd
--- /dev/null
+++ b/iocore/net/quic/test/test_QUICStreamState.cc
@@ -0,0 +1,47 @@
+/** @file
+ *
+ *  A brief file description
+ *
+ *  @section license License
+ *
+ *  Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+#include "catch.hpp"
+
+#include <memory>
+
+#include "quic/QUICFrame.h"
+#include "quic/QUICStreamState.h"
+
+TEST_CASE("QUICStreamState_Update", "[quic]")
+{
+  QUICStreamState ss;
+
+  std::shared_ptr<const QUICStreamFrame> streamFrame =
+    std::make_shared<const QUICStreamFrame>(reinterpret_cast<const uint8_t *>("foo"), 4, 1, 0);
+  std::shared_ptr<const QUICRstStreamFrame> rstStreamFrame =
+    std::make_shared<const QUICRstStreamFrame>(QUICErrorCode::QUIC_TRANSPORT_ERROR, 0, 0);
+
+  CHECK(ss.get() == QUICStreamState::State::idle);
+
+  ss.update_with_received_frame(*streamFrame);
+  CHECK(ss.get() == QUICStreamState::State::open);
+
+  ss.update_with_received_frame(*rstStreamFrame);
+  CHECK(ss.get() == QUICStreamState::State::closed);
+}
diff --git a/iocore/net/quic/test/test_QUICTransportParameters.cc b/iocore/net/quic/test/test_QUICTransportParameters.cc
new file mode 100644
index 0000000..7999789
--- /dev/null
+++ b/iocore/net/quic/test/test_QUICTransportParameters.cc
@@ -0,0 +1,164 @@
+/** @file
+ *
+ *  A brief file description
+ *
+ *  @section license License
+ *
+ *  Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+#include "catch.hpp"
+
+#include "QUICTransportParameters.h"
+
+TEST_CASE("QUICTransportParametersInClientHello_read", "[quic]")
+{
+  uint8_t buf[] = {
+    0x01, 0x02, 0x03, 0x04, // negotiated version
+    0x05, 0x06, 0x07, 0x08, // iinitial version
+    0x00, 0x04,             // number of parameters
+    0x00, 0x00,             // parameter id
+    0x00, 0x04,             // length of value
+    0x11, 0x22, 0x33, 0x44, // value
+    0x00, 0x01,             // parameter id
+    0x00, 0x04,             // length of value
+    0x12, 0x34, 0x56, 0x78, // value
+    0x00, 0x02,             // parameter id
+    0x00, 0x04,             // length of value
+    0x0a, 0x0b, 0x0c, 0x0d, // value
+    0x00, 0x03,             // parameter id
+    0x00, 0x02,             // length of value
+    0xab, 0xcd,             // value
+  };
+
+  QUICTransportParametersInClientHello params_in_ch(buf);
+  CHECK(params_in_ch.negotiated_version() == 0x01020304);
+  CHECK(params_in_ch.initial_version() == 0x05060708);
+  QUICTransportParameterValue value;
+  value = params_in_ch.get(QUICTransportParameterId::INITIAL_MAX_STREAM_DATA);
+  CHECK(value.len == 4);
+  CHECK(memcmp(value.data, "\x11\x22\x33\x44", 4) == 0);
+  value = params_in_ch.get(QUICTransportParameterId::INITIAL_MAX_DATA);
+  CHECK(value.len == 4);
+  CHECK(memcmp(value.data, "\x12\x34\x56\x78", 4) == 0);
+  value = params_in_ch.get(QUICTransportParameterId::INITIAL_MAX_STREAM_ID);
+  CHECK(value.len == 4);
+  CHECK(memcmp(value.data, "\x0a\x0b\x0c\x0d", 4) == 0);
+  value = params_in_ch.get(QUICTransportParameterId::IDLE_TIMEOUT);
+  CHECK(value.len == 2);
+  CHECK(memcmp(value.data, "\xab\xcd", 2) == 0);
+  value = params_in_ch.get(QUICTransportParameterId::MAX_PACKET_SIZE);
+  CHECK(value.len == 0);
+  CHECK(value.data == nullptr);
+}
+
+TEST_CASE("QUICTransportParametersInClientHello_write", "[quic]")
+{
+  uint8_t buf[65536];
+  uint16_t len;
+
+  uint8_t expected[] = {
+    0x01, 0x02, 0x03, 0x04, // negotiated version
+    0x05, 0x06, 0x07, 0x08, // iinitial version
+    0x00, 0x02,             // number of parameters
+    0x00, 0x00,             // parameter id
+    0x00, 0x04,             // length of value
+    0x11, 0x22, 0x33, 0x44, // value
+    0x00, 0x05,             // parameter id
+    0x00, 0x02,             // length of value
+    0xab, 0xcd,             // value
+  };
+
+  QUICTransportParametersInClientHello params_in_ch(0x01020304, 0x05060708);
+  params_in_ch.add(QUICTransportParameterId::INITIAL_MAX_STREAM_DATA, {reinterpret_cast<const uint8_t *>("\x11\x22\x33\x44"), 4});
+  params_in_ch.add(QUICTransportParameterId::MAX_PACKET_SIZE, {reinterpret_cast<const uint8_t *>("\xab\xcd"), 2});
+  params_in_ch.store(buf, &len);
+  CHECK(len == 24);
+  CHECK(memcmp(buf, expected, len) == 0);
+}
+
+TEST_CASE("QUICTransportParametersInEncryptedExtensions_read", "[quic]")
+{
+  uint8_t buf[] = {
+    0x00, 0x01,             // number of supported versions
+    0x01, 0x02, 0x03, 0x04, //
+    0x00, 0x04,             // number of parameters
+    0x00, 0x00,             // parameter id
+    0x00, 0x04,             // length of value
+    0x11, 0x22, 0x33, 0x44, // value
+    0x00, 0x01,             // parameter id
+    0x00, 0x04,             // length of value
+    0x12, 0x34, 0x56, 0x78, // value
+    0x00, 0x02,             // parameter id
+    0x00, 0x04,             // length of value
+    0x0a, 0x0b, 0x0c, 0x0d, // value
+    0x00, 0x03,             // parameter id
+    0x00, 0x02,             // length of value
+    0xab, 0xcd,             // value
+  };
+
+  QUICTransportParametersInEncryptedExtensions params_in_ee(buf);
+  const uint8_t *versions;
+  uint16_t nversion;
+  versions = params_in_ee.supported_versions(&nversion);
+  CHECK(nversion == 1);
+  CHECK(memcmp(versions, "\x01\x02\x03\x04", 4) == 0);
+  QUICTransportParameterValue value;
+  value = params_in_ee.get(QUICTransportParameterId::INITIAL_MAX_STREAM_DATA);
+  CHECK(value.len == 4);
+  CHECK(memcmp(value.data, "\x11\x22\x33\x44", 4) == 0);
+  value = params_in_ee.get(QUICTransportParameterId::INITIAL_MAX_DATA);
+  CHECK(value.len == 4);
+  CHECK(memcmp(value.data, "\x12\x34\x56\x78", 4) == 0);
+  value = params_in_ee.get(QUICTransportParameterId::INITIAL_MAX_STREAM_ID);
+  CHECK(value.len == 4);
+  CHECK(memcmp(value.data, "\x0a\x0b\x0c\x0d", 4) == 0);
+  value = params_in_ee.get(QUICTransportParameterId::IDLE_TIMEOUT);
+  CHECK(value.len == 2);
+  CHECK(memcmp(value.data, "\xab\xcd", 2) == 0);
+  value = params_in_ee.get(QUICTransportParameterId::MAX_PACKET_SIZE);
+  CHECK(value.len == 0);
+  CHECK(value.data == nullptr);
+}
+
+TEST_CASE("QUICTransportParametersEncryptedExtensions_write", "[quic]")
+{
+  uint8_t buf[65536];
+  uint16_t len;
+
+  uint8_t expected[] = {
+    0x00, 0x02,             // number of supported versions
+    0x01, 0x02, 0x03, 0x04, // version 1
+    0x05, 0x06, 0x07, 0x08, // version 2
+    0x00, 0x02,             // number of parameters
+    0x00, 0x00,             // parameter id
+    0x00, 0x04,             // length of value
+    0x11, 0x22, 0x33, 0x44, // value
+    0x00, 0x05,             // parameter id
+    0x00, 0x02,             // length of value
+    0xab, 0xcd,             // value
+  };
+
+  QUICTransportParametersInEncryptedExtensions params_in_ee;
+  params_in_ee.add(QUICTransportParameterId::INITIAL_MAX_STREAM_DATA, {reinterpret_cast<const uint8_t *>("\x11\x22\x33\x44"), 4});
+  params_in_ee.add(QUICTransportParameterId::MAX_PACKET_SIZE, {reinterpret_cast<const uint8_t *>("\xab\xcd"), 2});
+  params_in_ee.add_version(0x01020304);
+  params_in_ee.add_version(0x05060708);
+  params_in_ee.store(buf, &len);
+  CHECK(len == 26);
+  CHECK(memcmp(buf, expected, len) == 0);
+}
diff --git a/iocore/net/quic/test/test_QUICTypeUtil.cc b/iocore/net/quic/test/test_QUICTypeUtil.cc
new file mode 100644
index 0000000..1bea4ba
--- /dev/null
+++ b/iocore/net/quic/test/test_QUICTypeUtil.cc
@@ -0,0 +1,69 @@
+/** @file
+ *
+ *  A brief file description
+ *
+ *  @section license License
+ *
+ *  Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+#include "catch.hpp"
+
+#include "quic/QUICTypes.h"
+#include <memory>
+
+TEST_CASE("QUICTypeUtil", "[quic]")
+{
+  uint8_t buf[8];
+  size_t len;
+
+  QUICTypeUtil::write_uint_as_nbytes(0xff, 1, buf, &len);
+  INFO("1 byte to 1 byte");
+  CHECK(memcmp(buf, "\xff\x00\x00\x00\x00\x00\x00\x00", 1) == 0);
+
+  QUICTypeUtil::write_uint_as_nbytes(0xff, 2, buf, &len);
+  INFO("1 byte to 2 byte");
+  CHECK(memcmp(buf, "\x00\xff\x00\x00\x00\x00\x00\x00", 2) == 0);
+
+  QUICTypeUtil::write_uint_as_nbytes(0xff, 4, buf, &len);
+  INFO("1 byte to 4 byte");
+  CHECK(memcmp(buf, "\x00\x00\x00\xff\x00\x00\x00\x00", 4) == 0);
+
+  QUICTypeUtil::write_uint_as_nbytes(0xff, 6, buf, &len);
+  INFO("1 byte to 6 byte");
+  CHECK(memcmp(buf, "\x00\x00\x00\x00\x00\xff\x00\x00", 6) == 0);
+
+  QUICTypeUtil::write_uint_as_nbytes(0xff, 8, buf, &len);
+  INFO("1 byte to 8 byte");
+  CHECK(memcmp(buf, "\x00\x00\x00\x00\x00\x00\x00\xff", 8) == 0);
+
+  QUICTypeUtil::write_uint_as_nbytes(0x11ff, 2, buf, &len);
+  INFO("2 byte to 2 byte");
+  CHECK(memcmp(buf, "\x11\xff\x00\x00\x00\x00\x00\x00", 2) == 0);
+
+  QUICTypeUtil::write_uint_as_nbytes(0x11ff, 4, buf, &len);
+  INFO("2 byte to 4 byte");
+  CHECK(memcmp(buf, "\x00\x00\x11\xff\x00\x00\x00\x00", 4) == 0);
+
+  QUICTypeUtil::write_uint_as_nbytes(0x11ff, 6, buf, &len);
+  INFO("2 byte to 6 byte");
+  CHECK(memcmp(buf, "\x00\x00\x00\x00\x11\xff\x00\x00", 6) == 0);
+
+  QUICTypeUtil::write_uint_as_nbytes(0x11ff, 8, buf, &len);
+  INFO("2 byte to 8 byte");
+  CHECK(memcmp(buf, "\x00\x00\x00\x00\x00\x00\x11\xff", 8) == 0);
+}
diff --git a/iocore/net/quic/test/test_QUICVersionNegotiator.cc b/iocore/net/quic/test/test_QUICVersionNegotiator.cc
new file mode 100644
index 0000000..3d63e01
--- /dev/null
+++ b/iocore/net/quic/test/test_QUICVersionNegotiator.cc
@@ -0,0 +1,47 @@
+/** @file
+ *
+ *  A brief file description
+ *
+ *  @section license License
+ *
+ *  Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+#include "catch.hpp"
+
+#include "quic/QUICVersionNegotiator.h"
+#include "quic/Mock.h"
+
+TEST_CASE("QUICVersionNegotiator_Normal", "[quic]")
+{
+  QUICPacketFactory packet_factory;
+  QUICVersionNegotiator vn(&packet_factory, new MockQUICPacketTransmitter());
+
+  // Check initial state
+  CHECK(vn.status() == QUICVersionNegotiationStatus::NOT_NEGOTIATED);
+
+  // Negotiate version
+  std::unique_ptr<QUICPacket> initial_packet =
+    packet_factory.create_client_initial_packet({}, QUIC_SUPPORTED_VERSIONS[0], ats_unique_malloc(0), 0);
+  vn.negotiate(initial_packet.get());
+  CHECK(vn.status() == QUICVersionNegotiationStatus::NEGOTIATED);
+
+  // Revalidate version
+  vn.revalidate(QUIC_SUPPORTED_VERSIONS[0]);
+  // FIXME Currently, revalidate() is not implemented yet
+  // CHECK(vn.status() == QUICVersionNegotiationStatus::REVALIDATED);
+}
diff --git a/lib/records/I_RecHttp.h b/lib/records/I_RecHttp.h
index 75ddd6f..8fa2c69 100644
--- a/lib/records/I_RecHttp.h
+++ b/lib/records/I_RecHttp.h
@@ -231,7 +231,8 @@ public:
     TRANSPORT_COMPRESSED,   ///< Compressed HTTP.
     TRANSPORT_BLIND_TUNNEL, ///< Blind tunnel (no processing).
     TRANSPORT_SSL,          ///< SSL connection.
-    TRANSPORT_PLUGIN        /// < Protocol plugin connection
+    TRANSPORT_PLUGIN,       /// < Protocol plugin connection
+    TRANSPORT_QUIC,         ///< SSL connection.
   };
 
   int m_fd;             ///< Pre-opened file descriptor if present.
@@ -272,6 +273,9 @@ public:
   /// Check for SSL port.
   bool isSSL() const;
 
+  /// Check for QUIC port.
+  bool isQUIC() const;
+
   /// Check for SSL port.
   bool isPlugin() const;
 
@@ -298,6 +302,15 @@ public:
   /// @return @c true if any global port is an SSL port.
   static bool hasSSL();
 
+  /// Check for QUIC ports.
+  /// @return @c true if any port in @a ports is an QUIC port.
+  static bool hasQUIC(Group const &ports ///< Ports to check.
+                     );
+
+  /// Check for QUIC ports.
+  /// @return @c true if any global port is an QUIC port.
+  static bool hasQUIC();
+
   /** Load all relevant configuration data.
 
       This is hardwired to look up the appropriate values in the
@@ -389,6 +402,7 @@ public:
   static const char *const OPT_TRANSPARENT_FULL;        ///< Full transparency.
   static const char *const OPT_TRANSPARENT_PASSTHROUGH; ///< Pass-through non-HTTP.
   static const char *const OPT_SSL;                     ///< SSL (experimental)
+  static const char *const OPT_QUIC;                    ///< QUIC (experimental)
   static const char *const OPT_PLUGIN;                  ///< Protocol Plugin handle (experimental)
   static const char *const OPT_BLIND_TUNNEL;            ///< Blind tunnel.
   static const char *const OPT_COMPRESSED;              ///< Compressed.
@@ -421,6 +435,11 @@ HttpProxyPort::isSSL() const
   return TRANSPORT_SSL == m_type;
 }
 inline bool
+HttpProxyPort::isQUIC() const
+{
+  return TRANSPORT_QUIC == m_type;
+}
+inline bool
 HttpProxyPort::isPlugin() const
 {
   return TRANSPORT_PLUGIN == m_type;
@@ -463,6 +482,11 @@ HttpProxyPort::hasSSL()
 {
   return self::hasSSL(m_global);
 }
+inline bool
+HttpProxyPort::hasQUIC()
+{
+  return self::hasQUIC(m_global);
+}
 inline HttpProxyPort *
 HttpProxyPort::findHttp(uint16_t family)
 {
diff --git a/lib/records/RecHttp.cc b/lib/records/RecHttp.cc
index dc35bce..d302499 100644
--- a/lib/records/RecHttp.cc
+++ b/lib/records/RecHttp.cc
@@ -130,6 +130,7 @@ const char *const HttpProxyPort::OPT_SSL                     = "ssl";
 const char *const HttpProxyPort::OPT_PLUGIN                  = "plugin";
 const char *const HttpProxyPort::OPT_BLIND_TUNNEL            = "blind";
 const char *const HttpProxyPort::OPT_COMPRESSED              = "compressed";
+const char *const HttpProxyPort::OPT_QUIC                    = "quic";
 
 // File local constants.
 namespace
@@ -177,6 +178,18 @@ HttpProxyPort::hasSSL(Group const &ports)
   return zret;
 }
 
+bool
+HttpProxyPort::hasQUIC(Group const &ports)
+{
+  bool zret = false;
+  for (int i = 0, n = ports.length(); i < n && !zret; ++i) {
+    if (ports[i].isQUIC()) {
+      zret = true;
+    }
+  }
+  return zret;
+}
+
 HttpProxyPort *
 HttpProxyPort::findHttp(Group const &ports, uint16_t family)
 {
@@ -344,6 +357,8 @@ HttpProxyPort::processOptions(const char *opts)
       af_set_p = true;
     } else if (0 == strcasecmp(OPT_SSL, item)) {
       m_type = TRANSPORT_SSL;
+    } else if (0 == strcasecmp(OPT_QUIC, item)) {
+      m_type = TRANSPORT_QUIC;
     } else if (0 == strcasecmp(OPT_PLUGIN, item)) {
       m_type = TRANSPORT_PLUGIN;
     } else if (0 == strcasecmp(OPT_TRANSPARENT_INBOUND, item)) {
@@ -531,6 +546,8 @@ HttpProxyPort::print(char *out, size_t n)
     zret += snprintf(out + zret, n - zret, ":%s", OPT_BLIND_TUNNEL);
   } else if (TRANSPORT_SSL == m_type) {
     zret += snprintf(out + zret, n - zret, ":%s", OPT_SSL);
+  } else if (TRANSPORT_QUIC == m_type) {
+    zret += snprintf(out + zret, n - zret, ":%s", OPT_QUIC);
   } else if (TRANSPORT_PLUGIN == m_type) {
     zret += snprintf(out + zret, n - zret, ":%s", OPT_PLUGIN);
   } else if (TRANSPORT_COMPRESSED == m_type) {
diff --git a/proxy/Main.cc b/proxy/Main.cc
index d257a5e..2bf8796 100644
--- a/proxy/Main.cc
+++ b/proxy/Main.cc
@@ -91,6 +91,7 @@ extern "C" int plock(int);
 #include "I_Tasks.h"
 #include "InkAPIInternal.h"
 #include "HTTP2.h"
+#include "HQ.h"
 
 #include <ts/ink_cap.h>
 
@@ -1809,6 +1810,8 @@ main(int /* argc ATS_UNUSED */, const char **argv)
 
     // Initialize HTTP/2
     Http2::init();
+    // Initialize HTTP/QUIC
+    HQ::init();
 
     if (!HttpProxyPort::loadValue(http_accept_port_descriptor)) {
       HttpProxyPort::loadConfig();
@@ -1829,6 +1832,8 @@ main(int /* argc ATS_UNUSED */, const char **argv)
     SSLConfigParams::load_ssl_file_cb = load_ssl_file_callback;
     sslNetProcessor.start(-1, stacksize);
 
+    quic_NetProcessor.start(-1, stacksize);
+
     pmgmt->registerPluginCallbacks(global_config_cbs);
 
     cacheProcessor.afterInitCallbackSet(&CB_After_Cache_Init);
diff --git a/proxy/Makefile.am b/proxy/Makefile.am
index 0b5e394..220f8d8 100644
--- a/proxy/Makefile.am
+++ b/proxy/Makefile.am
@@ -19,7 +19,7 @@
 include $(top_srcdir)/build/tidy.mk
 
 # Note that hdrs is targeted from ../Makefile.am
-SUBDIRS = congest http http2 logging config
+SUBDIRS = congest http http2 hq logging config
 noinst_LIBRARIES =
 bin_PROGRAMS = \
   traffic_server \
@@ -35,6 +35,7 @@ AM_CPPFLAGS += \
   -I$(abs_top_srcdir)/lib/records \
   -I$(abs_srcdir)/http \
   -I$(abs_srcdir)/http2 \
+  -I$(abs_srcdir)/hq \
   -I$(abs_srcdir)/logging \
   -I$(abs_srcdir)/http/remap  \
   -I$(abs_srcdir)/hdrs \
@@ -179,6 +180,7 @@ endif
 traffic_server_LDADD = \
   http/libhttp.a \
   http2/libhttp2.a \
+  hq/libhq.a \
   http/remap/libhttp_remap.a \
   congest/libCongestionControl.a \
   logging/liblogging.a \
@@ -202,6 +204,7 @@ endif
 
 traffic_server_LDADD += \
   $(top_builddir)/iocore/net/libinknet.a \
+  $(top_builddir)/iocore/net/quic/libquic.a \
   $(top_builddir)/iocore/eventsystem/libinkevent.a \
   $(top_builddir)/lib/records/librecords_p.a \
   $(top_builddir)/iocore/eventsystem/libinkevent.a \
diff --git a/proxy/hq/HQ.cc b/proxy/hq/HQ.cc
new file mode 100644
index 0000000..1797a86
--- /dev/null
+++ b/proxy/hq/HQ.cc
@@ -0,0 +1,32 @@
+/** @file
+ *
+ *  A brief file description
+ *
+ *  @section license License
+ *
+ *  Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+#include "HQ.h"
+
+RecRawStatBlock *quic_rsb;
+
+void
+HQ::init()
+{
+  quic_rsb = RecAllocateRawStatBlock(static_cast<int>(HQ_N_STATS));
+}
diff --git a/proxy/hq/HQ.h b/proxy/hq/HQ.h
new file mode 100644
index 0000000..2d29e74
--- /dev/null
+++ b/proxy/hq/HQ.h
@@ -0,0 +1,44 @@
+/** @file
+ *
+ *  A brief file description
+ *
+ *  @section license License
+ *
+ *  Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+#ifndef __HTTP_QUIC_H__
+#define __HTTP_QUIC_H__
+
+#include "ts/ink_defs.h"
+#include "I_RecDefs.h"
+#include "I_RecProcess.h"
+
+extern RecRawStatBlock *hq_rsb; // Container for statistics.
+
+class HQ
+{
+public:
+  static void init();
+};
+
+// Statistics
+enum {
+  HQ_N_STATS // Terminal counter, NOT A STAT INDEX.
+};
+
+#endif // __HTTP_QUIC_H__
diff --git a/proxy/hq/HQSessionAccept.cc b/proxy/hq/HQSessionAccept.cc
new file mode 100644
index 0000000..a98e44e
--- /dev/null
+++ b/proxy/hq/HQSessionAccept.cc
@@ -0,0 +1,85 @@
+/** @file
+
+  A brief file description
+
+  @section license License
+
+  Licensed to the Apache Software Foundation (ASF) under one
+  or more contributor license agreements.  See the NOTICE file
+  distributed with this work for additional information
+  regarding copyright ownership.  The ASF licenses this file
+  to you under the Apache License, Version 2.0 (the
+  "License"); you may not use this file except in compliance
+  with the License.  You may obtain a copy of the License at
+
+      http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+ */
+
+#include "HQSessionAccept.h"
+
+#include "P_Net.h"
+#include "I_Machine.h"
+#include "../IPAllow.h"
+
+HQSessionAccept::HQSessionAccept(const HttpSessionAccept::Options &_o) : SessionAccept(nullptr), options(_o)
+{
+  SET_HANDLER(&HQSessionAccept::mainEvent);
+}
+
+HQSessionAccept::~HQSessionAccept()
+{
+}
+
+bool
+HQSessionAccept::accept(NetVConnection *netvc, MIOBuffer *iobuf, IOBufferReader *reader)
+{
+  sockaddr const *client_ip           = netvc->get_remote_addr();
+  const AclRecord *session_acl_record = testIpAllowPolicy(client_ip);
+  if (!session_acl_record) {
+    ip_port_text_buffer ipb;
+    Warning("QUIC client '%s' prohibited by ip-allow policy", ats_ip_ntop(client_ip, ipb, sizeof(ipb)));
+    return false;
+  }
+  netvc->attributes = this->options.transport_type;
+
+  if (is_debug_tag_set("quic_seq")) {
+    ip_port_text_buffer ipb;
+
+    Debug("quic_seq", "[HQSessionAccept:mainEvent %p] accepted connection from %s transport type = %d", netvc,
+          ats_ip_nptop(client_ip, ipb, sizeof(ipb)), netvc->attributes);
+  }
+
+  ink_assert(false);
+  // Not implemented yet
+
+  // HQClientSession *new_session = THREAD_ALLOC_INIT(quicClientSessionAllocator, this_ethread());
+  // new_session->acl_record         = session_acl_record;
+  // new_session->new_connection(netvc, iobuf, reader, false /* backdoor */);
+  // static_cast<QUICNetVConnection *>(netvc)->set_application(new_session);
+
+  return true;
+}
+
+int
+HQSessionAccept::mainEvent(int event, void *data)
+{
+  NetVConnection *netvc;
+  ink_release_assert(event == NET_EVENT_ACCEPT || event == EVENT_ERROR);
+  ink_release_assert((event == NET_EVENT_ACCEPT) ? (data != nullptr) : (1));
+
+  if (event == NET_EVENT_ACCEPT) {
+    netvc = static_cast<NetVConnection *>(data);
+    if (!this->accept(netvc, nullptr, nullptr)) {
+      netvc->do_io_close();
+    }
+    return EVENT_CONT;
+  }
+
+  return EVENT_CONT;
+}
diff --git a/proxy/hq/HQSessionAccept.h b/proxy/hq/HQSessionAccept.h
new file mode 100644
index 0000000..68fb584
--- /dev/null
+++ b/proxy/hq/HQSessionAccept.h
@@ -0,0 +1,56 @@
+/** @file
+
+  A brief file description
+
+  @section license License
+
+  Licensed to the Apache Software Foundation (ASF) under one
+  or more contributor license agreements.  See the NOTICE file
+  distributed with this work for additional information
+  regarding copyright ownership.  The ASF licenses this file
+  to you under the Apache License, Version 2.0 (the
+  "License"); you may not use this file except in compliance
+  with the License.  You may obtain a copy of the License at
+
+      http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+ */
+
+#ifndef __HTTP_QUIC_SESSION_ACCEPT_H__
+#define __HTTP_QUIC_SESSION_ACCEPT_H__
+
+#include "ts/ink_platform.h"
+#include "I_Net.h"
+
+// XXX HttpSessionAccept::Options needs to be refactored and separated from HttpSessionAccept so that
+// it can generically apply to all protocol implementations.
+#include "http/HttpSessionAccept.h"
+
+// HTTP/QUIC Session Accept.
+//
+// HTTP/QUIC needs to be explicitly enabled on a server port. The syntax is different for SSL and raw
+// ports. The example below configures QUIC on port 443 (with TLS).
+//
+// CONFIG proxy.config.http.server_ports STRING 443:quic
+
+class HQSessionAccept : public SessionAccept {
+public:
+  explicit HQSessionAccept(const HttpSessionAccept::Options &);
+  ~HQSessionAccept();
+
+  bool accept(NetVConnection *, MIOBuffer *, IOBufferReader *);
+  int mainEvent(int event, void *netvc);
+
+private:
+  HQSessionAccept(const HQSessionAccept &);
+  HQSessionAccept &operator=(const HQSessionAccept &);
+
+  HttpSessionAccept::Options options;
+};
+
+#endif // __HTTP_QUIC_SESSION_ACCEPT_H__
diff --git a/proxy/hq/Makefile.am b/proxy/hq/Makefile.am
new file mode 100644
index 0000000..b188de4
--- /dev/null
+++ b/proxy/hq/Makefile.am
@@ -0,0 +1,43 @@
+#
+#  Makefile.am for HTTP over QUIC
+#
+#  Licensed to the Apache Software Foundation (ASF) under one
+#  or more contributor license agreements.  See the NOTICE file
+#  distributed with this work for additional information
+#  regarding copyright ownership.  The ASF licenses this file
+#  to you under the Apache License, Version 2.0 (the
+#  "License"); you may not use this file except in compliance
+#  with the License.  You may obtain a copy of the License at
+#
+#      http://www.apache.org/licenses/LICENSE-2.0
+#
+#  Unless required by applicable law or agreed to in writing, software
+#  distributed under the License is distributed on an "AS IS" BASIS,
+#  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+#  See the License for the specific language governing permissions and
+#  limitations under the License.
+
+include $(top_srcdir)/build/tidy.mk
+
+AM_CPPFLAGS = \
+  $(iocore_include_dirs) \
+  -I$(abs_top_srcdir)/proxy/api/ts \
+  -I$(abs_top_srcdir)/lib \
+  -I$(abs_top_srcdir)/lib/records \
+  -I$(abs_top_srcdir)/mgmt \
+  -I$(abs_top_srcdir)/mgmt/utils \
+  -I$(abs_top_srcdir)/proxy \
+  -I$(abs_top_srcdir)/proxy/http \
+  -I$(abs_top_srcdir)/proxy/hdrs \
+  -I$(abs_top_srcdir)/proxy/shared \
+  -I$(abs_top_srcdir)/proxy/http/remap
+
+noinst_LIBRARIES = libhq.a
+
+libhq_a_SOURCES = \
+  HQ.cc \
+  HQSessionAccept.cc \
+  HQSessionAccept.h
+
+tidy-local: $(libhq_a_SOURCES) \
+	$(CXX_Clang_Tidy)
diff --git a/proxy/http/HttpProxyServerMain.cc b/proxy/http/HttpProxyServerMain.cc
index e88b604..9beb3ac 100644
--- a/proxy/http/HttpProxyServerMain.cc
+++ b/proxy/http/HttpProxyServerMain.cc
@@ -39,6 +39,7 @@
 #include "ProtocolProbeSessionAccept.h"
 #include "http2/Http2SessionAccept.h"
 #include "HttpConnectionCount.h"
+#include "hq/HQSessionAccept.h"
 
 HttpSessionAccept *plugin_http_accept             = nullptr;
 HttpSessionAccept *plugin_http_transparent_accept = nullptr;
@@ -218,6 +219,10 @@ MakeHttpProxyAcceptor(HttpProxyAcceptor &acceptor, HttpProxyPort &port, unsigned
     ssl_plugin_acceptors.push(ssl);
     ssl->proxyPort   = &port;
     acceptor._accept = ssl;
+  } else if (port.isQUIC()) {
+    // HTTP/QUIC
+    HQSessionAccept *hq = new HQSessionAccept(accept_opt);
+    acceptor._accept    = hq;
   } else {
     acceptor._accept = probe;
   }
@@ -286,7 +291,11 @@ start_HttpProxyServer()
   for (int i = 0, n = proxy_ports.length(); i < n; ++i) {
     HttpProxyAcceptor &acceptor = HttpProxyAcceptors[i];
     HttpProxyPort &port         = proxy_ports[i];
-    if (port.isSSL()) {
+    if (port.isQUIC()) {
+      if (nullptr == quic_NetProcessor.main_accept(acceptor._accept, port.m_fd, acceptor._net_opt)) {
+        return;
+      }
+    } else if (port.isSSL()) {
       if (nullptr == sslNetProcessor.main_accept(acceptor._accept, port.m_fd, acceptor._net_opt)) {
         return;
       }
diff --git a/proxy/http/Makefile.am b/proxy/http/Makefile.am
index fc8ee91..0d9605a 100644
--- a/proxy/http/Makefile.am
+++ b/proxy/http/Makefile.am
@@ -31,7 +31,8 @@ AM_CPPFLAGS += \
   -I$(abs_top_srcdir)/proxy/shared \
   -I$(abs_top_srcdir)/proxy/http/remap \
   -I$(abs_top_srcdir)/proxy/logging \
-  -I$(abs_top_srcdir)/proxy/http2
+  -I$(abs_top_srcdir)/proxy/http2 \
+  -I$(abs_top_srcdir)/proxy/hq
 
 noinst_HEADERS = HttpProxyServerMain.h
 noinst_LIBRARIES = libhttp.a

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