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:52 UTC

[trafficserver] branch quic-latest updated (997729b -> 4712f8a)

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

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


    from 997729b  Fix deleter of QUICPathResponseFrame unique pointer
     new e2c5a0f  Rename QUICConnectionInfo QUICConnectionInfoProvider
     new 976cec2  Print dcid and scid in debug log to identify quic connection
     new 4712f8a  Refer QUICConnectionInfoProviter when print debug log

The 3 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 cmd/traffic_quic/quic_client.cc                  |  3 +-
 iocore/net/P_QUICNetVConnection.h                | 18 +++--
 iocore/net/QUICNetProcessor.cc                   |  8 +--
 iocore/net/QUICNetVConnection.cc                 | 80 ++++++++++++++++------
 iocore/net/QUICPacketHandler.cc                  | 37 ++++++----
 iocore/net/quic/Mock.h                           | 87 +++++++++++++++++++++++-
 iocore/net/quic/QUICApplication.cc               |  2 +-
 iocore/net/quic/QUICCongestionController.cc      | 12 ++--
 iocore/net/quic/QUICConnection.h                 |  5 +-
 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/QUICTypes.cc                     |  6 ++
 iocore/net/quic/QUICTypes.h                      |  1 +
 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 +-
 26 files changed, 260 insertions(+), 115 deletions(-)

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

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

Posted by ma...@apache.org.
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.

[trafficserver] 02/03: Print dcid and scid in debug log to identify quic connection

Posted by ma...@apache.org.
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 976cec2ac7fc7932a0890ab8decdd532a7c1f3b5
Author: Masaori Koshiba <ma...@apache.org>
AuthorDate: Tue May 29 14:05:41 2018 +0900

    Print dcid and scid in debug log to identify quic connection
    
    Only change logs in QUICNetVConnection. Other components will be fixed.
---
 iocore/net/P_QUICNetVConnection.h | 16 ++++++++--
 iocore/net/QUICNetVConnection.cc  | 64 ++++++++++++++++++++++++++++-----------
 iocore/net/quic/Mock.h            |  6 ++++
 iocore/net/quic/QUICConnection.h  |  1 +
 iocore/net/quic/QUICTypes.cc      |  6 ++++
 iocore/net/quic/QUICTypes.h       |  1 +
 6 files changed, 73 insertions(+), 21 deletions(-)

diff --git a/iocore/net/P_QUICNetVConnection.h b/iocore/net/P_QUICNetVConnection.h
index 94f6ccf..3398aa6 100644
--- a/iocore/net/P_QUICNetVConnection.h
+++ b/iocore/net/P_QUICNetVConnection.h
@@ -74,6 +74,9 @@
 
 #define QUIC_OP_HANDSHAKE 0x16
 
+// Size of connection ids for debug log : e.g. aaaaaaaa-bbbbbbbb\0
+static constexpr size_t MAX_CIDS_SIZE = 8 + 1 + 8 + 1;
+
 // class QUICNextProtocolSet;
 // struct QUICCertLookup;
 
@@ -195,6 +198,7 @@ public:
   QUICConnectionId peer_connection_id() const override;
   QUICConnectionId original_connection_id() const override;
   QUICConnectionId connection_id() const override;
+  std::string_view cids() const override;
   const QUICFiveTuple five_tuple() const override;
   uint32_t maximum_quic_packet_size() const override;
   uint32_t minimum_quic_packet_size() override;
@@ -228,11 +232,14 @@ private:
   QUICPacketType _last_received_packet_type = QUICPacketType::UNINITIALIZED;
   std::random_device _rnd;
 
-  QUICConnectionId _peer_quic_connection_id;
-  QUICConnectionId _original_quic_connection_id;
-  QUICConnectionId _quic_connection_id;
+  QUICConnectionId _peer_quic_connection_id;     // dst cid
+  QUICConnectionId _original_quic_connection_id; // dst cid of initial packet from client
+  QUICConnectionId _quic_connection_id;          // src cid
   QUICFiveTuple _five_tuple;
 
+  char _cids_data[MAX_CIDS_SIZE] = {0};
+  std::string_view _cids;
+
   UDPConnection *_udp_con            = nullptr;
   QUICPacketHandler *_packet_handler = nullptr;
   QUICPacketFactory _packet_factory;
@@ -332,6 +339,9 @@ private:
 
   void _handle_idle_timeout();
 
+  void _update_cids();
+  void _update_peer_cid(const QUICConnectionId &new_cid);
+
   QUICPacketUPtr _the_final_packet = QUICPacketFactory::create_null_packet();
   QUICStatelessResetToken _reset_token;
 
diff --git a/iocore/net/QUICNetVConnection.cc b/iocore/net/QUICNetVConnection.cc
index 8126bac..bd216a7 100644
--- a/iocore/net/QUICNetVConnection.cc
+++ b/iocore/net/QUICNetVConnection.cc
@@ -42,13 +42,15 @@
 #define STATE_FROM_VIO(_x) ((NetState *)(((char *)(_x)) - STATE_VIO_OFFSET))
 #define STATE_VIO_OFFSET ((uintptr_t) & ((NetState *)0)->vio)
 
-#define QUICConDebug(fmt, ...) Debug("quic_net", "[%" PRIx64 "] " fmt, this->_peer_quic_connection_id.l64(), ##__VA_ARGS__)
+#define QUICConDebug(fmt, ...) Debug("quic_net", "[%s] " fmt, this->cids().data(), ##__VA_ARGS__)
 
-#define QUICConVDebug(fmt, ...) Debug("v_quic_net", "[%" PRIx64 "] " fmt, this->_peer_quic_connection_id.l64(), ##__VA_ARGS__)
+#define QUICConVDebug(fmt, ...) Debug("v_quic_net", "[%s] " fmt, this->cids().data(), ##__VA_ARGS__)
 
-#define QUICError(fmt, ...)                                                                     \
-  Debug("quic_net", "[%" PRIx64 "] " fmt, this->_peer_quic_connection_id.l64(), ##__VA_ARGS__); \
-  Error("quic_net [%" PRIx64 "] " fmt, this->_peer_quic_connection_id.l64(), ##__VA_ARGS__)
+#define QUICFCDebug(fmt, ...) Debug("quic_flow_ctrl", "[%s] " fmt, this->cids().data(), ##__VA_ARGS__)
+
+#define QUICError(fmt, ...)                                           \
+  Debug("quic_net", "[%s] " fmt, this->cids().data(), ##__VA_ARGS__); \
+  Error("quic_net [%s] " fmt, this->cids().data(), ##__VA_ARGS__)
 
 static constexpr uint32_t IPV4_HEADER_SIZE            = 20;
 static constexpr uint32_t IPV6_HEADER_SIZE            = 40;
@@ -84,8 +86,10 @@ QUICNetVConnection::init(QUICConnectionId peer_cid, QUICConnectionId original_ci
     this->_ctable->insert(this->_original_quic_connection_id, this);
   }
 
-  QUICConDebug("Connection ID %" PRIx64 " has been changed to %" PRIx64, this->_original_quic_connection_id.l64(),
-               this->_quic_connection_id.l64());
+  this->_update_cids();
+
+  // TODO: print full cids
+  QUICConDebug("dcid=%08" PRIx32 " scid=%08" PRIx32, this->_peer_quic_connection_id.h32(), this->_quic_connection_id.h32());
 }
 
 bool
@@ -339,6 +343,12 @@ QUICNetVConnection::connection_id() const
   return this->_quic_connection_id;
 }
 
+std::string_view
+QUICNetVConnection::cids() const
+{
+  return this->_cids;
+}
+
 const QUICFiveTuple
 QUICNetVConnection::five_tuple() const
 {
@@ -457,8 +467,8 @@ QUICNetVConnection::handle_frame(std::shared_ptr<const QUICFrame> frame)
   switch (frame->type()) {
   case QUICFrameType::MAX_DATA:
     this->_remote_flow_controller->forward_limit(std::static_pointer_cast<const QUICMaxDataFrame>(frame)->maximum_data());
-    Debug("quic_flow_ctrl", "Connection [%" PRIx64 "] [REMOTE] %" PRIu64 "/%" PRIu64, this->_peer_quic_connection_id.l64(),
-          this->_remote_flow_controller->current_offset(), this->_remote_flow_controller->current_limit());
+    QUICFCDebug("[REMOTE] %" PRIu64 "/%" PRIu64, this->_remote_flow_controller->current_offset(),
+                this->_remote_flow_controller->current_limit());
     this->_schedule_packet_write_ready();
 
     break;
@@ -1247,8 +1257,8 @@ QUICNetVConnection::_packetize_frames()
     ++frame_count;
     if (frame->type() == QUICFrameType::STREAM) {
       int ret = this->_remote_flow_controller->update(this->_stream_manager->total_offset_sent());
-      Debug("quic_flow_ctrl", "Connection [%" PRIx64 "] [REMOTE] %" PRIu64 "/%" PRIu64, this->_peer_quic_connection_id.l64(),
-            this->_remote_flow_controller->current_offset(), this->_remote_flow_controller->current_limit());
+      QUICFCDebug("[REMOTE] %" PRIu64 "/%" PRIu64, this->_remote_flow_controller->current_offset(),
+                  this->_remote_flow_controller->current_limit());
       ink_assert(ret == 0);
     }
     this->_store_frame(buf, len, retransmittable, current_packet_type, std::move(frame));
@@ -1319,8 +1329,8 @@ QUICNetVConnection::_recv_and_ack(QUICPacketUPtr packet)
   }
 
   int ret = this->_local_flow_controller->update(this->_stream_manager->total_offset_received());
-  Debug("quic_flow_ctrl", "Connection [%" PRIx64 "] [LOCAL] %" PRIu64 "/%" PRIu64, this->_peer_quic_connection_id.l64(),
-        this->_local_flow_controller->current_offset(), this->_local_flow_controller->current_limit());
+  QUICFCDebug("[LOCAL] %" PRIu64 "/%" PRIu64, this->_local_flow_controller->current_offset(),
+              this->_local_flow_controller->current_limit());
   if (ret != 0) {
     return QUICErrorUPtr(new QUICConnectionError(QUICTransErrorCode::FLOW_CONTROL_ERROR));
   }
@@ -1410,10 +1420,10 @@ QUICNetVConnection::_init_flow_control_params(const std::shared_ptr<const QUICTr
 
   this->_local_flow_controller->set_limit(local_initial_max_data);
   this->_remote_flow_controller->set_limit(remote_initial_max_data);
-  Debug("quic_flow_ctrl", "Connection [%" PRIx64 "] [LOCAL] %" PRIu64 "/%" PRIu64, this->_peer_quic_connection_id.l64(),
-        this->_local_flow_controller->current_offset(), this->_local_flow_controller->current_limit());
-  Debug("quic_flow_ctrl", "Connection [%" PRIx64 "] [REMOTE] %" PRIu64 "/%" PRIu64, this->_peer_quic_connection_id.l64(),
-        this->_remote_flow_controller->current_offset(), this->_remote_flow_controller->current_limit());
+  QUICFCDebug("[LOCAL] %" PRIu64 "/%" PRIu64, this->_local_flow_controller->current_offset(),
+              this->_local_flow_controller->current_limit());
+  QUICFCDebug("[REMOTE] %" PRIu64 "/%" PRIu64, this->_remote_flow_controller->current_offset(),
+              this->_remote_flow_controller->current_limit());
 }
 
 void
@@ -1451,7 +1461,7 @@ QUICNetVConnection::_dequeue_recv_packet(QUICPacketCreationResult &result)
       // FIXME src connection id could be zero ? if so, check packet header type.
       if (src_cid != QUICConnectionId::ZERO()) {
         if (this->_peer_quic_connection_id != src_cid) {
-          this->_peer_quic_connection_id = src_cid;
+          this->_update_peer_cid(src_cid);
         }
       }
     }
@@ -1752,3 +1762,21 @@ QUICNetVConnection::_validate_new_path()
   ink_hrtime rto = this->_loss_detector->current_rto_period();
   this->_schedule_path_validation_timeout(3 * rto);
 }
+
+void
+QUICNetVConnection::_update_cids()
+{
+  snprintf(this->_cids_data, sizeof(this->_cids_data), "%08" PRIx32 "-%08" PRIx32 "", this->_peer_quic_connection_id.h32(),
+           this->_quic_connection_id.h32());
+
+  this->_cids = {this->_cids_data, sizeof(this->_cids_data)};
+}
+
+void
+QUICNetVConnection::_update_peer_cid(const QUICConnectionId &new_cid)
+{
+  QUICConDebug("dcid: %08" PRIx32 " -> %08" PRIx32, this->_peer_quic_connection_id.h32(), new_cid.h32());
+
+  this->_peer_quic_connection_id = new_cid;
+  this->_update_cids();
+}
diff --git a/iocore/net/quic/Mock.h b/iocore/net/quic/Mock.h
index b713554..761e96f 100644
--- a/iocore/net/quic/Mock.h
+++ b/iocore/net/quic/Mock.h
@@ -169,6 +169,12 @@ public:
     return QUICFiveTuple();
   }
 
+  std::string_view
+  cids() const override
+  {
+    return std::string_view("00000000-00000000"sv);
+  }
+
   uint32_t
   transmit_packet(QUICPacketUPtr packet) override
   {
diff --git a/iocore/net/quic/QUICConnection.h b/iocore/net/quic/QUICConnection.h
index 1f7cd8f..2b4b5e1 100644
--- a/iocore/net/quic/QUICConnection.h
+++ b/iocore/net/quic/QUICConnection.h
@@ -39,6 +39,7 @@ public:
   virtual QUICConnectionId peer_connection_id() const     = 0;
   virtual QUICConnectionId original_connection_id() const = 0;
   virtual QUICConnectionId connection_id() const          = 0;
+  virtual std::string_view cids() const                   = 0;
   virtual const QUICFiveTuple five_tuple() const          = 0;
 
   /*
diff --git a/iocore/net/quic/QUICTypes.cc b/iocore/net/quic/QUICTypes.cc
index 33a43e2..be89426 100644
--- a/iocore/net/quic/QUICTypes.cc
+++ b/iocore/net/quic/QUICTypes.cc
@@ -285,3 +285,9 @@ QUICConnectionId::l64() const
   }
   return v;
 }
+
+uint32_t
+QUICConnectionId::h32() const
+{
+  return static_cast<uint32_t>(QUICIntUtil::read_nbytes_as_uint(this->_id, 4));
+}
diff --git a/iocore/net/quic/QUICTypes.h b/iocore/net/quic/QUICTypes.h
index abc8ea1..7b26777 100644
--- a/iocore/net/quic/QUICTypes.h
+++ b/iocore/net/quic/QUICTypes.h
@@ -245,6 +245,7 @@ public:
    * This is just for debugging.
    */
   uint64_t l64() const;
+  uint32_t h32() const;
 
   uint8_t length() const;
   bool is_zero() const;

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

[trafficserver] 01/03: Rename QUICConnectionInfo QUICConnectionInfoProvider

Posted by ma...@apache.org.
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 e2c5a0fdf4762592fec406eff9cc89ec79567596
Author: Masaori Koshiba <ma...@apache.org>
AuthorDate: Thu May 31 10:59:58 2018 +0900

    Rename QUICConnectionInfo QUICConnectionInfoProvider
---
 iocore/net/P_QUICNetVConnection.h | 2 +-
 iocore/net/quic/QUICConnection.h  | 4 ++--
 2 files changed, 3 insertions(+), 3 deletions(-)

diff --git a/iocore/net/P_QUICNetVConnection.h b/iocore/net/P_QUICNetVConnection.h
index 07884d2..94f6ccf 100644
--- a/iocore/net/P_QUICNetVConnection.h
+++ b/iocore/net/P_QUICNetVConnection.h
@@ -191,7 +191,7 @@ public:
   void close(QUICConnectionErrorUPtr error) override;
   void handle_received_packet(UDPPacket *packet) override;
 
-  // QUICConnection (QUICConnectionInfo)
+  // QUICConnection (QUICConnectionInfoProvider)
   QUICConnectionId peer_connection_id() const override;
   QUICConnectionId original_connection_id() const override;
   QUICConnectionId connection_id() const override;
diff --git a/iocore/net/quic/QUICConnection.h b/iocore/net/quic/QUICConnection.h
index f0048cd..1f7cd8f 100644
--- a/iocore/net/quic/QUICConnection.h
+++ b/iocore/net/quic/QUICConnection.h
@@ -33,7 +33,7 @@ class QUICStreamManager;
 class UDPPacket;
 class SSLNextProtocolSet;
 
-class QUICConnectionInfo
+class QUICConnectionInfoProvider
 {
 public:
   virtual QUICConnectionId peer_connection_id() const     = 0;
@@ -63,7 +63,7 @@ public:
   virtual QUICPacketNumber largest_acked_packet_number() const = 0;
 };
 
-class QUICConnection : public QUICPacketTransmitter, public QUICFrameHandler, public QUICConnectionInfo
+class QUICConnection : public QUICPacketTransmitter, public QUICFrameHandler, public QUICConnectionInfoProvider
 {
 public:
   virtual QUICStreamManager *stream_manager()             = 0;

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