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

[1/6] trafficserver git commit: TS-3580: cache generation ID configuration

Repository: trafficserver
Updated Branches:
  refs/heads/master eb75a9d6d -> 2f8c94108


TS-3580: cache generation ID configuration

Add a new overrideable proxy.config.http.cache.generation configuration
option. This is a generation number that will be hashed into the
cache key before accessing the cache layer.

Update the X-Debug plugin to publish the generation ID into the
X-Cache-Generation header.


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

Branch: refs/heads/master
Commit: 643208d71e27274490bdcc2116f6060043855093
Parents: eb75a9d
Author: James Peach <jp...@apache.org>
Authored: Fri Mar 13 21:07:50 2015 -0700
Committer: James Peach <jp...@apache.org>
Committed: Mon Jun 15 09:09:31 2015 -0700

----------------------------------------------------------------------
 doc/reference/plugins/xdebug.en.rst   |  4 ++++
 lib/ts/apidefs.h.in                   |  1 +
 mgmt/RecordsConfig.cc                 |  2 ++
 plugins/experimental/xdebug/xdebug.cc | 25 +++++++++++++++++++++++++
 proxy/InkAPI.cc                       |  6 ++++++
 proxy/InkAPITest.cc                   |  3 ++-
 proxy/hdrs/URL.cc                     | 20 ++++++++++++++------
 proxy/hdrs/URL.h                      | 10 ++++++----
 proxy/http/HttpConfig.cc              |  2 ++
 proxy/http/HttpConfig.h               |  3 ++-
 10 files changed, 64 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/trafficserver/blob/643208d7/doc/reference/plugins/xdebug.en.rst
----------------------------------------------------------------------
diff --git a/doc/reference/plugins/xdebug.en.rst b/doc/reference/plugins/xdebug.en.rst
index 0ee9197..16e0e76 100644
--- a/doc/reference/plugins/xdebug.en.rst
+++ b/doc/reference/plugins/xdebug.en.rst
@@ -59,6 +59,10 @@ X-Cache
     skipped     The cache lookup was skipped.
     ==========  ===========
 
+X-Cache-Generation
+  The cache generation ID for this transaction, as specified by the
+  :ts:cv:`proxy.config.http.cache.generation` configuration variable.
+
 X-Milestones
     The ``X-Milestones`` header contains detailed information about
     how long the transaction took to traverse portions of the HTTP

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/643208d7/lib/ts/apidefs.h.in
----------------------------------------------------------------------
diff --git a/lib/ts/apidefs.h.in b/lib/ts/apidefs.h.in
index bd847f2..e28f458 100644
--- a/lib/ts/apidefs.h.in
+++ b/lib/ts/apidefs.h.in
@@ -679,6 +679,7 @@ typedef enum {
   TS_CONFIG_HTTP_GLOBAL_USER_AGENT_HEADER,
   TS_CONFIG_HTTP_AUTH_SERVER_SESSION_PRIVATE,
   TS_CONFIG_HTTP_SLOW_LOG_THRESHOLD,
+  TS_CONFIG_HTTP_CACHE_GENERATION,
   TS_CONFIG_LAST_ENTRY
 } TSOverridableConfigKey;
 

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/643208d7/mgmt/RecordsConfig.cc
----------------------------------------------------------------------
diff --git a/mgmt/RecordsConfig.cc b/mgmt/RecordsConfig.cc
index c58c7fb..1e5d8e6 100644
--- a/mgmt/RecordsConfig.cc
+++ b/mgmt/RecordsConfig.cc
@@ -613,6 +613,8 @@ static const RecordElement RecordsConfig[] =
   //        #################
   {RECT_CONFIG, "proxy.config.http.cache.http", RECD_INT, "1", RECU_DYNAMIC, RR_NULL, RECC_INT, "[0-1]", RECA_NULL}
   ,
+  {RECT_CONFIG, "proxy.config.http.cache.generation", RECD_INT, "-1", RECU_DYNAMIC, RR_NULL, RECC_NULL, NULL, RECA_NULL}
+  ,
   // Enabling this setting allows the proxy to cache empty documents. This currently requires
   // that the response has a Content-Length: header, with a value of "0".
   {RECT_CONFIG, "proxy.config.http.cache.allow_empty_doc", RECD_INT, "1", RECU_DYNAMIC, RR_NULL, RECC_NULL, "[0-1]", RECA_NULL }

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/643208d7/plugins/experimental/xdebug/xdebug.cc
----------------------------------------------------------------------
diff --git a/plugins/experimental/xdebug/xdebug.cc b/plugins/experimental/xdebug/xdebug.cc
index 0fc38b6..ca45c30 100644
--- a/plugins/experimental/xdebug/xdebug.cc
+++ b/plugins/experimental/xdebug/xdebug.cc
@@ -28,6 +28,7 @@
 #define XHEADER_X_CACHE_KEY 0x0004u
 #define XHEADER_X_MILESTONES 0x0008u
 #define XHEADER_X_CACHE 0x0010u
+#define XHEADER_X_GENERATION 0x0020u
 
 static int XArgIndex = 0;
 static TSCont XInjectHeadersCont = NULL;
@@ -56,6 +57,24 @@ FindOrMakeHdrField(TSMBuffer buffer, TSMLoc hdr, const char *name, unsigned len)
 }
 
 static void
+InjectGenerationHeader(TSHttpTxn txn, TSMBuffer buffer, TSMLoc hdr)
+{
+  TSMgmtInt value;
+  TSMLoc dst = TS_NULL_MLOC;
+
+  if (TSHttpTxnConfigIntGet(txn, TS_CONFIG_HTTP_CACHE_GENERATION, &value) == TS_SUCCESS) {
+    dst = FindOrMakeHdrField(buffer, hdr, "X-Cache-Generation", lengthof("X-Cache-Generation"));
+    if (dst != TS_NULL_MLOC) {
+      TSReleaseAssert(TSMimeHdrFieldValueInt64Set(buffer, hdr, dst, -1 /* idx */, value) == TS_SUCCESS);
+    }
+  }
+
+  if (dst != TS_NULL_MLOC) {
+    TSHandleMLocRelease(buffer, hdr, dst);
+  }
+}
+
+static void
 InjectCacheKeyHeader(TSHttpTxn txn, TSMBuffer buffer, TSMLoc hdr)
 {
   TSMLoc url = TS_NULL_MLOC;
@@ -236,6 +255,10 @@ XInjectResponseHeaders(TSCont /* contp */, TSEvent event, void *edata)
     InjectMilestonesHeader(txn, buffer, hdr);
   }
 
+  if (xheaders & XHEADER_X_GENERATION) {
+    InjectGenerationHeader(txn, buffer, hdr);
+  }
+
 done:
   TSHttpTxnReenable(txn, TS_EVENT_HTTP_CONTINUE);
   return TS_EVENT_NONE;
@@ -282,6 +305,8 @@ XScanRequestHeaders(TSCont /* contp */, TSEvent event, void *edata)
         xheaders |= XHEADER_X_MILESTONES;
       } else if (header_field_eq("x-cache", value, vsize)) {
         xheaders |= XHEADER_X_CACHE;
+      } else if (header_field_eq("x-cache-generation", value, vsize)) {
+        xheaders |= XHEADER_X_GENERATION;
       } else if (header_field_eq("via", value, vsize)) {
         // If the client requests the Via header, enable verbose Via debugging for this transaction.
         TSHttpTxnConfigIntSet(txn, TS_CONFIG_HTTP_INSERT_RESPONSE_VIA_STR, 3);

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/643208d7/proxy/InkAPI.cc
----------------------------------------------------------------------
diff --git a/proxy/InkAPI.cc b/proxy/InkAPI.cc
index 31afa4c..295ae00 100644
--- a/proxy/InkAPI.cc
+++ b/proxy/InkAPI.cc
@@ -7958,6 +7958,10 @@ _conf_to_memberp(TSOverridableConfigKey conf, OverridableHttpConfigParams *overr
   case TS_CONFIG_HTTP_CACHE_RANGE_WRITE:
     ret = &overridableHttpConfig->cache_range_write;
     break;
+  case TS_CONFIG_HTTP_CACHE_GENERATION:
+    typ = OVERRIDABLE_TYPE_INT;
+    ret = &overridableHttpConfig->cache_generation_number;
+    break;
   case TS_CONFIG_HTTP_POST_CHECK_CONTENT_LENGTH_ENABLED:
     ret = &overridableHttpConfig->post_check_content_length_enabled;
     break;
@@ -8228,6 +8232,8 @@ TSHttpTxnConfigFind(const char *name, int length, TSOverridableConfigKey *conf,
   case 34:
     if (!strncmp(name, "proxy.config.http.chunking_enabled", length))
       cnf = TS_CONFIG_HTTP_CHUNKING_ENABLED;
+    else if (!strncmp(name, "proxy.config.http.cache.generation", length))
+      cnf = TS_CONFIG_HTTP_CACHE_GENERATION;
     break;
 
   case 35:

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/643208d7/proxy/InkAPITest.cc
----------------------------------------------------------------------
diff --git a/proxy/InkAPITest.cc b/proxy/InkAPITest.cc
index 9952225..2b744e8 100644
--- a/proxy/InkAPITest.cc
+++ b/proxy/InkAPITest.cc
@@ -7211,7 +7211,8 @@ const char *SDK_Overridable_Configs[TS_CONFIG_LAST_ENTRY] = {
   "proxy.config.ssl.hsts_max_age", "proxy.config.ssl.hsts_include_subdomains", "proxy.config.http.cache.open_read_retry_time",
   "proxy.config.http.cache.max_open_read_retries", "proxy.config.http.cache.range.write",
   "proxy.config.http.post.check.content_length.enabled", "proxy.config.http.global_user_agent_header",
-  "proxy.config.http.auth_server_session_private", "proxy.config.http.slow.log.threshold"};
+  "proxy.config.http.auth_server_session_private", "proxy.config.http.slow.log.threshold",
+  "proxy.config.http.cache.generation"};
 
 REGRESSION_TEST(SDK_API_OVERRIDABLE_CONFIGS)(RegressionTest *test, int /* atype ATS_UNUSED */, int *pstatus)
 {

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/643208d7/proxy/hdrs/URL.cc
----------------------------------------------------------------------
diff --git a/proxy/hdrs/URL.cc b/proxy/hdrs/URL.cc
index a0529c8..e6cde0b 100644
--- a/proxy/hdrs/URL.cc
+++ b/proxy/hdrs/URL.cc
@@ -1572,7 +1572,7 @@ memcpy_tolower(char *d, const char *s, int n)
 // no buffer overflow, no unescaping needed
 
 static inline void
-url_MD5_get_fast(URLImpl *url, CryptoContext &ctx, CryptoHash *hash)
+url_MD5_get_fast(const URLImpl *url, CryptoContext &ctx, CryptoHash *hash, cache_generation_t generation)
 {
   char buffer[BUFSIZE];
   char *p;
@@ -1603,12 +1603,16 @@ url_MD5_get_fast(URLImpl *url, CryptoContext &ctx, CryptoHash *hash)
   *p++ = ((char *)&port)[1];
 
   ctx.update(buffer, p - buffer);
+  if (generation != -1) {
+    ctx.update(&generation, sizeof(generation));
+  }
+
   ctx.finalize(hash);
 }
 
 
 static inline void
-url_MD5_get_general(URLImpl *url, CryptoContext &ctx, CryptoHash &hash)
+url_MD5_get_general(const URLImpl *url, CryptoContext &ctx, CryptoHash &hash, cache_generation_t generation)
 {
   char buffer[BUFSIZE];
   char *p, *e;
@@ -1675,25 +1679,29 @@ url_MD5_get_general(URLImpl *url, CryptoContext &ctx, CryptoHash &hash)
   port = url_canonicalize_port(url->m_url_type, url->m_port);
 
   ctx.update(&port, sizeof(port));
+  if (generation != -1) {
+    ctx.update(&generation, sizeof(generation));
+  }
+
   ctx.finalize(hash);
 }
 
 void
-url_MD5_get(URLImpl *url, CryptoHash *hash)
+url_MD5_get(const URLImpl *url, CryptoHash *hash, cache_generation_t generation)
 {
   URLHashContext ctx;
   if ((url_hash_method != 0) && (url->m_url_type == URL_TYPE_HTTP) &&
       ((url->m_len_user + url->m_len_password + url->m_len_params + url->m_len_query) == 0) &&
       (3 + 1 + 1 + 1 + 1 + 1 + 2 + url->m_len_scheme + url->m_len_host + url->m_len_path < BUFSIZE) &&
       (memchr(url->m_ptr_host, '%', url->m_len_host) == NULL) && (memchr(url->m_ptr_path, '%', url->m_len_path) == NULL)) {
-    url_MD5_get_fast(url, ctx, hash);
+    url_MD5_get_fast(url, ctx, hash, generation);
 #ifdef DEBUG
     CryptoHash md5_general;
-    url_MD5_get_general(url, ctx, md5_general);
+    url_MD5_get_general(url, ctx, md5_general, generation);
     ink_assert(*hash == md5_general);
 #endif
   } else {
-    url_MD5_get_general(url, ctx, *hash);
+    url_MD5_get_general(url, ctx, *hash, generation);
   }
 }
 

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/643208d7/proxy/hdrs/URL.h
----------------------------------------------------------------------
diff --git a/proxy/hdrs/URL.h b/proxy/hdrs/URL.h
index 4d362ce..de82af1 100644
--- a/proxy/hdrs/URL.h
+++ b/proxy/hdrs/URL.h
@@ -33,6 +33,8 @@
 
 #include "ink_apidefs.h"
 
+typedef int64_t cache_generation_t;
+
 enum URLType {
   URL_TYPE_NONE,
   URL_TYPE_HTTP,
@@ -212,7 +214,7 @@ void url_called_set(URLImpl *url);
 char *url_string_get_buf(URLImpl *url, char *dstbuf, int dstbuf_size, int *length);
 
 const char *url_scheme_get(URLImpl *url, int *length);
-void url_MD5_get(URLImpl *url, CryptoHash *md5);
+void url_MD5_get(const URLImpl *url, CryptoHash *md5, cache_generation_t generation = -1);
 void url_host_MD5_get(URLImpl *url, CryptoHash *md5);
 const char *url_scheme_set(HdrHeap *heap, URLImpl *url, const char *value, int value_wks_idx, int length, bool copy_string);
 
@@ -283,7 +285,7 @@ public:
   char *string_get(Arena *arena, int *length = NULL);
   char *string_get_ref(int *length = NULL);
   char *string_get_buf(char *dstbuf, int dsbuf_size, int *length = NULL);
-  void hash_get(CryptoHash *md5);
+  void hash_get(CryptoHash *md5, cache_generation_t generation = -1) const;
   void host_hash_get(CryptoHash *md5);
 
   const char *scheme_get(int *length);
@@ -470,10 +472,10 @@ URL::string_get_buf(char *dstbuf, int dsbuf_size, int *length)
   -------------------------------------------------------------------------*/
 
 inline void
-URL::hash_get(CryptoHash *md5)
+URL::hash_get(CryptoHash *md5, cache_generation_t generation) const
 {
   ink_assert(valid());
-  url_MD5_get(m_url_impl, md5);
+  url_MD5_get(m_url_impl, md5, generation);
 }
 
 /*-------------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/643208d7/proxy/http/HttpConfig.cc
----------------------------------------------------------------------
diff --git a/proxy/http/HttpConfig.cc b/proxy/http/HttpConfig.cc
index 02115bf..e3c5e17 100644
--- a/proxy/http/HttpConfig.cc
+++ b/proxy/http/HttpConfig.cc
@@ -1066,6 +1066,7 @@ HttpConfig::startup()
   // open read failure retries
   HttpEstablishStaticConfigLongLong(c.oride.max_cache_open_read_retries, "proxy.config.http.cache.max_open_read_retries");
   HttpEstablishStaticConfigLongLong(c.oride.cache_open_read_retry_time, "proxy.config.http.cache.open_read_retry_time");
+  HttpEstablishStaticConfigLongLong(c.oride.cache_generation_number, "proxy.config.http.cache.generation");
 
   // open write failure retries
   HttpEstablishStaticConfigLongLong(c.max_cache_open_write_retries, "proxy.config.http.cache.max_open_write_retries");
@@ -1329,6 +1330,7 @@ HttpConfig::reconfigure()
   // open read failure retries
   params->oride.max_cache_open_read_retries = m_master.oride.max_cache_open_read_retries;
   params->oride.cache_open_read_retry_time = m_master.oride.cache_open_read_retry_time;
+  params->oride.cache_generation_number = m_master.oride.cache_generation_number;
 
   // open write failure retries
   params->max_cache_open_write_retries = m_master.max_cache_open_write_retries;

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/643208d7/proxy/http/HttpConfig.h
----------------------------------------------------------------------
diff --git a/proxy/http/HttpConfig.h b/proxy/http/HttpConfig.h
index aaf48fa..3bbba9f 100644
--- a/proxy/http/HttpConfig.h
+++ b/proxy/http/HttpConfig.h
@@ -381,7 +381,7 @@ struct OverridableHttpConfigParams {
       transaction_active_timeout_out(0), origin_max_connections(0), connect_attempts_max_retries(0),
       connect_attempts_max_retries_dead_server(3), connect_attempts_rr_retries(3), connect_attempts_timeout(30),
       post_connect_attempts_timeout(1800), down_server_timeout(300), client_abort_threshold(10), freshness_fuzz_time(240),
-      freshness_fuzz_min_time(0), max_cache_open_read_retries(-1), cache_open_read_retry_time(10),
+      freshness_fuzz_min_time(0), max_cache_open_read_retries(-1), cache_open_read_retry_time(10), cache_generation_number(-1),
       background_fill_active_timeout(60), http_chunking_size(4096), flow_high_water_mark(0), flow_low_water_mark(0),
       default_buffer_size_index(8), default_buffer_water_mark(32768), slow_log_threshold(0),
 
@@ -546,6 +546,7 @@ struct OverridableHttpConfigParams {
   // open read failure retries.
   MgmtInt max_cache_open_read_retries;
   MgmtInt cache_open_read_retry_time; // time is in mseconds
+  MgmtInt cache_generation_number;
 
   MgmtInt background_fill_active_timeout;
 


[2/6] trafficserver git commit: TS-3580: Add tests for cache generation IDs.

Posted by jp...@apache.org.
TS-3580: Add tests for cache generation IDs.


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

Branch: refs/heads/master
Commit: 9e862d1ae83a75c9b7ea6fef37ff8d30e920511c
Parents: 4d7d892
Author: James Peach <jp...@apache.org>
Authored: Mon May 11 08:51:00 2015 -0700
Committer: James Peach <jp...@apache.org>
Committed: Mon Jun 15 09:16:00 2015 -0700

----------------------------------------------------------------------
 ci/tsqa/tests/test_cache_generation.py | 159 ++++++++++++++++++++++++++++
 1 file changed, 159 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/trafficserver/blob/9e862d1a/ci/tsqa/tests/test_cache_generation.py
----------------------------------------------------------------------
diff --git a/ci/tsqa/tests/test_cache_generation.py b/ci/tsqa/tests/test_cache_generation.py
new file mode 100644
index 0000000..6a79485
--- /dev/null
+++ b/ci/tsqa/tests/test_cache_generation.py
@@ -0,0 +1,159 @@
+'''
+Test the cache generation configuration
+'''
+
+#  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.
+
+import os
+import subprocess
+import logging
+import requests
+import random
+import uuid
+import time
+
+import helpers
+import tsqa.test_cases
+import tsqa.utils
+
+log = logging.getLogger(__name__)
+
+
+class TestCacheGeneration(helpers.EnvironmentCase):
+    '''
+    Test the cache object generation ID.
+    '''
+
+    def _fetch(self, path):
+        url = 'http://127.0.0.1:{}/{}'.format(
+                self.configs['records.config']['CONFIG']['proxy.config.http.server_ports'],
+                path
+        )
+        log.debug('get {}'.format(url))
+        return requests.get(url, headers={'x-debug': 'x-cache,x-cache-key,via,x-cache-generation'})
+
+    def _dump(self, response):
+        log.info('HTTP response {}'.format(response.status_code))
+        for k, v in response.headers.items():
+            log.info('    {}: {}'.format(k, v))
+
+    def _ctl(self, *args):
+        cmd = [os.path.join(self.environment.layout.bindir, 'traffic_ctl')] + list(args)
+        out, _ = tsqa.utils.run_sync_command(cmd,
+                env=self.environment.shell_env, stdout=subprocess.PIPE, stderr=subprocess.STDOUT
+        )
+        return out
+
+    @classmethod
+    def setUpEnv(cls, env):
+
+        cls.configs['plugin.config'].add_line('xdebug.so')
+
+        cls.configs['remap.config'].add_line(
+            'map /default/ http://127.0.0.1/ @plugin=generator.so'
+        )
+        cls.configs['remap.config'].add_line(
+            'map /generation1/ http://127.0.0.1/' +
+            ' @plugin=conf_remap.so @pparam=proxy.config.http.cache.generation=1' +
+            ' @plugin=generator.so'
+        )
+        cls.configs['remap.config'].add_line(
+            'map /generation2/ http://127.0.0.1/' +
+            ' @plugin=conf_remap.so @pparam=proxy.config.http.cache.generation=2' +
+            ' @plugin=generator.so'
+        )
+
+        # Start with cache generation turned off
+        cls.configs['records.config']['CONFIG']['proxy.config.http.cache.generation'] = -1
+        # Wait for the cache so we don't race client requests against it.
+        cls.configs['records.config']['CONFIG']['proxy.config.http.wait_for_cache'] = 1
+        cls.configs['records.config']['CONFIG']['proxy.config.config_update_interval_ms'] = 1
+
+    def test_generations_are_disjoint(self):
+        """Test that the same URL path in different cache generations creates disjoint objects"""
+        objectid = uuid.uuid4()
+
+        # First touch is a MISS.
+        ret = self._fetch('default/cache/10/{}'.format(objectid))
+        self.assertEqual(ret.status_code, 200)
+        self.assertEqual(ret.headers['x-cache'], 'miss', msg=ret)
+        self.assertEqual(ret.headers['x-cache-generation'], '-1')
+
+        # Same URL in generation 1 is a MISS.
+        ret = self._fetch('generation1/cache/10/{}'.format(objectid))
+        self.assertEqual(ret.status_code, 200)
+        self.assertEqual(ret.headers['x-cache'], 'miss')
+        self.assertEqual(ret.headers['x-cache-generation'], '1')
+
+        # Same URL in generation 2 is still a MISS.
+        ret = self._fetch('generation2/cache/10/{}'.format(objectid))
+        self.assertEqual(ret.status_code, 200)
+        self.assertEqual(ret.headers['x-cache'], 'miss')
+        self.assertEqual(ret.headers['x-cache-generation'], '2')
+
+        # Second touch is a HIT.
+        ret = self._fetch('default/cache/10/{}'.format(objectid))
+        self.assertEqual(ret.status_code, 200)
+        self.assertEqual(ret.headers['x-cache'], 'hit-fresh', msg=ret.headers['x-cache'])
+        self.assertEqual(ret.headers['x-cache-generation'], '-1')
+
+    def test_online_cache_clear(self):
+        """Test that incrementing the cache generation acts like a cache clear"""
+        objectid = uuid.uuid4()
+
+        # First touch is a MISS.
+        ret = self._fetch('default/cache/10/{}'.format(objectid))
+        self.assertEqual(ret.status_code, 200)
+        self.assertEqual(ret.headers['x-cache'], 'miss')
+
+        # Second touch is a HIT.
+        ret = self._fetch('default/cache/10/{}'.format(objectid))
+        self.assertEqual(ret.status_code, 200)
+        self.assertEqual(ret.headers['x-cache'], 'hit-fresh')
+
+        # Now update the generation number.
+        timeout = float(self._ctl('config', 'get', 'proxy.config.config_update_interval_ms').split(' ')[-1])
+        generation = random.randrange(65000)
+        gencount = 0
+
+        self._ctl('config', 'set', 'proxy.config.http.cache.generation', str(generation))
+        self._ctl('config', 'reload')
+
+        for _ in xrange(5):
+            if gencount == 0:
+                log.debug('waiting {} secs for the config to update'.format(timeout / 1000))
+                time.sleep(timeout / 1000)
+
+            ret = self._fetch('default/cache/10/{}'.format(objectid))
+            self.assertEqual(ret.status_code, 200)
+
+            if ret.headers['x-cache-generation'] == str(generation):
+                if gencount == 0:
+                    # First time we see the new generation, it should be a miss.
+                    self.assertEqual(ret.headers['x-cache'], 'miss')
+                else:
+                    # Now the previous hits should become misses.
+                    self.assertEqual(ret.headers['x-cache'], 'hit-fresh')
+            else:
+                # Config has not updated, so it should be a hit.
+                self.assertEqual(ret.headers['x-cache'], 'hit-fresh')
+                self.assertEqual(ret.headers['x-cache-generation'], '-1')
+
+                gencount = gencount + 1
+
+        self.assertNotEqual(gencount, 0, msg='proxy.config.http.cache.generation never updated')
+


[5/6] trafficserver git commit: TS-3580; Apply the cache generation number in ICP

Posted by jp...@apache.org.
TS-3580; Apply the cache generation number in ICP

In the ICP support, use the current cache generation number to
access cache objects. Note that at this point we cannot tell whether
the cache generation has been overridden for a specific object; we
can only use the global configuration.


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

Branch: refs/heads/master
Commit: 9f71421a15e1a8ee51fd00782947535764a60672
Parents: 7ff2d61
Author: James Peach <jp...@apache.org>
Authored: Mon May 4 20:25:00 2015 -0700
Committer: James Peach <jp...@apache.org>
Committed: Mon Jun 15 09:16:00 2015 -0700

----------------------------------------------------------------------
 proxy/ICP.cc       | 5 +++--
 proxy/ICP.h        | 6 ++++++
 proxy/ICPConfig.cc | 2 ++
 3 files changed, 11 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/trafficserver/blob/9f71421a/proxy/ICP.cc
----------------------------------------------------------------------
diff --git a/proxy/ICP.cc b/proxy/ICP.cc
index b3affe4..89232a8 100644
--- a/proxy/ICP.cc
+++ b/proxy/ICP.cc
@@ -454,12 +454,13 @@ ICPPeerReadCont::ICPPeerQueryCont(int /* event ATS_UNUSED */, Event * /* e ATS_U
   SET_HANDLER((ICPPeerReadContHandler)&ICPPeerReadCont::ICPPeerQueryEvent);
   if (_state->_rICPmsg->un.query.URL && *_state->_rICPmsg->un.query.URL) {
     HttpCacheKey key;
+    ICPConfigData *cfg = _ICPpr->GetConfig()->globalConfig();
 
-    Cache::generate_key(&key, &_state->_cachelookupURL); // XXX choose a cache generation number ...
+    Cache::generate_key(&key, &_state->_cachelookupURL, cfg->ICPCacheGeneration());
     _state->_queryResult = ~CACHE_EVENT_LOOKUP_FAILED;
     _start_time = ink_get_hrtime();
 
-    if (pluginFreshnessCalcFunc && _ICPpr->GetConfig()->globalConfig()->ICPStaleLookup()) {
+    if (pluginFreshnessCalcFunc && cfg->ICPStaleLookup()) {
       //////////////////////////////////////////////////////////////
       // Note: _cache_lookup_local is ignored in this case, since
       //       cache clustering is not used with stale lookup.

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/9f71421a/proxy/ICP.h
----------------------------------------------------------------------
diff --git a/proxy/ICP.h b/proxy/ICP.h
index 321c666..076b305 100644
--- a/proxy/ICP.h
+++ b/proxy/ICP.h
@@ -288,6 +288,11 @@ public:
   {
     return _default_reply_port;
   }
+  inline cache_generation_t
+  ICPCacheGeneration() const
+  {
+    return _cache_generation;
+  }
 
 private:
   //---------------------------------------------------------
@@ -302,6 +307,7 @@ private:
   int _stale_lookup;
   int _reply_to_unknown_peer;
   int _default_reply_port;
+  int64_t _cache_generation;
 };
 
 //----------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/9f71421a/proxy/ICPConfig.cc
----------------------------------------------------------------------
diff --git a/proxy/ICPConfig.cc b/proxy/ICPConfig.cc
index 7f4979b..cd56050 100644
--- a/proxy/ICPConfig.cc
+++ b/proxy/ICPConfig.cc
@@ -392,6 +392,8 @@ ICPConfiguration::ICPConfiguration() : _icp_config_callouts(0)
   ICP_EstablishStaticConfigInteger(_icp_cdata_current->_stale_lookup, "proxy.config.icp.stale_icp_enabled");
   ICP_EstablishStaticConfigInteger(_icp_cdata_current->_reply_to_unknown_peer, "proxy.config.icp.reply_to_unknown_peer");
   ICP_EstablishStaticConfigInteger(_icp_cdata_current->_default_reply_port, "proxy.config.icp.default_reply_port");
+  REC_EstablishStaticConfigInteger(_icp_cdata_current->_cache_generation, "proxy.config.http.cache.generation");
+
   UpdateGlobalConfig(); // sync working copy with current
 
   //**********************************************************


[6/6] trafficserver git commit: TS-3580: update CHANGES

Posted by jp...@apache.org.
TS-3580: update CHANGES


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

Branch: refs/heads/master
Commit: 2f8c94108dc32388488daaf9a3a15a22ae33edbd
Parents: 9e862d1
Author: James Peach <jp...@apache.org>
Authored: Mon Jun 15 09:27:50 2015 -0700
Committer: James Peach <jp...@apache.org>
Committed: Mon Jun 15 09:27:50 2015 -0700

----------------------------------------------------------------------
 CHANGES | 2 ++
 1 file changed, 2 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/trafficserver/blob/2f8c9410/CHANGES
----------------------------------------------------------------------
diff --git a/CHANGES b/CHANGES
index 2a76f46..986f548 100644
--- a/CHANGES
+++ b/CHANGES
@@ -1,6 +1,8 @@
                                                          -*- coding: utf-8 -*-
 Changes with Apache Traffic Server 6.0.0
 
+  *) [TS-3580] Implement a cache generation ID configuration.
+
   *) [TS-3365] Enable C99 by default.
 
   *) [TS-3689] Remove libck.


[4/6] trafficserver git commit: TS-3580: Use HttpCacheKey to enter the cache

Posted by jp...@apache.org.
TS-3580: Use HttpCacheKey to enter the cache

Refactor the cache interface so that the HTTP cache lookup path
uses the new HttpCacheKey type, rather than fussing with URL objects
directly. This lets us use the per-request cache generation number
to access cache objects while the cache remains oblivious.


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

Branch: refs/heads/master
Commit: 7ff2d6189d0170b422b8d1bc3df1c2145198ee68
Parents: 643208d
Author: James Peach <jp...@apache.org>
Authored: Mon May 4 09:44:18 2015 -0700
Committer: James Peach <jp...@apache.org>
Committed: Mon Jun 15 09:16:00 2015 -0700

----------------------------------------------------------------------
 iocore/cache/Cache.cc                   | 103 ++++++++++-----------------
 iocore/cache/CachePages.cc              |  18 ++---
 iocore/cache/I_Cache.h                  |  22 +++---
 iocore/cache/I_CacheDefs.h              |  18 +++++
 iocore/cache/P_CacheInternal.h          |  35 +++------
 iocore/cluster/ClusterCache.cc          |   4 +-
 iocore/cluster/P_ClusterCacheInternal.h |   4 +-
 iocore/cluster/P_ClusterInline.h        |  53 +++++++-------
 proxy/CacheControl.cc                   |   2 +-
 proxy/CacheControl.h                    |   2 +-
 proxy/ICP.cc                            |   8 ++-
 proxy/Prefetch.cc                       |   6 +-
 proxy/http/HttpCacheSM.cc               |  16 +++--
 proxy/http/HttpCacheSM.h                |   8 ++-
 proxy/http/HttpSM.cc                    |  16 ++++-
 15 files changed, 153 insertions(+), 162 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/trafficserver/blob/7ff2d618/iocore/cache/Cache.cc
----------------------------------------------------------------------
diff --git a/iocore/cache/Cache.cc b/iocore/cache/Cache.cc
index 370c516..e062c08 100644
--- a/iocore/cache/Cache.cc
+++ b/iocore/cache/Cache.cc
@@ -268,25 +268,18 @@ cache_stats_bytes_used_cb(const char *name, RecDataT data_type, RecData *data, R
 
 #ifdef CLUSTER_CACHE
 static Action *
-open_read_internal(int opcode, Continuation *cont, MIOBuffer *buf, CacheURL *url, CacheHTTPHdr *request,
-                   CacheLookupHttpConfig *params, CacheKey *key, time_t pin_in_cache, CacheFragType frag_type, char *hostname,
-                   int host_len)
+open_read_internal(int opcode, Continuation *cont, MIOBuffer *buf, const HttpCacheKey *key, CacheHTTPHdr *request,
+                   CacheLookupHttpConfig *params, time_t pin_in_cache, CacheFragType frag_type)
 {
-  INK_MD5 url_md5;
-  if ((opcode == CACHE_OPEN_READ_LONG) || (opcode == CACHE_OPEN_READ_BUFFER_LONG)) {
-    Cache::generate_key(&url_md5, url);
-  } else {
-    url_md5 = *key;
-  }
-  ClusterMachine *m = cluster_machine_at_depth(cache_hash(url_md5));
+  ClusterMachine *m = cluster_machine_at_depth(cache_hash(key->hash));
 
   if (m) {
-    return Cluster_read(m, opcode, cont, buf, url, request, params, key, pin_in_cache, frag_type, hostname, host_len);
+    return Cluster_read(m, opcode, cont, buf, request, params, &key->hash, pin_in_cache, frag_type, key->hostname, key->hostlen);
   } else {
     if ((opcode == CACHE_OPEN_READ_LONG) || (opcode == CACHE_OPEN_READ_BUFFER_LONG)) {
-      return caches[frag_type]->open_read(cont, &url_md5, request, params, frag_type, hostname, host_len);
+      return caches[frag_type]->open_read(cont, &key->hash, request, params, frag_type, key->hostname, key->hostlen);
     } else {
-      return caches[frag_type]->open_read(cont, key, frag_type, hostname, host_len);
+      return caches[frag_type]->open_read(cont, &key->hash, frag_type, key->hostname, key->hostlen);
     }
   }
 }
@@ -1205,8 +1198,8 @@ CacheProcessor::db_check(bool afix)
 }
 
 Action *
-CacheProcessor::lookup(Continuation *cont, CacheKey *key, bool cluster_cache_local ATS_UNUSED, bool local_only ATS_UNUSED,
-                       CacheFragType frag_type, char *hostname, int host_len)
+CacheProcessor::lookup(Continuation *cont, const CacheKey *key, bool cluster_cache_local ATS_UNUSED, bool local_only ATS_UNUSED,
+                       CacheFragType frag_type, const char *hostname, int host_len)
 {
 #ifdef CLUSTER_CACHE
   // Try to send remote, if not possible, handle locally
@@ -1221,16 +1214,20 @@ CacheProcessor::lookup(Continuation *cont, CacheKey *key, bool cluster_cache_loc
 }
 
 inkcoreapi Action *
-CacheProcessor::open_read(Continuation *cont, CacheKey *key, bool cluster_cache_local ATS_UNUSED, CacheFragType frag_type,
-                          char *hostname, int host_len)
+CacheProcessor::open_read(Continuation *cont, const CacheKey *key, bool cluster_cache_local ATS_UNUSED, CacheFragType frag_type,
+                          const char *hostname, int hostlen)
 {
 #ifdef CLUSTER_CACHE
   if (cache_clustering_enabled > 0 && !cluster_cache_local) {
-    return open_read_internal(CACHE_OPEN_READ, cont, (MIOBuffer *)0, (CacheURL *)0, (CacheHTTPHdr *)0, (CacheLookupHttpConfig *)0,
-                              key, 0, frag_type, hostname, host_len);
+    HttpCacheKey hkey;
+    hkey.hash = *key;
+    hkey.hostname = hostname;
+    hkey.hostlen = hostlen;
+    return open_read_internal(CACHE_OPEN_READ, cont, (MIOBuffer *)0, &hkey, (CacheHTTPHdr *)0, (CacheLookupHttpConfig *)0, 0,
+                              frag_type);
   }
 #endif
-  return caches[frag_type]->open_read(cont, key, frag_type, hostname, host_len);
+  return caches[frag_type]->open_read(cont, key, frag_type, hostname, hostlen);
 }
 
 inkcoreapi Action *
@@ -1241,15 +1238,15 @@ CacheProcessor::open_write(Continuation *cont, CacheKey *key, bool cluster_cache
   if (cache_clustering_enabled > 0 && !cluster_cache_local) {
     ClusterMachine *m = cluster_machine_at_depth(cache_hash(*key));
     if (m)
-      return Cluster_write(cont, expected_size, (MIOBuffer *)0, m, key, frag_type, options, pin_in_cache, CACHE_OPEN_WRITE, key,
-                           (CacheURL *)0, (CacheHTTPHdr *)0, (CacheHTTPInfo *)0, hostname, host_len);
+      return Cluster_write(cont, expected_size, (MIOBuffer *)0, m, key, frag_type, options, pin_in_cache, CACHE_OPEN_WRITE,
+                           (CacheHTTPHdr *)0, (CacheHTTPInfo *)0, hostname, host_len);
   }
 #endif
   return caches[frag_type]->open_write(cont, key, frag_type, options, pin_in_cache, hostname, host_len);
 }
 
 Action *
-CacheProcessor::remove(Continuation *cont, CacheKey *key, bool cluster_cache_local ATS_UNUSED, CacheFragType frag_type,
+CacheProcessor::remove(Continuation *cont, const CacheKey *key, bool cluster_cache_local ATS_UNUSED, CacheFragType frag_type,
                        const char *hostname, int host_len)
 {
   Debug("cache_remove", "[CacheProcessor::remove] Issuing cache delete for %u", cache_hash(*key));
@@ -1275,15 +1272,10 @@ scan(Continuation *cont, char *hostname = 0, int host_len = 0, int KB_per_second
 
 #ifdef HTTP_CACHE
 Action *
-CacheProcessor::lookup(Continuation *cont, CacheURL *url, bool cluster_cache_local, bool local_only, CacheFragType frag_type)
+CacheProcessor::lookup(Continuation *cont, const HttpCacheKey *key, bool cluster_cache_local, bool local_only,
+                       CacheFragType frag_type)
 {
-  (void)local_only;
-  INK_MD5 md5;
-  url->hash_get(&md5);
-  int host_len = 0;
-  const char *hostname = url->host_get(&host_len);
-
-  return lookup(cont, &md5, cluster_cache_local, local_only, frag_type, (char *)hostname, host_len);
+  return lookup(cont, &key->hash, cluster_cache_local, local_only, frag_type, key->hostname, key->hostlen);
 }
 
 #endif
@@ -3016,17 +3008,6 @@ Cache::lookup(Continuation *cont, const CacheKey *key, CacheFragType type, char
     return ACTION_RESULT_DONE;
 }
 
-Action *
-Cache::lookup(Continuation *cont, CacheURL *url, CacheFragType type)
-{
-  CryptoHash id;
-
-  url->hash_get(&id);
-  int len = 0;
-  char const *hostname = url->host_get(&len);
-  return lookup(cont, &id, type, hostname, len);
-}
-
 int
 CacheVC::removeEvent(int /* event ATS_UNUSED */, Event * /* e ATS_UNUSED */)
 {
@@ -3797,65 +3778,53 @@ ink_cache_init(ModuleVersion v)
 
 //----------------------------------------------------------------------------
 Action *
-CacheProcessor::open_read(Continuation *cont, CacheURL *url, bool cluster_cache_local, CacheHTTPHdr *request,
+CacheProcessor::open_read(Continuation *cont, const HttpCacheKey *key, bool cluster_cache_local, CacheHTTPHdr *request,
                           CacheLookupHttpConfig *params, time_t pin_in_cache, CacheFragType type)
 {
 #ifdef CLUSTER_CACHE
   if (cache_clustering_enabled > 0 && !cluster_cache_local) {
-    return open_read_internal(CACHE_OPEN_READ_LONG, cont, (MIOBuffer *)0, url, request, params, (CacheKey *)0, pin_in_cache, type,
-                              (char *)0, 0);
+    return open_read_internal(CACHE_OPEN_READ_LONG, cont, (MIOBuffer *)0, key, request, params, pin_in_cache, type);
   }
 #endif
-  return caches[type]->open_read(cont, url, request, params, type);
+
+  return caches[type]->open_read(cont, &key->hash, request, params, type, key->hostname, key->hostlen);
 }
 
 
 //----------------------------------------------------------------------------
 Action *
-CacheProcessor::open_write(Continuation *cont, int expected_size, CacheURL *url, bool cluster_cache_local, CacheHTTPHdr *request,
-                           CacheHTTPInfo *old_info, time_t pin_in_cache, CacheFragType type)
+CacheProcessor::open_write(Continuation *cont, int expected_size, const HttpCacheKey *key, bool cluster_cache_local,
+                           CacheHTTPHdr *request, CacheHTTPInfo *old_info, time_t pin_in_cache, CacheFragType type)
 {
 #ifdef CLUSTER_CACHE
   if (cache_clustering_enabled > 0 && !cluster_cache_local) {
-    INK_MD5 url_md5;
-    Cache::generate_key(&url_md5, url);
-    ClusterMachine *m = cluster_machine_at_depth(cache_hash(url_md5));
+    ClusterMachine *m = cluster_machine_at_depth(cache_hash(key->hash));
 
     if (m) {
       // Do remote open_write()
-      INK_MD5 url_only_md5;
-      Cache::generate_key(&url_only_md5, url);
-      return Cluster_write(cont, expected_size, (MIOBuffer *)0, m, &url_only_md5, type, false, pin_in_cache, CACHE_OPEN_WRITE_LONG,
-                           (CacheKey *)0, url, request, old_info, (char *)0, 0);
+      return Cluster_write(cont, expected_size, (MIOBuffer *)0, m, &key->hash, type, false, pin_in_cache, CACHE_OPEN_WRITE_LONG,
+                           request, old_info, key->hostname, key->hostlen);
     }
   }
 #endif
-  return caches[type]->open_write(cont, url, request, old_info, pin_in_cache, type);
+  return caches[type]->open_write(cont, &key->hash, old_info, pin_in_cache, NULL /* key1 */, type, key->hostname, key->hostlen);
 }
 
 //----------------------------------------------------------------------------
 // Note: this should not be called from from the cluster processor, or bad
 // recursion could occur. This is merely a convenience wrapper.
 Action *
-CacheProcessor::remove(Continuation *cont, CacheURL *url, bool cluster_cache_local, CacheFragType frag_type)
+CacheProcessor::remove(Continuation *cont, const HttpCacheKey *key, bool cluster_cache_local, CacheFragType frag_type)
 {
-  CryptoHash id;
-  int len = 0;
-  const char *hostname;
-
-  url->hash_get(&id);
-  hostname = url->host_get(&len);
-
-  Debug("cache_remove", "[CacheProcessor::remove] Issuing cache delete for %s", url->string_get_ref());
 #ifdef CLUSTER_CACHE
   if (cache_clustering_enabled > 0 && !cluster_cache_local) {
     // Remove from cluster
-    return remove(cont, &id, cluster_cache_local, frag_type, hostname, len);
+    return remove(cont, &key->hash, cluster_cache_local, frag_type, key->hostname, key->hostlen);
   }
 #endif
 
   // Remove from local cache only.
-  return caches[frag_type]->remove(cont, &id, frag_type, hostname, len);
+  return caches[frag_type]->remove(cont, &key->hash, frag_type, key->hostname, key->hostlen);
 }
 
 CacheDisk *

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/7ff2d618/iocore/cache/CachePages.cc
----------------------------------------------------------------------
diff --git a/iocore/cache/CachePages.cc b/iocore/cache/CachePages.cc
index 13f2b2a..721bffc 100644
--- a/iocore/cache/CachePages.cc
+++ b/iocore/cache/CachePages.cc
@@ -438,13 +438,13 @@ ShowCache::lookup_url(int event, Event *e)
   const char *s;
   s = show_cache_urlstrs[0];
   url.parse(&s, s + strlen(s));
-  INK_MD5 md5;
-  int len;
-  url.hash_get(&md5);
-  const char *hostname = url.host_get(&len);
+
+  HttpCacheKey key;
+  Cache::generate_key(&key, &url); // XXX choose a cache generation number ...
+
   SET_HANDLER(&ShowCache::handleCacheEvent);
   Action *lookup_result =
-    cacheProcessor.open_read(this, &md5, getClusterCacheLocal(&url, (char *)hostname), CACHE_FRAG_TYPE_HTTP, (char *)hostname, len);
+    cacheProcessor.open_read(this, &key.hash, getClusterCacheLocal(&url), CACHE_FRAG_TYPE_HTTP, key.hostname, key.hostlen);
   if (!lookup_result)
     lookup_result = ACTION_IO_ERROR;
   if (lookup_result == ACTION_RESULT_DONE)
@@ -482,9 +482,11 @@ ShowCache::delete_url(int event, Event *e)
   // increment the index so that the next time
   // delete_url is called you delete the next url
   urlstrs_index++;
-  int len;
-  const char *hostname = url.host_get(&len);
-  cacheProcessor.remove(this, &url, getClusterCacheLocal(&url, (char *)hostname), CACHE_FRAG_TYPE_HTTP);
+
+  HttpCacheKey key;
+  Cache::generate_key(&key, &url); // XXX choose a cache generation number ...
+
+  cacheProcessor.remove(this, &key, getClusterCacheLocal(&url), CACHE_FRAG_TYPE_HTTP);
   return EVENT_DONE;
 }
 

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/7ff2d618/iocore/cache/I_Cache.h
----------------------------------------------------------------------
diff --git a/iocore/cache/I_Cache.h b/iocore/cache/I_Cache.h
index 50b3185..77aafe9 100644
--- a/iocore/cache/I_Cache.h
+++ b/iocore/cache/I_Cache.h
@@ -76,25 +76,27 @@ struct CacheProcessor : public Processor {
   int dir_check(bool fix);
   int db_check(bool fix);
 
-  inkcoreapi Action *lookup(Continuation *cont, CacheKey *key, bool cluster_cache_local, bool local_only = false,
-                            CacheFragType frag_type = CACHE_FRAG_TYPE_NONE, char *hostname = 0, int host_len = 0);
-  inkcoreapi Action *open_read(Continuation *cont, CacheKey *key, bool cluster_cache_local,
-                               CacheFragType frag_type = CACHE_FRAG_TYPE_NONE, char *hostname = 0, int host_len = 0);
+  inkcoreapi Action *lookup(Continuation *cont, const CacheKey *key, bool cluster_cache_local, bool local_only = false,
+                            CacheFragType frag_type = CACHE_FRAG_TYPE_NONE, const char *hostname = 0, int host_len = 0);
+  inkcoreapi Action *open_read(Continuation *cont, const CacheKey *key, bool cluster_cache_local,
+                               CacheFragType frag_type = CACHE_FRAG_TYPE_NONE, const char *hostname = 0, int host_len = 0);
   inkcoreapi Action *open_write(Continuation *cont, CacheKey *key, bool cluster_cache_local,
                                 CacheFragType frag_type = CACHE_FRAG_TYPE_NONE, int expected_size = CACHE_EXPECTED_SIZE,
                                 int options = 0, time_t pin_in_cache = (time_t)0, char *hostname = 0, int host_len = 0);
-  inkcoreapi Action *remove(Continuation *cont, CacheKey *key, bool cluster_cache_local,
+  inkcoreapi Action *remove(Continuation *cont, const CacheKey *key, bool cluster_cache_local,
                             CacheFragType frag_type = CACHE_FRAG_TYPE_NONE, const char *hostname = 0, int host_len = 0);
   Action *scan(Continuation *cont, char *hostname = 0, int host_len = 0, int KB_per_second = SCAN_KB_PER_SECOND);
 #ifdef HTTP_CACHE
-  Action *lookup(Continuation *cont, CacheURL *url, bool cluster_cache_local, bool local_only = false,
+  Action *lookup(Continuation *cont, const HttpCacheKey *key, bool cluster_cache_local, bool local_only = false,
                  CacheFragType frag_type = CACHE_FRAG_TYPE_HTTP);
-  inkcoreapi Action *open_read(Continuation *cont, CacheURL *url, bool cluster_cache_local, CacheHTTPHdr *request,
+  inkcoreapi Action *open_read(Continuation *cont, const HttpCacheKey *key, bool cluster_cache_local, CacheHTTPHdr *request,
                                CacheLookupHttpConfig *params, time_t pin_in_cache = (time_t)0,
                                CacheFragType frag_type = CACHE_FRAG_TYPE_HTTP);
-  Action *open_write(Continuation *cont, int expected_size, CacheURL *url, bool cluster_cache_local, CacheHTTPHdr *request,
-                     CacheHTTPInfo *old_info, time_t pin_in_cache = (time_t)0, CacheFragType frag_type = CACHE_FRAG_TYPE_HTTP);
-  Action *remove(Continuation *cont, CacheURL *url, bool cluster_cache_local, CacheFragType frag_type = CACHE_FRAG_TYPE_HTTP);
+  Action *open_write(Continuation *cont, int expected_size, const HttpCacheKey *key, bool cluster_cache_local,
+                     CacheHTTPHdr *request, CacheHTTPInfo *old_info, time_t pin_in_cache = (time_t)0,
+                     CacheFragType frag_type = CACHE_FRAG_TYPE_HTTP);
+  Action *remove(Continuation *cont, const HttpCacheKey *key, bool cluster_cache_local,
+                 CacheFragType frag_type = CACHE_FRAG_TYPE_HTTP);
 #endif
   Action *link(Continuation *cont, CacheKey *from, CacheKey *to, bool cluster_cache_local,
                CacheFragType frag_type = CACHE_FRAG_TYPE_HTTP, char *hostname = 0, int host_len = 0);

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/7ff2d618/iocore/cache/I_CacheDefs.h
----------------------------------------------------------------------
diff --git a/iocore/cache/I_CacheDefs.h b/iocore/cache/I_CacheDefs.h
index 93b0cac..941ff0e 100644
--- a/iocore/cache/I_CacheDefs.h
+++ b/iocore/cache/I_CacheDefs.h
@@ -117,6 +117,24 @@ enum CacheFragType {
 };
 
 typedef CryptoHash CacheKey;
+
+struct HttpCacheKey {
+  uint64_t
+  slice64(int i) const
+  {
+    return hash.slice64(i);
+  }
+  uint32_t
+  slice32(int i) const
+  {
+    return hash.slice32(i);
+  }
+
+  int hostlen;
+  const char *hostname;
+  CacheKey hash;
+};
+
 #define CACHE_ALLOW_MULTIPLE_WRITES 1
 #define CACHE_EXPECTED_SIZE 32768
 

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/7ff2d618/iocore/cache/P_CacheInternal.h
----------------------------------------------------------------------
diff --git a/iocore/cache/P_CacheInternal.h b/iocore/cache/P_CacheInternal.h
index 96d08a3..7007bc2 100644
--- a/iocore/cache/P_CacheInternal.h
+++ b/iocore/cache/P_CacheInternal.h
@@ -1033,16 +1033,13 @@ struct Cache {
   Action *scan(Continuation *cont, const char *hostname = 0, int host_len = 0, int KB_per_second = 2500);
 
 #ifdef HTTP_CACHE
-  Action *lookup(Continuation *cont, CacheURL *url, CacheFragType type);
-  inkcoreapi Action *open_read(Continuation *cont, const CacheKey *key, CacheHTTPHdr *request, CacheLookupHttpConfig *params,
-                               CacheFragType type, const char *hostname, int host_len);
-  Action *open_read(Continuation *cont, CacheURL *url, CacheHTTPHdr *request, CacheLookupHttpConfig *params, CacheFragType type);
+  Action *open_read(Continuation *cont, const CacheKey *key, CacheHTTPHdr *request, CacheLookupHttpConfig *params,
+                    CacheFragType type, const char *hostname, int host_len);
   Action *open_write(Continuation *cont, const CacheKey *key, CacheHTTPInfo *old_info, time_t pin_in_cache = (time_t)0,
                      const CacheKey *key1 = NULL, CacheFragType type = CACHE_FRAG_TYPE_HTTP, const char *hostname = 0,
                      int host_len = 0);
-  Action *open_write(Continuation *cont, CacheURL *url, CacheHTTPHdr *request, CacheHTTPInfo *old_info,
-                     time_t pin_in_cache = (time_t)0, CacheFragType type = CACHE_FRAG_TYPE_HTTP);
   static void generate_key(INK_MD5 *md5, CacheURL *url);
+  static void generate_key(HttpCacheKey *md5, CacheURL *url, cache_generation_t generation = -1);
 #endif
 
   Action *link(Continuation *cont, const CacheKey *from, const CacheKey *to, CacheFragType type, const char *hostname,
@@ -1067,15 +1064,6 @@ extern Cache *theStreamCache;
 inkcoreapi extern Cache *caches[NUM_CACHE_FRAG_TYPES];
 
 #ifdef HTTP_CACHE
-TS_INLINE Action *
-Cache::open_read(Continuation *cont, CacheURL *url, CacheHTTPHdr *request, CacheLookupHttpConfig *params, CacheFragType type)
-{
-  INK_MD5 md5;
-  int len;
-  url->hash_get(&md5);
-  const char *hostname = url->host_get(&len);
-  return open_read(cont, &md5, request, params, type, hostname, len);
-}
 
 TS_INLINE void
 Cache::generate_key(INK_MD5 *md5, CacheURL *url)
@@ -1083,22 +1071,17 @@ Cache::generate_key(INK_MD5 *md5, CacheURL *url)
   url->hash_get(md5);
 }
 
-TS_INLINE Action *
-Cache::open_write(Continuation *cont, CacheURL *url, CacheHTTPHdr *request, CacheHTTPInfo *old_info, time_t pin_in_cache,
-                  CacheFragType type)
+TS_INLINE void
+Cache::generate_key(HttpCacheKey *key, CacheURL *url, cache_generation_t generation)
 {
-  (void)request;
-  INK_MD5 url_md5;
-  url->hash_get(&url_md5);
-  int len;
-  const char *hostname = url->host_get(&len);
-
-  return open_write(cont, &url_md5, old_info, pin_in_cache, NULL, type, hostname, len);
+  key->hostname = url->host_get(&key->hostlen);
+  url->hash_get(&key->hash, generation);
 }
+
 #endif
 
 TS_INLINE unsigned int
-cache_hash(INK_MD5 &md5)
+cache_hash(const INK_MD5 &md5)
 {
   uint64_t f = md5.fold();
   unsigned int mhash = (unsigned int)(f >> 32);

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/7ff2d618/iocore/cluster/ClusterCache.cc
----------------------------------------------------------------------
diff --git a/iocore/cluster/ClusterCache.cc b/iocore/cluster/ClusterCache.cc
index 6c8d424..c4480e1 100644
--- a/iocore/cluster/ClusterCache.cc
+++ b/iocore/cluster/ClusterCache.cc
@@ -2387,8 +2387,8 @@ CacheContinuation::lookupEvent(int /* event ATS_UNUSED */, void * /* d ATS_UNUSE
 //            Zero (Action *) if no probe
 //////////////////////////////////////////////////////////////////////////
 Action *
-CacheContinuation::do_remote_lookup(Continuation *cont, CacheKey *key, CacheContinuation *c, CacheFragType ft, char *hostname,
-                                    int hostname_len)
+CacheContinuation::do_remote_lookup(Continuation *cont, const CacheKey *key, CacheContinuation *c, CacheFragType ft,
+                                    const char *hostname, int hostname_len)
 {
   int probe_depth = 0;
   ClusterMachine *past_probes[CONFIGURATION_HISTORY_PROBE_DEPTH] = {0};

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/7ff2d618/iocore/cluster/P_ClusterCacheInternal.h
----------------------------------------------------------------------
diff --git a/iocore/cluster/P_ClusterCacheInternal.h b/iocore/cluster/P_ClusterCacheInternal.h
index eb7d5cd..bba6f17 100644
--- a/iocore/cluster/P_ClusterCacheInternal.h
+++ b/iocore/cluster/P_ClusterCacheInternal.h
@@ -309,7 +309,7 @@ struct CacheContinuation : public Continuation {
   static CacheContinuation *cacheContAllocator_alloc();
   static void cacheContAllocator_free(CacheContinuation *);
   inkcoreapi static Action *callback_failure(Action *, int, int, CacheContinuation *this_cc = 0);
-  static Action *do_remote_lookup(Continuation *, CacheKey *, CacheContinuation *, CacheFragType, char *, int);
+  static Action *do_remote_lookup(Continuation *, const CacheKey *, CacheContinuation *, CacheFragType, const char *, int);
   inkcoreapi static Action *do_op(Continuation *, ClusterMachine *, void *, int, char *, int, int nbytes = -1, MIOBuffer *b = 0);
   static int setup_local_vc(char *data, int data_len, CacheContinuation *cc, ClusterMachine *mp, Action **);
   static void disposeOfDataBuffer(void *buf);
@@ -331,7 +331,7 @@ struct CacheContinuation : public Continuation {
 #define CFL_MAX (1 << 15)
 
 struct CacheOpArgs_General {
-  INK_MD5 *url_md5;
+  const INK_MD5 *url_md5;
   time_t pin_in_cache; // open_write() specific arg
   CacheFragType frag_type;
   uint16_t cfl_flags;

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/7ff2d618/iocore/cluster/P_ClusterInline.h
----------------------------------------------------------------------
diff --git a/iocore/cluster/P_ClusterInline.h b/iocore/cluster/P_ClusterInline.h
index ed97c36..18aa72d 100644
--- a/iocore/cluster/P_ClusterInline.h
+++ b/iocore/cluster/P_ClusterInline.h
@@ -33,7 +33,7 @@
 #include "P_ClusterHandler.h"
 
 inline Action *
-Cluster_lookup(Continuation *cont, CacheKey *key, CacheFragType frag_type, char *hostname, int host_len)
+Cluster_lookup(Continuation *cont, const CacheKey *key, CacheFragType frag_type, const char *hostname, int host_len)
 {
   // Try to send remote, if not possible, handle locally
   Action *retAct;
@@ -59,9 +59,9 @@ Cluster_lookup(Continuation *cont, CacheKey *key, CacheFragType frag_type, char
 }
 
 inline Action *
-Cluster_read(ClusterMachine *owner_machine, int opcode, Continuation *cont, MIOBuffer *buf, CacheURL *url, CacheHTTPHdr *request,
-             CacheLookupHttpConfig *params, CacheKey *key, time_t pin_in_cache, CacheFragType frag_type, char *hostname,
-             int host_len)
+Cluster_read(ClusterMachine *owner_machine, int opcode, Continuation *cont, MIOBuffer *buf, CacheHTTPHdr *request,
+             CacheLookupHttpConfig *params, const CacheKey *key, time_t pin_in_cache, CacheFragType frag_type, const char *hostname,
+             int hostlen)
 {
   (void)params;
   if (clusterProcessor.disable_remote_cluster_ops(owner_machine)) {
@@ -78,19 +78,15 @@ Cluster_read(ClusterMachine *owner_machine, int opcode, Continuation *cont, MIOB
 
   if (vers == CacheOpMsg_long::CACHE_OP_LONG_MESSAGE_VERSION) {
     if ((opcode == CACHE_OPEN_READ_LONG) || (opcode == CACHE_OPEN_READ_BUFFER_LONG)) {
+      ink_assert(hostname);
+      ink_assert(hostlen);
+
       // Determine length of data to Marshal
       flen = op_to_sizeof_fixedlen_msg(opcode);
 
-      const char *url_hostname;
-      int url_hlen;
-      INK_MD5 url_only_md5;
-
-      Cache::generate_key(&url_only_md5, url);
-      url_hostname = url->host_get(&url_hlen);
-
       len += request->m_heap->marshal_length();
       len += params->marshal_length();
-      len += url_hlen;
+      len += hostlen;
 
       if ((flen + len) > DEFAULT_MAX_BUFFER_SIZE) // Bound marshalled data
         goto err_exit;
@@ -110,27 +106,27 @@ Cluster_read(ClusterMachine *owner_machine, int opcode, Continuation *cont, MIOB
       if ((res = params->marshal(data, cur_len)) < 0)
         goto err_exit;
       data += res;
-      memcpy(data, url_hostname, url_hlen);
+      memcpy(data, hostname, hostlen);
 
       CacheOpArgs_General readArgs;
-      readArgs.url_md5 = &url_only_md5;
+      readArgs.url_md5 = key;
       readArgs.pin_in_cache = pin_in_cache;
       readArgs.frag_type = frag_type;
       return CacheContinuation::do_op(cont, owner_machine, (void *)&readArgs, opcode, (char *)msg, (flen + len), -1, buf);
     } else {
       // Build message if we have host data.
 
-      if (host_len) {
+      if (hostlen) {
         // Determine length of data to Marshal
         flen = op_to_sizeof_fixedlen_msg(opcode);
-        len = host_len;
+        len = hostlen;
 
         if ((flen + len) > DEFAULT_MAX_BUFFER_SIZE) // Bound marshalled data
           goto err_exit;
 
         msg = (char *)ALLOCA_DOUBLE(flen + len);
         data = msg + flen;
-        memcpy(data, hostname, host_len);
+        memcpy(data, hostname, hostlen);
 
       } else {
         msg = 0;
@@ -156,11 +152,10 @@ err_exit:
 }
 
 inline Action *
-Cluster_write(Continuation *cont, int expected_size, MIOBuffer *buf, ClusterMachine *m, INK_MD5 *url_md5, CacheFragType ft,
-              int options, time_t pin_in_cache, int opcode, CacheKey *key, CacheURL *url, CacheHTTPHdr *request,
-              CacheHTTPInfo *old_info, char *hostname, int host_len)
+Cluster_write(Continuation *cont, int expected_size, MIOBuffer *buf, ClusterMachine *m, const CacheKey *url_md5, CacheFragType ft,
+              int options, time_t pin_in_cache, int opcode, CacheHTTPHdr *request, CacheHTTPInfo *old_info, const char *hostname,
+              int hostlen)
 {
-  (void)key;
   (void)request;
   if (clusterProcessor.disable_remote_cluster_ops(m)) {
     Action a;
@@ -177,10 +172,10 @@ Cluster_write(Continuation *cont, int expected_size, MIOBuffer *buf, ClusterMach
   switch (opcode) {
   case CACHE_OPEN_WRITE: {
     // Build message if we have host data
-    if (host_len) {
+    if (hostlen) {
       // Determine length of data to Marshal
       flen = op_to_sizeof_fixedlen_msg(CACHE_OPEN_WRITE);
-      len = host_len;
+      len = hostlen;
 
       if ((flen + len) > DEFAULT_MAX_BUFFER_SIZE) // Bound marshalled data
         goto err_exit;
@@ -188,13 +183,13 @@ Cluster_write(Continuation *cont, int expected_size, MIOBuffer *buf, ClusterMach
       msg = (char *)ALLOCA_DOUBLE(flen + len);
       data = msg + flen;
 
-      memcpy(data, hostname, host_len);
+      memcpy(data, hostname, hostlen);
     }
     break;
   }
   case CACHE_OPEN_WRITE_LONG: {
-    int url_hlen;
-    const char *url_hostname = url->host_get(&url_hlen);
+    ink_assert(hostname);
+    ink_assert(hostlen);
 
     // Determine length of data to Marshal
     flen = op_to_sizeof_fixedlen_msg(CACHE_OPEN_WRITE_LONG);
@@ -207,7 +202,7 @@ Cluster_write(Continuation *cont, int expected_size, MIOBuffer *buf, ClusterMach
     if (old_info) {
       len += old_info->marshal_length();
     }
-    len += url_hlen;
+    len += hostlen;
 
     if ((flen + len) > DEFAULT_MAX_BUFFER_SIZE) // Bound marshalled data
       goto err_exit;
@@ -224,7 +219,7 @@ Cluster_write(Continuation *cont, int expected_size, MIOBuffer *buf, ClusterMach
       }
       data += res;
     }
-    memcpy(data, url_hostname, url_hlen);
+    memcpy(data, hostname, hostlen);
     break;
   }
   default: {
@@ -343,7 +338,7 @@ err_exit:
 }
 
 inline Action *
-Cluster_remove(ClusterMachine *m, Continuation *cont, CacheKey *key, CacheFragType frag_type, const char *hostname, int host_len)
+Cluster_remove(ClusterMachine *m, Continuation *cont, const CacheKey *key, CacheFragType frag_type, const char *hostname, int host_len)
 {
   if (clusterProcessor.disable_remote_cluster_ops(m)) {
     Action a;

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/7ff2d618/proxy/CacheControl.cc
----------------------------------------------------------------------
diff --git a/proxy/CacheControl.cc b/proxy/CacheControl.cc
index 44a7fbc..77af5fe 100644
--- a/proxy/CacheControl.cc
+++ b/proxy/CacheControl.cc
@@ -177,7 +177,7 @@ getCacheControl(CacheControlResult *result, HttpRequestData *rdata, OverridableH
 }
 
 bool
-getClusterCacheLocal(URL *url, char * /* hostname ATS_UNUSED */)
+getClusterCacheLocal(URL *url)
 {
   HttpRequestData rdata;
   CacheControlResult result;

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/7ff2d618/proxy/CacheControl.h
----------------------------------------------------------------------
diff --git a/proxy/CacheControl.h b/proxy/CacheControl.h
index 2668443..de3820c 100644
--- a/proxy/CacheControl.h
+++ b/proxy/CacheControl.h
@@ -132,7 +132,7 @@ struct OverridableHttpConfigParams;
 
 inkcoreapi void getCacheControl(CacheControlResult *result, HttpRequestData *rdata, OverridableHttpConfigParams *h_txn_conf,
                                 char *tag = NULL);
-inkcoreapi bool getClusterCacheLocal(URL *url, char *hostname);
+inkcoreapi bool getClusterCacheLocal(URL *url);
 inkcoreapi bool host_rule_in_CacheControlTable();
 inkcoreapi bool ip_rule_in_CacheControlTable();
 

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/7ff2d618/proxy/ICP.cc
----------------------------------------------------------------------
diff --git a/proxy/ICP.cc b/proxy/ICP.cc
index af3d697..b3affe4 100644
--- a/proxy/ICP.cc
+++ b/proxy/ICP.cc
@@ -453,16 +453,20 @@ ICPPeerReadCont::ICPPeerQueryCont(int /* event ATS_UNUSED */, Event * /* e ATS_U
 
   SET_HANDLER((ICPPeerReadContHandler)&ICPPeerReadCont::ICPPeerQueryEvent);
   if (_state->_rICPmsg->un.query.URL && *_state->_rICPmsg->un.query.URL) {
+    HttpCacheKey key;
+
+    Cache::generate_key(&key, &_state->_cachelookupURL); // XXX choose a cache generation number ...
     _state->_queryResult = ~CACHE_EVENT_LOOKUP_FAILED;
     _start_time = ink_get_hrtime();
+
     if (pluginFreshnessCalcFunc && _ICPpr->GetConfig()->globalConfig()->ICPStaleLookup()) {
       //////////////////////////////////////////////////////////////
       // Note: _cache_lookup_local is ignored in this case, since
       //       cache clustering is not used with stale lookup.
       //////////////////////////////////////////////////////////////
-      a = cacheProcessor.open_read(this, &_state->_cachelookupURL, false, &gclient_request, &global_cache_lookup_config, (time_t)0);
+      a = cacheProcessor.open_read(this, &key, false, &gclient_request, &global_cache_lookup_config, (time_t)0);
     } else {
-      a = cacheProcessor.lookup(this, &_state->_cachelookupURL, false, _state->_cache_lookup_local);
+      a = cacheProcessor.lookup(this, &key, false, _state->_cache_lookup_local);
     }
     if (!a) {
       a = ACTION_IO_ERROR;

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/7ff2d618/proxy/Prefetch.cc
----------------------------------------------------------------------
diff --git a/proxy/Prefetch.cc b/proxy/Prefetch.cc
index 18393ee..521aa83 100644
--- a/proxy/Prefetch.cc
+++ b/proxy/Prefetch.cc
@@ -1446,12 +1446,16 @@ PrefetchBlaster::handleEvent(int event, void *data)
 
   switch (event) {
   case EVENT_IMMEDIATE: {
+    HttpCacheKey key;
+
     // Here, we need to decide if we need to prefetch based on whether it
     // is in the cache or not.
 
     // if (cache_lookup_necessary) do:
     initCacheLookupConfig();
-    cacheProcessor.open_read(this, request->url_get(), false, request, &cache_lookup_config, 0);
+
+    Cache::generate_key(&key, request->url_get()); // XXX choose a cache generation number ...
+    cacheProcessor.open_read(this, &key, false, request, &cache_lookup_config, 0);
 
     break;
   }

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/7ff2d618/proxy/http/HttpCacheSM.cc
----------------------------------------------------------------------
diff --git a/proxy/http/HttpCacheSM.cc b/proxy/http/HttpCacheSM.cc
index ff02e3a..c259b38 100644
--- a/proxy/http/HttpCacheSM.cc
+++ b/proxy/http/HttpCacheSM.cc
@@ -145,7 +145,7 @@ HttpCacheSM::state_cache_open_read(int event, void *data)
                         "retrying cache open read...",
           master_sm->sm_id, open_read_tries);
 
-    do_cache_open_read();
+    do_cache_open_read(cache_key);
     break;
 
   default:
@@ -200,7 +200,7 @@ HttpCacheSM::do_schedule_in()
 }
 
 Action *
-HttpCacheSM::do_cache_open_read()
+HttpCacheSM::do_cache_open_read(const HttpCacheKey &key)
 {
   open_read_tries++;
   ink_assert(pending_action == NULL);
@@ -211,7 +211,7 @@ HttpCacheSM::do_cache_open_read()
   }
   // Initialising read-while-write-inprogress flag
   this->readwhilewrite_inprogress = false;
-  Action *action_handle = cacheProcessor.open_read(this, this->lookup_url, master_sm->t_state.cache_control.cluster_cache_local,
+  Action *action_handle = cacheProcessor.open_read(this, &key, master_sm->t_state.cache_control.cluster_cache_local,
                                                    this->read_request_hdr, this->read_config, this->read_pin_in_cache);
 
   if (action_handle != ACTION_RESULT_DONE) {
@@ -230,10 +230,11 @@ HttpCacheSM::do_cache_open_read()
 }
 
 Action *
-HttpCacheSM::open_read(URL *url, HTTPHdr *hdr, CacheLookupHttpConfig *params, time_t pin_in_cache)
+HttpCacheSM::open_read(const HttpCacheKey *key, URL *url, HTTPHdr *hdr, CacheLookupHttpConfig *params, time_t pin_in_cache)
 {
   Action *act_return;
 
+  cache_key = *key;
   lookup_url = url;
   read_request_hdr = hdr;
   read_config = params;
@@ -244,7 +245,7 @@ HttpCacheSM::open_read(URL *url, HTTPHdr *hdr, CacheLookupHttpConfig *params, ti
   lookup_max_recursive++;
   current_lookup_level++;
   open_read_cb = false;
-  act_return = do_cache_open_read();
+  act_return = do_cache_open_read(cache_key);
   // the following logic is based on the assumption that the secnod
   // lookup won't happen if the HttpSM hasn't been called back for the
   // first lookup
@@ -267,7 +268,8 @@ HttpCacheSM::open_read(URL *url, HTTPHdr *hdr, CacheLookupHttpConfig *params, ti
 }
 
 Action *
-HttpCacheSM::open_write(URL *url, HTTPHdr *request, CacheHTTPInfo *old_info, time_t pin_in_cache, bool retry, bool allow_multiple)
+HttpCacheSM::open_write(const HttpCacheKey *key, URL *url, HTTPHdr *request, CacheHTTPInfo *old_info, time_t pin_in_cache,
+                        bool retry, bool allow_multiple)
 {
   SET_HANDLER(&HttpCacheSM::state_cache_open_write);
   ink_assert(pending_action == NULL);
@@ -297,7 +299,7 @@ HttpCacheSM::open_write(URL *url, HTTPHdr *request, CacheHTTPInfo *old_info, tim
   }
 
   Action *action_handle =
-    cacheProcessor.open_write(this, 0, url, master_sm->t_state.cache_control.cluster_cache_local, request,
+    cacheProcessor.open_write(this, 0, key, master_sm->t_state.cache_control.cluster_cache_local, request,
                               // INKqa11166
                               allow_multiple ? (CacheHTTPInfo *)CACHE_ALLOW_MULTIPLE_WRITES : old_info, pin_in_cache);
 

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/7ff2d618/proxy/http/HttpCacheSM.h
----------------------------------------------------------------------
diff --git a/proxy/http/HttpCacheSM.h b/proxy/http/HttpCacheSM.h
index d0ed2ad..3c14da3 100644
--- a/proxy/http/HttpCacheSM.h
+++ b/proxy/http/HttpCacheSM.h
@@ -68,9 +68,10 @@ public:
     captive_action.init(this);
   }
 
-  Action *open_read(URL *url, HTTPHdr *hdr, CacheLookupHttpConfig *params, time_t pin_in_cache);
+  Action *open_read(const HttpCacheKey *key, URL *url, HTTPHdr *hdr, CacheLookupHttpConfig *params, time_t pin_in_cache);
 
-  Action *open_write(URL *url, HTTPHdr *request, CacheHTTPInfo *old_info, time_t pin_in_cache, bool retry, bool allow_multiple);
+  Action *open_write(const HttpCacheKey *key, URL *url, HTTPHdr *request, CacheHTTPInfo *old_info, time_t pin_in_cache, bool retry,
+                     bool allow_multiple);
 
   CacheVConnection *cache_read_vc;
   CacheVConnection *cache_write_vc;
@@ -144,7 +145,7 @@ public:
 
 private:
   void do_schedule_in();
-  Action *do_cache_open_read();
+  Action *do_cache_open_read(const HttpCacheKey &);
 
   int state_cache_open_read(int event, void *data);
   int state_cache_open_write(int event, void *data);
@@ -165,6 +166,7 @@ private:
 
   // Common parameters
   URL *lookup_url;
+  HttpCacheKey cache_key;
 
   // to keep track of multiple cache lookups
   int lookup_max_recursive;

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/7ff2d618/proxy/http/HttpSM.cc
----------------------------------------------------------------------
diff --git a/proxy/http/HttpSM.cc b/proxy/http/HttpSM.cc
index b913bff..96030a3 100644
--- a/proxy/http/HttpSM.cc
+++ b/proxy/http/HttpSM.cc
@@ -4342,8 +4342,12 @@ HttpSM::do_cache_lookup_and_read()
 
   DebugSM("http_seq", "[HttpSM::do_cache_lookup_and_read] [%" PRId64 "] Issuing cache lookup for URL %s", sm_id,
           c_url->string_get(&t_state.arena));
+
+  HttpCacheKey key;
+  Cache::generate_key(&key, c_url, t_state.txn_conf->cache_generation_number);
+
   Action *cache_action_handle =
-    cache_sm.open_read(c_url, &t_state.hdr_info.client_request, &(t_state.cache_info.config),
+    cache_sm.open_read(&key, c_url, &t_state.hdr_info.client_request, &(t_state.cache_info.config),
                        (time_t)((t_state.cache_control.pin_in_cache_for < 0) ? 0 : t_state.cache_control.pin_in_cache_for));
   //
   // pin_in_cache value is an open_write parameter.
@@ -4372,7 +4376,9 @@ HttpSM::do_cache_delete_all_alts(Continuation *cont)
 
   Action *cache_action_handle = NULL;
 
-  cache_action_handle = cacheProcessor.remove(cont, t_state.cache_info.lookup_url, t_state.cache_control.cluster_cache_local);
+  HttpCacheKey key;
+  Cache::generate_key(&key, t_state.cache_info.lookup_url, t_state.txn_conf->cache_generation_number);
+  cache_action_handle = cacheProcessor.remove(cont, &key, t_state.cache_control.cluster_cache_local);
   if (cont != NULL) {
     if (cache_action_handle != ACTION_RESULT_DONE) {
       ink_assert(!pending_action);
@@ -4463,8 +4469,12 @@ HttpSM::do_cache_prepare_action(HttpCacheSM *c_sm, CacheHTTPInfo *object_read_in
 
   ink_assert(s_url != NULL && s_url->valid());
   DebugSM("http_cache_write", "[%" PRId64 "] writing to cache with URL %s", sm_id, s_url->string_get(&t_state.arena));
+
+  HttpCacheKey key;
+  Cache::generate_key(&key, s_url, t_state.txn_conf->cache_generation_number);
+
   Action *cache_action_handle = c_sm->open_write(
-    s_url, &t_state.hdr_info.client_request, object_read_info,
+    &key, s_url, &t_state.hdr_info.client_request, object_read_info,
     (time_t)((t_state.cache_control.pin_in_cache_for < 0) ? 0 : t_state.cache_control.pin_in_cache_for), retry, allow_multiple);
 
   if (cache_action_handle != ACTION_RESULT_DONE) {


[3/6] trafficserver git commit: TS-3580: Apply the cache generation number in Cache Pages

Posted by jp...@apache.org.
TS-3580: Apply the cache generation number in Cache Pages


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

Branch: refs/heads/master
Commit: 4d7d892febbf475312ab4517b884426c2c70f589
Parents: 9f71421
Author: James Peach <jp...@apache.org>
Authored: Mon May 4 20:25:23 2015 -0700
Committer: James Peach <jp...@apache.org>
Committed: Mon Jun 15 09:16:00 2015 -0700

----------------------------------------------------------------------
 iocore/cache/CachePages.cc | 6 ++++--
 1 file changed, 4 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/trafficserver/blob/4d7d892f/iocore/cache/CachePages.cc
----------------------------------------------------------------------
diff --git a/iocore/cache/CachePages.cc b/iocore/cache/CachePages.cc
index 721bffc..b850f0b 100644
--- a/iocore/cache/CachePages.cc
+++ b/iocore/cache/CachePages.cc
@@ -431,6 +431,8 @@ int
 ShowCache::lookup_url(int event, Event *e)
 {
   char header_str[300];
+  HttpCacheKey key;
+  cache_generation_t generation = -1;
 
   snprintf(header_str, sizeof(header_str), "<font color=red>%s</font>", show_cache_urlstrs[0]);
   CHECK_SHOW(begin(header_str));
@@ -439,8 +441,8 @@ ShowCache::lookup_url(int event, Event *e)
   s = show_cache_urlstrs[0];
   url.parse(&s, s + strlen(s));
 
-  HttpCacheKey key;
-  Cache::generate_key(&key, &url); // XXX choose a cache generation number ...
+  RecGetRecordInt("proxy.config.http.cache.generation", &generation);
+  Cache::generate_key(&key, &url, generation);
 
   SET_HANDLER(&ShowCache::handleCacheEvent);
   Action *lookup_result =