/* Adaptive Timeout stuff */
#define D_ADAPTTO D_OTHER
#define AT_BINS 4 /* "bin" means "N seconds of history" */
-#define AT_TIMEBASE_DEFAULT 600 /* remembered history (sec) (should be
- evenly divisible by AT_BINS) */
#define AT_FLG_NOHIST 0x1 /* use last reported value only */
#define AT_FLG_MIN 0x2 /* use a minimum limit */
struct adaptive_timeout {
time_t at_binstart; /* bin start time */
- time_t at_binlimit; /* bin time limit */
unsigned int at_hist[AT_BINS]; /* timeout history bins */
unsigned int at_flags;
unsigned int at_current; /* current timeout value */
};
/* import.c */
-static inline void at_init(struct adaptive_timeout *at, int val, int timebase,
- int flags) {
+static inline void at_init(struct adaptive_timeout *at, int val, int flags) {
memset(at, 0, sizeof(*at));
- at->at_binlimit = timebase / AT_BINS;
at->at_current = val;
at->at_worst_ever = val;
at->at_worst_time = cfs_time_current_sec();
return min(at->at_current, adaptive_timeout_max);
return at->at_current;
}
-void at_add(struct adaptive_timeout *at, unsigned int val);
+int at_add(struct adaptive_timeout *at, unsigned int val);
int import_at_get_index(struct obd_import *imp, int portal);
int import_at_get_ldlm(struct obd_import *imp);
#define AT_OFF (adaptive_timeout_max == 0)
void target_cleanup_recovery(struct obd_device *obd);
int target_queue_recovery_request(struct ptlrpc_request *req,
struct obd_device *obd);
-int target_queue_final_reply(struct ptlrpc_request *req, int rc);
+int target_queue_last_replay_reply(struct ptlrpc_request *req, int rc);
void target_send_reply(struct ptlrpc_request *req, int rc, int fail_id);
/* client.c */
__u64 rq_history_seq; /* history sequence # */
int rq_status;
spinlock_t rq_lock;
- /* client-side flags. */
+ /* client-side flags are serialized by rq_lock */
unsigned int rq_intr:1, rq_replied:1, rq_err:1,
rq_timedout:1, rq_resend:1, rq_restart:1,
/*
rq_replay:1,
rq_no_resend:1, rq_waiting:1, rq_receiving_reply:1,
rq_no_delay:1, rq_net_err:1, rq_early:1, rq_must_unlink:1,
- /* server-side: */
- rq_final:1; /* packed final reply */
+ /* server-side flags */
+ rq_packed_final:1; /* packed final reply */
enum rq_phase rq_phase; /* one of RQ_PHASE_* */
atomic_t rq_refcount; /* client-side refcount for SENT race,
server-side refcounf for multiple replies */
/* server-side... */
struct timeval rq_arrival_time; /* request arrival time */
struct ptlrpc_reply_state *rq_reply_state; /* separated reply state */
- struct semaphore rq_rs_sem; /* one reply at a time */
struct ptlrpc_request_buffer_desc *rq_rqbd; /* incoming request buffer*/
#ifdef CRAY_XT3
__u32 rq_uid; /* peer uid, used in MDS only */
/* client outgoing req */
time_t rq_sent; /* when request/reply sent (secs) */
- time_t rq_deadline; /* when request must finish */
+ volatile time_t rq_deadline; /* when request must finish. volatile
+ so that servers' early reply updates to the deadline aren't
+ kept in per-cpu cache */
int rq_timeout; /* service time estimate (secs) */
/* Multi-rpc bits */
struct ptlrpc_connection *ptlrpc_uuid_to_connection(struct obd_uuid *uuid);
static inline int
-ptlrpc_client_receiving_reply (struct ptlrpc_request *req)
+ptlrpc_client_recv_or_unlink (struct ptlrpc_request *req)
{
int rc;
spin_lock(&req->rq_lock);
- rc = req->rq_receiving_reply;
- spin_unlock(&req->rq_lock);
- return (rc);
-}
-
-static inline int
-ptlrpc_client_must_unlink (struct ptlrpc_request *req)
-{
- int rc;
-
- spin_lock(&req->rq_lock);
- rc = req->rq_must_unlink;
- spin_unlock(&req->rq_lock);
- return (rc);
-}
-
-static inline int
-ptlrpc_client_replied (struct ptlrpc_request *req)
-{
- int rc;
-
- spin_lock(&req->rq_lock);
- rc = req->rq_replied;
+ rc = req->rq_receiving_reply || req->rq_must_unlink;
spin_unlock(&req->rq_lock);
return (rc);
}
ptlrpc_rs_decref(req->rq_reply_state);
req->rq_reply_state = NULL;
req->rq_repmsg = NULL;
- up(&req->rq_rs_sem); /* held since lustre_pack_reply */
-}
-
-/* Check if we already packed a normal (non-early) reply.
- Single thread only! */
-static inline int lustre_packed_reply(struct ptlrpc_request *req)
-{
- return req->rq_final;
}
static inline __u32 lustre_request_magic(struct ptlrpc_request *req)
ptlrpc_req_set_repsize(struct ptlrpc_request *req, int count, int *lens)
{
int size = lustre_msg_size(req->rq_reqmsg->lm_magic, count, lens);
+
req->rq_replen = size + lustre_msg_early_size();
if (req->rq_reqmsg->lm_magic == LUSTRE_MSG_MAGIC_V2)
req->rq_reqmsg->lm_repsize = size;
oti->oti_xid = req->rq_xid;
- if (req->rq_reqmsg && req->rq_repmsg && req->rq_reply_state)
+ if ((req->rq_reqmsg != NULL) && (req->rq_repmsg != NULL))
oti->oti_transno = lustre_msg_get_transno(req->rq_repmsg);
oti->oti_thread_id = req->rq_svc_thread ? req->rq_svc_thread->t_id : -1;
oti->oti_conn_cnt = lustre_msg_get_conn_cnt(req->rq_reqmsg);
extern unsigned int ldlm_timeout; /* seconds */
extern unsigned int adaptive_timeout_min; /* seconds */
extern unsigned int adaptive_timeout_max; /* seconds */
+extern unsigned int adaptive_timeout_history; /* seconds */
extern unsigned int obd_sync_filter;
extern unsigned int obd_max_dirty_pages;
extern atomic_t obd_dirty_pages;
#define INITIAL_CONNECT_TIMEOUT max_t(int,CONNECTION_SWITCH_MIN,obd_timeout/2)
#endif
#define LND_TIMEOUT 50 /* LNET LND-level RPC timeout */
-#define FOREVER 300 /* Something taking this long is broken */
+#define LONG_UNLINK 300 /* Unlink should happen before now */
#define OBD_FAIL_MDS 0x100
return req->rq_export->exp_obd;
}
-int target_queue_final_reply(struct ptlrpc_request *req, int rc)
+int target_queue_last_replay_reply(struct ptlrpc_request *req, int rc)
{
struct obd_device *obd = target_req2obd(req);
struct ptlrpc_request *saved_req;
struct lustre_msg *reqmsg;
int recovery_done = 0;
- LASSERT ((rc == 0) == lustre_packed_reply(req));
+ LASSERT ((rc == 0) == req->rq_packed_final);
- if (!lustre_packed_reply(req)) {
+ if (!req->rq_packed_final) {
/* Just like ptlrpc_error, but without the sending. */
rc = lustre_pack_reply(req, 1, NULL, NULL);
if (rc)
- CERROR("pack error %d\n", rc);
+ return rc;
req->rq_type = PTL_RPC_MSG_ERR;
}
- LASSERT (!req->rq_reply_state->rs_difficult);
+ LASSERT(!req->rq_reply_state->rs_difficult);
LASSERT(list_empty(&req->rq_list));
/* XXX a bit like the request-dup code in queue_recovery_request */
OBD_ALLOC(saved_req, sizeof *saved_req);
if (!saved_req)
- LBUG();
+ return -ENOMEM;
OBD_ALLOC(reqmsg, req->rq_reqlen);
- if (!reqmsg)
- LBUG();
+ if (!reqmsg) {
+ OBD_FREE(saved_req, sizeof *req);
+ return -ENOMEM;
+ }
*saved_req = *req;
memcpy(reqmsg, req->rq_reqmsg, req->rq_reqlen);
EXIT;
out:
req->rq_status = rc ?: err; /* return either error - bug 11190 */
- if (!lustre_packed_reply(req)) {
+ if (!req->rq_packed_final) {
err = lustre_pack_reply(req, 1, NULL, NULL);
if (rc == 0)
rc = err;
static int ldlm_callback_reply(struct ptlrpc_request *req, int rc)
{
req->rq_status = rc;
- if (!lustre_packed_reply(req)) {
+ if (!req->rq_packed_final) {
rc = lustre_pack_reply(req, 1, NULL, NULL);
if (rc)
return rc;
EXPORT_SYMBOL(target_handle_ping);
EXPORT_SYMBOL(target_pack_pool_reply);
EXPORT_SYMBOL(target_handle_disconnect);
-EXPORT_SYMBOL(target_queue_final_reply);
+EXPORT_SYMBOL(target_queue_last_replay_reply);
/* l_lock.c */
EXPORT_SYMBOL(lock_res_and_lock);
{ "mds_conn_uuid", lprocfs_rd_conn_uuid, 0, 0 },
{ "max_rpcs_in_flight", mdc_rd_max_rpcs_in_flight,
mdc_wr_max_rpcs_in_flight, 0 },
- { "timeouts", lprocfs_rd_timeouts, lprocfs_wr_timeouts, 0 },
+ { "timeouts", lprocfs_rd_timeouts, 0, 0 },
{ 0 }
};
pop_ctxt(&saved, &obd->obd_lvfs_ctxt, &uc);
default:
mds_exit_ucred(&uc, mds);
- if (!lustre_packed_reply(req)) {
+ if (!req->rq_packed_final) {
req->rq_status = rc;
lustre_pack_reply(req, 1, NULL, NULL);
}
out_pop:
pop_ctxt(&saved, &obd->obd_lvfs_ctxt, &uc);
out_ucred:
- if (!lustre_packed_reply(req)) {
+ if (!req->rq_packed_final) {
req->rq_status = rc;
lustre_pack_reply(req, 1, NULL, NULL);
}
if (lustre_msg_get_flags(req->rq_reqmsg) & MSG_LAST_REPLAY) {
if (obd && obd->obd_recovering) {
DEBUG_REQ(D_HA, req, "LAST_REPLAY, queuing reply");
- return target_queue_final_reply(req, rc);
+ return target_queue_last_replay_reply(req, rc);
}
/* Lost a race with recovery; let the error path DTRT. */
rc = req->rq_status = -ENOTCONN;
unsigned int obd_dump_on_eviction;
unsigned int obd_timeout = OBD_TIMEOUT_DEFAULT; /* seconds */
unsigned int ldlm_timeout = LDLM_TIMEOUT_DEFAULT; /* seconds */
+/* Covers the maximum expected network latency */
unsigned int adaptive_timeout_min = 10; /* seconds */
unsigned int adaptive_timeout_max = 600; /* seconds */
+/* We remember the slowest event that took place within history */
+unsigned int adaptive_timeout_history = 600; /* seconds */
unsigned int obd_max_dirty_pages = 256;
atomic_t obd_dirty_pages;
EXPORT_SYMBOL(ldlm_timeout);
EXPORT_SYMBOL(adaptive_timeout_min);
EXPORT_SYMBOL(adaptive_timeout_max);
+EXPORT_SYMBOL(adaptive_timeout_history);
EXPORT_SYMBOL(obd_max_dirty_pages);
EXPORT_SYMBOL(obd_dirty_pages);
EXPORT_SYMBOL(ptlrpc_put_connection_superhack);
Since we can't say for sure how slow a network might be, we use
a user-defined max expected network latency. We will adapt to slow
increases, but a sudden jump can still kill us. */
- at_init(&at->iat_net_latency, adaptive_timeout_min, AT_TIMEBASE_DEFAULT,
- AT_FLG_MIN);
+ at_init(&at->iat_net_latency, adaptive_timeout_min, AT_FLG_MIN);
for (i = 0; i < IMP_AT_MAX_PORTALS; i++) {
/* max service estimates are tracked on the server side, so
don't use the AT history here, just use the last reported
val. (But keep hist for proc histogram, worst_ever) */
at_init(&at->iat_service_estimate[i], INITIAL_CONNECT_TIMEOUT,
- AT_TIMEBASE_DEFAULT, AT_FLG_NOHIST);
+ AT_FLG_NOHIST);
}
at->iat_drain = 0;
}
OBD_ALLOC_FAIL_RATE, /* memory allocation random failure rate */
ADAPTIVE_MIN, /* Adaptive timeout lower limit */
ADAPTIVE_MAX, /* Adaptive timeout upper limit */
+ ADAPTIVE_HISTORY, /* Adaptive timeout timebase */
};
int LL_PROC_PROTO(proc_fail_loc)
.mode = 0644,
.proc_handler = &proc_dointvec
},
+ {
+ .ctl_name = ADAPTIVE_HISTORY,
+ .procname = "adaptive_history",
+ .data = &adaptive_timeout_history,
+ .maxlen = sizeof(int),
+ .mode = 0644,
+ .proc_handler = &proc_dointvec
+ },
{ 0 }
};
"adaptive timeouts off, using obd_timeout %u\n",
obd_timeout);
- rc += snprintf(page + rc, count - rc,
- "%-10s : %ld sec\n", "timebase",
- imp->imp_at.iat_net_latency.at_binlimit * AT_BINS);
-
now = cfs_time_current_sec();
/* Some network health info for kicks */
return rc;
}
-int lprocfs_wr_timeouts(struct file *file, const char *buffer,
- unsigned long count, void *data)
-{
- struct obd_device *obd = (struct obd_device *)data;
- struct obd_import *imp;
- time_t bval;
- int val, i, rc;
-
- LASSERT(obd != NULL);
- LPROCFS_CLIMP_CHECK(obd);
- imp = obd->u.cli.cl_import;
-
- rc = lprocfs_write_helper(buffer, count, &val);
- if (rc < 0)
- return rc;
- if (val <= 0)
- return -ERANGE;
-
- bval = max(1, val / AT_BINS);
- spin_lock(&imp->imp_at.iat_net_latency.at_lock);
- imp->imp_at.iat_net_latency.at_binlimit = bval;
- spin_unlock(&imp->imp_at.iat_net_latency.at_lock);
- for(i = 0; i < IMP_AT_MAX_PORTALS; i++) {
- spin_lock(&imp->imp_at.iat_service_estimate[i].at_lock);
- imp->imp_at.iat_service_estimate[i].at_binlimit = bval;
- spin_unlock(&imp->imp_at.iat_service_estimate[i].at_lock);
- }
-
- LPROCFS_CLIMP_EXIT(obd);
- return count;
-}
-
static const char *obd_connect_names[] = {
"read_only",
"lov_index",
EXPORT_SYMBOL(lprocfs_rd_numrefs);
EXPORT_SYMBOL(lprocfs_at_hist_helper);
EXPORT_SYMBOL(lprocfs_rd_timeouts);
-EXPORT_SYMBOL(lprocfs_wr_timeouts);
EXPORT_SYMBOL(lprocfs_rd_blksize);
EXPORT_SYMBOL(lprocfs_rd_kbytestotal);
EXPORT_SYMBOL(lprocfs_rd_kbytesfree);
{ "prealloc_next_id", osc_rd_prealloc_next_id, 0, 0 },
{ "prealloc_last_id", osc_rd_prealloc_last_id, 0, 0 },
{ "checksums", osc_rd_checksum, osc_wr_checksum, 0 },
- { "timeouts", lprocfs_rd_timeouts, lprocfs_wr_timeouts, 0 },
+ { "timeouts", lprocfs_rd_timeouts, 0, 0 },
{ 0 }
};
if (lustre_msg_get_flags(req->rq_reqmsg) & MSG_LAST_REPLAY) {
if (obd && obd->obd_recovering) {
DEBUG_REQ(D_HA, req, "LAST_REPLAY, queuing reply");
- return target_queue_final_reply(req, rc);
+ return target_queue_last_replay_reply(req, rc);
}
/* Lost a race with recovery; let the error path DTRT. */
rc = req->rq_status = -ENOTCONN;
}
/* Set server timelimit for this req */
-static void ptlrpc_at_set_req_timeout(struct ptlrpc_request *req) {
+static void ptlrpc_at_set_req_timeout(struct ptlrpc_request *req)
+{
__u32 serv_est;
int idx;
struct imp_at *at;
+
LASSERT(req->rq_import);
- if (AT_OFF ||
- ((idx = import_at_get_index(req->rq_import,
- req->rq_request_portal)) < 0)) {
+ if (AT_OFF) {
/* non-AT settings */
req->rq_timeout = req->rq_import->imp_server_timeout ?
obd_timeout / 2 : obd_timeout;
- goto out;
+ lustre_msg_set_timeout(req->rq_reqmsg, req->rq_timeout);
+ return;
}
at = &req->rq_import->imp_at;
+ idx = import_at_get_index(req->rq_import,
+ req->rq_request_portal);
serv_est = at_get(&at->iat_service_estimate[idx]);
/* add an arbitrary minimum: 125% +5 sec */
req->rq_timeout = serv_est + (serv_est >> 2) + 5;
req->rq_timeout);
}
-out:
/* Let the server know what this RPC timeout is by putting it in the
reqmsg*/
lustre_msg_set_timeout(req->rq_reqmsg, req->rq_timeout);
}
/* Adjust max service estimate based on server value */
-static void ptlrpc_at_adj_service(struct ptlrpc_request *req) {
+static void ptlrpc_at_adj_service(struct ptlrpc_request *req)
+{
int idx;
unsigned int serv_est, oldse;
struct imp_at *at = &req->rq_import->imp_at;
+
LASSERT(req->rq_import);
/* service estimate is returned in the repmsg timeout field,
serv_est = lustre_msg_get_timeout(req->rq_repmsg);
idx = import_at_get_index(req->rq_import, req->rq_request_portal);
- if (idx < 0)
- return;
-
- oldse = at_get(&at->iat_service_estimate[idx]);
/* max service estimates are tracked on the server side,
so just keep minimal history here */
- at_add(&at->iat_service_estimate[idx], serv_est);
-
- if (at_get(&at->iat_service_estimate[idx]) != oldse)
+ oldse = at_add(&at->iat_service_estimate[idx], serv_est);
+ if (oldse != 0)
CDEBUG(D_ADAPTTO, "The RPC service estimate for %s ptl %d "
- "has changed %ds to %u\n",
- req->rq_import->imp_obd->obd_name, req->rq_request_portal,
- at_get(&at->iat_service_estimate[idx]) - oldse, serv_est);
+ "has changed from %d to %d\n",
+ req->rq_import->imp_obd->obd_name,req->rq_request_portal,
+ oldse, at_get(&at->iat_service_estimate[idx]));
}
/* Expected network latency per remote node (secs) */
-int ptlrpc_at_get_net_latency(struct ptlrpc_request *req) {
+int ptlrpc_at_get_net_latency(struct ptlrpc_request *req)
+{
return AT_OFF ? 0 : at_get(&req->rq_import->imp_at.iat_net_latency);
}
/* Adjust expected network latency */
-static void ptlrpc_at_adj_net_latency(struct ptlrpc_request *req) {
+static void ptlrpc_at_adj_net_latency(struct ptlrpc_request *req)
+{
unsigned int st, nl, oldnl;
struct imp_at *at = &req->rq_import->imp_at;
time_t now = cfs_time_current_sec();
+
LASSERT(req->rq_import);
st = lustre_msg_get_service_time(req->rq_repmsg);
CERROR("Reported service time %u > total measured time %ld\n",
st, now - req->rq_sent);
- oldnl = at_get(&at->iat_net_latency);
- at_add(&at->iat_net_latency, nl);
-
- if (at_get(&at->iat_net_latency) != oldnl)
- CDEBUG(D_ADAPTTO, "The network latency for %s (nid %s)"
- " has changed %ds to %u\n",
+ oldnl = at_add(&at->iat_net_latency, nl);
+ if (oldnl != 0)
+ CDEBUG(D_ADAPTTO, "The network latency for %s (nid %s) "
+ "has changed from %d to %d\n",
req->rq_import->imp_obd->obd_name,
obd_uuid2str(
&req->rq_import->imp_connection->c_remote_uuid),
- at_get(&at->iat_net_latency) - oldnl, nl);
+ oldnl, at_get(&at->iat_net_latency));
}
-static int unpack_reply(struct ptlrpc_request *req) {
+static int unpack_reply(struct ptlrpc_request *req)
+{
int rc;
/* Clear reply swab mask; we may have already swabbed an early reply */
/* Handle an early reply message.
We can't risk the real reply coming in and changing rq_repmsg,
so this fn must be called under the rq_lock */
-static int ptlrpc_at_early_reply(struct ptlrpc_request *req) {
+static int ptlrpc_at_recv_early_reply(struct ptlrpc_request *req) {
struct lustre_msg *oldmsg, *msgcpy;
time_t olddl;
int oldlen, rc;
GOTO(out, rc = 1);
if (req->rq_early) {
- ptlrpc_at_early_reply(req);
+ ptlrpc_at_recv_early_reply(req);
GOTO(out, rc = 0); /* keep waiting */
}
force_timer_recalc = 1;
}
+ spin_lock(&req->rq_lock);
+
/* Still waiting for a reply? */
- if (ptlrpc_client_receiving_reply(req))
+ if (req->rq_receiving_reply) {
+ spin_unlock(&req->rq_lock);
continue;
+ }
- spin_lock(&req->rq_lock);
if (req->rq_early) {
- ptlrpc_at_early_reply(req);
+ ptlrpc_at_recv_early_reply(req);
spin_unlock(&req->rq_lock);
continue;
}
- spin_unlock(&req->rq_lock);
/* Did we actually receive a reply? */
- if (!ptlrpc_client_replied(req))
+ if (!req->rq_replied) {
+ spin_unlock(&req->rq_lock);
continue;
+ }
+
+ spin_unlock(&req->rq_lock);
spin_lock(&imp->imp_lock);
list_del_init(&req->rq_list);
req = list_entry(tmp, struct ptlrpc_request, rq_set_chain);
/* request in-flight? */
- if (!((req->rq_phase == RQ_PHASE_RPC && !req->rq_waiting) ||
+ if (!(((req->rq_phase == RQ_PHASE_RPC) && !req->rq_waiting) ||
(req->rq_phase == RQ_PHASE_BULK)))
continue;
if (req->rq_deadline <= now) { /* actually expired already */
timeout = 1; /* ASAP */
break;
- } else if (timeout == 0 || timeout > req->rq_deadline - now) {
+ }
+
+ if ((timeout == 0) || (timeout > (req->rq_deadline - now))) {
timeout = req->rq_deadline - now;
}
}
ENTRY;
LASSERT(!in_interrupt ()); /* might sleep */
-
- if (!ptlrpc_client_receiving_reply(request) &&
- !ptlrpc_client_must_unlink(request))
+ if (!ptlrpc_client_recv_or_unlink(request))
/* Nothing left to do */
return;
for (;;) {
/* Network access will complete in finite time but the HUGE
* timeout lets us CWARN for visibility of sluggish NALs */
- lwi = LWI_TIMEOUT(cfs_time_seconds(FOREVER), NULL, NULL);
- rc = l_wait_event (*wq,
- !ptlrpc_client_receiving_reply(request) &&
- !ptlrpc_client_must_unlink(request),
+ lwi = LWI_TIMEOUT(cfs_time_seconds(LONG_UNLINK), NULL, NULL);
+ rc = l_wait_event (*wq, !ptlrpc_client_recv_or_unlink(request),
&lwi);
if (rc == 0)
return;
LASSERT (rc == -ETIMEDOUT);
DEBUG_REQ(D_WARNING, request, "Unexpectedly long timeout "
- "rvcng=%d unlnk=%d",
- ptlrpc_client_receiving_reply(request),
- ptlrpc_client_must_unlink(request));
+ "rvcng=%d unlnk=%d", request->rq_receiving_reply,
+ request->rq_must_unlink);
}
EXIT;
}
lustre_msg_get_status(req->rq_reqmsg), req->rq_xid,
libcfs_nid2str(imp->imp_connection->c_peer.nid),
lustre_msg_get_opc(req->rq_reqmsg));
+
spin_lock(&imp->imp_lock);
list_del_init(&req->rq_list);
spin_unlock(&imp->imp_lock);
out:
if (req->rq_bulk != NULL) {
- while(rc >= 0) {
+ if (rc >= 0) {
/* success so far. Note that anything going wrong
* with bulk now, is EXTREMELY strange, since the
* server must have believed that the bulk
* tranferred OK before she replied with success to
* me. */
- timeoutl = req->rq_deadline - cfs_time_current_sec();
- timeout = (timeoutl <= 0) ? CFS_TICK :
- cfs_time_seconds(timeoutl);
-
lwi = LWI_TIMEOUT(timeout, NULL, NULL);
brc = l_wait_event(req->rq_reply_waitq,
!ptlrpc_bulk_active(req->rq_bulk),
&lwi);
LASSERT(brc == 0 || brc == -ETIMEDOUT);
- if ((brc == -ETIMEDOUT) &&
- (req->rq_deadline > cfs_time_current_sec()))
- /* Wait again if we changed deadline */
- continue;
if (brc != 0) {
+ LASSERT(brc == -ETIMEDOUT);
DEBUG_REQ(D_ERROR, req, "bulk timed out");
rc = brc;
} else if (!req->rq_bulk->bd_success) {
DEBUG_REQ(D_ERROR, req, "bulk transfer failed");
rc = -EIO;
}
- break;
}
if (rc < 0)
ptlrpc_unregister_bulk (req);
LASSERT(ev->type == LNET_EVENT_PUT || ev->type == LNET_EVENT_UNLINK);
LASSERT(ev->md.start == req->rq_repbuf);
LASSERT(ev->mlength <= req->rq_replen);
+ /* We've set LNET_MD_MANAGE_REMOTE for all outgoing requests
+ for adaptive timeouts' early reply. */
+ LASSERT((ev->md.options & LNET_MD_MANAGE_REMOTE) != 0);
spin_lock(&req->rq_lock);
goto out_wake;
}
- /* We've set LNET_MD_MANAGE_REMOTE for all outgoing requests
- for adaptive timeouts' early reply. */
- LASSERT(ev->md.options & LNET_MD_MANAGE_REMOTE);
-
if ((ev->offset == 0) &&
- lustre_msg_get_flags(req->rq_reqmsg) & MSG_AT_SUPPORT) {
+ ((lustre_msg_get_flags(req->rq_reqmsg) & MSG_AT_SUPPORT) != 0)) {
/* Early reply */
DEBUG_REQ(D_ADAPTTO, req,
"Early reply received: mlen=%u offset=%d replen=%d "
req->rq_uid = ev->uid;
#endif
spin_lock_init(&req->rq_lock);
- sema_init(&req->rq_rs_sem, 1);
CFS_INIT_LIST_HEAD(&req->rq_timed_list);
atomic_set(&req->rq_refcount, 1);
if (ev->type == LNET_EVENT_PUT)
timeout = (int)(last - cfs_time_current_sec());
if (timeout > 0) {
lwi = LWI_TIMEOUT_INTERVAL(cfs_time_seconds(timeout),
- HZ, NULL, NULL);
+ cfs_time_seconds(1), NULL, NULL);
rc = l_wait_event(imp->imp_recovery_waitq,
(atomic_read(&imp->imp_inflight) == 0),
&lwi);
if (ptlrpc_import_in_recovery(imp)) {
struct l_wait_info lwi;
cfs_duration_t timeout;
- int idx;
- if (AT_OFF || (idx = import_at_get_index(imp,
- imp->imp_client->cli_request_portal)) < 0)
+ if (AT_OFF) {
timeout = cfs_time_seconds(obd_timeout);
- else
+ } else {
+ int idx = import_at_get_index(imp,
+ imp->imp_client->cli_request_portal);
timeout = cfs_time_seconds(
at_get(&imp->imp_at.iat_service_estimate[idx]));
+ }
lwi = LWI_TIMEOUT_INTR(cfs_timeout_cap(timeout),
back_to_sleep, LWI_ON_SIGNAL_NOOP, NULL);
rc = l_wait_event(imp->imp_recovery_waitq,
This gives us a max of the last binlimit*AT_BINS secs without the storage,
but still smoothing out a return to normalcy from a slow response.
(E.g. remember the maximum latency in each minute of the last 4 minutes.) */
-void at_add(struct adaptive_timeout *at, unsigned int val) {
- /*unsigned int old = at->at_current;*/
+int at_add(struct adaptive_timeout *at, unsigned int val)
+{
+ unsigned int old = at->at_current;
time_t now = cfs_time_current_sec();
+ time_t binlimit = max_t(time_t, adaptive_timeout_history / AT_BINS, 1);
LASSERT(at);
#if 0
- CDEBUG(D_INFO, "add %u to %p time=%lu tb=%lu v=%u (%u %u %u %u)\n",
- val, at, now - at->at_binstart, at->at_binlimit, at->at_current,
+ CDEBUG(D_INFO, "add %u to %p time=%lu v=%u (%u %u %u %u)\n",
+ val, at, now - at->at_binstart, at->at_current,
at->at_hist[0], at->at_hist[1], at->at_hist[2], at->at_hist[3]);
#endif
if (val == 0)
/* 0's don't count, because we never want our timeout to
drop to 0, and because 0 could mean an error */
- return;
+ return 0;
spin_lock(&at->at_lock);
at->at_worst_time = now;
at->at_hist[0] = val;
at->at_binstart = now;
- } else if (now - at->at_binstart < at->at_binlimit ) {
+ } else if (now - at->at_binstart < binlimit ) {
/* in bin 0 */
at->at_hist[0] = max(val, at->at_hist[0]);
at->at_current = max(val, at->at_current);
int i, shift;
unsigned int maxv = val;
/* move bins over */
- shift = (now - at->at_binstart) / at->at_binlimit;
+ shift = (now - at->at_binstart) / binlimit;
LASSERT(shift > 0);
for(i = AT_BINS - 1; i >= 0; i--) {
if (i >= shift) {
}
at->at_hist[0] = val;
at->at_current = maxv;
- at->at_binstart += shift * at->at_binlimit;
+ at->at_binstart += shift * binlimit;
}
if ((at->at_flags & AT_FLG_MIN) &&
at->at_hist[0], at->at_hist[1], at->at_hist[2],
at->at_hist[3]);
#endif
+
+ /* if we changed, report the old value */
+ old = (at->at_current != old) ? old : 0;
+
spin_unlock(&at->at_lock);
+ return old;
}
/* Find the imp_at index for a given portal; assign if space available */
-int import_at_get_index(struct obd_import *imp, int portal) {
+int import_at_get_index(struct obd_import *imp, int portal)
+{
struct imp_at *at = &imp->imp_at;
int i;
/* unused */
break;
}
+
+ /* Not enough portals? */
+ LASSERT(i < IMP_AT_MAX_PORTALS);
- if (i >= IMP_AT_MAX_PORTALS) {
- CERROR("Tried to use more than %d portals, not enough room "
- "in adaptive timeout stats.\n", IMP_AT_MAX_PORTALS);
- i = -1;
- goto out;
- }
at->iat_portal[i] = portal;
-
out:
spin_unlock(&imp->imp_lock);
return i;
Since any early reply will only affect the RPC wait time, and not
any local lock timer we set based on the return value here,
we should be conservative. */
-int import_at_get_ldlm(struct obd_import *imp) {
+int import_at_get_ldlm(struct obd_import *imp)
+{
int idx, tot;
if (!imp || !imp->imp_client || AT_OFF)
return obd_timeout;
- tot = at_get(&imp->imp_at.iat_net_latency);
idx = import_at_get_index(imp, imp->imp_client->cli_request_portal);
- if (idx < 0)
- tot += obd_timeout;
- else
- tot += at_get(&imp->imp_at.iat_service_estimate[idx]);
+ tot = at_get(&imp->imp_at.iat_net_latency) +
+ at_get(&imp->imp_at.iat_service_estimate[idx]);
/* add an arbitrary minimum: 150% + 10 sec */
tot += (tot >> 1) + 10;
"adaptive timeouts off, using obd_timeout %u\n",
obd_timeout);
rc += snprintf(page + rc, count - rc,
- "%10s : %ld sec\n", "timebase",
- svc->srv_at_estimate.at_binlimit * AT_BINS);
- rc += snprintf(page + rc, count - rc,
"%10s : cur %3u worst %3u (at %ld, "DHMS_FMT" ago) ",
"service", cur, worst, worstt,
DHMS_VARS(&ts));
return rc;
}
-static int ptlrpc_lprocfs_wr_timeouts(struct file *file, const char *buffer,
- unsigned long count, void *data)
-{
- struct ptlrpc_service *svc = data;
- int val, rc;
-
- rc = lprocfs_write_helper(buffer, count, &val);
- if (rc < 0)
- return rc;
- if (val <= 0)
- return -ERANGE;
-
- spin_lock(&svc->srv_at_estimate.at_lock);
- svc->srv_at_estimate.at_binlimit = max(1, val / AT_BINS);
- spin_unlock(&svc->srv_at_estimate.at_lock);
-
- return count;
-}
-
void ptlrpc_lprocfs_register_service(struct proc_dir_entry *entry,
struct ptlrpc_service *svc)
{
.read_fptr = ptlrpc_lprocfs_read_req_history_max,
.data = svc},
{.name = "timeouts",
- .write_fptr = ptlrpc_lprocfs_wr_timeouts,
.read_fptr = ptlrpc_lprocfs_rd_timeouts,
.data = svc},
{NULL}
req->rq_arrival_time.tv_sec, 1);
if (!(flags & PTLRPC_REPLY_EARLY) &&
(req->rq_type != PTL_RPC_MSG_ERR)) {
- int oldse = at_get(&svc->srv_at_estimate);
/* early replies and errors don't count toward our service
time estimate */
- at_add(&svc->srv_at_estimate, service_time);
- if (service_time > oldse)
+ int oldse = at_add(&svc->srv_at_estimate, service_time);
+ if (oldse != 0)
DEBUG_REQ(D_ADAPTTO, req,
- "svc %s increased estimate from %d to %d",
- svc->srv_name, oldse, service_time);
+ "svc %s changed estimate from %d to %d",
+ svc->srv_name, oldse,
+ at_get(&svc->srv_at_estimate));
}
/* Report actual service time for client latency calc */
lustre_msg_set_service_time(req->rq_repmsg, service_time);
int size;
ENTRY;
- /* Insure only 1 reply at a time: hold sem until ptlrpc_req_drop_rs */
- down(&req->rq_rs_sem);
LASSERT(req->rq_reply_state == NULL);
- if (req->rq_final && (flags & LPRFL_EARLY_REPLY)) {
+ if (req->rq_packed_final)
/* Already packed final, no more early */
- up(&req->rq_rs_sem);
RETURN(-EALREADY);
- }
+ if ((flags & LPRFL_EARLY_REPLY) == 0)
+ req->rq_packed_final = 1;
msg_len = lustre_msg_size_v1(count, lens);
size = sizeof(struct ptlrpc_reply_state) + msg_len;
OBD_ALLOC(rs, size);
if (unlikely(rs == NULL)) {
rs = lustre_get_emerg_rs(req->rq_rqbd->rqbd_service, size);
- if (!rs) {
- up(&req->rq_rs_sem);
+ if (!rs)
RETURN (-ENOMEM);
- }
}
atomic_set(&rs->rs_refcount, 1); /* 1 ref for rq_reply_state */
rs->rs_cb_id.cbid_fn = reply_out_callback;
req->rq_replen = msg_len;
req->rq_reply_state = rs;
req->rq_repmsg = rs->rs_msg;
- if (!flags)
- req->rq_final = 1; /* checked in lustre_packed_reply */
lustre_init_msg_v1(rs->rs_msg, count, lens, bufs);
int size;
ENTRY;
- /* Insure only 1 reply at a time: hold sem until ptlrpc_req_drop_rs */
- down(&req->rq_rs_sem);
LASSERT(req->rq_reply_state == NULL);
- if (req->rq_final && (flags & LPRFL_EARLY_REPLY)) {
+ if (req->rq_packed_final) {
/* Already packed final, no more early */
- up(&req->rq_rs_sem);
RETURN(-EALREADY);
}
+ if ((flags & LPRFL_EARLY_REPLY) == 0)
+ req->rq_packed_final = 1;
msg_len = lustre_msg_size_v2(count, lens);
size = sizeof(struct ptlrpc_reply_state) + msg_len;
OBD_ALLOC(rs, size);
if (unlikely(rs == NULL)) {
rs = lustre_get_emerg_rs(req->rq_rqbd->rqbd_service, size);
- if (!rs) {
- up(&req->rq_rs_sem);
+ if (!rs)
RETURN (-ENOMEM);
- }
}
atomic_set(&rs->rs_refcount, 1); /* 1 ref for rq_reply_state */
rs->rs_cb_id.cbid_fn = reply_out_callback;
req->rq_replen = msg_len;
req->rq_reply_state = rs;
req->rq_repmsg = rs->rs_msg;
- if (!(flags & LPRFL_EARLY_REPLY))
- req->rq_final = 1; /* checked in lustre_packed_reply */
/* server side, no rq_repbuf */
lustre_init_msg_v2(rs->rs_msg, count, lens, bufs);
case LUSTRE_MSG_MAGIC_V2_SWABBED: {
struct ptlrpc_body *pb;
- pb = lustre_msg_buf(msg, MSG_PTLRPC_BODY_OFF, sizeof(*pb));
+ pb = lustre_msg_buf_v2(msg, MSG_PTLRPC_BODY_OFF, sizeof(*pb));
if (!pb) {
CERROR("invalid msg %p: no ptlrpc body!\n", msg);
return 0;
case LUSTRE_MSG_MAGIC_V2_SWABBED: {
struct ptlrpc_body *pb;
- pb = lustre_msg_buf(msg, MSG_PTLRPC_BODY_OFF, sizeof(*pb));
+ pb = lustre_msg_buf_v2(msg, MSG_PTLRPC_BODY_OFF, sizeof(*pb));
if (!pb) {
CERROR("invalid msg %p: no ptlrpc body!\n", msg);
return 0;
CFS_MODULE_PARM(test_req_buffer_pressure, "i", int, 0444,
"set non-zero to put pressure on request buffer pools");
+static int at_early_margin = 3;
+CFS_MODULE_PARM(at_early_margin, "i", int, 0644,
+ "How far before the deadline we send an early reply");
+
+static int at_extra = 10;
+CFS_MODULE_PARM(at_extra, "i", int, 0644,
+ "How much extra time we give with an early reply");
+
/* forward ref */
static int ptlrpc_server_post_idle_rqbds (struct ptlrpc_service *svc);
cfs_timer_init(&service->srv_at_timer, ptlrpc_at_timer, service);
/* At SOW, service time should be quick; 10s seems generous. If client
timeout is less than this, we'll be sending an early reply. */
- at_init(&service->srv_at_estimate, 10, AT_TIMEBASE_DEFAULT, 0);
+ at_init(&service->srv_at_estimate, 10, 0);
spin_lock (&ptlrpc_all_services_lock);
list_add (&service->srv_list, &ptlrpc_all_services);
return NULL;
}
-static void ptlrpc_server_decref(struct ptlrpc_request *req)
+static void ptlrpc_server_req_decref(struct ptlrpc_request *req)
{
struct ptlrpc_request_buffer_desc *rqbd = req->rq_rqbd;
{
list_del(&req->rq_list);
ptlrpc_req_drop_rs(req);
- ptlrpc_server_decref(req);
+ ptlrpc_server_req_decref(req);
}
static void ptlrpc_server_free_request(struct ptlrpc_request *req)
struct list_head *tmp;
struct list_head *nxt;
- DEBUG_REQ(D_INFO, req, "free req");
+ if (req->rq_phase != RQ_PHASE_NEW) /* incorrect message magic */
+ DEBUG_REQ(D_INFO, req, "free req");
spin_lock(&svc->srv_at_lock);
list_del_init(&req->rq_timed_list);
spin_unlock(&svc->srv_at_lock);
return 0;
}
-/* If closest exipiration is within EARLY_MIN, send early replies to everybody
- expiring within EARLY_MAX, asking for AT_EXTRA time */
-#define AT_EARLY_MIN 2 /* Min time needed to send an early reply */
-#define AT_EARLY_MAX 5 /* Dont send early replies if deadline is beyond */
-#define AT_EXTRA 10 /* Early replies add this time to client timeout */
-
static void ptlrpc_at_set_timer(struct ptlrpc_service *svc)
{
struct ptlrpc_request *rq;
/* Set timer for closest deadline */
rq = list_entry(svc->srv_at_list.next, struct ptlrpc_request,
rq_timed_list);
- next = rq->rq_deadline - cfs_time_current_sec() - AT_EARLY_MIN;
+ next = rq->rq_deadline - cfs_time_current_sec() - at_early_margin;
if (next <= 0)
ptlrpc_at_timer((unsigned long)svc);
else
cfs_timer_arm(&svc->srv_at_timer, cfs_time_shift(next));
spin_unlock(&svc->srv_at_lock);
- CDEBUG(D_ADAPTTO, "armed %s at %+lds\n", svc->srv_name, next);
+ CDEBUG(D_INFO, "armed %s at %+lds\n", svc->srv_name, next);
}
/* Add rpc to early reply check list */
int extra_time)
{
struct ptlrpc_service *svc = req->rq_rqbd->rqbd_service;
+ struct ptlrpc_request *reqcopy;
+ struct lustre_msg *reqmsg;
long deadline = req->rq_deadline - cfs_time_current_sec();
int rc;
ENTRY;
if (AT_OFF)
RETURN(0);
+ if (deadline < 0) {
+ CERROR("Already past deadline (%+lds), not sending early "
+ "reply\n", deadline);
+ /* Return an error so we're not re-added to the timed list. */
+ RETURN(-ETIMEDOUT);
+ }
+
if ((lustre_msg_get_flags(req->rq_reqmsg) & MSG_AT_SUPPORT) == 0) {
CERROR("Wanted to ask client for more time, but no AT "
"support\n");
RETURN(-ENOSYS);
}
- rc = lustre_pack_reply_flags(req, 1, NULL, NULL, LPRFL_EARLY_REPLY);
+ OBD_ALLOC(reqcopy, sizeof *reqcopy);
+ if (reqcopy == NULL)
+ RETURN(-ENOMEM);
+ OBD_ALLOC(reqmsg, req->rq_reqlen);
+ if (!reqmsg) {
+ OBD_FREE(reqcopy, sizeof *reqcopy);
+ RETURN(-ENOMEM);
+ }
+
+ *reqcopy = *req;
+ /* We need the reqmsg for the magic */
+ reqcopy->rq_reqmsg = reqmsg;
+ memcpy(reqmsg, req->rq_reqmsg, req->rq_reqlen);
+
+ rc = lustre_pack_reply_flags(reqcopy, 1, NULL, NULL, LPRFL_EARLY_REPLY);
if (rc)
- /* EALREADY means final reply was already packed */
- RETURN(rc);
+ GOTO(out, rc);
if (extra_time) {
/* Fake our processing time into the future to ask the
clients for some extra amount of time */
extra_time += cfs_time_current_sec() -
- req->rq_arrival_time.tv_sec;
+ reqcopy->rq_arrival_time.tv_sec;
at_add(&svc->srv_at_estimate, extra_time);
}
- req->rq_early_count++; /* number sent, server side */
- rc = ptlrpc_send_reply(req, PTLRPC_REPLY_EARLY);
+ rc = ptlrpc_send_reply(reqcopy, PTLRPC_REPLY_EARLY);
if (!rc) {
/* Adjust our own deadline to what we told the client */
req->rq_deadline = req->rq_arrival_time.tv_sec +
at_get(&svc->srv_at_estimate);
+ req->rq_early_count++; /* number sent, server side */
} else {
DEBUG_REQ(D_ERROR, req, "Early reply send failed %d", rc);
}
- /* Reset reply */
- req->rq_rep_swab_mask = 0;
/* Free the (early) reply state from lustre_pack_reply.
(ptlrpc_send_reply takes it's own rs ref, so this is safe here) */
- ptlrpc_req_drop_rs(req);
-
+ ptlrpc_req_drop_rs(reqcopy);
+out:
+ OBD_FREE(reqmsg, req->rq_reqlen);
+ OBD_FREE(reqcopy, sizeof *reqcopy);
RETURN(rc);
}
-/* Check if we need to send any early replies, and send them */
+/* Send early replies to everybody expiring within at_early_margin
+ asking for at_extra time */
static int ptlrpc_at_check_timed(struct ptlrpc_service *svc)
{
struct ptlrpc_request *rq, *n;
rq = list_entry(svc->srv_at_list.next, struct ptlrpc_request,
rq_timed_list);
first = (int)(rq->rq_deadline - now);
- if (first > AT_EARLY_MIN) {
+ if (first > at_early_margin) {
/* We've still got plenty of time. Reset the timer. */
spin_unlock(&svc->srv_at_lock);
ptlrpc_at_set_timer(svc);
server will take. Send early replies to everyone expiring soon. */
CFS_INIT_LIST_HEAD(&work_list);
list_for_each_entry_safe(rq, n, &svc->srv_at_list, rq_timed_list) {
- if (rq->rq_deadline <= now + AT_EARLY_MAX) {
+ if (rq->rq_deadline <= now + at_early_margin) {
list_move(&rq->rq_timed_list, &work_list);
counter++;
} else {
spin_unlock(&svc->srv_at_lock);
CDEBUG(D_ADAPTTO, "timeout in %+ds, asking for %d secs on %d early "
- "replies\n", first, AT_EXTRA, counter);
+ "replies\n", first, at_extra, counter);
if (first < 0)
/* We're already past request deadlines before we even get a
chance to send early replies */
deleted, and is safe to take a ref to keep the req around */
atomic_inc(&rq->rq_refcount);
spin_unlock(&svc->srv_at_lock);
- if (!rq->rq_final &&
- (ptlrpc_at_send_early_reply(rq, AT_EXTRA) == 0)) {
+ if (!rq->rq_packed_final &&
+ (ptlrpc_at_send_early_reply(rq, at_extra) == 0)) {
counter++;
ptlrpc_at_add_timed(rq);
- } else if (rq->rq_final) {
- DEBUG_REQ(D_ADAPTTO, rq, "already packed final reply, "
- "not sending early");
}
- ptlrpc_server_decref(rq);
+ ptlrpc_server_req_decref(rq);
spin_lock(&svc->srv_at_lock);
}
spin_unlock(&svc->srv_at_lock);
RETURN(0);
}
-/* Handle freshly incoming reqs, check timeout, send early reply if needed,
+/* Handle freshly incoming reqs, add to timed early reply list,
pass on to regular request queue */
static int
ptlrpc_server_handle_req_in(struct ptlrpc_service *svc)
export = class_export_rpc_get(request->rq_export);
}
+ /* Discard requests queued for longer than the deadline.
+ The deadline is increased if we send an early reply. */
+ if (cfs_time_current_sec() > request->rq_deadline) {
+ CERROR("Dropping timed-out opc %d request from %s"
+ ": deadline %lds ago\n",
+ lustre_msg_get_opc(request->rq_reqmsg),
+ libcfs_id2str(request->rq_peer),
+ cfs_time_current_sec() - request->rq_deadline);
+ goto put_rpc_export;
+ }
+
request->rq_phase = RQ_PHASE_INTERPRET;
CDEBUG(D_RPCTRACE, "Handling RPC pname:cluuid+ref:pid:xid:nid:opc "
libcfs_id2str(request->rq_peer),
lustre_msg_get_opc(request->rq_reqmsg));
+put_rpc_export:
if (export != NULL)
class_export_rpc_put(export);
#ifdef WITH_GROUP_INFO
struct group_info *ginfo = NULL;
#endif
- int counter, rc = 0;
+ int counter = 0, rc = 0;
ENTRY;
ptlrpc_daemonize(data->name);
if (!list_empty(&svc->srv_reply_queue))
ptlrpc_server_handle_reply (svc);
- counter = 0;
- while(!list_empty(&svc->srv_req_in_queue)) {
+ if (!list_empty(&svc->srv_req_in_queue)) {
/* Process all incoming reqs before handling any */
ptlrpc_server_handle_req_in(svc);
/* but limit ourselves in case of flood */
- if (counter++ > 1000)
- break;
+ if (counter++ < 1000)
+ continue;
+ counter = 0;
}
if (svc->srv_at_check)
/* Network access will complete in finite time but the HUGE
* timeout lets us CWARN for visibility of sluggish NALs */
- lwi = LWI_TIMEOUT(cfs_time_seconds(FOREVER), NULL, NULL);
+ lwi = LWI_TIMEOUT(cfs_time_seconds(LONG_UNLINK), NULL, NULL);
rc = l_wait_event(service->srv_waitq,
service->srv_nrqbd_receiving == 0,
&lwi);
}
run_test 61c "test race mds llog sync vs llog cleanup"
-
+#Adaptive Timeouts
at_start() #bug 3055
{
if [ -z "$ATOLDBASE" ]; then
- ATOLDBASE=$(do_facet mds "grep timebase $LPROC/mdt/MDS/mds/timeouts" | awk '{print $3}' )
+ ATOLDBASE=$(do_facet mds "sysctl -n lustre.adaptive_history")
# speed up the timebase so we can check decreasing AT
- do_facet mds "echo 8 >> $LPROC/mdt/MDS/mds/timeouts"
- do_facet mds "echo 8 >> $LPROC/mdt/MDS/mds_readpage/timeouts"
- do_facet mds "echo 8 >> $LPROC/mdt/MDS/mds_setattr/timeouts"
- do_facet ost1 "echo 8 >> $LPROC/ost/OSS/ost/timeouts"
+ do_facet mds "sysctl -w lustre.adaptive_history=8"
+ do_facet ost1 "sysctl -w lustre.adaptive_history=8"
fi
}
at_start
$LCTL dk > /dev/null
# slow down a request
- sysctl -w lustre.fail_val=30000
+ do_facet mds sysctl -w lustre.fail_val=30000
#define OBD_FAIL_PTLRPC_PAUSE_REQ 0x50a
- sysctl -w lustre.fail_loc=0x8000050a
+ do_facet mds sysctl -w lustre.fail_loc=0x8000050a
createmany -o $DIR/$tfile 10 > /dev/null
unlinkmany $DIR/$tfile 10 > /dev/null
# check for log message
run_test 67b "AT: verify instant slowdown doesn't induce reconnects"
if [ -n "$ATOLDBASE" ]; then
- do_facet mds "echo $ATOLDBASE >> $LPROC/mdt/MDS/mds/timeouts"
- do_facet mds "echo $ATOLDBASE >> $LPROC/mdt/MDS/mds_readpage/timeouts"
- do_facet mds "echo $ATOLDBASE >> $LPROC/mdt/MDS/mds_setattr/timeouts"
- do_facet ost1 "echo $ATOLDBASE >> $LPROC/ost/OSS/ost/timeouts"
+ do_facet mds "sysctl -w lustre.adaptive_history=$ATOLDBASE"
+ do_facet ost1 "sysctl -w lustre.adaptive_history=$ATOLDBASE"
fi
# end of AT tests includes above lines