You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@trafficserver.apache.org by GitBox <gi...@apache.org> on 2021/03/25 17:01:47 UTC

[GitHub] [trafficserver] shinrich opened a new pull request #7622: HTTP/2 to origin

shinrich opened a new pull request #7622:
URL: https://github.com/apache/trafficserver/pull/7622


   This is work that has been going on in fits and starts for a couple years with contributions from @a-canary and Kees.  And discussions with @masaori335 and @maskit.  I have been using the project page https://github.com/apache/trafficserver/projects/9 to track issues and progress.
   
   This latest effort builds upon @a-canary's earlier PoolableSession restructuring work and has been going in earnest since last December.  I have run a version of this code against our 9.0.x (plus some other HTTP/2 commits to ease the merge) on one of our production boxes for several weeks.  That worked out many memory management issues.  In that environment it negotiated HTTP/2 with another ATS layer and an istio/envoy layer.  About 20% of our outbound transactions were over H2.
   
   I've just brought it back to master and rebased for this PR.  Locally on my Centos7 box is passes autest, unit tests, and regression tests.
   
   This code contains trailer logic to support gRPC.  I have run versions of this code against one of the python examples in the gRPC repository, but that was last done in January.  The gRPC/trailer support needs more testing, but it is a start.
   
   Some key code changes include
   
   * ConnectionPool/ConnectingEntry - This data structure tracks existing outstanding connection requests.  For HTTP/2, you may receive many simultaneous incoming streams all heading for the same location.  You do not want to start separate HTTP/2 sessions for each.  The code uses to history in the HostInfo to determine if it is likely that the origin will negotiate HTTP/2 and use the ConnnectionPool or connect directly ignoring other outstanding connection requests.
   * Server_session becomes server_txn in HttpSM. 
   * An overridable ALPN setting has been added to control whether ATS will offer to negotiate H2 with the origin.
   * Two h2 origin autests have been added leveraging proxy-verifier.
   
   The main code change is adding Http2 versions of transactions and sessions, and creating a Http1 server transaction.  The class hierarchies differ slightly between Http1 and Http2.  For Http1, both Http1ClientTransaction and Http1ServerTransaction inherit  directly from ProxyTransaction. 
   
   For Http2, there is still only the Http2Stream to represent both client and server transactions.  This PR adds an _outbound_flag to track the few cases where we do need to process inbound and outbound transactions differently.  For the sessions, the common elements of session processing are moved into Http2CommonSession.  The common class is used as a mix in for Http2ClientSession and Http2ServerSession.    Http2ClientSession continues to inherit directly from ProxySession.  Http2ServerSession inherits from PoolableSession as the Http1ServerSession does.
   
   One other thing to point out, is that the Http2ServerSessions can only be put in a Thread session pool.  I tested in hybrid mode, but pure thread pool mode should also work.  I need to see what happens with global pool mode.  That will likely require an addition check, warning, or error.
   
   This branch also includes connection failure cleanups identified in PR #7580.  We are running with that change in our production environment.  Pulling it back out to make the master PR seemed unnecessary since it will hopefully land first.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficserver] shinrich commented on a change in pull request #7622: HTTP/2 to origin

Posted by GitBox <gi...@apache.org>.
shinrich commented on a change in pull request #7622:
URL: https://github.com/apache/trafficserver/pull/7622#discussion_r602374401



##########
File path: iocore/net/I_NetVConnection.h
##########
@@ -39,6 +39,7 @@
 #include "YamlSNIConfig.h"
 #include "tscpp/util/TextView.h"
 #include "tscore/IpMap.h"
+#include "P_ALPNSupport.h"

Review comment:
       I moved it into ink_configs.h next to TS_MAX_HOST_NAME_LEN.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficserver] shinrich commented on a change in pull request #7622: HTTP/2 to origin

Posted by GitBox <gi...@apache.org>.
shinrich commented on a change in pull request #7622:
URL: https://github.com/apache/trafficserver/pull/7622#discussion_r612071871



##########
File path: proxy/ProxyTransaction.cc
##########
@@ -220,3 +220,78 @@ ProxyTransaction::has_request_body(int64_t request_content_length, bool is_chunk
 {
   return request_content_length > 0 || is_chunked;
 }
+
+bool
+ProxyTransaction::is_read_closed() const
+{
+  return false;
+}
+
+bool
+ProxyTransaction::expect_send_trailer() const
+{
+  return false;
+}
+
+void
+ProxyTransaction::set_expect_send_trailer()
+{
+}
+bool
+ProxyTransaction::expect_receive_trailer() const
+{
+  return false;
+}
+
+void
+ProxyTransaction::set_expect_receive_trailer()
+{
+}
+
+void
+ProxyTransaction::attach_transaction(HttpSM *attach_sm)
+{
+  _sm = attach_sm;
+}
+
+void
+ProxyTransaction::release()
+{
+  HttpTxnDebug("[%" PRId64 "] session released by sm [%" PRId64 "]", _proxy_ssn ? _proxy_ssn->connection_id() : 0,
+               _sm ? _sm->sm_id : 0);
+
+  this->decrement_transactions_stat();

Review comment:
       I think the double decrement has been dealt with.  The current_*_connection, current_*_transaction and current_*_stream counts seemed to be staying accurate in my production run earlier today.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficserver] bneradt commented on pull request #7622: HTTP/2 to origin

Posted by GitBox <gi...@apache.org>.
bneradt commented on pull request #7622:
URL: https://github.com/apache/trafficserver/pull/7622#issuecomment-815134601


   [approve ci]


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficserver] shinrich commented on a change in pull request #7622: HTTP/2 to origin

Posted by GitBox <gi...@apache.org>.
shinrich commented on a change in pull request #7622:
URL: https://github.com/apache/trafficserver/pull/7622#discussion_r612071871



##########
File path: proxy/ProxyTransaction.cc
##########
@@ -220,3 +220,78 @@ ProxyTransaction::has_request_body(int64_t request_content_length, bool is_chunk
 {
   return request_content_length > 0 || is_chunked;
 }
+
+bool
+ProxyTransaction::is_read_closed() const
+{
+  return false;
+}
+
+bool
+ProxyTransaction::expect_send_trailer() const
+{
+  return false;
+}
+
+void
+ProxyTransaction::set_expect_send_trailer()
+{
+}
+bool
+ProxyTransaction::expect_receive_trailer() const
+{
+  return false;
+}
+
+void
+ProxyTransaction::set_expect_receive_trailer()
+{
+}
+
+void
+ProxyTransaction::attach_transaction(HttpSM *attach_sm)
+{
+  _sm = attach_sm;
+}
+
+void
+ProxyTransaction::release()
+{
+  HttpTxnDebug("[%" PRId64 "] session released by sm [%" PRId64 "]", _proxy_ssn ? _proxy_ssn->connection_id() : 0,
+               _sm ? _sm->sm_id : 0);
+
+  this->decrement_transactions_stat();

Review comment:
       I think the double decrement has been dealt with.  The current_ * _connection, current_ * _transaction and current_ * _stream counts seemed to be staying accurate in my production run earlier today.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficserver] bneradt commented on pull request #7622: HTTP/2 to origin

Posted by GitBox <gi...@apache.org>.
bneradt commented on pull request #7622:
URL: https://github.com/apache/trafficserver/pull/7622#issuecomment-815120875


   [approve ci]


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficserver] shinrich commented on a change in pull request #7622: HTTP/2 to origin

Posted by GitBox <gi...@apache.org>.
shinrich commented on a change in pull request #7622:
URL: https://github.com/apache/trafficserver/pull/7622#discussion_r607154042



##########
File path: src/tscore/BufferWriterFormat.cc
##########
@@ -896,8 +897,12 @@ bwformat(BufferWriter &w, BWFSpec const &spec, bwf::Errno const &e)
   if (spec.has_numeric_type()) {              // if numeric type, print just the numeric part.
     w.print(number_fmt, e._e);
   } else {
-    w.write(short_name(e._e));
-    w.write(strerror(e._e));
+    if (e._e < 0) {

Review comment:
       This also is part of PR #7580.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficserver] shinrich commented on a change in pull request #7622: HTTP/2 to origin

Posted by GitBox <gi...@apache.org>.
shinrich commented on a change in pull request #7622:
URL: https://github.com/apache/trafficserver/pull/7622#discussion_r647649623



##########
File path: proxy/http/HttpSM.cc
##########
@@ -1801,19 +1945,51 @@ HttpSM::handle_api_return()
 }
 
 PoolableSession *
-HttpSM::create_server_session(NetVConnection *netvc)
+HttpSM::create_server_session(NetVConnection *netvc, MIOBuffer *netvc_read_buffer, IOBufferReader *netvc_reader)
 {
   HttpTransact::State &s  = this->t_state;
-  PoolableSession *retval = httpServerSessionAllocator.alloc();
+  PoolableSession *retval = nullptr;
+
+  // Figure out what protocol was negotiated
+  const unsigned char *proto  = nullptr;
+  unsigned int proto_length   = 0;
+  SSLNetVConnection *sslnetvc = dynamic_cast<SSLNetVConnection *>(netvc);
+  if (sslnetvc) {
+    SSL_get0_alpn_selected(sslnetvc->ssl, &proto, &proto_length);
+  }

Review comment:
       Done




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficserver] shinrich commented on a change in pull request #7622: HTTP/2 to origin

Posted by GitBox <gi...@apache.org>.
shinrich commented on a change in pull request #7622:
URL: https://github.com/apache/trafficserver/pull/7622#discussion_r602415912



##########
File path: proxy/http/Http1ClientTransaction.cc
##########
@@ -21,23 +21,28 @@
   limitations under the License.
  */
 
-#include "Http1Transaction.h"
+#include "Http1ClientTransaction.h"
 #include "Http1ClientSession.h"
 #include "HttpSM.h"
 
 void
-Http1Transaction::release(IOBufferReader *r)
+Http1ClientTransaction::release()
 {
-}
+  _proxy_ssn->clear_session_active();
+  /*
+    // Must set this inactivity count here rather than in the session because the state machine
+    // is not available then
+    MgmtInt ka_in = _sm->t_state.txn_conf->keep_alive_no_activity_timeout_in;
+    set_inactivity_timeout(HRTIME_SECONDS(ka_in));
 
-void
-Http1Transaction::reset()
-{
-  _sm = nullptr;
+    _proxy_ssn->ssn_last_txn_time = Thread::get_hrtime();

Review comment:
       removed




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficserver] SolidWallOfCode commented on a change in pull request #7622: HTTP/2 to origin

Posted by GitBox <gi...@apache.org>.
SolidWallOfCode commented on a change in pull request #7622:
URL: https://github.com/apache/trafficserver/pull/7622#discussion_r609019013



##########
File path: proxy/http/HttpTunnel.cc
##########
@@ -1161,6 +1165,8 @@ HttpTunnel::producer_handler(int event, HttpTunnelProducer *p)
     // If the write completes on the stack (as it can for http2), then
     // consumer could have called back by this point.  Must treat this as
     // a regular read complete (falling through to the following cases).
+    p->bytes_read = p->init_bytes_done;
+    // FALLTHROUGH

Review comment:
       Note - `[[fallthrough]]` is now official syntax - https://en.cppreference.com/w/cpp/language/attributes/fallthrough




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficserver] bneradt commented on pull request #7622: HTTP/2 to origin

Posted by GitBox <gi...@apache.org>.
bneradt commented on pull request #7622:
URL: https://github.com/apache/trafficserver/pull/7622#issuecomment-815166443


   [approve ci]


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficserver] shinrich commented on pull request #7622: HTTP/2 to origin

Posted by GitBox <gi...@apache.org>.
shinrich commented on pull request #7622:
URL: https://github.com/apache/trafficserver/pull/7622#issuecomment-950809796


   Closing in deference to PR #8447 against the 10-Dev branch.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@trafficserver.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficserver] maskit commented on a change in pull request #7622: HTTP/2 to origin

Posted by GitBox <gi...@apache.org>.
maskit commented on a change in pull request #7622:
URL: https://github.com/apache/trafficserver/pull/7622#discussion_r645321746



##########
File path: proxy/http/HttpSM.cc
##########
@@ -1801,19 +1945,51 @@ HttpSM::handle_api_return()
 }
 
 PoolableSession *
-HttpSM::create_server_session(NetVConnection *netvc)
+HttpSM::create_server_session(NetVConnection *netvc, MIOBuffer *netvc_read_buffer, IOBufferReader *netvc_reader)
 {
   HttpTransact::State &s  = this->t_state;
-  PoolableSession *retval = httpServerSessionAllocator.alloc();
+  PoolableSession *retval = nullptr;
+
+  // Figure out what protocol was negotiated
+  const unsigned char *proto  = nullptr;
+  unsigned int proto_length   = 0;
+  SSLNetVConnection *sslnetvc = dynamic_cast<SSLNetVConnection *>(netvc);
+  if (sslnetvc) {
+    SSL_get0_alpn_selected(sslnetvc->ssl, &proto, &proto_length);
+  }
 
-  retval->sharing_pool         = static_cast<TSServerSessionSharingPoolType>(s.http_config_param->server_session_sharing_pool);
-  retval->sharing_match        = static_cast<TSServerSessionSharingMatchMask>(s.txn_conf->server_session_sharing_match);
-  MIOBuffer *netvc_read_buffer = new_MIOBuffer(HTTP_SERVER_RESP_HDR_BUFFER_INDEX);
-  IOBufferReader *netvc_reader = netvc_read_buffer->alloc_reader();
+  bool add_session = false;
+  if (proto_length == 2 && memcmp(proto, "h2", 2) == 0) {
+    Http2ServerSession *session = http2ServerSessionAllocator.alloc();
+    add_session                 = true;
+    retval                      = session;
+  } else {
+    Http1ServerSession *session = httpServerSessionAllocator.alloc();
+    retval                      = session;
+  }

Review comment:
       I understand we need to do this here, but I don't want to have the detail here in HttpSM. We should probably have a mechanism like `ProtocolProbeSessionAccept`.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficserver] maskit edited a comment on pull request #7622: HTTP/2 to origin

Posted by GitBox <gi...@apache.org>.
maskit edited a comment on pull request #7622:
URL: https://github.com/apache/trafficserver/pull/7622#issuecomment-856473460


   Let's separate out the outbound ALPN part. We still need to discuss how to enable protocols, but the feature itself should work without H2 (e.g. use http/1.1 but not 1.0), and it should be testable.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficserver] shinrich commented on a change in pull request #7622: HTTP/2 to origin

Posted by GitBox <gi...@apache.org>.
shinrich commented on a change in pull request #7622:
URL: https://github.com/apache/trafficserver/pull/7622#discussion_r612068198



##########
File path: proxy/ProxySession.h
##########
@@ -156,6 +156,12 @@ class ProxySession : public VConnection, public PluginUserArgs<TS_USER_ARGS_SSN>
   void do_io_shutdown(ShutdownHowTo_t howto) override;
   void reenable(VIO *vio) override;
 
+  virtual ProxyTransaction *

Review comment:
       This is a reasonable default implementation.  Used by Http2ClientSession and maybe by some of the quic classes as well.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficserver] shinrich commented on pull request #7622: HTTP/2 to origin

Posted by GitBox <gi...@apache.org>.
shinrich commented on pull request #7622:
URL: https://github.com/apache/trafficserver/pull/7622#issuecomment-815256165


   [approve ci]


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficserver] bneradt commented on pull request #7622: HTTP/2 to origin

Posted by GitBox <gi...@apache.org>.
bneradt commented on pull request #7622:
URL: https://github.com/apache/trafficserver/pull/7622#issuecomment-815126966


   [approve ci]


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficserver] bneradt removed a comment on pull request #7622: HTTP/2 to origin

Posted by GitBox <gi...@apache.org>.
bneradt removed a comment on pull request #7622:
URL: https://github.com/apache/trafficserver/pull/7622#issuecomment-815172177


   [approve ci]


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficserver] maskit edited a comment on pull request #7622: HTTP/2 to origin

Posted by GitBox <gi...@apache.org>.
maskit edited a comment on pull request #7622:
URL: https://github.com/apache/trafficserver/pull/7622#issuecomment-840179475


   It seem like we could split this PR into a couple of separated PRs like:
   - To use server_txn instead of server_session in HttpSM
   - Changes around HttpSessionManager
   - Trailer header support
   - Something around HttpTunnel
   - ALPN setting for outbound connection
   - H2 class restructure
   - H2 to Origin support
   
   Although I'm not sure how much it would be hard and some may not be decoupled, the basic idea is to make this PR only adding H2 to Origin support. We need to make a lot of changes as preparation for supporting multiplexing server session, and some of them could affect existing features. I understand 9.1 is already branched and we basically should not add changes any more, but having some of the changes above on 9.1 would make debugging easier than adding all of them on 9.2.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficserver] maskit commented on a change in pull request #7622: HTTP/2 to origin

Posted by GitBox <gi...@apache.org>.
maskit commented on a change in pull request #7622:
URL: https://github.com/apache/trafficserver/pull/7622#discussion_r677061238



##########
File path: proxy/http/HttpSM.cc
##########
@@ -289,6 +277,160 @@ HttpVCTable::cleanup_all()
   }
 }
 
+class ConnectingEntry : public Continuation
+{
+public:
+  std::set<HttpSM *> _connect_sms;
+  NetVConnection *_netvc        = nullptr;
+  IOBufferReader *_netvc_reader = nullptr;
+  MIOBuffer *_netvc_read_buffer = nullptr;
+  std::string sni;
+  std::string cert_name;
+  IpEndpoint _ipaddr;
+  std::string hostname;
+  Action *_pending_action = nullptr;
+  NetVCOptions opt;
+
+  void remove_entry();
+  int state_http_server_open(int event, void *data);
+  static PoolableSession *create_server_session(HttpSM *root_sm, NetVConnection *netvc, MIOBuffer *netvc_read_buffer,
+                                                IOBufferReader *netvc_reader);
+};
+
+struct IpHelper {
+  size_t
+  operator()(IpEndpoint const &arg) const
+  {
+    return IpAddr{&arg.sa}.hash();
+  }
+  bool
+  operator()(IpEndpoint const &arg1, IpEndpoint const &arg2) const
+  {
+    return ats_ip_addr_port_eq(&arg1.sa, &arg2.sa);
+  }
+};
+using ConnectingIpPool = std::unordered_multimap<IpEndpoint, ConnectingEntry *, IpHelper>;
+
+class ConnectingPool
+{
+public:
+  ConnectingPool() {}
+  ConnectingIpPool m_ip_pool;
+};
+
+void
+initialize_thread_for_connecting_pools(EThread *thread)
+{
+  if (thread->connecting_pool == nullptr) {
+    thread->connecting_pool = new ConnectingPool();
+  }
+}
+
+int
+ConnectingEntry::state_http_server_open(int event, void *data)
+{
+  Debug("http_connect", "entered inside ConnectingEntry::state_http_server_open");
+
+  switch (event) {
+  case NET_EVENT_OPEN: {
+    _netvc                 = static_cast<NetVConnection *>(data);
+    UnixNetVConnection *vc = static_cast<UnixNetVConnection *>(_netvc);
+    ink_release_assert(_pending_action == nullptr || _pending_action->continuation == vc->get_action()->continuation);
+    _pending_action = nullptr;
+    Debug("http_connect", "ConnectingEntrysetting handler for TCP handshake");
+    // Just want to get a write-ready event so we know that the TCP handshake is complete.
+    // The buffer we create will be handed over to the eventually created server session
+    _netvc_read_buffer = new_MIOBuffer(HTTP_SERVER_RESP_HDR_BUFFER_INDEX);
+    _netvc_reader      = _netvc_read_buffer->alloc_reader();
+    _netvc->do_io_write(this, 1, _netvc_reader);
+    ink_release_assert(!_connect_sms.empty());
+    if (!_connect_sms.empty()) {
+      HttpSM *prime_connect_sm = *(_connect_sms.begin());
+      _netvc->set_inactivity_timeout(prime_connect_sm->get_server_connect_timeout());
+    }
+    ink_release_assert(_pending_action == nullptr);
+    return 0;
+  }
+  case VC_EVENT_READ_COMPLETE:
+  case VC_EVENT_WRITE_READY:
+  case VC_EVENT_WRITE_COMPLETE: {
+    Debug("http_connect", "Kick off %" PRId64 " state machines waiting for origin", _connect_sms.size());
+    this->remove_entry();
+    _netvc->do_io_write(nullptr, 0, nullptr);
+    if (!_connect_sms.empty()) {
+      auto prime_iter = _connect_sms.rbegin();
+      ink_release_assert(prime_iter != _connect_sms.rend());
+      PoolableSession *new_session = (*prime_iter)->create_server_session(_netvc, _netvc_read_buffer, _netvc_reader);
+      _netvc                       = nullptr;
+
+      // Did we end up with a multiplexing session?
+      int count = 0;
+      if (new_session->is_multiplexing()) {
+        // Hand off to all queued up ConnectSM's.

Review comment:
       > Actually, I think the code will pull new transactions under the reported max_concurrent_streams setting.
   
   I don't really understand this part. On the example above, the origin server tells us "max_concurrent_streams is 2" on the first connection. The origin server doesn't want to take the third request until either of the first two transactions ends. But we exploit a loophole and start third request by opening a new connection. I think this is the scenario that the paragraph on RFC7540 tries to avoid.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@trafficserver.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficserver] shinrich edited a comment on pull request #7622: HTTP/2 to origin

Posted by GitBox <gi...@apache.org>.
shinrich edited a comment on pull request #7622:
URL: https://github.com/apache/trafficserver/pull/7622#issuecomment-950395999


   Rebased against 10-Dev branch.  I need to close this PR and create another against 10-Dev.  Will coordinate with @bneradt to do that.  Need to sync up my last Yahoo testing changes too.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@trafficserver.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficserver] shinrich commented on a change in pull request #7622: HTTP/2 to origin

Posted by GitBox <gi...@apache.org>.
shinrich commented on a change in pull request #7622:
URL: https://github.com/apache/trafficserver/pull/7622#discussion_r607153426



##########
File path: proxy/http/HttpTransact.cc
##########
@@ -3969,47 +3977,21 @@ HttpTransact::handle_forward_server_connection_open(State *s)
   TxnDebug("http_seq", "[HttpTransact::handle_server_connection_open] ");
   ink_release_assert(s->current.state == CONNECTION_ALIVE);
 
-  if (s->hdr_info.server_response.version_get() == HTTPVersion(0, 9)) {
-    TxnDebug("http_trans", "[hfsco] server sent 0.9 response, reading...");
-    build_response(s, &s->hdr_info.client_response, s->client_info.http_version, HTTP_STATUS_OK, "Connection Established");
-
-    s->client_info.keep_alive = HTTP_NO_KEEPALIVE;
-    s->cache_info.action      = CACHE_DO_NO_ACTION;
-    s->next_action            = SM_ACTION_SERVER_READ;
-    return;
-
-  } else if (s->hdr_info.server_response.version_get() == HTTPVersion(1, 0)) {
-    if (s->current.server->http_version == HTTPVersion(0, 9)) {
-      // update_hostdb_to_indicate_server_version_is_1_0
-      s->updated_server_version = HostDBApplicationInfo::HTTP_VERSION_10;
-    } else if (s->current.server->http_version == HTTPVersion(1, 1)) {
-      // update_hostdb_to_indicate_server_version_is_1_0
-      s->updated_server_version = HostDBApplicationInfo::HTTP_VERSION_10;
-    } else {
-      // dont update the hostdb. let us try again with what we currently think.
-    }
-  } else if (s->hdr_info.server_response.version_get() == HTTPVersion(1, 1)) {
-    if (s->current.server->http_version == HTTPVersion(0, 9)) {
-      // update_hostdb_to_indicate_server_version_is_1_1
-      s->updated_server_version = HostDBApplicationInfo::HTTP_VERSION_11;
-    } else if (s->current.server->http_version == HTTPVersion(1, 0)) {
-      // update_hostdb_to_indicate_server_version_is_1_1
-      s->updated_server_version = HostDBApplicationInfo::HTTP_VERSION_11;
-    } else {
-      // dont update the hostdb. let us try again with what we currently think.
-    }
-  } else {
-    // dont update the hostdb. let us try again with what we currently think.
+  HostDBApplicationInfo::HttpVersion real_version = s->state_machine->server_txn->get_version(s->hdr_info.server_response);
+  if (real_version != s->host_db_info.app.http_data.http_version) {
+    TxnDebug("http_trans", "Update hostdb history of server HTTP version 0x%x", real_version);
+    // Need to update the hostdb
+    s->updated_server_version = real_version;

Review comment:
       Actually, this change is key to an efficient H2 to origin.  We store the previously HTTP2 version that the origin negotiated in hostdb.  So if it previously negotiated HTTP/1.x we don't do the potentially more expensive queuing of multiple connection requests to the same origin.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficserver] SolidWallOfCode commented on a change in pull request #7622: HTTP/2 to origin

Posted by GitBox <gi...@apache.org>.
SolidWallOfCode commented on a change in pull request #7622:
URL: https://github.com/apache/trafficserver/pull/7622#discussion_r609047726



##########
File path: proxy/ProxySession.h
##########
@@ -156,6 +156,12 @@ class ProxySession : public VConnection, public PluginUserArgs<TS_USER_ARGS_SSN>
   void do_io_shutdown(ShutdownHowTo_t howto) override;
   void reenable(VIO *vio) override;
 
+  virtual ProxyTransaction *

Review comment:
       Make this a pure virtual instead?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficserver] shinrich commented on a change in pull request #7622: HTTP/2 to origin

Posted by GitBox <gi...@apache.org>.
shinrich commented on a change in pull request #7622:
URL: https://github.com/apache/trafficserver/pull/7622#discussion_r647040474



##########
File path: proxy/http/HttpSM.cc
##########
@@ -289,6 +277,160 @@ HttpVCTable::cleanup_all()
   }
 }
 
+class ConnectingEntry : public Continuation
+{
+public:
+  std::set<HttpSM *> _connect_sms;
+  NetVConnection *_netvc        = nullptr;
+  IOBufferReader *_netvc_reader = nullptr;
+  MIOBuffer *_netvc_read_buffer = nullptr;
+  std::string sni;
+  std::string cert_name;
+  IpEndpoint _ipaddr;
+  std::string hostname;
+  Action *_pending_action = nullptr;
+  NetVCOptions opt;
+
+  void remove_entry();
+  int state_http_server_open(int event, void *data);
+  static PoolableSession *create_server_session(HttpSM *root_sm, NetVConnection *netvc, MIOBuffer *netvc_read_buffer,
+                                                IOBufferReader *netvc_reader);
+};
+
+struct IpHelper {
+  size_t
+  operator()(IpEndpoint const &arg) const
+  {
+    return IpAddr{&arg.sa}.hash();
+  }
+  bool
+  operator()(IpEndpoint const &arg1, IpEndpoint const &arg2) const
+  {
+    return ats_ip_addr_port_eq(&arg1.sa, &arg2.sa);
+  }
+};
+using ConnectingIpPool = std::unordered_multimap<IpEndpoint, ConnectingEntry *, IpHelper>;
+
+class ConnectingPool
+{
+public:
+  ConnectingPool() {}
+  ConnectingIpPool m_ip_pool;
+};
+
+void
+initialize_thread_for_connecting_pools(EThread *thread)
+{
+  if (thread->connecting_pool == nullptr) {
+    thread->connecting_pool = new ConnectingPool();
+  }
+}
+
+int
+ConnectingEntry::state_http_server_open(int event, void *data)
+{
+  Debug("http_connect", "entered inside ConnectingEntry::state_http_server_open");
+
+  switch (event) {
+  case NET_EVENT_OPEN: {
+    _netvc                 = static_cast<NetVConnection *>(data);
+    UnixNetVConnection *vc = static_cast<UnixNetVConnection *>(_netvc);
+    ink_release_assert(_pending_action == nullptr || _pending_action->continuation == vc->get_action()->continuation);
+    _pending_action = nullptr;
+    Debug("http_connect", "ConnectingEntrysetting handler for TCP handshake");
+    // Just want to get a write-ready event so we know that the TCP handshake is complete.
+    // The buffer we create will be handed over to the eventually created server session
+    _netvc_read_buffer = new_MIOBuffer(HTTP_SERVER_RESP_HDR_BUFFER_INDEX);
+    _netvc_reader      = _netvc_read_buffer->alloc_reader();
+    _netvc->do_io_write(this, 1, _netvc_reader);
+    ink_release_assert(!_connect_sms.empty());
+    if (!_connect_sms.empty()) {
+      HttpSM *prime_connect_sm = *(_connect_sms.begin());
+      _netvc->set_inactivity_timeout(prime_connect_sm->get_server_connect_timeout());
+    }
+    ink_release_assert(_pending_action == nullptr);
+    return 0;
+  }
+  case VC_EVENT_READ_COMPLETE:
+  case VC_EVENT_WRITE_READY:
+  case VC_EVENT_WRITE_COMPLETE: {
+    Debug("http_connect", "Kick off %" PRId64 " state machines waiting for origin", _connect_sms.size());
+    this->remove_entry();
+    _netvc->do_io_write(nullptr, 0, nullptr);
+    if (!_connect_sms.empty()) {
+      auto prime_iter = _connect_sms.rbegin();
+      ink_release_assert(prime_iter != _connect_sms.rend());
+      PoolableSession *new_session = (*prime_iter)->create_server_session(_netvc, _netvc_read_buffer, _netvc_reader);
+      _netvc                       = nullptr;
+
+      // Did we end up with a multiplexing session?
+      int count = 0;
+      if (new_session->is_multiplexing()) {
+        // Hand off to all queued up ConnectSM's.

Review comment:
       You don't know.  There is logic before requests are queued that tried to determine if the origin connection may be close to its limit of simultaneous active transactions (In the H2 case using a function of the current setting for max_concurrent_streams)




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficserver] shinrich commented on a change in pull request #7622: HTTP/2 to origin

Posted by GitBox <gi...@apache.org>.
shinrich commented on a change in pull request #7622:
URL: https://github.com/apache/trafficserver/pull/7622#discussion_r608067165



##########
File path: proxy/http/HttpSessionManager.cc
##########
@@ -190,9 +196,11 @@ ServerSessionPool::acquireSession(sockaddr const *addr, CryptoHash const &hostna
     }
     if (zret == HSM_DONE) {
       to_return = first;
-      HTTP_DECREMENT_DYN_STAT(http_pooled_server_connections_stat);
-      m_ip_pool.erase(first);
-      m_fqdn_pool.erase(to_return);
+      if (!to_return->is_multiplexing()) {
+        HTTP_DECREMENT_DYN_STAT(http_pooled_server_connections_stat);
+        m_ip_pool.erase(first);
+        m_fqdn_pool.erase(to_return);

Review comment:
       Good point, should be able to take advantage of the new removeSession method here.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficserver] SolidWallOfCode commented on a change in pull request #7622: HTTP/2 to origin

Posted by GitBox <gi...@apache.org>.
SolidWallOfCode commented on a change in pull request #7622:
URL: https://github.com/apache/trafficserver/pull/7622#discussion_r609083731



##########
File path: proxy/http/HttpSM.cc
##########
@@ -289,6 +278,160 @@ HttpVCTable::cleanup_all()
   }
 }
 
+class ConnectingEntry : public Continuation
+{
+public:
+  std::set<HttpSM *> _connect_sms;
+  NetVConnection *_netvc        = nullptr;
+  IOBufferReader *_netvc_reader = nullptr;
+  MIOBuffer *_netvc_read_buffer = nullptr;
+  std::string sni;
+  std::string cert_name;
+  IpEndpoint _ipaddr;
+  std::string hostname;
+  Action *_pending_action = nullptr;
+  NetVCOptions opt;
+
+  void remove_entry();
+  int state_http_server_open(int event, void *data);
+  static PoolableSession *create_server_session(HttpSM *root_sm, NetVConnection *netvc, MIOBuffer *netvc_read_buffer,
+                                                IOBufferReader *netvc_reader);
+};
+
+struct IpHelper {
+  size_t
+  operator()(IpEndpoint const &arg) const
+  {
+    return IpAddr{&arg.sa}.hash();
+  }
+  bool
+  operator()(IpEndpoint const &arg1, IpEndpoint const &arg2) const
+  {
+    return ats_ip_addr_port_eq(&arg1.sa, &arg2.sa);
+  }
+};
+using ConnectingIpPool = std::unordered_multimap<IpEndpoint, ConnectingEntry *, IpHelper>;
+
+class ConnectingPool
+{
+public:
+  ConnectingPool() {}
+  ConnectingIpPool m_ip_pool;
+};
+
+void
+initialize_thread_for_connecting_pools(EThread *thread)
+{
+  if (thread->connecting_pool == nullptr) {
+    thread->connecting_pool = new ConnectingPool();
+  }
+}
+
+int
+ConnectingEntry::state_http_server_open(int event, void *data)
+{
+  Debug("http_connect", "entered inside ConnectingEntry::state_http_server_open");
+
+  switch (event) {
+  case NET_EVENT_OPEN: {
+    _netvc                 = static_cast<NetVConnection *>(data);
+    UnixNetVConnection *vc = static_cast<UnixNetVConnection *>(_netvc);
+    ink_release_assert(_pending_action == nullptr || _pending_action->continuation == vc->get_action()->continuation);
+    _pending_action = nullptr;
+    Debug("http_connect", "ConnectingEntrysetting handler for TCP handshake");
+    // Just want to get a write-ready event so we know that the TCP handshake is complete.
+    // The buffer we create will be handed over to the eventually created server session
+    _netvc_read_buffer = new_MIOBuffer(HTTP_SERVER_RESP_HDR_BUFFER_INDEX);
+    _netvc_reader      = _netvc_read_buffer->alloc_reader();
+    _netvc->do_io_write(this, 1, _netvc_reader);
+    ink_release_assert(!_connect_sms.empty());
+    if (!_connect_sms.empty()) {
+      HttpSM *prime_connect_sm = *(_connect_sms.begin());
+      _netvc->set_inactivity_timeout(prime_connect_sm->get_server_connect_timeout());
+    }
+    ink_release_assert(_pending_action == nullptr);
+    return 0;
+  }
+  case VC_EVENT_READ_COMPLETE:
+  case VC_EVENT_WRITE_READY:
+  case VC_EVENT_WRITE_COMPLETE: {
+    Debug("http_connect", "Kick off %" PRId64 " state machines waiting for origin", _connect_sms.size());
+    this->remove_entry();
+    _netvc->do_io_write(nullptr, 0, nullptr);
+    if (!_connect_sms.empty()) {
+      auto prime_iter = _connect_sms.rbegin();
+      ink_release_assert(prime_iter != _connect_sms.rend());
+      PoolableSession *new_session = ConnectingEntry::create_server_session(*prime_iter, _netvc, _netvc_read_buffer, _netvc_reader);
+      _netvc                       = nullptr;
+
+      // Did we end up with a multiplexing session?
+      int count = 0;
+      if (new_session->is_multiplexing()) {
+        // Hand off to all queued up ConnectSM's.
+        while (!_connect_sms.empty()) {

Review comment:
       I'd be tempted to do
   ```
   for ( HttpSM* sm  : _connect_sms) {
     Debug("http_connect", "ConnectingEntry Pass along CONNECT_EVENT_TXN %d", count++);
     SCOPED_MUTEX_LOCK(lock, sm->mutex, this_ethread());
     sm->handleEvent(CONNECT_EVENT_TXN, new_session);
   }
   _connect_sms.clear();




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficserver] shinrich commented on pull request #7622: HTTP/2 to origin

Posted by GitBox <gi...@apache.org>.
shinrich commented on pull request #7622:
URL: https://github.com/apache/trafficserver/pull/7622#issuecomment-807577623


   [approve ci clang-analyzer]


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficserver] bneradt commented on pull request #7622: HTTP/2 to origin

Posted by GitBox <gi...@apache.org>.
bneradt commented on pull request #7622:
URL: https://github.com/apache/trafficserver/pull/7622#issuecomment-815119978


   [approve ci]


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficserver] keesspoelstra commented on a change in pull request #7622: HTTP/2 to origin

Posted by GitBox <gi...@apache.org>.
keesspoelstra commented on a change in pull request #7622:
URL: https://github.com/apache/trafficserver/pull/7622#discussion_r727282408



##########
File path: doc/admin-guide/files/records.config.en.rst
##########
@@ -3849,6 +3849,13 @@ Client-Related Configuration
 
    Enables (``1``) or disables (``0``) TLSv1_3 in the ATS client context. If not specified, enabled by default
 
+.. ts:cv:: CONFIG proxy.config.ssl.client.alpn_protocol STRING ""

Review comment:
       Typo, should be 
   `
   proxy.config.ssl.client.alpn_protocols
   `




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@trafficserver.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficserver] bneradt commented on pull request #7622: HTTP/2 to origin

Posted by GitBox <gi...@apache.org>.
bneradt commented on pull request #7622:
URL: https://github.com/apache/trafficserver/pull/7622#issuecomment-814197632


   [approve ci]


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficserver] shinrich commented on a change in pull request #7622: HTTP/2 to origin

Posted by GitBox <gi...@apache.org>.
shinrich commented on a change in pull request #7622:
URL: https://github.com/apache/trafficserver/pull/7622#discussion_r673511527



##########
File path: proxy/http/HttpSM.cc
##########
@@ -289,6 +277,160 @@ HttpVCTable::cleanup_all()
   }
 }
 
+class ConnectingEntry : public Continuation
+{
+public:
+  std::set<HttpSM *> _connect_sms;
+  NetVConnection *_netvc        = nullptr;
+  IOBufferReader *_netvc_reader = nullptr;
+  MIOBuffer *_netvc_read_buffer = nullptr;
+  std::string sni;
+  std::string cert_name;
+  IpEndpoint _ipaddr;
+  std::string hostname;
+  Action *_pending_action = nullptr;
+  NetVCOptions opt;
+
+  void remove_entry();
+  int state_http_server_open(int event, void *data);
+  static PoolableSession *create_server_session(HttpSM *root_sm, NetVConnection *netvc, MIOBuffer *netvc_read_buffer,
+                                                IOBufferReader *netvc_reader);
+};
+
+struct IpHelper {
+  size_t
+  operator()(IpEndpoint const &arg) const
+  {
+    return IpAddr{&arg.sa}.hash();
+  }
+  bool
+  operator()(IpEndpoint const &arg1, IpEndpoint const &arg2) const
+  {
+    return ats_ip_addr_port_eq(&arg1.sa, &arg2.sa);
+  }
+};
+using ConnectingIpPool = std::unordered_multimap<IpEndpoint, ConnectingEntry *, IpHelper>;
+
+class ConnectingPool
+{
+public:
+  ConnectingPool() {}
+  ConnectingIpPool m_ip_pool;
+};
+
+void
+initialize_thread_for_connecting_pools(EThread *thread)
+{
+  if (thread->connecting_pool == nullptr) {
+    thread->connecting_pool = new ConnectingPool();
+  }
+}
+
+int
+ConnectingEntry::state_http_server_open(int event, void *data)
+{
+  Debug("http_connect", "entered inside ConnectingEntry::state_http_server_open");
+
+  switch (event) {
+  case NET_EVENT_OPEN: {
+    _netvc                 = static_cast<NetVConnection *>(data);
+    UnixNetVConnection *vc = static_cast<UnixNetVConnection *>(_netvc);
+    ink_release_assert(_pending_action == nullptr || _pending_action->continuation == vc->get_action()->continuation);
+    _pending_action = nullptr;
+    Debug("http_connect", "ConnectingEntrysetting handler for TCP handshake");
+    // Just want to get a write-ready event so we know that the TCP handshake is complete.
+    // The buffer we create will be handed over to the eventually created server session
+    _netvc_read_buffer = new_MIOBuffer(HTTP_SERVER_RESP_HDR_BUFFER_INDEX);
+    _netvc_reader      = _netvc_read_buffer->alloc_reader();
+    _netvc->do_io_write(this, 1, _netvc_reader);
+    ink_release_assert(!_connect_sms.empty());
+    if (!_connect_sms.empty()) {
+      HttpSM *prime_connect_sm = *(_connect_sms.begin());
+      _netvc->set_inactivity_timeout(prime_connect_sm->get_server_connect_timeout());
+    }
+    ink_release_assert(_pending_action == nullptr);
+    return 0;
+  }
+  case VC_EVENT_READ_COMPLETE:
+  case VC_EVENT_WRITE_READY:
+  case VC_EVENT_WRITE_COMPLETE: {
+    Debug("http_connect", "Kick off %" PRId64 " state machines waiting for origin", _connect_sms.size());
+    this->remove_entry();
+    _netvc->do_io_write(nullptr, 0, nullptr);
+    if (!_connect_sms.empty()) {
+      auto prime_iter = _connect_sms.rbegin();
+      ink_release_assert(prime_iter != _connect_sms.rend());
+      PoolableSession *new_session = (*prime_iter)->create_server_session(_netvc, _netvc_read_buffer, _netvc_reader);
+      _netvc                       = nullptr;
+
+      // Did we end up with a multiplexing session?
+      int count = 0;
+      if (new_session->is_multiplexing()) {
+        // Hand off to all queued up ConnectSM's.

Review comment:
       Assuming 5 simultaneous (overlapping requests).  2 transactions will be made against the first HTTP/2 connection.  Then a new session will be created and 2 more transactions will be made there.  Then a third session session will be opened.
   
   Actually, I think the code will pull new transactions under the reported max_concurrent_streams setting. 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@trafficserver.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficserver] maskit commented on pull request #7622: HTTP/2 to origin

Posted by GitBox <gi...@apache.org>.
maskit commented on pull request #7622:
URL: https://github.com/apache/trafficserver/pull/7622#issuecomment-840179475


   It seem like we could split this PR into a couple of separated PRs like:
   - To use server_txn instead of server_session in HttpSM
   - Changes around HttpSessionManager
   - Trailer header support
   - Something around HttpTunnel
   - ALPN setting for outbound connection
   - H2 class restructure
   - H2 to Origin support
   
   Although I'm not sure how much it would be hard and some may not be decoupled, the basic idea is to make this PR only adding H2 to Origin support. We need to make a lot of changes as preparation for supporting multiplexing server session, and some of them could affect existing features. I understand 9.1 is already branched and we basically should not add changes any more, but having some of the changes above on 9.1 would debugging easier than adding all of them on 9.2.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficserver] maskit commented on a change in pull request #7622: HTTP/2 to origin

Posted by GitBox <gi...@apache.org>.
maskit commented on a change in pull request #7622:
URL: https://github.com/apache/trafficserver/pull/7622#discussion_r603038536



##########
File path: iocore/net/UnixNetVConnection.cc
##########
@@ -431,6 +432,9 @@ write_to_net_io(NetHandler *nh, UnixNetVConnection *vc, EThread *thread)
   if (towrite != ntodo && buf.writer()->write_avail()) {
     if (write_signal_and_update(VC_EVENT_WRITE_READY, vc) != EVENT_CONT) {
       return;
+    } else if (c != s->vio.cont) { /* The write vio was updated in the handler */

Review comment:
       Nitpick: Do we want variable `c`?

##########
File path: proxy/http/HttpTransact.h
##########
@@ -907,6 +901,16 @@ class HttpTransact
     }
 
     ProxyProtocol pp_info;

Review comment:
       I want an empty line here. I misread this as a return type.

##########
File path: proxy/logging/LogAccess.cc
##########
@@ -1782,6 +1782,40 @@ LogAccess::marshal_client_req_protocol_version(char *buf)
   return len;
 }
 
+/*-------------------------------------------------------------------------
+  -------------------------------------------------------------------------*/
+
+int
+LogAccess::marshal_server_req_protocol_version(char *buf)

Review comment:
       Not a big deal, but I think this part is reviewable even if a reviewer is not so familiar with H2 and server session. I'd be happy to add the new log field beforehand as a preparatory step.

##########
File path: tests/gold_tests/continuations/openclose.test.py
##########
@@ -51,7 +52,7 @@
         ts.Variables.port, server.Variables.Port)
 )
 
-cmd = 'curl -vs -H "host:oc.test" http://127.0.0.1:{0}'.format(ts.Variables.port)
+cmd = 'curl -H "Connection: close" -vs -H "host:oc.test" http://127.0.0.1:{0}'.format(ts.Variables.port)

Review comment:
       I'd like to make this change before landing H2 to Origin. That would ensure H2 from Client and H1 to Origin are not broken.

##########
File path: proxy/http/HttpTransact.h
##########
@@ -582,7 +576,7 @@ class HttpTransact
     ConnectionAttributes *server               = nullptr;
     ink_time_t now                             = 0;
     ServerState_t state                        = STATE_UNDEFINED;
-    unsigned attempts                          = 1;
+    unsigned attempts                          = 0;

Review comment:
       Is this intentional change?

##########
File path: proxy/http/HttpTransact.cc
##########
@@ -3969,47 +3977,21 @@ HttpTransact::handle_forward_server_connection_open(State *s)
   TxnDebug("http_seq", "[HttpTransact::handle_server_connection_open] ");
   ink_release_assert(s->current.state == CONNECTION_ALIVE);
 
-  if (s->hdr_info.server_response.version_get() == HTTPVersion(0, 9)) {
-    TxnDebug("http_trans", "[hfsco] server sent 0.9 response, reading...");
-    build_response(s, &s->hdr_info.client_response, s->client_info.http_version, HTTP_STATUS_OK, "Connection Established");
-
-    s->client_info.keep_alive = HTTP_NO_KEEPALIVE;
-    s->cache_info.action      = CACHE_DO_NO_ACTION;
-    s->next_action            = SM_ACTION_SERVER_READ;
-    return;
-
-  } else if (s->hdr_info.server_response.version_get() == HTTPVersion(1, 0)) {
-    if (s->current.server->http_version == HTTPVersion(0, 9)) {
-      // update_hostdb_to_indicate_server_version_is_1_0
-      s->updated_server_version = HostDBApplicationInfo::HTTP_VERSION_10;
-    } else if (s->current.server->http_version == HTTPVersion(1, 1)) {
-      // update_hostdb_to_indicate_server_version_is_1_0
-      s->updated_server_version = HostDBApplicationInfo::HTTP_VERSION_10;
-    } else {
-      // dont update the hostdb. let us try again with what we currently think.
-    }
-  } else if (s->hdr_info.server_response.version_get() == HTTPVersion(1, 1)) {
-    if (s->current.server->http_version == HTTPVersion(0, 9)) {
-      // update_hostdb_to_indicate_server_version_is_1_1
-      s->updated_server_version = HostDBApplicationInfo::HTTP_VERSION_11;
-    } else if (s->current.server->http_version == HTTPVersion(1, 0)) {
-      // update_hostdb_to_indicate_server_version_is_1_1
-      s->updated_server_version = HostDBApplicationInfo::HTTP_VERSION_11;
-    } else {
-      // dont update the hostdb. let us try again with what we currently think.
-    }
-  } else {
-    // dont update the hostdb. let us try again with what we currently think.
+  HostDBApplicationInfo::HttpVersion real_version = s->state_machine->server_txn->get_version(s->hdr_info.server_response);
+  if (real_version != s->host_db_info.app.http_data.http_version) {
+    TxnDebug("http_trans", "Update hostdb history of server HTTP version 0x%x", real_version);
+    // Need to update the hostdb
+    s->updated_server_version = real_version;

Review comment:
       Seems like we can do this without H2 to Origin.

##########
File path: proxy/http/HttpSessionManager.cc
##########
@@ -190,9 +196,11 @@ ServerSessionPool::acquireSession(sockaddr const *addr, CryptoHash const &hostna
     }
     if (zret == HSM_DONE) {
       to_return = first;
-      HTTP_DECREMENT_DYN_STAT(http_pooled_server_connections_stat);
-      m_ip_pool.erase(first);
-      m_fqdn_pool.erase(to_return);
+      if (!to_return->is_multiplexing()) {
+        HTTP_DECREMENT_DYN_STAT(http_pooled_server_connections_stat);
+        m_ip_pool.erase(first);
+        m_fqdn_pool.erase(to_return);

Review comment:
       Why don't you call removeSession? It seems like removeSession just does these three things with logging.
   
   I know H2 session calls removeSession directly somehow, but I think we could add removeSession separately for current code.

##########
File path: proxy/http/HttpSessionManager.h
##########
@@ -110,7 +113,7 @@ class HttpSessionManager
 public:
   HttpSessionManager() {}
   ~HttpSessionManager() {}
-  HSMresult_t acquire_session(Continuation *cont, sockaddr const *addr, const char *hostname, ProxyTransaction *ua_txn, HttpSM *sm);
+  HSMresult_t acquire_session(HttpSM *cont, sockaddr const *addr, const char *hostname, ProxyTransaction *ua_txn);

Review comment:
       Seems like a general cleanup.

##########
File path: proxy/http/HttpTransact.cc
##########
@@ -1826,10 +1830,14 @@ HttpTransact::ReDNSRoundRobin(State *s)
     s->next_action = how_to_open_connection(s);
   } else {
     // Our ReDNS failed so output the DNS failure error message
-    build_error_response(s, HTTP_STATUS_BAD_GATEWAY, "Cannot find server.", "connect#dns_failed");
+    // Set to internal server error so later logging will pick up SQUID_LOG_ERR_DNS_FAIL

Review comment:
       Not 100% sure if we can do this separately, but it doesn't seem like a change for H2 to Origin.

##########
File path: proxy/http2/HTTP2.cc
##########
@@ -729,7 +766,7 @@ http2_decode_header_blocks(HTTPHdr *hdr, const uint8_t *buf_start, const uint32_
   if (hdr->field_find(MIME_FIELD_CONNECTION, MIME_LEN_CONNECTION) != nullptr ||
       hdr->field_find(MIME_FIELD_KEEP_ALIVE, MIME_LEN_KEEP_ALIVE) != nullptr ||
       hdr->field_find(MIME_FIELD_PROXY_CONNECTION, MIME_LEN_PROXY_CONNECTION) != nullptr ||
-      hdr->field_find(MIME_FIELD_TRANSFER_ENCODING, MIME_LEN_TRANSFER_ENCODING) != nullptr ||
+      // hdr->field_find(MIME_FIELD_TRANSFER_ENCODING, MIME_LEN_TRANSFER_ENCODING) != nullptr ||

Review comment:
       Assuming this is related to gRPC. If normal H2 requests work fine with this line, I'd suggest making this change separately after this big change. That would allow us to back out only gRPC support but not entire H2 to Origin support.

##########
File path: proxy/http/HttpTransact.cc
##########
@@ -3704,26 +3717,16 @@ HttpTransact::handle_response_from_server(State *s)
   case OUTBOUND_CONGESTION:
     TxnDebug("http_trans", "[handle_response_from_server] Error. congestion control -- congested.");
     SET_VIA_STRING(VIA_DETAIL_SERVER_CONNECT, VIA_DETAIL_SERVER_FAILURE);
-    s->current.server->set_connect_fail(EUSERS); // too many users
+    s->set_connect_fail(EUSERS); // too many users
     handle_server_connection_not_open(s);
     break;
   case OPEN_RAW_ERROR:
-  /* fall through */

Review comment:
       I don't have strong opinion on this, but some may prefer to have `[[fallthrough]];`, which is available since C++17.
   In any cases, we could do this separately.

##########
File path: proxy/http/HttpTunnel.cc
##########
@@ -1161,6 +1165,8 @@ HttpTunnel::producer_handler(int event, HttpTunnelProducer *p)
     // If the write completes on the stack (as it can for http2), then
     // consumer could have called back by this point.  Must treat this as
     // a regular read complete (falling through to the following cases).
+    p->bytes_read = p->init_bytes_done;
+    // FALLTHROUGH

Review comment:
       Nitpick: If I feel like this comment is needed, I use `[[fallthrough]];`.

##########
File path: src/tscore/BufferWriterFormat.cc
##########
@@ -896,8 +897,12 @@ bwformat(BufferWriter &w, BWFSpec const &spec, bwf::Errno const &e)
   if (spec.has_numeric_type()) {              // if numeric type, print just the numeric part.
     w.print(number_fmt, e._e);
   } else {
-    w.write(short_name(e._e));
-    w.write(strerror(e._e));
+    if (e._e < 0) {

Review comment:
       Definitely not a part of H2 to Origin.

##########
File path: tests/gold_tests/redirect/redirect_actions.test.py
##########
@@ -184,7 +184,7 @@ class ActionE(Enum):
     Follow = {'config': 'follow', 'expectedStatusLine': 'HTTP/1.1 204 No Content\r\n'}
 
     # Added to test failure modes.
-    Break = {'expectedStatusLine': 'HTTP/1.1 502 Cannot find server.\r\n'}
+    Break = {'expectedStatusLine': 'HTTP/1.1 500 Cannot find server.\r\n'}

Review comment:
       Is this related to H2 to Origin?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficserver] shinrich commented on a change in pull request #7622: HTTP/2 to origin

Posted by GitBox <gi...@apache.org>.
shinrich commented on a change in pull request #7622:
URL: https://github.com/apache/trafficserver/pull/7622#discussion_r602577589



##########
File path: proxy/http/HttpSessionManager.cc
##########
@@ -147,28 +149,33 @@ ServerSessionPool::acquireSession(sockaddr const *addr, CryptoHash const &hostna
   to_return        = nullptr;
 
   if ((TS_SERVER_SESSION_SHARING_MATCH_MASK_HOSTONLY & match_style) && !(TS_SERVER_SESSION_SHARING_MATCH_MASK_IP & match_style)) {
+    Debug("http_ss", "Search for host name only not IP.  Pool size %" PRId64, m_fqdn_pool.count());
     // This is broken out because only in this case do we check the host hash first. The range must be checked
     // to verify an upstream that matches port and SNI name is selected. Walk backwards to select oldest.
     in_port_t port = ats_ip_port_cast(addr);
-    FQDNTable::iterator first, last;
-    // FreeBSD/clang++ bug workaround: explicit cast to super type to make overload work. Not needed on Fedora27 nor gcc.
-    // Not fixed on FreeBSD as of llvm 6.0.1.
-    std::tie(first, last) = static_cast<const decltype(m_fqdn_pool)::range::super_type &>(m_fqdn_pool.equal_range(hostname_hash));
-    while (last != first) {
-      --last;
-      if (port == ats_ip_port_cast(last->get_remote_addr()) &&
-          (!(match_style & TS_SERVER_SESSION_SHARING_MATCH_MASK_SNI) || validate_sni(sm, last->get_netvc())) &&
-          (!(match_style & TS_SERVER_SESSION_SHARING_MATCH_MASK_HOSTSNISYNC) || validate_host_sni(sm, last->get_netvc())) &&
-          (!(match_style & TS_SERVER_SESSION_SHARING_MATCH_MASK_CERT) || validate_cert(sm, last->get_netvc()))) {
+    auto first     = m_fqdn_pool.find(hostname_hash);
+    while (first != m_fqdn_pool.end() && first->hostname_hash == hostname_hash) {
+      Debug("http_ss", "Compare port 0x%x against 0x%x", port, ats_ip_port_cast(first->get_remote_addr()));
+      if (port == ats_ip_port_cast(first->get_remote_addr()) &&
+          (!(match_style & TS_SERVER_SESSION_SHARING_MATCH_MASK_SNI) || validate_sni(sm, first->get_netvc())) &&
+          (!(match_style & TS_SERVER_SESSION_SHARING_MATCH_MASK_HOSTSNISYNC) || validate_host_sni(sm, first->get_netvc())) &&
+          (!(match_style & TS_SERVER_SESSION_SHARING_MATCH_MASK_CERT) || validate_cert(sm, first->get_netvc()))) {

Review comment:
       I think these changes are due to a rebase error in a previous commit.  I set up PR #7631 to track.  That should definitely land independent of these changes.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficserver] shinrich commented on a change in pull request #7622: HTTP/2 to origin

Posted by GitBox <gi...@apache.org>.
shinrich commented on a change in pull request #7622:
URL: https://github.com/apache/trafficserver/pull/7622#discussion_r602366127



##########
File path: proxy/hdrs/HTTP.h
##########
@@ -580,6 +580,27 @@ class HTTPHdr : public MIMEHdr
   const char *path_get(int *length ///< Storage for path length.
   );
 
+  /** Get the URL matrix params.
+      This is a reference, not allocated.
+      @return A pointer to the matrix params or @c NULL if there is no valid URL.
+  */
+  const char *params_get(int *length ///< Storage for param length.

Review comment:
       Separated into PR #7628




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficserver] maskit commented on a change in pull request #7622: HTTP/2 to origin

Posted by GitBox <gi...@apache.org>.
maskit commented on a change in pull request #7622:
URL: https://github.com/apache/trafficserver/pull/7622#discussion_r647925324



##########
File path: proxy/http/HttpSM.cc
##########
@@ -289,6 +277,160 @@ HttpVCTable::cleanup_all()
   }
 }
 
+class ConnectingEntry : public Continuation
+{
+public:
+  std::set<HttpSM *> _connect_sms;
+  NetVConnection *_netvc        = nullptr;
+  IOBufferReader *_netvc_reader = nullptr;
+  MIOBuffer *_netvc_read_buffer = nullptr;
+  std::string sni;
+  std::string cert_name;
+  IpEndpoint _ipaddr;
+  std::string hostname;
+  Action *_pending_action = nullptr;
+  NetVCOptions opt;
+
+  void remove_entry();
+  int state_http_server_open(int event, void *data);
+  static PoolableSession *create_server_session(HttpSM *root_sm, NetVConnection *netvc, MIOBuffer *netvc_read_buffer,
+                                                IOBufferReader *netvc_reader);
+};
+
+struct IpHelper {
+  size_t
+  operator()(IpEndpoint const &arg) const
+  {
+    return IpAddr{&arg.sa}.hash();
+  }
+  bool
+  operator()(IpEndpoint const &arg1, IpEndpoint const &arg2) const
+  {
+    return ats_ip_addr_port_eq(&arg1.sa, &arg2.sa);
+  }
+};
+using ConnectingIpPool = std::unordered_multimap<IpEndpoint, ConnectingEntry *, IpHelper>;
+
+class ConnectingPool
+{
+public:
+  ConnectingPool() {}
+  ConnectingIpPool m_ip_pool;
+};
+
+void
+initialize_thread_for_connecting_pools(EThread *thread)
+{
+  if (thread->connecting_pool == nullptr) {
+    thread->connecting_pool = new ConnectingPool();
+  }
+}
+
+int
+ConnectingEntry::state_http_server_open(int event, void *data)
+{
+  Debug("http_connect", "entered inside ConnectingEntry::state_http_server_open");
+
+  switch (event) {
+  case NET_EVENT_OPEN: {
+    _netvc                 = static_cast<NetVConnection *>(data);
+    UnixNetVConnection *vc = static_cast<UnixNetVConnection *>(_netvc);
+    ink_release_assert(_pending_action == nullptr || _pending_action->continuation == vc->get_action()->continuation);
+    _pending_action = nullptr;
+    Debug("http_connect", "ConnectingEntrysetting handler for TCP handshake");
+    // Just want to get a write-ready event so we know that the TCP handshake is complete.
+    // The buffer we create will be handed over to the eventually created server session
+    _netvc_read_buffer = new_MIOBuffer(HTTP_SERVER_RESP_HDR_BUFFER_INDEX);
+    _netvc_reader      = _netvc_read_buffer->alloc_reader();
+    _netvc->do_io_write(this, 1, _netvc_reader);
+    ink_release_assert(!_connect_sms.empty());
+    if (!_connect_sms.empty()) {
+      HttpSM *prime_connect_sm = *(_connect_sms.begin());
+      _netvc->set_inactivity_timeout(prime_connect_sm->get_server_connect_timeout());
+    }
+    ink_release_assert(_pending_action == nullptr);
+    return 0;
+  }
+  case VC_EVENT_READ_COMPLETE:
+  case VC_EVENT_WRITE_READY:
+  case VC_EVENT_WRITE_COMPLETE: {
+    Debug("http_connect", "Kick off %" PRId64 " state machines waiting for origin", _connect_sms.size());
+    this->remove_entry();
+    _netvc->do_io_write(nullptr, 0, nullptr);
+    if (!_connect_sms.empty()) {
+      auto prime_iter = _connect_sms.rbegin();
+      ink_release_assert(prime_iter != _connect_sms.rend());
+      PoolableSession *new_session = (*prime_iter)->create_server_session(_netvc, _netvc_read_buffer, _netvc_reader);
+      _netvc                       = nullptr;
+
+      // Did we end up with a multiplexing session?
+      int count = 0;
+      if (new_session->is_multiplexing()) {
+        // Hand off to all queued up ConnectSM's.

Review comment:
       > I think we will want to be able to create multiple sessions to the same server. In the case of virtual IP's it will allow for better load distribution. Also relying on a single TCP connection to origin may worry some.
   
   I feel the same way, but RFC 7540 says:
   
   >    A client MAY open multiple connections to the same IP address and TCP
      port using different Server Name Indication [TLS-EXT] values or to
      provide different TLS client certificates but SHOULD avoid creating
      multiple connections with the same configuration.
   
   If we can open as many connections as we want, we will eventually make multiple connections to the same server through VIP, and then max_concurrent_stream doesn't make much sense.
   
   ---
   
   I'm not sure if I understand the entire process. Let's say we got 5 requests and all the requests will be send to the same origin server (or origin servers that have the same VIP). The origin server supports H2, and max_concurrent_stream is 2. What will happen?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficserver] shinrich commented on a change in pull request #7622: HTTP/2 to origin

Posted by GitBox <gi...@apache.org>.
shinrich commented on a change in pull request #7622:
URL: https://github.com/apache/trafficserver/pull/7622#discussion_r602383428



##########
File path: proxy/ProxySession.h
##########
@@ -127,6 +127,7 @@ class ProxySession : public VConnection, public PluginUserArgs<TS_USER_ARGS_SSN>
 
   // Non-Virtual Methods
   NetVConnection *get_netvc() const;
+  virtual void set_netvc(NetVConnection *netvc);

Review comment:
       Agreed.  Defining both in ProxySession and PoolableSession is confused..




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficserver] shinrich commented on pull request #7622: HTTP/2 to origin

Posted by GitBox <gi...@apache.org>.
shinrich commented on pull request #7622:
URL: https://github.com/apache/trafficserver/pull/7622#issuecomment-950395999


   Rebased against 10-Dev branch.  I need to close this PR and create another against 10-Dev.  Will coordinate with @bneradt to do that.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@trafficserver.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficserver] shinrich commented on a change in pull request #7622: HTTP/2 to origin

Posted by GitBox <gi...@apache.org>.
shinrich commented on a change in pull request #7622:
URL: https://github.com/apache/trafficserver/pull/7622#discussion_r607208232



##########
File path: proxy/http/HttpTransact.h
##########
@@ -582,7 +576,7 @@ class HttpTransact
     ConnectionAttributes *server               = nullptr;
     ink_time_t now                             = 0;
     ServerState_t state                        = STATE_UNDEFINED;
-    unsigned attempts                          = 1;
+    unsigned attempts                          = 0;

Review comment:
       yes, this was intentional.  And is part of PR #7580. 
   
   Without this change the logging on first failure was not working.  Really need to get someone to look at PR #7580.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficserver] maskit commented on a change in pull request #7622: HTTP/2 to origin

Posted by GitBox <gi...@apache.org>.
maskit commented on a change in pull request #7622:
URL: https://github.com/apache/trafficserver/pull/7622#discussion_r645334503



##########
File path: proxy/http/HttpSessionManager.cc
##########
@@ -165,9 +165,13 @@ ServerSessionPool::acquireSession(sockaddr const *addr, CryptoHash const &hostna
     }
     if (zret == HSM_DONE) {
       to_return = first;
-      this->removeSession(to_return);
-    } else if (first != m_fqdn_pool.end()) {
-      Debug("http_ss", "Failed find entry due to name mismatch %s", sm->t_state.current.server->name);
+      if (!to_return->is_multiplexing()) {
+        this->removeSession(to_return);
+      }
+    } else {
+      if (first != m_fqdn_pool.end()) {

Review comment:
       else if




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficserver] shinrich commented on a change in pull request #7622: HTTP/2 to origin

Posted by GitBox <gi...@apache.org>.
shinrich commented on a change in pull request #7622:
URL: https://github.com/apache/trafficserver/pull/7622#discussion_r603314448



##########
File path: proxy/ProxySession.cc
##########
@@ -106,7 +106,7 @@ ProxySession::state_api_callout(int event, void *data)
       if (!lock.is_locked()) {
         SET_HANDLER(&ProxySession::state_api_callout);
         if (!schedule_event) { // Don't bother if there is already one
-          schedule_event = mutex->thread_holding->schedule_in(this, HRTIME_MSECONDS(10));
+          schedule_event = this_ethread()->schedule_in(this, HRTIME_MSECONDS(10));

Review comment:
       Backed out this change for now.  Not at all critical.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficserver] bneradt removed a comment on pull request #7622: HTTP/2 to origin

Posted by GitBox <gi...@apache.org>.
bneradt removed a comment on pull request #7622:
URL: https://github.com/apache/trafficserver/pull/7622#issuecomment-815166443


   [approve ci]


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficserver] maskit commented on a change in pull request #7622: HTTP/2 to origin

Posted by GitBox <gi...@apache.org>.
maskit commented on a change in pull request #7622:
URL: https://github.com/apache/trafficserver/pull/7622#discussion_r630806172



##########
File path: src/traffic_server/InkAPI.cc
##########
@@ -4934,12 +4934,11 @@ TSHttpSsnClientVConnGet(TSHttpSsn ssnp)
 TSVConn
 TSHttpSsnServerVConnGet(TSHttpSsn ssnp)
 {
-  TSVConn vconn       = nullptr;
   PoolableSession *ss = reinterpret_cast<PoolableSession *>(ssnp);
   if (ss != nullptr) {
-    vconn = reinterpret_cast<TSVConn>(ss->get_netvc());
+    return reinterpret_cast<TSVConn>(ss->get_netvc());

Review comment:
       This is unnecessary for H2 to Origin, and inconsistent with a similar code in `TSHttpTxnServerVConnGet`.
   




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficserver] shinrich commented on a change in pull request #7622: HTTP/2 to origin

Posted by GitBox <gi...@apache.org>.
shinrich commented on a change in pull request #7622:
URL: https://github.com/apache/trafficserver/pull/7622#discussion_r602377433



##########
File path: proxy/PoolableSession.h
##########
@@ -72,16 +73,29 @@ class PoolableSession : public ProxySession
   TSServerSessionSharingMatchMask sharing_match = TS_SERVER_SESSION_SHARING_MATCH_MASK_NONE;
   TSServerSessionSharingPoolType sharing_pool   = TS_SERVER_SESSION_SHARING_POOL_GLOBAL;
 
-  // Keep track of connection limiting and a pointer to the
-  // singleton that keeps track of the connection counts.
-  OutboundConnTrack::Group *conn_track_group = nullptr;
+  void enable_outbound_connection_tracking(OutboundConnTrack::Group *group);
+  void release_outbound_comnection_tracking();

Review comment:
       Oops!  I blame my dying keyboard :-)




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficserver] SolidWallOfCode commented on a change in pull request #7622: HTTP/2 to origin

Posted by GitBox <gi...@apache.org>.
SolidWallOfCode commented on a change in pull request #7622:
URL: https://github.com/apache/trafficserver/pull/7622#discussion_r609079344



##########
File path: proxy/http/HttpSM.cc
##########
@@ -289,6 +278,160 @@ HttpVCTable::cleanup_all()
   }
 }
 
+class ConnectingEntry : public Continuation
+{
+public:
+  std::set<HttpSM *> _connect_sms;
+  NetVConnection *_netvc        = nullptr;
+  IOBufferReader *_netvc_reader = nullptr;
+  MIOBuffer *_netvc_read_buffer = nullptr;
+  std::string sni;
+  std::string cert_name;
+  IpEndpoint _ipaddr;
+  std::string hostname;
+  Action *_pending_action = nullptr;
+  NetVCOptions opt;
+
+  void remove_entry();
+  int state_http_server_open(int event, void *data);
+  static PoolableSession *create_server_session(HttpSM *root_sm, NetVConnection *netvc, MIOBuffer *netvc_read_buffer,
+                                                IOBufferReader *netvc_reader);
+};
+
+struct IpHelper {
+  size_t
+  operator()(IpEndpoint const &arg) const
+  {
+    return IpAddr{&arg.sa}.hash();
+  }
+  bool
+  operator()(IpEndpoint const &arg1, IpEndpoint const &arg2) const
+  {
+    return ats_ip_addr_port_eq(&arg1.sa, &arg2.sa);
+  }
+};
+using ConnectingIpPool = std::unordered_multimap<IpEndpoint, ConnectingEntry *, IpHelper>;
+
+class ConnectingPool
+{
+public:
+  ConnectingPool() {}
+  ConnectingIpPool m_ip_pool;
+};
+
+void
+initialize_thread_for_connecting_pools(EThread *thread)
+{
+  if (thread->connecting_pool == nullptr) {
+    thread->connecting_pool = new ConnectingPool();
+  }
+}
+
+int
+ConnectingEntry::state_http_server_open(int event, void *data)
+{
+  Debug("http_connect", "entered inside ConnectingEntry::state_http_server_open");
+
+  switch (event) {
+  case NET_EVENT_OPEN: {
+    _netvc                 = static_cast<NetVConnection *>(data);
+    UnixNetVConnection *vc = static_cast<UnixNetVConnection *>(_netvc);
+    ink_release_assert(_pending_action == nullptr || _pending_action->continuation == vc->get_action()->continuation);
+    _pending_action = nullptr;
+    Debug("http_connect", "ConnectingEntrysetting handler for TCP handshake");
+    // Just want to get a write-ready event so we know that the TCP handshake is complete.
+    // The buffer we create will be handed over to the eventually created server session
+    _netvc_read_buffer = new_MIOBuffer(HTTP_SERVER_RESP_HDR_BUFFER_INDEX);
+    _netvc_reader      = _netvc_read_buffer->alloc_reader();
+    _netvc->do_io_write(this, 1, _netvc_reader);
+    ink_release_assert(!_connect_sms.empty());
+    if (!_connect_sms.empty()) {
+      HttpSM *prime_connect_sm = *(_connect_sms.begin());
+      _netvc->set_inactivity_timeout(prime_connect_sm->get_server_connect_timeout());
+    }
+    ink_release_assert(_pending_action == nullptr);
+    return 0;
+  }
+  case VC_EVENT_READ_COMPLETE:
+  case VC_EVENT_WRITE_READY:
+  case VC_EVENT_WRITE_COMPLETE: {
+    Debug("http_connect", "Kick off %" PRId64 " state machines waiting for origin", _connect_sms.size());
+    this->remove_entry();
+    _netvc->do_io_write(nullptr, 0, nullptr);
+    if (!_connect_sms.empty()) {
+      auto prime_iter = _connect_sms.rbegin();
+      ink_release_assert(prime_iter != _connect_sms.rend());
+      PoolableSession *new_session = ConnectingEntry::create_server_session(*prime_iter, _netvc, _netvc_read_buffer, _netvc_reader);
+      _netvc                       = nullptr;
+
+      // Did we end up with a multiplexing session?
+      int count = 0;
+      if (new_session->is_multiplexing()) {
+        // Hand off to all queued up ConnectSM's.
+        while (!_connect_sms.empty()) {
+          Debug("http_connect", "ConnectingEntry Pass along CONNECT_EVENT_TXN %d", count++);
+          auto entry = _connect_sms.begin();
+
+          SCOPED_MUTEX_LOCK(lock, (*entry)->mutex, this_ethread());
+          (*entry)->handleEvent(CONNECT_EVENT_TXN, new_session);
+          _connect_sms.erase(entry);

Review comment:
       Is this thread safe? `_connect_sms.erase()` isn't thread safe and not protected by `(*entry)->mutex`.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficserver] shinrich commented on pull request #7622: HTTP/2 to origin

Posted by GitBox <gi...@apache.org>.
shinrich commented on pull request #7622:
URL: https://github.com/apache/trafficserver/pull/7622#issuecomment-837441567


   [approve ci autest]


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficserver] bneradt removed a comment on pull request #7622: HTTP/2 to origin

Posted by GitBox <gi...@apache.org>.
bneradt removed a comment on pull request #7622:
URL: https://github.com/apache/trafficserver/pull/7622#issuecomment-815120875






-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficserver] shinrich commented on a change in pull request #7622: HTTP/2 to origin

Posted by GitBox <gi...@apache.org>.
shinrich commented on a change in pull request #7622:
URL: https://github.com/apache/trafficserver/pull/7622#discussion_r607192036



##########
File path: tests/gold_tests/continuations/openclose.test.py
##########
@@ -51,7 +52,7 @@
         ts.Variables.port, server.Variables.Port)
 )
 
-cmd = 'curl -vs -H "host:oc.test" http://127.0.0.1:{0}'.format(ts.Variables.port)
+cmd = 'curl -H "Connection: close" -vs -H "host:oc.test" http://127.0.0.1:{0}'.format(ts.Variables.port)

Review comment:
       PR #7679




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficserver] shinrich commented on a change in pull request #7622: HTTP/2 to origin

Posted by GitBox <gi...@apache.org>.
shinrich commented on a change in pull request #7622:
URL: https://github.com/apache/trafficserver/pull/7622#discussion_r602386442



##########
File path: proxy/ProxyTransaction.cc
##########
@@ -220,3 +220,78 @@ ProxyTransaction::has_request_body(int64_t request_content_length, bool is_chunk
 {
   return request_content_length > 0 || is_chunked;
 }
+
+bool
+ProxyTransaction::is_read_closed() const
+{
+  return false;
+}
+
+bool
+ProxyTransaction::expect_send_trailer() const
+{
+  return false;
+}
+
+void
+ProxyTransaction::set_expect_send_trailer()
+{
+}
+bool
+ProxyTransaction::expect_receive_trailer() const
+{
+  return false;
+}
+
+void
+ProxyTransaction::set_expect_receive_trailer()
+{
+}
+
+void
+ProxyTransaction::attach_transaction(HttpSM *attach_sm)
+{
+  _sm = attach_sm;
+}
+
+void
+ProxyTransaction::release()
+{
+  HttpTxnDebug("[%" PRId64 "] session released by sm [%" PRId64 "]", _proxy_ssn ? _proxy_ssn->connection_id() : 0,
+               _sm ? _sm->sm_id : 0);
+
+  this->decrement_transactions_stat();
+
+  // Pass along the release to the session
+  if (_proxy_ssn) {
+    _proxy_ssn->release(this);
+  }
+}
+
+HostDBApplicationInfo::HttpVersion
+ProxyTransaction::get_version(HTTPHdr &hdr) const
+{
+  if (hdr.version_get() == HTTPVersion(1, 1)) {
+    return HostDBApplicationInfo::HTTP_VERSION_11;
+  } else if (hdr.version_get() == HTTPVersion(1, 0)) {
+    return HostDBApplicationInfo::HTTP_VERSION_10;
+  } else {
+    return HostDBApplicationInfo::HTTP_VERSION_09;
+  }
+}
+
+bool
+ProxyTransaction::allow_half_open() const
+{
+  return false;
+}
+
+void
+ProxyTransaction::increment_transactions_stat()

Review comment:
       No good reason.  Seemed like a good idea at the time I suppose.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficserver] maskit commented on a change in pull request #7622: HTTP/2 to origin

Posted by GitBox <gi...@apache.org>.
maskit commented on a change in pull request #7622:
URL: https://github.com/apache/trafficserver/pull/7622#discussion_r645340384



##########
File path: proxy/logging/LogAccess.cc
##########
@@ -1277,7 +1277,7 @@ LogAccess::marshal_client_sni_server_name(char *buf)
   std::string_view server_name = "";
 
   if (m_http_sm) {
-    auto txn = m_http_sm->get_ua_txn();
+    auto txn = m_http_sm->ua_txn;

Review comment:
       Let's use the getter.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficserver] shinrich commented on a change in pull request #7622: HTTP/2 to origin

Posted by GitBox <gi...@apache.org>.
shinrich commented on a change in pull request #7622:
URL: https://github.com/apache/trafficserver/pull/7622#discussion_r612586692



##########
File path: proxy/http/HttpSM.cc
##########
@@ -344,7 +487,7 @@ HttpSM::get_server_connect_timeout()
 {
   ink_hrtime retval = 0;
   if (t_state.api_txn_connect_timeout_value != -1) {
-    retval = HRTIME_MSECONDS(t_state.api_txn_connect_timeout_value);
+    retval = HRTIME_SECONDS(t_state.api_txn_connect_timeout_value);

Review comment:
       Good point.  Probably rebase fix up error.  Will fix.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficserver] shinrich commented on a change in pull request #7622: HTTP/2 to origin

Posted by GitBox <gi...@apache.org>.
shinrich commented on a change in pull request #7622:
URL: https://github.com/apache/trafficserver/pull/7622#discussion_r612595059



##########
File path: proxy/http/HttpTransact.cc
##########
@@ -3969,47 +3977,21 @@ HttpTransact::handle_forward_server_connection_open(State *s)
   TxnDebug("http_seq", "[HttpTransact::handle_server_connection_open] ");
   ink_release_assert(s->current.state == CONNECTION_ALIVE);
 
-  if (s->hdr_info.server_response.version_get() == HTTPVersion(0, 9)) {
-    TxnDebug("http_trans", "[hfsco] server sent 0.9 response, reading...");
-    build_response(s, &s->hdr_info.client_response, s->client_info.http_version, HTTP_STATUS_OK, "Connection Established");
-
-    s->client_info.keep_alive = HTTP_NO_KEEPALIVE;
-    s->cache_info.action      = CACHE_DO_NO_ACTION;
-    s->next_action            = SM_ACTION_SERVER_READ;
-    return;
-
-  } else if (s->hdr_info.server_response.version_get() == HTTPVersion(1, 0)) {
-    if (s->current.server->http_version == HTTPVersion(0, 9)) {
-      // update_hostdb_to_indicate_server_version_is_1_0
-      s->updated_server_version = HostDBApplicationInfo::HTTP_VERSION_10;
-    } else if (s->current.server->http_version == HTTPVersion(1, 1)) {
-      // update_hostdb_to_indicate_server_version_is_1_0
-      s->updated_server_version = HostDBApplicationInfo::HTTP_VERSION_10;
-    } else {
-      // dont update the hostdb. let us try again with what we currently think.
-    }
-  } else if (s->hdr_info.server_response.version_get() == HTTPVersion(1, 1)) {
-    if (s->current.server->http_version == HTTPVersion(0, 9)) {
-      // update_hostdb_to_indicate_server_version_is_1_1
-      s->updated_server_version = HostDBApplicationInfo::HTTP_VERSION_11;
-    } else if (s->current.server->http_version == HTTPVersion(1, 0)) {
-      // update_hostdb_to_indicate_server_version_is_1_1
-      s->updated_server_version = HostDBApplicationInfo::HTTP_VERSION_11;
-    } else {
-      // dont update the hostdb. let us try again with what we currently think.
-    }
-  } else {
-    // dont update the hostdb. let us try again with what we currently think.
+  HostDBApplicationInfo::HttpVersion real_version = s->state_machine->server_txn->get_version(s->hdr_info.server_response);
+  if (real_version != s->host_db_info.app.http_data.http_version) {
+    TxnDebug("http_trans", "Update hostdb history of server HTTP version 0x%x", real_version);
+    // Need to update the hostdb
+    s->updated_server_version = real_version;

Review comment:
       Agreed, the main relevant point here is adding the support for HTTP2 and rather than doing it as another if-then-else case I added a function in increase readability.
   
   The removal of the 0.9 logic is unrelated, but I removed it to further clean up the logic since we no longer support HTTP/0.9.  I'll put up a separate PR for that.  Perhaps for the whole thing and then just adding the HTTP2.0 enum would be all that would be needed here.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficserver] maskit commented on a change in pull request #7622: HTTP/2 to origin

Posted by GitBox <gi...@apache.org>.
maskit commented on a change in pull request #7622:
URL: https://github.com/apache/trafficserver/pull/7622#discussion_r607405859



##########
File path: iocore/net/UnixNetVConnection.cc
##########
@@ -431,6 +432,9 @@ write_to_net_io(NetHandler *nh, UnixNetVConnection *vc, EThread *thread)
   if (towrite != ntodo && buf.writer()->write_avail()) {
     if (write_signal_and_update(VC_EVENT_WRITE_READY, vc) != EVENT_CONT) {
       return;
+    } else if (c != s->vio.cont) { /* The write vio was updated in the handler */

Review comment:
       No, I meant it's used only once.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficserver] bneradt removed a comment on pull request #7622: HTTP/2 to origin

Posted by GitBox <gi...@apache.org>.
bneradt removed a comment on pull request #7622:
URL: https://github.com/apache/trafficserver/pull/7622#issuecomment-857050053


   [approve ci clang-format]


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficserver] shinrich commented on a change in pull request #7622: HTTP/2 to origin

Posted by GitBox <gi...@apache.org>.
shinrich commented on a change in pull request #7622:
URL: https://github.com/apache/trafficserver/pull/7622#discussion_r647041630



##########
File path: proxy/http/HttpSessionManager.cc
##########
@@ -398,7 +404,9 @@ HttpSessionManager::_acquire_session(sockaddr const *ip, CryptoHash const &hostn
     // Now check to see if we have a connection in our shared connection pool
     EThread *ethread = this_ethread();
     Ptr<ProxyMutex> pool_mutex =
-      (TS_SERVER_SESSION_SHARING_POOL_THREAD == pool_type) ? ethread->server_session_pool->mutex : m_g_pool->mutex;
+      (TS_SERVER_SESSION_SHARING_POOL_THREAD == sm->t_state.http_config_param->server_session_sharing_pool) ?

Review comment:
       Good point. That does seem to be an error.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficserver] shinrich commented on a change in pull request #7622: HTTP/2 to origin

Posted by GitBox <gi...@apache.org>.
shinrich commented on a change in pull request #7622:
URL: https://github.com/apache/trafficserver/pull/7622#discussion_r602514521



##########
File path: proxy/ProxyTransaction.cc
##########
@@ -220,3 +220,78 @@ ProxyTransaction::has_request_body(int64_t request_content_length, bool is_chunk
 {
   return request_content_length > 0 || is_chunked;
 }
+
+bool
+ProxyTransaction::is_read_closed() const
+{
+  return false;
+}
+
+bool
+ProxyTransaction::expect_send_trailer() const
+{
+  return false;
+}
+
+void
+ProxyTransaction::set_expect_send_trailer()
+{
+}
+bool
+ProxyTransaction::expect_receive_trailer() const
+{
+  return false;
+}
+
+void
+ProxyTransaction::set_expect_receive_trailer()
+{
+}
+
+void
+ProxyTransaction::attach_transaction(HttpSM *attach_sm)
+{
+  _sm = attach_sm;
+}
+
+void
+ProxyTransaction::release()
+{
+  HttpTxnDebug("[%" PRId64 "] session released by sm [%" PRId64 "]", _proxy_ssn ? _proxy_ssn->connection_id() : 0,
+               _sm ? _sm->sm_id : 0);
+
+  this->decrement_transactions_stat();

Review comment:
       Interesting.  The declaration has this set to 0.  The Http2Stream was calling the super version explicitly.  Http1 wasn't calling it at all.  I'm eliminating it in the ProxyTransaction and just inlining the necessary bits into the Http2Stream use cases.
   
   Need to think some more if there are double decrements coming from transaction_done.  Probably best to do the decrements in the shutdown logic instead of transaction_done.   Need to do another round of production testing.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficserver] SolidWallOfCode commented on a change in pull request #7622: HTTP/2 to origin

Posted by GitBox <gi...@apache.org>.
SolidWallOfCode commented on a change in pull request #7622:
URL: https://github.com/apache/trafficserver/pull/7622#discussion_r609046731



##########
File path: proxy/PoolableSession.h
##########
@@ -190,3 +204,39 @@ PoolableSession::FQDNLinkage::equal(CryptoHash const &lhs, CryptoHash const &rhs
 {
   return lhs == rhs;
 }
+
+inline void
+PoolableSession::enable_outbound_connection_tracking(OutboundConnTrack::Group *group)
+{
+  ink_assert(nullptr == conn_track_group);

Review comment:
       Not sure I'd do that - passing `nullptr` could be a way to disable connection tracking, and everything should function even if the value is `nullptr`.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficserver] shinrich commented on a change in pull request #7622: HTTP/2 to origin

Posted by GitBox <gi...@apache.org>.
shinrich commented on a change in pull request #7622:
URL: https://github.com/apache/trafficserver/pull/7622#discussion_r602368705



##########
File path: doc/admin-guide/files/records.config.en.rst
##########
@@ -3717,6 +3717,13 @@ Client-Related Configuration
 
    Enables (``1``) or disables (``0``) TLSv1_3 in the ATS client context. If not specified, enabled by default
 
+.. ts:cv:: CONFIG proxy.config.ssl.client.alpn_protocol STRING ""
+
+   Set the alpn string that ATS will send to origin during new connections.  By default no ALPN string will be set.
+   To enable HTTP/2 communication to the origin, set this to "h2,http1.1".

Review comment:
       Good questions.  I kept it empty by default so the initial integration would have less impact.  Of course, there are enough other code changes that landing this PR potentially have an impact even if you don't set the ALPN strings.
   
   We need to have a more general discussion about the preferred order, etc.  Should probably look how other similar proxies handle this.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficserver] shinrich commented on a change in pull request #7622: HTTP/2 to origin

Posted by GitBox <gi...@apache.org>.
shinrich commented on a change in pull request #7622:
URL: https://github.com/apache/trafficserver/pull/7622#discussion_r607152434



##########
File path: proxy/http/HttpTransact.cc
##########
@@ -1826,10 +1830,14 @@ HttpTransact::ReDNSRoundRobin(State *s)
     s->next_action = how_to_open_connection(s);
   } else {
     // Our ReDNS failed so output the DNS failure error message
-    build_error_response(s, HTTP_STATUS_BAD_GATEWAY, "Cannot find server.", "connect#dns_failed");
+    // Set to internal server error so later logging will pick up SQUID_LOG_ERR_DNS_FAIL

Review comment:
       This is actually part of PR #7580 as is the line with your previous comment.  I'm hoping it lands first and will rebase out of this PR.  Removing it makes my rebase back to a prod build more sketchy.  However, I've had no luck getting anyone to look at it.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficserver] shinrich commented on pull request #7622: HTTP/2 to origin

Posted by GitBox <gi...@apache.org>.
shinrich commented on pull request #7622:
URL: https://github.com/apache/trafficserver/pull/7622#issuecomment-883730311


   In testing, I found that I needed to defer the assignment of the stream_id for an outbound stream until it was ready to actually send a header.  Otherwise, I was getting GOAWAY frames from the next level ATS box with error code set to STREAM_CLOSED.  Presumably, another stream was getting assigned a later stream ID and sending its header out first.  So when the slower stream sends its header, the origin ATS box fails it because the stream ID is smaller than the "latest" seen stream id.   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@trafficserver.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficserver] bneradt commented on pull request #7622: HTTP/2 to origin

Posted by GitBox <gi...@apache.org>.
bneradt commented on pull request #7622:
URL: https://github.com/apache/trafficserver/pull/7622#issuecomment-815172177


   [approve ci]


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficserver] maskit commented on a change in pull request #7622: HTTP/2 to origin

Posted by GitBox <gi...@apache.org>.
maskit commented on a change in pull request #7622:
URL: https://github.com/apache/trafficserver/pull/7622#discussion_r645321364



##########
File path: proxy/http/HttpSM.cc
##########
@@ -1801,19 +1945,51 @@ HttpSM::handle_api_return()
 }
 
 PoolableSession *
-HttpSM::create_server_session(NetVConnection *netvc)
+HttpSM::create_server_session(NetVConnection *netvc, MIOBuffer *netvc_read_buffer, IOBufferReader *netvc_reader)
 {
   HttpTransact::State &s  = this->t_state;
-  PoolableSession *retval = httpServerSessionAllocator.alloc();
+  PoolableSession *retval = nullptr;
+
+  // Figure out what protocol was negotiated
+  const unsigned char *proto  = nullptr;
+  unsigned int proto_length   = 0;
+  SSLNetVConnection *sslnetvc = dynamic_cast<SSLNetVConnection *>(netvc);
+  if (sslnetvc) {
+    SSL_get0_alpn_selected(sslnetvc->ssl, &proto, &proto_length);
+  }

Review comment:
       This could be done by ALPNSupport.

##########
File path: proxy/http/HttpSM.cc
##########
@@ -1801,19 +1945,51 @@ HttpSM::handle_api_return()
 }
 
 PoolableSession *
-HttpSM::create_server_session(NetVConnection *netvc)
+HttpSM::create_server_session(NetVConnection *netvc, MIOBuffer *netvc_read_buffer, IOBufferReader *netvc_reader)
 {
   HttpTransact::State &s  = this->t_state;
-  PoolableSession *retval = httpServerSessionAllocator.alloc();
+  PoolableSession *retval = nullptr;
+
+  // Figure out what protocol was negotiated
+  const unsigned char *proto  = nullptr;
+  unsigned int proto_length   = 0;
+  SSLNetVConnection *sslnetvc = dynamic_cast<SSLNetVConnection *>(netvc);
+  if (sslnetvc) {
+    SSL_get0_alpn_selected(sslnetvc->ssl, &proto, &proto_length);
+  }
 
-  retval->sharing_pool         = static_cast<TSServerSessionSharingPoolType>(s.http_config_param->server_session_sharing_pool);
-  retval->sharing_match        = static_cast<TSServerSessionSharingMatchMask>(s.txn_conf->server_session_sharing_match);
-  MIOBuffer *netvc_read_buffer = new_MIOBuffer(HTTP_SERVER_RESP_HDR_BUFFER_INDEX);
-  IOBufferReader *netvc_reader = netvc_read_buffer->alloc_reader();
+  bool add_session = false;
+  if (proto_length == 2 && memcmp(proto, "h2", 2) == 0) {
+    Http2ServerSession *session = http2ServerSessionAllocator.alloc();
+    add_session                 = true;
+    retval                      = session;
+  } else {
+    Http1ServerSession *session = httpServerSessionAllocator.alloc();
+    retval                      = session;
+  }

Review comment:
       I understand we need to do this here, but I don't want to have the detail here in HttpSM. We should probably have a mechanism like `ProtocolProbeSessionAccept`.

##########
File path: proxy/http/HttpSessionManager.cc
##########
@@ -165,9 +165,13 @@ ServerSessionPool::acquireSession(sockaddr const *addr, CryptoHash const &hostna
     }
     if (zret == HSM_DONE) {
       to_return = first;
-      this->removeSession(to_return);
-    } else if (first != m_fqdn_pool.end()) {
-      Debug("http_ss", "Failed find entry due to name mismatch %s", sm->t_state.current.server->name);
+      if (!to_return->is_multiplexing()) {
+        this->removeSession(to_return);
+      }
+    } else {
+      if (first != m_fqdn_pool.end()) {

Review comment:
       else if

##########
File path: proxy/logging/LogAccess.cc
##########
@@ -1277,7 +1277,7 @@ LogAccess::marshal_client_sni_server_name(char *buf)
   std::string_view server_name = "";
 
   if (m_http_sm) {
-    auto txn = m_http_sm->get_ua_txn();
+    auto txn = m_http_sm->ua_txn;

Review comment:
       Let's use the getter.

##########
File path: proxy/http/HttpSessionManager.cc
##########
@@ -398,7 +404,9 @@ HttpSessionManager::_acquire_session(sockaddr const *ip, CryptoHash const &hostn
     // Now check to see if we have a connection in our shared connection pool
     EThread *ethread = this_ethread();
     Ptr<ProxyMutex> pool_mutex =
-      (TS_SERVER_SESSION_SHARING_POOL_THREAD == pool_type) ? ethread->server_session_pool->mutex : m_g_pool->mutex;
+      (TS_SERVER_SESSION_SHARING_POOL_THREAD == sm->t_state.http_config_param->server_session_sharing_pool) ?

Review comment:
       `pool_type` that is passed by the caller is used after acquiring the lock. Doesn't it cause mismatch?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficserver] maskit commented on a change in pull request #7622: HTTP/2 to origin

Posted by GitBox <gi...@apache.org>.
maskit commented on a change in pull request #7622:
URL: https://github.com/apache/trafficserver/pull/7622#discussion_r645321364



##########
File path: proxy/http/HttpSM.cc
##########
@@ -1801,19 +1945,51 @@ HttpSM::handle_api_return()
 }
 
 PoolableSession *
-HttpSM::create_server_session(NetVConnection *netvc)
+HttpSM::create_server_session(NetVConnection *netvc, MIOBuffer *netvc_read_buffer, IOBufferReader *netvc_reader)
 {
   HttpTransact::State &s  = this->t_state;
-  PoolableSession *retval = httpServerSessionAllocator.alloc();
+  PoolableSession *retval = nullptr;
+
+  // Figure out what protocol was negotiated
+  const unsigned char *proto  = nullptr;
+  unsigned int proto_length   = 0;
+  SSLNetVConnection *sslnetvc = dynamic_cast<SSLNetVConnection *>(netvc);
+  if (sslnetvc) {
+    SSL_get0_alpn_selected(sslnetvc->ssl, &proto, &proto_length);
+  }

Review comment:
       This could be done by ALPNSupport.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficserver] maskit commented on pull request #7622: HTTP/2 to origin

Posted by GitBox <gi...@apache.org>.
maskit commented on pull request #7622:
URL: https://github.com/apache/trafficserver/pull/7622#issuecomment-856473460


   Let's separate out the outbound ALPN part. We still need to discuss how to enable which protocol, but the feature itself should work without H2 (e.g. use http/1.1 but not 1.0), and it should be testable.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficserver] bneradt removed a comment on pull request #7622: HTTP/2 to origin

Posted by GitBox <gi...@apache.org>.
bneradt removed a comment on pull request #7622:
URL: https://github.com/apache/trafficserver/pull/7622#issuecomment-815134601


   [approve ci]


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficserver] shinrich commented on a change in pull request #7622: HTTP/2 to origin

Posted by GitBox <gi...@apache.org>.
shinrich commented on a change in pull request #7622:
URL: https://github.com/apache/trafficserver/pull/7622#discussion_r612641859



##########
File path: proxy/http/HttpTransact.cc
##########
@@ -3969,47 +3977,21 @@ HttpTransact::handle_forward_server_connection_open(State *s)
   TxnDebug("http_seq", "[HttpTransact::handle_server_connection_open] ");
   ink_release_assert(s->current.state == CONNECTION_ALIVE);
 
-  if (s->hdr_info.server_response.version_get() == HTTPVersion(0, 9)) {
-    TxnDebug("http_trans", "[hfsco] server sent 0.9 response, reading...");
-    build_response(s, &s->hdr_info.client_response, s->client_info.http_version, HTTP_STATUS_OK, "Connection Established");
-
-    s->client_info.keep_alive = HTTP_NO_KEEPALIVE;
-    s->cache_info.action      = CACHE_DO_NO_ACTION;
-    s->next_action            = SM_ACTION_SERVER_READ;
-    return;
-
-  } else if (s->hdr_info.server_response.version_get() == HTTPVersion(1, 0)) {
-    if (s->current.server->http_version == HTTPVersion(0, 9)) {
-      // update_hostdb_to_indicate_server_version_is_1_0
-      s->updated_server_version = HostDBApplicationInfo::HTTP_VERSION_10;
-    } else if (s->current.server->http_version == HTTPVersion(1, 1)) {
-      // update_hostdb_to_indicate_server_version_is_1_0
-      s->updated_server_version = HostDBApplicationInfo::HTTP_VERSION_10;
-    } else {
-      // dont update the hostdb. let us try again with what we currently think.
-    }
-  } else if (s->hdr_info.server_response.version_get() == HTTPVersion(1, 1)) {
-    if (s->current.server->http_version == HTTPVersion(0, 9)) {
-      // update_hostdb_to_indicate_server_version_is_1_1
-      s->updated_server_version = HostDBApplicationInfo::HTTP_VERSION_11;
-    } else if (s->current.server->http_version == HTTPVersion(1, 0)) {
-      // update_hostdb_to_indicate_server_version_is_1_1
-      s->updated_server_version = HostDBApplicationInfo::HTTP_VERSION_11;
-    } else {
-      // dont update the hostdb. let us try again with what we currently think.
-    }
-  } else {
-    // dont update the hostdb. let us try again with what we currently think.
+  HostDBApplicationInfo::HttpVersion real_version = s->state_machine->server_txn->get_version(s->hdr_info.server_response);
+  if (real_version != s->host_db_info.app.http_data.http_version) {
+    TxnDebug("http_trans", "Update hostdb history of server HTTP version 0x%x", real_version);
+    // Need to update the hostdb
+    s->updated_server_version = real_version;

Review comment:
       PR #7706. 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficserver] keesspoelstra commented on a change in pull request #7622: HTTP/2 to origin

Posted by GitBox <gi...@apache.org>.
keesspoelstra commented on a change in pull request #7622:
URL: https://github.com/apache/trafficserver/pull/7622#discussion_r727282408



##########
File path: doc/admin-guide/files/records.config.en.rst
##########
@@ -3849,6 +3849,13 @@ Client-Related Configuration
 
    Enables (``1``) or disables (``0``) TLSv1_3 in the ATS client context. If not specified, enabled by default
 
+.. ts:cv:: CONFIG proxy.config.ssl.client.alpn_protocol STRING ""

Review comment:
       Typo, should be 
   `
   proxy.config.ssl.client.alpn_protocols
   `




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@trafficserver.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficserver] shinrich commented on a change in pull request #7622: HTTP/2 to origin

Posted by GitBox <gi...@apache.org>.
shinrich commented on a change in pull request #7622:
URL: https://github.com/apache/trafficserver/pull/7622#discussion_r607877839



##########
File path: proxy/ProxyTransaction.cc
##########
@@ -220,3 +220,78 @@ ProxyTransaction::has_request_body(int64_t request_content_length, bool is_chunk
 {
   return request_content_length > 0 || is_chunked;
 }
+
+bool
+ProxyTransaction::is_read_closed() const
+{
+  return false;
+}
+
+bool
+ProxyTransaction::expect_send_trailer() const
+{
+  return false;
+}
+
+void
+ProxyTransaction::set_expect_send_trailer()
+{
+}
+bool
+ProxyTransaction::expect_receive_trailer() const
+{
+  return false;
+}
+
+void
+ProxyTransaction::set_expect_receive_trailer()
+{
+}
+
+void
+ProxyTransaction::attach_transaction(HttpSM *attach_sm)
+{
+  _sm = attach_sm;
+}
+
+void
+ProxyTransaction::release()
+{
+  HttpTxnDebug("[%" PRId64 "] session released by sm [%" PRId64 "]", _proxy_ssn ? _proxy_ssn->connection_id() : 0,
+               _sm ? _sm->sm_id : 0);
+
+  this->decrement_transactions_stat();

Review comment:
       Looking at my current prod run, we are double decrementing.  My current_transaction metrics are all going to 0.  Will track this down.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficserver] shinrich commented on pull request #7622: HTTP/2 to origin

Posted by GitBox <gi...@apache.org>.
shinrich commented on pull request #7622:
URL: https://github.com/apache/trafficserver/pull/7622#issuecomment-807122928


   [approve ci clang-format]


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficserver] maskit commented on a change in pull request #7622: HTTP/2 to origin

Posted by GitBox <gi...@apache.org>.
maskit commented on a change in pull request #7622:
URL: https://github.com/apache/trafficserver/pull/7622#discussion_r609157377



##########
File path: doc/admin-guide/files/records.config.en.rst
##########
@@ -3717,6 +3717,13 @@ Client-Related Configuration
 
    Enables (``1``) or disables (``0``) TLSv1_3 in the ATS client context. If not specified, enabled by default
 
+.. ts:cv:: CONFIG proxy.config.ssl.client.alpn_protocol STRING ""
+
+   Set the alpn string that ATS will send to origin during new connections.  By default no ALPN string will be set.
+   To enable HTTP/2 communication to the origin, set this to "h2,http1.1".

Review comment:
       I understand having "http1.1" by default doesn't make much sense in terms of the behavior, and I know no ALPN negotiation is the current behavior. But if I was not familiar with ALPN, I'd wonder why I have to have "http1.1"as well to enable HTTP/2. I might write only "h2" and say "Aha, H2 works. Seems like http/1.1 is not necessary.". Also listing numbers in descending order seem unnatural if I thought it's just a list. That's why I'm not keen on having this as ALPN setting.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficserver] keesspoelstra commented on a change in pull request #7622: HTTP/2 to origin

Posted by GitBox <gi...@apache.org>.
keesspoelstra commented on a change in pull request #7622:
URL: https://github.com/apache/trafficserver/pull/7622#discussion_r724109993



##########
File path: proxy/http/HttpSM.cc
##########
@@ -289,6 +277,160 @@ HttpVCTable::cleanup_all()
   }
 }
 
+class ConnectingEntry : public Continuation
+{
+public:
+  std::set<HttpSM *> _connect_sms;
+  NetVConnection *_netvc        = nullptr;
+  IOBufferReader *_netvc_reader = nullptr;
+  MIOBuffer *_netvc_read_buffer = nullptr;
+  std::string sni;
+  std::string cert_name;
+  IpEndpoint _ipaddr;
+  std::string hostname;
+  Action *_pending_action = nullptr;
+  NetVCOptions opt;
+
+  void remove_entry();
+  int state_http_server_open(int event, void *data);
+  static PoolableSession *create_server_session(HttpSM *root_sm, NetVConnection *netvc, MIOBuffer *netvc_read_buffer,
+                                                IOBufferReader *netvc_reader);
+};
+
+struct IpHelper {
+  size_t
+  operator()(IpEndpoint const &arg) const
+  {
+    return IpAddr{&arg.sa}.hash();
+  }
+  bool
+  operator()(IpEndpoint const &arg1, IpEndpoint const &arg2) const
+  {
+    return ats_ip_addr_port_eq(&arg1.sa, &arg2.sa);
+  }
+};
+using ConnectingIpPool = std::unordered_multimap<IpEndpoint, ConnectingEntry *, IpHelper>;
+
+class ConnectingPool
+{
+public:
+  ConnectingPool() {}
+  ConnectingIpPool m_ip_pool;
+};
+
+void
+initialize_thread_for_connecting_pools(EThread *thread)
+{
+  if (thread->connecting_pool == nullptr) {
+    thread->connecting_pool = new ConnectingPool();
+  }
+}
+
+int
+ConnectingEntry::state_http_server_open(int event, void *data)
+{
+  Debug("http_connect", "entered inside ConnectingEntry::state_http_server_open");
+
+  switch (event) {
+  case NET_EVENT_OPEN: {
+    _netvc                 = static_cast<NetVConnection *>(data);
+    UnixNetVConnection *vc = static_cast<UnixNetVConnection *>(_netvc);
+    ink_release_assert(_pending_action == nullptr || _pending_action->continuation == vc->get_action()->continuation);
+    _pending_action = nullptr;
+    Debug("http_connect", "ConnectingEntrysetting handler for TCP handshake");
+    // Just want to get a write-ready event so we know that the TCP handshake is complete.
+    // The buffer we create will be handed over to the eventually created server session
+    _netvc_read_buffer = new_MIOBuffer(HTTP_SERVER_RESP_HDR_BUFFER_INDEX);
+    _netvc_reader      = _netvc_read_buffer->alloc_reader();
+    _netvc->do_io_write(this, 1, _netvc_reader);
+    ink_release_assert(!_connect_sms.empty());
+    if (!_connect_sms.empty()) {
+      HttpSM *prime_connect_sm = *(_connect_sms.begin());
+      _netvc->set_inactivity_timeout(prime_connect_sm->get_server_connect_timeout());
+    }
+    ink_release_assert(_pending_action == nullptr);
+    return 0;
+  }
+  case VC_EVENT_READ_COMPLETE:
+  case VC_EVENT_WRITE_READY:
+  case VC_EVENT_WRITE_COMPLETE: {
+    Debug("http_connect", "Kick off %" PRId64 " state machines waiting for origin", _connect_sms.size());
+    this->remove_entry();
+    _netvc->do_io_write(nullptr, 0, nullptr);
+    if (!_connect_sms.empty()) {
+      auto prime_iter = _connect_sms.rbegin();
+      ink_release_assert(prime_iter != _connect_sms.rend());
+      PoolableSession *new_session = (*prime_iter)->create_server_session(_netvc, _netvc_read_buffer, _netvc_reader);
+      _netvc                       = nullptr;
+
+      // Did we end up with a multiplexing session?
+      int count = 0;
+      if (new_session->is_multiplexing()) {
+        // Hand off to all queued up ConnectSM's.

Review comment:
       It looks like hops or proxies were not fully considered in writing the spec.
   From a proxy's POV I really can imagine wanting to have multiple H2 connections to same origin, as you might be forward proxying a large organisation hiding behind a single outgoing IP or a busy reverse proxying in front of an origin.
   In those cases HOL blocking or too many interleaved response streams really might become a problem in user experience.
   
   I think we should have sane values (max h2 connections per server, configurable?) and then act upon the hints the H2 protocol will give us.

##########
File path: proxy/http/HttpSessionManager.cc
##########
@@ -190,7 +192,9 @@ ServerSessionPool::acquireSession(sockaddr const *addr, CryptoHash const &hostna
     }
     if (zret == HSM_DONE) {
       to_return = first;
-      this->removeSession(to_return);
+      if (!to_return->is_multiplexing()) {

Review comment:
       We probably want an early check in the while statement above to see if the session is maxed out, this to prevent ending up much later in the flow and finding out we can't add any more streams




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@trafficserver.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficserver] keesspoelstra commented on a change in pull request #7622: HTTP/2 to origin

Posted by GitBox <gi...@apache.org>.
keesspoelstra commented on a change in pull request #7622:
URL: https://github.com/apache/trafficserver/pull/7622#discussion_r724109993



##########
File path: proxy/http/HttpSM.cc
##########
@@ -289,6 +277,160 @@ HttpVCTable::cleanup_all()
   }
 }
 
+class ConnectingEntry : public Continuation
+{
+public:
+  std::set<HttpSM *> _connect_sms;
+  NetVConnection *_netvc        = nullptr;
+  IOBufferReader *_netvc_reader = nullptr;
+  MIOBuffer *_netvc_read_buffer = nullptr;
+  std::string sni;
+  std::string cert_name;
+  IpEndpoint _ipaddr;
+  std::string hostname;
+  Action *_pending_action = nullptr;
+  NetVCOptions opt;
+
+  void remove_entry();
+  int state_http_server_open(int event, void *data);
+  static PoolableSession *create_server_session(HttpSM *root_sm, NetVConnection *netvc, MIOBuffer *netvc_read_buffer,
+                                                IOBufferReader *netvc_reader);
+};
+
+struct IpHelper {
+  size_t
+  operator()(IpEndpoint const &arg) const
+  {
+    return IpAddr{&arg.sa}.hash();
+  }
+  bool
+  operator()(IpEndpoint const &arg1, IpEndpoint const &arg2) const
+  {
+    return ats_ip_addr_port_eq(&arg1.sa, &arg2.sa);
+  }
+};
+using ConnectingIpPool = std::unordered_multimap<IpEndpoint, ConnectingEntry *, IpHelper>;
+
+class ConnectingPool
+{
+public:
+  ConnectingPool() {}
+  ConnectingIpPool m_ip_pool;
+};
+
+void
+initialize_thread_for_connecting_pools(EThread *thread)
+{
+  if (thread->connecting_pool == nullptr) {
+    thread->connecting_pool = new ConnectingPool();
+  }
+}
+
+int
+ConnectingEntry::state_http_server_open(int event, void *data)
+{
+  Debug("http_connect", "entered inside ConnectingEntry::state_http_server_open");
+
+  switch (event) {
+  case NET_EVENT_OPEN: {
+    _netvc                 = static_cast<NetVConnection *>(data);
+    UnixNetVConnection *vc = static_cast<UnixNetVConnection *>(_netvc);
+    ink_release_assert(_pending_action == nullptr || _pending_action->continuation == vc->get_action()->continuation);
+    _pending_action = nullptr;
+    Debug("http_connect", "ConnectingEntrysetting handler for TCP handshake");
+    // Just want to get a write-ready event so we know that the TCP handshake is complete.
+    // The buffer we create will be handed over to the eventually created server session
+    _netvc_read_buffer = new_MIOBuffer(HTTP_SERVER_RESP_HDR_BUFFER_INDEX);
+    _netvc_reader      = _netvc_read_buffer->alloc_reader();
+    _netvc->do_io_write(this, 1, _netvc_reader);
+    ink_release_assert(!_connect_sms.empty());
+    if (!_connect_sms.empty()) {
+      HttpSM *prime_connect_sm = *(_connect_sms.begin());
+      _netvc->set_inactivity_timeout(prime_connect_sm->get_server_connect_timeout());
+    }
+    ink_release_assert(_pending_action == nullptr);
+    return 0;
+  }
+  case VC_EVENT_READ_COMPLETE:
+  case VC_EVENT_WRITE_READY:
+  case VC_EVENT_WRITE_COMPLETE: {
+    Debug("http_connect", "Kick off %" PRId64 " state machines waiting for origin", _connect_sms.size());
+    this->remove_entry();
+    _netvc->do_io_write(nullptr, 0, nullptr);
+    if (!_connect_sms.empty()) {
+      auto prime_iter = _connect_sms.rbegin();
+      ink_release_assert(prime_iter != _connect_sms.rend());
+      PoolableSession *new_session = (*prime_iter)->create_server_session(_netvc, _netvc_read_buffer, _netvc_reader);
+      _netvc                       = nullptr;
+
+      // Did we end up with a multiplexing session?
+      int count = 0;
+      if (new_session->is_multiplexing()) {
+        // Hand off to all queued up ConnectSM's.

Review comment:
       It looks like hops or proxies were not fully considered in writing the spec.
   From a proxy's POV I really can imagine wanting to have multiple H2 connections to same origin, as you might be forward proxying a large organisation hiding behind a single outgoing IP or a busy reverse proxying in front of an origin.
   In those cases HOL blocking or too many interleaved response streams really might become a problem in user experience.
   
   I think we should have sane values (max h2 connections per server, configurable?) and then act upon the hints the H2 protocol will give us.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@trafficserver.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficserver] maskit commented on a change in pull request #7622: HTTP/2 to origin

Posted by GitBox <gi...@apache.org>.
maskit commented on a change in pull request #7622:
URL: https://github.com/apache/trafficserver/pull/7622#discussion_r612304921



##########
File path: proxy/http/HttpSM.cc
##########
@@ -2851,6 +3062,50 @@ HttpSM::tunnel_handler_post(int event, void *data)
   return 0;
 }
 
+int
+HttpSM::tunnel_handler_trailer(int event, void *data)

Review comment:
       Is it too hard to separate out trailer support?

##########
File path: proxy/http/HttpSM.cc
##########
@@ -344,7 +487,7 @@ HttpSM::get_server_connect_timeout()
 {
   ink_hrtime retval = 0;
   if (t_state.api_txn_connect_timeout_value != -1) {
-    retval = HRTIME_MSECONDS(t_state.api_txn_connect_timeout_value);
+    retval = HRTIME_SECONDS(t_state.api_txn_connect_timeout_value);

Review comment:
       It does seem inconsistent, but the value is milliseconds.
   
   https://github.com/apache/trafficserver/blob/3fd2c608e3fa207c8d3ab6476f50349892f1d997/src/traffic_server/InkAPI.cc#L5549-L5560
   
   header_rewrite uses this API and the documentation says its milliseconds.
   https://github.com/apache/trafficserver/blob/3fd2c608e3fa207c8d3ab6476f50349892f1d997/doc/admin-guide/plugins/header_rewrite.en.rst#set-timeout-out
   
   In any cases, it's not related to H2 to Origin.

##########
File path: proxy/http/HttpTransact.cc
##########
@@ -3969,47 +3977,21 @@ HttpTransact::handle_forward_server_connection_open(State *s)
   TxnDebug("http_seq", "[HttpTransact::handle_server_connection_open] ");
   ink_release_assert(s->current.state == CONNECTION_ALIVE);
 
-  if (s->hdr_info.server_response.version_get() == HTTPVersion(0, 9)) {
-    TxnDebug("http_trans", "[hfsco] server sent 0.9 response, reading...");
-    build_response(s, &s->hdr_info.client_response, s->client_info.http_version, HTTP_STATUS_OK, "Connection Established");
-
-    s->client_info.keep_alive = HTTP_NO_KEEPALIVE;
-    s->cache_info.action      = CACHE_DO_NO_ACTION;
-    s->next_action            = SM_ACTION_SERVER_READ;
-    return;
-
-  } else if (s->hdr_info.server_response.version_get() == HTTPVersion(1, 0)) {
-    if (s->current.server->http_version == HTTPVersion(0, 9)) {
-      // update_hostdb_to_indicate_server_version_is_1_0
-      s->updated_server_version = HostDBApplicationInfo::HTTP_VERSION_10;
-    } else if (s->current.server->http_version == HTTPVersion(1, 1)) {
-      // update_hostdb_to_indicate_server_version_is_1_0
-      s->updated_server_version = HostDBApplicationInfo::HTTP_VERSION_10;
-    } else {
-      // dont update the hostdb. let us try again with what we currently think.
-    }
-  } else if (s->hdr_info.server_response.version_get() == HTTPVersion(1, 1)) {
-    if (s->current.server->http_version == HTTPVersion(0, 9)) {
-      // update_hostdb_to_indicate_server_version_is_1_1
-      s->updated_server_version = HostDBApplicationInfo::HTTP_VERSION_11;
-    } else if (s->current.server->http_version == HTTPVersion(1, 0)) {
-      // update_hostdb_to_indicate_server_version_is_1_1
-      s->updated_server_version = HostDBApplicationInfo::HTTP_VERSION_11;
-    } else {
-      // dont update the hostdb. let us try again with what we currently think.
-    }
-  } else {
-    // dont update the hostdb. let us try again with what we currently think.
+  HostDBApplicationInfo::HttpVersion real_version = s->state_machine->server_txn->get_version(s->hdr_info.server_response);
+  if (real_version != s->host_db_info.app.http_data.http_version) {
+    TxnDebug("http_trans", "Update hostdb history of server HTTP version 0x%x", real_version);
+    // Need to update the hostdb
+    s->updated_server_version = real_version;

Review comment:
       I mean it seems like the mechanism works regardless of H2 to Origin support. The key change here is to use `server_txn->get_version` if I understand correctly.
   
   Is it too hard to separate out this change as a mechanism to support multiple HTTP versions? It would be really nice if we could slim down this PR to a pure addition of support for another HTTP version.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficserver] SolidWallOfCode commented on a change in pull request #7622: HTTP/2 to origin

Posted by GitBox <gi...@apache.org>.
SolidWallOfCode commented on a change in pull request #7622:
URL: https://github.com/apache/trafficserver/pull/7622#discussion_r609028512



##########
File path: doc/admin-guide/files/records.config.en.rst
##########
@@ -3717,6 +3717,13 @@ Client-Related Configuration
 
    Enables (``1``) or disables (``0``) TLSv1_3 in the ATS client context. If not specified, enabled by default
 
+.. ts:cv:: CONFIG proxy.config.ssl.client.alpn_protocol STRING ""
+
+   Set the alpn string that ATS will send to origin during new connections.  By default no ALPN string will be set.
+   To enable HTTP/2 communication to the origin, set this to "h2,http1.1".

Review comment:
       I don't see the benefit of defaulting to "http1.1". To add HTTP/2, it would still be necessary to specific "http2,http1.1" in the configuration. I agree with Susan's view that, at least initially, this should be backwards compatible in that no HTTP/2 is done outbound unless explicitly configured. That's how HTTP/2 inbound was done - it originally required an explicit "enable" configuration variable to be set.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficserver] shinrich commented on a change in pull request #7622: HTTP/2 to origin

Posted by GitBox <gi...@apache.org>.
shinrich commented on a change in pull request #7622:
URL: https://github.com/apache/trafficserver/pull/7622#discussion_r607928422



##########
File path: include/tscore/Scalar.h
##########
@@ -33,15 +33,15 @@
 #include <type_traits>
 #include "tscore/BufferWriter.h"
 
-namespace tag
-{
-struct generic;
-}
-
 namespace ts
 {
 template <intmax_t N, typename C, typename T> class Scalar;
 
+namespace tag
+{
+  struct generic;
+}
+

Review comment:
       Needs to be in the ts namespace to avoid conflicts.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficserver] bneradt removed a comment on pull request #7622: HTTP/2 to origin

Posted by GitBox <gi...@apache.org>.
bneradt removed a comment on pull request #7622:
URL: https://github.com/apache/trafficserver/pull/7622#issuecomment-814195424






-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficserver] shinrich commented on a change in pull request #7622: HTTP/2 to origin

Posted by GitBox <gi...@apache.org>.
shinrich commented on a change in pull request #7622:
URL: https://github.com/apache/trafficserver/pull/7622#discussion_r608251450



##########
File path: proxy/http2/HTTP2.cc
##########
@@ -729,7 +766,7 @@ http2_decode_header_blocks(HTTPHdr *hdr, const uint8_t *buf_start, const uint32_
   if (hdr->field_find(MIME_FIELD_CONNECTION, MIME_LEN_CONNECTION) != nullptr ||
       hdr->field_find(MIME_FIELD_KEEP_ALIVE, MIME_LEN_KEEP_ALIVE) != nullptr ||
       hdr->field_find(MIME_FIELD_PROXY_CONNECTION, MIME_LEN_PROXY_CONNECTION) != nullptr ||
-      hdr->field_find(MIME_FIELD_TRANSFER_ENCODING, MIME_LEN_TRANSFER_ENCODING) != nullptr ||
+      // hdr->field_find(MIME_FIELD_TRANSFER_ENCODING, MIME_LEN_TRANSFER_ENCODING) != nullptr ||

Review comment:
       Backed this out.  Wasn't sure why I did this. My use of transfer_encoding headers evolved during this work.  Ending up with out using transfer encoding at all for HTTP/2.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficserver] shinrich commented on a change in pull request #7622: HTTP/2 to origin

Posted by GitBox <gi...@apache.org>.
shinrich commented on a change in pull request #7622:
URL: https://github.com/apache/trafficserver/pull/7622#discussion_r647401520



##########
File path: proxy/logging/LogAccess.cc
##########
@@ -1277,7 +1277,7 @@ LogAccess::marshal_client_sni_server_name(char *buf)
   std::string_view server_name = "";
 
   if (m_http_sm) {
-    auto txn = m_http_sm->get_ua_txn();
+    auto txn = m_http_sm->ua_txn;

Review comment:
       Done




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficserver] maskit commented on a change in pull request #7622: HTTP/2 to origin

Posted by GitBox <gi...@apache.org>.
maskit commented on a change in pull request #7622:
URL: https://github.com/apache/trafficserver/pull/7622#discussion_r645395815



##########
File path: proxy/http/HttpSM.cc
##########
@@ -289,6 +277,160 @@ HttpVCTable::cleanup_all()
   }
 }
 
+class ConnectingEntry : public Continuation
+{
+public:
+  std::set<HttpSM *> _connect_sms;
+  NetVConnection *_netvc        = nullptr;
+  IOBufferReader *_netvc_reader = nullptr;
+  MIOBuffer *_netvc_read_buffer = nullptr;
+  std::string sni;
+  std::string cert_name;
+  IpEndpoint _ipaddr;
+  std::string hostname;
+  Action *_pending_action = nullptr;
+  NetVCOptions opt;
+
+  void remove_entry();
+  int state_http_server_open(int event, void *data);
+  static PoolableSession *create_server_session(HttpSM *root_sm, NetVConnection *netvc, MIOBuffer *netvc_read_buffer,
+                                                IOBufferReader *netvc_reader);
+};
+
+struct IpHelper {
+  size_t
+  operator()(IpEndpoint const &arg) const
+  {
+    return IpAddr{&arg.sa}.hash();
+  }
+  bool
+  operator()(IpEndpoint const &arg1, IpEndpoint const &arg2) const
+  {
+    return ats_ip_addr_port_eq(&arg1.sa, &arg2.sa);
+  }
+};
+using ConnectingIpPool = std::unordered_multimap<IpEndpoint, ConnectingEntry *, IpHelper>;
+
+class ConnectingPool
+{
+public:
+  ConnectingPool() {}
+  ConnectingIpPool m_ip_pool;
+};
+
+void
+initialize_thread_for_connecting_pools(EThread *thread)
+{
+  if (thread->connecting_pool == nullptr) {
+    thread->connecting_pool = new ConnectingPool();
+  }
+}
+
+int
+ConnectingEntry::state_http_server_open(int event, void *data)
+{
+  Debug("http_connect", "entered inside ConnectingEntry::state_http_server_open");
+
+  switch (event) {
+  case NET_EVENT_OPEN: {
+    _netvc                 = static_cast<NetVConnection *>(data);
+    UnixNetVConnection *vc = static_cast<UnixNetVConnection *>(_netvc);
+    ink_release_assert(_pending_action == nullptr || _pending_action->continuation == vc->get_action()->continuation);
+    _pending_action = nullptr;
+    Debug("http_connect", "ConnectingEntrysetting handler for TCP handshake");
+    // Just want to get a write-ready event so we know that the TCP handshake is complete.
+    // The buffer we create will be handed over to the eventually created server session
+    _netvc_read_buffer = new_MIOBuffer(HTTP_SERVER_RESP_HDR_BUFFER_INDEX);
+    _netvc_reader      = _netvc_read_buffer->alloc_reader();
+    _netvc->do_io_write(this, 1, _netvc_reader);
+    ink_release_assert(!_connect_sms.empty());
+    if (!_connect_sms.empty()) {
+      HttpSM *prime_connect_sm = *(_connect_sms.begin());
+      _netvc->set_inactivity_timeout(prime_connect_sm->get_server_connect_timeout());
+    }
+    ink_release_assert(_pending_action == nullptr);
+    return 0;
+  }
+  case VC_EVENT_READ_COMPLETE:
+  case VC_EVENT_WRITE_READY:
+  case VC_EVENT_WRITE_COMPLETE: {
+    Debug("http_connect", "Kick off %" PRId64 " state machines waiting for origin", _connect_sms.size());
+    this->remove_entry();
+    _netvc->do_io_write(nullptr, 0, nullptr);
+    if (!_connect_sms.empty()) {
+      auto prime_iter = _connect_sms.rbegin();
+      ink_release_assert(prime_iter != _connect_sms.rend());
+      PoolableSession *new_session = (*prime_iter)->create_server_session(_netvc, _netvc_read_buffer, _netvc_reader);
+      _netvc                       = nullptr;
+
+      // Did we end up with a multiplexing session?
+      int count = 0;
+      if (new_session->is_multiplexing()) {
+        // Hand off to all queued up ConnectSM's.

Review comment:
       How do you know a multiplexed connection can process all the requests? An origin server may stop accepting new requests on a connection for some reasons (e.g. max requests per conn to mitigate memory pressure due to memory leak), and the queued up requests would just fail where those could be processed successfully on H1 if the server still accepts new connections.
   
   Also, there's no guarantee a multiplexed connection can process all the requests at once. The number of H2 concurrent stream might be set to 1 for some reasons.
   
   In that sense, whether a connection is multiplexing is not important. Non-multiplexed connection could be handled as a multiplexed connection that only process 1 request at once.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficserver] shinrich commented on pull request #7622: HTTP/2 to origin

Posted by GitBox <gi...@apache.org>.
shinrich commented on pull request #7622:
URL: https://github.com/apache/trafficserver/pull/7622#issuecomment-814898825


   [approve ci clang-analyzer]


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficserver] maskit commented on a change in pull request #7622: HTTP/2 to origin

Posted by GitBox <gi...@apache.org>.
maskit commented on a change in pull request #7622:
URL: https://github.com/apache/trafficserver/pull/7622#discussion_r601977186



##########
File path: doc/admin-guide/files/records.config.en.rst
##########
@@ -3717,6 +3717,13 @@ Client-Related Configuration
 
    Enables (``1``) or disables (``0``) TLSv1_3 in the ATS client context. If not specified, enabled by default
 
+.. ts:cv:: CONFIG proxy.config.ssl.client.alpn_protocol STRING ""
+
+   Set the alpn string that ATS will send to origin during new connections.  By default no ALPN string will be set.
+   To enable HTTP/2 communication to the origin, set this to "h2,http1.1".

Review comment:
       I'd like to know why it has no ALPN string by default. Having "http1.1" (or "http1.1,http1.0") by default and adding "h2" to enable HTTP/2 sound more natural to me. Also, it may be better to mention about the order.
   
   What will we do if we support H3 to origin? Will we internally filter the string to not offer protocols that are not supported on a transport, or have another setting for QUIC transport? 
   
   I'm not sure if users want to specify raw ALPN protocol names. For server_ports, we don't use it.
   
   I don't think this is really going to be a problem, but ALPN protocol name may have commas and any characters. The spec says protocol names are non-empty byte strings.

##########
File path: proxy/ProxySession.cc
##########
@@ -106,7 +106,7 @@ ProxySession::state_api_callout(int event, void *data)
       if (!lock.is_locked()) {
         SET_HANDLER(&ProxySession::state_api_callout);
         if (!schedule_event) { // Don't bother if there is already one
-          schedule_event = mutex->thread_holding->schedule_in(this, HRTIME_MSECONDS(10));
+          schedule_event = this_ethread()->schedule_in(this, HRTIME_MSECONDS(10));

Review comment:
       Can we make this change separately?

##########
File path: iocore/net/SSLConfig.cc
##########
@@ -177,7 +177,6 @@ set_paths_helper(const char *path, const char *filename, char **final_path, char
     *final_filename = filename ? ats_stringdup(Layout::get()->relative_to(path, filename)) : nullptr;
   }
 }
-

Review comment:
       I'm surprised that clang-format doesn't point out this. I want an empty line here.

##########
File path: proxy/ProxyTransaction.cc
##########
@@ -220,3 +220,78 @@ ProxyTransaction::has_request_body(int64_t request_content_length, bool is_chunk
 {
   return request_content_length > 0 || is_chunked;
 }
+
+bool
+ProxyTransaction::is_read_closed() const
+{
+  return false;
+}
+
+bool
+ProxyTransaction::expect_send_trailer() const
+{
+  return false;
+}
+
+void
+ProxyTransaction::set_expect_send_trailer()
+{
+}

Review comment:
       I want an empty line between functions.

##########
File path: proxy/ProxyTransaction.cc
##########
@@ -220,3 +220,78 @@ ProxyTransaction::has_request_body(int64_t request_content_length, bool is_chunk
 {
   return request_content_length > 0 || is_chunked;
 }
+
+bool
+ProxyTransaction::is_read_closed() const
+{
+  return false;
+}
+
+bool
+ProxyTransaction::expect_send_trailer() const
+{
+  return false;
+}
+
+void
+ProxyTransaction::set_expect_send_trailer()
+{
+}
+bool
+ProxyTransaction::expect_receive_trailer() const
+{
+  return false;
+}
+
+void
+ProxyTransaction::set_expect_receive_trailer()
+{
+}
+
+void
+ProxyTransaction::attach_transaction(HttpSM *attach_sm)
+{
+  _sm = attach_sm;
+}
+
+void
+ProxyTransaction::release()
+{
+  HttpTxnDebug("[%" PRId64 "] session released by sm [%" PRId64 "]", _proxy_ssn ? _proxy_ssn->connection_id() : 0,
+               _sm ? _sm->sm_id : 0);
+
+  this->decrement_transactions_stat();
+
+  // Pass along the release to the session
+  if (_proxy_ssn) {
+    _proxy_ssn->release(this);
+  }
+}
+
+HostDBApplicationInfo::HttpVersion
+ProxyTransaction::get_version(HTTPHdr &hdr) const
+{
+  if (hdr.version_get() == HTTPVersion(1, 1)) {
+    return HostDBApplicationInfo::HTTP_VERSION_11;
+  } else if (hdr.version_get() == HTTPVersion(1, 0)) {
+    return HostDBApplicationInfo::HTTP_VERSION_10;
+  } else {
+    return HostDBApplicationInfo::HTTP_VERSION_09;
+  }
+}
+
+bool
+ProxyTransaction::allow_half_open() const

Review comment:
       Can you do this separately to minimize this PR?
   
   We should remove `allow_falf_open()` from Http2Stream and Http3Transact if we have this here.

##########
File path: proxy/http/HttpConfig.cc
##########
@@ -413,6 +413,10 @@ register_stat_callbacks()
                      RECP_PERSISTENT, (int)http_ua_msecs_counts_errors_pre_accept_hangups_stat,
                      RecRawStatSyncIntMsecsToFloatSeconds);
 
+  RecRegisterRawStat(http_rsb, RECT_PROCESS, "proxy.process.http.pooled_server_connections", RECD_INT, RECP_NON_PERSISTENT,

Review comment:
       Seems like we could add this stat separately.

##########
File path: proxy/ProxyTransaction.cc
##########
@@ -220,3 +220,78 @@ ProxyTransaction::has_request_body(int64_t request_content_length, bool is_chunk
 {
   return request_content_length > 0 || is_chunked;
 }
+
+bool
+ProxyTransaction::is_read_closed() const
+{
+  return false;
+}
+
+bool
+ProxyTransaction::expect_send_trailer() const
+{
+  return false;
+}
+
+void
+ProxyTransaction::set_expect_send_trailer()
+{
+}
+bool
+ProxyTransaction::expect_receive_trailer() const
+{
+  return false;
+}
+
+void
+ProxyTransaction::set_expect_receive_trailer()
+{
+}
+
+void
+ProxyTransaction::attach_transaction(HttpSM *attach_sm)
+{
+  _sm = attach_sm;
+}
+
+void
+ProxyTransaction::release()
+{
+  HttpTxnDebug("[%" PRId64 "] session released by sm [%" PRId64 "]", _proxy_ssn ? _proxy_ssn->connection_id() : 0,
+               _sm ? _sm->sm_id : 0);
+
+  this->decrement_transactions_stat();
+
+  // Pass along the release to the session
+  if (_proxy_ssn) {
+    _proxy_ssn->release(this);
+  }
+}
+
+HostDBApplicationInfo::HttpVersion
+ProxyTransaction::get_version(HTTPHdr &hdr) const
+{
+  if (hdr.version_get() == HTTPVersion(1, 1)) {
+    return HostDBApplicationInfo::HTTP_VERSION_11;
+  } else if (hdr.version_get() == HTTPVersion(1, 0)) {
+    return HostDBApplicationInfo::HTTP_VERSION_10;
+  } else {
+    return HostDBApplicationInfo::HTTP_VERSION_09;
+  }
+}
+
+bool
+ProxyTransaction::allow_half_open() const
+{
+  return false;
+}
+
+void
+ProxyTransaction::increment_transactions_stat()

Review comment:
       I'd like to know why we want this empty implementation. Pure virtual function explicitly require subclasses to have appropriate implementation. With this empty implementation, we may forget to add the stat for HTTP/4.

##########
File path: proxy/http/Http1ClientTransaction.cc
##########
@@ -21,23 +21,28 @@
   limitations under the License.
  */
 
-#include "Http1Transaction.h"
+#include "Http1ClientTransaction.h"
 #include "Http1ClientSession.h"
 #include "HttpSM.h"
 
 void
-Http1Transaction::release(IOBufferReader *r)
+Http1ClientTransaction::release()
 {
-}
+  _proxy_ssn->clear_session_active();
+  /*
+    // Must set this inactivity count here rather than in the session because the state machine
+    // is not available then
+    MgmtInt ka_in = _sm->t_state.txn_conf->keep_alive_no_activity_timeout_in;
+    set_inactivity_timeout(HRTIME_SECONDS(ka_in));
 
-void
-Http1Transaction::reset()
-{
-  _sm = nullptr;
+    _proxy_ssn->ssn_last_txn_time = Thread::get_hrtime();

Review comment:
       `ssn_last_txn_time` is unused. I opened #7626 to remove it.

##########
File path: proxy/ProxyTransaction.h
##########
@@ -114,15 +125,19 @@ class ProxyTransaction : public VConnection
   // This function must return a non-negative number that is different for two in-progress transactions with the same proxy_ssn
   // session.
   //
-  void set_rx_error_code(ProxyError e);
-  void set_tx_error_code(ProxyError e);
+  virtual void set_rx_error_code(ProxyError e);
+  virtual void set_tx_error_code(ProxyError e);
 
   bool support_sni() const;
 
   /// Variables
   //
   HttpSessionAccept::Options upstream_outbound_options; // overwritable copy of options
 
+  void set_reader(IOBufferReader *reader);

Review comment:
       Seems like `set_reader` is only needed on Http1Transaction. The callers are only Http1ClientSession and Http1ServerSession. Nobody should call this for Http2Stream.

##########
File path: proxy/ProxySession.h
##########
@@ -127,6 +127,7 @@ class ProxySession : public VConnection, public PluginUserArgs<TS_USER_ARGS_SSN>
 
   // Non-Virtual Methods
   NetVConnection *get_netvc() const;
+  virtual void set_netvc(NetVConnection *netvc);

Review comment:
       This doesn't seem to be needed on client side. We should add `virtual` to `PoolableSession::set_netvc`.

##########
File path: proxy/PoolableSession.h
##########
@@ -72,16 +73,29 @@ class PoolableSession : public ProxySession
   TSServerSessionSharingMatchMask sharing_match = TS_SERVER_SESSION_SHARING_MATCH_MASK_NONE;
   TSServerSessionSharingPoolType sharing_pool   = TS_SERVER_SESSION_SHARING_POOL_GLOBAL;
 
-  // Keep track of connection limiting and a pointer to the
-  // singleton that keeps track of the connection counts.
-  OutboundConnTrack::Group *conn_track_group = nullptr;
+  void enable_outbound_connection_tracking(OutboundConnTrack::Group *group);
+  void release_outbound_comnection_tracking();

Review comment:
       Typo: comnection

##########
File path: proxy/ProxyTransaction.cc
##########
@@ -220,3 +220,78 @@ ProxyTransaction::has_request_body(int64_t request_content_length, bool is_chunk
 {
   return request_content_length > 0 || is_chunked;
 }
+
+bool
+ProxyTransaction::is_read_closed() const
+{
+  return false;
+}
+
+bool
+ProxyTransaction::expect_send_trailer() const
+{
+  return false;
+}
+
+void
+ProxyTransaction::set_expect_send_trailer()
+{
+}
+bool
+ProxyTransaction::expect_receive_trailer() const
+{
+  return false;
+}
+
+void
+ProxyTransaction::set_expect_receive_trailer()
+{
+}
+
+void
+ProxyTransaction::attach_transaction(HttpSM *attach_sm)
+{
+  _sm = attach_sm;
+}
+
+void
+ProxyTransaction::release()
+{
+  HttpTxnDebug("[%" PRId64 "] session released by sm [%" PRId64 "]", _proxy_ssn ? _proxy_ssn->connection_id() : 0,
+               _sm ? _sm->sm_id : 0);
+
+  this->decrement_transactions_stat();

Review comment:
       This is done in `ProxyTransaction::transaction_done` too. Maybe double decrement?

##########
File path: proxy/hdrs/HdrToken.cc
##########
@@ -227,7 +227,10 @@ static HdrTokenFieldInfo _hdrtoken_strs_field_initializers[] = {
   {"Strict-Transport-Security", MIME_SLOTID_NONE, MIME_PRESENCE_NONE, (HTIF_MULTVALS)},
   {"Subject", MIME_SLOTID_NONE, MIME_PRESENCE_SUBJECT, HTIF_NONE},
   {"Summary", MIME_SLOTID_NONE, MIME_PRESENCE_SUMMARY, HTIF_NONE},
-  {"TE", MIME_SLOTID_TE, MIME_PRESENCE_TE, (HTIF_COMMAS | HTIF_MULTVALS | HTIF_HOPBYHOP)},
+  // Need to figure out why this cannot be handled as hop by hop.  If it is hop-by-hop
+  // the information is not propagated for gRPC
+  //{"TE", MIME_SLOTID_TE, MIME_PRESENCE_TE, (HTIF_COMMAS | HTIF_MULTVALS | HTIF_HOPBYHOP)},
+  {"TE", MIME_SLOTID_TE, MIME_PRESENCE_TE, (HTIF_COMMAS | HTIF_MULTVALS)},

Review comment:
       Assuming this is not harmful for non-gRPC traffic. Having gRPC support may be worth temporally removing hop-by-hop flag, but in case we do that, we should file an issue for it.
   
   Making this change separately would make reverting the change easy in case it affects non-gRPC traffic.

##########
File path: iocore/net/I_NetVConnection.h
##########
@@ -39,6 +39,7 @@
 #include "YamlSNIConfig.h"
 #include "tscpp/util/TextView.h"
 #include "tscore/IpMap.h"
+#include "P_ALPNSupport.h"

Review comment:
       Hmmm, for MAX_ALPN_STRING? I don't want to bring in this dependency into I_NetVConnection.
   
   I can see you want to put ALPN stuff into one place, but the limitation (30) doesn't come from ALPN spec, ALPNSupport, or the new function you added. It's a random number defined on users' side. Furthermore, appropriate size depends on ATS user's configuration.
   
   How about having the constant value in ink_defs.h? I think it's pretty similar to TS_MAX_HOST_NAME_LEN in terms of that we need to have some number as an implementation restriction.

##########
File path: proxy/hdrs/HTTP.h
##########
@@ -580,6 +580,27 @@ class HTTPHdr : public MIMEHdr
   const char *path_get(int *length ///< Storage for path length.
   );
 
+  /** Get the URL matrix params.
+      This is a reference, not allocated.
+      @return A pointer to the matrix params or @c NULL if there is no valid URL.
+  */
+  const char *params_get(int *length ///< Storage for param length.

Review comment:
       We should add these functions separately. Other changes that may happen in the future probably don't want to depend on this H2 to origin commit. Those may be backported to old versions.

##########
File path: proxy/http/Http1ClientTransaction.h
##########
@@ -0,0 +1,52 @@
+/** @file
+
+  Http1ClientTransaction.h - The Client Transaction class for Http1*
+
+  @section license License
+
+  Licensed to the Apache Software Foundation (ASF) under one
+  or more contributor license agreements.  See the NOTICE file
+  distributed with this work for additional information
+  regarding copyright ownership.  The ASF licenses this file
+  to you under the Apache License, Version 2.0 (the
+  "License"); you may not use this file except in compliance
+  with the License.  You may obtain a copy of the License at
+
+      http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+ */
+
+#pragma once
+
+#include "Http1Transaction.h"
+
+class Http1ClientTransaction : public Http1Transaction
+{
+public:
+  using super_type = Http1Transaction;
+
+  Http1ClientTransaction() {}
+  Http1ClientTransaction(ProxySession *session) : super_type(session) {}
+
+  ////////////////////
+  // Methods
+  void release() override;
+  // void destroy() override; // todo make ~Http1Transaction()

Review comment:
       Remove it.

##########
File path: proxy/http/Http1ClientTransaction.h
##########
@@ -0,0 +1,52 @@
+/** @file
+
+  Http1ClientTransaction.h - The Client Transaction class for Http1*
+
+  @section license License
+
+  Licensed to the Apache Software Foundation (ASF) under one
+  or more contributor license agreements.  See the NOTICE file
+  distributed with this work for additional information
+  regarding copyright ownership.  The ASF licenses this file
+  to you under the Apache License, Version 2.0 (the
+  "License"); you may not use this file except in compliance
+  with the License.  You may obtain a copy of the License at
+
+      http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+ */
+
+#pragma once
+
+#include "Http1Transaction.h"
+
+class Http1ClientTransaction : public Http1Transaction
+{
+public:
+  using super_type = Http1Transaction;
+
+  Http1ClientTransaction() {}
+  Http1ClientTransaction(ProxySession *session) : super_type(session) {}
+
+  ////////////////////
+  // Methods
+  void release() override;
+  // void destroy() override; // todo make ~Http1Transaction()
+
+  bool allow_half_open() const override;
+  void transaction_done() override;
+  // int get_transaction_id() const override;

Review comment:
       Remove it.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficserver] shinrich commented on pull request #7622: HTTP/2 to origin

Posted by GitBox <gi...@apache.org>.
shinrich commented on pull request #7622:
URL: https://github.com/apache/trafficserver/pull/7622#issuecomment-814898162


   [approve ci autest]


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficserver] shinrich commented on a change in pull request #7622: HTTP/2 to origin

Posted by GitBox <gi...@apache.org>.
shinrich commented on a change in pull request #7622:
URL: https://github.com/apache/trafficserver/pull/7622#discussion_r607206823



##########
File path: iocore/net/UnixNetVConnection.cc
##########
@@ -431,6 +432,9 @@ write_to_net_io(NetHandler *nh, UnixNetVConnection *vc, EThread *thread)
   if (towrite != ntodo && buf.writer()->write_avail()) {
     if (write_signal_and_update(VC_EVENT_WRITE_READY, vc) != EVENT_CONT) {
       return;
+    } else if (c != s->vio.cont) { /* The write vio was updated in the handler */

Review comment:
       This is part of PR #7578.
   
   Are you objecting to the short variable name?
   




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficserver] bneradt removed a comment on pull request #7622: HTTP/2 to origin

Posted by GitBox <gi...@apache.org>.
bneradt removed a comment on pull request #7622:
URL: https://github.com/apache/trafficserver/pull/7622#issuecomment-815126966


   [approve ci]


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficserver] shinrich commented on a change in pull request #7622: HTTP/2 to origin

Posted by GitBox <gi...@apache.org>.
shinrich commented on a change in pull request #7622:
URL: https://github.com/apache/trafficserver/pull/7622#discussion_r609099358



##########
File path: proxy/http/HttpSM.cc
##########
@@ -289,6 +278,160 @@ HttpVCTable::cleanup_all()
   }
 }
 
+class ConnectingEntry : public Continuation
+{
+public:
+  std::set<HttpSM *> _connect_sms;
+  NetVConnection *_netvc        = nullptr;
+  IOBufferReader *_netvc_reader = nullptr;
+  MIOBuffer *_netvc_read_buffer = nullptr;
+  std::string sni;
+  std::string cert_name;
+  IpEndpoint _ipaddr;
+  std::string hostname;
+  Action *_pending_action = nullptr;
+  NetVCOptions opt;
+
+  void remove_entry();
+  int state_http_server_open(int event, void *data);
+  static PoolableSession *create_server_session(HttpSM *root_sm, NetVConnection *netvc, MIOBuffer *netvc_read_buffer,
+                                                IOBufferReader *netvc_reader);
+};
+
+struct IpHelper {
+  size_t
+  operator()(IpEndpoint const &arg) const
+  {
+    return IpAddr{&arg.sa}.hash();
+  }
+  bool
+  operator()(IpEndpoint const &arg1, IpEndpoint const &arg2) const
+  {
+    return ats_ip_addr_port_eq(&arg1.sa, &arg2.sa);
+  }
+};
+using ConnectingIpPool = std::unordered_multimap<IpEndpoint, ConnectingEntry *, IpHelper>;
+
+class ConnectingPool
+{
+public:
+  ConnectingPool() {}
+  ConnectingIpPool m_ip_pool;
+};
+
+void
+initialize_thread_for_connecting_pools(EThread *thread)
+{
+  if (thread->connecting_pool == nullptr) {
+    thread->connecting_pool = new ConnectingPool();
+  }
+}
+
+int
+ConnectingEntry::state_http_server_open(int event, void *data)
+{
+  Debug("http_connect", "entered inside ConnectingEntry::state_http_server_open");
+
+  switch (event) {
+  case NET_EVENT_OPEN: {
+    _netvc                 = static_cast<NetVConnection *>(data);
+    UnixNetVConnection *vc = static_cast<UnixNetVConnection *>(_netvc);
+    ink_release_assert(_pending_action == nullptr || _pending_action->continuation == vc->get_action()->continuation);
+    _pending_action = nullptr;
+    Debug("http_connect", "ConnectingEntrysetting handler for TCP handshake");
+    // Just want to get a write-ready event so we know that the TCP handshake is complete.
+    // The buffer we create will be handed over to the eventually created server session
+    _netvc_read_buffer = new_MIOBuffer(HTTP_SERVER_RESP_HDR_BUFFER_INDEX);
+    _netvc_reader      = _netvc_read_buffer->alloc_reader();
+    _netvc->do_io_write(this, 1, _netvc_reader);
+    ink_release_assert(!_connect_sms.empty());
+    if (!_connect_sms.empty()) {
+      HttpSM *prime_connect_sm = *(_connect_sms.begin());
+      _netvc->set_inactivity_timeout(prime_connect_sm->get_server_connect_timeout());
+    }
+    ink_release_assert(_pending_action == nullptr);
+    return 0;
+  }
+  case VC_EVENT_READ_COMPLETE:
+  case VC_EVENT_WRITE_READY:
+  case VC_EVENT_WRITE_COMPLETE: {
+    Debug("http_connect", "Kick off %" PRId64 " state machines waiting for origin", _connect_sms.size());
+    this->remove_entry();
+    _netvc->do_io_write(nullptr, 0, nullptr);
+    if (!_connect_sms.empty()) {
+      auto prime_iter = _connect_sms.rbegin();
+      ink_release_assert(prime_iter != _connect_sms.rend());
+      PoolableSession *new_session = ConnectingEntry::create_server_session(*prime_iter, _netvc, _netvc_read_buffer, _netvc_reader);
+      _netvc                       = nullptr;
+
+      // Did we end up with a multiplexing session?
+      int count = 0;
+      if (new_session->is_multiplexing()) {
+        // Hand off to all queued up ConnectSM's.
+        while (!_connect_sms.empty()) {
+          Debug("http_connect", "ConnectingEntry Pass along CONNECT_EVENT_TXN %d", count++);
+          auto entry = _connect_sms.begin();
+
+          SCOPED_MUTEX_LOCK(lock, (*entry)->mutex, this_ethread());
+          (*entry)->handleEvent(CONNECT_EVENT_TXN, new_session);
+          _connect_sms.erase(entry);

Review comment:
       It is a per thread data structure.  So yes it is thread safe.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficserver] shinrich commented on a change in pull request #7622: HTTP/2 to origin

Posted by GitBox <gi...@apache.org>.
shinrich commented on a change in pull request #7622:
URL: https://github.com/apache/trafficserver/pull/7622#discussion_r612072778



##########
File path: include/tscore/Scalar.h
##########
@@ -33,15 +33,15 @@
 #include <type_traits>
 #include "tscore/BufferWriter.h"
 
-namespace tag
-{
-struct generic;
-}
-
 namespace ts
 {
 template <intmax_t N, typename C, typename T> class Scalar;
 
+namespace tag
+{
+  struct generic;
+}
+

Review comment:
       @SolidWallOfCode pulled this into a separate PR which has landed. PR #7690




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficserver] maskit commented on a change in pull request #7622: HTTP/2 to origin

Posted by GitBox <gi...@apache.org>.
maskit commented on a change in pull request #7622:
URL: https://github.com/apache/trafficserver/pull/7622#discussion_r647127060



##########
File path: proxy/http/HttpSM.cc
##########
@@ -289,6 +277,160 @@ HttpVCTable::cleanup_all()
   }
 }
 
+class ConnectingEntry : public Continuation
+{
+public:
+  std::set<HttpSM *> _connect_sms;
+  NetVConnection *_netvc        = nullptr;
+  IOBufferReader *_netvc_reader = nullptr;
+  MIOBuffer *_netvc_read_buffer = nullptr;
+  std::string sni;
+  std::string cert_name;
+  IpEndpoint _ipaddr;
+  std::string hostname;
+  Action *_pending_action = nullptr;
+  NetVCOptions opt;
+
+  void remove_entry();
+  int state_http_server_open(int event, void *data);
+  static PoolableSession *create_server_session(HttpSM *root_sm, NetVConnection *netvc, MIOBuffer *netvc_read_buffer,
+                                                IOBufferReader *netvc_reader);
+};
+
+struct IpHelper {
+  size_t
+  operator()(IpEndpoint const &arg) const
+  {
+    return IpAddr{&arg.sa}.hash();
+  }
+  bool
+  operator()(IpEndpoint const &arg1, IpEndpoint const &arg2) const
+  {
+    return ats_ip_addr_port_eq(&arg1.sa, &arg2.sa);
+  }
+};
+using ConnectingIpPool = std::unordered_multimap<IpEndpoint, ConnectingEntry *, IpHelper>;
+
+class ConnectingPool
+{
+public:
+  ConnectingPool() {}
+  ConnectingIpPool m_ip_pool;
+};
+
+void
+initialize_thread_for_connecting_pools(EThread *thread)
+{
+  if (thread->connecting_pool == nullptr) {
+    thread->connecting_pool = new ConnectingPool();
+  }
+}
+
+int
+ConnectingEntry::state_http_server_open(int event, void *data)
+{
+  Debug("http_connect", "entered inside ConnectingEntry::state_http_server_open");
+
+  switch (event) {
+  case NET_EVENT_OPEN: {
+    _netvc                 = static_cast<NetVConnection *>(data);
+    UnixNetVConnection *vc = static_cast<UnixNetVConnection *>(_netvc);
+    ink_release_assert(_pending_action == nullptr || _pending_action->continuation == vc->get_action()->continuation);
+    _pending_action = nullptr;
+    Debug("http_connect", "ConnectingEntrysetting handler for TCP handshake");
+    // Just want to get a write-ready event so we know that the TCP handshake is complete.
+    // The buffer we create will be handed over to the eventually created server session
+    _netvc_read_buffer = new_MIOBuffer(HTTP_SERVER_RESP_HDR_BUFFER_INDEX);
+    _netvc_reader      = _netvc_read_buffer->alloc_reader();
+    _netvc->do_io_write(this, 1, _netvc_reader);
+    ink_release_assert(!_connect_sms.empty());
+    if (!_connect_sms.empty()) {
+      HttpSM *prime_connect_sm = *(_connect_sms.begin());
+      _netvc->set_inactivity_timeout(prime_connect_sm->get_server_connect_timeout());
+    }
+    ink_release_assert(_pending_action == nullptr);
+    return 0;
+  }
+  case VC_EVENT_READ_COMPLETE:
+  case VC_EVENT_WRITE_READY:
+  case VC_EVENT_WRITE_COMPLETE: {
+    Debug("http_connect", "Kick off %" PRId64 " state machines waiting for origin", _connect_sms.size());
+    this->remove_entry();
+    _netvc->do_io_write(nullptr, 0, nullptr);
+    if (!_connect_sms.empty()) {
+      auto prime_iter = _connect_sms.rbegin();
+      ink_release_assert(prime_iter != _connect_sms.rend());
+      PoolableSession *new_session = (*prime_iter)->create_server_session(_netvc, _netvc_read_buffer, _netvc_reader);
+      _netvc                       = nullptr;
+
+      // Did we end up with a multiplexing session?
+      int count = 0;
+      if (new_session->is_multiplexing()) {
+        // Hand off to all queued up ConnectSM's.

Review comment:
       Also, I know this is going to be complicated, but we can retry a transaction on another session if an H2 transaction was closed by RST_STREAM with REFUSED_STREAM error code.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficserver] shinrich commented on a change in pull request #7622: HTTP/2 to origin

Posted by GitBox <gi...@apache.org>.
shinrich commented on a change in pull request #7622:
URL: https://github.com/apache/trafficserver/pull/7622#discussion_r607936742



##########
File path: iocore/net/UnixNetVConnection.cc
##########
@@ -431,6 +432,9 @@ write_to_net_io(NetHandler *nh, UnixNetVConnection *vc, EThread *thread)
   if (towrite != ntodo && buf.writer()->write_avail()) {
     if (write_signal_and_update(VC_EVENT_WRITE_READY, vc) != EVENT_CONT) {
       return;
+    } else if (c != s->vio.cont) { /* The write vio was updated in the handler */

Review comment:
       Ah, it needs to be save aside since we are checking whether the cont assignment in the vio object changed during the call to write_signal_and_update.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficserver] shinrich commented on pull request #7622: HTTP/2 to origin

Posted by GitBox <gi...@apache.org>.
shinrich commented on pull request #7622:
URL: https://github.com/apache/trafficserver/pull/7622#issuecomment-886790190


   In my production testing, I am having problems with increased ERR_CONN_FAIL from another ATS box acting as origin as described in issue #8163.  Adjusting the error rate up helped some, but the ERR_CONN_FAIL was still significantly higher for that origin.   The mechanism was the same.  The origin was returning GOAWAY with no error which caused all other active streams to be terminated.
   
   Finally, Friday I notice that the H2 client logic was using the "Connection: closed" header as described below to start draining the session.
   
   https://docs.trafficserver.apache.org/en/latest/admin-guide/plugins/header_rewrite.en.html?highlight=drain#close-connections-for-draining
   
   Unfortunately, I cannot get the logic to keep around existing streams until the are finished working.  I just move my ERR_CONN_FAIL errors to ERR_CLIENT_ABORT or ERR_READ_TIMEOUT errors.  With recent changes to improve header error checking, I've been adding "Connection: close" headers not worrying about the HTTP/2 case since H2 should be ignoring the Connection header.  Forgetting about this draining feature.
   
   Not sure how to deal with this.  Could add H1/H2 checks into the HttpSM to only really add the "Connection: close" for Http/1 connections.  Or more likely add a virtual method onto the ProxyTransaction to do that logic.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@trafficserver.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficserver] maskit commented on a change in pull request #7622: HTTP/2 to origin

Posted by GitBox <gi...@apache.org>.
maskit commented on a change in pull request #7622:
URL: https://github.com/apache/trafficserver/pull/7622#discussion_r647124580



##########
File path: proxy/http/HttpSM.cc
##########
@@ -289,6 +277,160 @@ HttpVCTable::cleanup_all()
   }
 }
 
+class ConnectingEntry : public Continuation
+{
+public:
+  std::set<HttpSM *> _connect_sms;
+  NetVConnection *_netvc        = nullptr;
+  IOBufferReader *_netvc_reader = nullptr;
+  MIOBuffer *_netvc_read_buffer = nullptr;
+  std::string sni;
+  std::string cert_name;
+  IpEndpoint _ipaddr;
+  std::string hostname;
+  Action *_pending_action = nullptr;
+  NetVCOptions opt;
+
+  void remove_entry();
+  int state_http_server_open(int event, void *data);
+  static PoolableSession *create_server_session(HttpSM *root_sm, NetVConnection *netvc, MIOBuffer *netvc_read_buffer,
+                                                IOBufferReader *netvc_reader);
+};
+
+struct IpHelper {
+  size_t
+  operator()(IpEndpoint const &arg) const
+  {
+    return IpAddr{&arg.sa}.hash();
+  }
+  bool
+  operator()(IpEndpoint const &arg1, IpEndpoint const &arg2) const
+  {
+    return ats_ip_addr_port_eq(&arg1.sa, &arg2.sa);
+  }
+};
+using ConnectingIpPool = std::unordered_multimap<IpEndpoint, ConnectingEntry *, IpHelper>;
+
+class ConnectingPool
+{
+public:
+  ConnectingPool() {}
+  ConnectingIpPool m_ip_pool;
+};
+
+void
+initialize_thread_for_connecting_pools(EThread *thread)
+{
+  if (thread->connecting_pool == nullptr) {
+    thread->connecting_pool = new ConnectingPool();
+  }
+}
+
+int
+ConnectingEntry::state_http_server_open(int event, void *data)
+{
+  Debug("http_connect", "entered inside ConnectingEntry::state_http_server_open");
+
+  switch (event) {
+  case NET_EVENT_OPEN: {
+    _netvc                 = static_cast<NetVConnection *>(data);
+    UnixNetVConnection *vc = static_cast<UnixNetVConnection *>(_netvc);
+    ink_release_assert(_pending_action == nullptr || _pending_action->continuation == vc->get_action()->continuation);
+    _pending_action = nullptr;
+    Debug("http_connect", "ConnectingEntrysetting handler for TCP handshake");
+    // Just want to get a write-ready event so we know that the TCP handshake is complete.
+    // The buffer we create will be handed over to the eventually created server session
+    _netvc_read_buffer = new_MIOBuffer(HTTP_SERVER_RESP_HDR_BUFFER_INDEX);
+    _netvc_reader      = _netvc_read_buffer->alloc_reader();
+    _netvc->do_io_write(this, 1, _netvc_reader);
+    ink_release_assert(!_connect_sms.empty());
+    if (!_connect_sms.empty()) {
+      HttpSM *prime_connect_sm = *(_connect_sms.begin());
+      _netvc->set_inactivity_timeout(prime_connect_sm->get_server_connect_timeout());
+    }
+    ink_release_assert(_pending_action == nullptr);
+    return 0;
+  }
+  case VC_EVENT_READ_COMPLETE:
+  case VC_EVENT_WRITE_READY:
+  case VC_EVENT_WRITE_COMPLETE: {
+    Debug("http_connect", "Kick off %" PRId64 " state machines waiting for origin", _connect_sms.size());
+    this->remove_entry();
+    _netvc->do_io_write(nullptr, 0, nullptr);
+    if (!_connect_sms.empty()) {
+      auto prime_iter = _connect_sms.rbegin();
+      ink_release_assert(prime_iter != _connect_sms.rend());
+      PoolableSession *new_session = (*prime_iter)->create_server_session(_netvc, _netvc_read_buffer, _netvc_reader);
+      _netvc                       = nullptr;
+
+      // Did we end up with a multiplexing session?
+      int count = 0;
+      if (new_session->is_multiplexing()) {
+        // Hand off to all queued up ConnectSM's.

Review comment:
       You mean `is_peer_concurrent_stream_max` and `add_session`, right? That is fine (except direct access to the pool), but we have a session object here and could check the limit again to not assign too many transactions to the session.
   
   And that should work for the both multiplexed and non-multiplexed sessions. I'm trying to suggest something like below:
   ```cpp
   while (!_connect_sms.empty()) {
     sm = _sm_connect_sms->front();
     succeeded = new_session->assign_transaction(sm); // Returns false if a session has too many txn assigned
     if (succeeded) {
       _connect_sms.pop_front();
     } else {
       break;
     }
   }
   if (!_connect_sms.empty()) {
     // Prepare for remaining transactions, and wait for next opportunity
     // For H1, create another session
     // For H2, maybe do nothing (I'm not sure if we should have multiple sessions for the same server)
   }
   ```




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficserver] maskit commented on a change in pull request #7622: HTTP/2 to origin

Posted by GitBox <gi...@apache.org>.
maskit commented on a change in pull request #7622:
URL: https://github.com/apache/trafficserver/pull/7622#discussion_r645351479



##########
File path: proxy/http/HttpSessionManager.cc
##########
@@ -398,7 +404,9 @@ HttpSessionManager::_acquire_session(sockaddr const *ip, CryptoHash const &hostn
     // Now check to see if we have a connection in our shared connection pool
     EThread *ethread = this_ethread();
     Ptr<ProxyMutex> pool_mutex =
-      (TS_SERVER_SESSION_SHARING_POOL_THREAD == pool_type) ? ethread->server_session_pool->mutex : m_g_pool->mutex;
+      (TS_SERVER_SESSION_SHARING_POOL_THREAD == sm->t_state.http_config_param->server_session_sharing_pool) ?

Review comment:
       `pool_type` that is passed by the caller is used after acquiring the lock. Doesn't it cause mismatch?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficserver] shinrich commented on a change in pull request #7622: HTTP/2 to origin

Posted by GitBox <gi...@apache.org>.
shinrich commented on a change in pull request #7622:
URL: https://github.com/apache/trafficserver/pull/7622#discussion_r607204769



##########
File path: proxy/logging/LogAccess.cc
##########
@@ -1782,6 +1782,40 @@ LogAccess::marshal_client_req_protocol_version(char *buf)
   return len;
 }
 
+/*-------------------------------------------------------------------------
+  -------------------------------------------------------------------------*/
+
+int
+LogAccess::marshal_server_req_protocol_version(char *buf)

Review comment:
       Added PR #7680




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficserver] shinrich commented on a change in pull request #7622: HTTP/2 to origin

Posted by GitBox <gi...@apache.org>.
shinrich commented on a change in pull request #7622:
URL: https://github.com/apache/trafficserver/pull/7622#discussion_r607154810



##########
File path: tests/gold_tests/redirect/redirect_actions.test.py
##########
@@ -184,7 +184,7 @@ class ActionE(Enum):
     Follow = {'config': 'follow', 'expectedStatusLine': 'HTTP/1.1 204 No Content\r\n'}
 
     # Added to test failure modes.
-    Break = {'expectedStatusLine': 'HTTP/1.1 502 Cannot find server.\r\n'}
+    Break = {'expectedStatusLine': 'HTTP/1.1 500 Cannot find server.\r\n'}

Review comment:
       This also is related to PR #7580.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficserver] bneradt commented on pull request #7622: HTTP/2 to origin

Posted by GitBox <gi...@apache.org>.
bneradt commented on pull request #7622:
URL: https://github.com/apache/trafficserver/pull/7622#issuecomment-814195424


   [approve ci]


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficserver] shinrich commented on a change in pull request #7622: HTTP/2 to origin

Posted by GitBox <gi...@apache.org>.
shinrich commented on a change in pull request #7622:
URL: https://github.com/apache/trafficserver/pull/7622#discussion_r647401604



##########
File path: proxy/http/HttpSessionManager.cc
##########
@@ -165,9 +165,13 @@ ServerSessionPool::acquireSession(sockaddr const *addr, CryptoHash const &hostna
     }
     if (zret == HSM_DONE) {
       to_return = first;
-      this->removeSession(to_return);
-    } else if (first != m_fqdn_pool.end()) {
-      Debug("http_ss", "Failed find entry due to name mismatch %s", sm->t_state.current.server->name);
+      if (!to_return->is_multiplexing()) {
+        this->removeSession(to_return);
+      }
+    } else {
+      if (first != m_fqdn_pool.end()) {

Review comment:
       Done




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficserver] shinrich commented on a change in pull request #7622: HTTP/2 to origin

Posted by GitBox <gi...@apache.org>.
shinrich commented on a change in pull request #7622:
URL: https://github.com/apache/trafficserver/pull/7622#discussion_r612072949



##########
File path: proxy/ProxySession.h
##########
@@ -156,6 +156,12 @@ class ProxySession : public VConnection, public PluginUserArgs<TS_USER_ARGS_SSN>
   void do_io_shutdown(ShutdownHowTo_t howto) override;
   void reenable(VIO *vio) override;
 
+  virtual ProxyTransaction *

Review comment:
       This is a good default which is used by Http2ClientSession at least.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficserver] shinrich commented on a change in pull request #7622: HTTP/2 to origin

Posted by GitBox <gi...@apache.org>.
shinrich commented on a change in pull request #7622:
URL: https://github.com/apache/trafficserver/pull/7622#discussion_r607154668



##########
File path: tests/gold_tests/continuations/openclose.test.py
##########
@@ -51,7 +52,7 @@
         ts.Variables.port, server.Variables.Port)
 )
 
-cmd = 'curl -vs -H "host:oc.test" http://127.0.0.1:{0}'.format(ts.Variables.port)
+cmd = 'curl -H "Connection: close" -vs -H "host:oc.test" http://127.0.0.1:{0}'.format(ts.Variables.port)

Review comment:
       Sure, I'll pull this out.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficserver] shinrich commented on pull request #7622: HTTP/2 to origin

Posted by GitBox <gi...@apache.org>.
shinrich commented on pull request #7622:
URL: https://github.com/apache/trafficserver/pull/7622#issuecomment-818922232


   [approve ci autest]


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficserver] shinrich commented on a change in pull request #7622: HTTP/2 to origin

Posted by GitBox <gi...@apache.org>.
shinrich commented on a change in pull request #7622:
URL: https://github.com/apache/trafficserver/pull/7622#discussion_r608251574



##########
File path: proxy/http/HttpSessionManager.cc
##########
@@ -190,9 +196,11 @@ ServerSessionPool::acquireSession(sockaddr const *addr, CryptoHash const &hostna
     }
     if (zret == HSM_DONE) {
       to_return = first;
-      HTTP_DECREMENT_DYN_STAT(http_pooled_server_connections_stat);
-      m_ip_pool.erase(first);
-      m_fqdn_pool.erase(to_return);
+      if (!to_return->is_multiplexing()) {
+        HTTP_DECREMENT_DYN_STAT(http_pooled_server_connections_stat);
+        m_ip_pool.erase(first);
+        m_fqdn_pool.erase(to_return);

Review comment:
       Added uses of addSession and removeSession directly within the HttpSessionManager.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficserver] keesspoelstra commented on a change in pull request #7622: HTTP/2 to origin

Posted by GitBox <gi...@apache.org>.
keesspoelstra commented on a change in pull request #7622:
URL: https://github.com/apache/trafficserver/pull/7622#discussion_r724177804



##########
File path: proxy/http/HttpSessionManager.cc
##########
@@ -190,7 +192,9 @@ ServerSessionPool::acquireSession(sockaddr const *addr, CryptoHash const &hostna
     }
     if (zret == HSM_DONE) {
       to_return = first;
-      this->removeSession(to_return);
+      if (!to_return->is_multiplexing()) {

Review comment:
       We probably want an early check in the while statement above to see if the session is maxed out, this to prevent ending up much later in the flow and finding out we can't add any more streams




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@trafficserver.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficserver] shinrich closed pull request #7622: HTTP/2 to origin

Posted by GitBox <gi...@apache.org>.
shinrich closed pull request #7622:
URL: https://github.com/apache/trafficserver/pull/7622


   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@trafficserver.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficserver] shinrich commented on a change in pull request #7622: HTTP/2 to origin

Posted by GitBox <gi...@apache.org>.
shinrich commented on a change in pull request #7622:
URL: https://github.com/apache/trafficserver/pull/7622#discussion_r602358785



##########
File path: proxy/http/HttpConfig.cc
##########
@@ -413,6 +413,10 @@ register_stat_callbacks()
                      RECP_PERSISTENT, (int)http_ua_msecs_counts_errors_pre_accept_hangups_stat,
                      RecRawStatSyncIntMsecsToFloatSeconds);
 
+  RecRegisterRawStat(http_rsb, RECT_PROCESS, "proxy.process.http.pooled_server_connections", RECD_INT, RECP_NON_PERSISTENT,

Review comment:
       Yes, that is useful independent of the H2 to origin.  Pulled out into PR #7627 




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficserver] shinrich commented on a change in pull request #7622: HTTP/2 to origin

Posted by GitBox <gi...@apache.org>.
shinrich commented on a change in pull request #7622:
URL: https://github.com/apache/trafficserver/pull/7622#discussion_r607153838



##########
File path: proxy/logging/LogAccess.cc
##########
@@ -1782,6 +1782,40 @@ LogAccess::marshal_client_req_protocol_version(char *buf)
   return len;
 }
 
+/*-------------------------------------------------------------------------
+  -------------------------------------------------------------------------*/
+
+int
+LogAccess::marshal_server_req_protocol_version(char *buf)

Review comment:
       Sure we could land a very simple version of this first.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficserver] bneradt commented on pull request #7622: HTTP/2 to origin

Posted by GitBox <gi...@apache.org>.
bneradt commented on pull request #7622:
URL: https://github.com/apache/trafficserver/pull/7622#issuecomment-857050053


   [approve ci clang-format]


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficserver] shinrich commented on a change in pull request #7622: HTTP/2 to origin

Posted by GitBox <gi...@apache.org>.
shinrich commented on a change in pull request #7622:
URL: https://github.com/apache/trafficserver/pull/7622#discussion_r602413749



##########
File path: proxy/ProxyTransaction.h
##########
@@ -114,15 +125,19 @@ class ProxyTransaction : public VConnection
   // This function must return a non-negative number that is different for two in-progress transactions with the same proxy_ssn
   // session.
   //
-  void set_rx_error_code(ProxyError e);
-  void set_tx_error_code(ProxyError e);
+  virtual void set_rx_error_code(ProxyError e);
+  virtual void set_tx_error_code(ProxyError e);
 
   bool support_sni() const;
 
   /// Variables
   //
   HttpSessionAccept::Options upstream_outbound_options; // overwritable copy of options
 
+  void set_reader(IOBufferReader *reader);

Review comment:
       Adjusting




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficserver] shinrich commented on a change in pull request #7622: HTTP/2 to origin

Posted by GitBox <gi...@apache.org>.
shinrich commented on a change in pull request #7622:
URL: https://github.com/apache/trafficserver/pull/7622#discussion_r602527087



##########
File path: proxy/ProxyTransaction.cc
##########
@@ -220,3 +220,78 @@ ProxyTransaction::has_request_body(int64_t request_content_length, bool is_chunk
 {
   return request_content_length > 0 || is_chunked;
 }
+
+bool
+ProxyTransaction::is_read_closed() const
+{
+  return false;
+}
+
+bool
+ProxyTransaction::expect_send_trailer() const
+{
+  return false;
+}
+
+void
+ProxyTransaction::set_expect_send_trailer()
+{
+}
+bool
+ProxyTransaction::expect_receive_trailer() const
+{
+  return false;
+}
+
+void
+ProxyTransaction::set_expect_receive_trailer()
+{
+}
+
+void
+ProxyTransaction::attach_transaction(HttpSM *attach_sm)
+{
+  _sm = attach_sm;
+}
+
+void
+ProxyTransaction::release()
+{
+  HttpTxnDebug("[%" PRId64 "] session released by sm [%" PRId64 "]", _proxy_ssn ? _proxy_ssn->connection_id() : 0,
+               _sm ? _sm->sm_id : 0);
+
+  this->decrement_transactions_stat();
+
+  // Pass along the release to the session
+  if (_proxy_ssn) {
+    _proxy_ssn->release(this);
+  }
+}
+
+HostDBApplicationInfo::HttpVersion
+ProxyTransaction::get_version(HTTPHdr &hdr) const
+{
+  if (hdr.version_get() == HTTPVersion(1, 1)) {
+    return HostDBApplicationInfo::HTTP_VERSION_11;
+  } else if (hdr.version_get() == HTTPVersion(1, 0)) {
+    return HostDBApplicationInfo::HTTP_VERSION_10;
+  } else {
+    return HostDBApplicationInfo::HTTP_VERSION_09;
+  }
+}
+
+bool
+ProxyTransaction::allow_half_open() const

Review comment:
       Pulled into PR #7630




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficserver] shinrich commented on pull request #7622: HTTP/2 to origin

Posted by GitBox <gi...@apache.org>.
shinrich commented on pull request #7622:
URL: https://github.com/apache/trafficserver/pull/7622#issuecomment-815003611






-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficserver] shinrich commented on pull request #7622: HTTP/2 to origin

Posted by GitBox <gi...@apache.org>.
shinrich commented on pull request #7622:
URL: https://github.com/apache/trafficserver/pull/7622#issuecomment-856365532


   [approve ci autest]


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficserver] shinrich commented on pull request #7622: HTTP/2 to origin

Posted by GitBox <gi...@apache.org>.
shinrich commented on pull request #7622:
URL: https://github.com/apache/trafficserver/pull/7622#issuecomment-812051983


   Pushed an updated version of the branch based on fixes from my most recent 9.x based h2 to origin testing.  
   
   Will be working on transitioning this to a feature branch as requested in Monday's meeting.  And will be scheduling a PR walk through for next week.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficserver] shinrich commented on a change in pull request #7622: HTTP/2 to origin

Posted by GitBox <gi...@apache.org>.
shinrich commented on a change in pull request #7622:
URL: https://github.com/apache/trafficserver/pull/7622#discussion_r602431315



##########
File path: iocore/net/I_NetVConnection.h
##########
@@ -39,6 +39,7 @@
 #include "YamlSNIConfig.h"
 #include "tscpp/util/TextView.h"
 #include "tscore/IpMap.h"
+#include "P_ALPNSupport.h"

Review comment:
       Oh, nevermind.  ink_configs.h is a generated header file.  I will place it in ink.defs.h.  May eventually want to make a setting for this, but I don't think that is needed for the first phase.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficserver] shinrich commented on a change in pull request #7622: HTTP/2 to origin

Posted by GitBox <gi...@apache.org>.
shinrich commented on a change in pull request #7622:
URL: https://github.com/apache/trafficserver/pull/7622#discussion_r647638615



##########
File path: proxy/http/HttpSM.cc
##########
@@ -1801,19 +1945,51 @@ HttpSM::handle_api_return()
 }
 
 PoolableSession *
-HttpSM::create_server_session(NetVConnection *netvc)
+HttpSM::create_server_session(NetVConnection *netvc, MIOBuffer *netvc_read_buffer, IOBufferReader *netvc_reader)
 {
   HttpTransact::State &s  = this->t_state;
-  PoolableSession *retval = httpServerSessionAllocator.alloc();
+  PoolableSession *retval = nullptr;
+
+  // Figure out what protocol was negotiated
+  const unsigned char *proto  = nullptr;
+  unsigned int proto_length   = 0;
+  SSLNetVConnection *sslnetvc = dynamic_cast<SSLNetVConnection *>(netvc);
+  if (sslnetvc) {
+    SSL_get0_alpn_selected(sslnetvc->ssl, &proto, &proto_length);
+  }
 
-  retval->sharing_pool         = static_cast<TSServerSessionSharingPoolType>(s.http_config_param->server_session_sharing_pool);
-  retval->sharing_match        = static_cast<TSServerSessionSharingMatchMask>(s.txn_conf->server_session_sharing_match);
-  MIOBuffer *netvc_read_buffer = new_MIOBuffer(HTTP_SERVER_RESP_HDR_BUFFER_INDEX);
-  IOBufferReader *netvc_reader = netvc_read_buffer->alloc_reader();
+  bool add_session = false;
+  if (proto_length == 2 && memcmp(proto, "h2", 2) == 0) {
+    Http2ServerSession *session = http2ServerSessionAllocator.alloc();
+    add_session                 = true;
+    retval                      = session;
+  } else {
+    Http1ServerSession *session = httpServerSessionAllocator.alloc();
+    retval                      = session;
+  }

Review comment:
       pulled out into a static method.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficserver] shinrich commented on pull request #7622: HTTP/2 to origin

Posted by GitBox <gi...@apache.org>.
shinrich commented on pull request #7622:
URL: https://github.com/apache/trafficserver/pull/7622#issuecomment-856732336


   I'll look at breaking out the alpn logic.  It didn't occur to me you could use alpn to negotiate between HTTP/1.1 and HTTP/1.0 but I guess you can.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficserver] maskit commented on a change in pull request #7622: HTTP/2 to origin

Posted by GitBox <gi...@apache.org>.
maskit commented on a change in pull request #7622:
URL: https://github.com/apache/trafficserver/pull/7622#discussion_r630838877



##########
File path: proxy/ProxySession.h
##########
@@ -156,6 +156,12 @@ class ProxySession : public VConnection, public PluginUserArgs<TS_USER_ARGS_SSN>
   void do_io_shutdown(ShutdownHowTo_t howto) override;
   void reenable(VIO *vio) override;
 
+  virtual ProxyTransaction *

Review comment:
       I'm not sure if this is a good default implementation. What does returning `nullptr` mean?
   
   It could be:
   1. The session cannot make a new transaction at the moment due to some limitation (e.g. H2ServerSession)
   2. You are supposed to not call this function for the session (e.g. H2ClientSession)
   
   For 1, we may want to retry later.
   For 2, we may want to abort (assert), because it's a logic error.
   
   And it seems like there's no error handling for case 1.
   




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



[GitHub] [trafficserver] shinrich commented on a change in pull request #7622: HTTP/2 to origin

Posted by GitBox <gi...@apache.org>.
shinrich commented on a change in pull request #7622:
URL: https://github.com/apache/trafficserver/pull/7622#discussion_r647665919



##########
File path: proxy/http/HttpSM.cc
##########
@@ -289,6 +277,160 @@ HttpVCTable::cleanup_all()
   }
 }
 
+class ConnectingEntry : public Continuation
+{
+public:
+  std::set<HttpSM *> _connect_sms;
+  NetVConnection *_netvc        = nullptr;
+  IOBufferReader *_netvc_reader = nullptr;
+  MIOBuffer *_netvc_read_buffer = nullptr;
+  std::string sni;
+  std::string cert_name;
+  IpEndpoint _ipaddr;
+  std::string hostname;
+  Action *_pending_action = nullptr;
+  NetVCOptions opt;
+
+  void remove_entry();
+  int state_http_server_open(int event, void *data);
+  static PoolableSession *create_server_session(HttpSM *root_sm, NetVConnection *netvc, MIOBuffer *netvc_read_buffer,
+                                                IOBufferReader *netvc_reader);
+};
+
+struct IpHelper {
+  size_t
+  operator()(IpEndpoint const &arg) const
+  {
+    return IpAddr{&arg.sa}.hash();
+  }
+  bool
+  operator()(IpEndpoint const &arg1, IpEndpoint const &arg2) const
+  {
+    return ats_ip_addr_port_eq(&arg1.sa, &arg2.sa);
+  }
+};
+using ConnectingIpPool = std::unordered_multimap<IpEndpoint, ConnectingEntry *, IpHelper>;
+
+class ConnectingPool
+{
+public:
+  ConnectingPool() {}
+  ConnectingIpPool m_ip_pool;
+};
+
+void
+initialize_thread_for_connecting_pools(EThread *thread)
+{
+  if (thread->connecting_pool == nullptr) {
+    thread->connecting_pool = new ConnectingPool();
+  }
+}
+
+int
+ConnectingEntry::state_http_server_open(int event, void *data)
+{
+  Debug("http_connect", "entered inside ConnectingEntry::state_http_server_open");
+
+  switch (event) {
+  case NET_EVENT_OPEN: {
+    _netvc                 = static_cast<NetVConnection *>(data);
+    UnixNetVConnection *vc = static_cast<UnixNetVConnection *>(_netvc);
+    ink_release_assert(_pending_action == nullptr || _pending_action->continuation == vc->get_action()->continuation);
+    _pending_action = nullptr;
+    Debug("http_connect", "ConnectingEntrysetting handler for TCP handshake");
+    // Just want to get a write-ready event so we know that the TCP handshake is complete.
+    // The buffer we create will be handed over to the eventually created server session
+    _netvc_read_buffer = new_MIOBuffer(HTTP_SERVER_RESP_HDR_BUFFER_INDEX);
+    _netvc_reader      = _netvc_read_buffer->alloc_reader();
+    _netvc->do_io_write(this, 1, _netvc_reader);
+    ink_release_assert(!_connect_sms.empty());
+    if (!_connect_sms.empty()) {
+      HttpSM *prime_connect_sm = *(_connect_sms.begin());
+      _netvc->set_inactivity_timeout(prime_connect_sm->get_server_connect_timeout());
+    }
+    ink_release_assert(_pending_action == nullptr);
+    return 0;
+  }
+  case VC_EVENT_READ_COMPLETE:
+  case VC_EVENT_WRITE_READY:
+  case VC_EVENT_WRITE_COMPLETE: {
+    Debug("http_connect", "Kick off %" PRId64 " state machines waiting for origin", _connect_sms.size());
+    this->remove_entry();
+    _netvc->do_io_write(nullptr, 0, nullptr);
+    if (!_connect_sms.empty()) {
+      auto prime_iter = _connect_sms.rbegin();
+      ink_release_assert(prime_iter != _connect_sms.rend());
+      PoolableSession *new_session = (*prime_iter)->create_server_session(_netvc, _netvc_read_buffer, _netvc_reader);
+      _netvc                       = nullptr;
+
+      // Did we end up with a multiplexing session?
+      int count = 0;
+      if (new_session->is_multiplexing()) {
+        // Hand off to all queued up ConnectSM's.

Review comment:
       Added logic in HttpSM::state_http_server_open to detect if the transaction creation failed and retry the session if so.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org