Deprecate gfe2_reloadable_flag_quic_break_session_stream_close_loop.

Also inline CloseStreamInner.

PiperOrigin-RevId: 314176674
Change-Id: Iad90f3607bac0204b047f1c455d824b8ddfc84b8
diff --git a/quic/core/http/end_to_end_test.cc b/quic/core/http/end_to_end_test.cc
index a455238..f527271 100644
--- a/quic/core/http/end_to_end_test.cc
+++ b/quic/core/http/end_to_end_test.cc
@@ -1947,11 +1947,7 @@
   // Transmit the cancel, and ensure the connection is torn down properly.
   SetPacketLossPercentage(0);
   QuicStreamId stream_id = GetNthClientInitiatedBidirectionalId(0);
-  if (session->break_close_loop()) {
-    session->ResetStream(stream_id, QUIC_STREAM_CANCELLED, 0);
-  } else {
-    session->SendRstStream(stream_id, QUIC_STREAM_CANCELLED, 0);
-  }
+  session->ResetStream(stream_id, QUIC_STREAM_CANCELLED, 0);
 
   // WaitForEvents waits 50ms and returns true if there are outstanding
   // requests.
diff --git a/quic/core/http/quic_client_promised_info_test.cc b/quic/core/http/quic_client_promised_info_test.cc
index a599626..0836f9a 100644
--- a/quic/core/http/quic_client_promised_info_test.cc
+++ b/quic/core/http/quic_client_promised_info_test.cc
@@ -227,9 +227,6 @@
   EXPECT_CALL(*connection_, SendControlFrame(_));
   EXPECT_CALL(*connection_,
               OnStreamReset(promise_id_, QUIC_PROMISE_VARY_MISMATCH));
-  if (!session_.break_close_loop()) {
-    EXPECT_CALL(session_, CloseStream(promise_id_));
-  }
 
   promised->HandleClientRequest(client_request_, &delegate);
 }
@@ -305,9 +302,6 @@
   session_.GetOrCreateStream(promise_id_);
 
   // Cancel the promised stream.
-  if (!session_.break_close_loop()) {
-    EXPECT_CALL(session_, CloseStream(promise_id_));
-  }
   EXPECT_CALL(*connection_, SendControlFrame(_));
   EXPECT_CALL(*connection_, OnStreamReset(promise_id_, QUIC_STREAM_CANCELLED));
   promised->Cancel();
@@ -331,17 +325,10 @@
   promise_stream->OnStreamHeaderList(false, headers.uncompressed_header_bytes(),
                                      headers);
 
-  if (!session_.break_close_loop()) {
-    EXPECT_CALL(session_, CloseStream(promise_id_));
-  }
   EXPECT_CALL(*connection_, SendControlFrame(_));
   EXPECT_CALL(*connection_,
               OnStreamReset(promise_id_, QUIC_STREAM_PEER_GOING_AWAY));
-  if (session_.break_close_loop()) {
-    session_.ResetStream(promise_id_, QUIC_STREAM_PEER_GOING_AWAY, 0);
-  } else {
-    session_.SendRstStream(promise_id_, QUIC_STREAM_PEER_GOING_AWAY, 0);
-  }
+  session_.ResetStream(promise_id_, QUIC_STREAM_PEER_GOING_AWAY, 0);
 
   // Now initiate rendezvous.
   TestPushPromiseDelegate delegate(/*match=*/true);
diff --git a/quic/core/http/quic_spdy_client_session_base.cc b/quic/core/http/quic_spdy_client_session_base.cc
index 63de57c..21ecfee 100644
--- a/quic/core/http/quic_spdy_client_session_base.cc
+++ b/quic/core/http/quic_spdy_client_session_base.cc
@@ -204,26 +204,20 @@
     QuicStreamId id,
     QuicRstStreamErrorCode error_code) {
   DCHECK(QuicUtils::IsServerInitiatedStreamId(transport_version(), id));
-  if (break_close_loop()) {
-    ResetStream(id, error_code, 0);
-  } else {
-    SendRstStream(id, error_code, 0);
-  }
+  ResetStream(id, error_code, 0);
   if (!IsOpenStream(id) && !IsClosedStream(id)) {
     MaybeIncreaseLargestPeerStreamId(id);
   }
 }
 
-void QuicSpdyClientSessionBase::CloseStreamInner(QuicStreamId stream_id,
-                                                 bool rst_sent) {
-  QuicSpdySession::CloseStreamInner(stream_id, rst_sent);
+void QuicSpdyClientSessionBase::CloseStream(QuicStreamId stream_id) {
+  QuicSpdySession::CloseStream(stream_id);
   if (!VersionUsesHttp3(transport_version())) {
     headers_stream()->MaybeReleaseSequencerBuffer();
   }
 }
 
 void QuicSpdyClientSessionBase::OnStreamClosed(QuicStreamId stream_id) {
-  DCHECK(break_close_loop());
   QuicSpdySession::OnStreamClosed(stream_id);
   if (!VersionUsesHttp3(transport_version())) {
     headers_stream()->MaybeReleaseSequencerBuffer();
diff --git a/quic/core/http/quic_spdy_client_session_base.h b/quic/core/http/quic_spdy_client_session_base.h
index a109d18..a3c9aac 100644
--- a/quic/core/http/quic_spdy_client_session_base.h
+++ b/quic/core/http/quic_spdy_client_session_base.h
@@ -102,7 +102,7 @@
   void ResetPromised(QuicStreamId id, QuicRstStreamErrorCode error_code);
 
   // Release headers stream's sequencer buffer if it's empty.
-  void CloseStreamInner(QuicStreamId stream_id, bool rst_sent) override;
+  void CloseStream(QuicStreamId stream_id) override;
 
   // Release headers stream's sequencer buffer if it's empty.
   void OnStreamClosed(QuicStreamId stream_id) override;
diff --git a/quic/core/http/quic_spdy_client_session_test.cc b/quic/core/http/quic_spdy_client_session_test.cc
index c2a910f..e68199c 100644
--- a/quic/core/http/quic_spdy_client_session_test.cc
+++ b/quic/core/http/quic_spdy_client_session_test.cc
@@ -343,11 +343,7 @@
       .Times(AtLeast(1))
       .WillRepeatedly(Invoke(&ClearControlFrame));
   EXPECT_CALL(*connection_, OnStreamReset(_, _)).Times(1);
-  if (session_->break_close_loop()) {
-    session_->ResetStream(stream_id, QUIC_STREAM_PEER_GOING_AWAY, 0);
-  } else {
-    session_->SendRstStream(stream_id, QUIC_STREAM_PEER_GOING_AWAY, 0);
-  }
+  session_->ResetStream(stream_id, QUIC_STREAM_PEER_GOING_AWAY, 0);
 
   // A new stream cannot be created as the reset stream still counts as an open
   // outgoing stream until closed by the server.
@@ -399,11 +395,7 @@
       .Times(AtLeast(1))
       .WillRepeatedly(Invoke(&ClearControlFrame));
   EXPECT_CALL(*connection_, OnStreamReset(_, _)).Times(1);
-  if (session_->break_close_loop()) {
-    session_->ResetStream(stream_id, QUIC_STREAM_PEER_GOING_AWAY, 0);
-  } else {
-    session_->SendRstStream(stream_id, QUIC_STREAM_PEER_GOING_AWAY, 0);
-  }
+  session_->ResetStream(stream_id, QUIC_STREAM_PEER_GOING_AWAY, 0);
 
   // The stream receives trailers with final byte offset, but the header value
   // is non-numeric and should be treated as malformed.
@@ -853,12 +845,7 @@
   EXPECT_CALL(*connection_, SendControlFrame(_));
   EXPECT_CALL(*connection_,
               OnStreamReset(promised_stream_id_, QUIC_STREAM_PEER_GOING_AWAY));
-  if (session_->break_close_loop()) {
-    session_->ResetStream(promised_stream_id_, QUIC_STREAM_PEER_GOING_AWAY, 0);
-  } else {
-    session_->SendRstStream(promised_stream_id_, QUIC_STREAM_PEER_GOING_AWAY,
-                            0);
-  }
+  session_->ResetStream(promised_stream_id_, QUIC_STREAM_PEER_GOING_AWAY, 0);
   QuicClientPromisedInfo* promised =
       session_->GetPromisedById(promised_stream_id_);
   EXPECT_NE(promised, nullptr);
diff --git a/quic/core/quic_session.cc b/quic/core/quic_session.cc
index 6e0d894..7fb52dd 100644
--- a/quic/core/quic_session.cc
+++ b/quic/core/quic_session.cc
@@ -100,9 +100,7 @@
       supported_versions_(supported_versions),
       use_http2_priority_write_scheduler_(false),
       is_configured_(false),
-      enable_round_robin_scheduling_(false),
-      break_close_loop_(
-          GetQuicReloadableFlag(quic_break_session_stream_close_loop)) {
+      enable_round_robin_scheduling_(false) {
   closed_streams_clean_up_alarm_ =
       QuicWrapUnique<QuicAlarm>(connection_->alarm_factory()->CreateAlarm(
           new ClosedStreamsCleanUpDelegate(this)));
@@ -115,9 +113,6 @@
         config_.GetMaxUnidirectionalStreamsToSend() +
         num_expected_unidirectional_static_streams);
   }
-  if (break_close_loop_) {
-    QUIC_RELOADABLE_FLAG_COUNT(quic_break_session_stream_close_loop);
-  }
 }
 
 void QuicSession::Initialize() {
@@ -776,22 +771,11 @@
 
     connection_->OnStreamReset(id, error);
   }
-
-  if (break_close_loop_) {
-    return;
-  }
-
-  if (error != QUIC_STREAM_NO_ERROR && QuicContainsKey(zombie_streams_, id)) {
-    OnStreamDoneWaitingForAcks(id);
-    return;
-  }
-  CloseStreamInner(id, true);
 }
 
 void QuicSession::ResetStream(QuicStreamId id,
                               QuicRstStreamErrorCode error,
                               QuicStreamOffset bytes_written) {
-  DCHECK(break_close_loop_);
   QuicStream* stream = GetStream(id);
   if (stream != nullptr && stream->is_static()) {
     connection()->CloseConnection(
@@ -867,24 +851,11 @@
 }
 
 void QuicSession::CloseStream(QuicStreamId stream_id) {
-  CloseStreamInner(stream_id, false);
-}
-
-void QuicSession::InsertLocallyClosedStreamsHighestOffset(
-    const QuicStreamId id,
-    QuicStreamOffset offset) {
-  locally_closed_streams_highest_offset_[id] = offset;
-  if (IsIncomingStream(id)) {
-    ++num_locally_closed_incoming_streams_highest_offset_;
-  }
-}
-
-void QuicSession::CloseStreamInner(QuicStreamId stream_id, bool rst_sent) {
   QUIC_DVLOG(1) << ENDPOINT << "Closing stream " << stream_id;
 
   StreamMap::iterator it = stream_map_.find(stream_id);
   if (it == stream_map_.end()) {
-    // When CloseStreamInner has been called recursively (via
+    // When CloseStream has been called recursively (via
     // QuicStream::OnClose), the stream will already have been deleted
     // from stream_map_, so return immediately.
     QUIC_DVLOG(1) << ENDPOINT << "Stream is already closed: " << stream_id;
@@ -900,81 +871,21 @@
         ConnectionCloseBehavior::SEND_CONNECTION_CLOSE_PACKET);
     return;
   }
-  if (break_close_loop_) {
-    stream->CloseReadSide();
-    stream->CloseWriteSide();
-    return;
-  }
-  StreamType type = stream->type();
+  stream->CloseReadSide();
+  stream->CloseWriteSide();
+}
 
-  // Tell the stream that a RST has been sent.
-  if (rst_sent) {
-    stream->set_rst_sent(true);
-  }
-
-  if (stream->IsWaitingForAcks()) {
-    zombie_streams_[stream->id()] = std::move(it->second);
-  } else {
-    // Clean up the stream since it is no longer waiting for acks.
-    streams_waiting_for_acks_.erase(stream->id());
-    closed_streams_.push_back(std::move(it->second));
-    // Do not retransmit data of a closed stream.
-    streams_with_pending_retransmission_.erase(stream_id);
-    if (!closed_streams_clean_up_alarm_->IsSet()) {
-      closed_streams_clean_up_alarm_->Set(
-          connection_->clock()->ApproximateNow());
-    }
-  }
-
-  // If we haven't received a FIN or RST for this stream, we need to keep track
-  // of the how many bytes the stream's flow controller believes it has
-  // received, for accurate connection level flow control accounting.
-  const bool had_fin_or_rst = stream->HasReceivedFinalOffset();
-  if (!had_fin_or_rst) {
-    InsertLocallyClosedStreamsHighestOffset(
-        stream_id, stream->flow_controller()->highest_received_byte_offset());
-  }
-  const bool stream_was_draining = stream->was_draining();
-  QUIC_DVLOG_IF(1, stream_was_draining)
-      << ENDPOINT << "Stream " << stream_id << " was draining";
-  stream_map_.erase(it);
-  if (IsIncomingStream(stream_id)) {
-    --num_dynamic_incoming_streams_;
-  }
-  if (stream_was_draining) {
-    if (IsIncomingStream(stream_id)) {
-      QUIC_BUG_IF(num_draining_incoming_streams_ == 0);
-      --num_draining_incoming_streams_;
-    } else {
-      QUIC_BUG_IF(num_draining_outgoing_streams_ == 0);
-      --num_draining_outgoing_streams_;
-    }
-  } else if (VersionHasIetfQuicFrames(transport_version())) {
-    // Stream was not draining, but we did have a fin or rst, so we can now
-    // free the stream ID if version 99.
-    if (had_fin_or_rst && connection_->connected()) {
-      // Do not bother informing stream ID manager if connection is closed.
-      v99_streamid_manager_.OnStreamClosed(stream_id);
-    }
-  } else if (stream_id_manager_.handles_accounting() && had_fin_or_rst &&
-             connection_->connected()) {
-    stream_id_manager_.OnStreamClosed(
-        /*is_incoming=*/IsIncomingStream(stream_id));
-  }
-
-  stream->OnClose();
-
-  if (!stream_was_draining && !IsIncomingStream(stream_id) && had_fin_or_rst &&
-      !VersionHasIetfQuicFrames(transport_version())) {
-    // Streams that first became draining already called OnCanCreate...
-    // This covers the case where the stream went directly to being closed.
-    OnCanCreateNewOutgoingStream(type != BIDIRECTIONAL);
+void QuicSession::InsertLocallyClosedStreamsHighestOffset(
+    const QuicStreamId id,
+    QuicStreamOffset offset) {
+  locally_closed_streams_highest_offset_[id] = offset;
+  if (IsIncomingStream(id)) {
+    ++num_locally_closed_incoming_streams_highest_offset_;
   }
 }
 
 void QuicSession::OnStreamClosed(QuicStreamId stream_id) {
   QUIC_DVLOG(1) << ENDPOINT << "Closing stream: " << stream_id;
-  DCHECK(break_close_loop_);
   StreamMap::iterator it = stream_map_.find(stream_id);
   if (it == stream_map_.end()) {
     QUIC_DVLOG(1) << ENDPOINT << "Stream is already closed: " << stream_id;
@@ -1780,11 +1691,7 @@
     if (!stream_id_manager_.CanOpenIncomingStream(
             GetNumOpenIncomingStreams())) {
       // Refuse to open the stream.
-      if (break_close_loop_) {
-        ResetStream(stream_id, QUIC_REFUSED_STREAM, 0);
-      } else {
-        SendRstStream(stream_id, QUIC_REFUSED_STREAM, 0);
-      }
+      ResetStream(stream_id, QUIC_REFUSED_STREAM, 0);
       return nullptr;
     }
   }
diff --git a/quic/core/quic_session.h b/quic/core/quic_session.h
index 47ccce6..d7400d5 100644
--- a/quic/core/quic_session.h
+++ b/quic/core/quic_session.h
@@ -491,8 +491,6 @@
   // uses TLS handshake.
   virtual void OnAlpnSelected(quiche::QuicheStringPiece alpn);
 
-  bool break_close_loop() const { return break_close_loop_; }
-
   // Called on clients by the crypto handshaker to provide application state
   // necessary for sending application data in 0-RTT. The state provided here is
   // the same state that was provided to the crypto handshaker in
@@ -557,11 +555,6 @@
   // as a transport parameter, or in the most recent MAX_STREAMS frame.
   QuicStreamCount GetAdvertisedMaxIncomingBidirectionalStreams() const;
 
-  // Performs the work required to close |stream_id|.  If |rst_sent| then a
-  // Reset Stream frame has already been sent for this stream.
-  // TODO(fayang): Remove CloseStreamInner.
-  virtual void CloseStreamInner(QuicStreamId stream_id, bool rst_sent);
-
   // When a stream is closed locally, it may not yet know how many bytes the
   // peer sent on that stream.
   // When this data arrives (via stream frame w. FIN, trailing headers, or RST)
@@ -856,9 +849,6 @@
 
   // If true, enables round robin scheduling.
   bool enable_round_robin_scheduling_;
-
-  // Latched value of quic_break_session_stream_close_loop.
-  const bool break_close_loop_;
 };
 
 }  // namespace quic
diff --git a/quic/core/quic_session_test.cc b/quic/core/quic_session_test.cc
index d57180b..ad10f43 100644
--- a/quic/core/quic_session_test.cc
+++ b/quic/core/quic_session_test.cc
@@ -2444,14 +2444,9 @@
   session_.OnFrameLost(QuicFrame(frame));
   // Retransmit stream data causes connection close. Stream has not sent fin
   // yet, so an RST is sent.
-  if (session_.break_close_loop()) {
-    EXPECT_CALL(*stream, OnCanWrite()).WillOnce(Invoke([this, stream]() {
-      session_.CloseStream(stream->id());
-    }));
-  } else {
-    EXPECT_CALL(*stream, OnCanWrite())
-        .WillOnce(Invoke(stream, &QuicStream::OnClose));
-  }
+  EXPECT_CALL(*stream, OnCanWrite()).WillOnce(Invoke([this, stream]() {
+    session_.CloseStream(stream->id());
+  }));
   if (VersionHasIetfQuicFrames(transport_version())) {
     // Once for the RST_STREAM, once for the STOP_SENDING
     EXPECT_CALL(*connection_, SendControlFrame(_))
diff --git a/quic/core/quic_stream.cc b/quic/core/quic_stream.cc
index 6e5a5e9..6450b6d 100644
--- a/quic/core/quic_stream.cc
+++ b/quic/core/quic_stream.cc
@@ -580,14 +580,12 @@
   stream_error_ = error;
   session()->SendRstStream(id(), error, stream_bytes_written());
   rst_sent_ = true;
-  if (session_->break_close_loop()) {
-    if (read_side_closed_ && write_side_closed_ && !IsWaitingForAcks()) {
-      session()->OnStreamDoneWaitingForAcks(id_);
-      return;
-    }
-    CloseReadSide();
-    CloseWriteSide();
+  if (read_side_closed_ && write_side_closed_ && !IsWaitingForAcks()) {
+    session()->OnStreamDoneWaitingForAcks(id_);
+    return;
   }
+  CloseReadSide();
+  CloseWriteSide();
 }
 
 void QuicStream::OnUnrecoverableError(QuicErrorCode error,
@@ -779,12 +777,8 @@
 
   if (write_side_closed_) {
     QUIC_DVLOG(1) << ENDPOINT << "Closing stream " << id();
-    if (session_->break_close_loop()) {
-      session_->OnStreamClosed(id());
-      OnClose();
-    } else {
-      session_->CloseStream(id());
-    }
+    session_->OnStreamClosed(id());
+    OnClose();
   }
 }
 
@@ -797,12 +791,8 @@
   write_side_closed_ = true;
   if (read_side_closed_) {
     QUIC_DVLOG(1) << ENDPOINT << "Closing stream " << id();
-    if (session_->break_close_loop()) {
-      session_->OnStreamClosed(id());
-      OnClose();
-    } else {
-      session_->CloseStream(id());
-    }
+    session_->OnStreamClosed(id());
+    OnClose();
   }
 }
 
@@ -825,12 +815,7 @@
 }
 
 void QuicStream::OnClose() {
-  if (session()->break_close_loop()) {
-    DCHECK(read_side_closed_ && write_side_closed_);
-  } else {
-    CloseReadSide();
-    CloseWriteSide();
-  }
+  DCHECK(read_side_closed_ && write_side_closed_);
 
   if (!fin_sent_ && !rst_sent_) {
     // For flow control accounting, tell the peer how many bytes have been
diff --git a/quic/core/quic_stream.h b/quic/core/quic_stream.h
index 930fd17..80b2fb0 100644
--- a/quic/core/quic_stream.h
+++ b/quic/core/quic_stream.h
@@ -165,14 +165,6 @@
   // stream to write any pending data.
   virtual void OnCanWrite();
 
-  // Called just before the object is destroyed.
-  // The object should not be accessed after OnClose is called.
-  // Sends a RST_STREAM with code QUIC_RST_ACKNOWLEDGEMENT if neither a FIN nor
-  // a RST_STREAM has been sent.
-  // TODO(fayang): move this to protected when deprecating
-  // quic_break_session_stream_close_loop.
-  virtual void OnClose();
-
   // Called by the session when the endpoint receives a RST_STREAM from the
   // peer.
   virtual void OnStreamReset(const QuicRstStreamFrame& frame);
@@ -381,6 +373,12 @@
       const QuicReferenceCountedPointer<QuicAckListenerInterface>&
       /*ack_listener*/) {}
 
+  // Called just before the object is destroyed.
+  // The object should not be accessed after OnClose is called.
+  // Sends a RST_STREAM with code QUIC_RST_ACKNOWLEDGEMENT if neither a FIN nor
+  // a RST_STREAM has been sent.
+  virtual void OnClose();
+
   // True if buffered data in send buffer is below buffered_data_threshold_.
   bool CanWriteNewData() const;
 
diff --git a/quic/core/quic_stream_test.cc b/quic/core/quic_stream_test.cc
index f30a3be..700931c 100644
--- a/quic/core/quic_stream_test.cc
+++ b/quic/core/quic_stream_test.cc
@@ -455,12 +455,8 @@
 
   // Now close the stream, and expect that we send a RST.
   EXPECT_CALL(*session_, SendRstStream(_, _, _));
-  if (session_->break_close_loop()) {
-    stream_->CloseReadSide();
-    stream_->CloseWriteSide();
-  } else {
-    stream_->OnClose();
-  }
+  stream_->CloseReadSide();
+  stream_->CloseWriteSide();
   EXPECT_FALSE(session_->HasUnackedStreamData());
   EXPECT_FALSE(fin_sent());
   EXPECT_TRUE(rst_sent());
@@ -487,12 +483,8 @@
   EXPECT_FALSE(rst_sent());
 
   // Now close the stream, and expect that we do not send a RST.
-  if (session_->break_close_loop()) {
-    stream_->CloseReadSide();
-    stream_->CloseWriteSide();
-  } else {
-    stream_->OnClose();
-  }
+  stream_->CloseReadSide();
+  stream_->CloseWriteSide();
   EXPECT_TRUE(fin_sent());
   EXPECT_FALSE(rst_sent());
 }
@@ -516,12 +508,8 @@
 
   // Now close the stream (any further resets being sent would break the
   // expectation above).
-  if (session_->break_close_loop()) {
-    stream_->CloseReadSide();
-    stream_->CloseWriteSide();
-  } else {
-    stream_->OnClose();
-  }
+  stream_->CloseReadSide();
+  stream_->CloseWriteSide();
   EXPECT_FALSE(fin_sent());
   EXPECT_TRUE(rst_sent());
 }
@@ -657,12 +645,8 @@
               CloseConnection(QUIC_FLOW_CONTROL_RECEIVED_TOO_MUCH_DATA, _, _));
   stream_->OnStreamReset(rst_frame);
   EXPECT_TRUE(stream_->HasReceivedFinalOffset());
-  if (session_->break_close_loop()) {
-    stream_->CloseReadSide();
-    stream_->CloseWriteSide();
-  } else {
-    stream_->OnClose();
-  }
+  stream_->CloseReadSide();
+  stream_->CloseWriteSide();
 }
 
 TEST_P(QuicStreamTest, FinalByteOffsetFromZeroLengthStreamFrame) {
diff --git a/quic/quartc/quartc_stream_test.cc b/quic/quartc/quartc_stream_test.cc
index 1becc16..f21d1dc 100644
--- a/quic/quartc/quartc_stream_test.cc
+++ b/quic/quartc/quartc_stream_test.cc
@@ -398,12 +398,8 @@
 TEST_P(QuartcStreamTest, CloseStream) {
   CreateReliableQuicStream();
   EXPECT_FALSE(mock_stream_delegate_->closed());
-  if (GetQuicReloadableFlag(quic_break_session_stream_close_loop)) {
-    stream_->CloseWriteSide();
-    stream_->CloseReadSide();
-  } else {
-    stream_->OnClose();
-  }
+  stream_->CloseWriteSide();
+  stream_->CloseReadSide();
   EXPECT_TRUE(mock_stream_delegate_->closed());
 }
 
diff --git a/quic/test_tools/quic_test_client.cc b/quic/test_tools/quic_test_client.cc
index 2207d17..9c1c4f3 100644
--- a/quic/test_tools/quic_test_client.cc
+++ b/quic/test_tools/quic_test_client.cc
@@ -389,13 +389,8 @@
   QuicStreamId stream_id = GetNthClientInitiatedBidirectionalStreamId(
       session->transport_version(), 0);
   QuicStream* stream = session->GetOrCreateStream(stream_id);
-  if (session->break_close_loop()) {
-    session->ResetStream(stream_id, QUIC_STREAM_CANCELLED,
-                         stream->stream_bytes_written());
-  } else {
-    session->SendRstStream(stream_id, QUIC_STREAM_CANCELLED,
-                           stream->stream_bytes_written());
-  }
+  session->ResetStream(stream_id, QUIC_STREAM_CANCELLED,
+                       stream->stream_bytes_written());
   return ret;
 }