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 2019/02/05 03:16:42 UTC

[trafficserver] branch quic-latest updated: QUIC: Limit the sending after 3 packets with unavlidated path

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

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


The following commit(s) were added to refs/heads/quic-latest by this push:
     new 2bebf6c  QUIC: Limit the sending after 3 packets with unavlidated path
2bebf6c is described below

commit 2bebf6c7338e28c6ef0f8c1cdb3fb6c4e57fcc26
Author: scw00 <sc...@apache.org>
AuthorDate: Sat Feb 2 09:59:30 2019 +0800

    QUIC: Limit the sending after 3 packets with unavlidated path
---
 iocore/net/P_QUICNetVConnection.h                |  5 +-
 iocore/net/QUICNetVConnection.cc                 | 31 ++++++-----
 iocore/net/quic/Makefile.am                      | 13 ++++-
 iocore/net/quic/QUICAddrVerifyState.cc           | 66 ++++++++++++++++++++++++
 iocore/net/quic/QUICAddrVerifyState.h            | 43 +++++++++++++++
 iocore/net/quic/test/test_QUICAddrVerifyState.cc | 61 ++++++++++++++++++++++
 6 files changed, 199 insertions(+), 20 deletions(-)

diff --git a/iocore/net/P_QUICNetVConnection.h b/iocore/net/P_QUICNetVConnection.h
index 38b9284..8890ca9 100644
--- a/iocore/net/P_QUICNetVConnection.h
+++ b/iocore/net/P_QUICNetVConnection.h
@@ -59,6 +59,7 @@
 #include "quic/QUICPathValidator.h"
 #include "quic/QUICApplicationMap.h"
 #include "quic/QUICPacketReceiveQueue.h"
+#include "quic/QUICAddrVerifyState.h"
 
 // These are included here because older OpenQUIC libraries don't have them.
 // Don't copy these defines, or use their values directly, they are merely
@@ -363,7 +364,6 @@ private:
   void _update_peer_cid(const QUICConnectionId &new_cid);
   void _update_local_cid(const QUICConnectionId &new_cid);
   void _rerandomize_original_cid();
-  bool _is_src_addr_verified();
 
   QUICHandshakeProtocol *_setup_handshake_protocol(SSL_CTX *ctx);
 
@@ -379,9 +379,10 @@ private:
   uint64_t _stream_frames_sent     = 0;
 
   // TODO: Source addresses verification through an address validation token
-  bool _src_addr_verified       = false;
   bool _has_ack_only_packet_out = false;
 
+  QUICAddrVerifyState _verfied_state;
+
   // QUICFrameGenerator
   void _on_frame_lost(QUICFrameInformationUPtr &info) override;
   std::vector<QUICEncryptionLevel>
diff --git a/iocore/net/QUICNetVConnection.cc b/iocore/net/QUICNetVConnection.cc
index 9872d5c..453002f 100644
--- a/iocore/net/QUICNetVConnection.cc
+++ b/iocore/net/QUICNetVConnection.cc
@@ -69,8 +69,6 @@ static constexpr ink_hrtime WRITE_READY_INTERVAL      = HRTIME_MSECONDS(2);
 static constexpr uint32_t PACKET_PER_EVENT            = 256;
 static constexpr uint32_t MAX_CONSECUTIVE_STREAMS     = 8; //< Interrupt sending STREAM frames to send ACK frame
 
-static constexpr uint32_t MAX_PACKETS_WITHOUT_SRC_ADDR_VARIDATION = 3;
-
 static constexpr uint32_t STATE_CLOSING_MAX_SEND_PKT_NUM  = 8; // Max number of sending packets which contain a closing frame.
 static constexpr uint32_t STATE_CLOSING_MAX_RECV_PKT_WIND = 1 << STATE_CLOSING_MAX_SEND_PKT_NUM;
 
@@ -1052,8 +1050,8 @@ QUICNetVConnection::_state_handshake_process_handshake_packet(QUICPacketUPtr pac
 {
   // Source address is verified by receiving any message from the client encrypted using the
   // Handshake keys.
-  if (this->netvc_context == NET_VCONNECTION_IN && !this->_src_addr_verified) {
-    this->_src_addr_verified = true;
+  if (this->netvc_context == NET_VCONNECTION_IN && !this->_verfied_state.is_verified()) {
+    this->_verfied_state.set_addr_verifed();
   }
   return this->_recv_and_ack(*packet);
 }
@@ -1226,14 +1224,14 @@ QUICNetVConnection::_state_common_send_packet()
         continue;
       }
 
-      if (this->netvc_context == NET_VCONNECTION_IN && !this->_is_src_addr_verified() &&
-          this->_handshake_packets_sent >= MAX_PACKETS_WITHOUT_SRC_ADDR_VARIDATION) {
-        error = 1;
-        break;
+      uint32_t max_packet_size = udp_payload_len - written;
+      if (this->netvc_context == NET_VCONNECTION_IN && !this->_verfied_state.is_verified()) {
+        uint32_t windows = this->_verfied_state.windows();
+        QUICConDebug("send to unverified window: %u", windows);
+        max_packet_size = std::min(max_packet_size, windows);
       }
 
-      uint32_t max_packet_size = udp_payload_len - written;
-      QUICPacketUPtr packet    = this->_packetize_frames(level, max_packet_size);
+      QUICPacketUPtr packet = this->_packetize_frames(level, max_packet_size);
 
       if (packet) {
         if (this->netvc_context == NET_VCONNECTION_IN &&
@@ -1241,6 +1239,10 @@ QUICNetVConnection::_state_common_send_packet()
           ++this->_handshake_packets_sent;
         }
 
+        if (this->netvc_context == NET_VCONNECTION_IN && !this->_verfied_state.is_verified()) {
+          this->_verfied_state.consume(packet->size());
+        }
+
         // TODO: do not write two QUIC Short Header Packets
         uint8_t *buf = reinterpret_cast<uint8_t *>(udp_payload->end());
         size_t len   = 0;
@@ -1688,6 +1690,9 @@ QUICNetVConnection::_dequeue_recv_packet(QUICPacketCreationResult &result)
     }
 
     this->_last_received_packet_type = packet->type();
+    if (!this->_verfied_state.is_verified()) {
+      this->_verfied_state.fill(packet->size());
+    }
   }
 
   // Debug prints
@@ -2083,12 +2088,6 @@ QUICNetVConnection::_rerandomize_original_cid()
   }
 }
 
-bool
-QUICNetVConnection::_is_src_addr_verified()
-{
-  return this->_src_addr_verified;
-}
-
 QUICHandshakeProtocol *
 QUICNetVConnection::_setup_handshake_protocol(SSL_CTX *ctx)
 {
diff --git a/iocore/net/quic/Makefile.am b/iocore/net/quic/Makefile.am
index ba75683..2b12239 100644
--- a/iocore/net/quic/Makefile.am
+++ b/iocore/net/quic/Makefile.am
@@ -81,7 +81,8 @@ libquic_a_SOURCES = \
   QUICPathValidator.cc \
   QUICPinger.cc \
   QUICFrameGenerator.cc \
-  QUICFrameRetransmitter.cc
+  QUICFrameRetransmitter.cc \
+  QUICAddrVerifyState.cc
 
 #
 # Check Programs
@@ -107,7 +108,8 @@ check_PROGRAMS = \
   test_QUICType \
   test_QUICTypeUtil \
   test_QUICVersionNegotiator \
-  test_QUICFrameRetransmitter
+  test_QUICFrameRetransmitter \
+  test_QUICAddrVerifyState
 
 TESTS = $(check_PROGRAMS)
 
@@ -288,6 +290,13 @@ test_QUICFrameRetransmitter_SOURCES = \
   $(test_main_SOURCES) \
   ./test/test_QUICFrameRetransmitter.cc
 
+test_QUICAddrVerifyState_CPPFLAGS = $(test_CPPFLAGS)
+test_QUICAddrVerifyState_LDFLAGS = @AM_LDFLAGS@
+test_QUICAddrVerifyState_LDADD = $(test_LDADD)
+test_QUICAddrVerifyState_SOURCES = \
+  $(test_main_SOURCES) \
+  ./test/test_QUICAddrVerifyState.cc
+
 #
 # clang-tidy
 #
diff --git a/iocore/net/quic/QUICAddrVerifyState.cc b/iocore/net/quic/QUICAddrVerifyState.cc
new file mode 100644
index 0000000..67281b5
--- /dev/null
+++ b/iocore/net/quic/QUICAddrVerifyState.cc
@@ -0,0 +1,66 @@
+/** @file
+
+  A brief file description
+
+  @section license License
+
+  Licensed to the Apache Software Foundation (ASF) under one
+  or more contributor license agreements.  See the NOTICE file
+  distributed with this work for additional information
+  regarding copyright ownership.  The ASF licenses this file
+  to you under the Apache License, Version 2.0 (the
+  "License"); you may not use this file except in compliance
+  with the License.  You may obtain a copy of the License at
+
+      http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+ */
+
+#include "QUICAddrVerifyState.h"
+
+void
+QUICAddrVerifyState::fill(uint32_t windows)
+{
+  uint64_t tmp = this->_windows;
+  if (tmp + windows * 3 > UINT32_MAX) {
+    // overflow
+    this->_windows = UINT32_MAX;
+    return;
+  }
+
+  this->_windows += windows * 3;
+}
+
+void
+QUICAddrVerifyState::consume(uint32_t windows)
+{
+  if (this->_windows <= windows) {
+    this->_windows = 0;
+    return;
+  }
+
+  this->_windows -= windows;
+}
+
+uint32_t
+QUICAddrVerifyState::windows()
+{
+  return this->_windows;
+}
+
+void
+QUICAddrVerifyState::set_addr_verifed()
+{
+  this->_src_addr_verified = true;
+}
+
+bool
+QUICAddrVerifyState::is_verified() const
+{
+  return this->_src_addr_verified;
+}
diff --git a/iocore/net/quic/QUICAddrVerifyState.h b/iocore/net/quic/QUICAddrVerifyState.h
new file mode 100644
index 0000000..d6d6c7c
--- /dev/null
+++ b/iocore/net/quic/QUICAddrVerifyState.h
@@ -0,0 +1,43 @@
+/** @file
+
+  A brief file description
+
+  @section license License
+
+  Licensed to the Apache Software Foundation (ASF) under one
+  or more contributor license agreements.  See the NOTICE file
+  distributed with this work for additional information
+  regarding copyright ownership.  The ASF licenses this file
+  to you under the Apache License, Version 2.0 (the
+  "License"); you may not use this file except in compliance
+  with the License.  You may obtain a copy of the License at
+
+      http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+ */
+
+#pragma once
+
+#include <cstdint>
+
+class QUICAddrVerifyState
+{
+public:
+  QUICAddrVerifyState(uint8_t packets, uint32_t windows) : _windows(windows) {}
+  QUICAddrVerifyState() = default;
+
+  void fill(uint32_t windows);
+  void consume(uint32_t windows);
+  void set_addr_verifed();
+  uint32_t windows();
+  bool is_verified() const;
+
+private:
+  bool _src_addr_verified = false;
+  uint32_t _windows       = 0;
+};
diff --git a/iocore/net/quic/test/test_QUICAddrVerifyState.cc b/iocore/net/quic/test/test_QUICAddrVerifyState.cc
new file mode 100644
index 0000000..a90135c
--- /dev/null
+++ b/iocore/net/quic/test/test_QUICAddrVerifyState.cc
@@ -0,0 +1,61 @@
+/** @file
+ *
+ *  A brief file description
+ *
+ *  @section license License
+ *
+ *  Licensed to the Apache Software Foundation (ASF) under one
+ *  or more contributor license agreements.  See the NOTICE file
+ *  distributed with this work for additional information
+ *  regarding copyright ownership.  The ASF licenses this file
+ *  to you under the Apache License, Version 2.0 (the
+ *  "License"); you may not use this file except in compliance
+ *  with the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ *  Unless required by applicable law or agreed to in writing, software
+ *  distributed under the License is distributed on an "AS IS" BASIS,
+ *  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ *  See the License for the specific language governing permissions and
+ *  limitations under the License.
+ */
+
+#include "catch.hpp"
+
+#include "P_QUICNetVConnection.h"
+#include <memory>
+
+TEST_CASE("QUICAddrVerifyState", "[quic]")
+{
+  QUICAddrVerifyState state;
+
+  // without consuming
+  CHECK(state.windows() == 0);
+  state.fill(10240);
+  CHECK(state.windows() == 10240 * 3);
+
+  // consume
+  CHECK(state.windows() == 10240 * 3);
+  state.consume(10240);
+  CHECK(state.windows() == 10240 * 2);
+  state.consume(10240);
+  CHECK(state.windows() == 10240);
+  state.consume(10240);
+  CHECK(state.windows() == 0);
+
+  // fill
+  state.fill(1);
+  CHECK(state.windows() == 3);
+  state.consume(1);
+  CHECK(state.windows() == 2);
+  state.consume(1);
+  CHECK(state.windows() == 1);
+  state.consume(1);
+  CHECK(state.windows() == 0);
+
+  // fill overflow
+  state.fill(UINT32_MAX);
+  state.fill(2);
+  CHECK(state.windows() == UINT32_MAX);
+}