You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@trafficserver.apache.org by ma...@apache.org on 2016/01/09 10:07:24 UTC

[1/2] trafficserver git commit: TS-3478: Indexing header representations on HPACK encoder

Repository: trafficserver
Updated Branches:
  refs/heads/master 442511397 -> 770e7878c


TS-3478: Indexing header representations on HPACK encoder

This closes #391


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

Branch: refs/heads/master
Commit: 82ce75676f2ad614be20793b4a217308e6981e84
Parents: 4425113
Author: Ryo Okubo <ro...@yahoo-corp.jp>
Authored: Sun Dec 13 23:42:55 2015 +0900
Committer: Masaori Koshiba <ma...@apache.org>
Committed: Sat Jan 9 18:03:38 2016 +0900

----------------------------------------------------------------------
 proxy/http2/HPACK.cc                | 183 +++++++++++++++++-----
 proxy/http2/HPACK.h                 |  60 +++++---
 proxy/http2/HTTP2.cc                |  46 +++++-
 proxy/http2/HTTP2.h                 |   6 +-
 proxy/http2/Http2ConnectionState.cc |   8 +-
 proxy/http2/Http2ConnectionState.h  |  12 +-
 proxy/http2/Http2Stream.h           |   4 +-
 proxy/http2/RegressionHPACK.cc      | 254 ++++++++++++++++++++++---------
 8 files changed, 423 insertions(+), 150 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/trafficserver/blob/82ce7567/proxy/http2/HPACK.cc
----------------------------------------------------------------------
diff --git a/proxy/http2/HPACK.cc b/proxy/http2/HPACK.cc
index b54d923..3254aa5 100644
--- a/proxy/http2/HPACK.cc
+++ b/proxy/http2/HPACK.cc
@@ -174,18 +174,62 @@ const static struct {
                     {"via", ""},
                     {"www-authenticate", ""}};
 
+Http2LookupIndexResult
+Http2IndexingTable::get_index(const MIMEFieldWrapper &field) const
+{
+  Http2LookupIndexResult result;
+  int target_name_len = 0, target_value_len = 0;
+  const char *target_name = field.name_get(&target_name_len);
+  const char *target_value = field.value_get(&target_value_len);
+  const int entry_num = TS_HPACK_STATIC_TABLE_ENTRY_NUM + _dynamic_table.get_current_entry_num();
+
+  for (int index = 1; index < entry_num; ++index) {
+    const char *table_name, *table_value;
+    int table_name_len = 0, table_value_len = 0;
+
+    if (index < TS_HPACK_STATIC_TABLE_ENTRY_NUM) {
+      // static table
+      table_name = STATIC_TABLE[index].name;
+      table_value = STATIC_TABLE[index].value;
+      table_name_len = strlen(table_name);
+      table_value_len = strlen(table_value);
+    } else {
+      // dynamic table
+      const MIMEField *m_field = _dynamic_table.get_header_field(index - TS_HPACK_STATIC_TABLE_ENTRY_NUM);
+
+      table_name = m_field->name_get(&table_name_len);
+      table_value = m_field->value_get(&table_value_len);
+    }
+
+    // Check whether name (and value) are matched
+    if (ptr_len_casecmp(target_name, target_name_len, table_name, table_name_len) == 0) {
+      if (ptr_len_cmp(target_value, target_value_len, table_value, table_value_len) == 0) {
+        result.index = index;
+        result.value_is_indexed = true;
+        break;
+      } else if (!result.index) {
+        result.index = index;
+      }
+    }
+  }
+
+  return result;
+}
+
 int
-Http2DynamicTable::get_header_from_indexing_tables(uint32_t index, MIMEFieldWrapper &field) const
+Http2IndexingTable::get_header_field(uint32_t index, MIMEFieldWrapper &field) const
 {
   // Index Address Space starts at 1, so index == 0 is invalid.
   if (!index)
     return HPACK_ERROR_COMPRESSION_ERROR;
 
   if (index < TS_HPACK_STATIC_TABLE_ENTRY_NUM) {
+    // static table
     field.name_set(STATIC_TABLE[index].name, strlen(STATIC_TABLE[index].name));
     field.value_set(STATIC_TABLE[index].value, strlen(STATIC_TABLE[index].value));
-  } else if (index < TS_HPACK_STATIC_TABLE_ENTRY_NUM + get_current_entry_num()) {
-    const MIMEField *m_field = get_header(index - TS_HPACK_STATIC_TABLE_ENTRY_NUM + 1);
+  } else if (index < TS_HPACK_STATIC_TABLE_ENTRY_NUM + _dynamic_table.get_current_entry_num()) {
+    // dynamic table
+    const MIMEField *m_field = _dynamic_table.get_header_field(index - TS_HPACK_STATIC_TABLE_ENTRY_NUM);
 
     int name_len, value_len;
     const char *name = m_field->name_get(&name_len);
@@ -203,33 +247,34 @@ Http2DynamicTable::get_header_from_indexing_tables(uint32_t index, MIMEFieldWrap
   return 0;
 }
 
-//
-// [RFC 7541] 4.3. Entry Eviction when Header Table Size Changes
-//
-// Whenever the maximum size for the header table is reduced, entries
-// are evicted from the end of the header table until the size of the
-// header table is less than or equal to the maximum size.
-//
-bool
-Http2DynamicTable::set_dynamic_table_size(uint32_t new_size)
+void
+Http2IndexingTable::add_header_field_to_dynamic_table(const MIMEField *field)
 {
-  while (_current_size > new_size) {
-    if (_headers.n <= 0) {
-      return false;
-    }
-    int last_name_len, last_value_len;
-    MIMEField *last_field = _headers.last();
+  _dynamic_table.add_header_field(field);
+}
 
-    last_field->name_get(&last_name_len);
-    last_field->value_get(&last_value_len);
-    _current_size -= ADDITIONAL_OCTETS + last_name_len + last_value_len;
+uint32_t
+Http2IndexingTable::get_dynamic_table_size() const
+{
+  return _dynamic_table.get_size();
+}
 
-    _headers.remove_index(_headers.length() - 1);
-    _mhdr->field_delete(last_field, false);
-  }
+bool
+Http2IndexingTable::set_dynamic_table_size(uint32_t new_size)
+{
+  return _dynamic_table.set_size(new_size);
+}
 
-  _settings_dynamic_table_size = new_size;
-  return true;
+bool
+Http2IndexingTable::is_header_in_dynamic_table(const char *target_name, const char *target_value) const
+{
+  return _dynamic_table.is_header_in(target_name, target_value);
+}
+
+const MIMEField *
+Http2DynamicTable::get_header_field(uint32_t index) const
+{
+  return _headers.get(index);
 }
 
 void
@@ -264,11 +309,72 @@ Http2DynamicTable::add_header_field(const MIMEField *field)
 
     MIMEField *new_field = _mhdr->field_create(name, name_len);
     new_field->value_set(_mhdr->m_heap, _mhdr->m_mime, value, value_len);
+    _mhdr->field_attach(new_field);
     // XXX Because entire Vec instance is copied, Its too expensive!
     _headers.insert(0, new_field);
   }
 }
 
+uint32_t
+Http2DynamicTable::get_size() const
+{
+  return _current_size;
+}
+
+//
+// [RFC 7541] 4.3. Entry Eviction when Header Table Size Changes
+//
+// Whenever the maximum size for the header table is reduced, entries
+// are evicted from the end of the header table until the size of the
+// header table is less than or equal to the maximum size.
+//
+bool
+Http2DynamicTable::set_size(uint32_t new_size)
+{
+  while (_current_size > new_size) {
+    if (_headers.n <= 0) {
+      return false;
+    }
+    int last_name_len, last_value_len;
+    MIMEField *last_field = _headers.last();
+
+    last_field->name_get(&last_name_len);
+    last_field->value_get(&last_value_len);
+    _current_size -= ADDITIONAL_OCTETS + last_name_len + last_value_len;
+
+    _headers.remove_index(_headers.length() - 1);
+    _mhdr->field_delete(last_field, false);
+  }
+
+  _settings_dynamic_table_size = new_size;
+  return true;
+}
+
+const uint32_t
+Http2DynamicTable::get_current_entry_num() const
+{
+  return _headers.length();
+}
+
+bool
+Http2DynamicTable::is_header_in(const char *target_name, const char *target_value) const
+{
+  const MIMEField *field = _mhdr->field_find(target_name, strlen(target_name));
+
+  if (field) {
+    do {
+      int target_value_len = strlen(target_value);
+      int table_value_len = 0;
+      const char *table_value = field->value_get(&table_value_len);
+      if (ptr_len_cmp(target_value, target_value_len, table_value, table_value_len) == 0) {
+        return true;
+      }
+    } while (field->has_dups() && (field = field->m_next_dup) != NULL);
+  }
+
+  return false;
+}
+
 //
 // The first byte of an HPACK field unambiguously tells us what
 // kind of field it is. Field types are specified in the high 4 bits
@@ -402,12 +508,13 @@ encode_indexed_header_field(uint8_t *buf_start, const uint8_t *buf_end, uint32_t
   *p |= 0x80;
   p += len;
 
+  Debug("http2_hpack_encode", "Encoded field: %d", index);
   return p - buf_start;
 }
 
 int64_t
-encode_literal_header_field(uint8_t *buf_start, const uint8_t *buf_end, const MIMEFieldWrapper &header, uint32_t index,
-                            HpackFieldType type)
+encode_literal_header_field_with_indexed_name(uint8_t *buf_start, const uint8_t *buf_end, const MIMEFieldWrapper &header,
+                                              uint32_t index, Http2IndexingTable &indexing_table, HpackFieldType type)
 {
   uint8_t *p = buf_start;
   int64_t len;
@@ -417,6 +524,7 @@ encode_literal_header_field(uint8_t *buf_start, const uint8_t *buf_end, const MI
 
   switch (type) {
   case HPACK_FIELD_INDEXED_LITERAL:
+    indexing_table.add_header_field_to_dynamic_table(header.field_get());
     prefix = 6;
     flag = 0x40;
     break;
@@ -452,11 +560,13 @@ encode_literal_header_field(uint8_t *buf_start, const uint8_t *buf_end, const MI
     return -1;
   p += len;
 
+  Debug("http2_hpack_encode", "Encoded field: %d: %.*s", index, value_len, value);
   return p - buf_start;
 }
 
 int64_t
-encode_literal_header_field(uint8_t *buf_start, const uint8_t *buf_end, const MIMEFieldWrapper &header, HpackFieldType type)
+encode_literal_header_field_with_new_name(uint8_t *buf_start, const uint8_t *buf_end, const MIMEFieldWrapper &header,
+                                          Http2IndexingTable &indexing_table, HpackFieldType type)
 {
   uint8_t *p = buf_start;
   int64_t len;
@@ -466,6 +576,7 @@ encode_literal_header_field(uint8_t *buf_start, const uint8_t *buf_end, const MI
 
   switch (type) {
   case HPACK_FIELD_INDEXED_LITERAL:
+    indexing_table.add_header_field_to_dynamic_table(header.field_get());
     flag = 0x40;
     break;
   case HPACK_FIELD_NOINDEX_LITERAL:
@@ -592,7 +703,7 @@ decode_string(Arena &arena, char **str, uint32_t &str_length, const uint8_t *buf
 //
 int64_t
 decode_indexed_header_field(MIMEFieldWrapper &header, const uint8_t *buf_start, const uint8_t *buf_end,
-                            Http2DynamicTable &dynamic_table)
+                            Http2IndexingTable &indexing_table)
 {
   uint32_t index = 0;
   int64_t len = 0;
@@ -601,7 +712,7 @@ decode_indexed_header_field(MIMEFieldWrapper &header, const uint8_t *buf_start,
   if (len == HPACK_ERROR_COMPRESSION_ERROR)
     return HPACK_ERROR_COMPRESSION_ERROR;
 
-  if (dynamic_table.get_header_from_indexing_tables(index, header) == HPACK_ERROR_COMPRESSION_ERROR) {
+  if (indexing_table.get_header_field(index, header) == HPACK_ERROR_COMPRESSION_ERROR) {
     return HPACK_ERROR_COMPRESSION_ERROR;
   }
 
@@ -625,7 +736,7 @@ decode_indexed_header_field(MIMEFieldWrapper &header, const uint8_t *buf_start,
 //
 int64_t
 decode_literal_header_field(MIMEFieldWrapper &header, const uint8_t *buf_start, const uint8_t *buf_end,
-                            Http2DynamicTable &dynamic_table)
+                            Http2IndexingTable &indexing_table)
 {
   const uint8_t *p = buf_start;
   bool isIncremental = false;
@@ -652,7 +763,7 @@ decode_literal_header_field(MIMEFieldWrapper &header, const uint8_t *buf_start,
 
   // Decode header field name
   if (index) {
-    dynamic_table.get_header_from_indexing_tables(index, header);
+    indexing_table.get_header_field(index, header);
   } else {
     char *name_str = NULL;
     uint32_t name_str_len = 0;
@@ -685,7 +796,7 @@ decode_literal_header_field(MIMEFieldWrapper &header, const uint8_t *buf_start,
 
   // Incremental Indexing adds header to header table as new entry
   if (isIncremental) {
-    dynamic_table.add_header_field(header.field_get());
+    indexing_table.add_header_field_to_dynamic_table(header.field_get());
   }
 
   // Print decoded header field
@@ -706,7 +817,7 @@ decode_literal_header_field(MIMEFieldWrapper &header, const uint8_t *buf_start,
 // [RFC 7541] 6.3. Dynamic Table Size Update
 //
 int64_t
-update_dynamic_table_size(const uint8_t *buf_start, const uint8_t *buf_end, Http2DynamicTable &dynamic_table)
+update_dynamic_table_size(const uint8_t *buf_start, const uint8_t *buf_end, Http2IndexingTable &indexing_table)
 {
   if (buf_start == buf_end)
     return HPACK_ERROR_COMPRESSION_ERROR;
@@ -717,7 +828,7 @@ 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 (dynamic_table.set_dynamic_table_size(size) == false) {
+  if (indexing_table.set_dynamic_table_size(size) == false) {
     return HPACK_ERROR_COMPRESSION_ERROR;
   }
 

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/82ce7567/proxy/http2/HPACK.h
----------------------------------------------------------------------
diff --git a/proxy/http2/HPACK.h b/proxy/http2/HPACK.h
index ebe9a5d..2cda962 100644
--- a/proxy/http2/HPACK.h
+++ b/proxy/http2/HPACK.h
@@ -96,6 +96,14 @@ private:
   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
 {
@@ -114,23 +122,18 @@ public:
     delete _mhdr;
   }
 
+  const MIMEField * get_header_field(uint32_t index) const;
   void add_header_field(const MIMEField *field);
-  int get_header_from_indexing_tables(uint32_t index, MIMEFieldWrapper &header_field) const;
-  bool set_dynamic_table_size(uint32_t new_size);
 
-private:
-  const MIMEField *
-  get_header(uint32_t index) const
-  {
-    return _headers.get(index - 1);
-  }
+  uint32_t get_size() const;
+  bool set_size(uint32_t new_size);
 
-  const uint32_t
-  get_current_entry_num() const
-  {
-    return _headers.length();
-  }
+  const uint32_t get_current_entry_num() const;
+
+  // For regression test
+  bool is_header_in(const char *target_name, const char *target_value) const;
 
+private:
   uint32_t _current_size;
   uint32_t _settings_dynamic_table_size;
 
@@ -138,6 +141,23 @@ private:
   Vec<MIMEField *> _headers;
 };
 
+
+// [RFC 7541] 2.3. Indexing Table
+class Http2IndexingTable
+{
+public:
+  Http2LookupIndexResult get_index(const MIMEFieldWrapper &field) const;
+  int get_header_field(uint32_t index, MIMEFieldWrapper &header_field) const;
+
+  void add_header_field_to_dynamic_table(const MIMEField *field);
+  uint32_t get_dynamic_table_size() const;
+  bool set_dynamic_table_size(uint32_t new_size);
+  bool is_header_in_dynamic_table(const char *target_name, const char *target_value) const;
+
+private:
+  Http2DynamicTable _dynamic_table;
+};
+
 HpackFieldType hpack_parse_field_type(uint8_t ftype);
 
 static inline bool
@@ -152,17 +172,17 @@ int64_t encode_string(uint8_t *buf_start, const uint8_t *buf_end, const char *va
 int64_t decode_string(Arena &arena, char **str, uint32_t &str_length, const uint8_t *buf_start, const uint8_t *buf_end);
 
 int64_t encode_indexed_header_field(uint8_t *buf_start, const uint8_t *buf_end, uint32_t index);
-int64_t encode_literal_header_field(uint8_t *buf_start, const uint8_t *buf_end, const MIMEFieldWrapper &header, uint32_t index,
-                                    HpackFieldType type);
-int64_t encode_literal_header_field(uint8_t *buf_start, const uint8_t *buf_end, const MIMEFieldWrapper &header,
-                                    HpackFieldType type);
+int64_t encode_literal_header_field_with_indexed_name(uint8_t *buf_start, const uint8_t *buf_end, const MIMEFieldWrapper &header,
+                                                      uint32_t index, Http2IndexingTable &indexing_table, HpackFieldType type);
+int64_t encode_literal_header_field_with_new_name(uint8_t *buf_start, const uint8_t *buf_end, const MIMEFieldWrapper &header,
+    Http2IndexingTable &indexing_table, HpackFieldType type);
 
 // When these functions returns minus value, any error occurs
 // TODO Separate error code and length of processed buffer
 int64_t decode_indexed_header_field(MIMEFieldWrapper &header, const uint8_t *buf_start, const uint8_t *buf_end,
-                                    Http2DynamicTable &dynamic_table);
+                                    Http2IndexingTable &indexing_table);
 int64_t decode_literal_header_field(MIMEFieldWrapper &header, const uint8_t *buf_start, const uint8_t *buf_end,
-                                    Http2DynamicTable &dynamic_table);
-int64_t update_dynamic_table_size(const uint8_t *buf_start, const uint8_t *buf_end, Http2DynamicTable &dynamic_table);
+                                    Http2IndexingTable &indexing_table);
+int64_t update_dynamic_table_size(const uint8_t *buf_start, const uint8_t *buf_end, Http2IndexingTable &indexing_table);
 
 #endif /* __HPACK_H__ */

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/82ce7567/proxy/http2/HTTP2.cc
----------------------------------------------------------------------
diff --git a/proxy/http2/HTTP2.cc b/proxy/http2/HTTP2.cc
index a9b0e2b..27ee49c 100644
--- a/proxy/http2/HTTP2.cc
+++ b/proxy/http2/HTTP2.cc
@@ -484,8 +484,37 @@ 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
+
+  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);
+    }
+  } else {
+    return encode_literal_header_field_with_new_name(out, end, header, indexing_table, field_type);
+  }
+}
+
 int64_t
-http2_write_psuedo_headers(HTTPHdr *in, uint8_t *out, uint64_t out_len, Http2DynamicTable & /* dynamic_table */)
+http2_write_psuedo_headers(HTTPHdr *in, uint8_t *out, uint64_t out_len, Http2IndexingTable &indexing_table)
 {
   uint8_t *p = out;
   uint8_t *end = out + out_len;
@@ -508,7 +537,8 @@ http2_write_psuedo_headers(HTTPHdr *in, uint8_t *out, uint64_t out_len, Http2Dyn
 
     // Encode psuedo headers by HPACK
     MIMEFieldWrapper header(status_field, in->m_heap, in->m_http->m_fields_impl);
-    len = encode_literal_header_field(p, end, header, HPACK_FIELD_NEVERINDEX_LITERAL);
+
+    len = http2_write_header_field(p, end, header, indexing_table);
     if (len == -1)
       return -1;
     p += len;
@@ -522,7 +552,7 @@ http2_write_psuedo_headers(HTTPHdr *in, uint8_t *out, uint64_t out_len, Http2Dyn
 
 int64_t
 http2_write_header_fragment(HTTPHdr *in, MIMEFieldIter &field_iter, uint8_t *out, uint64_t out_len,
-                            Http2DynamicTable & /* dynamic_table */, bool &cont)
+                            Http2IndexingTable &indexing_table, bool &cont)
 {
   uint8_t *p = out;
   uint8_t *end = out + out_len;
@@ -559,7 +589,7 @@ http2_write_header_fragment(HTTPHdr *in, MIMEFieldIter &field_iter, uint8_t *out
     }
 
     MIMEFieldWrapper header(field, in->m_heap, in->m_http->m_fields_impl);
-    if ((len = encode_literal_header_field(p, end, header, HPACK_FIELD_INDEXED_LITERAL)) == -1) {
+    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;
@@ -584,7 +614,7 @@ http2_write_header_fragment(HTTPHdr *in, MIMEFieldIter &field_iter, uint8_t *out
  * Decode Header Blocks to Header List.
  */
 int64_t
-http2_decode_header_blocks(HTTPHdr *hdr, const uint8_t *buf_start, const uint8_t *buf_end, Http2DynamicTable &dynamic_table)
+http2_decode_header_blocks(HTTPHdr *hdr, const uint8_t *buf_start, const uint8_t *buf_end, Http2IndexingTable &indexing_table)
 {
   const uint8_t *cursor = buf_start;
   HdrHeap *heap = hdr->m_heap;
@@ -601,7 +631,7 @@ http2_decode_header_blocks(HTTPHdr *hdr, const uint8_t *buf_start, const uint8_t
 
     switch (ftype) {
     case HPACK_FIELD_INDEX:
-      read_bytes = decode_indexed_header_field(header, cursor, buf_end, dynamic_table);
+      read_bytes = decode_indexed_header_field(header, cursor, buf_end, indexing_table);
       if (read_bytes == HPACK_ERROR_COMPRESSION_ERROR) {
         return HPACK_ERROR_COMPRESSION_ERROR;
       }
@@ -611,7 +641,7 @@ http2_decode_header_blocks(HTTPHdr *hdr, const uint8_t *buf_start, const uint8_t
     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, dynamic_table);
+      read_bytes = decode_literal_header_field(header, cursor, buf_end, indexing_table);
       if (read_bytes == HPACK_ERROR_COMPRESSION_ERROR) {
         return HPACK_ERROR_COMPRESSION_ERROR;
       }
@@ -622,7 +652,7 @@ http2_decode_header_blocks(HTTPHdr *hdr, const uint8_t *buf_start, const uint8_t
       if (header_field_started) {
         return HPACK_ERROR_COMPRESSION_ERROR;
       }
-      read_bytes = update_dynamic_table_size(cursor, buf_end, dynamic_table);
+      read_bytes = update_dynamic_table_size(cursor, buf_end, indexing_table);
       if (read_bytes == HPACK_ERROR_COMPRESSION_ERROR) {
         return HPACK_ERROR_COMPRESSION_ERROR;
       }

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/82ce7567/proxy/http2/HTTP2.h
----------------------------------------------------------------------
diff --git a/proxy/http2/HTTP2.h b/proxy/http2/HTTP2.h
index ba322db..21f29e5 100644
--- a/proxy/http2/HTTP2.h
+++ b/proxy/http2/HTTP2.h
@@ -326,13 +326,13 @@ bool http2_parse_goaway(IOVec, Http2Goaway &);
 
 bool http2_parse_window_update(IOVec, uint32_t &);
 
-int64_t http2_decode_header_blocks(HTTPHdr *, const uint8_t *, const uint8_t *, Http2DynamicTable &);
+int64_t http2_decode_header_blocks(HTTPHdr *, const uint8_t *, const uint8_t *, Http2IndexingTable &);
 
 MIMEParseResult convert_from_2_to_1_1_header(HTTPHdr *);
 
-int64_t http2_write_psuedo_headers(HTTPHdr *, uint8_t *, uint64_t, Http2DynamicTable &);
+int64_t http2_write_psuedo_headers(HTTPHdr *, uint8_t *, uint64_t, Http2IndexingTable &);
 
-int64_t http2_write_header_fragment(HTTPHdr *, MIMEFieldIter &, uint8_t *, uint64_t, Http2DynamicTable &, bool &);
+int64_t http2_write_header_fragment(HTTPHdr *, MIMEFieldIter &, uint8_t *, uint64_t, Http2IndexingTable &, bool &);
 
 // Not sure where else to put this, but figure this is as good of a start as
 // anything else.

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/82ce7567/proxy/http2/Http2ConnectionState.cc
----------------------------------------------------------------------
diff --git a/proxy/http2/Http2ConnectionState.cc b/proxy/http2/Http2ConnectionState.cc
index 0f950e7..c073e2e 100644
--- a/proxy/http2/Http2ConnectionState.cc
+++ b/proxy/http2/Http2ConnectionState.cc
@@ -251,7 +251,7 @@ rcv_headers_frame(Http2ClientSession &cs, Http2ConnectionState &cstate, const Ht
       return Http2Error(HTTP2_ERROR_CLASS_CONNECTION, HTTP2_ERROR_PROTOCOL_ERROR);
     }
 
-    const int64_t decoded_bytes = stream->decode_header_blocks(*cstate.local_dynamic_table);
+    const int64_t decoded_bytes = stream->decode_header_blocks(*cstate.local_indexing_table);
 
     if (decoded_bytes == 0 || decoded_bytes == HPACK_ERROR_COMPRESSION_ERROR) {
       return Http2Error(HTTP2_ERROR_CLASS_CONNECTION, HTTP2_ERROR_COMPRESSION_ERROR);
@@ -644,7 +644,7 @@ rcv_continuation_frame(Http2ClientSession &cs, Http2ConnectionState &cstate, con
       return Http2Error(HTTP2_ERROR_CLASS_CONNECTION, HTTP2_ERROR_PROTOCOL_ERROR);
     }
 
-    const int64_t decoded_bytes = stream->decode_header_blocks(*cstate.local_dynamic_table);
+    const int64_t decoded_bytes = stream->decode_header_blocks(*cstate.local_indexing_table);
 
     if (decoded_bytes == 0 || decoded_bytes == HPACK_ERROR_COMPRESSION_ERROR) {
       return Http2Error(HTTP2_ERROR_CLASS_CONNECTION, HTTP2_ERROR_COMPRESSION_ERROR);
@@ -962,7 +962,7 @@ Http2ConnectionState::send_headers_frame(FetchSM *fetch_sm)
   HTTPHdr *resp_header = reinterpret_cast<HTTPHdr *>(fetch_sm->resp_hdr_bufp());
 
   // Write pseudo headers
-  payload_length += http2_write_psuedo_headers(resp_header, payload_buffer, buf_len, *(this->remote_dynamic_table));
+  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
@@ -979,7 +979,7 @@ Http2ConnectionState::send_headers_frame(FetchSM *fetch_sm)
 
     // Encode by HPACK naive
     payload_length += http2_write_header_fragment(resp_header, field_iter, payload_buffer + payload_length,
-                                                  buf_len - payload_length, *(this->remote_dynamic_table), cont);
+                                                  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) {

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/82ce7567/proxy/http2/Http2ConnectionState.h
----------------------------------------------------------------------
diff --git a/proxy/http2/Http2ConnectionState.h b/proxy/http2/Http2ConnectionState.h
index 200f3d3..44d0fca 100644
--- a/proxy/http2/Http2ConnectionState.h
+++ b/proxy/http2/Http2ConnectionState.h
@@ -112,8 +112,8 @@ public:
   }
 
   Http2ClientSession *ua_session;
-  Http2DynamicTable *local_dynamic_table;
-  Http2DynamicTable *remote_dynamic_table;
+  Http2IndexingTable *local_indexing_table;
+  Http2IndexingTable *remote_indexing_table;
 
   // Settings.
   Http2ConnectionSettings server_settings;
@@ -122,8 +122,8 @@ public:
   void
   init()
   {
-    local_dynamic_table = new Http2DynamicTable();
-    remote_dynamic_table = new Http2DynamicTable();
+    local_indexing_table = new Http2IndexingTable();
+    remote_indexing_table = new Http2IndexingTable();
 
     continued_buffer.iov_base = NULL;
     continued_buffer.iov_len = 0;
@@ -135,8 +135,8 @@ public:
     cleanup_streams();
 
     mutex = NULL; // magic happens - assigning to NULL frees the ProxyMutex
-    delete local_dynamic_table;
-    delete remote_dynamic_table;
+    delete local_indexing_table;
+    delete remote_indexing_table;
 
     ats_free(continued_buffer.iov_base);
   }

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/82ce7567/proxy/http2/Http2Stream.h
----------------------------------------------------------------------
diff --git a/proxy/http2/Http2Stream.h b/proxy/http2/Http2Stream.h
index c96c434..31f1d75 100644
--- a/proxy/http2/Http2Stream.h
+++ b/proxy/http2/Http2Stream.h
@@ -92,10 +92,10 @@ public:
   bool change_state(uint8_t type, uint8_t flags);
 
   int64_t
-  decode_header_blocks(Http2DynamicTable &dynamic_table)
+  decode_header_blocks(Http2IndexingTable &indexing_table)
   {
     return http2_decode_header_blocks(&_req_header, (const uint8_t *)header_blocks,
-                                      (const uint8_t *)header_blocks + header_blocks_length, dynamic_table);
+                                      (const uint8_t *)header_blocks + header_blocks_length, indexing_table);
   }
 
   // Check entire DATA payload length if content-length: header is exist

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/82ce7567/proxy/http2/RegressionHPACK.cc
----------------------------------------------------------------------
diff --git a/proxy/http2/RegressionHPACK.cc b/proxy/http2/RegressionHPACK.cc
index 4756aea..aee2f3c 100644
--- a/proxy/http2/RegressionHPACK.cc
+++ b/proxy/http2/RegressionHPACK.cc
@@ -27,6 +27,7 @@
 #include "ts/TestBox.h"
 
 // Constants for regression test
+const static int DYNAMIC_TABLE_SIZE_FOR_REGRESSION_TEST = 256;
 const static int BUFSIZE_FOR_REGRESSION_TEST = 128;
 const static int MAX_TEST_FIELD_NUM = 8;
 
@@ -161,49 +162,136 @@ const static struct {
 const static struct {
   char *raw_name;
   char *raw_value;
-} raw_field_test_case[][MAX_TEST_FIELD_NUM] = {{
-                                                 {(char *)":method", (char *) "GET"},
-                                                 {(char *)":scheme", (char *) "http"},
-                                                 {(char *)":path", (char *) "/"},
-                                                 {(char *)":authority", (char *) "www.example.com"},
-                                                 {(char *)"", (char *) ""} // End of this test case
-                                               },
-                                               {
-                                                 {(char *)":method", (char *) "GET"},
-                                                 {(char *)":scheme", (char *) "http"},
-                                                 {(char *)":path", (char *) "/"},
-                                                 {(char *)":authority", (char *) "www.example.com"},
-                                                 {(char *)"", (char *) ""} // End of this test case
-                                               }};
+} raw_field_request_test_case[][MAX_TEST_FIELD_NUM] = {{
+                                                         {(char *)":method", (char *) "GET"},
+                                                         {(char *)":scheme", (char *) "http"},
+                                                         {(char *)":path", (char *) "/"},
+                                                         {(char *)":authority", (char *) "www.example.com"},
+                                                         {(char *)"", (char *) ""} // End of this test case
+                                                       },
+                                                       {
+                                                         {(char *)":method", (char *) "GET"},
+                                                         {(char *)":scheme", (char *) "http"},
+                                                         {(char *)":path", (char *) "/"},
+                                                         {(char *)":authority", (char *) "www.example.com"},
+                                                         {(char *)"", (char *) ""} // End of this test case
+                                                       }};
 const static struct {
   uint8_t *encoded_field;
   int encoded_field_len;
-} encoded_field_test_case[] = {{(uint8_t *)"\x40"
-                                           "\x7:method"
-                                           "\x3GET"
-                                           "\x40"
-                                           "\x7:scheme"
-                                           "\x4http"
-                                           "\x40"
-                                           "\x5:path"
-                                           "\x1/"
-                                           "\x40"
-                                           "\xa:authority"
-                                           "\xfwww.example.com",
-                                64},
-                               {(uint8_t *)"\x40"
-                                           "\x85\xb9\x49\x53\x39\xe4"
-                                           "\x83\xc5\x83\x7f"
-                                           "\x40"
-                                           "\x85\xb8\x82\x4e\x5a\x4b"
-                                           "\x83\x9d\x29\xaf"
-                                           "\x40"
-                                           "\x84\xb9\x58\xd3\x3f"
-                                           "\x81\x63"
-                                           "\x40"
-                                           "\x88\xb8\x3b\x53\x39\xec\x32\x7d\x7f"
-                                           "\x8c\xf1\xe3\xc2\xe5\xf2\x3a\x6b\xa0\xab\x90\xf4\xff",
-                                53}};
+} encoded_field_request_test_case[] = {{(uint8_t *)"\x40"
+                                                   "\x7:method"
+                                                   "\x3GET"
+                                                   "\x40"
+                                                   "\x7:scheme"
+                                                   "\x4http"
+                                                   "\x40"
+                                                   "\x5:path"
+                                                   "\x1/"
+                                                   "\x40"
+                                                   "\xa:authority"
+                                                   "\xfwww.example.com",
+                                        64},
+                                       {(uint8_t *)"\x40"
+                                                   "\x85\xb9\x49\x53\x39\xe4"
+                                                   "\x83\xc5\x83\x7f"
+                                                   "\x40"
+                                                   "\x85\xb8\x82\x4e\x5a\x4b"
+                                                   "\x83\x9d\x29\xaf"
+                                                   "\x40"
+                                                   "\x84\xb9\x58\xd3\x3f"
+                                                   "\x81\x63"
+                                                   "\x40"
+                                                   "\x88\xb8\x3b\x53\x39\xec\x32\x7d\x7f"
+                                                   "\x8c\xf1\xe3\xc2\xe5\xf2\x3a\x6b\xa0\xab\x90\xf4\xff",
+                                        53}};
+
+// [RFC 7541] C.6. Response Examples with Huffman Coding
+const static struct {
+  char *raw_name;
+  char *raw_value;
+} raw_field_response_test_case[][MAX_TEST_FIELD_NUM] = {
+  {
+    {(char *)":status", (char *) "302"},
+    {(char *)"cache-control", (char *) "private"},
+    {(char *)"date", (char *) "Mon, 21 Oct 2013 20:13:21 GMT"},
+    {(char *)"location", (char *) "https://www.example.com"},
+    {(char *)"", (char *) ""} // End of this test case
+  },
+  {
+    {(char *)":status", (char *) "307"},
+    {(char *)"cache-control", (char *) "private"},
+    {(char *)"date", (char *) "Mon, 21 Oct 2013 20:13:21 GMT"},
+    {(char *)"location", (char *) "https://www.example.com"},
+    {(char *)"", (char *) ""} // End of this test case
+  },
+  {
+    {(char *)":status", (char *) "200"},
+    {(char *)"cache-control", (char *) "private"},
+    {(char *)"date", (char *) "Mon, 21 Oct 2013 20:13:22 GMT"},
+    {(char *)"location", (char *) "https://www.example.com"},
+    {(char *)"content-encoding", (char *) "gzip"},
+    {(char *)"set-cookie", (char *) "foo=ASDJKHQKBZXOQWEOPIUAXQWEOIU; max-age=3600; version=1"},
+    {(char *)"", (char *) ""} // End of this test case
+  }};
+const static struct {
+  uint8_t *encoded_field;
+  int encoded_field_len;
+} encoded_field_response_test_case[] = {{(uint8_t *)"\x48\x82"
+                                                    "\x64\x02"
+                                                    "\x58\x85"
+                                                    "\xae\xc3\x77\x1a\x4b"
+                                                    "\x61\x96"
+                                                    "\xd0\x7a\xbe\x94\x10\x54\xd4\x44\xa8\x20\x05\x95\x04\x0b\x81\x66"
+                                                    "\xe0\x82\xa6\x2d\x1b\xff"
+                                                    "\x6e\x91"
+                                                    "\x9d\x29\xad\x17\x18\x63\xc7\x8f\x0b\x97\xc8\xe9\xae\x82\xae\x43"
+                                                    "\xd3",
+                                         54},
+                                        {(uint8_t *)"\x48\x83"
+                                                    "\x64\x0e\xff"
+                                                    "\xc1"
+                                                    "\xc0"
+                                                    "\xbf",
+                                         8},
+                                        {(uint8_t *)"\x88"
+                                                    "\xc1"
+                                                    "\x61\x96"
+                                                    "\xd0\x7a\xbe\x94\x10\x54\xd4\x44\xa8\x20\x05\x95\x04\x0b\x81\x66"
+                                                    "\xe0\x84\xa6\x2d\x1b\xff"
+                                                    "\xc0"
+                                                    "\x5a\x83"
+                                                    "\x9b\xd9\xab"
+                                                    "\x77\xad"
+                                                    "\x94\xe7\x82\x1d\xd7\xf2\xe6\xc7\xb3\x35\xdf\xdf\xcd\x5b\x39\x60"
+                                                    "\xd5\xaf\x27\x08\x7f\x36\x72\xc1\xab\x27\x0f\xb5\x29\x1f\x95\x87"
+                                                    "\x31\x60\x65\xc0\x03\xed\x4e\xe5\xb1\x06\x3d\x50\x07",
+                                         79}};
+const static struct {
+  uint32_t size;
+  char *name;
+  char *value;
+} dynamic_table_response_test_case[][MAX_TEST_FIELD_NUM] = {
+  {
+    {63, (char *) "location", (char *) "https://www.example.com"},
+    {65, (char *) "date", (char *) "Mon, 21 Oct 2013 20:13:21 GMT"},
+    {52, (char *) "cache-control", (char *) "private"},
+    {42, (char *) ":status", (char *) "302"},
+    {0, (char *) "", (char *) ""} // End of this test case
+  },
+  {
+    {42, (char *) ":status", (char *) "307"},
+    {63, (char *) "location", (char *) "https://www.example.com"},
+    {65, (char *) "date", (char *) "Mon, 21 Oct 2013 20:13:21 GMT"},
+    {52, (char *) "cache-control", (char *) "private"},
+    {0, (char *) "", (char *) ""} // End of this test case
+  },
+  {
+    {98, (char *) "set-cookie", (char *) "foo=ASDJKHQKBZXOQWEOPIUAXQWEOIU; max-age=3600; version=1"},
+    {52, (char *) "content-encoding", (char *) "gzip"},
+    {65, (char *) "date", (char *) "Mon, 21 Oct 2013 20:13:22 GMT"},
+    {0, (char *) "", (char *) ""} // End of this test case
+  }};
 
 
 /***********************************************************************************
@@ -274,6 +362,7 @@ REGRESSION_TEST(HPACK_EncodeLiteralHeaderField)(RegressionTest *t, int, int *pst
 
   uint8_t buf[BUFSIZE_FOR_REGRESSION_TEST];
   int len;
+  Http2IndexingTable indexing_table;
 
   for (unsigned int i = 9; i < sizeof(literal_test_case) / sizeof(literal_test_case[0]); i++) {
     memset(buf, 0, BUFSIZE_FOR_REGRESSION_TEST);
@@ -283,13 +372,15 @@ REGRESSION_TEST(HPACK_EncodeLiteralHeaderField)(RegressionTest *t, int, int *pst
     MIMEField *field = mime_field_create(headers->m_heap, headers->m_http->m_fields_impl);
     MIMEFieldWrapper header(field, headers->m_heap, headers->m_http->m_fields_impl);
 
+    header.name_set(literal_test_case[i].raw_name, strlen(literal_test_case[i].raw_name));
     header.value_set(literal_test_case[i].raw_value, strlen(literal_test_case[i].raw_value));
     if (literal_test_case[i].index > 0) {
-      len = encode_literal_header_field(buf, buf + BUFSIZE_FOR_REGRESSION_TEST, header, literal_test_case[i].index,
-                                        literal_test_case[i].type);
+      len = encode_literal_header_field_with_indexed_name(buf, buf + BUFSIZE_FOR_REGRESSION_TEST, header,
+                                                          literal_test_case[i].index, indexing_table, literal_test_case[i].type);
     } else {
       header.name_set(literal_test_case[i].raw_name, strlen(literal_test_case[i].raw_name));
-      len = encode_literal_header_field(buf, buf + BUFSIZE_FOR_REGRESSION_TEST, header, literal_test_case[i].type);
+      len = encode_literal_header_field_with_new_name(buf, buf + BUFSIZE_FOR_REGRESSION_TEST, header, indexing_table,
+                                                      literal_test_case[i].type);
     }
 
     box.check(len == literal_test_case[i].encoded_field_len, "encoded length was %d, expecting %d", len,
@@ -304,37 +395,58 @@ REGRESSION_TEST(HPACK_Encode)(RegressionTest *t, int, int *pstatus)
   box = REGRESSION_TEST_PASSED;
 
   uint8_t buf[BUFSIZE_FOR_REGRESSION_TEST];
-  Http2DynamicTable dynamic_table;
+  Http2IndexingTable indexing_table;
+  indexing_table.set_dynamic_table_size(DYNAMIC_TABLE_SIZE_FOR_REGRESSION_TEST);
 
-  // FIXME Current encoder don't support indexing.
-  for (unsigned int i = 1; i < sizeof(encoded_field_test_case) / sizeof(encoded_field_test_case[0]); i++) {
+  for (unsigned int i = 0; i < sizeof(encoded_field_response_test_case) / sizeof(encoded_field_response_test_case[0]); i++) {
     ats_scoped_obj<HTTPHdr> headers(new HTTPHdr);
-    headers->create(HTTP_TYPE_REQUEST);
+    headers->create(HTTP_TYPE_RESPONSE);
 
-    for (unsigned int j = 0; j < sizeof(raw_field_test_case[i]) / sizeof(raw_field_test_case[i][0]); j++) {
-      const char *expected_name = raw_field_test_case[i][j].raw_name;
-      const char *expected_value = raw_field_test_case[i][j].raw_value;
+    for (unsigned int j = 0; j < sizeof(raw_field_response_test_case[i]) / sizeof(raw_field_response_test_case[i][0]); j++) {
+      const char *expected_name = raw_field_response_test_case[i][j].raw_name;
+      const char *expected_value = raw_field_response_test_case[i][j].raw_value;
       if (strlen(expected_name) == 0)
         break;
 
-      MIMEField *field = mime_field_create(headers->m_heap, headers->m_http->m_fields_impl);
-      mime_field_name_value_set(headers->m_heap, headers->m_http->m_fields_impl, field, -1, expected_name, strlen(expected_name),
-                                expected_value, strlen(expected_value), 1, strlen(expected_name) + strlen(expected_value), true);
-      mime_hdr_field_attach(headers->m_http->m_fields_impl, field, 1, NULL);
+      if (strlen(expected_name) == HPACK_LEN_STATUS && strncasecmp(expected_name, HPACK_VALUE_STATUS, HPACK_LEN_STATUS) == 0) {
+        headers->status_set(http_parse_status(expected_value, expected_value + strlen(expected_value)));
+      } else {
+        MIMEField *field = mime_field_create(headers->m_heap, headers->m_http->m_fields_impl);
+        field->name_set(headers->m_heap, headers->m_http->m_fields_impl, expected_name, strlen(expected_name));
+        field->value_set(headers->m_heap, headers->m_http->m_fields_impl, expected_value, strlen(expected_value));
+        mime_hdr_field_attach(headers->m_http->m_fields_impl, field, 1, NULL);
+      }
     }
 
     memset(buf, 0, BUFSIZE_FOR_REGRESSION_TEST);
     uint64_t buf_len = BUFSIZE_FOR_REGRESSION_TEST;
-    int64_t len = http2_write_psuedo_headers(headers, buf, buf_len, dynamic_table);
+    int64_t len = http2_write_psuedo_headers(headers, buf, buf_len, indexing_table);
     buf_len -= len;
 
     MIMEFieldIter field_iter;
     bool cont = false;
-    len += http2_write_header_fragment(headers, field_iter, buf, buf_len, dynamic_table, cont);
+    len += http2_write_header_fragment(headers, field_iter, buf + len, buf_len, indexing_table, cont);
+
+    box.check(len == encoded_field_response_test_case[i].encoded_field_len, "encoded length was %" PRId64 ", expecting %d", len,
+              encoded_field_response_test_case[i].encoded_field_len);
+    box.check(len > 0 && memcmp(buf, encoded_field_response_test_case[i].encoded_field, len) == 0, "encoded value was invalid");
+
+    // Check dynamic table
+    uint32_t expected_dynamic_table_size = 0;
+    for (unsigned int j = 0; j < sizeof(dynamic_table_response_test_case[i]) / sizeof(dynamic_table_response_test_case[i][0]);
+         j++) {
+      const char *expected_name = dynamic_table_response_test_case[i][j].name;
+      const char *expected_value = dynamic_table_response_test_case[i][j].value;
+
+      if (strlen(expected_name) == 0)
+        break;
 
-    box.check(len == encoded_field_test_case[i].encoded_field_len, "encoded length was %" PRId64 ", expecting %d", len,
-              encoded_field_test_case[i].encoded_field_len);
-    box.check(len > 0 && memcmp(buf, encoded_field_test_case[i].encoded_field, len) == 0, "encoded value was invalid");
+      box.check(indexing_table.is_header_in_dynamic_table(expected_name, expected_value), "dynamic table has unexpected entries");
+
+      expected_dynamic_table_size += dynamic_table_response_test_case[i][j].size;
+    }
+    box.check(indexing_table.get_dynamic_table_size() == expected_dynamic_table_size, "dynamic table is unexpected size: %d",
+              indexing_table.get_dynamic_table_size());
   }
 }
 
@@ -385,7 +497,7 @@ REGRESSION_TEST(HPACK_DecodeIndexedHeaderField)(RegressionTest *t, int, int *pst
   TestBox box(t, pstatus);
   box = REGRESSION_TEST_PASSED;
 
-  Http2DynamicTable dynamic_table;
+  Http2IndexingTable indexing_table;
 
   for (unsigned int i = 0; i < sizeof(indexed_test_case) / sizeof(indexed_test_case[0]); i++) {
     ats_scoped_obj<HTTPHdr> headers(new HTTPHdr);
@@ -395,7 +507,7 @@ REGRESSION_TEST(HPACK_DecodeIndexedHeaderField)(RegressionTest *t, int, int *pst
 
     int len =
       decode_indexed_header_field(header, indexed_test_case[i].encoded_field,
-                                  indexed_test_case[i].encoded_field + indexed_test_case[i].encoded_field_len, dynamic_table);
+                                  indexed_test_case[i].encoded_field + indexed_test_case[i].encoded_field_len, indexing_table);
 
     box.check(len == indexed_test_case[i].encoded_field_len, "decoded length was %d, expecting %d", len,
               indexed_test_case[i].encoded_field_len);
@@ -415,7 +527,7 @@ REGRESSION_TEST(HPACK_DecodeLiteralHeaderField)(RegressionTest *t, int, int *pst
   TestBox box(t, pstatus);
   box = REGRESSION_TEST_PASSED;
 
-  Http2DynamicTable dynamic_table;
+  Http2IndexingTable indexing_table;
 
   for (unsigned int i = 0; i < sizeof(literal_test_case) / sizeof(literal_test_case[0]); i++) {
     ats_scoped_obj<HTTPHdr> headers(new HTTPHdr);
@@ -425,7 +537,7 @@ REGRESSION_TEST(HPACK_DecodeLiteralHeaderField)(RegressionTest *t, int, int *pst
 
     int len =
       decode_literal_header_field(header, literal_test_case[i].encoded_field,
-                                  literal_test_case[i].encoded_field + literal_test_case[i].encoded_field_len, dynamic_table);
+                                  literal_test_case[i].encoded_field + literal_test_case[i].encoded_field_len, indexing_table);
 
     box.check(len == literal_test_case[i].encoded_field_len, "decoded length was %d, expecting %d", len,
               literal_test_case[i].encoded_field_len);
@@ -446,19 +558,19 @@ REGRESSION_TEST(HPACK_Decode)(RegressionTest *t, int, int *pstatus)
   TestBox box(t, pstatus);
   box = REGRESSION_TEST_PASSED;
 
-  Http2DynamicTable dynamic_table;
+  Http2IndexingTable indexing_table;
 
-  for (unsigned int i = 0; i < sizeof(encoded_field_test_case) / sizeof(encoded_field_test_case[0]); i++) {
+  for (unsigned int i = 0; i < sizeof(encoded_field_request_test_case) / sizeof(encoded_field_request_test_case[0]); i++) {
     ats_scoped_obj<HTTPHdr> headers(new HTTPHdr);
     headers->create(HTTP_TYPE_REQUEST);
 
-    http2_decode_header_blocks(headers, encoded_field_test_case[i].encoded_field,
-                               encoded_field_test_case[i].encoded_field + encoded_field_test_case[i].encoded_field_len,
-                               dynamic_table);
+    http2_decode_header_blocks(
+      headers, encoded_field_request_test_case[i].encoded_field,
+      encoded_field_request_test_case[i].encoded_field + encoded_field_request_test_case[i].encoded_field_len, indexing_table);
 
-    for (unsigned int j = 0; j < sizeof(raw_field_test_case[i]) / sizeof(raw_field_test_case[i][0]); j++) {
-      const char *expected_name = raw_field_test_case[i][j].raw_name;
-      const char *expected_value = raw_field_test_case[i][j].raw_value;
+    for (unsigned int j = 0; j < sizeof(raw_field_request_test_case[i]) / sizeof(raw_field_request_test_case[i][0]); j++) {
+      const char *expected_name = raw_field_request_test_case[i][j].raw_name;
+      const char *expected_value = raw_field_request_test_case[i][j].raw_value;
       if (strlen(expected_name) == 0)
         break;
 


[2/2] trafficserver git commit: TS-3478: clang-format

Posted by ma...@apache.org.
TS-3478: clang-format


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

Branch: refs/heads/master
Commit: 770e7878c0f4dde9516f6f93490eb8b2f66a8dd2
Parents: 82ce756
Author: Masaori Koshiba <ma...@apache.org>
Authored: Sat Jan 9 18:04:59 2016 +0900
Committer: Masaori Koshiba <ma...@apache.org>
Committed: Sat Jan 9 18:04:59 2016 +0900

----------------------------------------------------------------------
 proxy/http2/HPACK.h | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/trafficserver/blob/770e7878/proxy/http2/HPACK.h
----------------------------------------------------------------------
diff --git a/proxy/http2/HPACK.h b/proxy/http2/HPACK.h
index 2cda962..14430da 100644
--- a/proxy/http2/HPACK.h
+++ b/proxy/http2/HPACK.h
@@ -122,7 +122,7 @@ public:
     delete _mhdr;
   }
 
-  const MIMEField * get_header_field(uint32_t index) const;
+  const MIMEField *get_header_field(uint32_t index) const;
   void add_header_field(const MIMEField *field);
 
   uint32_t get_size() const;
@@ -175,7 +175,7 @@ int64_t encode_indexed_header_field(uint8_t *buf_start, const uint8_t *buf_end,
 int64_t encode_literal_header_field_with_indexed_name(uint8_t *buf_start, const uint8_t *buf_end, const MIMEFieldWrapper &header,
                                                       uint32_t index, Http2IndexingTable &indexing_table, HpackFieldType type);
 int64_t encode_literal_header_field_with_new_name(uint8_t *buf_start, const uint8_t *buf_end, const MIMEFieldWrapper &header,
-    Http2IndexingTable &indexing_table, HpackFieldType type);
+                                                  Http2IndexingTable &indexing_table, HpackFieldType type);
 
 // When these functions returns minus value, any error occurs
 // TODO Separate error code and length of processed buffer