]> Zhao Yanbai Git Server - minix.git/commitdiff
VFS: make all IPC asynchronous
authorThomas Veerman <thomas@minix3.org>
Tue, 28 Aug 2012 14:06:51 +0000 (14:06 +0000)
committerThomas Veerman <thomas@minix3.org>
Mon, 17 Sep 2012 11:01:45 +0000 (11:01 +0000)
By decoupling synchronous drivers from VFS, we are a big step closer to
supporting driver crashes under all circumstances. That is, VFS can't
become stuck on IPC with a synchronous driver (e.g., INET) and can
recover from crashing block drivers during open/close/ioctl or during
communication with an FS.

In order to maintain serialized communication with a synchronous driver,
the communication is wrapped by a mutex on a per driver basis (not major
numbers as there can be multiple majors with identical endpoints). Majors
that share a driver endpoint point to a single mutex object.

In order to support crashes from block drivers, the file reopen tactic
had to be changed; first reopen files associated with the crashed
driver, then send the new driver endpoint to FSes. This solves a
deadlock between the FS and the block driver;
  - VFS would send REQ_NEW_DRIVER to an FS, but he FS only receives it
    after retrying the current request to the newly started driver.
  - The block driver would refuse the retried request until all files
    had been reopened.
  - VFS would reopen files only after getting a reply from the initial
    REQ_NEW_DRIVER.

When a character special driver crashes, all associated files have to
be marked invalid and closed (or reopened if flagged as such). However,
they can only be closed if a thread holds exclusive access to it. To
obtain exclusive access, the worker thread (which handles the new driver
endpoint event from DS) schedules a new job to garbage collect invalid
files. This way, we can signal the worker thread that was talking to the
crashed driver and will release exclusive access to a file associated
with the crashed driver and prevent the garbage collecting worker thread
from dead locking on that file.

Also, when a character special driver crashes, RS will unmap the driver
and remap it upon restart. During unmapping, associated files are marked
invalid instead of waiting for an endpoint up event from DS, as that
event might come later than new read/write/select requests and thus
cause confusion in the freshly started driver.

When locking a filp, the usage counters are no longer checked. The usage
counter can legally go down to zero during filp invalidation while there
are locks pending.

DS events are handled by a separate worker thread instead of the main
thread as reopening files could lead to another crash and a stuck thread.
An additional worker thread is then necessary to unlock it.

Finally, with everything asynchronous a race condition in do_select
surfaced. A select entry was only marked in use after succesfully sending
initial select requests to drivers and having to wait. When multiple
select() calls were handled there was opportunity that these entries
were overwritten. This had as effect that some select results were
ignored (and select() remained blocking instead if returning) or do_select
tried to access filps that were not present (because thrown away by
secondary select()). This bug manifested itself with sendrecs, but was
very hard to reproduce. However, it became awfully easy to trigger with
asynsends only.

17 files changed:
drivers/tty/tty.c
servers/vfs/comm.c
servers/vfs/const.h
servers/vfs/device.c
servers/vfs/dmap.c
servers/vfs/dmap.h
servers/vfs/file.h
servers/vfs/filedes.c
servers/vfs/fproc.h
servers/vfs/main.c
servers/vfs/misc.c
servers/vfs/open.c
servers/vfs/proto.h
servers/vfs/read.c
servers/vfs/select.c
servers/vfs/threads.h
servers/vfs/worker.c

index f0330462075ee976d8bf9209d5b46f7db6e6fbbe..a42c0b40f4b26fc474266292390b736b68162d29 100644 (file)
@@ -430,7 +430,7 @@ message *m_ptr;
   }
 
   /* Almost done. Send back the reply message to the caller. */
-  status = sendnb(m_ptr->m_source, m_ptr);
+  status = send(m_ptr->m_source, m_ptr);
   if (status != OK) {
        printf("tty`do_status: send to %d failed: %d\n",
                m_ptr->m_source, status);
@@ -1487,7 +1487,7 @@ int status;                       /* reply code */
        panic("tty_reply sending TTY_REVIVE");
   }
 
-  status = sendnb(replyee, &tty_mess);
+  status = send(replyee, &tty_mess);
   if (status != OK)
        printf("tty`tty_reply: send to %d failed: %d\n", replyee, status);
 }
index 2cf2ab54a33474917fdfdc694bfda9d2dbc8a963..41760827fc6d007377d228a088ddacffa23c0ef1 100644 (file)
@@ -5,29 +5,27 @@
 #include <minix/vfsif.h>
 #include <assert.h>
 
-static int sendmsg(struct vmnt *vmp, struct fproc *rfp);
+static int sendmsg(struct vmnt *vmp, struct worker_thread *wp);
 static int queuemsg(struct vmnt *vmp);
 
 /*===========================================================================*
  *                             sendmsg                                      *
  *===========================================================================*/
-static int sendmsg(vmp, rfp)
-struct vmnt *vmp;
-struct fproc *rfp;
+static int sendmsg(struct vmnt *vmp, struct worker_thread *wp)
 {
 /* This is the low level function that sends requests to FS processes.
  */
   int r, transid;
 
   vmp->m_comm.c_cur_reqs++;    /* One more request awaiting a reply */
-  transid = rfp->fp_wtid + VFS_TRANSID;
-  rfp->fp_sendrec->m_type = TRNS_ADD_ID(rfp->fp_sendrec->m_type, transid);
-  rfp->fp_task = vmp->m_fs_e;
-  if ((r = asynsend3(vmp->m_fs_e, rfp->fp_sendrec, AMF_NOREPLY)) != OK) {
+  transid = wp->w_tid + VFS_TRANSID;
+  wp->w_fs_sendrec->m_type = TRNS_ADD_ID(wp->w_fs_sendrec->m_type, transid);
+  wp->w_task = vmp->m_fs_e;
+  if ((r = asynsend3(vmp->m_fs_e, wp->w_fs_sendrec, AMF_NOREPLY)) != OK) {
        printf("VFS: sendmsg: error sending message. "
-              "FS_e: %d req_nr: %d err: %d\n", vmp->m_fs_e,
-              rfp->fp_sendrec->m_type, r);
-               util_stacktrace();
+               "FS_e: %d req_nr: %d err: %d\n", vmp->m_fs_e,
+               wp->w_fs_sendrec->m_type, r);
+       util_stacktrace();
        return(r);
   }
 
@@ -83,7 +81,41 @@ void fs_sendmore(struct vmnt *vmp)
   worker->w_next = NULL;
   sending--;
   assert(sending >= 0);
-  (void) sendmsg(vmp, worker->w_job.j_fp);
+  (void) sendmsg(vmp, worker);
+}
+
+/*===========================================================================*
+ *                             drv_sendrec                                  *
+ *===========================================================================*/
+int drv_sendrec(endpoint_t drv_e, message *reqmp)
+{
+       int r;
+       struct dmap *dp;
+
+       if ((dp = get_dmap(drv_e)) == NULL)
+               panic("driver endpoint %d invalid", drv_e);
+
+       lock_dmap(dp);
+       if (dp->dmap_servicing != NONE)
+               panic("driver locking inconsistency");
+       dp->dmap_servicing = self->w_tid;
+       self->w_task = drv_e;
+       self->w_drv_sendrec = reqmp;
+
+       if ((r = asynsend3(drv_e, self->w_drv_sendrec, AMF_NOREPLY)) == OK) {
+               /* Yield execution until we've received the reply */
+               worker_wait();
+       } else {
+               printf("VFS: drv_sendrec: error sending msg to driver %d: %d\n",
+                       drv_e, r);
+               util_stacktrace();
+       }
+
+       dp->dmap_servicing = NONE;
+       self->w_task = NONE;
+       self->w_drv_sendrec = NULL;
+       unlock_dmap(dp);
+       return(OK);
 }
 
 /*===========================================================================*
@@ -100,13 +132,13 @@ int fs_sendrec(endpoint_t fs_e, message *reqmp)
   }
   if (fs_e == fp->fp_endpoint) return(EDEADLK);
 
-  fp->fp_sendrec = reqmp;      /* Where to store request and reply */
+  self->w_fs_sendrec = reqmp;  /* Where to store request and reply */
 
   /* Find out whether we can send right away or have to enqueue */
   if ( !(vmp->m_flags & VMNT_CALLBACK) &&
        vmp->m_comm.c_cur_reqs < vmp->m_comm.c_max_reqs) {
        /* There's still room to send more and no proc is queued */
-       r = sendmsg(vmp, fp);
+       r = sendmsg(vmp, self);
   } else {
        r = queuemsg(vmp);
   }
index 44a339d7c511f613676efe74af1b3a44110964dc..6c2ff41cda3edb5a1133cef4631758b9e97afd79 100644 (file)
@@ -27,6 +27,8 @@
 /* test if the process is blocked on something */
 #define fp_is_blocked(fp)      ((fp)->fp_blocked_on != FP_BLOCKED_ON_NONE)
 
+/* test if reply is a driver reply */
+#define IS_DRV_REPLY(x)        (IS_DEV_RS(x) || IS_BDEV_RS(x) || (x) == TASK_REPLY)
 #define DUP_MASK        0100   /* mask to distinguish dup2 from dup */
 
 #define LOOK_UP            0 /* tells search_dir to lookup string */
index 3e3e507e105e38c1217c205e2564c0c8d4a4efd9..53ba6a203fd7bca531a44956c4f941c18b2cd818 100644 (file)
@@ -228,7 +228,7 @@ endpoint_t find_suspended_ep(endpoint_t driver, cp_grant_id_t g)
 /*===========================================================================*
  *                             dev_status                                   *
  *===========================================================================*/
-void dev_status(message *m)
+void dev_status(endpoint_t drv_e)
 {
 /* A device sent us a notification it has something for us. Retrieve it. */
 
@@ -237,14 +237,14 @@ void dev_status(message *m)
   endpoint_t endpt;
 
   for (major = 0; major < NR_DEVICES; major++)
-       if (dmap_driver_match(m->m_source, major))
+       if (dmap_driver_match(drv_e, major))
                break; /* 'major' is the device that sent the message */
 
   if (major >= NR_DEVICES)     /* Device endpoint not found; nothing to do */
        return;
 
   if (dev_style_asyn(dmap[major].dmap_style)) {
-       printf("VFS: not doing dev_status for async driver %d\n", m->m_source);
+       printf("VFS: not doing dev_status for async driver %d\n", drv_e);
        return;
   }
 
@@ -253,10 +253,10 @@ void dev_status(message *m)
   do {
        int r;
        st.m_type = DEV_STATUS;
-       r = sendrec(m->m_source, &st);
+       r = drv_sendrec(drv_e, &st);
        if (r == OK && st.REP_STATUS == ERESTART) r = EDEADEPT;
        if (r != OK) {
-               printf("VFS: DEV_STATUS failed to %d: %d\n", m->m_source, r);
+               printf("VFS: DEV_STATUS failed to %d: %d\n", drv_e, r);
                if (r == EDEADSRCDST || r == EDEADEPT) return;
                panic("VFS: couldn't sendrec for DEV_STATUS: %d", r);
        }
@@ -267,7 +267,7 @@ void dev_status(message *m)
                 * synchronous character driver */
                endpt = st.REP_ENDPT;
                if (endpt == VFS_PROC_NR) {
-                       endpt = find_suspended_ep(m->m_source, st.REP_IO_GRANT);
+                       endpt = find_suspended_ep(drv_e, st.REP_IO_GRANT);
                        if (endpt == NONE) {
                          printf("VFS: proc with grant %d from %d not found\n",
                                 st.REP_IO_GRANT, st.m_source);
@@ -673,7 +673,7 @@ int do_ioctl()
   }
 
   if (r == OK) {
-       suspend_reopen = (f->filp_state != FS_NORMAL);
+       suspend_reopen = (f->filp_state & FS_NEEDS_REOPEN);
        dev = (dev_t) vp->v_sdev;
 
        if (S_ISBLK(vp->v_mode))
@@ -699,21 +699,30 @@ message *mess_ptr;                /* pointer to message for task */
 /* All file system I/O ultimately comes down to I/O on major/minor device
  * pairs.  These lead to calls on the following routines via the dmap table.
  */
-  int r, status, proc_e = NONE, is_bdev;
+  int r, status, proc_e = NONE, is_bdev, retry_count;
+  message mess_retry;
 
   is_bdev = IS_BDEV_RQ(mess_ptr->m_type);
+  mess_retry = *mess_ptr;
+  retry_count = 0;
 
   if (!is_bdev) proc_e = mess_ptr->USER_ENDPT;
 
-  r = sendrec(driver_e, mess_ptr);
-  if (r == OK) {
-       if (is_bdev)
-               status = mess_ptr->BDEV_STATUS;
-       else
-               status = mess_ptr->REP_STATUS;
-       if (status == ERESTART)
-               r = EDEADEPT;
-  }
+  do {
+       r = drv_sendrec(driver_e, mess_ptr);
+       if (r == OK) {
+               if (is_bdev)
+                       status = mess_ptr->BDEV_STATUS;
+               else
+                       status = mess_ptr->REP_STATUS;
+               if (status == ERESTART) {
+                       r = EDEADEPT;
+                       *mess_ptr = mess_retry;
+                       retry_count++;
+               }
+       }
+  } while (r == EDEADEPT && retry_count < 5);
+
   if (r != OK) {
        if (r == EDEADSRCDST || r == EDEADEPT) {
                printf("VFS: dead driver %d\n", driver_e);
@@ -727,13 +736,14 @@ message *mess_ptr;                /* pointer to message for task */
   }
 
   /* Did the process we did the sendrec() for get a result? */
-  if (!is_bdev && mess_ptr->REP_ENDPT != proc_e) {
+  if (!is_bdev && mess_ptr->REP_ENDPT != proc_e && mess_ptr->m_type != EIO) {
        printf("VFS: strange device reply from %d, type = %d, "
                "proc = %d (not %d) (2) ignored\n", mess_ptr->m_source,
                mess_ptr->m_type, proc_e, mess_ptr->REP_ENDPT);
 
        return(EIO);
-  }
+  } else if (!IS_DRV_REPLY(mess_ptr->m_type))
+       return(EIO);
 
   return(OK);
 }
@@ -742,9 +752,7 @@ message *mess_ptr;          /* pointer to message for task */
 /*===========================================================================*
  *                             asyn_io                                      *
  *===========================================================================*/
-int asyn_io(task_nr, mess_ptr)
-int task_nr;                   /* which task to call */
-message *mess_ptr;             /* pointer to message for task */
+int asyn_io(endpoint_t drv_e, message *mess_ptr)
 {
 /* All file system I/O ultimately comes down to I/O on major/minor device
  * pairs. These lead to calls on the following routines via the dmap table.
@@ -753,9 +761,10 @@ message *mess_ptr;         /* pointer to message for task */
   int r;
 
   assert(!IS_BDEV_RQ(mess_ptr->m_type));
+  self->w_drv_sendrec = mess_ptr; /* Remember where result should be stored */
+  self->w_task = drv_e;
 
-  fp->fp_sendrec = mess_ptr;   /* Remember where result should be stored */
-  r = asynsend3(task_nr, mess_ptr, AMF_NOREPLY);
+  r = asynsend3(drv_e, mess_ptr, AMF_NOREPLY);
 
   if (r != OK) panic("VFS: asynsend in asyn_io failed: %d", r);
 
@@ -941,23 +950,11 @@ void bdev_up(int maj)
 
   if (maj < 0 || maj >= NR_DEVICES) panic("VFS: out-of-bound major");
   label = dmap[maj].dmap_label;
-
-  /* Tell each affected mounted file system about the new endpoint. This code
-   * is currently useless, as driver endpoints do not change across restarts.
-   */
-  for (vmp = &vmnt[0]; vmp < &vmnt[NR_MNTS]; ++vmp) {
-       if (major(vmp->m_dev) != maj) continue;
-
-       /* Send the driver label to the mounted file system. */
-       if (OK != req_newdriver(vmp->m_fs_e, vmp->m_dev, label))
-               printf("VFS dev_up: error sending new driver label to %d\n",
-                      vmp->m_fs_e);
-  }
+  found = 0;
 
   /* For each block-special file that was previously opened on the affected
    * device, we need to reopen it on the new driver.
    */
-  found = 0;
   for (rfilp = filp; rfilp < &filp[NR_FILPS]; rfilp++) {
        if (rfilp->filp_count < 1 || !(vp = rfilp->filp_vno)) continue;
        if (major(vp->v_sdev) != maj) continue;
@@ -965,13 +962,27 @@ void bdev_up(int maj)
 
        /* Reopen the device on the driver, once per filp. */
        bits = mode_map[rfilp->filp_mode & O_ACCMODE];
-       if ((r = bdev_open(vp->v_sdev, bits)) != OK)
+       if ((r = bdev_open(vp->v_sdev, bits)) != OK) {
                printf("VFS: mounted dev %d/%d re-open failed: %d.\n",
                        maj, minor(vp->v_sdev), r);
+               dmap[maj].dmap_recovering = 0;
+               return; /* Give up entirely */
+       }
 
        found = 1;
   }
 
+  /* Tell each affected mounted file system about the new endpoint.
+   */
+  for (vmp = &vmnt[0]; vmp < &vmnt[NR_MNTS]; ++vmp) {
+       if (major(vmp->m_dev) != maj) continue;
+
+       /* Send the driver label to the mounted file system. */
+       if (OK != req_newdriver(vmp->m_fs_e, vmp->m_dev, label))
+               printf("VFS dev_up: error sending new driver label to %d\n",
+                      vmp->m_fs_e);
+  }
+
   /* If any block-special file was open for this major at all, also inform the
    * root file system about the new driver. We do this even if the
    * block-special file is linked to another mounted file system, merely
@@ -982,6 +993,7 @@ void bdev_up(int maj)
                printf("VFSdev_up: error sending new driver label to %d\n",
                        ROOT_FS_E);
   }
+
 }
 
 
@@ -997,7 +1009,7 @@ void cdev_up(int maj)
   struct fproc *rfp;
   struct vnode *vp;
 
-  /* Look for processes that are suspened in an OPEN call. Set FP_SUSP_REOPEN
+  /* Look for processes that are suspended in an OPEN call. Set FP_SUSP_REOPEN
    * to indicate that this process was suspended before the call to dev_up.
    */
   for (rfp = &fproc[0]; rfp < &fproc[NR_PROCS]; rfp++) {
@@ -1022,7 +1034,7 @@ void cdev_up(int maj)
        if (major(vp->v_sdev) != maj) continue;
        if (!S_ISCHR(vp->v_mode)) continue;
 
-       rfilp->filp_state = FS_NEEDS_REOPEN;
+       rfilp->filp_state |= FS_NEEDS_REOPEN;
        needs_reopen = TRUE;
   }
 
@@ -1036,15 +1048,45 @@ void cdev_up(int maj)
 void open_reply(void)
 {
   struct fproc *rfp;
+  struct worker_thread *wp;
   endpoint_t proc_e;
   int slot;
 
   proc_e = job_m_in.REP_ENDPT;
   if (isokendpt(proc_e, &slot) != OK) return;
   rfp = &fproc[slot];
-  *rfp->fp_sendrec = job_m_in;
-  if (rfp->fp_wtid != invalid_thread_id)
-       worker_signal(worker_get(rfp->fp_wtid));        /* Continue open */
+  wp = worker_get(rfp->fp_wtid);
+  if (wp == NULL || wp->w_task != who_e) {
+       printf("VFS: no worker thread waiting for a reply from %d\n", who_e);
+       return;
+  }
+  *wp->w_drv_sendrec = job_m_in;
+  wp->w_drv_sendrec = NULL;
+  wp->w_task = NONE;
+  worker_signal(wp);   /* Continue open */
+}
+
+/*===========================================================================*
+ *                             dev_reply                                    *
+ *===========================================================================*/
+void dev_reply(struct dmap *dp)
+{
+       struct worker_thread *wp;
+
+       assert(dp != NULL);
+       assert(dp->dmap_servicing != NONE);
+
+       wp = worker_get(dp->dmap_servicing);
+       if (wp == NULL || wp->w_task != who_e) {
+               printf("VFS: no worker thread waiting for a reply from %d\n",
+                       who_e);
+               return;
+       }
+
+       assert(wp->w_drv_sendrec != NULL);
+       *wp->w_drv_sendrec = m_in;
+       wp->w_drv_sendrec = NULL;
+       worker_signal(wp);
 }
 
 /*===========================================================================*
@@ -1062,36 +1104,38 @@ int maj;
   if (maj < 0 || maj >= NR_DEVICES) panic("VFS: out-of-bound major");
   for (rfilp = filp; rfilp < &filp[NR_FILPS]; rfilp++) {
        if (rfilp->filp_count < 1 || !(vp = rfilp->filp_vno)) continue;
-       if (rfilp->filp_state != FS_NEEDS_REOPEN) continue;
+       if (!(rfilp->filp_state & FS_NEEDS_REOPEN)) continue;
        if (!S_ISCHR(vp->v_mode)) continue;
 
        major_dev = major(vp->v_sdev);
        minor_dev = minor(vp->v_sdev);
        if (major_dev != maj) continue;
 
-       if (!(rfilp->filp_flags & O_REOPEN)) {
-               /* File descriptor is to be closed when driver restarts. */
-               n = invalidate_filp(rfilp);
-               if (n != rfilp->filp_count) {
-                       printf("VFS: warning: invalidate/count "
-                              "discrepancy (%d, %d)\n", n, rfilp->filp_count);
-               }
-               rfilp->filp_count = 0;
-               continue;
-       }
+       if (rfilp->filp_flags & O_REOPEN) {
+               /* Try to reopen a file upon driver restart */
+               r = dev_reopen(vp->v_sdev, rfilp-filp,
+                       vp->v_mode & (R_BIT|W_BIT));
+
+               if (r == OK)
+                       return;
 
-       r = dev_reopen(vp->v_sdev, rfilp-filp, vp->v_mode & (R_BIT|W_BIT));
-       if (r == OK) return;
+               printf("VFS: file on dev %d/%d re-open failed: %d; "
+                       "invalidated %d fd's.\n", major_dev, minor_dev, r, n);
+       }
 
-       /* Device could not be reopened. Invalidate all filps on that device.*/
+       /* File descriptor is to be closed when driver restarts. */
        n = invalidate_filp(rfilp);
        if (n != rfilp->filp_count) {
                printf("VFS: warning: invalidate/count "
-                       "discrepancy (%d, %d)\n", n, rfilp->filp_count);
+                      "discrepancy (%d, %d)\n", n, rfilp->filp_count);
        }
        rfilp->filp_count = 0;
-       printf("VFS: file on dev %d/%d re-open failed: %d; "
-               "invalidated %d fd's.\n", major_dev, minor_dev, r, n);
+
+       /* We have to clean up this filp and vnode, but can't do that yet as
+        * it's locked by a worker thread. Start a new job to garbage collect
+        * invalidated filps associated with this device driver.
+        */
+       sys_worker_start(do_filp_gc);
   }
 
   /* Nothing more to re-open. Restart suspended processes */
@@ -1172,7 +1216,7 @@ void reopen_reply()
        return;
   }
 
-  if (rfilp->filp_state != FS_NEEDS_REOPEN) {
+  if (!(rfilp->filp_state & FS_NEEDS_REOPEN)) {
        printf("VFS: reopen_reply: bad state %d for filp number %d"
               " (from driver %d)\n", rfilp->filp_state, filp_no, driver_e);
        return;
@@ -1194,7 +1238,7 @@ void reopen_reply()
   }
 
   if (status == OK) {
-       rfilp->filp_state= FS_NORMAL;
+       rfilp->filp_state &= ~FS_NEEDS_REOPEN;
   } else {
        printf("VFS: reopen_reply: should handle error status\n");
        return;
index 5ce4f73fc3d7d19d2b509afde30fbb2b82d9cf44..deccc8d361464a0821549a0a4207a02191b198d2 100644 (file)
@@ -4,6 +4,7 @@
  */
 
 #include "fs.h"
+#include <assert.h>
 #include <string.h>
 #include <stdlib.h>
 #include <ctype.h>
 
 struct dmap dmap[NR_DEVICES];
 
-#define DT_EMPTY { no_dev, no_dev_io, NONE, "", 0, STYLE_NDEV, NULL }
+#define DT_EMPTY { no_dev, no_dev_io, NONE, "", 0, STYLE_NDEV, NULL, NONE, \
+                  0, NULL, 0}
+
+/*===========================================================================*
+ *                             lock_dmap                                    *
+ *===========================================================================*/
+void lock_dmap(struct dmap *dp)
+{
+/* Lock a driver */
+       struct worker_thread *org_self;
+       struct fproc *org_fp;
+       int r;
+
+       assert(dp != NULL);
+       assert(dp->dmap_driver != NONE);
+
+       org_fp = fp;
+       org_self = self;
+
+       if ((r = mutex_lock(dp->dmap_lock_ref)) != 0)
+               panic("unable to get a lock on dmap: %d\n", r);
+
+       fp = org_fp;
+       self = org_self;
+}
+
+/*===========================================================================*
+ *                             unlock_dmap                                  *
+ *===========================================================================*/
+void unlock_dmap(struct dmap *dp)
+{
+/* Unlock a driver */
+       int r;
+
+       assert(dp != NULL);
+
+       if ((r = mutex_unlock(dp->dmap_lock_ref)) != 0)
+               panic("unable to unlock dmap lock: %d\n", r);
+}
 
 /*===========================================================================*
  *                             do_mapdriver                                 *
@@ -98,10 +137,19 @@ int flags;                 /* device flags */
 
   /* Check if we're supposed to unmap it. */
  if (proc_nr_e == NONE) {
+       /* Even when a driver is now unmapped and is shortly to be mapped in
+        * due to recovery, invalidate associated filps if they're character
+        * special files. More sophisticated recovery mechanisms which would
+        * reduce the need to invalidate files are possible, but would require
+        * cooperation of the driver and more recovery framework between RS,
+        * VFS, and DS.
+        */
+       invalidate_filp_by_char_major(major);
        dp->dmap_opcl = no_dev;
        dp->dmap_io = no_dev_io;
        dp->dmap_driver = NONE;
        dp->dmap_flags = flags;
+       dp->dmap_lock_ref = &dp->dmap_lock;
        return(OK);
   }
 
@@ -186,6 +234,7 @@ int map_service(struct rprocpub *rpub)
 {
 /* Map a new service by storing its device driver properties. */
   int r;
+  struct dmap *fdp, *sdp;
 
   /* Not a driver, nothing more to do. */
   if(rpub->dev_nr == NO_DEV) return(OK);
@@ -200,6 +249,17 @@ int map_service(struct rprocpub *rpub)
        r = map_driver(rpub->label, rpub->dev_nr+1, rpub->endpoint,
                       rpub->dev_style2, rpub->dev_flags);
        if(r != OK) return(r);
+
+       /* To ensure that future dmap lock attempts always lock the same driver
+        * regardless of major number, refer the second dmap lock reference
+        * to the first dmap entry.
+        */
+       fdp = get_dmap_by_major(rpub->dev_nr);
+       sdp = get_dmap_by_major(rpub->dev_nr+1);
+       assert(fdp != NULL);
+       assert(sdp != NULL);
+       assert(fdp != sdp);
+       sdp->dmap_lock_ref = &fdp->dmap_lock;
   }
 
   return(OK);
@@ -218,6 +278,20 @@ void init_dmap()
        dmap[i] = dmap_default;
 }
 
+/*===========================================================================*
+ *                             init_dmap_locks                              *
+ *===========================================================================*/
+void init_dmap_locks()
+{
+  int i;
+
+  for (i = 0; i < NR_DEVICES; i++) {
+       if (mutex_init(&dmap[i].dmap_lock, NULL) != 0)
+               panic("unable to initialize dmap lock");
+       dmap[i].dmap_lock_ref = &dmap[i].dmap_lock;
+  }
+}
+
 /*===========================================================================*
  *                             dmap_driver_match                            *
  *===========================================================================*/
@@ -230,6 +304,17 @@ int dmap_driver_match(endpoint_t proc, int major)
   return(0);
 }
 
+/*===========================================================================*
+ *                             dmap_by_major                                *
+ *===========================================================================*/
+struct dmap *
+get_dmap_by_major(int major)
+{
+       if (major < 0 || major >= NR_DEVICES) return(NULL);
+       if (dmap[major].dmap_driver == NONE) return(NULL);
+       return(&dmap[major]);
+}
+
 /*===========================================================================*
  *                             dmap_endpt_up                                *
  *===========================================================================*/
@@ -240,12 +325,35 @@ void dmap_endpt_up(endpoint_t proc_e, int is_blk)
  */
 
   int major;
+  struct dmap *dp;
+  struct worker_thread *worker;
+
+  if (proc_e == NONE) return;
+
   for (major = 0; major < NR_DEVICES; major++) {
-       if (dmap_driver_match(proc_e, major)) {
-               if (is_blk)
+       if ((dp = get_dmap_by_major(major)) == NULL) continue;
+       if (dp->dmap_driver == proc_e) {
+               if (is_blk) {
+                       if (dp->dmap_recovering) {
+                               printf("VFS: driver recovery failure for"
+                                       " major %d\n", major);
+                               if (dp->dmap_servicing != NONE) {
+                                       worker = worker_get(dp->dmap_servicing);
+                                       worker_stop(worker);
+                               }
+                               dp->dmap_recovering = 0;
+                               continue;
+                       }
+                       dp->dmap_recovering = 1;
                        bdev_up(major);
-               else
+                       dp->dmap_recovering = 0;
+               } else {
+                       if (dp->dmap_servicing != NONE) {
+                               worker = worker_get(dp->dmap_servicing);
+                               worker_stop(worker);
+                       }
                        cdev_up(major);
+               }
        }
   }
 }
index 0db127dfa80c47b6c477c93d2988433c2f0ae89d..e5455b6917d83319ce04067f0b40bb5237b46fcb 100644 (file)
@@ -1,9 +1,7 @@
 #ifndef __VFS_DMAP_H__
 #define __VFS_DMAP_H__
 
-/*
-dmap.h
-*/
+#include "threads.h"
 
 /*===========================================================================*
  *                      Device <-> Driver Table                             *
@@ -11,7 +9,7 @@ dmap.h
 
 /* Device table.  This table is indexed by major device number.  It provides
  * the link between major device numbers and the routines that process them.
- * The table can be update dynamically. The field 'dmap_flags' describe an
+ * The table can be updated dynamically. The field 'dmap_flags' describe an
  * entry's current status and determines what control options are possible.
  */
 
@@ -23,6 +21,10 @@ extern struct dmap {
   int dmap_flags;
   int dmap_style;
   struct filp *dmap_sel_filp;
+  endpoint_t dmap_servicing;
+  mutex_t dmap_lock;
+  mutex_t *dmap_lock_ref;
+  int dmap_recovering;
 } dmap[];
 
 #endif
index 52a5773c6825fb3658b7517914f1c8d41d1967ac..f2800124978889ac25dfc3e747733fd71b3815c0 100644 (file)
@@ -30,8 +30,9 @@ EXTERN struct filp {
 
 #define FILP_CLOSED    0       /* filp_mode: associated device closed */
 
-#define FS_NORMAL      0       /* file descriptor can be used normally */
-#define FS_NEEDS_REOPEN        1       /* file descriptor needs to be re-opened */
+#define FS_NORMAL      000     /* file descriptor can be used normally */
+#define FS_NEEDS_REOPEN        001     /* file descriptor needs to be re-opened */
+#define FS_INVALIDATED 002     /* file was invalidated */
 
 #define FSF_UPDATE     001     /* The driver should be informed about new
                                 * state.
index 936addcc1e6bd96b1054ddb3919beaac2e9e15c5..b51642b2cbf4b27000bb2641cca9265ab118fcbf 100644 (file)
@@ -76,6 +76,50 @@ void check_filp_locks(void)
 #endif
 }
 
+/*===========================================================================*
+ *                             do_filp_gc                                           *
+ *===========================================================================*/
+void *do_filp_gc(void *UNUSED(arg))
+{
+  struct filp *f;
+  struct vnode *vp;
+
+  for (f = &filp[0]; f < &filp[NR_FILPS]; f++) {
+       if (!(f->filp_state & FS_INVALIDATED)) continue;
+       assert(f->filp_vno != NULL);
+       vp = f->filp_vno;
+
+       /* Synchronize with worker thread that might hold a lock on the vp */
+       lock_vnode(vp, VNODE_OPCL);
+       unlock_vnode(vp);
+
+       /* If garbage collection was invoked due to a failed device open
+        * request, then common_open has already cleaned up and we have
+        * nothing to do.
+        */
+       if (!(f->filp_state & FS_INVALIDATED)) {
+               continue;
+       }
+
+       /* If garbage collection was invoked due to a failed device close
+        * request, the close_filp has already cleaned up and we have nothing
+        * to do.
+        */
+       if (f->filp_mode != FILP_CLOSED) {
+               assert(f->filp_count == 0);
+               f->filp_count = 1;      /* So lock_filp and close_filp will do
+                                        * their job */
+               lock_filp(f, VNODE_READ);
+               close_filp(f);
+       }
+
+       f->filp_state &= ~FS_INVALIDATED;
+  }
+
+  thread_cleanup(NULL);
+  return(NULL);
+}
+
 /*===========================================================================*
  *                             init_filps                                           *
  *===========================================================================*/
@@ -166,12 +210,16 @@ tll_access_t locktype;
 /* See if 'fild' refers to a valid file descr.  If so, return its filp ptr. */
   struct filp *filp;
 
-  err_code = EBADF;
-  if (fild < 0 || fild >= OPEN_MAX ) return(NULL);
-  if (rfp->fp_filp[fild] == NULL && FD_ISSET(fild, &rfp->fp_filp_inuse))
+  filp = NULL;
+  if (fild < 0 || fild >= OPEN_MAX)
+       err_code = EBADF;
+  else if (rfp->fp_filp[fild] == NULL && FD_ISSET(fild, &rfp->fp_filp_inuse))
        err_code = EIO; /* The filedes is not there, but is not closed either.
                         */
-  if ((filp = rfp->fp_filp[fild]) != NULL) lock_filp(filp, locktype);
+  else if ((filp = rfp->fp_filp[fild]) == NULL)
+       err_code = EBADF;
+  else
+       lock_filp(filp, locktype);      /* All is fine */
 
   return(filp);        /* may also be NULL */
 }
@@ -220,9 +268,27 @@ int invalidate_filp(struct filp *rfilp)
        }
   }
 
+  rfilp->filp_state |= FS_INVALIDATED;
   return(n);   /* Report back how often this filp has been invalidated. */
 }
 
+/*===========================================================================*
+ *                     invalidate_filp_by_char_major                        *
+ *===========================================================================*/
+void invalidate_filp_by_char_major(int major)
+{
+  struct filp *f;
+
+  for (f = &filp[0]; f < &filp[NR_FILPS]; f++) {
+       if (f->filp_count != 0 && f->filp_vno != NULL) {
+               if (major(f->filp_vno->v_sdev) == major &&
+                   S_ISCHR(f->filp_vno->v_mode)) {
+                       (void) invalidate_filp(f);
+               }
+       }
+  }
+}
+
 /*===========================================================================*
  *                     invalidate_filp_by_endpt                             *
  *===========================================================================*/
@@ -264,7 +330,6 @@ tll_access_t locktype;
 
   assert(vp->v_ref_count > 0); /* vnode still in use? */
   assert(filp->filp_vno == vp);        /* vnode still what we think it is? */
-  assert(filp->filp_count > 0); /* filp still in use? */
 
   /* First try to get filp lock right off the bat */
   if (mutex_trylock(&filp->filp_lock) != 0) {
@@ -279,8 +344,6 @@ tll_access_t locktype;
        fp = org_fp;
        self = org_self;
   }
-
-  assert(filp->filp_count > 0);        /* Yet again; filp still in use? */
 }
 
 /*===========================================================================*
@@ -293,7 +356,7 @@ struct filp *filp;
   if (filp->filp_softlock != NULL)
        assert(filp->filp_softlock == fp);
 
-  if (filp->filp_count > 0) {
+  if (filp->filp_count > 0 || filp->filp_state & FS_INVALIDATED) {
        /* Only unlock vnode if filp is still in use */
 
        /* and if we don't hold a soft lock */
@@ -557,9 +620,15 @@ struct filp *f;
                        }
                        unlock_bsf();
 
-                       (void) bdev_close(dev); /* Ignore errors on close */
+                       /* Attempt to close only when feasible */
+                       if (!(f->filp_state & FS_INVALIDATED)) {
+                               (void) bdev_close(dev); /* Ignore errors */
+                       }
                } else {
-                       (void) dev_close(dev, f-filp); /* Ignore errors */
+                       /* Attempt to close only when feasible */
+                       if (!(f->filp_state & FS_INVALIDATED)) {
+                               (void) dev_close(dev, f-filp);/*Ignore errors*/
+                       }
                }
 
                f->filp_mode = FILP_CLOSED;
@@ -572,8 +641,10 @@ struct filp *f;
        release(vp, rw, susp_count);
   }
 
-  /* If a write has been done, the inode is already marked as DIRTY. */
-  if (--f->filp_count == 0) {
+  f->filp_count--;     /* If filp got invalidated at device closure, the
+                        * count might've become negative now */
+  if (f->filp_count == 0 ||
+      (f->filp_count < 0 && f->filp_state & FS_INVALIDATED)) {
        if (S_ISFIFO(vp->v_mode)) {
                /* Last reader or writer is going. Tell PFS about latest
                 * pipe size.
@@ -585,6 +656,7 @@ struct filp *f;
        put_vnode(f->filp_vno);
        f->filp_vno = NULL;
        f->filp_mode = FILP_CLOSED;
+       f->filp_count = 0;
   } else if (f->filp_count < 0) {
        panic("VFS: invalid filp count: %d ino %d/%d", f->filp_count,
              vp->v_dev, vp->v_inode_nr);
index 46115da5f6460ef514c3f1b3f874007ee468eb10..49fa2f890e86151b94cfcb362a16b0b580fa7bcd 100644 (file)
@@ -42,7 +42,6 @@ EXTERN struct fproc {
   gid_t fp_sgroups[NGROUPS_MAX];/* supplemental groups */
   mode_t fp_umask;             /* mask set by umask system call */
 
-  message *fp_sendrec;         /* request/reply to/from FS/driver */
   mutex_t fp_lock;             /* mutex to lock fproc object */
   struct job fp_job;           /* pending job */
   thread_t fp_wtid;            /* Thread ID of worker */
index 7a4beb86fd67e05474c0fac0b17c1c9fec6b461d..3e4eb3dad1c1ec72ad5828b5f1ad8344ca779aae 100644 (file)
@@ -40,7 +40,6 @@ EXTERN unsigned long calls_stats[NCALLS];
 #endif
 
 /* Thread related prototypes */
-static void thread_cleanup(struct fproc *rfp);
 static void *do_async_dev_result(void *arg);
 static void *do_control_msgs(void *arg);
 static void *do_fs_reply(struct job *job);
@@ -110,7 +109,10 @@ int main(void)
                continue;
        } else if (is_notify(call_nr)) {
                /* A task notify()ed us */
-               sys_worker_start(do_control_msgs);
+               if (who_e == DS_PROC_NR)
+                       worker_start(ds_event);
+               else
+                       sys_worker_start(do_control_msgs);
                continue;
        } else if (who_p < 0) { /* i.e., message comes from a task */
                /* We're going to ignore this message. Tasks should
@@ -126,10 +128,28 @@ int main(void)
         * not a problem (requests/replies are simply queued), except when
         * they're from an FS endpoint, because these can cause a deadlock.
         * handle_work() takes care of the details. */
-       if (IS_DEV_RS(call_nr)) {
+       if (IS_DRV_REPLY(call_nr)) {
                /* We've got results for a device request */
-               handle_work(do_async_dev_result);
-               continue;
+
+               struct dmap *dp;
+
+               dp = get_dmap(who_e);
+               if (dp != NULL) {
+                       if (dev_style_asyn(dp->dmap_style)) {
+                               handle_work(do_async_dev_result);
+
+                       } else {
+                               if (dp->dmap_servicing == NONE) {
+                                       printf("Got spurious dev reply from %d",
+                                       who_e);
+                               } else {
+                                       dev_reply(dp);
+                               }
+                       }
+                       continue;
+               }
+               printf("VFS: ignoring dev reply from unknown driver %d\n",
+                       who_e);
        } else {
                /* Normal syscall. */
                handle_work(do_work);
@@ -253,12 +273,9 @@ static void *do_control_msgs(void *arg)
   if (job_m_in.m_source == CLOCK) {
        /* Alarm timer expired. Used only for select(). Check it. */
        expire_timers(job_m_in.NOTIFY_TIMESTAMP);
-  } else if (job_m_in.m_source == DS_PROC_NR) {
-       /* DS notifies us of an event. */
-       ds_event();
   } else {
        /* Device notifies us of an event. */
-       dev_status(&job_m_in);
+       dev_status(job_m_in.m_source);
   }
 
   thread_cleanup(NULL);
@@ -271,28 +288,26 @@ static void *do_control_msgs(void *arg)
 static void *do_fs_reply(struct job *job)
 {
   struct vmnt *vmp;
-  struct fproc *rfp;
+  struct worker_thread *wp;
 
   if ((vmp = find_vmnt(who_e)) == NULL)
        panic("Couldn't find vmnt for endpoint %d", who_e);
 
-  rfp = job->j_fp;
+  wp = worker_get(job->j_fp->fp_wtid);
 
-  if (rfp == NULL || rfp->fp_endpoint == NONE) {
+  if (wp == NULL) {
        printf("VFS: spurious reply from %d\n", who_e);
        return(NULL);
   }
 
-  if (rfp->fp_task != who_e)
-       printf("VFS: expected %d to reply, not %d\n", rfp->fp_task, who_e);
-  *rfp->fp_sendrec = m_in;
-  rfp->fp_task = NONE;
+  if (wp->w_task != who_e) {
+       printf("VFS: expected %d to reply, not %d\n", wp->w_task, who_e);
+       return(NULL);
+  }
+  *wp->w_fs_sendrec = m_in;
+  wp->w_task = NONE;
   vmp->m_comm.c_cur_reqs--; /* We've got our reply, make room for others */
-  if (rfp->fp_wtid != invalid_thread_id)
-       worker_signal(worker_get(rfp->fp_wtid)); /* Continue this thread */
-  else
-       printf("VFS: consistency error: reply for finished job\n");
-
+  worker_signal(wp); /* Continue this thread */
   return(NULL);
 }
 
@@ -576,6 +591,7 @@ static int sef_cb_init_fresh(int UNUSED(type), sef_init_info_t *info)
 #endif
   }
 
+  init_dmap_locks();           /* init dmap locks */
   init_vnodes();               /* init vnodes */
   init_vmnts();                        /* init vmnt structures */
   init_select();               /* init select() structures */
@@ -665,7 +681,7 @@ void unlock_proc(struct fproc *rfp)
 /*===========================================================================*
  *                             thread_cleanup                               *
  *===========================================================================*/
-static void thread_cleanup(struct fproc *rfp)
+void thread_cleanup(struct fproc *rfp)
 {
 /* Clean up worker thread. Skip parts if this thread is not associated
  * with a particular process (i.e., rfp is NULL) */
index 98cc99019acbeff63779b75c5089e85cf843b8aa..eaae794a9ad0484a41c09b6c6259c3925a19f495 100644 (file)
@@ -727,7 +727,8 @@ int pm_dumpcore(endpoint_t proc_e, int csig, vir_bytes exe_name)
 /*===========================================================================*
  *                              ds_event                                    *
  *===========================================================================*/
-void ds_event(void)
+void *
+ds_event(void *arg)
 {
   char key[DS_MAX_KEYLEN];
   char *blkdrv_prefix = "drv.blk.";
@@ -736,6 +737,11 @@ void ds_event(void)
   int type, r, is_blk;
   endpoint_t owner_endpoint;
 
+  struct job my_job;
+
+  my_job = *((struct job *) arg);
+  fp = my_job.j_fp;
+
   /* Get the event and the owner from DS. */
   while ((r = ds_check(key, &type, &owner_endpoint)) == OK) {
        /* Only check for block and character driver up events. */
@@ -749,7 +755,7 @@ void ds_event(void)
 
        if ((r = ds_retrieve_u32(key, &value)) != OK) {
                printf("VFS: ds_event: ds_retrieve_u32 failed\n");
-               return;
+               break;
        }
        if (value != DS_DRIVER_UP) continue;
 
@@ -758,4 +764,7 @@ void ds_event(void)
   }
 
   if (r != ENOENT) printf("VFS: ds_event: ds_check failed: %d\n", r);
+
+  thread_cleanup(NULL);
+  return(NULL);
 }
index f5ee4599c58a8359547823fe7bc5c5e2b3745658..0a0edf80f7968fe8f83e39d8249c676cd903b762 100644 (file)
@@ -299,6 +299,7 @@ int common_open(char path[PATH_MAX], int oflags, mode_t omode)
                FD_CLR(scratch(fp).file.fd_nr, &fp->fp_filp_inuse);
                filp->filp_count = 0;
                filp->filp_vno = NULL;
+               filp->filp_state &= ~FS_INVALIDATED; /* Prevent garbage col. */
                put_vnode(vp);
        }
   } else {
index fea576dd4ac910ab4c40ee25d7b02886f9b40a74..735624472d07a7ab7c12c5d599ab3e0bfbcf34db 100644 (file)
@@ -21,6 +21,7 @@ struct job;
 typedef struct filp * filp_id_t;
 
 /* comm.c */
+int drv_sendrec(endpoint_t drv_e, message *reqm);
 void fs_cancel(struct vmnt *vmp);
 int fs_sendrec(endpoint_t fs_e, message *reqm);
 void fs_sendmore(struct vmnt *vmp);
@@ -29,6 +30,7 @@ void send_work(void);
 /* device.c */
 int dev_open(dev_t dev, endpoint_t proc_e, int flags);
 int dev_reopen(dev_t dev, int filp_no, int flags);
+void dev_reply(struct dmap *dp);
 int dev_close(dev_t dev, int filp_no);
 int bdev_open(dev_t dev, int access);
 int bdev_close(dev_t dev);
@@ -36,7 +38,7 @@ int dev_io(int op, dev_t dev, endpoint_t proc_e, void *buf, u64_t pos,
        size_t bytes, int flags, int suspend_reopen);
 int gen_opcl(int op, dev_t dev, endpoint_t task_nr, int flags);
 int gen_io(endpoint_t driver_e, message *mess_ptr);
-int asyn_io(int task_nr, message *mess_ptr);
+int asyn_io(endpoint_t drv_e, message *mess_ptr);
 int no_dev(int op, dev_t dev, int proc, int flags);
 int no_dev_io(int, message *);
 int tty_opcl(int op, dev_t dev, endpoint_t proc, int flags);
@@ -45,7 +47,7 @@ int clone_opcl(int op, dev_t dev, int proc, int flags);
 int ctty_io(int task_nr, message *mess_ptr);
 int do_ioctl(void);
 void pm_setsid(endpoint_t proc_e);
-void dev_status(message *m);
+void dev_status(endpoint_t drv_e);
 void bdev_up(int major);
 void cdev_up(int major);
 endpoint_t find_suspended_ep(endpoint_t driver, cp_grant_id_t g);
@@ -53,12 +55,16 @@ void reopen_reply(void);
 void open_reply(void);
 
 /* dmap.c */
+void lock_dmap(struct dmap *dp);
+void unlock_dmap(struct dmap *dp);
 int do_mapdriver(void);
 void init_dmap(void);
+void init_dmap_locks(void);
 int dmap_driver_match(endpoint_t proc, int major);
 void dmap_endpt_up(int proc_nr, int is_blk);
 void dmap_unmap_by_endpt(int proc_nr);
 struct dmap *get_dmap(endpoint_t proc_e);
+struct dmap *get_dmap_by_major(int major);
 int do_mapdriver(void);
 int map_service(struct rprocpub *rpub);
 void dmap_unmap_by_endpt(int proc_nr);
@@ -78,6 +84,7 @@ int pm_exec(endpoint_t proc_e, vir_bytes path, size_t path_len, vir_bytes frame,
                        } while(0)
 
 /* filedes.c */
+void *do_filp_gc(void *arg);
 void check_filp_locks(void);
 void check_filp_locks_by_me(void);
 void init_filps(void);
@@ -91,6 +98,7 @@ void unlock_filp(struct filp *filp);
 void unlock_filps(struct filp *filp1, struct filp *filp2);
 int invalidate_filp(struct filp *);
 void invalidate_filp_by_endpt(endpoint_t proc_e);
+void invalidate_filp_by_char_major(int major);
 int do_verify_fd(void);
 int set_filp(filp_id_t sfilp);
 int do_set_filp(void);
@@ -120,10 +128,10 @@ void lock_revive(void);
 
 /* main.c */
 int main(void);
-void reply(endpoint_t whom, int result);
 void lock_proc(struct fproc *rfp, int force_lock);
+void reply(endpoint_t whom, int result);
+void thread_cleanup(struct fproc *rfp);
 void unlock_proc(struct fproc *rfp);
-void *do_dummy(void *arg);
 
 /* misc.c */
 int do_dup(void);
@@ -139,7 +147,7 @@ void pm_reboot(void);
 int do_svrctl(void);
 int do_getsysinfo(void);
 int pm_dumpcore(endpoint_t proc_e, int sig, vir_bytes exe_name);
-void ds_event(void);
+void * ds_event(void *arg);
 
 /* mount.c */
 int do_fsready(void);
index a0024ed9c4bc084a24386c8615748673f92ac6ae..4b7715017f9750c974f354d3f10be335c7269696 100644 (file)
@@ -135,7 +135,7 @@ int read_write(int rw_flag, struct filp *f, char *buf, size_t size,
        if (vp->v_sdev == NO_DEV)
                panic("VFS: read_write tries to access char dev NO_DEV");
 
-       suspend_reopen = (f->filp_state != FS_NORMAL);
+       suspend_reopen = (f->filp_state & FS_NEEDS_REOPEN);
        dev = (dev_t) vp->v_sdev;
 
        r = dev_io(op, dev, for_e, buf, position, size, oflags,
index 01d4305235e8e5516cf421e601ee755828db2d2b..0898229317341a80ed8b196a844d8ef18def8042 100644 (file)
@@ -112,25 +112,34 @@ int do_select(void)
 
   se = &selecttab[s];
   wipe_select(se);     /* Clear results of previous usage */
+  se->requestor = fp;
   se->req_endpt = who_e;
   se->vir_readfds = (fd_set *) job_m_in.SEL_READFDS;
   se->vir_writefds = (fd_set *) job_m_in.SEL_WRITEFDS;
   se->vir_errorfds = (fd_set *) job_m_in.SEL_ERRORFDS;
 
   /* Copy fdsets from the process */
-  if ((r = copy_fdsets(se, nfds, FROM_PROC)) != OK) return(r);
+  if ((r = copy_fdsets(se, nfds, FROM_PROC)) != OK) {
+       se->requestor = NULL;
+       return(r);
+  }
 
   /* Did the process set a timeout value? If so, retrieve it. */
   if (vtimeout != 0) {
        do_timeout = 1;
        r = sys_vircopy(who_e, (vir_bytes) vtimeout, SELF, 
                        (vir_bytes) &timeout, sizeof(timeout));
-       if (r != OK) return(r);
+       if (r != OK) {
+               se->requestor = NULL;
+               return(r);
+       }
   }
 
   /* No nonsense in the timeval */
-  if (do_timeout && (timeout.tv_sec < 0 || timeout.tv_usec < 0))
+  if (do_timeout && (timeout.tv_sec < 0 || timeout.tv_usec < 0)) {
+       se->requestor = NULL;
        return(EINVAL);
+  }
 
   /* If there is no timeout, we block forever. Otherwise, we block up to the
    * specified time interval.
@@ -165,6 +174,7 @@ int do_select(void)
                else /* File descriptor is 'ready' to return EIO */
                        r = EINTR;
 
+               se->requestor = NULL;
                return(r);
        }
 
@@ -191,8 +201,10 @@ int do_select(void)
                }
        }
        unlock_filp(f);
-       if (se->type[fd] == -1) /* Type not found */
+       if (se->type[fd] == -1) { /* Type not found */
+               se->requestor = NULL;
                return(EBADF);
+       }
   }
 
   /* Check all file descriptors in the set whether one is 'ready' now */
@@ -230,6 +242,7 @@ int do_select(void)
         */
        r = copy_fdsets(se, se->nfds, TO_PROC);
        select_cancel_all(se);
+       se->requestor = NULL;
 
        if (r != OK)
                return(r);
@@ -262,9 +275,6 @@ int do_select(void)
        set_timer(&se->timer, ticks, select_timeout_check, s);
   }
 
-  /* If we're blocking, the table entry is now valid  */
-  se->requestor = fp;
-
   /* process now blocked */
   suspend(FP_BLOCKED_ON_SELECT);
   return(SUSPEND);
index 02f03cdeecbce09efbaac32415c88069c9194a1c..cddb73e9ac96b6feba3b1e3ff37db52589d40ad8 100644 (file)
@@ -29,6 +29,10 @@ struct worker_thread {
   cond_t w_event;
   struct job w_job;
   struct fproc *w_fp;
+  message *w_fs_sendrec;
+  message *w_drv_sendrec;
+  endpoint_t w_task;
+  struct dmap *w_dmap;
   struct worker_thread *w_next;
 };
 
index f5eaa29cfc05b19712dfce456d1843fc083c7847..add1d371228455a6d2986f1c2980b858030e29b0 100644 (file)
@@ -291,7 +291,6 @@ static void worker_wake(struct worker_thread *worker)
 void worker_wait(void)
 {
   self->w_job.j_err_code = err_code;
-  assert(fp == self->w_job.j_fp);
   worker_sleep(self);
   /* We continue here after waking up */
   fp = self->w_job.j_fp;       /* Restore global data */
@@ -314,10 +313,18 @@ void worker_signal(struct worker_thread *worker)
 void worker_stop(struct worker_thread *worker)
 {
   ASSERTW(worker);             /* Make sure we have a valid thread */
-  if (worker->w_job.j_fp)
-       worker->w_job.j_fp->fp_sendrec->m_type = EIO;
-  else
+  if (worker->w_task != NONE) {
+       /* This thread is communicating with a driver or file server */
+       if (worker->w_drv_sendrec != NULL) {                    /* Driver */
+               worker->w_drv_sendrec->m_type = EIO;
+       } else if (worker->w_fs_sendrec != NULL) {              /* FS */
+               worker->w_fs_sendrec->m_type = EIO;
+       } else {
+               panic("reply storage consistency error");       /* Oh dear */
+       }
+  } else {
        worker->w_job.j_m_in.m_type = EIO;
+  }
   worker_wake(worker);
 }
 
@@ -387,7 +394,9 @@ static int worker_waiting_for(struct worker_thread *worker, endpoint_t proc_e)
   ASSERTW(worker);             /* Make sure we have a valid thread */
 
   if (worker->w_job.j_func != NULL) {
-       if (worker->w_job.j_fp != NULL) {
+       if (worker->w_task != NONE)
+               return(worker->w_task == proc_e);
+       else if (worker->w_job.j_fp != NULL) {
                return(worker->w_job.j_fp->fp_task == proc_e);
        }
   }