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/07/19 16:14:46 UTC

[6/8] trafficserver git commit: TS-974: Partial Object Caching.

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/1c06db83/iocore/cache/P_CacheHttp.h
----------------------------------------------------------------------
diff --git a/iocore/cache/P_CacheHttp.h b/iocore/cache/P_CacheHttp.h
index c5bf87e..ea0ac43 100644
--- a/iocore/cache/P_CacheHttp.h
+++ b/iocore/cache/P_CacheHttp.h
@@ -34,7 +34,6 @@ typedef URL CacheURL;
 typedef HTTPHdr CacheHTTPHdr;
 typedef HTTPInfo CacheHTTPInfo;
 
-
 #define OFFSET_BITS 24
 enum {
   OWNER_NONE = 0,
@@ -48,11 +47,57 @@ struct CacheHTTPInfo {
 
 #endif // HTTP_CACHE
 
-struct vec_info {
-  CacheHTTPInfo alternate;
-};
+LINK_FORWARD_DECLARATION(CacheVC, OpenDir_Link) // forward declaration
+LINK_FORWARD_DECLARATION(CacheVC, Active_Link)  // forward declaration
 
 struct CacheHTTPInfoVector {
+  typedef CacheHTTPInfoVector self; ///< Self reference type.
+
+  struct Item {
+    /// Descriptor for an alternate for this object.
+    CacheHTTPInfo _alternate;
+    /// CacheVCs which are writing data to this alternate.
+    DLL<CacheVC, Link_CacheVC_OpenDir_Link> _writers;
+    ///@{ Active I/O
+    /** These two lists tracks active / outstanding I/O operations on The @a _active list is for writers
+        and the CacheVC should be on this list iff it has initiated an I/O that has not yet
+        completed. The @a _waiting list is for CacheVCs that are waiting for a fragment that is being written
+        by a CacheVC on the @a _active list. That is, it is waiting on the same I/O operation as an @a _active
+        CacheVC.
+
+        @internal An alternative implementation would be to have an array with an element for each fragment. With
+        this scheme we will have to linear search these lists to find the corresponding fragment I/O if any.
+        However, these lists should be short (only very rarely more than 1 or 2) and an array, given the ever
+        larger objects to be stored, would be large and require allocation. For these reasons I think this is the
+        better choice.
+    */
+    /// CacheVCs with pending write I/O.
+    DLL<CacheVC, Link_CacheVC_Active_Link> _active;
+    /// CacheVCs waiting on fragments.
+    DLL<CacheVC, Link_CacheVC_Active_Link> _waiting;
+    // To minimize list walking, we track the convex hull of fragments for which readers are waiting.
+    // We update the values whenever we must actually walk the list.
+    // Otherwise we maintain the convex hull invariant so if a written fragment is outside the range,
+    // we can assume no reader was waiting for it.
+    /// lowest fragment index for which a reader is waiting.
+    int _wait_idx_min;
+    /// highest fragment inddex for which a reader is waiting.
+    int _wait_idx_max;
+    /// Flag
+    union {
+      uint16_t _flags;
+      struct {
+        unsigned int dirty : 1;
+      } f;
+    };
+    ///@}
+    /// Check if there are any writers.
+    /// @internal Need to augment this at some point to check for writers to a specific offset.
+    bool has_writers() const;
+  };
+
+  typedef CacheArray<Item> InfoVector;
+
   void *magic;
 
   CacheHTTPInfoVector();
@@ -63,6 +108,7 @@ struct CacheHTTPInfoVector {
   {
     return xcount;
   }
+
   int insert(CacheHTTPInfo *info, int id = -1);
   CacheHTTPInfo *get(int idx);
   void detach(int idx, CacheHTTPInfo *r);
@@ -79,19 +125,246 @@ struct CacheHTTPInfoVector {
   int marshal_length();
   int marshal(char *buf, int length);
   uint32_t 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;
+  /// Get the alternate index for the @a key.
+  int index_of(CacheKey const &key);
+  /// Check if there are any writers for the alternate of @a alt_key.
+  bool has_writer(CacheKey const &alt_key);
+  /// Mark a @c CacheVC as actively writing at @a offset on the alternate with @a alt_key.
+  self &write_active(CacheKey const &alt_key, CacheVC *vc, int64_t offset);
+  /// Mark an active write by @a vc as complete and indicate whether it had @a success.
+  /// If the write is not @a success then the fragment is not marked as cached.
+  self &write_complete(CacheKey const &alt_key, CacheVC *vc, bool success = true);
+  /// Indicate if a VC is currently writing to the fragment with this @a offset.
+  bool is_write_active(CacheKey const &alt_key, int64_t offset);
+  /// Mark a CacheVC as waiting for the fragment containing the byte at @a offset.
+  /// @return @c false if there is no writer scheduled to write that offset.
+  bool wait_for(CacheKey const &alt_key, CacheVC *vc, int64_t offset);
+  /// Get the fragment key for a specific @a offset.
+  CacheKey const &key_for(CacheKey const &alt_key, int64_t offset);
+  /// Close out anything related to this writer
+  self &close_writer(CacheKey const &alt_key, CacheVC *vc);
+  /** Compute the convex hull of the uncached parts of the @a request taking current writers in to account.
+
+      @return @c true if there is uncached data that must be retrieved.
+   */
+  HTTPRangeSpec::Range get_uncached_hull(CacheKey const &alt_key, HTTPRangeSpec const &request, int64_t initial);
+
+  /** Sigh, yet another custom array class.
+      @c Vec doesn't work because it really only works well with pointers, not objects.
+  */
+  InfoVector data;
+
   int xcount;
   Ptr<RefCountObj> vector_buf;
 };
 
+/** Range operation tracking.
+
+    This holds a range specification. It also tracks the current object offset and the individual range.
+
+    For simplification of the logic that uses this class it will pretend to be a single range of
+    the object size if it is empty. To return the correct response we still need to distinuish
+    those two cases.
+*/
+class CacheRange
+{
+public:
+  typedef CacheRange self; ///< Self reference type.
+
+  /// Default constructor
+  CacheRange() : _offset(0), _idx(-1), _ct_field(NULL), _pending_range_shift_p(false) {}
+
+  /// Test if the range spec has actual ranges in it
+  bool hasRanges() const;
+
+  /// Test for multiple ranges.
+  bool isMulti() const;
+
+  /// Get the current object offset
+  uint64_t getOffset() const;
+
+  /// Get the current range index.
+  int getIdx() const;
+
+  /// Get the number of ranges.
+  size_t count() const;
+
+  /// Get the remaining contiguous bytes for the current range.
+  uint64_t getRemnantSize() const;
+
+  /** Advance @a size bytes in the range spec.
+
+      @return The resulting offset in the object.
+  */
+  uint64_t consume(uint64_t size);
+
+  /** Initialize from a request header.
+   */
+  bool init(HTTPHdr *req);
+
+  /** Set the range to the start of the range set.
+      @return @c true if there is a valid range, @c false otherwise.
+  */
+  bool start();
+
+  /** Apply a content @a len to the ranges.
+
+      @return @c true if successfully applied, @c false otherwise.
+  */
+  bool apply(uint64_t len);
+
+  /** Get the range boundary string.
+      @a len if not @c NULL receives the length of the string.
+  */
+  char const *getBoundaryStr(int *len) const;
+
+  /** Generate the range boundary string */
+  self &generateBoundaryStr(CacheKey const &key);
+
+  /// Get the cached Content-Type field.
+  MIMEField *getContentTypeField() const;
+
+  /// Set the Content-Type field from a response header.
+  self &setContentTypeFromResponse(HTTPHdr *resp);
+
+  /** Calculate the effective HTTP content length value.
+   */
+  uint64_t calcContentLength() const;
+
+  /// Raw access to internal range spec.
+  HTTPRangeSpec &getRangeSpec();
+
+  /// Test if a consume moved across a range boundary.
+  bool hasPendingRangeShift() const;
+
+  /// Clear the pending range shift flag.
+  self &consumeRangeShift();
+
+  /// Range access.
+  HTTPRangeSpec::Range &operator[](int n);
+
+  /// Range access.
+  HTTPRangeSpec::Range const &operator[](int n) const;
+
+  /// Reset to re-usable state.
+  void clear();
+
+protected:
+  uint64_t _len;        ///< Total object length.
+  uint64_t _offset;     ///< Offset in content.
+  int _idx;             ///< Current range index. (< 0 means not in a range)
+  HTTPRangeSpec _r;     ///< The actual ranges.
+  MIMEField *_ct_field; ///< Content-Type field.
+  char _boundary[HTTP_RANGE_BOUNDARY_LEN];
+  bool _pending_range_shift_p;
+};
+
+TS_INLINE bool
+CacheHTTPInfoVector::Item::has_writers() const
+{
+  return NULL != _writers.head;
+}
+
 TS_INLINE CacheHTTPInfo *
 CacheHTTPInfoVector::get(int idx)
 {
   ink_assert(idx >= 0);
   ink_assert(idx < xcount);
-  return &data[idx].alternate;
+  return &data[idx]._alternate;
+}
+
+inline bool
+CacheRange::hasRanges() const
+{
+  return _r.isSingle() || _r.isMulti();
+}
+
+inline uint64_t
+CacheRange::getOffset() const
+{
+  return _offset;
+}
+
+inline int
+CacheRange::getIdx() const
+{
+  return _idx;
+}
+
+inline uint64_t
+CacheRange::getRemnantSize() const
+{
+  uint64_t zret = 0;
+
+  if (_r.isEmpty())
+    zret = _len - _offset;
+  else if (_r.isValid() && 0 <= _idx && _idx < static_cast<int>(_r.count()))
+    zret = (_r[_idx]._max - _offset) + 1;
+
+  return zret;
+}
+
+inline char const *
+CacheRange::getBoundaryStr(int *len) const
+{
+  if (len)
+    *len = sizeof(_boundary);
+  return _boundary;
+}
+
+inline HTTPRangeSpec &
+CacheRange::getRangeSpec()
+{
+  return _r;
+}
+
+inline bool
+CacheRange::isMulti() const
+{
+  return _r.isMulti();
+}
+
+inline bool
+CacheRange::hasPendingRangeShift() const
+{
+  return _pending_range_shift_p;
+}
+
+inline CacheRange &
+CacheRange::consumeRangeShift()
+{
+  _pending_range_shift_p = false;
+  return *this;
+}
+
+inline MIMEField *
+CacheRange::getContentTypeField() const
+{
+  return _ct_field;
+}
+
+inline size_t
+CacheRange::count() const
+{
+  return _r.count();
+}
+
+inline HTTPRangeSpec::Range &CacheRange::operator[](int n)
+{
+  return _r[n];
+}
+
+inline HTTPRangeSpec::Range const &CacheRange::operator[](int n) const
+{
+  return _r[n];
+}
+
+inline CacheRange &
+CacheRange::setContentTypeFromResponse(HTTPHdr *resp)
+{
+  _ct_field = resp->field_find(MIME_FIELD_CONTENT_TYPE, MIME_LEN_CONTENT_TYPE);
+  return *this;
 }
 
 #endif /* __CACHE_HTTP_H__ */

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/1c06db83/iocore/cache/P_CacheInternal.h
----------------------------------------------------------------------
diff --git a/iocore/cache/P_CacheInternal.h b/iocore/cache/P_CacheInternal.h
index 28cb44f..e215903 100644
--- a/iocore/cache/P_CacheInternal.h
+++ b/iocore/cache/P_CacheInternal.h
@@ -225,6 +225,10 @@ extern int cache_config_read_while_writer_max_retries;
 
 // CacheVC
 struct CacheVC : public CacheVConnection {
+  typedef CacheVC self;                                                  ///< Self reference type.
+  typedef HTTPCacheAlt::FragmentDescriptor FragmentDescriptor;           ///< Import type.
+  typedef HTTPCacheAlt::FragmentDescriptorTable FragmentDescriptorTable; ///< Import type.
+
   CacheVC();
 
   VIO *do_io_read(Continuation *c, int64_t nbytes, MIOBuffer *buf);
@@ -274,7 +278,9 @@ struct CacheVC : public CacheVConnection {
     return -1;
   }
 
-  bool writer_done();
+  Action *do_write_init();
+
+  //  bool writer_done();
   int calluser(int event);
   int callcont(int event);
   int die();
@@ -294,6 +300,7 @@ struct CacheVC : public CacheVConnection {
   int openReadReadDone(int event, Event *e);
   int openReadMain(int event, Event *e);
   int openReadStartEarliest(int event, Event *e);
+  int openReadWaitEarliest(int evid, Event *e);
 #ifdef HTTP_CACHE
   int openReadVecWrite(int event, Event *e);
 #endif
@@ -301,7 +308,7 @@ struct CacheVC : public CacheVConnection {
   int openReadFromWriter(int event, Event *e);
   int openReadFromWriterMain(int event, Event *e);
   int openReadFromWriterFailure(int event, Event *);
-  int openReadChooseWriter(int event, Event *e);
+  //  int openReadChooseWriter(int event, Event *e);
 
   int openWriteCloseDir(int event, Event *e);
   int openWriteCloseHeadDone(int event, Event *e);
@@ -311,13 +318,18 @@ struct CacheVC : public CacheVConnection {
   int openWriteRemoveVector(int event, Event *e);
   int openWriteWriteDone(int event, Event *e);
   int openWriteOverwrite(int event, Event *e);
+  int openWriteInit(int event, Event *e);
   int openWriteMain(int event, Event *e);
   int openWriteStartDone(int event, Event *e);
   int openWriteStartBegin(int event, Event *e);
+  int openWriteEmptyEarliestDone(int event, Event *e);
 
   int updateVector(int event, Event *e);
   int updateReadDone(int event, Event *e);
   int updateVecWrite(int event, Event *e);
+  int updateWriteStateFromRange();
+
+  int closeReadAndFree(int event, Event *e);
 
   int removeEvent(int event, Event *e);
 
@@ -357,12 +369,35 @@ struct CacheVC : public CacheVConnection {
       @return The address of the start of the fragment table,
       or @c NULL if there is no fragment table.
   */
-  virtual HTTPInfo::FragOffset *get_frag_table();
+  virtual HTTPInfo::FragmentDescriptorTable *get_frag_table();
   /** Load alt pointers and do fixups if needed.
       @return Length of header data used for alternates.
    */
   virtual uint32_t load_http_info(CacheHTTPInfoVector *info, struct Doc *doc, RefCountObj *block_ptr = NULL);
+
+  /// Change member @a key to be the key for the @a idx 'th fragment.
+  void update_key_to_frag_idx(int idx);
+  /// Compute the index of the fragment that contains the byte at content location @a offset.
+  int frag_idx_for_offset(uint64_t offset);
+
+  virtual char const *get_http_range_boundary_string(int *len) const;
+  virtual int64_t get_effective_content_size();
+  virtual void set_full_content_length(int64_t size);
+  virtual bool get_uncached(HTTPRangeSpec const &req, HTTPRangeSpec &result, int64_t initial);
+  /** This sets a range for data flowing in to the cache VC.
+      The CacheVC will write the incoming data to this part of the overall object.
+      @internal It's done this way to isolate the CacheVC from parsing range separators
+      in multi-range responses.
+  */
+  virtual int64_t set_inbound_range(int64_t min, int64_t max);
+  /** Select the ranges to apply to the content.
+      @internal In this case the CacheVC has to know the entire set of ranges so it can correctly
+      compute the actual output size (vs. the content size).
+  */
+  virtual void set_content_range(HTTPRangeSpec const &range);
+
 #endif
+
   virtual bool is_pread_capable();
   virtual bool set_pin_in_cache(time_t time_pin);
   virtual time_t get_pin_in_cache();
@@ -386,6 +421,9 @@ struct CacheVC : public CacheVConnection {
   // before being used by the CacheVC
   CacheKey key, first_key, earliest_key, update_key;
   Dir dir, earliest_dir, overwrite_dir, first_dir;
+  /// Thread to use to wake up this VC. Set when the VC puts itself on a wait list.
+  /// The waker should schedule @c EVENT_IMMEDIATE on this thread to wake up this VC.
+  EThread *wake_up_thread;
   // end Region A
 
   // Start Region B
@@ -405,11 +443,14 @@ struct CacheVC : public CacheVConnection {
 
   OpenDirEntry *od;
   AIOCallbackInternal io;
-  int alternate_index; // preferred position in vector
-  LINK(CacheVC, opendir_link);
+  int alternate_index;         // preferred position in vector
+  LINK(CacheVC, OpenDir_Link); ///< Reader/writer link per alternate in @c OpenDir.
+  LINK(CacheVC, Active_Link);  ///< Active I/O pending list in @c OpenDir.
 #ifdef CACHE_STAT_PAGES
   LINK(CacheVC, stat_link);
 #endif
+  CacheRange resp_range; ///< Tracking information for range data for response.
+  //  CacheRange uncached_range;      ///< The ranges in the request that are not in cache.
   // end Region B
 
   // Start Region C
@@ -451,6 +492,12 @@ struct CacheVC : public CacheVConnection {
   uint64_t total_len;    // total length written and available to write
   uint64_t doc_len;      // total_length (of the selected alternate for HTTP)
   uint64_t update_len;
+  HTTPRangeSpec::Range write_range; ///< Object based range for incoming partial content.
+  /// The offset in the content of the first byte beyond the end of the current fragment.
+  /// @internal This seems very weird but I couldn't figure out how to keep the more sensible
+  /// lower bound correctly updated.
+  /// The lower bound can can computed by subtracting doc->len from this value.
+  uint64_t frag_upper_bound;
   int fragment;
   int scan_msec_delay;
   CacheVC *write_vc;
@@ -522,7 +569,10 @@ extern CacheSync *cacheDirSync;
 // Function Prototypes
 #ifdef HTTP_CACHE
 int cache_write(CacheVC *, CacheHTTPInfoVector *);
-int get_alternate_index(CacheHTTPInfoVector *cache_vector, CacheKey key);
+/// Get the index for the alternate indentified by @a key in @a cache_vector.
+/// @a idx is a hint - that index is checked first and if not there the vector is scanned.
+/// This makes repeated access faster if the vector is not being updated.
+int get_alternate_index(CacheHTTPInfoVector *cache_vector, CacheKey key, int idx = -1);
 #endif
 CacheVC *new_DocEvacuator(int nbytes, Vol *d);
 
@@ -598,6 +648,7 @@ free_CacheVC(CacheVC *cont)
   cont->alternate_index = CACHE_ALT_INDEX_DEFAULT;
   if (cont->scan_vol_map)
     ats_free(cont->scan_vol_map);
+  cont->resp_range.clear();
   memset((char *)&cont->vio, 0, cont->size_to_init);
 #ifdef CACHE_STAT_PAGES
   ink_assert(!cont->stat_link.next && !cont->stat_link.prev);
@@ -723,6 +774,7 @@ CacheVC::do_write_lock_call()
   return handleWriteLock(EVENT_CALL, 0);
 }
 
+#if 0
 TS_INLINE bool
 CacheVC::writer_done()
 {
@@ -740,8 +792,9 @@ CacheVC::writer_done()
     return true;
   return false;
 }
+#endif
 
-TS_INLINE int
+TS_INLINE void
 Vol::close_write(CacheVC *cont)
 {
 #ifdef CACHE_STAT_PAGES
@@ -749,12 +802,12 @@ Vol::close_write(CacheVC *cont)
   stat_cache_vcs.remove(cont, cont->stat_link);
   ink_assert(!cont->stat_link.next && !cont->stat_link.prev);
 #endif
-  return open_dir.close_write(cont);
+  open_dir.close_entry(cont);
 }
 
 // Returns 0 on success or a positive error code on failure
 TS_INLINE int
-Vol::open_write(CacheVC *cont, int allow_if_writers, int max_writers)
+Vol::open_write(CacheVC *cont)
 {
   Vol *vol = this;
   bool agg_error = false;
@@ -768,7 +821,9 @@ Vol::open_write(CacheVC *cont, int allow_if_writers, int max_writers)
     CACHE_INCREMENT_DYN_STAT(cache_write_backlog_failure_stat);
     return ECACHE_WRITE_FAIL;
   }
-  if (open_dir.open_write(cont, allow_if_writers, max_writers)) {
+  ink_assert(NULL == cont->od);
+  if (NULL != (cont->od = open_dir.open_entry(this, cont->first_key, true))) {
+    cont->write_vector = &cont->od->vector;
 #ifdef CACHE_STAT_PAGES
     ink_assert(cont->mutex->thread_holding == this_ethread());
     ink_assert(!cont->stat_link.next && !cont->stat_link.prev);
@@ -786,26 +841,23 @@ Vol::close_write_lock(CacheVC *cont)
   CACHE_TRY_LOCK(lock, mutex, t);
   if (!lock.is_locked())
     return -1;
-  return close_write(cont);
+  this->close_write(cont);
+  return 0;
 }
 
 TS_INLINE int
-Vol::open_write_lock(CacheVC *cont, int allow_if_writers, int max_writers)
+Vol::open_write_lock(CacheVC *cont)
 {
   EThread *t = cont->mutex->thread_holding;
   CACHE_TRY_LOCK(lock, mutex, t);
-  if (!lock.is_locked())
-    return -1;
-  return open_write(cont, allow_if_writers, max_writers);
+  return lock.is_locked() ? this->open_write(cont) : -1;
 }
 
 TS_INLINE OpenDirEntry *
 Vol::open_read_lock(INK_MD5 *key, EThread *t)
 {
   CACHE_TRY_LOCK(lock, mutex, t);
-  if (!lock.is_locked())
-    return NULL;
-  return open_dir.open_read(key);
+  return lock.is_locked() ? open_dir.open_entry(this, *key, false) : NULL;
 }
 
 TS_INLINE int
@@ -871,6 +923,18 @@ rand_CacheKey(CacheKey *next_key, ProxyMutex *mutex)
   next_key->b[1] = mutex->thread_holding->generator.random();
 }
 
+#if 1
+void TS_INLINE
+next_CacheKey(CacheKey *next_key, CacheKey *key)
+{
+  next_key->next(*key);
+}
+void TS_INLINE
+prev_CacheKey(CacheKey *prev_key, CacheKey *key)
+{
+  prev_key->prev(*key);
+}
+#else
 extern uint8_t CacheKey_next_table[];
 void TS_INLINE
 next_CacheKey(CacheKey *next_key, CacheKey *key)
@@ -891,6 +955,7 @@ prev_CacheKey(CacheKey *prev_key, CacheKey *key)
     b[i] = 256 + CacheKey_prev_table[k[i]] - k[i - 1];
   b[0] = CacheKey_prev_table[k[0]];
 }
+#endif
 
 TS_INLINE unsigned int
 next_rand(unsigned int *p)
@@ -959,6 +1024,7 @@ struct Cache {
 
   Action *lookup(Continuation *cont, const CacheKey *key, CacheFragType type, const char *hostname, int host_len);
   inkcoreapi Action *open_read(Continuation *cont, const CacheKey *key, CacheFragType type, const char *hostname, int len);
+  inkcoreapi Action *open_read(Continuation *cont, CacheVConnection* writer, HTTPHdr* client_request);
   inkcoreapi Action *open_write(Continuation *cont, const CacheKey *key, CacheFragType frag_type, int options = 0,
                                 time_t pin_in_cache = (time_t)0, const char *hostname = 0, int host_len = 0);
   inkcoreapi Action *remove(Continuation *cont, const CacheKey *key, CacheFragType type = CACHE_FRAG_TYPE_HTTP,
@@ -1033,6 +1099,7 @@ cache_hash(const INK_MD5 &md5)
 #include "P_ClusterInline.h"
 #endif
 
-LINK_DEFINITION(CacheVC, opendir_link)
+LINK_DEFINITION(CacheVC, OpenDir_Link)
+LINK_DEFINITION(CacheVC, Active_Link)
 
 #endif /* _P_CACHE_INTERNAL_H__ */

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/1c06db83/iocore/cache/P_CacheVol.h
----------------------------------------------------------------------
diff --git a/iocore/cache/P_CacheVol.h b/iocore/cache/P_CacheVol.h
index 10eddc6..5b5e530 100644
--- a/iocore/cache/P_CacheVol.h
+++ b/iocore/cache/P_CacheVol.h
@@ -184,10 +184,10 @@ struct Vol : public Continuation {
 
   int recover_data();
 
-  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 open_write(CacheVC *cont);
+  int open_write_lock(CacheVC *cont);
+  void close_write(CacheVC *cont);
+  int close_write_lock(CacheVC *cont); // can fail lock
   int begin_read(CacheVC *cont);
   int begin_read_lock(CacheVC *cont);
   // unused read-write interlock code
@@ -482,7 +482,7 @@ free_EvacuationBlock(EvacuationBlock *b, EThread *t)
 TS_INLINE OpenDirEntry *
 Vol::open_read(const CryptoHash *key)
 {
-  return open_dir.open_read(key);
+  return open_dir.open_entry(this, *key, false);
 }
 
 TS_INLINE int

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/1c06db83/iocore/cluster/ClusterCache.cc
----------------------------------------------------------------------
diff --git a/iocore/cluster/ClusterCache.cc b/iocore/cluster/ClusterCache.cc
index c4480e1..534703d 100644
--- a/iocore/cluster/ClusterCache.cc
+++ b/iocore/cluster/ClusterCache.cc
@@ -2624,7 +2624,7 @@ CacheContinuation::getObjectSize(VConnection *vc, int opcode, CacheHTTPInfo *ret
     } else {
       new_ci.object_size_set(object_size);
     }
-    new_ci.m_alt->m_writeable = 1;
+    new_ci.m_alt->m_flag.writeable_p = true;
     ret_ci->copy_shallow(&new_ci);
   }
   ink_release_assert(object_size);

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/1c06db83/iocore/cluster/ClusterVConnection.cc
----------------------------------------------------------------------
diff --git a/iocore/cluster/ClusterVConnection.cc b/iocore/cluster/ClusterVConnection.cc
index 7a3185e..d831b39 100644
--- a/iocore/cluster/ClusterVConnection.cc
+++ b/iocore/cluster/ClusterVConnection.cc
@@ -631,4 +631,5 @@ ClusterVConnection::get_disk_io_priority()
   return disk_io_priority;
 }
 
+
 // End of ClusterVConnection.cc

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/1c06db83/iocore/cluster/P_Cluster.h
----------------------------------------------------------------------
diff --git a/iocore/cluster/P_Cluster.h b/iocore/cluster/P_Cluster.h
index 0376f63..b82f2e0 100644
--- a/iocore/cluster/P_Cluster.h
+++ b/iocore/cluster/P_Cluster.h
@@ -46,7 +46,7 @@
 #include "P_ClusterLoadMonitor.h"
 #include "P_TimeTrace.h"
 
-
+#if 0 // defined in InkErrno.h
 #define ECLUSTER_NO_VC (CLUSTER_ERRNO + 0)
 #define ECLUSTER_NO_MACHINE (CLUSTER_ERRNO + 1)
 #define ECLUSTER_OP_TIMEOUT (CLUSTER_ERRNO + 2)
@@ -54,6 +54,7 @@
 #define ECLUSTER_ORB_EIO (CLUSTER_ERRNO + 4)
 #define ECLUSTER_CHANNEL_INUSE (CLUSTER_ERRNO + 5)
 #define ECLUSTER_NOMORE_CHANNELS (CLUSTER_ERRNO + 6)
+#endif
 
 int init_clusterprocessor(void);
 enum {

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/1c06db83/iocore/cluster/P_ClusterCache.h
----------------------------------------------------------------------
diff --git a/iocore/cluster/P_ClusterCache.h b/iocore/cluster/P_ClusterCache.h
index 37f49e1..e68ce8f 100644
--- a/iocore/cluster/P_ClusterCache.h
+++ b/iocore/cluster/P_ClusterCache.h
@@ -357,6 +357,27 @@ struct ClusterVConnectionBase : public CacheVConnection {
   virtual void do_io_close(int lerrno = -1);
   virtual VIO *do_io_pread(Continuation *, int64_t, MIOBuffer *, int64_t);
 
+  // TODO - fix these to work for cluster.
+  // I think the best approach is to foist the work off to the source peer and have it do
+  // the range formatting which we then just pass through. For now, this just prevents
+  // link problems so I can get the base case to work.
+  virtual void
+  set_content_range(HTTPRangeSpec const &)
+  {
+    return;
+  }
+  virtual char const *
+  get_http_range_boundary_string(int *) const
+  {
+    return NULL;
+  }
+  virtual int64_t
+  get_effective_content_size()
+  {
+    return this->get_object_size();
+  }
+  virtual void set_full_content_length(int64_t) {} // only used when writing to cache
+
   // Set the timeouts associated with this connection.
   // active_timeout is for the total elasped time of the connection.
   // inactivity_timeout is the elapsed time *while an operation was
@@ -388,6 +409,7 @@ struct ClusterVConnectionBase : public CacheVConnection {
   ink_hrtime active_timeout_in;
   Event *inactivity_timeout;
   Event *active_timeout;
+  CacheRange resp_range;
 
   virtual void reenable(VIO *);
   virtual void reenable_re(VIO *);

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/1c06db83/lib/ts/CryptoHash.h
----------------------------------------------------------------------
diff --git a/lib/ts/CryptoHash.h b/lib/ts/CryptoHash.h
index 28b6296..316d9f1 100644
--- a/lib/ts/CryptoHash.h
+++ b/lib/ts/CryptoHash.h
@@ -26,8 +26,15 @@
 /// Apache Traffic Server commons.
 namespace ats
 {
+/// Permutation table for computing next hash.
+extern uint8_t const CRYPTO_HASH_NEXT_TABLE[256];
+/// Permutation table for computing previous hash.
+extern uint8_t const CRYPTO_HASH_PREV_TABLE[256];
+
 /// Crypto hash output.
 union CryptoHash {
+  typedef CryptoHash self; ///< Self reference type.
+
   uint64_t b[2]; // Legacy placeholder
   uint64_t u64[2];
   uint32_t u32[4];
@@ -51,7 +58,7 @@ union CryptoHash {
   /// Equality - bitwise identical.
   bool operator==(CryptoHash const &that) const { return u64[0] == that.u64[0] && u64[1] == that.u64[1]; }
 
-  /// Equality - bitwise identical.
+  /// Inequality - bitwise identical.
   bool operator!=(CryptoHash const &that) const { return !(*this == that); }
 
   /// Reduce to 64 bit value.
@@ -85,6 +92,44 @@ union CryptoHash {
   {
     return ink_code_to_hex_str(buffer, u8);
   }
+
+  /// Check for the zero key.
+  bool
+  is_zero() const
+  {
+    return 0 == (u64[0] | u64[1]);
+  }
+
+  /// Update the key to the computationally chained next key from @a that.
+  void
+  next(self const &that)
+  {
+    u8[0] = CRYPTO_HASH_NEXT_TABLE[that.u8[0]];
+    for (unsigned int i = 1; i < sizeof(u8); ++i)
+      u8[i] = CRYPTO_HASH_NEXT_TABLE[(u8[i - 1] + that.u8[i]) & 0xFF];
+  }
+  /// Update the key to the computationally chained next key.
+  void
+  next()
+  {
+    this->next(*this);
+  }
+
+  /// Update the key to the computationally chained previous key from @a that.
+  void
+  prev(self const &that)
+  {
+    for (unsigned int i = sizeof(u8) - 1; i > 0; --i)
+      u8[i] = 256 + CRYPTO_HASH_PREV_TABLE[that.u8[i]] - that.u8[i - 1];
+    u8[0] = CRYPTO_HASH_PREV_TABLE[that.u8[0]];
+  }
+
+  /// Update the key to the computationally chained previous key.
+  void
+  prev()
+  {
+    this->prev(*this);
+  }
 };
 
 extern CryptoHash const CRYPTO_HASH_ZERO;

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/1c06db83/lib/ts/InkErrno.h
----------------------------------------------------------------------
diff --git a/lib/ts/InkErrno.h b/lib/ts/InkErrno.h
index f3f796a..3915d4e 100644
--- a/lib/ts/InkErrno.h
+++ b/lib/ts/InkErrno.h
@@ -66,6 +66,8 @@
 #define ECACHE_NOT_READY (CACHE_ERRNO + 7)
 #define ECACHE_ALT_MISS (CACHE_ERRNO + 8)
 #define ECACHE_BAD_READ_REQUEST (CACHE_ERRNO + 9)
+#define ECACHE_INVALID_RANGE (CACHE_ERRNO + 10)
+#define ECACHE_UNSATISFIABLE_RANGE (CACHE_ERRNO + 11)
 
 #define EHTTP_ERROR (HTTP_ERRNO + 0)
 

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/1c06db83/lib/ts/ParseRules.cc
----------------------------------------------------------------------
diff --git a/lib/ts/ParseRules.cc b/lib/ts/ParseRules.cc
index 5864e70..a1885dd 100644
--- a/lib/ts/ParseRules.cc
+++ b/lib/ts/ParseRules.cc
@@ -306,3 +306,21 @@ ink_atoi64(const char *str, int len)
   }
   return num;
 }
+
+uint64_t
+ats_strto64(char const *s, size_t len, size_t *used)
+{
+  uint64_t zret = 0;
+  char const *spot = s;
+
+  if (s && len) {
+    for (char const *limit = spot + len; spot < limit && ParseRules::is_digit(*spot); ++spot) {
+      zret *= 10;
+      zret += *spot - '0';
+    }
+  }
+
+  if (used)
+    *used = spot - s;
+  return zret;
+}

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/1c06db83/lib/ts/ParseRules.h
----------------------------------------------------------------------
diff --git a/lib/ts/ParseRules.h b/lib/ts/ParseRules.h
index ce5c676..db65997 100644
--- a/lib/ts/ParseRules.h
+++ b/lib/ts/ParseRules.h
@@ -842,4 +842,24 @@ ink_atoui(const char *str)
     return static_cast<int>(val);
 }
 
+/** Convert a span of characters to an unsigned 64 bit value.
+
+    Parsing starts at @a s and continues for at most @a len characters.
+    Parsing stops when the first non-digit is encountered. Leading whitespace is not permitted.
+    @a *used is set to the number of characters parsed if @a used is not @c NULL.
+    If @a *used is set to 0 and 0 is returned, then no characters were parsed.
+
+    Key features
+
+    - No termination required.
+    - Number of parsed characters returned.
+    - Unsigned 64 bit return.
+    - Clip to UINT64_MAX;
+
+    @return The binary equivalent of @a s.
+
+    @internal All of these conversions and none work as I need. Sigh.
+*/
+uint64_t ats_strto64(char const *s, size_t len, size_t *used);
+
 #endif /* #if !defined (_ParseRules_h_) */

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/1c06db83/lib/ts/TsBuffer.h
----------------------------------------------------------------------
diff --git a/lib/ts/TsBuffer.h b/lib/ts/TsBuffer.h
index 4da12ff..4f4b2ee 100644
--- a/lib/ts/TsBuffer.h
+++ b/lib/ts/TsBuffer.h
@@ -219,6 +219,7 @@ struct ConstBuffer {
   self &set(char const *start, ///< First valid character.
             char const *end    ///< First invalid character.
             );
+
   /// Reset to empty.
   self &reset();
 
@@ -238,7 +239,7 @@ struct ConstBuffer {
       This is convenient when tokenizing and @a p points at the token
       separator.
 
-      @note If @a *p is not in the buffer then @a this is not changed
+      @note If @a *p is in the buffer then @a this is not changed
       and an empty buffer is returned. This means the caller can
       simply pass the result of @c find and check for an empty
       buffer returned to detect no more separators.
@@ -263,17 +264,20 @@ struct ConstBuffer {
       @return A buffer containing data up to but not including @a p.
   */
   self splitOn(char c);
+
   /** Get a trailing segment of the buffer.
 
       @return A buffer that contains all data after @a p.
   */
   self after(char const *p) const;
+
   /** Get a trailing segment of the buffer.
 
       @return A buffer that contains all data after the first
       occurrence of @a c.
   */
   self after(char c) const;
+
   /** Remove trailing segment.
 
       Data at @a p and beyond is removed from the buffer.
@@ -282,6 +286,48 @@ struct ConstBuffer {
       @return @a this.
   */
   self &clip(char const *p);
+
+  /** Remove initial instances of @a c.
+
+      @return @c true if not all characters were skipped, @c false if all characters matched @a c.
+      @see trim
+  */
+  bool skip(char c);
+
+  /** Remove leading characters that satisfy a @a predicate.
+      @return @c true if not all characters were skipped, @c false if all characters matched the @a predicate.
+
+      @internal We template this because the @c ParseRules predicates (which are the usual suspects)
+      return an integral type that is not @c bool.
+  */
+  template <typename BOOL_EQUIV ///< Type that can be automatically converted to bool
+            >
+  bool skip(BOOL_EQUIV (*predicate)(char));
+
+  /** Remove an initial instance the string @a str.
+
+      If the initial characters of the buffer match @a str (ignoring case) then the buffer is advanced past @a str.
+
+      @return @c true if matched and skipped, @c false otherwise.
+  */
+  bool skipNoCase(self const &str);
+
+  /** Remove trailing instances of @a c.
+
+      @return @c true if not all characters were trimmed, @c false if all characters matched @a c.
+      @see @c skip
+  */
+  bool trim(char c);
+
+  /** Remove trailing characters that satisfy a @a predicate.
+      @return @c true if not all characters were trimmed, @c false if all characters matched the @a predicate.
+
+      @internal We template this because the @c ParseRules predicates (which are the usual suspects)
+      return an integral type that is not @c bool.
+  */
+  template <typename BOOL_EQUIV ///< Type that can be automatically converted to bool
+            >
+  bool trim(BOOL_EQUIV (*predicate)(char));
 };
 
 // ----------------------------------------------------------
@@ -500,6 +546,55 @@ ConstBuffer::clip(char const *p)
   return *this;
 }
 
+template <typename BOOL_EQUIV>
+inline bool
+ConstBuffer::skip(BOOL_EQUIV (*predicate)(char))
+{
+  while (*this && predicate(**this))
+    ++*this;
+  return *this;
+}
+inline bool
+ConstBuffer::skip(char c)
+{
+  while (*this && c == **this)
+    ++*this;
+  return *this;
+}
+
+template <typename BOOL_EQUIV>
+inline bool
+ConstBuffer::trim(BOOL_EQUIV (*predicate)(char))
+{
+  if (NULL != _ptr) {
+    while (_size && predicate(_ptr[_size - 1]))
+      --_size;
+  }
+  return *this;
+}
+
+inline bool
+ConstBuffer::skipNoCase(self const &str)
+{
+  bool zret = true;
+  if (str._size <= _size && 0 == strncasecmp(_ptr, str._ptr, str._size))
+    *this += str._size;
+  else
+    zret = false;
+  return zret;
+}
+
+inline bool
+ConstBuffer::trim(char c)
+{
+  if (NULL != _ptr) {
+    while (_size && c == _ptr[_size - 1])
+      --_size;
+  }
+
+  return *this;
+}
+
 } // end namespace
 
 typedef ts::Buffer TsBuffer;

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/1c06db83/lib/ts/ink_code.cc
----------------------------------------------------------------------
diff --git a/lib/ts/ink_code.cc b/lib/ts/ink_code.cc
index e099b9c..f5db817 100644
--- a/lib/ts/ink_code.cc
+++ b/lib/ts/ink_code.cc
@@ -154,3 +154,32 @@ ink_code_to_hex_str(char *dest33, uint8_t const *hash)
   *d = '\0';
   return (dest33);
 }
+
+namespace ats
+{
+uint8_t const CRYPTO_HASH_NEXT_TABLE[256] = {
+  21,  53,  167, 51,  255, 126, 241, 151, 115, 66,  155, 174, 226, 215, 80,  188, 12,  95,  8,   24,  162, 201, 46,  104, 79,  172,
+  39,  68,  56,  144, 142, 217, 101, 62,  14,  108, 120, 90,  61,  47,  132, 199, 110, 166, 83,  125, 57,  65,  19,  130, 148, 116,
+  228, 189, 170, 1,   71,  0,   252, 184, 168, 177, 88,  229, 242, 237, 183, 55,  13,  212, 240, 81,  211, 74,  195, 205, 147, 93,
+  30,  87,  86,  63,  135, 102, 233, 106, 118, 163, 107, 10,  243, 136, 160, 119, 43,  161, 206, 141, 203, 78,  175, 36,  37,  140,
+  224, 197, 185, 196, 248, 84,  122, 73,  152, 157, 18,  225, 219, 145, 45,  2,   171, 249, 173, 32,  143, 137, 69,  41,  35,  89,
+  33,  98,  179, 214, 114, 231, 251, 123, 180, 194, 29,  3,   178, 31,  192, 164, 15,  234, 26,  230, 91,  156, 5,   16,  23,  244,
+  58,  50,  4,   67,  134, 165, 60,  235, 250, 7,   138, 216, 49,  139, 191, 154, 11,  52,  239, 59,  111, 245, 9,   64,  25,  129,
+  247, 232, 190, 246, 109, 22,  112, 210, 221, 181, 92,  169, 48,  100, 193, 77,  103, 133, 70,  220, 207, 223, 176, 204, 76,  186,
+  200, 208, 158, 182, 227, 222, 131, 38,  187, 238, 6,   34,  253, 128, 146, 44,  94,  127, 105, 153, 113, 20,  27,  124, 159, 17,
+  72,  218, 96,  149, 213, 42,  28,  254, 202, 40,  117, 82,  97,  209, 54,  236, 121, 75,  85,  150, 99,  198,
+};
+
+uint8_t const CRYPTO_HASH_PREV_TABLE[256] = {
+  57,  55,  119, 141, 158, 152, 218, 165, 18,  178, 89,  172, 16,  68,  34,  146, 153, 233, 114, 48,  229, 0,   187, 154, 19,  180,
+  148, 230, 240, 140, 78,  143, 123, 130, 219, 128, 101, 102, 215, 26,  243, 127, 239, 94,  223, 118, 22,  39,  194, 168, 157, 3,
+  173, 1,   248, 67,  28,  46,  156, 175, 162, 38,  33,  81,  179, 47,  9,   159, 27,  126, 200, 56,  234, 111, 73,  251, 206, 197,
+  99,  24,  14,  71,  245, 44,  109, 252, 80,  79,  62,  129, 37,  150, 192, 77,  224, 17,  236, 246, 131, 254, 195, 32,  83,  198,
+  23,  226, 85,  88,  35,  186, 42,  176, 188, 228, 134, 8,   51,  244, 86,  93,  36,  250, 110, 137, 231, 45,  5,   225, 221, 181,
+  49,  214, 40,  199, 160, 82,  91,  125, 166, 169, 103, 97,  30,  124, 29,  117, 222, 76,  50,  237, 253, 7,   112, 227, 171, 10,
+  151, 113, 210, 232, 92,  95,  20,  87,  145, 161, 43,  2,   60,  193, 54,  120, 25,  122, 11,  100, 204, 61,  142, 132, 138, 191,
+  211, 66,  59,  106, 207, 216, 15,  53,  184, 170, 144, 196, 139, 74,  107, 105, 255, 41,  208, 21,  242, 98,  205, 75,  96,  202,
+  209, 247, 189, 72,  69,  238, 133, 13,  167, 31,  235, 116, 201, 190, 213, 203, 104, 115, 12,  212, 52,  63,  149, 135, 183, 84,
+  147, 163, 249, 65,  217, 174, 70,  6,   64,  90,  155, 177, 185, 182, 108, 121, 164, 136, 58,  220, 241, 4,
+};
+}

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/1c06db83/mgmt/Alarms.cc
----------------------------------------------------------------------
diff --git a/mgmt/Alarms.cc b/mgmt/Alarms.cc
index 2b260b9..e087aa4 100644
--- a/mgmt/Alarms.cc
+++ b/mgmt/Alarms.cc
@@ -21,7 +21,6 @@
   limitations under the License.
  */
 
-
 #include "libts.h"
 #include "LocalManager.h"
 #include "ClusterCom.h"
@@ -67,7 +66,6 @@ Alarms::Alarms()
   alarmOEMcount = minOEMkey;
 } /* End Alarms::Alarms */
 
-
 Alarms::~Alarms()
 {
   ink_hash_table_destroy(cblist);
@@ -76,7 +74,6 @@ Alarms::~Alarms()
   ink_mutex_destroy(&mutex);
 } /* End Alarms::Alarms */
 
-
 void
 Alarms::registerCallback(AlarmCallbackFunc func)
 {
@@ -89,7 +86,6 @@ Alarms::registerCallback(AlarmCallbackFunc func)
   ink_mutex_release(&mutex);
 } /* End Alarms::registerCallback */
 
-
 bool
 Alarms::isCurrentAlarm(alarm_t a, char *ip)
 {
@@ -113,7 +109,6 @@ Alarms::isCurrentAlarm(alarm_t a, char *ip)
   return ret;
 } /* End Alarms::isCurrentAlarm */
 
-
 void
 Alarms::resolveAlarm(alarm_t a, char *ip)
 {
@@ -147,7 +142,6 @@ Alarms::resolveAlarm(alarm_t a, char *ip)
   return;
 } /* End Alarms::resolveAlarm */
 
-
 void
 Alarms::signalAlarm(alarm_t a, const char *desc, const char *ip)
 {
@@ -294,14 +288,14 @@ Alarms::signalAlarm(alarm_t a, const char *desc, const char *ip)
     (*(func))(a, ip, desc);
   }
 
-  /* Priority 2 alarms get signalled if they are the first unsolved occurence. */
+  /* Priority 2 alarms get signalled if they are the first unsolved occurence.
+   */
   if (priority == 2 && !ip) {
     execAlarmBin(desc);
   }
 
 } /* End Alarms::signalAlarm */
 
-
 /*
  * resetSeenFlag(...)
  *   Function resets the "seen" flag for a given peer's alarms. This allows
@@ -327,7 +321,6 @@ Alarms::resetSeenFlag(char *ip)
   return;
 } /* End Alarms::resetSeenFlag */
 
-
 /*
  * clearUnSeen(...)
  *   This function is a sweeper functionto clean up those alarms that have
@@ -357,7 +350,6 @@ Alarms::clearUnSeen(char *ip)
   return;
 } /* End Alarms::clearUnSeen */
 
-
 /*
  * constructAlarmMessage(...)
  *   This functions builds a message buffer for passing to peers. It basically
@@ -418,7 +410,6 @@ Alarms::constructAlarmMessage(const AppVersionInfo &version, char *ip, char *mes
   return;
 } /* End Alarms::constructAlarmMessage */
 
-
 /*
  * checkSystemNAlert(...)
  *   This function should test the system and signal local alarms. Sending

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/1c06db83/mgmt/Alarms.h
----------------------------------------------------------------------
diff --git a/mgmt/Alarms.h b/mgmt/Alarms.h
index b699d4c..be3f641 100644
--- a/mgmt/Alarms.h
+++ b/mgmt/Alarms.h
@@ -81,7 +81,6 @@ class AppVersionInfo;
 extern const char *alarmText[];
 extern const int alarmTextNum;
 
-
 /* OEM_ALARM: the alarm type is used as a key for hash tables;
    need offset and modulo constants which will keep the unique
    keys for OEM alarms within a specified range */

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/1c06db83/mgmt/BaseManager.cc
----------------------------------------------------------------------
diff --git a/mgmt/BaseManager.cc b/mgmt/BaseManager.cc
index c72c4a9..f0a275b 100644
--- a/mgmt/BaseManager.cc
+++ b/mgmt/BaseManager.cc
@@ -34,7 +34,6 @@
 #include "libts.h"
 #include "BaseManager.h"
 
-
 BaseManager::BaseManager()
 {
   /* Setup the event queue and callback tables */
@@ -43,7 +42,6 @@ BaseManager::BaseManager()
 
 } /* End BaseManager::BaseManager */
 
-
 BaseManager::~BaseManager()
 {
   InkHashTableEntry *entry;
@@ -69,7 +67,6 @@ BaseManager::~BaseManager()
   return;
 } /* End BaseManager::~BaseManager */
 
-
 /*
  * registerMgmtCallback(...)
  *   Function to register callback's for various management events, such
@@ -113,7 +110,6 @@ BaseManager::registerMgmtCallback(int msg_id, MgmtCallback cb, void *opaque_cb_d
   return msg_id;
 } /* End BaseManager::registerMgmtCallback */
 
-
 /*
  * signalMgmtEntity(...)
  */
@@ -151,7 +147,6 @@ BaseManager::signalMgmtEntity(int msg_id, char *data_raw, int data_len)
 
 } /* End BaseManager::signalMgmtEntity */
 
-
 void
 BaseManager::executeMgmtCallback(int msg_id, char *data_raw, int data_len)
 {

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/1c06db83/mgmt/BaseManager.h
----------------------------------------------------------------------
diff --git a/mgmt/BaseManager.h b/mgmt/BaseManager.h
index 2075d54..ba81ec9 100644
--- a/mgmt/BaseManager.h
+++ b/mgmt/BaseManager.h
@@ -40,7 +40,6 @@
 
 #include "MgmtDefs.h"
 
-
 /*******************************************
  * used by LocalManager and in Proxy Main. *
  */
@@ -106,14 +105,12 @@ typedef struct _mgmt_message_hdr_type {
   int data_len;
 } MgmtMessageHdr;
 
-
 typedef struct _mgmt_event_callback_list {
   MgmtCallback func;
   void *opaque_data;
   struct _mgmt_event_callback_list *next;
 } MgmtCallbackList;
 
-
 class BaseManager
 {
 public:
@@ -134,5 +131,4 @@ protected:
 private:
 }; /* End class BaseManager */
 
-
 #endif /* _BASE_MANAGER_H */

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/1c06db83/mgmt/FileManager.cc
----------------------------------------------------------------------
diff --git a/mgmt/FileManager.cc b/mgmt/FileManager.cc
index 55e1dd8..cd5dee0 100644
--- a/mgmt/FileManager.cc
+++ b/mgmt/FileManager.cc
@@ -55,8 +55,9 @@ FileManager::FileManager()
   // Check to see if the directory already exists, if not create it.
   if (mkdir(snapshotDir, DIR_MODE) < 0 && errno != EEXIST) {
     // Failed to create the snapshot directory
-    mgmt_fatal(stderr, 0, "[FileManager::FileManager] Failed to create the snapshot directory %s: %s\n", (const char *)snapshotDir,
-               strerror(errno));
+    mgmt_fatal(stderr, 0, "[FileManager::FileManager] Failed to create the "
+                          "snapshot directory %s: %s\n",
+               (const char *)snapshotDir, strerror(errno));
   }
 
   if (!ink_file_is_directory(snapshotDir)) {
@@ -124,7 +125,8 @@ FileManager::registerCallback(FileCallbackFunc func)
   ink_mutex_release(&cbListLock);
 }
 
-// void FileManager::addFile(char* baseFileName, const configFileInfo* file_info)
+// void FileManager::addFile(char* baseFileName, const configFileInfo*
+// file_info)
 //
 //  for the baseFile, creates a Rollback object for it
 //
@@ -291,7 +293,9 @@ FileManager::abortRestore(const char *abortTo)
 
     currentVersion = rb->getCurrentVersion();
     if (rb->revertToVersion_ml(currentVersion - 1) != OK_ROLLBACK) {
-      mgmt_fatal(stderr, 0, "[FileManager::abortRestore] Unable to abort a failed snapshot restore.  Configuration files have been "
+      mgmt_fatal(stderr, 0, "[FileManager::abortRestore] Unable to abort a "
+                            "failed snapshot restore.  Configuration files "
+                            "have been "
                             "left in a inconsistent state\n");
     }
   }
@@ -415,7 +419,9 @@ FileManager::removeSnap(const char *snapName, const char *snapDir)
   if (unlinkFailed == false) {
     if (rmdir(snapPath) < 0) {
       // strerror() isn't reentrant/thread-safe ... Problem? /leif
-      mgmt_log(stderr, "[FileManager::removeSnap] Unable to remove snapshot directory %s: %s\n", snapPath, strerror(errno));
+      mgmt_log(stderr, "[FileManager::removeSnap] Unable to remove snapshot "
+                       "directory %s: %s\n",
+               snapPath, strerror(errno));
       result = SNAP_REMOVE_FAILED;
     } else {
       result = SNAP_OK;
@@ -457,7 +463,9 @@ FileManager::takeSnap(const char *snapName, const char *snapDir)
   snapPath = newPathString(snapDir, snapName);
 
   if (mkdir(snapPath, DIR_MODE) < 0 && errno != EEXIST) {
-    mgmt_log(stderr, "[FileManager::takeSnap] Failed to create directory for snapshot %s: %s\n", snapName, strerror(errno));
+    mgmt_log(stderr, "[FileManager::takeSnap] Failed to create directory for "
+                     "snapshot %s: %s\n",
+             snapName, strerror(errno));
     delete[] snapPath;
     return SNAP_DIR_CREATE_FAILED;
   }
@@ -484,7 +492,9 @@ FileManager::takeSnap(const char *snapName, const char *snapDir)
       // Remove the failed snapshot so that we do not have a partial
       //   one hanging around
       if (removeSnap(snapName, snapDir) != SNAP_OK) {
-        mgmt_log(stderr, "[FileManager::takeSnap] Unable to remove failed snapshot %s.  This snapshot should be removed by hand\n",
+        mgmt_log(stderr, "[FileManager::takeSnap] Unable to remove failed "
+                         "snapshot %s.  This snapshot should be removed by "
+                         "hand\n",
                  snapName);
       }
       break;
@@ -596,8 +606,10 @@ FileManager::WalkSnaps(ExpandingArray *snapList)
 {
   MFresult r;
 
-  // The original code reset this->managedDir from proxy.config.snapshot_dir at this point. There doesn't appear to be
-  // any need for that, since managedDir is always set in the constructor and should not be changed.
+  // The original code reset this->managedDir from proxy.config.snapshot_dir at
+  // this point. There doesn't appear to be
+  // any need for that, since managedDir is always set in the constructor and
+  // should not be changed.
   ink_release_assert(this->managedDir != NULL);
 
   ink_mutex_acquire(&accessLock);
@@ -663,7 +675,8 @@ FileManager::isConfigStale()
   return stale;
 }
 
-// void FileManager::displaySnapPage(textBuffer* output, httpResponse& answerHdr)
+// void FileManager::displaySnapPage(textBuffer* output, httpResponse&
+// answerHdr)
 //
 //  Generates an HTML page with the add form and the list
 //    of current snapshots
@@ -684,7 +697,8 @@ FileManager::displaySnapOption(textBuffer *output)
   }
 }
 
-// void FileManger::createSelect(char* formVar, textBuffer* output, ExpandingArray*)
+// void FileManger::createSelect(char* formVar, textBuffer* output,
+// ExpandingArray*)
 //
 //  Creats a form with a select list.  The select options come
 //    from the expanding array.  Action is the value for the hidden input
@@ -693,7 +707,9 @@ FileManager::displaySnapOption(textBuffer *output)
 void
 FileManager::createSelect(char *action, textBuffer *output, ExpandingArray *options)
 {
-  const char formOpen[] = "<form method=POST action=\"/configure/snap_action.html\">\n<select name=snap>\n";
+  const char formOpen[] = "<form method=POST "
+                          "action=\"/configure/snap_action.html\">\n<select "
+                          "name=snap>\n";
   const char formEnd[] = "</form>";
   const char submitButton[] = "<input type=submit value=\"";
   const char hiddenInput[] = "<input type=hidden name=action value=";

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/1c06db83/mgmt/LocalManager.cc
----------------------------------------------------------------------
diff --git a/mgmt/LocalManager.cc b/mgmt/LocalManager.cc
index 4d8fddf..911d0d3 100644
--- a/mgmt/LocalManager.cc
+++ b/mgmt/LocalManager.cc
@@ -21,7 +21,6 @@
   limitations under the License.
  */
 
-
 #include "libts.h"
 #include "ink_platform.h"
 #include "MgmtUtils.h"
@@ -51,7 +50,6 @@ LocalManager::mgmtCleanup()
   return;
 }
 
-
 void
 LocalManager::mgmtShutdown()
 {
@@ -71,7 +69,6 @@ LocalManager::mgmtShutdown()
   mgmtCleanup();
 }
 
-
 void
 LocalManager::processShutdown(bool mainThread)
 {
@@ -84,7 +81,6 @@ LocalManager::processShutdown(bool mainThread)
   return;
 }
 
-
 void
 LocalManager::processRestart()
 {
@@ -93,7 +89,6 @@ LocalManager::processRestart()
   return;
 }
 
-
 void
 LocalManager::processBounce()
 {
@@ -211,9 +206,11 @@ LocalManager::LocalManager(bool proxy_on) : BaseManager(), run_proxy(proxy_on),
   RecHttpLoadIp("proxy.local.incoming_ip_to_bind", m_inbound_ip4, m_inbound_ip6);
 
   if (access(sysconfdir, R_OK) == -1) {
-    mgmt_elog(0, "[LocalManager::LocalManager] unable to access() directory '%s': %d, %s\n", (const char *)sysconfdir, errno,
-              strerror(errno));
-    mgmt_fatal(0, "[LocalManager::LocalManager] please set the 'TS_ROOT' environment variable\n");
+    mgmt_elog(0, "[LocalManager::LocalManager] unable to access() directory "
+                 "'%s': %d, %s\n",
+              (const char *)sysconfdir, errno, strerror(errno));
+    mgmt_fatal(0, "[LocalManager::LocalManager] please set the 'TS_ROOT' "
+                  "environment variable\n");
   }
 
 #if TS_HAS_WCCP
@@ -234,7 +231,8 @@ LocalManager::LocalManager(bool proxy_on) : BaseManager(), run_proxy(proxy_on),
     if (located) {
       wccp_cache.loadServicesFromFile(wccp_config_str);
     } else { // not located
-      mgmt_log("[LocalManager::LocalManager] WCCP service configuration file '%s' was specified but could not be found in the file "
+      mgmt_log("[LocalManager::LocalManager] WCCP service configuration file "
+               "'%s' was specified but could not be found in the file "
                "system.\n",
                static_cast<char *>(wccp_config_str));
     }
@@ -254,7 +252,8 @@ LocalManager::LocalManager(bool proxy_on) : BaseManager(), run_proxy(proxy_on),
   // coverity[fs_check_call]
   if (access(absolute_proxy_binary, R_OK | X_OK) == -1) {
     mgmt_elog(0, "[LocalManager::LocalManager] Unable to access() '%s': %d, %s\n", absolute_proxy_binary, errno, strerror(errno));
-    mgmt_fatal(0, "[LocalManager::LocalManager] please set bin path 'proxy.config.bin_path' \n");
+    mgmt_fatal(0, "[LocalManager::LocalManager] please set bin path "
+                  "'proxy.config.bin_path' \n");
   }
 
   internal_ticker = 0;
@@ -311,9 +310,13 @@ LocalManager::initCCom(const AppVersionInfo &version, FileManager *configFiles,
 
   found = mgmt_getAddrForIntr(intrName, &cluster_ip.sa);
   if (found == false) {
-    mgmt_fatal(stderr, 0, "[LocalManager::initCCom] Unable to find network interface %s.  Exiting...\n", intrName);
+    mgmt_fatal(stderr, 0, "[LocalManager::initCCom] Unable to find network "
+                          "interface %s.  Exiting...\n",
+               intrName);
   } else if (!ats_is_ip4(&cluster_ip)) {
-    mgmt_fatal(stderr, 0, "[LocalManager::initCCom] Unable to find IPv4 network interface %s.  Exiting...\n", intrName);
+    mgmt_fatal(stderr, 0, "[LocalManager::initCCom] Unable to find IPv4 "
+                          "network interface %s.  Exiting...\n",
+               intrName);
   }
 
   ats_ip_ntop(&cluster_ip, clusterAddrStr, sizeof(clusterAddrStr));
@@ -434,7 +437,9 @@ LocalManager::pollMgmtProcessServer()
         MgmtMessageHdr *mh;
         int data_len;
 
-        mgmt_log(stderr, "[LocalManager::pollMgmtProcessServer] New process connecting fd '%d'\n", new_sockfd);
+        mgmt_log(stderr, "[LocalManager::pollMgmtProcessServer] New process "
+                         "connecting fd '%d'\n",
+                 new_sockfd);
 
         if (new_sockfd < 0) {
           mgmt_elog(stderr, errno, "[LocalManager::pollMgmtProcessServer] ==> ");
@@ -446,7 +451,8 @@ LocalManager::pollMgmtProcessServer()
           mh->data_len = data_len;
           memcpy((char *)mh + sizeof(MgmtMessageHdr), &mgmt_sync_key, data_len);
           if (mgmt_write_pipe(new_sockfd, (char *)mh, sizeof(MgmtMessageHdr) + data_len) <= 0) {
-            mgmt_elog(errno, "[LocalManager::pollMgmtProcessServer] Error writing sync key message!\n");
+            mgmt_elog(errno, "[LocalManager::pollMgmtProcessServer] Error "
+                             "writing sync key message!\n");
             close_socket(new_sockfd);
             watched_process_fd = watched_process_pid = -1;
           }
@@ -470,10 +476,14 @@ LocalManager::pollMgmtProcessServer()
           if ((res = mgmt_read_pipe(watched_process_fd, data_raw, mh_hdr.data_len)) > 0) {
             handleMgmtMsgFromProcesses(mh_full);
           } else if (res < 0) {
-            mgmt_fatal(0, "[LocalManager::pollMgmtProcessServer] Error in read (errno: %d)\n", -res);
+            mgmt_fatal(0, "[LocalManager::pollMgmtProcessServer] Error in read "
+                          "(errno: %d)\n",
+                       -res);
           }
         } else if (res < 0) {
-          mgmt_fatal(0, "[LocalManager::pollMgmtProcessServer] Error in read (errno: %d)\n", -res);
+          mgmt_fatal(0, "[LocalManager::pollMgmtProcessServer] Error in read "
+                        "(errno: %d)\n",
+                     -res);
         }
         // handle EOF
         if (res == 0) {
@@ -512,12 +522,13 @@ LocalManager::pollMgmtProcessServer()
       ink_assert(num == 0); /* Invariant */
 
     } else if (num < 0) { /* Error */
-      mgmt_elog(stderr, 0, "[LocalManager::pollMgmtProcessServer] select failed or was interrupted (%d)\n", errno);
+      mgmt_elog(stderr, 0, "[LocalManager::pollMgmtProcessServer] select "
+                           "failed or was interrupted (%d)\n",
+                errno);
     }
   }
 }
 
-
 void
 LocalManager::handleMgmtMsgFromProcesses(MgmtMessageHdr *mh)
 {
@@ -629,7 +640,6 @@ LocalManager::handleMgmtMsgFromProcesses(MgmtMessageHdr *mh)
   }
 }
 
-
 void
 LocalManager::sendMgmtMsgToProcesses(int msg_id, const char *data_str)
 {
@@ -637,7 +647,6 @@ LocalManager::sendMgmtMsgToProcesses(int msg_id, const char *data_str)
   return;
 }
 
-
 void
 LocalManager::sendMgmtMsgToProcesses(int msg_id, const char *data_raw, int data_len)
 {
@@ -651,7 +660,6 @@ LocalManager::sendMgmtMsgToProcesses(int msg_id, const char *data_raw, int data_
   return;
 }
 
-
 void
 LocalManager::sendMgmtMsgToProcesses(MgmtMessageHdr *mh)
 {
@@ -672,7 +680,8 @@ LocalManager::sendMgmtMsgToProcesses(MgmtMessageHdr *mh)
     mh->msg_id = MGMT_EVENT_SHUTDOWN;
     break;
   case MGMT_EVENT_ROLL_LOG_FILES:
-    mgmt_log("[LocalManager::SendMgmtMsgsToProcesses]Event is being constructed .\n");
+    mgmt_log("[LocalManager::SendMgmtMsgsToProcesses]Event is being "
+             "constructed .\n");
     break;
   case MGMT_EVENT_CONFIG_FILE_UPDATE:
   case MGMT_EVENT_CONFIG_FILE_UPDATE_NO_INC_VERSION:
@@ -704,11 +713,15 @@ LocalManager::sendMgmtMsgToProcesses(MgmtMessageHdr *mh)
 
   if (watched_process_fd != -1) {
     if (mgmt_write_pipe(watched_process_fd, (char *)mh, sizeof(MgmtMessageHdr) + mh->data_len) <= 0) {
-      // In case of Linux, sometimes when the TS dies, the connection between TS and TM
-      // is not closed properly. the socket does not receive an EOF. So, the TM does
-      // not detect that the connection and hence TS has gone down. Hence it still
+      // In case of Linux, sometimes when the TS dies, the connection between TS
+      // and TM
+      // is not closed properly. the socket does not receive an EOF. So, the TM
+      // does
+      // not detect that the connection and hence TS has gone down. Hence it
+      // still
       // tries to send a message to TS, but encounters an error and enters here
-      // Also, ensure that this whole thing is done only once because there will be a
+      // Also, ensure that this whole thing is done only once because there will
+      // be a
       // deluge of message in the traffic.log otherwise
 
       static pid_t check_prev_pid = watched_process_pid;
@@ -744,7 +757,9 @@ LocalManager::sendMgmtMsgToProcesses(MgmtMessageHdr *mh)
             // End of TS down
           } else {
             // TS is still up, but the connection is lost
-            const char *err_msg = "The TS-TM connection is broken for some reason. Either restart TS and TM or correct this error "
+            const char *err_msg = "The TS-TM connection is broken for some "
+                                  "reason. Either restart TS and TM or correct "
+                                  "this error "
                                   "for TM to display TS statistics correctly";
             lmgmt->alarm_keeper->signalAlarm(MGMT_ALARM_PROXY_SYSTEM_ERROR, err_msg);
           }
@@ -761,7 +776,6 @@ LocalManager::sendMgmtMsgToProcesses(MgmtMessageHdr *mh)
   }
 }
 
-
 void
 LocalManager::signalFileChange(const char *var_name, bool incVersion)
 {
@@ -773,7 +787,6 @@ LocalManager::signalFileChange(const char *var_name, bool incVersion)
   return;
 }
 
-
 void
 LocalManager::signalEvent(int msg_id, const char *data_str)
 {
@@ -781,7 +794,6 @@ LocalManager::signalEvent(int msg_id, const char *data_str)
   return;
 }
 
-
 void
 LocalManager::signalEvent(int msg_id, const char *data_raw, int data_len)
 {
@@ -796,7 +808,6 @@ LocalManager::signalEvent(int msg_id, const char *data_raw, int data_len)
   return;
 }
 
-
 /*
  * processEventQueue()
  *   Function drains and processes the mgmt event queue
@@ -841,7 +852,6 @@ LocalManager::processEventQueue()
   }
 }
 
-
 /*
  * startProxy()
  *   Function fires up a proxy process.
@@ -866,7 +876,9 @@ LocalManager::startProxy()
   // traffic server binary exists, check permissions
   else if (access(absolute_proxy_binary, R_OK | X_OK) < 0) {
     // Error don't have proper permissions
-    mgmt_elog(stderr, errno, "[LocalManager::startProxy] Unable to access %s due to bad permisssions \n", absolute_proxy_binary);
+    mgmt_elog(stderr, errno, "[LocalManager::startProxy] Unable to access %s "
+                             "due to bad permisssions \n",
+              absolute_proxy_binary);
     return false;
   }
 
@@ -1000,7 +1012,8 @@ LocalManager::listenForProxy()
       this->bindProxyPort(p);
     }
 
-    // read backlong configuration value and overwrite the default value if found
+    // read backlong configuration value and overwrite the default value if
+    // found
     int backlog = 1024;
     bool found;
     RecInt config_backlog = REC_readInteger("proxy.config.net.listen_backlog", &found);
@@ -1016,7 +1029,6 @@ LocalManager::listenForProxy()
   return;
 }
 
-
 /*
  * bindProxyPort()
  *  Function binds the accept port of the proxy

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/1c06db83/mgmt/MultiFile.cc
----------------------------------------------------------------------
diff --git a/mgmt/MultiFile.cc b/mgmt/MultiFile.cc
index e00b1e7..d960881 100644
--- a/mgmt/MultiFile.cc
+++ b/mgmt/MultiFile.cc
@@ -147,7 +147,6 @@ MultiFile::WalkFiles(ExpandingArray *fileList)
   return MF_OK;
 }
 
-
 bool
 MultiFile::isManaged(const char *fileName)
 {

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/1c06db83/mgmt/ProcessManager.cc
----------------------------------------------------------------------
diff --git a/mgmt/ProcessManager.cc b/mgmt/ProcessManager.cc
index 6c9bdf9..d823400 100644
--- a/mgmt/ProcessManager.cc
+++ b/mgmt/ProcessManager.cc
@@ -1,6 +1,7 @@
 /** @file
 
-  File contains the member function defs and thread loop for the process manager.
+  File contains the member function defs and thread loop for the process
+  manager.
 
   @section license License
 
@@ -21,7 +22,6 @@
   limitations under the License.
  */
 
-
 #include "libts.h"
 #undef HTTP_CACHE
 #include "InkAPIInternal.h"
@@ -76,7 +76,6 @@ ProcessManager::ProcessManager(bool rlm) : BaseManager(), require_lm(rlm), mgmt_
   pid = getpid();
 } /* End ProcessManager::ProcessManager */
 
-
 void
 ProcessManager::reconfigure()
 {
@@ -87,7 +86,6 @@ ProcessManager::reconfigure()
   return;
 } /* End ProcessManager::reconfigure */
 
-
 void
 ProcessManager::signalManager(int msg_id, const char *data_str)
 {
@@ -95,7 +93,6 @@ ProcessManager::signalManager(int msg_id, const char *data_str)
   return;
 } /* End ProcessManager::signalManager */
 
-
 void
 ProcessManager::signalManager(int msg_id, const char *data_raw, int data_len)
 {
@@ -110,7 +107,6 @@ ProcessManager::signalManager(int msg_id, const char *data_raw, int data_len)
 
 } /* End ProcessManager::signalManager */
 
-
 bool
 ProcessManager::processEventQueue()
 {
@@ -126,7 +122,8 @@ ProcessManager::processEventQueue()
       executeMgmtCallback(mh->msg_id, NULL, 0);
     }
     if (mh->msg_id == MGMT_EVENT_SHUTDOWN) {
-      mgmt_log(stderr, "[ProcessManager::processEventQueue] Shutdown msg received, exiting\n");
+      mgmt_log(stderr, "[ProcessManager::processEventQueue] Shutdown msg "
+                       "received, exiting\n");
       _exit(0);
     } /* Exit on shutdown */
     ats_free(mh);
@@ -135,7 +132,6 @@ ProcessManager::processEventQueue()
   return ret;
 } /* End ProcessManager::processEventQueue */
 
-
 bool
 ProcessManager::processSignalQueue()
 {
@@ -158,7 +154,6 @@ ProcessManager::processSignalQueue()
   return ret;
 } /* End ProcessManager::processSignalQueue */
 
-
 void
 ProcessManager::initLMConnection()
 {
@@ -192,7 +187,8 @@ ProcessManager::initLMConnection()
   }
 
   if ((connect(local_manager_sockfd, (struct sockaddr *)&serv_addr, servlen)) < 0) {
-    mgmt_fatal(stderr, errno, "[ProcessManager::initLMConnection] failed to connect management socket '%s'\n",
+    mgmt_fatal(stderr, errno, "[ProcessManager::initLMConnection] failed to "
+                              "connect management socket '%s'\n",
                (const char *)sockpath);
   }
 
@@ -218,14 +214,12 @@ ProcessManager::initLMConnection()
     }
   }
 
-
   if (sync_key_raw)
     memcpy(&mgmt_sync_key, sync_key_raw, sizeof(mgmt_sync_key));
   Debug("pmgmt", "[ProcessManager::initLMConnection] Received key: %d\n", mgmt_sync_key);
 
 } /* End ProcessManager::initLMConnection */
 
-
 void
 ProcessManager::pollLMConnection()
 {
@@ -263,7 +257,9 @@ ProcessManager::pollLMConnection()
       }
 
     } else if (num < 0) { /* Error */
-      mgmt_elog(stderr, 0, "[ProcessManager::pollLMConnection] select failed or was interrupted (%d)\n", errno);
+      mgmt_elog(stderr, 0, "[ProcessManager::pollLMConnection] select failed "
+                           "or was interrupted (%d)\n",
+                errno);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/1c06db83/mgmt/ProxyConfig.h
----------------------------------------------------------------------
diff --git a/mgmt/ProxyConfig.h b/mgmt/ProxyConfig.h
index 8699296..0103b66 100644
--- a/mgmt/ProxyConfig.h
+++ b/mgmt/ProxyConfig.h
@@ -65,7 +65,8 @@ public:
   ConfigProcessor();
 
   enum {
-    // The number of seconds to wait before garbage collecting stale ConfigInfo objects. There's
+    // The number of seconds to wait before garbage collecting stale ConfigInfo
+    // objects. There's
     // no good reason to tune this, outside of regression tests, so don't.
     CONFIG_PROCESSOR_RELEASE_SECS = 60
   };
@@ -91,7 +92,8 @@ public:
   int ninfos;
 };
 
-// A Continuation wrapper that calls the static reconfigure() method of the given class.
+// A Continuation wrapper that calls the static reconfigure() method of the
+// given class.
 template <typename UpdateClass> struct ConfigUpdateContinuation : public Continuation {
   int
   update(int /* etype */, void * /* data */)

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/1c06db83/mgmt/RecordsConfig.h
----------------------------------------------------------------------
diff --git a/mgmt/RecordsConfig.h b/mgmt/RecordsConfig.h
index 8d73023..a332b1c 100644
--- a/mgmt/RecordsConfig.h
+++ b/mgmt/RecordsConfig.h
@@ -21,7 +21,6 @@
   limitations under the License.
  */
 
-
 #if !defined(_RECORDS_CONFIG_H_)
 #define _RECORDS_CONFIG_H_
 
@@ -50,7 +49,8 @@ typedef void (*RecordElementCallback)(const RecordElement *, void *);
 void RecordsConfigIterate(RecordElementCallback, void *);
 
 void LibRecordsConfigInit();                 // initializes RecordsConfigIndex
-void RecordsConfigOverrideFromEnvironment(); // Override records from the environment
+void RecordsConfigOverrideFromEnvironment(); // Override records from the
+                                             // environment
 void test_librecords();
 
 #endif

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/1c06db83/mgmt/RecordsConfigUtils.cc
----------------------------------------------------------------------
diff --git a/mgmt/RecordsConfigUtils.cc b/mgmt/RecordsConfigUtils.cc
index 7c16503..8cfdaa9 100644
--- a/mgmt/RecordsConfigUtils.cc
+++ b/mgmt/RecordsConfigUtils.cc
@@ -38,9 +38,12 @@ override_record(const RecordElement *record, void *)
 
     if ((value = RecConfigOverrideFromEnvironment(record->name, NULL))) {
       if (RecDataSetFromString(record->value_type, &data, value)) {
-        // WARNING: If we are not the record owner, RecSetRecord() doesn't set our copy
-        // of the record. It sends a set message to the local manager. This can cause
-        // "interesting" results if you are trying to override configuration values
+        // WARNING: If we are not the record owner, RecSetRecord() doesn't set
+        // our copy
+        // of the record. It sends a set message to the local manager. This can
+        // cause
+        // "interesting" results if you are trying to override configuration
+        // values
         // early in startup (before we have synced with the local manager).
         RecSetRecord(record->type, record->name, record->value_type, &data, NULL, REC_SOURCE_ENV, false);
         RecDataClear(record->value_type, &data);
@@ -49,8 +52,10 @@ override_record(const RecordElement *record, void *)
   }
 }
 
-// We process environment variable overrides when we parse the records.config configuration file, but the
-// operator might choose to override a variable that is not present in records.config so we have to post-
+// We process environment variable overrides when we parse the records.config
+// configuration file, but the
+// operator might choose to override a variable that is not present in
+// records.config so we have to post-
 // process the full set of configuration valriables as well.
 void
 RecordsConfigOverrideFromEnvironment()
@@ -85,7 +90,8 @@ initialize_record(const RecordElement *record, void *)
     RecData data = {0};
     RecSourceT source = value == record->value ? REC_SOURCE_DEFAULT : REC_SOURCE_ENV;
 
-    // If you specify a consistency check, you have to specify a regex expression. We abort here
+    // If you specify a consistency check, you have to specify a regex
+    // expression. We abort here
     // so that this breaks QA completely.
     if (record->check != RECC_NULL && record->regex == NULL) {
       ink_fatal("%s has a consistency check but no regular expression", record->name);
@@ -116,7 +122,8 @@ initialize_record(const RecordElement *record, void *)
     } // switch
 
     RecDataClear(record->value_type, &data);
-  } else { // Everything else, except PROCESS, are stats. TODO: Should modularize this too like PROCESS was done.
+  } else { // Everything else, except PROCESS, are stats. TODO: Should
+           // modularize this too like PROCESS was done.
     ink_assert(REC_TYPE_IS_STAT(type));
 
     switch (record->value_type) {

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/1c06db83/mgmt/Rollback.cc
----------------------------------------------------------------------
diff --git a/mgmt/Rollback.cc b/mgmt/Rollback.cc
index a218a7d..4969be9 100644
--- a/mgmt/Rollback.cc
+++ b/mgmt/Rollback.cc
@@ -107,11 +107,14 @@ Rollback::Rollback(const char *baseFileName, bool root_access_needed_) : configF
         activeVerStr = createPathStr(ACTIVE_VERSION);
 
         if (rename(highestSeenStr, activeVerStr) < 0) {
-          mgmt_log(stderr, "[RollBack::Rollback] Automatic Rollback to prior version failed for %s : %s\n", fileName,
-                   strerror(errno));
+          mgmt_log(stderr, "[RollBack::Rollback] Automatic Rollback to prior "
+                           "version failed for %s : %s\n",
+                   fileName, strerror(errno));
           needZeroLength = true;
         } else {
-          mgmt_log(stderr, "[RollBack::Rollback] Automatic Rollback to version succeded for %s\n", fileName, strerror(errno));
+          mgmt_log(stderr, "[RollBack::Rollback] Automatic Rollback to version "
+                           "succeded for %s\n",
+                   fileName, strerror(errno));
           needZeroLength = false;
           highestSeen--;
           // Since we've made the highestVersion active
@@ -134,8 +137,9 @@ Rollback::Rollback(const char *baseFileName, bool root_access_needed_) : configF
           ats_free(alarmMsg);
           closeFile(fd, true);
         } else {
-          mgmt_fatal(stderr, 0,
-                     "[RollBack::Rollback] Unable to find configuration file %s.\n\tCreation of a placeholder failed : %s\n",
+          mgmt_fatal(stderr, 0, "[RollBack::Rollback] Unable to find "
+                                "configuration file %s.\n\tCreation of a "
+                                "placeholder failed : %s\n",
                      fileName, strerror(errno));
         }
       }
@@ -145,8 +149,9 @@ Rollback::Rollback(const char *baseFileName, bool root_access_needed_) : configF
     } else {
       // If is there but we can not stat it, it is unusable to manager
       //   probably due to permissions problems.  Bail!
-      mgmt_fatal(stderr, 0, "[RollBack::Rollback] Unable to find configuration file %s.\n\tStat failed : %s\n", fileName,
-                 strerror(errno));
+      mgmt_fatal(stderr, 0, "[RollBack::Rollback] Unable to find configuration "
+                            "file %s.\n\tStat failed : %s\n",
+                 fileName, strerror(errno));
     }
   } else {
     fileLastModified = TS_ARCHIVE_STAT_MTIME(fileInfo);
@@ -204,7 +209,6 @@ Rollback::~Rollback()
   ats_free(fileName);
 }
 
-
 // Rollback::createPathStr(version_t version)
 //
 //   CALLEE DELETES STORAGE
@@ -296,7 +300,6 @@ Rollback::closeFile(int fd, bool callSync)
   return result;
 }
 
-
 RollBackCodes
 Rollback::updateVersion(textBuffer *buf, version_t basedOn, version_t newVersion, bool notifyChange, bool incVersion)
 {
@@ -323,7 +326,6 @@ Rollback::updateVersion_ml(textBuffer *buf, version_t basedOn, version_t newVers
   return returnCode;
 }
 
-
 RollBackCodes
 Rollback::forceUpdate(textBuffer *buf, version_t newVersion)
 {
@@ -361,7 +363,6 @@ Rollback::internalUpdate(textBuffer *buf, version_t newVersion, bool notifyChang
   bool failedLink = false;
   char *alarmMsg = NULL;
 
-
   // Check to see if the callee has specified a newVersion number
   //   If the newVersion argument is less than zero, the callee
   //   is telling us to use the next version in squence
@@ -413,7 +414,8 @@ Rollback::internalUpdate(textBuffer *buf, version_t newVersion, bool notifyChang
     //    install a new file so that we do not go around in
     //    an endless loop
     if (errno == ENOENT) {
-      mgmt_log(stderr, "[Rollback::internalUpdate] The active version of %s was lost.\n\tThe updated copy was installed.\n",
+      mgmt_log(stderr, "[Rollback::internalUpdate] The active version of %s "
+                       "was lost.\n\tThe updated copy was installed.\n",
                fileName);
       failedLink = true;
     } else {
@@ -424,7 +426,9 @@ Rollback::internalUpdate(textBuffer *buf, version_t newVersion, bool notifyChang
 
   if (rename(nextVersion, activeVersion) < 0) {
     mgmt_log(stderr, "[Rollback::internalUpdate] Rename failed : %s\n", strerror(errno));
-    mgmt_log(stderr, "[Rollback::internalUpdate] Unable to create new version of %s.  Using prior version\n", fileName);
+    mgmt_log(stderr, "[Rollback::internalUpdate] Unable to create new version "
+                     "of %s.  Using prior version\n",
+             fileName);
 
     returnCode = SYS_CALL_ERROR_ROLLBACK;
     goto UPDATE_CLEANUP;
@@ -458,7 +462,6 @@ Rollback::internalUpdate(textBuffer *buf, version_t newVersion, bool notifyChang
   this->numVersions++;
   this->currentVersion = newVersion;
 
-
   returnCode = OK_ROLLBACK;
 
   // Post the change to the config file manager
@@ -491,7 +494,6 @@ UPDATE_CLEANUP:
   return returnCode;
 }
 
-
 RollBackCodes
 Rollback::getVersion(version_t version, textBuffer **buffer)
 {
@@ -546,7 +548,8 @@ Rollback::getVersion_ml(version_t version, textBuffer **buffer)
   } while (readResult > 0);
 
   if ((off_t)newBuffer->spaceUsed() != fileInfo.st_size) {
-    mgmt_log(stderr, "[Rollback::getVersion] Incorrect amount of data retrieved from %s version %d.  Expected: %d   Got: %d\n",
+    mgmt_log(stderr, "[Rollback::getVersion] Incorrect amount of data "
+                     "retrieved from %s version %d.  Expected: %d   Got: %d\n",
              fileName, version, fileInfo.st_size, newBuffer->spaceUsed());
     returnCode = SYS_CALL_ERROR_ROLLBACK;
     delete newBuffer;
@@ -639,8 +642,9 @@ Rollback::findVersions_ml(ExpandingArray *listNames)
   dir = opendir(sysconfdir);
 
   if (dir == NULL) {
-    mgmt_log(stderr, "[Rollback::findVersions] Unable to open configuration directory: %s: %s\n", (const char *)sysconfdir,
-             strerror(errno));
+    mgmt_log(stderr, "[Rollback::findVersions] Unable to open configuration "
+                     "directory: %s: %s\n",
+             (const char *)sysconfdir, strerror(errno));
     return INVALID_VERSION;
   }
   // The fun of Solaris - readdir_r requires a buffer passed into it
@@ -757,7 +761,6 @@ Rollback::findVersions_ml(Queue<versionInfo> &q)
   return highest;
 }
 
-
 RollBackCodes
 Rollback::removeVersion(version_t version)
 {
@@ -908,7 +911,9 @@ Rollback::checkForUserUpdate(RollBackCheckType how)
         delete buf;
       }
       if (r != OK_ROLLBACK) {
-        mgmt_log(stderr, "[Rollback::checkForUserUpdate] Failed to roll changed user file %s: %s", fileName, RollbackStrings[r]);
+        mgmt_log(stderr, "[Rollback::checkForUserUpdate] Failed to roll "
+                         "changed user file %s: %s",
+                 fileName, RollbackStrings[r]);
       }
 
       mgmt_log(stderr, "User has changed config file %s\n", fileName);

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/1c06db83/mgmt/Rollback.h
----------------------------------------------------------------------
diff --git a/mgmt/Rollback.h b/mgmt/Rollback.h
index abc6388..b119a81 100644
--- a/mgmt/Rollback.h
+++ b/mgmt/Rollback.h
@@ -124,7 +124,8 @@ struct versionInfo {
 //    returned
 //
 //  findVersions(ExpandingArray* listNames) - scans the config directory for
-//    all versions of the file.  If listNames is not NULL, pointers to versionInfo
+//    all versions of the file.  If listNames is not NULL, pointers to
+// versionInfo
 //    structures are inserted into it.  If is the callee's responsibility
 //    to ats_free the versionInfo structures.  They are allocated by ats_malloc
 //

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/1c06db83/mgmt/WebMgmtUtils.cc
----------------------------------------------------------------------
diff --git a/mgmt/WebMgmtUtils.cc b/mgmt/WebMgmtUtils.cc
index b6572e2..5ae7686 100644
--- a/mgmt/WebMgmtUtils.cc
+++ b/mgmt/WebMgmtUtils.cc
@@ -36,7 +36,6 @@
  *
  ****************************************************************************/
 
-
 // bool varSetFromStr(const char*, const char* )
 //
 // Sets the named local manager variable from the value string
@@ -286,7 +285,6 @@ varDataFromName(RecDataT varType, const char *varName, RecData *value)
   return (err == REC_ERR_OKAY);
 }
 
-
 // bool varCounterFromName (const char*, RecFloat* )
 //
 //   Sets the *value to value of the varName.
@@ -435,7 +433,6 @@ varIntFromName(const char *varName, RecInt *value)
   return found;
 }
 
-
 // void percentStrFromFloat(MgmtFloat, char* bufVal)
 //
 //  Converts a float to a percent string
@@ -785,7 +782,6 @@ varType(const char *varName)
   return data_type;
 }
 
-
 // InkHashTable* processFormSubmission(char* submission)
 //
 //  A generic way to handle a HTML form submission.
@@ -1011,7 +1007,6 @@ substituteForHTMLChars(const char *buffer)
   return safeBuf;
 }
 
-
 // bool ProxyShutdown()
 //
 //  Attempts to turn the proxy off.  Returns
@@ -1212,8 +1207,9 @@ recordIPCheck(const char *pattern, const char *value)
   //  regex_t regex;
   //  int result;
   bool check;
-  const char *range_pattern =
-    "\\[[0-9]+\\-[0-9]+\\]\\\\\\.\\[[0-9]+\\-[0-9]+\\]\\\\\\.\\[[0-9]+\\-[0-9]+\\]\\\\\\.\\[[0-9]+\\-[0-9]+\\]";
+  const char *range_pattern = "\\[[0-9]+\\-[0-9]+\\]\\\\\\.\\[[0-9]+\\-[0-9]+"
+                              "\\]\\\\\\.\\[[0-9]+\\-[0-9]+\\]\\\\\\.\\[[0-9]+"
+                              "\\-[0-9]+\\]";
   const char *ip_pattern = "[0-9]*[0-9]*[0-9].[0-9]*[0-9]*[0-9].[0-9]*[0-9]*[0-9].[0-9]*[0-9]*[0-9]";
 
   Tokenizer dotTok1(".");

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/1c06db83/mgmt/api/APITestCliRemote.cc
----------------------------------------------------------------------
diff --git a/mgmt/api/APITestCliRemote.cc b/mgmt/api/APITestCliRemote.cc
index a783e68..f7c68d3 100644
--- a/mgmt/api/APITestCliRemote.cc
+++ b/mgmt/api/APITestCliRemote.cc
@@ -140,7 +140,6 @@ print_err(const char *module, TSMgmtError err)
     TSfree(err_msg);
 }
 
-
 /*--------------------------------------------------------------
  * print_ports
  *--------------------------------------------------------------*/
@@ -222,7 +221,6 @@ print_domain_list(TSDomainList list)
   }
 }
 
-
 void
 print_ip_addr_ele(TSIpAddrEle *ele)
 {
@@ -279,7 +277,6 @@ print_list_of_ip_list(TSList list)
   }
 }
 
-
 /*-------------------------------------------------------
  * print_pd_sspec
  *-------------------------------------------------------*/
@@ -349,7 +346,6 @@ print_pd_sspec(TSPdSsFormat info)
   }
   printf("\n");
 
-
   printf("\tscheme: ");
   switch (info.sec_spec.scheme) {
   case TS_SCHEME_NONE:
@@ -369,7 +365,6 @@ print_pd_sspec(TSPdSsFormat info)
   return;
 }
 
-
 void
 print_cache_ele(TSCacheEle *ele)
 {
@@ -434,13 +429,13 @@ print_cache_ele(TSCacheEle *ele)
 
   /*
      print_pd_sspec(ele->cache_info);
-     printf("Time: %d day, %d hr, %d min, %d sec\n", ele->time_period.d, ele->time_period.h,
+     printf("Time: %d day, %d hr, %d min, %d sec\n", ele->time_period.d,
+     ele->time_period.h,
      ele->time_period.m, ele->time_period.s);
    */
   return;
 }
 
-
 void
 print_hosting_ele(TSHostingEle *ele)
 {
@@ -839,7 +834,6 @@ print_ele_list(TSFileNameT file, TSCfgContext ctx)
   return;
 }
 
-
 /***************************************************************************
  * Control Testing
  ***************************************************************************/
@@ -936,13 +930,15 @@ test_action_need(void)
 
   // RU_NULL record
   TSRecordSetString("proxy.config.proxy_name", "proxy_dorky", &action);
-  printf("[TSRecordSetString] proxy.config.proxy_name \n\tAction Should: [%d]\n\tAction is    : [%d]\n", TS_ACTION_UNDEFINED,
-         action);
+  printf("[TSRecordSetString] proxy.config.proxy_name \n\tAction Should: "
+         "[%d]\n\tAction is    : [%d]\n",
+         TS_ACTION_UNDEFINED, action);
 
   // RU_RESTART_TS record
   TSRecordSetInt("proxy.config.cluster.cluster_port", 6666, &action);
-  printf("[TSRecordSetInt] proxy.config.cluster.cluster_port\n\tAction Should: [%d]\n\tAction is    : [%d]\n", TS_ACTION_RESTART,
-         action);
+  printf("[TSRecordSetInt] proxy.config.cluster.cluster_port\n\tAction Should: "
+         "[%d]\n\tAction is    : [%d]\n",
+         TS_ACTION_RESTART, action);
 }
 
 /* Bouncer the traffic_server process(es) */
@@ -991,12 +987,13 @@ test_error_records()
   ret = TSRecordSetInt("proy.config.cop.core_signal", new_port, &action);
   print_err("TSRecordSetInt", ret);
 
-
   printf("\n");
   if (TSRecordGetCounter("proxy.press.socks.connections_successful", &ctr1) != TS_ERR_OKAY)
     printf("TSRecordGetCounter FAILED!\n");
   else
-    printf("[TSRecordGetCounter]proxy.process.socks.connections_successful=%" PRId64 " \n", ctr1);
+    printf("[TSRecordGetCounter]proxy.process.socks.connections_successful="
+           "%" PRId64 " \n",
+           ctr1);
 
   printf("\n");
   if (TSRecordGetFloat("proxy.conig.http.cache.fuzz.probability", &flt1) != TS_ERR_OKAY)
@@ -1034,7 +1031,6 @@ test_records()
     printf("[TSRecordSetInt] proxy.config.cop.core_signal=%" PRId64 " \n", new_port);
 #endif
 
-
 #if TEST_REC_GET
   TSRecordEle *rec_ele;
   // retrieve a string value record using generic RecordGet
@@ -1048,7 +1044,6 @@ test_records()
   printf("\n\n");
 #endif
 
-
 #if TEST_REC_GET_2
   // retrieve a string value record using generic RecordGet
   rec_ele = TSRecordEleCreate();
@@ -1078,7 +1073,6 @@ test_records()
   else
     printf("[TSRecordSetString] proxy.config.proxy_name=%s\n", new_str);
 
-
   // get
   err = TSRecordGetString("proxy.config.proxy_name", &rec_value);
   if (err != TS_ERR_OKAY)
@@ -1116,17 +1110,23 @@ test_records()
   if (TSRecordGetCounter("proxy.process.socks.connections_successful", &ctr1) != TS_ERR_OKAY)
     printf("TSRecordGetCounter FAILED!\n");
   else
-    printf("[TSRecordGetCounter]proxy.process.socks.connections_successful=%" PRId64 " \n", ctr1);
+    printf("[TSRecordGetCounter]proxy.process.socks.connections_successful="
+           "%" PRId64 " \n",
+           ctr1);
 
   if (TSRecordSetCounter("proxy.process.socks.connections_successful", new_ctr, &action) != TS_ERR_OKAY)
     printf("TSRecordSetCounter FAILED!\n");
   else
-    printf("[TSRecordSetCounter] proxy.process.socks.connections_successful=%" PRId64 " \n", new_ctr);
+    printf("[TSRecordSetCounter] "
+           "proxy.process.socks.connections_successful=%" PRId64 " \n",
+           new_ctr);
 
   if (TSRecordGetCounter("proxy.process.socks.connections_successful", &ctr2) != TS_ERR_OKAY)
     printf("TSRecordGetCounter FAILED!\n");
   else
-    printf("[TSRecordGetCounter]proxy.process.socks.connections_successful=%" PRId64 " \n", ctr2);
+    printf("[TSRecordGetCounter]proxy.process.socks.connections_successful="
+           "%" PRId64 " \n",
+           ctr2);
   printf("\n");
 #endif
 
@@ -1251,7 +1251,6 @@ test_record_get_mlt(void)
   TSStringListEnqueue(name_list, v7);
   TSStringListEnqueue(name_list, v8);
 
-
   num = TSStringListLen(name_list);
   printf("Num Records to Get: %d\n", num);
   ret = TSRecordGetMlt(name_list, rec_list);
@@ -1335,7 +1334,6 @@ test_record_set_mlt(void)
   ele5->rec_type = TS_REC_INT;
   ele5->valueT.int_val = 555;
 
-
   TSListEnqueue(list, ele4);
   TSListEnqueue(list, ele1);
   TSListEnqueue(list, ele2);
@@ -1355,7 +1353,6 @@ test_record_set_mlt(void)
   TSListDestroy(list);
 }
 
-
 /***************************************************************************
  * File I/O Testing
  ***************************************************************************/
@@ -1625,7 +1622,8 @@ test_cfg_context_move(char *args)
 
   // shift all the ele's down so that the next to bottom ele is now top ele
   // move all ele's above the last ele down; bottom ele becomes top ele
-  printf("\nShift all Ele's above second to last ele down; bottom ele becomes top ele\n");
+  printf("\nShift all Ele's above second to last ele down; bottom ele becomes "
+         "top ele\n");
   for (i = count - 3; i >= 0; i--) {
     err = TSCfgContextMoveEleDown(ctx, i);
     if (err != TS_ERR_OKAY) {
@@ -1730,7 +1728,6 @@ test_cfg_context_ops()
   }
   // print_VirtIpAddr_ele_list(ctx);
 
-
   printf("\nMove ele at index %d to botoom of list\n", insert_at);
   for (i = insert_at; i < TSCfgContextGetCount(ctx); i++) {
     err = TSCfgContextMoveEleDown(ctx, i);
@@ -1752,7 +1749,6 @@ test_cfg_context_ops()
   }
   // print_VirtIpAddr_ele_list(ctx);
 
-
   // commit change
   TSCfgContextCommit(ctx, NULL, NULL);
 
@@ -1885,7 +1881,6 @@ test_cfg_socks()
   TSCfgContextDestroy(ctx);
 }
 
-
 /***************************************************************************
  * Events Testing
  ***************************************************************************/
@@ -2137,7 +2132,6 @@ set_stats()
 
   fprintf(stderr, "[set_stats] Set Dummy Stat Values\n");
 
-
   TSRecordSetInt("proxy.process.http.user_agent_response_document_total_size", 100, &action);
   TSRecordSetInt("proxy.process.http.user_agent_response_header_total_size", 100, &action);
   TSRecordSetInt("proxy.process.http.current_client_connections", 100, &action);
@@ -2147,7 +2141,6 @@ set_stats()
   TSRecordSetInt("proxy.process.http.current_server_connections", 100, &action);
   TSRecordSetInt("proxy.process.http.current_server_transactions", 100, &action);
 
-
   TSRecordSetFloat("proxy.node.bandwidth_hit_ratio", 110, &action);
   TSRecordSetFloat("proxy.node.hostdb.hit_ratio", 110, &action);
   TSRecordSetFloat("proxy.node.cache.percent_free", 110, &action);
@@ -2194,7 +2187,6 @@ print_stats()
 
   fprintf(stderr, "[print_stats]\n");
 
-
   TSRecordGetInt("proxy.process.http.user_agent_response_document_total_size", &i1);
   TSRecordGetInt("proxy.process.http.user_agent_response_header_total_size", &i2);
   TSRecordGetInt("proxy.process.http.current_client_connections", &i3);
@@ -2274,12 +2266,14 @@ sync_test()
   TSActionNeedT action;
 
   TSRecordSetString("proxy.config.proxy_name", "dorkface", &action);
-  printf("[TSRecordSetString] proxy.config.proxy_name \n\tAction Should: [%d]\n\tAction is    : [%d]\n", TS_ACTION_UNDEFINED,
-         action);
+  printf("[TSRecordSetString] proxy.config.proxy_name \n\tAction Should: "
+         "[%d]\n\tAction is    : [%d]\n",
+         TS_ACTION_UNDEFINED, action);
 
   TSRecordSetInt("proxy.config.cluster.cluster_port", 3333, &action);
-  printf("[TSRecordSetInt] proxy.config.cluster.cluster_port\n\tAction Should: [%d]\n\tAction is    : [%d]\n", TS_ACTION_RESTART,
-         action);
+  printf("[TSRecordSetInt] proxy.config.cluster.cluster_port\n\tAction Should: "
+         "[%d]\n\tAction is    : [%d]\n",
+         TS_ACTION_RESTART, action);
 
   if (TSRecordSet("proxy.config.http.cache.fuzz.probability", "-0.3333", &action) != TS_ERR_OKAY)
     printf("TSRecordSet FAILED!\n");
@@ -2396,12 +2390,10 @@ runInteractive()
       sync_test();
     }
 
-
   } // end while(1)
 
 } // end runInteractive
 
-
 /* ------------------------------------------------------------------------
  * main
  * ------------------------------------------------------------------------