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

[trafficserver] branch quic-latest updated (8185c20 -> 6ff1a3f)

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 8185c20  Add nullptr check before calling QUICConnectionTable::erase()
     new fc1bc36  Fix QUICPacketFactory for Version Negotiation Packet
     new b778c96  Remove version args from create_initial_packet
     new 528f1ee  Add VERSION_NEGOTIATION packet support to QUICVersionNegotiator
     new 6ff1a3f  Add Version Negotiation support on QUIC client

The 4 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:
 iocore/net/P_QUICNetVConnection.h                  |  1 +
 iocore/net/QUICNetVConnection.cc                   | 23 +++++++-
 iocore/net/quic/QUICConfig.cc                      |  8 +++
 iocore/net/quic/QUICConfig.h                       |  3 +
 iocore/net/quic/QUICHandshake.cc                   | 52 ++++++++++++++++--
 iocore/net/quic/QUICHandshake.h                    |  4 +-
 iocore/net/quic/QUICPacket.cc                      | 42 +++++++++-----
 iocore/net/quic/QUICPacket.h                       |  6 +-
 iocore/net/quic/QUICTransportParameters.cc         | 12 ++++
 iocore/net/quic/QUICTransportParameters.h          |  1 +
 iocore/net/quic/QUICTypes.h                        |  2 +
 iocore/net/quic/QUICVersionNegotiator.cc           | 63 +++++++++++++++++++--
 iocore/net/quic/QUICVersionNegotiator.h            |  1 +
 iocore/net/quic/test/test_QUICPacketFactory.cc     |  6 +-
 .../net/quic/test/test_QUICTransportParameters.cc  |  1 +
 iocore/net/quic/test/test_QUICVersionNegotiator.cc | 64 +++++++++++++++++++---
 mgmt/RecordsConfig.cc                              |  2 +
 17 files changed, 251 insertions(+), 40 deletions(-)

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

[trafficserver] 04/04: Add Version Negotiation support on QUIC client

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 6ff1a3f373f7af9339285c6f4b96d5636ca8912c
Author: Masaori Koshiba <ma...@apache.org>
AuthorDate: Wed Apr 4 15:15:47 2018 +0900

    Add Version Negotiation support on QUIC client
    
    To enforce version negotiation exercise, set below config 1.
    ```
    proxy.config.quic.client.vn_exercise_enabled
    ```
---
 iocore/net/P_QUICNetVConnection.h |  1 +
 iocore/net/QUICNetVConnection.cc  | 21 ++++++++++++++++-
 iocore/net/quic/QUICConfig.cc     |  8 +++++++
 iocore/net/quic/QUICConfig.h      |  3 +++
 iocore/net/quic/QUICHandshake.cc  | 49 +++++++++++++++++++++++++++++++++++----
 iocore/net/quic/QUICHandshake.h   |  4 +++-
 mgmt/RecordsConfig.cc             |  2 ++
 7 files changed, 82 insertions(+), 6 deletions(-)

diff --git a/iocore/net/P_QUICNetVConnection.h b/iocore/net/P_QUICNetVConnection.h
index eb7defd..e68b2e2 100644
--- a/iocore/net/P_QUICNetVConnection.h
+++ b/iocore/net/P_QUICNetVConnection.h
@@ -285,6 +285,7 @@ private:
   QUICErrorUPtr _recv_and_ack(QUICPacketUPtr packet);
 
   QUICErrorUPtr _state_handshake_process_packet(QUICPacketUPtr packet);
+  QUICErrorUPtr _state_handshake_process_version_negotiation_packet(QUICPacketUPtr packet);
   QUICErrorUPtr _state_handshake_process_initial_packet(QUICPacketUPtr packet);
   QUICErrorUPtr _state_handshake_process_retry_packet(QUICPacketUPtr packet);
   QUICErrorUPtr _state_handshake_process_handshake_packet(QUICPacketUPtr packet);
diff --git a/iocore/net/QUICNetVConnection.cc b/iocore/net/QUICNetVConnection.cc
index a22a7b1..6ea5de4 100644
--- a/iocore/net/QUICNetVConnection.cc
+++ b/iocore/net/QUICNetVConnection.cc
@@ -186,7 +186,7 @@ QUICNetVConnection::start()
     this->_handshake_handler = new QUICHandshake(this, params->server_ssl_ctx(), this->_reset_token, params->stateless_retry());
   } else {
     this->_handshake_handler = new QUICHandshake(this, params->client_ssl_ctx());
-    this->_handshake_handler->start(&this->_packet_factory);
+    this->_handshake_handler->start(&this->_packet_factory, params->vn_exercise_enabled());
   }
   this->_application_map = new QUICApplicationMap();
   this->_application_map->set(STREAM_ID_FOR_HANDSHAKE, this->_handshake_handler);
@@ -783,6 +783,9 @@ QUICNetVConnection::_state_handshake_process_packet(QUICPacketUPtr packet)
 {
   QUICErrorUPtr error = QUICErrorUPtr(new QUICNoError());
   switch (packet->type()) {
+  case QUICPacketType::VERSION_NEGOTIATION:
+    error = this->_state_handshake_process_version_negotiation_packet(std::move(packet));
+    break;
   case QUICPacketType::INITIAL:
     error = this->_state_handshake_process_initial_packet(std::move(packet));
     break;
@@ -806,6 +809,22 @@ QUICNetVConnection::_state_handshake_process_packet(QUICPacketUPtr packet)
 }
 
 QUICErrorUPtr
+QUICNetVConnection::_state_handshake_process_version_negotiation_packet(QUICPacketUPtr packet)
+{
+  QUICErrorUPtr error = QUICErrorUPtr(new QUICNoError());
+
+  if (packet->connection_id() != this->connection_id()) {
+    QUICConDebug("Ignore Version Negotiation packet");
+    return error;
+  }
+
+  error = this->_handshake_handler->negotiate_version(packet.get(), &this->_packet_factory);
+  // Initial packet will be retransmited with negotiated version
+
+  return error;
+}
+
+QUICErrorUPtr
 QUICNetVConnection::_state_handshake_process_initial_packet(QUICPacketUPtr packet)
 {
   if (packet->size() < MINIMUM_INITIAL_PACKET_SIZE) {
diff --git a/iocore/net/quic/QUICConfig.cc b/iocore/net/quic/QUICConfig.cc
index 5d5654b..897002b 100644
--- a/iocore/net/quic/QUICConfig.cc
+++ b/iocore/net/quic/QUICConfig.cc
@@ -121,6 +121,8 @@ QUICConfigParams::initialize()
   REC_EstablishStaticConfigInt32U(this->_server_id, "proxy.config.quic.server_id");
   REC_EstablishStaticConfigInt32(this->_connection_table_size, "proxy.config.quic.connection_table.size");
   REC_EstablishStaticConfigInt32U(this->_stateless_retry, "proxy.config.quic.stateless_retry");
+  REC_EstablishStaticConfigInt32U(this->_vn_exercise_enabled, "proxy.config.quic.client.vn_exercise_enabled");
+
   REC_ReadConfigStringAlloc(this->_server_supported_groups, "proxy.config.quic.server.supported_groups");
   REC_ReadConfigStringAlloc(this->_client_supported_groups, "proxy.config.quic.client.supported_groups");
 
@@ -184,6 +186,12 @@ QUICConfigParams::stateless_retry() const
 }
 
 uint32_t
+QUICConfigParams::vn_exercise_enabled() const
+{
+  return this->_vn_exercise_enabled;
+}
+
+uint32_t
 QUICConfigParams::initial_max_data() const
 {
   return this->_initial_max_data;
diff --git a/iocore/net/quic/QUICConfig.h b/iocore/net/quic/QUICConfig.h
index af76499..3de36c7 100644
--- a/iocore/net/quic/QUICConfig.h
+++ b/iocore/net/quic/QUICConfig.h
@@ -46,6 +46,8 @@ public:
   uint32_t server_id() const;
   static int connection_table_size();
   uint32_t stateless_retry() const;
+  uint32_t vn_exercise_enabled() const;
+
   const char *server_supported_groups() const;
   const char *client_supported_groups() const;
 
@@ -76,6 +78,7 @@ private:
   uint32_t _initial_max_stream_data = 0;
   uint32_t _server_id               = 0;
   uint32_t _stateless_retry         = 0;
+  uint32_t _vn_exercise_enabled     = 0;
 
   uint32_t _initial_max_stream_id_bidi_in  = 100;
   uint32_t _initial_max_stream_id_bidi_out = 101;
diff --git a/iocore/net/quic/QUICHandshake.cc b/iocore/net/quic/QUICHandshake.cc
index f6a6cec..5d2484e 100644
--- a/iocore/net/quic/QUICHandshake.cc
+++ b/iocore/net/quic/QUICHandshake.cc
@@ -116,10 +116,16 @@ QUICHandshake::~QUICHandshake()
 }
 
 QUICErrorUPtr
-QUICHandshake::start(QUICPacketFactory *packet_factory)
+QUICHandshake::start(QUICPacketFactory *packet_factory, bool vn_exercise_enabled)
 {
-  this->_load_local_client_transport_parameters(QUIC_SUPPORTED_VERSIONS[0]);
-  packet_factory->set_version(QUIC_SUPPORTED_VERSIONS[0]);
+  QUICVersion initital_version = QUIC_SUPPORTED_VERSIONS[0];
+  if (vn_exercise_enabled) {
+    initital_version = QUIC_EXERCISE_VERSIONS;
+  }
+
+  this->_load_local_client_transport_parameters(initital_version);
+  packet_factory->set_version(initital_version);
+
   return QUICErrorUPtr(new QUICNoError());
 }
 
@@ -147,6 +153,36 @@ QUICHandshake::start(const QUICPacket *initial_packet, QUICPacketFactory *packet
   return QUICErrorUPtr(new QUICNoError());
 }
 
+QUICErrorUPtr
+QUICHandshake::negotiate_version(const QUICPacket *vn, QUICPacketFactory *packet_factory)
+{
+  // Client side only
+  ink_assert(this->_netvc_context == NET_VCONNECTION_OUT);
+
+  // If already negotiated, just ignore it
+  if (this->_version_negotiator->status() == QUICVersionNegotiationStatus::NEGOTIATED ||
+      this->_version_negotiator->status() == QUICVersionNegotiationStatus::VALIDATED) {
+    QUICHSDebug("Ignore Version Negotiation packet");
+    return QUICErrorUPtr(new QUICNoError());
+  }
+
+  if (vn->version() != 0x00) {
+    QUICHSDebug("Version field must be 0x00000000");
+    return QUICErrorUPtr(new QUICConnectionError(QUICTransErrorCode::PROTOCOL_VIOLATION));
+  }
+
+  if (this->_version_negotiator->negotiate(vn) == QUICVersionNegotiationStatus::NEGOTIATED) {
+    QUICVersion version = this->_version_negotiator->negotiated_version();
+    QUICHSDebug("Version negotiation succeeded: 0x%x", version);
+    packet_factory->set_version(version);
+  } else {
+    QUICHSDebug("Version negotiation failed");
+    return QUICErrorUPtr(new QUICConnectionError(QUICTransErrorCode::VERSION_NEGOTIATION_ERROR));
+  }
+
+  return QUICErrorUPtr(new QUICNoError());
+}
+
 bool
 QUICHandshake::is_version_negotiated() const
 {
@@ -235,7 +271,12 @@ QUICHandshake::set_transport_parameters(std::shared_ptr<QUICTransportParametersI
 
   this->_remote_transport_parameters = tp;
 
-  // TODO Add client side implementation
+  // Version revalidation
+  if (this->_version_negotiator->validate(tp.get()) != QUICVersionNegotiationStatus::VALIDATED) {
+    QUICHSDebug("Version revalidation failed");
+    this->_abort_handshake(QUICTransErrorCode::VERSION_NEGOTIATION_ERROR);
+    return;
+  }
 
   return;
 }
diff --git a/iocore/net/quic/QUICHandshake.h b/iocore/net/quic/QUICHandshake.h
index 2005041..37e316e 100644
--- a/iocore/net/quic/QUICHandshake.h
+++ b/iocore/net/quic/QUICHandshake.h
@@ -52,7 +52,9 @@ public:
   ~QUICHandshake();
 
   // for client side
-  QUICErrorUPtr start(QUICPacketFactory *packet_factory);
+  QUICErrorUPtr start(QUICPacketFactory *packet_factory, bool vn_exercise_enabled);
+  QUICErrorUPtr negotiate_version(const QUICPacket *packet, QUICPacketFactory *packet_factory);
+
   // for server side
   QUICErrorUPtr start(const QUICPacket *initial_packet, QUICPacketFactory *packet_factory);
 
diff --git a/mgmt/RecordsConfig.cc b/mgmt/RecordsConfig.cc
index 4705eed..b69e2b5 100644
--- a/mgmt/RecordsConfig.cc
+++ b/mgmt/RecordsConfig.cc
@@ -1330,6 +1330,8 @@ static const RecordElement RecordsConfig[] =
   ,
   {RECT_CONFIG, "proxy.config.quic.stateless_retry", RECD_INT, "0", RECU_RESTART_TS, RR_NULL, RECC_INT, "[0-1]", RECA_NULL}
   ,
+  {RECT_CONFIG, "proxy.config.quic.client.vn_exercise_enabled", RECD_INT, "0", RECU_DYNAMIC, RR_NULL, RECC_INT, "[0-1]", RECA_NULL}
+  ,
   {RECT_CONFIG, "proxy.config.quic.server.supported_groups", RECD_STRING, "P-256:X25519:P-384:P-521" , RECU_RESTART_TS, RR_NULL, RECC_NULL, nullptr, RECA_NULL}
   ,
   {RECT_CONFIG, "proxy.config.quic.client.supported_groups", RECD_STRING, "P-256:X25519:P-384:P-521" , RECU_RESTART_TS, RR_NULL, RECC_NULL, nullptr, RECA_NULL}

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

[trafficserver] 02/04: Remove version args from create_initial_packet

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 b778c96d2cf62791168f9ac885761e79e380c090
Author: Masaori Koshiba <ma...@apache.org>
AuthorDate: Wed Apr 4 14:58:13 2018 +0900

    Remove version args from create_initial_packet
    
    To use PacketFactory's _version like other create_X_packet functions
---
 iocore/net/QUICNetVConnection.cc                   |  2 +-
 iocore/net/quic/QUICPacket.cc                      |  4 ++--
 iocore/net/quic/QUICPacket.h                       |  4 ++--
 iocore/net/quic/test/test_QUICVersionNegotiator.cc | 11 ++++++-----
 4 files changed, 11 insertions(+), 10 deletions(-)

diff --git a/iocore/net/QUICNetVConnection.cc b/iocore/net/QUICNetVConnection.cc
index cac33f3..a22a7b1 100644
--- a/iocore/net/QUICNetVConnection.cc
+++ b/iocore/net/QUICNetVConnection.cc
@@ -1228,7 +1228,7 @@ QUICNetVConnection::_build_packet(ats_unique_buf buf, size_t len, bool retransmi
   case QUICPacketType::INITIAL:
     ink_assert(this->get_context() == NET_VCONNECTION_OUT);
     packet = this->_packet_factory.create_initial_packet(this->_original_quic_connection_id, this->largest_acked_packet_number(),
-                                                         QUIC_SUPPORTED_VERSIONS[0], std::move(buf), len);
+                                                         std::move(buf), len);
     this->_handshake_handler->handleEvent(QUIC_EVENT_HANDSHAKE_PACKET_WRITE_COMPLETE, nullptr);
 
     break;
diff --git a/iocore/net/quic/QUICPacket.cc b/iocore/net/quic/QUICPacket.cc
index a195f20..61e19ba 100644
--- a/iocore/net/quic/QUICPacket.cc
+++ b/iocore/net/quic/QUICPacket.cc
@@ -800,12 +800,12 @@ QUICPacketFactory::create_version_negotiation_packet(const QUICPacket *packet_se
 }
 
 QUICPacketUPtr
-QUICPacketFactory::create_initial_packet(QUICConnectionId connection_id, QUICPacketNumber base_packet_number, QUICVersion version,
+QUICPacketFactory::create_initial_packet(QUICConnectionId connection_id, QUICPacketNumber base_packet_number,
                                          ats_unique_buf payload, size_t len)
 {
   QUICPacketHeaderUPtr header =
     QUICPacketHeader::build(QUICPacketType::INITIAL, connection_id, this->_packet_number_generator.next(), base_packet_number,
-                            version, std::move(payload), len);
+                            this->_version, std::move(payload), len);
   return this->_create_encrypted_packet(std::move(header), true);
 }
 
diff --git a/iocore/net/quic/QUICPacket.h b/iocore/net/quic/QUICPacket.h
index 1bc3229..6a87d1e 100644
--- a/iocore/net/quic/QUICPacket.h
+++ b/iocore/net/quic/QUICPacket.h
@@ -340,8 +340,8 @@ public:
 
   QUICPacketUPtr create(ats_unique_buf buf, size_t len, QUICPacketNumber base_packet_number, QUICPacketCreationResult &result);
   QUICPacketUPtr create_version_negotiation_packet(const QUICPacket *packet_sent_by_client);
-  QUICPacketUPtr create_initial_packet(QUICConnectionId connection_id, QUICPacketNumber base_packet_number, QUICVersion version,
-                                       ats_unique_buf payload, size_t len);
+  QUICPacketUPtr create_initial_packet(QUICConnectionId connection_id, QUICPacketNumber base_packet_number, ats_unique_buf payload,
+                                       size_t len);
   QUICPacketUPtr create_retry_packet(QUICConnectionId connection_id, QUICPacketNumber packet_number, ats_unique_buf payload,
                                      size_t len, bool retransmittable);
   QUICPacketUPtr create_handshake_packet(QUICConnectionId connection_id, QUICPacketNumber base_packet_number,
diff --git a/iocore/net/quic/test/test_QUICVersionNegotiator.cc b/iocore/net/quic/test/test_QUICVersionNegotiator.cc
index ea6d87b..be32502 100644
--- a/iocore/net/quic/test/test_QUICVersionNegotiator.cc
+++ b/iocore/net/quic/test/test_QUICVersionNegotiator.cc
@@ -39,8 +39,8 @@ TEST_CASE("QUICVersionNegotiator", "[quic]")
     CHECK(vn.status() == QUICVersionNegotiationStatus::NOT_NEGOTIATED);
 
     // Negotiate version
-    QUICPacketUPtr initial_packet =
-      packet_factory.create_initial_packet({}, 0, QUIC_SUPPORTED_VERSIONS[0], ats_unique_malloc(0), 0);
+    packet_factory.set_version(QUIC_SUPPORTED_VERSIONS[0]);
+    QUICPacketUPtr initial_packet = packet_factory.create_initial_packet({}, 0, ats_unique_malloc(0), 0);
     vn.negotiate(initial_packet.get());
     CHECK(vn.status() == QUICVersionNegotiationStatus::NEGOTIATED);
 
@@ -57,8 +57,8 @@ TEST_CASE("QUICVersionNegotiator", "[quic]")
     CHECK(vn.status() == QUICVersionNegotiationStatus::NOT_NEGOTIATED);
 
     // Negotiate version
-    QUICPacketUPtr initial_packet =
-      packet_factory.create_initial_packet({}, 0, QUIC_SUPPORTED_VERSIONS[0], ats_unique_malloc(0), 0);
+    packet_factory.set_version(QUIC_SUPPORTED_VERSIONS[0]);
+    QUICPacketUPtr initial_packet = packet_factory.create_initial_packet({}, 0, ats_unique_malloc(0), 0);
     vn.negotiate(initial_packet.get());
     CHECK(vn.status() == QUICVersionNegotiationStatus::NEGOTIATED);
 
@@ -75,7 +75,8 @@ TEST_CASE("QUICVersionNegotiator", "[quic]")
     CHECK(vn.status() == QUICVersionNegotiationStatus::NOT_NEGOTIATED);
 
     // Negotiate version
-    QUICPacketUPtr initial_packet = packet_factory.create_initial_packet({}, 0, 0xbabababa, ats_unique_malloc(0), 0);
+    packet_factory.set_version(0xbabababa);
+    QUICPacketUPtr initial_packet = packet_factory.create_initial_packet({}, 0, ats_unique_malloc(0), 0);
     vn.negotiate(initial_packet.get());
     CHECK(vn.status() == QUICVersionNegotiationStatus::NOT_NEGOTIATED);
 

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

[trafficserver] 01/04: Fix QUICPacketFactory for Version Negotiation Packet

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 fc1bc3600758cd0c712788896271db946dec669a
Author: Masaori Koshiba <ma...@apache.org>
AuthorDate: Wed Apr 4 14:28:43 2018 +0900

    Fix QUICPacketFactory for Version Negotiation Packet
---
 iocore/net/quic/QUICHandshake.cc               |  3 +-
 iocore/net/quic/QUICPacket.cc                  | 38 +++++++++++++++++---------
 iocore/net/quic/QUICPacket.h                   |  2 +-
 iocore/net/quic/test/test_QUICPacketFactory.cc |  6 ++--
 4 files changed, 31 insertions(+), 18 deletions(-)

diff --git a/iocore/net/quic/QUICHandshake.cc b/iocore/net/quic/QUICHandshake.cc
index aec0907..f6a6cec 100644
--- a/iocore/net/quic/QUICHandshake.cc
+++ b/iocore/net/quic/QUICHandshake.cc
@@ -137,8 +137,7 @@ QUICHandshake::start(const QUICPacket *initial_packet, QUICPacketFactory *packet
         this->_load_local_server_transport_parameters(initial_packet->version());
         packet_factory->set_version(this->_version_negotiator->negotiated_version());
       } else {
-        this->_client_qc->transmit_packet(
-          packet_factory->create_version_negotiation_packet(initial_packet, _client_qc->largest_acked_packet_number()));
+        this->_client_qc->transmit_packet(packet_factory->create_version_negotiation_packet(initial_packet));
         QUICHSDebug("Version negotiation failed: %x", initial_packet->version());
       }
     } else {
diff --git a/iocore/net/quic/QUICPacket.cc b/iocore/net/quic/QUICPacket.cc
index 66ed12c..a195f20 100644
--- a/iocore/net/quic/QUICPacket.cc
+++ b/iocore/net/quic/QUICPacket.cc
@@ -196,7 +196,11 @@ const uint8_t *
 QUICPacketLongHeader::payload() const
 {
   if (this->_buf) {
-    return this->_buf.get() + LONG_HDR_OFFSET_PAYLOAD;
+    if (this->type() == QUICPacketType::VERSION_NEGOTIATION) {
+      return this->_buf.get() + VERSION_NEGOTIATION_PKT_HEADER_LENGTH;
+    } else {
+      return this->_buf.get() + LONG_HDR_OFFSET_PAYLOAD;
+    }
   } else {
     return this->_payload.get();
   }
@@ -253,11 +257,13 @@ QUICPacketLongHeader::store(uint8_t *buf, size_t *len) const
   QUICTypeUtil::write_QUICVersion(this->_version, buf + *len, &n);
   *len += n;
 
-  QUICPacketNumber dst = 0;
-  size_t dst_len       = 4;
-  QUICPacket::encode_packet_number(dst, this->_packet_number, dst_len);
-  QUICTypeUtil::write_QUICPacketNumber(dst, dst_len, buf + *len, &n);
-  *len += n;
+  if (this->_type != QUICPacketType::VERSION_NEGOTIATION) {
+    QUICPacketNumber dst = 0;
+    size_t dst_len       = 4;
+    QUICPacket::encode_packet_number(dst, this->_packet_number, dst_len);
+    QUICTypeUtil::write_QUICPacketNumber(dst, dst_len, buf + *len, &n);
+    *len += n;
+  }
 }
 
 //
@@ -667,14 +673,20 @@ QUICPacketFactory::create(ats_unique_buf buf, size_t len, QUICPacketNumber base_
   QUICPacketHeaderUPtr header = QUICPacketHeader::load(std::move(buf), len, base_packet_number);
 
   if (header->has_version() && !QUICTypeUtil::is_supported_version(header->version())) {
-    // We can't decrypt packets that have unknown versions
-    result = QUICPacketCreationResult::UNSUPPORTED;
+    if (header->type() == QUICPacketType::VERSION_NEGOTIATION) {
+      // version of VN packet is 0x00000000
+      // This packet is unprotected. Just copy the payload
+      result = QUICPacketCreationResult::SUCCESS;
+    } else {
+      // We can't decrypt packets that have unknown versions
+      result = QUICPacketCreationResult::UNSUPPORTED;
+    }
+
     memcpy(plain_txt.get(), header->payload(), header->payload_size());
     plain_txt_len = header->payload_size();
   } else {
     Debug("quic_packet", "Decrypting %s packet #%" PRIu64, QUICDebugNames::packet_type(header->type()), header->packet_number());
     switch (header->type()) {
-    case QUICPacketType::VERSION_NEGOTIATION:
     case QUICPacketType::STATELESS_RESET:
       // These packets are unprotected. Just copy the payload
       memcpy(plain_txt.get(), header->payload(), header->payload_size());
@@ -768,7 +780,7 @@ QUICPacketFactory::create(ats_unique_buf buf, size_t len, QUICPacketNumber base_
 }
 
 QUICPacketUPtr
-QUICPacketFactory::create_version_negotiation_packet(const QUICPacket *packet_sent_by_client, QUICPacketNumber base_packet_number)
+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); });
@@ -780,9 +792,9 @@ QUICPacketFactory::create_version_negotiation_packet(const QUICPacket *packet_se
     p += n;
   }
 
-  QUICPacketHeaderUPtr header =
-    QUICPacketHeader::build(QUICPacketType::VERSION_NEGOTIATION, packet_sent_by_client->connection_id(),
-                            packet_sent_by_client->packet_number(), base_packet_number, 0x00, std::move(versions), len);
+  // VN packet dosen't have packet number field and version field is always 0x00000000
+  QUICPacketHeaderUPtr header = QUICPacketHeader::build(QUICPacketType::VERSION_NEGOTIATION, packet_sent_by_client->connection_id(),
+                                                        0x00, 0x00, 0x00, std::move(versions), len);
 
   return QUICPacketFactory::_create_unprotected_packet(std::move(header));
 }
diff --git a/iocore/net/quic/QUICPacket.h b/iocore/net/quic/QUICPacket.h
index cca3571..1bc3229 100644
--- a/iocore/net/quic/QUICPacket.h
+++ b/iocore/net/quic/QUICPacket.h
@@ -339,7 +339,7 @@ public:
   static QUICPacketUPtr create_null_packet();
 
   QUICPacketUPtr create(ats_unique_buf buf, size_t len, QUICPacketNumber base_packet_number, QUICPacketCreationResult &result);
-  QUICPacketUPtr create_version_negotiation_packet(const QUICPacket *packet_sent_by_client, QUICPacketNumber base_packet_number);
+  QUICPacketUPtr create_version_negotiation_packet(const QUICPacket *packet_sent_by_client);
   QUICPacketUPtr create_initial_packet(QUICConnectionId connection_id, QUICPacketNumber base_packet_number, QUICVersion version,
                                        ats_unique_buf payload, size_t len);
   QUICPacketUPtr create_retry_packet(QUICConnectionId connection_id, QUICPacketNumber packet_number, ats_unique_buf payload,
diff --git a/iocore/net/quic/test/test_QUICPacketFactory.cc b/iocore/net/quic/test/test_QUICPacketFactory.cc
index 78e3820..e0be9f1 100644
--- a/iocore/net/quic/test/test_QUICPacketFactory.cc
+++ b/iocore/net/quic/test/test_QUICPacketFactory.cc
@@ -46,12 +46,14 @@ TEST_CASE("QUICPacketFactory_Create_VersionNegotiationPacket", "[quic]")
   QUICPacket initial_packet(std::move(header), ats_unique_buf(initial_packet_payload, [](void *) {}),
                             sizeof(initial_packet_payload), 0);
 
-  QUICPacketUPtr packet = factory.create_version_negotiation_packet(&initial_packet, 0);
+  QUICPacketUPtr packet = factory.create_version_negotiation_packet(&initial_packet);
   CHECK(packet->type() == QUICPacketType::VERSION_NEGOTIATION);
   CHECK(packet->connection_id() == initial_packet.connection_id());
   CHECK(packet->packet_number() == initial_packet.packet_number());
   CHECK(packet->version() == 0x00);
-  CHECK(memcmp(packet->payload(), "\xff\x00\x00\x09", 4) == 0);
+
+  QUICVersion supported_version = QUICTypeUtil::read_QUICVersion(packet->payload());
+  CHECK(supported_version == QUIC_SUPPORTED_VERSIONS[0]);
 }
 
 TEST_CASE("QUICPacketFactory_Create_Retry", "[quic]")

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

[trafficserver] 03/04: Add VERSION_NEGOTIATION packet support to QUICVersionNegotiator

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 528f1ee56b7857c716fd74352b4ba04f19772027
Author: Masaori Koshiba <ma...@apache.org>
AuthorDate: Wed Apr 4 15:06:13 2018 +0900

    Add VERSION_NEGOTIATION packet support to QUICVersionNegotiator
---
 iocore/net/quic/QUICTransportParameters.cc         | 12 +++++
 iocore/net/quic/QUICTransportParameters.h          |  1 +
 iocore/net/quic/QUICTypes.h                        |  2 +
 iocore/net/quic/QUICVersionNegotiator.cc           | 63 ++++++++++++++++++++--
 iocore/net/quic/QUICVersionNegotiator.h            |  1 +
 .../net/quic/test/test_QUICTransportParameters.cc  |  1 +
 iocore/net/quic/test/test_QUICVersionNegotiator.cc | 55 +++++++++++++++++--
 7 files changed, 128 insertions(+), 7 deletions(-)

diff --git a/iocore/net/quic/QUICTransportParameters.cc b/iocore/net/quic/QUICTransportParameters.cc
index 7b74a6f..9a8182f 100644
--- a/iocore/net/quic/QUICTransportParameters.cc
+++ b/iocore/net/quic/QUICTransportParameters.cc
@@ -439,6 +439,18 @@ QUICTransportParametersInEncryptedExtensions::add_version(QUICVersion version)
   this->_versions[this->_n_versions++] = version;
 }
 
+bool
+QUICTransportParametersInEncryptedExtensions::is_valid_negotiated_version() const
+{
+  for (int i = 0; QUICVersion v = this->_versions[i]; i++) {
+    if (this->_negotiated_version == v) {
+      return true;
+    }
+  }
+
+  return false;
+}
+
 std::ptrdiff_t
 QUICTransportParametersInEncryptedExtensions::_parameters_offset(const uint8_t *buf) const
 {
diff --git a/iocore/net/quic/QUICTransportParameters.h b/iocore/net/quic/QUICTransportParameters.h
index 11bc8b0..96ff64b 100644
--- a/iocore/net/quic/QUICTransportParameters.h
+++ b/iocore/net/quic/QUICTransportParameters.h
@@ -136,6 +136,7 @@ public:
   QUICTransportParametersInEncryptedExtensions(const uint8_t *buf, size_t len);
   QUICVersion negotiated_version() const;
   void add_version(QUICVersion version);
+  bool is_valid_negotiated_version() const;
 
 protected:
   std::ptrdiff_t _parameters_offset(const uint8_t *buf) const override;
diff --git a/iocore/net/quic/QUICTypes.h b/iocore/net/quic/QUICTypes.h
index 78b9a01..57de391 100644
--- a/iocore/net/quic/QUICTypes.h
+++ b/iocore/net/quic/QUICTypes.h
@@ -49,6 +49,8 @@ using QUICOffset       = uint64_t;
 constexpr QUICVersion QUIC_SUPPORTED_VERSIONS[] = {
   0xff000009,
 };
+constexpr QUICVersion QUIC_EXERCISE_VERSIONS = 0x1a2a3a4a;
+
 constexpr QUICStreamId STREAM_ID_FOR_HANDSHAKE = 0;
 
 enum class QUICHandshakeMsgType {
diff --git a/iocore/net/quic/QUICVersionNegotiator.cc b/iocore/net/quic/QUICVersionNegotiator.cc
index 2329c31..8aff8a9 100644
--- a/iocore/net/quic/QUICVersionNegotiator.cc
+++ b/iocore/net/quic/QUICVersionNegotiator.cc
@@ -31,12 +31,40 @@ QUICVersionNegotiator::status()
 }
 
 QUICVersionNegotiationStatus
-QUICVersionNegotiator::negotiate(const QUICPacket *initial_packet)
+QUICVersionNegotiator::negotiate(const QUICPacket *packet)
 {
-  if (QUICTypeUtil::is_supported_version(initial_packet->version())) {
-    this->_status             = QUICVersionNegotiationStatus::NEGOTIATED;
-    this->_negotiated_version = initial_packet->version();
+  switch (packet->type()) {
+  case QUICPacketType::INITIAL: {
+    if (QUICTypeUtil::is_supported_version(packet->version())) {
+      this->_status             = QUICVersionNegotiationStatus::NEGOTIATED;
+      this->_negotiated_version = packet->version();
+    }
+
+    break;
+  }
+  case QUICPacketType::VERSION_NEGOTIATION: {
+    const uint8_t *supported_versions = packet->payload();
+    uint16_t supported_versions_len   = packet->payload_size();
+    uint16_t len                      = 0;
+
+    while (len < supported_versions_len) {
+      QUICVersion version = QUICTypeUtil::read_QUICVersion(supported_versions + len);
+      len += sizeof(QUICVersion);
+
+      if (QUICTypeUtil::is_supported_version(version)) {
+        this->_status             = QUICVersionNegotiationStatus::NEGOTIATED;
+        this->_negotiated_version = version;
+        break;
+      }
+    }
+
+    break;
   }
+  default:
+    ink_assert(false);
+    break;
+  }
+
   return this->_status;
 }
 
@@ -46,6 +74,7 @@ QUICVersionNegotiator::validate(const QUICTransportParametersInClientHello *tp)
   if (this->_negotiated_version == tp->initial_version()) {
     this->_status = QUICVersionNegotiationStatus::VALIDATED;
   } else {
+    // Version negotiation was performed
     if (QUICTypeUtil::is_supported_version(tp->initial_version())) {
       this->_status             = QUICVersionNegotiationStatus::FAILED;
       this->_negotiated_version = 0;
@@ -56,6 +85,32 @@ QUICVersionNegotiator::validate(const QUICTransportParametersInClientHello *tp)
   return this->_status;
 }
 
+QUICVersionNegotiationStatus
+QUICVersionNegotiator::validate(const QUICTransportParametersInEncryptedExtensions *tp)
+{
+  if (!tp->is_valid_negotiated_version()) {
+    this->_status             = QUICVersionNegotiationStatus::FAILED;
+    this->_negotiated_version = 0;
+
+    return this->_status;
+  }
+
+  if (this->_status == QUICVersionNegotiationStatus::NEGOTIATED) {
+    // Version negotiation was performed
+    if (this->_negotiated_version == tp->negotiated_version()) {
+      this->_status = QUICVersionNegotiationStatus::VALIDATED;
+    } else {
+      this->_status             = QUICVersionNegotiationStatus::FAILED;
+      this->_negotiated_version = 0;
+    }
+  } else {
+    this->_status             = QUICVersionNegotiationStatus::VALIDATED;
+    this->_negotiated_version = tp->negotiated_version();
+  }
+
+  return this->_status;
+}
+
 QUICVersion
 QUICVersionNegotiator::negotiated_version()
 {
diff --git a/iocore/net/quic/QUICVersionNegotiator.h b/iocore/net/quic/QUICVersionNegotiator.h
index 651a7db..c59d59e 100644
--- a/iocore/net/quic/QUICVersionNegotiator.h
+++ b/iocore/net/quic/QUICVersionNegotiator.h
@@ -37,6 +37,7 @@ public:
   QUICVersionNegotiationStatus status();
   QUICVersionNegotiationStatus negotiate(const QUICPacket *initial_packet);
   QUICVersionNegotiationStatus validate(const QUICTransportParametersInClientHello *tp);
+  QUICVersionNegotiationStatus validate(const QUICTransportParametersInEncryptedExtensions *tp);
   QUICVersion negotiated_version();
 
 private:
diff --git a/iocore/net/quic/test/test_QUICTransportParameters.cc b/iocore/net/quic/test/test_QUICTransportParameters.cc
index cc73714..3e26f93 100644
--- a/iocore/net/quic/test/test_QUICTransportParameters.cc
+++ b/iocore/net/quic/test/test_QUICTransportParameters.cc
@@ -153,6 +153,7 @@ TEST_CASE("QUICTransportParametersInEncryptedExtensions_read", "[quic]")
 
     QUICTransportParametersInEncryptedExtensions params_in_ee(buf, sizeof(buf));
     CHECK(params_in_ee.is_valid());
+    CHECK(params_in_ee.is_valid_negotiated_version());
     CHECK(params_in_ee.negotiated_version() == 0x01020304);
 
     uint16_t len        = 0;
diff --git a/iocore/net/quic/test/test_QUICVersionNegotiator.cc b/iocore/net/quic/test/test_QUICVersionNegotiator.cc
index be32502..a318c98 100644
--- a/iocore/net/quic/test/test_QUICVersionNegotiator.cc
+++ b/iocore/net/quic/test/test_QUICVersionNegotiator.cc
@@ -26,7 +26,7 @@
 #include "quic/QUICVersionNegotiator.h"
 #include "quic/Mock.h"
 
-TEST_CASE("QUICVersionNegotiator", "[quic]")
+TEST_CASE("QUICVersionNegotiator - Server Side", "[quic]")
 {
   QUICPacketFactory packet_factory;
   MockQUICHandshakeProtocol hs_protocol;
@@ -63,7 +63,7 @@ TEST_CASE("QUICVersionNegotiator", "[quic]")
     CHECK(vn.status() == QUICVersionNegotiationStatus::NEGOTIATED);
 
     // Validate version
-    QUICTransportParametersInClientHello tp(0xbabababa);
+    QUICTransportParametersInClientHello tp(QUIC_EXERCISE_VERSIONS);
     vn.validate(&tp);
     CHECK(vn.status() == QUICVersionNegotiationStatus::VALIDATED);
     CHECK(vn.negotiated_version() == QUIC_SUPPORTED_VERSIONS[0]);
@@ -75,7 +75,7 @@ TEST_CASE("QUICVersionNegotiator", "[quic]")
     CHECK(vn.status() == QUICVersionNegotiationStatus::NOT_NEGOTIATED);
 
     // Negotiate version
-    packet_factory.set_version(0xbabababa);
+    packet_factory.set_version(QUIC_EXERCISE_VERSIONS);
     QUICPacketUPtr initial_packet = packet_factory.create_initial_packet({}, 0, ats_unique_malloc(0), 0);
     vn.negotiate(initial_packet.get());
     CHECK(vn.status() == QUICVersionNegotiationStatus::NOT_NEGOTIATED);
@@ -87,3 +87,52 @@ TEST_CASE("QUICVersionNegotiator", "[quic]")
     CHECK(vn.negotiated_version() != QUIC_SUPPORTED_VERSIONS[0]);
   }
 }
+
+TEST_CASE("QUICVersionNegotiator - Client Side", "[quic]")
+{
+  QUICPacketFactory packet_factory;
+  MockQUICHandshakeProtocol hs_protocol;
+  packet_factory.set_hs_protocol(&hs_protocol);
+  QUICVersionNegotiator vn;
+
+  SECTION("Normal case")
+  {
+    // Check initial state
+    CHECK(vn.status() == QUICVersionNegotiationStatus::NOT_NEGOTIATED);
+
+    // No Version Negotiation packet from server
+
+    // Validate version
+    QUICTransportParametersInEncryptedExtensions tp(QUIC_SUPPORTED_VERSIONS[0]);
+    tp.add_version(QUIC_SUPPORTED_VERSIONS[0]);
+
+    vn.validate(&tp);
+    CHECK(vn.status() == QUICVersionNegotiationStatus::VALIDATED);
+    CHECK(vn.negotiated_version() == QUIC_SUPPORTED_VERSIONS[0]);
+  }
+
+  SECTION("Negotiation case")
+  {
+    // Check initial state
+    CHECK(vn.status() == QUICVersionNegotiationStatus::NOT_NEGOTIATED);
+
+    // Negotiate version
+    packet_factory.set_version(QUIC_EXERCISE_VERSIONS);
+    QUICPacketUPtr initial_packet = packet_factory.create_initial_packet({}, 0, ats_unique_malloc(0), 0);
+
+    // Server send VN packet based on Initial packet
+    QUICPacketUPtr vn_packet = packet_factory.create_version_negotiation_packet(initial_packet.get());
+
+    // Negotiate version
+    vn.negotiate(vn_packet.get());
+    CHECK(vn.status() == QUICVersionNegotiationStatus::NEGOTIATED);
+    CHECK(vn.negotiated_version() == QUIC_SUPPORTED_VERSIONS[0]);
+
+    // Validate version
+    QUICTransportParametersInEncryptedExtensions tp(QUIC_SUPPORTED_VERSIONS[0]);
+    tp.add_version(QUIC_SUPPORTED_VERSIONS[0]);
+
+    vn.validate(&tp);
+    CHECK(vn.status() == QUICVersionNegotiationStatus::VALIDATED);
+  }
+}

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