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/04/19 19:45:22 UTC

[8/8] trafficserver git commit: Full read multi range support passes basic testing.

Full read multi range support passes basic testing.


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

Branch: refs/heads/ts-974-multi-range-read
Commit: 2ddfc7ab52b4f24593a813e123e7dacb754eac58
Parents: 67261f1
Author: Alan M. Carroll <so...@yahoo-inc.com>
Authored: Sat Dec 6 00:27:32 2014 -0600
Committer: Alan M. Carroll <so...@yahoo-inc.com>
Committed: Sat Dec 6 11:56:04 2014 -0600

----------------------------------------------------------------------
 iocore/cache/CacheHttp.cc       |  76 ++++++++----
 iocore/cache/CacheRead.cc       | 221 +++++++++++++++++++----------------
 iocore/cache/I_Cache.h          |   7 ++
 iocore/cache/P_CacheHttp.h      | 127 ++++++++++++++------
 iocore/cache/P_CacheInternal.h  |   5 +-
 iocore/cluster/P_ClusterCache.h |   3 +
 lib/ts/InkErrno.h               |   3 +-
 proxy/hdrs/HTTP.cc              | 137 +++++++++++++++++++---
 proxy/hdrs/HTTP.h               |  78 +++++++++----
 proxy/http/HttpSM.cc            |   3 +-
 proxy/http/HttpTransact.cc      |   5 +-
 11 files changed, 469 insertions(+), 196 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/trafficserver/blob/2ddfc7ab/iocore/cache/CacheHttp.cc
----------------------------------------------------------------------
diff --git a/iocore/cache/CacheHttp.cc b/iocore/cache/CacheHttp.cc
index 4d82c18..b5fa82d 100644
--- a/iocore/cache/CacheHttp.cc
+++ b/iocore/cache/CacheHttp.cc
@@ -259,19 +259,58 @@ CacheHTTPInfoVector::get_handles(const char *buf, int length, RefCountObj * bloc
 /*-------------------------------------------------------------------------
   -------------------------------------------------------------------------*/
 
+void
+CacheRange::clear()
+{
+  _offset = 0;
+  _idx = -1;
+  _pending_range_shift_p = false;
+  _ct_field = NULL; // need to do real cleanup at some point.
+  _r.clear();
+}
+
+bool
+CacheRange::init(HTTPHdr* req)
+{
+  bool zret = true;
+  MIMEField* rf = req->field_find(MIME_FIELD_RANGE, MIME_LEN_RANGE);
+  if (rf) {
+    int len;
+    char const* val = rf->value_get(&len);
+    zret = _r.parse(val, len);
+  }
+  return zret;
+}
+
+bool
+CacheRange::apply(uint64_t len)
+{
+  bool zret = _r.apply(len);
+  if (zret) {
+    _len = len;
+    if (_r.hasRanges()) {
+      _offset = _r[_idx = 0]._min;
+      if (_r.isMulti()) _pending_range_shift_p = true;
+    }
+  }
+  return zret;
+}
+
 uint64_t
 CacheRange::consume(uint64_t size)
 {
-  switch (_r->_state) {
+  switch (_r._state) {
   case HTTPRangeSpec::EMPTY: _offset += size; break;
-  case HTTPRangeSpec::SINGLE: _offset += std::min(size, (_r->_single._max - _offset) + 1 ); break;
+  case HTTPRangeSpec::SINGLE: _offset += std::min(size, (_r._single._max - _offset) + 1 ); break;
   case HTTPRangeSpec::MULTI:
-    while (size && _idx < static_cast<int>(_r->count())) {
-      uint64_t r = std::min(size, ((*_r)[_idx]._max - _offset) + 1);
-      _offset += r;
-      size -= r;
-      if (_offset > (*_r)[_idx]._max)
-        _offset = (*_r)[++_idx]._min;
+    ink_assert(_idx < static_cast<int>(_r.count()));
+    // Must not consume more than 1 range or the boundary strings won't get sent.
+    ink_assert(!_pending_range_shift_p);
+    ink_assert(size <= (_r[_idx]._max - _offset) + 1);
+    _offset += size;
+    if (_offset > _r[_idx]._max && ++_idx < static_cast<int>(_r.count())) {
+      _offset = _r[_idx]._min;
+      _pending_range_shift_p = true;
     }
     break;
   default: break;
@@ -280,25 +319,22 @@ CacheRange::consume(uint64_t size)
   return _offset;
 }
 
-void
+CacheRange&
 CacheRange::generateBoundaryStr(CacheKey const& key)
 {
-  snprintf(_boundary, sizeof(_boundary), "%08" PRIu64 "%08" PRIu64 "..%08" PRIu64
-           , key.slice64(0), key.slice64(1), this_ethread()->generator.random()
-    );
-}
-
-bool
-CacheRange::setContentType(HTTPHdr* header)
-{
-  _ct_field = header->field_find(MIME_FIELD_CONTENT_TYPE, MIME_LEN_CONTENT_TYPE);
-  return NULL != _ct_field;
+  uint64_t rnd = this_ethread()->generator.random();
+  snprintf(_boundary, sizeof(_boundary), "%016" PRIx64 "%016" PRIx64 "..%016" PRIx64, key.slice64(0), key.slice64(1), rnd);
+  // GAH! snprintf null terminates so we can't actually print the last nybble that way and all of
+  // the internal hex converters do the same thing. This is crazy code I need to fix at some point.
+  // It is critical to print every nybble or the content lengths won't add up.
+  _boundary[HTTP_RANGE_BOUNDARY_LEN-1] = "0123456789abcdef"[rnd & 0xf];
+  return *this;
 }
 
 uint64_t
 CacheRange::calcContentLength() const
 {
-  return _r->calcContentLength(_len, _ct_field->m_len_value);
+  return _r.calcContentLength(_len, _ct_field ? _ct_field->m_len_value : 0);
 }
 
 /*-------------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/2ddfc7ab/iocore/cache/CacheRead.cc
----------------------------------------------------------------------
diff --git a/iocore/cache/CacheRead.cc b/iocore/cache/CacheRead.cc
index ac5e13e..c8ff47b 100644
--- a/iocore/cache/CacheRead.cc
+++ b/iocore/cache/CacheRead.cc
@@ -184,6 +184,18 @@ CacheVC::get_http_content_size()
   return resp_range.calcContentLength();
 }
 
+HTTPRangeSpec&
+CacheVC::get_http_range_spec()
+{
+  return resp_range.getRangeSpec();
+}
+
+bool
+CacheVC::is_http_partial_content()
+{
+  return resp_range.hasRanges();
+}
+
 int
 CacheVC::openReadFromWriterFailure(int event, Event * e)
 {
@@ -690,17 +702,64 @@ CacheVC::openReadMain(int /* event ATS_UNUSED */, Event * /* e ATS_UNUSED */)
 {
   cancel_trigger();
   Doc *doc = (Doc *) buf->data();
-  int64_t ntodo = vio.ntodo();
-  int64_t bytes = doc->len - doc_pos;
+  int64_t bytes = vio.ntodo();
   IOBufferBlock *b = NULL;
+  uint64_t target_offset = resp_range.getOffset();
+  uint64_t lower_bound = frag_upper_bound - doc->data_len();
+
+  if (bytes <= 0)
+    return EVENT_CONT;
+
+  // Start shipping
+  while (bytes > 0 && lower_bound <= target_offset && target_offset < frag_upper_bound) {
+    if (vio.buffer.writer()->max_read_avail() > vio.buffer.writer()->water_mark && vio.ndone) // wait for reader
+      return EVENT_CONT;
+
+    if (resp_range.hasPendingRangeShift()) { // in new range, shift to start location.
+      int b_len;
+      char const* b_str = resp_range.getBoundaryStr(&b_len);
+      size_t r_idx = resp_range.getIdx();
+
+      doc_pos = doc->prefix_len() + (target_offset - lower_bound);
+      
+      vio.ndone += HTTPRangeSpec::writePartBoundary(vio.buffer.writer(), b_str, b_len
+                                                    , doc_len, resp_range[r_idx]._min, resp_range[r_idx]._max
+                                                    , resp_range.getContentTypeField(), r_idx >= (resp_range.count() - 1)
+        );
+      resp_range.consumeRangeShift();
+      Debug("amc", "Range boundary for range %" PRIu64, r_idx);
+    }
+
+    bytes = std::min(doc->len - doc_pos, static_cast<int64_t>(resp_range.getRemnantSize()));
+    bytes = std::min(bytes, vio.ntodo());
+    if (bytes > 0) {
+      b = new_IOBufferBlock(buf, bytes, doc_pos);
+      b->_buf_end = b->_end;
+      vio.buffer.writer()->append_block(b);
+      vio.ndone += bytes;
+      doc_pos += bytes;
+      resp_range.consume(bytes);
+      Debug("amc", "shipped %" PRId64 " bytes at target offset %" PRIu64, bytes, target_offset);
+      target_offset = resp_range.getOffset();
+    }
+
+    if (vio.ntodo() <= 0)
+      return calluser(VC_EVENT_READ_COMPLETE);
+    else if (calluser(VC_EVENT_READ_READY) == EVENT_DONE)
+      return EVENT_DONE;
+  }
+
+
 #ifdef HTTP_CACHE
-  if (resp_range.isActive()) {
-    int target = -1; // target fragment index.
-    uint64_t target_offset = resp_range.getOffset();
-    uint64_t lower_bound = frag_upper_bound - doc->data_len();
+  if (resp_range.getRemnantSize()) {
+    HTTPInfo::FragOffset* frags = alternate.get_frag_table();
+    int n_frags = alternate.get_frag_offset_count();
 
-    if (target_offset < lower_bound || frag_upper_bound <= target_offset) {
-      HTTPInfo::FragOffset* frags = alternate.get_frag_table();
+    // Quick check for offset in next fragment - very common
+    if (target_offset >= frag_upper_bound && (!frags || fragment >= (n_frags-1) || target_offset < frags[fragment])) {
+      Debug("amc", "Non-seeking continuation to next fragment");
+    } else {
+      int target = -1; // target fragment index.
 
       if (is_debug_tag_set("amc")) {
         char b[33], c[33];
@@ -710,100 +769,60 @@ CacheVC::openReadMain(int /* event ATS_UNUSED */, Event * /* e ATS_UNUSED */)
           );
       }
 
-      if (frags) {
-        int n_frags = alternate.get_frag_offset_count();
-        target = this->frag_idx_for_offset(frags, n_frags, target_offset);
-        this->update_key_to_frag_idx(target);
-        /// one frag short, because it gets bumped when the fragment is actually read.
-        frag_upper_bound = target > 0 ? frags[target-1] : 0;
-      } else { // we don't support non-monotonic forward requests on objects with no frag table.
-        ink_release_assert(target_offset >= frag_upper_bound);
-      }
-      goto Lread;
-    } else {
-      // Adjust fragment starting offset to account for target offset.
-      int r_doc_pos = doc->prefix_len() + (target_offset - lower_bound);
-      if (doc_pos < r_doc_pos) {
-        doc_pos = r_doc_pos;
-        bytes = doc->len - doc_pos;
-      }
-      bytes = std::min(bytes, static_cast<int64_t>(resp_range.getRemnantSize()));
+      target = this->frag_idx_for_offset(frags, n_frags, target_offset);
+      this->update_key_to_frag_idx(target);
+      /// one frag short, because it gets bumped when the fragment is actually read.
+      frag_upper_bound = target > 0 ? frags[target-1] : 0;
+      Debug("amc", "Fragment seek from %d to %d target offset %" PRIu64, fragment - 1, target, target_offset);
     }
+  }
 #endif
+
+  if (vio.ntodo() > 0 && 0 == resp_range.getRemnantSize())
+    // reached the end of the document and the user still wants more
+    return calluser(VC_EVENT_EOS);
+  last_collision = 0;
+  writer_lock_retry = 0;
+  // if the state machine calls reenable on the callback from the cache,
+  // we set up a schedule_imm event. The openReadReadDone discards
+  // EVENT_IMMEDIATE events. So, we have to cancel that trigger and set
+  // a new EVENT_INTERVAL event.
+  cancel_trigger();
+  CACHE_TRY_LOCK(lock, vol->mutex, mutex->thread_holding);
+  if (!lock.is_locked()) {
+    SET_HANDLER(&CacheVC::openReadMain);
+    VC_SCHED_LOCK_RETRY();
   }
-  if (ntodo <= 0)
+  if (dir_probe(&key, vol, &dir, &last_collision)) {
+    SET_HANDLER(&CacheVC::openReadReadDone);
+    int ret = do_read_call(&key);
+    if (ret == EVENT_RETURN)
+      goto Lcallreturn;
     return EVENT_CONT;
-  if (vio.buffer.writer()->max_read_avail() > vio.buffer.writer()->water_mark && vio.ndone) // initiate read of first block
-    return EVENT_CONT;
-  if ((bytes <= 0) && vio.ntodo() >= 0)
-    goto Lread;
-  // Do the write to downstream consumers.
-  if (bytes > vio.ntodo())
-    bytes = vio.ntodo();
-  b = new_IOBufferBlock(buf, bytes, doc_pos);
-  b->_buf_end = b->_end;
-  vio.buffer.writer()->append_block(b);
-  vio.ndone += bytes;
-  doc_pos += bytes;
-  resp_range.consume(bytes);
-  if (vio.ntodo() <= 0)
-    return calluser(VC_EVENT_READ_COMPLETE);
-  else {
-    if (calluser(VC_EVENT_READ_READY) == EVENT_DONE)
-      return EVENT_DONE;
-    // we have to keep reading until we give the user all the
-    // bytes it wanted or we hit the watermark.
-    if (vio.ntodo() > 0 && !vio.buffer.writer()->high_water())
-      goto Lread;
-    return EVENT_CONT;
-  }
-Lread: {
-    if (vio.ndone >= (int64_t)doc_len)
-      // reached the end of the document and the user still wants more
-      return calluser(VC_EVENT_EOS);
-    last_collision = 0;
-    writer_lock_retry = 0;
-    // if the state machine calls reenable on the callback from the cache,
-    // we set up a schedule_imm event. The openReadReadDone discards
-    // EVENT_IMMEDIATE events. So, we have to cancel that trigger and set
-    // a new EVENT_INTERVAL event.
-    cancel_trigger();
-    CACHE_TRY_LOCK(lock, vol->mutex, mutex->thread_holding);
-    if (!lock.is_locked()) {
-      SET_HANDLER(&CacheVC::openReadMain);
-      VC_SCHED_LOCK_RETRY();
-    }
-    if (dir_probe(&key, vol, &dir, &last_collision)) {
-      SET_HANDLER(&CacheVC::openReadReadDone);
-      int ret = do_read_call(&key);
-      if (ret == EVENT_RETURN)
-        goto Lcallreturn;
-      return EVENT_CONT;
-    } else if (write_vc) {
-      if (writer_done()) {
-        last_collision = NULL;
-        while (dir_probe(&earliest_key, vol, &dir, &last_collision)) {
-          if (dir_offset(&dir) == dir_offset(&earliest_dir)) {
-            DDebug("cache_read_agg", "%p: key: %X ReadMain complete: %d",
-                  this, first_key.slice32(1), (int)vio.ndone);
-            doc_len = vio.ndone;
-            goto Leos;
-          }
+  } else if (write_vc) {
+    if (writer_done()) {
+      last_collision = NULL;
+      while (dir_probe(&earliest_key, vol, &dir, &last_collision)) {
+        if (dir_offset(&dir) == dir_offset(&earliest_dir)) {
+          DDebug("cache_read_agg", "%p: key: %X ReadMain complete: %d",
+                 this, first_key.slice32(1), (int)vio.ndone);
+          doc_len = vio.ndone;
+          goto Leos;
         }
-        DDebug("cache_read_agg", "%p: key: %X ReadMain writer aborted: %d",
-              this, first_key.slice32(1), (int)vio.ndone);
-        goto Lerror;
       }
-      DDebug("cache_read_agg", "%p: key: %X ReadMain retrying: %d", this, first_key.slice32(1), (int)vio.ndone);
-      SET_HANDLER(&CacheVC::openReadMain);
-      VC_SCHED_WRITER_RETRY();
+      DDebug("cache_read_agg", "%p: key: %X ReadMain writer aborted: %d",
+             this, first_key.slice32(1), (int)vio.ndone);
+      goto Lerror;
     }
-    if (is_action_tag_set("cache"))
-      ink_release_assert(false);
-    Warning("Document %X truncated at %d of %d, missing fragment %X", first_key.slice32(1), (int)vio.ndone, (int)doc_len, key.slice32(1));
-    // remove the directory entry
-    dir_delete(&earliest_key, vol, &earliest_dir);
+    DDebug("cache_read_agg", "%p: key: %X ReadMain retrying: %d", this, first_key.slice32(1), (int)vio.ndone);
+    SET_HANDLER(&CacheVC::openReadMain);
+    VC_SCHED_WRITER_RETRY();
   }
+  if (is_action_tag_set("cache"))
+    ink_release_assert(false);
+  Warning("Document %X truncated at %d of %d, missing fragment %X", first_key.slice32(1), (int)vio.ndone, (int)doc_len, key.slice32(1));
+  // remove the directory entry
+  dir_delete(&earliest_key, vol, &earliest_dir);
 Lerror:
   return calluser(VC_EVENT_ERROR);
 Leos:
@@ -867,6 +886,7 @@ CacheVC::openReadStartEarliest(int /* event ATS_UNUSED */, Event * /* e ATS_UNUS
     doc_pos = doc->prefix_len();
     next_CacheKey(&key, &doc->key);
     fragment = 1;
+    frag_upper_bound = doc->data_len();
     vol->begin_read(this);
     if (vol->within_hit_evacuate_window(&earliest_dir) &&
         (!cache_config_hit_evacuate_size_limit || doc_len <= (uint64_t)cache_config_hit_evacuate_size_limit)
@@ -1152,14 +1172,19 @@ CacheVC::openReadStartHead(int event, Event * e)
       alternate.copy_shallow(alternate_tmp);
       alternate.object_key_get(&key);
       doc_len = alternate.object_size_get();
-      resp_range.setRangeSpec(&(alternate.response_get()->getRangeSpec()));
-      if (!resp_range.apply(request.getRangeSpec(), doc_len)) {
+
+      // Handle any range related setup.
+      if (!resp_range.init(&request)) {
+        err = ECACHE_INVALID_RANGE;
+        goto Ldone;
+      }
+      if (!resp_range.apply(doc_len)) {
         err = ECACHE_UNSATISFIABLE_RANGE;
         goto Ldone;
       }
-      resp_range.setContentType(alternate.response_get());
       if (resp_range.isMulti())
-        resp_range.generateBoundaryStr(earliest_key);
+        resp_range.setContentTypeFromResponse(alternate.response_get()).generateBoundaryStr(earliest_key);
+
 
       if (key == doc->key) {      // is this my data?
         f.single_fragment = doc->single_fragment();

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/2ddfc7ab/iocore/cache/I_Cache.h
----------------------------------------------------------------------
diff --git a/iocore/cache/I_Cache.h b/iocore/cache/I_Cache.h
index ffdf1ba..19e0592 100644
--- a/iocore/cache/I_Cache.h
+++ b/iocore/cache/I_Cache.h
@@ -58,6 +58,7 @@ class CacheLookupHttpConfig;
 class URL;
 class HTTPHdr;
 class HTTPInfo;
+class HTTPRangeSpec;
 
 typedef HTTPHdr CacheHTTPHdr;
 typedef URL CacheURL;
@@ -231,6 +232,12 @@ struct CacheVConnection:public VConnection
       This is the actual content sized modified (if appropriate) by range data.
   */
   virtual uint64_t get_http_content_size() = 0;
+  /** Get the range spec for the response (request ranges modifed by content length).
+   */
+  virtual HTTPRangeSpec& get_http_range_spec() = 0;
+
+  /// Check if this is HTTP partial content (range request/response).
+  virtual bool is_http_partial_content() = 0;
 #endif
 
   virtual bool is_ram_cache_hit() const = 0;

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/2ddfc7ab/iocore/cache/P_CacheHttp.h
----------------------------------------------------------------------
diff --git a/iocore/cache/P_CacheHttp.h b/iocore/cache/P_CacheHttp.h
index f76e842..91756b9 100644
--- a/iocore/cache/P_CacheHttp.h
+++ b/iocore/cache/P_CacheHttp.h
@@ -101,14 +101,10 @@ class CacheRange
   typedef CacheRange self; ///< Self reference type.
 
   /// Default constructor
- CacheRange() : _offset(0), _idx(-1), _r(NULL), _ct_field(NULL) { }
+ CacheRange() : _offset(0), _idx(-1), _ct_field(NULL), _pending_range_shift_p(false) { }
 
-  /// Set the internal range spec pointer to @a src.
-  self& setRangeSpec(HTTPRangeSpec* src);
-
-  /// Test if the range should be active (used).
-  /// @internal This means it has ranges and should be used to do seeks on the content.
-  bool isActive() const;
+  /// Test if the range spec has actual ranges in it
+  bool hasRanges() const;
 
   /// Test for multiple ranges.
   bool isMulti() const;
@@ -119,6 +115,9 @@ class CacheRange
   /// 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;
 
@@ -128,11 +127,15 @@ class CacheRange
   */
   uint64_t consume(uint64_t size);
 
-  /** Apply a @a src range and content @a len to the contained range spec.
+  /** Initialize from a request header.
+   */
+  bool init(HTTPHdr* req);
+
+  /** Apply a content @a len to the ranges.
 
       @return @c true if successfully applied, @c false otherwise.
   */
-  bool apply(HTTPRangeSpec const& src, uint64_t len);
+  bool apply(uint64_t len);
 
   /** Get the range boundary string.
       @a len if not @c NULL receives the length of the string.
@@ -140,25 +143,44 @@ class CacheRange
   char const* getBoundaryStr(int* len) const;
 
   /** Generate the range boundary string */
-  void generateBoundaryStr(CacheKey const& key);
+  self& generateBoundaryStr(CacheKey const& key);
 
-  /** Stash the Content-Type field pointer from a @a header.
+  /// Get the cached Content-Type field.
+  MIMEField* getContentTypeField() const;
 
-      @return @c true if a Content-Type field was found in @a header, @c false if not.
-  */
-  bool setContentType(HTTPHdr* header);
+  /// 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.
+  HTTPRangeSpec _r; ///< The actual ranges.
   MIMEField* _ct_field; ///< Content-Type field.
   char _boundary[HTTP_RANGE_BOUNDARY_LEN];
+  bool _pending_range_shift_p;
 };
 
 TS_INLINE CacheHTTPInfo *
@@ -169,23 +191,10 @@ CacheHTTPInfoVector::get(int idx)
   return &data[idx].alternate;
 }
 
-inline CacheRange&
-CacheRange::setRangeSpec(HTTPRangeSpec* src)
-{
-  _r = src;
-  return *this;
-}
-
-inline bool
-CacheRange::apply(HTTPRangeSpec const& src, uint64_t len)
-{
-  return _r && _r->apply(src, len);
-}
-
 inline bool
-CacheRange::isActive() const
+CacheRange::hasRanges() const
 {
-  return _r && (_r->isSingle() || _r->isMulti());
+  return _r.isSingle() || _r.isMulti();
 }
 
 inline uint64_t
@@ -205,10 +214,10 @@ CacheRange::getRemnantSize() const
 {
   uint64_t zret = 0;
 
-  if (!_r || _r->isEmpty())
+  if (_r.isEmpty())
     zret = _len - _offset;
-  else if (_r->isValid() && 0 <= _idx && _idx < static_cast<int>(_r->count()))
-    zret = ((*_r)[_idx]._max - _offset) + 1;
+  else if (_r.isValid() && 0 <= _idx && _idx < static_cast<int>(_r.count()))
+    zret = (_r[_idx]._max - _offset) + 1;
 
   return zret;
 }
@@ -220,10 +229,60 @@ CacheRange::getBoundaryStr(int* len) const
   return _boundary;
 }
 
+inline HTTPRangeSpec&
+CacheRange::getRangeSpec()
+{
+  return _r;
+}
+
 inline bool
 CacheRange::isMulti() const
 {
-  return _r && _r->isMulti();
+  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/2ddfc7ab/iocore/cache/P_CacheInternal.h
----------------------------------------------------------------------
diff --git a/iocore/cache/P_CacheInternal.h b/iocore/cache/P_CacheInternal.h
index e386c98..c23112b 100644
--- a/iocore/cache/P_CacheInternal.h
+++ b/iocore/cache/P_CacheInternal.h
@@ -376,6 +376,8 @@ struct CacheVC: public CacheVConnection
 
   virtual char const* get_http_range_boundary_string(int* len) const;
   virtual uint64_t get_http_content_size();
+  virtual HTTPRangeSpec& get_http_range_spec();
+  virtual bool is_http_partial_content();
 
 #endif
 
@@ -426,6 +428,7 @@ struct CacheVC: public CacheVConnection
 #ifdef CACHE_STAT_PAGES
   LINK(CacheVC, stat_link);
 #endif
+  CacheRange resp_range;          ///< Tracking information for range data for response.
   // end Region B
 
   // Start Region C
@@ -467,7 +470,6 @@ 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;
-  CacheRange resp_range;          ///< Tracking information for range data for response.
   /// 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.
@@ -648,6 +650,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);

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/2ddfc7ab/iocore/cluster/P_ClusterCache.h
----------------------------------------------------------------------
diff --git a/iocore/cluster/P_ClusterCache.h b/iocore/cluster/P_ClusterCache.h
index c7a3033..8325f07 100644
--- a/iocore/cluster/P_ClusterCache.h
+++ b/iocore/cluster/P_ClusterCache.h
@@ -358,6 +358,8 @@ struct ClusterVConnectionBase: public CacheVConnection
   // link problems so I can get the base case to work.
   virtual char const* get_http_range_boundary_string(int*) const { return NULL; }
   virtual uint64_t get_http_content_size() { return this->get_object_size(); }
+  virtual HTTPRangeSpec& get_http_range_spec() { return resp_range.getRangeSpec(); }
+  virtual bool is_http_partial_content() { return false; }
 
   // Set the timeouts associated with this connection.
   // active_timeout is for the total elasped time of the connection.
@@ -390,6 +392,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/2ddfc7ab/lib/ts/InkErrno.h
----------------------------------------------------------------------
diff --git a/lib/ts/InkErrno.h b/lib/ts/InkErrno.h
index 45a846d..14c17ea 100644
--- a/lib/ts/InkErrno.h
+++ b/lib/ts/InkErrno.h
@@ -66,7 +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_UNSATISFIABLE_RANGE        (CACHE_ERRNO+10)
+#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/2ddfc7ab/proxy/hdrs/HTTP.cc
----------------------------------------------------------------------
diff --git a/proxy/hdrs/HTTP.cc b/proxy/hdrs/HTTP.cc
index d2d5f69..58beb4f 100644
--- a/proxy/hdrs/HTTP.cc
+++ b/proxy/hdrs/HTTP.cc
@@ -29,6 +29,7 @@
 #include "HTTP.h"
 #include "HdrToken.h"
 #include "Diags.h"
+#include "I_IOBuffer.h"
 
 /***********************************************************************
  *                                                                     *
@@ -2151,7 +2152,7 @@ HTTPRangeSpec::parse(char const* v, int len)
   src.skip(&ParseRules::is_ws);
 
   if (src.size() > sizeof(HTTP_LEN_BYTES)+1 &&
-      0 == memcmp(src.data(), HTTP_VALUE_BYTES, HTTP_LEN_BYTES) && '=' == src[HTTP_LEN_BYTES]
+      0 == strncasecmp(src.data(), HTTP_VALUE_BYTES, HTTP_LEN_BYTES) && '=' == src[HTTP_LEN_BYTES]
   ) {
     _state = INVALID; // something, it needs to be correct.
     src += HTTP_LEN_BYTES+1;
@@ -2226,10 +2227,55 @@ HTTPRangeSpec::add(uint64_t low, uint64_t high)
 }
 
 bool
+HTTPRangeSpec::apply(uint64_t len)
+{
+  if (!this->hasRanges()) {
+    // nothing - simplifying later logic.
+  } else if (0 == len) {
+    /* Must special case zero length content
+       - suffix ranges are OK but other ranges are not.
+       - Best option is to return a 200 (not 206 or 416) for all suffix range spec on zero length content.
+         (this is what Apache HTTPD does)
+       - So, mark result as either @c UNSATISFIABLE or @c EMPTY, clear all ranges.
+    */
+    _state = EMPTY;
+    if (!_single.isSuffix()) _state = UNSATISFIABLE;
+    for ( RangeBox::iterator spot = _ranges.begin(), limit = _ranges.end() ; spot != limit && EMPTY == _state ; ++spot ) {
+      if (!spot->isSuffix()) _state = UNSATISFIABLE;
+    }
+    _ranges.clear();
+  } else if (this->isSingle()) {
+    if (!_single.apply(len)) _state = UNSATISFIABLE;
+  } else { // gotta be MULTI
+    int src = 0, dst = 0;
+    int n = _ranges.size();
+    while (src < n) {
+      Range& r = _ranges[src];
+      if (r.apply(len)) {
+        if (src != dst) _ranges[dst] = r;
+        ++dst;
+      }
+      ++src;
+    }
+    // at this point, @a dst is the # of valid ranges.
+    if (dst > 0) {
+      _single = _ranges[0];
+      if (dst == 1) _state = SINGLE;
+      _ranges.resize(dst);
+    } else {
+      _state = UNSATISFIABLE;
+      _ranges.clear();
+    }
+  }
+  return this->isValid();
+}
+
+# if 0
+bool
 HTTPRangeSpec::apply(self const& that, uint64_t len)
 {
   _state = that._state;
-  if (INVALID == _state || EMPTY == _state) {
+  if (INVALID == _state || EMPTY == _state || UNSATISFIABLE == _state) {
     // nothing - simplifying later logic.
   } else if (0 == len) {
     /* Must special case zero length content
@@ -2278,12 +2324,27 @@ HTTPHdr::getRangeSpec()
   m_range_parsed = true;
   return m_range_spec;
 }
+# endif
 
-int
-Calc_Digital_Length(uint64_t x)
+namespace {
+
+  int
+  Calc_Digital_Length(uint64_t x)
+  {
+    char buff[32]; // big enough for 64 bit #
+    return snprintf(buff, sizeof(buff), "%" PRIu64, x);
+  }
+
+}
+
+uint64_t
+HTTPRangeSpec::calcPartBoundarySize(uint64_t object_size, uint64_t ct_val_len)
 {
-  char buff[32]; // big enough for 64 bit #
-  return snprintf(buff, sizeof(buff), "%" PRIu64, x);
+  size_t l_size = Calc_Digital_Length(object_size);
+  // CR LF "--" boundary-string CR LF "Content-Range" ": " "bytes " X "-" Y "/" Z CR LF Content-Type CR LF
+  uint64_t zret = 4 + HTTP_RANGE_BOUNDARY_LEN + 2 + MIME_LEN_CONTENT_RANGE + 2 + HTTP_LEN_BYTES + 1 + l_size + 1 +l_size + 1 + l_size + 2;
+  if (ct_val_len) zret += MIME_LEN_CONTENT_TYPE + 2 + ct_val_len + 2;
+  return zret;
 }
 
 uint64_t
@@ -2293,16 +2354,62 @@ HTTPRangeSpec::calcContentLength(uint64_t object_size, uint64_t ct_val_len) cons
   size_t nr = this->count();
 
   if (nr >= 1) {
-    size = this->size();
-    if (nr > 1) {
-      size_t l_size = Calc_Digital_Length(object_size);
-      // CR LF "--" boundary-string CR LF "Content-Range" ": " "bytes " X "-" Y "/" Z
-      uint64_t sep_size = 2 + 2 + HTTP_RANGE_BOUNDARY_LEN + 2 + MIME_LEN_CONTENT_RANGE + 2 + HTTP_LEN_BYTES + 1 + l_size + 1 +l_size + 1 + l_size + 2;
-    
-      sep_size += MIME_LEN_CONTENT_TYPE + 2 + ct_val_len + 2;
-      size += nr * sep_size;
-    }
+    size = this->size(); // the real content size.
+    if (nr > 1) // part boundaries
+      size += nr * self::calcPartBoundarySize(object_size, ct_val_len) + 2; // need trailing '--'
   }
   return size;
 }
 
+uint64_t
+HTTPRangeSpec::writePartBoundary(MIOBuffer* out, char const* boundary_str, size_t boundary_len, uint64_t total_size, uint64_t low, uint64_t high, MIMEField* ctf, bool final)
+{
+  size_t x; // tmp for printf results.
+  size_t loc_size = Calc_Digital_Length(total_size)*3 + 3; // precomputed size of all the location / size text.
+  size_t n = self::calcPartBoundarySize(total_size, ctf ? ctf->m_len_value : 0) + (final ? 2 : 0);
+  Ptr<IOBufferData> d(new_IOBufferData(iobuffer_size_to_index(n, MAX_BUFFER_SIZE_INDEX), MEMALIGNED));
+  char* spot = d->data();
+
+  x = snprintf(spot, n, "\r\n--%.*s", static_cast<int>(boundary_len), boundary_str);
+  spot += x;
+  n -= x;
+  if (final) {
+    memcpy(spot, "--", 2);
+    spot += 2;
+    n -= 2;
+  }
+
+  x = snprintf(spot, n, "\r\n%.*s: %.*s", MIME_LEN_CONTENT_RANGE, MIME_FIELD_CONTENT_RANGE, HTTP_LEN_BYTES, HTTP_VALUE_BYTES);
+  spot += x;
+  n -= x;
+  spot[-HTTP_LEN_BYTES] = tolower(spot[-HTTP_LEN_BYTES]); // ugly cleanup just to be careful of stupid user agents.
+
+  x = snprintf(spot, n, " %" PRIu64 "-%" PRIu64 "/%" PRIu64, low, high, total_size);
+  // Need to space fill to match pre-computed size
+  if (x < loc_size) memset(spot+x, ' ', loc_size - x);
+  spot += loc_size;
+  n -= loc_size;
+
+  if (ctf) {
+    int ctf_len;
+    char const* ctf_val = ctf->value_get(&ctf_len);
+    if (ctf_val) {
+      x = snprintf(spot, n, "\r\n%.*s: %.*s", MIME_LEN_CONTENT_TYPE, MIME_FIELD_CONTENT_TYPE, ctf_len, ctf_val);
+      spot += x;
+      n -= x;
+    }
+  }
+
+  // This also takes care of the snprintf null termination problem.
+  *spot++ = '\r';
+  *spot++ = '\n';
+  n -= 2;
+
+  ink_assert(n == 0);
+  
+  IOBufferBlock* b = new_IOBufferBlock(d, spot - d->data());
+  b->_buf_end = b->_end;
+  out->append_block(b);
+
+  return spot - d->data();
+}

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/2ddfc7ab/proxy/hdrs/HTTP.h
----------------------------------------------------------------------
diff --git a/proxy/hdrs/HTTP.h b/proxy/hdrs/HTTP.h
index 559a3b2..91316e2 100644
--- a/proxy/hdrs/HTTP.h
+++ b/proxy/hdrs/HTTP.h
@@ -38,6 +38,7 @@
 #define HTTP_MAJOR(v)      (((v) >> 16) & 0xFFFF)
 
 class Http2HeaderTable;
+class MIOBuffer;
 
 enum HTTPStatus
 {
@@ -578,11 +579,15 @@ struct HTTPRangeSpec {
   /// Default constructor - invalid range
   HTTPRangeSpec();
 
+  /// Reset to re-usable state.
+  void clear();
+
   /** Parse a range field @a value and update @a this with the results.
       @return @c true if @a value was a valid range specifier, @c false otherwise.
   */
   bool parse(char const* value, int len);
 
+# if 0
   /** Copy ranges from @a while applying them to the content @a length.
 
       Ranges are copied if valid for @a length and converted to absolute offsets. The number of ranges
@@ -594,6 +599,18 @@ struct HTTPRangeSpec {
       always satisfiable.
    */
   bool apply(self const& that, uint64_t length);
+# endif
+
+  /** Update ranges to be absolute based on content @a length.
+
+      Invalid ranges are removed, ranges will be clipped as needed, and suffix ranges will be
+      converted to absolute ranges.
+
+      @return @c true if the range spec is satisfiable (there remains at least one valid range), @c false otherwise.
+      Note a range spec with no ranges is always satisfiable and that suffix ranges are also
+      always satisfiable.
+   */
+  bool apply(uint64_t length);
 
   /** Number of distinct ranges.
       @return Number of ranges.
@@ -624,6 +641,9 @@ struct HTTPRangeSpec {
   /// Access the range at index @a idx.
   Range& operator [] (int n);
 
+  /// Access the range at index @a idx.
+  Range const& operator [] (int n) const;
+
   /** Calculate the content length for this range specification.
 
       @note If a specific content length has not been @c apply 'd this will not produce
@@ -636,6 +656,25 @@ struct HTTPRangeSpec {
 			     uint64_t ct_val_len ///< Length of Content-Type field value.
 			     ) const;
 
+  /// Calculate the length of the range part boundary header.
+  static uint64_t calcPartBoundarySize(
+				       uint64_t object_size ///< Base content size
+				       , uint64_t ct_val_len ///< Length of the Content-Type value (0 if none).
+				       );
+
+  /** Write the range part boundary to @a out.
+   */
+  static uint64_t writePartBoundary(
+				    MIOBuffer* out ///< Output IO Buffer
+				    , char const* boundary_str ///< Boundary marker string.
+				    , size_t boundary_len ///< Length of boundary marker string.
+				    , uint64_t total_size ///< Base content size.
+				    , uint64_t low ///< Low value for the range.
+				    , uint64_t high ///< High value for the raNGE.
+				    , MIMEField* ctf ///< Content-Type field (@c NULL if none)
+				    , bool final ///< Is this the final part boundary?
+				    );
+
   /// Iterator for first range.
   iterator begin();
   /// Iterator past last range.
@@ -664,13 +703,6 @@ public:
   /// also had a port, @c false otherwise.
   mutable bool m_port_in_header;
 
-  /// Parsed data from the RANGE field.
-  /// For requests, this is the RANGE field specification.
-  /// For responses, this is the RANGE field applied to the content length.
-  HTTPRangeSpec m_range_spec;
-  /// Have we parsed the range field yet?
-  bool m_range_parsed;
-
   HTTPHdr();
   ~HTTPHdr();
 
@@ -793,15 +825,6 @@ public:
   const char *reason_get(int *length);
   void reason_set(const char *value, int length);
 
-  /// Get the internal @c HTTPRangeSpec instance.
-  HTTPRangeSpec& getRangeSpec();
-  /// Check if this response is a partial content response.
-  /// @return @c true if there is a least one range, @c false if not.
-  bool isPartialContent() const;
-  /// Locate and parse (if present) the @c Range header field.
-  /// The results are put in to the internal @c HTTPRangeSpec instance.
-  bool parse_range();
-
   MIMEParseResult parse_req(HTTPParser *parser, const char **start, const char *end, bool eof);
   MIMEParseResult parse_resp(HTTPParser *parser, const char **start, const char *end, bool eof);
 
@@ -950,7 +973,7 @@ HTTPVersion::operator <=(const HTTPVersion & hv) const
 
 inline
 HTTPHdr::HTTPHdr()
-  : MIMEHdr(), m_http(NULL), m_url_cached(), m_target_cached(false), m_range_parsed(false)
+  : MIMEHdr(), m_http(NULL), m_url_cached(), m_target_cached(false)
 { }
 
 
@@ -1679,6 +1702,13 @@ HTTPRangeSpec::HTTPRangeSpec() : _state(EMPTY)
 {
 }
 
+inline void
+HTTPRangeSpec::clear()
+{
+  _state = EMPTY;
+  RangeBox().swap(_ranges); // force memory drop.
+}
+
 inline bool
 HTTPRangeSpec::isSingle() const
 {
@@ -1768,7 +1798,7 @@ HTTPRangeSpec::Range::apply(uint64_t len)
     _max = len - 1;
     _min = _min > len ? 0 : len - _min;
   } else if (_min < len) {
-    _max = MIN(_max,len);
+    _max = MIN(_max,len-1);
   } else {
     this->invalidate();
     zret = false;
@@ -1782,6 +1812,12 @@ HTTPRangeSpec::operator [] (int n)
   return SINGLE == _state ? _single : _ranges[n];
 }
 
+inline HTTPRangeSpec::Range const&
+HTTPRangeSpec::operator [] (int n) const
+{
+  return SINGLE == _state ? _single : _ranges[n];
+}
+
 inline HTTPRangeSpec::iterator
 HTTPRangeSpec::begin()
 {
@@ -1802,10 +1838,4 @@ HTTPRangeSpec::end()
   }
 }
 
-inline bool
-HTTPHdr::isPartialContent() const
-{
-  return m_range_spec.hasRanges();
-}
-
 #endif /* __HTTP_H__ */

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/2ddfc7ab/proxy/http/HttpSM.cc
----------------------------------------------------------------------
diff --git a/proxy/http/HttpSM.cc b/proxy/http/HttpSM.cc
index d1edc50..bb2b3fc 100644
--- a/proxy/http/HttpSM.cc
+++ b/proxy/http/HttpSM.cc
@@ -5812,7 +5812,8 @@ HttpSM::setup_cache_read_transfer()
 
   ink_assert(cache_sm.cache_read_vc != NULL);
 
-  doc_size = t_state.cache_info.object_read->object_size_get();
+//  doc_size = t_state.cache_info.object_read->object_size_get();
+  doc_size = cache_sm.cache_read_vc->get_http_content_size();
   alloc_index = buffer_size_to_index(doc_size + index_to_buffer_size(HTTP_HEADER_BUFFER_SIZE_INDEX));
 
 #ifndef USE_NEW_EMPTY_MIOBUFFER

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/2ddfc7ab/proxy/http/HttpTransact.cc
----------------------------------------------------------------------
diff --git a/proxy/http/HttpTransact.cc b/proxy/http/HttpTransact.cc
index 169e9ce..07c28d9 100644
--- a/proxy/http/HttpTransact.cc
+++ b/proxy/http/HttpTransact.cc
@@ -6585,7 +6585,7 @@ HttpTransact::handle_content_length_header(State* s, HTTPHdr* header, HTTPHdr* b
       case SOURCE_CACHE:
         // if we are doing a single Range: request, calculate the new
         // C-L: header
-        if (base->isPartialContent()) {
+        if (s->state_machine->get_cache_sm().cache_read_vc->is_http_partial_content()) {
           change_response_header_because_of_range_request(s,header);
           s->hdr_info.trust_response_cl = true;
         }
@@ -8762,7 +8762,7 @@ HttpTransact::change_response_header_because_of_range_request(State *s, HTTPHdr
   char *reason_phrase;
   CacheVConnection* cache_read_vc = s->state_machine->get_cache_sm().cache_read_vc;
   HTTPHdr* cached_response = find_appropriate_cached_resp(s);
-  HTTPRangeSpec& rs = cached_response->getRangeSpec();
+  HTTPRangeSpec& rs = cache_read_vc->get_http_range_spec();
 
   Debug("http_trans", "Partial content requested, re-calculating content-length");
 
@@ -8794,6 +8794,7 @@ HttpTransact::change_response_header_because_of_range_request(State *s, HTTPHdr
                 , rs[0]._min, rs[0]._max
                 , cached_response->get_content_length()
       );
+//    buff[0] = tolower(buff[0]);
     field->value_set(header->m_heap, header->m_mime, buff, n);
     header->field_attach(field);
   }