You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@trafficserver.apache.org by so...@apache.org on 2015/04/06 18:45:13 UTC

[3/6] trafficserver git commit: TS-3475 Make some HTTP options configurable

TS-3475 Make some HTTP options configurable

TS-3475 Add documentation for the configurations

TS-3475 Increase the window size to what Google says

(cherry picked from commit 4810f98e746c2ac0f777bca0351b31f82fc0594d)


Project: http://git-wip-us.apache.org/repos/asf/trafficserver/repo
Commit: http://git-wip-us.apache.org/repos/asf/trafficserver/commit/f810c09a
Tree: http://git-wip-us.apache.org/repos/asf/trafficserver/tree/f810c09a
Diff: http://git-wip-us.apache.org/repos/asf/trafficserver/diff/f810c09a

Branch: refs/heads/5.3.x
Commit: f810c09a3c63b92eb6a7fb2821c7ee02525a1107
Parents: 26fe759
Author: Leif Hedstrom <zw...@apache.org>
Authored: Thu Apr 2 11:57:58 2015 -0600
Committer: Phil Sorber <so...@apache.org>
Committed: Mon Apr 6 10:29:52 2015 -0600

----------------------------------------------------------------------
 CHANGES                                         |  3 ++
 .../configuration/records.config.en.rst         | 39 +++++++++++++++++++-
 mgmt/RecordsConfig.cc                           | 10 +++++
 proxy/Main.cc                                   |  5 +++
 proxy/http/HttpClientSession.cc                 |  1 +
 proxy/http2/HTTP2.cc                            | 23 +++++++++++-
 proxy/http2/HTTP2.h                             | 25 +++++++++++--
 proxy/http2/Http2ClientSession.cc               |  7 ++--
 proxy/http2/Http2ClientSession.h                |  3 ++
 proxy/http2/Http2ConnectionState.cc             |  6 +--
 proxy/http2/Http2ConnectionState.h              | 26 ++++++++++---
 11 files changed, 128 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f810c09a/CHANGES
----------------------------------------------------------------------
diff --git a/CHANGES b/CHANGES
index 29f8fd3..def7f30 100644
--- a/CHANGES
+++ b/CHANGES
@@ -1,6 +1,9 @@
                                                          -*- coding: utf-8 -*-
 Changes with Apache Traffic Server 5.3.0
 
+  *) [TS-3475] Add a few records.config options for controlling the HTTP/2
+   SETTINGS. Thanks to Masakazu for all the reviews and additions.
+
   *) [TS-3493] Grow window sizes as part of HTTP session setup.
    Author: Masakazu Kitajo <m4...@gmail.com>.
 

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f810c09a/doc/reference/configuration/records.config.en.rst
----------------------------------------------------------------------
diff --git a/doc/reference/configuration/records.config.en.rst b/doc/reference/configuration/records.config.en.rst
index cce3b6b..333e29f 100644
--- a/doc/reference/configuration/records.config.en.rst
+++ b/doc/reference/configuration/records.config.en.rst
@@ -2488,6 +2488,42 @@ ICP Configuration
 
    Specifies the timeout used for ICP queries.
 
+HTTP/2 Configuration
+====================
+
+
+.. ts:cv:: CONFIG proxy.config.http2.max_concurrent_streams_in INT 100
+   :reloadable:
+
+   The maximum number of concurrent streams per inbound connection.
+
+   .. note:: Reloading this value affects only new HTTP/2 connections, not the
+	     ones already established.
+
+.. ts:cv:: CONFIG proxy.config.http2.initial_window_size_in INT 65536
+   :reloadable:
+
+   The initial window size for inbound connections.
+
+.. ts:cv:: CONFIG proxy.config.http2.max_frame_size INT 16384
+   :reloadable:
+
+   Indicates the size of the largest frame payload that the sender is willing
+   to receive.
+
+.. ts:cv:: CONFIG proxy.config.http2.header_table_size INT 4096
+   :reloadable:
+
+   The maximum size of the header compression table used to decode header
+   blocks.
+
+.. ts:cv:: CONFIG proxy.config.http2.max_header_list_size INT 4294967295
+   :reloadable:
+
+   This advisory setting informs a peer of the maximum size of header list
+   that the sender is prepared to accept blocks. The default value, which is
+   the unsigned int maximum value in Traffic Server, implies unlimited size.
+
 SPDY Configuration
 ==================
 
@@ -2511,7 +2547,8 @@ SPDY Configuration
 
    The maximum number of concurrent streams per inbound connection.
 
-   .. note:: Reloading this value affects only new SPDY connections, not existing connects.
+   .. note:: Reloading this value affects only new SPDY connections, not the
+	     ones already established..
 
 Scheduled Update Configuration
 ==============================

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f810c09a/mgmt/RecordsConfig.cc
----------------------------------------------------------------------
diff --git a/mgmt/RecordsConfig.cc b/mgmt/RecordsConfig.cc
index 5729e97..14c9957 100644
--- a/mgmt/RecordsConfig.cc
+++ b/mgmt/RecordsConfig.cc
@@ -2002,6 +2002,16 @@ static const RecordElement RecordsConfig[] =
   //############
   {RECT_CONFIG, "proxy.config.http2.enabled", RECD_INT, "0", RECU_RESTART_TM, RR_NULL, RECC_INT, "[0-1]", RECA_NULL}
   ,
+  {RECT_CONFIG, "proxy.config.http2.max_concurrent_streams_in", RECD_INT, "100", RECU_DYNAMIC, RR_NULL, RECC_STR, "^[0-9]+$", RECA_NULL}
+  ,
+  {RECT_CONFIG, "proxy.config.http2.initial_window_size_in", RECD_INT, "1048576", RECU_DYNAMIC, RR_NULL, RECC_STR, "^[0-9]+$", RECA_NULL}
+  ,
+  {RECT_CONFIG, "proxy.config.http2.max_frame_size", RECD_INT, "16384", RECU_DYNAMIC, RR_NULL, RECC_STR, "^[0-9]+$", RECA_NULL}
+  ,
+  {RECT_CONFIG, "proxy.config.http2.header_table_size", RECD_INT, "4096", RECU_DYNAMIC, RR_NULL, RECC_STR, "^[0-9]+$", RECA_NULL}
+  ,
+  {RECT_CONFIG, "proxy.config.http2.max_header_list_size", RECD_INT, "4294967295", RECU_DYNAMIC, RR_NULL, RECC_STR, "^[0-9]+$", RECA_NULL}
+  ,
 
   //# Add LOCAL Records Here
   {RECT_LOCAL, "proxy.local.incoming_ip_to_bind", RECD_STRING, NULL, RECU_NULL, RR_NULL, RECC_NULL, NULL, RECA_NULL}

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f810c09a/proxy/Main.cc
----------------------------------------------------------------------
diff --git a/proxy/Main.cc b/proxy/Main.cc
index ef93db0..1e97448 100644
--- a/proxy/Main.cc
+++ b/proxy/Main.cc
@@ -89,6 +89,7 @@ extern "C" int plock(int);
 #include "RemapProcessor.h"
 #include "I_Tasks.h"
 #include "InkAPIInternal.h"
+#include "HTTP2.h"
 
 #include <ts/ink_cap.h>
 
@@ -1685,6 +1686,10 @@ main(int /* argc ATS_UNUSED */, const char **argv)
     extern int spdy_config_load();
     spdy_config_load(); // must be before HttpProxyPort init.
 #endif
+
+    // Initialize HTTP/2
+    Http2::init();
+
     // Load HTTP port data. getNumSSLThreads depends on this.
     if (!HttpProxyPort::loadValue(http_accept_port_descriptor))
       HttpProxyPort::loadConfig();

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f810c09a/proxy/http/HttpClientSession.cc
----------------------------------------------------------------------
diff --git a/proxy/http/HttpClientSession.cc b/proxy/http/HttpClientSession.cc
index 2182404..a07fcc0 100644
--- a/proxy/http/HttpClientSession.cc
+++ b/proxy/http/HttpClientSession.cc
@@ -295,6 +295,7 @@ HttpClientSession::do_io_close(int alerrno)
     read_state = HCS_CLOSED;
     if (upgrade_to_h2c) {
       Http2ClientSession *h2_session = http2ClientSessionAllocator.alloc();
+
       h2_session->set_upgrade_context(&current_reader->t_state.hdr_info.client_request);
       h2_session->new_connection(client_vc, NULL, NULL, false /* backdoor */);
       // TODO Consider about handling HTTP/1 hooks and stats

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f810c09a/proxy/http2/HTTP2.cc
----------------------------------------------------------------------
diff --git a/proxy/http2/HTTP2.cc b/proxy/http2/HTTP2.cc
index f6d4842..55f4f81 100644
--- a/proxy/http2/HTTP2.cc
+++ b/proxy/http2/HTTP2.cc
@@ -25,6 +25,7 @@
 #include "HPACK.h"
 #include "HuffmanCodec.h"
 #include "ink_assert.h"
+#include "I_RecCore.h"
 
 const char *const HTTP2_CONNECTION_PREFACE = "PRI * HTTP/2.0\r\n\r\nSM\r\n\r\n";
 static size_t HPACK_LEN_STATUS_VALUE_STR = 3;
@@ -106,13 +107,13 @@ http2_are_frame_flags_valid(uint8_t ftype, uint8_t fflags)
 }
 
 bool
-http2_frame_header_is_valid(const Http2FrameHeader &hdr)
+http2_frame_header_is_valid(const Http2FrameHeader &hdr, unsigned max_frame_size)
 {
   if (hdr.type >= HTTP2_FRAME_TYPE_MAX) {
     return false;
   }
 
-  if (hdr.length > HTTP2_MAX_FRAME_SIZE) {
+  if (hdr.length > max_frame_size) {
     return false;
   }
 
@@ -731,6 +732,24 @@ http2_parse_header_fragment(HTTPHdr *hdr, IOVec iov, Http2DynamicTable &dynamic_
   return cursor - buf_start;
 }
 
+// Initialize this subsystem with librecords configs (for now)
+uint32_t Http2::max_concurrent_streams = 100;
+uint32_t Http2::initial_window_size = 1048576;
+uint32_t Http2::max_frame_size = 16384;
+uint32_t Http2::header_table_size = 4096;
+uint32_t Http2::max_header_list_size = 4294967295;
+
+void
+Http2::init()
+{
+  REC_EstablishStaticConfigInt32U(max_concurrent_streams, "proxy.config.http2.max_concurrent_streams_in");
+  REC_EstablishStaticConfigInt32U(initial_window_size, "proxy.config.http2.initial_window_size_in");
+  REC_EstablishStaticConfigInt32U(max_frame_size, "proxy.config.http2.max_frame_size");
+  REC_EstablishStaticConfigInt32U(header_table_size, "proxy.config.http2.header_table_size");
+  REC_EstablishStaticConfigInt32U(max_header_list_size, "proxy.config.http2.max_header_list_size");
+}
+
+
 #if TS_HAS_TESTS
 
 #include "TestBox.h"

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f810c09a/proxy/http2/HTTP2.h
----------------------------------------------------------------------
diff --git a/proxy/http2/HTTP2.h b/proxy/http2/HTTP2.h
index 80ba374..bbeffd3 100644
--- a/proxy/http2/HTTP2.h
+++ b/proxy/http2/HTTP2.h
@@ -50,12 +50,13 @@ const size_t HTTP2_GOAWAY_LEN = 8;
 const size_t HTTP2_WINDOW_UPDATE_LEN = 4;
 const size_t HTTP2_SETTINGS_PARAMETER_LEN = 6;
 
-// SETTINGS initial values
-const uint32_t HTTP2_HEADER_TABLE_SIZE = 4096;
-const uint32_t HTTP2_ENABLE_PUSH = 0; // Server Push is NOT supported
+// SETTINGS initial values. NOTE: These should not be modified
+// unless the protocol changes! Do not change this thinking you
+// are changing server defaults. that is done via RecordsConfig.cc
 const uint32_t HTTP2_MAX_CONCURRENT_STREAMS = 100;
 const uint32_t HTTP2_INITIAL_WINDOW_SIZE = 65535;
 const uint32_t HTTP2_MAX_FRAME_SIZE = 16384;
+const uint32_t HTTP2_HEADER_TABLE_SIZE = 4096;
 const uint32_t HTTP2_MAX_HEADER_LIST_SIZE = UINT_MAX;
 
 // 6.9.1 The Flow Control Window
@@ -261,7 +262,7 @@ bool http2_write_goaway(const Http2Goaway &, IOVec);
 
 bool http2_write_window_update(const uint32_t new_size, const IOVec &);
 
-bool http2_frame_header_is_valid(const Http2FrameHeader &);
+bool http2_frame_header_is_valid(const Http2FrameHeader &, unsigned);
 
 bool http2_settings_parameter_is_valid(const Http2SettingsParameter &);
 
@@ -285,4 +286,20 @@ int64_t http2_write_psuedo_headers(HTTPHdr *, uint8_t *, uint64_t, Http2DynamicT
 
 int64_t http2_write_header_fragment(HTTPHdr *, MIMEFieldIter &, uint8_t *, uint64_t, Http2DynamicTable &, bool &);
 
+
+// Not sure where else to put this, but figure this is as good of a start as anything else.
+// Right now, only the static init() is available, which sets up some basic librecords
+// dependencies.
+class Http2
+{
+public:
+  static uint32_t max_concurrent_streams;
+  static uint32_t initial_window_size;
+  static uint32_t max_frame_size;
+  static uint32_t header_table_size;
+  static uint32_t max_header_list_size;
+
+  static void init();
+};
+
 #endif /* __HTTP2_H__ */

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f810c09a/proxy/http2/Http2ClientSession.cc
----------------------------------------------------------------------
diff --git a/proxy/http2/Http2ClientSession.cc b/proxy/http2/Http2ClientSession.cc
index 7ce0ade..def776a 100644
--- a/proxy/http2/Http2ClientSession.cc
+++ b/proxy/http2/Http2ClientSession.cc
@@ -116,7 +116,7 @@ Http2ClientSession::new_connection(NetVConnection *new_vc, MIOBuffer *iobuf, IOB
   DebugHttp2Ssn("session born, netvc %p", this->client_vc);
 
   this->read_buffer = iobuf ? iobuf : new_MIOBuffer(HTTP2_HEADER_BUFFER_SIZE_INDEX);
-  this->read_buffer->water_mark = HTTP2_MAX_FRAME_SIZE;
+  this->read_buffer->water_mark = connection_state.server_settings.get(HTTP2_SETTINGS_MAX_FRAME_SIZE);
   this->sm_reader = reader ? reader : this->read_buffer->alloc_reader();
 
   this->write_buffer = new_MIOBuffer(HTTP2_HEADER_BUFFER_SIZE_INDEX);
@@ -316,12 +316,13 @@ Http2ClientSession::state_start_frame_read(int event, void *edata)
 
     this->sm_reader->consume(nbytes);
 
-    if (!http2_frame_header_is_valid(this->current_hdr)) {
+    if (!http2_frame_header_is_valid(this->current_hdr,
+                                     this->connection_state.server_settings.get(HTTP2_SETTINGS_MAX_FRAME_SIZE))) {
       // XXX nuke it with HTTP2_ERROR_PROTOCOL_ERROR!
     }
 
     // If we know up front that the payload is too long, nuke this connection.
-    if (this->current_hdr.length > this->connection_state.client_settings.get(HTTP2_SETTINGS_MAX_FRAME_SIZE)) {
+    if (this->current_hdr.length > this->connection_state.server_settings.get(HTTP2_SETTINGS_MAX_FRAME_SIZE)) {
       MUTEX_LOCK(lock, this->connection_state.mutex, this_ethread());
       if (!this->connection_state.is_state_closed()) {
         this->connection_state.send_goaway_frame(this->current_hdr.streamid, HTTP2_ERROR_FRAME_SIZE_ERROR);

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f810c09a/proxy/http2/Http2ClientSession.h
----------------------------------------------------------------------
diff --git a/proxy/http2/Http2ClientSession.h b/proxy/http2/Http2ClientSession.h
index 37cedbb..21f8476 100644
--- a/proxy/http2/Http2ClientSession.h
+++ b/proxy/http2/Http2ClientSession.h
@@ -43,6 +43,8 @@ static size_t const HTTP2_HEADER_BUFFER_SIZE_INDEX = CLIENT_CONNECTION_FIRST_REA
 
 // To support Upgrade: h2c
 struct Http2UpgradeContext {
+  Http2UpgradeContext() { }
+
   ~Http2UpgradeContext()
   {
     if (req_header) {
@@ -185,6 +187,7 @@ public:
     return upgrade_context;
   }
 
+
 private:
   Http2ClientSession(Http2ClientSession &);                  // noncopyable
   Http2ClientSession &operator=(const Http2ClientSession &); // noncopyable

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f810c09a/proxy/http2/Http2ConnectionState.cc
----------------------------------------------------------------------
diff --git a/proxy/http2/Http2ConnectionState.cc b/proxy/http2/Http2ConnectionState.cc
index 44efe2b..bad37a0 100644
--- a/proxy/http2/Http2ConnectionState.cc
+++ b/proxy/http2/Http2ConnectionState.cc
@@ -616,10 +616,8 @@ Http2ConnectionState::main_event_handler(int event, void *edata)
     settings.finalize(HTTP2_SETTINGS_PARAMETER_LEN * (HTTP2_SETTINGS_MAX - 1));
     this->ua_session->handleEvent(HTTP2_SESSION_EVENT_XMIT, &settings);
 
-    // TODO: 65535 is the initial window size comes from the HTTP2 spec.
-    //       It should be defined in somewhere.
-    if (server_settings.get(HTTP2_SETTINGS_INITIAL_WINDOW_SIZE) > 65535) {
-      send_window_update_frame(0, server_settings.get(HTTP2_SETTINGS_INITIAL_WINDOW_SIZE) - 65535);
+    if (server_settings.get(HTTP2_SETTINGS_INITIAL_WINDOW_SIZE) > HTTP2_INITIAL_WINDOW_SIZE) {
+      send_window_update_frame(0, server_settings.get(HTTP2_SETTINGS_INITIAL_WINDOW_SIZE) - HTTP2_INITIAL_WINDOW_SIZE);
     }
 
     return 0;

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/f810c09a/proxy/http2/Http2ConnectionState.h
----------------------------------------------------------------------
diff --git a/proxy/http2/Http2ConnectionState.h b/proxy/http2/Http2ConnectionState.h
index 09d34fd..9d1ce1d 100644
--- a/proxy/http2/Http2ConnectionState.h
+++ b/proxy/http2/Http2ConnectionState.h
@@ -35,16 +35,27 @@ class Http2ConnectionSettings
 public:
   Http2ConnectionSettings()
   {
-    // 6.5.2.  Defined SETTINGS Parameters
-    // TODO these values should be configurable.
-    settings[indexof(HTTP2_SETTINGS_HEADER_TABLE_SIZE)] = HTTP2_HEADER_TABLE_SIZE;
-    settings[indexof(HTTP2_SETTINGS_ENABLE_PUSH)] = HTTP2_ENABLE_PUSH;
+    // 6.5.2.  Defined SETTINGS Parameters. These should generally not be modified,
+    // only if the protocol changes should these change.
+    settings[indexof(HTTP2_SETTINGS_ENABLE_PUSH)] = 0; // Disabled for now
+
     settings[indexof(HTTP2_SETTINGS_MAX_CONCURRENT_STREAMS)] = HTTP2_MAX_CONCURRENT_STREAMS;
     settings[indexof(HTTP2_SETTINGS_INITIAL_WINDOW_SIZE)] = HTTP2_INITIAL_WINDOW_SIZE;
     settings[indexof(HTTP2_SETTINGS_MAX_FRAME_SIZE)] = HTTP2_MAX_FRAME_SIZE;
+    settings[indexof(HTTP2_SETTINGS_HEADER_TABLE_SIZE)] = HTTP2_HEADER_TABLE_SIZE;
     settings[indexof(HTTP2_SETTINGS_MAX_HEADER_LIST_SIZE)] = HTTP2_MAX_HEADER_LIST_SIZE;
   }
 
+  void
+  settings_from_configs()
+  {
+    settings[indexof(HTTP2_SETTINGS_MAX_CONCURRENT_STREAMS)] = Http2::max_concurrent_streams;
+    settings[indexof(HTTP2_SETTINGS_INITIAL_WINDOW_SIZE)] = Http2::initial_window_size;
+    settings[indexof(HTTP2_SETTINGS_MAX_FRAME_SIZE)] = Http2::max_frame_size;
+    settings[indexof(HTTP2_SETTINGS_HEADER_TABLE_SIZE)] = Http2::header_table_size;
+    settings[indexof(HTTP2_SETTINGS_MAX_HEADER_LIST_SIZE)] = Http2::max_header_list_size;
+  }
+
   unsigned
   get(Http2SettingsIdentifier id) const
   {
@@ -85,7 +96,7 @@ class Http2ConnectionState;
 class Http2Stream
 {
 public:
-  Http2Stream(Http2StreamId sid = 0, ssize_t initial_rwnd = HTTP2_INITIAL_WINDOW_SIZE)
+  Http2Stream(Http2StreamId sid = 0, ssize_t initial_rwnd = Http2::initial_window_size)
     : client_rwnd(initial_rwnd), server_rwnd(initial_rwnd), _id(sid), _state(HTTP2_STREAM_STATE_IDLE), _fetch_sm(NULL),
       body_done(false), data_length(0)
   {
@@ -177,7 +188,7 @@ class Http2ConnectionState : public Continuation
 {
 public:
   Http2ConnectionState()
-    : Continuation(NULL), ua_session(NULL), client_rwnd(HTTP2_INITIAL_WINDOW_SIZE), server_rwnd(HTTP2_INITIAL_WINDOW_SIZE),
+    : Continuation(NULL), ua_session(NULL), client_rwnd(Http2::initial_window_size), server_rwnd(Http2::initial_window_size),
       stream_list(), latest_streamid(0), continued_id(0)
   {
     SET_HANDLER(&Http2ConnectionState::main_event_handler);
@@ -199,6 +210,9 @@ public:
 
     continued_buffer.iov_base = NULL;
     continued_buffer.iov_len = 0;
+
+    // Load the server settings from the records.config / RecordsConfig.cc settings.
+    server_settings.settings_from_configs();
   }
 
   void