proc.c 27.3 KB
Newer Older
1 2
/*-------------------------------------------------------------------------
 *
3
 * proc.c
4
 *	  routines to manage per-process shared memory data structure
5
 *
6
 * Portions Copyright (c) 1996-2001, PostgreSQL Global Development Group
B
Add:  
Bruce Momjian 已提交
7
 * 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.98 2001/01/26 18:23:12 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
 *		ProcSleep(), ProcWakeup(),
22 23
 *		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 52
#include "postgres.h"

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

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

64 65
#include "miscadmin.h"

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

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

75
#include "access/xact.h"
76 77
#include "storage/proc.h"

78

79
int DeadlockTimeout = 1000;
M
 
Marc G. Fournier 已提交
80

81 82 83 84 85 86 87
/* --------------------
 * 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
 * --------------------
 */
88
SPINLOCK	ProcStructLock;
89 90 91

static PROC_HDR *ProcGlobal = NULL;

92
PROC	   *MyProc = NULL;
93

94 95 96
static bool waitingForLock = false;

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

V
Vadim B. Mikheev 已提交
102

103 104
/*
 * InitProcGlobal -
105
 *	  initializes the global process table. We put it here so that
106
 *	  the postmaster can do this initialization. (ProcFreeAllSemaphores needs
107 108 109
 *	  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.)
110 111 112 113 114 115 116 117 118 119 120
 *
 *	  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.
121 122
 */
void
123
InitProcGlobal(int maxBackends)
124
{
125
	bool		found = false;
126

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

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

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

B
Bruce Momjian 已提交
148 149
		/*
		 * Arrange to delete semas on exit --- set this up now so that we
150 151 152 153
		 * 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.
154
		 */
155
		on_shmem_exit(ProcFreeAllSemaphores, 0);
156

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

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

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

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

	SpinAcquire(ProcStructLock);

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

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

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

	myOffset = ProcGlobal->freeProcs;

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

		/*
217 218 219 220
		 * 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).
221 222
		 */

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

231
	/*
232 233 234
	 * 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.
235
	 */
B
Bruce Momjian 已提交
236
	MemSet(MyProc->sLocks, 0, sizeof(MyProc->sLocks));
237 238
	MyProc->sLocks[ProcStructLock] = 1;

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

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

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

	/* -------------------------
274
	 * Install ourselves in the shmem index table.	The name to
275 276 277 278 279 280
	 * 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);
281 282
	if ((!ShmemPIDLookup(MyProcPid, &location)) ||
		(location != MAKE_OFFSET(MyProc)))
283
		elog(STOP, "InitProcess: ShmemPID table broken");
284

285 286 287
	/*
	 * Arrange to clean up at backend exit.
	 */
288
	on_shmem_exit(ProcKill, 0);
289 290 291 292 293 294

	/*
	 * Now that we have a PROC, we could try to acquire locks,
	 * so initialize the deadlock checker.
	 */
	InitDeadLockChecking();
295 296 297 298 299 300 301 302 303 304 305 306 307 308 309 310 311
}

/*
 * 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);
	}
312 313
}

314 315 316
/*
 * Cancel any pending wait for lock, when aborting a transaction.
 *
317 318
 * Returns true if we had been waiting for a lock, else false.
 *
319 320 321 322
 * (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.)
 */
323
bool
324 325 326 327
LockWaitCancel(void)
{
	/* Nothing to do if we weren't waiting for a lock */
	if (!waitingForLock)
328 329
		return false;

330 331 332 333
	waitingForLock = false;

	/* Turn off the deadlock timer, if it's still running (see ProcSleep) */
#ifndef __BEOS__
H
Hiroshi Inoue 已提交
334
	{
335 336 337 338 339
		struct itimerval timeval,
						 dummy;

		MemSet(&timeval, 0, sizeof(struct itimerval));
		setitimer(ITIMER_REAL, &timeval, &dummy);
H
Hiroshi Inoue 已提交
340
	}
341 342 343 344 345 346 347 348 349
#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 已提交
350 351
	UnlockLockTable();

352 353 354 355 356 357 358 359 360 361
	/*
	 * 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);
362 363 364 365 366 367

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

370

371
/*
372 373 374 375 376 377 378 379
 * 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.
380
 *
381
 * Note that user locks are not released in either case.
382 383
 */
void
384
ProcReleaseLocks(bool isCommit)
385
{
386 387
	if (!MyProc)
		return;
388 389 390
	/* If waiting, get off wait queue (should only be needed after error) */
	LockWaitCancel();
	/* Release locks */
391 392
	LockReleaseAll(DEFAULT_LOCKMETHOD, MyProc,
				   !isCommit, GetCurrentTransactionId());
393 394 395 396
}

/*
 * ProcRemove -
397 398
 *	  called by the postmaster to clean up the global tables after a
 *	  backend exits.  This also frees up the proc's wait semaphore.
399 400 401 402
 */
bool
ProcRemove(int pid)
{
403 404
	SHMEM_OFFSET location;
	PROC	   *proc;
405 406 407

	location = ShmemPIDDestroy(pid);
	if (location == INVALID_OFFSET)
408
		return FALSE;
409 410 411 412
	proc = (PROC *) MAKE_PTR(location);

	SpinAcquire(ProcStructLock);

413
	ProcFreeSem(proc->sem.semId, proc->sem.semNum);
414

415
	/* Add PROC struct to freelist so space can be recycled in future */
416 417 418 419 420
	proc->links.next = ProcGlobal->freeProcs;
	ProcGlobal->freeProcs = MAKE_OFFSET(proc);

	SpinRelease(ProcStructLock);

421
	return TRUE;
422 423 424 425
}

/*
 * ProcKill() -- Destroy the per-proc data structure for
426
 *		this process. Release any of its held spin locks.
427 428 429
 *
 * This is done inside the backend process before it exits.
 * ProcRemove, above, will be done by the postmaster afterwards.
430 431
 */
static void
432
ProcKill(void)
433
{
434
	Assert(MyProc);
435

436 437
	/* Release any spinlocks I am holding */
	ProcReleaseSpins(MyProc);
438

439 440
	/* Get off any wait queue I might be on */
	LockWaitCancel();
441

442
	/* Remove from the standard lock table */
443
	LockReleaseAll(DEFAULT_LOCKMETHOD, MyProc, true, InvalidTransactionId);
444

445 446
#ifdef USER_LOCKS
	/* Remove from the user lock table */
447
	LockReleaseAll(USER_LOCKMETHOD, MyProc, true, InvalidTransactionId);
448
#endif
449 450

	MyProc = NULL;
451 452 453 454
}

/*
 * ProcQueue package: routines for putting processes to sleep
455
 *		and  waking them up
456 457 458 459 460 461 462 463
 */

/*
 * 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
 */
464
#ifdef NOT_USED
465
PROC_QUEUE *
466 467
ProcQueueAlloc(char *name)
{
468 469
	bool		found;
	PROC_QUEUE *queue = (PROC_QUEUE *)
470
		ShmemInitStruct(name, sizeof(PROC_QUEUE), &found);
471 472

	if (!queue)
473
		return NULL;
474 475
	if (!found)
		ProcQueueInit(queue);
476
	return queue;
477
}
478

479
#endif
480 481 482 483 484

/*
 * ProcQueueInit -- initialize a shared memory process queue
 */
void
485
ProcQueueInit(PROC_QUEUE *queue)
486
{
487 488
	SHMQueueInit(&(queue->links));
	queue->size = 0;
489 490 491 492 493 494
}


/*
 * ProcSleep -- put a process to sleep
 *
495 496
 * Caller must have set MyProc->heldLocks to reflect locks already held
 * on the lockable object by this process (under all XIDs).
497 498 499
 *
 * Locktable's spinlock must be held at entry, and will be held
 * at exit.
500
 *
501
 * Result: STATUS_OK if we acquired the lock, STATUS_ERROR if not (deadlock).
502
 *
503
 * ASSUME: that no one will fiddle with the queue until after
504
 *		we release the spin lock.
505 506
 *
 * NOTES: The process queue is now a priority queue for locking.
507 508 509
 *
 * P() on the semaphore should put us to sleep.  The process
 * semaphore is normally zero, so when we try to acquire it, we sleep.
510 511
 */
int
512
ProcSleep(LOCKMETHODTABLE *lockMethodTable,
513 514 515
		  LOCKMODE lockmode,
		  LOCK *lock,
		  HOLDER *holder)
516
{
517
	LOCKMETHODCTL *lockctl = lockMethodTable->ctl;
V
Vadim B. Mikheev 已提交
518
	SPINLOCK	spinlock = lockctl->masterLock;
519 520
	PROC_QUEUE *waitQueue = &(lock->waitProcs);
	int			myHeldLocks = MyProc->heldLocks;
521 522
	PROC	   *proc;
	int			i;
523
#ifndef __BEOS__
B
Bruce Momjian 已提交
524 525
	struct itimerval timeval,
				dummy;
526 527 528
#else
    bigtime_t time_interval;
#endif
529

530 531 532 533 534 535 536 537 538 539
	/* ----------------------
	 * Determine where to add myself in the wait queue.
	 *
	 * Normally I should go at the end of the queue.  However, if I already
	 * hold locks that conflict with the request of any previous waiter,
	 * put myself in the queue just in front of the first such waiter.
	 * This is not a necessary step, since deadlock detection would move
	 * me to before that waiter anyway; but it's relatively cheap to detect
	 * such a conflict immediately, and avoid delaying till deadlock timeout.
	 *
540 541 542 543 544 545
	 * Special case: if I find I should go in front of some waiter, check
	 * to see if I conflict with already-held locks or the requests before
	 * that waiter.  If not, then just grant myself the requested lock
	 * immediately.  This is the same as the test for immediate grant in
	 * LockAcquire, except we are only considering the part of the wait queue
	 * before my insertion point.
546 547 548
	 * ----------------------
	 */
	if (myHeldLocks != 0)
V
Vadim B. Mikheev 已提交
549
	{
550 551
		int			aheadRequests = 0;

552 553
		proc = (PROC *) MAKE_PTR(waitQueue->links.next);
		for (i = 0; i < waitQueue->size; i++)
V
Vadim B. Mikheev 已提交
554
		{
555 556
			/* Must he wait for me? */
			if (lockctl->conflictTab[proc->waitLockMode] & myHeldLocks)
V
Vadim B. Mikheev 已提交
557
			{
558 559 560 561 562 563 564
				/* Must I wait for him ? */
				if (lockctl->conflictTab[lockmode] & proc->heldLocks)
				{
					/* Yes, can report deadlock failure immediately */
					MyProc->errType = STATUS_ERROR;
					return STATUS_ERROR;
				}
565 566 567 568 569 570 571 572
				/* I must go before this waiter.  Check special case. */
				if ((lockctl->conflictTab[lockmode] & aheadRequests) == 0 &&
					LockCheckConflicts(lockMethodTable,
									   lockmode,
									   lock,
									   holder,
									   MyProc,
									   NULL) == STATUS_OK)
573
				{
574 575 576
					/* Skip the wait and just grant myself the lock. */
					GrantLock(lock, holder, lockmode);
					return STATUS_OK;
577 578
				}
				/* Break out of loop to put myself before him */
V
Vadim B. Mikheev 已提交
579
				break;
580
			}
581 582
			/* Nope, so advance to next waiter */
			aheadRequests |= (1 << proc->waitLockMode);
583
			proc = (PROC *) MAKE_PTR(proc->links.next);
V
Vadim B. Mikheev 已提交
584
		}
585 586 587 588
		/*
		 * If we fall out of loop normally, proc points to waitQueue head,
		 * so we will insert at tail of queue as desired.
		 */
589 590 591 592 593
	}
	else
	{
		/* I hold no locks, so I can't push in front of anyone. */
		proc = (PROC *) &(waitQueue->links);
V
Vadim B. Mikheev 已提交
594
	}
595 596

	/* -------------------
597
	 * Insert self into queue, ahead of the given proc (or at tail of queue).
598 599
	 * -------------------
	 */
600
	SHMQueueInsertBefore(&(proc->links), &(MyProc->links));
B
Bruce Momjian 已提交
601
	waitQueue->size++;
602

603
	lock->waitMask |= (1 << lockmode);
604

605 606 607 608 609
	/* Set up wait information in PROC object, too */
	MyProc->waitLock = lock;
	MyProc->waitHolder = holder;
	MyProc->waitLockMode = lockmode;

610
	MyProc->errType = STATUS_OK; /* initialize result for success */
611

612 613 614 615 616 617 618 619 620 621 622 623
	/* 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.
	 * -------------------
	 */
624 625
	SpinRelease(spinlock);

626
	/* --------------
627 628 629 630 631 632 633
	 * 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 已提交
634
	 *
635
	 * Need to zero out struct to set the interval and the microseconds fields
B
Bruce Momjian 已提交
636
	 * to 0.
637 638
	 * --------------
	 */
639
#ifndef __BEOS__
B
Bruce Momjian 已提交
640
	MemSet(&timeval, 0, sizeof(struct itimerval));
641 642
	timeval.it_value.tv_sec = DeadlockTimeout / 1000;
	timeval.it_value.tv_usec = (DeadlockTimeout % 1000) * 1000;
643 644
	if (setitimer(ITIMER_REAL, &timeval, &dummy))
		elog(FATAL, "ProcSleep: Unable to set timer for process wakeup");
645
#else
646 647 648
    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");
649
#endif
650

651 652 653 654 655 656 657
	/* --------------
	 * 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...
658 659 660 661 662 663
	 *
	 * 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).
664 665
	 * --------------
	 */
666
	IpcSemaphoreLock(MyProc->sem.semId, MyProc->sem.semNum, true);
667

B
Bruce Momjian 已提交
668
	/* ---------------
669
	 * Disable the timer, if it's still running
B
Bruce Momjian 已提交
670 671
	 * ---------------
	 */
672
#ifndef __BEOS__
673
	MemSet(&timeval, 0, sizeof(struct itimerval));
B
Bruce Momjian 已提交
674
	if (setitimer(ITIMER_REAL, &timeval, &dummy))
675
		elog(FATAL, "ProcSleep: Unable to disable timer for process wakeup");
676 677
#else
    if (set_alarm(B_INFINITE_TIMEOUT, B_PERIODIC_ALARM) < 0)
678
		elog(FATAL, "ProcSleep: Unable to disable timer for process wakeup");
679
#endif
B
Bruce Momjian 已提交
680

681 682 683 684 685
	/*
	 * Now there is nothing for LockWaitCancel to do.
	 */
	waitingForLock = false;

686
	/* ----------------
687 688 689 690
	 * 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.
691 692 693 694
	 * ----------------
	 */
	SpinAcquire(spinlock);

695 696 697 698
	/*
	 * We don't have to do anything else, because the awaker did all the
	 * necessary update of the lock table and MyProc.
	 */
699
	return MyProc->errType;
700 701 702 703 704 705
}


/*
 * ProcWakeup -- wake up a process by releasing its private semaphore.
 *
706
 *	 Also remove the process from the wait queue and set its links invalid.
707
 *	 RETURN: the next process in the wait queue.
708
 */
B
Bruce Momjian 已提交
709
PROC *
710
ProcWakeup(PROC *proc, int errType)
711
{
712
	PROC	   *retProc;
713 714 715

	/* assume that spinlock has been acquired */

716
	/* Proc should be sleeping ... */
717 718
	if (proc->links.prev == INVALID_OFFSET ||
		proc->links.next == INVALID_OFFSET)
719
		return (PROC *) NULL;
720

721 722
	/* Save next process before we zap the list link */
	retProc = (PROC *) MAKE_PTR(proc->links.next);
723

724
	/* Remove process from wait queue */
725
	SHMQueueDelete(&(proc->links));
726
	(proc->waitLock->waitProcs.size)--;
727

728 729 730
	/* Clean up process' state and pass it the ok/fail signal */
	proc->waitLock = NULL;
	proc->waitHolder = NULL;
731 732
	proc->errType = errType;

733
	/* And awaken it */
734
	IpcSemaphoreUnlock(proc->sem.semId, proc->sem.semNum);
735 736

	return retProc;
737 738 739 740
}

/*
 * ProcLockWakeup -- routine for waking up processes when a lock is
741 742
 *		released (or a prior waiter is aborted).  Scan all waiters
 *		for lock, waken any that are no longer blocked.
743
 */
744 745
void
ProcLockWakeup(LOCKMETHODTABLE *lockMethodTable, LOCK *lock)
746
{
747 748 749
	LOCKMETHODCTL *lockctl = lockMethodTable->ctl;
	PROC_QUEUE *waitQueue = &(lock->waitProcs);
	int			queue_size = waitQueue->size;
750
	PROC	   *proc;
751
	int			aheadRequests = 0;
M
 
Marc G. Fournier 已提交
752

753
	Assert(queue_size >= 0);
754

755 756
	if (queue_size == 0)
		return;
757

758
	proc = (PROC *) MAKE_PTR(waitQueue->links.next);
759

760 761
	while (queue_size-- > 0)
	{
762
		LOCKMODE lockmode = proc->waitLockMode;
M
 
Marc G. Fournier 已提交
763 764

		/*
765 766
		 * Waken if (a) doesn't conflict with requests of earlier waiters,
		 * and (b) doesn't conflict with already-held locks.
M
 
Marc G. Fournier 已提交
767
		 */
768
		if ((lockctl->conflictTab[lockmode] & aheadRequests) == 0 &&
769 770 771 772 773 774
			LockCheckConflicts(lockMethodTable,
							   lockmode,
							   lock,
							   proc->waitHolder,
							   proc,
							   NULL) == STATUS_OK)
M
 
Marc G. Fournier 已提交
775
		{
776 777 778 779 780 781 782 783
			/* OK to waken */
			GrantLock(lock, proc->waitHolder, lockmode);
			proc = ProcWakeup(proc, STATUS_OK);
			/*
			 * ProcWakeup removes proc from the lock's waiting process queue
			 * and returns the next proc in chain; don't use proc's next-link,
			 * because it's been cleared.
			 */
M
 
Marc G. Fournier 已提交
784
		}
785
		else
786
		{
787 788
			/* Cannot wake this guy. Remember his request for later checks. */
			aheadRequests |= (1 << lockmode);
789
			proc = (PROC *) MAKE_PTR(proc->links.next);
790
		}
M
 
Marc G. Fournier 已提交
791
	}
792 793

	Assert(waitQueue->size >= 0);
794 795 796
}

/* --------------------
797
 * We only get to this routine if we got SIGALRM after DeadlockTimeout
798 799 800 801
 * 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.
802 803
 * --------------------
 */
804
void
805
HandleDeadLock(SIGNAL_ARGS)
806
{
807
	int			save_errno = errno;
808

809 810 811 812 813 814 815
	/*
	 * 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.
	 */
816 817 818 819 820 821 822 823 824
	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.
	 *
825 826 827
	 * 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.
828 829 830 831 832 833
	 * ---------------------
	 */
	if (MyProc->links.prev == INVALID_OFFSET ||
		MyProc->links.next == INVALID_OFFSET)
	{
		UnlockLockTable();
834
		errno = save_errno;
835 836 837
		return;
	}

838 839 840
#ifdef LOCK_DEBUG
    if (Debug_deadlocks)
        DumpAllLocks();
841 842
#endif

843
	if (!DeadLockCheck(MyProc))
B
Bruce Momjian 已提交
844
	{
845
		/* No deadlock, so keep waiting */
B
Bruce Momjian 已提交
846
		UnlockLockTable();
847
		errno = save_errno;
B
Bruce Momjian 已提交
848 849 850
		return;
	}

851
	/* ------------------------
852 853 854
	 * Oops.  We have a deadlock.
	 *
	 * Get this process out of wait state.
855 856
	 * ------------------------
	 */
857 858 859 860 861 862 863 864
	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;
865 866

	/* ------------------
867
	 * Unlock my semaphore so that the interrupted ProcSleep() call can finish.
868 869
	 * ------------------
	 */
870
	IpcSemaphoreUnlock(MyProc->sem.semId, MyProc->sem.semNum);
871

872 873 874 875 876 877 878 879 880 881
	/* ------------------
	 * 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.
	 * ------------------
882 883
	 */
	UnlockLockTable();
884
	errno = save_errno;
885 886 887
}

void
888
ProcReleaseSpins(PROC *proc)
889
{
890
	int			i;
891 892 893 894 895 896 897

	if (!proc)
		proc = MyProc;

	if (!proc)
		return;
	for (i = 0; i < (int) MAX_SPINS; i++)
898
	{
899
		if (proc->sLocks[i])
900
		{
901 902
			Assert(proc->sLocks[i] == 1);
			SpinRelease(i);
903 904
		}
	}
H
 
Hiroshi Inoue 已提交
905
	AbortBufferIO();
906 907 908
}

/*****************************************************************************
909
 *
910 911 912
 *****************************************************************************/

/*
913
 * ProcGetNewSemIdAndNum -
914
 *	  scan the free semaphore bitmap and allocate a single semaphore from
915
 *	  a semaphore set.
916 917
 */
static void
918
ProcGetNewSemIdAndNum(IpcSemaphoreId *semId, int *semNum)
919
{
920
	int			i;
921
	IpcSemaphoreId *procSemIds = ProcGlobal->procSemIds;
922
	int32	   *freeSemMap = ProcGlobal->freeSemMap;
923
	int32		fullmask = (1 << PROC_NSEMS_PER_SET) - 1;
924

925 926 927 928
	/*
	 * we hold ProcStructLock when entering this routine. We scan through
	 * the bitmap to look for a free semaphore.
	 */
929

930
	for (i = 0; i < PROC_SEM_MAP_ENTRIES; i++)
931
	{
932 933
		int			mask = 1;
		int			j;
934 935

		if (freeSemMap[i] == fullmask)
936
			continue;			/* this set is fully allocated */
937 938
		if (procSemIds[i] < 0)
			continue;			/* this set hasn't been initialized */
939 940 941 942 943 944 945

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

				/*
946
				 * a free semaphore found. Mark it as allocated.
947
				 */
948
				freeSemMap[i] |= mask;
949

950
				*semId = procSemIds[i];
951 952 953 954 955
				*semNum = j;
				return;
			}
			mask <<= 1;
		}
956 957
	}

958
	/* if we reach here, all the semaphores are in use. */
959
	elog(ERROR, "ProcGetNewSemIdAndNum: cannot allocate a free semaphore");
960 961 962 963
}

/*
 * ProcFreeSem -
964
 *	  free up our semaphore in the semaphore set.
965 966
 */
static void
967
ProcFreeSem(IpcSemaphoreId semId, int semNum)
968
{
969
	int32		mask;
970
	int			i;
971

972
	mask = ~(1 << semNum);
973

974 975 976 977 978 979 980 981 982
	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);
983 984 985 986
}

/*
 * ProcFreeAllSemaphores -
987 988 989
 *	  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.
990
 */
991
static void
992
ProcFreeAllSemaphores(void)
993
{
994
	int			i;
995

996
	for (i = 0; i < PROC_SEM_MAP_ENTRIES; i++)
997
	{
998 999
		if (ProcGlobal->procSemIds[i] >= 0)
			IpcSemaphoreKill(ProcGlobal->procSemIds[i]);
1000
	}
1001
}