You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@trafficserver.apache.org by zw...@apache.org on 2019/11/08 22:48:17 UTC

[trafficserver] branch 9.0.x updated: Remove Cache v23 support

This is an automated email from the ASF dual-hosted git repository.

zwoop pushed a commit to branch 9.0.x
in repository https://gitbox.apache.org/repos/asf/trafficserver.git


The following commit(s) were added to refs/heads/9.0.x by this push:
     new aec8f5d  Remove Cache v23 support
aec8f5d is described below

commit aec8f5d496acc8b40756d962ebec3570a161ca50
Author: scw00 <sc...@apache.org>
AuthorDate: Mon Apr 29 06:29:18 2019 +0000

    Remove Cache v23 support
    
    (cherry picked from commit d000f0cc0cb30d40755da408962a81c2745fbdf5)
---
 iocore/cache/Cache.cc      | 230 ++-------------------------------------------
 iocore/cache/I_CacheDefs.h |  10 +-
 iocore/cache/P_CacheBC.h   | 137 ---------------------------
 3 files changed, 11 insertions(+), 366 deletions(-)

diff --git a/iocore/cache/Cache.cc b/iocore/cache/Cache.cc
index 83b33b5..35291d2 100644
--- a/iocore/cache/Cache.cc
+++ b/iocore/cache/Cache.cc
@@ -34,7 +34,6 @@
 #include "HttpSM.h"
 #include "HttpCacheSM.h"
 #include "InkAPIInternal.h"
-#include "P_CacheBC.h"
 
 #include "tscore/hugepages.h"
 
@@ -2152,92 +2151,6 @@ unmarshal_helper(Doc *doc, Ptr<IOBufferData> &buf, int &okay)
   }
 }
 
-/** Upgrade a marshalled fragment buffer to the current version.
-
-    @internal I looked at doing this in place (rather than a copy & modify) but
-    - The in place logic would be even worse than this mess
-    - It wouldn't save you that much, since you end up doing inserts early in the buffer.
-      Without extreme care in the logic it could end up doing more copying than
-      the simpler copy & modify.
-
-    @internal This logic presumes the existence of some slack at the end of the buffer, which
-    is usually the case (because lots of rounding is done). If there isn't enough slack then
-    this fails and we have a cache miss. The assumption that this is sufficiently rare that
-    code simplicity takes precedence should be checked at some point.
- */
-static bool
-upgrade_doc_version(Ptr<IOBufferData> &buf)
-{
-  // Type definition is close enough to use for initial checking.
-  cache_bc::Doc_v23 *doc = reinterpret_cast<cache_bc::Doc_v23 *>(buf->data());
-  bool zret              = true;
-
-  if (DOC_MAGIC == doc->magic) {
-    if (0 == doc->hlen) {
-      Debug("cache_bc", "Doc %p without header, no upgrade needed.", doc);
-    } else if (CACHE_FRAG_TYPE_HTTP_V23 == doc->doc_type) {
-      cache_bc::HTTPCacheAlt_v21 *alt = reinterpret_cast<cache_bc::HTTPCacheAlt_v21 *>(doc->hdr());
-      if (alt && alt->is_unmarshalled_format()) {
-        Ptr<IOBufferData> d_buf(ioDataAllocator.alloc());
-        Doc *d_doc;
-        char *src;
-        char *dst;
-        char *hdr_limit = doc->data();
-        HTTPInfo::FragOffset *frags =
-          reinterpret_cast<HTTPInfo::FragOffset *>(static_cast<char *>(buf->data()) + sizeof(cache_bc::Doc_v23));
-        int frag_count      = doc->_flen / sizeof(HTTPInfo::FragOffset);
-        size_t n            = 0;
-        size_t content_size = doc->data_len();
-
-        Debug("cache_bc", "Doc %p is 3.2", doc);
-
-        // Use the same buffer size, fail if no fit.
-        d_buf->alloc(buf->_size_index, buf->_mem_type); // Duplicate.
-        d_doc = reinterpret_cast<Doc *>(d_buf->data());
-        n     = d_buf->block_size();
-
-        src = buf->data();
-        dst = d_buf->data();
-        memcpy(dst, src, sizeof(Doc));
-        src += sizeof(Doc) + doc->_flen;
-        dst += sizeof(Doc);
-        n -= sizeof(Doc);
-
-        // We copy the fragment table iff there is a fragment table and there is only one alternate.
-        if (frag_count > 0 && cache_bc::HTTPInfo_v21::marshalled_length(src) > doc->hlen) {
-          frag_count = 0; // inhibit fragment table insertion.
-        }
-
-        while (zret && src < hdr_limit) {
-          zret = cache_bc::HTTPInfo_v21::copy_and_upgrade_unmarshalled_to_v23(dst, src, n, frag_count, frags);
-        }
-        if (zret && content_size <= n) {
-          memcpy(dst, src, content_size); // content
-          // Must update new Doc::len and Doc::hlen
-          // dst points at the first byte of the content, or one past the last byte of the alt header.
-          d_doc->len  = (dst - reinterpret_cast<char *>(d_doc)) + content_size;
-          d_doc->hlen = (dst - reinterpret_cast<char *>(d_doc)) - sizeof(Doc);
-          buf         = d_buf; // replace original buffer with new buffer.
-        } else {
-          zret = false;
-        }
-      }
-      Doc *n_doc = reinterpret_cast<Doc *>(buf->data()); // access as current version.
-      // For now the base header size is the same. If that changes we'll need to handle the v22/23 case here
-      // as with the v21 and shift the content down to accommodate the bigger header.
-      ink_assert(sizeof(*n_doc) == sizeof(*doc));
-
-      n_doc->doc_type = CACHE_FRAG_TYPE_HTTP; // We converted so adjust doc_type.
-      // Set these to zero for debugging - they'll be updated to the current values if/when this is
-      // put in the aggregation buffer.
-      n_doc->v_major = 0;
-      n_doc->v_minor = 0;
-      n_doc->unused  = 0; // force to zero to make future use easier.
-    }
-  }
-  return zret;
-}
-
 // [amc] I think this is where all disk reads from cache funnel through here.
 int
 CacheVC::handleReadDone(int event, Event *e)
@@ -2270,37 +2183,12 @@ CacheVC::handleReadDone(int event, Event *e)
     ink_assert(vol->mutex->nthread_holding < 1000);
     ink_assert(doc->magic == DOC_MAGIC);
 
-    /* We've read the raw data from disk, time to deserialize it. We have to account for a variety of formats that
-       may be present.
-
-       As of cache version 24 we changed the @a doc_type to indicate a format change in the header which includes
-       version data inside the header. Prior to that we must use heuristics to deduce the actual format. For this reason
-       we send that header type off for special processing. Otherwise we can use the in object version to determine
-       the format.
-
-       All of this processing must produce a serialized header that is compliant with the current version. This includes
-       updating the doc_type.
-    */
-
-    if (doc->doc_type == CACHE_FRAG_TYPE_HTTP_V23) {
-      if (upgrade_doc_version(buf)) {
-        doc = reinterpret_cast<Doc *>(buf->data()); // buf may be a new copy
-      } else {
-        Debug("cache_bc", "Upgrade of fragment failed - disk %s - doc id = %" PRIx64 ":%" PRIx64 "", vol->hash_text.get(),
-              read_key->slice64(0), read_key->slice64(1));
-        doc->magic = DOC_CORRUPT;
-        // Should really trash the directory entry for this, as it's never going to work in the future.
-        // Or does that happen later anyway?
-        goto Ldone;
-      }
-    } else if (doc->doc_type == CACHE_FRAG_TYPE_HTTP) { // handle any version updates based on the object version
-      if (ts::VersionNumber(doc->v_major, doc->v_minor) > CACHE_DB_VERSION) {
-        // future version, count as corrupted
-        doc->magic = DOC_CORRUPT;
-        Debug("cache_bc", "Object is future version %d:%d - disk %s - doc id = %" PRIx64 ":%" PRIx64 "", doc->v_major, doc->v_minor,
-              vol->hash_text.get(), read_key->slice64(0), read_key->slice64(1));
-        goto Ldone;
-      }
+    if (ts::VersionNumber(doc->v_major, doc->v_minor) > CACHE_DB_VERSION) {
+      // future version, count as corrupted
+      doc->magic = DOC_CORRUPT;
+      Debug("cache_bc", "Object is future version %d:%d - disk %s - doc id = %" PRIx64 ":%" PRIx64 "", doc->v_major, doc->v_minor,
+            vol->hash_text.get(), read_key->slice64(0), read_key->slice64(1));
+      goto Ldone;
     }
 
 #ifdef VERIFY_JTEST_DATA
@@ -3300,109 +3188,3 @@ CacheProcessor::find_by_path(const char *path, int len)
 
   return nullptr;
 }
-
-// ----------------------------
-
-namespace cache_bc
-{
-static size_t const HTTP_ALT_MARSHAL_SIZE = HdrHeapMarshalBlocks{ts::round_up(sizeof(HTTPCacheAlt))}; // current size.
-size_t
-HTTPInfo_v21::marshalled_length(void *data)
-{
-  size_t zret           = HdrHeapMarshalBlocks{ts::round_up(sizeof(HTTPCacheAlt_v21))};
-  HTTPCacheAlt_v21 *alt = static_cast<HTTPCacheAlt_v21 *>(data);
-  HdrHeap *hdr;
-
-  hdr = reinterpret_cast<HdrHeap *>(reinterpret_cast<char *>(alt) + reinterpret_cast<uintptr_t>(alt->m_request_hdr.m_heap));
-  zret += HdrHeapMarshalBlocks{ts::round_up(hdr->unmarshal_size())};
-  hdr = reinterpret_cast<HdrHeap *>(reinterpret_cast<char *>(alt) + reinterpret_cast<uintptr_t>(alt->m_response_hdr.m_heap));
-  zret += HdrHeapMarshalBlocks{ts::round_up(hdr->unmarshal_size())};
-  return zret;
-}
-
-// Copy an unmarshalled instance from @a src to @a dst.
-// @a src is presumed to be Cache version 21 and the result
-// is Cache version 23. @a length is the buffer available in @a dst.
-// @return @c false if something went wrong (e.g., data overrun).
-bool
-HTTPInfo_v21::copy_and_upgrade_unmarshalled_to_v23(char *&dst, char *&src, size_t &length, int n_frags, FragOffset *frag_offsets)
-{
-  // Offsets of the data after the new stuff.
-  static const size_t OLD_OFFSET = offsetof(HTTPCacheAlt_v21, m_ext_buffer);
-  static const size_t NEW_OFFSET = offsetof(HTTPCacheAlt_v23, m_ext_buffer);
-
-  HTTPCacheAlt_v21 *s_alt = reinterpret_cast<HTTPCacheAlt_v21 *>(src);
-  HTTPCacheAlt_v23 *d_alt = reinterpret_cast<HTTPCacheAlt_v23 *>(dst);
-  HdrHeap_v23 *s_hdr;
-  HdrHeap_v23 *d_hdr;
-  size_t hdr_size;
-
-  if (length < HTTP_ALT_MARSHAL_SIZE) {
-    return false; // Absolutely no hope in this case.
-  }
-
-  memcpy(dst, src, OLD_OFFSET); // initially same data
-  // Now data that's now after extra
-  memcpy(static_cast<char *>(dst) + NEW_OFFSET, static_cast<char *>(src) + OLD_OFFSET, sizeof(HTTPCacheAlt_v21) - OLD_OFFSET);
-  dst += HTTP_ALT_MARSHAL_SIZE; // move past fixed data.
-  length -= HTTP_ALT_MARSHAL_SIZE;
-
-  // Extra data is fragment table - set that if we have it.
-  if (n_frags) {
-    static size_t const IFT_SIZE = HTTPCacheAlt_v23::N_INTEGRAL_FRAG_OFFSETS * sizeof(FragOffset);
-    size_t ift_actual            = std::min(n_frags, HTTPCacheAlt_v23::N_INTEGRAL_FRAG_OFFSETS) * sizeof(FragOffset);
-
-    if (length < (HTTP_ALT_MARSHAL_SIZE + n_frags * sizeof(FragOffset) - IFT_SIZE)) {
-      return false; // can't place fragment table.
-    }
-
-    d_alt->m_frag_offset_count = n_frags;
-    d_alt->m_frag_offsets      = reinterpret_cast<FragOffset *>(dst - reinterpret_cast<char *>(d_alt));
-
-    memcpy(d_alt->m_integral_frag_offsets, frag_offsets, ift_actual);
-    n_frags -= HTTPCacheAlt_v23::N_INTEGRAL_FRAG_OFFSETS;
-    if (n_frags > 0) {
-      size_t k = sizeof(FragOffset) * n_frags;
-      memcpy(dst, frag_offsets + IFT_SIZE, k);
-      dst += k;
-      length -= k;
-    } else if (n_frags < 0) {
-      memset(dst + ift_actual, 0, IFT_SIZE - ift_actual);
-    }
-  } else {
-    d_alt->m_frag_offset_count = 0;
-    d_alt->m_frag_offsets      = nullptr;
-    ink_zero(d_alt->m_integral_frag_offsets);
-  }
-
-  // Copy over the headers, tweaking the swizzled pointers.
-  s_hdr =
-    reinterpret_cast<HdrHeap_v23 *>(reinterpret_cast<char *>(s_alt) + reinterpret_cast<uintptr_t>(s_alt->m_request_hdr.m_heap));
-  d_hdr    = reinterpret_cast<HdrHeap_v23 *>(dst);
-  hdr_size = HdrHeapMarshalBlocks{ts::round_up(s_hdr->unmarshal_size())};
-  if (hdr_size > length) {
-    return false;
-  }
-  memcpy(static_cast<void *>(d_hdr), s_hdr, hdr_size);
-  d_alt->m_request_hdr.m_heap = reinterpret_cast<HdrHeap_v23 *>(reinterpret_cast<char *>(d_hdr) - reinterpret_cast<char *>(d_alt));
-  dst += hdr_size;
-  length -= hdr_size;
-
-  s_hdr =
-    reinterpret_cast<HdrHeap_v23 *>(reinterpret_cast<char *>(s_alt) + reinterpret_cast<uintptr_t>(s_alt->m_response_hdr.m_heap));
-  d_hdr    = reinterpret_cast<HdrHeap_v23 *>(dst);
-  hdr_size = HdrHeapMarshalBlocks{ts::round_up(s_hdr->unmarshal_size())};
-  if (hdr_size > length) {
-    return false;
-  }
-  memcpy(static_cast<void *>(d_hdr), s_hdr, hdr_size);
-  d_alt->m_response_hdr.m_heap = reinterpret_cast<HdrHeap_v23 *>(reinterpret_cast<char *>(d_hdr) - reinterpret_cast<char *>(d_alt));
-  dst += hdr_size;
-  length -= hdr_size;
-
-  src = reinterpret_cast<char *>(s_hdr) + hdr_size;
-
-  return true;
-}
-
-} // namespace cache_bc
diff --git a/iocore/cache/I_CacheDefs.h b/iocore/cache/I_CacheDefs.h
index cdd39bb..b8dfd5c 100644
--- a/iocore/cache/I_CacheDefs.h
+++ b/iocore/cache/I_CacheDefs.h
@@ -113,11 +113,11 @@ enum CacheDataType {
 };
 
 enum CacheFragType {
-  CACHE_FRAG_TYPE_NONE,
-  CACHE_FRAG_TYPE_HTTP_V23, ///< DB version 23 or prior.
-  CACHE_FRAG_TYPE_RTSP,     ///< Should be removed once Cache Toolkit is implemented.
-  CACHE_FRAG_TYPE_HTTP,
-  NUM_CACHE_FRAG_TYPES
+  CACHE_FRAG_TYPE_NONE     = 0,
+  CACHE_FRAG_TYPE_UNUSED_1 = 1, //. Because of the history we need to occupy a space
+  CACHE_FRAG_TYPE_RTSP     = 2, ///< Should be removed once Cache Toolkit is implemented.
+  CACHE_FRAG_TYPE_HTTP     = 3,
+  NUM_CACHE_FRAG_TYPES     = 4
 };
 
 typedef CryptoHash CacheKey;
diff --git a/iocore/cache/P_CacheBC.h b/iocore/cache/P_CacheBC.h
deleted file mode 100644
index ac4d925..0000000
--- a/iocore/cache/P_CacheBC.h
+++ /dev/null
@@ -1,137 +0,0 @@
-/** @file
-
-  Backwards compatibility support for the cache.
-
-  @section license License
-
-  Licensed to the Apache Software Foundation (ASF) under one
-  or more contributor license agreements.  See the NOTICE file
-  distributed with this work for additional information
-  regarding copyright ownership.  The ASF licenses this file
-  to you under the Apache License, Version 2.0 (the
-  "License"); you may not use this file except in compliance
-  with the License.  You may obtain a copy of the License at
-
-      http://www.apache.org/licenses/LICENSE-2.0
-
-  Unless required by applicable law or agreed to in writing, software
-  distributed under the License is distributed on an "AS IS" BASIS,
-  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-  See the License for the specific language governing permissions and
-  limitations under the License.
- */
-
-#pragma once
-
-namespace cache_bc
-{
-/* This looks kind of dumb, but I think it's useful. We import external structure
-   dependencies in to this namespace so we can at least (1) notice them and
-   (2) change them if the current structure changes.
-*/
-
-typedef HTTPHdr HTTPHdr_v21;
-typedef HdrHeap HdrHeap_v23;
-typedef CryptoHash CryptoHash_v23;
-typedef HTTPCacheAlt HTTPCacheAlt_v23;
-
-/** Cache backwards compatibility structure - the fragment table.
-    This is copied from @c HTTPCacheAlt in @c HTTP.h.
-*/
-struct HTTPCacheFragmentTable {
-  /// # of fragment offsets in this alternate.
-  /// @note This is one less than the number of fragments.
-  int m_frag_offset_count;
-  /// Type of offset for a fragment.
-  typedef uint64_t FragOffset;
-  /// Table of fragment offsets.
-  /// @note The offsets are forward looking so that frag[0] is the
-  /// first byte past the end of fragment 0 which is also the first
-  /// byte of fragment 1. For this reason there is no fragment offset
-  /// for the last fragment.
-  FragOffset *m_frag_offsets;
-  /// # of fragment offsets built in to object.
-  static int const N_INTEGRAL_FRAG_OFFSETS = 4;
-  /// Integral fragment offset table.
-  FragOffset m_integral_frag_offsets[N_INTEGRAL_FRAG_OFFSETS];
-};
-
-// From before moving the fragment table to the alternate.
-struct HTTPCacheAlt_v21 {
-  uint32_t m_magic;
-
-  int32_t m_writeable;
-  int32_t m_unmarshal_len;
-
-  int32_t m_id;
-  int32_t m_rid;
-
-  int32_t m_object_key[4];
-  int32_t m_object_size[2];
-
-  HTTPHdr_v21 m_request_hdr;
-  HTTPHdr_v21 m_response_hdr;
-
-  time_t m_request_sent_time;
-  time_t m_response_received_time;
-
-  RefCountObj *m_ext_buffer;
-
-  // The following methods were added for BC support.
-  // Checks itself to verify that it is unmarshalled and v21 format.
-  bool
-  is_unmarshalled_format() const
-  {
-    return CACHE_ALT_MAGIC_MARSHALED == m_magic && reinterpret_cast<intptr_t>(m_request_hdr.m_heap) == sizeof(*this);
-  }
-};
-
-/// Really just a namespace, doesn't depend on any of the members.
-struct HTTPInfo_v21 {
-  typedef uint64_t FragOffset;
-  /// Version upgrade methods
-  /// @a src , @a dst , and @a n are updated upon return.
-  /// @a n is the space in @a dst remaining.
-  /// @return @c false if something went wrong.
-  static bool copy_and_upgrade_unmarshalled_to_v23(char *&dst, char *&src, size_t &length, int n_frags, FragOffset *frag_offsets);
-  /// The size of the marshalled data of a marshalled alternate header.
-  static size_t marshalled_length(void *data);
-};
-
-/// Pre version 24.
-struct Doc_v23 {
-  uint32_t magic;           // DOC_MAGIC
-  uint32_t len;             // length of this segment (including hlen, flen & sizeof(Doc), unrounded)
-  uint64_t total_len;       // total length of document
-  CryptoHash_v23 first_key; ///< first key in object.
-  CryptoHash_v23 key;       ///< Key for this doc.
-  uint32_t hlen;            ///< Length of this header.
-  uint32_t doc_type : 8;    ///< Doc type - indicates the format of this structure and its content.
-  uint32_t _flen : 24;      ///< Fragment table length.
-  uint32_t sync_serial;
-  uint32_t write_serial;
-  uint32_t pinned; // pinned until
-  uint32_t checksum;
-
-  char *hdr();
-  char *data();
-  size_t data_len();
-};
-
-char *
-Doc_v23::data()
-{
-  return reinterpret_cast<char *>(this) + sizeof(Doc_v23) + _flen + hlen;
-}
-size_t
-Doc_v23::data_len()
-{
-  return len - sizeof(Doc_v23) - hlen;
-}
-char *
-Doc_v23::hdr()
-{
-  return reinterpret_cast<char *>(this) + sizeof(Doc_v23);
-}
-
-} // namespace cache_bc