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 2014/09/15 05:43:56 UTC

git commit: TS-3070: consistent span configuration across platforms

Repository: trafficserver
Updated Branches:
  refs/heads/master 25407ce75 -> 4747fe8f7


TS-3070: consistent span configuration across platforms

  - Remove unused Span::isRaw.
  - Add ink_file_is_mmappable().
  - Add ink_file_get_geometry().
  - Make Span geometry variables unsigned.
  - Use the symbolic constant for the Linux raw device major number.
  - Allow file spans to omit the size configuration parameter.
  - Use a scoped file descriptor to read Span configuration.
  - Consolidate span configuration so that it is the same for all
    platforms. Use ink_file_get_geometry to abstrct the method of
    probing block device geometry.
  - Use the inode number for the disk_id of cache files so that
    multiple cache files can be used.
  - Remove global Store theStore. The global Store theStore is not
    used for anything that matters. The cache and hostdb subsystems
    have their own separate Store instances.


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

Branch: refs/heads/master
Commit: 4747fe8f7c1c4bf6db4512f9d8ba0322b570dc8a
Parents: 25407ce
Author: James Peach <jp...@apache.org>
Authored: Thu Aug 28 13:28:16 2014 -0700
Committer: James Peach <jp...@apache.org>
Committed: Sun Sep 14 20:43:42 2014 -0700

----------------------------------------------------------------------
 CHANGES                                         |   2 +
 configure.ac                                    |   1 +
 .../configuration/storage.config.en.rst         |   2 +-
 iocore/cache/Cache.cc                           |  12 +-
 iocore/cache/I_Store.h                          |  96 ++--
 iocore/cache/Store.cc                           | 525 ++++++-------------
 iocore/hostdb/MultiCache.cc                     |   4 +-
 lib/ts/Makefile.am                              |   6 +-
 lib/ts/ink_file.cc                              | 132 +++++
 lib/ts/ink_file.h                               |  15 +
 lib/ts/test_geometry.cc                         |  56 ++
 proxy/Main.cc                                   |  23 -
 proxy/logcat.cc                                 |   5 +-
 proxy/sac.cc                                    |   4 +-
 14 files changed, 441 insertions(+), 442 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/trafficserver/blob/4747fe8f/CHANGES
----------------------------------------------------------------------
diff --git a/CHANGES b/CHANGES
index 09c4a5b..96b694a 100644
--- a/CHANGES
+++ b/CHANGES
@@ -1,6 +1,8 @@
                                                          -*- coding: utf-8 -*-
 Changes with Apache Traffic Server 5.2.0
 
+  *) [TS-3070] Make span configuration work consistently across platforms.
+
   *) [TS-3076] Fix minor strtok_r errors.
 
   *) [TS-2938] fix core dump in 307 redirect follow handling

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/4747fe8f/configure.ac
----------------------------------------------------------------------
diff --git a/configure.ac b/configure.ac
index e946d7e..a90ff40 100644
--- a/configure.ac
+++ b/configure.ac
@@ -1523,6 +1523,7 @@ AC_SUBST(ifaddrsh)
 AC_SUBST(readline_readlineh)
 
 TS_FLAG_HEADERS([sys/statfs.h sys/statvfs.h sys/disk.h sys/disklabel.h])
+TS_FLAG_HEADERS([linux/hdreg.h linux/fs.h linux/major.h])
 
 TS_FLAG_HEADERS([sys/sysctl.h], [], [],
                  [[#ifdef HAVE_SYS_PARAM_H

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/4747fe8f/doc/reference/configuration/storage.config.en.rst
----------------------------------------------------------------------
diff --git a/doc/reference/configuration/storage.config.en.rst b/doc/reference/configuration/storage.config.en.rst
index e1c1ea5..78293f2 100644
--- a/doc/reference/configuration/storage.config.en.rst
+++ b/doc/reference/configuration/storage.config.en.rst
@@ -36,7 +36,7 @@ The format of the :file:`storage.config` file is a series of lines of the form
 where :arg:`pathname` is the name of a partition, directory or file, :arg:`size` is the size of the
 named partition, directory or file (in bytes), and :arg:`volume` is the volume number used in the
 files :file:`volume.config` and :file:`hosting.config`. :arg:`id` is used for seeding the
-:ref:`assignment-table`. You must specify a size for directories or files; size is optional for raw
+:ref:`assignment-table`. You must specify a size for directories; size is optional for files and raw
 partitions. :arg:`volume` and arg:`seed` are optional.
 
 .. note::

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/4747fe8f/iocore/cache/Cache.cc
----------------------------------------------------------------------
diff --git a/iocore/cache/Cache.cc b/iocore/cache/Cache.cc
index f75fe0f..3625b71 100644
--- a/iocore/cache/Cache.cc
+++ b/iocore/cache/Cache.cc
@@ -661,7 +661,7 @@ CacheProcessor::start_internal(int flags)
         if (sector_size < cache_config_force_sector_size)
           sector_size = cache_config_force_sector_size;
         if (sd->hw_sector_size <= 0 || sector_size > STORE_BLOCK_SIZE) {
-          Warning("bad hardware sector size %d, resetting to %d", sector_size, STORE_BLOCK_SIZE);
+          Note("resetting hardware sector size from %d to %d", sector_size, STORE_BLOCK_SIZE);
           sector_size = STORE_BLOCK_SIZE;
         }
         off_t skip = ROUND_TO_STORE_BLOCK((sd->offset * STORE_BLOCK_SIZE < START_POS ? START_POS + sd->alignment :
@@ -748,12 +748,18 @@ CacheProcessor::start_internal(int flags)
 
         Debug("cache_hosting", "Disk: %d, blocks: %d", gndisks, blocks);
 
-        if (sector_size < cache_config_force_sector_size)
+        if (sector_size < cache_config_force_sector_size) {
           sector_size = cache_config_force_sector_size;
+        }
+
+        // It's actually common that the hardware I/O size is larger than the store block size as
+        // storage systems increasingly want larger I/Os. For example, on OS X, the filesystem block
+        // size is always reported as 1MB.
         if (sd->hw_sector_size <= 0 || sector_size > STORE_BLOCK_SIZE) {
-          Warning("bad hardware sector size %d, resetting to %d", sector_size, STORE_BLOCK_SIZE);
+          Note("resetting hardware sector size from %d to %d", sector_size, STORE_BLOCK_SIZE);
           sector_size = STORE_BLOCK_SIZE;
         }
+
         off_t skip = ROUND_TO_STORE_BLOCK((sd->offset < START_POS ? START_POS + sd->alignment : sd->offset));
         blocks = blocks - (skip >> STORE_BLOCK_SHIFT);
 #if AIO_MODE == AIO_MODE_NATIVE

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/4747fe8f/iocore/cache/I_Store.h
----------------------------------------------------------------------
diff --git a/iocore/cache/I_Store.h b/iocore/cache/I_Store.h
index f951e6b..c49ab38 100644
--- a/iocore/cache/I_Store.h
+++ b/iocore/cache/I_Store.h
@@ -37,34 +37,61 @@
 #define STORE_BLOCK_SHIFT      13
 #define DEFAULT_HW_SECTOR_SIZE 512
 
+enum span_error_t
+{
+  SPAN_ERROR_OK,
+  SPAN_ERROR_UNKNOWN,
+  SPAN_ERROR_NOT_FOUND,
+  SPAN_ERROR_NO_ACCESS,
+  SPAN_ERROR_MISSING_SIZE,
+  SPAN_ERROR_UNSUPPORTED_DEVTYPE,
+  SPAN_ERROR_MEDIA_PROBE,
+};
+
+struct span_diskid_t
+{
+  int64_t id[2];
+
+  bool operator < (const span_diskid_t& rhs) const {
+    return id[0] < rhs.id[0] && id[1] < rhs.id[1];
+  }
+
+  bool operator == (const span_diskid_t& rhs) const {
+    return id[0] == rhs.id[0] && id[1] == rhs.id[1];
+  }
+
+  int64_t& operator[] (unsigned i) {
+    return id[i];
+  }
+};
+
 //
 // A Store is a place to store data.
 // Those on the same disk should be in a linked list.
 //
 struct Span
 {
-  int64_t blocks;
-  int64_t offset;                 // used only if (file == true)
-  int hw_sector_size;
-  int alignment;
-  int disk_id;
+  int64_t blocks;                 // in STORE_BLOCK_SIZE blocks
+  int64_t offset;                 // used only if (file == true); in bytes
+  unsigned hw_sector_size;
+  unsigned alignment;
+  span_diskid_t disk_id;
   int forced_volume_num;  ///< Force span in to specific volume.
 private:
-    bool is_mmapable_internal;
+  bool is_mmapable_internal;
 public:
   bool file_pathname;           // the pathname is a file
-  bool isRaw;
   // v- used as a magic location for copy constructor.
   // we memcpy everything before this member and do explicit assignment for the rest.
   ats_scoped_str pathname;
   ats_scoped_str hash_base_string; ///< Used to seed the stripe assignment hash.
   SLINK(Span, link);
 
-  bool is_mmapable() { return is_mmapable_internal; }
+  bool is_mmapable() const { return is_mmapable_internal; }
   void set_mmapable(bool s) { is_mmapable_internal = s; }
-  int64_t size() { return blocks * STORE_BLOCK_SIZE; }
+  int64_t size() const { return blocks * STORE_BLOCK_SIZE; }
 
-  int64_t total_blocks() {
+  int64_t total_blocks() const {
     if (link.next) {
       return blocks + link.next->total_blocks();
     } else {
@@ -72,26 +99,27 @@ public:
     }
   }
 
-  Span *nth(int i) {
+  Span *nth(unsigned i) {
     Span *x = this;
     while (x && i--)
       x = x->link.next;
     return x;
   }
 
-  int paths() {
+  unsigned paths() const {
     int i = 0;
-    for (Span * x = this; x; i++, x = x->link.next);
+    for (const Span * x = this; x; i++, x = x->link.next);
     return i;
   }
-  int write(int fd);
+
+  int write(int fd) const;
   int read(int fd);
 
   /// Duplicate this span and all chained spans.
   Span *dup();
-  int64_t end() { return offset + blocks; }
+  int64_t end() const { return offset + blocks; }
 
-  const char *init(char *n, int64_t size);
+  const char *init(const char *n, int64_t size);
 
   // 0 on success -1 on failure
   int path(char *filename,      // for non-file, the filename in the director
@@ -108,12 +136,12 @@ public:
     , offset(0)
     , hw_sector_size(DEFAULT_HW_SECTOR_SIZE)
     , alignment(0)
-    , disk_id(0)
     , forced_volume_num(-1)
     , is_mmapable_internal(false)
     , file_pathname(false)
-    , isRaw(true)
-  { }
+  {
+    disk_id[0] = disk_id[1] = 0;
+  }
 
   /// Copy constructor.
   /// @internal Prior to this implementation handling the char* pointers was done manual
@@ -126,6 +154,8 @@ public:
   }
 
   ~Span();
+
+  static const char * errorstr(span_error_t serr);
 };
 
 struct Store
@@ -141,14 +171,14 @@ struct Store
 
   Span *alloc_one(unsigned int blocks, bool mmapable) {
     Store s;
-      alloc(s, blocks, true, mmapable);
-    if (s.n_disks)
-    {
+    alloc(s, blocks, true, mmapable);
+    if (s.n_disks) {
       Span *t = s.disk[0];
         s.disk[0] = NULL;
         return t;
-    } else
-        return NULL;
+    }
+
+    return NULL;
   }
   // try to allocate, return (s == gotten, diff == not gotten)
   void try_realloc(Store & s, Store & diff);
@@ -172,7 +202,7 @@ struct Store
   }
 
   // Non Thread-safe operations
-  unsigned int total_blocks(unsigned after = 0) {
+  unsigned int total_blocks(unsigned after = 0) const {
     int64_t t = 0;
     for (unsigned i = after; i < n_disks; i++) {
       if (disk[i]) {
@@ -184,7 +214,7 @@ struct Store
   // 0 on success -1 on failure
   // these operations are NOT thread-safe
   //
-  int write(int fd, char *name);
+  int write(int fd, const char *name) const;
   int read(int fd, char *name);
   int clear(char *filename, bool clear_dirs = true);
   void normalize();
@@ -205,25 +235,15 @@ struct Store
   // if fd >= 0 then on failure it returns an error string
   //            otherwise on failure it returns (char *)-1
   //
-  const char *read_config(int fd = -1);
-  int write_config_data(int fd);
+  const char *read_config();
+  int write_config_data(int fd) const;
 
   /// Additional configuration key values.
   static char const VOLUME_KEY[];
   static char const HASH_BASE_STRING_KEY[];
 };
 
-extern Store theStore;
-
 // store either free or in the cache, can be stolen for reconfiguration
 void stealStore(Store & s, int blocks);
-int initialize_store();
-
-struct storageConfigFile {
-  const char *parseFile(int fd) {
-    Store tStore;
-    return tStore.read_config(fd);
-  }
-};
 
 #endif

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/4747fe8f/iocore/cache/Store.cc
----------------------------------------------------------------------
diff --git a/iocore/cache/Store.cc b/iocore/cache/Store.cc
index 5d81822..96b1721 100644
--- a/iocore/cache/Store.cc
+++ b/iocore/cache/Store.cc
@@ -25,21 +25,10 @@
 #include "P_Cache.h"
 #include "I_Layout.h"
 
-#if HAVE_SYS_PARAM_H
-#include <sys/param.h>
+#if HAVE_LINUX_MAJOR_H
+#include <linux/major.h>
 #endif
 
-#if HAVE_SYS_DISK_H
-#include <sys/disk.h>
-#endif
-
-#if HAVE_SYS_DISKLABEL_H
-#include <sys/disklabel.h>
-#endif
-
-// Global
-Store theStore;
-
 //
 // Store
 //
@@ -47,6 +36,29 @@ Store theStore;
 char const Store::VOLUME_KEY[] = "volume";
 char const Store::HASH_BASE_STRING_KEY[] = "id";
 
+static span_error_t
+make_span_error(int error)
+{
+  switch (error) {
+  case ENOENT: return SPAN_ERROR_NOT_FOUND;
+  case EPERM: /* fallthru */
+  case EACCES: return SPAN_ERROR_NO_ACCESS;
+  default: return SPAN_ERROR_UNKNOWN;
+  }
+}
+
+static const char *
+span_file_typename(mode_t st_mode)
+{
+  switch (st_mode & S_IFMT) {
+  case S_IFBLK: return "block device";
+  case S_IFCHR: return "character device";
+  case S_IFDIR: return "directory";
+  case S_IFREG: return "file";
+  default: return "<unsupported>";
+  }
+}
+
 Ptr<ProxyMutex> tmp_p;
 Store::Store():n_disks(0), disk(NULL)
 #if TS_USE_INTERIM_CACHE == 1
@@ -55,12 +67,6 @@ Store::Store():n_disks(0), disk(NULL)
 {
 }
 
-int
-initialize_store()
-{
-  return theStore.read_config()? -1 : 0;
-}
-
 void
 Store::add(Span * ds)
 {
@@ -182,6 +188,28 @@ Store::sort()
   }
 }
 
+const char *
+Span::errorstr(span_error_t serr)
+{
+  switch (serr) {
+  case SPAN_ERROR_OK:
+    return "no error";
+  case SPAN_ERROR_NOT_FOUND:
+    return "file not found";
+  case SPAN_ERROR_NO_ACCESS:
+    return "unable to access file";
+  case SPAN_ERROR_MISSING_SIZE:
+    return "missing size specification";
+  case SPAN_ERROR_UNSUPPORTED_DEVTYPE:
+    return "unsupported cache file type";
+  case SPAN_ERROR_MEDIA_PROBE:
+    return "failed to probe device geometry";
+  case SPAN_ERROR_UNKNOWN: /* fallthru */
+  default:
+    return "unknown error";
+  }
+}
+
 int
 Span::path(char *filename, int64_t * aoffset, char *buf, int buflen)
 {
@@ -234,7 +262,7 @@ Span::~Span()
     delete link.next;
 }
 
-inline int
+static int
 get_int64(int fd, int64_t & data)
 {
   char buf[PATH_NAME_MAX + 1];
@@ -274,7 +302,7 @@ Lagain:
 }
 
 const char *
-Store::read_config(int fd)
+Store::read_config()
 {
   int n_dsstore = 0;
   int ln = 0;
@@ -282,19 +310,16 @@ Store::read_config(int fd)
   const char *err = NULL;
   Span *sd = NULL, *cur = NULL;
   Span *ns;
+  ats_scoped_fd fd;
+  ats_scoped_str storage_path(RecConfigReadConfigPath("proxy.config.cache.storage_filename", "storage.config"));
 
-  // Get pathname if not checking file
-
+  Debug("cache_init", "Store::read_config, fd = -1, \"%s\"", (const char *)storage_path);
+  fd = ::open(storage_path, O_RDONLY);
   if (fd < 0) {
-    ats_scoped_str storage_path(RecConfigReadConfigPath("proxy.config.cache.storage_filename", "storage.config"));
-
-    Debug("cache_init", "Store::read_config, fd = -1, \"%s\"", (const char *)storage_path);
-    fd = ::open(storage_path, O_RDONLY);
-    if (fd < 0) {
-      err = "error on open";
-      goto Lfail;
-    }
+    err = "error on open";
+    goto Lfail;
   }
+
   // For each line
 
   char line[1024];
@@ -384,12 +409,10 @@ Store::read_config(int fd)
   }
   sd = 0; // these are all used.
   sort();
-  ::close(fd);
   return NULL;
 
 Lfail:
   // Do clean up.
-  ::close(fd);
   if (sd)
     delete sd;
 
@@ -441,7 +464,7 @@ Store::read_interim_config() {
 #endif
 
 int
-Store::write_config_data(int fd)
+Store::write_config_data(int fd) const
 {
   for (unsigned i = 0; i < n_disks; i++)
     for (Span * sd = disk[i]; sd; sd = sd->link.next) {
@@ -453,378 +476,144 @@ Store::write_config_data(int fd)
   return 0;
 }
 
-#if defined(freebsd) || defined(darwin) || defined(openbsd)
-
 const char *
-Span::init(char *an, int64_t size)
+Span::init(const char * path, int64_t size)
 {
-  int devnum = 0;
-  const char *err = NULL;
-  int ret = 0;
-
-  is_mmapable_internal = true;
-
-  // handle symlinks
-
-  char *n = NULL;
-  int n_len = 0;
-  char real_n[PATH_NAME_MAX];
-
-  if ((n_len = readlink(an, real_n, sizeof(real_n) - 1)) > 0) {
-    real_n[n_len] = 0;
-    if (*real_n != '/') {
-      char *rs = strrchr(an, '/');
-      int l = 2;
-      const char *ann = "./";
+  struct stat     sbuf;
+  struct statvfs  vbuf;
+  span_error_t    serr;
+  ink_device_geometry geometry;
 
-      if (rs) {
-        ann = an;
-        l = (rs - an) + 1;
-      }
-      memmove(real_n + l, real_n, strlen(real_n) + 1);
-      memcpy(real_n, ann, l);
-    }
-    n = real_n;
-  } else {
-    n = an;
-  }
-
-  // stat the file system
-
-  struct stat s;
-  if ((ret = stat(n, &s)) < 0) {
-    Warning("unable to stat '%s': %d %d, %s", n, ret, errno, strerror(errno));
-    return "error stat of file";
-  }
-
-  ats_scoped_fd fd(socketManager.open(n, O_RDONLY));
+  ats_scoped_fd fd(socketManager.open(path, O_RDONLY));
   if (!fd) {
-    Warning("unable to open '%s': %s", n, strerror(errno));
-    return "unable to open";
+    serr = make_span_error(errno);
+    Warning("unable to open '%s': %s", path, strerror(errno));
+    goto fail;
   }
 
-  struct statvfs fs;
-  if ((ret = fstatvfs(fd, &fs)) < 0) {
-    Warning("unable to statvfs '%s': %d %d, %s", n, ret, errno, strerror(errno));
-    return "unable to statvfs";
+  if (fstat(fd, &sbuf) == -1) {
+    serr = make_span_error(errno);
+    Warning("unable to stat '%s': %s (%d)", path, strerror(errno), errno);
+    goto fail;
   }
 
-  hw_sector_size = fs.f_bsize;
-  int64_t fsize = (int64_t) fs.f_blocks * (int64_t) fs.f_bsize;
-
-  switch ((s.st_mode & S_IFMT)) {
-
-  case S_IFBLK:{
-  case S_IFCHR:
-    // These IOCTLs are standard across the BSD family; Darwin has a different set though.
-#if defined(DIOCGMEDIASIZE) && defined(DIOCGSECTORSIZE)
-      if (ioctl(fd, DIOCGMEDIASIZE, &size) < 0) {
-        Warning("unable to get disk information for '%s': %s", n, strerror(errno));
-        err = "unable to get label information";
-        goto Lfail;
-      }
-      if (ioctl(fd, DIOCGSECTORSIZE, &hw_sector_size) < 0) {
-        Warning("unable to get disk information for '%s': %s", n, strerror(errno));
-        err = "unable to get label information";
-        goto Lfail;
-      }
-      devnum = s.st_rdev;
-      break;
-#else
-      Warning("unable to get disk information for '%s': %s", n, strerror(errno));
-      err = "no raw disk support on this platform";
-      goto Lfail;
-#endif
-    }
-  case S_IFDIR:
-  case S_IFREG:
-    if (size <= 0 || size > fsize) {
-      Warning("bad or missing size for '%s': size %" PRId64 " fsize %" PRId64 "", n, (int64_t) size, fsize);
-      err = "bad or missing size";
-      goto Lfail;
-    }
-    devnum = s.st_dev;
-    break;
-
-  default:
-    Warning("unknown file type '%s': %d", n, s.st_mode);
-    return "unknown file type";
-    break;
+  if (fstatvfs(fd, &vbuf) == -1) {
+    serr = make_span_error(errno);
+    Warning("unable to statvfs '%s': %s (%d)", path, strerror(errno), errno);
+    goto fail;
   }
 
-  disk_id = devnum;
-
-  pathname = ats_strdup(an);
-  // igalic: blocks = size / hw_sector_size; was wrong TS-1707
-  // This code needs refactoring to unify the code-paths which are equal across platforms.
-  blocks = size / STORE_BLOCK_SIZE;
-  file_pathname = !((s.st_mode & S_IFMT) == S_IFDIR);
-
-  // This is so FreeBSD admins don't worry about our malicious code creating boot sector viruses:
-  if (((s.st_mode & S_IFMT) == S_IFBLK) || ((s.st_mode & S_IFMT) == S_IFCHR)) {
-    blocks--;
-    offset = 1;
+  // Directories require an explicit size parameter. For device nodes and files, we use
+  // the existing size.
+  if (S_ISDIR(sbuf.st_mode)) {
+    if (size <= 0) {
+      Warning("cache %s '%s' requires a size > 0", span_file_typename(sbuf.st_mode), path);
+      serr = SPAN_ERROR_MISSING_SIZE;
+      goto fail;
+    }
   }
 
-  Debug("cache_init", "Span::init - %s hw_sector_size = %d  size = %" PRId64 ", blocks = %" PRId64 ", disk_id = %d, file_pathname = %d", pathname.get(), hw_sector_size, size, blocks, disk_id, file_pathname);
+  // Should regular files use the IO size (vbuf.f_bsize), or the
+  // fundamental filesystem block size (vbuf.f_frsize)? That depends
+  // on whether we are using that block size for performance or for
+  // reliability.
 
-Lfail:
-  return err;
-}
+  switch (sbuf.st_mode & S_IFMT) {
+  case S_IFBLK:
+  case S_IFCHR:
 
+#if defined(linux)
+    if (major(sbuf.st_rdev) == RAW_MAJOR && minor(sbuf.st_rdev) == 0) {
+      Warning("cache %s '%s' is the raw device control interface", span_file_typename(sbuf.st_mode), path);
+      serr = SPAN_ERROR_UNSUPPORTED_DEVTYPE;
+      goto fail;
+    }
 #endif
 
-#if defined(solaris)
-
-const char *
-Span::init(char *filename, int64_t size)
-{
-  int devnum = 0;
-  const char *err = NULL;
-  int ret = 0;
-
-  //
-  // All file types on Solaris can be mmaped
-  //
-  is_mmapable_internal = true;
+    if (!ink_file_get_geometry(fd, geometry)) {
+      serr = make_span_error(errno);
 
-  ats_scoped_fd fd(socketManager.open(filename, O_RDONLY));
-  if (!fd) {
-    Warning("unable to open '%s': %s", filename, strerror(errno));
-    return "unable to open";
-  }
-
-  // stat the file system
-  struct stat s;
-  if ((ret = fstat(fd, &s)) < 0) {
-    Warning("unable to fstat '%s': %d %d, %s", filename, ret, errno, strerror(errno));
-    err = "unable to fstat";
-    goto Lfail;
-  }
+      if (errno == ENOTSUP) {
+        Warning("failed to query disk geometry for '%s', no raw device support", path);
+      } else {
+        Warning("failed to query disk geometry for '%s', %s (%d)", path, strerror(errno), errno);
+      }
 
+      goto fail;
+    }
 
-  switch ((s.st_mode & S_IFMT)) {
+    this->disk_id[0] = 0;
+    this->disk_id[1] = sbuf.st_rdev;
+    this->file_pathname = true;
+    this->hw_sector_size = geometry.blocksz;
+    this->alignment = geometry.alignsz;
+    this->blocks = geometry.totalsz / STORE_BLOCK_SIZE;
 
-  case S_IFBLK:
-  case S_IFCHR:
-    devnum = s.st_rdev;
-    // maybe we should use lseek(fd, 0, SEEK_END) here (it is portable)
-    size = (int64_t) s.st_size;
-    hw_sector_size = s.st_blksize;
     break;
+
   case S_IFDIR:
-  case S_IFREG:
-    int64_t fsize;
-    struct statvfs fs;
-    if ((ret = fstatvfs(fd, &fs)) < 0) {
-      Warning("unable to statvfs '%s': %d %d, %s", filename, ret, errno, strerror(errno));
-      err = "unable to statvfs";
-      goto Lfail;
+    if ((int64_t)(vbuf.f_frsize * vbuf.f_bavail) < size) {
+      Warning("not enough free space for cache %s '%s'", span_file_typename(sbuf.st_mode), path);
+      // Just warn for now; let the cache open fail later.
     }
 
-    hw_sector_size = fs.f_bsize;
-    fsize = (int64_t) fs.f_blocks * (int64_t) hw_sector_size;
-
-    if (size <= 0 || size > fsize) {
-      Warning("bad or missing size for '%s': size %" PRId64 " fsize %" PRId64 "", filename, (int64_t) size, fsize);
-      err = "bad or missing size";
-      goto Lfail;
-    }
+    // The cache initialization code in Cache.cc takes care of creating the actual cache file, naming it and making
+    // it the right size based on the "file_pathname" flag. That's something that we should clean up in the future.
+    this->file_pathname = false;
 
-    devnum = s.st_dev;
+    this->disk_id[0] = sbuf.st_dev;
+    this->disk_id[1] = sbuf.st_ino;
+    this->hw_sector_size = vbuf.f_bsize;
+    this->alignment = 0;
+    this->blocks = size / STORE_BLOCK_SIZE;
     break;
 
-  default:
-    Warning("unknown file type '%s': %" PRId64 "", filename, (int64_t)(s.st_mode));
-    err = "unknown file type";
-    goto Lfail;
-  }
-
-  // estimate the disk SOLARIS specific
-  if ((devnum >> 16) == 0x80) {
-    disk_id = (devnum >> 3) & 0x3F;
-  } else {
-    disk_id = devnum;
-  }
-
-  pathname = ats_strdup(filename);
-  // is this right Seems like this should be size / hw_sector_size
-  // igalic: No. See TS-1707
-  blocks = size / STORE_BLOCK_SIZE;
-  file_pathname = !((s.st_mode & S_IFMT) == S_IFDIR);
-
-  Debug("cache_init", "Span::init - %s hw_sector_size = %d  size = %" PRId64 ", blocks = %" PRId64 ", disk_id = %d, file_pathname = %d", filename, hw_sector_size, size, blocks, disk_id, file_pathname);
-
-Lfail:
-  return err;
-}
-#endif
-
-#if defined(linux)
-#include <unistd.h>             /* for close() */
-#include <sys/ioctl.h>
-#include <linux/hdreg.h>        /* for struct hd_geometry */
-#include <linux/fs.h>           /* for BLKGETSIZE.  sys/mount.h is another candidate */
-
+  case S_IFREG:
+    if (size > 0 && sbuf.st_size < size) {
+      int64_t needed = size - sbuf.st_size;
+      if ((int64_t)(vbuf.f_frsize * vbuf.f_bavail) < needed) {
+        Warning("not enough free space for cache %s '%s'", span_file_typename(sbuf.st_mode), path);
+        // Just warn for now; let the cache open fail later.
+      }
+    }
 
-const char *
-Span::init(char *filename, int64_t size)
-{
-  int devnum = 0, arg = 0;
-  int ret = 0, is_disk = 0;
-  uint64_t heads, sectors, cylinders, adjusted_sec;
-  ats_scoped_fd fd;
+    this->disk_id[0] = sbuf.st_dev;
+    this->disk_id[1] = sbuf.st_ino;
+    this->file_pathname = true;
+    this->hw_sector_size = vbuf.f_bsize;
+    this->alignment = 0;
+    this->blocks = sbuf.st_size / STORE_BLOCK_SIZE;
 
-  /* Fetch file type */
-  struct stat stat_buf;
-  Debug("cache_init", "Span::init(\"%s\",%" PRId64 ")", filename, size);
-  if ((ret = stat(filename, &stat_buf)) < 0) {
-    Warning("unable to stat '%s': %d %d, %s", filename, ret, errno, strerror(errno));
-    return "cannot stat file";
-  }
-  switch (stat_buf.st_mode & S_IFMT) {
-  case S_IFBLK:
-  case S_IFCHR:
-    devnum = stat_buf.st_rdev;
-    Debug("cache_init", "Span::init - %s - devnum = %d",
-          ((stat_buf.st_mode & S_IFMT) == S_IFBLK) ? "S_IFBLK" : "S_IFCHR", devnum);
-    break;
-  case S_IFDIR:
-    devnum = stat_buf.st_dev;
-    file_pathname = 0;
-    Debug("cache_init", "Span::init - S_IFDIR - devnum = %d", devnum);
-    break;
-  case S_IFREG:
-    devnum = stat_buf.st_dev;
-    file_pathname = 1;
-    size = stat_buf.st_size;
-    Debug("cache_init", "Span::init - S_IFREG - devnum = %d", devnum);
     break;
+
   default:
-    break;
+    serr = SPAN_ERROR_UNSUPPORTED_DEVTYPE;
+    goto fail;
   }
 
-  fd = socketManager.open(filename, O_RDONLY);
-  if (!fd) {
-    Warning("unable to open '%s': %s", filename, strerror(errno));
-    return "unable to open";
-  }
-  Debug("cache_init", "Span::init - socketManager.open(\"%s\", O_RDONLY) = %d", filename, (int)fd);
+  // The actual size of a span always trumps the configured size.
+  if (size > 0 && this->size() != size) {
+    int64_t newsz = MIN(size, this->size());
 
-  adjusted_sec = 1;
-#ifdef BLKPBSZGET
-  if (ioctl(fd, BLKPBSZGET, &arg) == 0)
-#else
-  if (ioctl(fd, BLKSSZGET, &arg) == 0)
-#endif
-  {
-    hw_sector_size = arg;
-    is_disk = 1;
-    adjusted_sec = hw_sector_size / 512;
-    Debug("cache_init", "Span::init - %s hw_sector_size=%d is_disk=%d adjusted_sec=%" PRId64,
-          filename, hw_sector_size, is_disk, adjusted_sec);
-  }
+    Note("cache %s '%s' is %" PRId64 " bytes, but the configured size is %" PRId64 " bytes, using the minimum",
+      span_file_typename(sbuf.st_mode), path, this->size(), size);
 
-  alignment = 0;
-#ifdef BLKALIGNOFF
-  if (ioctl(fd, BLKALIGNOFF, &arg) == 0) {
-    alignment = arg;
-    Debug("cache_init", "Span::init - %s alignment = %d", filename, alignment);
+    this->blocks = newsz / STORE_BLOCK_SIZE;
   }
-#endif
-
-  if (is_disk) {
-    uint32_t ioctl_sectors = 0;
-    uint64_t ioctl_bytes = 0;
-    uint64_t physsectors = 0;
-
-    /* Disks cannot be mmapped */
-    is_mmapable_internal = false;
-
-    if (!ioctl(fd, BLKGETSIZE64, &ioctl_bytes)) {
-      heads = 1;
-      cylinders = 1;
-      physsectors = ioctl_bytes / hw_sector_size;
-      sectors = physsectors;
-    } else if (!ioctl(fd, BLKGETSIZE, &ioctl_sectors)) {
-      heads = 1;
-      cylinders = 1;
-      physsectors = ioctl_sectors;
-      sectors = physsectors / adjusted_sec;
-    } else {
-      struct hd_geometry geometry;
-      if (!ioctl(fd, HDIO_GETGEO, &geometry)) {
-        heads = geometry.heads;
-        sectors = geometry.sectors;
-        cylinders = geometry.cylinders;
-        cylinders /= adjusted_sec;      /* do not round up */
-      } else {
-        /* Almost certainly something other than a disk device. */
-        Warning("unable to get geometry '%s': %d %s", filename, errno, strerror(errno));
-        return ("unable to get geometry");
-      }
-    }
-
-    blocks = heads * sectors * cylinders;
-
-    if (size > 0 && blocks * hw_sector_size != size) {
-      Warning("Warning: you specified a size of %" PRId64 " for %s,\n", size, filename);
-      Warning("but the device size is %" PRId64 ". Using minimum of the two.\n", (int64_t)blocks * (int64_t)hw_sector_size);
-      if ((int64_t)blocks * (int64_t)hw_sector_size < size)
-        size = (int64_t)blocks * (int64_t)hw_sector_size;
-    } else {
-      size = (int64_t)blocks * (int64_t)hw_sector_size;
-    }
 
-    /* I don't know why I'm redefining blocks to be something that is quite
-     * possibly something other than the actual number of blocks, but the
-     * code for other arches seems to.  Revisit this, perhaps. */
-    // igalic: No. See TS-1707
-    blocks = size / STORE_BLOCK_SIZE;
+  // A directory span means we will end up with a file, otherwise, we get what we asked for.
+  this->set_mmapable(ink_file_is_mmappable(S_ISDIR(sbuf.st_mode) ? (mode_t)S_IFREG : sbuf.st_mode));
+  this->pathname = ats_strdup(path);
 
-    Debug("cache_init", "Span::init physical sectors %" PRId64 " total size %" PRId64 " geometry size %" PRId64 " store blocks %" PRId64 "",
-          physsectors, hw_sector_size * physsectors, size, blocks);
-
-    pathname = ats_strdup(filename);
-    file_pathname = 1;
-  } else {
-    Debug("cache_init", "Span::init - is_disk = %d, raw device = %s", is_disk, (major(devnum) == 162) ? "yes" : "no");
-    if (major(devnum) == 162) {
-      /* Oh, a raw device, how cute. */
-
-      if (minor(devnum) == 0)
-        return "The raw device control file (usually /dev/raw; major 162, minor 0) is not a valid cache location.\n";
-
-      is_mmapable_internal = false;     /* I -think- */
-      file_pathname = 1;
-      pathname = ats_strdup(filename);
-      isRaw = 1;
-
-      if (size <= 0)
-        return "When using raw devices for cache storage, you must specify a size\n";
-    } else {
-      /* Files can be mmapped */
-      is_mmapable_internal = true;
-
-      /* The code for other arches seems to want to dereference symlinks, but I
-       * don't particularly understand that behaviour, so I'll just ignore it.
-       * :) */
-
-      pathname = ats_strdup(filename);
-      if (!file_pathname)
-        if (size <= 0)
-          return "When using directories for cache storage, you must specify a size\n";
-      Debug("cache_init", "Span::init - mapped file \"%s\", %" PRId64 "", pathname.get(), size);
-    }
-    blocks = size / STORE_BLOCK_SIZE;
-  }
-
-  disk_id = devnum;
+  Debug("cache_init", "initialized span '%s'", this->pathname.get());
+  Debug("cache_init", "hw_sector_size=%d, size=%" PRId64 ", blocks=%" PRId64 ", disk_id=%" PRId64 "/%" PRId64 ", file_pathname=%d",
+    this->hw_sector_size, this->size(), this->blocks, this->disk_id[0], this->disk_id[1], this->file_pathname);
 
   return NULL;
-}
-#endif
 
+fail:
+  return Span::errorstr(serr);
+}
 
 
 void
@@ -972,7 +761,7 @@ Store::alloc(Store & s, unsigned int blocks, bool one_only, bool mmapable)
 }
 
 int
-Span::write(int fd)
+Span::write(int fd) const
 {
   char buf[32];
 
@@ -1001,7 +790,7 @@ Span::write(int fd)
 }
 
 int
-Store::write(int fd, char *name)
+Store::write(int fd, const char *name) const
 {
   char buf[32];
 
@@ -1166,7 +955,7 @@ Store::clear(char *filename, bool clear_dirs)
   memset(z, 0, STORE_BLOCK_SIZE);
   for (unsigned i = 0; i < n_disks; i++) {
     Span *ds = disk[i];
-    for (int j = 0; j < disk[i]->paths(); j++) {
+    for (unsigned j = 0; j < disk[i]->paths(); j++) {
       char path[PATH_NAME_MAX + 1];
       Span *d = ds->nth(j);
       if (!clear_dirs && !d->file_pathname)

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/4747fe8f/iocore/hostdb/MultiCache.cc
----------------------------------------------------------------------
diff --git a/iocore/hostdb/MultiCache.cc b/iocore/hostdb/MultiCache.cc
index 108c4f4..65efbfd 100644
--- a/iocore/hostdb/MultiCache.cc
+++ b/iocore/hostdb/MultiCache.cc
@@ -232,7 +232,7 @@ MultiCacheBase::mmap_region(int blocks, int *fds, char *cur, size_t& total_lengt
     if (blocks - target > following)
       target = blocks - following;
     Span *ds = store->disk[i];
-    for (int j = 0; j < store->disk[i]->paths(); j++) {
+    for (unsigned j = 0; j < store->disk[i]->paths(); j++) {
       Span *d = ds->nth(j);
 
       ink_assert(d->is_mmapable());
@@ -307,7 +307,7 @@ MultiCacheBase::mmap_data(bool private_flag, bool zero_fill)
     goto Lalloc;
   for (unsigned i = 0; i < store->n_disks; i++) {
     Span *ds = store->disk[i];
-    for (int j = 0; j < store->disk[i]->paths(); j++) {
+    for (unsigned j = 0; j < store->disk[i]->paths(); j++) {
       char path[PATH_NAME_MAX + 1];
       Span *d = ds->nth(j);
       int r = d->path(filename, NULL, path, PATH_NAME_MAX);

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/4747fe8f/lib/ts/Makefile.am
----------------------------------------------------------------------
diff --git a/lib/ts/Makefile.am b/lib/ts/Makefile.am
index c0d4517..d1bcc89 100644
--- a/lib/ts/Makefile.am
+++ b/lib/ts/Makefile.am
@@ -21,7 +21,7 @@ library_includedir=$(includedir)/ts
 library_include_HEADERS = apidefs.h
 
 noinst_PROGRAMS = mkdfa CompileParseRules
-check_PROGRAMS = test_atomic test_freelist test_arena test_List test_Map test_Vec
+check_PROGRAMS = test_atomic test_freelist test_arena test_List test_Map test_Vec test_geometry
 TESTS = $(check_PROGRAMS)
 
 AM_CPPFLAGS = -I$(top_srcdir)/lib
@@ -217,6 +217,10 @@ test_Vec_SOURCES = test_Vec.cc
 test_Vec_LDADD = libtsutil.la @LIBTCL@ @LIBPCRE@
 test_Vec_LDFLAGS = @EXTRA_CXX_LDFLAGS@ @LIBTOOL_LINK_FLAGS@
 
+test_geometry_SOURCES = test_geometry.cc
+test_geometry_LDADD = libtsutil.la @LIBTCL@ @LIBPCRE@
+test_geometry_LDFLAGS = @EXTRA_CXX_LDFLAGS@ @LIBTOOL_LINK_FLAGS@
+
 CompileParseRules_SOURCES = CompileParseRules.cc
 
 test:: $(TESTS)

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/4747fe8f/lib/ts/ink_file.cc
----------------------------------------------------------------------
diff --git a/lib/ts/ink_file.cc b/lib/ts/ink_file.cc
index 41891f1..a5af471 100644
--- a/lib/ts/ink_file.cc
+++ b/lib/ts/ink_file.cc
@@ -27,6 +27,36 @@
 #include <sys/stat.h>
 #endif
 
+#if HAVE_SYS_PARAM_H
+#include <sys/param.h>
+#endif
+
+#if HAVE_SYS_DISK_H
+#include <sys/disk.h>
+#endif
+
+#if HAVE_SYS_DISKLABEL_H
+#include <sys/disklabel.h>
+#endif
+
+#if HAVE_SYS_IOCTL_H
+#include <sys/ioctl.h>
+#endif
+
+#if HAVE_LINUX_HDREG_H
+#include <linux/hdreg.h>        /* for struct hd_geometry */
+#endif
+
+#if HAVE_LINUX_FS_H
+#include <linux/fs.h>           /* for BLKGETSIZE.  sys/mount.h is another candidate */
+#endif
+
+typedef union {
+  uint64_t  u64;
+  uint32_t  u32;
+  off_t     off;
+} ioctl_arg_t;
+
 int
 ink_fputln(FILE * stream, const char *s)
 {
@@ -370,3 +400,105 @@ ink_file_is_directory(const char * path)
 
   return S_ISDIR(sbuf.st_mode);
 }
+
+bool
+ink_file_is_mmappable(mode_t st_mode)
+{
+  // Regular files are always ok;
+  if (S_ISREG(st_mode)) {
+    return true;
+  }
+
+#if defined(linux)
+  // Disks cannot be mmapped.
+  if (S_ISBLK(st_mode)) {
+    return false;
+  }
+
+  // At least some character devices can be mmapped. But should you?
+  if (S_ISCHR(st_mode)) {
+    return true;
+  }
+#endif
+
+#if defined(solaris)
+  // All file types on Solaris can be mmap(2)'ed.
+  return true;
+#endif
+
+  return false;
+
+}
+
+bool
+ink_file_get_geometry(int fd ATS_UNUSED, ink_device_geometry& geometry)
+{
+  ink_zero(geometry);
+
+#if defined(freebsd) || defined(darwin) || defined(openbsd)
+  ioctl_arg_t arg ATS_UNUSED;
+
+  // These IOCTLs are standard across the BSD family; Darwin has a different set though.
+#if defined(DIOCGMEDIASIZE)
+  if (ioctl(fd, DIOCGMEDIASIZE, &arg.off) == 0) {
+    geometry.totalsz = arg.off;
+  }
+#endif
+
+#if defined(DIOCGSECTORSIZE)
+  if (ioctl(fd, DIOCGSECTORSIZE, &arg.u32) == 0) {
+    geometry.blocksz = sector_size;
+  }
+#endif
+
+#if !defined(DIOCGMEDIASIZE) || !defined(DIOCGSECTORSIZE)
+  errno = ENOTSUP;
+#endif
+
+#elif defined(solaris)
+  struct stat sbuf;
+
+  if (fstat(fd, &sbuf) == 0) {
+    geometry.totalsz = sbuf.st_size;
+    geometry.blocksz = sbuf.st_blksize;
+  }
+
+#elif defined(linux)
+  ioctl_arg_t arg;
+
+  // The following set of ioctls work for both block and character devices. You can use the
+  // test_geometry program to test what happens for any specific use case or kernel.
+
+  // BLKGETSIZE64 gets the block device size in bytes.
+  if (ioctl(fd, BLKGETSIZE64, &arg.u64) == 0) {
+    geometry.totalsz = arg.u64;
+  }
+
+  // BLKSSZGET gets the logical block size in bytes.
+  if (ioctl(fd, BLKSSZGET, &arg.u32) == 0) {
+    geometry.blocksz = arg.u32;
+  }
+
+#if defined(BLKALIGNOFF)
+  // BLKALIGNOFF gets the number of bytes needed to align the I/Os to the block device with
+  // and underlying block devices. This might be non-zero when you are using a logical volume
+  // backed by JBOD or RAID device(s). BLKALIGNOFF was addeed in 2.6.32, so it's not present in
+  // RHEL 5.
+  if (ioctl(fd, BLKALIGNOFF, &arg.u32) == 0) {
+    geometry.alignsz = arg.u32;
+  }
+#endif
+
+#else /* No raw device support on this platform. */
+
+  errno = ENOTSUP;
+  return false;
+
+#endif
+
+  if (geometry.totalsz == 0 || geometry.blocksz == 0) {
+    return false;
+  }
+
+  return true;
+}

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/4747fe8f/lib/ts/ink_file.h
----------------------------------------------------------------------
diff --git a/lib/ts/ink_file.h b/lib/ts/ink_file.h
index c6429d7..326710c 100644
--- a/lib/ts/ink_file.h
+++ b/lib/ts/ink_file.h
@@ -103,4 +103,19 @@ int ink_file_fd_zerofill(int fd, off_t size);
  Return true if the path is a directory.
  */
 bool ink_file_is_directory(const char * path);
+
+/**
+ Return true if this file type can be mmap(2)'ed.
+ */
+bool ink_file_is_mmappable(mode_t st_mode);
+
+struct ink_device_geometry
+{
+  uint64_t  totalsz;  // Total device size in bytes.
+  unsigned  blocksz;  // Preferred I/O block size.
+  unsigned  alignsz;  // Block device alignment in bytes. Only relevant with stacked block devices.
+};
+
+bool ink_file_get_geometry(int fd, ink_device_geometry& geometry);
+
 #endif // _ink_file_h_

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/4747fe8f/lib/ts/test_geometry.cc
----------------------------------------------------------------------
diff --git a/lib/ts/test_geometry.cc b/lib/ts/test_geometry.cc
new file mode 100644
index 0000000..ffa01d7
--- /dev/null
+++ b/lib/ts/test_geometry.cc
@@ -0,0 +1,56 @@
+/** @file
+
+  Print block device geometry.
+
+  @section license License
+
+  Licensed to the Apache Software Foundation (ASF) under one
+  or more contributor license agreements.  See the NOTICE file
+  distributed with this work for additional information
+  regarding copyright ownership.  The ASF licenses this file
+  to you under the Apache License, Version 2.0 (the
+  "License"); you may not use this file except in compliance
+  with the License.  You may obtain a copy of the License at
+
+      http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+ */
+
+#include "libts.h"
+#include "ink_platform.h"
+
+// This isn't really a unit test. It's just a dumb little program to probe the disk
+// geometry of an arbitrary device file. That's useful when figuring out how ATS will
+// perceive different devices on differen operating systems.
+
+int main(int argc, const char ** argv)
+{
+  for (int i = 1; i < argc; ++i) {
+    int fd;
+    ink_device_geometry geometry;
+
+    fd = open(argv[i], O_RDONLY);
+    if (fd == -1) {
+      fprintf(stderr, "open(%s): %s\n", argv[i], strerror(errno));
+      continue;
+    }
+
+    if (ink_file_get_geometry(fd, geometry)) {
+      printf("%s:\n", argv[i]);
+      printf("\ttotalsz: %" PRId64 "\n", geometry.totalsz);
+      printf("\tblocksz: %u\n", geometry.blocksz);
+      printf("\talignsz: %u\n", geometry.alignsz);
+    } else {
+      printf("%s: %s (%d)\n", argv[i], strerror(errno), errno);
+    }
+
+    close(fd);
+  }
+
+  return 0;
+}

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/4747fe8f/proxy/Main.cc
----------------------------------------------------------------------
diff --git a/proxy/Main.cc b/proxy/Main.cc
index 1696151..c91969b 100644
--- a/proxy/Main.cc
+++ b/proxy/Main.cc
@@ -486,16 +486,6 @@ cmd_check_internal(char * /* cmd ATS_UNUSED */, bool fix = false)
 
   hostdb_current_interval = (ink_get_based_hrtime() / HRTIME_MINUTE);
 
-//#ifndef INK_NO_ACC
-//  acc.clear_cache();
-//#endif
-
-  const char *err = NULL;
-  theStore.delete_all();
-  if ((err = theStore.read_config())) {
-    printf("%s, %s failed\n", err, n);
-    return CMD_FAILED;
-  }
   printf("Host Database\n");
   HostDBCache hd;
   if (hd.start(fix) < 0) {
@@ -547,15 +537,6 @@ cmd_clear(char *cmd)
       Note("unable to unlink %s", (const char *)config);
   }
 
-  if (c_all || c_cache) {
-    const char *err = NULL;
-    theStore.delete_all();
-    if ((err = theStore.read_config())) {
-      printf("%s, CLEAR failed\n", err);
-      return CMD_FAILED;
-    }
-  }
-
   if (c_hdb || c_all) {
     Note("Clearing Host Database");
     if (hostDBProcessor.cache()->start(PROCESSOR_RECONFIGURE) < 0) {
@@ -1441,10 +1422,6 @@ main(int /* argc ATS_UNUSED */, char **argv)
   check_fd_limit();
   command_flag = command_flag || *command_string;
 
-  // Set up store
-  if (!command_flag && initialize_store())
-    ProcessFatal("unable to initialize storage, (Re)Configuration required\n");
-
   // Alter the frequecies at which the update threads will trigger
 #define SET_INTERVAL(scope, name, var) do { \
   RecInt tmpint; \

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/4747fe8f/proxy/logcat.cc
----------------------------------------------------------------------
diff --git a/proxy/logcat.cc b/proxy/logcat.cc
index 8123ee6..4a9cc56 100644
--- a/proxy/logcat.cc
+++ b/proxy/logcat.cc
@@ -20,10 +20,9 @@
   See the License for the specific language governing permissions and
   limitations under the License.
  */
-#include "ink_config.h"
-#include "ink_file.h"
+
+#include "libts.h"
 #include "I_Layout.h"
-#include "I_Version.h"
 
 #define PROGRAM_NAME        "traffic_logcat"
 #define MAX_LOGBUFFER_SIZE  65536

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/4747fe8f/proxy/sac.cc
----------------------------------------------------------------------
diff --git a/proxy/sac.cc b/proxy/sac.cc
index 2cea55d..df174c4 100644
--- a/proxy/sac.cc
+++ b/proxy/sac.cc
@@ -21,10 +21,8 @@
   limitations under the License.
  */
 
-#include "ink_config.h"
-#include "ink_file.h"
+#include "libts.h"
 #include "I_Layout.h"
-#include "I_Version.h"
 #include "P_Net.h"
 
 #define PROGRAM_NAME  "traffic_sac"