[MAJOR] convert all expiration timers from timeval to ticks

This is the first attempt at moving all internal parts from
using struct timeval to integer ticks. Those provides simpler
and faster code due to simplified operations, and this change
also saved about 64 bytes per session.

A new header file has been added : include/common/ticks.h.

It is possible that some functions should finally not be inlined
because they're used quite a lot (eg: tick_first, tick_add_ifset
and tick_is_expired). More measurements are required in order to
decide whether this is interesting or not.

Some function and variable names are still subject to change for
a better overall logics.
This commit is contained in:
Willy Tarreau 2008-07-07 00:09:58 +02:00
parent 75875a7c8c
commit 0c303eec87
32 changed files with 434 additions and 375 deletions

View File

@ -16,7 +16,7 @@
typedef struct appsessions { typedef struct appsessions {
char *sessid; char *sessid;
char *serverid; char *serverid;
struct timeval expire; /* next expiration time for this application session */ int expire; /* next expiration time for this application session (in tick) */
unsigned long int request_count; unsigned long int request_count;
struct list hash_list; struct list hash_list;
} appsess; } appsess;
@ -38,7 +38,7 @@ int match_str(const void *key1, const void *key2);
/* Callback for destroy */ /* Callback for destroy */
void destroy(appsess *data); void destroy(appsess *data);
void appsession_refresh(struct task *t, struct timeval *next); void appsession_refresh(struct task *t, int *next);
int appsession_task_init(void); int appsession_task_init(void);
int appsession_init(void); int appsession_init(void);
void appsession_cleanup(void); void appsession_cleanup(void);

118
include/common/ticks.h Normal file
View File

@ -0,0 +1,118 @@
/*
include/common/ticks.h
Functions and macros for manipulation of expiration timers
Copyright (C) 2000-2008 Willy Tarreau - w@1wt.eu
This library is free software; you can redistribute it and/or
modify it under the terms of the GNU Lesser General Public
License as published by the Free Software Foundation, version 2.1
exclusively.
This library is distributed in the hope that it will be useful,
but WITHOUT ANY WARRANTY; without even the implied warranty of
MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the GNU
Lesser General Public License for more details.
You should have received a copy of the GNU Lesser General Public
License along with this library; if not, write to the Free Software
Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301 USA
*/
/*
* Using a mix of milliseconds and timeval for internal timers is expensive and
* overkill, because we don't need such a precision to compute timeouts.
* So we're converting them to "ticks". Right now, one tick equals one
* millisecond, but that might change in the future. Ticks are stored as 32bit
* values, and sorted in four 30bit-wide rotating arrays, which means that any
* timer may be 2^30 ms in the future, or 12.4 days. The ticks are designed to
* wrap after they pass 2^32. That means that we cannot directly compare them,
* but we can check the sign of their difference.
*
* We must both support absolute dates (well in fact, dates relative to now+/-
* 12 days), and intervals (for timeouts). Both types need an "eternity" magic
* value. For optimal code generation, we'll use zero as the magic value
* indicating that an expiration timer or a timeout is not set. We have to
* check that we don't return this value when adding timeouts to <now>. If a
* computation returns 0, we must increase it to 1 (which will push the timeout
* 1 ms further).
*/
#ifndef _COMMON_TICKS_H
#define _COMMON_TICKS_H
#include <common/config.h>
#include <common/standard.h>
#define TICK_ETERNITY 0
/* right now, ticks are milliseconds. Both negative ms and negative ticks
* indicate eternity.
*/
#define MS_TO_TICKS(ms) (ms)
#define TICKS_TO_MS(tk) (tk)
/* return 1 if tick is set, otherwise 0 */
static inline int tick_isset(int expire)
{
return expire != 0;
}
/* Add <timeout> to <now>, and return the resulting expiration date.
* <timeout> will not be checked for null values.
*/
static inline int tick_add(int now, int timeout)
{
now += timeout;
if (unlikely(!now))
now++; /* unfortunate value */
return now;
}
/* add <timeout> to <now> if it is set, otherwise set it to eternity.
* Return the resulting expiration date.
*/
static inline int tick_add_ifset(int now, int timeout)
{
if (!timeout)
return TICK_ETERNITY;
return tick_add(now, timeout);
}
/* return 1 if timer <timer> is expired at date <now>, otherwise zero */
static inline int tick_is_expired(int timer, int now)
{
if (!tick_isset(timer))
return 0;
return (timer - now) <= 0;
}
/* return the first one of the two timers, both of which may be infinite */
static inline int tick_first(int t1, int t2)
{
if (!tick_isset(t1))
return t2;
if (!tick_isset(t2))
return t1;
if ((t1 - t2) <= 0)
return t1;
else
return t2;
}
/* return the number of ticks remaining from <now> to <exp>, or zero if expired */
static inline int tick_remain(int now, int exp)
{
if (tick_is_expired(exp, now))
return 0;
return exp - now;
}
#endif /* _COMMON_TICKS_H */
/*
* Local variables:
* c-indent-level: 8
* c-basic-offset: 8
* End:
*/

View File

@ -28,6 +28,7 @@
#include <common/config.h> #include <common/config.h>
#include <common/memory.h> #include <common/memory.h>
#include <common/ticks.h>
#include <common/time.h> #include <common/time.h>
#include <types/buffers.h> #include <types/buffers.h>
@ -68,14 +69,14 @@ static inline void buffer_flush(struct buffer *buf)
/* marks the buffer as "shutdown pending" for reads and cancels the timeout */ /* marks the buffer as "shutdown pending" for reads and cancels the timeout */
static inline void buffer_shutr(struct buffer *buf) static inline void buffer_shutr(struct buffer *buf)
{ {
tv_eternity(&buf->rex); buf->rex = TICK_ETERNITY;
buf->flags |= BF_SHUTR_PENDING; buf->flags |= BF_SHUTR_PENDING;
} }
/* marks the buffer as "shutdown pending" for writes and cancels the timeout */ /* marks the buffer as "shutdown pending" for writes and cancels the timeout */
static inline void buffer_shutw(struct buffer *buf) static inline void buffer_shutw(struct buffer *buf)
{ {
tv_eternity(&buf->wex); buf->wex = TICK_ETERNITY;
buf->flags |= BF_SHUTW_PENDING; buf->flags |= BF_SHUTW_PENDING;
} }

View File

@ -58,7 +58,7 @@ extern const char http_is_ver_token[256];
#define HTTP_IS_VER_TOKEN(x) (http_is_ver_token[(unsigned char)(x)]) #define HTTP_IS_VER_TOKEN(x) (http_is_ver_token[(unsigned char)(x)])
int event_accept(int fd); int event_accept(int fd);
void process_session(struct task *t, struct timeval *next); void process_session(struct task *t, int *next);
int process_cli(struct session *t); int process_cli(struct session *t);
int process_srv(struct session *t); int process_srv(struct session *t);

View File

@ -2,7 +2,7 @@
include/proto/proto_uxst.h include/proto/proto_uxst.h
This file contains UNIX-stream socket protocol definitions. This file contains UNIX-stream socket protocol definitions.
Copyright (C) 2000-2007 Willy Tarreau - w@1wt.eu Copyright (C) 2000-2008 Willy Tarreau - w@1wt.eu
This library is free software; you can redistribute it and/or This library is free software; you can redistribute it and/or
modify it under the terms of the GNU Lesser General Public modify it under the terms of the GNU Lesser General Public
@ -28,7 +28,7 @@
int uxst_event_accept(int fd); int uxst_event_accept(int fd);
void uxst_add_listener(struct listener *listener); void uxst_add_listener(struct listener *listener);
void process_uxst_stats(struct task *t, struct timeval *next); void process_uxst_stats(struct task *t, int *next);
#endif /* _PROTO_PROTO_UXST_H */ #endif /* _PROTO_PROTO_UXST_H */

View File

@ -23,11 +23,12 @@
#define _PROTO_PROXY_H #define _PROTO_PROXY_H
#include <common/config.h> #include <common/config.h>
#include <common/ticks.h>
#include <common/time.h> #include <common/time.h>
#include <types/proxy.h> #include <types/proxy.h>
int start_proxies(int verbose); int start_proxies(int verbose);
void maintain_proxies(struct timeval *next); void maintain_proxies(int *next);
void soft_stop(void); void soft_stop(void);
void pause_proxy(struct proxy *p); void pause_proxy(struct proxy *p);
void pause_proxies(void); void pause_proxies(void);
@ -52,14 +53,14 @@ static inline const char *proxy_type_str(struct proxy *proxy)
/* this function initializes all timeouts for proxy p */ /* this function initializes all timeouts for proxy p */
static inline void proxy_reset_timeouts(struct proxy *proxy) static inline void proxy_reset_timeouts(struct proxy *proxy)
{ {
tv_eternity(&proxy->timeout.client); proxy->timeout.client = TICK_ETERNITY;
tv_eternity(&proxy->timeout.tarpit); proxy->timeout.tarpit = TICK_ETERNITY;
tv_eternity(&proxy->timeout.queue); proxy->timeout.queue = TICK_ETERNITY;
tv_eternity(&proxy->timeout.connect); proxy->timeout.connect = TICK_ETERNITY;
tv_eternity(&proxy->timeout.server); proxy->timeout.server = TICK_ETERNITY;
tv_eternity(&proxy->timeout.appsession); proxy->timeout.appsession = TICK_ETERNITY;
tv_eternity(&proxy->timeout.httpreq); proxy->timeout.httpreq = TICK_ETERNITY;
tv_eternity(&proxy->timeout.check); proxy->timeout.check = TICK_ETERNITY;
} }
#endif /* _PROTO_PROXY_H */ #endif /* _PROTO_PROXY_H */

View File

@ -123,13 +123,13 @@ struct task *task_queue(struct task *task);
* - return the date of next event in <next> or eternity. * - return the date of next event in <next> or eternity.
*/ */
void process_runnable_tasks(struct timeval *next); void process_runnable_tasks(int *next);
/* /*
* Extract all expired timers from the timer queue, and wakes up all * Extract all expired timers from the timer queue, and wakes up all
* associated tasks. Returns the date of next event (or eternity). * associated tasks. Returns the date of next event (or eternity).
*/ */
void wake_expired_tasks(struct timeval *next); void wake_expired_tasks(int *next);
#endif /* _PROTO_TASK_H */ #endif /* _PROTO_TASK_H */

View File

@ -2,7 +2,7 @@
include/types/buffers.h include/types/buffers.h
Buffer management definitions, macros and inline functions. Buffer management definitions, macros and inline functions.
Copyright (C) 2000-2007 Willy Tarreau - w@1wt.eu Copyright (C) 2000-2008 Willy Tarreau - w@1wt.eu
This library is free software; you can redistribute it and/or This library is free software; you can redistribute it and/or
modify it under the terms of the GNU Lesser General Public modify it under the terms of the GNU Lesser General Public
@ -25,9 +25,6 @@
#include <common/config.h> #include <common/config.h>
#include <common/memory.h> #include <common/memory.h>
#include <sys/time.h>
#include <sys/types.h>
/* The BF_* macros designate Buffer Flags, which may be ORed in the bit field /* The BF_* macros designate Buffer Flags, which may be ORed in the bit field
* member 'flags' in struct buffer. * member 'flags' in struct buffer.
*/ */
@ -65,12 +62,12 @@ struct chunk {
struct buffer { struct buffer {
unsigned int flags; /* BF_* */ unsigned int flags; /* BF_* */
struct timeval rex; /* expiration date for a read */ int rex; /* expiration date for a read, in ticks */
struct timeval wex; /* expiration date for a write */ int wex; /* expiration date for a write, in ticks */
struct timeval cex; /* expiration date for a connect */ int cex; /* expiration date for a connect, in ticks */
struct timeval rto; /* read timeout */ int rto; /* read timeout, in ticks */
struct timeval wto; /* write timeout */ int wto; /* write timeout, in ticks */
struct timeval cto; /* connect timeout */ int cto; /* connect timeout, in ticks */
unsigned int l; /* data length */ unsigned int l; /* data length */
char *r, *w, *lr; /* read ptr, write ptr, last read */ char *r, *w, *lr; /* read ptr, write ptr, last read */
char *rlim; /* read limit, used for header rewriting */ char *rlim; /* read limit, used for header rewriting */

View File

@ -2,7 +2,7 @@
include/types/fd.h include/types/fd.h
File descriptors states. File descriptors states.
Copyright (C) 2000-2007 Willy Tarreau - w@1wt.eu Copyright (C) 2000-2008 Willy Tarreau - w@1wt.eu
This library is free software; you can redistribute it and/or This library is free software; you can redistribute it and/or
modify it under the terms of the GNU Lesser General Public modify it under the terms of the GNU Lesser General Public
@ -100,7 +100,7 @@ struct poller {
int REGPRM2 (*cond_c)(const int fd, int dir); /* clear polling on <fd> for <dir> if set */ int REGPRM2 (*cond_c)(const int fd, int dir); /* clear polling on <fd> for <dir> if set */
void REGPRM1 (*rem)(const int fd); /* remove any polling on <fd> */ void REGPRM1 (*rem)(const int fd); /* remove any polling on <fd> */
void REGPRM1 (*clo)(const int fd); /* mark <fd> as closed */ void REGPRM1 (*clo)(const int fd); /* mark <fd> as closed */
void REGPRM2 (*poll)(struct poller *p, struct timeval *exp); /* the poller itself */ void REGPRM2 (*poll)(struct poller *p, int exp); /* the poller itself */
int REGPRM1 (*init)(struct poller *p); /* poller initialization */ int REGPRM1 (*init)(struct poller *p); /* poller initialization */
void REGPRM1 (*term)(struct poller *p); /* termination of this poller */ void REGPRM1 (*term)(struct poller *p); /* termination of this poller */
int REGPRM1 (*test)(struct poller *p); /* pre-init check of the poller */ int REGPRM1 (*test)(struct poller *p); /* pre-init check of the poller */

View File

@ -66,7 +66,7 @@ struct global {
int maxaccept; /* max number of consecutive accept() */ int maxaccept; /* max number of consecutive accept() */
} tune; } tune;
struct listener stats_sock; /* unix socket listener for statistics */ struct listener stats_sock; /* unix socket listener for statistics */
struct timeval stats_timeout; int stats_timeout; /* in ticks */
}; };
extern struct global global; extern struct global global;

View File

@ -2,7 +2,7 @@
include/types/proto_http.h include/types/proto_http.h
This file contains HTTP protocol definitions. This file contains HTTP protocol definitions.
Copyright (C) 2000-2006 Willy Tarreau - w@1wt.eu Copyright (C) 2000-2008 Willy Tarreau - w@1wt.eu
This library is free software; you can redistribute it and/or This library is free software; you can redistribute it and/or
modify it under the terms of the GNU Lesser General Public modify it under the terms of the GNU Lesser General Public
@ -240,7 +240,7 @@ struct http_txn {
char *srv_cookie; /* cookie presented by the server, in capture mode */ char *srv_cookie; /* cookie presented by the server, in capture mode */
int status; /* HTTP status from the server, negative if from proxy */ int status; /* HTTP status from the server, negative if from proxy */
unsigned int flags; /* transaction flags */ unsigned int flags; /* transaction flags */
struct timeval exp; /* expiration date for the transaction (generally a request) */ int exp; /* expiration date for the transaction (generally a request), int ticks */
}; };
/* This structure is used by http_find_header() to return values of headers. /* This structure is used by http_find_header() to return values of headers.

View File

@ -83,8 +83,8 @@ struct listener {
struct listener *next; /* next address for the same proxy, or NULL */ struct listener *next; /* next address for the same proxy, or NULL */
struct list proto_list; /* list in the protocol header */ struct list proto_list; /* list in the protocol header */
int (*accept)(int fd); /* accept() function passed to fdtab[] */ int (*accept)(int fd); /* accept() function passed to fdtab[] */
void (*handler)(struct task *t, struct timeval *next); /* protocol handler */ void (*handler)(struct task *t, int *next); /* protocol handler */
struct timeval *timeout; /* pointer to client-side timeout */ int *timeout; /* pointer to client-side timeout */
void *private; /* any private data which may be used by accept() */ void *private; /* any private data which may be used by accept() */
union { /* protocol-dependant access restrictions */ union { /* protocol-dependant access restrictions */
struct { /* UNIX socket permissions */ struct { /* UNIX socket permissions */

View File

@ -182,14 +182,14 @@ struct proxy {
int monitor_uri_len; /* length of the string above. 0 if unused */ int monitor_uri_len; /* length of the string above. 0 if unused */
struct list mon_fail_cond; /* list of conditions to fail monitoring requests (chained) */ struct list mon_fail_cond; /* list of conditions to fail monitoring requests (chained) */
struct { /* WARNING! check proxy_reset_timeouts() in proxy.h !!! */ struct { /* WARNING! check proxy_reset_timeouts() in proxy.h !!! */
struct timeval client; /* client I/O timeout (in milliseconds) */ int client; /* client I/O timeout (in ticks) */
struct timeval tarpit; /* tarpit timeout, defaults to connect if unspecified */ int tarpit; /* tarpit timeout, defaults to connect if unspecified */
struct timeval queue; /* queue timeout, defaults to connect if unspecified */ int queue; /* queue timeout, defaults to connect if unspecified */
struct timeval connect; /* connect timeout (in milliseconds) */ int connect; /* connect timeout (in ticks) */
struct timeval server; /* server I/O timeout (in milliseconds) */ int server; /* server I/O timeout (in ticks) */
struct timeval appsession; /* appsession cookie expiration */ int appsession; /* appsession cookie expiration */
struct timeval httpreq; /* maximum time for complete HTTP request */ int httpreq; /* maximum time for complete HTTP request */
struct timeval check; /* maximum time for complete check */ int check; /* maximum time for complete check */
} timeout; } timeout;
char *id; /* proxy id */ char *id; /* proxy id */
struct list pendconns; /* pending connections with no server assigned yet */ struct list pendconns; /* pending connections with no server assigned yet */
@ -224,7 +224,7 @@ struct proxy {
signed char logfac1, logfac2; /* log facility for both servers. -1 = disabled */ signed char logfac1, logfac2; /* log facility for both servers. -1 = disabled */
int loglev1, loglev2; /* log level for each server, 7 by default */ int loglev1, loglev2; /* log level for each server, 7 by default */
int to_log; /* things to be logged (LW_*) */ int to_log; /* things to be logged (LW_*) */
struct timeval stop_time; /* date to stop listening, when stopping != 0 */ int stop_time; /* date to stop listening, when stopping != 0 (int ticks) */
int nb_reqadd, nb_rspadd; int nb_reqadd, nb_rspadd;
struct hdr_exp *req_exp; /* regular expressions for request headers */ struct hdr_exp *req_exp; /* regular expressions for request headers */
struct hdr_exp *rsp_exp; /* regular expressions for response headers */ struct hdr_exp *rsp_exp; /* regular expressions for response headers */

View File

@ -36,8 +36,8 @@
struct task { struct task {
struct eb32_node eb; /* ebtree node used to hold the task in the wait queue */ struct eb32_node eb; /* ebtree node used to hold the task in the wait queue */
int state; /* task state : IDLE or RUNNING */ int state; /* task state : IDLE or RUNNING */
struct timeval expire; /* next expiration time for this task, use only for fast sorting */ unsigned int expire; /* next expiration time for this task */
void (*process)(struct task *t, struct timeval *next); /* the function which processes the task */ void (*process)(struct task *t, int *next); /* the function which processes the task */
void *context; /* the task's context */ void *context; /* the task's context */
int nice; /* the task's current nice value from -1024 to +1024 */ int nice; /* the task's current nice value from -1024 to +1024 */
}; };

View File

@ -18,6 +18,7 @@
#include <common/config.h> #include <common/config.h>
#include <common/memory.h> #include <common/memory.h>
#include <common/sessionhash.h> #include <common/sessionhash.h>
#include <common/ticks.h>
#include <common/time.h> #include <common/time.h>
#include <types/buffers.h> #include <types/buffers.h>
@ -93,7 +94,7 @@ int appsession_task_init(void)
task_init(appsess_refresh); task_init(appsess_refresh);
appsess_refresh->context = NULL; appsess_refresh->context = NULL;
tv_ms_add(&appsess_refresh->expire, &now, TBLCHKINT); appsess_refresh->expire = tick_add(now_ms, MS_TO_TICKS(TBLCHKINT));
appsess_refresh->process = appsession_refresh; appsess_refresh->process = appsession_refresh;
task_queue(appsess_refresh); task_queue(appsess_refresh);
initialized ++; initialized ++;
@ -101,7 +102,7 @@ int appsession_task_init(void)
return 0; return 0;
} }
void appsession_refresh(struct task *t, struct timeval *next) void appsession_refresh(struct task *t, int *next)
{ {
struct proxy *p = proxy; struct proxy *p = proxy;
struct appsession_hash *htbl; struct appsession_hash *htbl;
@ -112,7 +113,7 @@ void appsession_refresh(struct task *t, struct timeval *next)
if (p->appsession_name != NULL) { if (p->appsession_name != NULL) {
htbl = &p->htbl_proxy; htbl = &p->htbl_proxy;
as_hash_for_each_entry_safe(i, element, back, &p->htbl_proxy, hash_list) { as_hash_for_each_entry_safe(i, element, back, &p->htbl_proxy, hash_list) {
if (tv_isle(&element->expire, &now)) { if (tick_is_expired(element->expire, now_ms)) {
if ((global.mode & MODE_DEBUG) && if ((global.mode & MODE_DEBUG) &&
(!(global.mode & MODE_QUIET) || (global.mode & MODE_VERBOSE))) { (!(global.mode & MODE_QUIET) || (global.mode & MODE_VERBOSE))) {
int len; int len;
@ -131,7 +132,7 @@ void appsession_refresh(struct task *t, struct timeval *next)
} }
p = p->next; p = p->next;
} }
tv_ms_add(&t->expire, &now, TBLCHKINT); /* check expiration every 5 seconds */ t->expire = tick_add(now_ms, MS_TO_TICKS(TBLCHKINT)); /* check expiration every 5 seconds */
task_queue(t); task_queue(t);
*next = t->expire; *next = t->expire;
} /* end appsession_refresh */ } /* end appsession_refresh */

View File

@ -22,6 +22,7 @@
#include <common/config.h> #include <common/config.h>
#include <common/debug.h> #include <common/debug.h>
#include <common/eb32tree.h> #include <common/eb32tree.h>
#include <common/ticks.h>
#include <common/time.h> #include <common/time.h>
#include <types/acl.h> #include <types/acl.h>
@ -1831,8 +1832,7 @@ int connect_server(struct session *s)
s->be->lbprm.server_take_conn(s->srv); s->be->lbprm.server_take_conn(s->srv);
} }
if (!tv_add_ifset(&s->req->cex, &now, &s->be->timeout.connect)) s->req->cex = tick_add_ifset(now_ms, s->be->timeout.connect);
tv_eternity(&s->req->cex);
return SN_ERR_NONE; /* connection is OK */ return SN_ERR_NONE; /* connection is OK */
} }
@ -1850,7 +1850,7 @@ int srv_count_retry_down(struct session *t, int conn_err)
if (t->conn_retries < 0) { if (t->conn_retries < 0) {
/* if not retryable anymore, let's abort */ /* if not retryable anymore, let's abort */
tv_eternity(&t->req->cex); t->req->cex = TICK_ETERNITY;
srv_close_with_err(t, conn_err, SN_FINST_C, srv_close_with_err(t, conn_err, SN_FINST_C,
503, error_message(t, HTTP_ERR_503)); 503, error_message(t, HTTP_ERR_503));
if (t->srv) if (t->srv)
@ -1894,7 +1894,7 @@ int srv_retryable_connect(struct session *t)
return 1; return 1;
case SN_ERR_INTERNAL: case SN_ERR_INTERNAL:
tv_eternity(&t->req->cex); t->req->cex = TICK_ETERNITY;
srv_close_with_err(t, SN_ERR_INTERNAL, SN_FINST_C, srv_close_with_err(t, SN_ERR_INTERNAL, SN_FINST_C,
500, error_message(t, HTTP_ERR_500)); 500, error_message(t, HTTP_ERR_500));
if (t->srv) if (t->srv)
@ -1965,7 +1965,7 @@ int srv_redispatch_connect(struct session *t)
goto redispatch; goto redispatch;
} }
tv_eternity(&t->req->cex); t->req->cex = TICK_ETERNITY;
srv_close_with_err(t, SN_ERR_SRVTO, SN_FINST_Q, srv_close_with_err(t, SN_ERR_SRVTO, SN_FINST_Q,
503, error_message(t, HTTP_ERR_503)); 503, error_message(t, HTTP_ERR_503));
@ -1975,7 +1975,7 @@ int srv_redispatch_connect(struct session *t)
case SRV_STATUS_NOSRV: case SRV_STATUS_NOSRV:
/* note: it is guaranteed that t->srv == NULL here */ /* note: it is guaranteed that t->srv == NULL here */
tv_eternity(&t->req->cex); t->req->cex = TICK_ETERNITY;
srv_close_with_err(t, SN_ERR_SRVTO, SN_FINST_C, srv_close_with_err(t, SN_ERR_SRVTO, SN_FINST_C,
503, error_message(t, HTTP_ERR_503)); 503, error_message(t, HTTP_ERR_503));
@ -1983,15 +1983,14 @@ int srv_redispatch_connect(struct session *t)
return 1; return 1;
case SRV_STATUS_QUEUED: case SRV_STATUS_QUEUED:
if (!tv_add_ifset(&t->req->cex, &now, &t->be->timeout.queue)) t->req->cex = tick_add_ifset(now_ms, t->be->timeout.queue);
tv_eternity(&t->req->cex);
t->srv_state = SV_STIDLE; t->srv_state = SV_STIDLE;
/* do nothing else and do not wake any other session up */ /* do nothing else and do not wake any other session up */
return 1; return 1;
case SRV_STATUS_INTERNAL: case SRV_STATUS_INTERNAL:
default: default:
tv_eternity(&t->req->cex); t->req->cex = TICK_ETERNITY;
srv_close_with_err(t, SN_ERR_INTERNAL, SN_FINST_C, srv_close_with_err(t, SN_ERR_INTERNAL, SN_FINST_C,
500, error_message(t, HTTP_ERR_500)); 500, error_message(t, HTTP_ERR_500));
if (t->srv) if (t->srv)

View File

@ -971,10 +971,7 @@ int cfg_parse_listen(const char *file, int linenum, char **args, int inv)
file, linenum, *err, args[0]); file, linenum, *err, args[0]);
return -1; return -1;
} }
if (val > 0) curproxy->timeout.appsession = val;
__tv_from_ms(&curproxy->timeout.appsession, val);
else
tv_eternity(&curproxy->timeout.appsession);
if (appsession_hash_init(&(curproxy->htbl_proxy), destroy) == 0) { if (appsession_hash_init(&(curproxy->htbl_proxy), destroy) == 0) {
Alert("parsing [%s:%d] : out of memory.\n", file, linenum); Alert("parsing [%s:%d] : out of memory.\n", file, linenum);
@ -2994,9 +2991,9 @@ int readcfgfile(const char *file)
} }
if ((curproxy->mode == PR_MODE_TCP || curproxy->mode == PR_MODE_HTTP) && if ((curproxy->mode == PR_MODE_TCP || curproxy->mode == PR_MODE_HTTP) &&
(((curproxy->cap & PR_CAP_FE) && !tv_isset(&curproxy->timeout.client)) || (((curproxy->cap & PR_CAP_FE) && !curproxy->timeout.client) ||
((curproxy->cap & PR_CAP_BE) && (curproxy->srv) && ((curproxy->cap & PR_CAP_BE) && (curproxy->srv) &&
(!tv_isset(&curproxy->timeout.connect) || !tv_isset(&curproxy->timeout.server))))) { (!curproxy->timeout.connect || !curproxy->timeout.server)))) {
Warning("parsing %s : missing timeouts for %s '%s'.\n" Warning("parsing %s : missing timeouts for %s '%s'.\n"
" | While not properly invalid, you will certainly encounter various problems\n" " | While not properly invalid, you will certainly encounter various problems\n"
" | with such a configuration. To fix this, please ensure that all following\n" " | with such a configuration. To fix this, please ensure that all following\n"
@ -3009,29 +3006,29 @@ int readcfgfile(const char *file)
* parameters have been set or must be copied from contimeouts. * parameters have been set or must be copied from contimeouts.
*/ */
if (curproxy != &defproxy) { if (curproxy != &defproxy) {
if (!tv_isset(&curproxy->timeout.tarpit) || if (!curproxy->timeout.tarpit ||
__tv_iseq(&curproxy->timeout.tarpit, &defproxy.timeout.tarpit)) { curproxy->timeout.tarpit == defproxy.timeout.tarpit) {
/* tarpit timeout not set. We search in the following order: /* tarpit timeout not set. We search in the following order:
* default.tarpit, curr.connect, default.connect. * default.tarpit, curr.connect, default.connect.
*/ */
if (tv_isset(&defproxy.timeout.tarpit)) if (defproxy.timeout.tarpit)
curproxy->timeout.tarpit = defproxy.timeout.tarpit; curproxy->timeout.tarpit = defproxy.timeout.tarpit;
else if (tv_isset(&curproxy->timeout.connect)) else if (curproxy->timeout.connect)
curproxy->timeout.tarpit = curproxy->timeout.connect; curproxy->timeout.tarpit = curproxy->timeout.connect;
else if (tv_isset(&defproxy.timeout.connect)) else if (defproxy.timeout.connect)
curproxy->timeout.tarpit = defproxy.timeout.connect; curproxy->timeout.tarpit = defproxy.timeout.connect;
} }
if ((curproxy->cap & PR_CAP_BE) && if ((curproxy->cap & PR_CAP_BE) &&
(!tv_isset(&curproxy->timeout.queue) || (!curproxy->timeout.queue ||
__tv_iseq(&curproxy->timeout.queue, &defproxy.timeout.queue))) { curproxy->timeout.queue == defproxy.timeout.queue)) {
/* queue timeout not set. We search in the following order: /* queue timeout not set. We search in the following order:
* default.queue, curr.connect, default.connect. * default.queue, curr.connect, default.connect.
*/ */
if (tv_isset(&defproxy.timeout.queue)) if (defproxy.timeout.queue)
curproxy->timeout.queue = defproxy.timeout.queue; curproxy->timeout.queue = defproxy.timeout.queue;
else if (tv_isset(&curproxy->timeout.connect)) else if (curproxy->timeout.connect)
curproxy->timeout.queue = curproxy->timeout.connect; curproxy->timeout.queue = curproxy->timeout.connect;
else if (tv_isset(&defproxy.timeout.connect)) else if (defproxy.timeout.connect)
curproxy->timeout.queue = defproxy.timeout.connect; curproxy->timeout.queue = defproxy.timeout.connect;
} }
} }

View File

@ -367,9 +367,7 @@ static int event_srv_chk_w(int fd)
#endif #endif
if (ret == s->proxy->check_len) { if (ret == s->proxy->check_len) {
/* we allow up to <timeout.check> if nonzero for a responce */ /* we allow up to <timeout.check> if nonzero for a responce */
//fprintf(stderr, "event_srv_chk_w, ms=%lu\n", __tv_to_ms(&s->proxy->timeout.check)); t->expire = tick_add_ifset(now_ms, s->proxy->timeout.check);
tv_add_ifset(&t->expire, &now, &s->proxy->timeout.check);
EV_FD_SET(fd, DIR_RD); /* prepare for reading reply */ EV_FD_SET(fd, DIR_RD); /* prepare for reading reply */
goto out_nowake; goto out_nowake;
} }
@ -526,7 +524,7 @@ static int event_srv_chk_r(int fd)
* manages a server health-check. Returns * manages a server health-check. Returns
* the time the task accepts to wait, or TIME_ETERNITY for infinity. * the time the task accepts to wait, or TIME_ETERNITY for infinity.
*/ */
void process_chk(struct task *t, struct timeval *next) void process_chk(struct task *t, int *next)
{ {
__label__ new_chk, out; __label__ new_chk, out;
struct server *s = t->context; struct server *s = t->context;
@ -540,7 +538,7 @@ void process_chk(struct task *t, struct timeval *next)
fd = s->curfd; fd = s->curfd;
if (fd < 0) { /* no check currently running */ if (fd < 0) { /* no check currently running */
//fprintf(stderr, "process_chk: 2\n"); //fprintf(stderr, "process_chk: 2\n");
if (!tv_isle(&t->expire, &now)) { /* not good time yet */ if (!tick_is_expired(t->expire, now_ms)) { /* not good time yet */
task_queue(t); /* restore t to its place in the task list */ task_queue(t); /* restore t to its place in the task list */
*next = t->expire; *next = t->expire;
goto out; goto out;
@ -550,8 +548,8 @@ void process_chk(struct task *t, struct timeval *next)
* the server should not be checked. * the server should not be checked.
*/ */
if (!(s->state & SRV_CHECKED) || s->proxy->state == PR_STSTOPPED) { if (!(s->state & SRV_CHECKED) || s->proxy->state == PR_STSTOPPED) {
while (tv_isle(&t->expire, &now)) while (tick_is_expired(t->expire, now_ms))
tv_ms_add(&t->expire, &t->expire, s->inter); t->expire = tick_add(t->expire, MS_TO_TICKS(s->inter));
task_queue(t); /* restore t to its place in the task list */ task_queue(t); /* restore t to its place in the task list */
*next = t->expire; *next = t->expire;
goto out; goto out;
@ -637,8 +635,6 @@ void process_chk(struct task *t, struct timeval *next)
if (s->result == SRV_CHK_UNKNOWN) { if (s->result == SRV_CHK_UNKNOWN) {
if ((connect(fd, (struct sockaddr *)&sa, sizeof(sa)) != -1) || (errno == EINPROGRESS)) { if ((connect(fd, (struct sockaddr *)&sa, sizeof(sa)) != -1) || (errno == EINPROGRESS)) {
struct timeval tv_con;
/* OK, connection in progress or established */ /* OK, connection in progress or established */
//fprintf(stderr, "process_chk: 4\n"); //fprintf(stderr, "process_chk: 4\n");
@ -662,11 +658,11 @@ void process_chk(struct task *t, struct timeval *next)
* to establish but only when timeout.check is set * to establish but only when timeout.check is set
* as it may be to short for a full check otherwise * as it may be to short for a full check otherwise
*/ */
tv_ms_add(&t->expire, &now, s->inter); t->expire = tick_add(now_ms, MS_TO_TICKS(s->inter));
if (tv_isset(&s->proxy->timeout.check) && tv_isset(&s->proxy->timeout.connect)) { if (s->proxy->timeout.check && s->proxy->timeout.connect) {
tv_add(&tv_con, &now, &s->proxy->timeout.connect); int t_con = tick_add(now_ms, s->proxy->timeout.connect);
tv_bound(&t->expire, &tv_con); t->expire = tick_first(t->expire, t_con);
} }
task_queue(t); /* restore t to its place in the task list */ task_queue(t); /* restore t to its place in the task list */
@ -683,8 +679,8 @@ void process_chk(struct task *t, struct timeval *next)
if (s->result == SRV_CHK_UNKNOWN) { /* nothing done */ if (s->result == SRV_CHK_UNKNOWN) { /* nothing done */
//fprintf(stderr, "process_chk: 6\n"); //fprintf(stderr, "process_chk: 6\n");
while (tv_isle(&t->expire, &now)) while (tick_is_expired(t->expire, now_ms))
tv_ms_add(&t->expire, &t->expire, s->inter); t->expire = tick_add(t->expire, MS_TO_TICKS(s->inter));
goto new_chk; /* may be we should initialize a new check */ goto new_chk; /* may be we should initialize a new check */
} }
@ -701,14 +697,14 @@ void process_chk(struct task *t, struct timeval *next)
* to establish but only when timeout.check is set * to establish but only when timeout.check is set
* as it may be to short for a full check otherwise * as it may be to short for a full check otherwise
*/ */
while (tv_isle(&t->expire, &now)) { while (tick_is_expired(t->expire, now_ms)) {
struct timeval tv_con; int t_con;
tv_add(&tv_con, &t->expire, &s->proxy->timeout.connect); t_con = tick_add(t->expire, s->proxy->timeout.connect);
tv_ms_add(&t->expire, &t->expire, s->inter); t->expire = tick_add(t->expire, MS_TO_TICKS(s->inter));
if (tv_isset(&s->proxy->timeout.check)) if (s->proxy->timeout.check)
tv_bound(&t->expire, &tv_con); t->expire = tick_first(t->expire, t_con);
} }
goto new_chk; goto new_chk;
} }
@ -763,10 +759,10 @@ void process_chk(struct task *t, struct timeval *next)
rv -= (int) (2 * rv * (rand() / (RAND_MAX + 1.0))); rv -= (int) (2 * rv * (rand() / (RAND_MAX + 1.0)));
//fprintf(stderr, "process_chk(%p): (%d+/-%d%%) random=%d\n", s, srv_getinter(s), global.spread_checks, rv); //fprintf(stderr, "process_chk(%p): (%d+/-%d%%) random=%d\n", s, srv_getinter(s), global.spread_checks, rv);
} }
tv_ms_add(&t->expire, &now, srv_getinter(s) + rv); t->expire = tick_add(now_ms, MS_TO_TICKS(srv_getinter(s) + rv));
goto new_chk; goto new_chk;
} }
else if ((s->result & SRV_CHK_ERROR) || tv_isle(&t->expire, &now)) { else if ((s->result & SRV_CHK_ERROR) || tick_is_expired(t->expire, now_ms)) {
//fprintf(stderr, "process_chk: 10\n"); //fprintf(stderr, "process_chk: 10\n");
/* failure or timeout detected */ /* failure or timeout detected */
if (s->health > s->rise) { if (s->health > s->rise) {
@ -784,7 +780,7 @@ void process_chk(struct task *t, struct timeval *next)
rv -= (int) (2 * rv * (rand() / (RAND_MAX + 1.0))); rv -= (int) (2 * rv * (rand() / (RAND_MAX + 1.0)));
//fprintf(stderr, "process_chk(%p): (%d+/-%d%%) random=%d\n", s, srv_getinter(s), global.spread_checks, rv); //fprintf(stderr, "process_chk(%p): (%d+/-%d%%) random=%d\n", s, srv_getinter(s), global.spread_checks, rv);
} }
tv_ms_add(&t->expire, &now, srv_getinter(s) + rv); t->expire = tick_add(now_ms, MS_TO_TICKS(srv_getinter(s) + rv));
goto new_chk; goto new_chk;
} }
/* if result is unknown and there's no timeout, we have to wait again */ /* if result is unknown and there's no timeout, we have to wait again */
@ -855,8 +851,9 @@ int start_checks() {
t->context = s; t->context = s;
/* check this every ms */ /* check this every ms */
tv_ms_add(&t->expire, &now, t->expire = tick_add(now_ms,
((mininter && mininter >= srv_getinter(s)) ? mininter : srv_getinter(s)) * srvpos / nbchk); MS_TO_TICKS(((mininter && mininter >= srv_getinter(s)) ?
mininter : srv_getinter(s)) * srvpos / nbchk));
task_queue(t); task_queue(t);
srvpos++; srvpos++;

View File

@ -354,7 +354,7 @@ int event_accept(int fd) {
s->rep->rto = s->be->timeout.server; s->rep->rto = s->be->timeout.server;
s->rep->wto = s->fe->timeout.client; s->rep->wto = s->fe->timeout.client;
tv_eternity(&s->rep->cto); s->rep->cto = TICK_ETERNITY;
fd_insert(cfd); fd_insert(cfd);
fdtab[cfd].owner = t; fdtab[cfd].owner = t;
@ -384,28 +384,28 @@ int event_accept(int fd) {
EV_FD_SET(cfd, DIR_RD); EV_FD_SET(cfd, DIR_RD);
} }
tv_eternity(&s->req->rex); s->req->rex = TICK_ETERNITY;
tv_eternity(&s->req->wex); s->req->wex = TICK_ETERNITY;
tv_eternity(&s->req->cex); s->req->cex = TICK_ETERNITY;
tv_eternity(&s->rep->rex); s->rep->rex = TICK_ETERNITY;
tv_eternity(&s->rep->wex); s->rep->wex = TICK_ETERNITY;
tv_eternity(&s->txn.exp); s->txn.exp = TICK_ETERNITY;
tv_eternity(&t->expire); t->expire = TICK_ETERNITY;
if (tv_isset(&s->fe->timeout.client)) { if (s->fe->timeout.client) {
if (EV_FD_ISSET(cfd, DIR_RD)) { if (EV_FD_ISSET(cfd, DIR_RD)) {
tv_add(&s->req->rex, &now, &s->fe->timeout.client); s->req->rex = tick_add(now_ms, s->fe->timeout.client);
t->expire = s->req->rex; t->expire = s->req->rex;
} }
if (EV_FD_ISSET(cfd, DIR_WR)) { if (EV_FD_ISSET(cfd, DIR_WR)) {
tv_add(&s->rep->wex, &now, &s->fe->timeout.client); s->rep->wex = tick_add(now_ms, s->fe->timeout.client);
t->expire = s->rep->wex; t->expire = s->rep->wex;
} }
} }
if (s->cli_state == CL_STHEADERS && tv_isset(&s->fe->timeout.httpreq)) { if (s->cli_state == CL_STHEADERS && s->fe->timeout.httpreq) {
tv_add(&s->txn.exp, &now, &s->fe->timeout.httpreq); s->txn.exp = tick_add(now_ms, s->fe->timeout.httpreq);
tv_bound(&t->expire, &s->txn.exp); t->expire = tick_first(t->expire, s->txn.exp);
} }
if (p->mode != PR_MODE_HEALTH) if (p->mode != PR_MODE_HEALTH)

View File

@ -1,7 +1,7 @@
/* /*
* Functions dedicated to statistics output * Functions dedicated to statistics output
* *
* Copyright 2000-2007 Willy Tarreau <w@1wt.eu> * Copyright 2000-2008 Willy Tarreau <w@1wt.eu>
* *
* This program is free software; you can redistribute it and/or * This program is free software; you can redistribute it and/or
* modify it under the terms of the GNU General Public License * modify it under the terms of the GNU General Public License
@ -29,6 +29,7 @@
#include <common/memory.h> #include <common/memory.h>
#include <common/mini-clist.h> #include <common/mini-clist.h>
#include <common/standard.h> #include <common/standard.h>
#include <common/ticks.h>
#include <common/time.h> #include <common/time.h>
#include <common/uri_auth.h> #include <common/uri_auth.h>
#include <common/version.h> #include <common/version.h>
@ -139,7 +140,7 @@ int stats_parse_global(const char **args, char *err, int errlen)
snprintf(err, errlen, "a positive value is expected for 'stats timeout' in 'global section'"); snprintf(err, errlen, "a positive value is expected for 'stats timeout' in 'global section'");
return -1; return -1;
} }
__tv_from_ms(&global.stats_timeout, timeout); global.stats_timeout = MS_TO_TICKS(timeout);
} }
else if (!strcmp(args[0], "maxconn")) { else if (!strcmp(args[0], "maxconn")) {
int maxconn = atol(args[1]); int maxconn = atol(args[1]);

View File

@ -17,6 +17,7 @@
#include <common/compat.h> #include <common/compat.h>
#include <common/config.h> #include <common/config.h>
#include <common/standard.h> #include <common/standard.h>
#include <common/ticks.h>
#include <common/time.h> #include <common/time.h>
#include <common/tools.h> #include <common/tools.h>
@ -221,7 +222,7 @@ REGPRM1 static void __fd_clo(int fd)
/* /*
* epoll() poller * epoll() poller
*/ */
REGPRM2 static void _do_poll(struct poller *p, struct timeval *exp) REGPRM2 static void _do_poll(struct poller *p, int exp)
{ {
int status; int status;
int fd; int fd;
@ -234,12 +235,12 @@ REGPRM2 static void _do_poll(struct poller *p, struct timeval *exp)
/* now let's wait for events */ /* now let's wait for events */
if (run_queue) if (run_queue)
wait_time = 0; wait_time = 0;
else if (tv_iseternity(exp)) else if (!exp)
wait_time = MAX_DELAY_MS; wait_time = MAX_DELAY_MS;
else if (tv_isge(&now, exp)) else if (tick_is_expired(exp, now_ms))
wait_time = 0; wait_time = 0;
else { else {
wait_time = __tv_ms_elapsed(&now, exp) + 1; wait_time = TICKS_TO_MS(tick_remain(now_ms, exp)) + 1;
if (wait_time > MAX_DELAY_MS) if (wait_time > MAX_DELAY_MS)
wait_time = MAX_DELAY_MS; wait_time = MAX_DELAY_MS;
} }

View File

@ -23,6 +23,7 @@
#include <common/compat.h> #include <common/compat.h>
#include <common/config.h> #include <common/config.h>
#include <common/ticks.h>
#include <common/time.h> #include <common/time.h>
#include <common/tools.h> #include <common/tools.h>
@ -99,45 +100,30 @@ REGPRM1 static void __fd_clo(int fd)
/* /*
* kqueue() poller * kqueue() poller
*/ */
REGPRM2 static void _do_poll(struct poller *p, struct timeval *exp) REGPRM2 static void _do_poll(struct poller *p, int exp)
{ {
int status; int status;
int count, fd, delta_ms; int count, fd, delta_ms;
struct timespec timeout; struct timespec timeout;
if (run_queue) {
timeout.tv_sec = timeout.tv_nsec = 0;
delta_ms = 0; delta_ms = 0;
} timeout.tv_sec = 0;
else if (tv_isset(exp)) { timeout.tv_nsec = 0;
const struct timeval max_delay = {
.tv_sec = MAX_DELAY_MS / 1000,
.tv_usec = (MAX_DELAY_MS % 1000) * 1000
};
struct timeval delta;
if (tv_isge(&now, exp)) { if (!run_queue) {
delta.tv_sec = delta.tv_usec = 0; if (!exp) {
delta_ms = 0;
}
else {
tv_remain(&now, exp, &delta);
if (__tv_isgt(&delta, &max_delay)) {
delta = max_delay;
delta_ms = MAX_DELAY_MS; delta_ms = MAX_DELAY_MS;
} else { timeout.tv_sec = (MAX_DELAY_MS / 1000);
delta_ms = delta.tv_sec * 1000 + delta.tv_usec / 1000;
}
}
timeout.tv_sec = delta.tv_sec;
timeout.tv_nsec = delta.tv_usec * 1000;
}
else {
delta_ms = MAX_DELAY_MS;
timeout.tv_sec = MAX_DELAY_MS / 1000;
timeout.tv_nsec = (MAX_DELAY_MS % 1000) * 1000000; timeout.tv_nsec = (MAX_DELAY_MS % 1000) * 1000000;
} }
else if (!tick_is_expired(exp, now_ms)) {
delta_ms = TICKS_TO_MS(tick_remain(now_ms, exp)) + 1;
if (delta_ms > MAX_DELAY_MS)
delta_ms = MAX_DELAY_MS;
timeout.tv_sec = (delta_ms / 1000);
timeout.tv_nsec = (delta_ms % 1000) * 1000000;
}
}
fd = MIN(maxfd, global.tune.maxpollevents); fd = MIN(maxfd, global.tune.maxpollevents);
status = kevent(kqueue_fd, // int kq status = kevent(kqueue_fd, // int kq

View File

@ -17,6 +17,7 @@
#include <common/compat.h> #include <common/compat.h>
#include <common/config.h> #include <common/config.h>
#include <common/ticks.h>
#include <common/time.h> #include <common/time.h>
#include <types/fd.h> #include <types/fd.h>
@ -81,7 +82,7 @@ REGPRM1 static void __fd_rem(const int fd)
/* /*
* Poll() poller * Poll() poller
*/ */
REGPRM2 static void _do_poll(struct poller *p, struct timeval *exp) REGPRM2 static void _do_poll(struct poller *p, int exp)
{ {
int status; int status;
int fd, nbfd; int fd, nbfd;
@ -126,12 +127,12 @@ REGPRM2 static void _do_poll(struct poller *p, struct timeval *exp)
/* now let's wait for events */ /* now let's wait for events */
if (run_queue) if (run_queue)
wait_time = 0; wait_time = 0;
else if (tv_iseternity(exp)) else if (!exp)
wait_time = MAX_DELAY_MS; wait_time = MAX_DELAY_MS;
else if (tv_isge(&now, exp)) else if (tick_is_expired(exp, now_ms))
wait_time = 0; wait_time = 0;
else { else {
wait_time = __tv_ms_elapsed(&now, exp) + 1; wait_time = TICKS_TO_MS(tick_remain(now_ms, exp)) + 1;
if (wait_time > MAX_DELAY_MS) if (wait_time > MAX_DELAY_MS)
wait_time = MAX_DELAY_MS; wait_time = MAX_DELAY_MS;
} }

View File

@ -16,6 +16,7 @@
#include <common/compat.h> #include <common/compat.h>
#include <common/config.h> #include <common/config.h>
#include <common/ticks.h>
#include <common/time.h> #include <common/time.h>
#include <types/fd.h> #include <types/fd.h>
@ -78,12 +79,8 @@ REGPRM1 static void __fd_rem(int fd)
/* /*
* Select() poller * Select() poller
*/ */
REGPRM2 static void _do_poll(struct poller *p, struct timeval *exp) REGPRM2 static void _do_poll(struct poller *p, int exp)
{ {
const struct timeval max_delay = {
.tv_sec = MAX_DELAY_MS / 1000,
.tv_usec = (MAX_DELAY_MS % 1000) * 1000
};
int status; int status;
int fd, i; int fd, i;
struct timeval delta; struct timeval delta;
@ -92,28 +89,22 @@ REGPRM2 static void _do_poll(struct poller *p, struct timeval *exp)
int fds; int fds;
char count; char count;
/* allow select to return immediately when needed */
delta.tv_sec = delta.tv_usec = 0;
delta_ms = 0; delta_ms = 0;
delta.tv_sec = 0;
delta.tv_usec = 0;
if (!run_queue) { if (!run_queue) {
if (!tv_isset(exp)) { if (!exp) {
delta = max_delay;
delta_ms = MAX_DELAY_MS; delta_ms = MAX_DELAY_MS;
delta.tv_sec = (MAX_DELAY_MS / 1000);
delta.tv_usec = (MAX_DELAY_MS % 1000) * 1000;
} }
else if (tv_islt(&now, exp)) { else if (!tick_is_expired(exp, now_ms)) {
tv_remain(&now, exp, &delta); delta_ms = TICKS_TO_MS(tick_remain(now_ms, exp)) + SCHEDULER_RESOLUTION;
/* To avoid eventual select loops due to timer precision */ if (delta_ms > MAX_DELAY_MS)
delta.tv_usec += SCHEDULER_RESOLUTION * 1000;
if (delta.tv_usec >= 1000000) {
delta.tv_usec -= 1000000;
delta.tv_sec ++;
}
if (__tv_isge(&delta, &max_delay)) {
delta = max_delay;
delta_ms = MAX_DELAY_MS; delta_ms = MAX_DELAY_MS;
} else { delta.tv_sec = (delta_ms / 1000);
delta_ms = delta.tv_sec * 1000 + delta.tv_usec / 1000; delta.tv_usec = (delta_ms % 1000) * 1000;
}
} }
} }

View File

@ -52,6 +52,7 @@
#include <common/config.h> #include <common/config.h>
#include <common/debug.h> #include <common/debug.h>
#include <common/standard.h> #include <common/standard.h>
#include <common/ticks.h>
#include <common/time.h> #include <common/time.h>
#include <common/tools.h> #include <common/tools.h>
@ -287,7 +288,7 @@ REGPRM1 static void __fd_clo(int fd)
/* /*
* speculative epoll() poller * speculative epoll() poller
*/ */
REGPRM2 static void _do_poll(struct poller *p, struct timeval *exp) REGPRM2 static void _do_poll(struct poller *p, int exp)
{ {
static unsigned int last_skipped; static unsigned int last_skipped;
static unsigned int spec_processed; static unsigned int spec_processed;
@ -434,12 +435,12 @@ REGPRM2 static void _do_poll(struct poller *p, struct timeval *exp)
wait_time = 0; wait_time = 0;
} }
else { else {
if (tv_iseternity(exp)) if (!exp)
wait_time = MAX_DELAY_MS; wait_time = MAX_DELAY_MS;
else if (tv_isge(&now, exp)) else if (tick_is_expired(exp, now_ms))
wait_time = 0; wait_time = 0;
else { else {
wait_time = __tv_ms_elapsed(&now, exp) + 1; wait_time = TICKS_TO_MS(tick_remain(now_ms, exp)) + 1;
if (wait_time > MAX_DELAY_MS) if (wait_time > MAX_DELAY_MS)
wait_time = MAX_DELAY_MS; wait_time = MAX_DELAY_MS;
} }

View File

@ -118,7 +118,7 @@ struct global global = {
logfac2 : -1, logfac2 : -1,
loglev1 : 7, /* max syslog level : debug */ loglev1 : 7, /* max syslog level : debug */
loglev2 : 7, loglev2 : 7,
.stats_timeout = { .tv_sec = 10, .tv_usec = 0 }, /* stats timeout = 10 seconds */ .stats_timeout = MS_TO_TICKS(10000), /* stats timeout = 10 seconds */
.stats_sock = { .stats_sock = {
.timeout = &global.stats_timeout, .timeout = &global.stats_timeout,
.maxconn = 10, /* 10 concurrent stats connections */ .maxconn = 10, /* 10 concurrent stats connections */
@ -894,7 +894,7 @@ static void tell_old_pids(int sig)
*/ */
void run_poll_loop() void run_poll_loop()
{ {
struct timeval next; int next;
tv_update_date(0,1); tv_update_date(0,1);
while (1) { while (1) {
@ -914,7 +914,7 @@ void run_poll_loop()
break; break;
/* The poller will ensure it returns around <next> */ /* The poller will ensure it returns around <next> */
cur_poller.poll(&cur_poller, &next); cur_poller.poll(&cur_poller, next);
} }
} }

View File

@ -30,6 +30,7 @@
#include <common/memory.h> #include <common/memory.h>
#include <common/mini-clist.h> #include <common/mini-clist.h>
#include <common/standard.h> #include <common/standard.h>
#include <common/ticks.h>
#include <common/time.h> #include <common/time.h>
#include <common/uri_auth.h> #include <common/uri_auth.h>
#include <common/version.h> #include <common/version.h>
@ -652,7 +653,7 @@ http_get_path(struct http_txn *txn)
* the time the task accepts to wait, or TIME_ETERNITY for * the time the task accepts to wait, or TIME_ETERNITY for
* infinity. * infinity.
*/ */
void process_session(struct task *t, struct timeval *next) void process_session(struct task *t, int *next)
{ {
struct session *s = t->context; struct session *s = t->context;
int fsm_resync = 0; int fsm_resync = 0;
@ -674,23 +675,15 @@ void process_session(struct task *t, struct timeval *next)
s->req->flags &= BF_CLEAR_READ & BF_CLEAR_WRITE; s->req->flags &= BF_CLEAR_READ & BF_CLEAR_WRITE;
s->rep->flags &= BF_CLEAR_READ & BF_CLEAR_WRITE; s->rep->flags &= BF_CLEAR_READ & BF_CLEAR_WRITE;
tv_min(&t->expire, &s->req->rex, &s->req->wex); t->expire = tick_first(tick_first(s->req->rex, s->req->wex),
tv_bound(&t->expire, &s->req->cex); tick_first(s->rep->rex, s->rep->wex));
tv_bound(&t->expire, &s->rep->rex); t->expire = tick_first(t->expire, s->req->cex);
tv_bound(&t->expire, &s->rep->wex);
if (s->cli_state == CL_STHEADERS) if (s->cli_state == CL_STHEADERS)
tv_bound(&t->expire, &s->txn.exp); t->expire = tick_first(t->expire, s->txn.exp);
/* restore t to its place in the task list */ /* restore t to its place in the task list */
task_queue(t); task_queue(t);
#ifdef DEBUG_FULL
/* DEBUG code : this should never ever happen, otherwise it indicates
* that a task still has something to do and will provoke a quick loop.
*/
if (tv_ms_remain2(&now, &t->expire) <= 0)
exit(100);
#endif
*next = t->expire; *next = t->expire;
return; /* nothing more to do */ return; /* nothing more to do */
} }
@ -726,7 +719,7 @@ void process_session(struct task *t, struct timeval *next)
task_delete(t); task_delete(t);
session_free(s); session_free(s);
task_free(t); task_free(t);
tv_eternity(next); *next = TICK_ETERNITY;
} }
@ -1647,8 +1640,8 @@ int process_cli(struct session *t)
} }
/* 3: has the read timeout expired ? */ /* 3: has the read timeout expired ? */
else if (unlikely(tv_isle(&req->rex, &now) || else if (unlikely(tick_is_expired(req->rex, now_ms) ||
tv_isle(&txn->exp, &now))) { tick_is_expired(txn->exp, now_ms))) {
/* read timeout : give up with an error message. */ /* read timeout : give up with an error message. */
txn->status = 408; txn->status = 408;
client_retnclose(t, error_message(t, HTTP_ERR_408)); client_retnclose(t, error_message(t, HTTP_ERR_408));
@ -1666,8 +1659,7 @@ int process_cli(struct session *t)
* full. We cannot loop here since stream_sock_read will disable it only if * full. We cannot loop here since stream_sock_read will disable it only if
* req->l == rlim-data * req->l == rlim-data
*/ */
if (!tv_add_ifset(&req->rex, &now, &t->fe->timeout.client)) req->rex = tick_add_ifset(now_ms, t->fe->timeout.client);
tv_eternity(&req->rex);
} }
return t->cli_state != CL_STHEADERS; return t->cli_state != CL_STHEADERS;
} }
@ -2245,8 +2237,8 @@ int process_cli(struct session *t)
t->logs.tv_request = now; t->logs.tv_request = now;
if (!tv_isset(&t->fe->timeout.client) || if (!t->fe->timeout.client ||
(t->srv_state < SV_STDATA && tv_isset(&t->be->timeout.server))) { (t->srv_state < SV_STDATA && t->be->timeout.server)) {
/* If the client has no timeout, or if the server is not ready yet, /* If the client has no timeout, or if the server is not ready yet,
* and we know for sure that it can expire, then it's cleaner to * and we know for sure that it can expire, then it's cleaner to
* disable the timeout on the client side so that too low values * disable the timeout on the client side so that too low values
@ -2256,7 +2248,7 @@ int process_cli(struct session *t)
* when it switches its state, otherwise a client can stay connected * when it switches its state, otherwise a client can stay connected
* indefinitely. This now seems to be OK. * indefinitely. This now seems to be OK.
*/ */
tv_eternity(&req->rex); req->rex = TICK_ETERNITY;
} }
/* When a connection is tarpitted, we use the tarpit timeout, /* When a connection is tarpitted, we use the tarpit timeout,
@ -2269,8 +2261,9 @@ int process_cli(struct session *t)
/* flush the request so that we can drop the connection early /* flush the request so that we can drop the connection early
* if the client closes first. * if the client closes first.
*/ */
if (!tv_add_ifset(&req->cex, &now, &t->be->timeout.tarpit)) req->cex = tick_add_ifset(now_ms, t->be->timeout.tarpit);
req->cex = now; if (!req->cex)
req->cex = now_ms;
} }
/* OK let's go on with the BODY now */ /* OK let's go on with the BODY now */
@ -2329,13 +2322,13 @@ int process_cli(struct session *t)
/* We must ensure that the read part is still alive when switching /* We must ensure that the read part is still alive when switching
* to shutw */ * to shutw */
EV_FD_SET(t->cli_fd, DIR_RD); EV_FD_SET(t->cli_fd, DIR_RD);
tv_add_ifset(&req->rex, &now, &t->fe->timeout.client); req->rex = tick_add_ifset(now_ms, t->fe->timeout.client);
t->cli_state = CL_STSHUTW; t->cli_state = CL_STSHUTW;
//fprintf(stderr,"%p:%s(%d), c=%d, s=%d\n", t, __FUNCTION__, __LINE__, t->cli_state, t->cli_state); //fprintf(stderr,"%p:%s(%d), c=%d, s=%d\n", t, __FUNCTION__, __LINE__, t->cli_state, t->cli_state);
return 1; return 1;
} }
/* read timeout */ /* read timeout */
else if (tv_isle(&req->rex, &now)) { else if (tick_is_expired(req->rex, now_ms)) {
EV_FD_CLR(t->cli_fd, DIR_RD); EV_FD_CLR(t->cli_fd, DIR_RD);
buffer_shutr(req); buffer_shutr(req);
t->cli_state = CL_STSHUTR; t->cli_state = CL_STSHUTR;
@ -2352,14 +2345,14 @@ int process_cli(struct session *t)
return 1; return 1;
} }
/* write timeout */ /* write timeout */
else if (tv_isle(&rep->wex, &now)) { else if (tick_is_expired(rep->wex, now_ms)) {
EV_FD_CLR(t->cli_fd, DIR_WR); EV_FD_CLR(t->cli_fd, DIR_WR);
buffer_shutw(rep); buffer_shutw(rep);
shutdown(t->cli_fd, SHUT_WR); shutdown(t->cli_fd, SHUT_WR);
/* We must ensure that the read part is still alive when switching /* We must ensure that the read part is still alive when switching
* to shutw */ * to shutw */
EV_FD_SET(t->cli_fd, DIR_RD); EV_FD_SET(t->cli_fd, DIR_RD);
tv_add_ifset(&req->rex, &now, &t->fe->timeout.client); req->rex = tick_add_ifset(now_ms, t->fe->timeout.client);
t->cli_state = CL_STSHUTW; t->cli_state = CL_STSHUTW;
if (!(t->flags & SN_ERR_MASK)) if (!(t->flags & SN_ERR_MASK))
@ -2379,21 +2372,21 @@ int process_cli(struct session *t)
/* no room to read more data */ /* no room to read more data */
if (EV_FD_COND_C(t->cli_fd, DIR_RD)) { if (EV_FD_COND_C(t->cli_fd, DIR_RD)) {
/* stop reading until we get some space */ /* stop reading until we get some space */
tv_eternity(&req->rex); req->rex = TICK_ETERNITY;
} }
} else { } else {
/* there's still some space in the buffer */ /* there's still some space in the buffer */
if (EV_FD_COND_S(t->cli_fd, DIR_RD)) { if (EV_FD_COND_S(t->cli_fd, DIR_RD)) {
if (!tv_isset(&t->fe->timeout.client) || if (!t->fe->timeout.client ||
(t->srv_state < SV_STDATA && tv_isset(&t->be->timeout.server))) (t->srv_state < SV_STDATA && t->be->timeout.server))
/* If the client has no timeout, or if the server not ready yet, and we /* If the client has no timeout, or if the server not ready yet, and we
* know for sure that it can expire, then it's cleaner to disable the * know for sure that it can expire, then it's cleaner to disable the
* timeout on the client side so that too low values cannot make the * timeout on the client side so that too low values cannot make the
* sessions abort too early. * sessions abort too early.
*/ */
tv_eternity(&req->rex); req->rex = TICK_ETERNITY;
else else
tv_add(&req->rex, &now, &t->fe->timeout.client); req->rex = tick_add(now_ms, t->fe->timeout.client);
} }
} }
@ -2401,19 +2394,18 @@ int process_cli(struct session *t)
((s < SV_STDATA) /* FIXME: this may be optimized && (rep->w == rep->h)*/)) { ((s < SV_STDATA) /* FIXME: this may be optimized && (rep->w == rep->h)*/)) {
if (EV_FD_COND_C(t->cli_fd, DIR_WR)) { if (EV_FD_COND_C(t->cli_fd, DIR_WR)) {
/* stop writing */ /* stop writing */
tv_eternity(&rep->wex); rep->wex = TICK_ETERNITY;
} }
} else { } else {
/* buffer not empty */ /* buffer not empty */
if (EV_FD_COND_S(t->cli_fd, DIR_WR)) { if (EV_FD_COND_S(t->cli_fd, DIR_WR)) {
/* restart writing */ /* restart writing */
if (tv_add_ifset(&rep->wex, &now, &t->fe->timeout.client)) { rep->wex = tick_add_ifset(now_ms, t->fe->timeout.client);
if (rep->wex) {
/* FIXME: to prevent the client from expiring read timeouts during writes, /* FIXME: to prevent the client from expiring read timeouts during writes,
* we refresh it. */ * we refresh it. */
req->rex = rep->wex; req->rex = rep->wex;
} }
else
tv_eternity(&rep->wex);
} }
} }
return 0; /* other cases change nothing */ return 0; /* other cases change nothing */
@ -2442,7 +2434,7 @@ int process_cli(struct session *t)
t->cli_state = CL_STCLOSE; t->cli_state = CL_STCLOSE;
return 1; return 1;
} }
else if (tv_isle(&rep->wex, &now)) { else if (tick_is_expired(rep->wex, now_ms)) {
buffer_shutw(rep); buffer_shutw(rep);
fd_delete(t->cli_fd); fd_delete(t->cli_fd);
t->cli_state = CL_STCLOSE; t->cli_state = CL_STCLOSE;
@ -2473,14 +2465,13 @@ int process_cli(struct session *t)
|| ((s == SV_STHEADERS) /* FIXME: this may be optimized && (rep->w == rep->h)*/)) { || ((s == SV_STHEADERS) /* FIXME: this may be optimized && (rep->w == rep->h)*/)) {
if (EV_FD_COND_C(t->cli_fd, DIR_WR)) { if (EV_FD_COND_C(t->cli_fd, DIR_WR)) {
/* stop writing */ /* stop writing */
tv_eternity(&rep->wex); rep->wex = TICK_ETERNITY;
} }
} else { } else {
/* buffer not empty */ /* buffer not empty */
if (EV_FD_COND_S(t->cli_fd, DIR_WR)) { if (EV_FD_COND_S(t->cli_fd, DIR_WR)) {
/* restart writing */ /* restart writing */
if (!tv_add_ifset(&rep->wex, &now, &t->fe->timeout.client)) rep->wex = tick_add_ifset(now_ms, t->fe->timeout.client);
tv_eternity(&rep->wex);
} }
} }
return 0; return 0;
@ -2508,7 +2499,7 @@ int process_cli(struct session *t)
t->cli_state = CL_STCLOSE; t->cli_state = CL_STCLOSE;
return 1; return 1;
} }
else if (tv_isle(&req->rex, &now)) { else if (tick_is_expired(req->rex, now_ms)) {
buffer_shutr(req); buffer_shutr(req);
fd_delete(t->cli_fd); fd_delete(t->cli_fd);
t->cli_state = CL_STCLOSE; t->cli_state = CL_STCLOSE;
@ -2533,15 +2524,12 @@ int process_cli(struct session *t)
if (EV_FD_COND_C(t->cli_fd, DIR_RD)) { if (EV_FD_COND_C(t->cli_fd, DIR_RD)) {
/* stop reading until we get some space */ /* stop reading until we get some space */
tv_eternity(&req->rex); req->rex = TICK_ETERNITY;
//fprintf(stderr,"%p:%s(%d), c=%d, s=%d\n", t, __FUNCTION__, __LINE__, t->cli_state, t->cli_state);
} }
} else { } else {
/* there's still some space in the buffer */ /* there's still some space in the buffer */
if (EV_FD_COND_S(t->cli_fd, DIR_RD)) { if (EV_FD_COND_S(t->cli_fd, DIR_RD)) {
if (!tv_add_ifset(&req->rex, &now, &t->fe->timeout.client)) req->rex = tick_add_ifset(now_ms, t->fe->timeout.client);
tv_eternity(&req->rex);
//fprintf(stderr,"%p:%s(%d), c=%d, s=%d\n", t, __FUNCTION__, __LINE__, t->cli_state, t->cli_state);
} }
} }
return 0; return 0;
@ -2615,7 +2603,7 @@ int process_srv(struct session *t)
else if (c == CL_STCLOSE || c == CL_STSHUTW || else if (c == CL_STCLOSE || c == CL_STSHUTW ||
(c == CL_STSHUTR && (c == CL_STSHUTR &&
(t->req->l == 0 || t->be->options & PR_O_ABRT_CLOSE))) { /* give up */ (t->req->l == 0 || t->be->options & PR_O_ABRT_CLOSE))) { /* give up */
tv_eternity(&req->cex); req->cex = TICK_ETERNITY;
if (t->pend_pos) if (t->pend_pos)
t->logs.t_queue = tv_ms_elapsed(&t->logs.tv_accept, &now); t->logs.t_queue = tv_ms_elapsed(&t->logs.tv_accept, &now);
/* note that this must not return any error because it would be able to /* note that this must not return any error because it would be able to
@ -2634,7 +2622,7 @@ int process_srv(struct session *t)
* already set the connect expiration date to the right * already set the connect expiration date to the right
* timeout. We just have to check that it has not expired. * timeout. We just have to check that it has not expired.
*/ */
if (!tv_isle(&req->cex, &now)) if (!tick_is_expired(req->cex, now_ms))
return 0; return 0;
/* We will set the queue timer to the time spent, just for /* We will set the queue timer to the time spent, just for
@ -2643,7 +2631,7 @@ int process_srv(struct session *t)
* It will not cause trouble to the logs because we can exclude * It will not cause trouble to the logs because we can exclude
* the tarpitted connections by filtering on the 'PT' status flags. * the tarpitted connections by filtering on the 'PT' status flags.
*/ */
tv_eternity(&req->cex); req->cex = TICK_ETERNITY;
t->logs.t_queue = tv_ms_elapsed(&t->logs.tv_accept, &now); t->logs.t_queue = tv_ms_elapsed(&t->logs.tv_accept, &now);
srv_close_with_err(t, SN_ERR_PRXCOND, SN_FINST_T, srv_close_with_err(t, SN_ERR_PRXCOND, SN_FINST_T,
500, error_message(t, HTTP_ERR_500)); 500, error_message(t, HTTP_ERR_500));
@ -2656,11 +2644,11 @@ int process_srv(struct session *t)
* to any other session to release it and wake us up again. * to any other session to release it and wake us up again.
*/ */
if (t->pend_pos) { if (t->pend_pos) {
if (!tv_isle(&req->cex, &now)) { if (!tick_is_expired(req->cex, now_ms)) {
return 0; return 0;
} else { } else {
/* we've been waiting too long here */ /* we've been waiting too long here */
tv_eternity(&req->cex); req->cex = TICK_ETERNITY;
t->logs.t_queue = tv_ms_elapsed(&t->logs.tv_accept, &now); t->logs.t_queue = tv_ms_elapsed(&t->logs.tv_accept, &now);
srv_close_with_err(t, SN_ERR_SRVTO, SN_FINST_Q, srv_close_with_err(t, SN_ERR_SRVTO, SN_FINST_Q,
503, error_message(t, HTTP_ERR_503)); 503, error_message(t, HTTP_ERR_503));
@ -2741,7 +2729,7 @@ int process_srv(struct session *t)
(c == CL_STSHUTR && (c == CL_STSHUTR &&
((t->req->l == 0 && !(req->flags & BF_WRITE_STATUS)) || ((t->req->l == 0 && !(req->flags & BF_WRITE_STATUS)) ||
t->be->options & PR_O_ABRT_CLOSE))) { /* give up */ t->be->options & PR_O_ABRT_CLOSE))) { /* give up */
tv_eternity(&req->cex); req->cex = TICK_ETERNITY;
if (!(t->flags & SN_CONN_TAR)) { if (!(t->flags & SN_CONN_TAR)) {
/* if we are in turn-around, we have already closed the FD */ /* if we are in turn-around, we have already closed the FD */
fd_delete(t->srv_fd); fd_delete(t->srv_fd);
@ -2757,8 +2745,7 @@ int process_srv(struct session *t)
srv_close_with_err(t, SN_ERR_CLICL, SN_FINST_C, 0, NULL); srv_close_with_err(t, SN_ERR_CLICL, SN_FINST_C, 0, NULL);
return 1; return 1;
} }
if (!(req->flags & BF_WRITE_STATUS) && !tv_isle(&req->cex, &now)) { if (!(req->flags & BF_WRITE_STATUS) && !tick_is_expired(req->cex, now_ms)) {
//fprintf(stderr,"1: c=%d, s=%d, now=%d.%06d, exp=%d.%06d\n", c, s, now.tv_sec, now.tv_usec, req->cex.tv_sec, req->cex.tv_usec);
return 0; /* nothing changed */ return 0; /* nothing changed */
} }
else if (!(req->flags & BF_WRITE_STATUS) || (req->flags & BF_WRITE_ERROR)) { else if (!(req->flags & BF_WRITE_STATUS) || (req->flags & BF_WRITE_ERROR)) {
@ -2767,7 +2754,7 @@ int process_srv(struct session *t)
if (t->flags & SN_CONN_TAR) { if (t->flags & SN_CONN_TAR) {
/* We are doing a turn-around waiting for a new connection attempt. */ /* We are doing a turn-around waiting for a new connection attempt. */
if (!tv_isle(&req->cex, &now)) if (!tick_is_expired(req->cex, now_ms))
return 0; return 0;
t->flags &= ~SN_CONN_TAR; t->flags &= ~SN_CONN_TAR;
} }
@ -2795,7 +2782,7 @@ int process_srv(struct session *t)
* time of 1 second. We will wait in the previous if block. * time of 1 second. We will wait in the previous if block.
*/ */
t->flags |= SN_CONN_TAR; t->flags |= SN_CONN_TAR;
tv_ms_add(&req->cex, &now, 1000); req->cex = tick_add(now_ms, MS_TO_TICKS(1000));
return 0; return 0;
} }
} }
@ -2843,23 +2830,20 @@ int process_srv(struct session *t)
//fprintf(stderr,"3: c=%d, s=%d\n", c, s); //fprintf(stderr,"3: c=%d, s=%d\n", c, s);
if (req->l == 0) /* nothing to write */ { if (req->l == 0) /* nothing to write */ {
EV_FD_CLR(t->srv_fd, DIR_WR); EV_FD_CLR(t->srv_fd, DIR_WR);
tv_eternity(&req->wex); req->wex = TICK_ETERNITY;
} else /* need the right to write */ { } else /* need the right to write */ {
EV_FD_SET(t->srv_fd, DIR_WR); EV_FD_SET(t->srv_fd, DIR_WR);
if (tv_add_ifset(&req->wex, &now, &t->be->timeout.server)) { req->wex = tick_add_ifset(now_ms, t->be->timeout.server);
if (req->wex) {
/* FIXME: to prevent the server from expiring read timeouts during writes, /* FIXME: to prevent the server from expiring read timeouts during writes,
* we refresh it. */ * we refresh it. */
rep->rex = req->wex; rep->rex = req->wex;
} }
else
tv_eternity(&req->wex);
} }
if (t->be->mode == PR_MODE_TCP) { /* let's allow immediate data connection in this case */ if (t->be->mode == PR_MODE_TCP) { /* let's allow immediate data connection in this case */
EV_FD_SET(t->srv_fd, DIR_RD); EV_FD_SET(t->srv_fd, DIR_RD);
if (!tv_add_ifset(&rep->rex, &now, &t->be->timeout.server)) rep->rex = tick_add_ifset(now_ms, t->be->timeout.server);
tv_eternity(&rep->rex);
t->srv_state = SV_STDATA; t->srv_state = SV_STDATA;
rep->rlim = rep->data + BUFSIZE; /* no rewrite needed */ rep->rlim = rep->data + BUFSIZE; /* no rewrite needed */
@ -2885,7 +2869,7 @@ int process_srv(struct session *t)
* hdr_idx_init(&t->txn.hdr_idx); * hdr_idx_init(&t->txn.hdr_idx);
*/ */
} }
tv_eternity(&req->cex); req->cex = TICK_ETERNITY;
return 1; return 1;
} }
} }
@ -2940,8 +2924,7 @@ int process_srv(struct session *t)
* full. We cannot loop here since stream_sock_read will disable it only if * full. We cannot loop here since stream_sock_read will disable it only if
* rep->l == rlim-data * rep->l == rlim-data
*/ */
if (!tv_add_ifset(&rep->rex, &now, &t->be->timeout.server)) req->rex = tick_add_ifset(now_ms, t->be->timeout.server);
tv_eternity(&rep->rex);
} }
@ -3003,10 +2986,9 @@ int process_srv(struct session *t)
return 1; return 1;
} }
/* read timeout : return a 504 to the client. /* read timeout : return a 504 to the client. */
*/
else if (unlikely(EV_FD_ISSET(t->srv_fd, DIR_RD) && else if (unlikely(EV_FD_ISSET(t->srv_fd, DIR_RD) &&
tv_isle(&rep->rex, &now))) { tick_is_expired(rep->rex, now_ms))) {
buffer_shutr(rep); buffer_shutr(rep);
buffer_shutw(req); buffer_shutw(req);
fd_delete(t->srv_fd); fd_delete(t->srv_fd);
@ -3047,7 +3029,7 @@ int process_srv(struct session *t)
/* We must ensure that the read part is still /* We must ensure that the read part is still
* alive when switching to shutw */ * alive when switching to shutw */
EV_FD_SET(t->srv_fd, DIR_RD); EV_FD_SET(t->srv_fd, DIR_RD);
tv_add_ifset(&rep->rex, &now, &t->be->timeout.server); rep->rex = tick_add_ifset(now_ms, t->be->timeout.server);
shutdown(t->srv_fd, SHUT_WR); shutdown(t->srv_fd, SHUT_WR);
t->srv_state = SV_STSHUTW; t->srv_state = SV_STSHUTW;
@ -3060,14 +3042,14 @@ int process_srv(struct session *t)
* some work to do on the headers. * some work to do on the headers.
*/ */
else if (unlikely(EV_FD_ISSET(t->srv_fd, DIR_WR) && else if (unlikely(EV_FD_ISSET(t->srv_fd, DIR_WR) &&
tv_isle(&req->wex, &now))) { tick_is_expired(req->wex, now_ms))) {
EV_FD_CLR(t->srv_fd, DIR_WR); EV_FD_CLR(t->srv_fd, DIR_WR);
buffer_shutw(req); buffer_shutw(req);
shutdown(t->srv_fd, SHUT_WR); shutdown(t->srv_fd, SHUT_WR);
/* We must ensure that the read part is still alive /* We must ensure that the read part is still alive
* when switching to shutw */ * when switching to shutw */
EV_FD_SET(t->srv_fd, DIR_RD); EV_FD_SET(t->srv_fd, DIR_RD);
tv_add_ifset(&rep->rex, &now, &t->be->timeout.server); rep->rex = tick_add_ifset(now_ms, t->be->timeout.server);
t->srv_state = SV_STSHUTW; t->srv_state = SV_STSHUTW;
if (!(t->flags & SN_ERR_MASK)) if (!(t->flags & SN_ERR_MASK))
@ -3088,13 +3070,12 @@ int process_srv(struct session *t)
else if (likely(req->l)) { else if (likely(req->l)) {
if (EV_FD_COND_S(t->srv_fd, DIR_WR)) { if (EV_FD_COND_S(t->srv_fd, DIR_WR)) {
/* restart writing */ /* restart writing */
if (tv_add_ifset(&req->wex, &now, &t->be->timeout.server)) { req->wex = tick_add_ifset(now_ms, t->be->timeout.server);
if (req->wex) {
/* FIXME: to prevent the server from expiring read timeouts during writes, /* FIXME: to prevent the server from expiring read timeouts during writes,
* we refresh it. */ * we refresh it. */
rep->rex = req->wex; rep->rex = req->wex;
} }
else
tv_eternity(&req->wex);
} }
} }
@ -3102,7 +3083,7 @@ int process_srv(struct session *t)
else { else {
if (EV_FD_COND_C(t->srv_fd, DIR_WR)) { if (EV_FD_COND_C(t->srv_fd, DIR_WR)) {
/* stop writing */ /* stop writing */
tv_eternity(&req->wex); req->wex = TICK_ETERNITY;
} }
} }
@ -3396,7 +3377,7 @@ int process_srv(struct session *t)
/* We must ensure that the read part is still alive when switching /* We must ensure that the read part is still alive when switching
* to shutw */ * to shutw */
EV_FD_SET(t->srv_fd, DIR_RD); EV_FD_SET(t->srv_fd, DIR_RD);
tv_add_ifset(&rep->rex, &now, &t->be->timeout.server); rep->rex = tick_add_ifset(now_ms, t->be->timeout.server);
shutdown(t->srv_fd, SHUT_WR); shutdown(t->srv_fd, SHUT_WR);
t->srv_state = SV_STSHUTW; t->srv_state = SV_STSHUTW;
@ -3469,13 +3450,13 @@ int process_srv(struct session *t)
/* We must ensure that the read part is still alive when switching /* We must ensure that the read part is still alive when switching
* to shutw */ * to shutw */
EV_FD_SET(t->srv_fd, DIR_RD); EV_FD_SET(t->srv_fd, DIR_RD);
tv_add_ifset(&rep->rex, &now, &t->be->timeout.server); rep->rex = tick_add_ifset(now_ms, t->be->timeout.server);
t->srv_state = SV_STSHUTW; t->srv_state = SV_STSHUTW;
return 1; return 1;
} }
/* read timeout */ /* read timeout */
else if (tv_isle(&rep->rex, &now)) { else if (tick_is_expired(rep->rex, now_ms)) {
EV_FD_CLR(t->srv_fd, DIR_RD); EV_FD_CLR(t->srv_fd, DIR_RD);
buffer_shutr(rep); buffer_shutr(rep);
t->srv_state = SV_STSHUTR; t->srv_state = SV_STSHUTR;
@ -3486,14 +3467,14 @@ int process_srv(struct session *t)
return 1; return 1;
} }
/* write timeout */ /* write timeout */
else if (tv_isle(&req->wex, &now)) { else if (tick_is_expired(req->wex, now_ms)) {
EV_FD_CLR(t->srv_fd, DIR_WR); EV_FD_CLR(t->srv_fd, DIR_WR);
buffer_shutw(req); buffer_shutw(req);
shutdown(t->srv_fd, SHUT_WR); shutdown(t->srv_fd, SHUT_WR);
/* We must ensure that the read part is still alive when switching /* We must ensure that the read part is still alive when switching
* to shutw */ * to shutw */
EV_FD_SET(t->srv_fd, DIR_RD); EV_FD_SET(t->srv_fd, DIR_RD);
tv_add_ifset(&rep->rex, &now, &t->be->timeout.server); rep->cex = tick_add_ifset(now_ms, t->be->timeout.server);
t->srv_state = SV_STSHUTW; t->srv_state = SV_STSHUTW;
if (!(t->flags & SN_ERR_MASK)) if (!(t->flags & SN_ERR_MASK))
t->flags |= SN_ERR_SRVTO; t->flags |= SN_ERR_SRVTO;
@ -3506,32 +3487,30 @@ int process_srv(struct session *t)
if (req->l == 0) { if (req->l == 0) {
if (EV_FD_COND_C(t->srv_fd, DIR_WR)) { if (EV_FD_COND_C(t->srv_fd, DIR_WR)) {
/* stop writing */ /* stop writing */
tv_eternity(&req->wex); req->wex = TICK_ETERNITY;
} }
} }
else { /* buffer not empty, there are still data to be transferred */ else { /* buffer not empty, there are still data to be transferred */
if (EV_FD_COND_S(t->srv_fd, DIR_WR)) { if (EV_FD_COND_S(t->srv_fd, DIR_WR)) {
/* restart writing */ /* restart writing */
if (tv_add_ifset(&req->wex, &now, &t->be->timeout.server)) { req->wex = tick_add_ifset(now_ms, t->be->timeout.server);
if (req->wex) {
/* FIXME: to prevent the server from expiring read timeouts during writes, /* FIXME: to prevent the server from expiring read timeouts during writes,
* we refresh it. */ * we refresh it. */
rep->rex = req->wex; rep->rex = req->wex;
} }
else
tv_eternity(&req->wex);
} }
} }
/* recompute response time-outs */ /* recompute response time-outs */
if (rep->l == BUFSIZE) { /* no room to read more data */ if (rep->l == BUFSIZE) { /* no room to read more data */
if (EV_FD_COND_C(t->srv_fd, DIR_RD)) { if (EV_FD_COND_C(t->srv_fd, DIR_RD)) {
tv_eternity(&rep->rex); rep->rex = TICK_ETERNITY;
} }
} }
else { else {
if (EV_FD_COND_S(t->srv_fd, DIR_RD)) { if (EV_FD_COND_S(t->srv_fd, DIR_RD)) {
if (!tv_add_ifset(&rep->rex, &now, &t->be->timeout.server)) rep->rex = tick_add_ifset(now_ms, t->be->timeout.server);
tv_eternity(&rep->rex);
} }
} }
@ -3580,7 +3559,7 @@ int process_srv(struct session *t)
return 1; return 1;
} }
else if (tv_isle(&req->wex, &now)) { else if (tick_is_expired(req->wex, now_ms)) {
//EV_FD_CLR(t->srv_fd, DIR_WR); //EV_FD_CLR(t->srv_fd, DIR_WR);
buffer_shutw(req); buffer_shutw(req);
fd_delete(t->srv_fd); fd_delete(t->srv_fd);
@ -3605,14 +3584,13 @@ int process_srv(struct session *t)
else if (req->l == 0) { else if (req->l == 0) {
if (EV_FD_COND_C(t->srv_fd, DIR_WR)) { if (EV_FD_COND_C(t->srv_fd, DIR_WR)) {
/* stop writing */ /* stop writing */
tv_eternity(&req->wex); req->wex = TICK_ETERNITY;
} }
} }
else { /* buffer not empty */ else { /* buffer not empty */
if (EV_FD_COND_S(t->srv_fd, DIR_WR)) { if (EV_FD_COND_S(t->srv_fd, DIR_WR)) {
/* restart writing */ /* restart writing */
if (!tv_add_ifset(&req->wex, &now, &t->be->timeout.server)) req->wex = tick_add_ifset(now_ms, t->be->timeout.server);
tv_eternity(&req->wex);
} }
} }
return 0; return 0;
@ -3660,7 +3638,7 @@ int process_srv(struct session *t)
return 1; return 1;
} }
else if (tv_isle(&rep->rex, &now)) { else if (tick_is_expired(rep->rex, now_ms)) {
//EV_FD_CLR(t->srv_fd, DIR_RD); //EV_FD_CLR(t->srv_fd, DIR_RD);
buffer_shutr(rep); buffer_shutr(rep);
fd_delete(t->srv_fd); fd_delete(t->srv_fd);
@ -3684,13 +3662,12 @@ int process_srv(struct session *t)
} }
else if (rep->l == BUFSIZE) { /* no room to read more data */ else if (rep->l == BUFSIZE) { /* no room to read more data */
if (EV_FD_COND_C(t->srv_fd, DIR_RD)) { if (EV_FD_COND_C(t->srv_fd, DIR_RD)) {
tv_eternity(&rep->rex); rep->rex = TICK_ETERNITY;
} }
} }
else { else {
if (EV_FD_COND_S(t->srv_fd, DIR_RD)) { if (EV_FD_COND_S(t->srv_fd, DIR_RD)) {
if (!tv_add_ifset(&rep->rex, &now, &t->be->timeout.server)) rep->rex = tick_add_ifset(now_ms, t->be->timeout.server);
tv_eternity(&rep->rex);
} }
} }
return 0; return 0;
@ -3699,7 +3676,7 @@ int process_srv(struct session *t)
/* this server state is set by the client to study the body for server assignment */ /* this server state is set by the client to study the body for server assignment */
/* Have we been through this long enough to timeout? */ /* Have we been through this long enough to timeout? */
if (!tv_isle(&req->rex, &now)) { if (!tick_is_expired(req->rex, now_ms)) {
/* balance url_param check_post should have been the only to get into this. /* balance url_param check_post should have been the only to get into this.
* just wait for data, check to compare how much * just wait for data, check to compare how much
*/ */
@ -4382,7 +4359,7 @@ void manage_client_side_cookies(struct session *t, struct buffer *req)
}/* end while(srv) */ }/* end while(srv) */
}/* end else if server == NULL */ }/* end else if server == NULL */
tv_add(&asession_temp->expire, &now, &t->be->timeout.appsession); asession_temp->expire = tick_add_ifset(now_ms, t->be->timeout.appsession);
}/* end if ((t->proxy->appsession_name != NULL) ... */ }/* end if ((t->proxy->appsession_name != NULL) ... */
} }
@ -4837,7 +4814,7 @@ void manage_server_side_cookies(struct session *t, struct buffer *rtr)
if (asession_temp->serverid[0] == '\0') if (asession_temp->serverid[0] == '\0')
memcpy(asession_temp->serverid, t->srv->id, server_id_len); memcpy(asession_temp->serverid, t->srv->id, server_id_len);
tv_add(&asession_temp->expire, &now, &t->be->timeout.appsession); asession_temp->expire = tick_add_ifset(now_ms, t->be->timeout.appsession);
#if defined(DEBUG_HASH) #if defined(DEBUG_HASH)
appsession_hash_dump(&(t->be->htbl_proxy)); appsession_hash_dump(&(t->be->htbl_proxy));
@ -5000,7 +4977,7 @@ void get_srv_from_appsession(struct session *t, const char *begin, int len)
pool_free2(apools.sessid, local_asession.sessid); pool_free2(apools.sessid, local_asession.sessid);
} }
tv_add(&asession_temp->expire, &now, &t->be->timeout.appsession); asession_temp->expire = tick_add_ifset(now_ms, t->be->timeout.appsession);
asession_temp->request_count++; asession_temp->request_count++;
#if defined(DEBUG_HASH) #if defined(DEBUG_HASH)

View File

@ -32,6 +32,7 @@
#include <common/memory.h> #include <common/memory.h>
#include <common/mini-clist.h> #include <common/mini-clist.h>
#include <common/standard.h> #include <common/standard.h>
#include <common/ticks.h>
#include <common/time.h> #include <common/time.h>
#include <common/version.h> #include <common/version.h>
@ -500,18 +501,18 @@ int uxst_event_accept(int fd) {
fdtab[cfd].peeraddr = (struct sockaddr *)&s->cli_addr; fdtab[cfd].peeraddr = (struct sockaddr *)&s->cli_addr;
fdtab[cfd].peerlen = sizeof(s->cli_addr); fdtab[cfd].peerlen = sizeof(s->cli_addr);
tv_eternity(&s->req->rex); s->req->rex = TICK_ETERNITY;
tv_eternity(&s->req->wex); s->req->wex = TICK_ETERNITY;
tv_eternity(&s->req->cex); s->req->cex = TICK_ETERNITY;
tv_eternity(&s->rep->rex); s->rep->rex = TICK_ETERNITY;
tv_eternity(&s->rep->wex); s->rep->wex = TICK_ETERNITY;
tv_eternity(&s->req->wto); s->req->wto = TICK_ETERNITY;
tv_eternity(&s->req->cto); s->req->cto = TICK_ETERNITY;
tv_eternity(&s->req->rto); s->req->rto = TICK_ETERNITY;
tv_eternity(&s->rep->rto); s->rep->rto = TICK_ETERNITY;
tv_eternity(&s->rep->cto); s->rep->cto = TICK_ETERNITY;
tv_eternity(&s->rep->wto); s->rep->wto = TICK_ETERNITY;
if (l->timeout) if (l->timeout)
s->req->rto = *l->timeout; s->req->rto = *l->timeout;
@ -519,10 +520,10 @@ int uxst_event_accept(int fd) {
if (l->timeout) if (l->timeout)
s->rep->wto = *l->timeout; s->rep->wto = *l->timeout;
tv_eternity(&t->expire); t->expire = TICK_ETERNITY;
if (l->timeout && tv_isset(l->timeout)) { if (l->timeout && *l->timeout) {
EV_FD_SET(cfd, DIR_RD); EV_FD_SET(cfd, DIR_RD);
tv_add(&s->req->rex, &now, &s->req->rto); s->req->rex = tick_add(now_ms, s->req->rto);
t->expire = s->req->rex; t->expire = s->req->rex;
} }
@ -592,13 +593,13 @@ static int process_uxst_cli(struct session *t)
/* We must ensure that the read part is still alive when switching /* We must ensure that the read part is still alive when switching
* to shutw */ * to shutw */
EV_FD_SET(t->cli_fd, DIR_RD); EV_FD_SET(t->cli_fd, DIR_RD);
tv_add_ifset(&req->rex, &now, &req->rto); req->rex = tick_add_ifset(now_ms, req->rto);
t->cli_state = CL_STSHUTW; t->cli_state = CL_STSHUTW;
//fprintf(stderr,"%p:%s(%d), c=%d, s=%d\n", t, __FUNCTION__, __LINE__, t->cli_state, t->cli_state); //fprintf(stderr,"%p:%s(%d), c=%d, s=%d\n", t, __FUNCTION__, __LINE__, t->cli_state, t->cli_state);
return 1; return 1;
} }
/* read timeout */ /* read timeout */
else if (tv_isle(&req->rex, &now)) { else if (tick_is_expired(req->rex, now_ms)) {
EV_FD_CLR(t->cli_fd, DIR_RD); EV_FD_CLR(t->cli_fd, DIR_RD);
buffer_shutr(req); buffer_shutr(req);
t->cli_state = CL_STSHUTR; t->cli_state = CL_STSHUTR;
@ -615,14 +616,14 @@ static int process_uxst_cli(struct session *t)
return 1; return 1;
} }
/* write timeout */ /* write timeout */
else if (tv_isle(&rep->wex, &now)) { else if (tick_is_expired(rep->wex, now_ms)) {
EV_FD_CLR(t->cli_fd, DIR_WR); EV_FD_CLR(t->cli_fd, DIR_WR);
buffer_shutw(rep); buffer_shutw(rep);
shutdown(t->cli_fd, SHUT_WR); shutdown(t->cli_fd, SHUT_WR);
/* We must ensure that the read part is still alive when switching /* We must ensure that the read part is still alive when switching
* to shutw */ * to shutw */
EV_FD_SET(t->cli_fd, DIR_RD); EV_FD_SET(t->cli_fd, DIR_RD);
tv_add_ifset(&req->rex, &now, &req->rto); req->rex = tick_add_ifset(now_ms, req->rto);
t->cli_state = CL_STSHUTW; t->cli_state = CL_STSHUTW;
if (!(t->flags & SN_ERR_MASK)) if (!(t->flags & SN_ERR_MASK))
@ -642,21 +643,21 @@ static int process_uxst_cli(struct session *t)
/* no room to read more data */ /* no room to read more data */
if (EV_FD_COND_C(t->cli_fd, DIR_RD)) { if (EV_FD_COND_C(t->cli_fd, DIR_RD)) {
/* stop reading until we get some space */ /* stop reading until we get some space */
tv_eternity(&req->rex); req->rex = TICK_ETERNITY;
} }
} else { } else {
/* there's still some space in the buffer */ /* there's still some space in the buffer */
if (EV_FD_COND_S(t->cli_fd, DIR_RD)) { if (EV_FD_COND_S(t->cli_fd, DIR_RD)) {
if (!tv_isset(&req->rto) || if (!req->rto ||
(t->srv_state < SV_STDATA && tv_isset(&req->wto))) (t->srv_state < SV_STDATA && req->wto))
/* If the client has no timeout, or if the server not ready yet, and we /* If the client has no timeout, or if the server not ready yet, and we
* know for sure that it can expire, then it's cleaner to disable the * know for sure that it can expire, then it's cleaner to disable the
* timeout on the client side so that too low values cannot make the * timeout on the client side so that too low values cannot make the
* sessions abort too early. * sessions abort too early.
*/ */
tv_eternity(&req->rex); req->rex = TICK_ETERNITY;
else else
tv_add(&req->rex, &now, &req->rto); req->rex = tick_add(now_ms, req->rto);
} }
} }
@ -664,19 +665,18 @@ static int process_uxst_cli(struct session *t)
((s < SV_STDATA) /* FIXME: this may be optimized && (rep->w == rep->h)*/)) { ((s < SV_STDATA) /* FIXME: this may be optimized && (rep->w == rep->h)*/)) {
if (EV_FD_COND_C(t->cli_fd, DIR_WR)) { if (EV_FD_COND_C(t->cli_fd, DIR_WR)) {
/* stop writing */ /* stop writing */
tv_eternity(&rep->wex); rep->wex = TICK_ETERNITY;
} }
} else { } else {
/* buffer not empty */ /* buffer not empty */
if (EV_FD_COND_S(t->cli_fd, DIR_WR)) { if (EV_FD_COND_S(t->cli_fd, DIR_WR)) {
/* restart writing */ /* restart writing */
if (tv_add_ifset(&rep->wex, &now, &rep->wto)) { rep->wex = tick_add_ifset(now_ms, rep->wto);
if (rep->wex) {
/* FIXME: to prevent the client from expiring read timeouts during writes, /* FIXME: to prevent the client from expiring read timeouts during writes,
* we refresh it. */ * we refresh it. */
req->rex = rep->wex; req->rex = rep->wex;
} }
else
tv_eternity(&rep->wex);
} }
} }
return 0; /* other cases change nothing */ return 0; /* other cases change nothing */
@ -704,7 +704,7 @@ static int process_uxst_cli(struct session *t)
t->cli_state = CL_STCLOSE; t->cli_state = CL_STCLOSE;
return 1; return 1;
} }
else if (tv_isle(&rep->wex, &now)) { else if (tick_is_expired(rep->wex, now_ms)) {
buffer_shutw(rep); buffer_shutw(rep);
fd_delete(t->cli_fd); fd_delete(t->cli_fd);
t->cli_state = CL_STCLOSE; t->cli_state = CL_STCLOSE;
@ -724,14 +724,13 @@ static int process_uxst_cli(struct session *t)
if (rep->l == 0) { if (rep->l == 0) {
if (EV_FD_COND_C(t->cli_fd, DIR_WR)) { if (EV_FD_COND_C(t->cli_fd, DIR_WR)) {
/* stop writing */ /* stop writing */
tv_eternity(&rep->wex); rep->wex = TICK_ETERNITY;
} }
} else { } else {
/* buffer not empty */ /* buffer not empty */
if (EV_FD_COND_S(t->cli_fd, DIR_WR)) { if (EV_FD_COND_S(t->cli_fd, DIR_WR)) {
/* restart writing */ /* restart writing */
if (!tv_add_ifset(&rep->wex, &now, &rep->wto)) rep->wex = tick_add_ifset(now_ms, rep->wto);
tv_eternity(&rep->wex);
} }
} }
return 0; return 0;
@ -759,7 +758,7 @@ static int process_uxst_cli(struct session *t)
t->cli_state = CL_STCLOSE; t->cli_state = CL_STCLOSE;
return 1; return 1;
} }
else if (tv_isle(&req->rex, &now)) { else if (tick_is_expired(req->rex, now_ms)) {
buffer_shutr(req); buffer_shutr(req);
fd_delete(t->cli_fd); fd_delete(t->cli_fd);
t->cli_state = CL_STCLOSE; t->cli_state = CL_STCLOSE;
@ -784,15 +783,12 @@ static int process_uxst_cli(struct session *t)
if (EV_FD_COND_C(t->cli_fd, DIR_RD)) { if (EV_FD_COND_C(t->cli_fd, DIR_RD)) {
/* stop reading until we get some space */ /* stop reading until we get some space */
tv_eternity(&req->rex); req->rex = TICK_ETERNITY;
//fprintf(stderr,"%p:%s(%d), c=%d, s=%d\n", t, __FUNCTION__, __LINE__, t->cli_state, t->cli_state);
} }
} else { } else {
/* there's still some space in the buffer */ /* there's still some space in the buffer */
if (EV_FD_COND_S(t->cli_fd, DIR_RD)) { if (EV_FD_COND_S(t->cli_fd, DIR_RD)) {
if (!tv_add_ifset(&req->rex, &now, &req->rto)) req->rex = tick_add_ifset(now_ms, req->rto);
tv_eternity(&req->rex);
//fprintf(stderr,"%p:%s(%d), c=%d, s=%d\n", t, __FUNCTION__, __LINE__, t->cli_state, t->cli_state);
} }
} }
return 0; return 0;
@ -1256,7 +1252,7 @@ static int process_uxst_srv(struct session *t)
* the time the task accepts to wait, or TIME_ETERNITY for * the time the task accepts to wait, or TIME_ETERNITY for
* infinity. * infinity.
*/ */
void process_uxst_session(struct task *t, struct timeval *next) void process_uxst_session(struct task *t, int *next)
{ {
struct session *s = t->context; struct session *s = t->context;
int fsm_resync = 0; int fsm_resync = 0;
@ -1357,7 +1353,7 @@ void process_uxst_session(struct task *t, struct timeval *next)
* for now. It only knows states SV_STIDLE, SV_STCONN, SV_STDATA, and * for now. It only knows states SV_STIDLE, SV_STCONN, SV_STDATA, and
* SV_STCLOSE. Returns in <next> the task's expiration date. * SV_STCLOSE. Returns in <next> the task's expiration date.
*/ */
void process_uxst_stats(struct task *t, struct timeval *next) void process_uxst_stats(struct task *t, int *next)
{ {
struct session *s = t->context; struct session *s = t->context;
struct listener *listener; struct listener *listener;
@ -1468,11 +1464,9 @@ void process_uxst_stats(struct task *t, struct timeval *next)
s->req->flags &= BF_CLEAR_READ & BF_CLEAR_WRITE; s->req->flags &= BF_CLEAR_READ & BF_CLEAR_WRITE;
s->rep->flags &= BF_CLEAR_READ & BF_CLEAR_WRITE; s->rep->flags &= BF_CLEAR_READ & BF_CLEAR_WRITE;
t->expire = s->req->rex; t->expire = tick_first(tick_first(s->req->rex, s->req->wex),
tv_min(&t->expire, &s->req->rex, &s->req->wex); tick_first(s->rep->rex, s->rep->wex));
tv_bound(&t->expire, &s->req->cex); t->expire = tick_first(t->expire, s->req->cex);
tv_bound(&t->expire, &s->rep->rex);
tv_bound(&t->expire, &s->rep->wex);
/* restore t to its place in the task list */ /* restore t to its place in the task list */
task_queue(t); task_queue(t);
@ -1497,7 +1491,7 @@ void process_uxst_stats(struct task *t, struct timeval *next)
task_delete(t); task_delete(t);
session_free(s); session_free(s);
task_free(t); task_free(t);
tv_eternity(next); *next = TICK_ETERNITY;
} }
__attribute__((constructor)) __attribute__((constructor))

View File

@ -90,8 +90,8 @@ int proxy_parse_timeout(const char **args, struct proxy *proxy,
unsigned timeout; unsigned timeout;
int retval, cap; int retval, cap;
const char *res, *name; const char *res, *name;
struct timeval *tv = NULL; int *tv = NULL;
struct timeval *td = NULL; int *td = NULL;
retval = 0; retval = 0;
name = args[0]; name = args[0];
@ -155,16 +155,12 @@ int proxy_parse_timeout(const char **args, struct proxy *proxy,
(cap & PR_CAP_BE) ? "backend" : "frontend"); (cap & PR_CAP_BE) ? "backend" : "frontend");
retval = 1; retval = 1;
} }
else if (defpx && !__tv_iseq(tv, td)) { else if (defpx && *tv != *td) {
snprintf(err, errlen, "overwriting %s timeout which was already specified", name); snprintf(err, errlen, "overwriting %s timeout which was already specified", name);
retval = 1; retval = 1;
} }
if (timeout) *tv = MS_TO_TICKS(timeout);
__tv_from_ms(tv, timeout);
else
tv_eternity(tv);
return retval; return retval;
} }
@ -303,7 +299,7 @@ int start_proxies(int verbose)
* select_loop(). It adjusts the date of next expiration event during stop * select_loop(). It adjusts the date of next expiration event during stop
* time if appropriate. * time if appropriate.
*/ */
void maintain_proxies(struct timeval *next) void maintain_proxies(int *next)
{ {
struct proxy *p; struct proxy *p;
struct listener *l; struct listener *l;
@ -346,7 +342,7 @@ void maintain_proxies(struct timeval *next)
while (p) { while (p) {
if (p->state != PR_STSTOPPED) { if (p->state != PR_STSTOPPED) {
int t; int t;
t = tv_ms_remain2(&now, &p->stop_time); t = tick_remain(now_ms, p->stop_time);
if (t == 0) { if (t == 0) {
Warning("Proxy %s stopped.\n", p->id); Warning("Proxy %s stopped.\n", p->id);
send_log(p, LOG_WARNING, "Proxy %s stopped.\n", p->id); send_log(p, LOG_WARNING, "Proxy %s stopped.\n", p->id);
@ -363,7 +359,7 @@ void maintain_proxies(struct timeval *next)
pool_gc2(); pool_gc2();
} }
else { else {
tv_bound(next, &p->stop_time); *next = tick_first(*next, p->stop_time);
} }
} }
p = p->next; p = p->next;
@ -389,7 +385,7 @@ void soft_stop(void)
if (p->state != PR_STSTOPPED) { if (p->state != PR_STSTOPPED) {
Warning("Stopping proxy %s in %d ms.\n", p->id, p->grace); Warning("Stopping proxy %s in %d ms.\n", p->id, p->grace);
send_log(p, LOG_WARNING, "Stopping proxy %s in %d ms.\n", p->id, p->grace); send_log(p, LOG_WARNING, "Stopping proxy %s in %d ms.\n", p->id, p->grace);
tv_ms_add(&p->stop_time, &now, p->grace); p->stop_time = tick_add(now_ms, p->grace);
} }
p = p->next; p = p->next;
} }

View File

@ -2,7 +2,7 @@
* Helper functions to send data over a socket and buffer. * Helper functions to send data over a socket and buffer.
* Should probably move somewhere else, but where ? * Should probably move somewhere else, but where ?
* *
* Copyright 2000-2007 Willy Tarreau <w@1wt.eu> * Copyright 2000-2008 Willy Tarreau <w@1wt.eu>
* *
* This program is free software; you can redistribute it and/or * This program is free software; you can redistribute it and/or
* modify it under the terms of the GNU General Public License * modify it under the terms of the GNU General Public License
@ -28,6 +28,7 @@
#include <common/debug.h> #include <common/debug.h>
#include <common/memory.h> #include <common/memory.h>
#include <common/standard.h> #include <common/standard.h>
#include <common/ticks.h>
#include <common/time.h> #include <common/time.h>
#include <common/version.h> #include <common/version.h>
@ -56,8 +57,7 @@ void client_retnclose(struct session *s, const struct chunk *msg)
EV_FD_CLR(s->cli_fd, DIR_RD); EV_FD_CLR(s->cli_fd, DIR_RD);
EV_FD_SET(s->cli_fd, DIR_WR); EV_FD_SET(s->cli_fd, DIR_WR);
buffer_shutr(s->req); buffer_shutr(s->req);
if (!tv_add_ifset(&s->rep->wex, &now, &s->rep->wto)) s->rep->wex = tick_add_ifset(now_ms, s->rep->wto);
tv_eternity(&s->rep->wex);
s->cli_state = CL_STSHUTR; s->cli_state = CL_STSHUTR;
buffer_flush(s->rep); buffer_flush(s->rep);
if (msg && msg->len) if (msg && msg->len)

View File

@ -1,7 +1,7 @@
/* /*
* Functions operating on SOCK_STREAM and buffers. * Functions operating on SOCK_STREAM and buffers.
* *
* Copyright 2000-2007 Willy Tarreau <w@1wt.eu> * Copyright 2000-2008 Willy Tarreau <w@1wt.eu>
* *
* This program is free software; you can redistribute it and/or * This program is free software; you can redistribute it and/or
* modify it under the terms of the GNU General Public License * modify it under the terms of the GNU General Public License
@ -23,6 +23,7 @@
#include <common/config.h> #include <common/config.h>
#include <common/debug.h> #include <common/debug.h>
#include <common/standard.h> #include <common/standard.h>
#include <common/ticks.h>
#include <common/time.h> #include <common/time.h>
#include <types/buffers.h> #include <types/buffers.h>
@ -42,7 +43,7 @@
* otherwise. * otherwise.
*/ */
int stream_sock_read(int fd) { int stream_sock_read(int fd) {
__label__ out_eternity, out_wakeup, out_shutdown_r, out_error; __label__ out_wakeup, out_shutdown_r, out_error;
struct buffer *b = fdtab[fd].cb[DIR_RD].b; struct buffer *b = fdtab[fd].cb[DIR_RD].b;
int ret, max, retval, cur_read; int ret, max, retval, cur_read;
int read_poll = MAX_READ_POLL_LOOPS; int read_poll = MAX_READ_POLL_LOOPS;
@ -88,7 +89,8 @@ int stream_sock_read(int fd) {
* never happen, but better safe than sorry ! * never happen, but better safe than sorry !
*/ */
EV_FD_CLR(fd, DIR_RD); EV_FD_CLR(fd, DIR_RD);
goto out_eternity; b->rex = TICK_ETERNITY;
goto out_wakeup;
} }
/* /*
@ -154,7 +156,8 @@ int stream_sock_read(int fd) {
} }
EV_FD_CLR(fd, DIR_RD); EV_FD_CLR(fd, DIR_RD);
goto out_eternity; b->rex = TICK_ETERNITY;
goto out_wakeup;
} }
/* if too many bytes were missing from last read, it means that /* if too many bytes were missing from last read, it means that
@ -218,12 +221,8 @@ int stream_sock_read(int fd) {
* have at least read something. * have at least read something.
*/ */
if (b->flags & BF_PARTIAL_READ) { if (b->flags & BF_PARTIAL_READ)
if (tv_add_ifset(&b->rex, &now, &b->rto)) b->rex = tick_add_ifset(now_ms, b->rto);
goto out_wakeup;
out_eternity:
tv_eternity(&b->rex);
}
out_wakeup: out_wakeup:
if (b->flags & BF_READ_STATUS) if (b->flags & BF_READ_STATUS)
@ -234,7 +233,8 @@ int stream_sock_read(int fd) {
out_shutdown_r: out_shutdown_r:
fdtab[fd].ev &= ~FD_POLL_HUP; fdtab[fd].ev &= ~FD_POLL_HUP;
b->flags |= BF_READ_NULL; b->flags |= BF_READ_NULL;
goto out_eternity; b->rex = TICK_ETERNITY;
goto out_wakeup;
out_error: out_error:
/* There was an error. we must wakeup the task. No need to clear /* There was an error. we must wakeup the task. No need to clear
@ -243,7 +243,8 @@ int stream_sock_read(int fd) {
fdtab[fd].state = FD_STERROR; fdtab[fd].state = FD_STERROR;
fdtab[fd].ev &= ~FD_POLL_STICKY; fdtab[fd].ev &= ~FD_POLL_STICKY;
b->flags |= BF_READ_ERROR; b->flags |= BF_READ_ERROR;
goto out_eternity; b->rex = TICK_ETERNITY;
goto out_wakeup;
} }
@ -254,7 +255,7 @@ int stream_sock_read(int fd) {
* otherwise. * otherwise.
*/ */
int stream_sock_write(int fd) { int stream_sock_write(int fd) {
__label__ out_eternity, out_wakeup, out_error; __label__ out_wakeup, out_error;
struct buffer *b = fdtab[fd].cb[DIR_WR].b; struct buffer *b = fdtab[fd].cb[DIR_WR].b;
int ret, max, retval; int ret, max, retval;
int write_poll = MAX_WRITE_POLL_LOOPS; int write_poll = MAX_WRITE_POLL_LOOPS;
@ -314,7 +315,8 @@ int stream_sock_write(int fd) {
* let's disable the write event and pretend we never came there. * let's disable the write event and pretend we never came there.
*/ */
EV_FD_CLR(fd, DIR_WR); EV_FD_CLR(fd, DIR_WR);
goto out_eternity; b->wex = TICK_ETERNITY;
goto out_wakeup;
} }
#ifndef MSG_NOSIGNAL #ifndef MSG_NOSIGNAL
@ -344,7 +346,8 @@ int stream_sock_write(int fd) {
if (!b->l) { if (!b->l) {
EV_FD_CLR(fd, DIR_WR); EV_FD_CLR(fd, DIR_WR);
goto out_eternity; b->wex = TICK_ETERNITY;
goto out_wakeup;
} }
/* if the system buffer is full, don't insist */ /* if the system buffer is full, don't insist */
@ -375,17 +378,15 @@ int stream_sock_write(int fd) {
*/ */
if (b->flags & BF_PARTIAL_WRITE) { if (b->flags & BF_PARTIAL_WRITE) {
if (tv_add_ifset(&b->wex, &now, &b->wto)) { b->wex = tick_add_ifset(now_ms, b->wto);
if (b->wex) {
/* FIXME: to prevent the client from expiring read timeouts during writes, /* FIXME: to prevent the client from expiring read timeouts during writes,
* we refresh it. A solution would be to merge read+write timeouts into a * we refresh it. A solution would be to merge read+write timeouts into a
* unique one, although that needs some study particularly on full-duplex * unique one, although that needs some study particularly on full-duplex
* TCP connections. */ * TCP connections. */
if (!(b->flags & BF_SHUTR_STATUS)) if (!(b->flags & BF_SHUTR_STATUS))
b->rex = b->wex; b->rex = b->wex;
goto out_wakeup;
} }
out_eternity:
tv_eternity(&b->wex);
} }
out_wakeup: out_wakeup:
@ -401,9 +402,8 @@ int stream_sock_write(int fd) {
fdtab[fd].state = FD_STERROR; fdtab[fd].state = FD_STERROR;
fdtab[fd].ev &= ~FD_POLL_STICKY; fdtab[fd].ev &= ~FD_POLL_STICKY;
b->flags |= BF_WRITE_ERROR; b->flags |= BF_WRITE_ERROR;
goto out_eternity; b->wex = TICK_ETERNITY;
goto out_wakeup;
} }

View File

@ -171,10 +171,10 @@ struct task *task_wakeup(struct task *t)
*/ */
struct task *task_queue(struct task *task) struct task *task_queue(struct task *task)
{ {
if (unlikely(tv_iseternity(&task->expire))) if (unlikely(!task->expire))
return task; return task;
task->eb.key = timeval_to_ticks(&task->expire); task->eb.key = task->expire;
#ifdef DEBUG_CHECK_INVALID_EXPIRATION_DATES #ifdef DEBUG_CHECK_INVALID_EXPIRATION_DATES
if ((task->eb.key - now_ms) & TIMER_SIGN_BIT) if ((task->eb.key - now_ms) & TIMER_SIGN_BIT)
/* we're queuing too far away or in the past (most likely) */ /* we're queuing too far away or in the past (most likely) */
@ -200,7 +200,7 @@ struct task *task_queue(struct task *task)
* Extract all expired timers from the timer queue, and wakes up all * Extract all expired timers from the timer queue, and wakes up all
* associated tasks. Returns the date of next event (or eternity). * associated tasks. Returns the date of next event (or eternity).
*/ */
void wake_expired_tasks(struct timeval *next) void wake_expired_tasks(int *next)
{ {
struct task *task; struct task *task;
struct eb32_node *eb; struct eb32_node *eb;
@ -238,7 +238,7 @@ void wake_expired_tasks(struct timeval *next)
} while (((tree - now_tree) & TIMER_TREE_MASK) < TIMER_TREES/2); } while (((tree - now_tree) & TIMER_TREE_MASK) < TIMER_TREES/2);
/* We have found no task to expire in any tree */ /* We have found no task to expire in any tree */
tv_eternity(next); *next = TICK_ETERNITY;
return; return;
} }
@ -257,9 +257,9 @@ void wake_expired_tasks(struct timeval *next)
* *
* The function adjusts <next> if a new event is closer. * The function adjusts <next> if a new event is closer.
*/ */
void process_runnable_tasks(struct timeval *next) void process_runnable_tasks(int *next)
{ {
struct timeval temp; int temp;
struct task *t; struct task *t;
struct eb32_node *eb; struct eb32_node *eb;
unsigned int tree, stop; unsigned int tree, stop;
@ -294,7 +294,7 @@ void process_runnable_tasks(struct timeval *next)
task_dequeue(t); task_dequeue(t);
t->process(t, &temp); t->process(t, &temp);
tv_bound(next, &temp); *next = tick_first(*next, temp);
if (!--max_processed) if (!--max_processed)
return; return;