You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@trafficserver.apache.org by ge...@apache.org on 2010/02/24 19:48:52 UTC

svn commit: r915922 [3/14] - in /incubator/trafficserver/traffic/trunk: ./ install/ iocore/aio/ iocore/block-cache/ iocore/cache/ iocore/cluster/ iocore/dns/ iocore/eventsystem/ iocore/hostdb/ iocore/net/ iocore/utils/ libev/ libev/CVS/ libinktomi++/ l...

Modified: incubator/trafficserver/traffic/trunk/iocore/cache/CacheDir.cc
URL: http://svn.apache.org/viewvc/incubator/trafficserver/traffic/trunk/iocore/cache/CacheDir.cc?rev=915922&r1=915921&r2=915922&view=diff
==============================================================================
--- incubator/trafficserver/traffic/trunk/iocore/cache/CacheDir.cc (original)
+++ incubator/trafficserver/traffic/trunk/iocore/cache/CacheDir.cc Wed Feb 24 18:48:42 2010
@@ -30,9 +30,6 @@
 #endif
 #include "ink_stack_trace.h"
 
-#define SYNC_MAX_WRITE                (256 * 1024)
-#define SYNC_DELAY                    HRTIME_MSECONDS(500)
-
 #define CACHE_INC_DIR_USED(_m) do { \
 ProxyMutex *mutex = _m; \
 CACHE_INCREMENT_DYN_STAT(cache_direntries_used_stat); \
@@ -49,15 +46,10 @@
 } while (0);
 
 
-// Debugging Options
-
-// #define CHECK_DIR_FAST
-// #define CHECK_DIR
-
 // Globals
 
 ClassAllocator<OpenDirEntry> openDirEntryAllocator("openDirEntry");
-Dir empty_dir = { 0 };
+Dir empty_dir;
 
 // OpenDir
 
@@ -74,12 +66,12 @@
    Returns 1 on success and 0 on failure.
    */
 int
-OpenDir::open_write(CacheVC * cont, int allow_if_writers, int max_writers)
+OpenDir::open_write(CacheVC *cont, int allow_if_writers, int max_writers)
 {
   ink_debug_assert(cont->part->mutex->thread_holding == this_ethread());
   unsigned int h = cont->first_key.word(0);
   int b = h % OPEN_DIR_BUCKETS;
-  for (OpenDirEntry * d = bucket[b].head; d; d = d->link.next) {
+  for (OpenDirEntry *d = bucket[b].head; d; d = d->link.next) {
     if (!(d->writers.head->first_key == cont->first_key))
       continue;
     if (allow_if_writers && d->num_writers < d->max_writers) {
@@ -110,7 +102,7 @@
 }
 
 int
-OpenDir::signal_readers(int event, Event * e)
+OpenDir::signal_readers(int event, Event *e)
 {
   NOWARN_UNUSED(e);
   NOWARN_UNUSED(event);
@@ -142,7 +134,7 @@
 }
 
 int
-OpenDir::close_write(CacheVC * cont)
+OpenDir::close_write(CacheVC *cont)
 {
   ink_debug_assert(cont->part->mutex->thread_holding == this_ethread());
   cont->od->writers.remove(cont, cont->opendir_link);
@@ -161,18 +153,18 @@
 }
 
 OpenDirEntry *
-OpenDir::open_read(INK_MD5 * key)
+OpenDir::open_read(INK_MD5 *key)
 {
   unsigned int h = key->word(0);
   int b = h % OPEN_DIR_BUCKETS;
-  for (OpenDirEntry * d = bucket[b].head; d; d = d->link.next)
+  for (OpenDirEntry *d = bucket[b].head; d; d = d->link.next)
     if (d->writers.head->first_key == *key)
       return d;
   return NULL;
 }
 
 int
-OpenDirEntry::wait(CacheVC * cont, int msec)
+OpenDirEntry::wait(CacheVC *cont, int msec)
 {
   ink_debug_assert(cont->part->mutex->thread_holding == this_ethread());
   cont->f.open_read_timeout = 1;
@@ -189,7 +181,7 @@
 // return value 1 means no loop
 // zero indicates loop
 int
-dir_bucket_loop_check(Dir * start_dir, Dir * seg)
+dir_bucket_loop_check(Dir *start_dir, Dir *seg)
 {
   if (start_dir == NULL)
     return 1;
@@ -216,7 +208,7 @@
 // adds all the directory entries
 // in a segment to the segment freelist
 void
-dir_init_segment(int s, Part * d)
+dir_init_segment(int s, Part *d)
 {
   d->header->freelist[s] = 0;
   Dir *seg = dir_segment(s, d);
@@ -225,7 +217,7 @@
   for (l = 1; l < DIR_DEPTH; l++) {
     for (b = 0; b < d->buckets; b++) {
       Dir *bucket = dir_bucket(b, seg);
-      dir_free_entry(&bucket[l], s, d);
+      dir_free_entry(dir_bucket_row(bucket, l), s, d);
     }
   }
 }
@@ -235,7 +227,7 @@
 // Note : abuse of the token bit in dir entries
 #if 0
 int
-dir_bucket_loop_fix(Dir * start_dir, int s, Part * d)
+dir_bucket_loop_fix(Dir *start_dir, int s, Part *d)
 {
   int ret = 0;
   if (start_dir == NULL)
@@ -257,13 +249,13 @@
     }
     p1 = p2;
   }
-  for (Dir * p3 = start_dir; p3; p3 = next_dir(p3, seg))
+  for (Dir *p3 = start_dir; p3; p3 = next_dir(p3, seg))
     dir_set_token(p3, 0);
   return ret;
 }
 #else
 int
-dir_bucket_loop_fix(Dir * start_dir, int s, Part * d)
+dir_bucket_loop_fix(Dir *start_dir, int s, Part *d)
 {
   if (!dir_bucket_loop_check(start_dir, dir_segment(s, d))) {
     Warning("Dir loop exists, clearing segment %d", s);
@@ -275,7 +267,7 @@
 #endif
 
 int
-dir_freelist_length(Part * d, int s)
+dir_freelist_length(Part *d, int s)
 {
   int free = 0;
   Dir *seg = dir_segment(s, d);
@@ -290,7 +282,7 @@
 }
 
 int
-dir_bucket_length(Dir * b, int s, Part * d)
+dir_bucket_length(Dir *b, int s, Part *d)
 {
   Dir *e = b;
   int i = 0;
@@ -309,11 +301,11 @@
 }
 
 int
-check_dir(Part * d)
+check_dir(Part *d)
 {
   int i, s;
   Debug("cache_check_dir", "inside check dir");
-  for (s = 0; s < DIR_SEGMENTS; s++) {
+  for (s = 0; s < d->segments; s++) {
     Dir *seg = dir_segment(s, d);
     for (i = 0; i < d->buckets; i++) {
       Dir *b = dir_bucket(i, seg);
@@ -326,7 +318,7 @@
 }
 
 inline void
-unlink_from_freelist(Dir * e, int s, Part * d)
+unlink_from_freelist(Dir *e, int s, Part *d)
 {
   Dir *seg = dir_segment(s, d);
   Dir *p = dir_from_offset(dir_prev(e), seg);
@@ -340,7 +332,7 @@
 }
 
 inline Dir *
-dir_delete_entry(Dir * e, Dir * p, int s, Part * d)
+dir_delete_entry(Dir *e, Dir *p, int s, Part *d)
 {
   Dir *seg = dir_segment(s, d);
   int no = dir_next(e);
@@ -369,7 +361,7 @@
 }
 
 inline void
-dir_clean_bucket(Dir * b, int s, Part * part)
+dir_clean_bucket(Dir *b, int s, Part *part)
 {
   Dir *e = b, *p = NULL;
   Dir *seg = dir_segment(s, part);
@@ -390,7 +382,6 @@
               (long) e, dir_tag(e), (int) dir_offset(e), (long) b, (long) p, dir_bucket_length(b, s, part));
       if (dir_offset(e))
         CACHE_DEC_DIR_USED(part->mutex);
-
       e = dir_delete_entry(e, p, s, part);
       continue;
     }
@@ -400,7 +391,7 @@
 }
 
 void
-dir_clean_segment(int s, Part * d)
+dir_clean_segment(int s, Part *d)
 {
   Dir *seg = dir_segment(s, d);
   for (int i = 0; i < d->buckets; i++) {
@@ -410,23 +401,20 @@
 }
 
 void
-dir_clean_part(Part * d)
+dir_clean_part(Part *d)
 {
-  for (int i = 0; i < DIR_SEGMENTS; i++)
+  for (int i = 0; i < d->segments; i++)
     dir_clean_segment(i, d);
-#if defined(DEBUG) && defined(CHECK_DIR)
-  check_dir(d);
-#endif
+  CHECK_DIR(d);
 }
 
 void
-dir_clear_range(int start, int end, Part * part)
+dir_clear_range(ink_off_t start, ink_off_t end, Part *part)
 {
-  for (int i = 0; i < part->buckets * DIR_DEPTH * DIR_SEGMENTS; i++) {
+  for (int i = 0; i < part->buckets * DIR_DEPTH * part->segments; i++) {
     Dir *e = dir_index(part, i);
     if (!dir_token(e) && (int) dir_offset(e) >= start && (int) dir_offset(e) < end) {
       CACHE_DEC_DIR_USED(part->mutex);
-
       dir_set_offset(e, 0);     // delete
     }
   }
@@ -434,7 +422,7 @@
 }
 
 void
-check_bucket_not_contains(Dir * b, Dir * e, Dir * seg)
+check_bucket_not_contains(Dir *b, Dir *e, Dir *seg)
 {
   Dir *x = b;
   do {
@@ -446,7 +434,7 @@
 }
 
 void
-freelist_clean(int s, Part * part)
+freelist_clean(int s, Part *part)
 {
   dir_clean_segment(s, part);
   if (part->header->freelist[s])
@@ -460,7 +448,6 @@
       Dir *e = dir_bucket_row(b, l);
       if (dir_head(e) && !(n++ % 10)) {
         CACHE_DEC_DIR_USED(part->mutex);
-
         dir_set_offset(e, 0);   // delete
       }
     }
@@ -469,7 +456,7 @@
 }
 
 inline Dir *
-freelist_pop(int s, Part * d)
+freelist_pop(int s, Part *d)
 {
   Dir *seg = dir_segment(s, d);
   Dir *e = dir_from_offset(d->header->freelist[s], seg);
@@ -490,7 +477,7 @@
 }
 
 int
-dir_segment_accounted(int s, Part * d, int offby, int *f, int *u, int *et, int *v, int *av, int *as)
+dir_segment_accounted(int s, Part *d, int offby, int *f, int *u, int *et, int *v, int *av, int *as)
 {
   int free = dir_freelist_length(d, s);
   int used = 0, empty = 0;
@@ -534,7 +521,7 @@
 }
 
 void
-dir_free_entry(Dir * e, int s, Part * d)
+dir_free_entry(Dir *e, int s, Part *d)
 {
   Dir *seg = dir_segment(s, d);
   unsigned int fo = d->header->freelist[s];
@@ -546,17 +533,15 @@
 }
 
 int
-dir_probe(CacheKey * key, Part * d, Dir * result, Dir ** last_collision)
+dir_probe(CacheKey *key, Part *d, Dir *result, Dir ** last_collision)
 {
   ink_debug_assert(d->mutex->thread_holding == this_ethread());
-  int s = key->word(0) % DIR_SEGMENTS;
-  int b = (key->word(0) / DIR_SEGMENTS) % d->buckets;
+  int s = key->word(0) % d->segments;
+  int b = key->word(1) % d->buckets;
   Dir *seg = dir_segment(s, d);
   Dir *e = NULL, *p = NULL, *collision = *last_collision;
   Part *part = d;
-#if defined(DEBUG) && defined(CHECK_DIR)
-  check_dir(d);
-#endif
+  CHECK_DIR(d);
 #ifdef LOOP_CHECK_MODE
   if (dir_bucket_loop_fix(dir_bucket(b, seg), s, d))
     return 0;
@@ -578,13 +563,13 @@
             // for the same document and so the collision stat
             // may not accurately reflect the number of documents
             // having the same first_key
-            Debug("cache_stats", "Incrementing dir collisions");
+            DDebug("cache_stats", "Incrementing dir collisions");
             CACHE_INC_DIR_COLLISIONS(d->mutex);
           }
           goto Lcont;
         }
         if (dir_valid(d, e)) {
-          Debug("dir_probe_hit", "found %X part %d bucket %d  boffset %d", key->word(0), d->fd, b, (int) dir_offset(e));
+          DDebug("dir_probe_hit", "found %X %X part %d bucket %d  boffset %d", key->word(0), key->word(1), d->fd, b, (int) dir_offset(e));
           dir_assign(result, e);
           *last_collision = e;
           ink_assert(dir_offset(e) * INK_BLOCK_SIZE < d->len);
@@ -595,46 +580,42 @@
           continue;
         }
       } else
-        Debug("dir_probe_tag", "tag mismatch %X %X vs expected %X", e, dir_tag(e), key->word(1));
+        DDebug("dir_probe_tag", "tag mismatch %X %X vs expected %X", e, dir_tag(e), key->word(3));
     Lcont:
       p = e;
       e = next_dir(e, seg);
     } while (e);
   if (collision) {              // last collision no longer in the list, retry
-    Debug("cache_stats", "Incrementing dir collisions");
+    DDebug("cache_stats", "Incrementing dir collisions");
     CACHE_INC_DIR_COLLISIONS(d->mutex);
     collision = NULL;
     goto Lagain;
   }
-  Debug("dir_probe_miss", "missed %X on part %d bucket %d at %X", key->word(0), d->fd, b, (long) seg);
-#if defined(DEBUG) && defined(CHECK_DIR)
-  check_dir(d);
-#endif
+  DDebug("dir_probe_miss", "missed %X %X on part %d bucket %d at %X", key->word(0), key->word(1), d->fd, b, (long) seg);
+  CHECK_DIR(d);
   return 0;
 }
 
 int
-dir_insert(CacheKey * key, Part * d, Dir * to_part)
+dir_insert(CacheKey *key, Part *d, Dir *to_part)
 {
   ink_debug_assert(d->mutex->thread_holding == this_ethread());
-  int s = key->word(0) % DIR_SEGMENTS, l;
-  int bi = (key->word(0) / DIR_SEGMENTS) % d->buckets;
-  ink_assert((unsigned int) dir_approx_size(to_part) <= (unsigned int) (MAX_FRAG_SIZE + sizeofDoc));    // XXX - size should be unsigned
+  int s = key->word(0) % d->segments, l;
+  int bi = key->word(1) % d->buckets;
+  ink_assert(dir_approx_size(to_part) <= MAX_FRAG_SIZE + sizeofDoc);
   Dir *seg = dir_segment(s, d);
   Dir *e = NULL;
   Dir *b = dir_bucket(bi, seg);
   Part *part = d;
-#if defined(DEBUG) && defined(CHECK_DIR_FAST)
-  unsigned int t = DIR_MASK_TAG(key->word(1));
+#if defined(DEBUG) && defined(DO_CHECK_DIR_FAST)
+  unsigned int t = DIR_MASK_TAG(key->word(2));
   Dir *col = b;
   while (col) {
     ink_assert((dir_tag(col) != t) || (dir_offset(col) != dir_offset(to_part)));
     col = next_dir(col, seg);
   }
 #endif
-#if defined(DEBUG) && defined(CHECK_DIR)
-  check_dir(d);
-#endif
+  CHECK_DIR(d);
 
 Lagain:
   // get from this row first
@@ -657,38 +638,34 @@
   dir_set_next(b, dir_to_offset(e, seg));
 Lfill:
   dir_assign_data(e, to_part);
-  dir_set_tag(e, key->word(1));
+  dir_set_tag(e, key->word(2));
   ink_assert(part_offset(d, e) < (d->skip + d->len));
-  Debug("dir_insert",
+  DDebug("dir_insert",
         "insert %X %X into part %d bucket %d at %X tag %X %X boffset %d",
         (long) e, key->word(0), d->fd, bi, (long) e, key->word(1), dir_tag(e), (int) dir_offset(e));
-#if defined(DEBUG) && defined(CHECK_DIR)
-  check_dir(d);
-#endif
+  CHECK_DIR(d);
   d->header->dirty = 1;
   CACHE_INC_DIR_USED(d->mutex);
   return 1;
 }
 
 int
-dir_overwrite(CacheKey * key, Part * d, Dir * dir, Dir * overwrite, bool must_overwrite)
+dir_overwrite(CacheKey *key, Part *d, Dir *dir, Dir *overwrite, bool must_overwrite)
 {
   ink_debug_assert(d->mutex->thread_holding == this_ethread());
-  int s = key->word(0) % DIR_SEGMENTS, l;
-  int bi = (key->word(0) / DIR_SEGMENTS) % d->buckets;
+  int s = key->word(0) % d->segments, l;
+  int bi = key->word(1) % d->buckets;
   Dir *seg = dir_segment(s, d);
   Dir *e = NULL;
   Dir *b = dir_bucket(bi, seg);
-  unsigned int t = DIR_MASK_TAG(key->word(1));
+  unsigned int t = DIR_MASK_TAG(key->word(2));
   int res = 1;
 #ifdef LOOP_CHECK_MODE
   int loop_count = 0;
   bool loop_possible = true;
 #endif
   Part *part = d;
-#if defined(DEBUG) && defined(CHECK_DIR)
-  check_dir(d);
-#endif
+  CHECK_DIR(d);
 
   ink_assert((unsigned int) dir_approx_size(dir) <= (unsigned int) (MAX_FRAG_SIZE + sizeofDoc));        // XXX - size should be unsigned
 Lagain:
@@ -737,31 +714,27 @@
   dir_assign_data(e, dir);
   dir_set_tag(e, t);
   ink_assert(part_offset(d, e) < d->skip + d->len);
-  Debug("dir_overwrite",
+  DDebug("dir_overwrite",
         "overwrite %X %X into part %d bucket %d at %X tag %X %X boffset %d",
         (long) e, key->word(0), d->fd, bi, (long) e, t, dir_tag(e), (int) dir_offset(e));
-#if defined(DEBUG) && defined(CHECK_DIR)
-  check_dir(d);
-#endif
+  CHECK_DIR(d);
   d->header->dirty = 1;
   return res;
 }
 
 int
-dir_delete(CacheKey * key, Part * d, Dir * del)
+dir_delete(CacheKey *key, Part *d, Dir *del)
 {
   ink_debug_assert(d->mutex->thread_holding == this_ethread());
-  int s = key->word(0) % DIR_SEGMENTS;
-  int b = (key->word(0) / DIR_SEGMENTS) % d->buckets;
+  int s = key->word(0) % d->segments;
+  int b = key->word(1) % d->buckets;
   Dir *seg = dir_segment(s, d);
   Dir *e = NULL, *p = NULL;
 #ifdef LOOP_CHECK_MODE
   int loop_count = 0;
 #endif
   Part *part = d;
-#if defined(DEBUG) && defined(CHECK_DIR)
-  check_dir(d);
-#endif
+  CHECK_DIR(d);
 
   e = dir_bucket(b, seg);
   if (dir_offset(e))
@@ -776,24 +749,20 @@
       if (dir_compare_tag(e, key) && dir_offset(e) == dir_offset(del)) {
         CACHE_DEC_DIR_USED(d->mutex);
         dir_delete_entry(e, p, s, d);
-#if defined(DEBUG) && defined(CHECK_DIR)
-        check_dir(d);
-#endif
+        CHECK_DIR(d);
         return 1;
       }
       p = e;
       e = next_dir(e, seg);
     } while (e);
-#if defined(DEBUG) && defined(CHECK_DIR)
-  check_dir(d);
-#endif
+  CHECK_DIR(d);
   return 0;
 }
 
 // Lookaside Cache
 
 int
-dir_lookaside_probe(CacheKey * key, Part * d, Dir * result, EvacuationBlock ** eblock)
+dir_lookaside_probe(CacheKey *key, Part *d, Dir *result, EvacuationBlock ** eblock)
 {
   ink_debug_assert(d->mutex->thread_holding == this_ethread());
   int i = key->word(3) % LOOKASIDE_SIZE;
@@ -802,7 +771,7 @@
     if (b->evac_frags.key == *key) {
       if (dir_valid(d, &b->new_dir)) {
         *result = b->new_dir;
-        Debug("dir_lookaside", "probe %X success", key->word(0));
+        DDebug("dir_lookaside", "probe %X success", key->word(0));
         if (eblock)
           *eblock = b;
         return 1;
@@ -810,15 +779,15 @@
     }
     b = b->link.next;
   }
-  Debug("dir_lookaside", "probe %X failed", key->word(0));
+  DDebug("dir_lookaside", "probe %X failed", key->word(0));
   return 0;
 }
 
 int
-dir_lookaside_insert(EvacuationBlock * eblock, Part * d, Dir * to)
+dir_lookaside_insert(EvacuationBlock *eblock, Part *d, Dir *to)
 {
   CacheKey *key = &eblock->evac_frags.earliest_key;
-  Debug("dir_lookaside", "insert %X, offset %d phase %d", key->word(0), (int) dir_offset(to), (int) dir_phase(to));
+  DDebug("dir_lookaside", "insert %X %X, offset %d phase %d", key->word(0), key->word(1), (int) dir_offset(to), (int) dir_phase(to));
   ink_debug_assert(d->mutex->thread_holding == this_ethread());
   int i = key->word(3) % LOOKASIDE_SIZE;
   EvacuationBlock *b = new_EvacuationBlock(d->mutex->thread_holding);
@@ -833,7 +802,7 @@
 }
 
 int
-dir_lookaside_fixup(CacheKey * key, Part * d)
+dir_lookaside_fixup(CacheKey *key, Part *d)
 {
   ink_debug_assert(d->mutex->thread_holding == this_ethread());
   int i = key->word(3) % LOOKASIDE_SIZE;
@@ -841,16 +810,16 @@
   while (b) {
     if (b->evac_frags.key == *key) {
       int res = dir_overwrite(key, d, &b->new_dir, &b->dir, false);
-      Debug("dir_lookaside", "fixup %X offset %d phase %d %d",
-            key->word(0), dir_offset(&b->new_dir), dir_phase(&b->new_dir), res);
+      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->lookaside[i].remove(b);
 #if 0
       // we need to do this because in case of a small cache, the scan
       // might have occured before we inserted this directory entry (if we 
       // wrapped around fast enough)
-      int part_end_offset = offset_to_part_offset(d, d->len + d->skip);
-      int part_write_offset = offset_to_part_offset(d, d->header->write_pos);
+      ink_off_t part_end_offset = offset_to_part_offset(d, d->len + d->skip);
+      ink_off_t part_write_offset = offset_to_part_offset(d, d->header->write_pos);
       if ((dir_offset(&b->new_dir) + part_end_offset - part_write_offset)
           % part_end_offset <= offset_to_part_offset(d, EVAC_SIZE + (d->len / PIN_SCAN_EVERY)))
         d->force_evacuate_head(&b->new_dir, dir_pinned(&b->new_dir));
@@ -860,12 +829,12 @@
     }
     b = b->link.next;
   }
-  Debug("dir_lookaside", "fixup %X failed", key->word(0));
+  DDebug("dir_lookaside", "fixup %X %X failed", key->word(0), key->word(1));
   return 0;
 }
 
 void
-dir_lookaside_cleanup(Part * d)
+dir_lookaside_cleanup(Part *d)
 {
   ink_debug_assert(d->mutex->thread_holding == this_ethread());
   for (int i = 0; i < LOOKASIDE_SIZE; i++) {
@@ -873,7 +842,8 @@
     while (b) {
       if (!dir_valid(d, &b->new_dir)) {
         EvacuationBlock *nb = b->link.next;
-        Debug("dir_lookaside", "cleanup %X cleaned up", b->evac_frags.earliest_key.word(0));
+        DDebug("dir_lookaside", "cleanup %X %X cleaned up", 
+              b->evac_frags.earliest_key.word(0), b->evac_frags.earliest_key.word(1));
         d->lookaside[i].remove(b);
         free_CacheVC(b->earliest_evacuator);
         free_EvacuationBlock(b, d->mutex->thread_holding);
@@ -887,22 +857,22 @@
 }
 
 void
-dir_lookaside_remove(CacheKey * key, Part * d)
+dir_lookaside_remove(CacheKey *key, Part *d)
 {
   ink_debug_assert(d->mutex->thread_holding == this_ethread());
   int i = key->word(3) % LOOKASIDE_SIZE;
   EvacuationBlock *b = d->lookaside[i].head;
   while (b) {
     if (b->evac_frags.key == *key) {
-      Debug("dir_lookaside", "remove %X offset %d phase %d",
-            key->word(0), dir_offset(&b->new_dir), dir_phase(&b->new_dir));
+      DDebug("dir_lookaside", "remove %X %X offset %d phase %d",
+            key->word(0), key->word(1), dir_offset(&b->new_dir), dir_phase(&b->new_dir));
       d->lookaside[i].remove(b);
       free_EvacuationBlock(b, d->mutex->thread_holding);
       return;
     }
     b = b->link.next;
   }
-  Debug("dir_lookaside", "remove %X failed", key->word(0));
+  DDebug("dir_lookaside", "remove %X %X failed", key->word(0), key->word(1));
   return;
 }
 
@@ -924,16 +894,16 @@
   io.aiocb.aio_nbytes = n;
   io.aiocb.aio_buf = b;
   io.action = this;
-  io.thread = mutex->thread_holding;
+  io.thread = AIO_CALLBACK_THREAD_ANY;
   ink_assert(ink_aio_write(&io) >= 0);
 }
 
 inku64
-dir_entries_used(Part * d)
+dir_entries_used(Part *d)
 {
   inku64 full = 0;
   inku64 sfull = 0;
-  for (int s = 0; s < DIR_SEGMENTS; full += sfull, s++) {
+  for (int s = 0; s < d->segments; full += sfull, s++) {
     Dir *seg = dir_segment(s, d);
     sfull = 0;
     for (int b = 0; b < d->buckets; b++) {
@@ -1027,13 +997,10 @@
       Debug("cache_dir_sync", "Periodic dir sync in progress -- overwriting");
     }
     d->footer->sync_serial = d->header->sync_serial;
-#ifdef DEBUG
-    check_dir(d);
-#endif
+    CHECK_DIR(d);
     memcpy(buf, d->raw_dir, dirlen);
     int B = d->header->sync_serial & 1;
     ink_off_t start = d->skip + (B ? dirlen : 0);
-
     B = pwrite(d->fd, buf, dirlen, start);
     ink_debug_assert(B == dirlen);
     Debug("cache_dir_sync", "done syncing dir for part %s", d->hash_id);
@@ -1046,7 +1013,7 @@
 
 
 int
-CacheSync::mainEvent(int event, Event * e)
+CacheSync::mainEvent(int event, Event *e)
 {
   NOWARN_UNUSED(e);
   NOWARN_UNUSED(event);
@@ -1065,7 +1032,10 @@
       buflen = 0;
     }
     Debug("cache_dir_sync", "sync done");
-    trigger = eventProcessor.schedule_in(this, HRTIME_SECONDS(cache_config_dir_sync_frequency));
+    if (event == EVENT_INTERVAL)
+      trigger = e->ethread->schedule_in(this, HRTIME_SECONDS(cache_config_dir_sync_frequency));
+    else
+      trigger = eventProcessor.schedule_in(this, HRTIME_SECONDS(cache_config_dir_sync_frequency));
     return EVENT_CONT;
   }
   if (event == AIO_EVENT_DONE) {
@@ -1126,9 +1096,7 @@
       }
       d->header->sync_serial++;
       d->footer->sync_serial = d->header->sync_serial;
-#ifdef DEBUG
-      check_dir(d);
-#endif
+      CHECK_DIR(d);
       memcpy(buf, d->raw_dir, dirlen);
       d->dir_sync_in_progress = 1;
     }
@@ -1174,11 +1142,12 @@
 {
   NOWARN_UNUSED(fix);
   int hist[HIST_DEPTH + 1] = { 0 };
-  int shist[DIR_SEGMENTS] = { 0 };
+  int *shist = (int*)xmalloc(segments * sizeof(int));
+  memset(shist, 0, segments * sizeof(int));
   int j;
   int stale = 0, full = 0, empty = 0;
   int last = 0, free = 0;
-  for (int s = 0; s < DIR_SEGMENTS; s++) {
+  for (int s = 0; s < segments; s++) {
     Dir *seg = dir_segment(s, this);
     for (int b = 0; b < buckets; b++) {
       int h = 0;
@@ -1206,11 +1175,11 @@
     last = t;
     free += dir_freelist_length(this, s);
   }
-  int total = buckets * DIR_SEGMENTS * DIR_DEPTH;
+  int total = buckets * segments * DIR_DEPTH;
   printf("    Directory for [%s]\n", hash_id);
   printf("        Bytes:     %d\n", total * SIZEOF_DIR);
-  printf("        Segments:  %d\n", DIR_SEGMENTS);
-  printf("        Buckets:   %d\n", buckets);
+  printf("        Segments:  %lld\n", (inku64)segments);
+  printf("        Buckets:   %lld\n", (inku64)buckets);
   printf("        Entries:   %d\n", total);
   printf("        Full:      %d\n", full);
   printf("        Empty:     %d\n", empty);
@@ -1218,25 +1187,26 @@
   printf("        Free:      %d\n", free);
   printf("        Bucket Fullness:   ");
   for (j = 0; j < HIST_DEPTH; j++) {
-    printf("%5d ", hist[j]);
-    if ((j % 5 == 4))
+    printf("%8d ", hist[j]);
+    if ((j % 4 == 3))
       printf("\n" "                           ");
   }
   printf("\n");
   printf("        Segment Fullness:  ");
-  for (j = 0; j < DIR_SEGMENTS; j++) {
+  for (j = 0; j < segments; j++) {
     printf("%5d ", shist[j]);
     if ((j % 5 == 4))
       printf("\n" "                           ");
   }
   printf("\n");
   printf("        Freelist Fullness: ");
-  for (j = 0; j < DIR_SEGMENTS; j++) {
+  for (j = 0; j < segments; j++) {
     printf("%5d ", dir_freelist_length(this, j));
     if ((j % 5 == 4))
       printf("\n" "                           ");
   }
   printf("\n");
+  ::xfree(shist);
   return 0;
 }
 
@@ -1327,7 +1297,7 @@
 }
 
 void
-regress_rand_CacheKey(CacheKey * key)
+regress_rand_CacheKey(CacheKey *key)
 {
   unsigned int *x = (unsigned int *) key;
   for (int i = 0; i < 4; i++)
@@ -1335,7 +1305,7 @@
 }
 
 void
-dir_corrupt_bucket(Dir * b, int s, Part * d)
+dir_corrupt_bucket(Dir *b, int s, Part *d)
 {
   // coverity[secure_coding]
   int l = ((int) (dir_bucket_length(b, s, d) * drand48()));
@@ -1348,29 +1318,7 @@
   dir_set_next(e, dir_to_offset(e, seg));
 }
 
-struct CacheDirReg:Continuation
-{
-  int *status;
-
-    CacheDirReg(int *_status):status(_status)
-  {
-    SET_HANDLER(&CacheDirReg::signal_reg);
-    eventProcessor.schedule_in(this, 120 * HRTIME_SECOND);
-
-  }
-
-  int signal_reg(int event, Event * e)
-  {
-    NOWARN_UNUSED(e);
-    NOWARN_UNUSED(event);
-
-    *status = REGRESSION_TEST_PASSED;
-    return EVENT_DONE;
-  }
-
-};
-
-EXCLUSIVE_REGRESSION_TEST(Cache_dir) (RegressionTest * t, int atype, int *status) {
+EXCLUSIVE_REGRESSION_TEST(Cache_dir) (RegressionTest *t, int atype, int *status) {
   NOWARN_UNUSED(atype);
   ink_hrtime ttime;
   int ret = REGRESSION_TEST_PASSED;
@@ -1399,7 +1347,7 @@
   CacheKey key;
   rand_CacheKey(&key, thread->mutex);
 
-  int s = key.word(0) % DIR_SEGMENTS, i, j;
+  int s = key.word(0) % d->segments, i, j;
   Dir *seg = dir_segment(s, d);
 
   // test insert
@@ -1463,15 +1411,16 @@
 
 
   Dir dir1;
-  memset(&dir1, 0, sizeof(Dir));
+  memset(&dir1, 0, sizeof(dir1));
   int s1, b1;
 
+  rprintf(t, "corrupt_bucket test\n");
   for (int ntimes = 0; ntimes < 10; ntimes++) {
 #ifdef LOOP_CHECK_MODE
-    rprintf(t, "dir_probe in bucket with loop\n");
+    // dir_probe in bucket with loop
     rand_CacheKey(&key, thread->mutex);
-    s1 = key.word(0) % DIR_SEGMENTS;
-    b1 = (key.word(0) / DIR_SEGMENTS) % d->buckets;
+    s1 = key.word(0) % d->segments;
+    b1 = key.word(1) % d->buckets;
     dir_corrupt_bucket(dir_bucket(b1, dir_segment(s1, d)), s1, d);
     dir_insert(&key, d, &dir);
     Dir *last_collision = 0;
@@ -1479,17 +1428,17 @@
 
 
     rand_CacheKey(&key, thread->mutex);
-    s1 = key.word(0) % DIR_SEGMENTS;
-    b1 = (key.word(0) / DIR_SEGMENTS) % d->buckets;
+    s1 = key.word(0) % d->segments;
+    b1 = key.word(1) % d->buckets;
     dir_corrupt_bucket(dir_bucket(b1, dir_segment(s1, d)), s1, d);
 
     last_collision = 0;
     dir_probe(&key, d, &dir, &last_collision);
 
-    rprintf(t, "dir_overwrite in bucket with loop\n");
+    // dir_overwrite in bucket with loop
     rand_CacheKey(&key, thread->mutex);
-    s1 = key.word(0) % DIR_SEGMENTS;
-    b1 = (key.word(0) / DIR_SEGMENTS) % d->buckets;
+    s1 = key.word(0) % d->segments;
+    b1 = key.word(1) % d->buckets;
     CacheKey key1;
     key1.b[1] = 127;
     dir1 = dir;
@@ -1502,41 +1451,39 @@
     dir_overwrite(&key, d, &dir, &dir, 1);
 
     rand_CacheKey(&key, thread->mutex);
-    s1 = key.word(0) % DIR_SEGMENTS;
-    b1 = (key.word(0) / DIR_SEGMENTS) % d->buckets;
+    s1 = key.word(0) % d->segments;
+    b1 = key.word(1) % d->buckets;
     key.b[1] = 23;
     dir_insert(&key, d, &dir1);
     dir_corrupt_bucket(dir_bucket(b1, dir_segment(s1, d)), s1, d);
     dir_overwrite(&key, d, &dir, &dir, 0);
 
-
     rand_CacheKey(&key, thread->mutex);
-    s1 = key.word(0) % DIR_SEGMENTS;
+    s1 = key.word(0) % d->segments;
     Dir *seg1 = dir_segment(s1, d);
-    rprintf(t, "dir_freelist_length in freelist with loop: segment %d\n", s1);
+    // dir_freelist_length in freelist with loop
     dir_corrupt_bucket(dir_from_offset(d->header->freelist[s], seg1), s1, d);
     dir_freelist_length(d, s1);
 
-
-
     rand_CacheKey(&key, thread->mutex);
-    s1 = key.word(0) % DIR_SEGMENTS;
-    b1 = (key.word(0) / DIR_SEGMENTS) % d->buckets;
-    rprintf(t, "dir_bucket_length in bucket with loop: segment %d\n", s1);
+    s1 = key.word(0) % d->segments;
+    b1 = key.word(1) % d->buckets;
+    // dir_bucket_length in bucket with loop
     dir_corrupt_bucket(dir_bucket(b1, dir_segment(s1, d)), s1, d);
     dir_bucket_length(dir_bucket(b1, dir_segment(s1, d)), s1, d);
-
     if (!check_dir(d))
       ret = REGRESSION_TEST_FAILED;
 #else
     // test corruption detection
     rand_CacheKey(&key, thread->mutex);
-    s1 = key.word(0) % DIR_SEGMENTS;
-    b1 = (key.word(0) / DIR_SEGMENTS) % d->buckets;
+    s1 = key.word(0) % d->segments;
+    b1 = key.word(1) % d->buckets;
 
     dir_insert(&key, d, &dir1);
     dir_insert(&key, d, &dir1);
-
+    dir_insert(&key, d, &dir1);
+    dir_insert(&key, d, &dir1);
+    dir_insert(&key, d, &dir1);
     dir_corrupt_bucket(dir_bucket(b1, dir_segment(s1, d)), s1, d);
     if (check_dir(d))
       ret = REGRESSION_TEST_FAILED;

Modified: incubator/trafficserver/traffic/trunk/iocore/cache/CacheDisk.cc
URL: http://svn.apache.org/viewvc/incubator/trafficserver/traffic/trunk/iocore/cache/CacheDisk.cc?rev=915922&r1=915921&r2=915922&view=diff
==============================================================================
--- incubator/trafficserver/traffic/trunk/iocore/cache/CacheDisk.cc (original)
+++ incubator/trafficserver/traffic/trunk/iocore/cache/CacheDisk.cc Wed Feb 24 18:48:42 2010
@@ -67,20 +67,17 @@
     return clearDisk();
   }
 
-
   SET_HANDLER(&CacheDisk::openStart);
   io.aiocb.aio_offset = skip;
   io.aiocb.aio_buf = (char *) header;
   io.aiocb.aio_nbytes = header_len;
-
-  io.thread = this_ethread();
+  io.thread = AIO_CALLBACK_THREAD_ANY;
   ink_aio_read(&io);
   return 0;
 }
 
 CacheDisk::~CacheDisk()
 {
-
   if (path) {
     xfree(path);
     for (int i = 0; i < (int) header->num_partitions; i++) {
@@ -104,13 +101,12 @@
 int
 CacheDisk::clearDisk()
 {
-
   delete_all_partitions();
 
   io.aiocb.aio_offset = skip;
   io.aiocb.aio_buf = header;
   io.aiocb.aio_nbytes = header_len;
-  io.thread = this_ethread();
+  io.thread = AIO_CALLBACK_THREAD_ANY;
   ink_aio_write(&io);
   return 0;
 }
@@ -157,7 +153,6 @@
 
   SET_HANDLER(&CacheDisk::openDone);
   return openDone(EVENT_IMMEDIATE, 0);
-
 }
 
 int
@@ -178,11 +173,10 @@
 int
 CacheDisk::sync()
 {
-
   io.aiocb.aio_offset = skip;
   io.aiocb.aio_buf = header;
   io.aiocb.aio_nbytes = header_len;
-  io.thread = this_ethread();
+  io.thread = AIO_CALLBACK_THREAD_ANY;
   ink_aio_write(&io);
   return 0;
 }
@@ -205,9 +199,8 @@
 
 /* size is in store blocks */
 DiskPartBlock *
-CacheDisk::create_partition(int number, int size_in_blocks, int scheme)
+CacheDisk::create_partition(int number, ink_off_t size_in_blocks, int scheme)
 {
-
   if (size_in_blocks == 0)
     return NULL;
 
@@ -216,7 +209,7 @@
 
   if (!q)
     return NULL;
-  int max_blocks = MAX_PART_SIZE >> STORE_BLOCK_SHIFT;
+  ink_off_t max_blocks = MAX_PART_SIZE >> STORE_BLOCK_SHIFT;
   size_in_blocks = (size_in_blocks <= max_blocks) ? size_in_blocks : max_blocks;
 
   int blocks_per_part = PART_BLOCK_SIZE / STORE_BLOCK_SIZE;
@@ -233,7 +226,6 @@
     }
   }
 
-
   if (!p && !closest_match)
     return NULL;
 
@@ -267,7 +259,6 @@
     free_blocks->size += dpb->len;
     free_space += dpb->len;
     header->num_diskpart_blks++;
-
   } else
     header->num_free--;
 
@@ -296,7 +287,6 @@
     disk_parts[i]->size = q->b->len;
     header->num_partitions++;
   }
-
   return p;
 }
 
@@ -304,7 +294,6 @@
 int
 CacheDisk::delete_partition(int number)
 {
-
   unsigned int i;
   for (i = 0; i < header->num_partitions; i++) {
     if (disk_parts[i]->part_number == number) {
@@ -340,7 +329,6 @@
 void
 CacheDisk::update_header()
 {
-
   unsigned int n = 0;
   unsigned int i, j;
   if (free_blocks) {
@@ -397,14 +385,12 @@
     }
   }
 
-
   ink_assert(n == header->num_partitions);
 }
 
 DiskPart *
 CacheDisk::get_diskpart(int part_number)
 {
-
   unsigned int i;
   for (i = 0; i < header->num_partitions; i++) {
     if (disk_parts[i]->part_number == part_number) {
@@ -417,7 +403,6 @@
 int
 CacheDisk::delete_all_partitions()
 {
-
   header->part_info[0].offset = start;
   header->part_info[0].len = num_usable_blocks;
   header->part_info[0].type = CACHE_NONE_TYPE;

Modified: incubator/trafficserver/traffic/trunk/iocore/cache/CacheHttp.cc
URL: http://svn.apache.org/viewvc/incubator/trafficserver/traffic/trunk/iocore/cache/CacheHttp.cc?rev=915922&r1=915921&r2=915922&view=diff
==============================================================================
--- incubator/trafficserver/traffic/trunk/iocore/cache/CacheHttp.cc (original)
+++ incubator/trafficserver/traffic/trunk/iocore/cache/CacheHttp.cc Wed Feb 24 18:48:42 2010
@@ -21,6 +21,7 @@
   limitations under the License.
  */
 
+#include "ink_config.h"
 #include <string.h>
 #include "P_Cache.h"
 
@@ -228,7 +229,7 @@
 
 /*-------------------------------------------------------------------------
   -------------------------------------------------------------------------*/
-int
+inku32
 CacheHTTPInfoVector::get_handles(const char *buf, int length, RefCountObj * block_ptr)
 {
   ink_assert(!(((long) buf) & 3));      // buf must be aligned

Modified: incubator/trafficserver/traffic/trunk/iocore/cache/CacheLink.cc
URL: http://svn.apache.org/viewvc/incubator/trafficserver/traffic/trunk/iocore/cache/CacheLink.cc?rev=915922&r1=915921&r2=915922&view=diff
==============================================================================
--- incubator/trafficserver/traffic/trunk/iocore/cache/CacheLink.cc (original)
+++ incubator/trafficserver/traffic/trunk/iocore/cache/CacheLink.cc Wed Feb 24 18:48:42 2010
@@ -84,35 +84,41 @@
 
   ink_assert(caches[type] == this);
 
-  Action *action = NULL;
-
   Part *part = key_to_part(key, hostname, host_len);
-  Dir result = { 0, 0, 0, 0, 0, 0, 0, 0, 0, 0 };
+  Dir result;
   Dir *last_collision = NULL;
+  CacheVC *c = NULL;
+  {
+    MUTEX_TRY_LOCK(lock, part->mutex, cont->mutex->thread_holding);
+    if (lock) {
+      if (!dir_probe(key, part, &result, &last_collision)) {
+        cont->handleEvent(CACHE_EVENT_DEREF_FAILED, (void *) -ECACHE_NO_DOC);
+        return ACTION_RESULT_DONE;
+      }
+    }
+    c = new_CacheVC(cont);
+    SET_CONTINUATION_HANDLER(c, &CacheVC::derefRead);
+    c->first_key = c->key = *key;
+    c->part = part;
+    c->dir = result;
+    c->last_collision = last_collision;
 
-  MUTEX_TRY_LOCK(lock, part->mutex, cont->mutex->thread_holding);
-  if (lock) {
-    if (!dir_probe(key, part, &result, &last_collision)) {
-      cont->handleEvent(CACHE_EVENT_DEREF_FAILED, (void *) -ECACHE_NO_DOC);
-      return ACTION_RESULT_DONE;
+    if (!lock) {
+      c->mutex->thread_holding->schedule_in_local(c, MUTEX_RETRY_DELAY);
+      return &c->_action;
     }
-  }
-  CacheVC *c = new_CacheVC(cont);
-  SET_CONTINUATION_HANDLER(c, &CacheVC::derefRead);
-  c->first_key = c->key = *key;
-  c->part = part;
-  c->dir = result;
-  c->last_collision = last_collision;
 
-  if (!lock) {
-    c->mutex->thread_holding->schedule_in_local(c, MUTEX_RETRY_DELAY);
-    return &c->_action;
+    switch (c->do_read_call(&c->key)) {
+      case EVENT_DONE: return ACTION_RESULT_DONE;
+      case EVENT_RETURN: goto Lcallreturn;
+      default: return &c->_action;
+    }
   }
-
-  if (c->do_read(&c->key) == EVENT_CONT)
-    return &c->_action;
+Lcallreturn:
+  if (c->handleEvent(AIO_EVENT_DONE, 0) == EVENT_DONE)
+    return ACTION_RESULT_DONE;
   else
-    return action;
+    return &c->_action;
 }
 
 int
@@ -149,10 +155,16 @@
       mutex->thread_holding->schedule_in_local(this, MUTEX_RETRY_DELAY);
       return EVENT_CONT;
     }
-    if (dir_probe(&key, part, &dir, &last_collision))
-      return do_read(&key);
+    if (dir_probe(&key, part, &dir, &last_collision)) {
+      int ret = do_read_call(&first_key);
+      if (ret == EVENT_RETURN)
+        goto Lcallreturn;
+      return ret;
+    }
   }
 Ldone:
   _action.continuation->handleEvent(CACHE_EVENT_DEREF_FAILED, (void *) -ECACHE_NO_DOC);
   return free_CacheVC(this);
+Lcallreturn:
+  return handleEvent(AIO_EVENT_DONE, 0); // hopefully a tail call
 }

Modified: incubator/trafficserver/traffic/trunk/iocore/cache/CachePages.cc
URL: http://svn.apache.org/viewvc/incubator/trafficserver/traffic/trunk/iocore/cache/CachePages.cc?rev=915922&r1=915921&r2=915922&view=diff
==============================================================================
--- incubator/trafficserver/traffic/trunk/iocore/cache/CachePages.cc (original)
+++ incubator/trafficserver/traffic/trunk/iocore/cache/CachePages.cc Wed Feb 24 18:48:42 2010
@@ -21,10 +21,11 @@
   limitations under the License.
  */
 
-#ifdef NON_MODULAR
 #include "P_Cache.h"
 #include "Show.h"
 
+#ifdef NON_MODULAR
+
 struct ShowCache:ShowCont
 {
   int part_index;
@@ -305,6 +306,8 @@
     CHECK_SHOW(show("<tr><td>sync_serial</td><td>%lu</tr>\n", d->sync_serial));
     CHECK_SHOW(show("<tr><td>write_serial</td><td>%lu</tr>\n", d->write_serial));
     CHECK_SHOW(show("<tr><td>header length</td><td>%lu</tr>\n", d->hlen));
+    CHECK_SHOW(show("<tr><td>fragment type</td><td>%lu</tr>\n", d->ftype));
+    CHECK_SHOW(show("<tr><td>fragment table length</td><td>%lu</tr>\n", d->flen));
     CHECK_SHOW(show("<tr><td>No of Alternates</td><td>%d</td></tr>\n", alt_count));
 
     CHECK_SHOW(show("<tr><td>Action</td>\n"

Modified: incubator/trafficserver/traffic/trunk/iocore/cache/CachePagesInternal.cc
URL: http://svn.apache.org/viewvc/incubator/trafficserver/traffic/trunk/iocore/cache/CachePagesInternal.cc?rev=915922&r1=915921&r2=915922&view=diff
==============================================================================
--- incubator/trafficserver/traffic/trunk/iocore/cache/CachePagesInternal.cc (original)
+++ incubator/trafficserver/traffic/trunk/iocore/cache/CachePagesInternal.cc Wed Feb 24 18:48:42 2010
@@ -21,10 +21,11 @@
   limitations under the License.
  */
 
-#ifdef NON_MODULAR
 #include "P_Cache.h"
 #include "Show.h"
 
+#ifdef NON_MODULAR
+
 struct ShowCacheInternal:ShowCont
 {
   int part_index;
@@ -153,23 +154,15 @@
     sprintf(nbytes, "%d", vc->vio.nbytes);
     sprintf(todo, "%d", vc->vio.ntodo());
 
-#if 1
-    if (vc->f.http_request && vc->request.valid()) {
+    if (vc->f.frag_type == CACHE_FRAG_TYPE_HTTP && vc->request.valid()) {
       URL *u = vc->request.url_get(&uu);
       u->print(url, 8000, &ib, &xd);
       url[ib] = 0;
-    }
-    //else if (vc->vector.get(vc->alternate_index)->valid()) {
-    //  URL* u = vc->vector.get(vc->alternate_index)->request_url_get(&uu);
-    //  u->print(url, 8000, &ib, &xd);
-    //  url[ib] = 0;
-    //} 
-    else if (vc->alternate.valid()) {
+    } else if (vc->alternate.valid()) {
       URL *u = vc->alternate.request_url_get(&uu);
       u->print(url, 8000, &ib, &xd);
       url[ib] = 0;
     } else
-#endif
       vc->key.string(url);
     CHECK_SHOW(show("<tr>" "<td>%s</td>"        // operation
                     "<td>%s</td>"       // Part
@@ -182,7 +175,7 @@
                     vc->part->hash_id,
                     url,
                     vc->vio.ndone,
-                    vc->vio.nbytes == INT_MAX ? "all" : nbytes, vc->vio.nbytes == INT_MAX ? "all" : todo));
+                    vc->vio.nbytes == INK64_MAX ? "all" : nbytes, vc->vio.nbytes == INK64_MAX ? "all" : todo));
   }
   part_index++;
   if (part_index < gnpart)
@@ -228,7 +221,7 @@
                       "<td>%d</td>"     // estimated size
                       "<td>%d</td>"     // reader count
                       "<td>%s</td>"     // done
-                      "</tr>\n", offset, (int) dir_approx_size(&b->dir), b->f.readers, b->f.done ? "yes" : "no"));
+                      "</tr>\n", offset, (int) dir_approx_size(&b->dir), b->readers, b->f.done ? "yes" : "no"));
     }
   }
   part_index++;
@@ -279,7 +272,7 @@
     agg_todo++;
   CHECK_SHOW(show("<tr>" "<td>%s</td>"  // ID
                   "<td>%d</td>" // blocks
-                  "<td>%d</td>" // directory entries
+                  "<td>%lld</td>" // directory entries
                   "<td>%d</td>" // write position
                   "<td>%d</td>" // write agg to do
                   "<td>%d</td>" // write agg to do size
@@ -291,7 +284,7 @@
                   "</tr>\n",
                   p->hash_id,
                   (int) ((p->len - (p->start - p->skip)) / INK_BLOCK_SIZE),
-                  p->buckets * DIR_DEPTH * DIR_SEGMENTS,
+                  (inku64)(p->buckets * DIR_DEPTH * p->segments),
                   (int) ((p->header->write_pos - p->start) / INK_BLOCK_SIZE),
                   agg_todo,
                   p->agg_todo_size,
@@ -330,7 +323,7 @@
                   "<td>%d</td>"
                   "<td>%d</td>" "<td>%d</td>" "<td>%d</td>" "</tr>\n", free, used, empty, valid, agg_valid, avg_size));
   seg_index++;
-  if (seg_index < DIR_SEGMENTS)
+  if (seg_index < p->segments)
     CONT_SCHED_LOCK_RETRY(this);
   else {
     CHECK_SHOW(show("</table>\n"));

Modified: incubator/trafficserver/traffic/trunk/iocore/cache/CachePart.cc
URL: http://svn.apache.org/viewvc/incubator/trafficserver/traffic/trunk/iocore/cache/CachePart.cc?rev=915922&r1=915921&r2=915922&view=diff
==============================================================================
--- incubator/trafficserver/traffic/trunk/iocore/cache/CachePart.cc (original)
+++ incubator/trafficserver/traffic/trunk/iocore/cache/CachePart.cc Wed Feb 24 18:48:42 2010
@@ -81,7 +81,7 @@
     goto Ldone;
   }
 Lcont:
-  segment = 0;
+  fragment = 0;
   SET_HANDLER(&CacheVC::scanObject);
   eventProcessor.schedule_in(this, HRTIME_MSECONDS(scan_msec_delay));
   return EVENT_CONT;
@@ -113,19 +113,19 @@
   if (_action.cancelled)
     return free_CacheVC(this);
 
-  MUTEX_TRY_LOCK(lock, part->mutex, mutex->thread_holding);
+  CACHE_TRY_LOCK(lock, part->mutex, mutex->thread_holding);
   if (!lock) {
     mutex->thread_holding->schedule_in_local(this, MUTEX_RETRY_DELAY);
     return EVENT_CONT;
   }
 
-  if (!segment) {               // initialize for first read
-    segment = 1;
+  if (!fragment) {               // initialize for first read
+    fragment = 1;
     io.aiocb.aio_offset = part_offset_to_offset(part, 0);
     io.aiocb.aio_nbytes = SCAN_BUF_SIZE;
     io.aiocb.aio_buf = buf->data();
     io.action = this;
-    io.thread = mutex->thread_holding;
+    io.thread = AIO_CALLBACK_THREAD_ANY;
     goto Lread;
   }
 
@@ -140,22 +140,22 @@
     int i;
     bool changed;
 
-    if (doc->magic != DOC_MAGIC || !doc->hlen)
+    if (doc->magic != DOC_MAGIC || doc->ftype != CACHE_FRAG_TYPE_HTTP || !doc->hlen)
       goto Lskip;
 
     last_collision = NULL;
     while (1) {
       if (!dir_probe(&doc->first_key, part, &dir, &last_collision))
         goto Lskip;
-      if (!dir_agg_valid(part, &dir) || !dir.head ||
+      if (!dir_agg_valid(part, &dir) || !dir_head(&dir) ||
           (part_offset(part, &dir) != io.aiocb.aio_offset + ((char *) doc - buf->data())))
         continue;
       break;
     }
-    if ((char *) doc - buf->data() + sizeofDoc + doc->hlen > (int) io.aiocb.aio_nbytes)
+    if (doc->data() - buf->data() > (int) io.aiocb.aio_nbytes)
       goto Lskip;
     {
-      char *tmp = doc->hdr;
+      char *tmp = doc->hdr();
       int len = doc->hlen;
       while (len > 0) {
         int r = HTTPInfo::unmarshal(tmp, len, buf._ptr());
@@ -167,7 +167,7 @@
         tmp += r;
       }
     }
-    if (vector.get_handles(doc->hdr, doc->hlen) != doc->hlen)
+    if (vector.get_handles(doc->hdr(), doc->hlen) != doc->hlen)
       goto Lskip;
     changed = false;
     hostinfo_copied = 0;
@@ -222,13 +222,12 @@
         ink_debug_assert(hostinfo_copied);
         SET_HANDLER(&CacheVC::scanRemoveDone);
         // force remove even if there is a writer
-        cacheProcessor.remove(this, &doc->first_key, true, false, CACHE_FRAG_TYPE_HTTP, (char *) hname, hlen);
+        cacheProcessor.remove(this, &doc->first_key, CACHE_FRAG_TYPE_HTTP, true, false, (char *) hname, hlen);
         return EVENT_CONT;
-//      dir_delete(&doc->first_key, part, &dir);
       } else {
         offset = (char *) doc - buf->data();
         write_len = 0;
-        f.http_request = 1;
+        frag_type = CACHE_FRAG_TYPE_HTTP;
         f.use_first_key = 1;
         f.evac_vector = 1;
         first_key = key = doc->first_key;
@@ -306,71 +305,69 @@
       return scanObject(EVENT_IMMEDIATE, 0);
     }
   }
-  MUTEX_TRY_LOCK(lock, part->mutex, mutex->thread_holding);
-  if (!lock)
-    VC_SCHED_LOCK_RETRY();
-
-  Debug("cache_scan", "trying for writer lock");
-  if (part->open_write(this, false, 1)) {
-    writer_lock_retry++;
-    SET_HANDLER(&CacheVC::scanOpenWrite);
-    mutex->thread_holding->schedule_in_local(this, scan_msec_delay);
-    return EVENT_CONT;
-  }
-
-  ink_debug_assert(this->od);
-  // put all the alternates in the open directory vector
-  int alt_count = vector.count();
-  for (int i = 0; i < alt_count; i++) {
-    write_vector->insert(vector.get(i));
-  }
-  od->writing_vec = 1;
-  vector.clear(false);
-  // check that the directory entry was not overwritten
-  // if so return failure
-  Debug("cache_scan", "got writer lock");
-  Dir *l = NULL;
-  Dir d;
-  Doc *doc = (Doc *) (buf->data() + offset);
-  offset = (char *) doc - buf->data() + round_to_approx_size(doc->len);
-  // if the doc contains some data, then we need to create
-  // a new directory entry for this fragment. Remember the
-  // offset and the key in earliest_key
-  dir_assign(&od->first_dir, &dir);
-  if (doc->total_len) {
-    dir_assign(&od->single_doc_dir, &dir);
-    dir_set_tag(&od->single_doc_dir, DIR_MASK_TAG(doc->key.word(1)));
-    od->single_doc_key = doc->key;
-    od->move_resident_alt = 1;
-  }
+  int ret = 0;
+  {
+    CACHE_TRY_LOCK(lock, part->mutex, mutex->thread_holding);
+    if (!lock)
+      VC_SCHED_LOCK_RETRY();
+
+    Debug("cache_scan", "trying for writer lock");
+    if (part->open_write(this, false, 1)) {
+      writer_lock_retry++;
+      SET_HANDLER(&CacheVC::scanOpenWrite);
+      mutex->thread_holding->schedule_in_local(this, scan_msec_delay);
+      return EVENT_CONT;
+    }
 
-  while (1) {
-    if (!dir_probe(&first_key, part, &d, &l)) {
-      part->close_write(this);
-      _action.continuation->handleEvent(CACHE_EVENT_SCAN_OPERATION_FAILED, 0);
-      SET_HANDLER(&CacheVC::scanObject);
-      return handleEvent(EVENT_IMMEDIATE, 0);
+    ink_debug_assert(this->od);
+    // put all the alternates in the open directory vector
+    int alt_count = vector.count();
+    for (int i = 0; i < alt_count; i++) {
+      write_vector->insert(vector.get(i));
     }
-/*     
-    if (!dir_agg_valid(part, &d) || !d.head || 
-	(part_offset(part, &d) !=
-	 io.aiocb.aio_offset + ((char*)doc - buf->data()))) {
-      Debug("cache_scan","dir entry is not valid");
-      continue;
+    od->writing_vec = 1;
+    vector.clear(false);
+    // check that the directory entry was not overwritten
+    // if so return failure
+    Debug("cache_scan", "got writer lock");
+    Dir *l = NULL;
+    Dir d;
+    Doc *doc = (Doc *) (buf->data() + offset);
+    offset = (char *) doc - buf->data() + round_to_approx_size(doc->len);
+    // if the doc contains some data, then we need to create
+    // a new directory entry for this fragment. Remember the
+    // offset and the key in earliest_key
+    dir_assign(&od->first_dir, &dir);
+    if (doc->total_len) {
+      dir_assign(&od->single_doc_dir, &dir);
+      dir_set_tag(&od->single_doc_dir, doc->key.word(2));
+      od->single_doc_key = doc->key;
+      od->move_resident_alt = 1;
     }
-*/
-    if (*((inku64 *) & dir) != *((inku64 *) & d)) {
-      Debug("cache_scan", "dir entry has changed");
-      continue;
+
+    while (1) {
+      if (!dir_probe(&first_key, part, &d, &l)) {
+        part->close_write(this);
+        _action.continuation->handleEvent(CACHE_EVENT_SCAN_OPERATION_FAILED, 0);
+        SET_HANDLER(&CacheVC::scanObject);
+        return handleEvent(EVENT_IMMEDIATE, 0);
+      }
+      if (memcmp(&dir, &d, SIZEOF_DIR)) {
+        Debug("cache_scan", "dir entry has changed");
+        continue;
+      }
+      break;
     }
-    break;
-  }
 
-  // the document was not modified 
-  // we are safe from now on as we hold the 
-  // writer lock on the doc
-  SET_HANDLER(&CacheVC::scanUpdateDone);
-  return do_write();
+    // the document was not modified 
+    // we are safe from now on as we hold the 
+    // writer lock on the doc
+    SET_HANDLER(&CacheVC::scanUpdateDone);
+    ret = do_write_call();
+  }
+  if (ret == EVENT_RETURN)
+    return handleEvent(AIO_EVENT_DONE, 0);
+  return ret;
 }
 
 int
@@ -381,7 +378,7 @@
   Debug("cache_scan_truss", "inside %p:scanUpdateDone", this);
   cancel_trigger();
   // get partition lock
-  MUTEX_TRY_LOCK(lock, part->mutex, mutex->thread_holding);
+  CACHE_TRY_LOCK(lock, part->mutex, mutex->thread_holding);
   if (lock) {
     // insert a directory entry for the previous fragment 
     dir_overwrite(&first_key, part, &dir, &od->first_dir, false);