You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@trafficserver.apache.org by am...@apache.org on 2014/08/09 18:03:41 UTC

[2/2] git commit: Cherry-pick - TS-2564: Fixup MIME presence bits and slot accelerators to recover from WKS_IDX changes, plus config to turn it off manually.

Cherry-pick - TS-2564: Fixup MIME presence bits and slot accelerators to recover from WKS_IDX changes, plus config to turn it off manually.


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

Branch: refs/heads/master
Commit: 8141ceae283a9bca5cb1d6830c11a0acccdd86d6
Parents: d6bcd2d
Author: Alan M. Carroll <am...@network-geographics.com>
Authored: Fri Apr 11 14:35:22 2014 -0600
Committer: Alan M. Carroll <am...@network-geographics.com>
Committed: Sat Aug 9 11:00:26 2014 -0500

----------------------------------------------------------------------
 CHANGES                        |  2 ++
 iocore/cache/Cache.cc          |  5 +++++
 iocore/cache/CacheRead.cc      | 20 +++++++++++++++++--
 iocore/cache/CacheVol.cc       |  2 +-
 iocore/cache/CacheWrite.cc     |  4 ++--
 iocore/cache/P_CacheInternal.h |  4 ++++
 mgmt/RecordsConfig.cc          |  8 ++++++++
 proxy/hdrs/MIME.cc             | 39 ++++++++++++++++++++++++++++++++-----
 proxy/hdrs/MIME.h              |  1 +
 9 files changed, 75 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/trafficserver/blob/8141ceae/CHANGES
----------------------------------------------------------------------
diff --git a/CHANGES b/CHANGES
index 0bc4568..6f6dac7 100644
--- a/CHANGES
+++ b/CHANGES
@@ -1,6 +1,8 @@
                                                          -*- coding: utf-8 -*-
 Changes with Apache Traffic Server 5.1.0
 
+  *) [TS-2564] Cherry pick for full cache compatibility.
+
   *) [TS-2362] Make cache backwards compatible to 3.2.0.
 
   *) [TS-2357] Add option to cache POST requests

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/8141ceae/iocore/cache/Cache.cc
----------------------------------------------------------------------
diff --git a/iocore/cache/Cache.cc b/iocore/cache/Cache.cc
index 52d5797..be3674f 100644
--- a/iocore/cache/Cache.cc
+++ b/iocore/cache/Cache.cc
@@ -81,6 +81,9 @@ int cache_config_read_while_writer = 0;
 int cache_config_mutex_retry_delay = 2;
 #ifdef HTTP_CACHE
 static int enable_cache_empty_http_doc = 0;
+/// Fix up a specific known problem with the 4.2.0 release.
+/// Not used for stripes with a cache version later than 4.2.0.
+int cache_config_compatibility_4_2_0_fixup = 1;
 #endif
 
 #if TS_USE_INTERIM_CACHE == 1
@@ -3520,6 +3523,8 @@ ink_cache_init(ModuleVersion v)
 
 #ifdef HTTP_CACHE
   REC_EstablishStaticConfigInt32(enable_cache_empty_http_doc, "proxy.config.http.cache.allow_empty_doc");
+
+  REC_EstablishStaticConfigInt32(cache_config_compatibility_4_2_0_fixup, "proxy.config.cache.http.compatibility.4-2-0-fixup");
 #endif
 
 #if TS_USE_INTERIM_CACHE == 1

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/8141ceae/iocore/cache/CacheRead.cc
----------------------------------------------------------------------
diff --git a/iocore/cache/CacheRead.cc b/iocore/cache/CacheRead.cc
index 79bd542..fa20f83 100644
--- a/iocore/cache/CacheRead.cc
+++ b/iocore/cache/CacheRead.cc
@@ -28,6 +28,7 @@
 #endif
 
 #define READ_WHILE_WRITER 1
+extern int cache_config_compatibility_4_2_0_fixup;
 
 Action *
 Cache::open_read(Continuation * cont, CacheKey * key, CacheFragType type, char *hostname, int host_len)
@@ -156,6 +157,21 @@ Lcallreturn:
 }
 #endif
 
+uint32_t
+CacheVC::load_http_info(CacheHTTPInfoVector* info, Doc* doc, RefCountObj * block_ptr)
+{
+  uint32_t zret = info->get_handles(doc->hdr(), doc->hlen, block_ptr);
+  if (cache_config_compatibility_4_2_0_fixup &&
+      vol->header->version.ink_major == 23 && vol->header->version.ink_minor == 0
+    ) {
+    for ( int i = info->xcount - 1 ; i >= 0 ; --i ) {
+      info->data(i).alternate.m_alt->m_response_hdr.m_mime->recompute_accelerators_and_presence_bits();
+      info->data(i).alternate.m_alt->m_request_hdr.m_mime->recompute_accelerators_and_presence_bits();
+    }
+  }
+  return zret;
+}
+
 int
 CacheVC::openReadFromWriterFailure(int event, Event * e)
 {
@@ -868,7 +884,7 @@ Lread:
       // don't want any writers while we are evacuating the vector
       if (!vol->open_write(this, false, 1)) {
         Doc *doc1 = (Doc *) first_buf->data();
-        uint32_t len = write_vector->get_handles(doc1->hdr(), doc1->hlen);
+        uint32_t len = this->load_http_info(write_vector, doc1);
         ink_assert(len == doc1->hlen && write_vector->count() > 0);
         write_vector->remove(alternate_index, true);
         // if the vector had one alternate, delete it's directory entry
@@ -1065,7 +1081,7 @@ CacheVC::openReadStartHead(int event, Event * e)
       ink_assert(doc->hlen);
       if (!doc->hlen)
         goto Ldone;
-      if ((uml = vector.get_handles(doc->hdr(), doc->hlen)) != doc->hlen) {
+      if ((uml = this->load_http_info(&vector, doc)) != doc->hlen) {
         if (buf) {
           HTTPCacheAlt* alt = reinterpret_cast<HTTPCacheAlt*>(doc->hdr());
           int32_t alt_length = 0;

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/8141ceae/iocore/cache/CacheVol.cc
----------------------------------------------------------------------
diff --git a/iocore/cache/CacheVol.cc b/iocore/cache/CacheVol.cc
index 72f8463..ccb6964 100644
--- a/iocore/cache/CacheVol.cc
+++ b/iocore/cache/CacheVol.cc
@@ -246,7 +246,7 @@ CacheVC::scanObject(int /* event ATS_UNUSED */, Event * /* e ATS_UNUSED */)
         tmp += r;
       }
     }
-    if (vector.get_handles(doc->hdr(), doc->hlen) != doc->hlen)
+    if (this->load_http_info(&vector, doc) != doc->hlen)
       goto Lskip;
     changed = false;
     hostinfo_copied = 0;

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/8141ceae/iocore/cache/CacheWrite.cc
----------------------------------------------------------------------
diff --git a/iocore/cache/CacheWrite.cc b/iocore/cache/CacheWrite.cc
index 2487938..287b5e8 100644
--- a/iocore/cache/CacheWrite.cc
+++ b/iocore/cache/CacheWrite.cc
@@ -402,7 +402,7 @@ CacheVC::evacuateReadHead(int /* event ATS_UNUSED */, Event * /* e ATS_UNUSED */
   alternate_tmp = 0;
   if (doc->doc_type == CACHE_FRAG_TYPE_HTTP && doc->hlen) {
     // its an http document
-    if (vector.get_handles(doc->hdr(), doc->hlen) != doc->hlen) {
+    if (this->load_http_info(&vector, doc) != doc->hlen) {
       Note("bad vector detected during evacuation");
       goto Ldone;
     }
@@ -1500,7 +1500,7 @@ CacheVC::openWriteStartDone(int event, Event *e)
         goto Lcollision;
 
       if (doc->magic != DOC_MAGIC || !doc->hlen ||
-          write_vector->get_handles(doc->hdr(), doc->hlen, buf) != doc->hlen) {
+          this->load_http_info(write_vector, doc, buf) != doc->hlen) {
         err = ECACHE_BAD_META_DATA;
 #if TS_USE_INTERIM_CACHE == 1
         if (dir_ininterim(&dir)) {

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/8141ceae/iocore/cache/P_CacheInternal.h
----------------------------------------------------------------------
diff --git a/iocore/cache/P_CacheInternal.h b/iocore/cache/P_CacheInternal.h
index 5afac60..4b840f0 100644
--- a/iocore/cache/P_CacheInternal.h
+++ b/iocore/cache/P_CacheInternal.h
@@ -361,6 +361,10 @@ struct CacheVC: public CacheVConnection
       or @c NULL if there is no fragment table.
   */
   virtual HTTPInfo::FragOffset* get_frag_table();
+  /** Load alt pointers and do fixups if needed.
+      @return Length of header data used for alternates.
+   */
+  virtual uint32_t load_http_info(CacheHTTPInfoVector* info, struct Doc* doc, RefCountObj * block_ptr = NULL);
 #endif
   virtual bool is_pread_capable();
   virtual bool set_pin_in_cache(time_t time_pin);

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/8141ceae/mgmt/RecordsConfig.cc
----------------------------------------------------------------------
diff --git a/mgmt/RecordsConfig.cc b/mgmt/RecordsConfig.cc
index 3e96b6b..1babc8b 100644
--- a/mgmt/RecordsConfig.cc
+++ b/mgmt/RecordsConfig.cc
@@ -2039,6 +2039,14 @@ RecordElement RecordsConfig[] = {
   {RECT_CONFIG, "proxy.config.remap.num_remap_threads", RECD_INT, "0", RECU_NULL, RR_NULL, RECC_NULL, NULL, RECA_NULL}
   ,
 
+  //###########
+  //#
+  //# Temporary and esoteric values.
+  //#
+  //###########
+  {RECT_CONFIG, "proxy.config.cache.http.compatibility.4-2-0-fixup", RECD_INT, "1", RECU_DYNAMIC, RR_NULL, RECC_NULL, NULL, RECA_NULL}
+  ,
+
   //##############################################################################
   //#
   //# The End

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/8141ceae/proxy/hdrs/MIME.cc
----------------------------------------------------------------------
diff --git a/proxy/hdrs/MIME.cc b/proxy/hdrs/MIME.cc
index 18976a2..2cb3122 100644
--- a/proxy/hdrs/MIME.cc
+++ b/proxy/hdrs/MIME.cc
@@ -433,6 +433,16 @@ mime_hdr_presence_unset(MIMEHdrImpl *h, int well_known_str_index)
  *                  S L O T    A C C E L E R A T O R S                 *
  *                                                                     *
  ***********************************************************************/
+inline void
+mime_hdr_init_accelerators_and_presence_bits(MIMEHdrImpl* mh)
+{
+  mh->m_presence_bits = 0;
+  mh->m_slot_accelerators[0] = 0xFFFFFFFF;
+  mh->m_slot_accelerators[1] = 0xFFFFFFFF;
+  mh->m_slot_accelerators[2] = 0xFFFFFFFF;
+  mh->m_slot_accelerators[3] = 0xFFFFFFFF;
+}
+
 inline uint32_t
 mime_hdr_get_accelerator_slotnum(MIMEHdrImpl *mh, int32_t slot_id)
 {
@@ -495,6 +505,24 @@ mime_hdr_unset_accelerators_and_presence_bits(MIMEHdrImpl *mh, MIMEField *field)
     mime_hdr_set_accelerator_slotnum(mh, slot_id, MIME_FIELD_SLOTNUM_MAX);
 }
 
+/// Reset data in the header.
+/// Clear all the presence bits and accelerators.
+/// Update all the m_wks_idx values, presence bits and accelerators.
+inline void
+mime_hdr_reset_accelerators_and_presence_bits(MIMEHdrImpl* mh) {
+  mime_hdr_init_accelerators_and_presence_bits(mh);
+
+  for (MIMEFieldBlockImpl* fblock = &(mh->m_first_fblock); fblock != NULL; fblock = fblock->m_next) {
+    for ( MIMEField *field = fblock->m_field_slots, *limit = field + fblock->m_freetop ; field < limit ; ++field) {
+      if (field->is_live()) {
+        field->m_wks_idx = hdrtoken_tokenize(field->m_ptr_name, field->m_len_name);
+        if (field->is_dup_head())
+          mime_hdr_set_accelerators_and_presence_bits(mh, field);
+      }
+    }
+  }
+}
+
 int
 checksum_block(const char *s, int len)
 {
@@ -995,11 +1023,7 @@ mime_hdr_cooked_stuff_init(MIMEHdrImpl *mh, MIMEField *changing_field_or_null)
 void
 mime_hdr_init(MIMEHdrImpl *mh)
 {
-  mh->m_presence_bits = 0;
-  mh->m_slot_accelerators[0] = 0xFFFFFFFF;
-  mh->m_slot_accelerators[1] = 0xFFFFFFFF;
-  mh->m_slot_accelerators[2] = 0xFFFFFFFF;
-  mh->m_slot_accelerators[3] = 0xFFFFFFFF;
+  mime_hdr_init_accelerators_and_presence_bits(mh);
 
   mime_hdr_cooked_stuff_init(mh, NULL);
 
@@ -3654,6 +3678,11 @@ MIMEHdrImpl::check_strings(HeapCheck *heaps, int num_heaps)
   m_first_fblock.check_strings(heaps, num_heaps);
 }
 
+void
+MIMEHdrImpl::recompute_accelerators_and_presence_bits() {
+  mime_hdr_reset_accelerators_and_presence_bits(this);
+}
+
 
 /***********************************************************************
  *                                                                     *

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/8141ceae/proxy/hdrs/MIME.h
----------------------------------------------------------------------
diff --git a/proxy/hdrs/MIME.h b/proxy/hdrs/MIME.h
index 74a63c3..6177cf3 100644
--- a/proxy/hdrs/MIME.h
+++ b/proxy/hdrs/MIME.h
@@ -250,6 +250,7 @@ struct MIMEHdrImpl:public HdrHeapObjImpl
 
   // Cooked values
   void recompute_cooked_stuff(MIMEField * changing_field_or_null = NULL);
+  void recompute_accelerators_and_presence_bits();
 };
 
 /***********************************************************************