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 2010/05/06 21:58:28 UTC

svn commit: r941885 - in /trafficserver/traffic/trunk: ./ iocore/cache/ iocore/eventsystem/ libinktomi++/ m4/ proxy/

Author: jplevyak
Date: Thu May  6 19:58:27 2010
New Revision: 941885

URL: http://svn.apache.org/viewvc?rev=941885&view=rev
Log:
TS-120, TS-121, TS-337: configurable RamCache algorith and new RamCache
algorithm: CLUFS which is both frequency and size sensitive and
which supports 3 levels of compression.

Added:
    trafficserver/traffic/trunk/iocore/cache/RamCacheCLFUS.cc
    trafficserver/traffic/trunk/iocore/cache/RamCacheLRU.cc
    trafficserver/traffic/trunk/libinktomi++/fastlz.c
    trafficserver/traffic/trunk/libinktomi++/fastlz.h
    trafficserver/traffic/trunk/m4/ax_check_zlib.m4
Removed:
    trafficserver/traffic/trunk/iocore/cache/RamCache.cc
Modified:
    trafficserver/traffic/trunk/configure.ac
    trafficserver/traffic/trunk/iocore/cache/Cache.cc
    trafficserver/traffic/trunk/iocore/cache/CacheDir.cc
    trafficserver/traffic/trunk/iocore/cache/CachePart.cc
    trafficserver/traffic/trunk/iocore/cache/CacheWrite.cc
    trafficserver/traffic/trunk/iocore/cache/I_Cache.h
    trafficserver/traffic/trunk/iocore/cache/Makefile.am
    trafficserver/traffic/trunk/iocore/cache/P_CacheInternal.h
    trafficserver/traffic/trunk/iocore/cache/P_CachePart.h
    trafficserver/traffic/trunk/iocore/cache/P_RamCache.h
    trafficserver/traffic/trunk/iocore/eventsystem/I_EThread.h
    trafficserver/traffic/trunk/libinktomi++/Makefile.am
    trafficserver/traffic/trunk/libinktomi++/inktomi++.h
    trafficserver/traffic/trunk/proxy/signals.cc

Modified: trafficserver/traffic/trunk/configure.ac
URL: http://svn.apache.org/viewvc/trafficserver/traffic/trunk/configure.ac?rev=941885&r1=941884&r2=941885&view=diff
==============================================================================
--- trafficserver/traffic/trunk/configure.ac (original)
+++ trafficserver/traffic/trunk/configure.ac Thu May  6 19:58:27 2010
@@ -582,6 +582,8 @@ AC_CHECK_LIB([posix4],[clock_gettime],[A
 AC_CHECK_LIB([iconv],[iconv_open],[AC_SUBST([LIBICONV],["-liconv"])])
 AC_CHECK_LIB([iconv],[libiconv_open],[AC_SUBST([LIBICONV],["-liconv"])])
 AC_CHECK_LIB([ev],[ev_sleep],[AC_SUBST([LIBEV],["-lev"])])
+AC_CHECK_LIB([ev],[ev_sleep],[AC_SUBST([LIBEV],["-lev"])])
+AX_CHECK_ZLIB()
 
 AC_CHECK_LIB([ssl],[SSL_CTX_new],
   [AC_SUBST([LIBSSL],["-lssl -lcrypto"])
@@ -591,6 +593,14 @@ AC_CHECK_LIB([ssl],[SSL_CTX_new],
   [-lcrypto]
 )
 
+AC_CHECK_HEADER(lzma.h,have_lzma_hdr=yes)
+AC_CHECK_LIB(lzma,lzma_code, have_lzma_lib=yes)
+if test x$have_lzma_hdr == xyes -a x$have_lzma_lib == xyes ; then
+  have_lzma=yes
+  LIBS="$LIBS -llzma"
+  AC_DEFINE(HAVE_LZMA,1,[Compiling with LZMA support])
+fi
+
 #
 # Tcl macros provided by m4/tcl.m4
 #

Modified: trafficserver/traffic/trunk/iocore/cache/Cache.cc
URL: http://svn.apache.org/viewvc/trafficserver/traffic/trunk/iocore/cache/Cache.cc?rev=941885&r1=941884&r2=941885&view=diff
==============================================================================
--- trafficserver/traffic/trunk/iocore/cache/Cache.cc (original)
+++ trafficserver/traffic/trunk/iocore/cache/Cache.cc Thu May  6 19:58:27 2010
@@ -53,6 +53,9 @@ do { \
 // Configuration
 
 ink64 cache_config_ram_cache_size = AUTO_SIZE_RAM_CACHE;
+int cache_config_ram_cache_algorithm = 0;
+int cache_config_ram_cache_compress = 0;
+int cache_config_ram_cache_compress_percent = 90;
 int cache_config_http_max_alts = 3;
 int cache_config_dir_sync_frequency = 60;
 int cache_config_permit_pinning = 0;
@@ -61,7 +64,7 @@ int cache_config_select_alternate = 1;
 int cache_config_max_doc_size = 0;
 int cache_config_min_average_object_size = ESTIMATED_OBJECT_SIZE;
 ink64 cache_config_ram_cache_cutoff = 1048576;  // 1 MB
-ink64 cache_config_ram_cache_mixt_cutoff = 1048576;     // 1 MB
+//ink64 cache_config_ram_cache_mixt_cutoff = 1048576;     // 1 MB
 int cache_config_max_disk_errors = 5;
 int cache_config_agg_write_backlog = 5242880;
 #ifdef HIT_EVACUATE
@@ -710,26 +713,24 @@ CacheProcessor::cacheInitialized()
   if (caches_ready) {
     Debug("cache_init", "CacheProcessor::cacheInitialized - caches_ready=0x%0lX, gnpart=%d",
           (unsigned long) caches_ready, gnpart);
+    ink64 ram_cache_bytes = 0;
     if (gnpart) {
-      ink64 ram_cache_bytes = 0;
-      ink32 ram_cache_object_size;
+      for (i = 0; i < gnpart; i++) {
+        switch (cache_config_ram_cache_algorithm) {
+          default: 
+          case RAM_CACHE_ALGORITHM_CLFUS:
+            gpart[i]->ram_cache = new_RamCacheCLFUS();
+            break;
+          case RAM_CACHE_ALGORITHM_LRU:
+            gpart[i]->ram_cache = new_RamCacheLRU();
+            break;
+        }
+      }
       if (cache_config_ram_cache_size == AUTO_SIZE_RAM_CACHE) {
         Debug("cache_init", "CacheProcessor::cacheInitialized - cache_config_ram_cache_size == AUTO_SIZE_RAM_CACHE");
         for (i = 0; i < gnpart; i++) {
           part = gpart[i];
-          if (gpart[i]->cache == theCache) {
-            ram_cache_object_size = ((cache_config_ram_cache_cutoff < cache_config_min_average_object_size) &&
-                                     cache_config_ram_cache_cutoff) ?
-              cache_config_ram_cache_cutoff : cache_config_min_average_object_size;
-            gpart[i]->ram_cache.init(part_dirlen(gpart[i]), part_dirlen(gpart[i]) / ram_cache_object_size,
-                                     cache_config_ram_cache_cutoff, gpart[i], gpart[i]->mutex);
-          } else {
-            ram_cache_object_size = ((cache_config_ram_cache_mixt_cutoff < cache_config_min_average_object_size) &&
-                                     cache_config_ram_cache_mixt_cutoff) ?
-              cache_config_ram_cache_mixt_cutoff : cache_config_min_average_object_size;
-            gpart[i]->ram_cache.init(part_dirlen(gpart[i]), part_dirlen(gpart[i]) / ram_cache_object_size,
-                                     cache_config_ram_cache_mixt_cutoff, gpart[i], gpart[i]->mutex);
-          }
+          gpart[i]->ram_cache->init(part_dirlen(part), part);
           ram_cache_bytes += part_dirlen(gpart[i]);
           Debug("cache_init", "CacheProcessor::cacheInitialized - ram_cache_bytes = %lld = %lldMb",
                 ram_cache_bytes, ram_cache_bytes / (1024 * 1024));
@@ -778,25 +779,15 @@ CacheProcessor::cacheInitialized()
           if (gpart[i]->cache == theCache) {
             factor = (double) (ink64) (gpart[i]->len >> STORE_BLOCK_SHIFT) / (ink64) theCache->cache_size;
             Debug("cache_init", "CacheProcessor::cacheInitialized - factor = %f", factor);
-            gpart[i]->ram_cache.init((ink64) (http_ram_cache_size * factor),
-                                     (ink64) ((http_ram_cache_size * factor) /
-                                              cache_config_min_average_object_size),
-                                     cache_config_ram_cache_cutoff, gpart[i], gpart[i]->mutex);
-
+            gpart[i]->ram_cache->init((ink64) (http_ram_cache_size * factor), part);
             ram_cache_bytes += (ink64) (http_ram_cache_size * factor);
             CACHE_PART_SUM_DYN_STAT(cache_ram_cache_bytes_total_stat, (ink64) (http_ram_cache_size * factor));
           } else {
             factor = (double) (ink64) (gpart[i]->len >> STORE_BLOCK_SHIFT) / (ink64) theStreamCache->cache_size;
             Debug("cache_init", "CacheProcessor::cacheInitialized - factor = %f", factor);
-            gpart[i]->ram_cache.init((ink64) (stream_ram_cache_size * factor),
-                                     (ink64) ((stream_ram_cache_size * factor) /
-                                              cache_config_min_average_object_size),
-                                     cache_config_ram_cache_mixt_cutoff, gpart[i], gpart[i]->mutex);
-
+            gpart[i]->ram_cache->init((ink64) (stream_ram_cache_size * factor), part);
             ram_cache_bytes += (ink64) (stream_ram_cache_size * factor);
             CACHE_PART_SUM_DYN_STAT(cache_ram_cache_bytes_total_stat, (ink64) (stream_ram_cache_size * factor));
-
-
           }
           Debug("cache_init", "CacheProcessor::cacheInitialized[%d] - ram_cache_bytes = %lld = %lldMb",
                 i, ram_cache_bytes, ram_cache_bytes / (1024 * 1024));
@@ -1768,6 +1759,21 @@ CacheVC::dead(int event, Event *e)
 
 #define STORE_COLLISION 1
 
+static void unmarshal_helper(Doc *doc, Ptr<IOBufferData> &buf, int &okay) {
+  char *tmp = doc->hdr();
+  int len = doc->hlen;
+  while (len > 0) {
+    int r = HTTPInfo::unmarshal(tmp, len, buf._ptr());
+    if (r < 0) {
+      ink_assert(!"CacheVC::handleReadDone unmarshal failed");
+      okay = 0;
+      break;
+    }
+    len -= r;
+    tmp += r;
+  }
+}
+
 int
 CacheVC::handleReadDone(int event, Event *e)
 {
@@ -1810,7 +1816,8 @@ CacheVC::handleReadDone(int event, Event
     if (io.ok() &&
         ((doc->first_key == *read_key) || (doc->key == *read_key) || STORE_COLLISION) && doc->magic == DOC_MAGIC) {
       int okay = 1;
-      f.not_from_ram_cache = 1;
+      if (!f.doc_from_ram_cache)
+        f.not_from_ram_cache = 1;
       if (cache_config_enable_checksum && doc->checksum != DOC_NO_CHECKSUM) {
         // verify that the checksum matches
         inku32 checksum = 0;
@@ -1825,23 +1832,14 @@ CacheVC::handleReadDone(int event, Event
           okay = 0;
         }
       }
-      // If http doc, we need to unmarshal the headers before putting
-      // in the ram cache. 
+      bool http_copy_hdr = false;
 #ifdef HTTP_CACHE
-      if (doc->ftype == CACHE_FRAG_TYPE_HTTP && doc->hlen && okay) {
-        char *tmp = doc->hdr();
-        int len = doc->hlen;
-        while (len > 0) {
-          int r = HTTPInfo::unmarshal(tmp, len, buf._ptr());
-          if (r < 0) {
-            ink_assert(!"CacheVC::handleReadDone unmarshal failed");
-            okay = 0;
-            break;
-          }
-          len -= r;
-          tmp += r;
-        }
-      }
+      http_copy_hdr = cache_config_ram_cache_compress && !f.doc_from_ram_cache && 
+        doc->ftype == CACHE_FRAG_TYPE_HTTP && doc->hlen;
+      // If http doc we need to unmarshal the headers before putting in the ram cache
+      // unless it could be compressed
+      if (!http_copy_hdr && doc->ftype == CACHE_FRAG_TYPE_HTTP && doc->hlen && okay)
+        unmarshal_helper(doc, buf, okay);
 #endif
       // Put the request in the ram cache only if its a open_read or lookup
       if (vio.op == VIO::READ && okay) {
@@ -1852,20 +1850,27 @@ CacheVC::handleReadDone(int event, Event
         //                doc->total_len
         // After that, the decision is based of doc_len (doc_len != 0)
         // (cache_config_ram_cache_cutoff == 0) : no cutoffs
-        cutoff_check = ((!doc_len && doc->total_len < part->ram_cache.cutoff_size)
-                        || (doc_len && doc_len < part->ram_cache.cutoff_size)
-                        || !part->ram_cache.cutoff_size);
-        if (cutoff_check)
-          part->ram_cache.put(read_key, buf, mutex->thread_holding, 0, dir_offset(&dir));
+        cutoff_check = ((!doc_len && (ink64)doc->total_len < cache_config_ram_cache_cutoff)
+                        || (doc_len && (ink64)doc_len < cache_config_ram_cache_cutoff)
+                        || !cache_config_ram_cache_cutoff);
+        if (cutoff_check && !f.doc_from_ram_cache) {
+          inku64 o = dir_offset(&dir);
+          part->ram_cache->put(read_key, buf, doc->len, http_copy_hdr, (inku32)(o >> 32), (inku32)o);
+        }
         if (!doc_len) {
           // keep a pointer to it. In case the state machine decides to
           // update this document, we don't have to read it back in memory
           // again
-          part->first_fragment.key = *read_key;
-          part->first_fragment.auxkey1 = dir_offset(&dir);
-          part->first_fragment.data = buf;
+          part->first_fragment_key = *read_key;
+          part->first_fragment_offset = dir_offset(&dir);
+          part->first_fragment_data = buf;
         }
       }                           // end VIO::READ check
+#ifdef HTTP_CACHE
+      // If it could be compressed, unmarshal after
+      if (http_copy_hdr && doc->ftype == CACHE_FRAG_TYPE_HTTP && doc->hlen && okay)
+        unmarshal_helper(doc, buf, okay);
+#endif
     }                             // end io.ok() check
   }
 Ldone:
@@ -1880,16 +1885,17 @@ CacheVC::handleRead(int event, Event *e)
   NOWARN_UNUSED(e);
   cancel_trigger();
 
+  f.doc_from_ram_cache = false;
   // check ram cache
   ink_debug_assert(part->mutex->thread_holding == this_ethread());
-  if (part->ram_cache.get(read_key, &buf, 0, dir_offset(&dir))) {
+  if (part->ram_cache->get(read_key, &buf, 0, dir_offset(&dir))) {
     CACHE_INCREMENT_DYN_STAT(cache_ram_cache_hits_stat);
     goto LramHit;
   }
-  // check if it was read in the last open_read call.
-  if (*read_key == part->first_fragment.key && dir_offset(&dir) == part->first_fragment.auxkey1) {
-    buf = part->first_fragment.data;
-    goto LramHit;
+  // check if it was read in the last open_read call
+  if (*read_key == part->first_fragment_key && dir_offset(&dir) == part->first_fragment_offset) {
+    buf = part->first_fragment_data;
+    goto LmemHit;
   }
 
   CACHE_INCREMENT_DYN_STAT(cache_ram_cache_misses_stat);
@@ -1920,7 +1926,16 @@ CacheVC::handleRead(int event, Event *e)
   CACHE_DEBUG_INCREMENT_DYN_STAT(cache_pread_count_stat);
   return EVENT_CONT;
 
-LramHit:
+LramHit: {
+    io.aio_result = io.aiocb.aio_nbytes;
+    Doc *doc = (Doc*)buf->data();
+    if (cache_config_ram_cache_compress && doc->ftype == CACHE_FRAG_TYPE_HTTP && doc->hlen) {
+      SET_HANDLER(&CacheVC::handleReadDone);
+      f.doc_from_ram_cache = true;
+      return EVENT_RETURN;
+    }
+  }
+LmemHit:
   io.aio_result = io.aiocb.aio_nbytes;
   POP_HANDLER;
   return EVENT_RETURN; // allow the caller to release the partition lock
@@ -1929,7 +1944,6 @@ LramHit:
 Action *
 Cache::lookup(Continuation *cont, CacheKey *key, CacheFragType type, char *hostname, int host_len)
 {
-
   if (!(CacheProcessor::cache_ready & type)) {
     cont->handleEvent(CACHE_EVENT_LOOKUP_FAILED, 0);
     return ACTION_RESULT_DONE;
@@ -2585,6 +2599,12 @@ ink_cache_init(ModuleVersion v)
   IOCORE_EstablishStaticConfigLLong(cache_config_ram_cache_size, "proxy.config.cache.ram_cache.size");
   Debug("cache_init", "proxy.config.cache.ram_cache.size = %lld = %lldMb",
         cache_config_ram_cache_size, cache_config_ram_cache_size / (1024 * 1024));
+  IOCORE_RegisterConfigInteger(RECT_CONFIG, "proxy.config.cache.ram_cache.algorithm", 0, RECU_DYNAMIC, RECC_NULL, NULL);
+  IOCORE_EstablishStaticConfigInt32(cache_config_ram_cache_algorithm, "proxy.config.cache.ram_cache.algorithm");
+  IOCORE_RegisterConfigInteger(RECT_CONFIG, "proxy.config.cache.ram_cache.compress", 0, RECU_DYNAMIC, RECC_NULL, NULL);
+  IOCORE_EstablishStaticConfigInt32(cache_config_ram_cache_compress, "proxy.config.cache.ram_cache.compress");
+  IOCORE_RegisterConfigInteger(RECT_CONFIG, "proxy.config.cache.ram_cache.compress_percent", 90, RECU_DYNAMIC, RECC_NULL, NULL);
+  IOCORE_EstablishStaticConfigInt32(cache_config_ram_cache_compress_percent, "proxy.config.cache.ram_cache.compress_percent");
 
   IOCORE_RegisterConfigInteger(RECT_CONFIG,
                                "proxy.config.cache.limits.http.max_alts", 3, RECU_DYNAMIC, RECC_NULL, NULL);
@@ -2597,11 +2617,13 @@ ink_cache_init(ModuleVersion v)
   Debug("cache_init", "cache_config_ram_cache_cutoff = %lld = %lldMb",
         cache_config_ram_cache_cutoff, cache_config_ram_cache_cutoff / (1024 * 1024));
 
+#if 0
   IOCORE_RegisterConfigInteger(RECT_CONFIG,
                                "proxy.config.cache.ram_cache_mixt_cutoff", 1048576, RECU_DYNAMIC, RECC_NULL, NULL);
   IOCORE_EstablishStaticConfigInteger(cache_config_ram_cache_mixt_cutoff, "proxy.config.cache.ram_cache_mixt_cutoff");
   Debug("cache_init", "proxy.config.cache.ram_cache_mixt_cutoff = %lld = %lldMb",
         cache_config_ram_cache_mixt_cutoff, cache_config_ram_cache_mixt_cutoff / (1024 * 1024));
+#endif
 
   IOCORE_RegisterConfigInteger(RECT_CONFIG, "proxy.config.cache.permit.pinning", 0, RECU_DYNAMIC, RECC_NULL, NULL);
   IOCORE_EstablishStaticConfigInt32(cache_config_permit_pinning, "proxy.config.cache.permit.pinning");

Modified: trafficserver/traffic/trunk/iocore/cache/CacheDir.cc
URL: http://svn.apache.org/viewvc/trafficserver/traffic/trunk/iocore/cache/CacheDir.cc?rev=941885&r1=941884&r2=941885&view=diff
==============================================================================
--- trafficserver/traffic/trunk/iocore/cache/CacheDir.cc (original)
+++ trafficserver/traffic/trunk/iocore/cache/CacheDir.cc Thu May  6 19:58:27 2010
@@ -812,7 +812,7 @@ dir_lookaside_fixup(CacheKey *key, Part 
       int res = dir_overwrite(key, d, &b->new_dir, &b->dir, false);
       DDebug("dir_lookaside", "fixup %X %X offset %d phase %d %d",
             key->word(0), key->word(1), dir_offset(&b->new_dir), dir_phase(&b->new_dir), res);
-      d->ram_cache.fixup(key, 0, dir_offset(&b->dir), 0, dir_offset(&b->new_dir));
+      d->ram_cache->fixup(key, 0, dir_offset(&b->dir), 0, dir_offset(&b->new_dir));
       d->lookaside[i].remove(b);
 #if 0
       // we need to do this because in case of a small cache, the scan

Modified: trafficserver/traffic/trunk/iocore/cache/CachePart.cc
URL: http://svn.apache.org/viewvc/trafficserver/traffic/trunk/iocore/cache/CachePart.cc?rev=941885&r1=941884&r2=941885&view=diff
==============================================================================
--- trafficserver/traffic/trunk/iocore/cache/CachePart.cc (original)
+++ trafficserver/traffic/trunk/iocore/cache/CachePart.cc Thu May  6 19:58:27 2010
@@ -398,9 +398,3 @@ CacheVC::scanUpdateDone(int event, Event
   }
 }
 
-void
-Cache::print_stats(FILE * fp, int verbose)
-{
-  for (int i = 0; i < gnpart; i++)
-    gpart[i]->ram_cache.print_stats(fp, verbose);
-}

Modified: trafficserver/traffic/trunk/iocore/cache/CacheWrite.cc
URL: http://svn.apache.org/viewvc/trafficserver/traffic/trunk/iocore/cache/CacheWrite.cc?rev=941885&r1=941884&r2=941885&view=diff
==============================================================================
--- trafficserver/traffic/trunk/iocore/cache/CacheWrite.cc (original)
+++ trafficserver/traffic/trunk/iocore/cache/CacheWrite.cc Thu May  6 19:58:27 2010
@@ -528,7 +528,7 @@ CacheVC::evacuateDocDone(int event, Even
 
           }
           if (dir_overwrite(&doc->first_key, part, &dir, &overwrite_dir)) {
-            part->ram_cache.fixup(&doc->first_key, 0, dir_offset(&overwrite_dir), 0, dir_offset(&dir));
+            part->ram_cache->fixup(&doc->first_key, 0, dir_offset(&overwrite_dir), 0, dir_offset(&dir));
           }
         } else {
           DDebug("cache_evac", "evacuating earliest: %X %d", (int) doc->key.word(0), (int) dir_offset(&overwrite_dir));

Modified: trafficserver/traffic/trunk/iocore/cache/I_Cache.h
URL: http://svn.apache.org/viewvc/trafficserver/traffic/trunk/iocore/cache/I_Cache.h?rev=941885&r1=941884&r2=941885&view=diff
==============================================================================
--- trafficserver/traffic/trunk/iocore/cache/I_Cache.h (original)
+++ trafficserver/traffic/trunk/iocore/cache/I_Cache.h Thu May  6 19:58:27 2010
@@ -51,6 +51,14 @@
 
 #define SCAN_KB_PER_SECOND      8192 // 1TB/8MB = 131072 = 36 HOURS to scan a TB
 
+#define RAM_CACHE_ALGORITHM_CLFUS        0
+#define RAM_CACHE_ALGORITHM_LRU          1
+
+#define CACHE_COMPRESSION_NONE           0
+#define CACHE_COMPRESSION_FASTLZ         1
+#define CACHE_COMPRESSION_LIBZ           2
+#define CACHE_COMPRESSION_LIBLZMA        3
+
 class CacheVC;
 #ifdef HTTP_CACHE
 class CacheLookupHttpConfig;

Modified: trafficserver/traffic/trunk/iocore/cache/Makefile.am
URL: http://svn.apache.org/viewvc/trafficserver/traffic/trunk/iocore/cache/Makefile.am?rev=941885&r1=941884&r2=941885&view=diff
==============================================================================
--- trafficserver/traffic/trunk/iocore/cache/Makefile.am (original)
+++ trafficserver/traffic/trunk/iocore/cache/Makefile.am Thu May  6 19:58:27 2010
@@ -70,7 +70,8 @@ libinkcache_a_SOURCES = \
   P_CacheInternal.h \
   P_CachePart.h \
   P_RamCache.h \
-  RamCache.cc \
+  RamCacheLRU.cc \
+  RamCacheCLFUS.cc \
   Store.cc \
   Inline.cc $(ADD_SRC)
 

Modified: trafficserver/traffic/trunk/iocore/cache/P_CacheInternal.h
URL: http://svn.apache.org/viewvc/trafficserver/traffic/trunk/iocore/cache/P_CacheInternal.h?rev=941885&r1=941884&r2=941885&view=diff
==============================================================================
--- trafficserver/traffic/trunk/iocore/cache/P_CacheInternal.h (original)
+++ trafficserver/traffic/trunk/iocore/cache/P_CacheInternal.h Thu May  6 19:58:27 2010
@@ -184,6 +184,10 @@ extern RecRawStatBlock *cache_rsb;
 	RecIncrRawStat(cache_rsb, mutex->thread_holding, (int) (x), (int) (y)); \
 	RecIncrRawStat(part->cache_part->part_rsb, mutex->thread_holding, (int) (x), (int) (y));
 
+#define CACHE_SUM_DYN_STAT_THREAD(x, y) \
+	RecIncrRawStat(cache_rsb, this_ethread(), (int) (x), (int) (y)); \
+	RecIncrRawStat(part->cache_part->part_rsb, this_ethread(), (int) (x), (int) (y));
+
 #define GLOBAL_CACHE_SUM_GLOBAL_DYN_STAT(x, y) \
 	RecIncrGlobalRawStatSum(cache_rsb,(x),(y))
 
@@ -214,6 +218,8 @@ extern int cache_config_read_while_write
 extern char cache_system_config_directory[PATH_NAME_MAX + 1];
 extern int cache_clustering_enabled;
 extern int cache_config_agg_write_backlog;
+extern int cache_config_ram_cache_compress;
+extern int cache_config_ram_cache_compress_percent;
 #ifdef HIT_EVACUATE
 extern int cache_config_hit_evacuate_percent;
 extern int cache_config_hit_evacuate_size_limit;
@@ -464,9 +470,10 @@ struct CacheVC:CacheVConnection
       unsigned int open_read_timeout:1; // UNUSED
       unsigned int data_done:1;
       unsigned int read_from_writer_called:1;
-      unsigned int not_from_ram_cache:1;        // entire doc was from ram cache
+      unsigned int not_from_ram_cache:1;        // entire object was from ram cache
       unsigned int rewrite_resident_alt:1;
       unsigned int readers:1;
+      unsigned int doc_from_ram_cache:1;
 #ifdef HIT_EVACUATE
       unsigned int hit_evacuate:1;
 #endif
@@ -974,7 +981,6 @@ struct Cache
   void part_initialized(bool result);
 
   int open_done();
-  static void print_stats(FILE *fp, int verbose = 0);
 
   Part *key_to_part(CacheKey *key, char *hostname, int host_len);
 

Modified: trafficserver/traffic/trunk/iocore/cache/P_CachePart.h
URL: http://svn.apache.org/viewvc/trafficserver/traffic/trunk/iocore/cache/P_CachePart.h?rev=941885&r1=941884&r2=941885&view=diff
==============================================================================
--- trafficserver/traffic/trunk/iocore/cache/P_CachePart.h (original)
+++ trafficserver/traffic/trunk/iocore/cache/P_CachePart.h Thu May  6 19:58:27 2010
@@ -158,7 +158,7 @@ struct Part:public Continuation
   Event *trigger;
 
   OpenDir open_dir;
-  RamCache ram_cache;
+  RamCache *ram_cache;
   int evacuate_size;
   DLL<EvacuationBlock> *evacuate;
   DLL<EvacuationBlock> lookaside[LOOKASIDE_SIZE];
@@ -175,7 +175,10 @@ struct Part:public Continuation
   bool dir_sync_waiting;
   bool dir_sync_in_progress;
   bool writing_end_marker;
-  RamCacheEntry first_fragment;
+
+  CacheKey first_fragment_key;
+  ink64 first_fragment_offset;
+  Ptr<IOBufferData> first_fragment_data;
 
   void cancel_trigger();
 

Modified: trafficserver/traffic/trunk/iocore/cache/P_RamCache.h
URL: http://svn.apache.org/viewvc/trafficserver/traffic/trunk/iocore/cache/P_RamCache.h?rev=941885&r1=941884&r2=941885&view=diff
==============================================================================
--- trafficserver/traffic/trunk/iocore/cache/P_RamCache.h (original)
+++ trafficserver/traffic/trunk/iocore/cache/P_RamCache.h Thu May  6 19:58:27 2010
@@ -24,71 +24,20 @@
 #ifndef _P_RAM_CACHE_H__
 #define _P_RAM_CACHE_H__
 
-
 #include "I_Cache.h"
 
-#define RAM_CACHE_SEEN_PER_OBJECT    2
-#define RAM_CACHE_FAST_LOAD_SIZE     32768
-#define n_partitions 1
-
-struct Part;
-
-struct RamCacheEntry
-{
-  INK_MD5 key;
-  inku32 auxkey1;
-  inku32 auxkey2;
-  LINK(RamCacheEntry, lru_link);
-  LINK(RamCacheEntry, hash_link);
-  Ptr<IOBufferData> data;
-};
+// Generic Ram Cache interface
 
-struct RamCachePartition
-{
-  int cur_bytes;
-  int cur_objects;
-  DList(RamCacheEntry, hash_link) *bucket;
-  Que(RamCacheEntry, lru_link) lru;
-  unsigned short *seen;
-  ProxyMutexPtr lock;
-
-  RamCachePartition():cur_bytes(0), cur_objects(0), bucket(0), seen(0), lock(NULL)
-  {
-  }
-};
+struct RamCache {
+  // returns 1 on found/stored, 0 on not found/stored, if provided auxkey1 and auxkey2 must match
+  virtual int get(INK_MD5 *key, Ptr<IOBufferData> *ret_data, inku32 auxkey1 = 0, inku32 auxkey2 = 0) = 0;
+  virtual int put(INK_MD5 *key, IOBufferData *data, inku32 len, bool copy = false, inku32 auxkey1 = 0, inku32 auxkey2 = 0) = 0;
+  virtual int fixup(INK_MD5 *key, inku32 old_auxkey1, inku32 old_auxkey2, inku32 new_auxkey1, inku32 new_auxkey2) = 0;
 
-struct RamCache
-{
-  // Partition Read-only data
-  ink64 bytes;
-  ink64 objects;
-  ink64 partition_size;
-  ink64 seen_size;
-  inku32 cutoff_size;
-  RamCachePartition *partition;
-  RamCachePartition one_partition;
-  Part *part;                   // back pointer to partition
-
-  // returns 1 on found/stored, 0 on not found/stored
-  // if provided, auxkey1 and auxkey2 must match
-  int get(INK_MD5 * key, Ptr<IOBufferData> *ret_data, inku32 auxkey1 = 0, inku32 auxkey2 = 0);
-  int put(INK_MD5 * key, IOBufferData * data, EThread * t, inku32 auxkey1 = 0, inku32 auxkey2 = 0);
-  int fixup(INK_MD5 * key, inku32 old_auxkey1, inku32 old_auxkey2, inku32 new_auxkey1, inku32 new_auxkey2);
-  // also returns -1 if locked 
-  int get_lock(INK_MD5 * key, Ptr<IOBufferData> *ret_data, EThread * t, inku32 auxkey1 = 0, inku32 auxkey2 = 0);
-  int put_lock(INK_MD5 * key, IOBufferData * data, EThread * t, inku32 auxkey1 = 0, inku32 auxkey2 = 0);
-
-  void remove_entry(RamCacheEntry * ee, RamCachePartition * p, EThread * t);
-
-  void print_stats(FILE * fp, int verbose = 0);
-
-
-  void init(ink64 bytes, ink64 objects, int cutoff, Part * _part, ProxyMutex * mutex = NULL);
-    RamCache():bytes(0), objects(0), partition_size(0), seen_size(0), cutoff_size(0), partition(0), part(NULL)
-  {
-  }
+  virtual void init(ink64 max_bytes, Part *part) = 0;
 };
 
-extern ClassAllocator<RamCacheEntry> ramCacheEntryAllocator;
+RamCache *new_RamCacheLRU();
+RamCache *new_RamCacheCLFUS();
 
 #endif /* _P_RAM_CACHE_H__ */

Added: trafficserver/traffic/trunk/iocore/cache/RamCacheCLFUS.cc
URL: http://svn.apache.org/viewvc/trafficserver/traffic/trunk/iocore/cache/RamCacheCLFUS.cc?rev=941885&view=auto
==============================================================================
--- trafficserver/traffic/trunk/iocore/cache/RamCacheCLFUS.cc (added)
+++ trafficserver/traffic/trunk/iocore/cache/RamCacheCLFUS.cc Thu May  6 19:58:27 2010
@@ -0,0 +1,590 @@
+/** @file
+
+  A brief file description
+
+  @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.
+ */
+
+// Clocked Least Frequently Used by Size (CLFUS) replacement policy
+// See https://cwiki.apache.org/confluence/display/TS/RamCache
+
+#include "P_Cache.h"
+#ifdef HAVE_LIBZ
+#include <zlib.h>
+#endif
+#ifdef HAVE_LZMA
+#include <lzma.h>
+#endif
+
+#define REQUIRED_COMPRESSION 0.9 // must get to this size or declared incompressible
+#define REQUIRED_SHRINK 0.8 // must get to this size or keep orignal buffer (with padding)
+#define HISTORY_HYSTERIA 10 // extra temporary history
+#define ENTRY_OVERHEAD 256 // per-entry overhead to consider when computing cache value/size
+#define LZMA_BASE_MEMLIMIT (64 * 1024 * 1024)
+//#define CHECK_ACOUNTING 1 // very expensive double checking of all sizes
+
+#define REQUEUE_HITS(_h) ((_h) ? 1 : 0)
+#define CACHE_VALUE_HITS_SIZE(_h, _s) ((float)((_h)+1) / ((_s) + ENTRY_OVERHEAD))
+#define CACHE_VALUE(_x) CACHE_VALUE_HITS_SIZE((_x)->hits, (_x)->size)
+
+struct RamCacheCLFUSEntry {
+  INK_MD5 key;
+  inku32 auxkey1;
+  inku32 auxkey2;
+  inku64 hits;
+  inku32 size; // memory used including paddding in buffer
+  inku32 len;  // actual data length
+  inku32 compressed_len;
+  union {
+    struct {
+      inku32 compressed:3; // compression type
+      inku32 incompressible:1;
+      inku32 lru:1;
+      inku32 copy:1; // copy-in-copy-out
+    };
+    inku32 flags;
+  };
+  LINK(RamCacheCLFUSEntry, lru_link);
+  LINK(RamCacheCLFUSEntry, hash_link);
+  Ptr<IOBufferData> data;
+};
+
+struct RamCacheCLFUS : RamCache {
+  ink64 max_bytes;
+  ink64 bytes;
+  ink64 objects;
+
+  // returns 1 on found/stored, 0 on not found/stored, if provided auxkey1 and auxkey2 must match
+  int get(INK_MD5 *key, Ptr<IOBufferData> *ret_data, inku32 auxkey1 = 0, inku32 auxkey2 = 0);
+  int put(INK_MD5 *key, IOBufferData *data, inku32 len, bool copy = false, inku32 auxkey1 = 0, inku32 auxkey2 = 0);
+  int fixup(INK_MD5 *key, inku32 old_auxkey1, inku32 old_auxkey2, inku32 new_auxkey1, inku32 new_auxkey2);
+
+  void init(ink64 max_bytes, Part *part);
+
+  // private
+  Part *part; // for stats
+  ink64 history;
+  int ibuckets;
+  int nbuckets;
+  DList(RamCacheCLFUSEntry, hash_link) *bucket;
+  Que(RamCacheCLFUSEntry, lru_link) lru[2];
+  inku16 *seen;
+  int ncompressed;
+  RamCacheCLFUSEntry *compressed; // first uncompressed lru[0] entry
+  void compress_entries(int do_at_most = INT_MAX);
+  void resize_hashtable();
+  void victimize(RamCacheCLFUSEntry *e);
+  void move_compressed(RamCacheCLFUSEntry *e);
+  RamCacheCLFUSEntry *destroy(RamCacheCLFUSEntry *e);
+  void requeue_victims(RamCacheCLFUS *c, Que(RamCacheCLFUSEntry, lru_link) &victims);
+  void tick(); // move CLOCK on history
+  RamCacheCLFUS(): max_bytes(0), bytes(0), objects(0), part(0), history(0), ibuckets(0), nbuckets(0), bucket(0),
+              seen(0), ncompressed(0), compressed(0) { }
+};
+
+ClassAllocator<RamCacheCLFUSEntry> ramCacheCLFUSEntryAllocator("RamCacheCLFUSEntry");
+
+const static int bucket_sizes[] = {
+  127, 251, 509, 1021, 2039, 4093, 8191, 16381, 32749, 65521, 131071, 262139,
+  524287, 1048573, 2097143, 4194301, 8388593, 16777213, 33554393, 67108859,
+  134217689, 268435399, 536870909, 1073741789, 2147483647
+};
+
+void RamCacheCLFUS::resize_hashtable() {
+  int anbuckets = bucket_sizes[ibuckets];
+  DDebug("ram_cache", "resize hashtable %d", anbuckets);
+  ink64 s = anbuckets * sizeof(DList(RamCacheCLFUSEntry, hash_link));
+  DList(RamCacheCLFUSEntry, hash_link) *new_bucket = (DList(RamCacheCLFUSEntry, hash_link) *)xmalloc(s);
+  memset(new_bucket, 0, s);
+  if (bucket) {
+    for (ink64 i = 0; i < nbuckets; i++) {
+      RamCacheCLFUSEntry *e = 0;
+      while ((e = bucket[i].pop()))
+        new_bucket[e->key.word(3) % anbuckets].push(e);
+    }
+    xfree(bucket);
+  }
+  bucket = new_bucket;
+  nbuckets = anbuckets;
+  if (seen) xfree(seen);
+  int size = bucket_sizes[ibuckets] * sizeof(inku16);
+  seen = (inku16*)xmalloc(size);
+  memset(seen, 0, size);
+}
+
+void RamCacheCLFUS::init(ink64 abytes, Part *apart) {
+  part = apart;
+  max_bytes = abytes;
+  DDebug("ram_cache", "initializing ram_cache %lld bytes", abytes);
+  if (!max_bytes)
+    return;
+  resize_hashtable();
+}
+
+#ifdef CHECK_ACOUNTING
+static void check_accounting(RamCacheCLFUS *c) {
+  ink64 x = 0, xsize = 0, h = 0;
+  RamCacheCLFUSEntry *y = c->lru[0].head;
+  while (y) { x++; xsize += y->size + ENTRY_OVERHEAD; y = y->lru_link.next; }
+  y = c->lru[1].head;
+  while (y) { h++; y = y->lru_link.next; }
+  ink_assert(x == c->objects);
+  ink_assert(xsize == c->bytes);
+  ink_assert(h == c->history);
+}
+#else
+#define check_accounting(_c)
+#endif
+
+int RamCacheCLFUS::get(INK_MD5 *key, Ptr<IOBufferData> *ret_data, inku32 auxkey1, inku32 auxkey2) {
+  if (!max_bytes)
+    return 0;
+  ink64 i = key->word(3) % nbuckets;
+  RamCacheCLFUSEntry *e = bucket[i].head;
+  char *b = 0;
+  while (e) {
+    if (e->key == *key && e->auxkey1 == auxkey1 && e->auxkey2 == auxkey2) {
+      move_compressed(e);
+      lru[e->lru].remove(e);
+      lru[e->lru].enqueue(e);
+      if (!e->lru) { // in memory
+        e->hits++;
+        if (e->compressed) {
+          b = (char*)xmalloc(e->len);
+          switch (e->compressed) {
+            default: goto Lfailed;
+            case CACHE_COMPRESSION_FASTLZ: {
+              int l = (int)e->len;
+              if ((l != (int)fastlz_decompress(e->data->data(), e->compressed_len, b, l)))
+                goto Lfailed;
+              break;
+            }
+#if HAVE_LIBZ
+            case CACHE_COMPRESSION_LIBZ: {
+              uLongf l = e->len;
+              if (Z_OK != uncompress((Bytef*)b, &l, (Bytef*)e->data->data(), e->compressed_len))
+                goto Lfailed;
+              break;
+            }
+#endif
+#ifdef HAVE_LZMA
+            case CACHE_COMPRESSION_LIBLZMA: {
+              size_t l = (size_t)e->len, ipos = 0, opos = 0;
+              uint64_t memlimit = e->len * 2 + LZMA_BASE_MEMLIMIT;
+              if (LZMA_OK != lzma_stream_buffer_decode(
+                    &memlimit, 0, NULL, (uint8_t*)e->data->data(), &ipos, e->compressed_len, (uint8_t*)b, &opos, l))
+                goto Lfailed;
+              break;
+            }
+#endif
+          }
+          IOBufferData *data = new_xmalloc_IOBufferData(b, e->len);
+          data->_mem_type = DEFAULT_ALLOC;
+          if (!e->copy) { // don't bother if we have to copy anyway
+            ink64 delta = ((ink64)e->compressed_len) - (ink64)e->size;
+            bytes += delta;
+            CACHE_SUM_DYN_STAT_THREAD(cache_ram_cache_bytes_stat, delta);
+            e->size = e->compressed_len;
+            check_accounting(this);
+            e->compressed = 0;
+            e->data = data;
+          }
+          (*ret_data) = data;
+        } else {
+          IOBufferData *data = e->data;
+          if (e->copy) {
+            data = new_IOBufferData(iobuffer_size_to_index(e->len, MAX_BUFFER_SIZE_INDEX), MEMALIGNED);
+            memcpy(data->data(), e->data->data(), e->len);
+          }
+          (*ret_data) = data;
+        }
+        CACHE_SUM_DYN_STAT_THREAD(cache_ram_cache_hits_stat, 1);
+        DDebug("ram_cache", "get %X %d %d size %d HIT", key->word(3), auxkey1, auxkey2, e->size);
+        return 1;
+      } else {
+        CACHE_SUM_DYN_STAT_THREAD(cache_ram_cache_misses_stat, 1);
+        DDebug("ram_cache", "get %X %d %d HISTORY", key->word(3), auxkey1, auxkey2);
+        return 0;
+      }
+    }
+    assert(e != e->hash_link.next);
+    e = e->hash_link.next;
+  }
+  DDebug("ram_cache", "get %X %d %d MISS", key->word(3), auxkey1, auxkey2);
+Lerror:
+  CACHE_SUM_DYN_STAT_THREAD(cache_ram_cache_misses_stat, 1);
+  return 0;
+Lfailed:
+  xfree(b);
+  e = destroy(e);
+  DDebug("ram_cache", "get %X %d %d Z_ERR", key->word(3), auxkey1, auxkey2);
+  goto Lerror;
+}
+
+void RamCacheCLFUS::tick() {
+  RamCacheCLFUSEntry *e = lru[1].dequeue();
+  if (!e)
+    return;
+  e->hits <<= 1;
+  if (e->hits) {
+    e->hits = REQUEUE_HITS(e->hits);
+    lru[1].enqueue(e);
+  } else
+    goto Lfree;
+  if (history <= objects + HISTORY_HYSTERIA)
+    return;
+  e = lru[1].dequeue();
+Lfree:
+  e->lru = 0;
+  history--;
+  inku32 b = e->key.word(3) % nbuckets;
+  bucket[b].remove(e);
+  DDebug("ram_cache", "put %X %d %d size %d FREED", e->key.word(3), e->auxkey1, e->auxkey2, e->size);
+  THREAD_FREE(e, ramCacheCLFUSEntryAllocator, this_ethread());
+}
+
+void RamCacheCLFUS::victimize(RamCacheCLFUSEntry *e) {
+  objects--;
+  DDebug("ram_cache", "put %X %d %d size %d VICTIMIZED", e->key.word(3), e->auxkey1, e->auxkey2, e->size);
+  e->data = NULL;
+  e->lru = 1;
+  lru[1].enqueue(e);
+  history++;
+}
+
+void RamCacheCLFUS::move_compressed(RamCacheCLFUSEntry *e) {
+  if (e == compressed) {
+    ncompressed--;
+    if (compressed->lru_link.next)
+      compressed = compressed->lru_link.next;
+    else {
+      ncompressed--;
+      compressed = compressed->lru_link.prev;
+    }
+  }
+}
+
+RamCacheCLFUSEntry *RamCacheCLFUS::destroy(RamCacheCLFUSEntry *e) {
+  RamCacheCLFUSEntry *ret = e->hash_link.next;
+  move_compressed(e);
+  lru[e->lru].remove(e);
+  if (!e->lru) {
+    objects--;
+    bytes -= e->size + ENTRY_OVERHEAD;
+    CACHE_SUM_DYN_STAT_THREAD(cache_ram_cache_bytes_stat, -e->size);
+    e->data = NULL;
+  } else
+    history--;
+  inku32 b = e->key.word(3) % nbuckets;
+  bucket[b].remove(e);
+  DDebug("ram_cache", "put %X %d %d DESTROYED", e->key.word(3), e->auxkey1, e->auxkey2);
+  THREAD_FREE(e, ramCacheCLFUSEntryAllocator, this_ethread());
+  return ret;
+}
+
+void RamCacheCLFUS::compress_entries(int do_at_most) {
+  if (!cache_config_ram_cache_compress)
+    return;
+  if (!compressed) {
+    compressed = lru[0].head;
+    ncompressed = 0;
+  }
+  float target = (cache_config_ram_cache_compress_percent / 100.0) * objects;
+  int n = 0;
+  char *b = 0, *bb = 0;
+  while (compressed && target > ncompressed) {
+    RamCacheCLFUSEntry *e = compressed;
+    if (e->incompressible || e->compressed)
+      goto Lcontinue; 
+    n++;
+    if (do_at_most < n)
+      break;
+    {
+      e->compressed_len = e->size;
+      uLongf l = 0;
+      int ctype = cache_config_ram_cache_compress;
+      switch (ctype) {
+        default: goto Lcontinue;
+        case CACHE_COMPRESSION_FASTLZ: l = e->len * 1.05 + 66; break;
+#ifdef HAVE_LIBZ
+        case CACHE_COMPRESSION_LIBZ: l = compressBound(e->len); break;
+#endif
+#ifdef HAVE_LZMA
+        case CACHE_COMPRESSION_LIBLZMA: l = e->len; break;
+#endif
+      }
+      b = (char*)xmalloc(l);
+      switch (ctype) {
+        default: goto Lfailed;
+        case CACHE_COMPRESSION_FASTLZ:
+          if (e->len < 16) goto Lfailed;
+          if ((l = fastlz_compress(e->data->data(), e->len, b)) <= 0)
+            goto Lfailed;
+          break;
+#if HAVE_LIBZ
+        case CACHE_COMPRESSION_LIBZ: {
+          uLongf ll = l;
+          if ((Z_OK != compress((Bytef*)b, &ll, (Bytef*)e->data->data(), e->len))) 
+            goto Lfailed;
+          l = (int)ll;
+          break;
+        }
+#endif
+#ifdef HAVE_LZMA
+        case CACHE_COMPRESSION_LIBLZMA: {
+          size_t pos = 0, ll = l;
+          if (LZMA_OK != lzma_easy_buffer_encode(LZMA_PRESET_DEFAULT, LZMA_CHECK_NONE, NULL, 
+                                                 (uint8_t*)e->data->data(), e->len, (uint8_t*)b, &pos, ll))
+            goto Lfailed;
+          l = (int)pos;
+          break;
+        }
+#endif
+      }
+      if (l > REQUIRED_COMPRESSION * e->len)
+        e->incompressible = ctype;
+      if (l > REQUIRED_SHRINK * e->size)
+        goto Lfailed;
+      if (l < e->len) {
+        e->compressed = cache_config_ram_cache_compress;
+        bb = (char*)xmalloc(l);
+        memcpy(bb, b, l);
+        xfree(b);
+        e->compressed_len = l;
+        ink64 delta = ((ink64)l) - (ink64)e->size;
+        bytes += delta;
+        CACHE_SUM_DYN_STAT_THREAD(cache_ram_cache_bytes_stat, delta);
+        e->size = l;
+      } else {
+        xfree(b);
+        e->compressed = 0;
+        bb = (char*)xmalloc(e->len);
+        memcpy(bb, e->data->data(), e->len);
+        ink64 delta = ((ink64)e->len) - (ink64)e->size;
+        bytes += delta;
+        CACHE_SUM_DYN_STAT_THREAD(cache_ram_cache_bytes_stat, delta);
+        e->size = e->len;
+        l = e->len;
+      }
+      e->data = new_xmalloc_IOBufferData(bb, l);
+      e->data->_mem_type = DEFAULT_ALLOC;
+      check_accounting(this);
+    }
+    goto Lcontinue;
+  Lfailed:    
+    xfree(b);
+    e->incompressible = 1;
+  Lcontinue:;
+    DDebug("ram_cache", "compress %X %d %d %d %d %d %d", 
+           e->key.word(3), e->auxkey1, e->auxkey2, e->incompressible, e->compressed, 
+           e->len, e->compressed_len);
+    if (!e->lru_link.next)
+      break;
+    compressed = e->lru_link.next;
+    ncompressed++;
+  }
+}
+
+void RamCacheCLFUS::requeue_victims(RamCacheCLFUS *c, Que(RamCacheCLFUSEntry, lru_link) &victims) {
+  RamCacheCLFUSEntry *victim = 0;
+  while ((victim = victims.dequeue())) { 
+    c->bytes += victim->size + ENTRY_OVERHEAD; 
+    CACHE_SUM_DYN_STAT_THREAD(cache_ram_cache_bytes_stat, victim->size);
+    victim->hits = REQUEUE_HITS(victim->hits);
+    c->lru[0].enqueue(victim); 
+  }
+}
+
+int RamCacheCLFUS::put(INK_MD5 *key, IOBufferData *data, inku32 len, bool copy, inku32 auxkey1, inku32 auxkey2) {
+  if (!max_bytes)
+    return 0;
+  inku32 i = key->word(3) % nbuckets;
+  RamCacheCLFUSEntry *e = bucket[i].head;
+  inku32 size = copy ? len : data->block_size();
+  while (e) {
+    if (e->key == *key) {
+      if (e->auxkey1 == auxkey1 && e->auxkey2 == auxkey2)
+        break;
+      else {
+        e = destroy(e); // discard when aux keys conflict
+        continue;
+      }
+    }
+    e = e->hash_link.next;
+  }
+  if (e) {
+    e->hits++;
+    if (!e->lru) { // already in cache
+      move_compressed(e);
+      lru[e->lru].remove(e);
+      lru[e->lru].enqueue(e);
+      ink64 delta = ((ink64)size) - (ink64)e->size;
+      bytes += delta;
+      CACHE_SUM_DYN_STAT_THREAD(cache_ram_cache_bytes_stat, delta);
+      if (!copy) {
+        e->size = size;
+        e->data = data;
+      } else {
+        char *b = (char*)xmalloc(len);
+        memcpy(b, data->data(), len);
+        e->data = new_xmalloc_IOBufferData(b, len);
+        e->data->_mem_type = DEFAULT_ALLOC;
+        e->size = size;
+      }
+      check_accounting(this);
+      e->copy = copy;
+      e->compressed = 0;
+      DDebug("ram_cache", "put %X %d %d size %d HIT", key->word(3), auxkey1, auxkey2, e->size);
+      return 1;
+    } else
+      lru[1].remove(e);
+  }
+  Que(RamCacheCLFUSEntry, lru_link) victims;
+  RamCacheCLFUSEntry *victim = 0;
+  if (!lru[1].head) // initial fill
+    if (bytes + size <= max_bytes)
+      goto Linsert;
+  if (!e) {
+    inku32 s = key->word(3) % bucket_sizes[ibuckets];
+    inku16 k = key->word(3) >> 16;
+    inku16 kk = seen[s];
+    seen[s] = k;
+    if (history >= objects && kk != k) {
+      DDebug("ram_cache", "put %X %d %d size %d UNSEEN", key->word(3), auxkey1, auxkey2, size);
+      return 0;
+    }
+  }
+  while (1) {
+    victim = lru[0].dequeue();
+    if (!victim) {
+      if (bytes + size <= max_bytes)
+        goto Linsert;
+      if (e)
+        lru[1].enqueue(e);
+      requeue_victims(this, victims);
+      DDebug("ram_cache", "put %X %d %d NO VICTIM", key->word(3), auxkey1, auxkey2);
+      return 0;
+    }
+    bytes -= victim->size + ENTRY_OVERHEAD;
+    CACHE_SUM_DYN_STAT_THREAD(cache_ram_cache_bytes_stat, -victim->size);
+    victims.enqueue(victim);
+    if (victim == compressed)
+      compressed = 0;
+    else
+      ncompressed--;
+    victim->hits <<= 1;
+    tick();
+    if (!e)
+      goto Lhistory;
+    else { // e from history
+      DDebug("ram_cache_compare", "put %f %f", CACHE_VALUE(victim), CACHE_VALUE(e));
+      if (bytes + victim->size + size > max_bytes && CACHE_VALUE(victim) > CACHE_VALUE(e)) {
+        requeue_victims(this, victims);
+        lru[1].enqueue(e);
+        DDebug("ram_cache", "put %X %d %d size %d INC %d HISTORY", 
+               key->word(3), auxkey1, auxkey2, e->size, e->hits);
+        return 0;
+      }
+    }
+    if (bytes + size <= max_bytes)
+      goto Linsert;
+  }
+Linsert:
+  while ((victim = victims.dequeue())) {
+    if (bytes + size + victim->size <= max_bytes) {
+      bytes += victim->size + ENTRY_OVERHEAD;
+      CACHE_SUM_DYN_STAT_THREAD(cache_ram_cache_bytes_stat, victim->size);
+      victim->hits = REQUEUE_HITS(victim->hits);
+      lru[0].enqueue(victim);
+    } else
+      victimize(victim);
+  }
+  if (e) {
+    history--; // move from history
+  } else {
+    e = THREAD_ALLOC(ramCacheCLFUSEntryAllocator, this_ethread());
+    e->key = *key;
+    e->auxkey1 = auxkey1;
+    e->auxkey2 = auxkey2;
+    e->hits = 1;
+    bucket[i].push(e);
+    if (objects > nbuckets) {
+      ++ibuckets;
+      resize_hashtable();
+    }
+  }
+  check_accounting(this);
+  e->flags = 0;
+  if (!copy)
+    e->data = data;
+  else {
+    char *b = (char*)xmalloc(len);
+    memcpy(b, data->data(), len);
+    e->data = new_xmalloc_IOBufferData(b, len);
+    e->data->_mem_type = DEFAULT_ALLOC;
+  }
+  e->copy = copy;
+  bytes += size + ENTRY_OVERHEAD;
+  CACHE_SUM_DYN_STAT_THREAD(cache_ram_cache_bytes_stat, size);
+  e->size = size;
+  objects++;
+  lru[0].enqueue(e);
+  e->len = len;
+  check_accounting(this);
+  DDebug("ram_cache", "put %X %d %d size %d INSERTED", key->word(3), auxkey1, auxkey2, e->size);
+  if (cache_config_ram_cache_compress_percent)
+    compress_entries();
+  return 1;
+Lhistory:
+  requeue_victims(this, victims);
+  check_accounting(this);
+  e = THREAD_ALLOC(ramCacheCLFUSEntryAllocator, this_ethread());
+  e->key = *key;
+  e->auxkey1 = auxkey1;
+  e->auxkey2 = auxkey2;
+  e->hits = 1;
+  e->size = data->block_size();
+  e->flags = 0;
+  bucket[i].push(e);
+  e->lru = 1;
+  lru[1].enqueue(e);
+  history++;
+  DDebug("ram_cache", "put %X %d %d HISTORY", key->word(3), auxkey1, auxkey2);
+  return 0;
+}
+
+int RamCacheCLFUS::fixup(INK_MD5 * key, inku32 old_auxkey1, inku32 old_auxkey2, inku32 new_auxkey1, inku32 new_auxkey2) {
+  if (!max_bytes)
+    return 0;
+  inku32 i = key->word(3) % nbuckets;
+  RamCacheCLFUSEntry *e = bucket[i].head;
+  while (e) {
+    if (e->key == *key && e->auxkey1 == old_auxkey1 && e->auxkey2 == old_auxkey2) {
+      e->auxkey1 = new_auxkey1;
+      e->auxkey2 = new_auxkey2;
+      return 1;
+    }
+    e = e->hash_link.next;
+  }
+  return 0;
+}
+
+RamCache *new_RamCacheCLFUS() {
+  return new RamCacheCLFUS;
+}

Added: trafficserver/traffic/trunk/iocore/cache/RamCacheLRU.cc
URL: http://svn.apache.org/viewvc/trafficserver/traffic/trunk/iocore/cache/RamCacheLRU.cc?rev=941885&view=auto
==============================================================================
--- trafficserver/traffic/trunk/iocore/cache/RamCacheLRU.cc (added)
+++ trafficserver/traffic/trunk/iocore/cache/RamCacheLRU.cc Thu May  6 19:58:27 2010
@@ -0,0 +1,197 @@
+/** @file
+
+  A brief file description
+
+  @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.
+ */
+
+#include "P_Cache.h"
+
+struct RamCacheLRUEntry {
+  INK_MD5 key;
+  inku32 auxkey1;
+  inku32 auxkey2;
+  LINK(RamCacheLRUEntry, lru_link);
+  LINK(RamCacheLRUEntry, hash_link);
+  Ptr<IOBufferData> data;
+};
+
+struct RamCacheLRU : RamCache {
+  ink64 max_bytes;
+  ink64 bytes;
+  ink64 objects;
+
+  // returns 1 on found/stored, 0 on not found/stored, if provided auxkey1 and auxkey2 must match
+  int get(INK_MD5 *key, Ptr<IOBufferData> *ret_data, inku32 auxkey1 = 0, inku32 auxkey2 = 0);
+  int put(INK_MD5 *key, IOBufferData *data, inku32 len, bool copy = false, inku32 auxkey1 = 0, inku32 auxkey2 = 0);
+  int fixup(INK_MD5 *key, inku32 old_auxkey1, inku32 old_auxkey2, inku32 new_auxkey1, inku32 new_auxkey2);
+
+  void init(ink64 max_bytes, Part *part);
+
+  // private
+  inku16 *seen;
+  Que(RamCacheLRUEntry, lru_link) lru;
+  DList(RamCacheLRUEntry, hash_link) *bucket;
+  int nbuckets;
+  int ibuckets;
+  Part *part;
+
+  void resize_hashtable();
+  RamCacheLRUEntry *remove(RamCacheLRUEntry *e);
+
+  RamCacheLRU():bytes(0), objects(0), seen(0), bucket(0), nbuckets(0), ibuckets(0), part(NULL) {}
+};
+
+ClassAllocator<RamCacheLRUEntry> ramCacheLRUEntryAllocator("RamCacheLRUEntry");
+
+const static int bucket_sizes[] = {
+  127, 251, 509, 1021, 2039, 4093, 8191, 16381, 32749, 65521, 131071, 262139,
+  524287, 1048573, 2097143, 4194301, 8388593, 16777213, 33554393, 67108859,
+  134217689, 268435399, 536870909
+};
+
+void RamCacheLRU::resize_hashtable() {
+  int anbuckets = bucket_sizes[ibuckets];
+  DDebug("ram_cache", "resize hashtable %d", anbuckets);
+  ink64 s = anbuckets * sizeof(DList(RamCacheLRUEntry, hash_link));
+  DList(RamCacheLRUEntry, hash_link) *new_bucket = (DList(RamCacheLRUEntry, hash_link) *)xmalloc(s);
+  memset(new_bucket, 0, s);
+  if (bucket) {
+    for (ink64 i = 0; i < nbuckets; i++) {
+      RamCacheLRUEntry *e = 0;
+      while ((e = bucket[i].pop()))
+        new_bucket[e->key.word(3) % anbuckets].push(e);
+    }
+    xfree(bucket);
+  }
+  bucket = new_bucket;
+  nbuckets = anbuckets;
+  if (seen) xfree(seen);
+  int size = bucket_sizes[ibuckets] * sizeof(inku16);
+  seen = (inku16*)xmalloc(size);
+  memset(seen, 0, size);
+}
+
+void
+RamCacheLRU::init(ink64 abytes, Part *apart) {
+  part = apart;
+  max_bytes = abytes;
+  DDebug("ram_cache", "initializing ram_cache %lld bytes", abytes);
+  if (!max_bytes)
+    return;
+  resize_hashtable();
+}
+
+int
+RamCacheLRU::get(INK_MD5 * key, Ptr<IOBufferData> *ret_data, inku32 auxkey1, inku32 auxkey2) {
+  if (!max_bytes)
+    return 0;
+  inku32 i = key->word(3) % nbuckets;
+  RamCacheLRUEntry *e = bucket[i].head;
+  while (e) {
+    if (e->key == *key && e->auxkey1 == auxkey1 && e->auxkey2 == auxkey2) {
+      lru.remove(e);
+      lru.enqueue(e);
+      (*ret_data) = e->data;
+      DDebug("ram_cache", "get %X %d %d HIT", key->word(3), auxkey1, auxkey2);
+      CACHE_SUM_DYN_STAT_THREAD(cache_ram_cache_hits_stat, 1);
+      return 1;
+    }
+    e = e->hash_link.next;
+  }
+  DDebug("ram_cache", "get %X %d %d MISS", key->word(3), auxkey1, auxkey2);
+  CACHE_SUM_DYN_STAT_THREAD(cache_ram_cache_misses_stat, 1);
+  return 0;
+}
+
+RamCacheLRUEntry * RamCacheLRU::remove(RamCacheLRUEntry *e) {
+  RamCacheLRUEntry *ret = e->hash_link.next;
+  inku32 b = e->key.word(3) % nbuckets;
+  bucket[b].remove(e);
+  lru.remove(e);
+  bytes -= e->data->block_size();
+  CACHE_SUM_DYN_STAT_THREAD(cache_ram_cache_bytes_stat, -e->data->block_size());
+  DDebug("ram_cache", "put %X %d %d FREED", e->key.word(3), e->auxkey1, e->auxkey2);
+  e->data = NULL;
+  THREAD_FREE(e, ramCacheLRUEntryAllocator, this_ethread());
+  objects--;
+  return ret;
+}
+
+// ignore 'len' and 'copy' since we don't touch the data
+int RamCacheLRU::put(INK_MD5 *key, IOBufferData *data, inku32, bool, inku32 auxkey1, inku32 auxkey2) {
+  if (!max_bytes)
+    return 0;
+  inku32 i = key->word(3) % nbuckets;
+  RamCacheLRUEntry *e = bucket[i].head;
+  while (e) {
+    if (e->key == *key) {
+      if (e->auxkey1 == auxkey1 && e->auxkey2 == auxkey2)
+        break;
+      else { // discard when aux keys conflict
+        e = remove(e);
+        continue;
+      }
+    }
+    e = e->hash_link.next;
+  }
+  e = THREAD_ALLOC(ramCacheLRUEntryAllocator, this_ethread());
+  e->key = *key;
+  e->auxkey1 = auxkey1;
+  e->auxkey2 = auxkey2;
+  e->data = data;
+  bucket[i].push(e);
+  lru.enqueue(e);
+  bytes += data->block_size();
+  objects++;
+  CACHE_SUM_DYN_STAT_THREAD(cache_ram_cache_bytes_stat, data->block_size());
+  while (bytes > max_bytes) {
+    RamCacheLRUEntry *ee = lru.dequeue();
+    if (ee)
+      remove(ee);
+    else
+      break;
+  }
+  DDebug("ram_cache", "put %X %d %d INSERTED", key->word(3), auxkey1, auxkey2);
+  if (objects > nbuckets) {
+    ++ibuckets;
+    resize_hashtable();
+  }
+  return 1;
+}
+
+int RamCacheLRU::fixup(INK_MD5 * key, inku32 old_auxkey1, inku32 old_auxkey2, inku32 new_auxkey1, inku32 new_auxkey2) {
+  if (!max_bytes)
+    return 0;
+  inku32 i = key->word(3) % nbuckets;
+  RamCacheLRUEntry *e = bucket[i].head;
+  while (e) {
+    if (e->key == *key && e->auxkey1 == old_auxkey1 && e->auxkey2 == old_auxkey2) {
+      e->auxkey1 = new_auxkey1;
+      e->auxkey2 = new_auxkey2;
+      return 1;
+    }
+    e = e->hash_link.next;
+  }
+  return 0;
+}
+
+RamCache *new_RamCacheLRU() {
+  return new RamCacheLRU;
+}

Modified: trafficserver/traffic/trunk/iocore/eventsystem/I_EThread.h
URL: http://svn.apache.org/viewvc/trafficserver/traffic/trunk/iocore/eventsystem/I_EThread.h?rev=941885&r1=941884&r2=941885&view=diff
==============================================================================
--- trafficserver/traffic/trunk/iocore/eventsystem/I_EThread.h (original)
+++ trafficserver/traffic/trunk/iocore/eventsystem/I_EThread.h Thu May  6 19:58:27 2010
@@ -273,7 +273,8 @@ public:
   ProxyAllocator cacheVConnectionAllocator;
   ProxyAllocator newCacheVConnectionAllocator;
   ProxyAllocator openDirEntryAllocator;
-  ProxyAllocator ramCacheEntryAllocator;
+  ProxyAllocator ramCacheCLFUSEntryAllocator;
+  ProxyAllocator ramCacheLRUEntryAllocator;
   ProxyAllocator evacuationBlockAllocator;
   ProxyAllocator ioDataAllocator;
   ProxyAllocator ioBlockAllocator;

Modified: trafficserver/traffic/trunk/libinktomi++/Makefile.am
URL: http://svn.apache.org/viewvc/trafficserver/traffic/trunk/libinktomi%2B%2B/Makefile.am?rev=941885&r1=941884&r2=941885&view=diff
==============================================================================
--- trafficserver/traffic/trunk/libinktomi++/Makefile.am (original)
+++ trafficserver/traffic/trunk/libinktomi++/Makefile.am Thu May  6 19:58:27 2010
@@ -117,6 +117,8 @@ libinktomi___a_SOURCES = \
   InkTime.h \
   inktomi++.h \
   ink_unused.h \
+  fastlz.h \
+  fastlz.c \
   I_Version.h \
   List.h \
   llqueue.cc \

Added: trafficserver/traffic/trunk/libinktomi++/fastlz.c
URL: http://svn.apache.org/viewvc/trafficserver/traffic/trunk/libinktomi%2B%2B/fastlz.c?rev=941885&view=auto
==============================================================================
--- trafficserver/traffic/trunk/libinktomi++/fastlz.c (added)
+++ trafficserver/traffic/trunk/libinktomi++/fastlz.c Thu May  6 19:58:27 2010
@@ -0,0 +1,551 @@
+/*  
+  FastLZ - lightning-fast lossless compression library
+
+  Copyright (C) 2007 Ariya Hidayat (ariya@kde.org)
+  Copyright (C) 2006 Ariya Hidayat (ariya@kde.org)
+  Copyright (C) 2005 Ariya Hidayat (ariya@kde.org)
+
+  Permission is hereby granted, free of charge, to any person obtaining a copy
+  of this software and associated documentation files (the "Software"), to deal
+  in the Software without restriction, including without limitation the rights
+  to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+  copies of the Software, and to permit persons to whom the Software is
+  furnished to do so, subject to the following conditions:
+
+  The above copyright notice and this permission notice shall be included in
+  all copies or substantial portions of the Software.
+
+  THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+  IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+  FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+  AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+  LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+  OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
+  THE SOFTWARE.
+*/
+
+#if !defined(FASTLZ__COMPRESSOR) && !defined(FASTLZ_DECOMPRESSOR)
+
+/*
+ * Always check for bound when decompressing.
+ * Generally it is best to leave it defined.
+ */
+#define FASTLZ_SAFE
+
+/*
+ * Give hints to the compiler for branch prediction optimization.
+ */
+#if defined(__GNUC__) && (__GNUC__ > 2)
+#define FASTLZ_EXPECT_CONDITIONAL(c)    (__builtin_expect((c), 1))
+#define FASTLZ_UNEXPECT_CONDITIONAL(c)  (__builtin_expect((c), 0))
+#else
+#define FASTLZ_EXPECT_CONDITIONAL(c)    (c)
+#define FASTLZ_UNEXPECT_CONDITIONAL(c)  (c)
+#endif
+
+/*
+ * Use inlined functions for supported systems.
+ */
+#if defined(__GNUC__) || defined(__DMC__) || defined(__POCC__) || defined(__WATCOMC__) || defined(__SUNPRO_C)
+#define FASTLZ_INLINE inline
+#elif defined(__BORLANDC__) || defined(_MSC_VER) || defined(__LCC__)
+#define FASTLZ_INLINE __inline
+#else 
+#define FASTLZ_INLINE
+#endif
+
+/*
+ * Prevent accessing more than 8-bit at once, except on x86 architectures.
+ */
+#if !defined(FASTLZ_STRICT_ALIGN)
+#define FASTLZ_STRICT_ALIGN
+#if defined(__i386__) || defined(__386)  /* GNU C, Sun Studio */
+#undef FASTLZ_STRICT_ALIGN
+#elif defined(__i486__) || defined(__i586__) || defined(__i686__) /* GNU C */
+#undef FASTLZ_STRICT_ALIGN
+#elif defined(_M_IX86) /* Intel, MSVC */
+#undef FASTLZ_STRICT_ALIGN
+#elif defined(__386)
+#undef FASTLZ_STRICT_ALIGN
+#elif defined(_X86_) /* MinGW */
+#undef FASTLZ_STRICT_ALIGN
+#elif defined(__I86__) /* Digital Mars */
+#undef FASTLZ_STRICT_ALIGN
+#endif
+#endif
+
+/*
+ * FIXME: use preprocessor magic to set this on different platforms!
+ */
+typedef unsigned char  flzuint8;
+typedef unsigned short flzuint16;
+typedef unsigned int   flzuint32;
+
+/* prototypes */
+int fastlz_compress(const void* input, int length, void* output);
+int fastlz_compress_level(int level, const void* input, int length, void* output);
+int fastlz_decompress(const void* input, int length, void* output, int maxout);
+
+#define MAX_COPY       32
+#define MAX_LEN       264  /* 256 + 8 */
+#define MAX_DISTANCE 8192
+
+#if !defined(FASTLZ_STRICT_ALIGN)
+#define FASTLZ_READU16(p) *((const flzuint16*)(p)) 
+#else
+#define FASTLZ_READU16(p) ((p)[0] | (p)[1]<<8)
+#endif
+
+#define HASH_LOG  13
+#define HASH_SIZE (1<< HASH_LOG)
+#define HASH_MASK  (HASH_SIZE-1)
+#define HASH_FUNCTION(v,p) { v = FASTLZ_READU16(p); v ^= FASTLZ_READU16(p+1)^(v>>(16-HASH_LOG));v &= HASH_MASK; }
+
+#undef FASTLZ_LEVEL
+#define FASTLZ_LEVEL 1
+
+#undef FASTLZ_COMPRESSOR
+#undef FASTLZ_DECOMPRESSOR
+#define FASTLZ_COMPRESSOR fastlz1_compress
+#define FASTLZ_DECOMPRESSOR fastlz1_decompress
+static FASTLZ_INLINE int FASTLZ_COMPRESSOR(const void* input, int length, void* output);
+static FASTLZ_INLINE int FASTLZ_DECOMPRESSOR(const void* input, int length, void* output, int maxout);
+#include "fastlz.c"
+
+#undef FASTLZ_LEVEL
+#define FASTLZ_LEVEL 2
+
+#undef MAX_DISTANCE
+#define MAX_DISTANCE 8191
+#define MAX_FARDISTANCE (65535+MAX_DISTANCE-1)
+
+#undef FASTLZ_COMPRESSOR
+#undef FASTLZ_DECOMPRESSOR
+#define FASTLZ_COMPRESSOR fastlz2_compress
+#define FASTLZ_DECOMPRESSOR fastlz2_decompress
+static FASTLZ_INLINE int FASTLZ_COMPRESSOR(const void* input, int length, void* output);
+static FASTLZ_INLINE int FASTLZ_DECOMPRESSOR(const void* input, int length, void* output, int maxout);
+#include "fastlz.c"
+
+int fastlz_compress(const void* input, int length, void* output)
+{
+  /* for short block, choose fastlz1 */
+  if(length < 65536)
+    return fastlz1_compress(input, length, output);
+
+  /* else... */
+  return fastlz2_compress(input, length, output);
+}
+
+int fastlz_decompress(const void* input, int length, void* output, int maxout)
+{
+  /* magic identifier for compression level */
+  int level = ((*(const flzuint8*)input) >> 5) + 1;
+
+  if(level == 1)
+    return fastlz1_decompress(input, length, output, maxout);
+  if(level == 2)
+    return fastlz2_decompress(input, length, output, maxout);
+
+  /* unknown level, trigger error */
+  return 0;
+}
+
+int fastlz_compress_level(int level, const void* input, int length, void* output)
+{
+  if(level == 1)
+    return fastlz1_compress(input, length, output);
+  if(level == 2)
+    return fastlz2_compress(input, length, output);
+
+  return 0;
+}
+
+#else /* !defined(FASTLZ_COMPRESSOR) && !defined(FASTLZ_DECOMPRESSOR) */
+
+static FASTLZ_INLINE int FASTLZ_COMPRESSOR(const void* input, int length, void* output)
+{
+  const flzuint8* ip = (const flzuint8*) input;
+  const flzuint8* ip_bound = ip + length - 2;
+  const flzuint8* ip_limit = ip + length - 12;
+  flzuint8* op = (flzuint8*) output;
+
+  const flzuint8* htab[HASH_SIZE];
+  const flzuint8** hslot;
+  flzuint32 hval;
+
+  flzuint32 copy;
+
+  /* sanity check */
+  if(FASTLZ_UNEXPECT_CONDITIONAL(length < 4))
+  {
+    if(length)
+    {
+      /* create literal copy only */
+      *op++ = length-1;
+      ip_bound++;
+      while(ip <= ip_bound)
+        *op++ = *ip++;
+      return length+1;
+    }
+    else
+      return 0;
+  }
+
+  /* initializes hash table */
+  for (hslot = htab; hslot < htab + HASH_SIZE; hslot++)
+    *hslot = ip;
+
+  /* we start with literal copy */
+  copy = 2;
+  *op++ = MAX_COPY-1;
+  *op++ = *ip++;
+  *op++ = *ip++;
+
+  /* main loop */
+  while(FASTLZ_EXPECT_CONDITIONAL(ip < ip_limit))
+  {
+    const flzuint8* ref;
+    flzuint32 distance;
+
+    /* minimum match length */
+    flzuint32 len = 3;
+
+    /* comparison starting-point */
+    const flzuint8* anchor = ip;
+
+    /* check for a run */
+#if FASTLZ_LEVEL==2
+    if(ip[0] == ip[-1] && FASTLZ_READU16(ip-1)==FASTLZ_READU16(ip+1))
+    {
+      distance = 1;
+      ip += 3;
+      ref = anchor - 1 + 3;
+      goto match;
+    }
+#endif
+
+    /* find potential match */
+    HASH_FUNCTION(hval,ip);
+    hslot = htab + hval;
+    ref = htab[hval];
+
+    /* calculate distance to the match */
+    distance = anchor - ref;
+
+    /* update hash table */
+    *hslot = anchor;
+
+    /* is this a match? check the first 3 bytes */
+    if(distance==0 || 
+#if FASTLZ_LEVEL==1
+    (distance >= MAX_DISTANCE) ||
+#else
+    (distance >= MAX_FARDISTANCE) ||
+#endif
+    *ref++ != *ip++ || *ref++!=*ip++ || *ref++!=*ip++)
+      goto literal;
+
+#if FASTLZ_LEVEL==2
+    /* far, needs at least 5-byte match */
+    if(distance >= MAX_DISTANCE)
+    {
+      if(*ip++ != *ref++ || *ip++!= *ref++) 
+        goto literal;
+      len += 2;
+    }
+    
+    match:
+#endif
+
+    /* last matched byte */
+    ip = anchor + len;
+
+    /* distance is biased */
+    distance--;
+
+    if(!distance)
+    {
+      /* zero distance means a run */
+      flzuint8 x = ip[-1];
+      while(ip < ip_bound)
+        if(*ref++ != x) break; else ip++;
+    }
+    else
+    for(;;)
+    {
+      /* safe because the outer check against ip limit */
+      if(*ref++ != *ip++) break;
+      if(*ref++ != *ip++) break;
+      if(*ref++ != *ip++) break;
+      if(*ref++ != *ip++) break;
+      if(*ref++ != *ip++) break;
+      if(*ref++ != *ip++) break;
+      if(*ref++ != *ip++) break;
+      if(*ref++ != *ip++) break;
+      while(ip < ip_bound)
+        if(*ref++ != *ip++) break;
+      break;
+    }
+
+    /* if we have copied something, adjust the copy count */
+    if(copy)
+      /* copy is biased, '0' means 1 byte copy */
+      *(op-copy-1) = copy-1;
+    else
+      /* back, to overwrite the copy count */
+      op--;
+
+    /* reset literal counter */
+    copy = 0;
+
+    /* length is biased, '1' means a match of 3 bytes */
+    ip -= 3;
+    len = ip - anchor;
+
+    /* encode the match */
+#if FASTLZ_LEVEL==2
+    if(distance < MAX_DISTANCE)
+    {
+      if(len < 7)
+      {
+        *op++ = (len << 5) + (distance >> 8);
+        *op++ = (distance & 255);
+      }
+      else
+      {
+        *op++ = (7 << 5) + (distance >> 8);
+        for(len-=7; len >= 255; len-= 255)
+          *op++ = 255;
+        *op++ = len;
+        *op++ = (distance & 255);
+      }
+    }
+    else
+    {
+      /* far away, but not yet in the another galaxy... */
+      if(len < 7)
+      {
+        distance -= MAX_DISTANCE;
+        *op++ = (len << 5) + 31;
+        *op++ = 255;
+        *op++ = distance >> 8;
+        *op++ = distance & 255;
+      }
+      else
+      {
+        distance -= MAX_DISTANCE;
+        *op++ = (7 << 5) + 31;
+        for(len-=7; len >= 255; len-= 255)
+          *op++ = 255;
+        *op++ = len;
+        *op++ = 255;
+        *op++ = distance >> 8;
+        *op++ = distance & 255;
+      }
+    }
+#else
+
+    if(FASTLZ_UNEXPECT_CONDITIONAL(len > MAX_LEN-2))
+      while(len > MAX_LEN-2)
+      {
+        *op++ = (7 << 5) + (distance >> 8);
+        *op++ = MAX_LEN - 2 - 7 -2; 
+        *op++ = (distance & 255);
+        len -= MAX_LEN-2;
+      }
+
+    if(len < 7)
+    {
+      *op++ = (len << 5) + (distance >> 8);
+      *op++ = (distance & 255);
+    }
+    else
+    {
+      *op++ = (7 << 5) + (distance >> 8);
+      *op++ = len - 7;
+      *op++ = (distance & 255);
+    }
+#endif
+
+    /* update the hash at match boundary */
+    HASH_FUNCTION(hval,ip);
+    htab[hval] = ip++;
+    HASH_FUNCTION(hval,ip);
+    htab[hval] = ip++;
+
+    /* assuming literal copy */
+    *op++ = MAX_COPY-1;
+
+    continue;
+
+    literal:
+      *op++ = *anchor++;
+      ip = anchor;
+      copy++;
+      if(FASTLZ_UNEXPECT_CONDITIONAL(copy == MAX_COPY))
+      {
+        copy = 0;
+        *op++ = MAX_COPY-1;
+      }
+  }
+
+  /* left-over as literal copy */
+  ip_bound++;
+  while(ip <= ip_bound)
+  {
+    *op++ = *ip++;
+    copy++;
+    if(copy == MAX_COPY)
+    {
+      copy = 0;
+      *op++ = MAX_COPY-1;
+    }
+  }
+
+  /* if we have copied something, adjust the copy length */
+  if(copy)
+    *(op-copy-1) = copy-1;
+  else
+    op--;
+
+#if FASTLZ_LEVEL==2
+  /* marker for fastlz2 */
+  *(flzuint8*)output |= (1 << 5);
+#endif
+
+  return op - (flzuint8*)output;
+}
+
+static FASTLZ_INLINE int FASTLZ_DECOMPRESSOR(const void* input, int length, void* output, int maxout)
+{
+  const flzuint8* ip = (const flzuint8*) input;
+  const flzuint8* ip_limit  = ip + length;
+  flzuint8* op = (flzuint8*) output;
+  flzuint8* op_limit = op + maxout;
+  flzuint32 ctrl = (*ip++) & 31;
+  int loop = 1;
+
+  do
+  {
+    const flzuint8* ref = op;
+    flzuint32 len = ctrl >> 5;
+    flzuint32 ofs = (ctrl & 31) << 8;
+
+    if(ctrl >= 32)
+    {
+#if FASTLZ_LEVEL==2
+      flzuint8 code;
+#endif
+      len--;
+      ref -= ofs;
+      if (len == 7-1)
+#if FASTLZ_LEVEL==1
+        len += *ip++;
+      ref -= *ip++;
+#else
+        do
+        {
+          code = *ip++;
+          len += code;
+        } while (code==255);
+      code = *ip++;
+      ref -= code;
+
+      /* match from 16-bit distance */
+      if(FASTLZ_UNEXPECT_CONDITIONAL(code==255))
+      if(FASTLZ_EXPECT_CONDITIONAL(ofs==(31 << 8)))
+      {
+        ofs = (*ip++) << 8;
+        ofs += *ip++;
+        ref = op - ofs - MAX_DISTANCE;
+      }
+#endif
+      
+#ifdef FASTLZ_SAFE
+      if (FASTLZ_UNEXPECT_CONDITIONAL(op + len + 3 > op_limit))
+        return 0;
+
+      if (FASTLZ_UNEXPECT_CONDITIONAL(ref-1 < (flzuint8 *)output))
+        return 0;
+#endif
+
+      if(FASTLZ_EXPECT_CONDITIONAL(ip < ip_limit))
+        ctrl = *ip++;
+      else
+        loop = 0;
+
+      if(ref == op)
+      {
+        /* optimize copy for a run */
+        flzuint8 b = ref[-1];
+        *op++ = b;
+        *op++ = b;
+        *op++ = b;
+        for(; len; --len)
+          *op++ = b;
+      }
+      else
+      {
+#if !defined(FASTLZ_STRICT_ALIGN)
+        const flzuint16* p;
+        flzuint16* q;
+#endif
+        /* copy from reference */
+        ref--;
+        *op++ = *ref++;
+        *op++ = *ref++;
+        *op++ = *ref++;
+
+#if !defined(FASTLZ_STRICT_ALIGN)
+        /* copy a byte, so that now it's word aligned */
+        if(len & 1)
+        {
+          *op++ = *ref++;
+          len--;
+        }
+
+        /* copy 16-bit at once */
+        q = (flzuint16*) op;
+        op += len;
+        p = (const flzuint16*) ref;
+        for(len>>=1; len > 4; len-=4)
+        {
+          *q++ = *p++;
+          *q++ = *p++;
+          *q++ = *p++;
+          *q++ = *p++;
+        }
+        for(; len; --len)
+          *q++ = *p++;
+#else
+        for(; len; --len)
+          *op++ = *ref++;
+#endif
+      }
+    }
+    else
+    {
+      ctrl++;
+#ifdef FASTLZ_SAFE
+      if (FASTLZ_UNEXPECT_CONDITIONAL(op + ctrl > op_limit))
+        return 0;
+      if (FASTLZ_UNEXPECT_CONDITIONAL(ip + ctrl > ip_limit))
+        return 0;
+#endif
+
+      *op++ = *ip++; 
+      for(--ctrl; ctrl; ctrl--)
+        *op++ = *ip++;
+
+      loop = FASTLZ_EXPECT_CONDITIONAL(ip < ip_limit);
+      if(loop)
+        ctrl = *ip++;
+    }
+  }
+  while(FASTLZ_EXPECT_CONDITIONAL(loop));
+
+  return op - (flzuint8*)output;
+}
+
+#endif /* !defined(FASTLZ_COMPRESSOR) && !defined(FASTLZ_DECOMPRESSOR) */

Added: trafficserver/traffic/trunk/libinktomi++/fastlz.h
URL: http://svn.apache.org/viewvc/trafficserver/traffic/trunk/libinktomi%2B%2B/fastlz.h?rev=941885&view=auto
==============================================================================
--- trafficserver/traffic/trunk/libinktomi++/fastlz.h (added)
+++ trafficserver/traffic/trunk/libinktomi++/fastlz.h Thu May  6 19:58:27 2010
@@ -0,0 +1,100 @@
+/*  
+  FastLZ - lightning-fast lossless compression library
+
+  Copyright (C) 2007 Ariya Hidayat (ariya@kde.org)
+  Copyright (C) 2006 Ariya Hidayat (ariya@kde.org)
+  Copyright (C) 2005 Ariya Hidayat (ariya@kde.org)
+
+  Permission is hereby granted, free of charge, to any person obtaining a copy
+  of this software and associated documentation files (the "Software"), to deal
+  in the Software without restriction, including without limitation the rights
+  to use, copy, modify, merge, publish, distribute, sublicense, and/or sell
+  copies of the Software, and to permit persons to whom the Software is
+  furnished to do so, subject to the following conditions:
+
+  The above copyright notice and this permission notice shall be included in
+  all copies or substantial portions of the Software.
+
+  THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR
+  IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY,
+  FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE
+  AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
+  LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
+  OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
+  THE SOFTWARE.
+*/
+
+#ifndef FASTLZ_H
+#define FASTLZ_H
+
+#define FASTLZ_VERSION 0x000100
+
+#define FASTLZ_VERSION_MAJOR     0
+#define FASTLZ_VERSION_MINOR     0
+#define FASTLZ_VERSION_REVISION  0
+
+#define FASTLZ_VERSION_STRING "0.1.0"
+
+#if defined (__cplusplus)
+extern "C" {
+#endif
+
+/**
+  Compress a block of data in the input buffer and returns the size of 
+  compressed block. The size of input buffer is specified by length. The 
+  minimum input buffer size is 16.
+
+  The output buffer must be at least 5% larger than the input buffer  
+  and can not be smaller than 66 bytes.
+
+  If the input is not compressible, the return value might be larger than
+  length (input buffer size).
+
+  The input buffer and the output buffer can not overlap.
+*/
+
+int fastlz_compress(const void* input, int length, void* output);
+
+/**
+  Decompress a block of compressed data and returns the size of the 
+  decompressed block. If error occurs, e.g. the compressed data is 
+  corrupted or the output buffer is not large enough, then 0 (zero) 
+  will be returned instead.
+
+  The input buffer and the output buffer can not overlap.
+
+  Decompression is memory safe and guaranteed not to write the output buffer
+  more than what is specified in maxout.
+ */
+
+int fastlz_decompress(const void* input, int length, void* output, int maxout); 
+
+/**
+  Compress a block of data in the input buffer and returns the size of 
+  compressed block. The size of input buffer is specified by length. The 
+  minimum input buffer size is 16.
+
+  The output buffer must be at least 5% larger than the input buffer  
+  and can not be smaller than 66 bytes.
+
+  If the input is not compressible, the return value might be larger than
+  length (input buffer size).
+
+  The input buffer and the output buffer can not overlap.
+
+  Compression level can be specified in parameter level. At the moment, 
+  only level 1 and level 2 are supported.
+  Level 1 is the fastest compression and generally useful for short data.
+  Level 2 is slightly slower but it gives better compression ratio.
+
+  Note that the compressed data, regardless of the level, can always be
+  decompressed using the function fastlz_decompress above.
+*/  
+
+int fastlz_compress_level(int level, const void* input, int length, void* output);
+
+#if defined (__cplusplus)
+}
+#endif
+
+#endif /* FASTLZ_H */

Modified: trafficserver/traffic/trunk/libinktomi++/inktomi++.h
URL: http://svn.apache.org/viewvc/trafficserver/traffic/trunk/libinktomi%2B%2B/inktomi%2B%2B.h?rev=941885&r1=941884&r2=941885&view=diff
==============================================================================
--- trafficserver/traffic/trunk/libinktomi++/inktomi++.h (original)
+++ trafficserver/traffic/trunk/libinktomi++/inktomi++.h Thu May  6 19:58:27 2010
@@ -117,6 +117,7 @@ extern "C"
 #include "ink_thread.h"
 #include "ink_time.h"
 #include "ink_unused.h"
+#include "fastlz.h"
 
 #include "Allocator.h"
 #include "Arena.h"

Added: trafficserver/traffic/trunk/m4/ax_check_zlib.m4
URL: http://svn.apache.org/viewvc/trafficserver/traffic/trunk/m4/ax_check_zlib.m4?rev=941885&view=auto
==============================================================================
--- trafficserver/traffic/trunk/m4/ax_check_zlib.m4 (added)
+++ trafficserver/traffic/trunk/m4/ax_check_zlib.m4 Thu May  6 19:58:27 2010
@@ -0,0 +1,125 @@
+# ===========================================================================
+#       http://www.gnu.org/software/autoconf-archive/ax_check_zlib.html
+# ===========================================================================
+#
+# SYNOPSIS
+#
+#   AX_CHECK_ZLIB()
+#
+# DESCRIPTION
+#
+#   This macro searches for an installed zlib library. If nothing was
+#   specified when calling configure, it searches first in /usr/local and
+#   then in /usr. If the --with-zlib=DIR is specified, it will try to find
+#   it in DIR/include/zlib.h and DIR/lib/libz.a. If --without-zlib is
+#   specified, the library is not searched at all.
+#
+#   If either the header file (zlib.h) or the library (libz) is not found,
+#   the configuration exits on error, asking for a valid zlib installation
+#   directory or --without-zlib.
+#
+#   The macro defines the symbol HAVE_LIBZ if the library is found. You
+#   should use autoheader to include a definition for this symbol in a
+#   config.h file. Sample usage in a C/C++ source is as follows:
+#
+#     #ifdef HAVE_LIBZ
+#     #include <zlib.h>
+#     #endif /* HAVE_LIBZ */
+#
+# LICENSE
+#
+#   Copyright (c) 2008 Loic Dachary <lo...@senga.org>
+#
+#   This program is free software; you can redistribute it and/or modify it
+#   under the terms of the GNU General Public License as published by the
+#   Free Software Foundation; either version 2 of the License, or (at your
+#   option) any later version.
+#
+#   This program is distributed in the hope that it will be useful, but
+#   WITHOUT ANY WARRANTY; without even the implied warranty of
+#   MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU General
+#   Public License for more details.
+#
+#   You should have received a copy of the GNU General Public License along
+#   with this program. If not, see <http://www.gnu.org/licenses/>.
+#
+#   As a special exception, the respective Autoconf Macro's copyright owner
+#   gives unlimited permission to copy, distribute and modify the configure
+#   scripts that are the output of Autoconf when processing the Macro. You
+#   need not follow the terms of the GNU General Public License when using
+#   or distributing such scripts, even though portions of the text of the
+#   Macro appear in them. The GNU General Public License (GPL) does govern
+#   all other use of the material that constitutes the Autoconf Macro.
+#
+#   This special exception to the GPL applies to versions of the Autoconf
+#   Macro released by the Autoconf Archive. When you make and distribute a
+#   modified version of the Autoconf Macro, you may extend this special
+#   exception to the GPL to apply to your modified version as well.
+
+#serial 6
+
+AU_ALIAS([CHECK_ZLIB], [AX_CHECK_ZLIB])
+AC_DEFUN([AX_CHECK_ZLIB],
+#
+# Handle user hints
+#
+[AC_MSG_CHECKING(if zlib is wanted)
+AC_ARG_WITH(zlib,
+[  --with-zlib=DIR root directory path of zlib installation [defaults to
+                    /usr/local or /usr if not found in /usr/local]
+  --without-zlib to disable zlib usage completely],
+[if test "$withval" != no ; then
+  AC_MSG_RESULT(yes)
+  if test -d "$withval"
+  then
+    ZLIB_HOME="$withval"
+  else
+    AC_MSG_WARN([Sorry, $withval does not exist, checking usual places])
+  fi
+else
+  AC_MSG_RESULT(no)
+fi], 
+[AC_MSG_RESULT(yes)])
+
+ZLIB_HOME=/usr/local
+if test ! -f "${ZLIB_HOME}/include/zlib.h"
+then
+        ZLIB_HOME=/usr
+fi
+
+#
+# Locate zlib, if wanted
+#
+if test -n "${ZLIB_HOME}"
+then
+        ZLIB_OLD_LDFLAGS=$LDFLAGS
+        ZLIB_OLD_CPPFLAGS=$LDFLAGS
+        LDFLAGS="$LDFLAGS -L${ZLIB_HOME}/lib"
+        CPPFLAGS="$CPPFLAGS -I${ZLIB_HOME}/include"
+        AC_LANG_SAVE
+        AC_LANG_C
+        AC_CHECK_LIB(z, inflateEnd, [zlib_cv_libz=yes], [zlib_cv_libz=no])
+        AC_CHECK_HEADER(zlib.h, [zlib_cv_zlib_h=yes], [zlib_cv_zlib_h=no])
+        AC_LANG_RESTORE
+        if test "$zlib_cv_libz" = "yes" -a "$zlib_cv_zlib_h" = "yes"
+        then
+                #
+                # If both library and header were found, use them
+                #
+                AC_CHECK_LIB(z, inflateEnd)
+                AC_MSG_CHECKING(zlib in ${ZLIB_HOME})
+                AC_MSG_RESULT(ok)
+        else
+                #
+                # If either header or library was not found, revert and bomb
+                #
+                AC_MSG_CHECKING(zlib in ${ZLIB_HOME})
+                LDFLAGS="$ZLIB_OLD_LDFLAGS"
+                CPPFLAGS="$ZLIB_OLD_CPPFLAGS"
+                AC_MSG_RESULT(failed)
+                AC_MSG_ERROR(either specify a valid zlib installation with --with-zlib=DIR or disable zlib usage with --without-zlib)
+        fi
+fi
+
+])
+

Modified: trafficserver/traffic/trunk/proxy/signals.cc
URL: http://svn.apache.org/viewvc/trafficserver/traffic/trunk/proxy/signals.cc?rev=941885&r1=941884&r2=941885&view=diff
==============================================================================
--- trafficserver/traffic/trunk/proxy/signals.cc (original)
+++ trafficserver/traffic/trunk/proxy/signals.cc Thu May  6 19:58:27 2010
@@ -86,7 +86,6 @@ public:
     if (sigusr1_received) {
       sigusr1_received = 0;
 
-      Cache::print_stats(stderr);
       xdump_to_file(stderr);
       ink_freelists_dump(stderr);
       if (!end)