gfe-relnote: Deprecate gfe2_reloadable_flag_quic_treat_queued_packets_as_sent.

PiperOrigin-RevId: 283750614
Change-Id: I4303df56680d761b4140a4c22a5e605f08ff65a1
diff --git a/quic/core/quic_connection.cc b/quic/core/quic_connection.cc
index 9ecfa94..9349bf2 100644
--- a/quic/core/quic_connection.cc
+++ b/quic/core/quic_connection.cc
@@ -330,9 +330,6 @@
       bytes_received_before_address_validation_(0),
       bytes_sent_before_address_validation_(0),
       address_validated_(false),
-      treat_queued_packets_as_sent_(
-          GetQuicReloadableFlag(quic_treat_queued_packets_as_sent) ||
-          version().CanSendCoalescedPackets()),
       quic_version_negotiated_by_default_at_server_(
           GetQuicReloadableFlag(quic_version_negotiated_by_default_at_server)),
       use_handshake_delegate_(
@@ -403,15 +400,6 @@
 }
 
 void QuicConnection::ClearQueuedPackets() {
-  for (auto it = queued_packets_.begin(); it != queued_packets_.end(); ++it) {
-    // Delete the buffer before calling ClearSerializedPacket, which sets
-    // encrypted_buffer to nullptr.
-    DCHECK(!treat_queued_packets_as_sent_);
-    delete[] it->encrypted_buffer;
-    ClearSerializedPacket(&(*it));
-  }
-  queued_packets_.clear();
-
   buffered_packets_.clear();
 }
 
@@ -1686,28 +1674,6 @@
     ScopedPacketFlusher flusher(this);
     packet_creator_.FlushCurrentPacket();
   }
-
-  // Remove all queued packets which only contain data for the reset stream.
-  // TODO(fayang): consider removing this because it should be rarely executed.
-  auto packet_iterator = queued_packets_.begin();
-  while (packet_iterator != queued_packets_.end()) {
-    QuicFrames* retransmittable_frames =
-        &packet_iterator->retransmittable_frames;
-    if (retransmittable_frames->empty()) {
-      ++packet_iterator;
-      continue;
-    }
-    // NOTE THAT RemoveFramesForStream removes only STREAM frames
-    // for the specified stream.
-    RemoveFramesForStream(retransmittable_frames, id);
-    if (!retransmittable_frames->empty()) {
-      ++packet_iterator;
-      continue;
-    }
-    delete[] packet_iterator->encrypted_buffer;
-    ClearSerializedPacket(&(*packet_iterator));
-    packet_iterator = queued_packets_.erase(packet_iterator);
-  }
   // TODO(ianswett): Consider checking for 3 RTOs when the last stream is
   // cancelled as well.
 }
@@ -2033,37 +1999,9 @@
   }
 
   QUIC_CLIENT_HISTOGRAM_COUNTS("QuicSession.NumQueuedPacketsBeforeWrite",
-                               queued_packets_.size(), 1, 1000, 50, "");
-  while (!queued_packets_.empty()) {
-    DCHECK(!treat_queued_packets_as_sent_);
-    // WritePacket() can potentially clear all queued packets, so we need to
-    // save the first queued packet to a local variable before calling it.
-    SerializedPacket packet(std::move(queued_packets_.front()));
-    queued_packets_.pop_front();
-
-    const bool write_result = WritePacket(&packet);
-
-    if (connected_ && !write_result) {
-      // Write failed but connection is open, re-insert |packet| into the
-      // front of the queue, it will be retried later.
-      queued_packets_.emplace_front(std::move(packet));
-      break;
-    }
-
-    delete[] packet.encrypted_buffer;
-    ClearSerializedPacket(&packet);
-    if (!connected_) {
-      DCHECK(queued_packets_.empty()) << "Queued packets should have been "
-                                         "cleared while closing connection";
-      break;
-    }
-
-    // Continue to send the next packet in queue.
-  }
+                               buffered_packets_.size(), 1, 1000, 50, "");
 
   while (!buffered_packets_.empty()) {
-    DCHECK(treat_queued_packets_as_sent_);
-    QUIC_RELOADABLE_FLAG_COUNT_N(quic_treat_queued_packets_as_sent, 1, 3);
     if (HandleWriteBlocked()) {
       break;
     }
@@ -2221,7 +2159,7 @@
     QUIC_BUG << "Attempt to write packet:" << packet->packet_number
              << " after:" << sent_packet_manager_.GetLargestSentPacket();
     QUIC_CLIENT_HISTOGRAM_COUNTS("QuicSession.NumQueuedPacketsAtOutOfOrder",
-                                 queued_packets_.size(), 1, 1000, 50, "");
+                                 buffered_packets_.size(), 1, 1000, 50, "");
     CloseConnection(QUIC_INTERNAL_ERROR, "Packet written out of order.",
                     ConnectionCloseBehavior::SEND_CONNECTION_CLOSE_PACKET);
     return true;
@@ -2230,13 +2168,7 @@
       /*is_mtu_discovery=*/packet->encrypted_length > long_term_mtu_);
   // Termination packets are encrypted and saved, so don't exit early.
   const bool is_termination_packet = IsTerminationPacket(*packet);
-  if (!treat_queued_packets_as_sent_ && HandleWriteBlocked() &&
-      !is_termination_packet) {
-    return false;
-  }
-
   QuicPacketNumber packet_number = packet->packet_number;
-
   QuicPacketLength encrypted_length = packet->encrypted_length;
   // Termination packets are eventually owned by TimeWaitListManager.
   // Others are deleted at the end of this call.
@@ -2249,11 +2181,6 @@
     char* buffer_copy = CopyBuffer(*packet);
     termination_packets_->emplace_back(
         new QuicEncryptedPacket(buffer_copy, encrypted_length, true));
-    // This assures we won't try to write *forced* packets when blocked.
-    // Return true to stop processing.
-    if (!treat_queued_packets_as_sent_ && HandleWriteBlocked()) {
-      return true;
-    }
   }
 
   const bool looks_like_mtu_probe = packet->retransmittable_frames.empty() &&
@@ -2322,8 +2249,6 @@
       }
       break;
     case BUFFER:
-      DCHECK(treat_queued_packets_as_sent_);
-      QUIC_RELOADABLE_FLAG_COUNT_N(quic_treat_queued_packets_as_sent, 2, 3);
       QUIC_DVLOG(1) << ENDPOINT << "Adding packet: " << packet->packet_number
                     << " to buffered packets";
       buffered_packets_.emplace_back(*packet, self_address(), peer_address());
@@ -2358,14 +2283,9 @@
     // duplicate packet being sent.  The helper must call OnCanWrite
     // when the write completes, and OnWriteError if an error occurs.
     if (result.status != WRITE_STATUS_BLOCKED_DATA_BUFFERED) {
-      if (treat_queued_packets_as_sent_) {
-        QUIC_RELOADABLE_FLAG_COUNT_N(quic_treat_queued_packets_as_sent, 3, 3);
-        QUIC_DVLOG(1) << ENDPOINT << "Adding packet: " << packet->packet_number
-                      << " to buffered packets";
-        buffered_packets_.emplace_back(*packet, self_address(), peer_address());
-      } else {
-        return false;
-      }
+      QUIC_DVLOG(1) << ENDPOINT << "Adding packet: " << packet->packet_number
+                    << " to buffered packets";
+      buffered_packets_.emplace_back(*packet, self_address(), peer_address());
     }
   }
 
@@ -2626,17 +2546,7 @@
     QUIC_BUG << "packet.encrypted_buffer == nullptr in to SendOrQueuePacket";
     return;
   }
-  // If there are already queued packets, queue this one immediately to ensure
-  // it's written in sequence number order.
-  if (!queued_packets_.empty() || !WritePacket(packet)) {
-    if (!treat_queued_packets_as_sent_) {
-      // Take ownership of the underlying encrypted packet.
-      packet->encrypted_buffer = CopyBuffer(*packet);
-      queued_packets_.push_back(*packet);
-      packet->retransmittable_frames.clear();
-    }
-  }
-
+  WritePacket(packet);
   ClearSerializedPacket(packet);
 }
 
@@ -3089,15 +2999,14 @@
 }
 
 bool QuicConnection::HasQueuedData() const {
-  return pending_version_negotiation_packet_ || !queued_packets_.empty() ||
+  return pending_version_negotiation_packet_ ||
          packet_creator_.HasPendingFrames() || !buffered_packets_.empty();
 }
 
 bool QuicConnection::CanWriteStreamData() {
   // Don't write stream data if there are negotiation or queued data packets
   // to send. Otherwise, continue and bundle as many frames as possible.
-  if (pending_version_negotiation_packet_ || !queued_packets_.empty() ||
-      !buffered_packets_.empty()) {
+  if (pending_version_negotiation_packet_ || !buffered_packets_.empty()) {
     return false;
   }
 
@@ -3700,9 +3609,8 @@
     return;
   }
 
-  bool application_limited = queued_packets_.empty() &&
-                             buffered_packets_.empty() &&
-                             !visitor_->WillingAndAbleToWrite();
+  bool application_limited =
+      buffered_packets_.empty() && !visitor_->WillingAndAbleToWrite();
 
   if (!application_limited) {
     return;
@@ -4087,9 +3995,6 @@
 
 SerializedPacketFate QuicConnection::DeterminePacketFate(
     bool is_mtu_discovery) {
-  if (!treat_queued_packets_as_sent_) {
-    return SEND_TO_WRITER;
-  }
   if (version().CanSendCoalescedPackets() &&
       sent_packet_manager_.handshake_state() <
           QuicSentPacketManager::HANDSHAKE_CONFIRMED &&
diff --git a/quic/core/quic_connection.h b/quic/core/quic_connection.h
index 2fcc604..abd836f 100644
--- a/quic/core/quic_connection.h
+++ b/quic/core/quic_connection.h
@@ -599,12 +599,7 @@
   }
 
   // Testing only.
-  size_t NumQueuedPackets() const {
-    if (treat_queued_packets_as_sent_) {
-      return buffered_packets_.size();
-    }
-    return queued_packets_.size();
-  }
+  size_t NumQueuedPackets() const { return buffered_packets_.size(); }
 
   // Returns true if the underlying UDP socket is writable, there is
   // no queued data and the connection is not congestion-control
@@ -893,10 +888,6 @@
   // or the one sent after an IETF Retry.
   void InstallInitialCrypters(QuicConnectionId connection_id);
 
-  bool treat_queued_packets_as_sent() const {
-    return treat_queued_packets_as_sent_;
-  }
-
   // Called when version is considered negotiated.
   void OnSuccessfulVersionNegotiation();
 
@@ -1285,15 +1276,6 @@
   bool send_ietf_version_negotiation_packet_;
   bool send_version_negotiation_packet_with_prefixed_lengths_;
 
-  // When packets could not be sent because the socket was not writable,
-  // they are added to this list.  All corresponding frames are in
-  // unacked_packets_ if they are to be retransmitted.  Packets encrypted_buffer
-  // fields are owned by the QueuedPacketList, in order to ensure they outlast
-  // the original scope of the SerializedPacket.
-  // TODO(fayang): Remove this when deprecating
-  // quic_treat_queued_packets_as_sent.
-  QueuedPacketList queued_packets_;
-
   // Contains the connection close packets if the connection has been closed.
   std::unique_ptr<std::vector<std::unique_ptr<QuicEncryptedPacket>>>
       termination_packets_;
@@ -1510,8 +1492,7 @@
   // Used to store content of packets which cannot be sent because of write
   // blocked. Packets' encrypted buffers are copied and owned by
   // buffered_packets_. From unacked_packet_map (and congestion control)'s
-  // perspective, those packets are considered sent. This is only used when
-  // treat_queued_packets_as_sent_ is true.
+  // perspective, those packets are considered sent.
   std::list<BufferedPacket> buffered_packets_;
 
   // Used to coalesce packets of different encryption level into the same UDP
@@ -1519,9 +1500,6 @@
   // packet gets acknowledged.
   QuicCoalescedPacket coalesced_packet_;
 
-  // Latched value of quic_treat_queued_packets_as_sent.
-  const bool treat_queued_packets_as_sent_;
-
   QuicConnectionMtuDiscoverer mtu_discoverer_;
 
   // Latched value of quic_version_negotiated_by_default_at_server.
diff --git a/quic/core/quic_connection_test.cc b/quic/core/quic_connection_test.cc
index 37d3d36..3babced 100644
--- a/quic/core/quic_connection_test.cc
+++ b/quic/core/quic_connection_test.cc
@@ -2034,21 +2034,8 @@
   writer_->SetWritable();
   connection_.SendConnectivityProbingPacket(writer_.get(),
                                             connection_.peer_address());
-  if (connection_.treat_queued_packets_as_sent()) {
-    EXPECT_CALL(visitor_, OnConnectionClosed(_, _)).Times(0);
-    connection_.OnCanWrite();
-    return;
-  }
-  EXPECT_CALL(visitor_,
-              OnConnectionClosed(_, ConnectionCloseSource::FROM_SELF));
-  EXPECT_QUIC_BUG(connection_.OnCanWrite(),
-                  "Attempt to write packet:1 after:2");
-  EXPECT_FALSE(connection_.connected());
-  TestConnectionCloseQuicErrorCode(QUIC_INTERNAL_ERROR);
-  const std::vector<QuicConnectionCloseFrame>& connection_close_frames =
-      writer_->connection_close_frames();
-  EXPECT_EQ("Packet written out of order.",
-            connection_close_frames[0].error_details);
+  EXPECT_CALL(visitor_, OnConnectionClosed(_, _)).Times(0);
+  connection_.OnCanWrite();
 }
 
 TEST_P(QuicConnectionTest, DiscardQueuedPacketsAfterConnectionClose) {
@@ -2070,12 +2057,8 @@
   connection_.SendStreamDataWithString(/*id=*/2, "foo", 0, NO_FIN);
 
   EXPECT_FALSE(connection_.connected());
-  if (connection_.treat_queued_packets_as_sent()) {
-    // No need to buffer packets.
-    EXPECT_EQ(0u, connection_.NumQueuedPackets());
-  } else {
-    EXPECT_EQ(1u, connection_.NumQueuedPackets());
-  }
+  // No need to buffer packets.
+  EXPECT_EQ(0u, connection_.NumQueuedPackets());
 
   EXPECT_EQ(0u, connection_.GetStats().packets_discarded);
   connection_.OnCanWrite();
@@ -3371,11 +3354,7 @@
   BlockOnNextWrite();
 
   QuicStreamId stream_id = 2;
-  if (connection_.treat_queued_packets_as_sent()) {
-    EXPECT_CALL(*send_algorithm_, OnPacketSent(_, _, _, _, _)).Times(1);
-  } else {
-    EXPECT_CALL(*send_algorithm_, OnPacketSent(_, _, _, _, _)).Times(0);
-  }
+  EXPECT_CALL(*send_algorithm_, OnPacketSent(_, _, _, _, _)).Times(1);
   connection_.SendStreamDataWithString(stream_id, "foo", 0, NO_FIN);
 
   // Now that there is a queued packet, reset the stream.
@@ -3383,13 +3362,7 @@
 
   // Unblock the connection and verify that the RST_STREAM is sent and the data
   // packet is sent.
-  if (connection_.treat_queued_packets_as_sent()) {
-    EXPECT_CALL(*send_algorithm_, OnPacketSent(_, _, _, _, _))
-        .Times(AtLeast(1));
-  } else {
-    EXPECT_CALL(*send_algorithm_, OnPacketSent(_, _, _, _, _))
-        .Times(AtLeast(2));
-  }
+  EXPECT_CALL(*send_algorithm_, OnPacketSent(_, _, _, _, _)).Times(AtLeast(1));
   writer_->SetWritable();
   connection_.OnCanWrite();
   size_t padding_frame_count = writer_->padding_frames().size();
@@ -3588,12 +3561,8 @@
   EXPECT_CALL(*loss_algorithm_, DetectLosses(_, _, _, _, _, _))
       .WillOnce(SetArgPointee<5>(lost_packets));
   EXPECT_CALL(*send_algorithm_, OnCongestionEvent(true, _, _, _, _));
-  if (connection_.treat_queued_packets_as_sent()) {
-    EXPECT_CALL(*send_algorithm_, OnPacketSent(_, _, QuicPacketNumber(4), _, _))
-        .Times(1);
-  } else {
-    EXPECT_CALL(*send_algorithm_, OnPacketSent(_, _, _, _, _)).Times(0);
-  }
+  EXPECT_CALL(*send_algorithm_, OnPacketSent(_, _, QuicPacketNumber(4), _, _))
+      .Times(1);
   ProcessAckPacket(&nack_two);
   EXPECT_EQ(1u, connection_.NumQueuedPackets());
 
@@ -3603,15 +3572,8 @@
   QuicAckFrame ack_all = InitAckFrame(3);
   ProcessAckPacket(&ack_all);
 
-  if (connection_.treat_queued_packets_as_sent()) {
-    EXPECT_CALL(*send_algorithm_, OnPacketSent(_, _, QuicPacketNumber(4), _, _))
-        .Times(0);
-  } else {
-    // Unblock the socket and attempt to send the queued packets. We will always
-    // send the retransmission.
-    EXPECT_CALL(*send_algorithm_, OnPacketSent(_, _, QuicPacketNumber(4), _, _))
-        .Times(1);
-  }
+  EXPECT_CALL(*send_algorithm_, OnPacketSent(_, _, QuicPacketNumber(4), _, _))
+      .Times(0);
 
   writer_->SetWritable();
   connection_.OnCanWrite();
@@ -3662,11 +3624,7 @@
   // Block the writer and ensure they're queued.
   BlockOnNextWrite();
   clock_.AdvanceTime(DefaultRetransmissionTime());
-  if (connection_.treat_queued_packets_as_sent()) {
-    EXPECT_CALL(*send_algorithm_, OnPacketSent(_, _, _, _, _)).Times(2);
-  } else {
-    EXPECT_CALL(*send_algorithm_, OnPacketSent(_, _, _, _, _)).Times(0);
-  }
+  EXPECT_CALL(*send_algorithm_, OnPacketSent(_, _, _, _, _)).Times(2);
   connection_.GetRetransmissionAlarm()->Fire();
   EXPECT_TRUE(connection_.HasQueuedData());
 
@@ -3674,12 +3632,7 @@
   writer_->SetWritable();
   clock_.AdvanceTime(QuicTime::Delta::FromMicroseconds(
       2 * DefaultRetransmissionTime().ToMicroseconds()));
-  if (connection_.treat_queued_packets_as_sent()) {
-    EXPECT_CALL(*send_algorithm_, OnPacketSent(_, _, _, _, _)).Times(2);
-  } else {
-    // 2 RTOs + 1 TLP, which is buggy.
-    EXPECT_CALL(*send_algorithm_, OnPacketSent(_, _, _, _, _)).Times(3);
-  }
+  EXPECT_CALL(*send_algorithm_, OnPacketSent(_, _, _, _, _)).Times(2);
   connection_.GetRetransmissionAlarm()->Fire();
   connection_.OnCanWrite();
 }
@@ -3699,36 +3652,20 @@
 TEST_P(QuicConnectionTest, WriteBlockedThenSent) {
   EXPECT_CALL(*send_algorithm_, OnPacketSent(_, _, _, _, _)).Times(0);
   BlockOnNextWrite();
-  if (connection_.treat_queued_packets_as_sent()) {
-    EXPECT_CALL(*send_algorithm_, OnPacketSent(_, _, _, _, _)).Times(1);
-  } else {
-    EXPECT_CALL(*send_algorithm_, OnPacketSent(_, _, _, _, _)).Times(0);
-  }
+  EXPECT_CALL(*send_algorithm_, OnPacketSent(_, _, _, _, _)).Times(1);
   connection_.SendStreamDataWithString(1, "foo", 0, NO_FIN);
-  if (connection_.treat_queued_packets_as_sent()) {
-    EXPECT_TRUE(connection_.GetRetransmissionAlarm()->IsSet());
-  } else {
-    EXPECT_FALSE(connection_.GetRetransmissionAlarm()->IsSet());
-  }
+  EXPECT_TRUE(connection_.GetRetransmissionAlarm()->IsSet());
   EXPECT_EQ(1u, connection_.NumQueuedPackets());
 
   // The second packet should also be queued, in order to ensure packets are
   // never sent out of order.
   writer_->SetWritable();
-  if (connection_.treat_queued_packets_as_sent()) {
-    EXPECT_CALL(*send_algorithm_, OnPacketSent(_, _, _, _, _)).Times(1);
-  } else {
-    EXPECT_CALL(*send_algorithm_, OnPacketSent(_, _, _, _, _)).Times(0);
-  }
+  EXPECT_CALL(*send_algorithm_, OnPacketSent(_, _, _, _, _)).Times(1);
   connection_.SendStreamDataWithString(1, "foo", 0, NO_FIN);
   EXPECT_EQ(2u, connection_.NumQueuedPackets());
 
   // Now both are sent in order when we unblock.
-  if (connection_.treat_queued_packets_as_sent()) {
-    EXPECT_CALL(*send_algorithm_, OnPacketSent(_, _, _, _, _)).Times(0);
-  } else {
-    EXPECT_CALL(*send_algorithm_, OnPacketSent(_, _, _, _, _)).Times(2);
-  }
+  EXPECT_CALL(*send_algorithm_, OnPacketSent(_, _, _, _, _)).Times(0);
   connection_.OnCanWrite();
   EXPECT_TRUE(connection_.GetRetransmissionAlarm()->IsSet());
   EXPECT_EQ(0u, connection_.NumQueuedPackets());
@@ -4299,11 +4236,7 @@
   // Simulate the retransmission alarm firing and the socket blocking.
   BlockOnNextWrite();
   clock_.AdvanceTime(DefaultRetransmissionTime());
-  if (connection_.treat_queued_packets_as_sent()) {
-    EXPECT_CALL(*send_algorithm_, OnPacketSent(_, _, _, _, _)).Times(1);
-  } else {
-    EXPECT_CALL(*send_algorithm_, OnPacketSent(_, _, _, _, _)).Times(0);
-  }
+  EXPECT_CALL(*send_algorithm_, OnPacketSent(_, _, _, _, _)).Times(1);
   connection_.GetRetransmissionAlarm()->Fire();
   EXPECT_EQ(1u, connection_.NumQueuedPackets());
 
@@ -4453,26 +4386,13 @@
 
 TEST_P(QuicConnectionTest, SetRTOAfterWritingToSocket) {
   BlockOnNextWrite();
-  if (connection_.treat_queued_packets_as_sent()) {
-    EXPECT_CALL(*send_algorithm_, OnPacketSent(_, _, _, _, _)).Times(1);
-  } else {
-    EXPECT_CALL(*send_algorithm_, OnPacketSent(_, _, _, _, _)).Times(0);
-  }
+  EXPECT_CALL(*send_algorithm_, OnPacketSent(_, _, _, _, _)).Times(1);
   connection_.SendStreamDataWithString(1, "foo", 0, NO_FIN);
-  if (connection_.treat_queued_packets_as_sent()) {
-    EXPECT_TRUE(connection_.GetRetransmissionAlarm()->IsSet());
-  } else {
-    // Make sure that RTO is not started when the packet is queued.
-    EXPECT_FALSE(connection_.GetRetransmissionAlarm()->IsSet());
-  }
+  EXPECT_TRUE(connection_.GetRetransmissionAlarm()->IsSet());
 
   // Test that RTO is started once we write to the socket.
   writer_->SetWritable();
-  if (connection_.treat_queued_packets_as_sent()) {
-    EXPECT_CALL(*send_algorithm_, OnPacketSent(_, _, _, _, _)).Times(0);
-  } else {
-    EXPECT_CALL(*send_algorithm_, OnPacketSent(_, _, _, _, _)).Times(1);
-  }
+  EXPECT_CALL(*send_algorithm_, OnPacketSent(_, _, _, _, _)).Times(0);
   connection_.OnCanWrite();
   EXPECT_TRUE(connection_.GetRetransmissionAlarm()->IsSet());
 }
@@ -4969,9 +4889,7 @@
   SendStreamDataToPeer(3, "!", packets_between_probes_base - 1, NO_FIN,
                        nullptr);
   ASSERT_TRUE(connection_.GetMtuDiscoveryAlarm()->IsSet());
-  if (GetQuicReloadableFlag(quic_treat_queued_packets_as_sent)) {
-    EXPECT_CALL(*send_algorithm_, OnPacketSent(_, _, _, _, _));
-  }
+  EXPECT_CALL(*send_algorithm_, OnPacketSent(_, _, _, _, _));
   BlockOnNextWrite();
   EXPECT_EQ(0u, connection_.NumQueuedPackets());
   connection_.GetMtuDiscoveryAlarm()->Fire();
@@ -7982,11 +7900,7 @@
   EXPECT_CALL(visitor_, WillingAndAbleToWrite()).WillRepeatedly(Return(true));
   BlockOnNextWrite();
 
-  if (connection_.treat_queued_packets_as_sent()) {
-    EXPECT_CALL(*send_algorithm_, OnPacketSent(_, _, _, _, _)).Times(1);
-  } else {
-    EXPECT_CALL(*send_algorithm_, OnPacketSent(_, _, _, _, _)).Times(0);
-  }
+  EXPECT_CALL(*send_algorithm_, OnPacketSent(_, _, _, _, _)).Times(1);
   connection_.SendStreamData3();
 
   // Now unblock the writer, become congestion control blocked,
@@ -7994,11 +7908,7 @@
   writer_->SetWritable();
   CongestionBlockWrites();
   EXPECT_CALL(visitor_, WillingAndAbleToWrite()).WillRepeatedly(Return(false));
-  if (connection_.treat_queued_packets_as_sent()) {
-    EXPECT_CALL(*send_algorithm_, OnPacketSent(_, _, _, _, _)).Times(0);
-  } else {
-    EXPECT_CALL(*send_algorithm_, OnPacketSent(_, _, _, _, _)).Times(1);
-  }
+  EXPECT_CALL(*send_algorithm_, OnPacketSent(_, _, _, _, _)).Times(0);
   EXPECT_CALL(*send_algorithm_, OnApplicationLimited(_)).Times(1);
   connection_.OnCanWrite();
 }
@@ -8682,32 +8592,15 @@
 
 TEST_P(QuicConnectionTest, WriteBlockedWithInvalidAck) {
   EXPECT_CALL(visitor_, OnSuccessfulVersionNegotiation(_));
-  if (connection_.treat_queued_packets_as_sent()) {
-    EXPECT_CALL(visitor_, OnConnectionClosed(_, _)).Times(0);
-  } else {
-    EXPECT_CALL(visitor_, OnConnectionClosed(_, _))
-        .WillOnce(Invoke(this, &QuicConnectionTest::SaveConnectionCloseFrame));
-  }
+  EXPECT_CALL(visitor_, OnConnectionClosed(_, _)).Times(0);
   BlockOnNextWrite();
-  if (connection_.treat_queued_packets_as_sent()) {
-    EXPECT_CALL(*send_algorithm_, OnPacketSent(_, _, _, _, _)).Times(1);
-  } else {
-    EXPECT_CALL(*send_algorithm_, OnPacketSent(_, _, _, _, _)).Times(0);
-  }
+  EXPECT_CALL(*send_algorithm_, OnPacketSent(_, _, _, _, _)).Times(1);
   connection_.SendStreamDataWithString(5, "foo", 0, FIN);
   // This causes connection to be closed because packet 1 has not been sent yet.
   QuicAckFrame frame = InitAckFrame(1);
-  if (connection_.treat_queued_packets_as_sent()) {
-    EXPECT_CALL(*send_algorithm_, OnCongestionEvent(_, _, _, _, _));
-  }
+  EXPECT_CALL(*send_algorithm_, OnCongestionEvent(_, _, _, _, _));
   ProcessAckPacket(1, &frame);
-  if (connection_.treat_queued_packets_as_sent()) {
-    EXPECT_EQ(0, connection_close_frame_count_);
-  } else {
-    EXPECT_EQ(1, connection_close_frame_count_);
-    EXPECT_THAT(saved_connection_close_frame_.quic_error_code,
-                IsError(QUIC_INVALID_ACK_DATA));
-  }
+  EXPECT_EQ(0, connection_close_frame_count_);
 }
 
 TEST_P(QuicConnectionTest, SendMessage) {
@@ -9361,11 +9254,7 @@
   EXPECT_CALL(visitor_, OnWriteBlocked()).Times(AtLeast(1));
   SendRstStream(stream_id, QUIC_ERROR_PROCESSING_STREAM, 3);
 
-  if (connection_.treat_queued_packets_as_sent()) {
-    EXPECT_CALL(*send_algorithm_, OnPacketSent(_, _, _, _, _)).Times(1);
-  } else {
-    EXPECT_CALL(*send_algorithm_, OnPacketSent(_, _, _, _, _)).Times(0);
-  }
+  EXPECT_CALL(*send_algorithm_, OnPacketSent(_, _, _, _, _)).Times(1);
   // Retransmission timer fires in TLP mode.
   connection_.GetRetransmissionAlarm()->Fire();
   // Verify one packets is forced flushed when writer is blocked.