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 2009/12/18 17:48:16 UTC

svn commit: r892310 [3/4] - in /incubator/trafficserver/traffic/branches/dev: ./ iocore/aio/ iocore/cache/ iocore/cluster/ iocore/eventsystem/ iocore/net/ libinktomi++/ proxy/ proxy/http2/

Modified: incubator/trafficserver/traffic/branches/dev/iocore/cache/P_CacheDir.h
URL: http://svn.apache.org/viewvc/incubator/trafficserver/traffic/branches/dev/iocore/cache/P_CacheDir.h?rev=892310&r1=892309&r2=892310&view=diff
==============================================================================
--- incubator/trafficserver/traffic/branches/dev/iocore/cache/P_CacheDir.h (original)
+++ incubator/trafficserver/traffic/branches/dev/iocore/cache/P_CacheDir.h Fri Dec 18 16:47:37 2009
@@ -30,107 +30,81 @@
 struct Part;
 struct CacheVC;
 
+/*
+  Directory layout
+*/
+
 // Constants
 
 #define DIR_TAG_WIDTH			12
-#define DIR_SIZE_WIDTH			6
 #define DIR_MASK_TAG(_t)                ((_t) & ((1 << DIR_TAG_WIDTH) - 1))
-#define DIR_MASK_TOKEN_TAG(_t)          ((_t) & ((1 << (DIR_TAG_WIDTH+DIR_SIZE_WIDTH)) - 1))
-#define SIZEOF_DIR           		8
+#define SIZEOF_DIR           		10
 #define ESTIMATED_OBJECT_SIZE           8000
 
-
-/* This number was changed from 16 to 32 so that we could support
-   partititon size upt 8G ( 8 * 1024 * 1024 * 1024) with average
-   object sizes as low as 4 KB.
-*/
-#define DIR_SEGMENTS                    32
- // between 2 (faster/more waste) and 5 (slower/less waste)
+#define MAX_DIR_SEGMENTS                (32 * (1<<16))
 #define DIR_DEPTH                       4
+#define DIR_SIZE_WIDTH                  6
+#define DIR_BLOCK_SIZES                 4
+#define DIR_BLOCK_SHIFT(_i)             (3*(_i))
+#define DIR_BLOCK_SIZE(_i)              (INK_BLOCK_SIZE << DIR_BLOCK_SHIFT(_i))
+#define DIR_SIZE_WITH_BLOCK(_i)         ((1<<DIR_SIZE_WIDTH) * DIR_BLOCK_SIZE(_i))
+#define DIR_OFFSET_BITS                 40
+#define DIR_OFFSET_MAX                  ((((ink_off_t)1) << DIR_OFFSET_BITS) - 1)
+#define MAX_DOC_SIZE                    ((1<<DIR_SIZE_WIDTH)*(1<<B8K_SHIFT)) // 1MB
+
+#define SYNC_MAX_WRITE                  (2 * 1024 * 1024)
+#define SYNC_DELAY                      HRTIME_MSECONDS(500)
+
+// Debugging Options
 
-#define DIR_SIZE_BIG                    32768
-#define DIR_NULL                        0
-#define DIR_OFFSET_BITS                 24
-#define DIR_OFFSET_MAX                  ((1 << DIR_OFFSET_BITS) - 1)
-#define DIR_OFFSET_MASK(_o)             ((_o) & DIR_OFFSET_MAX)
+//#define DO_CHECK_DIR_FAST
+//#define DO_CHECK_DIR
 
+// Macros
 
-// Dir accessors: use these
-
-#define dir_offset(_e) ((ink_off_t)(_e)->offset)
-#define dir_set_offset(_e,_o) (_e)->offset = _o
+#ifdef DO_CHECK_DIR
+#define CHECK_DIR(_d) ink_debug_assert(check_dir(_d))
+#else
+#define CHECK_DIR(_d) ((void)0)
+#endif
 
 #define dir_index(_e, _i) ((Dir*)((char*)(_e)->dir + (SIZEOF_DIR * (_i))))
-#define round_to_approx_size(_s) \
-(_s <= DIR_SIZE_BIG ? ROUND_TO_BLOCK(_s) : ROUND_TO_8K(_s))
-#define dir_set_approx_size(_e, _s) do {                          \
-  if (_s > DIR_SIZE_BIG) {                                        \
-      (_e)->big_size = 1;                                         \
-      (_e)->size = (((_s)-1) >> B8K_SHIFT);                       \
-   } else {                                                       \
-      (_e)->big_size = 0;                                         \
-      (_e)->size = (((_s)-1) >> INK_BLOCK_SHIFT);                 \
-    }                                                             \
-} while (0)
-#define dir_approx_size(_e)		         \
-  ((_e)->big_size ?                              \
-    (((_e)->size + 1) << B8K_SHIFT) :            \
-    (((_e)->size + 1) << INK_BLOCK_SHIFT))
-#define dir_phase(_e) (_e)->phase
-#define dir_set_phase(_e,_p) (_e)->phase = _p
-#define dir_tag(_e) (_e)->tag
-#define dir_token(_e) (_e)->token
-#define dir_set_token(_e, _v) (_e)->token = _v
-#define dir_set_tag(_e,_t) (_e)->tag = _t
 #define dir_assign(_e,_x) do {                \
-  ((inku32*)(_e))[0] = ((inku32*)(_x))[0];    \
-  ((inku32*)(_e))[1] = ((inku32*)(_x))[1];    \
+    (_e)->w[0] = (_x)->w[0];                  \
+    (_e)->w[1] = (_x)->w[1];                  \
+    (_e)->w[2] = (_x)->w[2];                  \
+    (_e)->w[3] = (_x)->w[3];                  \
+    (_e)->w[4] = (_x)->w[4];                  \
 } while (0)
 #define dir_assign_data(_e,_x) do {           \
-  unsigned short next = (_e)->next;           \
+  unsigned short next = dir_next(_e);         \
   dir_assign(_e, _x);		              \
-  (_e)->next = next;                          \
+  dir_set_next(_e, next);                     \
 } while(0)
 // entry is valid
-#define dir_valid(_d, _e)                                                \
-  (_d->header->phase == dir_phase(_e) ? part_in_phase_valid(_d, _e) :    \
+#define dir_valid(_d, _e)                                               \
+  (_d->header->phase == dir_phase(_e) ? part_in_phase_valid(_d, _e) :   \
                                         part_out_of_phase_valid(_d, _e))
 // entry is valid and outside of write aggregation region
 #define dir_agg_valid(_d, _e)                                            \
   (_d->header->phase == dir_phase(_e) ? part_in_phase_valid(_d, _e) :    \
                                         part_out_of_phase_agg_valid(_d, _e))
 // entry may be valid or overwritten in the last aggregated write
-#define dir_write_valid(_d, _e)                                            \
-  (_d->header->phase == dir_phase(_e) ? part_in_phase_valid(_d, _e) :    \
+#define dir_write_valid(_d, _e)                                         \
+  (_d->header->phase == dir_phase(_e) ? part_in_phase_valid(_d, _e) :   \
                                         part_out_of_phase_write_valid(_d, _e))
 #define dir_agg_buf_valid(_d, _e)                                          \
   (_d->header->phase == dir_phase(_e) && part_in_phase_agg_buf_valid(_d, _e))
-
-
-#define dir_set_dirinfo(_e, _i) *(DirInfo*)(_e) = *(DirInfo*)_i
-#define dir_dirinfo(_e) (*(DirInfo*)(_e))
-#define dir_next(_e) (_e)->next
-#define dir_set_next(_e, _o) (_e)->next = _o
-#define dir_prev(_e) ((FreeDir*)(_e))->prev
-#define dir_set_prev(_e,_o) ((FreeDir*)(_e))->prev = _o
-#define dir_head(_e) (_e)->head
-#define dir_set_head(_e, _v) (_e)->head = _v
-#define dir_pinned(_e) (_e)->pinned
-#define dir_set_pinned(_e, _v) (_e)->pinned = _v
-#define dir_is_empty(_e) (!(_e)->offset)
-#define dir_clear(_e)  *((inku64*)(_e)) = 0
-#define dir_clean(_e)  *((inku32*)(_e)) = 0
-
-#if 1
-#define dir_segment(_s, _d) (_d)->segment[_s]
-#else
-#define dir_segment(_s, _d) part_dir_segment(_d, _s)    // slower?
-#endif
-
-// DirInfo
-
-#define dirinfo_clear(_x) (_x) = 0
-#define dirinfo_is_empty(_x) (!(_x))
+#define dir_is_empty(_e) (!dir_offset(_e))
+#define dir_clear(_e) do { \
+    (_e)->w[0] = 0; \
+    (_e)->w[1] = 0; \
+    (_e)->w[2] = 0; \
+    (_e)->w[3] = 0; \
+    (_e)->w[4] = 0; \
+} while (0)
+#define dir_clean(_e) dir_set_offset(_e,0)
+#define dir_segment(_s, _d) part_dir_segment(_d, _s)
 
 // OpenDir
 
@@ -143,31 +117,98 @@
 // Cache Directory
 
 // INTERNAL: do not access these members directly, use the
-// accessors below (e.g. dir_offset, dir_set_offset)
+// accessors below (e.g. dir_offset, dir_set_offset).
+// These structures are stored in memory 2 byte aligned.
+// The accessors prevent unaligned memory access which
+// is often either less efficient or unsupported depending
+// on the processor.
 struct Dir
 {
-  unsigned int offset:24;       // 16M * 512 = 8GB
-  unsigned int big_size:1;
-  unsigned int size:6;          // 64k/512 = 128 log2 = 7
-  unsigned int phase:1;
-  unsigned int tag:12;          // 2048 / 8 entries/bucket = .4%
-  unsigned int head:1;          // first segment in a document
-  unsigned int pinned:1;
-  unsigned int token:1;
-  unsigned int reserved:1;
-  unsigned int next:16;
+#if 0
+  // bits are numbered from lowest in u16 to highest
+  // always index as u16 to avoid byte order issues
+  unsigned int offset:24;       // (0,1:0-7) 16M * 512 = 8GB
+  unsigned int big:2;           // (1:8-9) 512 << (3 * big)
+  unsigned int size:6;          // (1:10-15) 6**2 = 64, 64*512 = 32768 .. 64*256=16MB
+  unsigned int tag:12;          // (2:0-11) 2048 / 8 entries/bucket = .4%
+  unsigned int phase:1;         // (2:12)
+  unsigned int head:1;          // (2:13) first segment in a document
+  unsigned int pinned:1;        // (2:14)
+  unsigned int token:1;         // (2:15)
+  unsigned int next:16;         // (3)
+  inku16 offset_high;           // 8GB * 65k = 0.5PB (4)
+#else
+  inku16 w[5];
+  Dir() { dir_clear(this); }
+#endif
 };
 
 // INTERNAL: do not access these members directly, use the
 // accessors below (e.g. dir_offset, dir_set_offset)
 struct FreeDir
 {
+#if 0
   unsigned int offset:24;       // 0: empty
   unsigned int reserved:8;
-  unsigned int prev:16;
-  unsigned int next:16;
+  unsigned int prev:16;         // (2)
+  unsigned int next:16;         // (3)
+  inku16 offset_high;           // 0: empty
+#else
+  inku16 w[5];
+  FreeDir() { dir_clear(this); }
+#endif
 };
 
+#define dir_bit(_e, _w, _b) (((_e)->w[_w] >> (_b)) & 1)
+#define dir_set_bit(_e, _w, _b, _v) (_e)->w[_w] = (inku16)(((_e)->w[_w] & ~(1<<(_b))) | (((_v)?1:0)<<(_b)))
+#define dir_offset(_e) ((ink_off_t)                                     \
+                         (((inku64)(_e)->w[0]) |                        \
+                          (((inku64)((_e)->w[1] & 0xFF)) << 16) |       \
+                          (((inku64)(_e)->w[4]) << 24)))
+#define dir_set_offset(_e,_o) do { \
+    (_e)->w[0] = (inku16)_o;                                                    \
+    (_e)->w[1] = (inku16)((((_o) >> 16) & 0xFF) | ((_e)->w[1] & 0xFF00));       \
+    (_e)->w[4] = (inku16)((_o) >> 24);                                          \
+} while (0)
+#define dir_big(_e) ((inku32)((((_e)->w[1]) >> 8)&0x3))
+#define dir_set_big(_e, _v) (_e)->w[1] = (inku16)(((_e)->w[1] & 0xFCFF) | (((inku16)(_v))&0x3)<<8)
+#define dir_size(_e) ((inku32)(((_e)->w[1]) >> 10))
+#define dir_set_size(_e, _v) (_e)->w[1] = (inku16)(((_e)->w[1] & ((1<<10)-1)) | ((_v)<<10))
+#define dir_set_approx_size(_e, _s) do {                         \
+  if ((_s) <= DIR_SIZE_WITH_BLOCK(0)) {                          \
+    dir_set_big(_e,0);                                           \
+    dir_set_size(_e,((_s)-1) / DIR_BLOCK_SIZE(0));               \
+  } else if ((_s) <= DIR_SIZE_WITH_BLOCK(1)) {                   \
+    dir_set_big(_e,1);                                           \
+    dir_set_size(_e,((_s)-1) / DIR_BLOCK_SIZE(1));               \
+  } else if ((_s) <= DIR_SIZE_WITH_BLOCK(2)) {                   \
+    dir_set_big(_e,2);                                           \
+    dir_set_size(_e,((_s)-1) / DIR_BLOCK_SIZE(2));               \
+  } else {                                                       \
+    dir_set_big(_e,3);                                           \
+    dir_set_size(_e,((_s)-1) / DIR_BLOCK_SIZE(3));               \
+  }                                                              \
+} while (0)
+#define dir_approx_size(_e) ((dir_size(_e) + 1) * DIR_BLOCK_SIZE(dir_big(_e)))
+#define round_to_approx_size(_s) (_s <= DIR_SIZE_WITH_BLOCK(0) ? ROUND_TO(_s, DIR_BLOCK_SIZE(0)) : \
+                                  (_s <= DIR_SIZE_WITH_BLOCK(1) ? ROUND_TO(_s, DIR_BLOCK_SIZE(1)) : \
+                                   (_s <= DIR_SIZE_WITH_BLOCK(2) ? ROUND_TO(_s, DIR_BLOCK_SIZE(2)) : \
+                                    ROUND_TO(_s, DIR_BLOCK_SIZE(3)))))
+#define dir_tag(_e) ((_e)->w[2]&((1<<DIR_TAG_WIDTH)-1))
+#define dir_set_tag(_e,_t) (_e)->w[2] = (inku16)(((_e)->w[2]&~((1<<DIR_TAG_WIDTH)-1)) | ((_t)&((1<<DIR_TAG_WIDTH)-1)))
+#define dir_phase(_e) dir_bit(_e,2,12)
+#define dir_set_phase(_e,_v) dir_set_bit(_e,2,12,_v)
+#define dir_head(_e) dir_bit(_e,2,13)
+#define dir_set_head(_e, _v) dir_set_bit(_e,2,13,_v)
+#define dir_pinned(_e) dir_bit(_e,2,14)
+#define dir_set_pinned(_e, _v) dir_set_bit(_e,2,14,_v)
+#define dir_token(_e) dir_bit(_e,2,15)
+#define dir_set_token(_e, _v) dir_set_bit(_e,2,15,_v)
+#define dir_next(_e) (_e)->w[3]
+#define dir_set_next(_e, _o) (_e)->w[3] = (inku16)(_o)
+#define dir_prev(_e) (_e)->w[2]
+#define dir_set_prev(_e,_o) (_e)->w[2] = (inku16)(_o)
+
 // INKqa11166 - Cache can not store 2 HTTP alternates simultaneously.
 // To allow this, move the vector from the CacheVC to the OpenDirEntry.
 // Each CacheVC now maintains a pointer to this vector. Adding/Deleting
@@ -176,15 +217,15 @@
 // (CacheVC::updateVector). 
 struct OpenDirEntry
 {
-  DLL<CacheVC> writers;      // list of all the current writers
-  DLL<CacheVC> readers;      // list of all the current readers - not used
+  DLL<CacheVC> writers;         // list of all the current writers
+  DLL<CacheVC> readers;         // list of all the current readers - not used
   CacheHTTPInfoVector vector;   // Vector for the http document. Each writer 
-  // maintains a pointer to this vector and 
-  // writes it down to disk. 
+                                // maintains a pointer to this vector and 
+                                // writes it down to disk. 
   CacheKey single_doc_key;      // Key for the resident alternate. 
   Dir single_doc_dir;           // Directory for the resident alternate
   Dir first_dir;                // Dir for the vector. If empty, a new dir is 
-  // inserted, otherwise this dir is overwritten
+                                // inserted, otherwise this dir is overwritten
   inku16 num_writers;           // num of current writers
   inku16 max_writers;           // max number of simultaneous writers allowed
   bool dont_update_directory;   // if set, the first_dir is not updated.
@@ -192,7 +233,7 @@
   volatile bool reading_vec;    // somebody is currently reading the vector
   volatile bool writing_vec;    // somebody is currently writing the vector
 
-    Link<OpenDirEntry> link;
+  Link<OpenDirEntry> link;
 
   int wait(CacheVC * c, int msec);
 
@@ -212,7 +253,7 @@
   OpenDirEntry *open_read(INK_MD5 * key);
   int signal_readers(int event, Event * e);
 
-    OpenDir();
+  OpenDir();
 };
 
 struct CacheSync:Continuation
@@ -226,7 +267,7 @@
   int mainEvent(int event, Event * e);
   void aio_write(int fd, char *b, int n, ink_off_t o);
 
-    CacheSync():Continuation(new_ProxyMutex()), part(0), buf(0), buflen(0), writepos(0), trigger(0)
+  CacheSync():Continuation(new_ProxyMutex()), part(0), buf(0), buflen(0), writepos(0), trigger(0)
   {
     SET_HANDLER(&CacheSync::mainEvent);
   }
@@ -247,24 +288,27 @@
 void dir_lookaside_remove(CacheKey * key, Part * d);
 void dir_free_entry(Dir * e, int s, Part * d);
 void dir_sync_init();
-void check_dir(Part * d);
+int check_dir(Part * d);
 void dir_clean_part(Part * d);
-void dir_clear_range(int start, int end, Part * d);
+void dir_clear_range(ink_off_t start, ink_off_t end, Part * d);
 int dir_segment_accounted(int s, Part * d, int offby = 0,
                           int *free = 0, int *used = 0,
                           int *empty = 0, int *valid = 0, int *agg_valid = 0, int *avg_size = 0);
-void dir_compute_stats();
-
 inku64 dir_entries_used(Part * d);
+void sync_cache_dir_on_shutdown();
+
+// Global Data
 
 extern Dir empty_dir;
 
-// inline Funtions
+// Inline Funtions
+
+#define dir_in_seg(_s, _i) ((Dir*)(((char*)(_s)) + (SIZEOF_DIR * (_i))))
 
 inline bool
 dir_compare_tag(Dir * e, CacheKey * key)
 {
-  return (dir_tag(e) == DIR_MASK_TAG(key->word(1)));
+  return (dir_tag(e) == DIR_MASK_TAG(key->word(2)));
 }
 
 inline Dir *
@@ -273,10 +317,10 @@
 #if DIR_DEPTH < 5
   if (!i)
     return 0;
-  return &seg[i];
+  return dir_in_seg(seg, i);
 #else
   i = i + ((i - 1) / (DIR_DEPTH - 1));
-  return &seg[i];
+  return dir_in_seg(seg, i);
 #endif
 }
 inline Dir *
@@ -289,9 +333,9 @@
 dir_to_offset(Dir * d, Dir * seg)
 {
 #if DIR_DEPTH < 5
-  return d - seg;
+  return (((char*)d) - ((char*)seg))/SIZEOF_DIR;
 #else
-  int i = d - seg;
+  int i = (int)((((char*)d) - ((char*)seg))/SIZEOF_DIR);
   i = i - (i / DIR_DEPTH);
   return i;
 #endif
@@ -299,14 +343,12 @@
 inline Dir *
 dir_bucket(int b, Dir * seg)
 {
-  return &seg[b * DIR_DEPTH];
+  return dir_in_seg(seg, b * DIR_DEPTH);
 }
 inline Dir *
 dir_bucket_row(Dir * b, int i)
 {
-  return &b[i];
+  return dir_in_seg(b, i);
 }
 
-extern void sync_cache_dir_on_shutdown(void);
-
 #endif /* _P_CACHE_DIR_H__ */

Modified: incubator/trafficserver/traffic/branches/dev/iocore/cache/P_CacheDisk.h
URL: http://svn.apache.org/viewvc/incubator/trafficserver/traffic/branches/dev/iocore/cache/P_CacheDisk.h?rev=892310&r1=892309&r2=892310&view=diff
==============================================================================
--- incubator/trafficserver/traffic/branches/dev/iocore/cache/P_CacheDisk.h (original)
+++ incubator/trafficserver/traffic/branches/dev/iocore/cache/P_CacheDisk.h Fri Dec 18 16:47:37 2009
@@ -40,7 +40,7 @@
 #define ROUND_DOWN_TO_STORE_BLOCK(_x)   (((_x) >> STORE_BLOCK_SHIFT) << STORE_BLOCK_SHIFT)
 
 #define STORE_BLOCKS_PER_PART  (PART_BLOCK_SIZE / STORE_BLOCK_SIZE)
-#define DISK_HEADER_MAGIC               0xABCD1234
+#define DISK_HEADER_MAGIC               0xABCD1235
 
 /* each disk part block has a corresponding Part object */
 struct CacheDisk;
@@ -70,9 +70,9 @@
   int num_partblocks;           /* number of disk partition blocks in this discrete 
                                    partition */
   int part_number;              /* the partition number of this partition */
-  int size;                     /* size in store blocks */
+  ink_off_t size;               /* size in store blocks */
   CacheDisk *disk;
-    Queue<DiskPartBlockQueue> dpb_queue;
+  Queue<DiskPartBlockQueue> dpb_queue;
 };
 
 struct DiskHeader
@@ -129,7 +129,7 @@
 
   int syncDone(int event, void *data);
 
-  DiskPartBlock *create_partition(int number, int size, int scheme);
+  DiskPartBlock *create_partition(int number, ink_off_t size, int scheme);
 
   int delete_partition(int number);
 

Modified: incubator/trafficserver/traffic/branches/dev/iocore/cache/P_CacheHttp.h
URL: http://svn.apache.org/viewvc/incubator/trafficserver/traffic/branches/dev/iocore/cache/P_CacheHttp.h?rev=892310&r1=892309&r2=892310&view=diff
==============================================================================
--- incubator/trafficserver/traffic/branches/dev/iocore/cache/P_CacheHttp.h (original)
+++ incubator/trafficserver/traffic/branches/dev/iocore/cache/P_CacheHttp.h Fri Dec 18 16:47:37 2009
@@ -80,7 +80,7 @@
 
   int marshal_length();
   int marshal(char *buf, int length);
-  int get_handles(const char *buf, int length, RefCountObj * block_ptr = NULL);
+  inku32 get_handles(const char *buf, int length, RefCountObj * block_ptr = NULL);
   int unmarshal(const char *buf, int length, RefCountObj * block_ptr);
 
   CacheArray<vec_info> data;

Modified: incubator/trafficserver/traffic/branches/dev/iocore/cache/P_CacheInternal.h
URL: http://svn.apache.org/viewvc/incubator/trafficserver/traffic/branches/dev/iocore/cache/P_CacheInternal.h?rev=892310&r1=892309&r2=892310&view=diff
==============================================================================
--- incubator/trafficserver/traffic/branches/dev/iocore/cache/P_CacheInternal.h (original)
+++ incubator/trafficserver/traffic/branches/dev/iocore/cache/P_CacheInternal.h Fri Dec 18 16:47:37 2009
@@ -42,6 +42,7 @@
 // #define CACHE_INSPECTOR_PAGES
 #define MAX_CACHE_VCS_PER_THREAD        500
 
+#define INTEGRAL_FRAGS                  4
 
 #ifdef CACHE_INSPECTOR_PAGES
 #ifdef DEBUG
@@ -64,6 +65,12 @@
 #endif
 
 
+#define VC_LOCK_RETRY_EVENT() \
+  do { \
+    trigger = mutex->thread_holding->schedule_in_local(this,MUTEX_RETRY_DELAY,event); \
+    return EVENT_CONT; \
+  } while (0)
+
 #define VC_SCHED_LOCK_RETRY() \
   do { \
     trigger = mutex->thread_holding->schedule_in_local(this,MUTEX_RETRY_DELAY); \
@@ -190,8 +197,6 @@
 } while (0);
 
 // Configuration
-extern int cache_config_mmap_directory;
-extern int cache_config_sem_key;
 extern int cache_config_dir_sync_frequency;
 extern int cache_config_http_max_alts;
 extern int cache_config_permit_pinning;
@@ -200,15 +205,15 @@
 extern int cache_config_max_doc_size;
 extern int cache_config_min_average_object_size;
 extern int cache_config_agg_write_backlog;
-#ifdef HIT_EVACUATE
-extern int cache_config_hit_evacuate_percent;
-extern int cache_config_hit_evacuate_size_limit;
-#endif
 extern int cache_config_enable_checksum;
 extern int cache_config_alt_rewrite_max_size;
 extern int cache_config_read_while_writer;
 extern char cache_system_config_directory[PATH_NAME_MAX + 1];
-extern int CacheClusteringEnabled;
+extern int cache_clustering_enabled;
+#ifdef HIT_EVACUATE
+extern int cache_config_hit_evacuate_percent;
+extern int cache_config_hit_evacuate_size_limit;
+#endif
 
 // CacheVC
 struct CacheVC:CacheVConnection
@@ -216,6 +221,7 @@
   CacheVC();
 
   VIO *do_io_read(Continuation * c, int nbytes, MIOBuffer * buf);
+  VIO *do_io_pread(Continuation *c, ink64 nbytes, MIOBuffer *buf, ink_off_t off);
   VIO *do_io_write(Continuation * c, int nbytes, IOBufferReader * buf, bool owner = false);
   void do_io_close(int lerrno = -1);
   void reenable(VIO * avio);
@@ -231,6 +237,20 @@
     ink_assert(vio.op == VIO::READ);
     return !f.not_from_ram_cache;
   }
+  int get_header(void **ptr, int *len) 
+  {
+    Doc *doc = (Doc*)first_buf->data();
+    *ptr = doc->hdr();
+    *len = doc->hlen;   
+    return 0;
+  }
+  int set_header(void *ptr, int len) 
+  {
+    header_to_write = ptr;
+    header_to_write_len = len;
+    return 0;
+  }
+
   bool writer_done();
   int calluser(int event);
   int die();
@@ -238,11 +258,12 @@
 
   int handleReadDone(int event, Event * e);
   int handleRead(int event, Event * e);
-  int do_read(CacheKey * akey);
+  int do_read_call(CacheKey * akey);
   int handleWrite(int event, Event * e);
   int handleWriteLock(int event, Event * e);
-  int do_write();
+  int do_write_call();
   int do_write_lock();
+  int do_write_lock_call();
 
   int openReadClose(int event, Event * e);
   int openReadReadDone(int event, Event * e);
@@ -362,12 +383,19 @@
   // NOTE: NOTE: NOTE: If vio is NOT the start, then CHANGE the 
   // size_to_init initialization
   VIO vio;
+  EThread *initial_thread;  // initial thread open_XX was called on
+  CacheFragType frag_type;
   CacheHTTPInfo *info;
   CacheHTTPInfoVector *write_vector;
 #ifdef HTTP_CACHE
   CacheLookupHttpConfig *params;
 #endif
-  int vec_len;
+  int header_len;       // for communicating with agg_copy
+  int frag_len;         // for communicating with agg_copy
+  inku32 write_len;     // for communicating with agg_copy
+  inku32 agg_len;       // for communicating with aggWrite
+  Frag *frag;           // arraylist of fragment offset
+  Frag integral_frags[INTEGRAL_FRAGS];
   Part *part;
   Dir *last_collision;
   Event *trigger;
@@ -378,20 +406,22 @@
   int base_stat;
   int recursive;
   int closed;
-  int docpos;                   // read position in 'buf'
-  int write_len;
-  int length;                   // length of data available to write
+  inku32 docpos;                // read position in 'buf'
   int offset;                   // offset into 'blocks' of data to write
-  int total_len;
-  int doc_len;
-  int update_len;
-  int segment;
-  int agg_len;
+  inku64 length;                // length of data available to write
+  inku64 write_pos;             // length written
+  inku64 total_len;             // total length written and available to write
+  inku64 doc_len;
+  inku64 update_len;
+  inku64 seek_to;
+  int fragment;
   int scan_msec_delay;
   int writer_offset;
   CacheVC *write_vc;
   char *hostname;
   int host_len;
+  int header_to_write_len;  
+  void *header_to_write;
   short writer_lock_retry;
   union
   {
@@ -401,8 +431,7 @@
       unsigned int use_first_key:1;
       unsigned int overwrite:1; // overwrite first_key Dir if it exists
       unsigned int evacuator:1;
-      unsigned int http_request:1;
-      unsigned int single_segment:1;
+      unsigned int single_fragment:1;
       unsigned int evac_vector:1;
       unsigned int lookup:1;
 #ifdef HIT_EVACUATE
@@ -421,21 +450,21 @@
   //end region C
 };
 
-#define PUSH_HANDLER(_x) do { \
-save_handler = handler; handler = (ContinuationHandler)(_x); \
+#define PUSH_HANDLER(_x) do {                                           \
+    ink_assert(handler != (ContinuationHandler)(&CacheVC::dead));       \
+    save_handler = handler; handler = (ContinuationHandler)(_x);        \
 } while (0)
 
-#define POP_HANDLER handler = save_handler
-
+#define POP_HANDLER do {                                          \
+    handler = save_handler;                                       \
+    ink_assert(handler != (ContinuationHandler)(&CacheVC::dead)); \
+  } while (0) 
 
 struct CacheRemoveCont:Continuation
 {
-
   int event_handler(int event, void *data);
-
-    CacheRemoveCont():Continuation(NULL)
-  {
-  }
+  
+  CacheRemoveCont():Continuation(NULL) { }
 };
 
 
@@ -463,6 +492,7 @@
   c->vector.data.data = &c->vector.data.fast_data[0];
 #endif
   c->_action = cont;
+  c->initial_thread = t;
   c->mutex = cont->mutex;
   c->start_time = ink_get_hrtime();
   ink_assert(c->trigger == NULL);
@@ -485,11 +515,9 @@
   if (cont->closed > 0) {
     CACHE_INCREMENT_DYN_STAT(cont->base_stat + CACHE_STAT_SUCCESS);
   }                             // else abort,cancel
-  EThread *t = mutex->thread_holding;
-  ink_debug_assert(t == this_ethread());
-  if (cont->trigger) {
+  ink_debug_assert(mutex->thread_holding == this_ethread());
+  if (cont->trigger)
     cont->trigger->cancel();
-  }
   ink_assert(!cont->is_io_in_progress());
   ink_assert(!cont->od);
   /* calling cont->io.action = NULL causes compile problem on 2.6 solaris
@@ -520,7 +548,8 @@
   cont->blocks.clear();
   cont->writer_buf.clear();
   cont->alternate_index = CACHE_ALT_INDEX_DEFAULT;
-
+  if (cont->frag && cont->frag != cont->integral_frags)
+    xfree(cont->frag);
   memset((char *) &cont->vio, 0, cont->size_to_init);
 #ifdef CACHE_STAT_PAGES
   ink_assert(!cont->stat_link.next && !cont->stat_link.prev);
@@ -528,7 +557,7 @@
 #ifdef DEBUG
   SET_CONTINUATION_HANDLER(cont, &CacheVC::dead);
 #endif
-  THREAD_FREE_TO(cont, cacheVConnectionAllocator, t, MAX_CACHE_VCS_PER_THREAD);
+  THREAD_FREE_TO(cont, cacheVConnectionAllocator, this_ethread(), MAX_CACHE_VCS_PER_THREAD);
   return EVENT_DONE;
 }
 
@@ -536,25 +565,8 @@
 CacheVC::calluser(int event)
 {
   recursive++;
-#ifdef AUTO_PILOT_MODE
-  if (vio.buffer.mbuf->autopilot) {
-    switch (event) {
-    case VC_EVENT_READ_READY:
-      vio.buffer.mbuf->reenable_readers();
-      break;
-    case VC_EVENT_WRITE_READY:
-      vio.buffer.mbuf->reenable_writer();
-      break;
-    default:
-      vio._cont->handleEvent(event, (void *) &vio);
-      break;
-    }
-  } else {
-    vio._cont->handleEvent(event, (void *) &vio);
-  }
-#else
+  ink_debug_assert(!part || this_ethread() != part->mutex->thread_holding);
   vio._cont->handleEvent(event, (void *) &vio);
-#endif
   recursive--;
   if (closed) {
     die();
@@ -564,20 +576,20 @@
 }
 
 inline int
-CacheVC::do_read(CacheKey * akey)
+CacheVC::do_read_call(CacheKey * akey)
 {
   docpos = 0;
   read_key = akey;
   io.aiocb.aio_nbytes = dir_approx_size(&dir);
   PUSH_HANDLER(&CacheVC::handleRead);
-  return handleRead(EVENT_NONE, 0);
+  return handleRead(EVENT_CALL, 0);
 }
 
 inline int
-CacheVC::do_write()
+CacheVC::do_write_call()
 {
   PUSH_HANDLER(&CacheVC::handleWrite);
-  return handleWrite(EVENT_NONE, 0);
+  return handleWrite(EVENT_CALL, 0);
 }
 
 inline void
@@ -620,13 +632,19 @@
 CacheVC::handleWriteLock(int event, Event * e)
 {
   cancel_trigger();
-  MUTEX_TRY_LOCK(lock, part->mutex, mutex->thread_holding);
-  if (!lock) {
-    set_agg_write_in_progress();
-    trigger = mutex->thread_holding->schedule_in_local(this, MUTEX_RETRY_DELAY);
-    return EVENT_CONT;
+  int ret = 0;
+  {
+    CACHE_TRY_LOCK(lock, part->mutex, mutex->thread_holding);
+    if (!lock) {
+      set_agg_write_in_progress();
+      trigger = mutex->thread_holding->schedule_in_local(this, MUTEX_RETRY_DELAY);
+      return EVENT_CONT;
+    }
+    ret = handleWrite(event, e);
   }
-  return handleWrite(event, e);
+  if (ret == EVENT_RETURN)
+    return handleEvent(AIO_EVENT_DONE, 0);
+  return EVENT_CONT;
 }
 
 inline int
@@ -636,6 +654,13 @@
   return handleWriteLock(EVENT_NONE, 0);
 }
 
+inline int
+CacheVC::do_write_lock_call()
+{
+  PUSH_HANDLER(&CacheVC::handleWriteLock);
+  return handleWriteLock(EVENT_CALL, 0);
+}
+
 inline bool
 CacheVC::writer_done()
 {
@@ -697,7 +722,7 @@
 Part::close_write_lock(CacheVC * cont)
 {
   EThread *t = cont->mutex->thread_holding;
-  MUTEX_TRY_LOCK(lock, mutex, t);
+  CACHE_TRY_LOCK(lock, mutex, t);
   if (!lock)
     return -1;
   return close_write(cont);
@@ -707,7 +732,7 @@
 Part::open_write_lock(CacheVC * cont, int allow_if_writers, int max_writers)
 {
   EThread *t = cont->mutex->thread_holding;
-  MUTEX_TRY_LOCK(lock, mutex, t);
+  CACHE_TRY_LOCK(lock, mutex, t);
   if (!lock)
     return -1;
   return open_write(cont, allow_if_writers, max_writers);
@@ -727,7 +752,7 @@
 {
   // no need for evacuation as the entire document is already in memory
 #ifndef  CACHE_STAT_PAGES
-  if (cont->f.single_segment)
+  if (cont->f.single_fragment)
     return 0;
 #endif
   // VC is enqueued in stat_cache_vcs in the begin_read call
@@ -862,7 +887,6 @@
 
 struct Cache
 {
-
   volatile int cache_read_done;
   volatile int total_good_npart;
   int total_npart;
@@ -901,7 +925,6 @@
                      CacheFragType type = CACHE_FRAG_TYPE_HTTP);
   Action *remove(Continuation * cont, URL * url, CacheFragType type);
   static void generate_key(INK_MD5 * md5, URL * url, CacheHTTPHdr * request);
-
 #endif
 
   Action *link(Continuation * cont, CacheKey * from, CacheKey * to, CacheFragType type, char *hostname, int host_len);
@@ -914,12 +937,10 @@
 
   Part *key_to_part(CacheKey * key, char *hostname, int host_len);
 
-    Cache():cache_read_done(0), total_good_npart(0), total_npart(0), ready(CACHE_INITIALIZING), cache_size(0),  // in store block size
-    hosttable(NULL), total_initialized_part(0), scheme(CACHE_NONE_TYPE)
-  {
-  }
-
-
+  Cache():cache_read_done(0), total_good_npart(0), total_npart(0), ready(CACHE_INITIALIZING), cache_size(0),  // in store block size
+          hosttable(NULL), total_initialized_part(0), scheme(CACHE_NONE_TYPE)
+    {
+    }
 };
 
 extern Cache *theCache;
@@ -1025,7 +1046,7 @@
   (void) local_only;
 #ifdef CLUSTER_CACHE
   // Try to send remote, if not possible, handle locally
-  if ((CacheClusteringEnabled > 0) && !local_only) {
+  if ((cache_clustering_enabled > 0) && !local_only) {
     Action *a = Cluster_lookup(cont, key, frag_type, hostname, host_len);
     if (a) {
       return a;
@@ -1039,7 +1060,7 @@
 CacheProcessor::open_read(Continuation * cont, CacheKey * key, CacheFragType frag_type, char *hostname, int host_len)
 {
 #ifdef CLUSTER_CACHE
-  if (CacheClusteringEnabled > 0) {
+  if (cache_clustering_enabled > 0) {
     return open_read_internal(CACHE_OPEN_READ, cont, (MIOBuffer *) 0,
                               (CacheURL *) 0, (CacheHTTPHdr *) 0,
                               (CacheLookupHttpConfig *) 0, key, 0, frag_type, hostname, host_len);
@@ -1054,7 +1075,7 @@
 {
   (void) buf;
 #ifdef CLUSTER_CACHE
-  if (CacheClusteringEnabled > 0) {
+  if (cache_clustering_enabled > 0) {
     return open_read_internal(CACHE_OPEN_READ_BUFFER, cont, buf,
                               (CacheURL *) 0, (CacheHTTPHdr *) 0,
                               (CacheLookupHttpConfig *) 0, key, 0, frag_type, hostname, host_len);
@@ -1072,7 +1093,7 @@
 #ifdef CLUSTER_CACHE
   ClusterMachine *m = cluster_machine_at_depth(cache_hash(*key));
 
-  if (m && (CacheClusteringEnabled > 0)) {
+  if (m && (cache_clustering_enabled > 0)) {
     return Cluster_write(cont, expected_size, (MIOBuffer *) 0, m,
                          key, frag_type, overwrite, pin_in_cache,
                          CACHE_OPEN_WRITE, key, (CacheURL *) 0,
@@ -1103,7 +1124,7 @@
                        bool rm_user_agents, bool rm_link, CacheFragType frag_type, char *hostname, int host_len)
 {
 #ifdef CLUSTER_CACHE
-  if (CacheClusteringEnabled > 0) {
+  if (cache_clustering_enabled > 0) {
     ClusterMachine *m = cluster_machine_at_depth(cache_hash(*key));
 
     if (m) {
@@ -1141,7 +1162,7 @@
 {
   (void) buf;
 #ifdef CLUSTER_CACHE
-  if (CacheClusteringEnabled > 0) {
+  if (cache_clustering_enabled > 0) {
     return open_read_internal(CACHE_OPEN_READ_BUFFER_LONG, cont, buf, url,
                               request, params, (CacheKey *) 0, 0, type, (char *) 0, 0);
   }
@@ -1164,28 +1185,6 @@
   return NULL;
 }
 
-/*INK_INLINE Action* CacheProcessor::remove (Continuation *cont, URL *url, 
-				       CacheFragType frag_type) {
-#ifdef CLUSTER_CACHE
-  if (CacheClusteringEnabled > 0) {
-  }
-#endif
- if (cache_global_hooks != NULL and cache_global_hooks->hooks_set > 0) {
-    Debug("cache_plugin", "Cache hooks are set");
-    APIHook *cache_lookup = cache_global_hooks->get(INK_CACHE_DELETE_HOOK);
-    if (cache_lookup != NULL) {
-       NewCacheVC *vc = NewCacheVC::alloc(cont, url, sm, VIO::READ);
-       int rval = cache_lookup->invoke(INK_EVENT_DELETE_HOOK, (void*)vc);
-       if (rval == INK_SUCCESS) {
-          return ACTION_RESULT_DONE;
-        } else {
-          abort();
-        }
-    }
- }
-  return caches[frag_type]->remove(cont, url, frag_type);
-}*/
-
 #endif
 
 
@@ -1228,7 +1227,7 @@
 CacheProcessor::link(Continuation * cont, CacheKey * from, CacheKey * to,
                      CacheFragType type, char *hostname, int host_len)
 {
-  if (CacheClusteringEnabled > 0) {
+  if (cache_clustering_enabled > 0) {
     // Use INK_MD5 in "from" to determine target machine
     ClusterMachine *m = cluster_machine_at_depth(cache_hash(*from));
     if (m) {
@@ -1241,7 +1240,7 @@
 INK_INLINE Action *
 CacheProcessor::deref(Continuation * cont, CacheKey * key, CacheFragType type, char *hostname, int host_len)
 {
-  if (CacheClusteringEnabled > 0) {
+  if (cache_clustering_enabled > 0) {
     ClusterMachine *m = cluster_machine_at_depth(cache_hash(*key));
     if (m) {
       return Cluster_deref(m, cont, key, type, hostname, host_len);

Modified: incubator/trafficserver/traffic/branches/dev/iocore/cache/P_CachePart.h
URL: http://svn.apache.org/viewvc/incubator/trafficserver/traffic/branches/dev/iocore/cache/P_CachePart.h?rev=892310&r1=892309&r2=892310&view=diff
==============================================================================
--- incubator/trafficserver/traffic/branches/dev/iocore/cache/P_CachePart.h (original)
+++ incubator/trafficserver/traffic/branches/dev/iocore/cache/P_CachePart.h Fri Dec 18 16:47:37 2009
@@ -25,47 +25,37 @@
 #ifndef _P_CACHE_PART_H__
 #define _P_CACHE_PART_H__
 
-// Definitions
-#define AIO_NOT_IN_PROGRESS             0
-#define AIO_AGG_WRITE_IN_PROGRESS       -1
-#define SHM_MAGIC                       "TrafficServerCache"
-#define SHM_SIZE                        128
-#define AUTO_SIZE_RAM_CACHE             -1      // 1-1 with directory size
+#define INK_BLOCK_SHIFT			9
+#define INK_BLOCK_SIZE			(1<<INK_BLOCK_SHIFT)
+#define ROUND_TO_BLOCK(_x)	        (((_x)+(INK_BLOCK_SIZE-1))&~(INK_BLOCK_SIZE-1))
+#define ROUND_TO(_x, _y)	        (((_x)+((_y)-1))&~((_y)-1))
+
+// Part
 
-#define INK_BLOCK_SIZE			512
-#define INK_BLOCK_SHIFT                 9
-#define B8K_SIZE			8192
-#define B8K_SHIFT                       13
-#define ROUND_TO_16(_x)                 (((_x)+(16-1))&~(16-1))
-#define ROUND_TO_BLOCK(_x)	(((_x)+(INK_BLOCK_SIZE-1))&~(INK_BLOCK_SIZE-1))
-#define ROUND_TO_8K(_x) 		(((_x)+(B8K_SIZE-1))&~(B8K_SIZE-1))
-#define ROUND_TO_PAGE(_x) 		(((_x)+(STORE_BLOCK_SIZE-1))&\
-                                          ~(STORE_BLOCK_SIZE-1))
+#define PART_MAGIC			0xF1D0F00D
 #define START_BLOCKS                    32      // 8k
 #define START_POS			((ink_off_t)START_BLOCKS * INK_BLOCK_SIZE)
-#define AGG_BLOCKS                      1024
 #define AGG_HEADER_SIZE                 INK_BLOCK_SIZE
-#define AGG_SIZE                        (AGG_BLOCKS * INK_BLOCK_SIZE)   // 512k
-#define EVAC_SIZE                       (2 * AGG_SIZE)  // 1MB
-#define MAX_PART_SIZE                   ((ink_off_t)8 * 1024 * 1024 * 1024)
+#define AGG_SIZE                        (4 * 1024 * 1024) // 4MB
+#define AGG_HIGH_WATER                  (AGG_SIZE / 2) // 2MB
+#define EVACUATION_SIZE                 (2 * AGG_SIZE)  // 8MB
+#define MAX_PART_SIZE                   ((ink_off_t)512 * 1024 * 1024 * 1024 * 1024)
 #define STORE_BLOCKS_PER_DISK_BLOCK     (STORE_BLOCK_SIZE / INK_BLOCK_SIZE)
 #define MAX_PART_BLOCKS                 (MAX_PART_SIZE / INK_BLOCK_SIZE)
-#define TARGET_FRAG_SIZE                (32768 - sizeofDoc)
-#define SHRINK_TARGET_FRAG_SIZE         (32768 + 8192)
+#define TARGET_FRAG_SIZE                (DEFAULT_MAX_BUFFER_SIZE - sizeofDoc)
+#define SHRINK_TARGET_FRAG_SIZE         (DEFAULT_MAX_BUFFER_SIZE + (DEFAULT_MAX_BUFFER_SIZE/4))
 #define MAX_FRAG_SIZE                   ((256 * 1024) - sizeofDoc)
-#define AGG_TODO_SIZE_MAX               (5242880)       // 5 MB
-#define LEAVE_FREE                      32768
+#define LEAVE_FREE                      DEFAULT_MAX_BUFFER_SIZE
 #define PIN_SCAN_EVERY                  16      // scan every 1/16 of disk
-
 #define PART_HASH_TABLE_SIZE            32707
 #define PART_HASH_EMPTY                 0xFFFF
-
 #define LOOKASIDE_SIZE                  256
+#define EVACUATION_BUCKET_SIZE          (2 * EVACUATION_SIZE) // 16MB
+#define RECOVERY_SIZE                   EVACUATION_SIZE // 8MB
+#define AIO_NOT_IN_PROGRESS             0
+#define AIO_AGG_WRITE_IN_PROGRESS       -1
+#define AUTO_SIZE_RAM_CACHE             -1      // 1-1 with directory size
 
-// Part
-
-#define PART_MAGIC			0xF1D0F00D
-#define EVACUATION_BUCKET_SIZE          (2 * 1024 * 1024)
 
 #define dir_offset_evac_bucket(_o) \
   (_o / (EVACUATION_BUCKET_SIZE / INK_BLOCK_SIZE))
@@ -79,15 +69,10 @@
 #define DOC_CORRUPT                     ((inku32)0xDEADBABE)
 #define DOC_NO_CHECKSUM                 ((inku32)0xA0B0C0D0)
 
-#define sizeofDoc ((int)(long)&((Doc*)0)->hdr[0])
-
-
-enum
-{ CACHE_METADATA_INSERT, CACHE_METADATA_UPDATE,
-  CACHE_METADATA_RESERVED, CACHE_METADATA_RESERVED2
-};
+#define sizeofDoc (((inku32)(uintptr_t)&((Doc*)0)->checksum)+(inku32)sizeof(inku32))
 
 struct Cache;
+struct Part;
 struct CacheDisk;
 struct PartInitInfo;
 struct DiskPart;
@@ -97,17 +82,17 @@
 {
   unsigned int magic;
   VersionNumber version;
+  time_t create_time;
   ink_off_t write_pos;
   ink_off_t last_write_pos;
-  inku32 generation;            // token generation (vary), this cannot be 0
   ink_off_t agg_pos;
-  unsigned int phase;
-  unsigned int cycle;
-  unsigned int sync_serial;
-  unsigned int write_serial;
-  time_t create_time;
-  unsigned int dirty;
-  unsigned short freelist[DIR_SEGMENTS];
+  inku32 generation;            // token generation (vary), this cannot be 0
+  inku32 phase;
+  inku32 cycle;
+  inku32 sync_serial;
+  inku32 write_serial;
+  inku32 dirty;
+  inku16 freelist[1];
 };
 
 // Key and Earliest key for each fragment that needs to be evacuated
@@ -140,52 +125,6 @@
     Link<EvacuationBlock> link;
 };
 
-struct MetaData
-{
-  union
-  {
-    struct
-    {
-      unsigned int op:2;
-      unsigned int unused:30;
-      unsigned int doc_serial;
-      ink_off_t write_pos;
-    } hdr;
-    struct
-    {
-      unsigned int op:2;
-      unsigned int hosthash:14;
-      unsigned int segment:4;
-      unsigned int bucket:12;
-      unsigned int pin_time;
-      Dir dir;
-    } insert;
-    struct
-    {
-      unsigned int op:2;
-      unsigned int hosthash:14;
-      unsigned int segment:4;
-      unsigned int bucket:12;
-      unsigned int new_offset;
-      Dir dir;
-    } update;
-  };
-};
-
-struct PartCallback:public Continuation
-{
-
-  Queue<CacheVC> write_done;
-  Event *trigger;
-
-  int aggWriteDone(int event, Event * e);
-
-    PartCallback(ProxyMutex * m):Continuation(m), trigger(0)
-  {
-    SET_HANDLER(&PartCallback::aggWriteDone);
-  }
-};
-
 struct Part:public Continuation
 {
   char *path;
@@ -197,24 +136,20 @@
   Dir *dir;
   PartHeaderFooter *header;
   PartHeaderFooter *footer;
-  int buckets;
-  Dir *segment[DIR_SEGMENTS];
-  MetaData *metadata;
+  int segments;
+  ink_off_t buckets;
   ink_off_t recover_pos;
   ink_off_t prev_recover_pos;
   ink_off_t scan_pos;
-  ink_off_t metadata_pos;
   ink_off_t skip;               // start of headers
   ink_off_t start;              // start of data
   ink_off_t len;
-  int data_blocks;
+  ink_off_t data_blocks;
   int hit_evacuate_window;
-
   AIOCallbackInternal io;
 
-    Queue<CacheVC> agg;
-    Queue<CacheVC> stat_cache_vcs;
-  PartCallback *callback_cont;
+  Queue<CacheVC> agg;
+  Queue<CacheVC> stat_cache_vcs;
   char *agg_buffer;
   int agg_todo_size;
   int agg_buf_pos;
@@ -224,8 +159,8 @@
   OpenDir open_dir;
   RamCache ram_cache;
   int evacuate_size;
-    DLL<EvacuationBlock> *evacuate;
-    DLL<EvacuationBlock> lookaside[LOOKASIDE_SIZE];
+  DLL<EvacuationBlock> *evacuate;
+  DLL<EvacuationBlock> lookaside[LOOKASIDE_SIZE];
   CacheVC *doc_evacuator;
 
   PartInitInfo *init_info;
@@ -243,12 +178,12 @@
 
   void cancel_trigger();
 
-  int open_write(CacheVC * cont, int allow_if_writers, int max_writers);
-  int open_write_lock(CacheVC * cont, int allow_if_writers, int max_writers);
-  int close_write(CacheVC * cont);
-  int close_write_lock(CacheVC * cont);
-  int begin_read(CacheVC * cont);
-  int begin_read_lock(CacheVC * cont);
+  int open_write(CacheVC *cont, int allow_if_writers, int max_writers);
+  int open_write_lock(CacheVC *cont, int allow_if_writers, int max_writers);
+  int close_write(CacheVC *cont);
+  int close_write_lock(CacheVC *cont);
+  int begin_read(CacheVC *cont);
+  int begin_read_lock(CacheVC *cont);
   // unused read-write interlock code
   // currently http handles a write-lock failure by retrying the read
   OpenDirEntry *open_read(INK_MD5 * key);
@@ -289,29 +224,28 @@
     io.aiocb.aio_fildes = AIO_NOT_IN_PROGRESS;
   }
 
-  int aggWriteDone(int event, Event * e);
-  int aggWrite(int event, Event * e);
+  int aggWriteDone(int event, Event *e);
+  int aggWrite(int event, void *e);
   void agg_wrap();
 
-  int evacuateWrite(CacheVC * evacuator, int event, Event * e);
-  int evacuateDocReadDone(int event, Event * e);
-  int evacuateDoc(int event, Event * e);
+  int evacuateWrite(CacheVC *evacuator, int event, Event *e);
+  int evacuateDocReadDone(int event, Event *e);
+  int evacuateDoc(int event, Event *e);
 
   int evac_range(ink_off_t start, ink_off_t end, int evac_phase);
   void periodic_scan();
   void scan_for_pinned_documents();
   void evacuate_cleanup_blocks(int i);
   void evacuate_cleanup();
-  EvacuationBlock *force_evacuate_head(Dir * dir, int pinned);
-  int within_hit_evacuate_window(Dir * dir);
+  EvacuationBlock *force_evacuate_head(Dir *dir, int pinned);
+  int within_hit_evacuate_window(Dir *dir);
 
 Part():Continuation(new_ProxyMutex()), path(NULL), fd(-1),
-    dir(0), buckets(0), recover_pos(0), prev_recover_pos(0), scan_pos(0), metadata_pos(0), skip(0), start(0),
+    dir(0), buckets(0), recover_pos(0), prev_recover_pos(0), scan_pos(0), skip(0), start(0),
     len(0), data_blocks(0), hit_evacuate_window(0), agg_todo_size(0), agg_buf_pos(0), trigger(0),
     evacuate_size(0), disk(NULL), last_sync_serial(0), last_write_serial(0), recover_wrapped(false),
     dir_sync_waiting(0), dir_sync_in_progress(0) {
     open_dir.mutex = mutex;
-    callback_cont = NEW(new PartCallback(mutex));
 #if defined(_WIN32)
     agg_buffer = (char *) malloc(AGG_SIZE);
 #else
@@ -322,21 +256,17 @@
   }
 
   ~Part() {
-    delete callback_cont;
     ink_memalign_free(agg_buffer);
   }
 };
 
 struct AIO_Callback_handler:public Continuation
 {
-
   int handle_disk_failure(int event, void *data);
 
-    AIO_Callback_handler():Continuation(new_ProxyMutex())
-  {
+  AIO_Callback_handler():Continuation(new_ProxyMutex()) {
     SET_HANDLER(&AIO_Callback_handler::handle_disk_failure);
   }
-
 };
 
 struct CachePart
@@ -351,32 +281,38 @@
   // per partition stats
   RecRawStatBlock *part_rsb;
 
-    CachePart():part_number(-1), scheme(0), size(0), num_parts(0), parts(NULL), disk_parts(0), part_rsb(0)
-  {
-  }
-
+  CachePart():part_number(-1), scheme(0), size(0), num_parts(0), parts(NULL), disk_parts(0), part_rsb(0) { }
+};
 
+// element of the fragment table in the head of a multi-fragment document
+struct Frag {
+  inku64 offset; // start offset of data stored in this fragment
 };
 
-// Note : the hdr field needs to be 8 byte aligned. 
+// Note : hdr() needs to be 8 byte aligned. 
+// If you change this, change sizeofDoc above
 struct Doc
 {
   inku32 magic;                 // DOC_MAGIC
-  ink32 len;                    // length of this segment
-  ink32 hlen;                   // header length
-  ink32 total_len;              // total length of document
+  inku32 len;                   // length of this segment
+  inku64 total_len;             // total length of document
   INK_MD5 first_key;            // first key in document (http: vector)
   INK_MD5 key;
+  inku32 hlen;                  // header length
+  inku32 ftype:8;               // fragment type CACHE_FRAG_TYPE_XX
+  inku32 flen:24;               // fragment table length
   inku32 sync_serial;
   inku32 write_serial;
   inku32 pinned;                // pinned until
   inku32 checksum;
-  char hdr[1];
 
-
-  int data_len();
-  int single_segment();
-  int no_data_in_segment();
+  inku32 data_len();
+  inku32 prefix_len();
+  int single_fragment();
+  int no_data_in_fragment();
+  inku32 nfrags();
+  char *hdr();
+  Frag *frags();
   char *data();
 };
 
@@ -392,21 +328,19 @@
 // inline Functions
 
 inline int
-part_dirlen(Part * d)
-{
-  return ROUND_TO_BLOCK(d->buckets *
-                        DIR_DEPTH * DIR_SEGMENTS * SIZEOF_DIR) + 2 * ROUND_TO_BLOCK(sizeof(PartHeaderFooter));
+part_headerlen(Part *d) {
+  return ROUND_TO_BLOCK(sizeof(PartHeaderFooter) + sizeof(inku16) * (d->segments-1));
 }
 inline int
-part_direntries(Part * d)
+part_dirlen(Part * d)
 {
-  return d->buckets * DIR_DEPTH * DIR_SEGMENTS;
+  return ROUND_TO_BLOCK(d->buckets * DIR_DEPTH * d->segments * SIZEOF_DIR) + 
+    part_headerlen(d) + ROUND_TO_BLOCK(sizeof(PartHeaderFooter));
 }
 inline int
-part_metalen(Part * e)
+part_direntries(Part * d)
 {
-  (void) e;
-  return 0;
+  return d->buckets * DIR_DEPTH * d->segments;
 }
 inline int
 part_out_of_phase_valid(Part * d, Dir * e)
@@ -433,10 +367,10 @@
 {
   return d->start + (ink_off_t) dir_offset(e) * INK_BLOCK_SIZE - INK_BLOCK_SIZE;
 }
-inline int
+inline ink_off_t
 offset_to_part_offset(Part * d, ink_off_t pos)
 {
-  return (int) ((pos - d->start + INK_BLOCK_SIZE) / INK_BLOCK_SIZE);
+  return ((pos - d->start + INK_BLOCK_SIZE) / INK_BLOCK_SIZE);
 }
 inline ink_off_t
 part_offset_to_offset(Part * d, ink_off_t pos)
@@ -453,21 +387,39 @@
 {
   return (part_offset(d, e) >= d->header->write_pos && part_offset(d, e) < (d->header->write_pos + d->agg_buf_pos));
 }
-
-inline int
+inline inku32
+Doc::prefix_len()
+{
+  return sizeofDoc + hlen + flen;
+}
+inline inku32
 Doc::data_len()
 {
-  return len - sizeofDoc - hlen;
+  return len - sizeofDoc - hlen - flen;
 }
 inline int
-Doc::single_segment()
+Doc::single_fragment()
 {
   return (total_len && (data_len() == total_len));
 }
+inline inku32
+Doc::nfrags() { 
+  return flen / sizeof(Frag);
+}
+inline Frag *
+Doc::frags()
+{
+  return (Frag*)(((char *) this) + sizeofDoc);
+}
+inline char *
+Doc::hdr()
+{
+  return ((char *) this) + sizeofDoc + flen;
+}
 inline char *
 Doc::data()
 {
-  return ((char *) this) + sizeofDoc + hlen;
+  return ((char *) this) + sizeofDoc + flen + hlen;
 }
 
 int part_dir_clear(Part * d);

Modified: incubator/trafficserver/traffic/branches/dev/iocore/cache/P_RamCache.h
URL: http://svn.apache.org/viewvc/incubator/trafficserver/traffic/branches/dev/iocore/cache/P_RamCache.h?rev=892310&r1=892309&r2=892310&view=diff
==============================================================================
--- incubator/trafficserver/traffic/branches/dev/iocore/cache/P_RamCache.h (original)
+++ incubator/trafficserver/traffic/branches/dev/iocore/cache/P_RamCache.h Fri Dec 18 16:47:37 2009
@@ -64,7 +64,7 @@
   ink64 objects;
   ink64 partition_size;
   ink64 seen_size;
-  int cutoff_size;
+  inku32 cutoff_size;
   RamCachePartition *partition;
   RamCachePartition one_partition;
   Part *part;                   // back pointer to partition

Modified: incubator/trafficserver/traffic/branches/dev/iocore/cache/RamCache.cc
URL: http://svn.apache.org/viewvc/incubator/trafficserver/traffic/branches/dev/iocore/cache/RamCache.cc?rev=892310&r1=892309&r2=892310&view=diff
==============================================================================
--- incubator/trafficserver/traffic/branches/dev/iocore/cache/RamCache.cc (original)
+++ incubator/trafficserver/traffic/branches/dev/iocore/cache/RamCache.cc Fri Dec 18 16:47:37 2009
@@ -82,7 +82,7 @@
   if (partition_size == 0)
     return 0;
 
-  inku32 k = key->word(2);
+  inku32 k = key->word(3);
   inku32 pp = k % n_partitions;
   RamCachePartition *p = &partition[pp];
   inku32 o = k / n_partitions;
@@ -105,11 +105,11 @@
 int
 RamCache::get_lock(INK_MD5 * key, Ptr<IOBufferData> *ret_data, EThread * t, inku32 auxkey1, inku32 auxkey2)
 {
-  inku32 k = key->word(2);
+  inku32 k = key->word(3);
   int pp = k % n_partitions;
   RamCachePartition *p = &partition[pp];
   (void) p;
-  MUTEX_TRY_LOCK(l, p->lock, t);
+  CACHE_TRY_LOCK(l, p->lock, t);
   if (!l)
     return -1;
   return get(key, ret_data, auxkey1, auxkey2);
@@ -118,13 +118,13 @@
 void
 RamCache::remove_entry(RamCacheEntry * ee, RamCachePartition * p, EThread * t)
 {
-  inku32 oo = ee->key.word(2) / n_partitions;
+  inku32 oo = ee->key.word(3) / n_partitions;
   inku32 ii = oo % partition_size;
   p->bucket[ii].remove(ee, ee->hash_link);
   p->cur_bytes -= ee->data->block_size();
   ProxyMutex *mutex = part->mutex;
   CACHE_SUM_DYN_STAT(cache_ram_cache_bytes_stat, -ee->data->block_size());
-  Debug("ram_cache", "put %X %d %d FREED", ee->key.word(2), ee->auxkey1, ee->auxkey2);
+  Debug("ram_cache", "put %X %d %d FREED", ee->key.word(3), ee->auxkey1, ee->auxkey2);
   free_RamCacheEntry(ee, t);
 }
 
@@ -138,7 +138,7 @@
 
   ProxyMutex *mutex = t->mutex;
   (void) mutex;
-  inku32 k = key->word(2);
+  inku32 k = key->word(3);
   inku32 pp = k % n_partitions;
   inku32 o = k / n_partitions;
   inku32 i = o % partition_size;
@@ -192,11 +192,11 @@
 int
 RamCache::put_lock(INK_MD5 * key, IOBufferData * data, EThread * t, inku32 auxkey1, inku32 auxkey2)
 {
-  inku32 k = key->word(2);
+  inku32 k = key->word(3);
   int pp = k % n_partitions;
   RamCachePartition *p = &partition[pp];
   (void) p;
-  MUTEX_TRY_LOCK(l, p->lock, t);
+  CACHE_TRY_LOCK(l, p->lock, t);
   if (!l)
     return -1;
   return put(key, data, t, auxkey1, auxkey2);
@@ -209,7 +209,7 @@
   if (partition_size == 0)
     return 0;
   Debug("ram_cache", "fixup %d", key);
-  inku32 k = key->word(2);
+  inku32 k = key->word(3);
   inku32 pp = k % n_partitions;
   RamCachePartition *p = &partition[pp];
   inku32 o = k / n_partitions;

Modified: incubator/trafficserver/traffic/branches/dev/iocore/cluster/ClusterCache.cc
URL: http://svn.apache.org/viewvc/incubator/trafficserver/traffic/branches/dev/iocore/cluster/ClusterCache.cc?rev=892310&r1=892309&r2=892310&view=diff
==============================================================================
--- incubator/trafficserver/traffic/branches/dev/iocore/cluster/ClusterCache.cc (original)
+++ incubator/trafficserver/traffic/branches/dev/iocore/cluster/ClusterCache.cc Fri Dec 18 16:47:37 2009
@@ -1232,7 +1232,7 @@
         // Save hostname and attach it to the continuation since we may
         //  need it if we convert this to an open_write.
 
-        c->ic_hostname = new_IOBufferData(buffer_size_to_index(host_len));
+        c->ic_hostname = new_IOBufferData(iobuffer_size_to_index(host_len));
         c->ic_hostname_len = host_len;
 
         memcpy(c->ic_hostname->data(), hostname, host_len);
@@ -1489,7 +1489,7 @@
     ink_release_assert(caller_buf_freebytes);
     SET_HANDLER((CacheContHandler) & CacheContinuation::VCdataRead);
 
-    int size_index = buffer_size_to_index(caller_buf_freebytes);
+    int size_index = iobuffer_size_to_index(caller_buf_freebytes);
     MIOBuffer *buf = new_MIOBuffer(size_index);
     readahead_reader = buf->alloc_reader();
 
@@ -2781,7 +2781,7 @@
     }
 
   } else {
-    object_size = vc->get_object_size();
+    object_size = ((CacheVC *)vc)->get_object_size();
   }
 
   if (ret_ci && !ret_ci->valid()) {

Modified: incubator/trafficserver/traffic/branches/dev/iocore/cluster/ClusterConfig.cc
URL: http://svn.apache.org/viewvc/incubator/trafficserver/traffic/branches/dev/iocore/cluster/ClusterConfig.cc?rev=892310&r1=892309&r2=892310&view=diff
==============================================================================
--- incubator/trafficserver/traffic/branches/dev/iocore/cluster/ClusterConfig.cc (original)
+++ incubator/trafficserver/traffic/branches/dev/iocore/cluster/ClusterConfig.cc Fri Dec 18 16:47:37 2009
@@ -403,7 +403,7 @@
 cluster_machine_at_depth(unsigned int hash, int *pprobe_depth, ClusterMachine ** past_probes)
 {
 #ifdef CLUSTER_TOMCAT
-  if (!CacheClusteringEnabled)
+  if (!cache_clustering_enabled)
     return NULL;
 #endif
   ClusterConfiguration *

Modified: incubator/trafficserver/traffic/branches/dev/iocore/cluster/ClusterHandler.cc
URL: http://svn.apache.org/viewvc/incubator/trafficserver/traffic/branches/dev/iocore/cluster/ClusterHandler.cc?rev=892310&r1=892309&r2=892310&view=diff
==============================================================================
--- incubator/trafficserver/traffic/branches/dev/iocore/cluster/ClusterHandler.cc (original)
+++ incubator/trafficserver/traffic/branches/dev/iocore/cluster/ClusterHandler.cc Fri Dec 18 16:47:37 2009
@@ -592,8 +592,7 @@
               //////////////////////////////////////////////////////////
               ink_release_assert(s.msg.descriptor[i].length <= DEFAULT_MAX_BUFFER_SIZE);
               vc->read_block = new_IOBufferBlock();
-              int
-                index = buffer_size_to_index(s.msg.descriptor[i].length);
+              int index = buffer_size_to_index(s.msg.descriptor[i].length, MAX_BUFFER_SIZE_INDEX);
               vc->read_block->alloc(index);
 
               s.iov[new_n_iov].iov_base = 0;
@@ -610,8 +609,7 @@
             s.block[new_n_iov]->_buf_end = s.block[new_n_iov]->end() + s.msg.descriptor[i].length;
 
           } else {
-            bool
-              remote_write_fill = (vc->pending_remote_fill && vc->remote_write_block);
+            bool remote_write_fill = (vc->pending_remote_fill && vc->remote_write_block);
             // Sanity check, assert we have the lock
             if (!remote_write_fill) {
               ink_assert((ProxyMutex *) vc->write_locked);

Modified: incubator/trafficserver/traffic/branches/dev/iocore/cluster/ClusterHandlerBase.cc
URL: http://svn.apache.org/viewvc/incubator/trafficserver/traffic/branches/dev/iocore/cluster/ClusterHandlerBase.cc?rev=892310&r1=892309&r2=892310&view=diff
==============================================================================
--- incubator/trafficserver/traffic/branches/dev/iocore/cluster/ClusterHandlerBase.cc (original)
+++ incubator/trafficserver/traffic/branches/dev/iocore/cluster/ClusterHandlerBase.cc Fri Dec 18 16:47:37 2009
@@ -73,7 +73,7 @@
 
   ink_assert(!data);
   if ((len + DATA_HDR + sizeof(ink32)) <= DEFAULT_MAX_BUFFER_SIZE) {
-    size_index = buffer_size_to_index(len + DATA_HDR + sizeof(ink32));
+    size_index = buffer_size_to_index(len + DATA_HDR + sizeof(ink32), MAX_BUFFER_SIZE_INDEX);
     iob_block = new_IOBufferBlock();
     iob_block->alloc(size_index);       // aligns on 8 byte boundary
     real_data = (ink64 *) iob_block->buf();

Modified: incubator/trafficserver/traffic/branches/dev/iocore/cluster/ClusterProcessor.cc
URL: http://svn.apache.org/viewvc/incubator/trafficserver/traffic/branches/dev/iocore/cluster/ClusterProcessor.cc?rev=892310&r1=892309&r2=892310&view=diff
==============================================================================
--- incubator/trafficserver/traffic/branches/dev/iocore/cluster/ClusterProcessor.cc (original)
+++ incubator/trafficserver/traffic/branches/dev/iocore/cluster/ClusterProcessor.cc Fri Dec 18 16:47:37 2009
@@ -32,7 +32,7 @@
 // ClusterProcessor member functions (Public class)
 /*************************************************************************/
 int cluster_port_number = DEFAULT_CLUSTER_PORT_NUMBER;
-int CacheClusteringEnabled = 0;
+int cache_clustering_enabled = 0;
 
 ClusterProcessor clusterProcessor;
 RecRawStatBlock *cluster_rsb = NULL;
@@ -367,117 +367,64 @@
 ////////////////////////////////////////////////////////////////////////////
 #define GS RecRawStat
 #if 0
-GS *
-  p_CLUSTER_CONNECTIONS_OPEN_STAT = RecGetGlobalRawStatPtr(cluster_rsb, CLUSTER_CONNECTIONS_OPEN_STAT);
-GS *
-  p_CLUSTER_CONNECTIONS_OPENNED_STAT = RecGetGlobalRawStatPtr(cluster_rsb, CLUSTER_CONNECTIONS_OPENNED_STAT);
-GS *
-  p_CLUSTER_CON_TOTAL_TIME_STAT = RecGetGlobalRawStatPtr(cluster_rsb, CLUSTER_CON_TOTAL_TIME_STAT);
-GS *
-  p_CLUSTER_CTRL_MSGS_SENT_STAT = RecGetGlobalRawStatPtr(cluster_rsb, CLUSTER_CTRL_MSGS_SENT_STAT);
-GS *
-  p_CLUSTER_SLOW_CTRL_MSGS_SENT_STAT = RecGetGlobalRawStatPtr(cluster_rsb, CLUSTER_SLOW_CTRL_MSGS_SENT_STAT);
-GS *
-  p_CLUSTER_CTRL_MSGS_RECVD_STAT = RecGetGlobalRawStatPtr(cluster_rsb, CLUSTER_CTRL_MSGS_RECVD_STAT);
-GS *
-  p_CLUSTER_SLOW_CTRL_MSGS_RECVD_STAT = RecGetGlobalRawStatPtr(cluster_rsb, CLUSTER_SLOW_CTRL_MSGS_RECVD_STAT);
-GS *
-  p_CLUSTER_CTRL_MSGS_SEND_TIME_STAT = RecGetGlobalRawStatPtr(cluster_rsb, CLUSTER_CTRL_MSGS_SEND_TIME_STAT);
-GS *
-  p_CLUSTER_CTRL_MSGS_RECV_TIME_STAT = RecGetGlobalRawStatPtr(cluster_rsb, CLUSTER_CTRL_MSGS_RECV_TIME_STAT);
-GS *
-  p_CLUSTER_READ_BYTES_STAT = RecGetGlobalRawStatPtr(cluster_rsb, CLUSTER_READ_BYTES_STAT);
-GS *
-  p_CLUSTER_WRITE_BYTES_STAT = RecGetGlobalRawStatPtr(cluster_rsb, CLUSTER_WRITE_BYTES_STAT);
-GS *
-  p_CLUSTER_OP_DELAYED_FOR_LOCK_STAT = RecGetGlobalRawStatPtr(cluster_rsb, CLUSTER_OP_DELAYED_FOR_LOCK_STAT);
-GS *
-  p_CLUSTER_CONNECTIONS_LOCKED_STAT = RecGetGlobalRawStatPtr(cluster_rsb, CLUSTER_CONNECTIONS_LOCKED_STAT);
-GS *
-  p_CLUSTER_CONNECTIONS_BUMPED_STAT = RecGetGlobalRawStatPtr(cluster_rsb, CLUSTER_CONNECTIONS_BUMPED_STAT);
-GS *
-  p_CLUSTER_NODES_STAT = RecGetGlobalRawStatPtr(cluster_rsb, CLUSTER_NODES_STAT);
-GS *
-  p_CLUSTER_NET_BACKUP_STAT = RecGetGlobalRawStatPtr(cluster_rsb, CLUSTER_NET_BACKUP_STAT);
-GS *
-  p_machines_allocated_stat = RecGetGlobalRawStatPtr(cluster_rsb, CLUSTER_MACHINES_ALLOCATED_STAT);
-GS *
-  p_machines_freed_stat = RecGetGlobalRawStatPtr(cluster_rsb, CLUSTER_MACHINES_FREED_STAT);
-GS *
-  p_configuration_changes_stat = RecGetGlobalRawStatPtr(cluster_rsb, CLUSTER_CONFIGURATION_CHANGES_STAT);
-GS *
-  p_delayed_reads_stat = RecGetGlobalRawStatPtr(cluster_rsb, CLUSTER_DELAYED_READS_STAT);
-GS *
-  p_byte_bank_used_stat = RecGetGlobalRawStatPtr(cluster_rsb, CLUSTER_BYTE_BANK_USED_STAT);
-GS *
-  p_alloc_data_news_stat = RecGetGlobalRawStatPtr(cluster_rsb, CLUSTER_ALLOC_DATA_NEWS_STAT);
-GS *
-  p_write_bb_mallocs_stat = RecGetGlobalRawStatPtr(cluster_rsb, CLUSTER_WRITE_BB_MALLOCS_STAT);
-GS *
-  p_partial_reads_stat = RecGetGlobalRawStatPtr(cluster_rsb, CLUSTER_PARTIAL_READS_STAT);
-GS *
-  p_partial_writes_stat = RecGetGlobalRawStatPtr(cluster_rsb, CLUSTER_PARTIAL_WRITES_STAT);
-GS *
-  p_cache_outstanding = RecGetGlobalRawStatPtr(cluster_rsb, CLUSTER_CACHE_OUTSTANDING_STAT);
-GS *
-  p_remote_op_timeouts = RecGetGlobalRawStatPtr(cluster_rsb, CLUSTER_REMOTE_OP_TIMEOUTS_STAT);
-GS *
-  p_remote_op_reply_timeouts = RecGetGlobalRawStatPtr(cluster_rsb,
+GS *p_CLUSTER_CONNECTIONS_OPEN_STAT = RecGetGlobalRawStatPtr(cluster_rsb, CLUSTER_CONNECTIONS_OPEN_STAT);
+GS *p_CLUSTER_CONNECTIONS_OPENNED_STAT = RecGetGlobalRawStatPtr(cluster_rsb, CLUSTER_CONNECTIONS_OPENNED_STAT);
+GS *p_CLUSTER_CON_TOTAL_TIME_STAT = RecGetGlobalRawStatPtr(cluster_rsb, CLUSTER_CON_TOTAL_TIME_STAT);
+GS *p_CLUSTER_CTRL_MSGS_SENT_STAT = RecGetGlobalRawStatPtr(cluster_rsb, CLUSTER_CTRL_MSGS_SENT_STAT);
+GS *p_CLUSTER_SLOW_CTRL_MSGS_SENT_STAT = RecGetGlobalRawStatPtr(cluster_rsb, CLUSTER_SLOW_CTRL_MSGS_SENT_STAT);
+GS *p_CLUSTER_CTRL_MSGS_RECVD_STAT = RecGetGlobalRawStatPtr(cluster_rsb, CLUSTER_CTRL_MSGS_RECVD_STAT);
+GS *p_CLUSTER_SLOW_CTRL_MSGS_RECVD_STAT = RecGetGlobalRawStatPtr(cluster_rsb, CLUSTER_SLOW_CTRL_MSGS_RECVD_STAT);
+GS *p_CLUSTER_CTRL_MSGS_SEND_TIME_STAT = RecGetGlobalRawStatPtr(cluster_rsb, CLUSTER_CTRL_MSGS_SEND_TIME_STAT);
+GS *p_CLUSTER_CTRL_MSGS_RECV_TIME_STAT = RecGetGlobalRawStatPtr(cluster_rsb, CLUSTER_CTRL_MSGS_RECV_TIME_STAT);
+GS *p_CLUSTER_READ_BYTES_STAT = RecGetGlobalRawStatPtr(cluster_rsb, CLUSTER_READ_BYTES_STAT);
+GS *p_CLUSTER_WRITE_BYTES_STAT = RecGetGlobalRawStatPtr(cluster_rsb, CLUSTER_WRITE_BYTES_STAT);
+GS *p_CLUSTER_OP_DELAYED_FOR_LOCK_STAT = RecGetGlobalRawStatPtr(cluster_rsb, CLUSTER_OP_DELAYED_FOR_LOCK_STAT);
+GS *p_CLUSTER_CONNECTIONS_LOCKED_STAT = RecGetGlobalRawStatPtr(cluster_rsb, CLUSTER_CONNECTIONS_LOCKED_STAT);
+GS *p_CLUSTER_CONNECTIONS_BUMPED_STAT = RecGetGlobalRawStatPtr(cluster_rsb, CLUSTER_CONNECTIONS_BUMPED_STAT);
+GS *p_CLUSTER_NODES_STAT = RecGetGlobalRawStatPtr(cluster_rsb, CLUSTER_NODES_STAT);
+GS *p_CLUSTER_NET_BACKUP_STAT = RecGetGlobalRawStatPtr(cluster_rsb, CLUSTER_NET_BACKUP_STAT);
+GS *p_machines_allocated_stat = RecGetGlobalRawStatPtr(cluster_rsb, CLUSTER_MACHINES_ALLOCATED_STAT);
+GS *p_machines_freed_stat = RecGetGlobalRawStatPtr(cluster_rsb, CLUSTER_MACHINES_FREED_STAT);
+GS *p_configuration_changes_stat = RecGetGlobalRawStatPtr(cluster_rsb, CLUSTER_CONFIGURATION_CHANGES_STAT);
+GS *p_delayed_reads_stat = RecGetGlobalRawStatPtr(cluster_rsb, CLUSTER_DELAYED_READS_STAT);
+GS *p_byte_bank_used_stat = RecGetGlobalRawStatPtr(cluster_rsb, CLUSTER_BYTE_BANK_USED_STAT);
+GS *p_alloc_data_news_stat = RecGetGlobalRawStatPtr(cluster_rsb, CLUSTER_ALLOC_DATA_NEWS_STAT);
+GS *p_write_bb_mallocs_stat = RecGetGlobalRawStatPtr(cluster_rsb, CLUSTER_WRITE_BB_MALLOCS_STAT);
+GS *p_partial_reads_stat = RecGetGlobalRawStatPtr(cluster_rsb, CLUSTER_PARTIAL_READS_STAT);
+GS *p_partial_writes_stat = RecGetGlobalRawStatPtr(cluster_rsb, CLUSTER_PARTIAL_WRITES_STAT);
+GS *p_cache_outstanding = RecGetGlobalRawStatPtr(cluster_rsb, CLUSTER_CACHE_OUTSTANDING_STAT);
+GS *p_remote_op_timeouts = RecGetGlobalRawStatPtr(cluster_rsb, CLUSTER_REMOTE_OP_TIMEOUTS_STAT);
+GS *p_remote_op_reply_timeouts = RecGetGlobalRawStatPtr(cluster_rsb,
                                                       CLUSTER_REMOTE_OP_REPLY_TIMEOUTS_STAT);
-GS *
-  p_chan_inuse = RecGetGlobalRawStatPtr(cluster_rsb, CLUSTER_CHAN_INUSE_STAT);
-GS *
-  p_CLUSTER_OPEN_DELAYS_STAT = RecGetGlobalRawStatPtr(cluster_rsb, CLUSTER_OPEN_DELAYS_STAT);
-GS *
-  p_CLUSTER_OPEN_DELAY_TIME_STAT = RecGetGlobalRawStatPtr(cluster_rsb, CLUSTER_OPEN_DELAY_TIME_STAT);
-GS *
-  p_CLUSTER_CACHE_CALLBACKS_STAT = RecGetGlobalRawStatPtr(cluster_rsb, CLUSTER_CACHE_CALLBACKS_STAT);
-GS *
-  p_CLUSTER_CACHE_CALLBACK_TIME_STAT = RecGetGlobalRawStatPtr(cluster_rsb, CLUSTER_CACHE_CALLBACK_TIME_STAT);
-GS *
-  p_cluster_thread_steal_expires = RecGetGlobalRawStatPtr(cluster_rsb, CLUSTER_THREAD_STEAL_EXPIRES_STAT);
-GS *
-  p_CLUSTER_RDMSG_ASSEMBLE_TIME_STAT = RecGetGlobalRawStatPtr(cluster_rsb, CLUSTER_RDMSG_ASSEMBLE_TIME_STAT);
-GS *
-  p_cluster_ping_time = RecGetGlobalRawStatPtr(cluster_rsb, CLUSTER_PING_TIME_STAT);
-GS *
-  p_cluster_setdata_no_clustervc = RecGetGlobalRawStatPtr(cluster_rsb, CLUSTER_SETDATA_NO_CLUSTERVC_STAT);
-GS *
-  p_cluster_setdata_no_tunnel = RecGetGlobalRawStatPtr(cluster_rsb, CLUSTER_SETDATA_NO_TUNNEL_STAT);
-GS *
-  p_cluster_setdata_no_cachevc = RecGetGlobalRawStatPtr(cluster_rsb, CLUSTER_SETDATA_NO_CACHEVC_STAT);
-GS *
-  p_cluster_setdata_no_cluster = RecGetGlobalRawStatPtr(cluster_rsb, CLUSTER_SETDATA_NO_CLUSTER_STAT);
-GS *
-  p_cluster_vc_write_stall = RecGetGlobalRawStatPtr(cluster_rsb, CLUSTER_VC_WRITE_STALL_STAT);
-GS *
-  p_cluster_no_remote_space = RecGetGlobalRawStatPtr(cluster_rsb, CLUSTER_NO_REMOTE_SPACE_STAT);
-GS *
-  p_cluster_level1_bank = RecGetGlobalRawStatPtr(cluster_rsb, CLUSTER_LEVEL1_BANK_STAT);
-GS *
-  p_cluster_multilevel_bank = RecGetGlobalRawStatPtr(cluster_rsb, CLUSTER_MULTILEVEL_BANK_STAT);
-GS *
-  p_cluster_vc_cache_insert_lock_misses = RecGetGlobalRawStatPtr(cluster_rsb,
+GS *p_chan_inuse = RecGetGlobalRawStatPtr(cluster_rsb, CLUSTER_CHAN_INUSE_STAT);
+GS *p_CLUSTER_OPEN_DELAYS_STAT = RecGetGlobalRawStatPtr(cluster_rsb, CLUSTER_OPEN_DELAYS_STAT);
+GS *p_CLUSTER_OPEN_DELAY_TIME_STAT = RecGetGlobalRawStatPtr(cluster_rsb, CLUSTER_OPEN_DELAY_TIME_STAT);
+GS *p_CLUSTER_CACHE_CALLBACKS_STAT = RecGetGlobalRawStatPtr(cluster_rsb, CLUSTER_CACHE_CALLBACKS_STAT);
+GS *p_CLUSTER_CACHE_CALLBACK_TIME_STAT = RecGetGlobalRawStatPtr(cluster_rsb, CLUSTER_CACHE_CALLBACK_TIME_STAT);
+GS *p_cluster_thread_steal_expires = RecGetGlobalRawStatPtr(cluster_rsb, CLUSTER_THREAD_STEAL_EXPIRES_STAT);
+GS *p_CLUSTER_RDMSG_ASSEMBLE_TIME_STAT = RecGetGlobalRawStatPtr(cluster_rsb, CLUSTER_RDMSG_ASSEMBLE_TIME_STAT);
+GS *p_cluster_ping_time = RecGetGlobalRawStatPtr(cluster_rsb, CLUSTER_PING_TIME_STAT);
+GS *p_cluster_setdata_no_clustervc = RecGetGlobalRawStatPtr(cluster_rsb, CLUSTER_SETDATA_NO_CLUSTERVC_STAT);
+GS *p_cluster_setdata_no_tunnel = RecGetGlobalRawStatPtr(cluster_rsb, CLUSTER_SETDATA_NO_TUNNEL_STAT);
+GS *p_cluster_setdata_no_cachevc = RecGetGlobalRawStatPtr(cluster_rsb, CLUSTER_SETDATA_NO_CACHEVC_STAT);
+GS *p_cluster_setdata_no_cluster = RecGetGlobalRawStatPtr(cluster_rsb, CLUSTER_SETDATA_NO_CLUSTER_STAT);
+GS *p_cluster_vc_write_stall = RecGetGlobalRawStatPtr(cluster_rsb, CLUSTER_VC_WRITE_STALL_STAT);
+GS *p_cluster_no_remote_space = RecGetGlobalRawStatPtr(cluster_rsb, CLUSTER_NO_REMOTE_SPACE_STAT);
+GS *p_cluster_level1_bank = RecGetGlobalRawStatPtr(cluster_rsb, CLUSTER_LEVEL1_BANK_STAT);
+GS *p_cluster_multilevel_bank = RecGetGlobalRawStatPtr(cluster_rsb, CLUSTER_MULTILEVEL_BANK_STAT);
+GS *p_cluster_vc_cache_insert_lock_misses = RecGetGlobalRawStatPtr(cluster_rsb,
                                                                  CLUSTER_VC_CACHE_INSERT_LOCK_MISSES_STAT);
-GS *
-  p_cluster_vc_cache_inserts = RecGetGlobalRawStatPtr(cluster_rsb, CLUSTER_VC_CACHE_INSERTS_STAT);
-GS *
-  p_cluster_vc_cache_lookup_lock_misses = RecGetGlobalRawStatPtr(cluster_rsb,
+GS *p_cluster_vc_cache_inserts = RecGetGlobalRawStatPtr(cluster_rsb, CLUSTER_VC_CACHE_INSERTS_STAT);
+GS *p_cluster_vc_cache_lookup_lock_misses = RecGetGlobalRawStatPtr(cluster_rsb,
                                                                  CLUSTER_VC_CACHE_LOOKUP_LOCK_MISSES_STAT);
-GS *
-  p_cluster_vc_cache_lookup_hits = RecGetGlobalRawStatPtr(cluster_rsb, CLUSTER_VC_CACHE_LOOKUP_HITS_STAT);
-GS *
-  p_cluster_vc_cache_lookup_misses = RecGetGlobalRawStatPtr(cluster_rsb,
+GS *p_cluster_vc_cache_lookup_hits = RecGetGlobalRawStatPtr(cluster_rsb, CLUSTER_VC_CACHE_LOOKUP_HITS_STAT);
+GS *p_cluster_vc_cache_lookup_misses = RecGetGlobalRawStatPtr(cluster_rsb,
                                                             CLUSTER_VC_CACHE_LOOKUP_MISSES_STAT);
-GS *
-  p_cluster_vc_cache_scans = RecGetGlobalRawStatPtr(cluster_rsb, CLUSTER_VC_CACHE_SCANS_STAT);
-GS *
-  p_cluster_vc_cache_scan_lock_misses = RecGetGlobalRawStatPtr(cluster_rsb,
+GS *p_cluster_vc_cache_scans = RecGetGlobalRawStatPtr(cluster_rsb, CLUSTER_VC_CACHE_SCANS_STAT);
+GS *p_cluster_vc_cache_scan_lock_misses = RecGetGlobalRawStatPtr(cluster_rsb,
                                                                CLUSTER_VC_CACHE_SCAN_LOCK_MISSES_STAT);
-GS *
-  p_cluster_vc_cache_purges = RecGetGlobalRawStatPtr(cluster_rsb, CLUSTER_VC_CACHE_PURGES_STAT);
-GS *
-  p_cluster_write_lock_misses = RecGetGlobalRawStatPtr(cluster_rsb, CLUSTER_WRITE_LOCK_MISSES_STAT);
+GS *p_cluster_vc_cache_purges = RecGetGlobalRawStatPtr(cluster_rsb, CLUSTER_VC_CACHE_PURGES_STAT);
+GS *p_cluster_write_lock_misses = RecGetGlobalRawStatPtr(cluster_rsb, CLUSTER_WRITE_LOCK_MISSES_STAT);
 #endif
 ////////////////////////////////////////////////////////////////////////////
 
@@ -485,33 +432,21 @@
   PeriodicClusterEvent;
 
 #ifdef CLUSTER_TOMCAT
-extern int
-  CacheClusteringEnabled;
+extern int cache_clustering_enabled;
 
-int
-  CacheClusterMonitorEnabled = 0;
-int
-  CacheClusterMonitorIntervalSecs = 1;
+int CacheClusterMonitorEnabled = 0;
+int CacheClusterMonitorIntervalSecs = 1;
 
-int
-  cluster_send_buffer_size = 0;
-int
-  cluster_receive_buffer_size = 0;
-unsigned long
-  cluster_sockopt_flags = 0;
+int cluster_send_buffer_size = 0;
+int cluster_receive_buffer_size = 0;
+unsigned long cluster_sockopt_flags = 0;
 
-int
-  RPC_only_CacheCluster = 0;
+int RPC_only_CacheCluster = 0;
 #endif
 
 int
 ClusterProcessor::init()
 {
-  // This isn't used. /leif
-  //EThread *thread = this_ethread();
-  //ProxyMutex *mutex = thread->mutex;
-
-
   cluster_rsb = RecAllocateRawStatBlock((int) cluster_stat_count);
   //
   // Statistics callbacks
@@ -886,11 +821,11 @@
   memset(channel_dummy_output, 0, sizeof(channel_dummy_output));
 
   if (cluster_type == 1) {
-    CacheClusteringEnabled = 1;
+    cache_clustering_enabled = 1;
     Note("cache clustering enabled");
     compute_cluster_mode();
   } else {
-    CacheClusteringEnabled = 0;
+    cache_clustering_enabled = 0;
     Note("cache clustering disabled");
   }
   return 0;
@@ -909,7 +844,7 @@
 #ifdef LOCAL_CLUSTER_TEST_MODE
   this_cluster_machine()->cluster_port = cluster_port;
 #endif
-  if (CacheClusteringEnabled && (cacheProcessor.IsCacheEnabled() == CACHE_INITIALIZED)) {
+  if (cache_clustering_enabled && (cacheProcessor.IsCacheEnabled() == CACHE_INITIALIZED)) {
 
     ET_CLUSTER = eventProcessor.spawn_event_threads(1);
     for (int i = 0; i < eventProcessor.n_threads_for_type[ET_CLUSTER]; i++) {
@@ -1003,13 +938,13 @@
 ClusterProcessor::compute_cluster_mode()
 {
   if (RPC_only_CacheCluster) {
-    if (CacheClusteringEnabled > 0) {
-      CacheClusteringEnabled = -1;
+    if (cache_clustering_enabled > 0) {
+      cache_clustering_enabled = -1;
       Note("RPC only cache clustering");
     }
   } else {
-    if (CacheClusteringEnabled < 0) {
-      CacheClusteringEnabled = 1;
+    if (cache_clustering_enabled < 0) {
+      cache_clustering_enabled = 1;
       Note("RPC only cache clustering disabled");
     }
   }

Modified: incubator/trafficserver/traffic/branches/dev/iocore/cluster/ClusterVConnection.cc
URL: http://svn.apache.org/viewvc/incubator/trafficserver/traffic/branches/dev/iocore/cluster/ClusterVConnection.cc?rev=892310&r1=892309&r2=892310&view=diff
==============================================================================
--- incubator/trafficserver/traffic/branches/dev/iocore/cluster/ClusterVConnection.cc (original)
+++ incubator/trafficserver/traffic/branches/dev/iocore/cluster/ClusterVConnection.cc Fri Dec 18 16:47:37 2009
@@ -87,6 +87,27 @@
 }
 
 VIO *
+ClusterVConnectionBase::do_io_pread(Continuation * acont, ink64 anbytes, MIOBuffer * abuffer, ink_off_t off)
+{
+  ink_assert(!"implemented");
+  return 0;
+}
+
+int
+ClusterVConnection::get_header(void **ptr, int *len) 
+{
+  ink_assert(!"implemented");
+  return -1;
+}
+
+int
+ClusterVConnection::set_header(void *ptr, int len) 
+{
+  ink_assert(!"implemented");
+  return -1;
+}
+
+VIO *
 ClusterVConnectionBase::do_io_write(Continuation * acont, int anbytes, IOBufferReader * abuffer, bool owner)
 {
   ink_assert(!closed);

Modified: incubator/trafficserver/traffic/branches/dev/iocore/cluster/P_ClusterCache.h
URL: http://svn.apache.org/viewvc/incubator/trafficserver/traffic/branches/dev/iocore/cluster/P_ClusterCache.h?rev=892310&r1=892309&r2=892310&view=diff
==============================================================================
--- incubator/trafficserver/traffic/branches/dev/iocore/cluster/P_ClusterCache.h (original)
+++ incubator/trafficserver/traffic/branches/dev/iocore/cluster/P_ClusterCache.h Fri Dec 18 16:47:37 2009
@@ -379,6 +379,7 @@
     ink_assert(!"shutdown of cluster connection");
   }
   virtual void do_io_close(int lerrno = -1);
+  virtual VIO* do_io_pread(Continuation*, ink64, MIOBuffer*, ink_off_t);
 
   // Set the timeouts associated with this connection.
   // active_timeout is for the total elasped time of the connection.
@@ -530,8 +531,8 @@
   // 0 on success -1 on failure
   int start(EThread * t);       // New connect protocol
 
-    ClusterVConnection(int is_new_connect_read = 0);
-   ~ClusterVConnection();
+  ClusterVConnection(int is_new_connect_read = 0);
+  ~ClusterVConnection();
   void free();                  // Destructor actions (we are using ClassAllocator)
 
   virtual void do_io_close(int lerrno = -1);
@@ -564,28 +565,28 @@
   void set_type(int);
   ink_hrtime start_time;
   ink_hrtime last_activity_time;
-    Queue<ByteBankDescriptor> byte_bank_q;   // done awaiting completion
+  Queue<ByteBankDescriptor> byte_bank_q;   // done awaiting completion
   int n_set_data_msgs;          // # pending set_data() msgs on VC
   int n_recv_set_data_msgs;     // # set_data() msgs received on VC
   volatile int pending_remote_fill;     // Remote fill pending on connection
-    Ptr<IOBufferBlock> read_block;   // Hold current data for open read
+  Ptr<IOBufferBlock> read_block;   // Hold current data for open read
   bool have_all_data;           // All data in read_block
   int initial_data_bytes;       // bytes in open_read buffer
-    Ptr<IOBufferBlock> remote_write_block;   // Write side data for remote fill
+  Ptr<IOBufferBlock> remote_write_block;   // Write side data for remote fill
   void *current_cont;           // Track current continuation (debug)
 
 #define CLUSTER_IOV_NOT_OPEN               -2
 #define CLUSTER_IOV_NONE                   -1
   int iov_map;                  // which iov?
 
-    Ptr<ProxyMutex> read_locked;
-    Ptr<ProxyMutex> write_locked;
+  Ptr<ProxyMutex> read_locked;
+  Ptr<ProxyMutex> write_locked;
 
   // Data buffer for unmarshaled objects from remote node.
-    Ptr<IOBufferData> marshal_buf;
+  Ptr<IOBufferData> marshal_buf;
 
   // Pending write data
-    Ptr<IOBufferBlock> write_list;
+  Ptr<IOBufferBlock> write_list;
   IOBufferBlock *write_list_tail;
   int write_list_bytes;
   int write_bytes_in_transit;
@@ -614,6 +615,8 @@
   {
     return 0;
   }
+  virtual int get_header(void **ptr, int *len);
+  virtual int set_header(void *ptr, int len);
 };
 
 //

Modified: incubator/trafficserver/traffic/branches/dev/iocore/cluster/P_ClusterCacheInternal.h
URL: http://svn.apache.org/viewvc/incubator/trafficserver/traffic/branches/dev/iocore/cluster/P_ClusterCacheInternal.h?rev=892310&r1=892309&r2=892310&view=diff
==============================================================================
--- incubator/trafficserver/traffic/branches/dev/iocore/cluster/P_ClusterCacheInternal.h (original)
+++ incubator/trafficserver/traffic/branches/dev/iocore/cluster/P_ClusterCacheInternal.h Fri Dec 18 16:47:37 2009
@@ -218,7 +218,7 @@
     ink_assert(rw_buf_msg == 0);
     ink_assert(rw_buf_msg_len);
     if (rw_buf_msg_len <= DEFAULT_MAX_BUFFER_SIZE) {
-      rw_buf_msg = new_IOBufferData(buffer_size_to_index(rw_buf_msg_len));
+      rw_buf_msg = new_IOBufferData(buffer_size_to_index(rw_buf_msg_len, MAX_BUFFER_SIZE_INDEX));
     } else {
       rw_buf_msg = new_xmalloc_IOBufferData(xmalloc(rw_buf_msg_len), rw_buf_msg_len);
     }

Modified: incubator/trafficserver/traffic/branches/dev/iocore/eventsystem/I_EThread.h
URL: http://svn.apache.org/viewvc/incubator/trafficserver/traffic/branches/dev/iocore/eventsystem/I_EThread.h?rev=892310&r1=892309&r2=892310&view=diff
==============================================================================
--- incubator/trafficserver/traffic/branches/dev/iocore/eventsystem/I_EThread.h (original)
+++ incubator/trafficserver/traffic/branches/dev/iocore/eventsystem/I_EThread.h Fri Dec 18 16:47:37 2009
@@ -277,7 +277,7 @@
   ProxyAllocator evacuationBlockAllocator;
   ProxyAllocator ioDataAllocator;
   ProxyAllocator ioBlockAllocator;
-  ProxyAllocator ioBufAllocator[9];
+  ProxyAllocator ioBufAllocator[DEFAULT_BUFFER_SIZES];
 
 private:
 

Modified: incubator/trafficserver/traffic/branches/dev/iocore/eventsystem/I_IOBuffer.h
URL: http://svn.apache.org/viewvc/incubator/trafficserver/traffic/branches/dev/iocore/eventsystem/I_IOBuffer.h?rev=892310&r1=892309&r2=892310&view=diff
==============================================================================
--- incubator/trafficserver/traffic/branches/dev/iocore/eventsystem/I_IOBuffer.h (original)
+++ incubator/trafficserver/traffic/branches/dev/iocore/eventsystem/I_IOBuffer.h Fri Dec 18 16:47:37 2009
@@ -92,17 +92,18 @@
 #define BUFFER_SIZE_INDEX_8K            6
 #define BUFFER_SIZE_INDEX_16K           7
 #define BUFFER_SIZE_INDEX_32K           8
+#define BUFFER_SIZE_INDEX_64K           9
+#define MAX_BUFFER_SIZE_INDEX           9
+#define DEFAULT_BUFFER_SIZES            (MAX_BUFFER_SIZE_INDEX+1)
 
 #define BUFFER_SIZE_FOR_INDEX(_i)    (DEFAULT_BUFFER_BASE_SIZE * (1 << (_i)))
 #define DEFAULT_SMALL_BUFFER_SIZE    BUFFER_SIZE_INDEX_512
 #define DEFAULT_LARGE_BUFFER_SIZE    BUFFER_SIZE_INDEX_4K
 #define DEFAULT_TS_BUFFER_SIZE       BUFFER_SIZE_INDEX_8K
-#define DEFAULT_MAX_BUFFER_SIZE      \
-BUFFER_SIZE_FOR_INDEX(DEFAULT_BUFFER_SIZES-1)
+#define DEFAULT_MAX_BUFFER_SIZE      BUFFER_SIZE_FOR_INDEX(MAX_BUFFER_SIZE_INDEX)
 #define MIN_IOBUFFER_SIZE            BUFFER_SIZE_INDEX_128
 #define MAX_IOBUFFER_SIZE            (DEFAULT_BUFFER_SIZES-1)
 
-#define DEFAULT_BUFFER_SIZES         (BUFFER_SIZE_INDEX_32K+1)
 
 #define BUFFER_SIZE_ALLOCATED(_i)     \
   (BUFFER_SIZE_INDEX_IS_FAST_ALLOCATED(_i) || \

Modified: incubator/trafficserver/traffic/branches/dev/iocore/eventsystem/I_VConnection.h
URL: http://svn.apache.org/viewvc/incubator/trafficserver/traffic/branches/dev/iocore/eventsystem/I_VConnection.h?rev=892310&r1=892309&r2=892310&view=diff
==============================================================================
--- incubator/trafficserver/traffic/branches/dev/iocore/eventsystem/I_VConnection.h (original)
+++ incubator/trafficserver/traffic/branches/dev/iocore/eventsystem/I_VConnection.h Fri Dec 18 16:47:37 2009
@@ -366,40 +366,6 @@
     return NULL;
   }
 
-  /** Cache and cluster specific. */
-  virtual int get_object_size()
-  {
-    ink_debug_assert(!"get_object_size not implemented");
-    return 0;
-  }
-  virtual bool set_pin_in_cache(time_t time_pin)
-  {
-    (void) time_pin;
-    ink_debug_assert(!"set_pin_in_cache not implemented");
-    return false;
-  }
-  virtual time_t get_pin_in_cache()
-  {
-    ink_debug_assert(!"get_pin_in_cache not implemented");
-    return (time_t) 0;
-  }
-  virtual bool set_disk_io_priority(int priority)
-  {
-    (void) priority;
-    ink_debug_assert(!"set_disk_io_priority not implemented");
-    return false;
-  }
-  virtual int get_disk_io_priority()
-  {
-    ink_debug_assert(!"get_disk_io_priority not implemented");
-    return 0;
-  }
-  virtual bool is_ram_cache_hit()
-  {
-    ink_debug_assert(!"is_ram_cache_hit not implemented");
-    return 0;
-  }
-
 public:
 
   /**

Modified: incubator/trafficserver/traffic/branches/dev/iocore/eventsystem/P_IOBuffer.h
URL: http://svn.apache.org/viewvc/incubator/trafficserver/traffic/branches/dev/iocore/eventsystem/P_IOBuffer.h?rev=892310&r1=892309&r2=892310&view=diff
==============================================================================
--- incubator/trafficserver/traffic/branches/dev/iocore/eventsystem/P_IOBuffer.h (original)
+++ incubator/trafficserver/traffic/branches/dev/iocore/eventsystem/P_IOBuffer.h Fri Dec 18 16:47:37 2009
@@ -45,7 +45,7 @@
 INK_INLINE int
 iobuffer_size_to_index(unsigned int size, int max)
 {
-  if (size > DEFAULT_MAX_BUFFER_SIZE) {
+  if (size > (unsigned int)BUFFER_SIZE_FOR_INDEX(max)) {
     ////////////////////////////////////////////////////
     // BUFFER_SIZE_INDEX_FOR_XMALLOC_SIZE returns the //
     // negative of its argument. The argument must be //

Modified: incubator/trafficserver/traffic/branches/dev/iocore/net/P_UnixNetVConnection.h
URL: http://svn.apache.org/viewvc/incubator/trafficserver/traffic/branches/dev/iocore/net/P_UnixNetVConnection.h?rev=892310&r1=892309&r2=892310&view=diff
==============================================================================
--- incubator/trafficserver/traffic/branches/dev/iocore/net/P_UnixNetVConnection.h (original)
+++ incubator/trafficserver/traffic/branches/dev/iocore/net/P_UnixNetVConnection.h Fri Dec 18 16:47:37 2009
@@ -187,10 +187,6 @@
   void readReschedule(NetHandler * nh);
   void netActivity(EThread * lthread);
 
-#ifdef DEBUG
-  // Class static data
-  static int enable_debug_trace;
-#endif
   Action *action()
   {
     return &action_;

Modified: incubator/trafficserver/traffic/branches/dev/iocore/net/UnixNetVConnection.cc
URL: http://svn.apache.org/viewvc/incubator/trafficserver/traffic/branches/dev/iocore/net/UnixNetVConnection.cc?rev=892310&r1=892309&r2=892310&view=diff
==============================================================================
--- incubator/trafficserver/traffic/branches/dev/iocore/net/UnixNetVConnection.cc (original)
+++ incubator/trafficserver/traffic/branches/dev/iocore/net/UnixNetVConnection.cc Fri Dec 18 16:47:37 2009
@@ -38,12 +38,6 @@
 #define NET_MAX_IOV UIO_MAXIOV
 #endif
 
-#ifdef DEBUG
-// Initialize class UnixNetVConnection static data
-int
-  UnixNetVConnection::enable_debug_trace = 0;
-#endif
-
 // Global
 ClassAllocator<UnixNetVConnection> netVCAllocator("netVCAllocator");
 
@@ -852,14 +846,6 @@
   ink_debug_assert(vio->mutex->thread_holding == this_ethread());
   ink_assert(!closed);
   STATE_FROM_VIO(vio)->enabled = 1;
-#ifdef DEBUG
-  if (vio == &read.vio) {
-    if (enable_debug_trace && (vio->buffer.mbuf && !vio->buffer.writer()->write_avail()));
-  } else {
-    ink_assert(vio == &write.vio);
-    if (enable_debug_trace && (vio->buffer.mbuf && !vio->buffer.reader()->read_avail()));
-  }
-#endif
 #ifdef INACTIVITY_TIMEOUT
   if (!inactivity_timeout && inactivity_timeout_in)
     inactivity_timeout = vio->mutex->thread_holding->schedule_in_local(this, inactivity_timeout_in);

Modified: incubator/trafficserver/traffic/branches/dev/libinktomi++/ink_port.h
URL: http://svn.apache.org/viewvc/incubator/trafficserver/traffic/branches/dev/libinktomi%2B%2B/ink_port.h?rev=892310&r1=892309&r2=892310&view=diff
==============================================================================
--- incubator/trafficserver/traffic/branches/dev/libinktomi++/ink_port.h (original)
+++ incubator/trafficserver/traffic/branches/dev/libinktomi++/ink_port.h Fri Dec 18 16:47:37 2009
@@ -45,6 +45,10 @@
 typedef unsigned long long inku64;
 typedef off_t ink_off_t;
 
+#define INKU64_MAX 18446744073709551615ULL
+#define INK64_MAX 9223372036854775807LL
+#define INK64_MIN (-INK64_MAX -1LL)
+
 /*******************************************************************
  ** x86
   ******************************************************************/

Modified: incubator/trafficserver/traffic/branches/dev/libinktomi++/ink_queue.cc
URL: http://svn.apache.org/viewvc/incubator/trafficserver/traffic/branches/dev/libinktomi%2B%2B/ink_queue.cc?rev=892310&r1=892309&r2=892310&view=diff
==============================================================================
--- incubator/trafficserver/traffic/branches/dev/libinktomi++/ink_queue.cc (original)
+++ incubator/trafficserver/traffic/branches/dev/libinktomi++/ink_queue.cc Fri Dec 18 16:47:37 2009
@@ -76,7 +76,7 @@
 #define DEADBEEF
 #endif
 
-/* #define MEMPROTECT */
+// #define MEMPROTECT 1
 
 #define MEMPROTECT_SIZE  0x200
 
@@ -167,7 +167,6 @@
 #endif                          /* !INK_USE_MUTEX_FOR_FREELISTS */
 {                               //static inku32 cntf = 0;
 
-
 #if (defined(USE_SPINLOCK_FOR_FREELIST) || defined(CHECK_FOR_DOUBLE_FREE))
   void *foo;
   inku32 type_size = f->type_size;
@@ -321,10 +320,6 @@
     } else {
       SET_FREELIST_POINTER_VERSION(next, *ADDRESS_OF_NEXT(TO_PTR(FREELIST_POINTER(item)), f->offset),
                                    FREELIST_VERSION(item) + 1);
-#ifdef SANITY
-      if (item.s.pointer == TO_PTR(next.s.pointer))
-        ink_fatal(1, "ink_freelist_new: loop detected");
-#endif /* SANITY */
 #if !defined(INK_USE_MUTEX_FOR_FREELISTS)
       result = ink_atomic_cas64((ink64 *) & f->head.data, item.data, next.data);
 #else
@@ -334,7 +329,9 @@
 
 #ifdef SANITY
       if (result) {
-        if (((uintptr_t) (TO_PTR(next.s.pointer))) & 3)
+        if (FREELIST_POINTER(item) == TO_PTR(FREELIST_POINTER(next)))
+          ink_fatal(1, "ink_freelist_new: loop detected");
+        if (((uintptr_t) (TO_PTR(FREELIST_POINTER(next)))) & 3)
           ink_fatal(1, "ink_freelist_new: bad list");
         if (TO_PTR(FREELIST_POINTER(next)))
           fake_global_for_ink_queue = *(int *) TO_PTR(FREELIST_POINTER(next));
@@ -434,11 +431,7 @@
 #ifdef SANITY
     if (TO_PTR(FREELIST_POINTER(h)) == item)
       ink_fatal(1, "ink_freelist_free: trying to free item twice");
-#ifdef __alpha
-    if (((uintptr_t) (TO_PTR(h.data))) & 3)
-#else
-    if (((uintptr_t) (TO_PTR(h.s.pointer))) & 3)
-#endif
+    if (((uintptr_t) (TO_PTR(FREELIST_POINTER(h)))) & 3)
       ink_fatal(1, "ink_freelist_free: bad list");
     if (TO_PTR(FREELIST_POINTER(h)))
       fake_global_for_ink_queue = *(int *) TO_PTR(FREELIST_POINTER(h));

Modified: incubator/trafficserver/traffic/branches/dev/libinktomi++/ink_queue.h
URL: http://svn.apache.org/viewvc/incubator/trafficserver/traffic/branches/dev/libinktomi%2B%2B/ink_queue.h?rev=892310&r1=892309&r2=892310&view=diff
==============================================================================
--- incubator/trafficserver/traffic/branches/dev/libinktomi++/ink_queue.h (original)
+++ incubator/trafficserver/traffic/branches/dev/libinktomi++/ink_queue.h Fri Dec 18 16:47:37 2009
@@ -71,11 +71,13 @@
 
   typedef union
   {
+#if defined(__i386__)
     struct
     {
       void *pointer;
       ink32 version;
     } s;
+#endif
     ink64 data;
   } head_p;
 

Modified: incubator/trafficserver/traffic/branches/dev/libinktomi++/ink_queue_utils.c
URL: http://svn.apache.org/viewvc/incubator/trafficserver/traffic/branches/dev/libinktomi%2B%2B/ink_queue_utils.c?rev=892310&r1=892309&r2=892310&view=diff
==============================================================================
--- incubator/trafficserver/traffic/branches/dev/libinktomi++/ink_queue_utils.c (original)
+++ incubator/trafficserver/traffic/branches/dev/libinktomi++/ink_queue_utils.c Fri Dec 18 16:47:37 2009
@@ -68,9 +68,13 @@
 void
 ink_queue_load_64(void *dst, void *src)
 {
+#if defined(__i386__)
   ink32 src_version = (*(head_p *) src).s.version;
   void *src_pointer = (*(head_p *) src).s.pointer;
 
   (*(head_p *) dst).s.version = src_version;
   (*(head_p *) dst).s.pointer = src_pointer;
+#else
+  *(void**)dst = *(void**)src;
+#endif
 }

Modified: incubator/trafficserver/traffic/branches/dev/proxy/ICP.cc
URL: http://svn.apache.org/viewvc/incubator/trafficserver/traffic/branches/dev/proxy/ICP.cc?rev=892310&r1=892309&r2=892310&view=diff
==============================================================================
--- incubator/trafficserver/traffic/branches/dev/proxy/ICP.cc (original)
+++ incubator/trafficserver/traffic/branches/dev/proxy/ICP.cc Fri Dec 18 16:47:37 2009
@@ -1928,7 +1928,7 @@
   // Setup custom allocators
   //
   // replaced with generic IOBufferBlock allocator
-  ICPHandlerCont::ICPDataBuf_IOBuffer_sizeindex = iobuffer_size_to_index(MAX_ICP_MSGSIZE);
+  ICPHandlerCont::ICPDataBuf_IOBuffer_sizeindex = iobuffer_size_to_index(MAX_ICP_MSGSIZE, MAX_BUFFER_SIZE_INDEX);
 
   //
   // Setup ICP stats callbacks

Modified: incubator/trafficserver/traffic/branches/dev/proxy/Main.cc
URL: http://svn.apache.org/viewvc/incubator/trafficserver/traffic/branches/dev/proxy/Main.cc?rev=892310&r1=892309&r2=892310&view=diff
==============================================================================
--- incubator/trafficserver/traffic/branches/dev/proxy/Main.cc (original)
+++ incubator/trafficserver/traffic/branches/dev/proxy/Main.cc Fri Dec 18 16:47:37 2009
@@ -159,7 +159,7 @@
 int lock_process = DEFAULT_LOCK_PROCESS;
 extern int fds_limit;
 extern int cluster_port_number;
-extern int CacheClusteringEnabled;
+extern int cache_clustering_enabled;
 char cluster_host[DOMAIN_NAME_MAX + 1] = DEFAULT_CLUSTER_HOST;
 
 //         = DEFAULT_CLUSTER_PORT_NUMBER;
@@ -1708,10 +1708,10 @@
   RecGetRecordInt("proxy.local.cluster.type", &temp_int);
   cluster_type = (int) temp_int;
   if (cluster_type == 1) {
-    CacheClusteringEnabled = 1;
+    cache_clustering_enabled = 1;
     Note("cache clustering enabled");
   } else {
-    CacheClusteringEnabled = 0;
+    cache_clustering_enabled = 0;
     /* 3com does not want these messages to be seen */
     Note("cache clustering disabled");