You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@trafficserver.apache.org by zw...@apache.org on 2018/01/22 23:30:23 UTC

[trafficserver] branch 7.1.x updated: move the postbuf to HttpSM

This is an automated email from the ASF dual-hosted git repository.

zwoop pushed a commit to branch 7.1.x
in repository https://gitbox.apache.org/repos/asf/trafficserver.git


The following commit(s) were added to refs/heads/7.1.x by this push:
     new e879caf  move the postbuf to HttpSM
e879caf is described below

commit e879caf54f5c5e12464b41c62b9103125ebdf5fe
Author: scw00 <sc...@apache.org>
AuthorDate: Sat Nov 4 09:30:36 2017 +0800

    move the postbuf to HttpSM
    
    (cherry picked from commit b9988fd5ccdcc0a85a18b1d92777fc30aeb13b77)
---
 proxy/http/HttpSM.cc     |  88 ++++++++++++++++++++++++++++---------
 proxy/http/HttpSM.h      |  64 +++++++++++++++++++++++++++
 proxy/http/HttpTunnel.cc | 110 +++--------------------------------------------
 proxy/http/HttpTunnel.h  |  41 +++---------------
 4 files changed, 144 insertions(+), 159 deletions(-)

diff --git a/proxy/http/HttpSM.cc b/proxy/http/HttpSM.cc
index 53812bd..d63bea1 100644
--- a/proxy/http/HttpSM.cc
+++ b/proxy/http/HttpSM.cc
@@ -1984,8 +1984,7 @@ HttpSM::state_read_server_response_header(int event, void *data)
     if (enable_redirection && (redirection_tries <= t_state.txn_conf->number_of_redirections)) {
       ++redirection_tries;
     } else {
-      tunnel.deallocate_redirect_postdata_buffers();
-      enable_redirection = false;
+      this->disable_redirect();
     }
 
     do_api_callout();
@@ -2852,7 +2851,7 @@ HttpSM::tunnel_handler_cache_fill(int event, void *data)
   ink_release_assert(cache_sm.cache_write_vc);
 
   tunnel.deallocate_buffers();
-  tunnel.deallocate_redirect_postdata_buffers();
+  this->postbuf_clear();
   tunnel.reset();
 
   setup_server_transfer_to_cache_only();
@@ -2879,7 +2878,7 @@ HttpSM::tunnel_handler_100_continue(int event, void *data)
     //  does not free the memory from the header
     t_state.hdr_info.client_response.destroy();
     tunnel.deallocate_buffers();
-    tunnel.deallocate_redirect_postdata_buffers();
+    this->postbuf_clear();
     tunnel.reset();
 
     if (server_entry->eos) {
@@ -2930,7 +2929,7 @@ HttpSM::tunnel_handler_push(int event, void *data)
   // Reset tunneling state since we need to send a response
   //  to client as whether we succeeded
   tunnel.deallocate_buffers();
-  tunnel.deallocate_redirect_postdata_buffers();
+  this->postbuf_clear();
   tunnel.reset();
 
   if (cache_write_success) {
@@ -3596,8 +3595,6 @@ HttpSM::tunnel_handler_for_partial_post(int event, void * /* data ATS_UNUSED */)
   tunnel.deallocate_buffers();
   tunnel.reset();
 
-  tunnel.allocate_redirect_postdata_producer_buffer();
-
   t_state.redirect_info.redirect_in_process = false;
 
   if (post_failed) {
@@ -5406,8 +5403,7 @@ HttpSM::handle_post_failure()
 
   // disable redirection in case we got a partial response and then EOS, because the buffer might not
   // have the full post and it's deallocating the post buffers here
-  enable_redirection = false;
-  tunnel.deallocate_redirect_postdata_buffers();
+  this->disable_redirect();
 
   // Don't even think about doing keep-alive after this debacle
   t_state.client_info.keep_alive     = HTTP_NO_KEEPALIVE;
@@ -5681,19 +5677,17 @@ HttpSM::do_setup_post_tunnel(HttpVC_t to_vc_type)
   // YTS Team, yamsat Plugin
   // if redirect_in_process and redirection is enabled add static producer
 
-  if (t_state.redirect_info.redirect_in_process && enable_redirection &&
-      (tunnel.postbuf && tunnel.postbuf->postdata_copy_buffer_start != nullptr &&
-       tunnel.postbuf->postdata_producer_buffer != nullptr)) {
+  if (t_state.redirect_info.redirect_in_process && enable_redirection && (this->_postbuf.postdata_copy_buffer_start != nullptr)) {
     post_redirect = true;
     // copy the post data into a new producer buffer for static producer
-    tunnel.postbuf->postdata_producer_buffer->write(tunnel.postbuf->postdata_copy_buffer_start);
-    int64_t post_bytes = tunnel.postbuf->postdata_producer_reader->read_avail();
+    MIOBuffer *postdata_producer_buffer      = new_empty_MIOBuffer();
+    IOBufferReader *postdata_producer_reader = postdata_producer_buffer->alloc_reader();
+
+    postdata_producer_buffer->write(this->_postbuf.postdata_copy_buffer_start);
+    int64_t post_bytes = postdata_producer_reader->read_avail();
     transfered_bytes   = post_bytes;
-    p                  = tunnel.add_producer(HTTP_TUNNEL_STATIC_PRODUCER, post_bytes, tunnel.postbuf->postdata_producer_reader,
-                            (HttpProducerHandler) nullptr, HT_STATIC, "redirect static agent post");
-    // the tunnel has taken over the buffer and will free it
-    tunnel.postbuf->postdata_producer_buffer = nullptr;
-    tunnel.postbuf->postdata_producer_reader = nullptr;
+    p = tunnel.add_producer(HTTP_TUNNEL_STATIC_PRODUCER, post_bytes, postdata_producer_reader, (HttpProducerHandler) nullptr,
+                            HT_STATIC, "redirect static agent post");
   } else {
     int64_t alloc_index;
     // content length is undefined, use default buffer size
@@ -5709,6 +5703,10 @@ HttpSM::do_setup_post_tunnel(HttpVC_t to_vc_type)
     IOBufferReader *buf_start = post_buffer->alloc_reader();
     int64_t post_bytes        = chunked ? INT64_MAX : t_state.hdr_info.request_content_length;
 
+    if (enable_redirection) {
+      this->_postbuf.init(post_buffer->clone_reader(buf_start));
+    }
+
     // Note: Many browsers, Netscape and IE included send two extra
     //  bytes (CRLF) at the end of the post.  We just ignore those
     //  bytes since the sending them is not spec
@@ -6849,7 +6847,7 @@ void
 HttpSM::kill_this()
 {
   ink_release_assert(reentrancy_count == 1);
-  tunnel.deallocate_redirect_postdata_buffers();
+  this->postbuf_clear();
   enable_redirection = false;
 
   if (kill_this_async_done == false) {
@@ -7766,7 +7764,7 @@ HttpSM::do_redirect()
 {
   DebugSM("http_redirect", "[HttpSM::do_redirect]");
   if (!enable_redirection || redirection_tries > t_state.txn_conf->number_of_redirections) {
-    tunnel.deallocate_redirect_postdata_buffers();
+    this->postbuf_clear();
     return;
   }
 
@@ -8153,3 +8151,51 @@ HttpSM::find_proto_string(HTTPVersion version) const
   }
   return nullptr;
 }
+
+// YTS Team, yamsat Plugin
+// Function to copy the partial Post data while tunnelling
+void
+PostDataBuffers::copy_partial_post_data()
+{
+  this->postdata_copy_buffer->write(this->ua_buffer_reader);
+  Debug("http_redirect", "[PostDataBuffers::copy_partial_post_data] wrote %" PRId64 " bytes to buffers %" PRId64 "",
+        this->ua_buffer_reader->read_avail(), this->postdata_copy_buffer_start->read_avail());
+  this->ua_buffer_reader->consume(this->ua_buffer_reader->read_avail());
+}
+
+// YTS Team, yamsat Plugin
+// Allocating the post data buffers
+void
+PostDataBuffers::init(IOBufferReader *ua_reader)
+{
+  Debug("http_redirect", "[PostDataBuffers::init]");
+
+  this->ua_buffer_reader = ua_reader;
+
+  if (this->postdata_copy_buffer == nullptr) {
+    ink_assert(this->postdata_copy_buffer_start == nullptr);
+    this->postdata_copy_buffer       = new_empty_MIOBuffer(BUFFER_SIZE_INDEX_4K);
+    this->postdata_copy_buffer_start = this->postdata_copy_buffer->alloc_reader();
+  }
+
+  ink_assert(this->ua_buffer_reader != nullptr);
+}
+
+// YTS Team, yamsat Plugin
+// Deallocating the post data buffers
+void
+PostDataBuffers::clear()
+{
+  Debug("http_redirect", "[PostDataBuffers::clear]");
+
+  if (this->postdata_copy_buffer != nullptr) {
+    free_MIOBuffer(this->postdata_copy_buffer);
+    this->postdata_copy_buffer       = nullptr;
+    this->postdata_copy_buffer_start = nullptr; // deallocated by the buffer
+  }
+}
+
+PostDataBuffers::~PostDataBuffers()
+{
+  this->clear();
+}
diff --git a/proxy/http/HttpSM.h b/proxy/http/HttpSM.h
index e9bd219..e14e941 100644
--- a/proxy/http/HttpSM.h
+++ b/proxy/http/HttpSM.h
@@ -177,6 +177,22 @@ enum HttpPluginTunnel_t {
 class CoreUtils;
 class PluginVCCore;
 
+class PostDataBuffers
+{
+public:
+  PostDataBuffers() { Debug("http_redirect", "[PostDataBuffers::PostDataBuffers]"); }
+
+  MIOBuffer *postdata_copy_buffer            = nullptr;
+  IOBufferReader *postdata_copy_buffer_start = nullptr;
+  IOBufferReader *ua_buffer_reader           = nullptr;
+
+  void clear();
+  void init(IOBufferReader *ua_reader);
+  void copy_partial_post_data();
+
+  ~PostDataBuffers();
+};
+
 class HttpSM : public Continuation
 {
   friend class HttpPagesHandler;
@@ -289,6 +305,14 @@ public:
 
   HttpTransact::State t_state;
 
+  // _postbuf api
+  int64_t postbuf_reader_avail();
+  int64_t postbuf_buffer_avail();
+  void postbuf_clear();
+  void disable_redirect();
+  void postbuf_copy_partial_data();
+  void postbuf_init(IOBufferReader *ua_reader);
+
 protected:
   int reentrancy_count;
 
@@ -561,6 +585,9 @@ public:
   {
     return terminate_sm;
   }
+
+private:
+  PostDataBuffers _postbuf;
 };
 
 // Function to get the cache_sm object - YTS Team, yamsat
@@ -668,4 +695,41 @@ HttpSM::is_transparent_passthrough_allowed()
           ua_session->get_transact_count() == 1);
 }
 
+inline int64_t
+HttpSM::postbuf_reader_avail()
+{
+  return this->_postbuf.ua_buffer_reader->read_avail();
+}
+
+inline int64_t
+HttpSM::postbuf_buffer_avail()
+{
+  return this->_postbuf.postdata_copy_buffer_start->read_avail();
+}
+
+inline void
+HttpSM::postbuf_clear()
+{
+  this->_postbuf.clear();
+}
+
+inline void
+HttpSM::disable_redirect()
+{
+  this->enable_redirection = false;
+  this->_postbuf.clear();
+}
+
+inline void
+HttpSM::postbuf_copy_partial_data()
+{
+  this->_postbuf.copy_partial_post_data();
+}
+
+inline void
+HttpSM::postbuf_init(IOBufferReader *ua_reader)
+{
+  this->_postbuf.init(ua_reader);
+}
+
 #endif
diff --git a/proxy/http/HttpTunnel.cc b/proxy/http/HttpTunnel.cc
index d21e79d..403c6fd 100644
--- a/proxy/http/HttpTunnel.cc
+++ b/proxy/http/HttpTunnel.cc
@@ -528,15 +528,7 @@ HttpTunnelConsumer::HttpTunnelConsumer()
 {
 }
 
-HttpTunnel::HttpTunnel()
-  : Continuation(nullptr),
-    num_producers(0),
-    num_consumers(0),
-    sm(nullptr),
-    active(false),
-    postbuf(nullptr),
-    reentrancy_count(0),
-    call_sm(false)
+HttpTunnel::HttpTunnel() : Continuation(nullptr)
 {
 }
 
@@ -589,7 +581,6 @@ HttpTunnel::kill_tunnel()
     ink_assert(producer.alive == false);
   }
   active = false;
-  this->deallocate_redirect_postdata_buffers();
   this->deallocate_buffers();
   this->reset();
 }
@@ -630,9 +621,6 @@ HttpTunnel::deallocate_buffers()
   for (auto &producer : producers) {
     if (producer.read_buffer != nullptr) {
       ink_assert(producer.vc != nullptr);
-      if (postbuf && postbuf->ua_buffer_reader && postbuf->ua_buffer_reader->mbuf == producer.read_buffer) {
-        postbuf->ua_buffer_reader = nullptr;
-      }
       free_MIOBuffer(producer.read_buffer);
       producer.read_buffer  = nullptr;
       producer.buffer_start = nullptr;
@@ -998,11 +986,9 @@ HttpTunnel::producer_run(HttpTunnelProducer *p)
     if (p->buffer_start->read_avail() > HttpConfig::m_master.post_copy_size) {
       Debug("http_redirect", "[HttpTunnel::producer_handler] post exceeds buffer limit, buffer_avail=%" PRId64 " limit=%" PRId64 "",
             p->buffer_start->read_avail(), HttpConfig::m_master.post_copy_size);
-      sm->enable_redirection = false;
+      sm->disable_redirect();
     } else {
-      // allocate post buffers with a new reader. The reader will be freed when p->read_buffer is freed
-      allocate_redirect_postdata_buffers(p->read_buffer->clone_reader(p->buffer_start));
-      copy_partial_post_data();
+      sm->postbuf_copy_partial_data();
     }
   } // end of added logic for partial POST
 
@@ -1199,16 +1185,13 @@ HttpTunnel::producer_handler(int event, HttpTunnelProducer *p)
       (event == VC_EVENT_READ_READY || event == VC_EVENT_READ_COMPLETE) && (p->vc_type == HT_HTTP_CLIENT)) {
     Debug("http_redirect", "[HttpTunnel::producer_handler] [%s %s]", p->name, HttpDebugNames::get_event_name(event));
 
-    if ((postbuf->postdata_copy_buffer_start->read_avail() + postbuf->ua_buffer_reader->read_avail()) >
-        HttpConfig::m_master.post_copy_size) {
+    if ((sm->postbuf_buffer_avail() + sm->postbuf_reader_avail()) > HttpConfig::m_master.post_copy_size) {
       Debug("http_redirect", "[HttpTunnel::producer_handler] post exceeds buffer limit, buffer_avail=%" PRId64
                              " reader_avail=%" PRId64 " limit=%" PRId64 "",
-            postbuf->postdata_copy_buffer_start->read_avail(), postbuf->ua_buffer_reader->read_avail(),
-            HttpConfig::m_master.post_copy_size);
-      deallocate_redirect_postdata_buffers();
-      sm->enable_redirection = false;
+            sm->postbuf_buffer_avail(), sm->postbuf_reader_avail(), HttpConfig::m_master.post_copy_size);
+      sm->disable_redirect();
     } else {
-      copy_partial_post_data();
+      sm->postbuf_copy_partial_data();
     }
   } // end of added logic for partial copy of POST
 
@@ -1692,82 +1675,3 @@ void
 HttpTunnel::internal_error()
 {
 }
-
-// YTS Team, yamsat Plugin
-// Function to copy the partial Post data while tunnelling
-void
-HttpTunnel::copy_partial_post_data()
-{
-  postbuf->postdata_copy_buffer->write(postbuf->ua_buffer_reader);
-  Debug("http_redirect", "[HttpTunnel::copy_partial_post_data] wrote %" PRId64 " bytes to buffers %" PRId64 "",
-        postbuf->ua_buffer_reader->read_avail(), postbuf->postdata_copy_buffer_start->read_avail());
-  postbuf->ua_buffer_reader->consume(postbuf->ua_buffer_reader->read_avail());
-}
-
-// YTS Team, yamsat Plugin
-// Allocate a new buffer for static producers
-void
-HttpTunnel::allocate_redirect_postdata_producer_buffer()
-{
-  int64_t alloc_index = buffer_size_to_index(sm->t_state.hdr_info.request_content_length);
-
-  ink_release_assert(postbuf->postdata_producer_buffer == nullptr);
-
-  postbuf->postdata_producer_buffer = new_MIOBuffer(alloc_index);
-  postbuf->postdata_producer_reader = postbuf->postdata_producer_buffer->alloc_reader();
-}
-
-// YTS Team, yamsat Plugin
-// Allocating the post data buffers
-void
-HttpTunnel::allocate_redirect_postdata_buffers(IOBufferReader *ua_reader)
-{
-  int64_t alloc_index = buffer_size_to_index(sm->t_state.hdr_info.request_content_length);
-
-  Debug("http_redirect", "[HttpTunnel::allocate_postdata_buffers]");
-
-  // TODO: This is uncool, shouldn't this use the class allocator or proxy allocator ?
-  // If fixed, obviously also fix the deallocator.
-  if (postbuf == nullptr) {
-    postbuf                             = new PostDataBuffers();
-    postbuf->ua_buffer_reader           = ua_reader;
-    postbuf->postdata_copy_buffer       = new_MIOBuffer(alloc_index);
-    postbuf->postdata_copy_buffer_start = postbuf->postdata_copy_buffer->alloc_reader();
-    allocate_redirect_postdata_producer_buffer();
-  } else {
-    // Reset the buffer readers
-    postbuf->postdata_copy_buffer->dealloc_reader(postbuf->postdata_copy_buffer_start);
-    postbuf->postdata_copy_buffer_start = postbuf->postdata_copy_buffer->alloc_reader();
-    postbuf->postdata_producer_buffer->dealloc_reader(postbuf->postdata_producer_reader);
-    postbuf->postdata_producer_reader = postbuf->postdata_producer_buffer->alloc_reader();
-  }
-}
-
-// YTS Team, yamsat Plugin
-// Deallocating the post data buffers
-void
-HttpTunnel::deallocate_redirect_postdata_buffers()
-{
-  Debug("http_redirect", "[HttpTunnel::deallocate_postdata_copy_buffers]");
-
-  if (postbuf != nullptr) {
-    if (postbuf->postdata_producer_buffer != nullptr) {
-      free_MIOBuffer(postbuf->postdata_producer_buffer);
-      postbuf->postdata_producer_buffer = nullptr;
-      postbuf->postdata_producer_reader = nullptr; // deallocated by the buffer
-    }
-    if (postbuf->postdata_copy_buffer != nullptr) {
-      free_MIOBuffer(postbuf->postdata_copy_buffer);
-      postbuf->postdata_copy_buffer       = nullptr;
-      postbuf->postdata_copy_buffer_start = nullptr; // deallocated by the buffer
-    }
-
-    if (postbuf->ua_buffer_reader != nullptr) {
-      postbuf->ua_buffer_reader->mbuf->dealloc_reader(postbuf->ua_buffer_reader);
-      postbuf->ua_buffer_reader = nullptr;
-    }
-
-    delete postbuf;
-    postbuf = nullptr;
-  }
-}
diff --git a/proxy/http/HttpTunnel.h b/proxy/http/HttpTunnel.h
index 54f5314..42c6fd4 100644
--- a/proxy/http/HttpTunnel.h
+++ b/proxy/http/HttpTunnel.h
@@ -253,26 +253,6 @@ struct HttpTunnelProducer {
                         );
 };
 
-class PostDataBuffers
-{
-public:
-  PostDataBuffers()
-    : postdata_producer_buffer(NULL),
-      postdata_copy_buffer(NULL),
-      postdata_producer_reader(NULL),
-      postdata_copy_buffer_start(NULL),
-      ua_buffer_reader(NULL)
-  {
-    Debug("http_redirect", "[PostDataBuffers::PostDataBuffers]");
-  }
-
-  MIOBuffer *postdata_producer_buffer;
-  MIOBuffer *postdata_copy_buffer;
-  IOBufferReader *postdata_producer_reader;
-  IOBufferReader *postdata_copy_buffer_start;
-  IOBufferReader *ua_buffer_reader;
-};
-
 class HttpTunnel : public Continuation
 {
   friend class HttpPagesHandler;
@@ -311,12 +291,6 @@ public:
   bool is_tunnel_alive() const;
   bool has_cache_writer() const;
 
-  // YTS Team, yamsat Plugin
-  void copy_partial_post_data();
-  void allocate_redirect_postdata_producer_buffer();
-  void allocate_redirect_postdata_buffers(IOBufferReader *ua_reader);
-  void deallocate_redirect_postdata_buffers();
-
   HttpTunnelProducer *add_producer(VConnection *vc, int64_t nbytes, IOBufferReader *reader_start, HttpProducerHandler sm_handler,
                                    HttpTunnelType_t vc_type, const char *name);
 
@@ -373,23 +347,20 @@ private:
   HttpTunnelProducer *alloc_producer();
   HttpTunnelConsumer *alloc_consumer();
 
-  int num_producers;
-  int num_consumers;
+  int num_producers = 0;
+  int num_consumers = 0;
   HttpTunnelConsumer consumers[MAX_CONSUMERS];
   HttpTunnelProducer producers[MAX_PRODUCERS];
-  HttpSM *sm;
+  HttpSM *sm = nullptr;
 
-  bool active;
+  bool active = false;
 
   /// State data about flow control.
   FlowControl flow_state;
 
-public:
-  PostDataBuffers *postbuf;
-
 private:
-  int reentrancy_count;
-  bool call_sm;
+  int reentrancy_count = 0;
+  bool call_sm         = false;
 };
 
 // void HttpTunnel::abort_cache_write_finish_others

-- 
To stop receiving notification emails like this one, please contact
zwoop@apache.org.