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 2017/10/12 06:51:55 UTC

[trafficserver] branch quic-latest updated: Fix heap-use-after-free

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 88b5d7c  Fix heap-use-after-free
88b5d7c is described below

commit 88b5d7c71e1288400356adb6103f4ec18794652b
Author: Masaori Koshiba <ma...@apache.org>
AuthorDate: Thu Oct 12 15:39:31 2017 +0900

    Fix heap-use-after-free
    
    Make data of QUICStramFrame ats_unique_buf. Copy data of _write_vio
    to the buffer, when QUICStream sends frame. Ideally this malloc and
    copy should be avoided.
---
 iocore/net/quic/QUICFrame.cc                       |  10 +-
 iocore/net/quic/QUICFrame.h                        |   6 +-
 iocore/net/quic/QUICStream.cc                      |   6 +-
 iocore/net/quic/test/test_QUICFrame.cc             |  50 ++++-
 iocore/net/quic/test/test_QUICFrameDispatcher.cc   |   7 +-
 .../net/quic/test/test_QUICIncomingFrameBuffer.cc  |  52 +++--
 iocore/net/quic/test/test_QUICStream.cc            | 215 ++++++++++++---------
 iocore/net/quic/test/test_QUICStreamManager.cc     |  40 ++--
 iocore/net/quic/test/test_QUICStreamState.cc       |  48 ++++-
 proxy/hq/HQClientTransaction.cc                    |   1 +
 10 files changed, 273 insertions(+), 162 deletions(-)

diff --git a/iocore/net/quic/QUICFrame.cc b/iocore/net/quic/QUICFrame.cc
index 69951d4..8d2cc7f 100644
--- a/iocore/net/quic/QUICFrame.cc
+++ b/iocore/net/quic/QUICFrame.cc
@@ -72,9 +72,9 @@ QUICFrame::reset(const uint8_t *buf, size_t len)
 // STREAM Frame
 //
 
-QUICStreamFrame::QUICStreamFrame(const uint8_t *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)
 {
-  this->_data      = data;
+  this->_data      = std::move(data);
   this->_data_len  = data_len;
   this->_stream_id = stream_id;
   this->_offset    = offset;
@@ -183,7 +183,7 @@ QUICStreamFrame::data() const
   if (this->_buf) {
     return this->_buf + this->_get_data_offset();
   } else {
-    return this->_data;
+    return this->_data.get();
   }
 }
 
@@ -1313,10 +1313,10 @@ 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(ats_unique_buf data, size_t data_len, QUICStreamId stream_id, QUICOffset offset, bool last)
 {
   QUICStreamFrame *frame = quicStreamFrameAllocator.alloc();
-  new (frame) QUICStreamFrame(data, data_len, stream_id, offset, last);
+  new (frame) QUICStreamFrame(std::move(data), data_len, stream_id, offset, last);
   return QUICStreamFrameUPtr(frame, &QUICFrameDeleter::delete_stream_frame);
 }
 
diff --git a/iocore/net/quic/QUICFrame.h b/iocore/net/quic/QUICFrame.h
index 772d65a..f84d328 100644
--- a/iocore/net/quic/QUICFrame.h
+++ b/iocore/net/quic/QUICFrame.h
@@ -59,7 +59,7 @@ class QUICStreamFrame : public QUICFrame
 public:
   QUICStreamFrame() : QUICFrame() {}
   QUICStreamFrame(const uint8_t *buf, size_t len) : QUICFrame(buf, len) {}
-  QUICStreamFrame(const uint8_t *buf, size_t len, QUICStreamId streamid, QUICOffset offset, bool last = false);
+  QUICStreamFrame(ats_unique_buf buf, size_t len, QUICStreamId streamid, QUICOffset offset, bool last = false);
   virtual QUICFrameType type() const override;
   virtual size_t size() const override;
   virtual void store(uint8_t *buf, size_t *len) const override;
@@ -74,7 +74,7 @@ public:
   LINK(QUICStreamFrame, link);
 
 private:
-  const uint8_t *_data    = nullptr;
+  ats_unique_buf _data    = {nullptr, [](void *p) { ats_free(p); }};
   size_t _data_len        = 0;
   QUICStreamId _stream_id = 0;
   QUICOffset _offset      = 0;
@@ -591,7 +591,7 @@ public:
    * Creates a STREAM frame.
    * 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,
+  static QUICStreamFrameUPtr create_stream_frame(ats_unique_buf data, size_t data_len, QUICStreamId stream_id, QUICOffset offset,
                                                  bool last = false);
   /*
    * Creates a ACK frame.
diff --git a/iocore/net/quic/QUICStream.cc b/iocore/net/quic/QUICStream.cc
index f0ec0ff..3e99573 100644
--- a/iocore/net/quic/QUICStream.cc
+++ b/iocore/net/quic/QUICStream.cc
@@ -376,8 +376,10 @@ QUICStream::_send()
       break;
     }
 
-    QUICStreamFrameUPtr frame = QUICFrameFactory::create_stream_frame(reinterpret_cast<const uint8_t *>(reader->start()), len,
-                                                                      this->_id, this->_send_offset, fin);
+    // TODO: avoid malloc & copy
+    ats_unique_buf data = ats_unique_malloc(len);
+    memcpy(data.get(), reader->start(), len);
+    QUICStreamFrameUPtr frame = QUICFrameFactory::create_stream_frame(std::move(data), len, this->_id, this->_send_offset, fin);
 
     this->_send_offset += len;
     reader->consume(len);
diff --git a/iocore/net/quic/test/test_QUICFrame.cc b/iocore/net/quic/test/test_QUICFrame.cc
index 90c82fb..36f1eee 100644
--- a/iocore/net/quic/test/test_QUICFrame.cc
+++ b/iocore/net/quic/test/test_QUICFrame.cc
@@ -55,11 +55,14 @@ TEST_CASE("QUICFrame Type", "[quic]")
 
 TEST_CASE("Construct QUICFrame", "[quic]")
 {
-  uint8_t payload[] = "foo";
+  uint8_t raw[]          = "foo";
+  ats_unique_buf payload = ats_unique_malloc(sizeof(raw));
+  memcpy(payload.get(), raw, sizeof(raw));
+
   uint8_t buf[65536];
   size_t len;
 
-  QUICStreamFrame frame1(payload, sizeof(payload), 0xffcc9966, 0xffddbb9977553311);
+  QUICStreamFrame frame1(std::move(payload), sizeof(raw), 0xffcc9966, 0xffddbb9977553311);
   frame1.store(buf, &len);
   CHECK(frame1.type() == QUICFrameType::STREAM);
   CHECK(frame1.size() == 19);
@@ -119,7 +122,12 @@ TEST_CASE("Store STREAM Frame", "[quic]")
     0x00, 0x05,                   // Data Length
     0x01, 0x02, 0x03, 0x04, 0x05, // Stream Data
   };
-  QUICStreamFrame streamFrame1(reinterpret_cast<const uint8_t *>("\x01\x02\x03\x04\x05"), 5, 0x01, 0x00);
+
+  uint8_t raw1[]          = "\x01\x02\x03\x04\x05";
+  ats_unique_buf payload1 = ats_unique_malloc(5);
+  memcpy(payload1.get(), raw1, 5);
+
+  QUICStreamFrame streamFrame1(std::move(payload1), 5, 0x01, 0x00);
   streamFrame1.store(buf, &len);
   CHECK(len == 9);
   CHECK(memcmp(buf, expected1, len) == 0);
@@ -132,7 +140,11 @@ TEST_CASE("Store STREAM Frame", "[quic]")
     0x00, 0x05,                   // Data Length
     0x01, 0x02, 0x03, 0x04, 0x05, // Stream Data
   };
-  QUICStreamFrame streamFrame2(reinterpret_cast<const uint8_t *>("\x01\x02\x03\x04\x05"), 5, 0x01, 0x01);
+  uint8_t raw2[]          = "\x01\x02\x03\x04\x05";
+  ats_unique_buf payload2 = ats_unique_malloc(5);
+  memcpy(payload2.get(), raw2, 5);
+
+  QUICStreamFrame streamFrame2(std::move(payload2), 5, 0x01, 0x01);
   streamFrame2.store(buf, &len);
   CHECK(len == 11);
   CHECK(memcmp(buf, expected2, len) == 0);
@@ -145,7 +157,11 @@ TEST_CASE("Store STREAM Frame", "[quic]")
     0x00, 0x05,                   // Data Length
     0x01, 0x02, 0x03, 0x04, 0x05, // Stream Data
   };
-  QUICStreamFrame streamFrame3(reinterpret_cast<const uint8_t *>("\x01\x02\x03\x04\x05"), 5, 0x01, 0x010000);
+  uint8_t raw3[]          = "\x01\x02\x03\x04\x05";
+  ats_unique_buf payload3 = ats_unique_malloc(5);
+  memcpy(payload3.get(), raw3, 5);
+
+  QUICStreamFrame streamFrame3(std::move(payload3), 5, 0x01, 0x010000);
   streamFrame3.store(buf, &len);
   CHECK(len == 13);
   CHECK(memcmp(buf, expected3, len) == 0);
@@ -158,7 +174,11 @@ TEST_CASE("Store STREAM Frame", "[quic]")
     0x00, 0x05,                                     // Data Length
     0x01, 0x02, 0x03, 0x04, 0x05,                   // Stream Data
   };
-  QUICStreamFrame streamFrame4(reinterpret_cast<const uint8_t *>("\x01\x02\x03\x04\x05"), 5, 0x01, 0x0100000000);
+  uint8_t raw4[]          = "\x01\x02\x03\x04\x05";
+  ats_unique_buf payload4 = ats_unique_malloc(5);
+  memcpy(payload4.get(), raw4, 5);
+
+  QUICStreamFrame streamFrame4(std::move(payload4), 5, 0x01, 0x0100000000);
   streamFrame4.store(buf, &len);
   CHECK(len == 17);
   CHECK(memcmp(buf, expected4, len) == 0);
@@ -171,7 +191,11 @@ TEST_CASE("Store STREAM Frame", "[quic]")
     0x00, 0x05,                                     // Data Length
     0x01, 0x02, 0x03, 0x04, 0x05,                   // Stream Data
   };
-  QUICStreamFrame streamFrame5(reinterpret_cast<const uint8_t *>("\x01\x02\x03\x04\x05"), 5, 0x0100, 0x0100000000);
+  uint8_t raw5[]          = "\x01\x02\x03\x04\x05";
+  ats_unique_buf payload5 = ats_unique_malloc(5);
+  memcpy(payload5.get(), raw5, 5);
+
+  QUICStreamFrame streamFrame5(std::move(payload5), 5, 0x0100, 0x0100000000);
   streamFrame5.store(buf, &len);
   CHECK(len == 18);
   CHECK(memcmp(buf, expected5, len) == 0);
@@ -184,7 +208,11 @@ TEST_CASE("Store STREAM Frame", "[quic]")
     0x00, 0x05,                                     // Data Length
     0x01, 0x02, 0x03, 0x04, 0x05,                   // Stream Data
   };
-  QUICStreamFrame streamFrame6(reinterpret_cast<const uint8_t *>("\x01\x02\x03\x04\x05"), 5, 0x010000, 0x0100000000);
+  uint8_t raw6[]          = "\x01\x02\x03\x04\x05";
+  ats_unique_buf payload6 = ats_unique_malloc(5);
+  memcpy(payload6.get(), raw6, 5);
+
+  QUICStreamFrame streamFrame6(std::move(payload6), 5, 0x010000, 0x0100000000);
   streamFrame6.store(buf, &len);
   CHECK(len == 19);
   CHECK(memcmp(buf, expected6, len) == 0);
@@ -197,7 +225,11 @@ TEST_CASE("Store STREAM Frame", "[quic]")
     0x00, 0x05,                                     // Data Length
     0x01, 0x02, 0x03, 0x04, 0x05,                   // Stream Data
   };
-  QUICStreamFrame streamFrame7(reinterpret_cast<const uint8_t *>("\x01\x02\x03\x04\x05"), 5, 0x01000000, 0x0100000000);
+  uint8_t raw7[]          = "\x01\x02\x03\x04\x05";
+  ats_unique_buf payload7 = ats_unique_malloc(5);
+  memcpy(payload7.get(), raw7, 5);
+
+  QUICStreamFrame streamFrame7(std::move(payload7), 5, 0x01000000, 0x0100000000);
   streamFrame7.store(buf, &len);
   CHECK(len == 20);
   CHECK(memcmp(buf, expected7, len) == 0);
diff --git a/iocore/net/quic/test/test_QUICFrameDispatcher.cc b/iocore/net/quic/test/test_QUICFrameDispatcher.cc
index 8573dce..215303f 100644
--- a/iocore/net/quic/test/test_QUICFrameDispatcher.cc
+++ b/iocore/net/quic/test/test_QUICFrameDispatcher.cc
@@ -29,8 +29,11 @@
 
 TEST_CASE("QUICFrameHandler", "[quic]")
 {
-  uint8_t payload[] = {0x01};
-  QUICStreamFrame streamFrame(payload, 1, 0x03, 0);
+  uint8_t raw[]          = {0x01};
+  ats_unique_buf payload = ats_unique_malloc(1);
+  memcpy(payload.get(), raw, 1);
+
+  QUICStreamFrame streamFrame(std::move(payload), 1, 0x03, 0);
 
   auto connection           = new MockQUICConnection();
   auto streamManager        = new MockQUICStreamManager();
diff --git a/iocore/net/quic/test/test_QUICIncomingFrameBuffer.cc b/iocore/net/quic/test/test_QUICIncomingFrameBuffer.cc
index 844607a..d4e06a7 100644
--- a/iocore/net/quic/test/test_QUICIncomingFrameBuffer.cc
+++ b/iocore/net/quic/test/test_QUICIncomingFrameBuffer.cc
@@ -33,13 +33,15 @@ TEST_CASE("QUICIncomingFrameBuffer_fin_offset", "[quic]")
   QUICIncomingFrameBuffer buffer(stream);
   QUICErrorUPtr err = nullptr;
 
-  uint8_t data[1024] = {0};
-
-  std::shared_ptr<QUICStreamFrame> stream1_frame_0_r = QUICFrameFactory::create_stream_frame(data, 1024, 1, 0);
-  std::shared_ptr<QUICStreamFrame> stream1_frame_1_r = QUICFrameFactory::create_stream_frame(data, 1024, 1, 1024);
-  std::shared_ptr<QUICStreamFrame> stream1_frame_2_r = QUICFrameFactory::create_stream_frame(data, 1024, 1, 2048, true);
-  std::shared_ptr<QUICStreamFrame> stream1_frame_3_r = QUICFrameFactory::create_stream_frame(data, 1024, 1, 3072, true);
-  std::shared_ptr<QUICStreamFrame> stream1_frame_4_r = QUICFrameFactory::create_stream_frame(data, 1024, 1, 4096);
+  std::shared_ptr<QUICStreamFrame> stream1_frame_0_r = QUICFrameFactory::create_stream_frame(ats_unique_malloc(1024), 1024, 1, 0);
+  std::shared_ptr<QUICStreamFrame> stream1_frame_1_r =
+    QUICFrameFactory::create_stream_frame(ats_unique_malloc(1024), 1024, 1, 1024);
+  std::shared_ptr<QUICStreamFrame> stream1_frame_2_r =
+    QUICFrameFactory::create_stream_frame(ats_unique_malloc(1024), 1024, 1, 2048, true);
+  std::shared_ptr<QUICStreamFrame> stream1_frame_3_r =
+    QUICFrameFactory::create_stream_frame(ats_unique_malloc(1024), 1024, 1, 3072, true);
+  std::shared_ptr<QUICStreamFrame> stream1_frame_4_r =
+    QUICFrameFactory::create_stream_frame(ats_unique_malloc(1024), 1024, 1, 4096);
 
   buffer.insert(stream1_frame_0_r);
   buffer.insert(stream1_frame_1_r);
@@ -72,11 +74,15 @@ TEST_CASE("QUICIncomingFrameBuffer_pop", "[quic]")
 
   uint8_t data[1024] = {0};
 
-  std::shared_ptr<QUICStreamFrame> stream1_frame_0_r = QUICFrameFactory::create_stream_frame(data, 1024, 1, 0);
-  std::shared_ptr<QUICStreamFrame> stream1_frame_1_r = QUICFrameFactory::create_stream_frame(data, 1024, 1, 1024);
-  std::shared_ptr<QUICStreamFrame> stream1_frame_2_r = QUICFrameFactory::create_stream_frame(data, 1024, 1, 2048);
-  std::shared_ptr<QUICStreamFrame> stream1_frame_3_r = QUICFrameFactory::create_stream_frame(data, 1024, 1, 3072);
-  std::shared_ptr<QUICStreamFrame> stream1_frame_4_r = QUICFrameFactory::create_stream_frame(data, 1024, 1, 4096, true);
+  std::shared_ptr<QUICStreamFrame> stream1_frame_0_r = QUICFrameFactory::create_stream_frame(ats_unique_malloc(1024), 1024, 1, 0);
+  std::shared_ptr<QUICStreamFrame> stream1_frame_1_r =
+    QUICFrameFactory::create_stream_frame(ats_unique_malloc(1024), 1024, 1, 1024);
+  std::shared_ptr<QUICStreamFrame> stream1_frame_2_r =
+    QUICFrameFactory::create_stream_frame(ats_unique_malloc(1024), 1024, 1, 2048);
+  std::shared_ptr<QUICStreamFrame> stream1_frame_3_r =
+    QUICFrameFactory::create_stream_frame(ats_unique_malloc(1024), 1024, 1, 3072);
+  std::shared_ptr<QUICStreamFrame> stream1_frame_4_r =
+    QUICFrameFactory::create_stream_frame(ats_unique_malloc(1024), 1024, 1, 4096, true);
 
   buffer.insert(stream1_frame_0_r);
   buffer.insert(stream1_frame_1_r);
@@ -129,10 +135,13 @@ TEST_CASE("QUICIncomingFrameBuffer_dup_frame", "[quic]")
 
   uint8_t data[1024] = {0};
 
-  std::shared_ptr<QUICStreamFrame> stream1_frame_0_r = QUICFrameFactory::create_stream_frame(data, 1024, 1, 0);
-  std::shared_ptr<QUICStreamFrame> stream1_frame_1_r = QUICFrameFactory::create_stream_frame(data, 1024, 1, 1024);
-  std::shared_ptr<QUICStreamFrame> stream1_frame_2_r = QUICFrameFactory::create_stream_frame(data, 1024, 1, 2048, true);
-  std::shared_ptr<QUICStreamFrame> stream1_frame_3_r = QUICFrameFactory::create_stream_frame(data, 1024, 1, 2048, true);
+  std::shared_ptr<QUICStreamFrame> stream1_frame_0_r = QUICFrameFactory::create_stream_frame(ats_unique_malloc(1024), 1024, 1, 0);
+  std::shared_ptr<QUICStreamFrame> stream1_frame_1_r =
+    QUICFrameFactory::create_stream_frame(ats_unique_malloc(1024), 1024, 1, 1024);
+  std::shared_ptr<QUICStreamFrame> stream1_frame_2_r =
+    QUICFrameFactory::create_stream_frame(ats_unique_malloc(1024), 1024, 1, 2048, true);
+  std::shared_ptr<QUICStreamFrame> stream1_frame_3_r =
+    QUICFrameFactory::create_stream_frame(ats_unique_malloc(1024), 1024, 1, 2048, true);
 
   buffer.insert(stream1_frame_0_r);
   buffer.insert(stream1_frame_1_r);
@@ -152,10 +161,13 @@ TEST_CASE("QUICIncomingFrameBuffer_dup_frame", "[quic]")
 
   buffer.clear();
 
-  std::shared_ptr<QUICStreamFrame> stream2_frame_0_r = QUICFrameFactory::create_stream_frame(data, 1024, 1, 0);
-  std::shared_ptr<QUICStreamFrame> stream2_frame_1_r = QUICFrameFactory::create_stream_frame(data, 1024, 1, 1024);
-  std::shared_ptr<QUICStreamFrame> stream2_frame_2_r = QUICFrameFactory::create_stream_frame(data, 1024, 1, 1024);
-  std::shared_ptr<QUICStreamFrame> stream2_frame_3_r = QUICFrameFactory::create_stream_frame(data, 1024, 1, 2048, true);
+  std::shared_ptr<QUICStreamFrame> stream2_frame_0_r = QUICFrameFactory::create_stream_frame(ats_unique_malloc(1024), 1024, 1, 0);
+  std::shared_ptr<QUICStreamFrame> stream2_frame_1_r =
+    QUICFrameFactory::create_stream_frame(ats_unique_malloc(1024), 1024, 1, 1024);
+  std::shared_ptr<QUICStreamFrame> stream2_frame_2_r =
+    QUICFrameFactory::create_stream_frame(ats_unique_malloc(1024), 1024, 1, 1024);
+  std::shared_ptr<QUICStreamFrame> stream2_frame_3_r =
+    QUICFrameFactory::create_stream_frame(ats_unique_malloc(1024), 1024, 1, 2048, true);
 
   buffer.insert(stream2_frame_0_r);
   buffer.insert(stream2_frame_1_r);
diff --git a/iocore/net/quic/test/test_QUICStream.cc b/iocore/net/quic/test/test_QUICStream.cc
index 632ad91..ef523ab 100644
--- a/iocore/net/quic/test/test_QUICStream.cc
+++ b/iocore/net/quic/test/test_QUICStream.cc
@@ -26,101 +26,124 @@
 #include "quic/QUICStream.h"
 #include "quic/Mock.h"
 
-namespace
+TEST_CASE("QUICStream", "[quic]")
 {
-// Test Data
-uint8_t payload[]  = {0x01, 0x02, 0x03, 0x04, 0x05, 0x06, 0x07, 0x08, 0x09, 0x0a, 0x0b, 0x0c, 0x0d, 0x0e, 0x0f, 0x10};
-uint32_t stream_id = 0x03;
-
-std::shared_ptr<QUICStreamFrame> frame_1 = std::make_shared<QUICStreamFrame>(payload, 2, stream_id, 0);
-std::shared_ptr<QUICStreamFrame> frame_2 = std::make_shared<QUICStreamFrame>(payload + 2, 2, stream_id, 2);
-std::shared_ptr<QUICStreamFrame> frame_3 = std::make_shared<QUICStreamFrame>(payload + 4, 2, stream_id, 4);
-std::shared_ptr<QUICStreamFrame> frame_4 = std::make_shared<QUICStreamFrame>(payload + 6, 2, stream_id, 6);
-std::shared_ptr<QUICStreamFrame> frame_5 = std::make_shared<QUICStreamFrame>(payload + 8, 2, stream_id, 8);
-std::shared_ptr<QUICStreamFrame> frame_6 = std::make_shared<QUICStreamFrame>(payload + 10, 2, stream_id, 10);
-std::shared_ptr<QUICStreamFrame> frame_7 = std::make_shared<QUICStreamFrame>(payload + 12, 2, stream_id, 12);
-std::shared_ptr<QUICStreamFrame> frame_8 = std::make_shared<QUICStreamFrame>(payload + 14, 2, stream_id, 14);
-
-TEST_CASE("QUICStream_assembling_byte_stream_1", "[quic]")
-{
-  MIOBuffer *read_buffer = new_MIOBuffer(BUFFER_SIZE_INDEX_4K);
-  IOBufferReader *reader = read_buffer->alloc_reader();
-  MockQUICFrameTransmitter tx;
-
-  std::unique_ptr<QUICStream> stream(new QUICStream());
-  stream->init(&tx, 0, stream_id, 1024, 1024);
-  stream->do_io_read(nullptr, 0, read_buffer);
-
-  stream->recv(frame_1);
-  stream->recv(frame_2);
-  stream->recv(frame_3);
-  stream->recv(frame_4);
-  stream->recv(frame_5);
-  stream->recv(frame_6);
-  stream->recv(frame_7);
-  stream->recv(frame_8);
-
-  uint8_t buf[32];
-  int64_t len = reader->read_avail();
-  reader->read(buf, len);
-
-  CHECK(len == 16);
-  CHECK(memcmp(buf, payload, len) == 0);
-}
-
-TEST_CASE("QUICStream_assembling_byte_stream_2", "[quic]")
-{
-  MIOBuffer *read_buffer = new_MIOBuffer(BUFFER_SIZE_INDEX_4K);
-  IOBufferReader *reader = read_buffer->alloc_reader();
-  MockQUICFrameTransmitter tx;
-
-  std::unique_ptr<QUICStream> stream(new QUICStream());
-  stream->init(&tx, 0, stream_id);
-  stream->do_io_read(nullptr, 0, read_buffer);
-
-  stream->recv(frame_8);
-  stream->recv(frame_7);
-  stream->recv(frame_6);
-  stream->recv(frame_5);
-  stream->recv(frame_4);
-  stream->recv(frame_3);
-  stream->recv(frame_2);
-  stream->recv(frame_1);
-
-  uint8_t buf[32];
-  int64_t len = reader->read_avail();
-  reader->read(buf, len);
-
-  CHECK(len == 16);
-  CHECK(memcmp(buf, payload, len) == 0);
-}
-
-TEST_CASE("QUICStream_assembling_byte_stream_3", "[quic]")
-{
-  MIOBuffer *read_buffer = new_MIOBuffer(BUFFER_SIZE_INDEX_4K);
-  IOBufferReader *reader = read_buffer->alloc_reader();
-  MockQUICFrameTransmitter tx;
-
-  std::unique_ptr<QUICStream> stream(new QUICStream());
-  stream->init(&tx, 0, stream_id);
-  stream->do_io_read(nullptr, 0, read_buffer);
-
-  stream->recv(frame_8);
-  stream->recv(frame_7);
-  stream->recv(frame_6);
-  stream->recv(frame_7); // duplicated frame
-  stream->recv(frame_5);
-  stream->recv(frame_3);
-  stream->recv(frame_1);
-  stream->recv(frame_2);
-  stream->recv(frame_4);
-  stream->recv(frame_5); // duplicated frame
-
-  uint8_t buf[32];
-  int64_t len = reader->read_avail();
-  reader->read(buf, len);
-
-  CHECK(len == 16);
-  CHECK(memcmp(buf, payload, len) == 0);
-}
+  // Test Data
+  uint8_t payload[]  = {0x01, 0x02, 0x03, 0x04, 0x05, 0x06, 0x07, 0x08, 0x09, 0x0a, 0x0b, 0x0c, 0x0d, 0x0e, 0x0f, 0x10};
+  uint32_t stream_id = 0x03;
+
+  ats_unique_buf payload1 = ats_unique_malloc(2);
+  memcpy(payload1.get(), payload, 2);
+  std::shared_ptr<QUICStreamFrame> frame_1 = std::make_shared<QUICStreamFrame>(std::move(payload1), 2, stream_id, 0);
+
+  ats_unique_buf payload2 = ats_unique_malloc(2);
+  memcpy(payload2.get(), payload + 2, 2);
+  std::shared_ptr<QUICStreamFrame> frame_2 = std::make_shared<QUICStreamFrame>(std::move(payload2), 2, stream_id, 2);
+
+  ats_unique_buf payload3 = ats_unique_malloc(2);
+  memcpy(payload3.get(), payload + 4, 2);
+  std::shared_ptr<QUICStreamFrame> frame_3 = std::make_shared<QUICStreamFrame>(std::move(payload3), 2, stream_id, 4);
+
+  ats_unique_buf payload4 = ats_unique_malloc(2);
+  memcpy(payload4.get(), payload + 6, 2);
+  std::shared_ptr<QUICStreamFrame> frame_4 = std::make_shared<QUICStreamFrame>(std::move(payload4), 2, stream_id, 6);
+
+  ats_unique_buf payload5 = ats_unique_malloc(2);
+  memcpy(payload5.get(), payload + 8, 2);
+  std::shared_ptr<QUICStreamFrame> frame_5 = std::make_shared<QUICStreamFrame>(std::move(payload5), 2, stream_id, 8);
+
+  ats_unique_buf payload6 = ats_unique_malloc(2);
+  memcpy(payload6.get(), payload + 10, 2);
+  std::shared_ptr<QUICStreamFrame> frame_6 = std::make_shared<QUICStreamFrame>(std::move(payload6), 2, stream_id, 10);
+
+  ats_unique_buf payload7 = ats_unique_malloc(2);
+  memcpy(payload7.get(), payload + 12, 2);
+  std::shared_ptr<QUICStreamFrame> frame_7 = std::make_shared<QUICStreamFrame>(std::move(payload7), 2, stream_id, 12);
+
+  ats_unique_buf payload8 = ats_unique_malloc(2);
+  memcpy(payload8.get(), payload + 14, 2);
+  std::shared_ptr<QUICStreamFrame> frame_8 = std::make_shared<QUICStreamFrame>(std::move(payload8), 2, stream_id, 14);
+
+  SECTION("QUICStream_assembling_byte_stream_1")
+  {
+    MIOBuffer *read_buffer = new_MIOBuffer(BUFFER_SIZE_INDEX_4K);
+    IOBufferReader *reader = read_buffer->alloc_reader();
+    MockQUICFrameTransmitter tx;
+
+    std::unique_ptr<QUICStream> stream(new QUICStream());
+    stream->init(&tx, 0, stream_id, 1024, 1024);
+    stream->do_io_read(nullptr, 0, read_buffer);
+
+    stream->recv(frame_1);
+    stream->recv(frame_2);
+    stream->recv(frame_3);
+    stream->recv(frame_4);
+    stream->recv(frame_5);
+    stream->recv(frame_6);
+    stream->recv(frame_7);
+    stream->recv(frame_8);
+
+    uint8_t buf[32];
+    int64_t len = reader->read_avail();
+    reader->read(buf, len);
+
+    CHECK(len == 16);
+    CHECK(memcmp(buf, payload, len) == 0);
+  }
+
+  SECTION("QUICStream_assembling_byte_stream_2")
+  {
+    MIOBuffer *read_buffer = new_MIOBuffer(BUFFER_SIZE_INDEX_4K);
+    IOBufferReader *reader = read_buffer->alloc_reader();
+    MockQUICFrameTransmitter tx;
+
+    std::unique_ptr<QUICStream> stream(new QUICStream());
+    stream->init(&tx, 0, stream_id);
+    stream->do_io_read(nullptr, 0, read_buffer);
+
+    stream->recv(frame_8);
+    stream->recv(frame_7);
+    stream->recv(frame_6);
+    stream->recv(frame_5);
+    stream->recv(frame_4);
+    stream->recv(frame_3);
+    stream->recv(frame_2);
+    stream->recv(frame_1);
+
+    uint8_t buf[32];
+    int64_t len = reader->read_avail();
+    reader->read(buf, len);
+
+    CHECK(len == 16);
+    CHECK(memcmp(buf, payload, len) == 0);
+  }
+
+  SECTION("QUICStream_assembling_byte_stream_3")
+  {
+    MIOBuffer *read_buffer = new_MIOBuffer(BUFFER_SIZE_INDEX_4K);
+    IOBufferReader *reader = read_buffer->alloc_reader();
+    MockQUICFrameTransmitter tx;
+
+    std::unique_ptr<QUICStream> stream(new QUICStream());
+    stream->init(&tx, 0, stream_id);
+    stream->do_io_read(nullptr, 0, read_buffer);
+
+    stream->recv(frame_8);
+    stream->recv(frame_7);
+    stream->recv(frame_6);
+    stream->recv(frame_7); // duplicated frame
+    stream->recv(frame_5);
+    stream->recv(frame_3);
+    stream->recv(frame_1);
+    stream->recv(frame_2);
+    stream->recv(frame_4);
+    stream->recv(frame_5); // duplicated frame
+
+    uint8_t buf[32];
+    int64_t len = reader->read_avail();
+    reader->read(buf, len);
+
+    CHECK(len == 16);
+    CHECK(memcmp(buf, payload, len) == 0);
+  }
 }
diff --git a/iocore/net/quic/test/test_QUICStreamManager.cc b/iocore/net/quic/test/test_QUICStreamManager.cc
index f9490c9..5c2f485 100644
--- a/iocore/net/quic/test/test_QUICStreamManager.cc
+++ b/iocore/net/quic/test/test_QUICStreamManager.cc
@@ -42,10 +42,14 @@ TEST_CASE("QUICStreamManager_NewStream", "[quic]")
   sm.init_flow_control_params(local_tp, remote_tp);
 
   // STREAM frames create new streams
-  std::shared_ptr<QUICFrame> stream_frame_0 =
-    QUICFrameFactory::create_stream_frame(reinterpret_cast<const uint8_t *>("abc"), 3, 0, 0);
-  std::shared_ptr<QUICFrame> stream_frame_1 =
-    QUICFrameFactory::create_stream_frame(reinterpret_cast<const uint8_t *>("abc"), 3, 1, 0);
+  uint8_t raw[]        = "abc";
+  ats_unique_buf data1 = ats_unique_malloc(sizeof(raw));
+  memcpy(data1.get(), raw, sizeof(raw));
+  ats_unique_buf data2 = ats_unique_malloc(sizeof(raw));
+  memcpy(data2.get(), raw, sizeof(raw));
+
+  std::shared_ptr<QUICFrame> stream_frame_0 = QUICFrameFactory::create_stream_frame(std::move(data1), sizeof(raw), 0, 0);
+  std::shared_ptr<QUICFrame> stream_frame_1 = QUICFrameFactory::create_stream_frame(std::move(data2), sizeof(raw), 1, 0);
   CHECK(sm.stream_count() == 0);
   sm.handle_frame(stream_frame_0);
   CHECK(sm.stream_count() == 1);
@@ -87,8 +91,11 @@ TEST_CASE("QUICStreamManager_first_initial_map", "[quic]")
   sm.init_flow_control_params(local_tp, remote_tp);
 
   // STREAM frames create new streams
-  std::shared_ptr<QUICFrame> stream_frame_0 =
-    QUICFrameFactory::create_stream_frame(reinterpret_cast<const uint8_t *>("abc"), 3, 0, 7);
+  uint8_t raw[]       = "abc";
+  ats_unique_buf data = ats_unique_malloc(sizeof(raw));
+  memcpy(data.get(), raw, sizeof(raw));
+
+  std::shared_ptr<QUICFrame> stream_frame_0 = QUICFrameFactory::create_stream_frame(std::move(data), sizeof(raw), 0, 7);
 
   sm.handle_frame(stream_frame_0);
   CHECK("succeed");
@@ -107,7 +114,6 @@ TEST_CASE("QUICStreamManager_total_offset_received", "[quic]")
   std::shared_ptr<QUICTransportParameters> remote_tp =
     std::make_shared<QUICTransportParametersInClientHello>(static_cast<QUICVersion>(0), static_cast<QUICVersion>(0));
   sm.init_flow_control_params(local_tp, remote_tp);
-  uint8_t data[1024] = {0};
 
   // Create a stream with STREAM_BLOCKED (== noop)
   std::shared_ptr<QUICFrame> stream_blocked_frame_0 = QUICFrameFactory::create_stream_blocked_frame(0);
@@ -118,12 +124,12 @@ TEST_CASE("QUICStreamManager_total_offset_received", "[quic]")
   CHECK(sm.total_offset_received() == 0);
 
   // Stream 0 shoud be out of flow control
-  std::shared_ptr<QUICFrame> stream_frame_0 = QUICFrameFactory::create_stream_frame(data, 1024, 0, 0);
+  std::shared_ptr<QUICFrame> stream_frame_0 = QUICFrameFactory::create_stream_frame(ats_unique_malloc(1024), 1024, 0, 0);
   sm.handle_frame(stream_frame_0);
   CHECK(sm.total_offset_received() == 0);
 
   // total_offset should be a integer in unit of 1024 octets
-  std::shared_ptr<QUICFrame> stream_frame_1 = QUICFrameFactory::create_stream_frame(data, 1024, 1, 0);
+  std::shared_ptr<QUICFrame> stream_frame_1 = QUICFrameFactory::create_stream_frame(ats_unique_malloc(1024), 1024, 1, 0);
   sm.handle_frame(stream_frame_1);
   CHECK(sm.total_offset_received() == 1);
 }
@@ -144,23 +150,27 @@ TEST_CASE("QUICStreamManager_total_offset_sent", "[quic]")
   uint8_t data[1024] = {0};
 
   // Create a stream with STREAM_BLOCKED (== noop)
-  std::shared_ptr<QUICFrame> stream_frame_0_r =
-    QUICFrameFactory::create_stream_frame(reinterpret_cast<const uint8_t *>("abc"), 3, 0, 0);
-  std::shared_ptr<QUICFrame> stream_frame_1_r =
-    QUICFrameFactory::create_stream_frame(reinterpret_cast<const uint8_t *>("abc"), 3, 1, 0);
+  uint8_t raw[]        = "abc";
+  ats_unique_buf data1 = ats_unique_malloc(sizeof(raw));
+  memcpy(data1.get(), raw, sizeof(raw));
+  ats_unique_buf data2 = ats_unique_malloc(sizeof(raw));
+  memcpy(data2.get(), raw, sizeof(raw));
+
+  std::shared_ptr<QUICFrame> stream_frame_0_r = QUICFrameFactory::create_stream_frame(std::move(data1), sizeof(raw), 0, 0);
+  std::shared_ptr<QUICFrame> stream_frame_1_r = QUICFrameFactory::create_stream_frame(std::move(data2), sizeof(raw), 1, 0);
   sm.handle_frame(stream_frame_0_r);
   sm.handle_frame(stream_frame_1_r);
   CHECK(sm.stream_count() == 2);
   CHECK(sm.total_offset_sent() == 0);
 
   // Stream 0 shoud be out of flow control
-  QUICFrameUPtr stream_frame_0 = QUICFrameFactory::create_stream_frame(data, 1024, 0, 0);
+  QUICFrameUPtr stream_frame_0 = QUICFrameFactory::create_stream_frame(ats_unique_malloc(1024), 1024, 0, 0);
   mock_app.send(data, 1024, 0);
   sleep(2);
   CHECK(sm.total_offset_sent() == 0);
 
   // total_offset should be a integer in unit of octets
-  QUICFrameUPtr stream_frame_1 = QUICFrameFactory::create_stream_frame(data, 1024, 1, 0);
+  QUICFrameUPtr stream_frame_1 = QUICFrameFactory::create_stream_frame(ats_unique_malloc(1024), 1024, 1, 0);
   mock_app.send(data, 1024, 1);
   sm.add_total_offset_sent(1024);
   sleep(2);
diff --git a/iocore/net/quic/test/test_QUICStreamState.cc b/iocore/net/quic/test/test_QUICStreamState.cc
index ec30434..c9a37a4 100644
--- a/iocore/net/quic/test/test_QUICStreamState.cc
+++ b/iocore/net/quic/test/test_QUICStreamState.cc
@@ -30,7 +30,11 @@
 
 TEST_CASE("QUICStreamState_Idle", "[quic]")
 {
-  auto stream_frame          = QUICFrameFactory::create_stream_frame(reinterpret_cast<const uint8_t *>("foo"), 4, 1, 0);
+  uint8_t raw[]       = "foo";
+  ats_unique_buf data = ats_unique_malloc(sizeof(raw));
+  memcpy(data.get(), raw, sizeof(raw));
+
+  auto stream_frame          = QUICFrameFactory::create_stream_frame(std::move(data), sizeof(raw), 1, 0);
   auto rst_stream_frame      = QUICFrameFactory::create_rst_stream_frame(0, QUICErrorCode::NO_ERROR, 0);
   auto max_stream_data_frame = QUICFrameFactory::create_max_stream_data_frame(0, 0);
   auto stream_blocked_frame  = QUICFrameFactory::create_stream_blocked_frame(0);
@@ -68,9 +72,17 @@ TEST_CASE("QUICStreamState_Idle", "[quic]")
 
 TEST_CASE("QUICStreamState_Open", "[quic]")
 {
-  auto stream_frame          = QUICFrameFactory::create_stream_frame(reinterpret_cast<const uint8_t *>("foo"), 4, 1, 0);
-  auto stream_frame_with_fin = QUICFrameFactory::create_stream_frame(reinterpret_cast<const uint8_t *>("bar"), 4, 1, 0, true);
-  auto rst_stream_frame      = QUICFrameFactory::create_rst_stream_frame(0, QUICErrorCode::NO_ERROR, 0);
+  uint8_t raw1[]       = "foo";
+  ats_unique_buf data1 = ats_unique_malloc(sizeof(raw1));
+  memcpy(data1.get(), raw1, sizeof(raw1));
+  auto stream_frame = QUICFrameFactory::create_stream_frame(std::move(data1), sizeof(raw1), 1, 0);
+
+  uint8_t raw2[]       = "bar";
+  ats_unique_buf data2 = ats_unique_malloc(sizeof(raw2));
+  memcpy(data2.get(), raw2, sizeof(raw2));
+  auto stream_frame_with_fin = QUICFrameFactory::create_stream_frame(std::move(data2), sizeof(raw2), 1, 0, true);
+
+  auto rst_stream_frame = QUICFrameFactory::create_rst_stream_frame(0, QUICErrorCode::NO_ERROR, 0);
 
   // Case1. Send FIN in a STREAM
   QUICStreamState ss1;
@@ -103,9 +115,17 @@ TEST_CASE("QUICStreamState_Open", "[quic]")
 
 TEST_CASE("QUICStreamState_Half_Closed_Remote", "[quic]")
 {
-  auto stream_frame          = QUICFrameFactory::create_stream_frame(reinterpret_cast<const uint8_t *>("foo"), 4, 1, 0);
-  auto stream_frame_with_fin = QUICFrameFactory::create_stream_frame(reinterpret_cast<const uint8_t *>("bar"), 4, 1, 0, true);
-  auto rst_stream_frame      = QUICFrameFactory::create_rst_stream_frame(0, QUICErrorCode::NO_ERROR, 0);
+  uint8_t raw1[]       = "foo";
+  ats_unique_buf data1 = ats_unique_malloc(sizeof(raw1));
+  memcpy(data1.get(), raw1, sizeof(raw1));
+  auto stream_frame = QUICFrameFactory::create_stream_frame(std::move(data1), sizeof(raw1), 1, 0);
+
+  uint8_t raw2[]       = "bar";
+  ats_unique_buf data2 = ats_unique_malloc(sizeof(raw2));
+  memcpy(data2.get(), raw2, sizeof(raw2));
+  auto stream_frame_with_fin = QUICFrameFactory::create_stream_frame(std::move(data2), sizeof(raw2), 1, 0, true);
+
+  auto rst_stream_frame = QUICFrameFactory::create_rst_stream_frame(0, QUICErrorCode::NO_ERROR, 0);
 
   // Case1. Send FIN in a STREAM
   QUICStreamState ss1;
@@ -124,9 +144,17 @@ TEST_CASE("QUICStreamState_Half_Closed_Remote", "[quic]")
 
 TEST_CASE("QUICStreamState_Half_Closed_Local", "[quic]")
 {
-  auto stream_frame          = QUICFrameFactory::create_stream_frame(reinterpret_cast<const uint8_t *>("foo"), 4, 1, 0);
-  auto stream_frame_with_fin = QUICFrameFactory::create_stream_frame(reinterpret_cast<const uint8_t *>("bar"), 4, 1, 0, true);
-  auto rst_stream_frame      = QUICFrameFactory::create_rst_stream_frame(0, QUICErrorCode::NO_ERROR, 0);
+  uint8_t raw1[]       = "foo";
+  ats_unique_buf data1 = ats_unique_malloc(sizeof(raw1));
+  memcpy(data1.get(), raw1, sizeof(raw1));
+  auto stream_frame = QUICFrameFactory::create_stream_frame(std::move(data1), sizeof(raw1), 1, 0);
+
+  uint8_t raw2[]       = "bar";
+  ats_unique_buf data2 = ats_unique_malloc(sizeof(raw2));
+  memcpy(data2.get(), raw2, sizeof(raw2));
+  auto stream_frame_with_fin = QUICFrameFactory::create_stream_frame(std::move(data2), sizeof(raw2), 1, 0, true);
+
+  auto rst_stream_frame = QUICFrameFactory::create_rst_stream_frame(0, QUICErrorCode::NO_ERROR, 0);
 
   // Case1. Recv FIN in a STREAM
   QUICStreamState ss1;
diff --git a/proxy/hq/HQClientTransaction.cc b/proxy/hq/HQClientTransaction.cc
index 8c168d5..7cedc18 100644
--- a/proxy/hq/HQClientTransaction.cc
+++ b/proxy/hq/HQClientTransaction.cc
@@ -36,6 +36,7 @@ dump_io_buffer(IOBufferReader *reader)
   int64_t msg_len              = 1024;
   int64_t read_len             = debug_reader->read(msg, msg_len);
   Debug("hq_trans", "len=%" PRId64 "\n%s\n", read_len, msg);
+  debug_reader->dealloc();
 }
 
 HQClientTransaction::HQClientTransaction(HQClientSession *session, QUICStreamIO *stream_io) : super(), _stream_io(stream_io)

-- 
To stop receiving notification emails like this one, please contact
['"commits@trafficserver.apache.org" <co...@trafficserver.apache.org>'].