summaryrefslogtreecommitdiffstats
path: root/modules/http2/mod_proxy_http2.c
diff options
context:
space:
mode:
Diffstat (limited to 'modules/http2/mod_proxy_http2.c')
-rw-r--r--modules/http2/mod_proxy_http2.c494
1 files changed, 151 insertions, 343 deletions
diff --git a/modules/http2/mod_proxy_http2.c b/modules/http2/mod_proxy_http2.c
index a7e0dcd..ebf8f61 100644
--- a/modules/http2/mod_proxy_http2.c
+++ b/modules/http2/mod_proxy_http2.c
@@ -16,13 +16,14 @@
#include <nghttp2/nghttp2.h>
+#include <ap_mmn.h>
#include <httpd.h>
#include <mod_proxy.h>
#include "mod_http2.h"
#include "mod_proxy_http2.h"
-#include "h2_request.h"
+#include "h2.h"
#include "h2_proxy_util.h"
#include "h2_version.h"
#include "h2_proxy_session.h"
@@ -46,19 +47,11 @@ AP_DECLARE_MODULE(proxy_http2) = {
/* Optional functions from mod_http2 */
static int (*is_h2)(conn_rec *c);
-static apr_status_t (*req_engine_push)(const char *name, request_rec *r,
- http2_req_engine_init *einit);
-static apr_status_t (*req_engine_pull)(h2_req_engine *engine,
- apr_read_type_e block,
- int capacity,
- request_rec **pr);
-static void (*req_engine_done)(h2_req_engine *engine, conn_rec *r_conn,
- apr_status_t status);
-
+
typedef struct h2_proxy_ctx {
- conn_rec *owner;
+ const char *id;
+ conn_rec *cfront;
apr_pool_t *pool;
- request_rec *rbase;
server_rec *server;
const char *proxy_func;
char server_portstr[32];
@@ -66,20 +59,16 @@ typedef struct h2_proxy_ctx {
proxy_worker *worker;
proxy_server_conf *conf;
- h2_req_engine *engine;
- const char *engine_id;
- const char *engine_type;
- apr_pool_t *engine_pool;
apr_size_t req_buffer_size;
- h2_proxy_fifo *requests;
int capacity;
- unsigned standalone : 1;
unsigned is_ssl : 1;
- unsigned flushall : 1;
- apr_status_t r_status; /* status of our first request work */
- h2_proxy_session *session; /* current http2 session against backend */
+ request_rec *r; /* the request processed in this ctx */
+ int r_status; /* status of request work */
+ int r_done; /* request was processed, not necessarily successfully */
+ int r_may_retry; /* request may be retried */
+ int has_reusable_session; /* http2 session is live and clean */
} h2_proxy_ctx;
static int h2_proxy_post_config(apr_pool_t *p, apr_pool_t *plog,
@@ -104,16 +93,6 @@ static int h2_proxy_post_config(apr_pool_t *p, apr_pool_t *plog,
MOD_HTTP2_VERSION, ngh2? ngh2->version_str : "unknown");
is_h2 = APR_RETRIEVE_OPTIONAL_FN(http2_is_h2);
- req_engine_push = APR_RETRIEVE_OPTIONAL_FN(http2_req_engine_push);
- req_engine_pull = APR_RETRIEVE_OPTIONAL_FN(http2_req_engine_pull);
- req_engine_done = APR_RETRIEVE_OPTIONAL_FN(http2_req_engine_done);
-
- /* we need all of them */
- if (!req_engine_push || !req_engine_pull || !req_engine_done) {
- req_engine_push = NULL;
- req_engine_pull = NULL;
- req_engine_done = NULL;
- }
return status;
}
@@ -174,9 +153,24 @@ static int proxy_http2_canon(request_rec *r, char *url)
if (apr_table_get(r->notes, "proxy-nocanon")) {
path = url; /* this is the raw path */
}
+ else if (apr_table_get(r->notes, "proxy-noencode")) {
+ path = url; /* this is the encoded path already */
+ search = r->args;
+ }
else {
+#ifdef PROXY_CANONENC_NOENCODEDSLASHENCODING
+ core_dir_config *d = ap_get_core_module_config(r->per_dir_config);
+ int flags = d->allow_encoded_slashes && !d->decode_encoded_slashes ? PROXY_CANONENC_NOENCODEDSLASHENCODING : 0;
+
+ path = ap_proxy_canonenc_ex(r->pool, url, (int)strlen(url),
+ enc_path, flags, r->proxyreq);
+#else
path = ap_proxy_canonenc(r->pool, url, (int)strlen(url),
enc_path, 0, r->proxyreq);
+#endif
+ if (!path) {
+ return HTTP_BAD_REQUEST;
+ }
search = r->args;
}
break;
@@ -184,9 +178,21 @@ static int proxy_http2_canon(request_rec *r, char *url)
path = url;
break;
}
-
- if (path == NULL) {
- return HTTP_BAD_REQUEST;
+ /*
+ * If we have a raw control character or a ' ' in nocanon path or
+ * r->args, correct encoding was missed.
+ */
+ if (path == url && *ap_scan_vchar_obstext(path)) {
+ ap_log_rerror(APLOG_MARK, APLOG_ERR, 0, r, APLOGNO(10420)
+ "To be forwarded path contains control "
+ "characters or spaces");
+ return HTTP_FORBIDDEN;
+ }
+ if (search && *ap_scan_vchar_obstext(search)) {
+ ap_log_rerror(APLOG_MARK, APLOG_ERR, 0, r, APLOGNO(10412)
+ "To be forwarded query string contains control "
+ "characters or spaces");
+ return HTTP_FORBIDDEN;
}
if (port != def_port) {
@@ -204,45 +210,6 @@ static int proxy_http2_canon(request_rec *r, char *url)
return OK;
}
-static void out_consumed(void *baton, conn_rec *c, apr_off_t bytes)
-{
- h2_proxy_ctx *ctx = baton;
-
- if (ctx->session) {
- h2_proxy_session_update_window(ctx->session, c, bytes);
- }
-}
-
-static apr_status_t proxy_engine_init(h2_req_engine *engine,
- const char *id,
- const char *type,
- apr_pool_t *pool,
- apr_size_t req_buffer_size,
- request_rec *r,
- http2_output_consumed **pconsumed,
- void **pctx)
-{
- h2_proxy_ctx *ctx = ap_get_module_config(r->connection->conn_config,
- &proxy_http2_module);
- if (!ctx) {
- ap_log_rerror(APLOG_MARK, APLOG_WARNING, 0, r, APLOGNO(03368)
- "h2_proxy_session, engine init, no ctx found");
- return APR_ENOTIMPL;
- }
-
- ctx->pool = pool;
- ctx->engine = engine;
- ctx->engine_id = id;
- ctx->engine_type = type;
- ctx->engine_pool = pool;
- ctx->req_buffer_size = req_buffer_size;
- ctx->capacity = H2MIN(100, h2_proxy_fifo_capacity(ctx->requests));
-
- *pconsumed = out_consumed;
- *pctx = ctx;
- return APR_SUCCESS;
-}
-
static apr_status_t add_request(h2_proxy_session *session, request_rec *r)
{
h2_proxy_ctx *ctx = session->user_data;
@@ -252,7 +219,7 @@ static apr_status_t add_request(h2_proxy_session *session, request_rec *r)
url = apr_table_get(r->notes, H2_PROXY_REQ_URL_NOTE);
apr_table_setn(r->notes, "proxy-source-port", apr_psprintf(r->pool, "%hu",
ctx->p_conn->connection->local_addr->port));
- status = h2_proxy_session_submit(session, url, r, ctx->standalone);
+ status = h2_proxy_session_submit(session, url, r, 1);
if (status != APR_SUCCESS) {
ap_log_cerror(APLOG_MARK, APLOG_ERR, status, r->connection, APLOGNO(03351)
"pass request body failed to %pI (%s) from %s (%s)",
@@ -264,201 +231,84 @@ static apr_status_t add_request(h2_proxy_session *session, request_rec *r)
}
static void request_done(h2_proxy_ctx *ctx, request_rec *r,
- apr_status_t status, int touched)
+ apr_status_t status, int touched, int error_code)
{
- const char *task_id = apr_table_get(r->connection->notes, H2_TASK_ID_NOTE);
-
- ap_log_cerror(APLOG_MARK, APLOG_TRACE1, status, r->connection,
- "h2_proxy_session(%s): request done %s, touched=%d",
- ctx->engine_id, task_id, touched);
- if (status != APR_SUCCESS) {
- if (!touched) {
- /* untouched request, need rescheduling */
- status = h2_proxy_fifo_push(ctx->requests, r);
- ap_log_cerror(APLOG_MARK, APLOG_DEBUG, status, r->connection,
- APLOGNO(03369)
- "h2_proxy_session(%s): rescheduled request %s",
- ctx->engine_id, task_id);
- return;
- }
- else {
- const char *uri;
- uri = apr_uri_unparse(r->pool, &r->parsed_uri, 0);
- ap_log_cerror(APLOG_MARK, APLOG_DEBUG, status, r->connection,
- APLOGNO(03471) "h2_proxy_session(%s): request %s -> %s "
- "not complete, cannot repeat",
- ctx->engine_id, task_id, uri);
- }
- }
-
- if (r == ctx->rbase) {
- ctx->r_status = ((status == APR_SUCCESS)? APR_SUCCESS
- : HTTP_SERVICE_UNAVAILABLE);
- }
-
- if (req_engine_done && ctx->engine) {
- ap_log_cerror(APLOG_MARK, APLOG_DEBUG, status, r->connection,
- APLOGNO(03370)
- "h2_proxy_session(%s): finished request %s",
- ctx->engine_id, task_id);
- req_engine_done(ctx->engine, r->connection, status);
+ if (r == ctx->r) {
+ ap_log_cerror(APLOG_MARK, APLOG_TRACE1, status, r->connection,
+ "h2_proxy_session(%s): request done, touched=%d, error=%d",
+ ctx->id, touched, error_code);
+ ctx->r_done = 1;
+ if (touched) ctx->r_may_retry = 0;
+ ctx->r_status = error_code? HTTP_BAD_GATEWAY :
+ ((status == APR_SUCCESS)? OK :
+ ap_map_http_request_error(status, HTTP_SERVICE_UNAVAILABLE));
}
}
static void session_req_done(h2_proxy_session *session, request_rec *r,
- apr_status_t status, int touched)
-{
- request_done(session->user_data, r, status, touched);
-}
-
-static apr_status_t next_request(h2_proxy_ctx *ctx, int before_leave)
+ apr_status_t status, int touched, int error_code)
{
- if (h2_proxy_fifo_count(ctx->requests) > 0) {
- return APR_SUCCESS;
- }
- else if (req_engine_pull && ctx->engine) {
- apr_status_t status;
- request_rec *r = NULL;
-
- status = req_engine_pull(ctx->engine, before_leave?
- APR_BLOCK_READ: APR_NONBLOCK_READ,
- ctx->capacity, &r);
- if (status == APR_SUCCESS && r) {
- ap_log_cerror(APLOG_MARK, APLOG_TRACE3, status, ctx->owner,
- "h2_proxy_engine(%s): pulled request (%s) %s",
- ctx->engine_id,
- before_leave? "before leave" : "regular",
- r->the_request);
- h2_proxy_fifo_push(ctx->requests, r);
- }
- return APR_STATUS_IS_EAGAIN(status)? APR_SUCCESS : status;
- }
- return APR_EOF;
+ request_done(session->user_data, r, status, touched, error_code);
}
-static apr_status_t proxy_engine_run(h2_proxy_ctx *ctx) {
+static apr_status_t ctx_run(h2_proxy_ctx *ctx) {
apr_status_t status = OK;
+ h2_proxy_session *session;
int h2_front;
- request_rec *r;
/* Step Four: Send the Request in a new HTTP/2 stream and
* loop until we got the response or encounter errors.
*/
- ap_log_cerror(APLOG_MARK, APLOG_TRACE1, 0, ctx->owner,
- "eng(%s): setup session", ctx->engine_id);
- h2_front = is_h2? is_h2(ctx->owner) : 0;
- ctx->session = h2_proxy_session_setup(ctx->engine_id, ctx->p_conn, ctx->conf,
- h2_front, 30,
- h2_proxy_log2((int)ctx->req_buffer_size),
- session_req_done);
- if (!ctx->session) {
- ap_log_cerror(APLOG_MARK, APLOG_DEBUG, 0, ctx->owner,
+ ctx->has_reusable_session = 0; /* don't know yet */
+ h2_front = is_h2? is_h2(ctx->cfront) : 0;
+ session = h2_proxy_session_setup(ctx->id, ctx->p_conn, ctx->conf,
+ h2_front, 30,
+ h2_proxy_log2((int)ctx->req_buffer_size),
+ session_req_done);
+ if (!session) {
+ ap_log_cerror(APLOG_MARK, APLOG_DEBUG, 0, ctx->cfront,
APLOGNO(03372) "session unavailable");
return HTTP_SERVICE_UNAVAILABLE;
}
- ap_log_cerror(APLOG_MARK, APLOG_DEBUG, 0, ctx->owner, APLOGNO(03373)
- "eng(%s): run session %s", ctx->engine_id, ctx->session->id);
- ctx->session->user_data = ctx;
+ ap_log_cerror(APLOG_MARK, APLOG_DEBUG, 0, ctx->cfront, APLOGNO(03373)
+ "eng(%s): run session %s", ctx->id, session->id);
+ session->user_data = ctx;
- while (!ctx->owner->aborted) {
- if (APR_SUCCESS == h2_proxy_fifo_try_pull(ctx->requests, (void**)&r)) {
- add_request(ctx->session, r);
- }
-
- status = h2_proxy_session_process(ctx->session);
-
- if (status == APR_SUCCESS) {
- apr_status_t s2;
- /* ongoing processing, call again */
- if (ctx->session->remote_max_concurrent > 0
- && ctx->session->remote_max_concurrent != ctx->capacity) {
- ctx->capacity = H2MIN((int)ctx->session->remote_max_concurrent,
- h2_proxy_fifo_capacity(ctx->requests));
- }
- s2 = next_request(ctx, 0);
- if (s2 == APR_ECONNABORTED) {
- /* master connection gone */
- ap_log_cerror(APLOG_MARK, APLOG_DEBUG, s2, ctx->owner,
- APLOGNO(03374) "eng(%s): pull request",
- ctx->engine_id);
- /* give notice that we're leaving and cancel all ongoing
- * streams. */
- next_request(ctx, 1);
- h2_proxy_session_cancel_all(ctx->session);
- h2_proxy_session_process(ctx->session);
- status = ctx->r_status = APR_SUCCESS;
- break;
- }
- if ((h2_proxy_fifo_count(ctx->requests) == 0)
- && h2_proxy_ihash_empty(ctx->session->streams)) {
- break;
- }
- }
- else {
- /* end of processing, maybe error */
- ap_log_cerror(APLOG_MARK, APLOG_DEBUG, status, ctx->owner,
+ ctx->r_done = 0;
+ add_request(session, ctx->r);
+
+ while (!ctx->cfront->aborted && !ctx->r_done) {
+
+ status = h2_proxy_session_process(session);
+ if (status != APR_SUCCESS) {
+ /* Encountered an error during session processing */
+ ap_log_cerror(APLOG_MARK, APLOG_DEBUG, status, ctx->cfront,
APLOGNO(03375) "eng(%s): end of session %s",
- ctx->engine_id, ctx->session->id);
- /*
- * Any open stream of that session needs to
+ ctx->id, session->id);
+ /* Any open stream of that session needs to
* a) be reopened on the new session iff safe to do so
* b) reported as done (failed) otherwise
*/
- h2_proxy_session_cleanup(ctx->session, session_req_done);
- break;
+ h2_proxy_session_cleanup(session, session_req_done);
+ goto out;
}
}
- ctx->session->user_data = NULL;
- ctx->session = NULL;
-
+out:
+ if (ctx->cfront->aborted) {
+ /* master connection gone */
+ ap_log_cerror(APLOG_MARK, APLOG_DEBUG, status, ctx->cfront,
+ APLOGNO(03374) "eng(%s): master connection gone", ctx->id);
+ /* cancel all ongoing requests */
+ h2_proxy_session_cancel_all(session);
+ h2_proxy_session_process(session);
+ }
+ ctx->has_reusable_session = h2_proxy_session_is_reusable(session);
+ session->user_data = NULL;
return status;
}
-static apr_status_t push_request_somewhere(h2_proxy_ctx *ctx, request_rec *r)
-{
- conn_rec *c = ctx->owner;
- const char *engine_type, *hostname;
-
- hostname = (ctx->p_conn->ssl_hostname?
- ctx->p_conn->ssl_hostname : ctx->p_conn->hostname);
- engine_type = apr_psprintf(ctx->pool, "proxy_http2 %s%s", hostname,
- ctx->server_portstr);
-
- if (c->master && req_engine_push && r && is_h2 && is_h2(c)) {
- /* If we are have req_engine capabilities, push the handling of this
- * request (e.g. slave connection) to a proxy_http2 engine which
- * uses the same backend. We may be called to create an engine
- * ourself. */
- if (req_engine_push(engine_type, r, proxy_engine_init) == APR_SUCCESS) {
- if (ctx->engine == NULL) {
- /* request has been assigned to an engine in another thread */
- return SUSPENDED;
- }
- }
- }
-
- if (!ctx->engine) {
- /* No engine was available or has been initialized, handle this
- * request just by ourself. */
- ctx->engine_id = apr_psprintf(ctx->pool, "eng-proxy-%ld", c->id);
- ctx->engine_type = engine_type;
- ctx->engine_pool = ctx->pool;
- ctx->req_buffer_size = (32*1024);
- ctx->standalone = 1;
- ap_log_cerror(APLOG_MARK, APLOG_TRACE1, 0, c,
- "h2_proxy_http2(%ld): setup standalone engine for type %s",
- c->id, engine_type);
- }
- else {
- ap_log_cerror(APLOG_MARK, APLOG_TRACE1, 0, c,
- "H2: hosting engine %s", ctx->engine_id);
- }
-
- return h2_proxy_fifo_push(ctx->requests, r);
-}
-
static int proxy_http2_handler(request_rec *r,
proxy_worker *worker,
proxy_server_conf *conf,
@@ -498,29 +348,32 @@ static int proxy_http2_handler(request_rec *r,
default:
return DECLINED;
}
-
+
ctx = apr_pcalloc(r->pool, sizeof(*ctx));
- ctx->owner = r->connection;
- ctx->pool = r->pool;
- ctx->rbase = r;
- ctx->server = r->server;
+ ctx->id = apr_psprintf(r->pool, "%ld", (long)r->connection->id);
+ ctx->cfront = r->connection;
+ ctx->pool = r->pool;
+ ctx->server = r->server;
ctx->proxy_func = proxy_func;
- ctx->is_ssl = is_ssl;
- ctx->worker = worker;
- ctx->conf = conf;
- ctx->flushall = apr_table_get(r->subprocess_env, "proxy-flushall")? 1 : 0;
- ctx->r_status = HTTP_SERVICE_UNAVAILABLE;
+ ctx->is_ssl = is_ssl;
+ ctx->worker = worker;
+ ctx->conf = conf;
+ ctx->req_buffer_size = (32*1024);
+ ctx->r = r;
+ ctx->r_status = status = HTTP_SERVICE_UNAVAILABLE;
+ ctx->r_done = 0;
+ ctx->r_may_retry = 1;
- h2_proxy_fifo_set_create(&ctx->requests, ctx->pool, 100);
-
- ap_set_module_config(ctx->owner->conn_config, &proxy_http2_module, ctx);
+ ap_set_module_config(ctx->cfront->conn_config, &proxy_http2_module, ctx);
/* scheme says, this is for us. */
- apr_table_setn(ctx->rbase->notes, H2_PROXY_REQ_URL_NOTE, url);
- ap_log_rerror(APLOG_MARK, APLOG_TRACE1, 0, ctx->rbase,
+ apr_table_setn(ctx->r->notes, H2_PROXY_REQ_URL_NOTE, url);
+ ap_log_rerror(APLOG_MARK, APLOG_TRACE1, 0, ctx->r,
"H2: serving URL %s", url);
run_connect:
+ if (ctx->cfront->aborted) goto cleanup;
+
/* Get a proxy_conn_rec from the worker, might be a new one, might
* be one still open from another request, or it might fail if the
* worker is stopped or in error. */
@@ -530,25 +383,11 @@ run_connect:
}
ctx->p_conn->is_ssl = ctx->is_ssl;
- if (ctx->is_ssl && ctx->p_conn->connection) {
- /* If there are some metadata on the connection (e.g. TLS alert),
- * let mod_ssl detect them, and create a new connection below.
- */
- apr_bucket_brigade *tmp_bb;
- tmp_bb = apr_brigade_create(ctx->rbase->pool,
- ctx->rbase->connection->bucket_alloc);
- status = ap_get_brigade(ctx->p_conn->connection->input_filters, tmp_bb,
- AP_MODE_SPECULATIVE, APR_NONBLOCK_READ, 1);
- if (status != APR_SUCCESS && !APR_STATUS_IS_EAGAIN(status)) {
- ctx->p_conn->close = 1;
- }
- apr_brigade_cleanup(tmp_bb);
- }
/* Step One: Determine the URL to connect to (might be a proxy),
* initialize the backend accordingly and determine the server
* port string we can expect in responses. */
- if ((status = ap_proxy_determine_connection(ctx->pool, ctx->rbase, conf, worker,
+ if ((status = ap_proxy_determine_connection(ctx->pool, ctx->r, conf, worker,
ctx->p_conn, &uri, &locurl,
proxyname, proxyport,
ctx->server_portstr,
@@ -556,111 +395,80 @@ run_connect:
goto cleanup;
}
- /* If we are not already hosting an engine, try to push the request
- * to an already existing engine or host a new engine here. */
- if (r && !ctx->engine) {
- ctx->r_status = push_request_somewhere(ctx, r);
- r = NULL;
- if (ctx->r_status == SUSPENDED) {
- /* request was pushed to another thread, leave processing here */
- goto cleanup;
- }
- }
-
/* Step Two: Make the Connection (or check that an already existing
* socket is still usable). On success, we have a socket connected to
* backend->hostname. */
if (ap_proxy_connect_backend(ctx->proxy_func, ctx->p_conn, ctx->worker,
ctx->server)) {
- ap_log_cerror(APLOG_MARK, APLOG_DEBUG, 0, ctx->owner, APLOGNO(03352)
+ ap_log_cerror(APLOG_MARK, APLOG_DEBUG, 0, ctx->cfront, APLOGNO(03352)
"H2: failed to make connection to backend: %s",
ctx->p_conn->hostname);
- goto reconnect;
+ goto cleanup;
}
/* Step Three: Create conn_rec for the socket we have open now. */
- if (!ctx->p_conn->connection) {
- status = ap_proxy_connection_create_ex(ctx->proxy_func,
- ctx->p_conn, ctx->rbase);
- if (status != OK) {
- ap_log_cerror(APLOG_MARK, APLOG_DEBUG, status, ctx->owner, APLOGNO(03353)
- "setup new connection: is_ssl=%d %s %s %s",
- ctx->p_conn->is_ssl, ctx->p_conn->ssl_hostname,
- locurl, ctx->p_conn->hostname);
- goto reconnect;
- }
-
- if (!ctx->p_conn->data) {
- /* New conection: set a note on the connection what CN is
- * requested and what protocol we want */
- if (ctx->p_conn->ssl_hostname) {
- ap_log_cerror(APLOG_MARK, APLOG_TRACE1, status, ctx->owner,
- "set SNI to %s for (%s)",
- ctx->p_conn->ssl_hostname,
- ctx->p_conn->hostname);
- apr_table_setn(ctx->p_conn->connection->notes,
- "proxy-request-hostname", ctx->p_conn->ssl_hostname);
- }
- if (ctx->is_ssl) {
- apr_table_setn(ctx->p_conn->connection->notes,
- "proxy-request-alpn-protos", "h2");
- }
- }
+ status = ap_proxy_connection_create_ex(ctx->proxy_func, ctx->p_conn, ctx->r);
+ if (status != OK) {
+ ap_log_cerror(APLOG_MARK, APLOG_DEBUG, status, ctx->cfront, APLOGNO(03353)
+ "setup new connection: is_ssl=%d %s %s %s",
+ ctx->p_conn->is_ssl, ctx->p_conn->ssl_hostname,
+ locurl, ctx->p_conn->hostname);
+ ctx->r_status = ap_map_http_request_error(status, HTTP_SERVICE_UNAVAILABLE);
+ goto cleanup;
}
-
-run_session:
- status = proxy_engine_run(ctx);
- if (status == APR_SUCCESS) {
- /* session and connection still ok */
- if (next_request(ctx, 1) == APR_SUCCESS) {
- /* more requests, run again */
- ap_log_cerror(APLOG_MARK, APLOG_DEBUG, 0, ctx->owner, APLOGNO(03376)
- "run_session, again");
- goto run_session;
- }
- /* done */
- ctx->engine = NULL;
+
+ if (!ctx->p_conn->data && ctx->is_ssl) {
+ /* New SSL connection: set a note on the connection about what
+ * protocol we need. */
+ apr_table_setn(ctx->p_conn->connection->notes,
+ "proxy-request-alpn-protos", "h2");
}
-reconnect:
- if (next_request(ctx, 1) == APR_SUCCESS) {
- /* Still more to do, tear down old conn and start over */
+ if (ctx->cfront->aborted) goto cleanup;
+ status = ctx_run(ctx);
+
+ if (ctx->r_status != OK && ctx->r_may_retry && !ctx->cfront->aborted) {
+ /* Not successfully processed, but may retry, tear down old conn and start over */
if (ctx->p_conn) {
ctx->p_conn->close = 1;
- /*only in trunk so far */
- /*proxy_run_detach_backend(r, ctx->p_conn);*/
+#if AP_MODULE_MAGIC_AT_LEAST(20140207, 2)
+ proxy_run_detach_backend(r, ctx->p_conn);
+#endif
ap_proxy_release_connection(ctx->proxy_func, ctx->p_conn, ctx->server);
ctx->p_conn = NULL;
}
++reconnects;
- if (reconnects < 5 && !ctx->owner->aborted) {
+ if (reconnects < 2) {
goto run_connect;
}
- ap_log_cerror(APLOG_MARK, APLOG_DEBUG, 0, ctx->owner, APLOGNO(10023)
- "giving up after %d reconnects, %d requests todo",
- reconnects, h2_proxy_fifo_count(ctx->requests));
+ ap_log_cerror(APLOG_MARK, APLOG_DEBUG, 0, ctx->cfront, APLOGNO(10023)
+ "giving up after %d reconnects, request-done=%d",
+ reconnects, ctx->r_done);
}
cleanup:
if (ctx->p_conn) {
- if (status != APR_SUCCESS) {
- /* close socket when errors happened or session shut down (EOF) */
+ if (status != APR_SUCCESS || !ctx->has_reusable_session) {
+ /* close socket when errors happened or session is not "clean",
+ * meaning in a working condition with no open streams */
ctx->p_conn->close = 1;
}
- /*only in trunk so far */
- /*proxy_run_detach_backend(ctx->rbase, ctx->p_conn);*/
+#if AP_MODULE_MAGIC_AT_LEAST(20140207, 2)
+ proxy_run_detach_backend(ctx->r, ctx->p_conn);
+#endif
ap_proxy_release_connection(ctx->proxy_func, ctx->p_conn, ctx->server);
ctx->p_conn = NULL;
}
- /* Any requests will still have need to fail */
- while (APR_SUCCESS == h2_proxy_fifo_try_pull(ctx->requests, (void**)&r)) {
- request_done(ctx, r, HTTP_SERVICE_UNAVAILABLE, 1);
- }
-
- ap_set_module_config(ctx->owner->conn_config, &proxy_http2_module, NULL);
- ap_log_cerror(APLOG_MARK, APLOG_DEBUG, status, ctx->owner,
+ ap_set_module_config(ctx->cfront->conn_config, &proxy_http2_module, NULL);
+ ap_log_cerror(APLOG_MARK, APLOG_DEBUG, status, ctx->cfront,
APLOGNO(03377) "leaving handler");
+ if (ctx->r_status != OK) {
+ ap_die(ctx->r_status, r);
+ }
+ else if (status != APR_SUCCESS) {
+ ap_die(ap_map_http_request_error(status, HTTP_SERVICE_UNAVAILABLE), r);
+ }
return ctx->r_status;
}