You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@trafficserver.apache.org by maskit <gi...@git.apache.org> on 2016/02/08 16:20:45 UTC

[GitHub] trafficserver pull request: TS-4092: Decouple HPACK from HTTP/2

GitHub user maskit opened a pull request:

    https://github.com/apache/trafficserver/pull/460

    TS-4092: Decouple HPACK from HTTP/2

    I tried to decouple HPACK from HTTP2, and it partially succeeded. It's very difficult to completely decouple HPACK unless we use MIMEField in it.
    
    # Changes:
    - Added two functions to HPACK module
       - hpack_decode_header_block
       - hpack_encode_header_block
       
       These functions handle a header block(uint8_t *) and a header list(HTTPHdr *), and convert them from one to another. They simply take care of RFC7541(HPACK). Which means no special treatment for particular headers such as pseudo headers.
       
    - Simplified HPACK related functions in HTTP2 module
       - http2_write_psuedo_headers(removed)
       - http2_write_header_fragment(removed)
       - http2_convert_header_from_1_1_to_2(added)
       - http2_convert_header_from_2_to_1_1(renamed)
       - http2_decode_header_blocks(simplified)
       - http2_encode_header_blocks(added)
       
       The last two functions handle a header block and a header list too, but they take care of RFC7540(HTTP2). They are responsible for handling of sensitive headers, effectiveness (RFC7540 Section 8.1.2.5) and HTTP2 related limitations.
       
    - Added a flag of sensitivity to MIMEField
       The flag is used for determining a representation of a header field when encoding response headers. It will also be used when we support HTTP2 on server side connections because an intermediary needs to remember representation of each header fields to ensure that a never-indexed representation field will not be re-encoded as a indexed representation field (RFC7541 Section 7.1.3). 
    
    # Benefits:
    
    - HPACK module has been simple
       Simple is best.
       
    - HPACK module has been testable
       Our HPACK module has been a pure HPACK implementation. It means we can use external common test cases which other implementations use. It will ensure compatibility.
       https://github.com/http2jp/hpack-test-case
       
    - HPACK module has been (almost) decoupled from HTTP2
       We can use the module not only for HTTP2 but also QUIC (if we support it).
    
    # Issues:
    
    - MIMEField converts header names old fashioned
       Because RFC 7541(HPACK) does not mention about case of characters, we should not change them in HPACK module but MIMEField does against WKSs. So we have to convert the strings to lowercase just before encoding in HPACK module to follow RFC 7540(HTTP2). This blocks complete decoupling.
    
    - Cannot prepend a header field to HTTPHdr
       HTTP2 cares order of header fields. Pseudo header fields must be at the beginning of header fields. To convert HTTP1.1 header to HTTP2 header, we need to prepend pseudo header fields to a existing HTTPHdr. However, the only way to achieve this is creating new HTTPHdr for now. It does cost. The same thing will happen when we support HTTP2 on server side connection. This would affect performance.

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/maskit/trafficserver ts4092

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/trafficserver/pull/460.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #460
    
----
commit 3f5ac05a50443ce77201746061f43f5804eff4ec
Author: Masakazu Kitajo <ma...@apache.org>
Date:   2016-02-08T14:52:59Z

    TS-4092: Decouple HPACK from HTTP/2

----


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] trafficserver pull request: TS-4092: Decouple HPACK from HTTP/2

Posted by bryancall <gi...@git.apache.org>.
Github user bryancall commented on the pull request:

    https://github.com/apache/trafficserver/pull/460#issuecomment-187824312
  
    I will take a look at this.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] trafficserver pull request: TS-4092: Decouple HPACK from HTTP/2

Posted by bryancall <gi...@git.apache.org>.
Github user bryancall commented on a diff in the pull request:

    https://github.com/apache/trafficserver/pull/460#discussion_r56909414
  
    --- Diff: proxy/http2/HPACK.h ---
    @@ -90,31 +97,29 @@ class MIMEFieldWrapper
         return _field;
       }
     
    +  bool
    +  is_sensitive() const
    +  {
    +    return _field->is_sensitive();
    +  }
    +
     private:
       MIMEField *_field;
       HdrHeap *_heap;
       MIMEHdrImpl *_mh;
     };
     
    -// Result of looking for a header field in IndexingTable
    -struct Http2LookupIndexResult {
    -  Http2LookupIndexResult() : index(0), value_is_indexed(false) {}
    -
    -  int index;
    -  bool value_is_indexed;
    -};
    -
     // [RFC 7541] 2.3.2. Dynamic Table
    -class Http2DynamicTable
    +class HpackDynamicTable
     {
     public:
    -  Http2DynamicTable() : _current_size(0), _settings_dynamic_table_size(4096)
    +  HpackDynamicTable() : _current_size(0), _maximum_size(4096)
    --- End diff --
    
    4096 should a constant in HPACK.h


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] trafficserver pull request: TS-4092: Decouple HPACK from HTTP/2

Posted by maskit <gi...@git.apache.org>.
Github user maskit commented on a diff in the pull request:

    https://github.com/apache/trafficserver/pull/460#discussion_r56920295
  
    --- Diff: proxy/http2/HPACK.cc ---
    @@ -828,9 +828,111 @@ update_dynamic_table_size(const uint8_t *buf_start, const uint8_t *buf_end, Http
       if (len == HPACK_ERROR_COMPRESSION_ERROR)
         return HPACK_ERROR_COMPRESSION_ERROR;
     
    -  if (indexing_table.set_dynamic_table_size(size) == false) {
    +  if (indexing_table.update_maximum_size(size) == false) {
         return HPACK_ERROR_COMPRESSION_ERROR;
       }
     
       return len;
     }
    +
    +int64_t
    +hpack_decode_header_block(HpackIndexingTable &indexing_table, HTTPHdr *hdr, const uint8_t *in_buf, const size_t in_buf_len)
    +{
    +  const uint8_t *cursor = in_buf;
    +  const uint8_t *const in_buf_end = in_buf + in_buf_len;
    +  HdrHeap *heap = hdr->m_heap;
    +  HTTPHdrImpl *hh = hdr->m_http;
    +  bool header_field_started = false;
    +  bool has_http2_violation = false;
    +
    +  while (cursor < in_buf_end) {
    +    int64_t read_bytes = 0;
    +
    +    // decode a header field encoded by HPACK
    +    MIMEField *field = mime_field_create(heap, hh->m_fields_impl);
    +    MIMEFieldWrapper header(field, heap, hh->m_fields_impl);
    +    HpackFieldType ftype = hpack_parse_field_type(*cursor);
    +
    +    switch (ftype) {
    +    case HPACK_FIELD_INDEX:
    +      read_bytes = decode_indexed_header_field(header, cursor, in_buf_end, indexing_table);
    +      if (read_bytes == HPACK_ERROR_COMPRESSION_ERROR) {
    +        return HPACK_ERROR_COMPRESSION_ERROR;
    +      }
    +      cursor += read_bytes;
    +      header_field_started = true;
    +      break;
    +    case HPACK_FIELD_INDEXED_LITERAL:
    +    case HPACK_FIELD_NOINDEX_LITERAL:
    +    case HPACK_FIELD_NEVERINDEX_LITERAL:
    +      read_bytes = decode_literal_header_field(header, cursor, in_buf_end, indexing_table);
    +      if (read_bytes == HPACK_ERROR_COMPRESSION_ERROR) {
    +        return HPACK_ERROR_COMPRESSION_ERROR;
    +      }
    +      if (read_bytes < 0) {
    +        has_http2_violation = true;
    +        read_bytes = -read_bytes;
    +      }
    +      cursor += read_bytes;
    +      header_field_started = true;
    +      break;
    +    case HPACK_FIELD_TABLESIZE_UPDATE:
    +      if (header_field_started) {
    +        return HPACK_ERROR_COMPRESSION_ERROR;
    +      }
    +      read_bytes = update_dynamic_table_size(cursor, in_buf_end, indexing_table);
    +      if (read_bytes == HPACK_ERROR_COMPRESSION_ERROR) {
    +        return HPACK_ERROR_COMPRESSION_ERROR;
    +      }
    +      cursor += read_bytes;
    +      continue;
    +    }
    +    // Store to HdrHeap
    +    mime_hdr_field_attach(hh->m_fields_impl, field, 1, NULL);
    +  }
    +  // Parsing all headers is done
    +  if (has_http2_violation) {
    +    return -(cursor - in_buf);
    +  } else {
    +    return cursor - in_buf;
    +  }
    +}
    +
    +int64_t
    +hpack_encode_header_block(HpackIndexingTable &indexing_table, uint8_t *out_buf, const size_t out_buf_len, HTTPHdr *hdr)
    +{
    +  uint8_t *cursor = out_buf;
    +  const uint8_t *const out_buf_end = out_buf + out_buf_len;
    +  int64_t written;
    +
    +  ink_assert(http_hdr_type_get(hdr->m_http) != HTTP_TYPE_UNKNOWN);
    +
    +  MIMEFieldIter field_iter;
    +  for (MIMEField *field = hdr->iter_get_first(&field_iter); field != NULL; field = hdr->iter_get_next(&field_iter)) {
    +    HpackFieldType field_type;
    +    MIMEFieldWrapper header(field, hdr->m_heap, hdr->m_http->m_fields_impl);
    +    if (header.is_sensitive()) {
    --- End diff --
    
    It's possible for now but the flag would be needed when we support HTTP/2 on both client and server session because a proxy must keep field's representation.
    
    > An intermediary MUST NOT re-encode a value that uses the never-
       indexed literal representation with another representation that would
       index it.  If HPACK is used for re-encoding, the never-indexed
       literal representation MUST be used.
    
    I intended to use the flag to remember a field's representation in more general way.
    Should we consider it when we support HTTP/2 on server side?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] trafficserver pull request: TS-4092: Decouple HPACK from HTTP/2

Posted by bryancall <gi...@git.apache.org>.
Github user bryancall commented on a diff in the pull request:

    https://github.com/apache/trafficserver/pull/460#discussion_r57029193
  
    --- Diff: proxy/http2/HPACK.cc ---
    @@ -828,9 +828,111 @@ update_dynamic_table_size(const uint8_t *buf_start, const uint8_t *buf_end, Http
       if (len == HPACK_ERROR_COMPRESSION_ERROR)
         return HPACK_ERROR_COMPRESSION_ERROR;
     
    -  if (indexing_table.set_dynamic_table_size(size) == false) {
    +  if (indexing_table.update_maximum_size(size) == false) {
         return HPACK_ERROR_COMPRESSION_ERROR;
       }
     
       return len;
     }
    +
    +int64_t
    +hpack_decode_header_block(HpackIndexingTable &indexing_table, HTTPHdr *hdr, const uint8_t *in_buf, const size_t in_buf_len)
    +{
    +  const uint8_t *cursor = in_buf;
    +  const uint8_t *const in_buf_end = in_buf + in_buf_len;
    +  HdrHeap *heap = hdr->m_heap;
    +  HTTPHdrImpl *hh = hdr->m_http;
    +  bool header_field_started = false;
    +  bool has_http2_violation = false;
    +
    +  while (cursor < in_buf_end) {
    +    int64_t read_bytes = 0;
    +
    +    // decode a header field encoded by HPACK
    +    MIMEField *field = mime_field_create(heap, hh->m_fields_impl);
    +    MIMEFieldWrapper header(field, heap, hh->m_fields_impl);
    +    HpackFieldType ftype = hpack_parse_field_type(*cursor);
    +
    +    switch (ftype) {
    +    case HPACK_FIELD_INDEX:
    +      read_bytes = decode_indexed_header_field(header, cursor, in_buf_end, indexing_table);
    +      if (read_bytes == HPACK_ERROR_COMPRESSION_ERROR) {
    +        return HPACK_ERROR_COMPRESSION_ERROR;
    +      }
    +      cursor += read_bytes;
    +      header_field_started = true;
    +      break;
    +    case HPACK_FIELD_INDEXED_LITERAL:
    +    case HPACK_FIELD_NOINDEX_LITERAL:
    +    case HPACK_FIELD_NEVERINDEX_LITERAL:
    +      read_bytes = decode_literal_header_field(header, cursor, in_buf_end, indexing_table);
    +      if (read_bytes == HPACK_ERROR_COMPRESSION_ERROR) {
    +        return HPACK_ERROR_COMPRESSION_ERROR;
    +      }
    +      if (read_bytes < 0) {
    +        has_http2_violation = true;
    +        read_bytes = -read_bytes;
    +      }
    +      cursor += read_bytes;
    +      header_field_started = true;
    +      break;
    +    case HPACK_FIELD_TABLESIZE_UPDATE:
    +      if (header_field_started) {
    +        return HPACK_ERROR_COMPRESSION_ERROR;
    +      }
    +      read_bytes = update_dynamic_table_size(cursor, in_buf_end, indexing_table);
    +      if (read_bytes == HPACK_ERROR_COMPRESSION_ERROR) {
    +        return HPACK_ERROR_COMPRESSION_ERROR;
    +      }
    +      cursor += read_bytes;
    +      continue;
    +    }
    +    // Store to HdrHeap
    +    mime_hdr_field_attach(hh->m_fields_impl, field, 1, NULL);
    +  }
    +  // Parsing all headers is done
    +  if (has_http2_violation) {
    +    return -(cursor - in_buf);
    +  } else {
    +    return cursor - in_buf;
    +  }
    +}
    +
    +int64_t
    +hpack_encode_header_block(HpackIndexingTable &indexing_table, uint8_t *out_buf, const size_t out_buf_len, HTTPHdr *hdr)
    +{
    +  uint8_t *cursor = out_buf;
    +  const uint8_t *const out_buf_end = out_buf + out_buf_len;
    +  int64_t written;
    +
    +  ink_assert(http_hdr_type_get(hdr->m_http) != HTTP_TYPE_UNKNOWN);
    +
    +  MIMEFieldIter field_iter;
    +  for (MIMEField *field = hdr->iter_get_first(&field_iter); field != NULL; field = hdr->iter_get_next(&field_iter)) {
    +    HpackFieldType field_type;
    +    MIMEFieldWrapper header(field, hdr->m_heap, hdr->m_http->m_fields_impl);
    +    if (header.is_sensitive()) {
    --- End diff --
    
    I believe the logic could still stay in the HPACK layer when doing server side.  I would remove it for now and when the server side is implemented we can deal with it.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] trafficserver pull request: TS-4092: Decouple HPACK from HTTP/2

Posted by maskit <gi...@git.apache.org>.
Github user maskit commented on the pull request:

    https://github.com/apache/trafficserver/pull/460#issuecomment-185757269
  
    Also, this fixes a bug in the HPACK decoding process. Current decoding process stops if there is a HTTP2 violation (not HPACK violation) but it should not. As I commented above, the decoder should keep decoding unless a compression error (HPACK violation) occurs.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] trafficserver pull request: TS-4092: Decouple HPACK from HTTP/2

Posted by bryancall <gi...@git.apache.org>.
Github user bryancall commented on the pull request:

    https://github.com/apache/trafficserver/pull/460#issuecomment-199522681
  
    Overall it looks good.  I had a couple comments in the code.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] trafficserver pull request: TS-4092: Decouple HPACK from HTTP/2

Posted by masaori335 <gi...@git.apache.org>.
Github user masaori335 commented on a diff in the pull request:

    https://github.com/apache/trafficserver/pull/460#discussion_r53417506
  
    --- Diff: proxy/http2/HTTP2.cc ---
    @@ -484,246 +498,162 @@ convert_from_2_to_1_1_header(HTTPHdr *headers)
       return PARSE_DONE;
     }
     
    -static int64_t
    -http2_write_header_field(uint8_t *out, const uint8_t *end, MIMEFieldWrapper &header, Http2IndexingTable &indexing_table)
    -{
    -  HpackFieldType field_type = HPACK_FIELD_INDEXED_LITERAL;
    -
    -  // Cookie less that 20 bytes and Authorization are never indexed
    -  // This policy is refer to Firefox and nghttp2
    -  int name_len = 0, value_len = 0;
    -  const char *name = header.name_get(&name_len);
    -  header.value_get(&value_len);
    -  if ((ptr_len_casecmp(name, name_len, MIME_FIELD_COOKIE, MIME_LEN_COOKIE) == 0 && value_len < 20) ||
    -      (ptr_len_casecmp(name, name_len, MIME_FIELD_AUTHORIZATION, MIME_LEN_AUTHORIZATION) == 0)) {
    -    field_type = HPACK_FIELD_NEVERINDEX_LITERAL;
    -  }
    -
    -  // TODO Enable to configure selecting header field representation
    +void
    +http2_convert_header_from_1_1_to_2(HTTPHdr *headers)
    +{
    +  HTTPHdr tmp;
    +  tmp.create(http_hdr_type_get(headers->m_http));
    +  tmp.copy(headers);
    +  headers->fields_clear();
    +
    +  if (http_hdr_type_get(tmp.m_http) == HTTP_TYPE_RESPONSE) {
    +    char status_str[HTTP2_LEN_STATUS_VALUE_STR + 1];
    +    snprintf(status_str, sizeof(status_str), "%d", tmp.status_get());
    +
    +    // Add ':status' header field
    +    MIMEField *status_field = headers->field_create(HTTP2_VALUE_STATUS, HTTP2_LEN_STATUS);
    +    status_field->value_set(headers->m_heap, headers->m_mime, status_str, HTTP2_LEN_STATUS_VALUE_STR);
    +    headers->field_attach(status_field);
    +
    +    MIMEFieldIter field_iter;
    +    for (MIMEField *field = tmp.iter_get_first(&field_iter); field != NULL; field = tmp.iter_get_next(&field_iter)) {
    +      // Intermediaries SHOULD remove connection-specific header fields.
    +      const char *name;
    +      int name_len;
    +      const char *value;
    +      int value_len;
    +      name = field->name_get(&name_len);
    +      if ((name_len == MIME_LEN_CONNECTION && strncasecmp(name, MIME_FIELD_CONNECTION, name_len) == 0) ||
    +          (name_len == MIME_LEN_KEEP_ALIVE && strncasecmp(name, MIME_FIELD_KEEP_ALIVE, name_len) == 0) ||
    +          (name_len == MIME_LEN_PROXY_CONNECTION && strncasecmp(name, MIME_FIELD_PROXY_CONNECTION, name_len) == 0) ||
    +          (name_len == MIME_LEN_TRANSFER_ENCODING && strncasecmp(name, MIME_FIELD_TRANSFER_ENCODING, name_len) == 0) ||
    +          (name_len == MIME_LEN_UPGRADE && strncasecmp(name, MIME_FIELD_UPGRADE, name_len) == 0)) {
    +        continue;
    +      }
     
    -  const Http2LookupIndexResult &result = indexing_table.get_index(header);
    -  if (result.index > 0) {
    -    if (result.value_is_indexed) {
    -      return encode_indexed_header_field(out, end, result.index);
    -    } else {
    -      return encode_literal_header_field_with_indexed_name(out, end, header, result.index, indexing_table, field_type);
    +      MIMEField *newfield;
    +      name = field->name_get(&name_len);
    +      newfield = headers->field_create(name, name_len);
    +      value = field->value_get(&value_len);
    +      newfield->value_set(headers->m_heap, headers->m_mime, value, value_len);
    +      tmp.field_delete(field);
    +      headers->field_attach(newfield);
    +      // Set sensitive flag (See RFC7541 7.1.3)
    +      // - Authorization header obviously should not be indexed
    +      // - Short Cookie header should not be indexed because of low entropy
    +      if ((ptr_len_casecmp(name, name_len, MIME_FIELD_COOKIE, MIME_LEN_COOKIE) == 0 && value_len < 20) ||
    +          (ptr_len_casecmp(name, name_len, MIME_FIELD_AUTHORIZATION, MIME_LEN_AUTHORIZATION) == 0)) {
    +        newfield->m_sensitive = 1;
    +      }
         }
    -  } else {
    -    return encode_literal_header_field_with_new_name(out, end, header, indexing_table, field_type);
       }
    +  tmp.destroy();
     }
     
     int64_t
    -http2_write_psuedo_headers(HTTPHdr *in, uint8_t *out, uint64_t out_len, Http2IndexingTable &indexing_table)
    +http2_encode_header_blocks(HTTPHdr *in, uint8_t *out, uint32_t out_len, HpackHandle &handle)
     {
    -  uint8_t *p = out;
    -  uint8_t *end = out + out_len;
    -  int64_t len;
    -
    -  ink_assert(http_hdr_type_get(in->m_http) != HTTP_TYPE_UNKNOWN);
    -
    -  // TODO Check whether buffer size is enough
    -
    -  // Set psuedo header
    -  if (http_hdr_type_get(in->m_http) == HTTP_TYPE_RESPONSE) {
    -    char status_str[HPACK_LEN_STATUS_VALUE_STR + 1];
    -    snprintf(status_str, sizeof(status_str), "%d", in->status_get());
    -
    -    // Add 'Status:' dummy header field
    -    MIMEField *status_field = mime_field_create(in->m_heap, in->m_http->m_fields_impl);
    -    mime_field_name_value_set(in->m_heap, in->m_mime, status_field, -1, HPACK_VALUE_STATUS, HPACK_LEN_STATUS, status_str,
    -                              HPACK_LEN_STATUS_VALUE_STR, 0, HPACK_LEN_STATUS + HPACK_LEN_STATUS_VALUE_STR, true);
    -    mime_hdr_field_attach(in->m_mime, status_field, 1, NULL);
    -
    -    // Encode psuedo headers by HPACK
    -    MIMEFieldWrapper header(status_field, in->m_heap, in->m_http->m_fields_impl);
    -
    -    len = http2_write_header_field(p, end, header, indexing_table);
    -    if (len == -1)
    -      return -1;
    -    p += len;
    -
    -    // Remove dummy header field
    -    in->field_delete(HPACK_VALUE_STATUS, HPACK_LEN_STATUS);
    -  }
    -
    -  return p - out;
    -}
    -
    -int64_t
    -http2_write_header_fragment(HTTPHdr *in, MIMEFieldIter &field_iter, uint8_t *out, uint64_t out_len,
    -                            Http2IndexingTable &indexing_table, bool &cont)
    -{
    -  uint8_t *p = out;
    -  uint8_t *end = out + out_len;
    -  int64_t len;
    -
    -  ink_assert(http_hdr_type_get(in->m_http) != HTTP_TYPE_UNKNOWN);
    -  ink_assert(in);
    -
    -  // TODO Get a index value from the tables for the header field, and then
    -  // choose a representation type.
    -  // TODO Each indexing types per field should be passed by a caller, HTTP/2
    -  // implementation.
    -
    -  // Get first header field which is required encoding
    -  MIMEField *field;
    -  if (!field_iter.m_block) {
    -    field = in->iter_get_first(&field_iter);
    -  } else {
    -    field = in->iter_get(&field_iter);
    -  }
    -
    -  // Set mime headers
    -  cont = false;
    -  for (; field != NULL; field = in->iter_get_next(&field_iter)) {
    -    // Intermediaries SHOULD remove connection-specific header fields.
    -    int name_len;
    -    const char *name = field->name_get(&name_len);
    -    if ((name_len == MIME_LEN_CONNECTION && strncasecmp(name, MIME_FIELD_CONNECTION, name_len) == 0) ||
    -        (name_len == MIME_LEN_KEEP_ALIVE && strncasecmp(name, MIME_FIELD_KEEP_ALIVE, name_len) == 0) ||
    -        (name_len == MIME_LEN_PROXY_CONNECTION && strncasecmp(name, MIME_FIELD_PROXY_CONNECTION, name_len) == 0) ||
    -        (name_len == MIME_LEN_TRANSFER_ENCODING && strncasecmp(name, MIME_FIELD_TRANSFER_ENCODING, name_len) == 0) ||
    -        (name_len == MIME_LEN_UPGRADE && strncasecmp(name, MIME_FIELD_UPGRADE, name_len) == 0)) {
    -      continue;
    -    }
    -
    -    MIMEFieldWrapper header(field, in->m_heap, in->m_http->m_fields_impl);
    -    if ((len = http2_write_header_field(p, end, header, indexing_table)) == -1) {
    -      if (p == out) {
    -        // no progress was made, header was too big for the buffer, skipping for now
    -        continue;
    -      }
    -      if (!cont) {
    -        // Parsing a part of headers is done
    -        cont = true;
    -        return p - out;
    -      } else {
    -        // Parse error
    -        return -1;
    -      }
    -    }
    -    p += len;
    -  }
    -
    -  // Parsing all headers is done
    -  return p - out;
    +  // TODO: It would be better to split Cookie header value
    +  return hpack_encode_header_block(handle, out, out_len, in);
     }
     
     /*
      * Decode Header Blocks to Header List.
      */
     int64_t
    -http2_decode_header_blocks(HTTPHdr *hdr, const uint8_t *buf_start, const uint8_t *buf_end, Http2IndexingTable &indexing_table,
    +http2_decode_header_blocks(HTTPHdr *hdr, const uint8_t *buf_start, const uint32_t buf_len, HpackHandle &handle,
                                bool &trailing_header)
     {
    -  const uint8_t *cursor = buf_start;
    -  HdrHeap *heap = hdr->m_heap;
    -  HTTPHdrImpl *hh = hdr->m_http;
    -  bool header_field_started = false;
    +  const MIMEField *field;
    +  const char *value;
    +  int len;
       bool is_trailing_header = trailing_header;
    +  int64_t result = hpack_decode_header_block(handle, hdr, buf_start, buf_len);
     
    -  while (cursor < buf_end) {
    -    int64_t read_bytes = 0;
    +  if (result < 0) {
    +    return result;
    +  }
     
    -    // decode a header field encoded by HPACK
    -    MIMEField *field = mime_field_create(heap, hh->m_fields_impl);
    -    MIMEFieldWrapper header(field, heap, hh->m_fields_impl);
    -    HpackFieldType ftype = hpack_parse_field_type(*cursor);
     
    -    switch (ftype) {
    -    case HPACK_FIELD_INDEX:
    -      read_bytes = decode_indexed_header_field(header, cursor, buf_end, indexing_table);
    -      if (read_bytes == HPACK_ERROR_COMPRESSION_ERROR) {
    -        return HPACK_ERROR_COMPRESSION_ERROR;
    -      }
    -      cursor += read_bytes;
    -      header_field_started = true;
    -      break;
    -    case HPACK_FIELD_INDEXED_LITERAL:
    -    case HPACK_FIELD_NOINDEX_LITERAL:
    -    case HPACK_FIELD_NEVERINDEX_LITERAL:
    -      read_bytes = decode_literal_header_field(header, cursor, buf_end, indexing_table);
    -      if (read_bytes == HPACK_ERROR_COMPRESSION_ERROR) {
    -        return HPACK_ERROR_COMPRESSION_ERROR;
    -      }
    -      cursor += read_bytes;
    -      header_field_started = true;
    -      break;
    -    case HPACK_FIELD_TABLESIZE_UPDATE:
    -      if (header_field_started) {
    -        return HPACK_ERROR_COMPRESSION_ERROR;
    +  MIMEFieldIter iter;
    +  unsigned int expected_pseudo_header_count = 4;
    +  unsigned int pseudo_header_count = 0;
    +
    +  if (is_trailing_header) {
    +    expected_pseudo_header_count = 0;
    +  }
    +  for (field = hdr->iter_get_first(&iter); field != NULL; field = hdr->iter_get_next(&iter)) {
    +    value = field->name_get(&len);
    +    // Pseudo headers must appear before regular headers
    +    if (len && value[0] == ':') {
    +      ++pseudo_header_count;
    +      if (pseudo_header_count > expected_pseudo_header_count) {
    +        return -result;
    --- End diff --
    
    Is it necessary to return this? Those values are used in `decode_header_blocks` in Http2Stream.h to detect COMPRESSION_ERROR or PROTOCOL_ERROR.
    How about returning Http2ErrorCode directly?



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] trafficserver pull request: TS-4092: Decouple HPACK from HTTP/2

Posted by maskit <gi...@git.apache.org>.
Github user maskit commented on the pull request:

    https://github.com/apache/trafficserver/pull/460#issuecomment-185754698
  
    @jpeach , I changed some function names as per your comments. But some are not exactly the same as you suggested because I think the names should correspond to the spec.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] trafficserver pull request: TS-4092: Decouple HPACK from HTTP/2

Posted by maskit <gi...@git.apache.org>.
Github user maskit commented on a diff in the pull request:

    https://github.com/apache/trafficserver/pull/460#discussion_r53446831
  
    --- Diff: proxy/http2/HTTP2.cc ---
    @@ -484,246 +498,162 @@ convert_from_2_to_1_1_header(HTTPHdr *headers)
       return PARSE_DONE;
     }
     
    -static int64_t
    -http2_write_header_field(uint8_t *out, const uint8_t *end, MIMEFieldWrapper &header, Http2IndexingTable &indexing_table)
    -{
    -  HpackFieldType field_type = HPACK_FIELD_INDEXED_LITERAL;
    -
    -  // Cookie less that 20 bytes and Authorization are never indexed
    -  // This policy is refer to Firefox and nghttp2
    -  int name_len = 0, value_len = 0;
    -  const char *name = header.name_get(&name_len);
    -  header.value_get(&value_len);
    -  if ((ptr_len_casecmp(name, name_len, MIME_FIELD_COOKIE, MIME_LEN_COOKIE) == 0 && value_len < 20) ||
    -      (ptr_len_casecmp(name, name_len, MIME_FIELD_AUTHORIZATION, MIME_LEN_AUTHORIZATION) == 0)) {
    -    field_type = HPACK_FIELD_NEVERINDEX_LITERAL;
    -  }
    -
    -  // TODO Enable to configure selecting header field representation
    +void
    +http2_convert_header_from_1_1_to_2(HTTPHdr *headers)
    +{
    +  HTTPHdr tmp;
    +  tmp.create(http_hdr_type_get(headers->m_http));
    +  tmp.copy(headers);
    +  headers->fields_clear();
    +
    +  if (http_hdr_type_get(tmp.m_http) == HTTP_TYPE_RESPONSE) {
    +    char status_str[HTTP2_LEN_STATUS_VALUE_STR + 1];
    +    snprintf(status_str, sizeof(status_str), "%d", tmp.status_get());
    +
    +    // Add ':status' header field
    +    MIMEField *status_field = headers->field_create(HTTP2_VALUE_STATUS, HTTP2_LEN_STATUS);
    +    status_field->value_set(headers->m_heap, headers->m_mime, status_str, HTTP2_LEN_STATUS_VALUE_STR);
    +    headers->field_attach(status_field);
    +
    +    MIMEFieldIter field_iter;
    +    for (MIMEField *field = tmp.iter_get_first(&field_iter); field != NULL; field = tmp.iter_get_next(&field_iter)) {
    +      // Intermediaries SHOULD remove connection-specific header fields.
    +      const char *name;
    +      int name_len;
    +      const char *value;
    +      int value_len;
    +      name = field->name_get(&name_len);
    +      if ((name_len == MIME_LEN_CONNECTION && strncasecmp(name, MIME_FIELD_CONNECTION, name_len) == 0) ||
    +          (name_len == MIME_LEN_KEEP_ALIVE && strncasecmp(name, MIME_FIELD_KEEP_ALIVE, name_len) == 0) ||
    +          (name_len == MIME_LEN_PROXY_CONNECTION && strncasecmp(name, MIME_FIELD_PROXY_CONNECTION, name_len) == 0) ||
    +          (name_len == MIME_LEN_TRANSFER_ENCODING && strncasecmp(name, MIME_FIELD_TRANSFER_ENCODING, name_len) == 0) ||
    +          (name_len == MIME_LEN_UPGRADE && strncasecmp(name, MIME_FIELD_UPGRADE, name_len) == 0)) {
    +        continue;
    +      }
     
    -  const Http2LookupIndexResult &result = indexing_table.get_index(header);
    -  if (result.index > 0) {
    -    if (result.value_is_indexed) {
    -      return encode_indexed_header_field(out, end, result.index);
    -    } else {
    -      return encode_literal_header_field_with_indexed_name(out, end, header, result.index, indexing_table, field_type);
    +      MIMEField *newfield;
    +      name = field->name_get(&name_len);
    +      newfield = headers->field_create(name, name_len);
    +      value = field->value_get(&value_len);
    +      newfield->value_set(headers->m_heap, headers->m_mime, value, value_len);
    +      tmp.field_delete(field);
    +      headers->field_attach(newfield);
    +      // Set sensitive flag (See RFC7541 7.1.3)
    +      // - Authorization header obviously should not be indexed
    +      // - Short Cookie header should not be indexed because of low entropy
    +      if ((ptr_len_casecmp(name, name_len, MIME_FIELD_COOKIE, MIME_LEN_COOKIE) == 0 && value_len < 20) ||
    +          (ptr_len_casecmp(name, name_len, MIME_FIELD_AUTHORIZATION, MIME_LEN_AUTHORIZATION) == 0)) {
    +        newfield->m_sensitive = 1;
    +      }
         }
    -  } else {
    -    return encode_literal_header_field_with_new_name(out, end, header, indexing_table, field_type);
       }
    +  tmp.destroy();
     }
     
     int64_t
    -http2_write_psuedo_headers(HTTPHdr *in, uint8_t *out, uint64_t out_len, Http2IndexingTable &indexing_table)
    +http2_encode_header_blocks(HTTPHdr *in, uint8_t *out, uint32_t out_len, HpackHandle &handle)
     {
    -  uint8_t *p = out;
    -  uint8_t *end = out + out_len;
    -  int64_t len;
    -
    -  ink_assert(http_hdr_type_get(in->m_http) != HTTP_TYPE_UNKNOWN);
    -
    -  // TODO Check whether buffer size is enough
    -
    -  // Set psuedo header
    -  if (http_hdr_type_get(in->m_http) == HTTP_TYPE_RESPONSE) {
    -    char status_str[HPACK_LEN_STATUS_VALUE_STR + 1];
    -    snprintf(status_str, sizeof(status_str), "%d", in->status_get());
    -
    -    // Add 'Status:' dummy header field
    -    MIMEField *status_field = mime_field_create(in->m_heap, in->m_http->m_fields_impl);
    -    mime_field_name_value_set(in->m_heap, in->m_mime, status_field, -1, HPACK_VALUE_STATUS, HPACK_LEN_STATUS, status_str,
    -                              HPACK_LEN_STATUS_VALUE_STR, 0, HPACK_LEN_STATUS + HPACK_LEN_STATUS_VALUE_STR, true);
    -    mime_hdr_field_attach(in->m_mime, status_field, 1, NULL);
    -
    -    // Encode psuedo headers by HPACK
    -    MIMEFieldWrapper header(status_field, in->m_heap, in->m_http->m_fields_impl);
    -
    -    len = http2_write_header_field(p, end, header, indexing_table);
    -    if (len == -1)
    -      return -1;
    -    p += len;
    -
    -    // Remove dummy header field
    -    in->field_delete(HPACK_VALUE_STATUS, HPACK_LEN_STATUS);
    -  }
    -
    -  return p - out;
    -}
    -
    -int64_t
    -http2_write_header_fragment(HTTPHdr *in, MIMEFieldIter &field_iter, uint8_t *out, uint64_t out_len,
    -                            Http2IndexingTable &indexing_table, bool &cont)
    -{
    -  uint8_t *p = out;
    -  uint8_t *end = out + out_len;
    -  int64_t len;
    -
    -  ink_assert(http_hdr_type_get(in->m_http) != HTTP_TYPE_UNKNOWN);
    -  ink_assert(in);
    -
    -  // TODO Get a index value from the tables for the header field, and then
    -  // choose a representation type.
    -  // TODO Each indexing types per field should be passed by a caller, HTTP/2
    -  // implementation.
    -
    -  // Get first header field which is required encoding
    -  MIMEField *field;
    -  if (!field_iter.m_block) {
    -    field = in->iter_get_first(&field_iter);
    -  } else {
    -    field = in->iter_get(&field_iter);
    -  }
    -
    -  // Set mime headers
    -  cont = false;
    -  for (; field != NULL; field = in->iter_get_next(&field_iter)) {
    -    // Intermediaries SHOULD remove connection-specific header fields.
    -    int name_len;
    -    const char *name = field->name_get(&name_len);
    -    if ((name_len == MIME_LEN_CONNECTION && strncasecmp(name, MIME_FIELD_CONNECTION, name_len) == 0) ||
    -        (name_len == MIME_LEN_KEEP_ALIVE && strncasecmp(name, MIME_FIELD_KEEP_ALIVE, name_len) == 0) ||
    -        (name_len == MIME_LEN_PROXY_CONNECTION && strncasecmp(name, MIME_FIELD_PROXY_CONNECTION, name_len) == 0) ||
    -        (name_len == MIME_LEN_TRANSFER_ENCODING && strncasecmp(name, MIME_FIELD_TRANSFER_ENCODING, name_len) == 0) ||
    -        (name_len == MIME_LEN_UPGRADE && strncasecmp(name, MIME_FIELD_UPGRADE, name_len) == 0)) {
    -      continue;
    -    }
    -
    -    MIMEFieldWrapper header(field, in->m_heap, in->m_http->m_fields_impl);
    -    if ((len = http2_write_header_field(p, end, header, indexing_table)) == -1) {
    -      if (p == out) {
    -        // no progress was made, header was too big for the buffer, skipping for now
    -        continue;
    -      }
    -      if (!cont) {
    -        // Parsing a part of headers is done
    -        cont = true;
    -        return p - out;
    -      } else {
    -        // Parse error
    -        return -1;
    -      }
    -    }
    -    p += len;
    -  }
    -
    -  // Parsing all headers is done
    -  return p - out;
    +  // TODO: It would be better to split Cookie header value
    +  return hpack_encode_header_block(handle, out, out_len, in);
     }
     
     /*
      * Decode Header Blocks to Header List.
      */
     int64_t
    -http2_decode_header_blocks(HTTPHdr *hdr, const uint8_t *buf_start, const uint8_t *buf_end, Http2IndexingTable &indexing_table,
    +http2_decode_header_blocks(HTTPHdr *hdr, const uint8_t *buf_start, const uint32_t buf_len, HpackHandle &handle,
                                bool &trailing_header)
     {
    -  const uint8_t *cursor = buf_start;
    -  HdrHeap *heap = hdr->m_heap;
    -  HTTPHdrImpl *hh = hdr->m_http;
    -  bool header_field_started = false;
    +  const MIMEField *field;
    +  const char *value;
    +  int len;
       bool is_trailing_header = trailing_header;
    +  int64_t result = hpack_decode_header_block(handle, hdr, buf_start, buf_len);
     
    -  while (cursor < buf_end) {
    -    int64_t read_bytes = 0;
    +  if (result < 0) {
    +    return result;
    +  }
     
    -    // decode a header field encoded by HPACK
    -    MIMEField *field = mime_field_create(heap, hh->m_fields_impl);
    -    MIMEFieldWrapper header(field, heap, hh->m_fields_impl);
    -    HpackFieldType ftype = hpack_parse_field_type(*cursor);
     
    -    switch (ftype) {
    -    case HPACK_FIELD_INDEX:
    -      read_bytes = decode_indexed_header_field(header, cursor, buf_end, indexing_table);
    -      if (read_bytes == HPACK_ERROR_COMPRESSION_ERROR) {
    -        return HPACK_ERROR_COMPRESSION_ERROR;
    -      }
    -      cursor += read_bytes;
    -      header_field_started = true;
    -      break;
    -    case HPACK_FIELD_INDEXED_LITERAL:
    -    case HPACK_FIELD_NOINDEX_LITERAL:
    -    case HPACK_FIELD_NEVERINDEX_LITERAL:
    -      read_bytes = decode_literal_header_field(header, cursor, buf_end, indexing_table);
    -      if (read_bytes == HPACK_ERROR_COMPRESSION_ERROR) {
    -        return HPACK_ERROR_COMPRESSION_ERROR;
    -      }
    -      cursor += read_bytes;
    -      header_field_started = true;
    -      break;
    -    case HPACK_FIELD_TABLESIZE_UPDATE:
    -      if (header_field_started) {
    -        return HPACK_ERROR_COMPRESSION_ERROR;
    +  MIMEFieldIter iter;
    +  unsigned int expected_pseudo_header_count = 4;
    +  unsigned int pseudo_header_count = 0;
    +
    +  if (is_trailing_header) {
    +    expected_pseudo_header_count = 0;
    +  }
    +  for (field = hdr->iter_get_first(&iter); field != NULL; field = hdr->iter_get_next(&iter)) {
    +    value = field->name_get(&len);
    +    // Pseudo headers must appear before regular headers
    +    if (len && value[0] == ':') {
    +      ++pseudo_header_count;
    +      if (pseudo_header_count > expected_pseudo_header_count) {
    +        return -result;
    --- End diff --
    
    I could do that. Actually, I changed it once and restored it to keep consistency of return value type between encoding and decoding.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] trafficserver pull request: TS-4092: Decouple HPACK from HTTP/2

Posted by masaori335 <gi...@git.apache.org>.
Github user masaori335 commented on a diff in the pull request:

    https://github.com/apache/trafficserver/pull/460#discussion_r53414029
  
    --- Diff: proxy/http2/Http2ConnectionState.cc ---
    @@ -992,43 +998,50 @@ Http2ConnectionState::send_headers_frame(FetchSM *fetch_sm)
     
       DebugHttp2Stream(ua_session, stream->get_id(), "Send HEADERS frame");
     
    -  // Write pseudo headers
    -  payload_length += http2_write_psuedo_headers(resp_header, payload_buffer, buf_len, *(this->remote_indexing_table));
    -
    -  // If response body is empty, set END_STREAM flag to HEADERS frame
    -  // Must check to ensure content-length is there.  Otherwise the value defaults
    -  // to 0
    -  if (resp_header->presence(MIME_PRESENCE_CONTENT_LENGTH) && resp_header->get_content_length() == 0) {
    -    flags |= HTTP2_FLAGS_HEADERS_END_STREAM;
    +  http2_convert_header_from_1_1_to_2(resp_header);
    +  buf_len = resp_header->length_get() * 2; // Make it double just in case
    +  buf = (uint8_t *)ats_malloc(buf_len);
    +  if (buf == NULL) {
    +    return;
    +  }
    +  header_blocks_size = http2_encode_header_blocks(resp_header, buf, buf_len, *(this->remote_hpack_handle));
    +  if (header_blocks_size < 0) {
    +    ats_free(buf);
    +    return;
       }
     
    -  MIMEFieldIter field_iter;
    -  bool cont = false;
    -  do {
    -    // Handle first sending frame is as HEADERS
    -    Http2FrameType type = cont ? HTTP2_FRAME_TYPE_CONTINUATION : HTTP2_FRAME_TYPE_HEADERS;
    -
    -    // Encode by HPACK naive
    -    payload_length += http2_write_header_fragment(resp_header, field_iter, payload_buffer + payload_length,
    -                                                  buf_len - payload_length, *(this->remote_indexing_table), cont);
    -
    -    // If buffer size is enough to send rest of headers, set END_HEADERS flag
    -    if (buf_len >= payload_length && !cont) {
    -      flags |= HTTP2_FLAGS_HEADERS_END_HEADERS;
    -    }
    -
    -    // Create HEADERS or CONTINUATION frame
    -    Http2Frame headers(type, stream->get_id(), flags);
    -    headers.alloc(buffer_size_index[type]);
    -    http2_write_headers(payload_buffer, payload_length, headers.write());
    +  // Send a HEADERS frame
    +  if (header_blocks_size <= BUFFER_SIZE_FOR_INDEX(buffer_size_index[HTTP2_FRAME_TYPE_HEADERS]) - HTTP2_FRAME_HEADER_LEN) {
    +    payload_length = header_blocks_size;
    +    flags |= HTTP2_FLAGS_HEADERS_END_HEADERS;
    +  } else {
    +    payload_length = BUFFER_SIZE_FOR_INDEX(buffer_size_index[HTTP2_FRAME_TYPE_HEADERS]) - HTTP2_FRAME_HEADER_LEN;
    +  }
    +  Http2Frame headers(HTTP2_FRAME_TYPE_HEADERS, stream->get_id(), flags);
    +  headers.alloc(buffer_size_index[HTTP2_FRAME_TYPE_HEADERS]);
    +  http2_write_headers(buf, payload_length, headers.write());
    +  headers.finalize(payload_length);
    +  // xmit event
    +  SCOPED_MUTEX_LOCK(lock, this->ua_session->mutex, this_ethread());
    +  this->ua_session->handleEvent(HTTP2_SESSION_EVENT_XMIT, &headers);
    +  sent += payload_length;
    +
    +  // Send CONTINUATION frames
    +  while (sent < header_blocks_size) {
    +    DebugSsn(this->ua_session, "http2_cs", "[%" PRId64 "] Send CONTINUATION frame.", this->ua_session->connection_id());
    --- End diff --
    
    It is better to use `DebugHttp2Stream`.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] trafficserver pull request: TS-4092: Decouple HPACK from HTTP/2

Posted by bryancall <gi...@git.apache.org>.
Github user bryancall commented on a diff in the pull request:

    https://github.com/apache/trafficserver/pull/460#discussion_r56908946
  
    --- Diff: proxy/http2/HPACK.cc ---
    @@ -828,9 +828,111 @@ update_dynamic_table_size(const uint8_t *buf_start, const uint8_t *buf_end, Http
       if (len == HPACK_ERROR_COMPRESSION_ERROR)
         return HPACK_ERROR_COMPRESSION_ERROR;
     
    -  if (indexing_table.set_dynamic_table_size(size) == false) {
    +  if (indexing_table.update_maximum_size(size) == false) {
         return HPACK_ERROR_COMPRESSION_ERROR;
       }
     
       return len;
     }
    +
    +int64_t
    +hpack_decode_header_block(HpackIndexingTable &indexing_table, HTTPHdr *hdr, const uint8_t *in_buf, const size_t in_buf_len)
    +{
    +  const uint8_t *cursor = in_buf;
    +  const uint8_t *const in_buf_end = in_buf + in_buf_len;
    +  HdrHeap *heap = hdr->m_heap;
    +  HTTPHdrImpl *hh = hdr->m_http;
    +  bool header_field_started = false;
    +  bool has_http2_violation = false;
    +
    +  while (cursor < in_buf_end) {
    +    int64_t read_bytes = 0;
    +
    +    // decode a header field encoded by HPACK
    +    MIMEField *field = mime_field_create(heap, hh->m_fields_impl);
    +    MIMEFieldWrapper header(field, heap, hh->m_fields_impl);
    +    HpackFieldType ftype = hpack_parse_field_type(*cursor);
    +
    +    switch (ftype) {
    +    case HPACK_FIELD_INDEX:
    +      read_bytes = decode_indexed_header_field(header, cursor, in_buf_end, indexing_table);
    +      if (read_bytes == HPACK_ERROR_COMPRESSION_ERROR) {
    +        return HPACK_ERROR_COMPRESSION_ERROR;
    +      }
    +      cursor += read_bytes;
    +      header_field_started = true;
    +      break;
    +    case HPACK_FIELD_INDEXED_LITERAL:
    +    case HPACK_FIELD_NOINDEX_LITERAL:
    +    case HPACK_FIELD_NEVERINDEX_LITERAL:
    +      read_bytes = decode_literal_header_field(header, cursor, in_buf_end, indexing_table);
    +      if (read_bytes == HPACK_ERROR_COMPRESSION_ERROR) {
    +        return HPACK_ERROR_COMPRESSION_ERROR;
    +      }
    +      if (read_bytes < 0) {
    +        has_http2_violation = true;
    +        read_bytes = -read_bytes;
    +      }
    +      cursor += read_bytes;
    +      header_field_started = true;
    +      break;
    +    case HPACK_FIELD_TABLESIZE_UPDATE:
    +      if (header_field_started) {
    +        return HPACK_ERROR_COMPRESSION_ERROR;
    +      }
    +      read_bytes = update_dynamic_table_size(cursor, in_buf_end, indexing_table);
    +      if (read_bytes == HPACK_ERROR_COMPRESSION_ERROR) {
    +        return HPACK_ERROR_COMPRESSION_ERROR;
    +      }
    +      cursor += read_bytes;
    +      continue;
    +    }
    +    // Store to HdrHeap
    +    mime_hdr_field_attach(hh->m_fields_impl, field, 1, NULL);
    +  }
    +  // Parsing all headers is done
    +  if (has_http2_violation) {
    +    return -(cursor - in_buf);
    +  } else {
    +    return cursor - in_buf;
    +  }
    +}
    +
    +int64_t
    +hpack_encode_header_block(HpackIndexingTable &indexing_table, uint8_t *out_buf, const size_t out_buf_len, HTTPHdr *hdr)
    +{
    +  uint8_t *cursor = out_buf;
    +  const uint8_t *const out_buf_end = out_buf + out_buf_len;
    +  int64_t written;
    +
    +  ink_assert(http_hdr_type_get(hdr->m_http) != HTTP_TYPE_UNKNOWN);
    +
    +  MIMEFieldIter field_iter;
    +  for (MIMEField *field = hdr->iter_get_first(&field_iter); field != NULL; field = hdr->iter_get_next(&field_iter)) {
    +    HpackFieldType field_type;
    +    MIMEFieldWrapper header(field, hdr->m_heap, hdr->m_http->m_fields_impl);
    +    if (header.is_sensitive()) {
    --- End diff --
    
    Can't we just do the Cookie and Authentication header check here instead of adding another mime filed flag?  I would rather have HPACK know about Cookie and Authentication headers instead of adding the sensitive flag to MIME.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] trafficserver pull request: TS-4092: Decouple HPACK from HTTP/2

Posted by maskit <gi...@git.apache.org>.
Github user maskit commented on a diff in the pull request:

    https://github.com/apache/trafficserver/pull/460#discussion_r53549624
  
    --- Diff: proxy/http2/Http2ConnectionState.cc ---
    @@ -992,43 +998,50 @@ Http2ConnectionState::send_headers_frame(FetchSM *fetch_sm)
     
       DebugHttp2Stream(ua_session, stream->get_id(), "Send HEADERS frame");
     
    -  // Write pseudo headers
    -  payload_length += http2_write_psuedo_headers(resp_header, payload_buffer, buf_len, *(this->remote_indexing_table));
    -
    -  // If response body is empty, set END_STREAM flag to HEADERS frame
    -  // Must check to ensure content-length is there.  Otherwise the value defaults
    -  // to 0
    -  if (resp_header->presence(MIME_PRESENCE_CONTENT_LENGTH) && resp_header->get_content_length() == 0) {
    -    flags |= HTTP2_FLAGS_HEADERS_END_STREAM;
    --- End diff --
    
    It's not intended but it seems OK, actually.
    
    I've looked into it and I found that current implementation (on master) sends END_STREAM flag twice. The first one is on a HEADERS frame and the second one is on an empty DATA frame. I'll create a jira ticket for it.
    
    But this logic should be here for efficient framing. I'm going to restore the logic.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] trafficserver pull request: TS-4092: Decouple HPACK from HTTP/2

Posted by maskit <gi...@git.apache.org>.
Github user maskit commented on a diff in the pull request:

    https://github.com/apache/trafficserver/pull/460#discussion_r57849578
  
    --- Diff: proxy/http2/HPACK.h ---
    @@ -90,31 +97,29 @@ class MIMEFieldWrapper
         return _field;
       }
     
    +  bool
    +  is_sensitive() const
    +  {
    +    return _field->is_sensitive();
    +  }
    +
     private:
       MIMEField *_field;
       HdrHeap *_heap;
       MIMEHdrImpl *_mh;
     };
     
    -// Result of looking for a header field in IndexingTable
    -struct Http2LookupIndexResult {
    -  Http2LookupIndexResult() : index(0), value_is_indexed(false) {}
    -
    -  int index;
    -  bool value_is_indexed;
    -};
    -
     // [RFC 7541] 2.3.2. Dynamic Table
    -class Http2DynamicTable
    +class HpackDynamicTable
     {
     public:
    -  Http2DynamicTable() : _current_size(0), _settings_dynamic_table_size(4096)
    +  HpackDynamicTable() : _current_size(0), _maximum_size(4096)
    --- End diff --
    
    The value of 4096 comes from HTTP2, so I changed to pass HTTP2_HEADER_TABLE_SIZE on to the constructor. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] trafficserver pull request: TS-4092: Decouple HPACK from HTTP/2

Posted by maskit <gi...@git.apache.org>.
Github user maskit commented on a diff in the pull request:

    https://github.com/apache/trafficserver/pull/460#discussion_r57056410
  
    --- Diff: proxy/http2/HPACK.cc ---
    @@ -828,9 +828,111 @@ update_dynamic_table_size(const uint8_t *buf_start, const uint8_t *buf_end, Http
       if (len == HPACK_ERROR_COMPRESSION_ERROR)
         return HPACK_ERROR_COMPRESSION_ERROR;
     
    -  if (indexing_table.set_dynamic_table_size(size) == false) {
    +  if (indexing_table.update_maximum_size(size) == false) {
         return HPACK_ERROR_COMPRESSION_ERROR;
       }
     
       return len;
     }
    +
    +int64_t
    +hpack_decode_header_block(HpackIndexingTable &indexing_table, HTTPHdr *hdr, const uint8_t *in_buf, const size_t in_buf_len)
    +{
    +  const uint8_t *cursor = in_buf;
    +  const uint8_t *const in_buf_end = in_buf + in_buf_len;
    +  HdrHeap *heap = hdr->m_heap;
    +  HTTPHdrImpl *hh = hdr->m_http;
    +  bool header_field_started = false;
    +  bool has_http2_violation = false;
    +
    +  while (cursor < in_buf_end) {
    +    int64_t read_bytes = 0;
    +
    +    // decode a header field encoded by HPACK
    +    MIMEField *field = mime_field_create(heap, hh->m_fields_impl);
    +    MIMEFieldWrapper header(field, heap, hh->m_fields_impl);
    +    HpackFieldType ftype = hpack_parse_field_type(*cursor);
    +
    +    switch (ftype) {
    +    case HPACK_FIELD_INDEX:
    +      read_bytes = decode_indexed_header_field(header, cursor, in_buf_end, indexing_table);
    +      if (read_bytes == HPACK_ERROR_COMPRESSION_ERROR) {
    +        return HPACK_ERROR_COMPRESSION_ERROR;
    +      }
    +      cursor += read_bytes;
    +      header_field_started = true;
    +      break;
    +    case HPACK_FIELD_INDEXED_LITERAL:
    +    case HPACK_FIELD_NOINDEX_LITERAL:
    +    case HPACK_FIELD_NEVERINDEX_LITERAL:
    +      read_bytes = decode_literal_header_field(header, cursor, in_buf_end, indexing_table);
    +      if (read_bytes == HPACK_ERROR_COMPRESSION_ERROR) {
    +        return HPACK_ERROR_COMPRESSION_ERROR;
    +      }
    +      if (read_bytes < 0) {
    +        has_http2_violation = true;
    +        read_bytes = -read_bytes;
    +      }
    +      cursor += read_bytes;
    +      header_field_started = true;
    +      break;
    +    case HPACK_FIELD_TABLESIZE_UPDATE:
    +      if (header_field_started) {
    +        return HPACK_ERROR_COMPRESSION_ERROR;
    +      }
    +      read_bytes = update_dynamic_table_size(cursor, in_buf_end, indexing_table);
    +      if (read_bytes == HPACK_ERROR_COMPRESSION_ERROR) {
    +        return HPACK_ERROR_COMPRESSION_ERROR;
    +      }
    +      cursor += read_bytes;
    +      continue;
    +    }
    +    // Store to HdrHeap
    +    mime_hdr_field_attach(hh->m_fields_impl, field, 1, NULL);
    +  }
    +  // Parsing all headers is done
    +  if (has_http2_violation) {
    +    return -(cursor - in_buf);
    +  } else {
    +    return cursor - in_buf;
    +  }
    +}
    +
    +int64_t
    +hpack_encode_header_block(HpackIndexingTable &indexing_table, uint8_t *out_buf, const size_t out_buf_len, HTTPHdr *hdr)
    +{
    +  uint8_t *cursor = out_buf;
    +  const uint8_t *const out_buf_end = out_buf + out_buf_len;
    +  int64_t written;
    +
    +  ink_assert(http_hdr_type_get(hdr->m_http) != HTTP_TYPE_UNKNOWN);
    +
    +  MIMEFieldIter field_iter;
    +  for (MIMEField *field = hdr->iter_get_first(&field_iter); field != NULL; field = hdr->iter_get_next(&field_iter)) {
    +    HpackFieldType field_type;
    +    MIMEFieldWrapper header(field, hdr->m_heap, hdr->m_http->m_fields_impl);
    +    if (header.is_sensitive()) {
    --- End diff --
    
    Yea, we can still do the special treatment in the HPACK layer after the server side implementation. I meant that we should use the flag to determine the representation if we add the flag and I think the flag would be needed in the future.
    
    But, OK, I'll remove the flag for now. It's not a big deal. Let's think about it if we face issues.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] trafficserver pull request: TS-4092: Decouple HPACK from HTTP/2

Posted by maskit <gi...@git.apache.org>.
Github user maskit commented on a diff in the pull request:

    https://github.com/apache/trafficserver/pull/460#discussion_r53552779
  
    --- Diff: proxy/http2/Http2ConnectionState.cc ---
    @@ -992,43 +998,50 @@ Http2ConnectionState::send_headers_frame(FetchSM *fetch_sm)
     
       DebugHttp2Stream(ua_session, stream->get_id(), "Send HEADERS frame");
     
    -  // Write pseudo headers
    -  payload_length += http2_write_psuedo_headers(resp_header, payload_buffer, buf_len, *(this->remote_indexing_table));
    -
    -  // If response body is empty, set END_STREAM flag to HEADERS frame
    -  // Must check to ensure content-length is there.  Otherwise the value defaults
    -  // to 0
    -  if (resp_header->presence(MIME_PRESENCE_CONTENT_LENGTH) && resp_header->get_content_length() == 0) {
    -    flags |= HTTP2_FLAGS_HEADERS_END_STREAM;
    --- End diff --
    
    Restored the logic. The END_STREAM issue will be fixed on TS-4217.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] trafficserver pull request: TS-4092: Decouple HPACK from HTTP/2

Posted by masaori335 <gi...@git.apache.org>.
Github user masaori335 commented on a diff in the pull request:

    https://github.com/apache/trafficserver/pull/460#discussion_r53448759
  
    --- Diff: proxy/http2/HTTP2.cc ---
    @@ -484,246 +498,162 @@ convert_from_2_to_1_1_header(HTTPHdr *headers)
       return PARSE_DONE;
     }
     
    -static int64_t
    -http2_write_header_field(uint8_t *out, const uint8_t *end, MIMEFieldWrapper &header, Http2IndexingTable &indexing_table)
    -{
    -  HpackFieldType field_type = HPACK_FIELD_INDEXED_LITERAL;
    -
    -  // Cookie less that 20 bytes and Authorization are never indexed
    -  // This policy is refer to Firefox and nghttp2
    -  int name_len = 0, value_len = 0;
    -  const char *name = header.name_get(&name_len);
    -  header.value_get(&value_len);
    -  if ((ptr_len_casecmp(name, name_len, MIME_FIELD_COOKIE, MIME_LEN_COOKIE) == 0 && value_len < 20) ||
    -      (ptr_len_casecmp(name, name_len, MIME_FIELD_AUTHORIZATION, MIME_LEN_AUTHORIZATION) == 0)) {
    -    field_type = HPACK_FIELD_NEVERINDEX_LITERAL;
    -  }
    -
    -  // TODO Enable to configure selecting header field representation
    +void
    +http2_convert_header_from_1_1_to_2(HTTPHdr *headers)
    +{
    +  HTTPHdr tmp;
    +  tmp.create(http_hdr_type_get(headers->m_http));
    +  tmp.copy(headers);
    +  headers->fields_clear();
    +
    +  if (http_hdr_type_get(tmp.m_http) == HTTP_TYPE_RESPONSE) {
    +    char status_str[HTTP2_LEN_STATUS_VALUE_STR + 1];
    +    snprintf(status_str, sizeof(status_str), "%d", tmp.status_get());
    +
    +    // Add ':status' header field
    +    MIMEField *status_field = headers->field_create(HTTP2_VALUE_STATUS, HTTP2_LEN_STATUS);
    +    status_field->value_set(headers->m_heap, headers->m_mime, status_str, HTTP2_LEN_STATUS_VALUE_STR);
    +    headers->field_attach(status_field);
    +
    +    MIMEFieldIter field_iter;
    +    for (MIMEField *field = tmp.iter_get_first(&field_iter); field != NULL; field = tmp.iter_get_next(&field_iter)) {
    +      // Intermediaries SHOULD remove connection-specific header fields.
    +      const char *name;
    +      int name_len;
    +      const char *value;
    +      int value_len;
    +      name = field->name_get(&name_len);
    +      if ((name_len == MIME_LEN_CONNECTION && strncasecmp(name, MIME_FIELD_CONNECTION, name_len) == 0) ||
    +          (name_len == MIME_LEN_KEEP_ALIVE && strncasecmp(name, MIME_FIELD_KEEP_ALIVE, name_len) == 0) ||
    +          (name_len == MIME_LEN_PROXY_CONNECTION && strncasecmp(name, MIME_FIELD_PROXY_CONNECTION, name_len) == 0) ||
    +          (name_len == MIME_LEN_TRANSFER_ENCODING && strncasecmp(name, MIME_FIELD_TRANSFER_ENCODING, name_len) == 0) ||
    +          (name_len == MIME_LEN_UPGRADE && strncasecmp(name, MIME_FIELD_UPGRADE, name_len) == 0)) {
    +        continue;
    +      }
     
    -  const Http2LookupIndexResult &result = indexing_table.get_index(header);
    -  if (result.index > 0) {
    -    if (result.value_is_indexed) {
    -      return encode_indexed_header_field(out, end, result.index);
    -    } else {
    -      return encode_literal_header_field_with_indexed_name(out, end, header, result.index, indexing_table, field_type);
    +      MIMEField *newfield;
    +      name = field->name_get(&name_len);
    +      newfield = headers->field_create(name, name_len);
    +      value = field->value_get(&value_len);
    +      newfield->value_set(headers->m_heap, headers->m_mime, value, value_len);
    +      tmp.field_delete(field);
    +      headers->field_attach(newfield);
    +      // Set sensitive flag (See RFC7541 7.1.3)
    +      // - Authorization header obviously should not be indexed
    +      // - Short Cookie header should not be indexed because of low entropy
    +      if ((ptr_len_casecmp(name, name_len, MIME_FIELD_COOKIE, MIME_LEN_COOKIE) == 0 && value_len < 20) ||
    +          (ptr_len_casecmp(name, name_len, MIME_FIELD_AUTHORIZATION, MIME_LEN_AUTHORIZATION) == 0)) {
    +        newfield->m_sensitive = 1;
    +      }
         }
    -  } else {
    -    return encode_literal_header_field_with_new_name(out, end, header, indexing_table, field_type);
       }
    +  tmp.destroy();
     }
     
     int64_t
    -http2_write_psuedo_headers(HTTPHdr *in, uint8_t *out, uint64_t out_len, Http2IndexingTable &indexing_table)
    +http2_encode_header_blocks(HTTPHdr *in, uint8_t *out, uint32_t out_len, HpackHandle &handle)
     {
    -  uint8_t *p = out;
    -  uint8_t *end = out + out_len;
    -  int64_t len;
    -
    -  ink_assert(http_hdr_type_get(in->m_http) != HTTP_TYPE_UNKNOWN);
    -
    -  // TODO Check whether buffer size is enough
    -
    -  // Set psuedo header
    -  if (http_hdr_type_get(in->m_http) == HTTP_TYPE_RESPONSE) {
    -    char status_str[HPACK_LEN_STATUS_VALUE_STR + 1];
    -    snprintf(status_str, sizeof(status_str), "%d", in->status_get());
    -
    -    // Add 'Status:' dummy header field
    -    MIMEField *status_field = mime_field_create(in->m_heap, in->m_http->m_fields_impl);
    -    mime_field_name_value_set(in->m_heap, in->m_mime, status_field, -1, HPACK_VALUE_STATUS, HPACK_LEN_STATUS, status_str,
    -                              HPACK_LEN_STATUS_VALUE_STR, 0, HPACK_LEN_STATUS + HPACK_LEN_STATUS_VALUE_STR, true);
    -    mime_hdr_field_attach(in->m_mime, status_field, 1, NULL);
    -
    -    // Encode psuedo headers by HPACK
    -    MIMEFieldWrapper header(status_field, in->m_heap, in->m_http->m_fields_impl);
    -
    -    len = http2_write_header_field(p, end, header, indexing_table);
    -    if (len == -1)
    -      return -1;
    -    p += len;
    -
    -    // Remove dummy header field
    -    in->field_delete(HPACK_VALUE_STATUS, HPACK_LEN_STATUS);
    -  }
    -
    -  return p - out;
    -}
    -
    -int64_t
    -http2_write_header_fragment(HTTPHdr *in, MIMEFieldIter &field_iter, uint8_t *out, uint64_t out_len,
    -                            Http2IndexingTable &indexing_table, bool &cont)
    -{
    -  uint8_t *p = out;
    -  uint8_t *end = out + out_len;
    -  int64_t len;
    -
    -  ink_assert(http_hdr_type_get(in->m_http) != HTTP_TYPE_UNKNOWN);
    -  ink_assert(in);
    -
    -  // TODO Get a index value from the tables for the header field, and then
    -  // choose a representation type.
    -  // TODO Each indexing types per field should be passed by a caller, HTTP/2
    -  // implementation.
    -
    -  // Get first header field which is required encoding
    -  MIMEField *field;
    -  if (!field_iter.m_block) {
    -    field = in->iter_get_first(&field_iter);
    -  } else {
    -    field = in->iter_get(&field_iter);
    -  }
    -
    -  // Set mime headers
    -  cont = false;
    -  for (; field != NULL; field = in->iter_get_next(&field_iter)) {
    -    // Intermediaries SHOULD remove connection-specific header fields.
    -    int name_len;
    -    const char *name = field->name_get(&name_len);
    -    if ((name_len == MIME_LEN_CONNECTION && strncasecmp(name, MIME_FIELD_CONNECTION, name_len) == 0) ||
    -        (name_len == MIME_LEN_KEEP_ALIVE && strncasecmp(name, MIME_FIELD_KEEP_ALIVE, name_len) == 0) ||
    -        (name_len == MIME_LEN_PROXY_CONNECTION && strncasecmp(name, MIME_FIELD_PROXY_CONNECTION, name_len) == 0) ||
    -        (name_len == MIME_LEN_TRANSFER_ENCODING && strncasecmp(name, MIME_FIELD_TRANSFER_ENCODING, name_len) == 0) ||
    -        (name_len == MIME_LEN_UPGRADE && strncasecmp(name, MIME_FIELD_UPGRADE, name_len) == 0)) {
    -      continue;
    -    }
    -
    -    MIMEFieldWrapper header(field, in->m_heap, in->m_http->m_fields_impl);
    -    if ((len = http2_write_header_field(p, end, header, indexing_table)) == -1) {
    -      if (p == out) {
    -        // no progress was made, header was too big for the buffer, skipping for now
    -        continue;
    -      }
    -      if (!cont) {
    -        // Parsing a part of headers is done
    -        cont = true;
    -        return p - out;
    -      } else {
    -        // Parse error
    -        return -1;
    -      }
    -    }
    -    p += len;
    -  }
    -
    -  // Parsing all headers is done
    -  return p - out;
    +  // TODO: It would be better to split Cookie header value
    +  return hpack_encode_header_block(handle, out, out_len, in);
     }
     
     /*
      * Decode Header Blocks to Header List.
      */
     int64_t
    -http2_decode_header_blocks(HTTPHdr *hdr, const uint8_t *buf_start, const uint8_t *buf_end, Http2IndexingTable &indexing_table,
    +http2_decode_header_blocks(HTTPHdr *hdr, const uint8_t *buf_start, const uint32_t buf_len, HpackHandle &handle,
                                bool &trailing_header)
     {
    -  const uint8_t *cursor = buf_start;
    -  HdrHeap *heap = hdr->m_heap;
    -  HTTPHdrImpl *hh = hdr->m_http;
    -  bool header_field_started = false;
    +  const MIMEField *field;
    +  const char *value;
    +  int len;
       bool is_trailing_header = trailing_header;
    +  int64_t result = hpack_decode_header_block(handle, hdr, buf_start, buf_len);
     
    -  while (cursor < buf_end) {
    -    int64_t read_bytes = 0;
    +  if (result < 0) {
    +    return result;
    +  }
     
    -    // decode a header field encoded by HPACK
    -    MIMEField *field = mime_field_create(heap, hh->m_fields_impl);
    -    MIMEFieldWrapper header(field, heap, hh->m_fields_impl);
    -    HpackFieldType ftype = hpack_parse_field_type(*cursor);
     
    -    switch (ftype) {
    -    case HPACK_FIELD_INDEX:
    -      read_bytes = decode_indexed_header_field(header, cursor, buf_end, indexing_table);
    -      if (read_bytes == HPACK_ERROR_COMPRESSION_ERROR) {
    -        return HPACK_ERROR_COMPRESSION_ERROR;
    -      }
    -      cursor += read_bytes;
    -      header_field_started = true;
    -      break;
    -    case HPACK_FIELD_INDEXED_LITERAL:
    -    case HPACK_FIELD_NOINDEX_LITERAL:
    -    case HPACK_FIELD_NEVERINDEX_LITERAL:
    -      read_bytes = decode_literal_header_field(header, cursor, buf_end, indexing_table);
    -      if (read_bytes == HPACK_ERROR_COMPRESSION_ERROR) {
    -        return HPACK_ERROR_COMPRESSION_ERROR;
    -      }
    -      cursor += read_bytes;
    -      header_field_started = true;
    -      break;
    -    case HPACK_FIELD_TABLESIZE_UPDATE:
    -      if (header_field_started) {
    -        return HPACK_ERROR_COMPRESSION_ERROR;
    +  MIMEFieldIter iter;
    +  unsigned int expected_pseudo_header_count = 4;
    +  unsigned int pseudo_header_count = 0;
    +
    +  if (is_trailing_header) {
    +    expected_pseudo_header_count = 0;
    +  }
    +  for (field = hdr->iter_get_first(&iter); field != NULL; field = hdr->iter_get_next(&iter)) {
    +    value = field->name_get(&len);
    +    // Pseudo headers must appear before regular headers
    +    if (len && value[0] == ':') {
    +      ++pseudo_header_count;
    +      if (pseudo_header_count > expected_pseudo_header_count) {
    +        return -result;
    --- End diff --
    
    How about changing both of them?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] trafficserver pull request: TS-4092: Decouple HPACK from HTTP/2

Posted by bryancall <gi...@git.apache.org>.
Github user bryancall commented on a diff in the pull request:

    https://github.com/apache/trafficserver/pull/460#discussion_r56910134
  
    --- Diff: proxy/http2/HTTP2.cc ---
    @@ -484,246 +498,175 @@ convert_from_2_to_1_1_header(HTTPHdr *headers)
       return PARSE_DONE;
     }
     
    -static int64_t
    -http2_write_header_field(uint8_t *out, const uint8_t *end, MIMEFieldWrapper &header, Http2IndexingTable &indexing_table)
    -{
    -  HpackFieldType field_type = HPACK_FIELD_INDEXED_LITERAL;
    -
    -  // Cookie less that 20 bytes and Authorization are never indexed
    -  // This policy is refer to Firefox and nghttp2
    -  int name_len = 0, value_len = 0;
    -  const char *name = header.name_get(&name_len);
    -  header.value_get(&value_len);
    -  if ((ptr_len_casecmp(name, name_len, MIME_FIELD_COOKIE, MIME_LEN_COOKIE) == 0 && value_len < 20) ||
    -      (ptr_len_casecmp(name, name_len, MIME_FIELD_AUTHORIZATION, MIME_LEN_AUTHORIZATION) == 0)) {
    -    field_type = HPACK_FIELD_NEVERINDEX_LITERAL;
    -  }
    -
    -  // TODO Enable to configure selecting header field representation
    +void
    +http2_convert_header_from_1_1_to_2(HTTPHdr *headers)
    +{
    +  HTTPHdr tmp;
    +  tmp.create(http_hdr_type_get(headers->m_http));
    +  tmp.copy(headers);
    +  headers->fields_clear();
    +
    +  if (http_hdr_type_get(tmp.m_http) == HTTP_TYPE_RESPONSE) {
    +    char status_str[HTTP2_LEN_STATUS_VALUE_STR + 1];
    +    snprintf(status_str, sizeof(status_str), "%d", tmp.status_get());
    +
    +    // Add ':status' header field
    +    MIMEField *status_field = headers->field_create(HTTP2_VALUE_STATUS, HTTP2_LEN_STATUS);
    +    status_field->value_set(headers->m_heap, headers->m_mime, status_str, HTTP2_LEN_STATUS_VALUE_STR);
    +    headers->field_attach(status_field);
    +
    +    MIMEFieldIter field_iter;
    +    for (MIMEField *field = tmp.iter_get_first(&field_iter); field != NULL; field = tmp.iter_get_next(&field_iter)) {
    +      // Intermediaries SHOULD remove connection-specific header fields.
    +      const char *name;
    +      int name_len;
    +      const char *value;
    +      int value_len;
    +      name = field->name_get(&name_len);
    +      if ((name_len == MIME_LEN_CONNECTION && strncasecmp(name, MIME_FIELD_CONNECTION, name_len) == 0) ||
    +          (name_len == MIME_LEN_KEEP_ALIVE && strncasecmp(name, MIME_FIELD_KEEP_ALIVE, name_len) == 0) ||
    +          (name_len == MIME_LEN_PROXY_CONNECTION && strncasecmp(name, MIME_FIELD_PROXY_CONNECTION, name_len) == 0) ||
    +          (name_len == MIME_LEN_TRANSFER_ENCODING && strncasecmp(name, MIME_FIELD_TRANSFER_ENCODING, name_len) == 0) ||
    +          (name_len == MIME_LEN_UPGRADE && strncasecmp(name, MIME_FIELD_UPGRADE, name_len) == 0)) {
    +        continue;
    +      }
     
    -  const Http2LookupIndexResult &result = indexing_table.get_index(header);
    -  if (result.index > 0) {
    -    if (result.value_is_indexed) {
    -      return encode_indexed_header_field(out, end, result.index);
    -    } else {
    -      return encode_literal_header_field_with_indexed_name(out, end, header, result.index, indexing_table, field_type);
    +      MIMEField *newfield;
    +      name = field->name_get(&name_len);
    +      newfield = headers->field_create(name, name_len);
    +      value = field->value_get(&value_len);
    +      newfield->value_set(headers->m_heap, headers->m_mime, value, value_len);
    +      tmp.field_delete(field);
    +      headers->field_attach(newfield);
    +      // Set sensitive flag (See RFC7541 7.1.3)
    +      // - Authorization header obviously should not be indexed
    +      // - Short Cookie header should not be indexed because of low entropy
    +      if ((ptr_len_casecmp(name, name_len, MIME_FIELD_COOKIE, MIME_LEN_COOKIE) == 0 && value_len < 20) ||
    --- End diff --
    
    This wouldn't need to be done if the logic is in HPACK


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] trafficserver pull request: TS-4092: Decouple HPACK from HTTP/2

Posted by masaori335 <gi...@git.apache.org>.
Github user masaori335 commented on a diff in the pull request:

    https://github.com/apache/trafficserver/pull/460#discussion_r53414542
  
    --- Diff: proxy/http2/Http2ConnectionState.cc ---
    @@ -992,43 +998,50 @@ Http2ConnectionState::send_headers_frame(FetchSM *fetch_sm)
     
       DebugHttp2Stream(ua_session, stream->get_id(), "Send HEADERS frame");
     
    -  // Write pseudo headers
    -  payload_length += http2_write_psuedo_headers(resp_header, payload_buffer, buf_len, *(this->remote_indexing_table));
    -
    -  // If response body is empty, set END_STREAM flag to HEADERS frame
    -  // Must check to ensure content-length is there.  Otherwise the value defaults
    -  // to 0
    -  if (resp_header->presence(MIME_PRESENCE_CONTENT_LENGTH) && resp_header->get_content_length() == 0) {
    -    flags |= HTTP2_FLAGS_HEADERS_END_STREAM;
    --- End diff --
    
    It looks like this logic is gone. Is this OK?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

[GitHub] trafficserver pull request: TS-4092: Decouple HPACK from HTTP/2

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

    https://github.com/apache/trafficserver/pull/460


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastructure@apache.org or file a JIRA ticket
with INFRA.
---