Remove notion of static stream in QuicStreamIdManager.

QuicStreamId manager now doesn't keep track of the number of static stream QuicSession has. Instead, when an application of QuicSession sends out its config for max allowed incoming stream, it should add the number of expected static streams.

This allows creation of static streams later in a QuicSession.

gfe-relnote: v99 only, not protected.
PiperOrigin-RevId: 265728450
Change-Id: I3c9ed20f7d439fc9feb3b7f867e8b70c713c33ae
diff --git a/quic/core/http/end_to_end_test.cc b/quic/core/http/end_to_end_test.cc
index 14d2e7b..236fbdb 100644
--- a/quic/core/http/end_to_end_test.cc
+++ b/quic/core/http/end_to_end_test.cc
@@ -1709,10 +1709,7 @@
       VersionHasIetfQuicFrames(
           client_session->connection()->transport_version())
           ? QuicSessionPeer::v99_streamid_manager(client_session)
-                    ->max_allowed_outgoing_bidirectional_streams() -
-                QuicSessionPeer::v99_bidirectional_stream_id_manager(
-                    client_session)
-                    ->outgoing_static_stream_count()
+                ->max_allowed_outgoing_bidirectional_streams()
           : QuicSessionPeer::GetStreamIdManager(client_session)
                 ->max_open_outgoing_streams();
   size_t client_max_open_outgoing_unidirectional_streams =
@@ -1720,9 +1717,7 @@
           client_session->connection()->transport_version())
           ? QuicSessionPeer::v99_streamid_manager(client_session)
                     ->max_allowed_outgoing_unidirectional_streams() -
-                QuicSessionPeer::v99_unidirectional_stream_id_manager(
-                    client_session)
-                    ->outgoing_static_stream_count()
+                client_session->num_expected_unidirectional_static_streams()
           : QuicSessionPeer::GetStreamIdManager(client_session)
                 ->max_open_outgoing_streams();
   EXPECT_EQ(kServerMaxIncomingDynamicStreams,
@@ -1743,9 +1738,7 @@
           server_session->connection()->transport_version())
           ? QuicSessionPeer::v99_streamid_manager(server_session)
                     ->max_allowed_outgoing_unidirectional_streams() -
-                QuicSessionPeer::v99_unidirectional_stream_id_manager(
-                    server_session)
-                    ->outgoing_static_stream_count()
+                server_session->num_expected_unidirectional_static_streams()
           : QuicSessionPeer::GetStreamIdManager(server_session)
                 ->max_open_outgoing_streams();
   EXPECT_EQ(kClientMaxIncomingDynamicStreams,
diff --git a/quic/core/http/quic_spdy_client_session_test.cc b/quic/core/http/quic_spdy_client_session_test.cc
index e79170b..3601873 100644
--- a/quic/core/http/quic_spdy_client_session_test.cc
+++ b/quic/core/http/quic_spdy_client_session_test.cc
@@ -156,7 +156,8 @@
     QuicConfig config = DefaultQuicConfig();
     if (VersionHasIetfQuicFrames(connection_->transport_version())) {
       config.SetMaxIncomingUnidirectionalStreamsToSend(
-          server_max_incoming_streams);
+          server_max_incoming_streams +
+          session_->num_expected_unidirectional_static_streams());
       config.SetMaxIncomingBidirectionalStreamsToSend(
           server_max_incoming_streams);
     } else {
@@ -292,22 +293,14 @@
     // In V99 the stream limit increases only if we get a MAX_STREAMS
     // frame; pretend we got one.
 
-    // Note that this is to be the second stream created, hence
-    // the stream count is 3 (the two streams created as a part of
-    // the test plus the header stream, internally created).
-    QuicMaxStreamsFrame frame(
-        0,
-        QuicSessionPeer::v99_bidirectional_stream_id_manager(&*session_)
-                ->outgoing_static_stream_count() +
-            2,
-        /*unidirectional=*/false);
+    QuicMaxStreamsFrame frame(0, 2,
+                              /*unidirectional=*/false);
     session_->OnMaxStreamsFrame(frame);
   }
   stream = session_->CreateOutgoingBidirectionalStream();
   EXPECT_NE(nullptr, stream);
   if (VersionHasIetfQuicFrames(GetParam().transport_version)) {
-    // Ensure that we have/have had three open streams: two test streams and the
-    // header stream.
+    // Ensure that we have 2 total streams, 1 open and 1 closed.
     QuicStreamCount expected_stream_count = 2;
     EXPECT_EQ(expected_stream_count,
               QuicSessionPeer::v99_bidirectional_stream_id_manager(&*session_)
@@ -372,23 +365,15 @@
   // be able to create a new outgoing stream.
   EXPECT_EQ(0u, session_->GetNumOpenOutgoingStreams());
   if (VersionHasIetfQuicFrames(GetParam().transport_version)) {
-    // Note that this is to be the second stream created, hence
-    // the stream count is 3 (the two streams created as a part of
-    // the test plus the header stream, internally created).
-    QuicMaxStreamsFrame frame(
-        0,
-        QuicSessionPeer::v99_bidirectional_stream_id_manager(&*session_)
-                ->outgoing_static_stream_count() +
-            2,
-        /*unidirectional=*/false);
+    QuicMaxStreamsFrame frame(0, 2,
+                              /*unidirectional=*/false);
 
     session_->OnMaxStreamsFrame(frame);
   }
   stream = session_->CreateOutgoingBidirectionalStream();
   EXPECT_NE(nullptr, stream);
   if (VersionHasIetfQuicFrames(GetParam().transport_version)) {
-    // Ensure that we have/have had three open streams: two test streams and the
-    // header stream.
+    // Ensure that we have 2 open streams.
     QuicStreamCount expected_stream_count = 2;
     EXPECT_EQ(expected_stream_count,
               QuicSessionPeer::v99_bidirectional_stream_id_manager(&*session_)
diff --git a/quic/core/http/quic_spdy_session.cc b/quic/core/http/quic_spdy_session.cc
index 491f1de..101b33d 100644
--- a/quic/core/http/quic_spdy_session.cc
+++ b/quic/core/http/quic_spdy_session.cc
@@ -311,7 +311,12 @@
     QuicSession::Visitor* visitor,
     const QuicConfig& config,
     const ParsedQuicVersionVector& supported_versions)
-    : QuicSession(connection, visitor, config, supported_versions),
+    : QuicSession(connection,
+                  visitor,
+                  config,
+                  supported_versions,
+                  /*num_expected_unidirectional_static_streams = */
+                  VersionUsesQpack(connection->transport_version()) ? 3 : 0),
       send_control_stream_(nullptr),
       receive_control_stream_(nullptr),
       qpack_encoder_receive_stream_(nullptr),
@@ -377,9 +382,10 @@
               headers_stream->id());
 
     headers_stream_ = headers_stream.get();
-    RegisterStaticStream(std::move(headers_stream),
-                         /*stream_already_counted = */ false);
+    RegisterStaticStream(std::move(headers_stream));
   } else {
+    ConfigureMaxIncomingDynamicStreamsToSend(
+        config()->GetMaxIncomingUnidirectionalStreamsToSend());
     qpack_encoder_ = QuicMakeUnique<QpackEncoder>(this);
     qpack_decoder_ =
         QuicMakeUnique<QpackDecoder>(qpack_maximum_dynamic_table_capacity_,
@@ -915,8 +921,7 @@
       }
       auto receive_stream = QuicMakeUnique<QuicReceiveControlStream>(pending);
       receive_control_stream_ = receive_stream.get();
-      RegisterStaticStream(std::move(receive_stream),
-                           /*stream_already_counted = */ true);
+      RegisterStaticStream(std::move(receive_stream));
       receive_control_stream_->SetUnblocked();
       QUIC_DVLOG(1) << "Receive Control stream is created";
       return true;
@@ -934,8 +939,7 @@
       auto encoder_receive = QuicMakeUnique<QpackReceiveStream>(
           pending, qpack_decoder_->encoder_stream_receiver());
       qpack_encoder_receive_stream_ = encoder_receive.get();
-      RegisterStaticStream(std::move(encoder_receive),
-                           /*stream_already_counted = */ true);
+      RegisterStaticStream(std::move(encoder_receive));
       qpack_encoder_receive_stream_->SetUnblocked();
       QUIC_DVLOG(1) << "Receive QPACK Encoder stream is created";
       return true;
@@ -948,8 +952,7 @@
       auto decoder_receive = QuicMakeUnique<QpackReceiveStream>(
           pending, qpack_encoder_->decoder_stream_receiver());
       qpack_decoder_receive_stream_ = decoder_receive.get();
-      RegisterStaticStream(std::move(decoder_receive),
-                           /*stream_already_counted = */ true);
+      RegisterStaticStream(std::move(decoder_receive));
       qpack_decoder_receive_stream_->SetUnblocked();
       QUIC_DVLOG(1) << "Receive Qpack Decoder stream is created";
       return true;
@@ -968,8 +971,7 @@
         GetNextOutgoingUnidirectionalStreamId(), this,
         max_inbound_header_list_size_);
     send_control_stream_ = send_control.get();
-    RegisterStaticStream(std::move(send_control),
-                         /*stream_already_counted = */ false);
+    RegisterStaticStream(std::move(send_control));
   }
 
   if (!qpack_decoder_send_stream_ &&
@@ -977,8 +979,7 @@
     auto decoder_send = QuicMakeUnique<QpackSendStream>(
         GetNextOutgoingUnidirectionalStreamId(), this, kQpackDecoderStream);
     qpack_decoder_send_stream_ = decoder_send.get();
-    RegisterStaticStream(std::move(decoder_send),
-                         /*stream_already_counted = */ false);
+    RegisterStaticStream(std::move(decoder_send));
     qpack_decoder_->set_qpack_stream_sender_delegate(
         qpack_decoder_send_stream_);
   }
@@ -988,8 +989,7 @@
     auto encoder_send = QuicMakeUnique<QpackSendStream>(
         GetNextOutgoingUnidirectionalStreamId(), this, kQpackEncoderStream);
     qpack_encoder_send_stream_ = encoder_send.get();
-    RegisterStaticStream(std::move(encoder_send),
-                         /*stream_already_counted = */ false);
+    RegisterStaticStream(std::move(encoder_send));
     qpack_encoder_->set_qpack_stream_sender_delegate(
         qpack_encoder_send_stream_);
   }
diff --git a/quic/core/http/quic_spdy_session_test.cc b/quic/core/http/quic_spdy_session_test.cc
index 34f0b92..55de8b6 100644
--- a/quic/core/http/quic_spdy_session_test.cc
+++ b/quic/core/http/quic_spdy_session_test.cc
@@ -304,6 +304,11 @@
         kInitialStreamFlowControlWindowForTest);
     session_.config()->SetInitialSessionFlowControlWindowToSend(
         kInitialSessionFlowControlWindowForTest);
+    if (VersionUsesQpack(transport_version())) {
+      QuicConfigPeer::SetReceivedMaxIncomingUnidirectionalStreams(
+          session_.config(),
+          session_.num_expected_unidirectional_static_streams());
+    }
     connection_->AdvanceTime(QuicTime::Delta::FromSeconds(1));
     TestCryptoStream* crypto_stream = session_.GetMutableCryptoStream();
     EXPECT_CALL(*crypto_stream, HasPendingRetransmission())
diff --git a/quic/core/quic_session.cc b/quic/core/quic_session.cc
index fc4ec23..98aa242 100644
--- a/quic/core/quic_session.cc
+++ b/quic/core/quic_session.cc
@@ -47,10 +47,12 @@
 #define ENDPOINT \
   (perspective() == Perspective::IS_SERVER ? "Server: " : "Client: ")
 
-QuicSession::QuicSession(QuicConnection* connection,
-                         Visitor* owner,
-                         const QuicConfig& config,
-                         const ParsedQuicVersionVector& supported_versions)
+QuicSession::QuicSession(
+    QuicConnection* connection,
+    Visitor* owner,
+    const QuicConfig& config,
+    const ParsedQuicVersionVector& supported_versions,
+    QuicStreamCount num_expected_unidirectional_static_streams)
     : connection_(connection),
       visitor_(owner),
       write_blocked_streams_(connection->transport_version()),
@@ -90,7 +92,9 @@
       closed_streams_clean_up_alarm_(nullptr),
       supported_versions_(supported_versions),
       use_http2_priority_write_scheduler_(false),
-      is_configured_(false) {
+      is_configured_(false),
+      num_expected_unidirectional_static_streams_(
+          num_expected_unidirectional_static_streams) {
   closed_streams_clean_up_alarm_ =
       QuicWrapUnique<QuicAlarm>(connection_->alarm_factory()->CreateAlarm(
           new ClosedStreamsCleanUpDelegate(this)));
@@ -112,32 +116,28 @@
 
   DCHECK_EQ(QuicUtils::GetCryptoStreamId(connection_->transport_version()),
             GetMutableCryptoStream()->id());
-
-  QuicStreamId id =
-      QuicUtils::GetCryptoStreamId(connection_->transport_version());
-  if (VersionHasIetfQuicFrames(connection_->transport_version())) {
-    v99_streamid_manager_.RegisterStaticStream(id, false);
-  }
 }
 
 QuicSession::~QuicSession() {
   QUIC_LOG_IF(WARNING, !zombie_streams_.empty()) << "Still have zombie streams";
 }
 
-void QuicSession::RegisterStaticStream(std::unique_ptr<QuicStream> stream,
-                                       bool stream_already_counted) {
+void QuicSession::RegisterStaticStream(std::unique_ptr<QuicStream> stream) {
   DCHECK(stream->is_static());
   QuicStreamId stream_id = stream->id();
   stream_map_[stream_id] = std::move(stream);
-  if (VersionHasIetfQuicFrames(connection_->transport_version())) {
-    v99_streamid_manager_.RegisterStaticStream(stream_id,
-                                               stream_already_counted);
-  }
   if (IsIncomingStream(stream_id)) {
     ++num_incoming_static_streams_;
   } else {
     ++num_outgoing_static_streams_;
   }
+  if (VersionHasIetfQuicFrames(transport_version()) &&
+      !QuicUtils::IsBidirectionalStreamId(stream_id)) {
+    DCHECK_LE(num_incoming_static_streams_,
+              num_expected_unidirectional_static_streams_);
+    DCHECK_LE(num_outgoing_static_streams_,
+              num_expected_unidirectional_static_streams_);
+  }
 }
 
 void QuicSession::PendingStreamOnStreamFrame(const QuicStreamFrame& frame) {
@@ -970,8 +970,7 @@
     }
     QUIC_DVLOG(1) << "Setting Bidirectional outgoing_max_streams_ to "
                   << max_streams;
-    v99_streamid_manager_.AdjustMaxOpenOutgoingBidirectionalStreams(
-        max_streams);
+    v99_streamid_manager_.SetMaxOpenOutgoingBidirectionalStreams(max_streams);
 
     max_streams = 0;
     if (config_.HasReceivedMaxIncomingUnidirectionalStreams()) {
@@ -979,8 +978,7 @@
     }
     QUIC_DVLOG(1) << "Setting Unidirectional outgoing_max_streams_ to "
                   << max_streams;
-    v99_streamid_manager_.AdjustMaxOpenOutgoingUnidirectionalStreams(
-        max_streams);
+    v99_streamid_manager_.SetMaxOpenOutgoingUnidirectionalStreams(max_streams);
   } else {
     uint32_t max_streams = 0;
     if (config_.HasReceivedMaxIncomingBidirectionalStreams()) {
diff --git a/quic/core/quic_session.h b/quic/core/quic_session.h
index f57ae0f..a402057 100644
--- a/quic/core/quic_session.h
+++ b/quic/core/quic_session.h
@@ -8,6 +8,7 @@
 #define QUICHE_QUIC_CORE_QUIC_SESSION_H_
 
 #include <cstddef>
+#include <cstdint>
 #include <map>
 #include <memory>
 #include <string>
@@ -83,7 +84,8 @@
   QuicSession(QuicConnection* connection,
               Visitor* owner,
               const QuicConfig& config,
-              const ParsedQuicVersionVector& supported_versions);
+              const ParsedQuicVersionVector& supported_versions,
+              QuicStreamCount num_expected_unidirectional_static_streams);
   QuicSession(const QuicSession&) = delete;
   QuicSession& operator=(const QuicSession&) = delete;
 
@@ -446,6 +448,17 @@
 
   bool is_configured() const { return is_configured_; }
 
+  QuicStreamCount num_expected_unidirectional_static_streams() const {
+    return num_expected_unidirectional_static_streams_;
+  }
+
+  // Set the number of unidirectional stream that the peer is allowed to open to
+  // be |max_stream| + |num_expected_static_streams_|.
+  void ConfigureMaxIncomingDynamicStreamsToSend(QuicStreamCount max_stream) {
+    config_.SetMaxIncomingUnidirectionalStreamsToSend(
+        max_stream + num_expected_unidirectional_static_streams_);
+  }
+
  protected:
   using StreamMap = QuicSmallMap<QuicStreamId, std::unique_ptr<QuicStream>, 10>;
 
@@ -519,8 +532,7 @@
   // |stream| as static in stream id manager. |stream_already_counted| is true
   // if |stream| is created from pending stream and is already known as an open
   // stream.
-  void RegisterStaticStream(std::unique_ptr<QuicStream> stream,
-                            bool stream_already_counted);
+  void RegisterStaticStream(std::unique_ptr<QuicStream> stream);
 
   StreamMap& stream_map() { return stream_map_; }
   const StreamMap& stream_map() const { return stream_map_; }
@@ -746,6 +758,9 @@
   // Initialized to false. Set to true when the session has been properly
   // configured and is ready for general operation.
   bool is_configured_;
+
+  // The number of expected static streams.
+  QuicStreamCount num_expected_unidirectional_static_streams_;
 };
 
 }  // namespace quic
diff --git a/quic/core/quic_session_test.cc b/quic/core/quic_session_test.cc
index 267b2df..7480eee 100644
--- a/quic/core/quic_session_test.cc
+++ b/quic/core/quic_session_test.cc
@@ -142,7 +142,8 @@
       : QuicSession(connection,
                     session_visitor,
                     DefaultQuicConfig(),
-                    CurrentSupportedVersions()),
+                    CurrentSupportedVersions(),
+                    /*num_expected_unidirectional_static_streams = */ 0),
         crypto_stream_(this),
         writev_consumes_all_data_(false),
         uses_pending_streams_(false),
diff --git a/quic/core/quic_stream_id_manager.cc b/quic/core/quic_stream_id_manager.cc
index ca6d441..440b97a 100644
--- a/quic/core/quic_stream_id_manager.cc
+++ b/quic/core/quic_stream_id_manager.cc
@@ -30,14 +30,12 @@
       outgoing_max_streams_(max_allowed_outgoing_streams),
       next_outgoing_stream_id_(GetFirstOutgoingStreamId()),
       outgoing_stream_count_(0),
-      outgoing_static_stream_count_(0),
       using_default_max_streams_(true),
       incoming_actual_max_streams_(max_allowed_incoming_streams),
       // Advertised max starts at actual because it's communicated in the
       // handshake.
       incoming_advertised_max_streams_(max_allowed_incoming_streams),
       incoming_initial_max_open_streams_(max_allowed_incoming_streams),
-      incoming_static_stream_count_(0),
       incoming_stream_count_(0),
       largest_peer_created_stream_id_(
           QuicUtils::GetInvalidStreamId(transport_version())),
@@ -55,7 +53,7 @@
   const QuicStreamCount current_outgoing_max_streams = outgoing_max_streams_;
 
   // Set the limit to be exactly the stream count in the frame.
-  if (!ConfigureMaxOpenOutgoingStreams(frame.stream_count)) {
+  if (!SetMaxOpenOutgoingStreams(frame.stream_count)) {
     return false;
   }
   // If we were at the previous limit and this MAX_STREAMS frame
@@ -99,8 +97,7 @@
 
 // Used when configuration has been done and we have an initial
 // maximum stream count from the peer.
-bool QuicStreamIdManager::ConfigureMaxOpenOutgoingStreams(
-    size_t max_open_streams) {
+bool QuicStreamIdManager::SetMaxOpenOutgoingStreams(size_t max_open_streams) {
   if (using_default_max_streams_) {
     // This is the first MAX_STREAMS/transport negotiation we've received. Treat
     // this a bit differently than later ones. The difference is that
@@ -133,43 +130,11 @@
   return true;
 }
 
-void QuicStreamIdManager::SetMaxOpenOutgoingStreams(size_t max_open_streams) {
-  QUIC_BUG_IF(!using_default_max_streams_);
-  // TODO(fkastenholz): when static streams are removed from I-Quic, this
-  // should be revised to invoke ConfigureMaxOpen...
-  AdjustMaxOpenOutgoingStreams(max_open_streams);
-}
-
-// Adjust the outgoing stream limit - max_open_streams is the limit, not
-// including static streams. If the new stream limit wraps, will peg
-// the limit at the implementation max.
-// TODO(fkastenholz): AdjustMax is cognizant of the number of static streams and
-// sets the maximum to be max_streams + number_of_statics. This should be
-// removed from IETF QUIC when static streams are gone.
-void QuicStreamIdManager::AdjustMaxOpenOutgoingStreams(
-    size_t max_open_streams) {
-  if ((outgoing_static_stream_count_ + max_open_streams) < max_open_streams) {
-    // New limit causes us to wrap, set limit to be the implementation maximum.
-    ConfigureMaxOpenOutgoingStreams(
-        QuicUtils::GetMaxStreamCount(unidirectional_, perspective()));
-    return;
-  }
-  // Does not wrap, set limit to what is requested.
-  ConfigureMaxOpenOutgoingStreams(outgoing_static_stream_count_ +
-                                  max_open_streams);
-}
-
 void QuicStreamIdManager::SetMaxOpenIncomingStreams(size_t max_open_streams) {
   QuicStreamCount implementation_max =
       QuicUtils::GetMaxStreamCount(unidirectional_, perspective());
-  QuicStreamCount new_max =
-      std::min(implementation_max,
-               static_cast<QuicStreamCount>(max_open_streams +
-                                            incoming_static_stream_count_));
-  if (new_max < max_open_streams) {
-    // wrapped around ...
-    new_max = implementation_max;
-  }
+  QuicStreamCount new_max = std::min(
+      implementation_max, static_cast<QuicStreamCount>(max_open_streams));
   if (new_max < incoming_stream_count_) {
     session_->connection()->CloseConnection(
         QUIC_MAX_STREAMS_ERROR, "Stream limit less than existing stream count",
@@ -240,61 +205,6 @@
   return false;
 }
 
-bool QuicStreamIdManager::RegisterStaticStream(QuicStreamId stream_id,
-                                               bool stream_already_counted) {
-  DCHECK_NE(QuicUtils::IsBidirectionalStreamId(stream_id), unidirectional_);
-  if (IsIncomingStream(stream_id)) {
-    // This code is predicated on static stream ids being allocated densely, in
-    // order, and starting with the first stream allowed. QUIC_BUG if this is
-    // not so.
-    // This is a stream id for a stream that is started by the peer, deal with
-    // the incoming stream ids. Increase the floor and adjust everything
-    // accordingly.
-
-    QUIC_BUG_IF(incoming_actual_max_streams_ >
-                QuicUtils::GetMaxStreamCount(unidirectional_, perspective()));
-
-    // If we have reached the limit on stream creation, do not create
-    // the static stream; return false.
-    if (incoming_stream_count_ >=
-        QuicUtils::GetMaxStreamCount(unidirectional_, perspective())) {
-      return false;
-    }
-
-    if (incoming_actual_max_streams_ <
-        QuicUtils::GetMaxStreamCount(unidirectional_, perspective())) {
-      incoming_actual_max_streams_++;
-    }
-    if (incoming_advertised_max_streams_ <
-        QuicUtils::GetMaxStreamCount(unidirectional_, perspective())) {
-      incoming_advertised_max_streams_++;
-    }
-
-    if (!stream_already_counted) {
-      incoming_stream_count_++;
-    }
-    incoming_static_stream_count_++;
-    return true;
-  }
-
-  // If we have reached the limit on stream creation, do not create
-  // the static stream; return false.
-  if (outgoing_max_streams_ >=
-      QuicUtils::GetMaxStreamCount(unidirectional_, perspective())) {
-    return false;
-  }
-
-  // Increase the outgoing_max_streams_ limit to reflect the semantic that
-  // outgoing_max_streams_ was inialized to a "maximum request/response" count
-  // and only becomes a maximum stream count when we receive the first
-  // MAX_STREAMS.
-  if (using_default_max_streams_) {
-    outgoing_max_streams_++;
-  }
-  outgoing_static_stream_count_++;
-  return true;
-}
-
 // Stream_id is the id of a new incoming stream. Check if it can be
 // created (doesn't violate limits, etc).
 bool QuicStreamIdManager::MaybeIncreaseLargestPeerStreamId(
diff --git a/quic/core/quic_stream_id_manager.h b/quic/core/quic_stream_id_manager.h
index d0b0824..baba5ac 100644
--- a/quic/core/quic_stream_id_manager.h
+++ b/quic/core/quic_stream_id_manager.h
@@ -47,12 +47,10 @@
         ", outgoing_max_streams_: ", outgoing_max_streams_,
         ", next_outgoing_stream_id_: ", next_outgoing_stream_id_,
         ", outgoing_stream_count_: ", outgoing_stream_count_,
-        ", outgoing_static_stream_count_: ", outgoing_static_stream_count_,
         ", using_default_max_streams_: ", using_default_max_streams_,
         ", incoming_actual_max_streams_: ", incoming_actual_max_streams_,
         ", incoming_advertised_max_streams_: ",
         incoming_advertised_max_streams_,
-        ", incoming_static_stream_count_: ", incoming_static_stream_count_,
         ", incoming_stream_count_: ", incoming_stream_count_,
         ", available_streams_.size(): ", available_streams_.size(),
         ", largest_peer_created_stream_id_: ", largest_peer_created_stream_id_,
@@ -88,37 +86,14 @@
   // allocates a stream ID past the peer specified limit.
   QuicStreamId GetNextOutgoingStreamId();
 
-  // Set the outgoing stream limits to be |max_open_streams| plus the number
-  // of static streams that have been opened. For outgoing and incoming,
-  // respectively.
-  // SetMaxOpenOutgoingStreams will QUIC_BUG if it is called after
-  // a MAX_STREAMS frame has been received.
-  // TODO(fkastenholz): When static streams disappear, these should be removed.
-  void SetMaxOpenOutgoingStreams(size_t max_open_streams);
   void SetMaxOpenIncomingStreams(size_t max_open_streams);
 
-  // Adjust the outgoing stream limit - max_open_streams is the limit, not
-  // including static streams. Does not QUIC_BUG if it is called _after_
-  // receiving a MAX_STREAMS.
-  void AdjustMaxOpenOutgoingStreams(size_t max_open_streams);
-
   // Sets the maximum number of outgoing streams to max_open_streams.
   // Used when configuration has been done and we have an initial
   // maximum stream count from the peer. Note that if the stream count is such
   // that it would result in stream ID values that are greater than the
   // implementation limit, it pegs the count at the implementation limit.
-  bool ConfigureMaxOpenOutgoingStreams(size_t max_open_streams);
-
-  // Register a new stream as a static stream. This is used so that the
-  // advertised MAX STREAMS can be calculated based on the start of the
-  // dynamic stream space. This method will take any stream ID, one that either
-  // this node or the peer will initiate.
-  // If |stream_already_counted| is true, the stream is already counted as an
-  // open stream else where, so no need to count it again.
-  // Returns false if this fails because the new static stream would cause the
-  // stream limit to be exceeded.
-  bool RegisterStaticStream(QuicStreamId stream_id,
-                            bool stream_already_counted);
+  bool SetMaxOpenOutgoingStreams(size_t max_open_streams);
 
   // Checks if the incoming stream ID exceeds the MAX_STREAMS limit.  If the
   // limit is exceeded, closes the connection and returns false.  Uses the
@@ -134,10 +109,6 @@
   // Return true if given stream is peer initiated.
   bool IsIncomingStream(QuicStreamId id) const;
 
-  size_t outgoing_static_stream_count() const {
-    return outgoing_static_stream_count_;
-  }
-
   size_t incoming_initial_max_open_streams() const {
     return incoming_initial_max_open_streams_;
   }
@@ -203,8 +174,6 @@
   bool unidirectional_;
 
   // This is the number of streams that this node can initiate.
-  // This limit applies to both static and dynamic streams - the total
-  // of the two can not exceed this count.
   // This limit is:
   //   - Initiated to a value specified in the constructor
   //   - May be updated when the config is received.
@@ -219,11 +188,6 @@
   // outgoing_max_streams_.
   QuicStreamCount outgoing_stream_count_;
 
-  // Number of outgoing static streams created.
-  // TODO(fkastenholz): Remove when static streams no longer supported for IETF
-  // QUIC.
-  QuicStreamCount outgoing_static_stream_count_;
-
   // Set to true while the default (from the constructor) outgoing stream limit
   // is in use. It is set to false when either a MAX STREAMS frame is received
   // or the transport negotiation completes and sets the stream limit (this is
@@ -232,12 +196,7 @@
   // until we receive an authoritative value from the peer.
   // outgoing_max_streams_ is initialized in the constructor
   // to some hard-coded value, which may or may not be consistent
-  // with what the peer wants. Furthermore, as we create outgoing
-  // static streams, the cap raises as static streams get inserted
-  // "beneath" the dynamic streams because, prior to receiving
-  // a MAX_STREAMS, the values setting the limit are interpreted
-  // as "number of request/responses" that can be created. Once
-  // a MAX_STREAMS is received, it becomes a hard limit.
+  // with what the peer wants.
   bool using_default_max_streams_;
 
   // FOR INCOMING STREAMS
@@ -249,11 +208,6 @@
   // Initial maximum on the number of open streams allowed.
   QuicStreamCount incoming_initial_max_open_streams_;
 
-  // Number of outgoing static streams created.
-  // TODO(fkastenholz): Remove when static streams no longer supported for IETF
-  // QUIC.
-  QuicStreamCount incoming_static_stream_count_;
-
   // This is the number of streams that have been created -- some are still
   // open, the others have been closed. It is the number that is compared
   // against MAX_STREAMS when deciding whether to accept a new stream or not.
diff --git a/quic/core/quic_stream_id_manager_test.cc b/quic/core/quic_stream_id_manager_test.cc
index 8c6766b..19e4d59 100644
--- a/quic/core/quic_stream_id_manager_test.cc
+++ b/quic/core/quic_stream_id_manager_test.cc
@@ -220,21 +220,6 @@
   EXPECT_EQ(1u, stream_id_manager_->max_streams_window());
 }
 
-// Test that stream counts that would exceed the implementation maximum are
-// safely handled.
-// First, check that setting up to and including the implementation maximum
-// is OK.
-TEST_P(QuicStreamIdManagerTestClient, CheckMaxStreamsBadValuesToMaxOkOutgoing) {
-  QuicStreamCount implementation_max =
-      QuicUtils::GetMaxStreamCount(!GetParam(), /* GetParam==true for bidi */
-                                   Perspective::IS_CLIENT);
-  stream_id_manager_->AdjustMaxOpenOutgoingStreams(implementation_max);
-  EXPECT_EQ(implementation_max, stream_id_manager_->outgoing_max_streams());
-
-  stream_id_manager_->AdjustMaxOpenOutgoingStreams(implementation_max);
-  EXPECT_EQ(implementation_max, stream_id_manager_->outgoing_max_streams());
-}
-
 // Now check that setting to a value larger than the maximum fails.
 TEST_P(QuicStreamIdManagerTestClient,
        CheckMaxStreamsBadValuesOverMaxFailsOutgoing) {
@@ -245,7 +230,7 @@
   EXPECT_LT(stream_id_manager_->outgoing_max_streams(), implementation_max);
 
   // Try to go over.
-  stream_id_manager_->AdjustMaxOpenOutgoingStreams(implementation_max + 1);
+  stream_id_manager_->SetMaxOpenOutgoingStreams(implementation_max + 1);
   // Should be pegged at the max.
   EXPECT_EQ(implementation_max, stream_id_manager_->outgoing_max_streams());
 }
@@ -370,9 +355,8 @@
   QuicStreamCount initial_stream_count =
       // need to know the number of request/response streams.
       // This is the total number of outgoing streams (which includes both
-      // req/resp and statics) minus just the statics...
-      stream_id_manager_->outgoing_max_streams() -
-      stream_id_manager_->outgoing_static_stream_count();
+      // req/resp and statics).
+      stream_id_manager_->outgoing_max_streams();
 
   QuicMaxStreamsFrame frame;
 
@@ -599,46 +583,6 @@
             session_->save_frame().max_streams_frame.stream_count);
 }
 
-// Test that registering static stream IDs causes the stream limit to rise
-// accordingly. This is server/client agnostic.
-TEST_P(QuicStreamIdManagerTestClient, TestStaticStreamAdjustment) {
-  QuicStreamId first_dynamic =
-      QuicStreamIdManagerPeer::GetFirstIncomingStreamId(stream_id_manager_);
-  QuicStreamCount actual_max =
-      stream_id_manager_->incoming_actual_max_streams();
-
-  // First test will register the first dynamic stream id as being for a static
-  // stream.
-  stream_id_manager_->RegisterStaticStream(first_dynamic,
-                                           /*stream_already_counted = */ false);
-  // Should go up by 1 stream/stream id.
-  EXPECT_EQ(actual_max + 1u, stream_id_manager_->incoming_actual_max_streams());
-}
-
-// Check that the OnMaxStreamFrame logic properly handles all the
-// cases of offered max streams and outgoing_static_stream_count_,
-// checking for the wrap conditions. Tests in client perspective, necessary
-// because internally, some calculations depend on the client/server
-// perspective.
-TEST_P(QuicStreamIdManagerTestClient, TestMaxStreamsWrapChecks) {
-  QuicStreamCount max_stream_count =
-      QuicUtils::GetMaxStreamCount(IsUnidi(), Perspective::IS_CLIENT);
-  QuicMaxStreamsFrame frame;
-  frame.unidirectional = IsUnidi();
-
-  // Check the case where the offered stream count is less than the
-  // maximum
-  frame.stream_count = max_stream_count - 10;
-  EXPECT_TRUE(stream_id_manager_->OnMaxStreamsFrame(frame));
-  EXPECT_EQ(max_stream_count - 10u, stream_id_manager_->outgoing_max_streams());
-
-  // Now check if the offered count is larger than the max.
-  // The count should be pegged at the max.
-  frame.stream_count = max_stream_count + 10;
-  EXPECT_TRUE(stream_id_manager_->OnMaxStreamsFrame(frame));
-  EXPECT_EQ(max_stream_count, stream_id_manager_->outgoing_max_streams());
-}
-
 // Check that edge conditions of the stream count in a STREAMS_BLOCKED frame
 // are. properly handled.
 TEST_P(QuicStreamIdManagerTestClient, StreamsBlockedEdgeConditions) {
@@ -810,54 +754,6 @@
       stream_id_manager_->MaybeIncreaseLargestPeerStreamId(too_big_stream_id));
 }
 
-// Check that the OnMaxStreamFrame logic properly handles all the
-// cases of offered max streams and outgoing_static_stream_count_,
-// checking for the wrap conditions. Tests in server perspective, necessary
-// because internally, some calculations depend on the client/server
-// perspective.
-TEST_P(QuicStreamIdManagerTestServer, TestMaxStreamsWrapChecks) {
-  QuicStreamCount max_stream_count =
-      QuicUtils::GetMaxStreamCount(IsUnidi(), Perspective::IS_SERVER);
-  QuicMaxStreamsFrame frame;
-  frame.unidirectional = IsUnidi();
-
-  // Check the case where the offered stream count is less than the
-  // implementation maximum,
-  frame.stream_count = max_stream_count - 10;
-  EXPECT_TRUE(stream_id_manager_->OnMaxStreamsFrame(frame));
-  EXPECT_EQ(max_stream_count - 10u, stream_id_manager_->outgoing_max_streams());
-
-  // Check the case where the offered stream count is greater than the
-  // implementation maximum. The count should peg at the maximum.
-  frame.stream_count = max_stream_count + 10;
-  EXPECT_TRUE(stream_id_manager_->OnMaxStreamsFrame(frame));
-  EXPECT_EQ(max_stream_count, stream_id_manager_->outgoing_max_streams());
-}
-
-// Check that static streams can be created before and after MAX_STREAM frame is
-// received.
-TEST_P(QuicStreamIdManagerTestServer, RegisterStaticStreams) {
-  EXPECT_EQ(0u, stream_id_manager_->outgoing_static_stream_count());
-  QuicStreamCount previous_max = stream_id_manager_->outgoing_max_streams();
-  stream_id_manager_->RegisterStaticStream(
-      stream_id_manager_->GetNextOutgoingStreamId(),
-      /*stream_already_counted = */ false);
-  EXPECT_EQ(1u, stream_id_manager_->outgoing_static_stream_count());
-  EXPECT_EQ(previous_max + 1, stream_id_manager_->outgoing_max_streams());
-
-  QuicMaxStreamsFrame frame;
-  frame.unidirectional = IsUnidi();
-  frame.stream_count = 20;
-  EXPECT_TRUE(stream_id_manager_->OnMaxStreamsFrame(frame));
-  EXPECT_EQ(20u, stream_id_manager_->outgoing_max_streams());
-
-  stream_id_manager_->RegisterStaticStream(
-      stream_id_manager_->GetNextOutgoingStreamId(),
-      /*stream_already_counted = */ false);
-  EXPECT_EQ(2u, stream_id_manager_->outgoing_static_stream_count());
-  EXPECT_EQ(20u, stream_id_manager_->outgoing_max_streams());
-}
-
 }  // namespace
 }  // namespace test
 }  // namespace quic
diff --git a/quic/core/uber_quic_stream_id_manager.cc b/quic/core/uber_quic_stream_id_manager.cc
index c69bf24..a2b3b7a 100644
--- a/quic/core/uber_quic_stream_id_manager.cc
+++ b/quic/core/uber_quic_stream_id_manager.cc
@@ -24,30 +24,7 @@
           /*unidirectional=*/true,
           max_open_outgoing_unidirectional_streams,
           max_open_incoming_unidirectional_streams) {}
-void UberQuicStreamIdManager::RegisterStaticStream(
-    QuicStreamId id,
-    bool stream_already_counted) {
-  if (QuicUtils::IsBidirectionalStreamId(id)) {
-    bidirectional_stream_id_manager_.RegisterStaticStream(
-        id, stream_already_counted);
-    return;
-  }
-  unidirectional_stream_id_manager_.RegisterStaticStream(
-      id, stream_already_counted);
-}
 
-void UberQuicStreamIdManager::AdjustMaxOpenOutgoingUnidirectionalStreams(
-    size_t max_streams) {
-  unidirectional_stream_id_manager_.AdjustMaxOpenOutgoingStreams(max_streams);
-}
-void UberQuicStreamIdManager::AdjustMaxOpenOutgoingBidirectionalStreams(
-    size_t max_streams) {
-  bidirectional_stream_id_manager_.AdjustMaxOpenOutgoingStreams(max_streams);
-}
-
-// TODO(fkastenholz): SetMax is cognizant of the number of static streams and
-// sets the maximum to be max_streams + number_of_statics. This should
-// eventually be removed from IETF QUIC.
 void UberQuicStreamIdManager::SetMaxOpenOutgoingBidirectionalStreams(
     size_t max_open_streams) {
   bidirectional_stream_id_manager_.SetMaxOpenOutgoingStreams(max_open_streams);
diff --git a/quic/core/uber_quic_stream_id_manager.h b/quic/core/uber_quic_stream_id_manager.h
index 06edb68..61eaf62 100644
--- a/quic/core/uber_quic_stream_id_manager.h
+++ b/quic/core/uber_quic_stream_id_manager.h
@@ -27,29 +27,12 @@
       QuicStreamCount max_open_incoming_bidirectional_streams,
       QuicStreamCount max_open_incoming_unidirectional_streams);
 
-  // Called when a stream with |stream_id| is registered as a static stream.
-  // If |stream_already_counted| is true, the static stream is already counted
-  // as an open stream earlier, so no need to count it again.
-  void RegisterStaticStream(QuicStreamId id, bool stream_already_counted);
-
-  // Sets the limits to max_open_streams + number of static streams
-  // in existence. SetMaxOpenOutgoingStreams will QUIC_BUG if it is called
-  // after getting the first MAX_STREAMS frame or the transport configuration
-  // was done.
-  // TODO(fkastenholz): SetMax is cognizant of the number of static streams and
-  // sets the maximum to be max_streams + number_of_statics. This should
-  // eventually be removed from IETF QUIC.
+  // Sets the limits to max_open_streams.
   void SetMaxOpenOutgoingBidirectionalStreams(size_t max_open_streams);
   void SetMaxOpenOutgoingUnidirectionalStreams(size_t max_open_streams);
   void SetMaxOpenIncomingBidirectionalStreams(size_t max_open_streams);
   void SetMaxOpenIncomingUnidirectionalStreams(size_t max_open_streams);
 
-  // Sets the outgoing stream count to the number of static streams + max
-  // outgoing streams.  Unlike SetMaxOpenOutgoingStreams, this method will
-  // not QUIC_BUG if called after getting  the first MAX_STREAMS frame.
-  void AdjustMaxOpenOutgoingBidirectionalStreams(size_t max_streams);
-  void AdjustMaxOpenOutgoingUnidirectionalStreams(size_t max_streams);
-
   // Returns true if next outgoing bidirectional stream ID can be allocated.
   bool CanOpenNextOutgoingBidirectionalStream();
 
diff --git a/quic/core/uber_quic_stream_id_manager_test.cc b/quic/core/uber_quic_stream_id_manager_test.cc
index 203dbb8..6f5d34e 100644
--- a/quic/core/uber_quic_stream_id_manager_test.cc
+++ b/quic/core/uber_quic_stream_id_manager_test.cc
@@ -123,38 +123,6 @@
   }
 }
 
-TEST_P(UberQuicStreamIdManagerTest, RegisterStaticStream) {
-  QuicStreamId first_incoming_bidirectional_stream_id =
-      GetParam() == Perspective::IS_SERVER
-          ? GetNthClientInitiatedBidirectionalId(0)
-          : GetNthServerInitiatedBidirectionalId(0);
-  QuicStreamId first_incoming_unidirectional_stream_id =
-      GetParam() == Perspective::IS_SERVER
-          ? GetNthClientInitiatedUnidirectionalId(0)
-          : GetNthServerInitiatedUnidirectionalId(0);
-
-  QuicStreamCount actual_max_allowed_incoming_bidirectional_streams =
-      manager_->actual_max_allowed_incoming_bidirectional_streams();
-  QuicStreamCount actual_max_allowed_incoming_unidirectional_streams =
-      manager_->actual_max_allowed_incoming_unidirectional_streams();
-  manager_->RegisterStaticStream(first_incoming_bidirectional_stream_id,
-                                 /*stream_already_counted = */ false);
-  // Verify actual_max_allowed_incoming_bidirectional_streams increases.
-  EXPECT_EQ(actual_max_allowed_incoming_bidirectional_streams + 1u,
-            manager_->actual_max_allowed_incoming_bidirectional_streams());
-  // Verify actual_max_allowed_incoming_unidirectional_streams does not
-  // change.
-  EXPECT_EQ(actual_max_allowed_incoming_unidirectional_streams,
-            manager_->actual_max_allowed_incoming_unidirectional_streams());
-
-  manager_->RegisterStaticStream(first_incoming_unidirectional_stream_id,
-                                 /*stream_already_counted = */ false);
-  EXPECT_EQ(actual_max_allowed_incoming_bidirectional_streams + 1u,
-            manager_->actual_max_allowed_incoming_bidirectional_streams());
-  EXPECT_EQ(actual_max_allowed_incoming_unidirectional_streams + 1u,
-            manager_->actual_max_allowed_incoming_unidirectional_streams());
-}
-
 TEST_P(UberQuicStreamIdManagerTest, SetMaxOpenOutgoingStreams) {
   const size_t kNumMaxOutgoingStream = 123;
   // Set the uni- and bi- directional limits to different values to ensure
diff --git a/quic/qbone/qbone_client_session.cc b/quic/qbone/qbone_client_session.cc
index 7bd401d..c2972ea 100644
--- a/quic/qbone/qbone_client_session.cc
+++ b/quic/qbone/qbone_client_session.cc
@@ -44,8 +44,7 @@
   auto control_stream =
       QuicMakeUnique<QboneClientControlStream>(this, handler_);
   control_stream_ = control_stream.get();
-  RegisterStaticStream(std::move(control_stream),
-                       /*stream_already_counted = */ false);
+  RegisterStaticStream(std::move(control_stream));
 }
 
 int QboneClientSession::GetNumSentClientHellos() const {
diff --git a/quic/qbone/qbone_server_session.cc b/quic/qbone/qbone_server_session.cc
index 9b2ebdd..37cdd4c 100644
--- a/quic/qbone/qbone_server_session.cc
+++ b/quic/qbone/qbone_server_session.cc
@@ -59,8 +59,7 @@
   auto control_stream =
       QuicMakeUnique<QboneServerControlStream>(this, handler_);
   control_stream_ = control_stream.get();
-  RegisterStaticStream(std::move(control_stream),
-                       /*stream_already_counted = */ false);
+  RegisterStaticStream(std::move(control_stream));
 }
 
 bool QboneServerSession::SendClientRequest(const QboneClientRequest& request) {
diff --git a/quic/qbone/qbone_session_base.cc b/quic/qbone/qbone_session_base.cc
index da2f4ec..3ab3472 100644
--- a/quic/qbone/qbone_session_base.cc
+++ b/quic/qbone/qbone_session_base.cc
@@ -21,14 +21,18 @@
     const QuicConfig& config,
     const ParsedQuicVersionVector& supported_versions,
     QbonePacketWriter* writer)
-    : QuicSession(connection, owner, config, supported_versions) {
+    : QuicSession(connection,
+                  owner,
+                  config,
+                  supported_versions,
+                  /*num_expected_unidirectional_static_streams = */ 0) {
   set_writer(writer);
   const uint32_t max_streams =
       (std::numeric_limits<uint32_t>::max() / kMaxAvailableStreamsMultiplier) -
       1;
   this->config()->SetMaxIncomingBidirectionalStreamsToSend(max_streams);
   if (VersionHasIetfQuicFrames(transport_version())) {
-    this->config()->SetMaxIncomingUnidirectionalStreamsToSend(max_streams);
+    ConfigureMaxIncomingDynamicStreamsToSend(max_streams);
   }
   write_blocked_streams()->SwitchWriteScheduler(
       spdy::WriteSchedulerType::LIFO, connection->transport_version());
diff --git a/quic/quartc/quartc_session.cc b/quic/quartc/quartc_session.cc
index f206d62..117b137 100644
--- a/quic/quartc/quartc_session.cc
+++ b/quic/quartc/quartc_session.cc
@@ -24,7 +24,11 @@
                              const QuicConfig& config,
                              const ParsedQuicVersionVector& supported_versions,
                              const QuicClock* clock)
-    : QuicSession(connection.get(), visitor, config, supported_versions),
+    : QuicSession(connection.get(),
+                  visitor,
+                  config,
+                  supported_versions,
+                  /*num_expected_unidirectional_static_streams = */ 0),
       connection_(std::move(connection)),
       clock_(clock),
       per_packet_options_(QuicMakeUnique<QuartcPerPacketOptions>()) {
diff --git a/quic/quartc/quartc_stream_test.cc b/quic/quartc/quartc_stream_test.cc
index b689e57..eb4d412 100644
--- a/quic/quartc/quartc_stream_test.cc
+++ b/quic/quartc/quartc_stream_test.cc
@@ -55,7 +55,8 @@
       : QuicSession(connection,
                     nullptr /*visitor*/,
                     config,
-                    CurrentSupportedVersions()),
+                    CurrentSupportedVersions(),
+                    /*num_expected_unidirectional_static_streams = */ 0),
         write_buffer_(write_buffer) {}
 
   ~MockQuicSession() override {}
diff --git a/quic/test_tools/quic_session_peer.cc b/quic/test_tools/quic_session_peer.cc
index 1f83909..ab4e5af 100644
--- a/quic/test_tools/quic_session_peer.cc
+++ b/quic/test_tools/quic_session_peer.cc
@@ -161,8 +161,7 @@
 // static
 void QuicSessionPeer::RegisterStaticStream(QuicSession* session,
                                            std::unique_ptr<QuicStream> stream) {
-  return session->RegisterStaticStream(std::move(stream),
-                                       /*stream_already_counted = */ false);
+  return session->RegisterStaticStream(std::move(stream));
 }
 
 // static
diff --git a/quic/test_tools/quic_test_utils.cc b/quic/test_tools/quic_test_utils.cc
index 9274567..5ff5e05 100644
--- a/quic/test_tools/quic_test_utils.cc
+++ b/quic/test_tools/quic_test_utils.cc
@@ -529,7 +529,8 @@
     : QuicSession(connection,
                   nullptr,
                   DefaultQuicConfig(),
-                  connection->supported_versions()) {
+                  connection->supported_versions(),
+                  /*num_expected_unidirectional_static_streams = */ 0) {
   if (create_mock_crypto_stream) {
     crypto_stream_ = QuicMakeUnique<MockQuicCryptoStream>(this);
   }
diff --git a/quic/tools/quic_simple_server_session_test.cc b/quic/tools/quic_simple_server_session_test.cc
index 899db9c..6cc97ed 100644
--- a/quic/tools/quic_simple_server_session_test.cc
+++ b/quic/tools/quic_simple_server_session_test.cc
@@ -198,13 +198,18 @@
     QuicConfigPeer::SetReceivedMaxIncomingBidirectionalStreams(
         &config_, kMaxStreamsForTest);
     config_.SetMaxIncomingUnidirectionalStreamsToSend(kMaxStreamsForTest);
-    QuicConfigPeer::SetReceivedMaxIncomingUnidirectionalStreams(
-        &config_, kMaxStreamsForTest);
 
     config_.SetInitialStreamFlowControlWindowToSend(
         kInitialStreamFlowControlWindowForTest);
     config_.SetInitialSessionFlowControlWindowToSend(
         kInitialSessionFlowControlWindowForTest);
+    if (VersionUsesQpack(GetParam().transport_version)) {
+      QuicConfigPeer::SetReceivedMaxIncomingUnidirectionalStreams(
+          &config_, kMaxStreamsForTest + 3);
+    } else {
+      QuicConfigPeer::SetReceivedMaxIncomingUnidirectionalStreams(
+          &config_, kMaxStreamsForTest);
+    }
 
     ParsedQuicVersionVector supported_versions = SupportedVersions(GetParam());
     connection_ = new StrictMock<MockQuicConnectionWithSendStreamData>(