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/03/23 03:01:20 UTC

[trafficserver] branch quic-latest updated: Don't ack protected packets with a handshake packet

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

maskit 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 bef6fba  Don't ack protected packets with a handshake packet
bef6fba is described below

commit bef6fbad1085423d4b1a1b4228fea4f2e62a1a2e
Author: Masakazu Kitajo <ma...@apache.org>
AuthorDate: Fri Mar 23 09:29:43 2018 +0900

    Don't ack protected packets with a handshake packet
---
 iocore/net/P_QUICNetVConnection.h          |  2 +-
 iocore/net/QUICNetVConnection.cc           | 22 ++++---
 iocore/net/quic/QUICAckFrameCreator.cc     | 21 +++++--
 iocore/net/quic/QUICAckFrameCreator.h      |  5 +-
 iocore/net/quic/QUICFrame.cc               | 65 +++++++++++++++------
 iocore/net/quic/QUICFrame.h                | 94 ++++++++++++++++++------------
 iocore/net/quic/QUICIncomingFrameBuffer.cc |  2 +-
 iocore/net/quic/QUICStream.cc              |  3 +-
 8 files changed, 140 insertions(+), 74 deletions(-)

diff --git a/iocore/net/P_QUICNetVConnection.h b/iocore/net/P_QUICNetVConnection.h
index 0cca49d..2709926 100644
--- a/iocore/net/P_QUICNetVConnection.h
+++ b/iocore/net/P_QUICNetVConnection.h
@@ -289,7 +289,7 @@ private:
   QUICPacketUPtr _build_packet(ats_unique_buf buf, size_t len, bool retransmittable,
                                QUICPacketType type = QUICPacketType::UNINITIALIZED);
 
-  QUICErrorUPtr _recv_and_ack(const uint8_t *payload, uint16_t size, QUICPacketNumber packet_numm);
+  QUICErrorUPtr _recv_and_ack(QUICPacketUPtr packet);
 
   QUICErrorUPtr _state_handshake_process_packet(QUICPacketUPtr packet);
   QUICErrorUPtr _state_handshake_process_initial_client_packet(QUICPacketUPtr packet);
diff --git a/iocore/net/QUICNetVConnection.cc b/iocore/net/QUICNetVConnection.cc
index 1c3af53..a361c3b 100644
--- a/iocore/net/QUICNetVConnection.cc
+++ b/iocore/net/QUICNetVConnection.cc
@@ -819,7 +819,7 @@ QUICNetVConnection::_state_handshake_process_initial_client_packet(QUICPacketUPt
   // Start handshake
   QUICErrorUPtr error = this->_handshake_handler->start(packet.get(), &this->_packet_factory);
   if (this->_handshake_handler->is_version_negotiated()) {
-    error = this->_recv_and_ack(packet->payload(), packet->payload_size(), packet->packet_number());
+    error = this->_recv_and_ack(std::move(packet));
   } else {
     // Perhaps response packets for initial client packet were lost, but no need to start handshake again because loss detector will
     // retransmit the packets.
@@ -830,20 +830,20 @@ QUICNetVConnection::_state_handshake_process_initial_client_packet(QUICPacketUPt
 QUICErrorUPtr
 QUICNetVConnection::_state_handshake_process_client_cleartext_packet(QUICPacketUPtr packet)
 {
-  return this->_recv_and_ack(packet->payload(), packet->payload_size(), packet->packet_number());
+  return this->_recv_and_ack(std::move(packet));
 }
 
 QUICErrorUPtr
 QUICNetVConnection::_state_handshake_process_zero_rtt_protected_packet(QUICPacketUPtr packet)
 {
   this->_start_application();
-  return this->_recv_and_ack(packet->payload(), packet->payload_size(), packet->packet_number());
+  return this->_recv_and_ack(std::move(packet));
 }
 
 QUICErrorUPtr
 QUICNetVConnection::_state_connection_established_process_packet(QUICPacketUPtr packet)
 {
-  return this->_recv_and_ack(packet->payload(), packet->payload_size(), packet->packet_number());
+  return this->_recv_and_ack(std::move(packet));
 }
 
 QUICErrorUPtr
@@ -906,7 +906,7 @@ QUICNetVConnection::_state_connection_closing_and_draining_receive_packet()
   QUICPacketCreationResult result;
   QUICPacketUPtr packet = this->_dequeue_recv_packet(result);
   if (result == QUICPacketCreationResult::SUCCESS) {
-    this->_recv_and_ack(packet->payload(), packet->payload_size(), packet->packet_number());
+    this->_recv_and_ack(std::move(packet));
     this->_schedule_packet_write_ready();
   }
 
@@ -995,8 +995,7 @@ QUICNetVConnection::_store_frame(ats_unique_buf &buf, size_t &len, bool &retrans
   QUICRetransmissionFrame *rf         = dynamic_cast<QUICRetransmissionFrame *>(frame.get());
   if (rf) {
     current_packet_type = rf->packet_type();
-  } else if (frame->type() == QUICFrameType::STREAM &&
-             static_cast<const QUICStreamFrame *>(frame.get())->stream_id() != STREAM_ID_FOR_HANDSHAKE) {
+  } else if (frame->is_protected()) {
     current_packet_type = QUICPacketType::PROTECTED;
   } else {
     current_packet_type = QUICPacketType::UNINITIALIZED;
@@ -1101,8 +1100,12 @@ QUICNetVConnection::_packetize_frames()
 }
 
 QUICErrorUPtr
-QUICNetVConnection::_recv_and_ack(const uint8_t *payload, uint16_t size, QUICPacketNumber packet_num)
+QUICNetVConnection::_recv_and_ack(QUICPacketUPtr packet)
 {
+  const uint8_t *payload = packet->payload();
+  uint16_t size = packet->payload_size();
+  QUICPacketNumber packet_num = packet->packet_number();
+
   if (packet_num > this->_largest_received_packet_number) {
     this->_largest_received_packet_number = packet_num;
   }
@@ -1123,7 +1126,8 @@ QUICNetVConnection::_recv_and_ack(const uint8_t *payload, uint16_t size, QUICPac
     return QUICErrorUPtr(new QUICConnectionError(QUICTransErrorCode::FLOW_CONTROL_ERROR));
   }
 
-  this->_ack_frame_creator.update(packet_num, should_send_ack);
+  bool protection = packet->type() == QUICPacketType::PROTECTED || packet->type() == QUICPacketType::ZERO_RTT_PROTECTED;
+  this->_ack_frame_creator.update(packet_num, protection, should_send_ack);
   static_cast<QUICConnection *>(this)->transmit_frame();
 
   return error;
diff --git a/iocore/net/quic/QUICAckFrameCreator.cc b/iocore/net/quic/QUICAckFrameCreator.cc
index 9c81166..ab9de24 100644
--- a/iocore/net/quic/QUICAckFrameCreator.cc
+++ b/iocore/net/quic/QUICAckFrameCreator.cc
@@ -26,7 +26,7 @@
 #include <algorithm>
 
 int
-QUICAckFrameCreator::update(QUICPacketNumber packet_number, bool should_send)
+QUICAckFrameCreator::update(QUICPacketNumber packet_number, bool protection, bool should_send)
 {
   if (this->_packet_numbers.size() == MAXIMUM_PACKET_COUNT) {
     return -1;
@@ -34,6 +34,10 @@ QUICAckFrameCreator::update(QUICPacketNumber packet_number, bool should_send)
 
   this->_packet_numbers.push_back(packet_number);
 
+  if (!protection) {
+    this->_unprotected_packets.insert(packet_number);
+  }
+
   if (!this->_can_send) {
     this->_can_send = true;
   }
@@ -54,6 +58,7 @@ QUICAckFrameCreator::create()
     this->_should_send  = false;
     this->_packet_count = 0;
     this->_packet_numbers.clear();
+    this->_unprotected_packets.clear();
   }
   return ack_frame;
 }
@@ -85,8 +90,14 @@ QUICAckFrameCreator::_create_ack_frame()
   size_t i        = 0;
   uint8_t gap     = 0;
   uint64_t length = 0;
+  bool protection = false;
 
   while (i < this->_packet_numbers.size()) {
+    if (!protection) {
+      if (this->_unprotected_packets.find(last_ack_number) == this->_unprotected_packets.end()) {
+        protection = true;
+      }
+    }
     if (this->_packet_numbers[i] == last_ack_number) {
       last_ack_number--;
       length++;
@@ -95,10 +106,10 @@ QUICAckFrameCreator::_create_ack_frame()
     }
 
     if (ack_frame) {
-      ack_frame->ack_block_section()->add_ack_block({static_cast<uint8_t>(gap - 1), length - 1});
+      ack_frame->ack_block_section()->add_ack_block({static_cast<uint8_t>(gap - 1), length - 1}, protection);
     } else {
       uint64_t delay = this->_calculate_delay();
-      ack_frame      = QUICFrameFactory::create_ack_frame(largest_ack_number, delay, length - 1);
+      ack_frame      = QUICFrameFactory::create_ack_frame(largest_ack_number, delay, length - 1, protection);
     }
 
     gap             = last_ack_number - this->_packet_numbers[i];
@@ -107,10 +118,10 @@ QUICAckFrameCreator::_create_ack_frame()
   }
 
   if (ack_frame) {
-    ack_frame->ack_block_section()->add_ack_block({static_cast<uint8_t>(gap - 1), length - 1});
+    ack_frame->ack_block_section()->add_ack_block({static_cast<uint8_t>(gap - 1), length - 1}, protection);
   } else {
     uint64_t delay = this->_calculate_delay();
-    ack_frame      = QUICFrameFactory::create_ack_frame(largest_ack_number, delay, length - 1);
+    ack_frame      = QUICFrameFactory::create_ack_frame(largest_ack_number, delay, length - 1, protection);
   }
   return ack_frame;
 }
diff --git a/iocore/net/quic/QUICAckFrameCreator.h b/iocore/net/quic/QUICAckFrameCreator.h
index 5a8e799..088d574 100644
--- a/iocore/net/quic/QUICAckFrameCreator.h
+++ b/iocore/net/quic/QUICAckFrameCreator.h
@@ -26,6 +26,8 @@
 #include "ts/ink_hrtime.h"
 #include "QUICTypes.h"
 #include "QUICFrame.h"
+#include <vector>
+#include <set>
 
 class QUICAckPacketNumbers
 {
@@ -59,7 +61,7 @@ public:
    * All packet numbers ATS received need to be passed to this method.
    * Returns 0 if updated successfully.
    */
-  int update(QUICPacketNumber packet_number, bool should_send);
+  int update(QUICPacketNumber packet_number, bool protection, bool should_send);
 
   /*
    * Returns QUICAckFrame only if ACK frame is able to be sent.
@@ -80,6 +82,7 @@ private:
 
   QUICAckPacketNumbers _packet_numbers;
   uint16_t _packet_count = 0;
+  std::set<QUICPacketNumber> _unprotected_packets;
 
   void _sort_packet_numbers();
   std::unique_ptr<QUICAckFrame, QUICFrameDeleterFunc> _create_ack_frame();
diff --git a/iocore/net/quic/QUICFrame.cc b/iocore/net/quic/QUICFrame.cc
index 5663d7c..2e4bc1d 100644
--- a/iocore/net/quic/QUICFrame.cc
+++ b/iocore/net/quic/QUICFrame.cc
@@ -70,11 +70,19 @@ QUICFrame::reset(const uint8_t *buf, size_t len)
   this->_len = len;
 }
 
+bool
+QUICFrame::is_protected() const
+{
+  return this->_protection;
+}
+
 //
 // STREAM Frame
 //
 
-QUICStreamFrame::QUICStreamFrame(ats_unique_buf data, size_t data_len, QUICStreamId stream_id, QUICOffset offset, bool last)
+QUICStreamFrame::QUICStreamFrame(ats_unique_buf data, size_t data_len, QUICStreamId stream_id, QUICOffset offset, bool last,
+                                 bool protection)
+  : QUICFrame(protection)
 {
   this->_data      = std::move(data);
   this->_data_len  = data_len;
@@ -309,12 +317,14 @@ QUICStreamFrame::_get_length_field_len() const
 // ACK frame
 //
 
-QUICAckFrame::QUICAckFrame(const uint8_t *buf, size_t len) : QUICFrame(buf, len)
+QUICAckFrame::QUICAckFrame(const uint8_t *buf, size_t len, bool protection) : QUICFrame(buf, len, protection)
 {
   this->reset(buf, len);
 }
 
-QUICAckFrame::QUICAckFrame(QUICPacketNumber largest_acknowledged, uint64_t ack_delay, uint64_t first_ack_block_length)
+QUICAckFrame::QUICAckFrame(QUICPacketNumber largest_acknowledged, uint64_t ack_delay, uint64_t first_ack_block_length,
+                           bool protection)
+  : QUICFrame(protection)
 {
   this->_largest_acknowledged = largest_acknowledged;
   this->_ack_delay            = ack_delay;
@@ -373,6 +383,12 @@ QUICAckFrame::store(uint8_t *buf, size_t *len) const
   return;
 }
 
+bool
+QUICAckFrame::is_protected() const
+{
+  return QUICFrame::is_protected() || this->_ack_block_section->has_protected();
+}
+
 QUICPacketNumber
 QUICAckFrame::largest_acknowledged() const
 {
@@ -619,9 +635,16 @@ QUICAckFrame::AckBlockSection::first_ack_block_length() const
 }
 
 void
-QUICAckFrame::AckBlockSection::add_ack_block(AckBlock block)
+QUICAckFrame::AckBlockSection::add_ack_block(AckBlock block, bool protection)
 {
   this->_ack_blocks.push_back(block);
+  this->_protection |= protection;
+}
+
+bool
+QUICAckFrame::AckBlockSection::has_protected() const
+{
+  return this->_protection;
 }
 
 QUICAckFrame::AckBlockSection::const_iterator
@@ -715,8 +738,9 @@ QUICAckFrame::AckBlockSection::const_iterator::operator==(const const_iterator &
 // RST_STREAM frame
 //
 
-QUICRstStreamFrame::QUICRstStreamFrame(QUICStreamId stream_id, QUICAppErrorCode error_code, QUICOffset final_offset)
-  : _stream_id(stream_id), _error_code(error_code), _final_offset(final_offset)
+QUICRstStreamFrame::QUICRstStreamFrame(QUICStreamId stream_id, QUICAppErrorCode error_code, QUICOffset final_offset,
+                                       bool protection)
+  : QUICFrame(protection), _stream_id(stream_id), _error_code(error_code), _final_offset(final_offset)
 {
 }
 
@@ -904,7 +928,8 @@ QUICPaddingFrame::store(uint8_t *buf, size_t *len) const
 // CONNECTION_CLOSE frame
 //
 QUICConnectionCloseFrame::QUICConnectionCloseFrame(QUICTransErrorCode error_code, uint64_t reason_phrase_length,
-                                                   const char *reason_phrase)
+                                                   const char *reason_phrase, bool protection)
+  : QUICFrame(protection)
 {
   this->_error_code           = error_code;
   this->_reason_phrase_length = reason_phrase_length;
@@ -1004,7 +1029,8 @@ QUICConnectionCloseFrame::_get_reason_phrase_field_offset() const
 // APPLICATION_CLOSE frame
 //
 QUICApplicationCloseFrame::QUICApplicationCloseFrame(QUICAppErrorCode error_code, uint64_t reason_phrase_length,
-                                                     const char *reason_phrase)
+                                                     const char *reason_phrase, bool protection)
+  : QUICFrame(protection)
 {
   this->_error_code           = error_code;
   this->_reason_phrase_length = reason_phrase_length;
@@ -1103,7 +1129,7 @@ QUICApplicationCloseFrame::_get_reason_phrase_field_offset() const
 //
 // MAX_DATA frame
 //
-QUICMaxDataFrame::QUICMaxDataFrame(uint64_t maximum_data)
+QUICMaxDataFrame::QUICMaxDataFrame(uint64_t maximum_data, bool protection)
 {
   this->_maximum_data = maximum_data;
 }
@@ -1161,7 +1187,8 @@ QUICMaxDataFrame::_get_max_data_field_length() const
 //
 // MAX_STREAM_DATA
 //
-QUICMaxStreamDataFrame::QUICMaxStreamDataFrame(QUICStreamId stream_id, uint64_t maximum_stream_data)
+QUICMaxStreamDataFrame::QUICMaxStreamDataFrame(QUICStreamId stream_id, uint64_t maximum_stream_data, bool protection)
+  : QUICFrame(protection)
 {
   this->_stream_id           = stream_id;
   this->_maximum_stream_data = maximum_stream_data;
@@ -1254,7 +1281,7 @@ QUICMaxStreamDataFrame::_get_max_stream_data_field_length() const
 //
 // MAX_STREAM_ID
 //
-QUICMaxStreamIdFrame::QUICMaxStreamIdFrame(QUICStreamId maximum_stream_id)
+QUICMaxStreamIdFrame::QUICMaxStreamIdFrame(QUICStreamId maximum_stream_id, bool protection) : QUICFrame(protection)
 {
   this->_maximum_stream_id = maximum_stream_id;
 }
@@ -1586,8 +1613,8 @@ QUICNewConnectionIdFrame::_get_connection_id_field_offset() const
 // STOP_SENDING frame
 //
 
-QUICStopSendingFrame::QUICStopSendingFrame(QUICStreamId stream_id, QUICAppErrorCode error_code)
-  : _stream_id(stream_id), _error_code(error_code)
+QUICStopSendingFrame::QUICStopSendingFrame(QUICStreamId stream_id, QUICAppErrorCode error_code, bool protection)
+  : QUICFrame(protection), _stream_id(stream_id), _error_code(error_code)
 {
 }
 
@@ -1719,7 +1746,7 @@ QUICPongFrame::_data_offset() const
 // QUICRetransmissionFrame
 //
 QUICRetransmissionFrame::QUICRetransmissionFrame(QUICFrameUPtr original_frame, const QUICPacket &original_packet)
-  : QUICFrame(), _packet_type(original_packet.type())
+  : QUICFrame(original_frame->is_protected()), _packet_type(original_packet.type())
 {
   size_t dummy;
   this->_size = original_frame->size();
@@ -1862,21 +1889,23 @@ QUICFrameFactory::fast_create(const uint8_t *buf, size_t len)
 }
 
 QUICStreamFrameUPtr
-QUICFrameFactory::create_stream_frame(const uint8_t *data, size_t data_len, QUICStreamId stream_id, QUICOffset offset, bool last)
+QUICFrameFactory::create_stream_frame(const uint8_t *data, size_t data_len, QUICStreamId stream_id, QUICOffset offset, bool last,
+                                      bool protection)
 {
   ats_unique_buf buf = ats_unique_malloc(data_len);
   memcpy(buf.get(), data, data_len);
 
   QUICStreamFrame *frame = quicStreamFrameAllocator.alloc();
-  new (frame) QUICStreamFrame(std::move(buf), data_len, stream_id, offset, last);
+  new (frame) QUICStreamFrame(std::move(buf), data_len, stream_id, offset, last, protection);
   return QUICStreamFrameUPtr(frame, &QUICFrameDeleter::delete_stream_frame);
 }
 
 std::unique_ptr<QUICAckFrame, QUICFrameDeleterFunc>
-QUICFrameFactory::create_ack_frame(QUICPacketNumber largest_acknowledged, uint64_t ack_delay, uint64_t first_ack_block_length)
+QUICFrameFactory::create_ack_frame(QUICPacketNumber largest_acknowledged, uint64_t ack_delay, uint64_t first_ack_block_length,
+                                   bool protection)
 {
   QUICAckFrame *frame = quicAckFrameAllocator.alloc();
-  new (frame) QUICAckFrame(largest_acknowledged, ack_delay, first_ack_block_length);
+  new (frame) QUICAckFrame(largest_acknowledged, ack_delay, first_ack_block_length, protection);
   return std::unique_ptr<QUICAckFrame, QUICFrameDeleterFunc>(frame, &QUICFrameDeleter::delete_ack_frame);
 }
 
diff --git a/iocore/net/quic/QUICFrame.h b/iocore/net/quic/QUICFrame.h
index fb4729d..d2ef572 100644
--- a/iocore/net/quic/QUICFrame.h
+++ b/iocore/net/quic/QUICFrame.h
@@ -34,19 +34,24 @@
 class QUICFrame
 {
 public:
-  QUICFrame(const uint8_t *buf, size_t len) : _buf(buf), _len(len){};
+  QUICFrame(const uint8_t *buf, size_t len, bool protection) : _buf(buf), _len(len), _protection(protection) {}
+  QUICFrame(bool protection) : _protection(protection) {}
+  virtual ~QUICFrame() {}
+
+  static QUICFrameType type(const uint8_t *buf);
+
   virtual QUICFrameType type() const;
   virtual size_t size() const = 0;
   virtual void store(uint8_t *buf, size_t *len) const = 0;
   virtual void reset(const uint8_t *buf, size_t len);
-  static QUICFrameType type(const uint8_t *buf);
-  virtual ~QUICFrame() {}
+  virtual bool is_protected() const;
   LINK(QUICFrame, link);
 
 protected:
   QUICFrame() {}
   const uint8_t *_buf = nullptr;
   size_t _len         = 0;
+  bool _protection    = true;
 };
 
 //
@@ -57,8 +62,9 @@ class QUICStreamFrame : public QUICFrame
 {
 public:
   QUICStreamFrame() : QUICFrame() {}
-  QUICStreamFrame(const uint8_t *buf, size_t len) : QUICFrame(buf, len) {}
-  QUICStreamFrame(ats_unique_buf buf, size_t len, QUICStreamId streamid, QUICOffset offset, bool last = false);
+  QUICStreamFrame(const uint8_t *buf, size_t len, bool protection = true) : QUICFrame(buf, len, protection) {}
+  QUICStreamFrame(ats_unique_buf buf, size_t len, QUICStreamId streamid, QUICOffset offset, bool last = false,
+                  bool protection = true);
 
   virtual QUICFrameType type() const override;
   virtual size_t size() const override;
@@ -168,9 +174,10 @@ public:
     size_t size() const;
     void store(uint8_t *buf, size_t *len) const;
     uint64_t first_ack_block_length() const;
-    void add_ack_block(const AckBlock block);
+    void add_ack_block(const AckBlock block, bool protection = true);
     const_iterator begin() const;
     const_iterator end() const;
+    bool has_protected() const;
 
   private:
     size_t _get_first_ack_block_length_size() const;
@@ -179,17 +186,19 @@ public:
     uint64_t _first_ack_block_length = 0;
     uint8_t _ack_block_count         = 0;
     std::vector<QUICAckFrame::AckBlock> _ack_blocks;
+    bool _protection = false;
   };
 
   QUICAckFrame() : QUICFrame() {}
-  QUICAckFrame(const uint8_t *buf, size_t len);
-  QUICAckFrame(QUICPacketNumber largest_acknowledged, uint64_t ack_delay, uint64_t first_ack_block_length);
+  QUICAckFrame(const uint8_t *buf, size_t len, bool protection = true);
+  QUICAckFrame(QUICPacketNumber largest_acknowledged, uint64_t ack_delay, uint64_t first_ack_block_length, bool protection = true);
 
   virtual ~QUICAckFrame();
   virtual void reset(const uint8_t *buf, size_t len) override;
   virtual QUICFrameType type() const override;
   virtual size_t size() const override;
   virtual void store(uint8_t *buf, size_t *len) const override;
+  bool is_protected() const override;
 
   QUICPacketNumber largest_acknowledged() const;
   uint64_t ack_delay() const;
@@ -219,8 +228,8 @@ class QUICRstStreamFrame : public QUICFrame
 {
 public:
   QUICRstStreamFrame() : QUICFrame() {}
-  QUICRstStreamFrame(const uint8_t *buf, size_t len) : QUICFrame(buf, len) {}
-  QUICRstStreamFrame(QUICStreamId stream_id, QUICAppErrorCode error_code, QUICOffset final_offset);
+  QUICRstStreamFrame(const uint8_t *buf, size_t len, bool protection = true) : QUICFrame(buf, len, protection) {}
+  QUICRstStreamFrame(QUICStreamId stream_id, QUICAppErrorCode error_code, QUICOffset final_offset, bool protection = true);
 
   virtual QUICFrameType type() const override;
   virtual size_t size() const override;
@@ -250,8 +259,11 @@ class QUICPingFrame : public QUICFrame
 {
 public:
   QUICPingFrame() : QUICFrame() {}
-  QUICPingFrame(const uint8_t *buf, size_t len) : QUICFrame(buf, len) {}
-  QUICPingFrame(ats_unique_buf data, size_t data_len) : _data(std::move(data)), _data_len(data_len) {}
+  QUICPingFrame(const uint8_t *buf, size_t len, bool protection = true) : QUICFrame(buf, len, protection) {}
+  QUICPingFrame(ats_unique_buf data, size_t data_len, bool protection = true)
+    : QUICFrame(protection), _data(std::move(data)), _data_len(data_len)
+  {
+  }
 
   virtual QUICFrameType type() const override;
   virtual size_t size() const override;
@@ -273,7 +285,7 @@ class QUICPaddingFrame : public QUICFrame
 {
 public:
   QUICPaddingFrame() : QUICFrame() {}
-  QUICPaddingFrame(const uint8_t *buf, size_t len) : QUICFrame(buf, len) {}
+  QUICPaddingFrame(const uint8_t *buf, size_t len, bool protection = true) : QUICFrame(buf, len, protection) {}
   virtual QUICFrameType type() const override;
   virtual size_t size() const override;
   virtual void store(uint8_t *buf, size_t *len) const override;
@@ -287,8 +299,9 @@ class QUICConnectionCloseFrame : public QUICFrame
 {
 public:
   QUICConnectionCloseFrame() : QUICFrame() {}
-  QUICConnectionCloseFrame(const uint8_t *buf, size_t len) : QUICFrame(buf, len) {}
-  QUICConnectionCloseFrame(QUICTransErrorCode error_code, uint64_t reason_phrase_length, const char *reason_phrase);
+  QUICConnectionCloseFrame(const uint8_t *buf, size_t len, bool protection = true) : QUICFrame(buf, len, protection) {}
+  QUICConnectionCloseFrame(QUICTransErrorCode error_code, uint64_t reason_phrase_length, const char *reason_phrase,
+                           bool protection = true);
   virtual QUICFrameType type() const override;
   virtual size_t size() const override;
   virtual void store(uint8_t *buf, size_t *len) const override;
@@ -314,8 +327,9 @@ class QUICApplicationCloseFrame : public QUICFrame
 {
 public:
   QUICApplicationCloseFrame() : QUICFrame() {}
-  QUICApplicationCloseFrame(const uint8_t *buf, size_t len) : QUICFrame(buf, len) {}
-  QUICApplicationCloseFrame(QUICAppErrorCode error_code, uint64_t reason_phrase_length, const char *reason_phrase);
+  QUICApplicationCloseFrame(const uint8_t *buf, size_t len, bool protection = true) : QUICFrame(buf, len, protection) {}
+  QUICApplicationCloseFrame(QUICAppErrorCode error_code, uint64_t reason_phrase_length, const char *reason_phrase,
+                            bool protection = true);
   virtual QUICFrameType type() const override;
   virtual size_t size() const override;
   virtual void store(uint8_t *buf, size_t *len) const override;
@@ -341,8 +355,8 @@ class QUICMaxDataFrame : public QUICFrame
 {
 public:
   QUICMaxDataFrame() : QUICFrame() {}
-  QUICMaxDataFrame(const uint8_t *buf, size_t len) : QUICFrame(buf, len) {}
-  QUICMaxDataFrame(uint64_t maximum_data);
+  QUICMaxDataFrame(const uint8_t *buf, size_t len, bool protection = true) : QUICFrame(buf, len, protection) {}
+  QUICMaxDataFrame(uint64_t maximum_data, bool protection = true);
   virtual QUICFrameType type() const override;
   virtual size_t size() const override;
   virtual void store(uint8_t *buf, size_t *len) const override;
@@ -362,8 +376,8 @@ class QUICMaxStreamDataFrame : public QUICFrame
 {
 public:
   QUICMaxStreamDataFrame() : QUICFrame() {}
-  QUICMaxStreamDataFrame(const uint8_t *buf, size_t len) : QUICFrame(buf, len) {}
-  QUICMaxStreamDataFrame(QUICStreamId stream_id, uint64_t maximum_stream_data);
+  QUICMaxStreamDataFrame(const uint8_t *buf, size_t len, bool protection = true) : QUICFrame(buf, len, protection) {}
+  QUICMaxStreamDataFrame(QUICStreamId stream_id, uint64_t maximum_stream_data, bool protection = true);
   virtual QUICFrameType type() const override;
   virtual size_t size() const override;
   virtual void store(uint8_t *buf, size_t *len) const override;
@@ -388,8 +402,8 @@ class QUICMaxStreamIdFrame : public QUICFrame
 {
 public:
   QUICMaxStreamIdFrame() : QUICFrame() {}
-  QUICMaxStreamIdFrame(const uint8_t *buf, size_t len) : QUICFrame(buf, len) {}
-  QUICMaxStreamIdFrame(QUICStreamId maximum_stream_id);
+  QUICMaxStreamIdFrame(const uint8_t *buf, size_t len, bool protection = true) : QUICFrame(buf, len, protection) {}
+  QUICMaxStreamIdFrame(QUICStreamId maximum_stream_id, bool protection = true);
   virtual QUICFrameType type() const override;
   virtual size_t size() const override;
   virtual void store(uint8_t *buf, size_t *len) const override;
@@ -408,8 +422,8 @@ class QUICBlockedFrame : public QUICFrame
 {
 public:
   QUICBlockedFrame() : QUICFrame() {}
-  QUICBlockedFrame(const uint8_t *buf, size_t len) : QUICFrame(buf, len) {}
-  QUICBlockedFrame(QUICOffset offset) : _offset(offset){};
+  QUICBlockedFrame(const uint8_t *buf, size_t len, bool protection = true) : QUICFrame(buf, len, protection) {}
+  QUICBlockedFrame(QUICOffset offset, bool protection = true) : QUICFrame(protection), _offset(offset){};
 
   virtual QUICFrameType type() const override;
   virtual size_t size() const override;
@@ -431,8 +445,8 @@ class QUICStreamBlockedFrame : public QUICFrame
 {
 public:
   QUICStreamBlockedFrame() : QUICFrame() {}
-  QUICStreamBlockedFrame(const uint8_t *buf, size_t len) : QUICFrame(buf, len) {}
-  QUICStreamBlockedFrame(QUICStreamId s, QUICOffset o) : _stream_id(s), _offset(o){};
+  QUICStreamBlockedFrame(const uint8_t *buf, size_t len, bool protection = true) : QUICFrame(buf, len, protection) {}
+  QUICStreamBlockedFrame(QUICStreamId s, QUICOffset o, bool protection = true) : QUICFrame(protection), _stream_id(s), _offset(o){};
 
   virtual QUICFrameType type() const override;
   virtual size_t size() const override;
@@ -457,8 +471,8 @@ class QUICStreamIdBlockedFrame : public QUICFrame
 {
 public:
   QUICStreamIdBlockedFrame() : QUICFrame() {}
-  QUICStreamIdBlockedFrame(const uint8_t *buf, size_t len) : QUICFrame(buf, len) {}
-  QUICStreamIdBlockedFrame(QUICStreamId s) : _stream_id(s) {}
+  QUICStreamIdBlockedFrame(const uint8_t *buf, size_t len, bool protection = true) : QUICFrame(buf, len, protection) {}
+  QUICStreamIdBlockedFrame(QUICStreamId s, bool protection = true) : QUICFrame(protection), _stream_id(s) {}
 
   virtual QUICFrameType type() const override;
   virtual size_t size() const override;
@@ -480,9 +494,9 @@ class QUICNewConnectionIdFrame : public QUICFrame
 {
 public:
   QUICNewConnectionIdFrame() : QUICFrame() {}
-  QUICNewConnectionIdFrame(const uint8_t *buf, size_t len) : QUICFrame(buf, len) {}
-  QUICNewConnectionIdFrame(uint64_t seq, QUICConnectionId id, QUICStatelessResetToken token)
-    : _sequence(seq), _connection_id(id), _stateless_reset_token(token){};
+  QUICNewConnectionIdFrame(const uint8_t *buf, size_t len, bool protection = true) : QUICFrame(buf, len, protection) {}
+  QUICNewConnectionIdFrame(uint64_t seq, QUICConnectionId id, QUICStatelessResetToken token, bool protection = true)
+    : QUICFrame(protection), _sequence(seq), _connection_id(id), _stateless_reset_token(token){};
 
   virtual QUICFrameType type() const override;
   virtual size_t size() const override;
@@ -509,8 +523,8 @@ class QUICStopSendingFrame : public QUICFrame
 {
 public:
   QUICStopSendingFrame() : QUICFrame() {}
-  QUICStopSendingFrame(const uint8_t *buf, size_t len) : QUICFrame(buf, len) {}
-  QUICStopSendingFrame(QUICStreamId stream_id, QUICAppErrorCode error_code);
+  QUICStopSendingFrame(const uint8_t *buf, size_t len, bool protection = true) : QUICFrame(buf, len, protection) {}
+  QUICStopSendingFrame(QUICStreamId stream_id, QUICAppErrorCode error_code, bool protection = true);
 
   virtual QUICFrameType type() const override;
   virtual size_t size() const override;
@@ -539,8 +553,11 @@ class QUICPongFrame : public QUICFrame
 {
 public:
   QUICPongFrame() : QUICFrame() {}
-  QUICPongFrame(const uint8_t *buf, size_t len) : QUICFrame(buf, len) {}
-  QUICPongFrame(ats_unique_buf data, size_t data_len) : _data(std::move(data)), _data_len(data_len) {}
+  QUICPongFrame(const uint8_t *buf, size_t len, bool protection = true) : QUICFrame(buf, len, protection) {}
+  QUICPongFrame(ats_unique_buf data, size_t data_len, bool protection = true)
+    : QUICFrame(protection), _data(std::move(data)), _data_len(data_len)
+  {
+  }
 
   virtual QUICFrameType type() const override;
   virtual size_t size() const override;
@@ -753,14 +770,15 @@ public:
    * You have to make sure that the data size won't exceed the maximum size of QUIC packet.
    */
   static QUICStreamFrameUPtr create_stream_frame(const uint8_t *data, size_t data_len, QUICStreamId stream_id, QUICOffset offset,
-                                                 bool last = false);
+                                                 bool last = false, bool protection = true);
   /*
    * Creates a ACK frame.
    * You shouldn't call this directly but through QUICAckFrameCreator because QUICAckFrameCreator manages packet numbers that we
    * need to ack.
    */
   static std::unique_ptr<QUICAckFrame, QUICFrameDeleterFunc> create_ack_frame(QUICPacketNumber largest_acknowledged,
-                                                                              uint64_t ack_delay, uint64_t first_ack_block_length);
+                                                                              uint64_t ack_delay, uint64_t first_ack_block_length,
+                                                                              bool protection = true);
   /*
    * Creates a CONNECTION_CLOSE frame.
    */
diff --git a/iocore/net/quic/QUICIncomingFrameBuffer.cc b/iocore/net/quic/QUICIncomingFrameBuffer.cc
index a00d6d7..f2cea29 100644
--- a/iocore/net/quic/QUICIncomingFrameBuffer.cc
+++ b/iocore/net/quic/QUICIncomingFrameBuffer.cc
@@ -101,7 +101,7 @@ std::shared_ptr<const QUICStreamFrame>
 QUICIncomingFrameBuffer::_clone(std::shared_ptr<const QUICStreamFrame> frame)
 {
   return QUICFrameFactory::create_stream_frame(frame->data(), frame->data_length(), frame->stream_id(), frame->offset(),
-                                               frame->has_fin_flag());
+                                               frame->has_fin_flag(), frame->is_protected());
 }
 
 QUICErrorUPtr
diff --git a/iocore/net/quic/QUICStream.cc b/iocore/net/quic/QUICStream.cc
index 195ed4c..f82c81a 100644
--- a/iocore/net/quic/QUICStream.cc
+++ b/iocore/net/quic/QUICStream.cc
@@ -476,8 +476,9 @@ QUICStream::_process_write_vio()
       }
     }
 
+    bool protection           = this->_id != STREAM_ID_FOR_HANDSHAKE;
     QUICStreamFrameUPtr frame = QUICFrameFactory::create_stream_frame(reinterpret_cast<const uint8_t *>(reader->start()), len,
-                                                                      this->_id, this->_send_offset, fin);
+                                                                      this->_id, this->_send_offset, fin, protection);
     if (!this->_state.is_allowed_to_send(*frame)) {
       QUICStreamDebug("Canceled sending %s frame due to the stream state", QUICDebugNames::frame_type(frame->type()));
       break;

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