proc.c 28.5 KB
Newer Older
1 2
/*-------------------------------------------------------------------------
 *
3
 * proc.c
4
 *	  routines to manage per-process shared memory data structure
5
 *
B
Add:  
Bruce Momjian 已提交
6 7
 * Portions Copyright (c) 1996-2000, PostgreSQL, Inc
 * Portions Copyright (c) 1994, Regents of the University of California
8 9 10
 *
 *
 * IDENTIFICATION
11
 *	  $Header: /cvsroot/pgsql/src/backend/storage/lmgr/proc.c,v 1.95 2001/01/22 22:30:06 tgl Exp $
12 13 14 15
 *
 *-------------------------------------------------------------------------
 */
/*
16 17
 *	Each postgres backend gets one of these.  We'll use it to
 *	clean up after the process should the process suddenly die.
18 19 20
 *
 *
 * Interface (a):
21 22 23
 *		ProcSleep(), ProcWakeup(), ProcWakeupNext(),
 *		ProcQueueAlloc() -- create a shm queue for sleeping processes
 *		ProcQueueInit() -- create a queue without allocing memory
24 25 26 27 28 29 30 31
 *
 * Locking and waiting for buffers can cause the backend to be
 * put to sleep.  Whoever releases the lock, etc. wakes the
 * process up again (and gives it an error code so it knows
 * whether it was awoken on an error condition).
 *
 * Interface (b):
 *
32 33
 * ProcReleaseLocks -- frees the locks associated with current transaction
 *
34
 * ProcKill -- destroys the shared memory state (and locks)
35
 *		associated with the process.
36 37
 *
 * 5/15/91 -- removed the buffer pool based lock chain in favor
38 39 40 41 42 43
 *		of a shared memory lock chain.	The write-protection is
 *		more expensive if the lock chain is in the buffer pool.
 *		The only reason I kept the lock chain in the buffer pool
 *		in the first place was to allow the lock table to grow larger
 *		than available shared memory and that isn't going to work
 *		without a lot of unimplemented support anyway.
44 45
 *
 * 4/7/95 -- instead of allocating a set of 1 semaphore per process, we
46 47 48 49
 *		allocate a semaphore from a set of PROC_NSEMS_PER_SET semaphores
 *		shared among backends (we keep a few sets of semaphores around).
 *		This is so that we can support more backends. (system-wide semaphore
 *		sets run out pretty fast.)				  -ay 4/95
50
 *
51
 * $Header: /cvsroot/pgsql/src/backend/storage/lmgr/proc.c,v 1.95 2001/01/22 22:30:06 tgl Exp $
52
 */
53 54
#include "postgres.h"

55
#include <errno.h>
56 57
#include <sys/time.h>
#include <unistd.h>
58
#include <signal.h>
59
#include <sys/types.h>
M
Marc G. Fournier 已提交
60

61
#if defined(solaris_sparc) || defined(__CYGWIN__)
62 63 64 65
#include <sys/ipc.h>
#include <sys/sem.h>
#endif

66 67
#include "miscadmin.h"

68 69 70 71
#if defined(__darwin__)
#include "port/darwin/sem.h"
#endif

72
/* In Ultrix and QNX, sem.h must be included after ipc.h */
73
#ifdef HAVE_SYS_SEM_H
74
#include <sys/sem.h>
75
#endif
B
Bruce Momjian 已提交
76

77
#include "access/xact.h"
78 79
#include "storage/proc.h"

80

81
int DeadlockTimeout = 1000;
M
 
Marc G. Fournier 已提交
82

83 84 85 86 87 88 89
/* --------------------
 * Spin lock for manipulating the shared process data structure:
 * ProcGlobal.... Adding an extra spin lock seemed like the smallest
 * hack to get around reading and updating this structure in shared
 * memory. -mer 17 July 1991
 * --------------------
 */
90
SPINLOCK	ProcStructLock;
91 92 93

static PROC_HDR *ProcGlobal = NULL;

94
PROC	   *MyProc = NULL;
95

96 97 98
static bool waitingForLock = false;

static void ProcKill(void);
99 100
static void ProcGetNewSemIdAndNum(IpcSemaphoreId *semId, int *semNum);
static void ProcFreeSem(IpcSemaphoreId semId, int semNum);
101 102 103
static void ZeroProcSemaphore(PROC *proc);
static void ProcFreeAllSemaphores(void);

V
Vadim B. Mikheev 已提交
104

105 106
/*
 * InitProcGlobal -
107
 *	  initializes the global process table. We put it here so that
108
 *	  the postmaster can do this initialization. (ProcFreeAllSemaphores needs
109 110 111
 *	  to read this table on exiting the postmaster. If we have the first
 *	  backend do this, starting up and killing the postmaster without
 *	  starting any backends will be a problem.)
112 113 114 115 116 117 118 119 120 121 122
 *
 *	  We also allocate all the per-process semaphores we will need to support
 *	  the requested number of backends.  We used to allocate semaphores
 *	  only when backends were actually started up, but that is bad because
 *	  it lets Postgres fail under load --- a lot of Unix systems are
 *	  (mis)configured with small limits on the number of semaphores, and
 *	  running out when trying to start another backend is a common failure.
 *	  So, now we grab enough semaphores to support the desired max number
 *	  of backends immediately at initialization --- if the sysadmin has set
 *	  MaxBackends higher than his kernel will support, he'll find out sooner
 *	  rather than later.
123 124
 */
void
125
InitProcGlobal(int maxBackends)
126
{
127
	bool		found = false;
128

129 130
	/* attach to the free list */
	ProcGlobal = (PROC_HDR *)
131
		ShmemInitStruct("Proc Header", sizeof(PROC_HDR), &found);
132

133 134
	/* --------------------
	 * We're the first - initialize.
135 136
	 * XXX if found should ever be true, it is a sign of impending doom ...
	 * ought to complain if so?
137 138 139
	 * --------------------
	 */
	if (!found)
140
	{
141
		int			i;
142

143
		ProcGlobal->freeProcs = INVALID_OFFSET;
144 145 146
		for (i = 0; i < PROC_SEM_MAP_ENTRIES; i++)
		{
			ProcGlobal->procSemIds[i] = -1;
147
			ProcGlobal->freeSemMap[i] = 0;
148
		}
149

B
Bruce Momjian 已提交
150 151
		/*
		 * Arrange to delete semas on exit --- set this up now so that we
152 153 154 155
		 * will clean up if pre-allocation fails.  We use our own freeproc,
		 * rather than IpcSemaphoreCreate's removeOnExit option, because
		 * we don't want to fill up the on_shmem_exit list with a separate
		 * entry for each semaphore set.
156
		 */
157
		on_shmem_exit(ProcFreeAllSemaphores, 0);
158

B
Bruce Momjian 已提交
159
		/*
160
		 * Pre-create the semaphores for the first maxBackends processes.
161
		 */
162 163 164
		Assert(maxBackends > 0 && maxBackends <= MAXBACKENDS);

		for (i = 0; i < ((maxBackends-1)/PROC_NSEMS_PER_SET+1); i++)
165
		{
166 167 168 169 170 171 172
			IpcSemaphoreId		semId;

			semId = IpcSemaphoreCreate(PROC_NSEMS_PER_SET,
									   IPCProtection,
									   1,
									   false);
			ProcGlobal->procSemIds[i] = semId;
173
		}
174 175 176 177 178 179 180 181 182
	}
}

/* ------------------------
 * InitProc -- create a per-process data structure for this process
 * used by the lock manager on semaphore queues.
 * ------------------------
 */
void
183
InitProcess(void)
184
{
185 186 187
	bool		found = false;
	unsigned long location,
				myOffset;
188 189 190

	SpinAcquire(ProcStructLock);

191
	/* attach to the ProcGlobal structure */
192
	ProcGlobal = (PROC_HDR *)
193
		ShmemInitStruct("Proc Header", sizeof(PROC_HDR), &found);
194
	if (!found)
195
	{
196
		/* this should not happen. InitProcGlobal() is called before this. */
197
		elog(STOP, "InitProcess: Proc Header uninitialized");
198
	}
199 200

	if (MyProc != NULL)
201
	{
202
		SpinRelease(ProcStructLock);
203
		elog(ERROR, "ProcInit: you already exist");
204
	}
205

206
	/* try to get a proc struct from the free list first */
207 208 209 210

	myOffset = ProcGlobal->freeProcs;

	if (myOffset != INVALID_OFFSET)
211
	{
212 213 214 215 216 217 218
		MyProc = (PROC *) MAKE_PTR(myOffset);
		ProcGlobal->freeProcs = MyProc->links.next;
	}
	else
	{

		/*
219 220 221 222
		 * have to allocate one.  We can't use the normal shmem index
		 * table mechanism because the proc structure is stored by PID
		 * instead of by a global name (need to look it up by PID when we
		 * cleanup dead processes).
223 224
		 */

225
		MyProc = (PROC *) ShmemAlloc(sizeof(PROC));
226
		if (!MyProc)
227
		{
228 229
			SpinRelease(ProcStructLock);
			elog(FATAL, "cannot create new proc: out of memory");
230 231
		}
	}
232

233
	/*
234 235 236
	 * zero out the spin lock counts and set the sLocks field for
	 * ProcStructLock to 1 as we have acquired this spinlock above but
	 * didn't record it since we didn't have MyProc until now.
237
	 */
B
Bruce Momjian 已提交
238
	MemSet(MyProc->sLocks, 0, sizeof(MyProc->sLocks));
239 240
	MyProc->sLocks[ProcStructLock] = 1;

241 242 243
	/*
	 * Set up a wait-semaphore for the proc.
	 */
244 245
	if (IsUnderPostmaster)
	{
246
		ProcGetNewSemIdAndNum(&MyProc->sem.semId, &MyProc->sem.semNum);
247 248 249 250
		/*
		 * we might be reusing a semaphore that belongs to a dead backend.
		 * So be careful and reinitialize its value here.
		 */
251
		ZeroProcSemaphore(MyProc);
252 253
	}
	else
254
	{
255
		MyProc->sem.semId = -1;
256 257
		MyProc->sem.semNum = -1;
	}
258

259 260
	SHMQueueElemInit(&(MyProc->links));
	MyProc->errType = NO_ERROR;
B
Bruce Momjian 已提交
261
	MyProc->pid = MyProcPid;
262
	MyProc->databaseId = MyDatabaseId;
263
	MyProc->xid = InvalidTransactionId;
264
	MyProc->xmin = InvalidTransactionId;
265 266 267
	MyProc->waitLock = NULL;
	MyProc->waitHolder = NULL;
	SHMQueueInit(&(MyProc->procHolders));
268

269 270 271
	/* ----------------------
	 * Release the lock.
	 * ----------------------
272
	 */
273
	SpinRelease(ProcStructLock);
274 275

	/* -------------------------
276
	 * Install ourselves in the shmem index table.	The name to
277 278 279 280 281 282
	 * use is determined by the OS-assigned process id.  That
	 * allows the cleanup process to find us after any untimely
	 * exit.
	 * -------------------------
	 */
	location = MAKE_OFFSET(MyProc);
283 284
	if ((!ShmemPIDLookup(MyProcPid, &location)) ||
		(location != MAKE_OFFSET(MyProc)))
285
		elog(STOP, "InitProcess: ShmemPID table broken");
286

287 288 289 290 291 292 293 294 295 296 297 298 299 300 301 302 303 304
	on_shmem_exit(ProcKill, 0);
}

/*
 * Initialize the proc's wait-semaphore to count zero.
 */
static void
ZeroProcSemaphore(PROC *proc)
{
	union semun		semun;

	semun.val = 0;
	if (semctl(proc->sem.semId, proc->sem.semNum, SETVAL, semun) < 0)
	{
		fprintf(stderr, "ZeroProcSemaphore: semctl(id=%d,SETVAL) failed: %s\n",
				proc->sem.semId, strerror(errno));
		proc_exit(255);
	}
305 306
}

307 308 309 310
/*
 * Remove a proc from the wait-queue it is on
 * (caller must know it is on one).
 * Locktable lock must be held by caller.
311 312
 *
 * NB: this does not remove the process' holder object, nor the lock object,
313 314
 * even though their counts might now have gone to zero.  That will happen
 * during a subsequent LockReleaseAll call, which we expect will happen
315 316
 * during transaction cleanup.  (Removal of a proc from its wait queue by
 * this routine can only happen if we are aborting the transaction.)
H
Hiroshi Inoue 已提交
317
 */
318 319
static void
RemoveFromWaitQueue(PROC *proc)
H
Hiroshi Inoue 已提交
320
{
321 322
	LOCK   *waitLock = proc->waitLock;
	LOCKMODE lockmode = proc->waitLockMode;
323

324 325 326 327 328 329 330 331 332
	/* Make sure proc is waiting */
	Assert(proc->links.next != INVALID_OFFSET);
	Assert(waitLock);
	Assert(waitLock->waitProcs.size > 0);

	/* Remove proc from lock's wait queue */
	SHMQueueDelete(&(proc->links));
	waitLock->waitProcs.size--;

333 334 335 336 337 338
	/* Undo increments of request counts by waiting process */
	Assert(waitLock->nRequested > 0);
	Assert(waitLock->nRequested > proc->waitLock->nGranted);
	waitLock->nRequested--;
	Assert(waitLock->requested[lockmode] > 0);
	waitLock->requested[lockmode]--;
339
	/* don't forget to clear waitMask bit if appropriate */
340
	if (waitLock->granted[lockmode] == waitLock->requested[lockmode])
341 342 343 344 345 346 347 348 349 350 351 352 353
		waitLock->waitMask &= ~(1 << lockmode);

	/* Clean up the proc's own state */
	proc->waitLock = NULL;
	proc->waitHolder = NULL;

	/* See if any other waiters for the lock can be woken up now */
	ProcLockWakeup(LOCK_LOCKMETHOD(*waitLock), waitLock);
}

/*
 * Cancel any pending wait for lock, when aborting a transaction.
 *
354 355
 * Returns true if we had been waiting for a lock, else false.
 *
356 357 358 359
 * (Normally, this would only happen if we accept a cancel/die
 * interrupt while waiting; but an elog(ERROR) while waiting is
 * within the realm of possibility, too.)
 */
360
bool
361 362 363 364
LockWaitCancel(void)
{
	/* Nothing to do if we weren't waiting for a lock */
	if (!waitingForLock)
365 366
		return false;

367 368 369 370
	waitingForLock = false;

	/* Turn off the deadlock timer, if it's still running (see ProcSleep) */
#ifndef __BEOS__
H
Hiroshi Inoue 已提交
371
	{
372 373 374 375 376
		struct itimerval timeval,
						 dummy;

		MemSet(&timeval, 0, sizeof(struct itimerval));
		setitimer(ITIMER_REAL, &timeval, &dummy);
H
Hiroshi Inoue 已提交
377
	}
378 379 380 381 382 383 384 385 386
#else
	/* BeOS doesn't have setitimer, but has set_alarm */
    set_alarm(B_INFINITE_TIMEOUT, B_PERIODIC_ALARM);
#endif /* __BEOS__ */

	/* Unlink myself from the wait queue, if on it (might not be anymore!) */
	LockLockTable();
	if (MyProc->links.next != INVALID_OFFSET)
		RemoveFromWaitQueue(MyProc);
H
Hiroshi Inoue 已提交
387 388
	UnlockLockTable();

389 390 391 392 393 394 395 396 397 398
	/*
	 * Reset the proc wait semaphore to zero.  This is necessary in the
	 * scenario where someone else granted us the lock we wanted before we
	 * were able to remove ourselves from the wait-list.  The semaphore will
	 * have been bumped to 1 by the would-be grantor, and since we are no
	 * longer going to wait on the sema, we have to force it back to zero.
	 * Otherwise, our next attempt to wait for a lock will fall through
	 * prematurely.
	 */
	ZeroProcSemaphore(MyProc);
399 400 401 402 403 404

	/*
	 * Return true even if we were kicked off the lock before we were
	 * able to remove ourselves.
	 */
	return true;
H
Hiroshi Inoue 已提交
405
}
406

407

408
/*
409 410 411 412 413 414 415 416
 * ProcReleaseLocks() -- release locks associated with current transaction
 *			at transaction commit or abort
 *
 * At commit, we release only locks tagged with the current transaction's XID,
 * leaving those marked with XID 0 (ie, session locks) undisturbed.  At abort,
 * we release all locks including XID 0, because we need to clean up after
 * a failure.  This logic will need extension if we ever support nested
 * transactions.
417
 *
418
 * Note that user locks are not released in either case.
419 420
 */
void
421
ProcReleaseLocks(bool isCommit)
422
{
423 424
	if (!MyProc)
		return;
425 426 427
	/* If waiting, get off wait queue (should only be needed after error) */
	LockWaitCancel();
	/* Release locks */
428 429
	LockReleaseAll(DEFAULT_LOCKMETHOD, MyProc,
				   !isCommit, GetCurrentTransactionId());
430 431 432 433
}

/*
 * ProcRemove -
434 435
 *	  called by the postmaster to clean up the global tables after a
 *	  backend exits.  This also frees up the proc's wait semaphore.
436 437 438 439
 */
bool
ProcRemove(int pid)
{
440 441
	SHMEM_OFFSET location;
	PROC	   *proc;
442 443 444

	location = ShmemPIDDestroy(pid);
	if (location == INVALID_OFFSET)
445
		return FALSE;
446 447 448 449
	proc = (PROC *) MAKE_PTR(location);

	SpinAcquire(ProcStructLock);

450
	ProcFreeSem(proc->sem.semId, proc->sem.semNum);
451

452
	/* Add PROC struct to freelist so space can be recycled in future */
453 454 455 456 457
	proc->links.next = ProcGlobal->freeProcs;
	ProcGlobal->freeProcs = MAKE_OFFSET(proc);

	SpinRelease(ProcStructLock);

458
	return TRUE;
459 460 461 462
}

/*
 * ProcKill() -- Destroy the per-proc data structure for
463
 *		this process. Release any of its held spin locks.
464 465 466
 *
 * This is done inside the backend process before it exits.
 * ProcRemove, above, will be done by the postmaster afterwards.
467 468
 */
static void
469
ProcKill(void)
470
{
471
	Assert(MyProc);
472

473 474
	/* Release any spinlocks I am holding */
	ProcReleaseSpins(MyProc);
475

476 477
	/* Get off any wait queue I might be on */
	LockWaitCancel();
478

479
	/* Remove from the standard lock table */
480
	LockReleaseAll(DEFAULT_LOCKMETHOD, MyProc, true, InvalidTransactionId);
481

482 483
#ifdef USER_LOCKS
	/* Remove from the user lock table */
484
	LockReleaseAll(USER_LOCKMETHOD, MyProc, true, InvalidTransactionId);
485
#endif
486 487

	MyProc = NULL;
488 489 490 491
}

/*
 * ProcQueue package: routines for putting processes to sleep
492
 *		and  waking them up
493 494 495 496 497 498 499 500
 */

/*
 * ProcQueueAlloc -- alloc/attach to a shared memory process queue
 *
 * Returns: a pointer to the queue or NULL
 * Side Effects: Initializes the queue if we allocated one
 */
501
#ifdef NOT_USED
502
PROC_QUEUE *
503 504
ProcQueueAlloc(char *name)
{
505 506
	bool		found;
	PROC_QUEUE *queue = (PROC_QUEUE *)
507
		ShmemInitStruct(name, sizeof(PROC_QUEUE), &found);
508 509

	if (!queue)
510
		return NULL;
511 512
	if (!found)
		ProcQueueInit(queue);
513
	return queue;
514
}
515

516
#endif
517 518 519 520 521

/*
 * ProcQueueInit -- initialize a shared memory process queue
 */
void
522
ProcQueueInit(PROC_QUEUE *queue)
523
{
524 525
	SHMQueueInit(&(queue->links));
	queue->size = 0;
526 527 528 529 530 531 532
}


/*
 * ProcSleep -- put a process to sleep
 *
 * P() on the semaphore should put us to sleep.  The process
533 534 535 536
 * semaphore is normally zero, so when we try to acquire it, we sleep.
 *
 * Locktable's spinlock must be held at entry, and will be held
 * at exit.
537
 *
538 539
 * Result is NO_ERROR if we acquired the lock, STATUS_ERROR if not (deadlock).
 *
540
 * ASSUME: that no one will fiddle with the queue until after
541
 *		we release the spin lock.
542 543 544 545
 *
 * NOTES: The process queue is now a priority queue for locking.
 */
int
546 547 548 549
ProcSleep(LOCKMETHODCTL *lockctl,
		  LOCKMODE lockmode,
		  LOCK *lock,
		  HOLDER *holder)
550
{
551
	PROC_QUEUE *waitQueue = &(lock->waitProcs);
V
Vadim B. Mikheev 已提交
552
	SPINLOCK	spinlock = lockctl->masterLock;
553
	int			myMask = (1 << lockmode);
V
Vadim B. Mikheev 已提交
554
	int			waitMask = lock->waitMask;
555 556
	PROC	   *proc;
	int			i;
557
	int			aheadGranted[MAX_LOCKMODES];
558
	bool		selfConflict = (lockctl->conflictTab[lockmode] & myMask),
V
Vadim B. Mikheev 已提交
559
				prevSame = false;
560
#ifndef __BEOS__
B
Bruce Momjian 已提交
561 562
	struct itimerval timeval,
				dummy;
563 564 565
#else
    bigtime_t time_interval;
#endif
566

567
	proc = (PROC *) MAKE_PTR(waitQueue->links.next);
568

V
Vadim B. Mikheev 已提交
569
	/* if we don't conflict with any waiter - be first in queue */
570
	if (!(lockctl->conflictTab[lockmode] & waitMask))
V
Vadim B. Mikheev 已提交
571
		goto ins;
572

573
	/* otherwise, determine where we should go into the queue */
V
Vadim B. Mikheev 已提交
574
	for (i = 1; i < MAX_LOCKMODES; i++)
575 576
		aheadGranted[i] = lock->granted[i];
	(aheadGranted[lockmode])++;
577

V
Vadim B. Mikheev 已提交
578 579
	for (i = 0; i < waitQueue->size; i++)
	{
580 581 582 583
		LOCKMODE	procWaitMode = proc->waitLockMode;

		/* must I wait for him ? */
		if (lockctl->conflictTab[lockmode] & proc->heldLocks)
V
Vadim B. Mikheev 已提交
584 585
		{
			/* is he waiting for me ? */
586
			if (lockctl->conflictTab[procWaitMode] & MyProc->heldLocks)
V
Vadim B. Mikheev 已提交
587
			{
588
				/* Yes, report deadlock failure */
V
Vadim B. Mikheev 已提交
589
				MyProc->errType = STATUS_ERROR;
590
				return STATUS_ERROR;
V
Vadim B. Mikheev 已提交
591
			}
592
			/* I must go after him in queue - so continue loop */
V
Vadim B. Mikheev 已提交
593
		}
594 595
		/* if he waits for me, go before him in queue */
		else if (lockctl->conflictTab[procWaitMode] & MyProc->heldLocks)
V
Vadim B. Mikheev 已提交
596 597
			break;
		/* if conflicting locks requested */
598
		else if (lockctl->conflictTab[procWaitMode] & myMask)
V
Vadim B. Mikheev 已提交
599
		{
B
Bruce Momjian 已提交
600

V
Vadim B. Mikheev 已提交
601
			/*
B
Bruce Momjian 已提交
602
			 * If I request non self-conflicting lock and there are others
603
			 * requesting the same lock just before this guy - stop here.
V
Vadim B. Mikheev 已提交
604 605 606 607
			 */
			if (!selfConflict && prevSame)
				break;
		}
B
Bruce Momjian 已提交
608

V
Vadim B. Mikheev 已提交
609
		/*
610 611
		 * Last attempt to not move any further to the back of the queue:
		 * if we don't conflict with remaining waiters, stop here.
V
Vadim B. Mikheev 已提交
612
		 */
613
		else if (!(lockctl->conflictTab[lockmode] & waitMask))
V
Vadim B. Mikheev 已提交
614
			break;
615

616 617 618 619 620
		/* Move past this guy, and update state accordingly */
		prevSame = (procWaitMode == lockmode);
		(aheadGranted[procWaitMode])++;
		if (aheadGranted[procWaitMode] == lock->requested[procWaitMode])
			waitMask &= ~(1 << procWaitMode);
621
		proc = (PROC *) MAKE_PTR(proc->links.next);
V
Vadim B. Mikheev 已提交
622
	}
623

V
Vadim B. Mikheev 已提交
624
ins:;
625
	/* -------------------
626
	 * Insert self into queue, ahead of the given proc (or at tail of queue).
627 628
	 * -------------------
	 */
629
	SHMQueueInsertBefore(&(proc->links), &(MyProc->links));
B
Bruce Momjian 已提交
630
	waitQueue->size++;
631

V
Vadim B. Mikheev 已提交
632
	lock->waitMask |= myMask;
633

634 635 636 637 638 639
	/* Set up wait information in PROC object, too */
	MyProc->waitLock = lock;
	MyProc->waitHolder = holder;
	MyProc->waitLockMode = lockmode;
	/* We assume the caller set up MyProc->heldLocks */

640 641
	MyProc->errType = NO_ERROR;		/* initialize result for success */

642 643 644 645 646 647 648 649 650 651 652 653
	/* mark that we are waiting for a lock */
	waitingForLock = true;

	/* -------------------
	 * Release the locktable's spin lock.
	 *
	 * NOTE: this may also cause us to exit critical-section state,
	 * possibly allowing a cancel/die interrupt to be accepted.
	 * This is OK because we have recorded the fact that we are waiting for
	 * a lock, and so LockWaitCancel will clean up if cancel/die happens.
	 * -------------------
	 */
654 655
	SpinRelease(spinlock);

656
	/* --------------
657 658 659 660 661 662 663
	 * Set timer so we can wake up after awhile and check for a deadlock.
	 * If a deadlock is detected, the handler releases the process's
	 * semaphore and sets MyProc->errType = STATUS_ERROR, allowing us to
	 * know that we must report failure rather than success.
	 *
	 * By delaying the check until we've waited for a bit, we can avoid
	 * running the rather expensive deadlock-check code in most cases.
B
Bruce Momjian 已提交
664 665 666
	 *
	 * Need to zero out struct to set the interval and the micro seconds fields
	 * to 0.
667 668
	 * --------------
	 */
669
#ifndef __BEOS__
B
Bruce Momjian 已提交
670
	MemSet(&timeval, 0, sizeof(struct itimerval));
671 672
	timeval.it_value.tv_sec = DeadlockTimeout / 1000;
	timeval.it_value.tv_usec = (DeadlockTimeout % 1000) * 1000;
673 674
	if (setitimer(ITIMER_REAL, &timeval, &dummy))
		elog(FATAL, "ProcSleep: Unable to set timer for process wakeup");
675
#else
676 677 678
    time_interval = DeadlockTimeout * 1000000; /* usecs */
	if (set_alarm(time_interval, B_ONE_SHOT_RELATIVE_ALARM) < 0)
		elog(FATAL, "ProcSleep: Unable to set timer for process wakeup");
679
#endif
680

681 682 683 684 685 686 687
	/* --------------
	 * If someone wakes us between SpinRelease and IpcSemaphoreLock,
	 * IpcSemaphoreLock will not block.  The wakeup is "saved" by
	 * the semaphore implementation.  Note also that if HandleDeadLock
	 * is invoked but does not detect a deadlock, IpcSemaphoreLock()
	 * will continue to wait.  There used to be a loop here, but it
	 * was useless code...
688 689 690 691 692 693
	 *
	 * We pass interruptOK = true, which eliminates a window in which
	 * cancel/die interrupts would be held off undesirably.  This is a
	 * promise that we don't mind losing control to a cancel/die interrupt
	 * here.  We don't, because we have no state-change work to do after
	 * being granted the lock (the grantor did it all).
694 695
	 * --------------
	 */
696
	IpcSemaphoreLock(MyProc->sem.semId, MyProc->sem.semNum, true);
697

B
Bruce Momjian 已提交
698
	/* ---------------
699
	 * Disable the timer, if it's still running
B
Bruce Momjian 已提交
700 701
	 * ---------------
	 */
702
#ifndef __BEOS__
703
	MemSet(&timeval, 0, sizeof(struct itimerval));
B
Bruce Momjian 已提交
704
	if (setitimer(ITIMER_REAL, &timeval, &dummy))
705
		elog(FATAL, "ProcSleep: Unable to disable timer for process wakeup");
706 707
#else
    if (set_alarm(B_INFINITE_TIMEOUT, B_PERIODIC_ALARM) < 0)
708
		elog(FATAL, "ProcSleep: Unable to disable timer for process wakeup");
709
#endif
B
Bruce Momjian 已提交
710

711 712 713 714 715
	/*
	 * Now there is nothing for LockWaitCancel to do.
	 */
	waitingForLock = false;

716
	/* ----------------
717 718 719 720
	 * Re-acquire the locktable's spin lock.
	 *
	 * We could accept a cancel/die interrupt here.  That's OK because
	 * the lock is now registered as being held by this process.
721 722 723 724
	 * ----------------
	 */
	SpinAcquire(spinlock);

725 726 727 728
	/*
	 * We don't have to do anything else, because the awaker did all the
	 * necessary update of the lock table and MyProc.
	 */
729
	return MyProc->errType;
730 731 732 733 734 735
}


/*
 * ProcWakeup -- wake up a process by releasing its private semaphore.
 *
736
 *	 Also remove the process from the wait queue and set its links invalid.
737
 *	 RETURN: the next process in the wait queue.
738
 */
B
Bruce Momjian 已提交
739
PROC *
740
ProcWakeup(PROC *proc, int errType)
741
{
742
	PROC	   *retProc;
743 744 745

	/* assume that spinlock has been acquired */

746
	/* Proc should be sleeping ... */
747 748
	if (proc->links.prev == INVALID_OFFSET ||
		proc->links.next == INVALID_OFFSET)
749
		return (PROC *) NULL;
750

751 752
	/* Save next process before we zap the list link */
	retProc = (PROC *) MAKE_PTR(proc->links.next);
753

754
	/* Remove process from wait queue */
755
	SHMQueueDelete(&(proc->links));
756
	(proc->waitLock->waitProcs.size)--;
757

758 759 760
	/* Clean up process' state and pass it the ok/fail signal */
	proc->waitLock = NULL;
	proc->waitHolder = NULL;
761 762
	proc->errType = errType;

763
	/* And awaken it */
764
	IpcSemaphoreUnlock(proc->sem.semId, proc->sem.semNum);
765 766

	return retProc;
767 768 769 770
}

/*
 * ProcLockWakeup -- routine for waking up processes when a lock is
771
 *		released.
772 773
 */
int
774
ProcLockWakeup(LOCKMETHOD lockmethod, LOCK *lock)
775
{
776
	PROC_QUEUE *queue = &(lock->waitProcs);
777
	PROC	   *proc;
778 779
	int			awoken = 0;
	LOCKMODE	last_lockmode = 0;
M
 
Marc G. Fournier 已提交
780 781
	int			queue_size = queue->size;

782
	Assert(queue_size >= 0);
783

784
	if (!queue_size)
785
		return STATUS_NOT_FOUND;
786

787
	proc = (PROC *) MAKE_PTR(queue->links.next);
788

789 790 791 792 793 794 795 796 797 798
	while (queue_size-- > 0)
	{
		if (proc->waitLockMode == last_lockmode)
		{
			/*
			 * This proc will conflict as the previous one did, don't even
			 * try.
			 */
			goto nextProc;
		}
M
 
Marc G. Fournier 已提交
799 800

		/*
V
Vadim B. Mikheev 已提交
801
		 * Does this proc conflict with locks held by others ?
M
 
Marc G. Fournier 已提交
802 803
		 */
		if (LockResolveConflicts(lockmethod,
804
								 proc->waitLockMode,
805
								 lock,
806 807 808
								 proc->waitHolder,
								 proc,
								 NULL) != STATUS_OK)
M
 
Marc G. Fournier 已提交
809
		{
810 811
			/* Yes.  Quit if we already awoke at least one process. */
			if (awoken != 0)
V
Vadim B. Mikheev 已提交
812
				break;
813 814 815
			/* Otherwise, see if any later waiters can be awoken. */
			last_lockmode = proc->waitLockMode;
			goto nextProc;
M
 
Marc G. Fournier 已提交
816
		}
817 818

		/*
819
		 * OK to wake up this sleeping process.
820
		 */
821 822 823
		GrantLock(lock, proc->waitHolder, proc->waitLockMode);
		proc = ProcWakeup(proc, NO_ERROR);
		awoken++;
824 825

		/*
826
		 * ProcWakeup removes proc from the lock's waiting process queue
827 828
		 * and returns the next proc in chain; don't use proc's next-link,
		 * because it's been cleared.
829
		 */
830
		continue;
831

832
nextProc:
833
		proc = (PROC *) MAKE_PTR(proc->links.next);
834
	}
835

M
 
Marc G. Fournier 已提交
836 837
	Assert(queue->size >= 0);

838
	if (awoken)
839
		return STATUS_OK;
840 841
	else
	{
842
		/* Something is still blocking us.	May have deadlocked. */
843 844 845
#ifdef LOCK_DEBUG
		if (lock->tag.lockmethod == USER_LOCKMETHOD ? Trace_userlocks : Trace_locks)
		{
846 847
			elog(DEBUG, "ProcLockWakeup: lock(%lx) can't wake up any process",
				 MAKE_OFFSET(lock));
848
			if (Debug_deadlocks)
849
				DumpAllLocks();
850
		}
M
 
Marc G. Fournier 已提交
851
#endif
852
		return STATUS_NOT_FOUND;
M
 
Marc G. Fournier 已提交
853
	}
854 855 856
}

/* --------------------
857
 * We only get to this routine if we got SIGALRM after DeadlockTimeout
858 859 860 861
 * while waiting for a lock to be released by some other process.  Look
 * to see if there's a deadlock; if not, just return and continue waiting.
 * If we have a real deadlock, remove ourselves from the lock's wait queue
 * and signal an error to ProcSleep.
862 863
 * --------------------
 */
864
void
865
HandleDeadLock(SIGNAL_ARGS)
866
{
867
	int			save_errno = errno;
868

869 870 871 872 873 874 875
	/*
	 * Acquire locktable lock.  Note that the SIGALRM interrupt had better
	 * not be enabled anywhere that this process itself holds the locktable
	 * lock, else this will wait forever.  Also note that this calls
	 * SpinAcquire which creates a critical section, so that this routine
	 * cannot be interrupted by cancel/die interrupts.
	 */
876 877 878 879 880 881 882 883 884
	LockLockTable();

	/* ---------------------
	 * Check to see if we've been awoken by anyone in the interim.
	 *
	 * If we have we can return and resume our transaction -- happy day.
	 * Before we are awoken the process releasing the lock grants it to
	 * us so we know that we don't have to wait anymore.
	 *
885 886 887
	 * We check by looking to see if we've been unlinked from the wait queue.
	 * This is quicker than checking our semaphore's state, since no kernel
	 * call is needed, and it is safe because we hold the locktable lock.
888 889 890 891 892 893
	 * ---------------------
	 */
	if (MyProc->links.prev == INVALID_OFFSET ||
		MyProc->links.next == INVALID_OFFSET)
	{
		UnlockLockTable();
894
		errno = save_errno;
895 896 897
		return;
	}

898 899 900
#ifdef LOCK_DEBUG
    if (Debug_deadlocks)
        DumpAllLocks();
901 902
#endif

B
Bruce Momjian 已提交
903
	if (!DeadLockCheck(MyProc, MyProc->waitLock))
B
Bruce Momjian 已提交
904
	{
905
		/* No deadlock, so keep waiting */
B
Bruce Momjian 已提交
906
		UnlockLockTable();
907
		errno = save_errno;
B
Bruce Momjian 已提交
908 909 910
		return;
	}

911
	/* ------------------------
912 913 914
	 * Oops.  We have a deadlock.
	 *
	 * Get this process out of wait state.
915 916
	 * ------------------------
	 */
917 918 919 920 921 922 923 924
	RemoveFromWaitQueue(MyProc);

	/* -------------
	 * Set MyProc->errType to STATUS_ERROR so that ProcSleep will
	 * report an error after we return from this signal handler.
	 * -------------
	 */
	MyProc->errType = STATUS_ERROR;
925 926

	/* ------------------
927
	 * Unlock my semaphore so that the interrupted ProcSleep() call can finish.
928 929
	 * ------------------
	 */
930
	IpcSemaphoreUnlock(MyProc->sem.semId, MyProc->sem.semNum);
931

932 933 934 935 936 937 938 939 940 941
	/* ------------------
	 * We're done here.  Transaction abort caused by the error that ProcSleep
	 * will raise will cause any other locks we hold to be released, thus
	 * allowing other processes to wake up; we don't need to do that here.
	 * NOTE: an exception is that releasing locks we hold doesn't consider
	 * the possibility of waiters that were blocked behind us on the lock
	 * we just failed to get, and might now be wakable because we're not
	 * in front of them anymore.  However, RemoveFromWaitQueue took care of
	 * waking up any such processes.
	 * ------------------
942 943
	 */
	UnlockLockTable();
944
	errno = save_errno;
945 946 947
}

void
948
ProcReleaseSpins(PROC *proc)
949
{
950
	int			i;
951 952 953 954 955 956 957

	if (!proc)
		proc = MyProc;

	if (!proc)
		return;
	for (i = 0; i < (int) MAX_SPINS; i++)
958
	{
959
		if (proc->sLocks[i])
960
		{
961 962
			Assert(proc->sLocks[i] == 1);
			SpinRelease(i);
963 964
		}
	}
H
 
Hiroshi Inoue 已提交
965
	AbortBufferIO();
966 967 968
}

/*****************************************************************************
969
 *
970 971 972
 *****************************************************************************/

/*
973
 * ProcGetNewSemIdAndNum -
974
 *	  scan the free semaphore bitmap and allocate a single semaphore from
975
 *	  a semaphore set.
976 977
 */
static void
978
ProcGetNewSemIdAndNum(IpcSemaphoreId *semId, int *semNum)
979
{
980
	int			i;
981
	IpcSemaphoreId *procSemIds = ProcGlobal->procSemIds;
982
	int32	   *freeSemMap = ProcGlobal->freeSemMap;
983
	int32		fullmask = (1 << PROC_NSEMS_PER_SET) - 1;
984

985 986 987 988
	/*
	 * we hold ProcStructLock when entering this routine. We scan through
	 * the bitmap to look for a free semaphore.
	 */
989

990
	for (i = 0; i < PROC_SEM_MAP_ENTRIES; i++)
991
	{
992 993
		int			mask = 1;
		int			j;
994 995

		if (freeSemMap[i] == fullmask)
996
			continue;			/* this set is fully allocated */
997 998
		if (procSemIds[i] < 0)
			continue;			/* this set hasn't been initialized */
999 1000 1001 1002 1003 1004 1005

		for (j = 0; j < PROC_NSEMS_PER_SET; j++)
		{
			if ((freeSemMap[i] & mask) == 0)
			{

				/*
1006
				 * a free semaphore found. Mark it as allocated.
1007
				 */
1008
				freeSemMap[i] |= mask;
1009

1010
				*semId = procSemIds[i];
1011 1012 1013 1014 1015
				*semNum = j;
				return;
			}
			mask <<= 1;
		}
1016 1017
	}

1018
	/* if we reach here, all the semaphores are in use. */
1019
	elog(ERROR, "ProcGetNewSemIdAndNum: cannot allocate a free semaphore");
1020 1021 1022 1023
}

/*
 * ProcFreeSem -
1024
 *	  free up our semaphore in the semaphore set.
1025 1026
 */
static void
1027
ProcFreeSem(IpcSemaphoreId semId, int semNum)
1028
{
1029
	int32		mask;
1030
	int			i;
1031

1032
	mask = ~(1 << semNum);
1033

1034 1035 1036 1037 1038 1039 1040 1041 1042
	for (i = 0; i < PROC_SEM_MAP_ENTRIES; i++)
	{
		if (ProcGlobal->procSemIds[i] == semId)
		{
			ProcGlobal->freeSemMap[i] &= mask;
			return;
		}
	}
	fprintf(stderr, "ProcFreeSem: no ProcGlobal entry for semId %d\n", semId);
1043 1044 1045 1046
}

/*
 * ProcFreeAllSemaphores -
1047 1048 1049
 *	  called at shmem_exit time, ie when exiting the postmaster or
 *	  destroying shared state for a failed set of backends.
 *	  Free up all the semaphores allocated to the lmgrs of the backends.
1050
 */
1051
static void
1052
ProcFreeAllSemaphores(void)
1053
{
1054
	int			i;
1055

1056
	for (i = 0; i < PROC_SEM_MAP_ENTRIES; i++)
1057
	{
1058 1059
		if (ProcGlobal->procSemIds[i] >= 0)
			IpcSemaphoreKill(ProcGlobal->procSemIds[i]);
1060
	}
1061
}