You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@trafficserver.apache.org by am...@apache.org on 2015/03/31 16:50:54 UTC

trafficserver git commit: TS-3468: Fix check option to be a useful stripe inspector.

Repository: trafficserver
Updated Branches:
  refs/heads/master 68b4d8536 -> d0d0afb9a


TS-3468: Fix check option to be a useful stripe inspector.


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

Branch: refs/heads/master
Commit: d0d0afb9a97c9449881d1275c93672e1010a2628
Parents: 68b4d85
Author: Alan M. Carroll <am...@apache.org>
Authored: Fri Mar 13 01:41:32 2015 -0500
Committer: Alan M. Carroll <am...@apache.org>
Committed: Tue Mar 31 09:50:01 2015 -0500

----------------------------------------------------------------------
 CHANGES                    |   2 +
 iocore/cache/Cache.cc      |  60 +++++++++++---
 iocore/cache/CacheDir.cc   | 171 +++++++++++++++++++++++++++-------------
 iocore/cache/CacheDisk.cc  |  20 ++++-
 iocore/cache/I_Cache.h     |   5 +-
 iocore/cache/P_CacheDir.h  |   2 +
 iocore/cache/P_CacheDisk.h |   4 +-
 proxy/Main.cc              | 152 ++++++++++++++++-------------------
 8 files changed, 259 insertions(+), 157 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/trafficserver/blob/d0d0afb9/CHANGES
----------------------------------------------------------------------
diff --git a/CHANGES b/CHANGES
index 24a6811..49325fb 100644
--- a/CHANGES
+++ b/CHANGES
@@ -8,6 +8,8 @@ Changes with Apache Traffic Server 6.0.0
 
 Changes with Apache Traffic Server 5.3.0
 
+  *) [TS-3468] Make command 'check' work usefully.
+
   *) [TS-2709] ATS does not send close-notify on shutdown. Confuses some clients.
 
   *) [TS-3467] Cleanup tmp files created from traffic_via tests.

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/d0d0afb9/iocore/cache/Cache.cc
----------------------------------------------------------------------
diff --git a/iocore/cache/Cache.cc b/iocore/cache/Cache.cc
index d8e7bed..c818cc7 100644
--- a/iocore/cache/Cache.cc
+++ b/iocore/cache/Cache.cc
@@ -103,8 +103,9 @@ Store theCacheStore;
 volatile int CacheProcessor::initialized = CACHE_INITIALIZING;
 volatile uint32_t CacheProcessor::cache_ready = 0;
 volatile int CacheProcessor::start_done = 0;
-int CacheProcessor::clear = 0;
-int CacheProcessor::fix = 0;
+bool CacheProcessor::clear = false;
+bool CacheProcessor::fix = false;
+bool CacheProcessor::check = false;
 int CacheProcessor::start_internal_flags = 0;
 int CacheProcessor::auto_clear_flag = 0;
 CacheProcessor cacheProcessor;
@@ -615,6 +616,7 @@ CacheProcessor::start_internal(int flags)
   start_internal_flags = flags;
   clear = !!(flags & PROCESSOR_RECONFIGURE) || auto_clear_flag;
   fix = !!(flags & PROCESSOR_FIX);
+  check = (flags & PROCESSOR_CHECK) != 0;
   start_done = 0;
   int diskok = 1;
   Span *sd;
@@ -646,18 +648,37 @@ CacheProcessor::start_internal(int flags)
 #ifdef O_DSYNC
     opts |= O_DSYNC;
 #endif
+    if (check) {
+      opts &= ~O_CREAT;
+      opts |= O_RDONLY;
+    }
 
     int fd = open(path, opts, 0644);
     int blocks = sd->blocks;
     if (fd > 0) {
       if (!sd->file_pathname) {
-        if (ftruncate(fd, ((uint64_t)blocks) * STORE_BLOCK_SIZE) < 0) {
-          Warning("unable to truncate cache file '%s' to %d blocks", path, blocks);
+        if (!check) {
+          if (ftruncate(fd, ((uint64_t)blocks) * STORE_BLOCK_SIZE) < 0) {
+            Warning("unable to truncate cache file '%s' to %d blocks", path, blocks);
+            diskok = 0;
+          }
+        } else { // read-only mode
+          struct stat sbuf;
           diskok = 0;
+          if (-1 == fstat(fd, &sbuf)) {
+            fprintf(stderr, "Failed to stat cache file for directory %s", path)
+          } else if (blocks != sbuf.st_size / STORE_BLOCK_SIZE) {
+            fprintf(stderr, "Cache file for directory %s is %" PRId64 " bytes, expected %" PRId64, path, sbuf.st_size,
+                    blocks * STORE_BLOCK_SIZE);
+          } else {
+            diskok = 1;
+          }
         }
       }
       if (diskok) {
         CacheDisk *disk = new CacheDisk();
+        if (check)
+          disk->read_only_p = true;
         Debug("cache_hosting", "interim Disk: %d, blocks: %d", gn_interim_disks, blocks);
         int sector_size = sd->hw_sector_size;
         if (sector_size < cache_config_force_sector_size)
@@ -726,6 +747,10 @@ CacheProcessor::start_internal(int flags)
 #ifdef O_DSYNC
     opts |= O_DSYNC;
 #endif
+    if (check) {
+      opts &= ~O_CREAT;
+      opts |= O_RDONLY;
+    }
 
     int fd = open(path, opts, 0644);
     int blocks = sd->blocks;
@@ -735,15 +760,30 @@ CacheProcessor::start_internal(int flags)
 
     if (fd >= 0) {
       if (!sd->file_pathname) {
-        if (ftruncate(fd, ((uint64_t)blocks) * STORE_BLOCK_SIZE) < 0) {
-          Warning("unable to truncate cache file '%s' to %d blocks", path, blocks);
+        if (check) {
+          if (ftruncate(fd, ((uint64_t)blocks) * STORE_BLOCK_SIZE) < 0) {
+            Warning("unable to truncate cache file '%s' to %d blocks", path, blocks);
+            diskok = 0;
+          }
+        } else { // read-only mode checks
+          struct stat sbuf;
           diskok = 0;
+          if (-1 == fstat(fd, &sbuf)) {
+            fprintf(stderr, "Failed to stat cache file for directory %s", path);
+          } else if (blocks != sbuf.st_size / STORE_BLOCK_SIZE) {
+            fprintf(stderr, "Cache file for directory %s is %" PRId64 " bytes, expected %" PRId64, path, sbuf.st_size,
+                    blocks * static_cast<int64_t>(STORE_BLOCK_SIZE));
+          } else {
+            diskok = 1;
+          }
         }
       }
       if (diskok) {
         int sector_size = sd->hw_sector_size;
 
         gdisks[gndisks] = new CacheDisk();
+        if (check)
+          gdisks[gndisks]->read_only_p = true;
         gdisks[gndisks]->forced_volume_num = sd->forced_volume_num;
         if (sd->hash_base_string)
           gdisks[gndisks]->hash_base_string = ats_strdup(sd->hash_base_string);
@@ -872,7 +912,8 @@ CacheProcessor::diskInitialized()
                 d->header->vol_info[j].len, d->header->vol_info[j].free);
         }
       }
-      d->sync();
+      if (!check)
+        d->sync();
     }
     if (config_volumes.num_volumes == 0) {
       theCache = new Cache();
@@ -1087,7 +1128,8 @@ CacheProcessor::cacheInitialized()
       GLOBAL_CACHE_SET_DYN_STAT(cache_bytes_total_stat, total_cache_bytes);
       GLOBAL_CACHE_SET_DYN_STAT(cache_direntries_total_stat, total_direntries);
       GLOBAL_CACHE_SET_DYN_STAT(cache_direntries_used_stat, used_direntries);
-      dir_sync_init();
+      if (!check)
+        dir_sync_init();
       cache_init_ok = 1;
     } else
       Warning("cache unable to open any vols, disabled");
@@ -1098,7 +1140,7 @@ CacheProcessor::cacheInitialized()
     CacheProcessor::cache_ready = caches_ready;
     Note("cache enabled");
 #ifdef CLUSTER_CACHE
-    if (!(start_internal_flags & PROCESSOR_RECONFIGURE)) {
+    if (!(start_internal_flags & (PROCESSOR_RECONFIGURE | PROCESSOR_CHECK))) {
       CacheContinuation::init();
       clusterProcessor.start();
     }

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/d0d0afb9/iocore/cache/CacheDir.cc
----------------------------------------------------------------------
diff --git a/iocore/cache/CacheDir.cc b/iocore/cache/CacheDir.cc
index 779c16b..24d0843 100644
--- a/iocore/cache/CacheDir.cc
+++ b/iocore/cache/CacheDir.cc
@@ -1247,6 +1247,15 @@ Ldone:
   goto Lrestart;
 }
 
+namespace {
+  int
+  compare_ushort(void const* a, void const* b)
+  {
+    return *static_cast<unsigned short const*>(a) - *static_cast<unsigned short const*>(b);
+  }
+}
+
+
 //
 // Check
 //
@@ -1254,75 +1263,127 @@ Ldone:
 #define HIST_DEPTH 8
 int Vol::dir_check(bool /* fix ATS_UNUSED */) // TODO: we should eliminate this parameter ?
 {
-  int hist[HIST_DEPTH + 1] = {0};
-  int *shist = (int *)ats_malloc(segments * sizeof(int));
-  memset(shist, 0, segments * sizeof(int));
+  static int const SEGMENT_HISTOGRAM_WIDTH = 16;
+  int hist[SEGMENT_HISTOGRAM_WIDTH + 1] = { 0 };
+  unsigned short chain_tag[MAX_ENTRIES_PER_SEGMENT];
+  int32_t chain_mark[MAX_ENTRIES_PER_SEGMENT];
+  uint64_t total_buckets = buckets * segments;
+  uint64_t total_entries = total_buckets * DIR_DEPTH;
+
   int j;
   int stale = 0, full = 0, empty = 0;
-  int last = 0, free = 0;
+  int free = 0, head = 0;
+
+  int max_chain_length = 0;
+  int64_t bytes_in_use = 0;
+
+  printf("Stripe '[%s]'\n", hash_text.get());
+  printf("  Directory Bytes: %" PRIu64 "\n", total_buckets * SIZEOF_DIR);
+  printf("  Segments:  %d\n", segments);
+  printf("  Buckets:   %" PRIu64 "\n", buckets);
+  printf("  Entries:   %" PRIu64 "\n", total_entries);
+
   for (int s = 0; s < segments; s++) {
     Dir *seg = dir_segment(s, this);
+    int seg_chain_max = 0;
+    int seg_empty = 0;
+    int seg_full = 0;
+    int seg_stale = 0;
+    int seg_bytes_in_use = 0;
+    int seg_dups = 0;
+
+    ink_zero(chain_tag);
+    memset(chain_mark, -1, sizeof(chain_mark));
+
     for (int b = 0; b < buckets; b++) {
-      int h = 0;
-      Dir *e = dir_bucket(b, seg);
-      while (e) {
-        if (!dir_offset(e))
-          empty++;
-        else {
-          h++;
-          if (!dir_valid(this, e))
-            stale++;
-          else
-            full++;
+      Dir *root = dir_bucket(b, seg);
+      int h = 0; // chain length starting in this bucket
+
+      // Walk the chain starting in this bucket
+      int chain_idx = 0;
+      int mark = 0;
+      for ( Dir* e = root ; e ; e = next_dir(e, seg) ) {
+        if (!dir_offset(e)) { // this should only happen on the first dir in a bucket
+          ++seg_empty;
+        } else {
+          int e_idx = e - seg;
+          ++h;
+          chain_tag[chain_idx++] = dir_tag(e);
+          if (chain_mark[e_idx] == mark) {
+            printf("    - Cycle of length %d detected for bucket %d\n", h, b);
+          } else if (chain_mark[e_idx] >= 0) {
+            printf("    - Entry %" PRIu64 " is in chain %d and %d", e - seg, chain_mark[e_idx], mark);
+          } else {
+            chain_mark[e_idx] = mark;
+          }
+
+          if (dir_head(e)) ++head;
+
+          if (!dir_valid(this, e)) {
+            ++seg_stale;
+          } else {
+            ++seg_full;
+            seg_bytes_in_use += dir_approx_size(e);
+          }
         }
         e = next_dir(e, seg);
         if (!e)
           break;
       }
-      if (h > HIST_DEPTH)
-        h = HIST_DEPTH;
-      hist[h]++;
+
+      // Check for duplicates (identical tags in the same bucket).
+      if (h > 1) {
+        unsigned short last;
+        qsort(chain_tag, h, sizeof(chain_tag[0]), &compare_ushort);
+        last = chain_tag[0];
+        for ( int k = 1 ; k < h ; ++k ) {
+          if (last == chain_tag[k]) ++seg_dups;
+          last = chain_tag[k];
+        }
+      }
+
+      ++hist[std::min(h, SEGMENT_HISTOGRAM_WIDTH)];
+      seg_chain_max = std::max(seg_chain_max, h);
     }
-    int t = stale + full;
-    shist[s] = t - last;
-    last = t;
+    int seg_chains = buckets - seg_empty;
+    full += seg_full;
+    empty += seg_empty;
+    stale += seg_stale;
     free += dir_freelist_length(this, s);
+    max_chain_length = std::max(max_chain_length, seg_chain_max);
+    bytes_in_use += seg_bytes_in_use;
+
+    printf("  - Segment-%d: full:%d stale:%d empty:%d bytes-used:%d chain-count:%d chain-max:%d chain-avg:%.2f chain-dups:%d\n"
+           , s, seg_full, seg_stale, seg_empty, seg_bytes_in_use
+           , seg_chains, seg_chain_max, seg_chains ? static_cast<float>(seg_full+seg_stale)/seg_chains : 0.0, seg_dups
+      );
   }
-  int total = buckets * segments * DIR_DEPTH;
-  printf("    Directory for [%s]\n", hash_text.get());
-  printf("        Bytes:     %d\n", total * SIZEOF_DIR);
-  printf("        Segments:  %" PRIu64 "\n", (uint64_t)segments);
-  printf("        Buckets:   %" PRIu64 "\n", (uint64_t)buckets);
-  printf("        Entries:   %d\n", total);
-  printf("        Full:      %d\n", full);
-  printf("        Empty:     %d\n", empty);
-  printf("        Stale:     %d\n", stale);
-  printf("        Free:      %d\n", free);
-  printf("        Bucket Fullness:   ");
-  for (j = 0; j < HIST_DEPTH; j++) {
-    printf("%8d ", hist[j]);
-    if ((j % 4 == 3))
-      printf("\n"
-             "                           ");
-  }
-  printf("\n");
-  printf("        Segment Fullness:  ");
-  for (j = 0; j < segments; j++) {
-    printf("%5d ", shist[j]);
-    if ((j % 5 == 4))
-      printf("\n"
-             "                           ");
-  }
-  printf("\n");
-  printf("        Freelist Fullness: ");
-  for (j = 0; j < segments; j++) {
-    printf("%5d ", dir_freelist_length(this, j));
-    if ((j % 5 == 4))
-      printf("\n"
-             "                           ");
-  }
+
+  int chain_count = total_buckets - empty;
+  printf("  - Stripe: full:%d stale:%d empty:%d free:%d chain-count:%d chain-max:%d chain-avg:%.2f\n"
+         , full, stale, empty, free, chain_count, max_chain_length
+         , chain_count ? static_cast<float>(full + stale)/chain_count : 0
+    );
+
+  printf("    Chain lengths:  ");
+  for (j = 0; j < SEGMENT_HISTOGRAM_WIDTH; ++j)
+    printf(" %d=%d ", j, hist[j]);
+  printf(" %d>=%d\n", SEGMENT_HISTOGRAM_WIDTH, hist[SEGMENT_HISTOGRAM_WIDTH]);
+
+  char tt[256];
+  printf("    Total Size:      %" PRIu64 "\n", (uint64_t)len);
+  printf("    Bytes in Use:    %" PRIu64 "\n", bytes_in_use);
+  printf("    Objects:         %d\n", head);
+  printf("    Average Size:    %" PRIu64 "\n", head ? (bytes_in_use/head) : 0);
+  printf("    Write Position:  %" PRIu64 "\n", (uint64_t) (header->write_pos - skip - start));
+  printf("    Phase:           %d\n", (int)!!header->phase);
+  ink_ctime_r(&header->create_time, tt);
+  tt[strlen(tt) - 1] = 0;
+  printf("    Create Time:     %s\n", tt);
+  printf("    Sync Serial:     %u\n", (unsigned int)header->sync_serial);
+  printf("    Write Serial:    %u\n", (unsigned int)header->write_serial);
   printf("\n");
-  ats_free(shist);
+
   return 0;
 }
 

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/d0d0afb9/iocore/cache/CacheDisk.cc
----------------------------------------------------------------------
diff --git a/iocore/cache/CacheDisk.cc b/iocore/cache/CacheDisk.cc
index 0372d8d..e11acec 100644
--- a/iocore/cache/CacheDisk.cc
+++ b/iocore/cache/CacheDisk.cc
@@ -57,8 +57,15 @@ CacheDisk::open(char *s, off_t blocks, off_t askip, int ahw_sector_size, int fil
   header = (DiskHeader *)ats_memalign(ats_pagesize(), header_len);
   memset(header, 0, header_len);
   if (clear) {
-    SET_HANDLER(&CacheDisk::clearDone);
-    return clearDisk();
+    if (read_only_p) {
+      fprintf(stderr, "Could not read disk header for disk %s", path);
+      SET_DISK_BAD(this);
+      SET_HANDLER(&CacheDisk::openDone);
+      return openDone(EVENT_IMMEDIATE, 0);
+    } else {
+      SET_HANDLER(&CacheDisk::clearDone);
+      return clearDisk();
+    }
   }
 
   SET_HANDLER(&CacheDisk::openStart);
@@ -142,8 +149,13 @@ CacheDisk::openStart(int event, void * /* data ATS_UNUSED */)
       // stripe isn't the short one, the split will be different this time.
       // Further - the size is encoded in to the disk hash so if the size changes, the data is effectively lost anyway.
       // So no space recovery.
-      //      if (header->num_diskvol_blks == 1)
-      //        header->vol_info[0].len += delta_3_2;
+//      if (header->num_diskvol_blks == 1)
+//        header->vol_info[0].len += delta_3_2;
+    } else if (read_only_p) {
+      fprintf(stderr, "Disk header is different than expected for disk %s", path);
+      SET_DISK_BAD(this);
+      SET_HANDLER(&CacheDisk::openDone);
+      return EVENT_DONE;
     } else {
       Warning("disk header different for disk %s: clearing the disk", path);
       SET_HANDLER(&CacheDisk::clearDone);

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/d0d0afb9/iocore/cache/I_Cache.h
----------------------------------------------------------------------
diff --git a/iocore/cache/I_Cache.h b/iocore/cache/I_Cache.h
index af99d30..857f312 100644
--- a/iocore/cache/I_Cache.h
+++ b/iocore/cache/I_Cache.h
@@ -152,8 +152,9 @@ struct CacheProcessor : public Processor {
   static volatile uint32_t cache_ready;
   static volatile int initialized;
   static volatile int start_done;
-  static int clear;
-  static int fix;
+  static bool clear;
+  static bool fix;
+  static bool check;
   static int start_internal_flags;
   static int auto_clear_flag;
 

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/d0d0afb9/iocore/cache/P_CacheDir.h
----------------------------------------------------------------------
diff --git a/iocore/cache/P_CacheDir.h b/iocore/cache/P_CacheDir.h
index 0a31c32..a4aef0c 100644
--- a/iocore/cache/P_CacheDir.h
+++ b/iocore/cache/P_CacheDir.h
@@ -44,6 +44,8 @@ struct CacheVC;
 
 #define MAX_DIR_SEGMENTS (32 * (1 << 16))
 #define DIR_DEPTH 4
+#define MAX_ENTRIES_PER_SEGMENT (1 << 16)
+#define MAX_BUCKETS_PER_SEGMENT (MAX_ENTRIES_PER_SEGMENT / DIR_DEPTH)
 #define DIR_SIZE_WIDTH 6
 #define DIR_BLOCK_SIZES 4
 #define DIR_BLOCK_SHIFT(_i) (3 * (_i))

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/d0d0afb9/iocore/cache/P_CacheDisk.h
----------------------------------------------------------------------
diff --git a/iocore/cache/P_CacheDisk.h b/iocore/cache/P_CacheDisk.h
index 8b51de2..7b470a7 100644
--- a/iocore/cache/P_CacheDisk.h
+++ b/iocore/cache/P_CacheDisk.h
@@ -96,6 +96,7 @@ struct CacheDisk : public Continuation {
   DiskVol *free_blocks;
   int num_errors;
   int cleared;
+  bool read_only_p;
 
   // Extra configuration values
   int forced_volume_num;           ///< Volume number for this disk.
@@ -103,7 +104,8 @@ struct CacheDisk : public Continuation {
 
   CacheDisk()
     : Continuation(new_ProxyMutex()), header(NULL), path(NULL), header_len(0), len(0), start(0), skip(0), num_usable_blocks(0),
-      fd(-1), free_space(0), wasted_space(0), disk_vols(NULL), free_blocks(NULL), num_errors(0), cleared(0), forced_volume_num(-1)
+      fd(-1), free_space(0), wasted_space(0), disk_vols(NULL), free_blocks(NULL), num_errors(0), cleared(0), read_only_p(false),
+      forced_volume_num(-1)
   {
   }
 

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/d0d0afb9/proxy/Main.cc
----------------------------------------------------------------------
diff --git a/proxy/Main.cc b/proxy/Main.cc
index ef93db0..98d6dba 100644
--- a/proxy/Main.cc
+++ b/proxy/Main.cc
@@ -132,6 +132,10 @@ int http_accept_file_descriptor = NO_FD;
 static char core_file[255] = "";
 static bool enable_core_file_p = false; // Enable core file dump?
 int command_flag = DEFAULT_COMMAND_FLAG;
+int command_index = -1;
+bool command_valid = false;
+// Commands that have special processing / requirements.
+static char const *CMD_VERIFY_CONFIG = "verify_config";
 #if TS_HAS_TESTS
 static char regression_test[1024] = "";
 #endif
@@ -546,64 +550,39 @@ CB_After_Cache_Init()
   }
 }
 
-struct CmdCacheCont : public Continuation {
-  int cache_fix;
-
-  int
-  ClearEvent(int event, Event *e)
-  {
-    (void)event;
-    (void)e;
-    if (cacheProcessor.IsCacheEnabled() == CACHE_INITIALIZED) {
-      Note("CLEAR, succeeded");
-      _exit(0);
-    } else if (cacheProcessor.IsCacheEnabled() == CACHE_INIT_FAILED) {
-      Note("unable to open Cache, CLEAR failed");
-      _exit(1);
-    }
-    return EVENT_CONT;
+void
+CB_cmd_cache_clear()
+{
+  if (cacheProcessor.IsCacheEnabled() == CACHE_INITIALIZED) {
+    Note("CLEAR, succeeded");
+    _exit(0);
+  } else if (cacheProcessor.IsCacheEnabled() == CACHE_INIT_FAILED) {
+    Note("unable to open Cache, CLEAR failed");
+    _exit(1);
   }
+}
 
-  int
-  CheckEvent(int event, Event *e)
-  {
-    (void)event;
-    (void)e;
-    int res = 0;
-    Note("Cache Directory");
-    if (cacheProcessor.IsCacheEnabled() == CACHE_INITIALIZED) {
-      res = cacheProcessor.dir_check(cache_fix) < 0 || res;
-
-      Note("Cache");
-      res = cacheProcessor.db_check(cache_fix) < 0 || res;
-
-      cacheProcessor.stop();
-
-      const char *n = cache_fix ? "REPAIR" : "CHECK";
-
-      if (res) {
-        printf("\n%s failed", n);
-        _exit(1);
-      } else {
-        printf("\n%s succeeded\n", n);
-        _exit(0);
-      }
-    } else if (cacheProcessor.IsCacheEnabled() == CACHE_INIT_FAILED) {
-      Note("unable to open Cache, Check failed");
+void
+CB_cmd_cache_check()
+{
+  int res = 0;
+  if (cacheProcessor.IsCacheEnabled() == CACHE_INITIALIZED) {
+    res = cacheProcessor.dir_check(false) < 0 || res;
+    cacheProcessor.stop();
+    const char *n = "CHECK";
+
+    if (res) {
+      printf("\n%s failed", n);
       _exit(1);
+    } else {
+      printf("\n%s succeeded\n", n);
+      _exit(0);
     }
-    return EVENT_CONT;
-  }
-
-  CmdCacheCont(bool check, bool fix = false) : Continuation(new_ProxyMutex())
-  {
-    cache_fix = fix;
-    if (check)
-      SET_HANDLER(&CmdCacheCont::CheckEvent);
-    else
-      SET_HANDLER(&CmdCacheCont::ClearEvent);
+  } else if (cacheProcessor.IsCacheEnabled() == CACHE_INIT_FAILED) {
+    Note("unable to open Cache, Check failed");
+    _exit(1);
   }
-};
+}
 
 static int
 cmd_check_internal(char * /* cmd ATS_UNUSED */, bool fix = false)
@@ -614,6 +593,7 @@ cmd_check_internal(char * /* cmd ATS_UNUSED */, bool fix = false)
 
   hostdb_current_interval = (ink_get_based_hrtime() / HRTIME_MINUTE);
 
+#if 0
   printf("Host Database\n");
   HostDBCache hd;
   if (hd.start(fix) < 0) {
@@ -622,13 +602,13 @@ cmd_check_internal(char * /* cmd ATS_UNUSED */, bool fix = false)
   }
   hd.check("hostdb.config", fix);
   hd.reset();
+#endif
 
-  if (cacheProcessor.start() < 0) {
+  cacheProcessor.set_after_init_callback(&CB_cmd_cache_check);
+  if (cacheProcessor.start_internal(PROCESSOR_CHECK) < 0) {
     printf("\nbad cache configuration, %s failed\n", n);
     return CMD_FAILED;
   }
-  eventProcessor.schedule_every(new CmdCacheCont(true, fix), HRTIME_SECONDS(1));
-
   return CMD_IN_PROGRESS;
 }
 
@@ -689,11 +669,11 @@ cmd_clear(char *cmd)
   if (c_all || c_cache) {
     Note("Clearing Cache");
 
+    cacheProcessor.set_after_init_callback(&CB_cmd_cache_clear);
     if (cacheProcessor.start_internal(PROCESSOR_RECONFIGURE) < 0) {
       Note("unable to open Cache, CLEAR failed");
       return CMD_FAILED;
     }
-    eventProcessor.schedule_every(new CmdCacheCont(false), HRTIME_SECONDS(1));
     return CMD_IN_PROGRESS;
   }
 
@@ -770,6 +750,7 @@ static const struct CMD {
   const char *d; // description (part of a line)
   const char *h; // help string (multi-line)
   int (*f)(char *);
+  bool no_process_lock; /// If set this command doesn't need a process level lock.
 } commands[] = {
   {"list", "List cache configuration", "LIST\n"
                                        "\n"
@@ -786,7 +767,7 @@ static const struct CMD {
                                                          "CHECK does not make any changes to the data stored in\n"
                                                          "the cache. CHECK requires a scan of the contents of the\n"
                                                          "cache and may take a long time for large caches.\n",
-   cmd_check},
+   cmd_check, true},
   {"clear", "Clear the entire cache", "CLEAR\n"
                                       "\n"
                                       "FORMAT: clear\n"
@@ -810,12 +791,12 @@ static const struct CMD {
                                              "Clear the entire hostdb cache.  All host name resolution\n"
                                              "information is lost.\n",
    cmd_clear},
-  {"verify_config", "Verify the config", "\n"
-                                         "\n"
-                                         "FORMAT: verify_config\n"
-                                         "\n"
-                                         "Load the config and verify traffic_server comes up correctly. \n",
-   cmd_verify},
+  {CMD_VERIFY_CONFIG, "Verify the config", "\n"
+                                           "\n"
+                                           "FORMAT: verify_config\n"
+                                           "\n"
+                                           "Load the config and verify traffic_server comes up correctly. \n",
+   cmd_verify, true},
   {"help", "Obtain a short description of a command (e.g. 'help clear')", "HELP\n"
                                                                           "\n"
                                                                           "FORMAT: help [command_name]\n"
@@ -828,14 +809,14 @@ static const struct CMD {
 };
 
 static int
-cmd_index(char *p)
+find_cmd_index(char const *p)
 {
   p += strspn(p, " \t");
   for (unsigned c = 0; c < countof(commands); c++) {
-    const char *l = commands[c].n;
+    char const *l = commands[c].n;
     while (l) {
-      const char *s = strchr(l, '/');
-      char *e = strpbrk(p, " \t\n");
+      char const *s = strchr(l, '/');
+      char const *e = strpbrk(p, " \t\n");
       int len = s ? s - l : strlen(l);
       int lenp = e ? e - p : strlen(p);
       if ((len == lenp) && !strncasecmp(p, l, len))
@@ -858,7 +839,7 @@ cmd_help(char *cmd)
     }
   } else {
     int i;
-    if ((i = cmd_index(cmd)) < 0) {
+    if ((i = find_cmd_index(cmd)) < 0) {
       printf("\nno help found for: %s\n", cmd);
       return CMD_FAILED;
     }
@@ -892,14 +873,11 @@ check_fd_limit()
 static int
 cmd_mode()
 {
-  if (*command_string) {
-    int c = cmd_index(command_string);
-    if (c >= 0) {
-      return commands[c].f(command_string);
-    } else {
-      Warning("unrecognized command: '%s'", command_string);
-      return CMD_FAILED; // in error
-    }
+  if (command_index >= 0) {
+    return commands[command_index].f(command_string);
+  } else if (*command_string) {
+    Warning("unrecognized command: '%s'", command_string);
+    return CMD_FAILED; // in error
   } else {
     printf("\n");
     printf("WARNING\n");
@@ -1429,6 +1407,14 @@ main(int /* argc ATS_UNUSED */, const char **argv)
 
   process_args(&appVersionInfo, argument_descriptions, countof(argument_descriptions), argv);
   command_flag = command_flag || *command_string;
+  command_index = find_cmd_index(command_string);
+  command_valid = command_flag && command_index >= 0;
+
+  // Specific validity checks.
+  if (*conf_dir && command_index != find_cmd_index(CMD_VERIFY_CONFIG)) {
+    fprintf(stderr, "-D option can only be used with the %s command\n", CMD_VERIFY_CONFIG);
+    _exit(1);
+  }
 
   // Set stdout/stdin to be unbuffered
   setbuf(stdout, NULL);
@@ -1456,16 +1442,10 @@ main(int /* argc ATS_UNUSED */, const char **argv)
   // Local process manager
   initialize_process_manager();
 
-  if ((*command_string) && (cmd_index(command_string) == cmd_index((char *)"verify_config"))) {
-    fprintf(stderr, "\n\n skip lock check for %s \n\n", command_string);
-  } else {
-    if (*conf_dir) {
-      fprintf(stderr, "-D option should be used with -Cverify_config\n");
-      _exit(0);
-    }
-    // Ensure only one copy of traffic server is running
+  // Ensure only one copy of traffic server is running, unless it's a command
+  // that doesn't require a lock.
+  if (!(command_valid && commands[command_index].no_process_lock))
     check_lockfile();
-  }
 
   // Set the core limit for the process
   init_core_size();