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 2011/05/11 08:20:58 UTC

svn commit: r1101758 - in /trafficserver/traffic/trunk/iocore/cache: Cache.cc CacheDisk.cc CacheHosting.cc P_CacheDisk.h P_CacheInternal.h

Author: jplevyak
Date: Wed May 11 06:20:57 2011
New Revision: 1101758

URL: http://svn.apache.org/viewvc?rev=1101758&view=rev
Log:
Fix (at least partial) for the 512GB limit bug.  We need to rationalize the off_t
vs int64_t vs uint64_t issue. 

NOTE: this changes the disk header format which will result in a cache *WIPE*.  This is
unfortunately unavoidable.

Modified:
    trafficserver/traffic/trunk/iocore/cache/Cache.cc
    trafficserver/traffic/trunk/iocore/cache/CacheDisk.cc
    trafficserver/traffic/trunk/iocore/cache/CacheHosting.cc
    trafficserver/traffic/trunk/iocore/cache/P_CacheDisk.h
    trafficserver/traffic/trunk/iocore/cache/P_CacheInternal.h

Modified: trafficserver/traffic/trunk/iocore/cache/Cache.cc
URL: http://svn.apache.org/viewvc/trafficserver/traffic/trunk/iocore/cache/Cache.cc?rev=1101758&r1=1101757&r2=1101758&view=diff
==============================================================================
--- trafficserver/traffic/trunk/iocore/cache/Cache.cc (original)
+++ trafficserver/traffic/trunk/iocore/cache/Cache.cc Wed May 11 06:20:57 2011
@@ -146,7 +146,7 @@ struct VolInitInfo
 void cplist_init();
 static void cplist_update();
 int cplist_reconfigure();
-static int create_volume(int volume_number, int size_in_blocks, int scheme, CacheVol *cp);
+static int create_volume(int volume_number, off_t size_in_blocks, int scheme, CacheVol *cp);
 static void rebuild_host_table(Cache *cache);
 void register_cache_stats(RecRawStatBlock *rsb, const char *prefix);
 
@@ -2174,16 +2174,15 @@ CacheVConnection::CacheVConnection()
   : VConnection(NULL)
 { }
 
+
 void
 cplist_init()
 {
-  int i;
-  unsigned int j;
   cp_list_len = 0;
-  for (i = 0; i < gndisks; i++) {
+  for (int i = 0; i < gndisks; i++) {
     CacheDisk *d = gdisks[i];
     DiskVol **dp = d->disk_vols;
-    for (j = 0; j < d->header->num_volumes; j++) {
+    for (unsigned int j = 0; j < d->header->num_volumes; j++) {
       ink_assert(dp[j]->dpb_queue.head);
       CacheVol *p = cp_list.head;
       while (p) {
@@ -2263,10 +2262,9 @@ cplist_update()
 int
 cplist_reconfigure()
 {
-  int i, j;
-  int size;
+  int64_t size;
   int volume_number;
-  int size_in_blocks;
+  off_t size_in_blocks;
 
   gnvol = 0;
   if (config_volumes.num_volumes == 0) {
@@ -2278,7 +2276,7 @@ cplist_reconfigure()
     memset(cp->disk_vols, 0, gndisks * sizeof(DiskVol *));
     cp_list.enqueue(cp);
     cp_list_len++;
-    for (i = 0; i < gndisks; i++) {
+    for (int i = 0; i < gndisks; i++) {
       if (gdisks[i]->header->num_volumes != 1 || gdisks[i]->disk_vols[0]->vol_number != 0) {
         /* The user had created several volumes before - clear the disk
            and create one volume for http */
@@ -2292,7 +2290,7 @@ cplist_reconfigure()
           off_t b = gdisks[i]->free_space / (vols - p);
           Debug("cache_hosting", "blocks = %d\n", b);
           DiskVolBlock *dpb = gdisks[i]->create_volume(0, b, CACHE_HTTP_TYPE);
-          ink_assert(dpb && dpb->len == b);
+          ink_assert(dpb && dpb->len == (uint64_t)b);
         }
         ink_assert(gdisks[i]->free_space == 0);
       }
@@ -2306,7 +2304,7 @@ cplist_reconfigure()
     }
 
   } else {
-    for (i = 0; i < gndisks; i++) {
+    for (int i = 0; i < gndisks; i++) {
       if (gdisks[i]->header->num_volumes == 1 && gdisks[i]->disk_vols[0]->vol_number == 0) {
         /* The user had created several volumes before - clear the disk
            and create one volume for http */
@@ -2316,11 +2314,11 @@ cplist_reconfigure()
     }
 
     /* change percentages in the config patitions to absolute value */
-    int64_t tot_space_in_blks = 0;
-    int blocks_per_vol = VOL_BLOCK_SIZE / STORE_BLOCK_SIZE;
+    off_t tot_space_in_blks = 0;
+    off_t blocks_per_vol = VOL_BLOCK_SIZE / STORE_BLOCK_SIZE;
     /* sum up the total space available on all the disks.
        round down the space to 128 megabytes */
-    for (i = 0; i < gndisks; i++)
+    for (int i = 0; i < gndisks; i++)
       tot_space_in_blks += (gdisks[i]->num_usable_blocks / blocks_per_vol) * blocks_per_vol;
 
     double percent_remaining = 100.00;
@@ -2332,7 +2330,7 @@ cplist_reconfigure()
           Warning("no volumes created");
           return -1;
         }
-        int space_in_blks = (int) (((double) (config_vol->percent / percent_remaining)) * tot_space_in_blks);
+        int64_t space_in_blks = (int64_t) (((double) (config_vol->percent / percent_remaining)) * tot_space_in_blks);
 
         space_in_blks = space_in_blks >> (20 - STORE_BLOCK_SHIFT);
         /* round down to 128 megabyte multiple */
@@ -2388,13 +2386,12 @@ cplist_reconfigure()
       /* search the cp_list */
 
       int *sorted_vols = new int[gndisks];
-      for (i = 0; i < gndisks; i++) {
+      for (int i = 0; i < gndisks; i++)
         sorted_vols[i] = i;
-      }
-      for (i = 0; i < gndisks - 1; i++) {
+      for (int i = 0; i < gndisks - 1; i++) {
         int smallest = sorted_vols[i];
         int smallest_ndx = i;
-        for (j = i + 1; j < gndisks; j++) {
+        for (int j = i + 1; j < gndisks; j++) {
           int curr = sorted_vols[j];
           DiskVol *dvol = cp->disk_vols[curr];
           if (gdisks[curr]->cleared) {
@@ -2415,9 +2412,9 @@ cplist_reconfigure()
         sorted_vols[i] = smallest;
       }
 
-      int size_to_alloc = size_in_blocks - cp->size;
+      int64_t size_to_alloc = size_in_blocks - cp->size;
       int disk_full = 0;
-      for (i = 0; (i < gndisks) && size_to_alloc; i++) {
+      for (int i = 0; (i < gndisks) && size_to_alloc; i++) {
 
         int disk_no = sorted_vols[i];
         ink_assert(cp->disk_vols[sorted_vols[gndisks - 1]]);
@@ -2427,7 +2424,7 @@ cplist_reconfigure()
            between the biggest volume on any disk and
            the volume on this disk and try to make
            them equal */
-        int size_diff = (cp->disk_vols[disk_no]) ? largest_vol - cp->disk_vols[disk_no]->size : largest_vol;
+        int64_t size_diff = (cp->disk_vols[disk_no]) ? largest_vol - cp->disk_vols[disk_no]->size : largest_vol;
         size_diff = (size_diff < size_to_alloc) ? size_diff : size_to_alloc;
         /* if size_diff == 0, then then the disks have volumes of the
            same sizes, so we don't need to balance the disks */
@@ -2452,15 +2449,13 @@ cplist_reconfigure()
           disk_full++;
 
         size_to_alloc = size_in_blocks - cp->size;
-
       }
 
       delete[]sorted_vols;
 
       if (size_to_alloc) {
-        if (create_volume(volume_number, size_to_alloc, cp->scheme, cp)) {
+        if (create_volume(volume_number, size_to_alloc, cp->scheme, cp))
           return -1;
-        }
       }
       gnvol += cp->num_vols;
     }
@@ -2468,12 +2463,13 @@ cplist_reconfigure()
   return 0;
 }
 
+// This is some really bad code, and needs to be rewritten!
 int
-create_volume(int volume_number, int size_in_blocks, int scheme, CacheVol *cp)
+create_volume(int volume_number, off_t size_in_blocks, int scheme, CacheVol *cp)
 {
-  static int curr_vol = 0;
-  int to_create = size_in_blocks;
-  int blocks_per_vol = VOL_BLOCK_SIZE >> STORE_BLOCK_SHIFT;
+  static int curr_vol = 0;  // FIXME: this will not reinitialize correctly
+  off_t to_create = size_in_blocks;
+  off_t blocks_per_vol = VOL_BLOCK_SIZE >> STORE_BLOCK_SHIFT;
   int full_disks = 0;
 
   int *sp = new int[gndisks];
@@ -2511,7 +2507,6 @@ create_volume(int volume_number, int siz
     if (sp[i] > 0) {
       while (sp[i] > 0) {
         DiskVolBlock *p = gdisks[i]->create_volume(volume_number, sp[i], scheme);
-
         ink_assert(p && (p->len >= (unsigned int) blocks_per_vol));
         sp[i] -= p->len;
         cp->num_vols++;

Modified: trafficserver/traffic/trunk/iocore/cache/CacheDisk.cc
URL: http://svn.apache.org/viewvc/trafficserver/traffic/trunk/iocore/cache/CacheDisk.cc?rev=1101758&r1=1101757&r2=1101758&view=diff
==============================================================================
--- trafficserver/traffic/trunk/iocore/cache/CacheDisk.cc (original)
+++ trafficserver/traffic/trunk/iocore/cache/CacheDisk.cc Wed May 11 06:20:57 2011
@@ -140,7 +140,7 @@ CacheDisk::openStart(int event, void *da
     return openDone(EVENT_IMMEDIATE, 0);
   }
 
-  if (header->magic != DISK_HEADER_MAGIC || header->num_blocks != len) {
+  if (header->magic != DISK_HEADER_MAGIC || header->num_blocks != (uint64_t)len) {
     Warning("disk header different for disk %s: clearing the disk", path);
     SET_HANDLER(&CacheDisk::clearDone);
     clearDisk();

Modified: trafficserver/traffic/trunk/iocore/cache/CacheHosting.cc
URL: http://svn.apache.org/viewvc/trafficserver/traffic/trunk/iocore/cache/CacheHosting.cc?rev=1101758&r1=1101757&r2=1101758&view=diff
==============================================================================
--- trafficserver/traffic/trunk/iocore/cache/CacheHosting.cc (original)
+++ trafficserver/traffic/trunk/iocore/cache/CacheHosting.cc Wed May 11 06:20:57 2011
@@ -963,9 +963,9 @@ create_config(RegressionTest * t, int nu
       }
 
       // calculate the total free space
-      uint64_t total_space = 0;
+      off_t total_space = 0;
       for (i = 0; i < gndisks; i++) {
-        int vol_blocks = gdisks[i]->num_usable_blocks;
+        off_t vol_blocks = gdisks[i]->num_usable_blocks;
         /* round down the blocks to the nearest
            multiple of STORE_BLOCKS_PER_VOL */
         vol_blocks = (vol_blocks / STORE_BLOCKS_PER_VOL)
@@ -1003,7 +1003,7 @@ create_config(RegressionTest * t, int nu
     {
       /* calculate the total disk space */
       InkRand *gen = &this_ethread()->generator;
-      uint64_t total_space = 0;
+      off_t total_space = 0;
       vol_num = 1;
       if (num == 2) {
         rprintf(t, "Random Volumes after clearing the disks\n");
@@ -1012,7 +1012,7 @@ create_config(RegressionTest * t, int nu
       }
 
       for (i = 0; i < gndisks; i++) {
-        int vol_blocks = gdisks[i]->num_usable_blocks;
+        off_t vol_blocks = gdisks[i]->num_usable_blocks;
         /* round down the blocks to the nearest
            multiple of STORE_BLOCKS_PER_VOL */
         vol_blocks = (vol_blocks / STORE_BLOCKS_PER_VOL)
@@ -1037,7 +1037,7 @@ create_config(RegressionTest * t, int nu
         /* convert to 128 megs multiple */
         int scheme = (random_size % 2) ? CACHE_HTTP_TYPE : CACHE_RTSP_TYPE;
         random_size = ROUND_TO_VOL_SIZE(random_size);
-        int blocks = random_size / STORE_BLOCK_SIZE;
+        off_t blocks = random_size / STORE_BLOCK_SIZE;
         ink_assert(blocks <= (int) total_space);
         total_space -= blocks;
 
@@ -1072,7 +1072,6 @@ create_config(RegressionTest * t, int nu
 int
 execute_and_verify(RegressionTest * t)
 {
-  int i;
   cplist_init();
   cplist_reconfigure();
 
@@ -1086,7 +1085,7 @@ execute_and_verify(RegressionTest * t)
   ConfigVol *cp = config_volumes.cp_queue.head;
   CacheVol *cachep;
 
-  for (i = 0; i < config_volumes.num_volumes; i++) {
+  for (int i = 0; i < config_volumes.num_volumes; i++) {
     cachep = cp_list.head;
     while (cachep) {
       if (cachep->vol_number == cp->number) {
@@ -1141,18 +1140,17 @@ execute_and_verify(RegressionTest * t)
 
   ClearCacheVolList(&cp_list, cp_list_len);
 
-  for (i = 0; i < gndisks; i++) {
+  for (int i = 0; i < gndisks; i++) {
     CacheDisk *d = gdisks[i];
     if (is_debug_tag_set("cache_hosting")) {
-      int j;
 
       Debug("cache_hosting", "Disk: %d: Vol Blocks: %ld: Free space: %ld",
             i, d->header->num_diskvol_blks, d->free_space);
-      for (j = 0; j < (int) d->header->num_volumes; j++) {
+      for (int j = 0; j < (int) d->header->num_volumes; j++) {
 
         Debug("cache_hosting", "\tVol: %d Size: %d", d->disk_vols[j]->vol_number, d->disk_vols[j]->size);
       }
-      for (j = 0; j < (int) d->header->num_diskvol_blks; j++) {
+      for (int j = 0; j < (int) d->header->num_diskvol_blks; j++) {
         Debug("cache_hosting", "\tBlock No: %d Size: %d Free: %d",
               d->header->vol_info[j].number, d->header->vol_info[j].len, d->header->vol_info[j].free);
       }

Modified: trafficserver/traffic/trunk/iocore/cache/P_CacheDisk.h
URL: http://svn.apache.org/viewvc/trafficserver/traffic/trunk/iocore/cache/P_CacheDisk.h?rev=1101758&r1=1101757&r2=1101758&view=diff
==============================================================================
--- trafficserver/traffic/trunk/iocore/cache/P_CacheDisk.h (original)
+++ trafficserver/traffic/trunk/iocore/cache/P_CacheDisk.h Wed May 11 06:20:57 2011
@@ -40,19 +40,18 @@ extern int cache_config_max_disk_errors;
 #define ROUND_DOWN_TO_STORE_BLOCK(_x)   (((_x) >> STORE_BLOCK_SHIFT) << STORE_BLOCK_SHIFT)
 
 #define STORE_BLOCKS_PER_VOL            (VOL_BLOCK_SIZE / STORE_BLOCK_SIZE)
-#define DISK_HEADER_MAGIC               0xABCD1236
+#define DISK_HEADER_MAGIC               0xABCD1237
 
 /* each disk vol block has a corresponding Vol object */
 struct CacheDisk;
 
 struct DiskVolBlock
 {
-  off_t offset;
-  unsigned short number;
-  unsigned int len:26;
+  uint64_t offset;  // offset in bytes from the start of the disk
+  uint64_t len;  // length in in store blocks
+  int number;
   unsigned int type:3;
   unsigned int free:1;
-  unsigned int unused:2;
 };
 
 struct DiskVolBlockQueue
@@ -70,7 +69,7 @@ struct DiskVol
 {
   int num_volblocks;           /* number of disk volume blocks in this volume */
   int vol_number;              /* the volume number of this volume */
-  off_t size;                  /* size in store blocks */
+  uint64_t size;                  /* size in store blocks */
   CacheDisk *disk;
   Queue<DiskVolBlockQueue> dpb_queue;
 };
@@ -82,7 +81,7 @@ struct DiskHeader
   unsigned int num_free;               /* number of disk volume blocks free */
   unsigned int num_used;               /* number of disk volume blocks in use */
   unsigned int num_diskvol_blks;       /* number of disk volume blocks */
-  off_t num_blocks;
+  uint64_t num_blocks;
   DiskVolBlock vol_info[1];
 };
 
@@ -95,7 +94,7 @@ struct CacheDisk: public Continuation
   off_t len;                // in blocks (STORE_BLOCK)
   off_t start;
   off_t skip;
-  int num_usable_blocks;
+  off_t num_usable_blocks;
   int hw_sector_size;
   int fd;
   off_t free_space;

Modified: trafficserver/traffic/trunk/iocore/cache/P_CacheInternal.h
URL: http://svn.apache.org/viewvc/trafficserver/traffic/trunk/iocore/cache/P_CacheInternal.h?rev=1101758&r1=1101757&r2=1101758&view=diff
==============================================================================
--- trafficserver/traffic/trunk/iocore/cache/P_CacheInternal.h (original)
+++ trafficserver/traffic/trunk/iocore/cache/P_CacheInternal.h Wed May 11 06:20:57 2011
@@ -948,7 +948,7 @@ struct Cache
 {
   volatile int cache_read_done;
   volatile int total_good_nvol;
-  int total_nvol;
+  volatile int total_nvol;
   volatile int ready;
   int64_t cache_size;             //in store block size
   CacheHostTable *hosttable;