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 2014/04/03 00:44:48 UTC

git commit: TS-2657 Eliminate TSHttpTxnSetHttpRetBody

Repository: trafficserver
Updated Branches:
  refs/heads/master 21e2f33c9 -> 7bac4a99c


TS-2657 Eliminate TSHttpTxnSetHttpRetBody

This patch does the following:

1. First, both TSHttpTxnSetHttpRetBody() and TSHttpTxnGetMaxHttpRetBodySize()
are removed. I’ve patched all “core” plugins to use the
TSHttpTxnErrorBodySet() API, but any non-Apache plugin would break. Hence,
this change is going in for v5.0.0, which allows breaking compatibility.

2. More importantly, TSHttpTxnErrorBodySet() requires for the body to be heap
allocated, whereas TSHttpTxnSetHttpRetBody() used a fixed size (2KB) array on
the HttpSM. This was the primary reason why I started looking into this, since
this “simple” change reduces the size of the HttpSM by 25%.

3. In order to make the TSHttpTxnErrorBodySet() API easier to use, I’ve
made the content type value NULL imply a default of "text/html". This is by
far the most common use case, and saves a good chunk of allocations.

4. The TSHttpTxnErrorBodySet() also imposes the body to be sent through the
body factory, for log field expansion. For starters, this is incredibly
inefficient , as well as imposing an 8KB size limit. This is something I
intend to ameliorate soon by improvements to the body factory. However, this
is an unnecessary inefficiency IMO; As part of the body factory improvements,
I also plan on exposing the body factory “log field” expansion as a generic
API. This would allow any plugin to decide if they want to go through the
pains of body factory expansions or not (and not limited to just error pages,
e.g. Header Rewrite would use this new API).


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

Branch: refs/heads/master
Commit: 7bac4a99c1ad86b1226df84aa497ff11b9b623a7
Parents: 21e2f33
Author: Leif Hedstrom <zw...@apache.org>
Authored: Mon Mar 24 15:10:10 2014 -0600
Committer: Leif Hedstrom <zw...@apache.org>
Committed: Wed Apr 2 16:40:19 2014 -0600

----------------------------------------------------------------------
 example/remap/remap.cc                      | 13 ++---
 lib/atscppapi/src/Transaction.cc            | 11 +----
 plugins/experimental/authproxy/authproxy.cc |  4 +-
 plugins/experimental/geoip_acl/acl.h        | 12 +++--
 plugins/experimental/lua/lapi.cc            |  9 ++--
 plugins/experimental/ts_lua/ts_lua_http.c   |  4 +-
 proxy/InkAPI.cc                             | 62 ++++++++----------------
 proxy/api/ts/ts.h                           | 17 +++++--
 proxy/http/HttpBodyFactory.cc               |  2 +-
 proxy/http/HttpSM.cc                        | 25 ++++++----
 proxy/http/HttpTransact.cc                  | 51 ++++++++++---------
 proxy/http/HttpTransact.h                   | 45 +++++++----------
 tools/apichecker.pl                         |  3 +-
 13 files changed, 120 insertions(+), 138 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/trafficserver/blob/7bac4a99/example/remap/remap.cc
----------------------------------------------------------------------
diff --git a/example/remap/remap.cc b/example/remap/remap.cc
index 941bcaa..f6d8bc0 100644
--- a/example/remap/remap.cc
+++ b/example/remap/remap.cc
@@ -310,13 +310,14 @@ TSRemapDoRemap(void* ih, TSHttpTxn rh, TSRemapRequestInfo *rri)
   // How to cancel request processing and return error message to the client
   // We wiil do it each other request
   if (_processing_counter & 1) {
-    char tmp[256];
+    char* tmp = (char*)TSmalloc(256);
     static int my_local_counter = 0;
-    snprintf(tmp, sizeof(tmp) - 1,
-             "This is very small example of TS API usage!\nIteration %d!\nHTTP return code %d\n",
-             my_local_counter, TS_HTTP_STATUS_CONTINUE + my_local_counter);
-    TSHttpTxnSetHttpRetStatus((TSHttpTxn) rh, (TSHttpStatus) ((int) TS_HTTP_STATUS_CONTINUE + my_local_counter));   //TS_HTTP_STATUS_SERVICE_UNAVAILABLE); //TS_HTTP_STATUS_NOT_ACCEPTABLE);
-    TSHttpTxnSetHttpRetBody((TSHttpTxn) rh, (const char *) tmp, true);
+
+    size_t len = snprintf(tmp, sizeof(tmp) - 1,
+                          "This is very small example of TS API usage!\nIteration %d!\nHTTP return code %d\n",
+                          my_local_counter, TS_HTTP_STATUS_CONTINUE + my_local_counter);
+    TSHttpTxnSetHttpRetStatus((TSHttpTxn) rh, (TSHttpStatus) ((int) TS_HTTP_STATUS_CONTINUE + my_local_counter)); 
+    TSHttpTxnErrorBodySet((TSHttpTxn) rh, tmp, len, NULL); // Defaults to text/html
     my_local_counter++;
   }
   // hardcoded case for remapping

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/7bac4a99/lib/atscppapi/src/Transaction.cc
----------------------------------------------------------------------
diff --git a/lib/atscppapi/src/Transaction.cc b/lib/atscppapi/src/Transaction.cc
index f5bde35..2470e7f 100644
--- a/lib/atscppapi/src/Transaction.cc
+++ b/lib/atscppapi/src/Transaction.cc
@@ -113,16 +113,7 @@ void Transaction::error(const std::string &page) {
 
 void Transaction::setErrorBody(const std::string &page) {
   LOG_DEBUG("Transaction tshttptxn=%p setting error body page: %s", state_->txn_, page.c_str());
-  char *res_bdy = static_cast<char*>(TSmalloc(page.length() + 1));
-  strncpy(res_bdy, page.c_str(), page.length());
-  res_bdy[page.length()] = '\0';
-
-  std::string str_content_type = "text/html";
-  char *content_type = static_cast<char*>(TSmalloc(str_content_type.length() + 1));
-  strncpy(content_type, str_content_type.c_str(), str_content_type.length());
-  content_type[str_content_type.length()] = '\0';
-
-  TSHttpTxnErrorBodySet(state_->txn_, res_bdy, page.length(), content_type);
+  TSHttpTxnErrorBodySet(state_->txn_, TSstrdup(page.c_str()), page.length(), NULL); // Default to text/html
 }
 
 bool Transaction::isInternalRequest() const {

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/7bac4a99/plugins/experimental/authproxy/authproxy.cc
----------------------------------------------------------------------
diff --git a/plugins/experimental/authproxy/authproxy.cc b/plugins/experimental/authproxy/authproxy.cc
index c11d1bf..a27614d 100644
--- a/plugins/experimental/authproxy/authproxy.cc
+++ b/plugins/experimental/authproxy/authproxy.cc
@@ -538,7 +538,7 @@ StateAuthProxySendResponse(AuthRequestContext * auth, void * /* edata ATS_UNUSED
     status = TSHttpHdrStatusGet(mbuf, mhdr),
     snprintf(msg, sizeof(msg), "%d %s\n", status, TSHttpHdrReasonLookup(status));
 
-    TSHttpTxnErrorBodySet(auth->txn, TSstrdup(msg), strlen(msg), NULL);
+    TSHttpTxnErrorBodySet(auth->txn, TSstrdup(msg), strlen(msg), TSstrdup("text/plain"));
 
     // We must not whack the content length for HEAD responses, since the
     // client already knows that there is no body. Forcing content length to
@@ -664,7 +664,7 @@ StateUnauthorized(AuthRequestContext * auth, void *)
     static const char msg[] = "authorization denied\n";
 
     TSHttpTxnSetHttpRetStatus(auth->txn, TS_HTTP_STATUS_FORBIDDEN);
-    TSHttpTxnErrorBodySet(auth->txn, TSstrdup(msg), sizeof(msg) - 1, NULL);
+    TSHttpTxnErrorBodySet(auth->txn, TSstrdup(msg), sizeof(msg) - 1, TSstrdup("text/plain"));
 
     TSHttpTxnReenable(auth->txn, TS_EVENT_HTTP_ERROR);
     return TS_EVENT_CONTINUE;

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/7bac4a99/plugins/experimental/geoip_acl/acl.h
----------------------------------------------------------------------
diff --git a/plugins/experimental/geoip_acl/acl.h b/plugins/experimental/geoip_acl/acl.h
index adc91b9..ee7657a 100644
--- a/plugins/experimental/geoip_acl/acl.h
+++ b/plugins/experimental/geoip_acl/acl.h
@@ -43,7 +43,7 @@ class Acl
 {
 public:
   Acl()
-    : _html(""), _allow(true), _added_tokens(0)
+    : _allow(true), _added_tokens(0)
   { }
 
   virtual ~Acl()
@@ -53,15 +53,17 @@ public:
   virtual void read_regex(const char* fn) = 0;
   virtual void process_args(int argc, char* argv[]) = 0;
   virtual bool eval(TSRemapRequestInfo *rri, TSHttpTxn txnp) const = 0;
-  virtual void add_token(const std::string& /* str */) { ++_added_tokens; }
 
+  virtual void add_token(const std::string& /* str */) { ++_added_tokens; }
   void set_allow(bool allow) { _allow = allow; }
-  const char* get_html() const { return _html.c_str(); }
 
   void send_html(TSHttpTxn txnp) const
   {
-    if (_html.size() > 0)
-      TSHttpTxnSetHttpRetBody(txnp, _html.c_str(), true);
+    if (_html.size() > 0) {
+      char* msg = TSstrdup(_html.c_str());
+
+      TSHttpTxnErrorBodySet(txnp, msg, _html.size(), NULL); // Defaults to text/html
+    }
   }
 
   void read_html(const char* fn);

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/7bac4a99/plugins/experimental/lua/lapi.cc
----------------------------------------------------------------------
diff --git a/plugins/experimental/lua/lapi.cc b/plugins/experimental/lua/lapi.cc
index efa1b3d..6121113 100644
--- a/plugins/experimental/lua/lapi.cc
+++ b/plugins/experimental/lua/lapi.cc
@@ -263,22 +263,21 @@ LuaRemapReject(lua_State * lua)
 {
   LuaRemapRequest * rq;
   int status;
+  size_t body_len;
   const char * body = NULL;
 
   rq = LuaRemapRequest::get(lua, 1);
   status = luaL_checkint(lua, 2);
   if (!lua_isnoneornil(lua, 3)) {
-    body = luaL_checkstring(lua, 3);
+    body = luaL_checklstring(lua, 3, &body_len);
+    // body = luaL_checkstring(lua, 3);
   }
 
   LuaLogDebug("rejecting request %p with status %d", rq->rri, status);
 
   TSHttpTxnSetHttpRetStatus(rq->txn, (TSHttpStatus)status);
   if (body && *body) {
-    // XXX Dubiously guess the content type from the body. This doesn't actually seem to work
-    // so it doesn't matter that our guess is pretty bad.
-    int isplain = (*body != '<');
-    TSHttpTxnSetHttpRetBody(rq->txn, body, isplain);
+    TSHttpTxnErrorBodySet(rq->txn, TSstrdup(body), body_len, NULL); // Defaults to text/html
   }
 
   // A reject terminates plugin chain evaluation but does not update the request URL.

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/7bac4a99/plugins/experimental/ts_lua/ts_lua_http.c
----------------------------------------------------------------------
diff --git a/plugins/experimental/ts_lua/ts_lua_http.c b/plugins/experimental/ts_lua/ts_lua_http.c
index b800881..ba28364 100644
--- a/plugins/experimental/ts_lua/ts_lua_http.c
+++ b/plugins/experimental/ts_lua/ts_lua_http.c
@@ -131,7 +131,7 @@ ts_lua_http_set_retbody(lua_State *L)
     http_ctx = ts_lua_get_http_ctx(L);
 
     body = luaL_checklstring(L, 1, &body_len);
-    TSHttpTxnSetHttpRetBody(http_ctx->txnp, body, 1);
+    TSHttpTxnErrorBodySet(http_ctx->txnp, TSstrdup(body), body_len, NULL); // Defaults to text/html
     return 0;
 }
 
@@ -152,7 +152,7 @@ ts_lua_http_set_resp(lua_State *L)
 
     if (n == 2) {
         body = luaL_checklstring(L, 2, &body_len);
-        TSHttpTxnSetHttpRetBody(http_ctx->txnp, body, 1);
+        TSHttpTxnErrorBodySet(http_ctx->txnp, TSstrdup(body), body_len, NULL); // Defaults to text/html
     }
 
     return 0;

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/7bac4a99/proxy/InkAPI.cc
----------------------------------------------------------------------
diff --git a/proxy/InkAPI.cc b/proxy/InkAPI.cc
index c9bb9ec..241ab45 100644
--- a/proxy/InkAPI.cc
+++ b/proxy/InkAPI.cc
@@ -5423,46 +5423,46 @@ TSHttpTxnServerPacketTosSet(TSHttpTxn txnp, int tos)
   return TS_SUCCESS;
 }
 
+// Set the body, or, if you provide a NULL buffer, clear the body message
 void
-TSHttpTxnErrorBodySet(TSHttpTxn txnp, char *buf, int buflength, char *mimetype)
+TSHttpTxnErrorBodySet(TSHttpTxn txnp, char *buf, size_t buflength, char *mimetype)
 {
   sdk_assert(sdk_sanity_check_txn(txnp) == TS_SUCCESS);
-  sdk_assert(sdk_sanity_check_null_ptr((void*)buf) == TS_SUCCESS);
-  sdk_assert(buflength > 0);
 
   HttpSM *sm = (HttpSM *) txnp;
   HttpTransact::State *s = &(sm->t_state);
 
-  if (s->internal_msg_buffer)
-    HttpTransact::free_internal_msg_buffer(s->internal_msg_buffer, s->internal_msg_buffer_fast_allocator_size);
-
-  ats_free_null(s->internal_msg_buffer_type);
+  // Cleanup anything already set.
+  s->free_internal_msg_buffer();
+  ats_free(s->internal_msg_buffer_type);
 
   s->internal_msg_buffer = buf;
-  s->internal_msg_buffer_type = mimetype;
-  s->internal_msg_buffer_size = buflength;
+  s->internal_msg_buffer_size = buf ? buflength : 0;
   s->internal_msg_buffer_fast_allocator_size = -1;
+
+  s->internal_msg_buffer_type = mimetype;
 }
 
 void
 TSHttpTxnServerRequestBodySet(TSHttpTxn txnp, char *buf, int64_t buflength)
 {
   sdk_assert(sdk_sanity_check_txn(txnp) == TS_SUCCESS);
-  sdk_assert(sdk_sanity_check_null_ptr((void*)buf) == TS_SUCCESS);
-  sdk_assert(buflength > 0);
 
   HttpSM *sm = (HttpSM *) txnp;
   HttpTransact::State *s = &(sm->t_state);
 
-  if (s->method != HTTP_WKSIDX_GET)
-    return;
-
-  if (s->internal_msg_buffer)
-    HttpTransact::free_internal_msg_buffer(s->internal_msg_buffer, s->internal_msg_buffer_fast_allocator_size);
+  // Cleanup anything already set.
+  s->free_internal_msg_buffer();
 
-  s->api_server_request_body_set = true;
-  s->internal_msg_buffer = buf;
-  s->internal_msg_buffer_size = buflength;
+  if (buf) {
+    s->api_server_request_body_set = true;
+    s->internal_msg_buffer = buf;
+    s->internal_msg_buffer_size = buflength;
+  } else {
+    s->api_server_request_body_set = false;
+    s->internal_msg_buffer = NULL;
+    s->internal_msg_buffer_size = 0;
+  }
   s->internal_msg_buffer_fast_allocator_size = -1;
 }
 
@@ -5663,30 +5663,6 @@ TSHttpTxnSetHttpRetStatus(TSHttpTxn txnp, TSHttpStatus http_retstatus)
   sm->t_state.http_return_code = (HTTPStatus) http_retstatus;
 }
 
-int
-TSHttpTxnGetMaxHttpRetBodySize(void)
-{
-  return HTTP_TRANSACT_STATE_MAX_XBUF_SIZE;
-}
-
-void
-TSHttpTxnSetHttpRetBody(TSHttpTxn txnp, const char *body_msg, int plain_msg_flag)
-{
-  sdk_assert(sdk_sanity_check_txn(txnp) == TS_SUCCESS);
-
-  HttpSM *sm = (HttpSM *) txnp;
-  HttpTransact::State *s = &(sm->t_state);
-
-  s->return_xbuf_size = 0;
-  s->return_xbuf[0] = 0;
-  s->return_xbuf_plain = false;
-  if (body_msg) {
-    ink_strlcpy(s->return_xbuf, body_msg, HTTP_TRANSACT_STATE_MAX_XBUF_SIZE);
-    s->return_xbuf_size = strlen(s->return_xbuf);
-    s->return_xbuf_plain = plain_msg_flag;
-  }
-}
-
 /* control channel for HTTP */
 TSReturnCode
 TSHttpTxnCntl(TSHttpTxn txnp, TSHttpCntlType cntl, void *data)

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/7bac4a99/proxy/api/ts/ts.h
----------------------------------------------------------------------
diff --git a/proxy/api/ts/ts.h b/proxy/api/ts/ts.h
index 02f4bf0..367317b 100644
--- a/proxy/api/ts/ts.h
+++ b/proxy/api/ts/ts.h
@@ -1394,7 +1394,19 @@ extern "C"
   tsapi TSReturnCode TSHttpTxnServerPacketTosSet(TSHttpTxn txnp, int tos);
 
 
-  tsapi void TSHttpTxnErrorBodySet(TSHttpTxn txnp, char* buf, int buflength, char* mimetype);
+  /**
+     Sets an error type body to a transaction. Note that both string arguments
+     must be allocated with TSmalloc() or TSstrdup(). The mimetype argument is
+     optional, if not provided it defaults to "text/html". Sending an emptry
+     string would prevent setting a content type header (but that is not adviced).
+
+     @param txnp HTTP transaction whose parent proxy to get.
+     @param buf The body message (must be heap allocated).
+     @param buflength Length of the body message.
+     @param mimetype The MIME type to set the response to (can be NULL, but must
+            be heap allocated if non-NULL).
+  */
+  tsapi void TSHttpTxnErrorBodySet(TSHttpTxn txnp, char* buf, size_t buflength, char* mimetype);
 
   /**
       Retrieves the parent proxy hostname and port, if parent
@@ -1458,8 +1470,7 @@ extern "C"
   tsapi TSReturnCode TSHttpArgIndexNameLookup(const char* name, int* arg_idx, const char** description);
   tsapi TSReturnCode TSHttpArgIndexLookup(int arg_idx, const char** name, const char** description);
 
-  tsapi int TSHttpTxnGetMaxHttpRetBodySize(void);
-  tsapi void TSHttpTxnSetHttpRetBody(TSHttpTxn txnp, const char* body_msg, int plain_msg);
+  /* ToDo: This is a leftover from olden days, can we eliminate? */
   tsapi void TSHttpTxnSetHttpRetStatus(TSHttpTxn txnp, TSHttpStatus http_retstatus);
 
   tsapi void TSHttpTxnActiveTimeoutSet(TSHttpTxn txnp, int timeout);

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/7bac4a99/proxy/http/HttpBodyFactory.cc
----------------------------------------------------------------------
diff --git a/proxy/http/HttpBodyFactory.cc b/proxy/http/HttpBodyFactory.cc
index 632d499..0650b48 100644
--- a/proxy/http/HttpBodyFactory.cc
+++ b/proxy/http/HttpBodyFactory.cc
@@ -122,7 +122,7 @@ HttpBodyFactory::fabricate_with_old_api(const char *type, HttpTransact::State *
   ///////////////////////////////////////////
   // check if we don't need to format body //
   ///////////////////////////////////////////
-  if (context->return_xbuf_plain && format) {
+  if (format) {
     int l = ink_bvsprintf(NULL, format, ap);
     if (l < max_buffer_length) {
       buffer = (char *)ats_malloc(l + 1);

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/7bac4a99/proxy/http/HttpSM.cc
----------------------------------------------------------------------
diff --git a/proxy/http/HttpSM.cc b/proxy/http/HttpSM.cc
index 0f25682..e60f5a4 100644
--- a/proxy/http/HttpSM.cc
+++ b/proxy/http/HttpSM.cc
@@ -2186,11 +2186,12 @@ HttpSM::state_handle_stat_page(int event, void *data)
 
     if (data) {
       StatPageData *spd = (StatPageData *) data;
+
       t_state.internal_msg_buffer = spd->data;
       if (spd->type)
         t_state.internal_msg_buffer_type = spd->type;
       else
-        t_state.internal_msg_buffer_type = ats_strdup("text/html");
+        t_state.internal_msg_buffer_type = NULL; // Defaults to text/html
       t_state.internal_msg_buffer_size = spd->length;
       t_state.internal_msg_buffer_fast_allocator_size = -1;
     }
@@ -5511,7 +5512,6 @@ HttpSM::setup_server_send_request_api()
 void
 HttpSM::setup_server_send_request()
 {
-  bool api_set;
   int hdr_length;
   int64_t msg_len = 0;  /* lv: just make gcc happy */
 
@@ -5523,8 +5523,7 @@ HttpSM::setup_server_send_request()
   server_entry->vc_handler = &HttpSM::state_send_server_request_header;
   server_entry->write_buffer = new_MIOBuffer(buffer_size_to_index(HTTP_HEADER_BUFFER_SIZE));
 
-  api_set = t_state.api_server_request_body_set ? true : false;
-  if (api_set) {
+  if (t_state.api_server_request_body_set) {
     msg_len = t_state.internal_msg_buffer_size;
     t_state.hdr_info.server_request.value_set_int64(MIME_FIELD_CONTENT_LENGTH, MIME_LEN_CONTENT_LENGTH, msg_len);
   }
@@ -5535,7 +5534,7 @@ HttpSM::setup_server_send_request()
     write_header_into_buffer(&t_state.hdr_info.server_request, server_entry->write_buffer);
 
   // the plugin decided to append a message to the request
-  if (api_set) {
+  if (t_state.api_server_request_body_set) {
     DebugSM("http", "[%" PRId64 "] appending msg of %" PRId64" bytes to request %s", sm_id, msg_len, t_state.internal_msg_buffer);
     hdr_length += server_entry->write_buffer->write(t_state.internal_msg_buffer, msg_len);
     server_request_body_bytes = msg_len;
@@ -5790,6 +5789,7 @@ void
 HttpSM::setup_internal_transfer(HttpSMHandler handler_arg)
 {
   bool is_msg_buf_present;
+
   if (t_state.internal_msg_buffer) {
     is_msg_buf_present = true;
     ink_assert(t_state.internal_msg_buffer_size > 0);
@@ -5800,12 +5800,19 @@ HttpSM::setup_internal_transfer(HttpSMHandler handler_arg)
 
     // set internal_msg_buffer_type if available
     if (t_state.internal_msg_buffer_type) {
-      t_state.hdr_info.client_response.value_set(MIME_FIELD_CONTENT_TYPE,
-                                                 MIME_LEN_CONTENT_TYPE,
-                                                 t_state.internal_msg_buffer_type,
-                                                 strlen(t_state.internal_msg_buffer_type));
+      int len = strlen(t_state.internal_msg_buffer_type);
+
+      if (len > 0) {
+        t_state.hdr_info.client_response.value_set(MIME_FIELD_CONTENT_TYPE,
+                                                   MIME_LEN_CONTENT_TYPE,
+                                                   t_state.internal_msg_buffer_type, len);
+      }
       ats_free(t_state.internal_msg_buffer_type);
       t_state.internal_msg_buffer_type = NULL;
+    } else {
+      t_state.hdr_info.client_response.value_set(MIME_FIELD_CONTENT_TYPE,
+                                                 MIME_LEN_CONTENT_TYPE,
+                                                 "text/html", 9);
     }
   } else {
     is_msg_buf_present = false;

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/7bac4a99/proxy/http/HttpTransact.cc
----------------------------------------------------------------------
diff --git a/proxy/http/HttpTransact.cc b/proxy/http/HttpTransact.cc
index 1eca460..3f14b4e 100644
--- a/proxy/http/HttpTransact.cc
+++ b/proxy/http/HttpTransact.cc
@@ -870,10 +870,11 @@ HttpTransact::EndRemapRequest(State* s)
   // Check if remap plugin set HTTP return code and return body  //
   /////////////////////////////////////////////////////////////////
   if (s->http_return_code != HTTP_STATUS_NONE) {
-    build_error_response(s, s->http_return_code, NULL, NULL, s->return_xbuf_size ? s->return_xbuf : NULL);
+    build_error_response(s, s->http_return_code, NULL, NULL, s->internal_msg_buffer_size ? s->internal_msg_buffer : NULL);
     s->reverse_proxy = false;
     goto done;
   }
+
   ///////////////////////////////////////////////////////////////
   // if no mapping was found, handle the cases where:          //
   //                                                           //
@@ -3422,8 +3423,15 @@ HttpTransact::HandleStatPage(State* s)
   s->hdr_info.client_response.set_content_length(s->internal_msg_buffer_size);
 
   if (s->internal_msg_buffer_type) {
-    s->hdr_info.client_response.value_set(MIME_FIELD_CONTENT_TYPE, MIME_LEN_CONTENT_TYPE, s->internal_msg_buffer_type,
-                                          strlen(s->internal_msg_buffer_type));
+    int len = strlen(s->internal_msg_buffer_type);
+
+    if (len > 0) {
+      s->hdr_info.client_response.value_set(MIME_FIELD_CONTENT_TYPE, MIME_LEN_CONTENT_TYPE,
+                                            s->internal_msg_buffer_type, len);
+    }
+  } else {
+    s->hdr_info.client_response.value_set(MIME_FIELD_CONTENT_TYPE, MIME_LEN_CONTENT_TYPE,
+                                          "text/plain", 9);
   }
 
   s->cache_info.action = CACHE_DO_NO_ACTION;
@@ -5512,9 +5520,7 @@ HttpTransact::handle_trace_and_options_requests(State* s, HTTPHdr* incoming_hdr)
 
       s->internal_msg_buffer_index = 0;
       s->internal_msg_buffer_size = req_length * 2;
-      if (s->internal_msg_buffer) {
-        free_internal_msg_buffer(s->internal_msg_buffer, s->internal_msg_buffer_fast_allocator_size);
-      }
+      s->free_internal_msg_buffer();
 
       if (s->internal_msg_buffer_size <= max_iobuffer_size) {
         s->internal_msg_buffer_fast_allocator_size = buffer_size_to_index(s->internal_msg_buffer_size);
@@ -8175,16 +8181,6 @@ HttpTransact::build_error_response(State *s, HTTPStatus status_code, const char
   s->hdr_info.client_response.field_delete(MIME_FIELD_LAST_MODIFIED, MIME_LEN_LAST_MODIFIED);
 
 
-  /////////////////////////////////////////////////////////////
-  // deallocate any existing response body --- it's possible //
-  // that we have previous created an error msg but retried  //
-  // the connection and are now making a new one.            //
-  /////////////////////////////////////////////////////////////
-  s->internal_msg_buffer_index = 0;
-  if (s->internal_msg_buffer) {
-    free_internal_msg_buffer(s->internal_msg_buffer, s->internal_msg_buffer_fast_allocator_size);
-  }
-  s->internal_msg_buffer_fast_allocator_size = -1;
 
   ////////////////////////////////////////////////////////////////////
   // create the error message using the "body factory", which will  //
@@ -8193,11 +8189,20 @@ HttpTransact::build_error_response(State *s, HTTPStatus status_code, const char
   // supports language targeting using the Accept-Language header   //
   ////////////////////////////////////////////////////////////////////
 
-  s->internal_msg_buffer = body_factory->fabricate_with_old_api(error_body_type, s, 8192,
-                                                                &s->internal_msg_buffer_size,
-                                                                body_language, sizeof(body_language), 
-                                                                body_type, sizeof(body_type), 
-                                                                format, ap);
+  int64_t len;
+  char *new_msg = body_factory->fabricate_with_old_api(error_body_type, s, 8192,
+                                                       &len,
+                                                       body_language, sizeof(body_language),
+                                                       body_type, sizeof(body_type),
+                                                       format, ap);
+
+  // After the body factory is called, a new "body" is allocated, and we must replace it. It is
+  // unfortunate that there's no way to avoid this fabrication even when there is no substitutions...
+  s->free_internal_msg_buffer();
+  s->internal_msg_buffer = new_msg;
+  s->internal_msg_buffer_size = len;
+  s->internal_msg_buffer_index = 0;
+  s->internal_msg_buffer_fast_allocator_size = -1;
 
   s->hdr_info.client_response.value_set(MIME_FIELD_CONTENT_TYPE, MIME_LEN_CONTENT_TYPE, body_type, strlen(body_type));
   s->hdr_info.client_response.value_set(MIME_FIELD_CONTENT_LANGUAGE, MIME_LEN_CONTENT_LANGUAGE, body_language,
@@ -8280,9 +8285,7 @@ HttpTransact::build_redirect_response(State* s)
   // set descriptive text //
   //////////////////////////
   s->internal_msg_buffer_index = 0;
-  if (s->internal_msg_buffer) {
-    free_internal_msg_buffer(s->internal_msg_buffer, s->internal_msg_buffer_fast_allocator_size);
-  }
+  s->free_internal_msg_buffer();
   s->internal_msg_buffer_fast_allocator_size = -1;
   s->internal_msg_buffer = body_factory->fabricate_with_old_api_build_va("redirect#moved_temporarily", s, 8192,
                                                                          &s->internal_msg_buffer_size,

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/7bac4a99/proxy/http/HttpTransact.h
----------------------------------------------------------------------
diff --git a/proxy/http/HttpTransact.h b/proxy/http/HttpTransact.h
index b712b63..652c5ac 100644
--- a/proxy/http/HttpTransact.h
+++ b/proxy/http/HttpTransact.h
@@ -953,9 +953,6 @@ public:
     remap_plugin_info::_tsremap_os_response *fp_tsremap_os_response;
     void* remap_plugin_instance;
     HTTPStatus http_return_code;
-    int return_xbuf_size;
-    bool return_xbuf_plain;
-    char return_xbuf[HTTP_TRANSACT_STATE_MAX_XBUF_SIZE];
     void *user_args[HTTP_SSN_TXN_MAX_USER_ARG];
 
     int api_txn_active_timeout_value;
@@ -1050,8 +1047,8 @@ public:
         upgrade_token_wks(NULL),
         is_websocket(false),
         did_upgrade_succeed(false),
-        internal_msg_buffer(0),
-        internal_msg_buffer_type(0),
+        internal_msg_buffer(NULL),
+        internal_msg_buffer_type(NULL),
         internal_msg_buffer_size(0),
         internal_msg_buffer_fast_allocator_size(-1),
         internal_msg_buffer_index(0),
@@ -1076,8 +1073,6 @@ public:
         fp_tsremap_os_response(NULL),
         remap_plugin_instance(0),
         http_return_code(HTTP_STATUS_NONE),
-        return_xbuf_size(0),
-        return_xbuf_plain(false),
         api_txn_active_timeout_value(-1),
         api_txn_connect_timeout_value(-1),
         api_txn_dns_timeout_value(-1),
@@ -1137,7 +1132,6 @@ public:
       via_string[VIA_DETAIL_SERVER_DESCRIPTOR] = VIA_DETAIL_SERVER_DESCRIPTOR_STRING;
       via_string[MAX_VIA_INDICES] = '\0';
 
-      memset(return_xbuf, 0, sizeof(return_xbuf));
       memset(user_args, 0, sizeof(user_args));
       memset(&host_db_info, 0, sizeof(host_db_info));
     }
@@ -1168,11 +1162,8 @@ public:
       record_transaction_stats();
       m_magic = HTTP_TRANSACT_MAGIC_DEAD;
 
-      if (internal_msg_buffer) {
-        free_internal_msg_buffer(internal_msg_buffer, internal_msg_buffer_fast_allocator_size);
-      }
-      if (internal_msg_buffer_type)
-        ats_free(internal_msg_buffer_type);
+      free_internal_msg_buffer();
+      ats_free(internal_msg_buffer_type);
 
       ParentConfig::release(parent_params);
       parent_params = NULL;
@@ -1219,10 +1210,21 @@ public:
       }
     }
 
-  }; // End of State struct.
-
+    void
+    free_internal_msg_buffer()
+    {
+      if (internal_msg_buffer) {
+        if (internal_msg_buffer_fast_allocator_size >= 0) {
+          THREAD_FREE(internal_msg_buffer, ioBufAllocator[internal_msg_buffer_fast_allocator_size], this_thread());
+        } else {
+          ats_free(internal_msg_buffer);
+        }
+        internal_msg_buffer = NULL;
+      }
+      internal_msg_buffer_size = 0;
+    }
 
-  static void free_internal_msg_buffer(char *buffer, int64_t size);
+  }; // End of State struct.
 
   static void HandleBlindTunnel(State* s);
   static void StartRemapRequest(State* s);
@@ -1379,17 +1381,6 @@ public:
 
 typedef void (*TransactEntryFunc_t) (HttpTransact::State* s);
 
-inline void
-HttpTransact::free_internal_msg_buffer(char *buffer, int64_t size)
-{
-  ink_assert(buffer);
-  if (size >= 0) {
-    THREAD_FREE(buffer, ioBufAllocator[size], this_thread());
-  } else {
-    ats_free(buffer);
-  }
-}
-
 inline bool
 is_response_body_precluded(HTTPStatus status_code, int method)
 {

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/7bac4a99/tools/apichecker.pl
----------------------------------------------------------------------
diff --git a/tools/apichecker.pl b/tools/apichecker.pl
index 706a9b8..26563c7 100755
--- a/tools/apichecker.pl
+++ b/tools/apichecker.pl
@@ -145,6 +145,7 @@ my $W_CHAR_NOT_NULL = "returns the char* pointer directly, it can never be NULL"
 my $W_TIME_T = "returns the time_t directly";
 my $W_NOT_NULL_LEN = "the length parameter can no longer be a NULL pointer";
 my $W_TSCACHEKEY = "returns a TSCacheKey directly";
+my $W_ARG_CHANGES = "the argument list or types have changes";
 my $W_TSAIOCALLBACK = "uses the new TSAIOCallback data type";
 my $W_DEPRECATED = "is deprecated, do not use (ever!)";
 my $W_NO_NULL_LENGTH = "1";
@@ -266,7 +267,7 @@ my %TWO_2_THREE = (
   "TSHttpTxnArgGet" => [$W_VOID_PTR_RETURN],
   "TSHttpSsnArgSet" => [$W_VOID_RETURN],
   "TSHttpSsnArgGet" => [$W_VOID_PTR_RETURN],
-  "TSHttpTxnSetHttpRetBody" => [$W_VOID_RETURN],
+  "TSHttpTxnSetHttpRetBody" => [$W_DEPRECATED],
   "TSHttpTxnSetHttpRetStatus" => [$W_VOID_RETURN],
   "TSHttpTxnActiveTimeoutSet" => [$W_VOID_RETURN],
   "TSHttpTxnConnectTimeoutSet" => [$W_VOID_RETURN],