#include <proto/arg.h>
#include <proto/backend.h>
#include <proto/channel.h>
+#include <proto/checks.h>
#include <proto/frontend.h>
#include <proto/http_htx.h>
#include <proto/lb_chash.h>
return 0;
}
+/* Check if the connection request is in such a state that it can be aborted. */
+static int back_may_abort_req(struct channel *req, struct stream *s)
+{
+ return ((req->flags & (CF_READ_ERROR)) ||
+ ((req->flags & (CF_SHUTW_NOW|CF_SHUTW)) && /* empty and client aborted */
+ (channel_is_empty(req) || (s->be->options & PR_O_ABRT_CLOSE))));
+}
+
+/* Update back stream interface status for input states SI_ST_ASS, SI_ST_QUE,
+ * SI_ST_TAR. Other input states are simply ignored.
+ * Possible output states are SI_ST_CLO, SI_ST_TAR, SI_ST_ASS, SI_ST_REQ, SI_ST_CON
+ * and SI_ST_EST. Flags must have previously been updated for timeouts and other
+ * conditions.
+ */
+void back_try_conn_req(struct stream *s)
+{
+ struct server *srv = objt_server(s->target);
+ struct stream_interface *si = &s->si[1];
+ struct channel *req = &s->req;
+
+ DBG_TRACE_ENTER(STRM_EV_STRM_PROC|STRM_EV_SI_ST, s);
+
+ if (si->state == SI_ST_ASS) {
+ /* Server assigned to connection request, we have to try to connect now */
+ int conn_err;
+
+ /* Before we try to initiate the connection, see if the
+ * request may be aborted instead.
+ */
+ if (back_may_abort_req(req, s)) {
+ si->err_type |= SI_ET_CONN_ABRT;
+ DBG_TRACE_STATE("connection aborted", STRM_EV_STRM_PROC|STRM_EV_SI_ST|STRM_EV_STRM_ERR, s);
+ goto abort_connection;
+ }
+
+ conn_err = connect_server(s);
+ srv = objt_server(s->target);
+
+ if (conn_err == SF_ERR_NONE) {
+ /* state = SI_ST_CON or SI_ST_EST now */
+ if (srv)
+ srv_inc_sess_ctr(srv);
+ if (srv)
+ srv_set_sess_last(srv);
+ DBG_TRACE_STATE("connection attempt", STRM_EV_STRM_PROC|STRM_EV_SI_ST, s);
+ goto end;
+ }
+
+ /* We have received a synchronous error. We might have to
+ * abort, retry immediately or redispatch.
+ */
+ if (conn_err == SF_ERR_INTERNAL) {
+ if (!si->err_type) {
+ si->err_type = SI_ET_CONN_OTHER;
+ }
+
+ if (srv)
+ srv_inc_sess_ctr(srv);
+ if (srv)
+ srv_set_sess_last(srv);
+ if (srv)
+ _HA_ATOMIC_ADD(&srv->counters.failed_conns, 1);
+ _HA_ATOMIC_ADD(&s->be->be_counters.failed_conns, 1);
+
+ /* release other streams waiting for this server */
+ sess_change_server(s, NULL);
+ if (may_dequeue_tasks(srv, s->be))
+ process_srv_queue(srv);
+
+ /* Failed and not retryable. */
+ si_shutr(si);
+ si_shutw(si);
+ req->flags |= CF_WRITE_ERROR;
+
+ s->logs.t_queue = tv_ms_elapsed(&s->logs.tv_accept, &now);
+
+ /* we may need to know the position in the queue for logging */
+ pendconn_cond_unlink(s->pend_pos);
+
+ /* no stream was ever accounted for this server */
+ si->state = SI_ST_CLO;
+ if (s->srv_error)
+ s->srv_error(s, si);
+ DBG_TRACE_STATE("internal error during connection", STRM_EV_STRM_PROC|STRM_EV_SI_ST|STRM_EV_STRM_ERR, s);
+ goto end;
+ }
+
+ /* We are facing a retryable error, but we don't want to run a
+ * turn-around now, as the problem is likely a source port
+ * allocation problem, so we want to retry now.
+ */
+ si->state = SI_ST_CER;
+ si->flags &= ~SI_FL_ERR;
+ back_handle_st_cer(s);
+
+ DBG_TRACE_STATE("connection error, retry", STRM_EV_STRM_PROC|STRM_EV_SI_ST|STRM_EV_STRM_ERR, s);
+ /* now si->state is one of SI_ST_CLO, SI_ST_TAR, SI_ST_ASS, SI_ST_REQ */
+ }
+ else if (si->state == SI_ST_QUE) {
+ /* connection request was queued, check for any update */
+ if (!pendconn_dequeue(s)) {
+ /* The connection is not in the queue anymore. Either
+ * we have a server connection slot available and we
+ * go directly to the assigned state, or we need to
+ * load-balance first and go to the INI state.
+ */
+ si->exp = TICK_ETERNITY;
+ if (unlikely(!(s->flags & SF_ASSIGNED)))
+ si->state = SI_ST_REQ;
+ else {
+ s->logs.t_queue = tv_ms_elapsed(&s->logs.tv_accept, &now);
+ si->state = SI_ST_ASS;
+ }
+ DBG_TRACE_STATE("dequeue connection request", STRM_EV_STRM_PROC|STRM_EV_SI_ST, s);
+ goto end;
+ }
+
+ /* Connection request still in queue... */
+ if (si->flags & SI_FL_EXP) {
+ /* ... and timeout expired */
+ si->exp = TICK_ETERNITY;
+ si->flags &= ~SI_FL_EXP;
+ s->logs.t_queue = tv_ms_elapsed(&s->logs.tv_accept, &now);
+
+ /* we may need to know the position in the queue for logging */
+ pendconn_cond_unlink(s->pend_pos);
+
+ if (srv)
+ _HA_ATOMIC_ADD(&srv->counters.failed_conns, 1);
+ _HA_ATOMIC_ADD(&s->be->be_counters.failed_conns, 1);
+ si_shutr(si);
+ si_shutw(si);
+ req->flags |= CF_WRITE_TIMEOUT;
+ if (!si->err_type)
+ si->err_type = SI_ET_QUEUE_TO;
+ si->state = SI_ST_CLO;
+ if (s->srv_error)
+ s->srv_error(s, si);
+ DBG_TRACE_STATE("connection request still queued", STRM_EV_STRM_PROC|STRM_EV_SI_ST, s);
+ goto end;
+ }
+
+ /* Connection remains in queue, check if we have to abort it */
+ if (back_may_abort_req(req, s)) {
+ s->logs.t_queue = tv_ms_elapsed(&s->logs.tv_accept, &now);
+
+ /* we may need to know the position in the queue for logging */
+ pendconn_cond_unlink(s->pend_pos);
+
+ si->err_type |= SI_ET_QUEUE_ABRT;
+ DBG_TRACE_STATE("abort queued connection request", STRM_EV_STRM_PROC|STRM_EV_SI_ST|STRM_EV_STRM_ERR, s);
+ goto abort_connection;
+ }
+
+ /* Nothing changed */
+ }
+ else if (si->state == SI_ST_TAR) {
+ /* Connection request might be aborted */
+ if (back_may_abort_req(req, s)) {
+ si->err_type |= SI_ET_CONN_ABRT;
+ DBG_TRACE_STATE("connection aborted", STRM_EV_STRM_PROC|STRM_EV_SI_ST|STRM_EV_STRM_ERR, s);
+ goto abort_connection;
+ }
+
+ if (!(si->flags & SI_FL_EXP))
+ return; /* still in turn-around */
+
+ si->flags &= ~SI_FL_EXP;
+ si->exp = TICK_ETERNITY;
+
+ /* we keep trying on the same server as long as the stream is
+ * marked "assigned".
+ * FIXME: Should we force a redispatch attempt when the server is down ?
+ */
+ if (s->flags & SF_ASSIGNED)
+ si->state = SI_ST_ASS;
+ else
+ si->state = SI_ST_REQ;
+
+ DBG_TRACE_STATE("retry connection now", STRM_EV_STRM_PROC|STRM_EV_SI_ST, s);
+ }
+
+ end:
+ DBG_TRACE_LEAVE(STRM_EV_STRM_PROC|STRM_EV_SI_ST, s);
+ return;
+
+abort_connection:
+ /* give up */
+ si->exp = TICK_ETERNITY;
+ si->flags &= ~SI_FL_EXP;
+ si_shutr(si);
+ si_shutw(si);
+ si->state = SI_ST_CLO;
+ if (s->srv_error)
+ s->srv_error(s, si);
+ DBG_TRACE_DEVEL("leaving on error", STRM_EV_STRM_PROC|STRM_EV_SI_ST|STRM_EV_STRM_ERR, s);
+ return;
+}
+
+/* This function initiates a server connection request on a stream interface
+ * already in SI_ST_REQ state. Upon success, the state goes to SI_ST_ASS for
+ * a real connection to a server, indicating that a server has been assigned,
+ * or SI_ST_EST for a successful connection to an applet. It may also return
+ * SI_ST_QUE, or SI_ST_CLO upon error.
+ */
+void back_handle_st_req(struct stream *s)
+{
+ struct stream_interface *si = &s->si[1];
+
+ if (si->state != SI_ST_REQ)
+ return;
+
+ DBG_TRACE_ENTER(STRM_EV_STRM_PROC|STRM_EV_SI_ST, s);
+
+ if (unlikely(obj_type(s->target) == OBJ_TYPE_APPLET)) {
+ /* the applet directly goes to the EST state */
+ struct appctx *appctx = objt_appctx(si->end);
+
+ if (!appctx || appctx->applet != __objt_applet(s->target))
+ appctx = si_register_handler(si, objt_applet(s->target));
+
+ if (!appctx) {
+ /* No more memory, let's immediately abort. Force the
+ * error code to ignore the ERR_LOCAL which is not a
+ * real error.
+ */
+ s->flags &= ~(SF_ERR_MASK | SF_FINST_MASK);
+
+ si_shutr(si);
+ si_shutw(si);
+ s->req.flags |= CF_WRITE_ERROR;
+ si->err_type = SI_ET_CONN_RES;
+ si->state = SI_ST_CLO;
+ if (s->srv_error)
+ s->srv_error(s, si);
+ DBG_TRACE_STATE("failed to register applet", STRM_EV_STRM_PROC|STRM_EV_SI_ST|STRM_EV_STRM_ERR, s);
+ goto end;
+ }
+
+ if (tv_iszero(&s->logs.tv_request))
+ s->logs.tv_request = now;
+ s->logs.t_queue = tv_ms_elapsed(&s->logs.tv_accept, &now);
+ si->state = SI_ST_EST;
+ si->err_type = SI_ET_NONE;
+ be_set_sess_last(s->be);
+
+ DBG_TRACE_STATE("applet registered", STRM_EV_STRM_PROC|STRM_EV_SI_ST, s);
+ /* let back_establish() finish the job */
+ goto end;
+ }
+
+ /* Try to assign a server */
+ if (srv_redispatch_connect(s) != 0) {
+ /* We did not get a server. Either we queued the
+ * connection request, or we encountered an error.
+ */
+ if (si->state == SI_ST_QUE) {
+ DBG_TRACE_STATE("connection request queued", STRM_EV_STRM_PROC|STRM_EV_SI_ST, s);
+ goto end;
+ }
+
+ /* we did not get any server, let's check the cause */
+ si_shutr(si);
+ si_shutw(si);
+ s->req.flags |= CF_WRITE_ERROR;
+ if (!si->err_type)
+ si->err_type = SI_ET_CONN_OTHER;
+ si->state = SI_ST_CLO;
+ if (s->srv_error)
+ s->srv_error(s, si);
+ DBG_TRACE_STATE("connection request failed", STRM_EV_STRM_PROC|STRM_EV_SI_ST|STRM_EV_STRM_ERR, s);
+ goto end;
+ }
+
+ /* The server is assigned */
+ s->logs.t_queue = tv_ms_elapsed(&s->logs.tv_accept, &now);
+ si->state = SI_ST_ASS;
+ be_set_sess_last(s->be);
+ DBG_TRACE_STATE("connection request assigned to a server", STRM_EV_STRM_PROC|STRM_EV_SI_ST, s);
+
+ end:
+ DBG_TRACE_LEAVE(STRM_EV_STRM_PROC|STRM_EV_SI_ST, s);
+}
+
+/* This function is called with (si->state == SI_ST_CON) meaning that a
+ * connection was attempted and that the file descriptor is already allocated.
+ * We must check for timeout, error and abort. Possible output states are
+ * SI_ST_CER (error), SI_ST_DIS (abort), and SI_ST_CON (no change). This only
+ * works with connection-based streams. We know that there were no I/O event
+ * when reaching this function. Timeouts and errors are *not* cleared.
+ */
+void back_handle_st_con(struct stream *s)
+{
+ struct stream_interface *si = &s->si[1];
+ struct channel *req = &s->req;
+ struct channel *rep = &s->res;
+ struct conn_stream *srv_cs = objt_cs(si->end);
+ struct connection *conn = srv_cs ? srv_cs->conn : objt_conn(si->end);
+
+ DBG_TRACE_ENTER(STRM_EV_STRM_PROC|STRM_EV_SI_ST, s);
+
+ /* the client might want to abort */
+ if ((rep->flags & CF_SHUTW) ||
+ ((req->flags & CF_SHUTW_NOW) &&
+ (channel_is_empty(req) || (s->be->options & PR_O_ABRT_CLOSE)))) {
+ si->flags |= SI_FL_NOLINGER;
+ si_shutw(si);
+ si->err_type |= SI_ET_CONN_ABRT;
+ if (s->srv_error)
+ s->srv_error(s, si);
+ /* Note: state = SI_ST_DIS now */
+ DBG_TRACE_STATE("client abort during connection attempt", STRM_EV_STRM_PROC|STRM_EV_SI_ST|STRM_EV_STRM_ERR, s);
+ }
+
+ /* retryable error ? */
+ else if (si->flags & (SI_FL_EXP|SI_FL_ERR)) {
+ if (!(s->flags & SF_SRV_REUSED) && conn) {
+ conn_stop_tracking(conn);
+ conn_full_close(conn);
+ }
+
+ if (!si->err_type) {
+ if (si->flags & SI_FL_ERR)
+ si->err_type = SI_ET_CONN_ERR;
+ else
+ si->err_type = SI_ET_CONN_TO;
+ }
+
+ si->state = SI_ST_CER;
+ DBG_TRACE_STATE("connection failed, retry", STRM_EV_STRM_PROC|STRM_EV_SI_ST|STRM_EV_STRM_ERR, s);
+ }
+
+ DBG_TRACE_LEAVE(STRM_EV_STRM_PROC|STRM_EV_SI_ST, s);
+}
+
+/* This function is called with (si->state == SI_ST_CER) meaning that a
+ * previous connection attempt has failed and that the file descriptor
+ * has already been released. Possible causes include asynchronous error
+ * notification and time out. Possible output states are SI_ST_CLO when
+ * retries are exhausted, SI_ST_TAR when a delay is wanted before a new
+ * connection attempt, SI_ST_ASS when it's wise to retry on the same server,
+ * and SI_ST_REQ when an immediate redispatch is wanted. The buffers are
+ * marked as in error state. Timeouts and errors are cleared before retrying.
+ */
+void back_handle_st_cer(struct stream *s)
+{
+ struct stream_interface *si = &s->si[1];
+ struct conn_stream *cs = objt_cs(si->end);
+ struct connection *conn = cs_conn(cs);
+
+ DBG_TRACE_ENTER(STRM_EV_STRM_PROC|STRM_EV_SI_ST, s);
+
+ si->exp = TICK_ETERNITY;
+ si->flags &= ~SI_FL_EXP;
+
+ /* we probably have to release last stream from the server */
+ if (objt_server(s->target)) {
+ health_adjust(objt_server(s->target), HANA_STATUS_L4_ERR);
+
+ if (s->flags & SF_CURR_SESS) {
+ s->flags &= ~SF_CURR_SESS;
+ _HA_ATOMIC_SUB(&__objt_server(s->target)->cur_sess, 1);
+ }
+
+ if ((si->flags & SI_FL_ERR) &&
+ conn && conn->err_code == CO_ER_SSL_MISMATCH_SNI) {
+ /* We tried to connect to a server which is configured
+ * with "verify required" and which doesn't have the
+ * "verifyhost" directive. The server presented a wrong
+ * certificate (a certificate for an unexpected name),
+ * which implies that we have used SNI in the handshake,
+ * and that the server doesn't have the associated cert
+ * and presented a default one.
+ *
+ * This is a serious enough issue not to retry. It's
+ * especially important because this wrong name might
+ * either be the result of a configuration error, and
+ * retrying will only hammer the server, or is caused
+ * by the use of a wrong SNI value, most likely
+ * provided by the client and we don't want to let the
+ * client provoke retries.
+ */
+ si->conn_retries = 0;
+ DBG_TRACE_DEVEL("Bad SSL cert, disable connection retries", STRM_EV_STRM_PROC|STRM_EV_SI_ST|STRM_EV_STRM_ERR, s);
+ }
+ }
+
+ /* ensure that we have enough retries left */
+ si->conn_retries--;
+ if (si->conn_retries < 0 || !(s->be->retry_type & PR_RE_CONN_FAILED)) {
+ if (!si->err_type) {
+ si->err_type = SI_ET_CONN_ERR;
+ }
+
+ if (objt_server(s->target))
+ _HA_ATOMIC_ADD(&objt_server(s->target)->counters.failed_conns, 1);
+ _HA_ATOMIC_ADD(&s->be->be_counters.failed_conns, 1);
+ sess_change_server(s, NULL);
+ if (may_dequeue_tasks(objt_server(s->target), s->be))
+ process_srv_queue(objt_server(s->target));
+
+ /* shutw is enough so stop a connecting socket */
+ si_shutw(si);
+ s->req.flags |= CF_WRITE_ERROR;
+ s->res.flags |= CF_READ_ERROR;
+
+ si->state = SI_ST_CLO;
+ if (s->srv_error)
+ s->srv_error(s, si);
+
+ DBG_TRACE_STATE("connection failed", STRM_EV_STRM_PROC|STRM_EV_SI_ST|STRM_EV_STRM_ERR, s);
+ goto end;
+ }
+
+ stream_choose_redispatch(s);
+
+ if (si->flags & SI_FL_ERR) {
+ /* The error was an asynchronous connection error, and we will
+ * likely have to retry connecting to the same server, most
+ * likely leading to the same result. To avoid this, we wait
+ * MIN(one second, connect timeout) before retrying. We don't
+ * do it when the failure happened on a reused connection
+ * though.
+ */
+
+ int delay = 1000;
+
+ if (s->be->timeout.connect && s->be->timeout.connect < delay)
+ delay = s->be->timeout.connect;
+
+ if (!si->err_type)
+ si->err_type = SI_ET_CONN_ERR;
+
+ /* only wait when we're retrying on the same server */
+ if ((si->state == SI_ST_ASS ||
+ (s->be->lbprm.algo & BE_LB_KIND) != BE_LB_KIND_RR ||
+ (s->be->srv_act <= 1)) && !(s->flags & SF_SRV_REUSED)) {
+ si->state = SI_ST_TAR;
+ si->exp = tick_add(now_ms, MS_TO_TICKS(delay));
+ }
+ si->flags &= ~SI_FL_ERR;
+ DBG_TRACE_STATE("retry a new connection", STRM_EV_STRM_PROC|STRM_EV_SI_ST, s);
+ }
+
+ end:
+ DBG_TRACE_LEAVE(STRM_EV_STRM_PROC|STRM_EV_SI_ST, s);
+}
+
+/* This function is called with (si->state == SI_ST_RDY) meaning that a
+ * connection was attempted, that the file descriptor is already allocated,
+ * and that it has succeeded. We must still check for errors and aborts.
+ * Possible output states are SI_ST_EST (established), SI_ST_CER (error),
+ * and SI_ST_DIS (abort). This only works with connection-based streams.
+ * Timeouts and errors are *not* cleared.
+ */
+void back_handle_st_rdy(struct stream *s)
+{
+ struct stream_interface *si = &s->si[1];
+ struct channel *req = &s->req;
+ struct channel *rep = &s->res;
+ struct conn_stream *srv_cs = objt_cs(si->end);
+ struct connection *conn = srv_cs ? srv_cs->conn : objt_conn(si->end);
+
+ DBG_TRACE_ENTER(STRM_EV_STRM_PROC|STRM_EV_SI_ST, s);
+ /* We know the connection at least succeeded, though it could have
+ * since met an error for any other reason. At least it didn't time out
+ * eventhough the timeout might have been reported right after success.
+ * We need to take care of various situations here :
+ * - everything might be OK. We have to switch to established.
+ * - an I/O error might have been reported after a successful transfer,
+ * which is not retryable and needs to be logged correctly, and needs
+ * established as well
+ * - SI_ST_CON implies !CF_WROTE_DATA but not conversely as we could
+ * have validated a connection with incoming data (e.g. TCP with a
+ * banner protocol), or just a successful connect() probe.
+ * - the client might have requested a connection abort, this needs to
+ * be checked before we decide to retry anything.
+ */
+
+ /* it's still possible to handle client aborts or connection retries
+ * before any data were sent.
+ */
+ if (!(req->flags & CF_WROTE_DATA)) {
+ /* client abort ? */
+ if ((rep->flags & CF_SHUTW) ||
+ ((req->flags & CF_SHUTW_NOW) &&
+ (channel_is_empty(req) || (s->be->options & PR_O_ABRT_CLOSE)))) {
+ /* give up */
+ si->flags |= SI_FL_NOLINGER;
+ si_shutw(si);
+ si->err_type |= SI_ET_CONN_ABRT;
+ if (s->srv_error)
+ s->srv_error(s, si);
+ DBG_TRACE_STATE("client abort during connection attempt", STRM_EV_STRM_PROC|STRM_EV_SI_ST|STRM_EV_STRM_ERR, s);
+ goto end;
+ }
+
+ /* retryable error ? */
+ if (si->flags & SI_FL_ERR) {
+ if (!(s->flags & SF_SRV_REUSED) && conn) {
+ conn_stop_tracking(conn);
+ conn_full_close(conn);
+ }
+
+ if (!si->err_type)
+ si->err_type = SI_ET_CONN_ERR;
+ si->state = SI_ST_CER;
+ DBG_TRACE_STATE("connection failed, retry", STRM_EV_STRM_PROC|STRM_EV_SI_ST|STRM_EV_STRM_ERR, s);
+ goto end;
+ }
+ }
+
+ /* data were sent and/or we had no error, back_establish() will
+ * now take over.
+ */
+ DBG_TRACE_STATE("connection established", STRM_EV_STRM_PROC|STRM_EV_SI_ST, s);
+ si->err_type = SI_ET_NONE;
+ si->state = SI_ST_EST;
+
+ end:
+ DBG_TRACE_LEAVE(STRM_EV_STRM_PROC|STRM_EV_SI_ST, s);
+}
+
/* sends a log message when a backend goes down, and also sets last
* change date.
*/
}
}
-/* This function is called with (si->state == SI_ST_CON) meaning that a
- * connection was attempted and that the file descriptor is already allocated.
- * We must check for timeout, error and abort. Possible output states are
- * SI_ST_CER (error), SI_ST_DIS (abort), and SI_ST_CON (no change). This only
- * works with connection-based streams. We know that there were no I/O event
- * when reaching this function. Timeouts and errors are *not* cleared.
- */
-static void sess_update_st_con_tcp(struct stream *s)
-{
- struct stream_interface *si = &s->si[1];
- struct channel *req = &s->req;
- struct channel *rep = &s->res;
- struct conn_stream *srv_cs = objt_cs(si->end);
- struct connection *conn = srv_cs ? srv_cs->conn : objt_conn(si->end);
-
- DBG_TRACE_ENTER(STRM_EV_STRM_PROC|STRM_EV_SI_ST, s);
-
- /* the client might want to abort */
- if ((rep->flags & CF_SHUTW) ||
- ((req->flags & CF_SHUTW_NOW) &&
- (channel_is_empty(req) || (s->be->options & PR_O_ABRT_CLOSE)))) {
- si->flags |= SI_FL_NOLINGER;
- si_shutw(si);
- si->err_type |= SI_ET_CONN_ABRT;
- if (s->srv_error)
- s->srv_error(s, si);
- /* Note: state = SI_ST_DIS now */
- DBG_TRACE_STATE("client abort during connection attempt", STRM_EV_STRM_PROC|STRM_EV_SI_ST|STRM_EV_STRM_ERR, s);
- }
-
- /* retryable error ? */
- else if (si->flags & (SI_FL_EXP|SI_FL_ERR)) {
- if (!(s->flags & SF_SRV_REUSED) && conn) {
- conn_stop_tracking(conn);
- conn_full_close(conn);
- }
-
- if (!si->err_type) {
- if (si->flags & SI_FL_ERR)
- si->err_type = SI_ET_CONN_ERR;
- else
- si->err_type = SI_ET_CONN_TO;
- }
-
- si->state = SI_ST_CER;
- DBG_TRACE_STATE("connection failed, retry", STRM_EV_STRM_PROC|STRM_EV_SI_ST|STRM_EV_STRM_ERR, s);
- }
-
- DBG_TRACE_LEAVE(STRM_EV_STRM_PROC|STRM_EV_SI_ST, s);
-}
-
-/* This function is called with (si->state == SI_ST_CER) meaning that a
- * previous connection attempt has failed and that the file descriptor
- * has already been released. Possible causes include asynchronous error
- * notification and time out. Possible output states are SI_ST_CLO when
- * retries are exhausted, SI_ST_TAR when a delay is wanted before a new
- * connection attempt, SI_ST_ASS when it's wise to retry on the same server,
- * and SI_ST_REQ when an immediate redispatch is wanted. The buffers are
- * marked as in error state. Timeouts and errors are cleared before retrying.
- */
-static void sess_update_st_cer(struct stream *s)
-{
- struct stream_interface *si = &s->si[1];
- struct conn_stream *cs = objt_cs(si->end);
- struct connection *conn = cs_conn(cs);
-
- DBG_TRACE_ENTER(STRM_EV_STRM_PROC|STRM_EV_SI_ST, s);
-
- si->exp = TICK_ETERNITY;
- si->flags &= ~SI_FL_EXP;
-
- /* we probably have to release last stream from the server */
- if (objt_server(s->target)) {
- health_adjust(objt_server(s->target), HANA_STATUS_L4_ERR);
-
- if (s->flags & SF_CURR_SESS) {
- s->flags &= ~SF_CURR_SESS;
- _HA_ATOMIC_SUB(&__objt_server(s->target)->cur_sess, 1);
- }
-
- if ((si->flags & SI_FL_ERR) &&
- conn && conn->err_code == CO_ER_SSL_MISMATCH_SNI) {
- /* We tried to connect to a server which is configured
- * with "verify required" and which doesn't have the
- * "verifyhost" directive. The server presented a wrong
- * certificate (a certificate for an unexpected name),
- * which implies that we have used SNI in the handshake,
- * and that the server doesn't have the associated cert
- * and presented a default one.
- *
- * This is a serious enough issue not to retry. It's
- * especially important because this wrong name might
- * either be the result of a configuration error, and
- * retrying will only hammer the server, or is caused
- * by the use of a wrong SNI value, most likely
- * provided by the client and we don't want to let the
- * client provoke retries.
- */
- si->conn_retries = 0;
- DBG_TRACE_DEVEL("Bad SSL cert, disable connection retries", STRM_EV_STRM_PROC|STRM_EV_SI_ST|STRM_EV_STRM_ERR, s);
- }
- }
-
- /* ensure that we have enough retries left */
- si->conn_retries--;
- if (si->conn_retries < 0 || !(s->be->retry_type & PR_RE_CONN_FAILED)) {
- if (!si->err_type) {
- si->err_type = SI_ET_CONN_ERR;
- }
-
- if (objt_server(s->target))
- _HA_ATOMIC_ADD(&objt_server(s->target)->counters.failed_conns, 1);
- _HA_ATOMIC_ADD(&s->be->be_counters.failed_conns, 1);
- sess_change_server(s, NULL);
- if (may_dequeue_tasks(objt_server(s->target), s->be))
- process_srv_queue(objt_server(s->target));
-
- /* shutw is enough so stop a connecting socket */
- si_shutw(si);
- s->req.flags |= CF_WRITE_ERROR;
- s->res.flags |= CF_READ_ERROR;
-
- si->state = SI_ST_CLO;
- if (s->srv_error)
- s->srv_error(s, si);
-
- DBG_TRACE_STATE("connection failed", STRM_EV_STRM_PROC|STRM_EV_SI_ST|STRM_EV_STRM_ERR, s);
- goto end;
- }
-
- stream_choose_redispatch(s);
-
- if (si->flags & SI_FL_ERR) {
- /* The error was an asynchronous connection error, and we will
- * likely have to retry connecting to the same server, most
- * likely leading to the same result. To avoid this, we wait
- * MIN(one second, connect timeout) before retrying. We don't
- * do it when the failure happened on a reused connection
- * though.
- */
-
- int delay = 1000;
-
- if (s->be->timeout.connect && s->be->timeout.connect < delay)
- delay = s->be->timeout.connect;
-
- if (!si->err_type)
- si->err_type = SI_ET_CONN_ERR;
-
- /* only wait when we're retrying on the same server */
- if ((si->state == SI_ST_ASS ||
- (s->be->lbprm.algo & BE_LB_KIND) != BE_LB_KIND_RR ||
- (s->be->srv_act <= 1)) && !(s->flags & SF_SRV_REUSED)) {
- si->state = SI_ST_TAR;
- si->exp = tick_add(now_ms, MS_TO_TICKS(delay));
- }
- si->flags &= ~SI_FL_ERR;
- DBG_TRACE_STATE("retry a new connection", STRM_EV_STRM_PROC|STRM_EV_SI_ST, s);
- }
-
- end:
- DBG_TRACE_LEAVE(STRM_EV_STRM_PROC|STRM_EV_SI_ST, s);
-}
-
-/* This function is called with (si->state == SI_ST_RDY) meaning that a
- * connection was attempted, that the file descriptor is already allocated,
- * and that it has succeeded. We must still check for errors and aborts.
- * Possible output states are SI_ST_EST (established), SI_ST_CER (error),
- * and SI_ST_DIS (abort). This only works with connection-based streams.
- * Timeouts and errors are *not* cleared.
- */
-static void sess_update_st_rdy_tcp(struct stream *s)
-{
- struct stream_interface *si = &s->si[1];
- struct channel *req = &s->req;
- struct channel *rep = &s->res;
- struct conn_stream *srv_cs = objt_cs(si->end);
- struct connection *conn = srv_cs ? srv_cs->conn : objt_conn(si->end);
-
- DBG_TRACE_ENTER(STRM_EV_STRM_PROC|STRM_EV_SI_ST, s);
- /* We know the connection at least succeeded, though it could have
- * since met an error for any other reason. At least it didn't time out
- * eventhough the timeout might have been reported right after success.
- * We need to take care of various situations here :
- * - everything might be OK. We have to switch to established.
- * - an I/O error might have been reported after a successful transfer,
- * which is not retryable and needs to be logged correctly, and needs
- * established as well
- * - SI_ST_CON implies !CF_WROTE_DATA but not conversely as we could
- * have validated a connection with incoming data (e.g. TCP with a
- * banner protocol), or just a successful connect() probe.
- * - the client might have requested a connection abort, this needs to
- * be checked before we decide to retry anything.
- */
-
- /* it's still possible to handle client aborts or connection retries
- * before any data were sent.
- */
- if (!(req->flags & CF_WROTE_DATA)) {
- /* client abort ? */
- if ((rep->flags & CF_SHUTW) ||
- ((req->flags & CF_SHUTW_NOW) &&
- (channel_is_empty(req) || (s->be->options & PR_O_ABRT_CLOSE)))) {
- /* give up */
- si->flags |= SI_FL_NOLINGER;
- si_shutw(si);
- si->err_type |= SI_ET_CONN_ABRT;
- if (s->srv_error)
- s->srv_error(s, si);
- DBG_TRACE_STATE("client abort during connection attempt", STRM_EV_STRM_PROC|STRM_EV_SI_ST|STRM_EV_STRM_ERR, s);
- goto end;
- }
-
- /* retryable error ? */
- if (si->flags & SI_FL_ERR) {
- if (!(s->flags & SF_SRV_REUSED) && conn) {
- conn_stop_tracking(conn);
- conn_full_close(conn);
- }
-
- if (!si->err_type)
- si->err_type = SI_ET_CONN_ERR;
- si->state = SI_ST_CER;
- DBG_TRACE_STATE("connection failed, retry", STRM_EV_STRM_PROC|STRM_EV_SI_ST|STRM_EV_STRM_ERR, s);
- goto end;
- }
- }
-
- /* data were sent and/or we had no error, sess_establish() will
- * now take over.
- */
- DBG_TRACE_STATE("connection established", STRM_EV_STRM_PROC|STRM_EV_SI_ST, s);
- si->err_type = SI_ET_NONE;
- si->state = SI_ST_EST;
-
- end:
- DBG_TRACE_LEAVE(STRM_EV_STRM_PROC|STRM_EV_SI_ST, s);
-}
-
/*
* This function handles the transition between the SI_ST_CON state and the
* SI_ST_EST state. It must only be called after switching from SI_ST_CON (or
* the CF_SHUTR flag, it means we were able to forward the request, and
* receive the response, before process_stream() had the opportunity to
* make the switch from SI_ST_CON to SI_ST_EST. When that happens, we want
- * to go through sess_establish() anyway, to make sure the analysers run.
+ * to go through back_establish() anyway, to make sure the analysers run.
* Timeouts are cleared. Error are reported on the channel so that analysers
* can handle them.
*/
-static void sess_establish(struct stream *s)
+static void back_establish(struct stream *s)
{
struct stream_interface *si = &s->si[1];
struct conn_stream *srv_cs = objt_cs(si->end);
DBG_TRACE_LEAVE(STRM_EV_STRM_PROC|STRM_EV_SI_ST, s);
}
-/* Check if the connection request is in such a state that it can be aborted. */
-static int check_req_may_abort(struct channel *req, struct stream *s)
-{
- return ((req->flags & (CF_READ_ERROR)) ||
- ((req->flags & (CF_SHUTW_NOW|CF_SHUTW)) && /* empty and client aborted */
- (channel_is_empty(req) || (s->be->options & PR_O_ABRT_CLOSE))));
-}
-
-/* Update back stream interface status for input states SI_ST_ASS, SI_ST_QUE,
- * SI_ST_TAR. Other input states are simply ignored.
- * Possible output states are SI_ST_CLO, SI_ST_TAR, SI_ST_ASS, SI_ST_REQ, SI_ST_CON
- * and SI_ST_EST. Flags must have previously been updated for timeouts and other
- * conditions.
- */
-static void sess_update_stream_int(struct stream *s)
-{
- struct server *srv = objt_server(s->target);
- struct stream_interface *si = &s->si[1];
- struct channel *req = &s->req;
-
- DBG_TRACE_ENTER(STRM_EV_STRM_PROC|STRM_EV_SI_ST, s);
-
- if (si->state == SI_ST_ASS) {
- /* Server assigned to connection request, we have to try to connect now */
- int conn_err;
-
- /* Before we try to initiate the connection, see if the
- * request may be aborted instead.
- */
- if (check_req_may_abort(req, s)) {
- si->err_type |= SI_ET_CONN_ABRT;
- DBG_TRACE_STATE("connection aborted", STRM_EV_STRM_PROC|STRM_EV_SI_ST|STRM_EV_STRM_ERR, s);
- goto abort_connection;
- }
-
- conn_err = connect_server(s);
- srv = objt_server(s->target);
-
- if (conn_err == SF_ERR_NONE) {
- /* state = SI_ST_CON or SI_ST_EST now */
- if (srv)
- srv_inc_sess_ctr(srv);
- if (srv)
- srv_set_sess_last(srv);
- DBG_TRACE_STATE("connection attempt", STRM_EV_STRM_PROC|STRM_EV_SI_ST, s);
- goto end;
- }
-
- /* We have received a synchronous error. We might have to
- * abort, retry immediately or redispatch.
- */
- if (conn_err == SF_ERR_INTERNAL) {
- if (!si->err_type) {
- si->err_type = SI_ET_CONN_OTHER;
- }
-
- if (srv)
- srv_inc_sess_ctr(srv);
- if (srv)
- srv_set_sess_last(srv);
- if (srv)
- _HA_ATOMIC_ADD(&srv->counters.failed_conns, 1);
- _HA_ATOMIC_ADD(&s->be->be_counters.failed_conns, 1);
-
- /* release other streams waiting for this server */
- sess_change_server(s, NULL);
- if (may_dequeue_tasks(srv, s->be))
- process_srv_queue(srv);
-
- /* Failed and not retryable. */
- si_shutr(si);
- si_shutw(si);
- req->flags |= CF_WRITE_ERROR;
-
- s->logs.t_queue = tv_ms_elapsed(&s->logs.tv_accept, &now);
-
- /* we may need to know the position in the queue for logging */
- pendconn_cond_unlink(s->pend_pos);
-
- /* no stream was ever accounted for this server */
- si->state = SI_ST_CLO;
- if (s->srv_error)
- s->srv_error(s, si);
- DBG_TRACE_STATE("internal error during connection", STRM_EV_STRM_PROC|STRM_EV_SI_ST|STRM_EV_STRM_ERR, s);
- goto end;
- }
-
- /* We are facing a retryable error, but we don't want to run a
- * turn-around now, as the problem is likely a source port
- * allocation problem, so we want to retry now.
- */
- si->state = SI_ST_CER;
- si->flags &= ~SI_FL_ERR;
- sess_update_st_cer(s);
-
- DBG_TRACE_STATE("connection error, retry", STRM_EV_STRM_PROC|STRM_EV_SI_ST|STRM_EV_STRM_ERR, s);
- /* now si->state is one of SI_ST_CLO, SI_ST_TAR, SI_ST_ASS, SI_ST_REQ */
- }
- else if (si->state == SI_ST_QUE) {
- /* connection request was queued, check for any update */
- if (!pendconn_dequeue(s)) {
- /* The connection is not in the queue anymore. Either
- * we have a server connection slot available and we
- * go directly to the assigned state, or we need to
- * load-balance first and go to the INI state.
- */
- si->exp = TICK_ETERNITY;
- if (unlikely(!(s->flags & SF_ASSIGNED)))
- si->state = SI_ST_REQ;
- else {
- s->logs.t_queue = tv_ms_elapsed(&s->logs.tv_accept, &now);
- si->state = SI_ST_ASS;
- }
- DBG_TRACE_STATE("dequeue connection request", STRM_EV_STRM_PROC|STRM_EV_SI_ST, s);
- goto end;
- }
-
- /* Connection request still in queue... */
- if (si->flags & SI_FL_EXP) {
- /* ... and timeout expired */
- si->exp = TICK_ETERNITY;
- si->flags &= ~SI_FL_EXP;
- s->logs.t_queue = tv_ms_elapsed(&s->logs.tv_accept, &now);
-
- /* we may need to know the position in the queue for logging */
- pendconn_cond_unlink(s->pend_pos);
-
- if (srv)
- _HA_ATOMIC_ADD(&srv->counters.failed_conns, 1);
- _HA_ATOMIC_ADD(&s->be->be_counters.failed_conns, 1);
- si_shutr(si);
- si_shutw(si);
- req->flags |= CF_WRITE_TIMEOUT;
- if (!si->err_type)
- si->err_type = SI_ET_QUEUE_TO;
- si->state = SI_ST_CLO;
- if (s->srv_error)
- s->srv_error(s, si);
- DBG_TRACE_STATE("connection request still queued", STRM_EV_STRM_PROC|STRM_EV_SI_ST, s);
- goto end;
- }
-
- /* Connection remains in queue, check if we have to abort it */
- if (check_req_may_abort(req, s)) {
- s->logs.t_queue = tv_ms_elapsed(&s->logs.tv_accept, &now);
-
- /* we may need to know the position in the queue for logging */
- pendconn_cond_unlink(s->pend_pos);
-
- si->err_type |= SI_ET_QUEUE_ABRT;
- DBG_TRACE_STATE("abort queued connection request", STRM_EV_STRM_PROC|STRM_EV_SI_ST|STRM_EV_STRM_ERR, s);
- goto abort_connection;
- }
-
- /* Nothing changed */
- }
- else if (si->state == SI_ST_TAR) {
- /* Connection request might be aborted */
- if (check_req_may_abort(req, s)) {
- si->err_type |= SI_ET_CONN_ABRT;
- DBG_TRACE_STATE("connection aborted", STRM_EV_STRM_PROC|STRM_EV_SI_ST|STRM_EV_STRM_ERR, s);
- goto abort_connection;
- }
-
- if (!(si->flags & SI_FL_EXP))
- return; /* still in turn-around */
-
- si->flags &= ~SI_FL_EXP;
- si->exp = TICK_ETERNITY;
-
- /* we keep trying on the same server as long as the stream is
- * marked "assigned".
- * FIXME: Should we force a redispatch attempt when the server is down ?
- */
- if (s->flags & SF_ASSIGNED)
- si->state = SI_ST_ASS;
- else
- si->state = SI_ST_REQ;
-
- DBG_TRACE_STATE("retry connection now", STRM_EV_STRM_PROC|STRM_EV_SI_ST, s);
- }
-
- end:
- DBG_TRACE_LEAVE(STRM_EV_STRM_PROC|STRM_EV_SI_ST, s);
- return;
-
-abort_connection:
- /* give up */
- si->exp = TICK_ETERNITY;
- si->flags &= ~SI_FL_EXP;
- si_shutr(si);
- si_shutw(si);
- si->state = SI_ST_CLO;
- if (s->srv_error)
- s->srv_error(s, si);
- DBG_TRACE_DEVEL("leaving on error", STRM_EV_STRM_PROC|STRM_EV_SI_ST|STRM_EV_STRM_ERR, s);
- return;
-}
-
/* Set correct stream termination flags in case no analyser has done it. It
* also counts a failed request if the server state has not reached the request
* stage.
}
}
-/* This function initiates a server connection request on a stream interface
- * already in SI_ST_REQ state. Upon success, the state goes to SI_ST_ASS for
- * a real connection to a server, indicating that a server has been assigned,
- * or SI_ST_EST for a successful connection to an applet. It may also return
- * SI_ST_QUE, or SI_ST_CLO upon error.
- */
-static void sess_prepare_conn_req(struct stream *s)
-{
- struct stream_interface *si = &s->si[1];
-
- if (si->state != SI_ST_REQ)
- return;
-
- DBG_TRACE_ENTER(STRM_EV_STRM_PROC|STRM_EV_SI_ST, s);
-
- if (unlikely(obj_type(s->target) == OBJ_TYPE_APPLET)) {
- /* the applet directly goes to the EST state */
- struct appctx *appctx = objt_appctx(si->end);
-
- if (!appctx || appctx->applet != __objt_applet(s->target))
- appctx = si_register_handler(si, objt_applet(s->target));
-
- if (!appctx) {
- /* No more memory, let's immediately abort. Force the
- * error code to ignore the ERR_LOCAL which is not a
- * real error.
- */
- s->flags &= ~(SF_ERR_MASK | SF_FINST_MASK);
-
- si_shutr(si);
- si_shutw(si);
- s->req.flags |= CF_WRITE_ERROR;
- si->err_type = SI_ET_CONN_RES;
- si->state = SI_ST_CLO;
- if (s->srv_error)
- s->srv_error(s, si);
- DBG_TRACE_STATE("failed to register applet", STRM_EV_STRM_PROC|STRM_EV_SI_ST|STRM_EV_STRM_ERR, s);
- goto end;
- }
-
- if (tv_iszero(&s->logs.tv_request))
- s->logs.tv_request = now;
- s->logs.t_queue = tv_ms_elapsed(&s->logs.tv_accept, &now);
- si->state = SI_ST_EST;
- si->err_type = SI_ET_NONE;
- be_set_sess_last(s->be);
-
- DBG_TRACE_STATE("applet registered", STRM_EV_STRM_PROC|STRM_EV_SI_ST, s);
- /* let sess_establish() finish the job */
- goto end;
- }
-
- /* Try to assign a server */
- if (srv_redispatch_connect(s) != 0) {
- /* We did not get a server. Either we queued the
- * connection request, or we encountered an error.
- */
- if (si->state == SI_ST_QUE) {
- DBG_TRACE_STATE("connection request queued", STRM_EV_STRM_PROC|STRM_EV_SI_ST, s);
- goto end;
- }
-
- /* we did not get any server, let's check the cause */
- si_shutr(si);
- si_shutw(si);
- s->req.flags |= CF_WRITE_ERROR;
- if (!si->err_type)
- si->err_type = SI_ET_CONN_OTHER;
- si->state = SI_ST_CLO;
- if (s->srv_error)
- s->srv_error(s, si);
- DBG_TRACE_STATE("connection request failed", STRM_EV_STRM_PROC|STRM_EV_SI_ST|STRM_EV_STRM_ERR, s);
- goto end;
- }
-
- /* The server is assigned */
- s->logs.t_queue = tv_ms_elapsed(&s->logs.tv_accept, &now);
- si->state = SI_ST_ASS;
- be_set_sess_last(s->be);
- DBG_TRACE_STATE("connection request assigned to a server", STRM_EV_STRM_PROC|STRM_EV_SI_ST, s);
-
- end:
- DBG_TRACE_LEAVE(STRM_EV_STRM_PROC|STRM_EV_SI_ST, s);
-}
-
/* This function parses the use-service action ruleset. It executes
* the associated ACL and set an applet as a stream or txn final node.
* it returns ACT_RET_ERR if an error occurs, the proxy left in
* maybe it succeeded, maybe it failed, maybe we timed out, ...
*/
if (si_b->state == SI_ST_RDY)
- sess_update_st_rdy_tcp(s);
+ back_handle_st_rdy(s);
else if (si_b->state == SI_ST_CON)
- sess_update_st_con_tcp(s);
+ back_handle_st_con(s);
if (si_b->state == SI_ST_CER)
- sess_update_st_cer(s);
+ back_handle_st_cer(s);
else if (si_b->state == SI_ST_EST)
- sess_establish(s);
+ back_establish(s);
/* state is now one of SI_ST_CON (still in progress), SI_ST_EST
* (established), SI_ST_DIS (abort), SI_ST_CLO (last error),
* to give a chance to step 2 to perform a redirect if needed.
*/
if (si_b->state != SI_ST_REQ)
- sess_update_stream_int(s);
+ back_try_conn_req(s);
if (si_b->state == SI_ST_REQ)
- sess_prepare_conn_req(s);
+ back_handle_st_req(s);
/* applets directly go to the ESTABLISHED state. Similarly,
* servers experience the same fate when their connection
* is reused.
*/
if (unlikely(si_b->state == SI_ST_EST))
- sess_establish(s);
+ back_establish(s);
srv = objt_server(s->target);
if (si_b->state == SI_ST_ASS && srv && srv->rdr_len && (s->flags & SF_REDIRECTABLE))