patch-2.3.25 linux/ipc/sem.c
Next file: linux/ipc/shm.c
Previous file: linux/ipc/msg.c
Back to the patch index
Back to the overall index
-  Lines: 1141
-  Date:
Thu Oct 28 14:35:57 1999
-  Orig file: 
v2.3.24/linux/ipc/sem.c
-  Orig date: 
Thu Aug 26 13:05:41 1999
diff -u --recursive --new-file v2.3.24/linux/ipc/sem.c linux/ipc/sem.c
@@ -50,11 +50,13 @@
  *      array we do the same as before.
  *
  * /proc/sysvipc/sem support (c) 1999 Dragos Acostachioaie <dragos@iname.com>
+ *
+ * SMP-threaded (c) 1999 Manfred Spraul <manfreds@colorfullife.com>
  */
 
 #include <linux/config.h>
 #include <linux/malloc.h>
-#include <linux/smp_lock.h>
+#include <linux/spinlock.h>
 #include <linux/init.h>
 #include <linux/proc_fs.h>
 
@@ -68,27 +70,42 @@
 static int sysvipc_sem_read_proc(char *buffer, char **start, off_t offset, int length, int *eof, void *data);
 #endif
 
-static struct semid_ds *semary[SEMMNI];
-static int used_sems = 0, used_semids = 0;
-static DECLARE_WAIT_QUEUE_HEAD(sem_lock);
+struct semid_ary
+{
+	spinlock_t lock;
+	struct semid_ds* s;
+};
+
+static struct semid_ary semary[SEMMNI];
+
+static DECLARE_MUTEX(sem_lock);
 static int max_semid = 0;
+static int used_sems = 0;
+static int used_semids = 0;
 
 static unsigned short sem_seq = 0;
 
+/* anti-deadlock ordering:
+ *	sem_lock < semary[].lock
+ * linked list protection:
+ *	sem_undo.id_next,
+ *	semid_ds.sem_pending{,last},
+ *	semid_ds.sem_undo: semary[].lock for read/write
+ *	sem_undo.proc_next: only "current" is allowed to read/write that field.
+ *	
+ */
+
 void __init sem_init (void)
 {
 	int i;
-#ifdef CONFIG_PROC_FS
-	struct proc_dir_entry *ent;
-#endif
 
-	init_waitqueue_head(&sem_lock);
 	used_sems = used_semids = max_semid = sem_seq = 0;
-	for (i = 0; i < SEMMNI; i++)
-		semary[i] = (struct semid_ds *) IPC_UNUSED;
+	for (i = 0; i < SEMMNI; i++) {
+		semary[i].lock = SPIN_LOCK_UNLOCKED;
+		semary[i].s = NULL;
+	}
 #ifdef CONFIG_PROC_FS
-	ent = create_proc_entry("sysvipc/sem", 0, 0);
-	ent->read_proc = sysvipc_sem_read_proc;
+	create_proc_read_entry("sysvipc/sem", 0, 0, sysvipc_sem_read_proc, NULL);
 #endif
 	return;
 }
@@ -99,10 +116,10 @@
 	struct semid_ds *sma;
 
 	for (id = 0; id <= max_semid; id++) {
-		while ((sma = semary[id]) == IPC_NOID)
-			interruptible_sleep_on (&sem_lock);
-		if (sma == IPC_UNUSED)
+		sma = semary[id].s;
+		if(sma==NULL)
 			continue;
+
 		if (key == sma->sem_perm.key)
 			return id;
 	}
@@ -120,20 +137,17 @@
 		return -EINVAL;
 	if (used_sems + nsems > SEMMNS)
 		return -ENOSPC;
-	for (id = 0; id < SEMMNI; id++)
-		if (semary[id] == IPC_UNUSED) {
-			semary[id] = (struct semid_ds *) IPC_NOID;
+	for (id = 0; id < SEMMNI; id++) {
+		if(semary[id].s == NULL)
 			goto found;
-		}
+	}
 	return -ENOSPC;
 found:
 	size = sizeof (*sma) + nsems * sizeof (struct sem);
 	used_sems += nsems;
 	sma = (struct semid_ds *) kmalloc (size, GFP_KERNEL);
 	if (!sma) {
-		semary[id] = (struct semid_ds *) IPC_UNUSED;
 		used_sems -= nsems;
-		wake_up (&sem_lock);
 		return -ENOMEM;
 	}
 	memset (sma, 0, size);
@@ -143,7 +157,10 @@
 	ipcp->key = key;
 	ipcp->cuid = ipcp->uid = current->euid;
 	ipcp->gid = ipcp->cgid = current->egid;
-	sma->sem_perm.seq = sem_seq;
+	/* sma->sem_perm.seq*MSGMNI must be a positive integer.
+	 * this limits MSGMNI to 32768
+	 */
+	sma->sem_perm.seq = sem_seq++;
 	/* sma->sem_pending = NULL; */
 	sma->sem_pending_last = &sma->sem_pending;
 	/* sma->undo = NULL; */
@@ -152,8 +169,9 @@
 	if (id > max_semid)
 		max_semid = id;
 	used_semids++;
-	semary[id] = sma;
-	wake_up (&sem_lock);
+	spin_lock(&semary[id].lock);
+	semary[id].s = sma;
+	spin_unlock(&semary[id].lock);
 	return (unsigned int) sma->sem_perm.seq * SEMMNI + id;
 }
 
@@ -162,9 +180,10 @@
 	int id, err = -EINVAL;
 	struct semid_ds *sma;
 
-	lock_kernel();
 	if (nsems < 0 || nsems > SEMMSL)
-		goto out;
+		return -EINVAL;
+	down(&sem_lock);
+	
 	if (key == IPC_PRIVATE) {
 		err = newary(key, nsems, semflg);
 	} else if ((id = findkey (key)) == -1) {  /* key not used */
@@ -175,7 +194,7 @@
 	} else if (semflg & IPC_CREAT && semflg & IPC_EXCL) {
 		err = -EEXIST;
 	} else {
-		sma = semary[id];
+		sma = semary[id].s;
 		if (nsems > sma->sem_nsems)
 			err = -EINVAL;
 		else if (ipcperms(&sma->sem_perm, semflg))
@@ -183,8 +202,8 @@
 		else
 			err = (int) sma->sem_perm.seq * SEMMNI + id;
 	}
-out:
-	unlock_kernel();
+
+	up(&sem_lock);
 	return err;
 }
 
@@ -192,25 +211,25 @@
  * insert new queue elements at the tail sma->sem_pending_last.
  */
 static inline void append_to_queue (struct semid_ds * sma,
-                                    struct sem_queue * q)
+				    struct sem_queue * q)
 {
 	*(q->prev = sma->sem_pending_last) = q;
 	*(sma->sem_pending_last = &q->next) = NULL;
 }
 
 static inline void prepend_to_queue (struct semid_ds * sma,
-                                     struct sem_queue * q)
+				     struct sem_queue * q)
 {
-        q->next = sma->sem_pending;
-        *(q->prev = &sma->sem_pending) = q;
-        if (q->next)
-                q->next->prev = &q->next;
-        else /* sma->sem_pending_last == &sma->sem_pending */
-                sma->sem_pending_last = &q->next;
+	q->next = sma->sem_pending;
+	*(q->prev = &sma->sem_pending) = q;
+	if (q->next)
+		q->next->prev = &q->next;
+	else /* sma->sem_pending_last == &sma->sem_pending */
+		sma->sem_pending_last = &q->next;
 }
 
 static inline void remove_from_queue (struct semid_ds * sma,
-                                      struct sem_queue * q)
+				      struct sem_queue * q)
 {
 	*(q->prev) = q->next;
 	if (q->next)
@@ -226,8 +245,8 @@
  */
 
 static int try_atomic_semop (struct semid_ds * sma, struct sembuf * sops,
-                             int nsops, struct sem_undo *un, int pid,
-                             int do_undo)
+			     int nsops, struct sem_undo *un, int pid,
+			     int do_undo)
 {
 	int result, sem_op;
 	struct sembuf *sop;
@@ -251,12 +270,12 @@
 			goto out_of_range;
 	}
 
-        if (do_undo)
-        {
-                sop--;
-                result = 0;
-                goto undo;
-        }
+	if (do_undo)
+	{
+		sop--;
+		result = 0;
+		goto undo;
+	}
 
 	sma->sem_otime = CURRENT_TIME;
 	return 0;
@@ -272,7 +291,7 @@
 		result = 1;
 
 undo:
-        while (sop >= sops) {
+	while (sop >= sops) {
 		curr = sma->sem_base + sop->sem_num;
 		curr->semval -= sop->sem_op;
 		curr->sempid >>= 16;
@@ -293,27 +312,27 @@
 	int error;
 	struct sem_queue * q;
 
-        for (q = sma->sem_pending; q; q = q->next) {
-                        
-                if (q->status == 1)
-                        return; /* wait for other process */
-
-                error = try_atomic_semop(sma, q->sops, q->nsops,
-                                         q->undo, q->pid, q->alter);
-
-                /* Does q->sleeper still need to sleep? */
-                if (error <= 0) {
-                                /* Found one, wake it up */
-                        wake_up_interruptible(&q->sleeper);
-                        if (error == 0 && q->alter) {
-                                /* if q-> alter let it self try */
-                                q->status = 1;
-                                return;
-                        }
-                        q->status = error;
-                        remove_from_queue(sma,q);
-                }
-        }
+	for (q = sma->sem_pending; q; q = q->next) {
+			
+		if (q->status == 1)
+			return; /* wait for other process */
+
+		error = try_atomic_semop(sma, q->sops, q->nsops,
+					 q->undo, q->pid, q->alter);
+
+		/* Does q->sleeper still need to sleep? */
+		if (error <= 0) {
+				/* Found one, wake it up */
+			wake_up_process(q->sleeper);
+			if (error == 0 && q->alter) {
+				/* if q-> alter let it self try */
+				q->status = 1;
+				return;
+			}
+			q->status = error;
+			remove_from_queue(sma,q);
+		}
+	}
 }
 
 /* The following counts are associated to each semaphore:
@@ -365,21 +384,22 @@
 /* Free a semaphore set. */
 static void freeary (int id)
 {
-	struct semid_ds *sma = semary[id];
+	struct semid_ds *sma;
 	struct sem_undo *un;
 	struct sem_queue *q;
 
-	/* Invalidate this semaphore set */
-	sma->sem_perm.seq++;
-	sem_seq = (sem_seq+1) % ((unsigned)(1<<31)/SEMMNI); /* increment, but avoid overflow */
+	/* we own both locks, noone can get in */
+	sma = semary[id].s;
+	semary[id].s = NULL;
+
 	used_sems -= sma->sem_nsems;
 	if (id == max_semid)
-		while (max_semid && (semary[--max_semid] == IPC_UNUSED));
-	semary[id] = (struct semid_ds *) IPC_UNUSED;
+		while (max_semid && (semary[--max_semid].s == NULL));
 	used_semids--;
 
 	/* Invalidate the existing undo structures for this semaphore set.
-	 * (They will be freed without any further action in sem_exit().)
+	 * (They will be freed without any further action in sem_exit()
+	 * or during the next semop.)
 	 */
 	for (un = sma->undo; un; un = un->id_next)
 		un->semid = -1;
@@ -388,35 +408,23 @@
 	for (q = sma->sem_pending; q; q = q->next) {
 		q->status = -EIDRM;
 		q->prev = NULL;
-		wake_up_interruptible(&q->sleeper); /* doesn't sleep! */
+		wake_up_process(q->sleeper); /* doesn't sleep */
 	}
 
 	kfree(sma);
 }
 
-asmlinkage long sys_semctl (int semid, int semnum, int cmd, union semun arg)
+int semctl_nolock(int semid, int semnum, int cmd, union semun arg)
 {
-	struct semid_ds *buf = NULL;
-	struct semid_ds tbuf;
-	int i, id, val = 0;
-	struct semid_ds *sma;
-	struct ipc_perm *ipcp;
-	struct sem *curr = NULL;
-	struct sem_undo *un;
-	unsigned int nsems;
-	ushort *array = NULL;
-	ushort sem_io[SEMMSL];
 	int err = -EINVAL;
+	int lid = semid % SEMMNI;
 
-	lock_kernel();
-	if (semid < 0 || semnum < 0 || cmd < 0)
-		goto out;
-
-	switch (cmd) {
+	switch(cmd) {
 	case IPC_INFO:
 	case SEM_INFO:
 	{
-		struct seminfo seminfo, *tmp = arg.__buf;
+		struct seminfo seminfo;
+
 		seminfo.semmni = SEMMNI;
 		seminfo.semmns = SEMMNS;
 		seminfo.semmsl = SEMMSL;
@@ -428,174 +436,201 @@
 		seminfo.semusz = SEMUSZ;
 		seminfo.semaem = SEMAEM;
 		if (cmd == SEM_INFO) {
+			down(&sem_lock);
 			seminfo.semusz = used_semids;
 			seminfo.semaem = used_sems;
+			up(&sem_lock);
 		}
-		err = -EFAULT;
-		if (copy_to_user (tmp, &seminfo, sizeof(struct seminfo))) 
-			goto out;
-		err = max_semid;
-		goto out;
+		if (copy_to_user (arg.__buf, &seminfo, sizeof(struct seminfo))) 
+			return -EFAULT;
+		return max_semid;
 	}
-
 	case SEM_STAT:
-		buf = arg.buf;
-		err = -EINVAL;
+	{
+		struct semid_ds *sma;
+		struct semid_ds tbuf;
+		int id;
+
 		if (semid > max_semid)
-			goto out;
-		sma = semary[semid];
-		if (sma == IPC_UNUSED || sma == IPC_NOID)
-			goto out;
+			return -EINVAL;
+
+		spin_lock(&semary[lid].lock);
+		err = -EINVAL;
+		sma = semary[semid].s;
+		if (sma ==  NULL)
+			goto out_unlock;
 		err = -EACCES;
 		if (ipcperms (&sma->sem_perm, S_IRUGO))
-			goto out;
+			goto out_unlock;
 		id = (unsigned int) sma->sem_perm.seq * SEMMNI + semid;
+		memset(&tbuf,0,sizeof(tbuf));
 		tbuf.sem_perm   = sma->sem_perm;
 		tbuf.sem_otime  = sma->sem_otime;
 		tbuf.sem_ctime  = sma->sem_ctime;
 		tbuf.sem_nsems  = sma->sem_nsems;
-		err = -EFAULT;
-		if (copy_to_user (buf, &tbuf, sizeof(*buf)) == 0)
-			err = id;
-		goto out;
+		spin_unlock(&semary[lid].lock);
+		if (copy_to_user (arg.buf, &tbuf, sizeof(tbuf)))
+			return -EFAULT;
+		return id;
+	}
+	default:
+		return -EINVAL;
 	}
+	return err;
+out_unlock:
+	spin_unlock(&semary[lid].lock);
+	return err;
+}
 
-	id = (unsigned int) semid % SEMMNI;
-	sma = semary [id];
-	err = -EINVAL;
-	if (sma == IPC_UNUSED || sma == IPC_NOID)
-		goto out;
-	ipcp = &sma->sem_perm;
-	nsems = sma->sem_nsems;
-	err = -EIDRM;
+int semctl_locked_unlock(int semid, int semnum, int cmd, union semun arg)
+{
+	struct semid_ds *sma;
+	struct semid_ds tbuf;
+	int err;
+	int lid = semid % SEMMNI;
+
+	sma = semary[lid].s;
+	err=-EINVAL;
+	if (sma == NULL)
+		goto out_unlock;
+	err=-EIDRM;
 	if (sma->sem_perm.seq != (unsigned int) semid / SEMMNI)
-		goto out;
+		goto out_unlock;
 
+	err = -EACCES;
+	if (ipcperms(&sma->sem_perm, S_IRUGO))
+			goto out_unlock;	
+	
 	switch (cmd) {
-	case GETVAL:
-	case GETPID:
-	case GETNCNT:
-	case GETZCNT:
-	case SETVAL:
+	case GETALL:
+	{
+		ushort *array = arg.array;
+		ushort sem_io[SEMMSL];
+		int i;
+		int nsems = sma->sem_nsems;
+
+		for (i = 0; i < sma->sem_nsems; i++)
+			sem_io[i] = sma->sem_base[i].semval;
+		spin_unlock(&semary[lid].lock);
+		if (copy_to_user (array, sem_io, nsems*sizeof(ushort)))
+			return -EFAULT;
+		return 0;
+	}
+	case IPC_STAT:
+		memset(&tbuf,0,sizeof(tbuf));
+		tbuf.sem_perm   = sma->sem_perm;
+		tbuf.sem_otime  = sma->sem_otime;
+		tbuf.sem_ctime  = sma->sem_ctime;
+		tbuf.sem_nsems  = sma->sem_nsems;
+		spin_unlock(&semary[lid].lock);
+		if (copy_to_user (arg.buf, &tbuf, sizeof(tbuf)))
+			return -EFAULT;
+		return 0;
+default:
 		err = -EINVAL;
-		if (semnum >= nsems)
-			goto out;
-		curr = &sma->sem_base[semnum];
-		break;
 	}
+out_unlock:
+	spin_unlock(&semary[lid].lock);
+	return err;
+
+}
+
+int semctl_locked(int semid, int semnum, int cmd, union semun arg)
+{
+	struct semid_ds *sma;
+	int lid = semid % SEMMNI;
+	struct sem *curr;
+
+	sma = semary[lid].s;
+	if (sma == NULL)
+		return -EINVAL;
+
+	if (ipcperms (&sma->sem_perm, (cmd==SETVAL)?S_IWUGO:S_IRUGO))
+		return -EACCES;
+
+	if (sma->sem_perm.seq != (unsigned int) semid / SEMMNI)
+		return -EIDRM;
+
+	if (semnum >= sma->sem_nsems)
+		return -EINVAL;
+
+	curr = &sma->sem_base[semnum];
 
 	switch (cmd) {
 	case GETVAL:
+		return curr->semval;
 	case GETPID:
+		return curr->sempid & 0xffff;
 	case GETNCNT:
+		return count_semncnt(sma,semnum);
 	case GETZCNT:
-	case GETALL:
-		err = -EACCES;
-		if (ipcperms (ipcp, S_IRUGO))
-			goto out;
-		switch (cmd) {
-		case GETVAL : err = curr->semval; goto out;
-		case GETPID : err = curr->sempid & 0xffff; goto out;
-		case GETNCNT: err = count_semncnt(sma,semnum); goto out;
-		case GETZCNT: err = count_semzcnt(sma,semnum); goto out;
-		case GETALL:
-			array = arg.array;
-			break;
-		}
-		break;
+		return count_semzcnt(sma,semnum);
 	case SETVAL:
-		val = arg.val;
-		err = -ERANGE;
+	{
+		int val = arg.val;
+		struct sem_undo *un;
 		if (val > SEMVMX || val < 0)
-			goto out;
-		break;
-	case IPC_RMID:
-		if (current->euid == ipcp->cuid || 
-		    current->euid == ipcp->uid || capable(CAP_SYS_ADMIN)) {
-			freeary (id);
-			err = 0;
-			goto out;
-		}
-		err = -EPERM;
-		goto out;
-	case SETALL: /* arg is a pointer to an array of ushort */
-		array = arg.array;
-		err = -EFAULT;
-		if (copy_from_user (sem_io, array, nsems*sizeof(ushort)))
-		       goto out;
-		err = 0;
-		for (i = 0; i < nsems; i++)
-			if (sem_io[i] > SEMVMX) {
-				err = -ERANGE;
-				goto out;
-			}
-		break;
-	case IPC_STAT:
-		buf = arg.buf;
-		break;
-	case IPC_SET:
-		buf = arg.buf;
-		err = copy_from_user (&tbuf, buf, sizeof (*buf));
-		if (err)
-			err = -EFAULT;
-		break;
-	}
-
-	err = -EIDRM;
-	if (semary[id] == IPC_UNUSED || semary[id] == IPC_NOID)
-		goto out;
-	if (sma->sem_perm.seq != (unsigned int) semid / SEMMNI)
-		goto out;
+			return -ERANGE;
 
-	switch (cmd) {
-	case GETALL:
-		err = -EACCES;
-		if (ipcperms (ipcp, S_IRUGO))
-			goto out;
-		for (i = 0; i < sma->sem_nsems; i++)
-			sem_io[i] = sma->sem_base[i].semval;
-		if (copy_to_user (array, sem_io, nsems*sizeof(ushort)))
-			err = -EFAULT;
-		break;
-	case SETVAL:
-		err = -EACCES;
-		if (ipcperms (ipcp, S_IWUGO))
-			goto out;
 		for (un = sma->undo; un; un = un->id_next)
 			un->semadj[semnum] = 0;
 		curr->semval = val;
 		sma->sem_ctime = CURRENT_TIME;
 		/* maybe some queued-up processes were waiting for this */
 		update_queue(sma);
-		break;
-	case IPC_SET:
-		if (current->euid == ipcp->cuid || 
-		    current->euid == ipcp->uid || capable(CAP_SYS_ADMIN)) {
-			ipcp->uid = tbuf.sem_perm.uid;
-			ipcp->gid = tbuf.sem_perm.gid;
-			ipcp->mode = (ipcp->mode & ~S_IRWXUGO)
-				| (tbuf.sem_perm.mode & S_IRWXUGO);
-			sma->sem_ctime = CURRENT_TIME;
-			err = 0;
-			goto out;
-		}
-		err = -EPERM;
-		goto out;
-	case IPC_STAT:
-		err = -EACCES;
-		if (ipcperms (ipcp, S_IRUGO))
-			goto out;
-		tbuf.sem_perm   = sma->sem_perm;
-		tbuf.sem_otime  = sma->sem_otime;
-		tbuf.sem_ctime  = sma->sem_ctime;
-		tbuf.sem_nsems  = sma->sem_nsems;
-		if (copy_to_user (buf, &tbuf, sizeof(*buf)))
-			err = -EFAULT;
-		break;
-	case SETALL:
-		err = -EACCES;
+		return 0;
+	}
+	}
+	return -EINVAL;
+}
+
+int semctl_down(int semid, int semnum, int cmd, union semun arg)
+{
+	struct semid_ds *sma;
+	int err;
+	int lid = semid % SEMMNI;
+	struct semid_ds tbuf;
+	struct ipc_perm *ipcp;
+
+	if(cmd == IPC_SET) {
+		if(copy_from_user (&tbuf, arg.buf, sizeof (tbuf)))
+			return -EFAULT;
+	}
+	spin_lock(&semary[lid].lock);
+	sma = semary[lid].s;
+	err=-EINVAL;
+	if (sma == NULL)
+		goto out_unlock;
+	ipcp = &sma->sem_perm;
+	
+	if(cmd == SETALL) {
+		int i;
+		struct sem_undo *un;
+		unsigned int nsems;
+		ushort sem_io[SEMMSL];
+		/* SETALL doesn't belong into this
+		 * group, but I need the semaphore
+		 * for atomically reading nsems
+		 * and changing the semaphore values
+		 */
+		err=-EACCES;
 		if (ipcperms (ipcp, S_IWUGO))
-			goto out;
+			goto out_unlock;
+		nsems=sma->sem_nsems;
+		spin_unlock(&semary[lid].lock);
+
+		if (copy_from_user (sem_io, arg.array, nsems*sizeof(ushort)))
+			return -EFAULT;
+		for (i = 0; i < nsems; i++) {
+			if (sem_io[i] > SEMVMX) {
+				return -ERANGE;
+			}
+		}
+		/* we still own sem_lock, ie neither ownership
+		 * nor permissions of the sem array could
+		 * have changed. Just continue.
+		 */
+		spin_lock(&semary[lid].lock);
 		for (i = 0; i < nsems; i++)
 			sma->sem_base[i].semval = sem_io[i];
 		for (un = sma->undo; un; un = un->id_next)
@@ -604,44 +639,169 @@
 		sma->sem_ctime = CURRENT_TIME;
 		/* maybe some queued-up processes were waiting for this */
 		update_queue(sma);
+		err = 0;
+		goto out_unlock;
+	}
+
+	if (current->euid != ipcp->cuid && 
+	    current->euid != ipcp->uid && !capable(CAP_SYS_ADMIN)) {
+	    	err=-EPERM;
+		goto out_unlock;
+	}
+		
+	if (sma->sem_perm.seq != (unsigned int) semid / SEMMNI) {
+		err=-EIDRM;
+		goto out_unlock;
+	}	
+	switch(cmd){
+	case IPC_RMID:
+		freeary(lid);
+		err = 0;
+		break;
+	case IPC_SET:
+		ipcp->uid = tbuf.sem_perm.uid;
+		ipcp->gid = tbuf.sem_perm.gid;
+		ipcp->mode = (ipcp->mode & ~S_IRWXUGO)
+				| (tbuf.sem_perm.mode & S_IRWXUGO);
+		sma->sem_ctime = CURRENT_TIME;
+		err = 0;
 		break;
 	default:
 		err = -EINVAL;
-		goto out;
 	}
-	err = 0;
-out:
-	unlock_kernel();
+
+out_unlock:
+	spin_unlock(&semary[lid].lock);
 	return err;
 }
 
+asmlinkage long sys_semctl (int semid, int semnum, int cmd, union semun arg)
+{
+	int lid; /* lock id */
+	int err = -EINVAL;
+
+	if (semid < 0 || semnum < 0 || cmd < 0)
+		return -EINVAL;
+
+	lid = semid % SEMMNI;
+
+	switch(cmd) {
+	case IPC_INFO:
+	case SEM_INFO:
+	case SEM_STAT:
+		err = semctl_nolock(semid,semnum,cmd,arg);
+		return err;
+	case GETALL:
+	case IPC_STAT:
+		spin_lock(&semary[lid].lock);
+		err = semctl_locked_unlock(semid,semnum,cmd,arg);
+		return err;
+	case GETVAL:
+	case GETPID:
+	case GETNCNT:
+	case GETZCNT:
+	case SETVAL:
+		spin_lock(&semary[lid].lock);
+		err= semctl_locked(semid,semnum,cmd,arg);
+		spin_unlock(&semary[lid].lock);
+		return err;
+	case SETALL:
+	case IPC_RMID:
+	case IPC_SET:
+		down(&sem_lock);
+		err= semctl_down(semid,semnum,cmd,arg);
+		up(&sem_lock);
+		return err;
+	default:
+		return -EINVAL;
+	}
+}
+
+static struct sem_undo* freeundos(struct semid_ds *sma, struct sem_undo* un)
+{
+	struct sem_undo* u;
+	struct sem_undo** up;
+
+	for(up = ¤t->semundo;(u=*up);up=&u->proc_next) {
+		if(un==u) {
+			un=u->proc_next;
+			*up=un;
+			kfree(u);
+			return un;
+		}
+	}
+	printk ("freeundos undo list error id=%d\n", un->semid);
+	return un->proc_next;
+}
+
+static int alloc_undo(struct semid_ds *sma, struct sem_undo** unp, int semid, int alter)
+{
+	int size;
+	struct sem_undo *un;
+	int error,id;
+	id = (unsigned int) semid % SEMMNI;
+	size = sizeof(struct sem_undo) + sizeof(short)*sma->sem_nsems;
+	spin_unlock(&semary[id].lock);
+
+	un = (struct sem_undo *) kmalloc(size, GFP_KERNEL);
+	spin_lock(&semary[id].lock);
+	if (!un) {
+		return -ENOMEM;
+	}
+	sma = semary[id].s;
+	error = -EIDRM;
+	if (sma == NULL)
+		goto out;
+	if (sma->sem_perm.seq != (unsigned int) semid / SEMMNI)
+		goto out;
+	if (size != sizeof(struct sem_undo) + sizeof(short)*sma->sem_nsems)
+		goto out;
+
+	error = -EACCES;
+	if (ipcperms(&sma->sem_perm, alter ? S_IWUGO : S_IRUGO))
+		goto out;
+	memset(un, 0, size);
+	un->semadj = (short *) &un[1];
+	un->semid = semid;
+	un->proc_next = current->semundo;
+	current->semundo = un;
+	un->id_next = sma->undo;
+	sma->undo = un;
+	*unp = un;
+	return 0;
+out:
+	kfree(un);
+	return error;
+}
+
 asmlinkage long sys_semop (int semid, struct sembuf *tsops, unsigned nsops)
 {
-	int id, size, error = -EINVAL;
+	int id, error = -EINVAL;
 	struct semid_ds *sma;
 	struct sembuf sops[SEMOPM], *sop;
 	struct sem_undo *un;
 	int undos = 0, decrease = 0, alter = 0;
 	struct sem_queue queue;
 
-	lock_kernel();
 	if (nsops < 1 || semid < 0)
-		goto out;
-	error = -E2BIG;
+		return -EINVAL;
+
 	if (nsops > SEMOPM)
-		goto out;
-	error = -EFAULT;
+		return -E2BIG;
 	if (copy_from_user (sops, tsops, nsops * sizeof(*tsops)))
-		goto out;
+		return -EFAULT;
+
 	id = (unsigned int) semid % SEMMNI;
+	spin_lock(&semary[id].lock);
+	sma = semary[id].s;
 	error = -EINVAL;
-	if ((sma = semary[id]) == IPC_UNUSED || sma == IPC_NOID)
+	if (sma == NULL)
 		goto out;
 	error = -EIDRM;
 	if (sma->sem_perm.seq != (unsigned int) semid / SEMMNI)
 		goto out;
 
-		error = -EFBIG;
+	error = -EFBIG;
 	for (sop = sops; sop < sops + nsops; sop++) {
 		if (sop->sem_num >= sma->sem_nsems)
 			goto out;
@@ -649,10 +809,10 @@
 			undos++;
 		if (sop->sem_op < 0)
 			decrease = 1;
-                if (sop->sem_op > 0)
-                        alter = 1;
+		if (sop->sem_op > 0)
+			alter = 1;
 	}
-        alter |= decrease;
+	alter |= decrease;
 
 	error = -EACCES;
 	if (ipcperms(&sma->sem_perm, alter ? S_IWUGO : S_IRUGO))
@@ -661,81 +821,85 @@
 		/* Make sure we have an undo structure
 		 * for this process and this semaphore set.
 		 */
-		for (un = current->semundo; un; un = un->proc_next)
-			if (un->semid == semid)
+		un=current->semundo;
+		while(un != NULL) {
+			if(un->semid==semid)
 				break;
+			if(un->semid==-1)
+				un=freeundos(sma,un);
+			 else
+				un=un->proc_next;
+		}
 		if (!un) {
-			size = sizeof(struct sem_undo) + sizeof(short)*sma->sem_nsems;
-			un = (struct sem_undo *) kmalloc(size, GFP_ATOMIC);
-			if (!un) {
-				error = -ENOMEM;
+			error = alloc_undo(sma,&un,semid,alter);
+			if(error<0)
 				goto out;
-			}
-			memset(un, 0, size);
-			un->semadj = (short *) &un[1];
-			un->semid = semid;
-			un->proc_next = current->semundo;
-			current->semundo = un;
-			un->id_next = sma->undo;
-			sma->undo = un;
 		}
 	} else
 		un = NULL;
 
 	error = try_atomic_semop (sma, sops, nsops, un, current->pid, 0);
 	if (error <= 0)
-                goto update;
+		goto update;
+
+	/* We need to sleep on this operation, so we put the current
+	 * task into the pending queue and go to sleep.
+	 */
+		
+	queue.sma = sma;
+	queue.sops = sops;
+	queue.nsops = nsops;
+	queue.undo = un;
+	queue.pid = current->pid;
+	queue.alter = decrease;
+	queue.id = id;
+	current->semsleeping = &queue;
+	if (alter)
+		append_to_queue(sma ,&queue);
+	else
+		prepend_to_queue(sma ,&queue);
+
+	for (;;) {
+		queue.status = -EINTR;
+		queue.sleeper = current;
+		current->state = TASK_INTERRUPTIBLE;
+		spin_unlock(&semary[id].lock);
 
-        /* We need to sleep on this operation, so we put the current
-         * task into the pending queue and go to sleep.
-         */
-                
-        queue.sma = sma;
-        queue.sops = sops;
-        queue.nsops = nsops;
-        queue.undo = un;
-        queue.pid = current->pid;
-        queue.alter = decrease;
-        current->semsleeping = &queue;
-        if (alter)
-                append_to_queue(sma ,&queue);
-        else
-                prepend_to_queue(sma ,&queue);
-
-        for (;;) {
-                queue.status = -EINTR;
-                init_waitqueue_head(&queue.sleeper);
-                interruptible_sleep_on(&queue.sleeper);
-
-                /*
-                 * If queue.status == 1 we where woken up and
-                 * have to retry else we simply return.
-                 * If an interrupt occurred we have to clean up the
-                 * queue
-                 *
-                 */
-                if (queue.status == 1)
-                {
-                        error = try_atomic_semop (sma, sops, nsops, un,
-                                                  current->pid,0);
-                        if (error <= 0) 
-                                break;
-                } else {
-                        error = queue.status;;
-                        if (queue.prev) /* got Interrupt */
-                                break;
-                        /* Everything done by update_queue */
-                        current->semsleeping = NULL;
-                        goto out;
-                }
-        }
-        current->semsleeping = NULL;
-        remove_from_queue(sma,&queue);
+		schedule();
+
+		/* we can lock the semary even if it was
+		 * deleted.
+		 */
+		spin_lock(&semary[id].lock);
+		/*
+		 * If queue.status == 1 we where woken up and
+		 * have to retry else we simply return.
+		 * If an interrupt occurred we have to clean up the
+		 * queue
+		 *
+		 */
+		if (queue.status == 1)
+		{
+			error = try_atomic_semop (sma, sops, nsops, un,
+						  current->pid,0);
+			if (error <= 0) 
+				break;
+		} else {
+			error = queue.status;
+			if (queue.prev) /* got Interrupt */
+				break;
+			/* Everything done by update_queue */
+			current->semsleeping = NULL;
+			goto out;
+		}
+	}
+	current->semsleeping = NULL;
+	remove_from_queue(sma,&queue);
 update:
-        if (alter)
-                update_queue (sma);
+	if (alter)
+		update_queue (sma);
 out:
-	unlock_kernel();
+	spin_unlock(&semary[id].lock);
 	return error;
 }
 
@@ -761,27 +925,43 @@
 	/* If the current process was sleeping for a semaphore,
 	 * remove it from the queue.
 	 */
+	/* semsleeping is part of "current", and it
+	 * is never modified by another thread.
+	 * No synchronization required.
+	 */
 	if ((q = current->semsleeping)) {
+		spin_lock(&semary[current->semsleeping->id].lock);
+
 		if (q->prev)
 			remove_from_queue(q->sma,q);
 		current->semsleeping = NULL;
+		spin_unlock(&semary[current->semsleeping->id].lock);
 	}
 
 	for (up = ¤t->semundo; (u = *up); *up = u->proc_next, kfree(u)) {
-		if (u->semid == -1)
-			continue;
-		sma = semary[(unsigned int) u->semid % SEMMNI];
-		if (sma == IPC_UNUSED || sma == IPC_NOID)
+		int semid = u->semid;
+		int lid;
+		if(semid == -1)
 			continue;
+		lid = semid % SEMMNI;
+		spin_lock(&semary[lid].lock);
+
+		if (u->semid == -1)
+			goto next_entry;
+
+		sma = semary[lid].s;
+		if (sma == NULL)
+			goto next_entry;
 		if (sma->sem_perm.seq != (unsigned int) u->semid / SEMMNI)
-			continue;
+			goto next_entry;
+
 		/* remove u from the sma->undo list */
 		for (unp = &sma->undo; (un = *unp); unp = &un->id_next) {
 			if (u == un)
 				goto found;
 		}
 		printk ("sem_exit undo list error id=%d\n", u->semid);
-		break;
+		goto next_entry;
 found:
 		*unp = un->id_next;
 		/* perform adjustments registered in u */
@@ -796,6 +976,8 @@
 		sma->sem_otime = CURRENT_TIME;
 		/* maybe some queued-up processes were waiting for this */
 		update_queue(sma);
+next_entry:
+		spin_unlock(&semary[lid].lock);
 	}
 	current->semundo = NULL;
 }
@@ -808,20 +990,23 @@
 	int i, len = 0;
 
 	len += sprintf(buffer, "       key      semid perms nsems   uid   gid  cuid  cgid      otime      ctime\n");
+	down(&sem_lock);
 
 	for(i = 0; i < SEMMNI; i++)
-		if(semary[i] != IPC_UNUSED) {
+		if(semary[i].s != NULL) {
+			spin_lock(&semary[i].lock);
 			len += sprintf(buffer + len, "%10d %10d  %4o %5u %5u %5u %5u %5u %10lu %10lu\n",
-				semary[i]->sem_perm.key,
-				semary[i]->sem_perm.seq * SEMMNI + i,
-				semary[i]->sem_perm.mode,
-				semary[i]->sem_nsems,
-				semary[i]->sem_perm.uid,
-				semary[i]->sem_perm.gid,
-				semary[i]->sem_perm.cuid,
-				semary[i]->sem_perm.cgid,
-				semary[i]->sem_otime,
-				semary[i]->sem_ctime);
+				semary[i].s->sem_perm.key,
+				semary[i].s->sem_perm.seq * SEMMNI + i,
+				semary[i].s->sem_perm.mode,
+				semary[i].s->sem_nsems,
+				semary[i].s->sem_perm.uid,
+				semary[i].s->sem_perm.gid,
+				semary[i].s->sem_perm.cuid,
+				semary[i].s->sem_perm.cgid,
+				semary[i].s->sem_otime,
+				semary[i].s->sem_ctime);
+			spin_unlock(&semary[i].lock);
 
 			pos += len;
 			if(pos < offset) {
@@ -833,6 +1018,7 @@
 		}
 	*eof = 1;
 done:
+	up(&sem_lock);
 	*start = buffer + (offset - begin);
 	len -= (offset - begin);
 	if(len > length)
FUNET's LINUX-ADM group, linux-adm@nic.funet.fi
TCL-scripts by Sam Shen (who was at: slshen@lbl.gov)