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

[trafficserver] 03/03: Refer QUICConnectionInfoProviter when print debug log

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

commit 4712f8a260aefb3e2c0a2c21af98074660d18650
Author: Masaori Koshiba <ma...@apache.org>
AuthorDate: Thu May 31 10:34:14 2018 +0900

    Refer QUICConnectionInfoProviter when print debug log
---
 cmd/traffic_quic/quic_client.cc                  |  3 +-
 iocore/net/P_QUICNetVConnection.h                |  4 +-
 iocore/net/QUICNetProcessor.cc                   |  8 +--
 iocore/net/QUICNetVConnection.cc                 | 16 +++--
 iocore/net/QUICPacketHandler.cc                  | 37 +++++++----
 iocore/net/quic/Mock.h                           | 81 +++++++++++++++++++++++-
 iocore/net/quic/QUICApplication.cc               |  2 +-
 iocore/net/quic/QUICCongestionController.cc      | 12 ++--
 iocore/net/quic/QUICHandshake.cc                 |  4 +-
 iocore/net/quic/QUICLossDetector.cc              | 11 ++--
 iocore/net/quic/QUICLossDetector.h               | 11 ++--
 iocore/net/quic/QUICPacketRetransmitter.cc       |  1 -
 iocore/net/quic/QUICStream.cc                    | 18 +++---
 iocore/net/quic/QUICStream.h                     |  5 +-
 iocore/net/quic/QUICStreamManager.cc             |  6 +-
 iocore/net/quic/QUICStreamManager.h              |  6 +-
 iocore/net/quic/test/test_QUICFrame.cc           |  2 +-
 iocore/net/quic/test/test_QUICFrameDispatcher.cc |  6 +-
 iocore/net/quic/test/test_QUICLossDetector.cc    | 18 +++---
 iocore/net/quic/test/test_QUICStream.cc          | 11 ++--
 iocore/net/quic/test/test_QUICStreamManager.cc   |  8 +--
 proxy/hq/HQClientSession.cc                      |  2 +-
 proxy/hq/HQClientTransaction.cc                  |  7 +-
 23 files changed, 186 insertions(+), 93 deletions(-)

diff --git a/cmd/traffic_quic/quic_client.cc b/cmd/traffic_quic/quic_client.cc
index da43583..d01bd09 100644
--- a/cmd/traffic_quic/quic_client.cc
+++ b/cmd/traffic_quic/quic_client.cc
@@ -112,8 +112,7 @@ QUICClient::state_http_server_open(int event, void *data)
 //
 // QUICClientApp
 //
-#define QUICClientAppDebug(fmt, ...) \
-  Debug("quic_client_app", "[%" PRIx64 "] " fmt, static_cast<uint64_t>(this->_qc->connection_id()), ##__VA_ARGS__)
+#define QUICClientAppDebug(fmt, ...) Debug("quic_client_app", "[%s] " fmt, this->_qc->cids().data(), ##__VA_ARGS__)
 
 QUICClientApp::QUICClientApp(QUICNetVConnection *qvc, const char *filename) : QUICApplication(qvc), _filename(filename)
 {
diff --git a/iocore/net/P_QUICNetVConnection.h b/iocore/net/P_QUICNetVConnection.h
index 3398aa6..3b255f2 100644
--- a/iocore/net/P_QUICNetVConnection.h
+++ b/iocore/net/P_QUICNetVConnection.h
@@ -232,9 +232,9 @@ private:
   QUICPacketType _last_received_packet_type = QUICPacketType::UNINITIALIZED;
   std::random_device _rnd;
 
-  QUICConnectionId _peer_quic_connection_id;     // dst cid
+  QUICConnectionId _peer_quic_connection_id;     // dst cid in local
   QUICConnectionId _original_quic_connection_id; // dst cid of initial packet from client
-  QUICConnectionId _quic_connection_id;          // src cid
+  QUICConnectionId _quic_connection_id;          // src cid in local
   QUICFiveTuple _five_tuple;
 
   char _cids_data[MAX_CIDS_SIZE] = {0};
diff --git a/iocore/net/QUICNetProcessor.cc b/iocore/net/QUICNetProcessor.cc
index 1d55a73..8807a15 100644
--- a/iocore/net/QUICNetProcessor.cc
+++ b/iocore/net/QUICNetProcessor.cc
@@ -139,11 +139,9 @@ QUICNetProcessor::connect_re(Continuation *cont, sockaddr const *remote_addr, Ne
   }
 
   // Setup QUICNetVConnection
-  QUICConnectionId scid;
-  QUICConnectionId dcid;
-  scid.randomize();
-  dcid.randomize();
-  vc->init(scid, dcid, con, packet_handler);
+  QUICConnectionId client_dst_cid;
+  client_dst_cid.randomize();
+  vc->init(client_dst_cid, client_dst_cid, con, packet_handler);
   packet_handler->init(vc);
 
   // Connection ID will be changed
diff --git a/iocore/net/QUICNetVConnection.cc b/iocore/net/QUICNetVConnection.cc
index bd216a7..2c1f1a5 100644
--- a/iocore/net/QUICNetVConnection.cc
+++ b/iocore/net/QUICNetVConnection.cc
@@ -205,13 +205,12 @@ QUICNetVConnection::start()
   this->_packet_factory.set_hs_protocol(this->_hs_protocol);
 
   // Create frame handlers
-  this->_congestion_controller  = new QUICCongestionController(this->peer_connection_id());
-  this->_loss_detector          = new QUICLossDetector(this, this->_congestion_controller);
+  this->_congestion_controller  = new QUICCongestionController(this);
+  this->_loss_detector          = new QUICLossDetector(this, this, this->_congestion_controller);
   this->_remote_flow_controller = new QUICRemoteConnectionFlowController(UINT64_MAX);
   this->_local_flow_controller  = new QUICLocalConnectionFlowController(this->_loss_detector, UINT64_MAX);
   this->_path_validator         = new QUICPathValidator();
-  this->_stream_manager =
-    new QUICStreamManager(this->_loss_detector, this->peer_connection_id(), this->_application_map, this->netvc_context);
+  this->_stream_manager         = new QUICStreamManager(this->_loss_detector, this, this->_application_map, this->netvc_context);
 
   this->_frame_dispatcher->add_handler(this);
   this->_frame_dispatcher->add_handler(this->_stream_manager);
@@ -343,6 +342,12 @@ QUICNetVConnection::connection_id() const
   return this->_quic_connection_id;
 }
 
+/*
+ Return combination of dst connection id and src connection id for debug log
+ e.g. "aaaaaaaa-bbbbbbbb"
+   - "aaaaaaaa" : high 32 bit of dst connection id
+   - "bbbbbbbb" : high 32 bit of src connection id
+ */
 std::string_view
 QUICNetVConnection::cids() const
 {
@@ -427,6 +432,8 @@ QUICNetVConnection::transmit_packet(QUICPacketUPtr packet)
 void
 QUICNetVConnection::retransmit_packet(const QUICPacket &packet)
 {
+  QUICConDebug("Retransmit packet #%" PRIu64 " type %s", packet.packet_number(), QUICDebugNames::packet_type(packet.type()));
+
   this->_packet_retransmitter.retransmit_packet(packet);
 }
 
@@ -1775,6 +1782,7 @@ QUICNetVConnection::_update_cids()
 void
 QUICNetVConnection::_update_peer_cid(const QUICConnectionId &new_cid)
 {
+  // TODO: print full cids
   QUICConDebug("dcid: %08" PRIx32 " -> %08" PRIx32, this->_peer_quic_connection_id.h32(), new_cid.h32());
 
   this->_peer_quic_connection_id = new_cid;
diff --git a/iocore/net/QUICPacketHandler.cc b/iocore/net/QUICPacketHandler.cc
index 17241a8..6717880 100644
--- a/iocore/net/QUICPacketHandler.cc
+++ b/iocore/net/QUICPacketHandler.cc
@@ -30,6 +30,12 @@
 #include "QUICDebugNames.h"
 #include "QUICEvents.h"
 
+#define QUICDebugQC(qc, fmt, ...) Debug("quic_sec", "[%s] " fmt, qc->cids().data(), ##__VA_ARGS__)
+
+// ["local dcid" - "local scid"]
+#define QUICDebugDS(dcid, scid, fmt, ...) \
+  Debug("quic_sec", "[%08" PRIx32 "-%08" PRIx32 "] " fmt, dcid.h32(), scid.h32(), ##__VA_ARGS__)
+
 //
 // QUICPacketHandler
 //
@@ -74,8 +80,13 @@ QUICPacketHandler::_send_packet(Continuation *c, const QUICPacket &packet, UDPCo
 
   // NOTE: p will be enqueued to udpOutQueue of UDPNetHandler
   ip_port_text_buffer ipb;
-  Debug("quic_sec", "[%" PRIx64 "] send %s packet to %s, size=%" PRId64, packet.destination_cid().l64(),
-        QUICDebugNames::packet_type(packet.type()), ats_ip_nptop(&udp_packet->to.sa, ipb, sizeof(ipb)), udp_packet->getPktLength());
+  QUICConnectionId dcid = packet.destination_cid();
+  QUICConnectionId scid = QUICConnectionId::ZERO();
+  if (packet.type() != QUICPacketType::PROTECTED) {
+    scid = packet.source_cid();
+  }
+  QUICDebugDS(dcid, scid, "send %s packet to %s size=%" PRId64, QUICDebugNames::packet_type(packet.type()),
+              ats_ip_nptop(&udp_packet->to.sa, ipb, sizeof(ipb)), udp_packet->getPktLength());
 
   udp_con->send(c, udp_packet);
 }
@@ -178,14 +189,15 @@ QUICPacketHandlerIn::_recv_packet(int event, UDPPacket *udp_packet)
 
   if (is_debug_tag_set("quic_sec")) {
     ip_port_text_buffer ipb;
+    QUICConnectionId dcid = this->_read_destination_connection_id(block);
+    QUICConnectionId scid = QUICConnectionId::ZERO();
     if (QUICTypeUtil::has_long_header(reinterpret_cast<const uint8_t *>(block->buf()))) {
-      QUICConnectionId cid = this->_read_source_connection_id(block);
-      Debug("quic_sec", "[%" PRIx64 "] received packet from %s, size=%" PRId64, cid.l64(),
-            ats_ip_nptop(&udp_packet->from.sa, ipb, sizeof(ipb)), udp_packet->getPktLength());
-    } else {
-      Debug("quic_sec", "received packet from %s, size=%" PRId64, ats_ip_nptop(&udp_packet->from.sa, ipb, sizeof(ipb)),
-            udp_packet->getPktLength());
+      scid = this->_read_source_connection_id(block);
     }
+    // Remote dst cid is src cid in local
+    // TODO: print packet type
+    QUICDebugDS(scid, dcid, "recv packet from %s, size=%" PRId64, ats_ip_nptop(&udp_packet->from.sa, ipb, sizeof(ipb)),
+                udp_packet->getPktLength());
   }
 
   QUICConnection *qc =
@@ -299,13 +311,10 @@ QUICPacketHandlerOut::send_packet(const QUICPacket &packet, QUICNetVConnection *
 void
 QUICPacketHandlerOut::_recv_packet(int event, UDPPacket *udp_packet)
 {
-  IOBufferBlock *block = udp_packet->getIOBlockChain();
-
-  QUICConnectionId cid = this->_read_destination_connection_id(block);
-
   ip_port_text_buffer ipb;
-  Debug("quic_sec", "[%" PRIx64 "] received packet from %s, size=%" PRId64, cid.l64(),
-        ats_ip_nptop(&udp_packet->from.sa, ipb, sizeof(ipb)), udp_packet->getPktLength());
+  // TODO: print packet type
+  QUICDebugQC(this->_vc, "recv packet from %s size=%" PRId64, ats_ip_nptop(&udp_packet->from.sa, ipb, sizeof(ipb)),
+              udp_packet->getPktLength());
 
   this->_vc->handle_received_packet(udp_packet);
   eventProcessor.schedule_imm(this->_vc, ET_CALL, QUIC_EVENT_PACKET_READ_READY, nullptr);
diff --git a/iocore/net/quic/Mock.h b/iocore/net/quic/Mock.h
index 761e96f..a798a34 100644
--- a/iocore/net/quic/Mock.h
+++ b/iocore/net/quic/Mock.h
@@ -284,6 +284,81 @@ public:
   NetVConnectionContext_t _direction;
 };
 
+class MockQUICConnectionInfoProvider : public QUICConnectionInfoProvider
+{
+  QUICConnectionId
+  connection_id() const override
+  {
+    return {reinterpret_cast<const uint8_t *>("\x00"), 1};
+  }
+
+  QUICConnectionId
+  peer_connection_id() const override
+  {
+    return {reinterpret_cast<const uint8_t *>("\x00"), 1};
+  }
+
+  QUICConnectionId
+  original_connection_id() const override
+  {
+    return {reinterpret_cast<const uint8_t *>("\x00"), 1};
+  }
+
+  const QUICFiveTuple
+  five_tuple() const override
+  {
+    return QUICFiveTuple();
+  }
+
+  std::string_view
+  cids() const override
+  {
+    return std::string_view("00000000-00000000"sv);
+  }
+
+  uint32_t
+  minimum_quic_packet_size() override
+  {
+    return 1200;
+  }
+
+  uint32_t
+  maximum_quic_packet_size() const override
+  {
+    return 1200;
+  }
+
+  uint32_t
+  pmtu() const override
+  {
+    return 1280;
+  }
+
+  QUICPacketNumber
+  largest_acked_packet_number() const override
+  {
+    return 0;
+  }
+
+  NetVConnectionContext_t
+  direction() const override
+  {
+    return NET_VCONNECTION_OUT;
+  }
+
+  SSLNextProtocolSet *
+  next_protocol_set() const override
+  {
+    return nullptr;
+  }
+
+  bool
+  is_closed() const override
+  {
+    return false;
+  }
+};
+
 class MockQUICPacketTransmitter : public QUICPacketTransmitter
 {
 public:
@@ -319,6 +394,7 @@ public:
 class MockQUICCongestionController : public QUICCongestionController
 {
 public:
+  MockQUICCongestionController(QUICConnectionInfoProvider *info) : QUICCongestionController(info) {}
   // Override
   virtual void
   on_packets_lost(std::map<QUICPacketNumber, PacketInfo *> &packets) override
@@ -363,7 +439,10 @@ private:
 class MockQUICLossDetector : public QUICLossDetector
 {
 public:
-  MockQUICLossDetector() : QUICLossDetector(new MockQUICPacketTransmitter(), new MockQUICCongestionController()) {}
+  MockQUICLossDetector(QUICPacketTransmitter *transmitter, QUICConnectionInfoProvider *info, QUICCongestionController *cc)
+    : QUICLossDetector(transmitter, info, cc)
+  {
+  }
   void
   rcv_frame(std::shared_ptr<const QUICFrame>)
   {
diff --git a/iocore/net/quic/QUICApplication.cc b/iocore/net/quic/QUICApplication.cc
index cd178f0..4a960d5 100644
--- a/iocore/net/quic/QUICApplication.cc
+++ b/iocore/net/quic/QUICApplication.cc
@@ -199,7 +199,7 @@ QUICApplication::reenable(QUICStream *stream)
     stream_io->read_reenable();
     stream_io->write_reenable();
   } else {
-    Debug(tag, "[%" PRIx64 "] Unknown Stream, id: %" PRIx64, this->_qc->connection_id().l64(), stream->id());
+    Debug(tag, "[%s] Unknown Stream id=%" PRIx64, this->_qc->cids().data(), stream->id());
   }
 
   return;
diff --git a/iocore/net/quic/QUICCongestionController.cc b/iocore/net/quic/QUICCongestionController.cc
index 9a756d8..fa580e6 100644
--- a/iocore/net/quic/QUICCongestionController.cc
+++ b/iocore/net/quic/QUICCongestionController.cc
@@ -28,19 +28,17 @@
 
 #define QUICCCDebug(fmt, ...)                                                \
   Debug("quic_cc",                                                           \
-        "[%" PRIx64 "] "                                                     \
+        "[%s] "                                                              \
         "window: %" PRIu32 " bytes: %" PRIu32 " ssthresh: %" PRIu32 " " fmt, \
-        this->_connection_id.l64(), this->_congestion_window, this->_bytes_in_flight, this->_ssthresh, ##__VA_ARGS__)
+        this->_info->cids().data(), this->_congestion_window, this->_bytes_in_flight, this->_ssthresh, ##__VA_ARGS__)
 
 #define QUICCCError(fmt, ...)                                                \
   Error("quic_cc",                                                           \
-        "[%" PRIx64 "] "                                                     \
+        "[%s] "                                                              \
         "window: %" PRIu32 " bytes: %" PRIu32 " ssthresh: %" PRIu32 " " fmt, \
-        this->_connection_id.l64(), this->_congestion_window, this->_bytes_in_flight, this->_ssthresh, ##__VA_ARGS__)
+        this->_info->cids().data(), this->_congestion_window, this->_bytes_in_flight, this->_ssthresh, ##__VA_ARGS__)
 
-QUICCongestionController::QUICCongestionController() : QUICCongestionController(QUICConnectionId::ZERO()) {}
-
-QUICCongestionController::QUICCongestionController(QUICConnectionId connection_id) : _connection_id(connection_id)
+QUICCongestionController::QUICCongestionController(QUICConnectionInfoProvider *info) : _info(info)
 {
   QUICConfig::scoped_config params;
   this->_k_default_mss           = params->cc_default_mss();
diff --git a/iocore/net/quic/QUICHandshake.cc b/iocore/net/quic/QUICHandshake.cc
index 715c01c..e765e67 100644
--- a/iocore/net/quic/QUICHandshake.cc
+++ b/iocore/net/quic/QUICHandshake.cc
@@ -36,9 +36,9 @@
 
 static constexpr char dump_tag[] = "v_quic_handshake_dump_pkt";
 
-#define QUICHSDebug(fmt, ...) Debug("quic_handshake", "[%" PRIx64 "] " fmt, this->_qc->peer_connection_id().l64(), ##__VA_ARGS__)
+#define QUICHSDebug(fmt, ...) Debug("quic_handshake", "[%s] " fmt, this->_qc->cids().data(), ##__VA_ARGS__)
 
-#define QUICVHSDebug(fmt, ...) Debug("v_quic_handshake", "[%" PRIx64 "] " fmt, this->_qc->peer_connection_id().l64(), ##__VA_ARGS__)
+#define QUICVHSDebug(fmt, ...) Debug("v_quic_handshake", "[%s] " fmt, this->_qc->cids().data(), ##__VA_ARGS__)
 
 #define I_WANNA_DUMP_THIS_BUF(buf, len)                                                                                            \
   {                                                                                                                                \
diff --git a/iocore/net/quic/QUICLossDetector.cc b/iocore/net/quic/QUICLossDetector.cc
index e58fce2..a2eca13 100644
--- a/iocore/net/quic/QUICLossDetector.cc
+++ b/iocore/net/quic/QUICLossDetector.cc
@@ -28,10 +28,11 @@
 #include "QUICConfig.h"
 #include "QUICEvents.h"
 
-#define QUICLDDebug(fmt, ...) Debug("quic_loss_detector", "[%" PRIx64 "] " fmt, this->_connection_id.l64(), ##__VA_ARGS__)
+#define QUICLDDebug(fmt, ...) Debug("quic_loss_detector", "[%s] " fmt, this->_info->cids().data(), ##__VA_ARGS__)
 
-QUICLossDetector::QUICLossDetector(QUICPacketTransmitter *transmitter, QUICCongestionController *cc)
-  : _transmitter(transmitter), _cc(cc)
+QUICLossDetector::QUICLossDetector(QUICPacketTransmitter *transmitter, QUICConnectionInfoProvider *info,
+                                   QUICCongestionController *cc)
+  : _transmitter(transmitter), _info(info), _cc(cc)
 {
   this->mutex                 = new_ProxyMutex();
   this->_loss_detection_mutex = new_ProxyMutex();
@@ -131,10 +132,6 @@ QUICLossDetector::largest_acked_packet_number()
 void
 QUICLossDetector::on_packet_sent(QUICPacketUPtr packet)
 {
-  if (this->_connection_id.is_zero() && packet->type() != QUICPacketType::VERSION_NEGOTIATION) {
-    this->_connection_id = packet->destination_cid();
-  }
-
   bool is_handshake   = false;
   QUICPacketType type = packet->type();
 
diff --git a/iocore/net/quic/QUICLossDetector.h b/iocore/net/quic/QUICLossDetector.h
index c448727..c8a1deb 100644
--- a/iocore/net/quic/QUICLossDetector.h
+++ b/iocore/net/quic/QUICLossDetector.h
@@ -36,6 +36,7 @@
 #include "QUICFrame.h"
 #include "QUICFrameHandler.h"
 #include "QUICPacketTransmitter.h"
+#include "QUICConnection.h"
 
 class QUICLossDetector;
 
@@ -57,8 +58,7 @@ public:
 class QUICCongestionController
 {
 public:
-  QUICCongestionController();
-  QUICCongestionController(QUICConnectionId connection_id);
+  QUICCongestionController(QUICConnectionInfoProvider *info);
   virtual ~QUICCongestionController() {}
   void on_packet_sent(size_t bytes_sent);
   void on_packet_acked(QUICPacketNumber acked_packet_number, size_t acked_packet_size);
@@ -85,7 +85,7 @@ private:
   QUICPacketNumber _end_of_recovery = 0;
   uint32_t _ssthresh                = UINT32_MAX;
 
-  QUICConnectionId _connection_id = QUICConnectionId::ZERO();
+  QUICConnectionInfoProvider *_info = nullptr;
 
   bool _in_recovery(QUICPacketNumber packet_number);
 };
@@ -93,7 +93,7 @@ private:
 class QUICLossDetector : public Continuation, public QUICFrameHandler, public QUICRTTProvider
 {
 public:
-  QUICLossDetector(QUICPacketTransmitter *transmitter, QUICCongestionController *cc);
+  QUICLossDetector(QUICPacketTransmitter *transmitter, QUICConnectionInfoProvider *info, QUICCongestionController *cc);
   ~QUICLossDetector();
 
   int event_handler(int event, Event *edata);
@@ -109,8 +109,6 @@ public:
 private:
   Ptr<ProxyMutex> _loss_detection_mutex;
 
-  QUICConnectionId _connection_id = QUICConnectionId::ZERO();
-
   // TODO QUICCongestionController *cc = nullptr;
 
   // 3.4.1.  Constants of interest (draft-10)
@@ -174,5 +172,6 @@ private:
   void _send_two_packets();
 
   QUICPacketTransmitter *_transmitter = nullptr;
+  QUICConnectionInfoProvider *_info   = nullptr;
   QUICCongestionController *_cc       = nullptr;
 };
diff --git a/iocore/net/quic/QUICPacketRetransmitter.cc b/iocore/net/quic/QUICPacketRetransmitter.cc
index 24e7946..5f9ba33 100644
--- a/iocore/net/quic/QUICPacketRetransmitter.cc
+++ b/iocore/net/quic/QUICPacketRetransmitter.cc
@@ -27,7 +27,6 @@
 void
 QUICPacketRetransmitter::retransmit_packet(const QUICPacket &packet)
 {
-  Debug("quic_con", "Retransmit packet #%" PRIu64 " type %s", packet.packet_number(), QUICDebugNames::packet_type(packet.type()));
   ink_assert(packet.type() != QUICPacketType::VERSION_NEGOTIATION && packet.type() != QUICPacketType::UNINITIALIZED);
 
   // Get payload from a header because packet.payload() is encrypted
diff --git a/iocore/net/quic/QUICStream.cc b/iocore/net/quic/QUICStream.cc
index b4d4507..61df854 100644
--- a/iocore/net/quic/QUICStream.cc
+++ b/iocore/net/quic/QUICStream.cc
@@ -28,22 +28,22 @@
 #include "QUICStreamManager.h"
 #include "QUICDebugNames.h"
 
-#define QUICStreamDebug(fmt, ...)                                                                      \
-  Debug("quic_stream", "[%" PRIx64 "] [%" PRIx64 "] [%s] " fmt, this->_connection_id.l64(), this->_id, \
+#define QUICStreamDebug(fmt, ...)                                                             \
+  Debug("quic_stream", "[%s] [%" PRIx64 "] [%s] " fmt, this->_info->cids().data(), this->_id, \
         QUICDebugNames::stream_state(this->_state), ##__VA_ARGS__)
 
-#define QUICVStreamDebug(fmt, ...)                                                                       \
-  Debug("v_quic_stream", "[%" PRIx64 "] [%" PRIx64 "] [%s] " fmt, this->_connection_id.l64(), this->_id, \
+#define QUICVStreamDebug(fmt, ...)                                                              \
+  Debug("v_quic_stream", "[%s] [%" PRIx64 "] [%s] " fmt, this->_info->cids().data(), this->_id, \
         QUICDebugNames::stream_state(this->_state), ##__VA_ARGS__)
 
-#define QUICStreamFCDebug(fmt, ...)                                                                       \
-  Debug("quic_flow_ctrl", "[%" PRIx64 "] [%" PRIx64 "] [%s] " fmt, this->_connection_id.l64(), this->_id, \
+#define QUICStreamFCDebug(fmt, ...)                                                              \
+  Debug("quic_flow_ctrl", "[%s] [%" PRIx64 "] [%s] " fmt, this->_info->cids().data(), this->_id, \
         QUICDebugNames::stream_state(this->_state), ##__VA_ARGS__)
 
-QUICStream::QUICStream(QUICRTTProvider *rtt_provider, QUICConnectionId cid, QUICStreamId sid, uint64_t recv_max_stream_data,
-                       uint64_t send_max_stream_data)
+QUICStream::QUICStream(QUICRTTProvider *rtt_provider, QUICConnectionInfoProvider *info, QUICStreamId sid,
+                       uint64_t recv_max_stream_data, uint64_t send_max_stream_data)
   : VConnection(nullptr),
-    _connection_id(cid),
+    _info(info),
     _id(sid),
     _remote_flow_controller(send_max_stream_data, _id),
     _local_flow_controller(rtt_provider, recv_max_stream_data, _id),
diff --git a/iocore/net/quic/QUICStream.h b/iocore/net/quic/QUICStream.h
index 5515225..af3afcc 100644
--- a/iocore/net/quic/QUICStream.h
+++ b/iocore/net/quic/QUICStream.h
@@ -34,6 +34,7 @@
 #include "QUICIncomingFrameBuffer.h"
 #include "QUICFrameGenerator.h"
 #include "QUICLossDetector.h"
+#include "QUICConnection.h"
 
 class QUICNetVConnection;
 class QUICStreamState;
@@ -53,7 +54,7 @@ public:
       _received_stream_frame_buffer(this)
   {
   }
-  QUICStream(QUICRTTProvider *rtt_provider, QUICConnectionId cid, QUICStreamId sid, uint64_t recv_max_stream_data = 0,
+  QUICStream(QUICRTTProvider *rtt_provider, QUICConnectionInfoProvider *info, QUICStreamId sid, uint64_t recv_max_stream_data = 0,
              uint64_t send_max_stream_data = 0);
   ~QUICStream();
   // void start();
@@ -102,7 +103,7 @@ private:
 
   QUICStreamState _state;
   QUICStreamErrorUPtr _reset_reason = nullptr;
-  QUICConnectionId _connection_id   = QUICConnectionId::ZERO();
+  QUICConnectionInfoProvider *_info = nullptr;
   QUICStreamId _id                  = 0;
   QUICOffset _send_offset           = 0;
 
diff --git a/iocore/net/quic/QUICStreamManager.cc b/iocore/net/quic/QUICStreamManager.cc
index 026a0f9..b5cd442 100644
--- a/iocore/net/quic/QUICStreamManager.cc
+++ b/iocore/net/quic/QUICStreamManager.cc
@@ -32,9 +32,9 @@ static constexpr char tag[] = "quic_stream_manager";
 ClassAllocator<QUICStreamManager> quicStreamManagerAllocator("quicStreamManagerAllocator");
 ClassAllocator<QUICStream> quicStreamAllocator("quicStreamAllocator");
 
-QUICStreamManager::QUICStreamManager(QUICRTTProvider *rtt_provider, QUICConnectionId cid, QUICApplicationMap *app_map,
+QUICStreamManager::QUICStreamManager(QUICRTTProvider *rtt_provider, QUICConnectionInfoProvider *info, QUICApplicationMap *app_map,
                                      NetVConnectionContext_t context)
-  : _connection_id(cid), _app_map(app_map), _netvc_context(context), _rtt_provider(rtt_provider)
+  : _info(info), _rtt_provider(rtt_provider), _app_map(app_map), _netvc_context(context)
 {
   if (this->_netvc_context == NET_VCONNECTION_OUT) {
     // stream 0 is for handshake, smallest client bidi stream id is 4
@@ -290,7 +290,7 @@ QUICStreamManager::_find_or_create_stream(QUICStreamId stream_id)
 
     // TODO Free the stream somewhere
     stream = THREAD_ALLOC(quicStreamAllocator, this_ethread());
-    new (stream) QUICStream(this->_rtt_provider, this->_connection_id, stream_id, local_max_stream_data, remote_max_stream_data);
+    new (stream) QUICStream(this->_rtt_provider, this->_info, stream_id, local_max_stream_data, remote_max_stream_data);
 
     this->stream_list.push(stream);
   }
diff --git a/iocore/net/quic/QUICStreamManager.h b/iocore/net/quic/QUICStreamManager.h
index 449db9e..88ae800 100644
--- a/iocore/net/quic/QUICStreamManager.h
+++ b/iocore/net/quic/QUICStreamManager.h
@@ -38,7 +38,7 @@ class QUICStreamManager : public QUICFrameHandler, public QUICFrameGenerator
 {
 public:
   QUICStreamManager(){};
-  QUICStreamManager(QUICRTTProvider *rtt_provider, QUICConnectionId cid, QUICApplicationMap *app_map,
+  QUICStreamManager(QUICRTTProvider *rtt_provider, QUICConnectionInfoProvider *info, QUICApplicationMap *app_map,
                     NetVConnectionContext_t context);
 
   void init_flow_control_params(const std::shared_ptr<const QUICTransportParameters> &local_tp,
@@ -77,7 +77,8 @@ private:
   QUICErrorUPtr _handle_frame(const std::shared_ptr<const QUICStreamBlockedFrame> &);
   QUICErrorUPtr _handle_frame(const std::shared_ptr<const QUICMaxStreamIdFrame> &);
 
-  QUICConnectionId _connection_id                           = QUICConnectionId::ZERO();
+  QUICConnectionInfoProvider *_info                         = nullptr;
+  QUICRTTProvider *_rtt_provider                            = nullptr;
   QUICApplicationMap *_app_map                              = nullptr;
   NetVConnectionContext_t _netvc_context                    = NET_VCONNECTION_UNSET;
   std::shared_ptr<const QUICTransportParameters> _local_tp  = nullptr;
@@ -89,5 +90,4 @@ private:
   QUICStreamId _next_stream_id_uni                          = 0;
   QUICStreamId _next_stream_id_bidi                         = 0;
   uint64_t _total_offset_sent                               = 0;
-  QUICRTTProvider *_rtt_provider                            = nullptr;
 };
diff --git a/iocore/net/quic/test/test_QUICFrame.cc b/iocore/net/quic/test/test_QUICFrame.cc
index 27d8279..c455be4 100644
--- a/iocore/net/quic/test/test_QUICFrame.cc
+++ b/iocore/net/quic/test/test_QUICFrame.cc
@@ -1158,7 +1158,7 @@ TEST_CASE("QUICFrameFactory Create CONNECTION_CLOSE with a QUICConnectionError",
 
 TEST_CASE("QUICFrameFactory Create RST_STREAM with a QUICStreamError", "[quic]")
 {
-  QUICStream stream(new MockQUICRTTProvider(), QUICConnectionId::ZERO(), 0x1234, 0, 0);
+  QUICStream stream(new MockQUICRTTProvider(), new MockQUICConnection(), 0x1234, 0, 0);
   std::unique_ptr<QUICStreamError> error =
     std::unique_ptr<QUICStreamError>(new QUICStreamError(&stream, static_cast<QUICAppErrorCode>(0x01)));
   std::unique_ptr<QUICRstStreamFrame, QUICFrameDeleterFunc> rst_stream_frame1 =
diff --git a/iocore/net/quic/test/test_QUICFrameDispatcher.cc b/iocore/net/quic/test/test_QUICFrameDispatcher.cc
index d307d29..e1cbddf 100644
--- a/iocore/net/quic/test/test_QUICFrameDispatcher.cc
+++ b/iocore/net/quic/test/test_QUICFrameDispatcher.cc
@@ -37,7 +37,11 @@ TEST_CASE("QUICFrameHandler", "[quic]")
 
   auto connection    = new MockQUICConnection();
   auto streamManager = new MockQUICStreamManager();
-  auto lossDetector  = new MockQUICLossDetector();
+  auto tx            = new MockQUICPacketTransmitter();
+  auto info          = new MockQUICConnectionInfoProvider();
+  auto cc            = new MockQUICCongestionController(info);
+  auto lossDetector  = new MockQUICLossDetector(tx, info, cc);
+
   QUICFrameDispatcher quicFrameDispatcher;
   quicFrameDispatcher.add_handler(connection);
   quicFrameDispatcher.add_handler(streamManager);
diff --git a/iocore/net/quic/test/test_QUICLossDetector.cc b/iocore/net/quic/test/test_QUICLossDetector.cc
index ae3dce7..26d0f56 100644
--- a/iocore/net/quic/test/test_QUICLossDetector.cc
+++ b/iocore/net/quic/test/test_QUICLossDetector.cc
@@ -34,11 +34,12 @@ TEST_CASE("QUICLossDetector_Loss", "[quic]")
   QUICPacketFactory pf;
   pf.set_hs_protocol(&hs_protocol);
 
-  QUICAckFrameCreator *afc         = new QUICAckFrameCreator();
-  QUICConnectionId connection_id   = {reinterpret_cast<const uint8_t *>("\x01"), 1};
-  MockQUICPacketTransmitter *tx    = new MockQUICPacketTransmitter();
-  MockQUICCongestionController *cc = new MockQUICCongestionController();
-  QUICLossDetector detector(tx, cc);
+  QUICAckFrameCreator *afc             = new QUICAckFrameCreator();
+  QUICConnectionId connection_id       = {reinterpret_cast<const uint8_t *>("\x01"), 1};
+  MockQUICPacketTransmitter *tx        = new MockQUICPacketTransmitter();
+  MockQUICConnectionInfoProvider *info = new MockQUICConnectionInfoProvider();
+  MockQUICCongestionController *cc     = new MockQUICCongestionController(info);
+  QUICLossDetector detector(tx, info, cc);
   ats_unique_buf payload              = ats_unique_malloc(16);
   size_t payload_len                  = 16;
   QUICPacketUPtr packet               = QUICPacketFactory::create_null_packet();
@@ -160,9 +161,10 @@ TEST_CASE("QUICLossDetector_Loss", "[quic]")
 
 TEST_CASE("QUICLossDetector_HugeGap", "[quic]")
 {
-  MockQUICPacketTransmitter *tx    = new MockQUICPacketTransmitter();
-  MockQUICCongestionController *cc = new MockQUICCongestionController();
-  QUICLossDetector detector(tx, cc);
+  MockQUICPacketTransmitter *tx        = new MockQUICPacketTransmitter();
+  MockQUICConnectionInfoProvider *info = new MockQUICConnectionInfoProvider();
+  MockQUICCongestionController *cc     = new MockQUICCongestionController(info);
+  QUICLossDetector detector(tx, info, cc);
 
   // Check initial state
   CHECK(tx->retransmitted.size() == 0);
diff --git a/iocore/net/quic/test/test_QUICStream.cc b/iocore/net/quic/test/test_QUICStream.cc
index b942b03..28d6166 100644
--- a/iocore/net/quic/test/test_QUICStream.cc
+++ b/iocore/net/quic/test/test_QUICStream.cc
@@ -69,7 +69,8 @@ TEST_CASE("QUICStream", "[quic]")
     MIOBuffer *read_buffer = new_MIOBuffer(BUFFER_SIZE_INDEX_4K);
     IOBufferReader *reader = read_buffer->alloc_reader();
 
-    std::unique_ptr<QUICStream> stream(new QUICStream(new MockQUICRTTProvider(), QUICConnectionId::ZERO(), stream_id, 1024, 1024));
+    std::unique_ptr<QUICStream> stream(
+      new QUICStream(new MockQUICRTTProvider(), new MockQUICConnectionInfoProvider(), stream_id, 1024, 1024));
     stream->do_io_read(nullptr, 0, read_buffer);
 
     stream->recv(frame_1);
@@ -95,7 +96,7 @@ TEST_CASE("QUICStream", "[quic]")
     IOBufferReader *reader = read_buffer->alloc_reader();
 
     std::unique_ptr<QUICStream> stream(
-      new QUICStream(new MockQUICRTTProvider(), QUICConnectionId::ZERO(), stream_id, UINT64_MAX, UINT64_MAX));
+      new QUICStream(new MockQUICRTTProvider(), new MockQUICConnectionInfoProvider(), stream_id, UINT64_MAX, UINT64_MAX));
     stream->do_io_read(nullptr, 0, read_buffer);
 
     stream->recv(frame_8);
@@ -121,7 +122,7 @@ TEST_CASE("QUICStream", "[quic]")
     IOBufferReader *reader = read_buffer->alloc_reader();
 
     std::unique_ptr<QUICStream> stream(
-      new QUICStream(new MockQUICRTTProvider(), QUICConnectionId::ZERO(), stream_id, UINT64_MAX, UINT64_MAX));
+      new QUICStream(new MockQUICRTTProvider(), new MockQUICConnectionInfoProvider(), stream_id, UINT64_MAX, UINT64_MAX));
     stream->do_io_read(nullptr, 0, read_buffer);
 
     stream->recv(frame_8);
@@ -150,7 +151,7 @@ TEST_CASE("QUICStream", "[quic]")
     MIOBuffer *read_buffer = new_MIOBuffer(BUFFER_SIZE_INDEX_4K);
     IOBufferReader *reader = read_buffer->alloc_reader();
 
-    std::unique_ptr<QUICStream> stream(new QUICStream(new MockQUICRTTProvider(), QUICConnectionId::ZERO(), stream_id));
+    std::unique_ptr<QUICStream> stream(new QUICStream(new MockQUICRTTProvider(), new MockQUICConnectionInfoProvider(), stream_id));
     stream->init_flow_control_params(4096, 4096);
     stream->do_io_read(nullptr, 0, read_buffer);
 
@@ -186,7 +187,7 @@ TEST_CASE("QUICStream", "[quic]")
     IOBufferReader *read_buffer_reader  = read_buffer->alloc_reader();
     IOBufferReader *write_buffer_reader = write_buffer->alloc_reader();
 
-    std::unique_ptr<QUICStream> stream(new QUICStream(new MockQUICRTTProvider(), QUICConnectionId::ZERO(), stream_id));
+    std::unique_ptr<QUICStream> stream(new QUICStream(new MockQUICRTTProvider(), new MockQUICConnectionInfoProvider(), stream_id));
     stream->init_flow_control_params(4096, 4096);
     MockContinuation mock_cont(stream->mutex);
     stream->do_io_read(nullptr, 0, read_buffer);
diff --git a/iocore/net/quic/test/test_QUICStreamManager.cc b/iocore/net/quic/test/test_QUICStreamManager.cc
index e500db0..4ee6f10 100644
--- a/iocore/net/quic/test/test_QUICStreamManager.cc
+++ b/iocore/net/quic/test/test_QUICStreamManager.cc
@@ -34,7 +34,7 @@ TEST_CASE("QUICStreamManager_NewStream", "[quic]")
   QUICApplicationMap app_map;
   MockQUICApplication mock_app;
   app_map.set_default(&mock_app);
-  QUICStreamManager sm(new MockQUICRTTProvider(), QUICConnectionId::ZERO(), &app_map, NET_VCONNECTION_IN);
+  QUICStreamManager sm(new MockQUICRTTProvider(), new MockQUICConnectionInfoProvider(), &app_map, NET_VCONNECTION_IN);
   std::shared_ptr<QUICTransportParameters> local_tp =
     std::make_shared<QUICTransportParametersInEncryptedExtensions>(static_cast<QUICVersion>(0));
   std::shared_ptr<QUICTransportParameters> remote_tp =
@@ -80,7 +80,7 @@ TEST_CASE("QUICStreamManager_first_initial_map", "[quic]")
   QUICApplicationMap app_map;
   MockQUICApplication mock_app;
   app_map.set_default(&mock_app);
-  QUICStreamManager sm(new MockQUICRTTProvider(), QUICConnectionId::ZERO(), &app_map, NET_VCONNECTION_IN);
+  QUICStreamManager sm(new MockQUICRTTProvider(), new MockQUICConnectionInfoProvider(), &app_map, NET_VCONNECTION_IN);
   std::shared_ptr<QUICTransportParameters> local_tp =
     std::make_shared<QUICTransportParametersInEncryptedExtensions>(static_cast<QUICVersion>(0));
   std::shared_ptr<QUICTransportParameters> remote_tp =
@@ -100,7 +100,7 @@ TEST_CASE("QUICStreamManager_total_offset_received", "[quic]")
   QUICApplicationMap app_map;
   MockQUICApplication mock_app;
   app_map.set_default(&mock_app);
-  QUICStreamManager sm(new MockQUICRTTProvider(), QUICConnectionId::ZERO(), &app_map, NET_VCONNECTION_IN);
+  QUICStreamManager sm(new MockQUICRTTProvider(), new MockQUICConnectionInfoProvider(), &app_map, NET_VCONNECTION_IN);
   std::shared_ptr<QUICTransportParameters> local_tp =
     std::make_shared<QUICTransportParametersInEncryptedExtensions>(static_cast<QUICVersion>(0));
   local_tp->set(QUICTransportParameterId::INITIAL_MAX_STREAM_DATA, UINT32_C(4096));
@@ -133,7 +133,7 @@ TEST_CASE("QUICStreamManager_total_offset_sent", "[quic]")
   QUICApplicationMap app_map;
   MockQUICApplication mock_app;
   app_map.set_default(&mock_app);
-  QUICStreamManager sm(new MockQUICRTTProvider(), QUICConnectionId::ZERO(), &app_map, NET_VCONNECTION_IN);
+  QUICStreamManager sm(new MockQUICRTTProvider(), new MockQUICConnectionInfoProvider(), &app_map, NET_VCONNECTION_IN);
   std::shared_ptr<QUICTransportParameters> local_tp =
     std::make_shared<QUICTransportParametersInEncryptedExtensions>(static_cast<QUICVersion>(0));
   local_tp->set(QUICTransportParameterId::INITIAL_MAX_STREAM_DATA, UINT32_C(4096));
diff --git a/proxy/hq/HQClientSession.cc b/proxy/hq/HQClientSession.cc
index f946847..3761e7f 100644
--- a/proxy/hq/HQClientSession.cc
+++ b/proxy/hq/HQClientSession.cc
@@ -85,7 +85,7 @@ HQClientSession::start()
 void
 HQClientSession::new_connection(NetVConnection *new_vc, MIOBuffer *iobuf, IOBufferReader *reader, bool backdoor)
 {
-  this->con_id = ProxyClientSession::next_connection_id();
+  this->con_id = static_cast<QUICConnection *>(reinterpret_cast<QUICNetVConnection *>(new_vc))->connection_id();
 
   return;
 }
diff --git a/proxy/hq/HQClientTransaction.cc b/proxy/hq/HQClientTransaction.cc
index 6766b2c..455f645 100644
--- a/proxy/hq/HQClientTransaction.cc
+++ b/proxy/hq/HQClientTransaction.cc
@@ -32,10 +32,9 @@
 #include "HQDataFramer.h"
 #include "HttpSM.h"
 
-#define HQTransDebug(fmt, ...)                                                                                                \
-  Debug("hq_trans", "[%" PRIx64 "] [%" PRIx32 "] " fmt,                                                                       \
-        static_cast<uint64_t>(                                                                                                \
-          static_cast<QUICConnection *>(reinterpret_cast<QUICNetVConnection *>(this->parent->get_netvc()))->connection_id()), \
+#define HQTransDebug(fmt, ...)                                                                                           \
+  Debug("hq_trans", "[%s] [%" PRIx32 "] " fmt,                                                                           \
+        static_cast<QUICConnection *>(reinterpret_cast<QUICNetVConnection *>(this->parent->get_netvc()))->cids().data(), \
         this->get_transaction_id(), ##__VA_ARGS__)
 
 // static void

-- 
To stop receiving notification emails like this one, please contact
masaori@apache.org.