You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@trafficserver.apache.org by bc...@apache.org on 2014/04/18 01:14:06 UTC

git commit: TS-1125: POST's with Expect: 100-continue are slowed by delayed 100 response Additional Authors: Feifei Cai Sudheer Vinukonda

Repository: trafficserver
Updated Branches:
  refs/heads/master 700875f18 -> 9da123014


TS-1125: POST's with Expect: 100-continue are slowed by delayed 100 response
Additional Authors:
Feifei Cai <ff...@yahoo-inc.com>
Sudheer Vinukonda <su...@yahoo-inc.com>


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

Branch: refs/heads/master
Commit: 9da123014382c00bccb1869782a4f2502229a459
Parents: 700875f
Author: Bryan Call <bc...@apache.org>
Authored: Thu Apr 17 16:10:40 2014 -0700
Committer: Bryan Call <bc...@apache.org>
Committed: Thu Apr 17 16:10:40 2014 -0700

----------------------------------------------------------------------
 iocore/net/P_SSLNetVConnection.h  |  1 +
 iocore/net/SSLNetVConnection.cc   |  2 +-
 mgmt/RecordsConfig.cc             |  2 ++
 proxy/hdrs/HTTP.cc                |  4 ++++
 proxy/hdrs/HTTP.h                 |  2 ++
 proxy/http/HttpConfig.cc          |  3 +++
 proxy/http/HttpConfig.h           |  3 +++
 proxy/http/HttpSM.cc              | 24 ++++++++++++++++++++++++
 proxy/http/HttpTransact.cc        |  5 +++++
 proxy/http/HttpTransactHeaders.cc | 12 ++++++++++++
 proxy/http/HttpTransactHeaders.h  |  1 +
 11 files changed, 58 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/trafficserver/blob/9da12301/iocore/net/P_SSLNetVConnection.h
----------------------------------------------------------------------
diff --git a/iocore/net/P_SSLNetVConnection.h b/iocore/net/P_SSLNetVConnection.h
index 47861e4..577995b 100644
--- a/iocore/net/P_SSLNetVConnection.h
+++ b/iocore/net/P_SSLNetVConnection.h
@@ -133,5 +133,6 @@ private:
 typedef int (SSLNetVConnection::*SSLNetVConnHandler) (int, void *);
 
 extern ClassAllocator<SSLNetVConnection> sslNetVCAllocator;
+extern int do_SSL_write(SSL *ssl, void *buf, int size);
 
 #endif /* _SSLNetVConnection_h_ */

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/9da12301/iocore/net/SSLNetVConnection.cc
----------------------------------------------------------------------
diff --git a/iocore/net/SSLNetVConnection.cc b/iocore/net/SSLNetVConnection.cc
index 02f3f4a..c62b1c1 100644
--- a/iocore/net/SSLNetVConnection.cc
+++ b/iocore/net/SSLNetVConnection.cc
@@ -80,7 +80,7 @@ debug_certificate_name(const char * msg, X509_NAME * name)
   BIO_free(bio);
 }
 
-static inline int
+int
 do_SSL_write(SSL * ssl, void *buf, int size)
 {
   int r = 0;

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/9da12301/mgmt/RecordsConfig.cc
----------------------------------------------------------------------
diff --git a/mgmt/RecordsConfig.cc b/mgmt/RecordsConfig.cc
index 9e45111..1516d1a 100644
--- a/mgmt/RecordsConfig.cc
+++ b/mgmt/RecordsConfig.cc
@@ -426,6 +426,8 @@ RecordElement RecordsConfig[] = {
   //       #
   {RECT_CONFIG, "proxy.config.http.send_http11_requests", RECD_INT, "1", RECU_DYNAMIC, RR_NULL, RECC_NULL, NULL, RECA_NULL}
   ,
+  {RECT_CONFIG, "proxy.config.http.send_100_continue_response", RECD_INT, "0", RECU_DYNAMIC, RR_NULL, RECC_NULL, NULL, RECA_NULL}
+  ,
   {RECT_CONFIG, "proxy.config.http.share_server_sessions", RECD_INT, "2", RECU_RESTART_TS, RR_NULL, RECC_NULL, NULL, RECA_NULL}
   ,
   {RECT_CONFIG, "proxy.config.http.server_session_sharing.match", RECD_STRING, "both", RECU_RESTART_TS, RR_NULL, RECC_NULL, NULL, RECA_NULL}

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/9da12301/proxy/hdrs/HTTP.cc
----------------------------------------------------------------------
diff --git a/proxy/hdrs/HTTP.cc b/proxy/hdrs/HTTP.cc
index 4ce25c8..20f702a 100644
--- a/proxy/hdrs/HTTP.cc
+++ b/proxy/hdrs/HTTP.cc
@@ -109,6 +109,7 @@ const char *HTTP_VALUE_PROXY_REVALIDATE;
 const char *HTTP_VALUE_PUBLIC;
 const char *HTTP_VALUE_S_MAXAGE;
 const char *HTTP_VALUE_NEED_REVALIDATE_ONCE;
+const char *HTTP_VALUE_100_CONTINUE;
 // Cache-control: extension "need-revalidate-once" is used internally by T.S.
 // to invalidate a document, and it is not returned/forwarded.
 // If a cached document has this extension set (ie, is invalidated),
@@ -143,6 +144,7 @@ int HTTP_LEN_PROXY_REVALIDATE;
 int HTTP_LEN_PUBLIC;
 int HTTP_LEN_S_MAXAGE;
 int HTTP_LEN_NEED_REVALIDATE_ONCE;
+int HTTP_LEN_100_CONTINUE;
 
 Arena* const HTTPHdr::USE_HDR_HEAP_MAGIC = reinterpret_cast<Arena*>(1);
 
@@ -257,6 +259,7 @@ http_init()
     HTTP_VALUE_PUBLIC = hdrtoken_string_to_wks("public");
     HTTP_VALUE_S_MAXAGE = hdrtoken_string_to_wks("s-maxage");
     HTTP_VALUE_NEED_REVALIDATE_ONCE = hdrtoken_string_to_wks("need-revalidate-once");
+    HTTP_VALUE_100_CONTINUE = hdrtoken_string_to_wks("100-continue");
 
     HTTP_LEN_BYTES = hdrtoken_wks_to_length(HTTP_VALUE_BYTES);
     HTTP_LEN_CHUNKED = hdrtoken_wks_to_length(HTTP_VALUE_CHUNKED);
@@ -280,6 +283,7 @@ http_init()
     HTTP_LEN_PUBLIC = hdrtoken_wks_to_length(HTTP_VALUE_PUBLIC);
     HTTP_LEN_S_MAXAGE = hdrtoken_wks_to_length(HTTP_VALUE_S_MAXAGE);
     HTTP_LEN_NEED_REVALIDATE_ONCE = hdrtoken_wks_to_length(HTTP_VALUE_NEED_REVALIDATE_ONCE);
+    HTTP_LEN_100_CONTINUE = hdrtoken_wks_to_length(HTTP_VALUE_100_CONTINUE);
 
     // TODO: We need to look into enable these CC values as WKS XXX
 #if 0

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/9da12301/proxy/hdrs/HTTP.h
----------------------------------------------------------------------
diff --git a/proxy/hdrs/HTTP.h b/proxy/hdrs/HTTP.h
index e6eabac..afbc1fd 100644
--- a/proxy/hdrs/HTTP.h
+++ b/proxy/hdrs/HTTP.h
@@ -402,6 +402,7 @@ extern const char *HTTP_VALUE_PROXY_REVALIDATE;
 extern const char *HTTP_VALUE_PUBLIC;
 extern const char *HTTP_VALUE_S_MAXAGE;
 extern const char *HTTP_VALUE_NEED_REVALIDATE_ONCE;
+extern const char *HTTP_VALUE_100_CONTINUE;
 
 extern int HTTP_LEN_BYTES;
 extern int HTTP_LEN_CHUNKED;
@@ -425,6 +426,7 @@ extern int HTTP_LEN_PROXY_REVALIDATE;
 extern int HTTP_LEN_PUBLIC;
 extern int HTTP_LEN_S_MAXAGE;
 extern int HTTP_LEN_NEED_REVALIDATE_ONCE;
+extern int HTTP_LEN_100_CONTINUE;
 
 /* Private */
 void http_hdr_adjust(HTTPHdrImpl *hdrp, int32_t offset, int32_t length, int32_t delta);

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/9da12301/proxy/http/HttpConfig.cc
----------------------------------------------------------------------
diff --git a/proxy/http/HttpConfig.cc b/proxy/http/HttpConfig.cc
index cdd3ea6..ff487b6 100644
--- a/proxy/http/HttpConfig.cc
+++ b/proxy/http/HttpConfig.cc
@@ -1452,6 +1452,8 @@ HttpConfig::startup()
 
   HttpEstablishStaticConfigByte(c.oride.send_http11_requests, "proxy.config.http.send_http11_requests");
 
+  HttpEstablishStaticConfigByte(c.oride.send_100_continue_response, "proxy.config.http.send_100_continue_response");
+
   // HTTP Referer Filtering
   HttpEstablishStaticConfigByte(c.referer_filter_enabled, "proxy.config.http.referer_filter");
   HttpEstablishStaticConfigByte(c.referer_format_redirect, "proxy.config.http.referer_format_redirect");
@@ -1691,6 +1693,7 @@ params->push_method_enabled = INT_TO_BOOL(m_master.push_method_enabled);
   params->record_cop_page = INT_TO_BOOL(m_master.record_cop_page);
   params->record_tcp_mem_hit = INT_TO_BOOL(m_master.record_tcp_mem_hit);
   params->oride.send_http11_requests = m_master.oride.send_http11_requests;
+  params->oride.send_100_continue_response = INT_TO_BOOL(m_master.oride.send_100_continue_response);
   params->oride.doc_in_cache_skip_dns = INT_TO_BOOL(m_master.oride.doc_in_cache_skip_dns);
   params->oride.default_buffer_size_index = m_master.oride.default_buffer_size_index;
   params->oride.default_buffer_water_mark = m_master.oride.default_buffer_water_mark;

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/9da12301/proxy/http/HttpConfig.h
----------------------------------------------------------------------
diff --git a/proxy/http/HttpConfig.h b/proxy/http/HttpConfig.h
index 343148f..031272d 100644
--- a/proxy/http/HttpConfig.h
+++ b/proxy/http/HttpConfig.h
@@ -412,6 +412,7 @@ struct OverridableHttpConfigParams {
       anonymize_remove_cookie(0), anonymize_remove_client_ip(0), anonymize_insert_client_ip(1),
       proxy_response_server_enabled(1), proxy_response_hsts_max_age(-1), proxy_response_hsts_include_subdomains(0),
       insert_squid_x_forwarded_for(1), send_http11_requests(1),
+      send_100_continue_response(0),
       cache_http(1), cache_cluster_cache_local(0), cache_ignore_client_no_cache(1), cache_ignore_client_cc_max_age(0),
       cache_ims_on_client_no_cache(1), cache_ignore_server_no_cache(0), cache_responses_to_cookies(1),
       cache_ignore_auth(0), cache_urls_that_look_dynamic(1), cache_required_headers(2), cache_range_lookup(1),
@@ -491,6 +492,8 @@ struct OverridableHttpConfigParams {
   //////////////////////
   MgmtByte send_http11_requests;
 
+  MgmtByte send_100_continue_response;
+
   ///////////////////
   // cache control //
   ///////////////////

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/9da12301/proxy/http/HttpSM.cc
----------------------------------------------------------------------
diff --git a/proxy/http/HttpSM.cc b/proxy/http/HttpSM.cc
index 767d252..45dd36f 100644
--- a/proxy/http/HttpSM.cc
+++ b/proxy/http/HttpSM.cc
@@ -91,6 +91,9 @@ static int scat_count = 0;
 static const int sub_header_size = sizeof("Content-type: ") - 1 + 2 + sizeof("Content-range: bytes ") - 1 + 4;
 static const int boundary_size = 2 + sizeof("RANGE_SEPARATOR") - 1 + 2;
 
+const char *str_100_continue_response = "HTTP/1.1 100 Continue\r\n\r\n";
+const int len_100_continue_response = strlen(str_100_continue_response);
+
 /**
  * Takes two milestones and returns the difference.
  * @param start The start time
@@ -1886,6 +1889,27 @@ HttpSM::state_send_server_request_header(int event, void *data)
       if (post_transform_info.vc) {
         setup_transform_to_server_transfer();
       } else {
+        if (t_state.txn_conf->send_100_continue_response) {
+          int len = 0;
+          const char *expect = t_state.hdr_info.client_request.value_get(MIME_FIELD_EXPECT, MIME_LEN_EXPECT, &len);
+          // When receive an "Expect: 100-continue" request from client, ATS sends a "100 Continue" response to client
+          // imediately, before receive the real response from original server.
+          if ((len == HTTP_LEN_100_CONTINUE) && (strncasecmp(expect, HTTP_VALUE_100_CONTINUE, HTTP_LEN_100_CONTINUE) == 0)) {
+            DebugSM("http_seq", "send 100 Continue response to client");
+
+            UnixNetVConnection* unix_vc = (UnixNetVConnection*)ua_session->get_netvc();
+            SSLNetVConnection *ssl_vc = dynamic_cast<SSLNetVConnection *>(unix_vc);
+
+            if (ssl_vc == NULL) {
+              DebugSM("http_seq", "send 100 Continue response to http client using raw socket");
+              int fd = unix_vc->get_socket();
+              write(fd, str_100_continue_response, len_100_continue_response);
+            } else {
+              DebugSM("http_seq", "send 100 Continue response to https client via openssl");
+              do_SSL_write(ssl_vc->ssl, (void*)str_100_continue_response, len_100_continue_response);
+            }
+          }
+        }
         do_setup_post_tunnel(HTTP_SERVER_VC);
       }
     } else {

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/9da12301/proxy/http/HttpTransact.cc
----------------------------------------------------------------------
diff --git a/proxy/http/HttpTransact.cc b/proxy/http/HttpTransact.cc
index aa671fa..6b00e98 100644
--- a/proxy/http/HttpTransact.cc
+++ b/proxy/http/HttpTransact.cc
@@ -7858,6 +7858,11 @@ HttpTransact::build_request(State* s, HTTPHdr* base_request, HTTPHdr* outgoing_r
     }
   }
 
+  if (s->txn_conf->send_100_continue_response) {
+    HttpTransactHeaders::remove_100_continue_headers(s, outgoing_request);
+    DebugTxn("http_trans", "[build_request] request expect 100-continue headers removed");
+  }
+
   s->request_sent_time = ink_cluster_time();
   s->current.now = s->request_sent_time;
   // The assert is backwards in this case because request is being (re)sent.

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/9da12301/proxy/http/HttpTransactHeaders.cc
----------------------------------------------------------------------
diff --git a/proxy/http/HttpTransactHeaders.cc b/proxy/http/HttpTransactHeaders.cc
index 4574c21..9be81d0 100644
--- a/proxy/http/HttpTransactHeaders.cc
+++ b/proxy/http/HttpTransactHeaders.cc
@@ -1007,6 +1007,18 @@ HttpTransactHeaders::remove_conditional_headers(HTTPHdr *outgoing)
   // TODO: how about RANGE and IF_RANGE?
 }
 
+void
+HttpTransactHeaders::remove_100_continue_headers(HttpTransact::State *s, HTTPHdr *outgoing)
+{
+  int len = 0;
+  const char *expect = s->hdr_info.client_request.value_get(MIME_FIELD_EXPECT, MIME_LEN_EXPECT, &len);
+
+  if ((len == HTTP_LEN_100_CONTINUE) && (strncasecmp(expect, HTTP_VALUE_100_CONTINUE, HTTP_LEN_100_CONTINUE) == 0)) {
+    outgoing->field_delete(MIME_FIELD_EXPECT, MIME_LEN_EXPECT);
+  }
+}
+
+
 
 ////////////////////////////////////////////////////////////////////////
 // Deal with lame-o servers by removing the host name from the url.

http://git-wip-us.apache.org/repos/asf/trafficserver/blob/9da12301/proxy/http/HttpTransactHeaders.h
----------------------------------------------------------------------
diff --git a/proxy/http/HttpTransactHeaders.h b/proxy/http/HttpTransactHeaders.h
index 0fa3a03..cc6f475 100644
--- a/proxy/http/HttpTransactHeaders.h
+++ b/proxy/http/HttpTransactHeaders.h
@@ -79,6 +79,7 @@ public:
   static void insert_basic_realm_in_proxy_authenticate(const char *realm, HTTPHdr * header, bool bRevPrxy);
 
   static void remove_conditional_headers(HTTPHdr * outgoing);
+  static void remove_100_continue_headers(HttpTransact::State *s, HTTPHdr * outgoing);
   static void remove_host_name_from_url(HTTPHdr * outgoing_request);
   static void add_global_user_agent_header_to_request(HttpConfigParams *http_config_param, HTTPHdr * header);
   static void add_server_header_to_response(OverridableHttpConfigParams *http_txn_conf, HTTPHdr * header);


Re: git commit: TS-1125: POST's with Expect: 100-continue are slowed by delayed 100 response Additional Authors: Feifei Cai Sudheer Vinukonda

Posted by James Peach <jp...@apache.org>.
This crashes traffic_server and traffic_manager on startup; please revert.

In addition, the actual implementation seems very ugly ... were you planning to revisit that? Since we have a good place for docs, now, we should document proxy.config.http.send_100_continue_response.

On Apr 17, 2014, at 4:14 PM, bcall@apache.org wrote:

> Repository: trafficserver
> Updated Branches:
> refs/heads/master 700875f18 -> 9da123014
> 
> 
> TS-1125: POST's with Expect: 100-continue are slowed by delayed 100 response
> Additional Authors:
> Feifei Cai <ff...@yahoo-inc.com>
> Sudheer Vinukonda <su...@yahoo-inc.com>
> 
> 
> Project: http://git-wip-us.apache.org/repos/asf/trafficserver/repo
> Commit: http://git-wip-us.apache.org/repos/asf/trafficserver/commit/9da12301
> Tree: http://git-wip-us.apache.org/repos/asf/trafficserver/tree/9da12301
> Diff: http://git-wip-us.apache.org/repos/asf/trafficserver/diff/9da12301
> 
> Branch: refs/heads/master
> Commit: 9da123014382c00bccb1869782a4f2502229a459
> Parents: 700875f
> Author: Bryan Call <bc...@apache.org>
> Authored: Thu Apr 17 16:10:40 2014 -0700
> Committer: Bryan Call <bc...@apache.org>
> Committed: Thu Apr 17 16:10:40 2014 -0700
> 
> ----------------------------------------------------------------------
> iocore/net/P_SSLNetVConnection.h  |  1 +
> iocore/net/SSLNetVConnection.cc   |  2 +-
> mgmt/RecordsConfig.cc             |  2 ++
> proxy/hdrs/HTTP.cc                |  4 ++++
> proxy/hdrs/HTTP.h                 |  2 ++
> proxy/http/HttpConfig.cc          |  3 +++
> proxy/http/HttpConfig.h           |  3 +++
> proxy/http/HttpSM.cc              | 24 ++++++++++++++++++++++++
> proxy/http/HttpTransact.cc        |  5 +++++
> proxy/http/HttpTransactHeaders.cc | 12 ++++++++++++
> proxy/http/HttpTransactHeaders.h  |  1 +
> 11 files changed, 58 insertions(+), 1 deletion(-)
> ----------------------------------------------------------------------
> 
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/9da12301/iocore/net/P_SSLNetVConnection.h
> ----------------------------------------------------------------------
> diff --git a/iocore/net/P_SSLNetVConnection.h b/iocore/net/P_SSLNetVConnection.h
> index 47861e4..577995b 100644
> --- a/iocore/net/P_SSLNetVConnection.h
> +++ b/iocore/net/P_SSLNetVConnection.h
> @@ -133,5 +133,6 @@ private:
> typedef int (SSLNetVConnection::*SSLNetVConnHandler) (int, void *);
> 
> extern ClassAllocator<SSLNetVConnection> sslNetVCAllocator;
> +extern int do_SSL_write(SSL *ssl, void *buf, int size);
> 
> #endif /* _SSLNetVConnection_h_ */
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/9da12301/iocore/net/SSLNetVConnection.cc
> ----------------------------------------------------------------------
> diff --git a/iocore/net/SSLNetVConnection.cc b/iocore/net/SSLNetVConnection.cc
> index 02f3f4a..c62b1c1 100644
> --- a/iocore/net/SSLNetVConnection.cc
> +++ b/iocore/net/SSLNetVConnection.cc
> @@ -80,7 +80,7 @@ debug_certificate_name(const char * msg, X509_NAME * name)
>  BIO_free(bio);
> }
> 
> -static inline int
> +int
> do_SSL_write(SSL * ssl, void *buf, int size)
> {
>  int r = 0;
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/9da12301/mgmt/RecordsConfig.cc
> ----------------------------------------------------------------------
> diff --git a/mgmt/RecordsConfig.cc b/mgmt/RecordsConfig.cc
> index 9e45111..1516d1a 100644
> --- a/mgmt/RecordsConfig.cc
> +++ b/mgmt/RecordsConfig.cc
> @@ -426,6 +426,8 @@ RecordElement RecordsConfig[] = {
>  //       #
>  {RECT_CONFIG, "proxy.config.http.send_http11_requests", RECD_INT, "1", RECU_DYNAMIC, RR_NULL, RECC_NULL, NULL, RECA_NULL}
>  ,
> +  {RECT_CONFIG, "proxy.config.http.send_100_continue_response", RECD_INT, "0", RECU_DYNAMIC, RR_NULL, RECC_NULL, NULL, RECA_NULL}
> +  ,
>  {RECT_CONFIG, "proxy.config.http.share_server_sessions", RECD_INT, "2", RECU_RESTART_TS, RR_NULL, RECC_NULL, NULL, RECA_NULL}
>  ,
>  {RECT_CONFIG, "proxy.config.http.server_session_sharing.match", RECD_STRING, "both", RECU_RESTART_TS, RR_NULL, RECC_NULL, NULL, RECA_NULL}
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/9da12301/proxy/hdrs/HTTP.cc
> ----------------------------------------------------------------------
> diff --git a/proxy/hdrs/HTTP.cc b/proxy/hdrs/HTTP.cc
> index 4ce25c8..20f702a 100644
> --- a/proxy/hdrs/HTTP.cc
> +++ b/proxy/hdrs/HTTP.cc
> @@ -109,6 +109,7 @@ const char *HTTP_VALUE_PROXY_REVALIDATE;
> const char *HTTP_VALUE_PUBLIC;
> const char *HTTP_VALUE_S_MAXAGE;
> const char *HTTP_VALUE_NEED_REVALIDATE_ONCE;
> +const char *HTTP_VALUE_100_CONTINUE;
> // Cache-control: extension "need-revalidate-once" is used internally by T.S.
> // to invalidate a document, and it is not returned/forwarded.
> // If a cached document has this extension set (ie, is invalidated),
> @@ -143,6 +144,7 @@ int HTTP_LEN_PROXY_REVALIDATE;
> int HTTP_LEN_PUBLIC;
> int HTTP_LEN_S_MAXAGE;
> int HTTP_LEN_NEED_REVALIDATE_ONCE;
> +int HTTP_LEN_100_CONTINUE;
> 
> Arena* const HTTPHdr::USE_HDR_HEAP_MAGIC = reinterpret_cast<Arena*>(1);
> 
> @@ -257,6 +259,7 @@ http_init()
>    HTTP_VALUE_PUBLIC = hdrtoken_string_to_wks("public");
>    HTTP_VALUE_S_MAXAGE = hdrtoken_string_to_wks("s-maxage");
>    HTTP_VALUE_NEED_REVALIDATE_ONCE = hdrtoken_string_to_wks("need-revalidate-once");
> +    HTTP_VALUE_100_CONTINUE = hdrtoken_string_to_wks("100-continue");
> 
>    HTTP_LEN_BYTES = hdrtoken_wks_to_length(HTTP_VALUE_BYTES);
>    HTTP_LEN_CHUNKED = hdrtoken_wks_to_length(HTTP_VALUE_CHUNKED);
> @@ -280,6 +283,7 @@ http_init()
>    HTTP_LEN_PUBLIC = hdrtoken_wks_to_length(HTTP_VALUE_PUBLIC);
>    HTTP_LEN_S_MAXAGE = hdrtoken_wks_to_length(HTTP_VALUE_S_MAXAGE);
>    HTTP_LEN_NEED_REVALIDATE_ONCE = hdrtoken_wks_to_length(HTTP_VALUE_NEED_REVALIDATE_ONCE);
> +    HTTP_LEN_100_CONTINUE = hdrtoken_wks_to_length(HTTP_VALUE_100_CONTINUE);
> 
>    // TODO: We need to look into enable these CC values as WKS XXX
> #if 0
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/9da12301/proxy/hdrs/HTTP.h
> ----------------------------------------------------------------------
> diff --git a/proxy/hdrs/HTTP.h b/proxy/hdrs/HTTP.h
> index e6eabac..afbc1fd 100644
> --- a/proxy/hdrs/HTTP.h
> +++ b/proxy/hdrs/HTTP.h
> @@ -402,6 +402,7 @@ extern const char *HTTP_VALUE_PROXY_REVALIDATE;
> extern const char *HTTP_VALUE_PUBLIC;
> extern const char *HTTP_VALUE_S_MAXAGE;
> extern const char *HTTP_VALUE_NEED_REVALIDATE_ONCE;
> +extern const char *HTTP_VALUE_100_CONTINUE;
> 
> extern int HTTP_LEN_BYTES;
> extern int HTTP_LEN_CHUNKED;
> @@ -425,6 +426,7 @@ extern int HTTP_LEN_PROXY_REVALIDATE;
> extern int HTTP_LEN_PUBLIC;
> extern int HTTP_LEN_S_MAXAGE;
> extern int HTTP_LEN_NEED_REVALIDATE_ONCE;
> +extern int HTTP_LEN_100_CONTINUE;
> 
> /* Private */
> void http_hdr_adjust(HTTPHdrImpl *hdrp, int32_t offset, int32_t length, int32_t delta);
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/9da12301/proxy/http/HttpConfig.cc
> ----------------------------------------------------------------------
> diff --git a/proxy/http/HttpConfig.cc b/proxy/http/HttpConfig.cc
> index cdd3ea6..ff487b6 100644
> --- a/proxy/http/HttpConfig.cc
> +++ b/proxy/http/HttpConfig.cc
> @@ -1452,6 +1452,8 @@ HttpConfig::startup()
> 
>  HttpEstablishStaticConfigByte(c.oride.send_http11_requests, "proxy.config.http.send_http11_requests");
> 
> +  HttpEstablishStaticConfigByte(c.oride.send_100_continue_response, "proxy.config.http.send_100_continue_response");
> +
>  // HTTP Referer Filtering
>  HttpEstablishStaticConfigByte(c.referer_filter_enabled, "proxy.config.http.referer_filter");
>  HttpEstablishStaticConfigByte(c.referer_format_redirect, "proxy.config.http.referer_format_redirect");
> @@ -1691,6 +1693,7 @@ params->push_method_enabled = INT_TO_BOOL(m_master.push_method_enabled);
>  params->record_cop_page = INT_TO_BOOL(m_master.record_cop_page);
>  params->record_tcp_mem_hit = INT_TO_BOOL(m_master.record_tcp_mem_hit);
>  params->oride.send_http11_requests = m_master.oride.send_http11_requests;
> +  params->oride.send_100_continue_response = INT_TO_BOOL(m_master.oride.send_100_continue_response);
>  params->oride.doc_in_cache_skip_dns = INT_TO_BOOL(m_master.oride.doc_in_cache_skip_dns);
>  params->oride.default_buffer_size_index = m_master.oride.default_buffer_size_index;
>  params->oride.default_buffer_water_mark = m_master.oride.default_buffer_water_mark;
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/9da12301/proxy/http/HttpConfig.h
> ----------------------------------------------------------------------
> diff --git a/proxy/http/HttpConfig.h b/proxy/http/HttpConfig.h
> index 343148f..031272d 100644
> --- a/proxy/http/HttpConfig.h
> +++ b/proxy/http/HttpConfig.h
> @@ -412,6 +412,7 @@ struct OverridableHttpConfigParams {
>      anonymize_remove_cookie(0), anonymize_remove_client_ip(0), anonymize_insert_client_ip(1),
>      proxy_response_server_enabled(1), proxy_response_hsts_max_age(-1), proxy_response_hsts_include_subdomains(0),
>      insert_squid_x_forwarded_for(1), send_http11_requests(1),
> +      send_100_continue_response(0),
>      cache_http(1), cache_cluster_cache_local(0), cache_ignore_client_no_cache(1), cache_ignore_client_cc_max_age(0),
>      cache_ims_on_client_no_cache(1), cache_ignore_server_no_cache(0), cache_responses_to_cookies(1),
>      cache_ignore_auth(0), cache_urls_that_look_dynamic(1), cache_required_headers(2), cache_range_lookup(1),
> @@ -491,6 +492,8 @@ struct OverridableHttpConfigParams {
>  //////////////////////
>  MgmtByte send_http11_requests;
> 
> +  MgmtByte send_100_continue_response;
> +
>  ///////////////////
>  // cache control //
>  ///////////////////
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/9da12301/proxy/http/HttpSM.cc
> ----------------------------------------------------------------------
> diff --git a/proxy/http/HttpSM.cc b/proxy/http/HttpSM.cc
> index 767d252..45dd36f 100644
> --- a/proxy/http/HttpSM.cc
> +++ b/proxy/http/HttpSM.cc
> @@ -91,6 +91,9 @@ static int scat_count = 0;
> static const int sub_header_size = sizeof("Content-type: ") - 1 + 2 + sizeof("Content-range: bytes ") - 1 + 4;
> static const int boundary_size = 2 + sizeof("RANGE_SEPARATOR") - 1 + 2;
> 
> +const char *str_100_continue_response = "HTTP/1.1 100 Continue\r\n\r\n";
> +const int len_100_continue_response = strlen(str_100_continue_response);
> +
> /**
> * Takes two milestones and returns the difference.
> * @param start The start time
> @@ -1886,6 +1889,27 @@ HttpSM::state_send_server_request_header(int event, void *data)
>      if (post_transform_info.vc) {
>        setup_transform_to_server_transfer();
>      } else {
> +        if (t_state.txn_conf->send_100_continue_response) {
> +          int len = 0;
> +          const char *expect = t_state.hdr_info.client_request.value_get(MIME_FIELD_EXPECT, MIME_LEN_EXPECT, &len);
> +          // When receive an "Expect: 100-continue" request from client, ATS sends a "100 Continue" response to client
> +          // imediately, before receive the real response from original server.
> +          if ((len == HTTP_LEN_100_CONTINUE) && (strncasecmp(expect, HTTP_VALUE_100_CONTINUE, HTTP_LEN_100_CONTINUE) == 0)) {
> +            DebugSM("http_seq", "send 100 Continue response to client");
> +
> +            UnixNetVConnection* unix_vc = (UnixNetVConnection*)ua_session->get_netvc();
> +            SSLNetVConnection *ssl_vc = dynamic_cast<SSLNetVConnection *>(unix_vc);
> +
> +            if (ssl_vc == NULL) {
> +              DebugSM("http_seq", "send 100 Continue response to http client using raw socket");
> +              int fd = unix_vc->get_socket();
> +              write(fd, str_100_continue_response, len_100_continue_response);
> +            } else {
> +              DebugSM("http_seq", "send 100 Continue response to https client via openssl");
> +              do_SSL_write(ssl_vc->ssl, (void*)str_100_continue_response, len_100_continue_response);
> +            }
> +          }
> +        }
>        do_setup_post_tunnel(HTTP_SERVER_VC);
>      }
>    } else {
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/9da12301/proxy/http/HttpTransact.cc
> ----------------------------------------------------------------------
> diff --git a/proxy/http/HttpTransact.cc b/proxy/http/HttpTransact.cc
> index aa671fa..6b00e98 100644
> --- a/proxy/http/HttpTransact.cc
> +++ b/proxy/http/HttpTransact.cc
> @@ -7858,6 +7858,11 @@ HttpTransact::build_request(State* s, HTTPHdr* base_request, HTTPHdr* outgoing_r
>    }
>  }
> 
> +  if (s->txn_conf->send_100_continue_response) {
> +    HttpTransactHeaders::remove_100_continue_headers(s, outgoing_request);
> +    DebugTxn("http_trans", "[build_request] request expect 100-continue headers removed");
> +  }
> +
>  s->request_sent_time = ink_cluster_time();
>  s->current.now = s->request_sent_time;
>  // The assert is backwards in this case because request is being (re)sent.
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/9da12301/proxy/http/HttpTransactHeaders.cc
> ----------------------------------------------------------------------
> diff --git a/proxy/http/HttpTransactHeaders.cc b/proxy/http/HttpTransactHeaders.cc
> index 4574c21..9be81d0 100644
> --- a/proxy/http/HttpTransactHeaders.cc
> +++ b/proxy/http/HttpTransactHeaders.cc
> @@ -1007,6 +1007,18 @@ HttpTransactHeaders::remove_conditional_headers(HTTPHdr *outgoing)
>  // TODO: how about RANGE and IF_RANGE?
> }
> 
> +void
> +HttpTransactHeaders::remove_100_continue_headers(HttpTransact::State *s, HTTPHdr *outgoing)
> +{
> +  int len = 0;
> +  const char *expect = s->hdr_info.client_request.value_get(MIME_FIELD_EXPECT, MIME_LEN_EXPECT, &len);
> +
> +  if ((len == HTTP_LEN_100_CONTINUE) && (strncasecmp(expect, HTTP_VALUE_100_CONTINUE, HTTP_LEN_100_CONTINUE) == 0)) {
> +    outgoing->field_delete(MIME_FIELD_EXPECT, MIME_LEN_EXPECT);
> +  }
> +}
> +
> +
> 
> ////////////////////////////////////////////////////////////////////////
> // Deal with lame-o servers by removing the host name from the url.
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/9da12301/proxy/http/HttpTransactHeaders.h
> ----------------------------------------------------------------------
> diff --git a/proxy/http/HttpTransactHeaders.h b/proxy/http/HttpTransactHeaders.h
> index 0fa3a03..cc6f475 100644
> --- a/proxy/http/HttpTransactHeaders.h
> +++ b/proxy/http/HttpTransactHeaders.h
> @@ -79,6 +79,7 @@ public:
>  static void insert_basic_realm_in_proxy_authenticate(const char *realm, HTTPHdr * header, bool bRevPrxy);
> 
>  static void remove_conditional_headers(HTTPHdr * outgoing);
> +  static void remove_100_continue_headers(HttpTransact::State *s, HTTPHdr * outgoing);
>  static void remove_host_name_from_url(HTTPHdr * outgoing_request);
>  static void add_global_user_agent_header_to_request(HttpConfigParams *http_config_param, HTTPHdr * header);
>  static void add_server_header_to_response(OverridableHttpConfigParams *http_txn_conf, HTTPHdr * header);
> 


Re: git commit: TS-1125: POST's with Expect: 100-continue are slowed by delayed 100 response Additional Authors: Feifei Cai Sudheer Vinukonda

Posted by James Peach <jp...@apache.org>.
This crashes traffic_server and traffic_manager on startup; please revert.

In addition, the actual implementation seems very ugly ... were you planning to revisit that? Since we have a good place for docs, now, we should document proxy.config.http.send_100_continue_response.

On Apr 17, 2014, at 4:14 PM, bcall@apache.org wrote:

> Repository: trafficserver
> Updated Branches:
> refs/heads/master 700875f18 -> 9da123014
> 
> 
> TS-1125: POST's with Expect: 100-continue are slowed by delayed 100 response
> Additional Authors:
> Feifei Cai <ff...@yahoo-inc.com>
> Sudheer Vinukonda <su...@yahoo-inc.com>
> 
> 
> Project: http://git-wip-us.apache.org/repos/asf/trafficserver/repo
> Commit: http://git-wip-us.apache.org/repos/asf/trafficserver/commit/9da12301
> Tree: http://git-wip-us.apache.org/repos/asf/trafficserver/tree/9da12301
> Diff: http://git-wip-us.apache.org/repos/asf/trafficserver/diff/9da12301
> 
> Branch: refs/heads/master
> Commit: 9da123014382c00bccb1869782a4f2502229a459
> Parents: 700875f
> Author: Bryan Call <bc...@apache.org>
> Authored: Thu Apr 17 16:10:40 2014 -0700
> Committer: Bryan Call <bc...@apache.org>
> Committed: Thu Apr 17 16:10:40 2014 -0700
> 
> ----------------------------------------------------------------------
> iocore/net/P_SSLNetVConnection.h  |  1 +
> iocore/net/SSLNetVConnection.cc   |  2 +-
> mgmt/RecordsConfig.cc             |  2 ++
> proxy/hdrs/HTTP.cc                |  4 ++++
> proxy/hdrs/HTTP.h                 |  2 ++
> proxy/http/HttpConfig.cc          |  3 +++
> proxy/http/HttpConfig.h           |  3 +++
> proxy/http/HttpSM.cc              | 24 ++++++++++++++++++++++++
> proxy/http/HttpTransact.cc        |  5 +++++
> proxy/http/HttpTransactHeaders.cc | 12 ++++++++++++
> proxy/http/HttpTransactHeaders.h  |  1 +
> 11 files changed, 58 insertions(+), 1 deletion(-)
> ----------------------------------------------------------------------
> 
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/9da12301/iocore/net/P_SSLNetVConnection.h
> ----------------------------------------------------------------------
> diff --git a/iocore/net/P_SSLNetVConnection.h b/iocore/net/P_SSLNetVConnection.h
> index 47861e4..577995b 100644
> --- a/iocore/net/P_SSLNetVConnection.h
> +++ b/iocore/net/P_SSLNetVConnection.h
> @@ -133,5 +133,6 @@ private:
> typedef int (SSLNetVConnection::*SSLNetVConnHandler) (int, void *);
> 
> extern ClassAllocator<SSLNetVConnection> sslNetVCAllocator;
> +extern int do_SSL_write(SSL *ssl, void *buf, int size);
> 
> #endif /* _SSLNetVConnection_h_ */
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/9da12301/iocore/net/SSLNetVConnection.cc
> ----------------------------------------------------------------------
> diff --git a/iocore/net/SSLNetVConnection.cc b/iocore/net/SSLNetVConnection.cc
> index 02f3f4a..c62b1c1 100644
> --- a/iocore/net/SSLNetVConnection.cc
> +++ b/iocore/net/SSLNetVConnection.cc
> @@ -80,7 +80,7 @@ debug_certificate_name(const char * msg, X509_NAME * name)
>  BIO_free(bio);
> }
> 
> -static inline int
> +int
> do_SSL_write(SSL * ssl, void *buf, int size)
> {
>  int r = 0;
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/9da12301/mgmt/RecordsConfig.cc
> ----------------------------------------------------------------------
> diff --git a/mgmt/RecordsConfig.cc b/mgmt/RecordsConfig.cc
> index 9e45111..1516d1a 100644
> --- a/mgmt/RecordsConfig.cc
> +++ b/mgmt/RecordsConfig.cc
> @@ -426,6 +426,8 @@ RecordElement RecordsConfig[] = {
>  //       #
>  {RECT_CONFIG, "proxy.config.http.send_http11_requests", RECD_INT, "1", RECU_DYNAMIC, RR_NULL, RECC_NULL, NULL, RECA_NULL}
>  ,
> +  {RECT_CONFIG, "proxy.config.http.send_100_continue_response", RECD_INT, "0", RECU_DYNAMIC, RR_NULL, RECC_NULL, NULL, RECA_NULL}
> +  ,
>  {RECT_CONFIG, "proxy.config.http.share_server_sessions", RECD_INT, "2", RECU_RESTART_TS, RR_NULL, RECC_NULL, NULL, RECA_NULL}
>  ,
>  {RECT_CONFIG, "proxy.config.http.server_session_sharing.match", RECD_STRING, "both", RECU_RESTART_TS, RR_NULL, RECC_NULL, NULL, RECA_NULL}
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/9da12301/proxy/hdrs/HTTP.cc
> ----------------------------------------------------------------------
> diff --git a/proxy/hdrs/HTTP.cc b/proxy/hdrs/HTTP.cc
> index 4ce25c8..20f702a 100644
> --- a/proxy/hdrs/HTTP.cc
> +++ b/proxy/hdrs/HTTP.cc
> @@ -109,6 +109,7 @@ const char *HTTP_VALUE_PROXY_REVALIDATE;
> const char *HTTP_VALUE_PUBLIC;
> const char *HTTP_VALUE_S_MAXAGE;
> const char *HTTP_VALUE_NEED_REVALIDATE_ONCE;
> +const char *HTTP_VALUE_100_CONTINUE;
> // Cache-control: extension "need-revalidate-once" is used internally by T.S.
> // to invalidate a document, and it is not returned/forwarded.
> // If a cached document has this extension set (ie, is invalidated),
> @@ -143,6 +144,7 @@ int HTTP_LEN_PROXY_REVALIDATE;
> int HTTP_LEN_PUBLIC;
> int HTTP_LEN_S_MAXAGE;
> int HTTP_LEN_NEED_REVALIDATE_ONCE;
> +int HTTP_LEN_100_CONTINUE;
> 
> Arena* const HTTPHdr::USE_HDR_HEAP_MAGIC = reinterpret_cast<Arena*>(1);
> 
> @@ -257,6 +259,7 @@ http_init()
>    HTTP_VALUE_PUBLIC = hdrtoken_string_to_wks("public");
>    HTTP_VALUE_S_MAXAGE = hdrtoken_string_to_wks("s-maxage");
>    HTTP_VALUE_NEED_REVALIDATE_ONCE = hdrtoken_string_to_wks("need-revalidate-once");
> +    HTTP_VALUE_100_CONTINUE = hdrtoken_string_to_wks("100-continue");
> 
>    HTTP_LEN_BYTES = hdrtoken_wks_to_length(HTTP_VALUE_BYTES);
>    HTTP_LEN_CHUNKED = hdrtoken_wks_to_length(HTTP_VALUE_CHUNKED);
> @@ -280,6 +283,7 @@ http_init()
>    HTTP_LEN_PUBLIC = hdrtoken_wks_to_length(HTTP_VALUE_PUBLIC);
>    HTTP_LEN_S_MAXAGE = hdrtoken_wks_to_length(HTTP_VALUE_S_MAXAGE);
>    HTTP_LEN_NEED_REVALIDATE_ONCE = hdrtoken_wks_to_length(HTTP_VALUE_NEED_REVALIDATE_ONCE);
> +    HTTP_LEN_100_CONTINUE = hdrtoken_wks_to_length(HTTP_VALUE_100_CONTINUE);
> 
>    // TODO: We need to look into enable these CC values as WKS XXX
> #if 0
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/9da12301/proxy/hdrs/HTTP.h
> ----------------------------------------------------------------------
> diff --git a/proxy/hdrs/HTTP.h b/proxy/hdrs/HTTP.h
> index e6eabac..afbc1fd 100644
> --- a/proxy/hdrs/HTTP.h
> +++ b/proxy/hdrs/HTTP.h
> @@ -402,6 +402,7 @@ extern const char *HTTP_VALUE_PROXY_REVALIDATE;
> extern const char *HTTP_VALUE_PUBLIC;
> extern const char *HTTP_VALUE_S_MAXAGE;
> extern const char *HTTP_VALUE_NEED_REVALIDATE_ONCE;
> +extern const char *HTTP_VALUE_100_CONTINUE;
> 
> extern int HTTP_LEN_BYTES;
> extern int HTTP_LEN_CHUNKED;
> @@ -425,6 +426,7 @@ extern int HTTP_LEN_PROXY_REVALIDATE;
> extern int HTTP_LEN_PUBLIC;
> extern int HTTP_LEN_S_MAXAGE;
> extern int HTTP_LEN_NEED_REVALIDATE_ONCE;
> +extern int HTTP_LEN_100_CONTINUE;
> 
> /* Private */
> void http_hdr_adjust(HTTPHdrImpl *hdrp, int32_t offset, int32_t length, int32_t delta);
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/9da12301/proxy/http/HttpConfig.cc
> ----------------------------------------------------------------------
> diff --git a/proxy/http/HttpConfig.cc b/proxy/http/HttpConfig.cc
> index cdd3ea6..ff487b6 100644
> --- a/proxy/http/HttpConfig.cc
> +++ b/proxy/http/HttpConfig.cc
> @@ -1452,6 +1452,8 @@ HttpConfig::startup()
> 
>  HttpEstablishStaticConfigByte(c.oride.send_http11_requests, "proxy.config.http.send_http11_requests");
> 
> +  HttpEstablishStaticConfigByte(c.oride.send_100_continue_response, "proxy.config.http.send_100_continue_response");
> +
>  // HTTP Referer Filtering
>  HttpEstablishStaticConfigByte(c.referer_filter_enabled, "proxy.config.http.referer_filter");
>  HttpEstablishStaticConfigByte(c.referer_format_redirect, "proxy.config.http.referer_format_redirect");
> @@ -1691,6 +1693,7 @@ params->push_method_enabled = INT_TO_BOOL(m_master.push_method_enabled);
>  params->record_cop_page = INT_TO_BOOL(m_master.record_cop_page);
>  params->record_tcp_mem_hit = INT_TO_BOOL(m_master.record_tcp_mem_hit);
>  params->oride.send_http11_requests = m_master.oride.send_http11_requests;
> +  params->oride.send_100_continue_response = INT_TO_BOOL(m_master.oride.send_100_continue_response);
>  params->oride.doc_in_cache_skip_dns = INT_TO_BOOL(m_master.oride.doc_in_cache_skip_dns);
>  params->oride.default_buffer_size_index = m_master.oride.default_buffer_size_index;
>  params->oride.default_buffer_water_mark = m_master.oride.default_buffer_water_mark;
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/9da12301/proxy/http/HttpConfig.h
> ----------------------------------------------------------------------
> diff --git a/proxy/http/HttpConfig.h b/proxy/http/HttpConfig.h
> index 343148f..031272d 100644
> --- a/proxy/http/HttpConfig.h
> +++ b/proxy/http/HttpConfig.h
> @@ -412,6 +412,7 @@ struct OverridableHttpConfigParams {
>      anonymize_remove_cookie(0), anonymize_remove_client_ip(0), anonymize_insert_client_ip(1),
>      proxy_response_server_enabled(1), proxy_response_hsts_max_age(-1), proxy_response_hsts_include_subdomains(0),
>      insert_squid_x_forwarded_for(1), send_http11_requests(1),
> +      send_100_continue_response(0),
>      cache_http(1), cache_cluster_cache_local(0), cache_ignore_client_no_cache(1), cache_ignore_client_cc_max_age(0),
>      cache_ims_on_client_no_cache(1), cache_ignore_server_no_cache(0), cache_responses_to_cookies(1),
>      cache_ignore_auth(0), cache_urls_that_look_dynamic(1), cache_required_headers(2), cache_range_lookup(1),
> @@ -491,6 +492,8 @@ struct OverridableHttpConfigParams {
>  //////////////////////
>  MgmtByte send_http11_requests;
> 
> +  MgmtByte send_100_continue_response;
> +
>  ///////////////////
>  // cache control //
>  ///////////////////
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/9da12301/proxy/http/HttpSM.cc
> ----------------------------------------------------------------------
> diff --git a/proxy/http/HttpSM.cc b/proxy/http/HttpSM.cc
> index 767d252..45dd36f 100644
> --- a/proxy/http/HttpSM.cc
> +++ b/proxy/http/HttpSM.cc
> @@ -91,6 +91,9 @@ static int scat_count = 0;
> static const int sub_header_size = sizeof("Content-type: ") - 1 + 2 + sizeof("Content-range: bytes ") - 1 + 4;
> static const int boundary_size = 2 + sizeof("RANGE_SEPARATOR") - 1 + 2;
> 
> +const char *str_100_continue_response = "HTTP/1.1 100 Continue\r\n\r\n";
> +const int len_100_continue_response = strlen(str_100_continue_response);
> +
> /**
> * Takes two milestones and returns the difference.
> * @param start The start time
> @@ -1886,6 +1889,27 @@ HttpSM::state_send_server_request_header(int event, void *data)
>      if (post_transform_info.vc) {
>        setup_transform_to_server_transfer();
>      } else {
> +        if (t_state.txn_conf->send_100_continue_response) {
> +          int len = 0;
> +          const char *expect = t_state.hdr_info.client_request.value_get(MIME_FIELD_EXPECT, MIME_LEN_EXPECT, &len);
> +          // When receive an "Expect: 100-continue" request from client, ATS sends a "100 Continue" response to client
> +          // imediately, before receive the real response from original server.
> +          if ((len == HTTP_LEN_100_CONTINUE) && (strncasecmp(expect, HTTP_VALUE_100_CONTINUE, HTTP_LEN_100_CONTINUE) == 0)) {
> +            DebugSM("http_seq", "send 100 Continue response to client");
> +
> +            UnixNetVConnection* unix_vc = (UnixNetVConnection*)ua_session->get_netvc();
> +            SSLNetVConnection *ssl_vc = dynamic_cast<SSLNetVConnection *>(unix_vc);
> +
> +            if (ssl_vc == NULL) {
> +              DebugSM("http_seq", "send 100 Continue response to http client using raw socket");
> +              int fd = unix_vc->get_socket();
> +              write(fd, str_100_continue_response, len_100_continue_response);
> +            } else {
> +              DebugSM("http_seq", "send 100 Continue response to https client via openssl");
> +              do_SSL_write(ssl_vc->ssl, (void*)str_100_continue_response, len_100_continue_response);
> +            }
> +          }
> +        }
>        do_setup_post_tunnel(HTTP_SERVER_VC);
>      }
>    } else {
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/9da12301/proxy/http/HttpTransact.cc
> ----------------------------------------------------------------------
> diff --git a/proxy/http/HttpTransact.cc b/proxy/http/HttpTransact.cc
> index aa671fa..6b00e98 100644
> --- a/proxy/http/HttpTransact.cc
> +++ b/proxy/http/HttpTransact.cc
> @@ -7858,6 +7858,11 @@ HttpTransact::build_request(State* s, HTTPHdr* base_request, HTTPHdr* outgoing_r
>    }
>  }
> 
> +  if (s->txn_conf->send_100_continue_response) {
> +    HttpTransactHeaders::remove_100_continue_headers(s, outgoing_request);
> +    DebugTxn("http_trans", "[build_request] request expect 100-continue headers removed");
> +  }
> +
>  s->request_sent_time = ink_cluster_time();
>  s->current.now = s->request_sent_time;
>  // The assert is backwards in this case because request is being (re)sent.
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/9da12301/proxy/http/HttpTransactHeaders.cc
> ----------------------------------------------------------------------
> diff --git a/proxy/http/HttpTransactHeaders.cc b/proxy/http/HttpTransactHeaders.cc
> index 4574c21..9be81d0 100644
> --- a/proxy/http/HttpTransactHeaders.cc
> +++ b/proxy/http/HttpTransactHeaders.cc
> @@ -1007,6 +1007,18 @@ HttpTransactHeaders::remove_conditional_headers(HTTPHdr *outgoing)
>  // TODO: how about RANGE and IF_RANGE?
> }
> 
> +void
> +HttpTransactHeaders::remove_100_continue_headers(HttpTransact::State *s, HTTPHdr *outgoing)
> +{
> +  int len = 0;
> +  const char *expect = s->hdr_info.client_request.value_get(MIME_FIELD_EXPECT, MIME_LEN_EXPECT, &len);
> +
> +  if ((len == HTTP_LEN_100_CONTINUE) && (strncasecmp(expect, HTTP_VALUE_100_CONTINUE, HTTP_LEN_100_CONTINUE) == 0)) {
> +    outgoing->field_delete(MIME_FIELD_EXPECT, MIME_LEN_EXPECT);
> +  }
> +}
> +
> +
> 
> ////////////////////////////////////////////////////////////////////////
> // Deal with lame-o servers by removing the host name from the url.
> 
> http://git-wip-us.apache.org/repos/asf/trafficserver/blob/9da12301/proxy/http/HttpTransactHeaders.h
> ----------------------------------------------------------------------
> diff --git a/proxy/http/HttpTransactHeaders.h b/proxy/http/HttpTransactHeaders.h
> index 0fa3a03..cc6f475 100644
> --- a/proxy/http/HttpTransactHeaders.h
> +++ b/proxy/http/HttpTransactHeaders.h
> @@ -79,6 +79,7 @@ public:
>  static void insert_basic_realm_in_proxy_authenticate(const char *realm, HTTPHdr * header, bool bRevPrxy);
> 
>  static void remove_conditional_headers(HTTPHdr * outgoing);
> +  static void remove_100_continue_headers(HttpTransact::State *s, HTTPHdr * outgoing);
>  static void remove_host_name_from_url(HTTPHdr * outgoing_request);
>  static void add_global_user_agent_header_to_request(HttpConfigParams *http_config_param, HTTPHdr * header);
>  static void add_server_header_to_response(OverridableHttpConfigParams *http_txn_conf, HTTPHdr * header);
>