proc.c 29.2 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.107 2001/09/07 00:27:29 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
#include "storage/proc.h"
77
#include "storage/sinval.h"
78

79

B
Bruce Momjian 已提交
80
int			DeadlockTimeout = 1000;
M
 
Marc G. Fournier 已提交
81

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

static PROC_HDR *ProcGlobal = NULL;

93
PROC	   *MyProc = NULL;
94

95
static bool waitingForLock = false;
96
static bool waitingForSignal = false;
97 98

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 128
	int			semMapEntries;
	Size		procGlobalSize;
129
	bool		found = false;
130

131 132 133 134 135 136
	/* Compute size for ProcGlobal structure */
	Assert(maxBackends > 0);
	semMapEntries = PROC_SEM_MAP_ENTRIES(maxBackends);
	procGlobalSize = sizeof(PROC_HDR) + (semMapEntries-1) * sizeof(SEM_MAP_ENTRY);

	/* Create or attach to the ProcGlobal shared structure */
137
	ProcGlobal = (PROC_HDR *)
138
		ShmemInitStruct("Proc Header", procGlobalSize, &found);
139

140 141
	/* --------------------
	 * We're the first - initialize.
142 143
	 * XXX if found should ever be true, it is a sign of impending doom ...
	 * ought to complain if so?
144 145 146
	 * --------------------
	 */
	if (!found)
147
	{
148
		int			i;
149

150
		ProcGlobal->freeProcs = INVALID_OFFSET;
151 152 153
		ProcGlobal->semMapEntries = semMapEntries;

		for (i = 0; i < semMapEntries; i++)
154
		{
155 156
			ProcGlobal->procSemMap[i].procSemId = -1;
			ProcGlobal->procSemMap[i].freeSemMap = 0;
157
		}
158

B
Bruce Momjian 已提交
159 160
		/*
		 * Arrange to delete semas on exit --- set this up now so that we
B
Bruce Momjian 已提交
161 162 163 164
		 * 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.
165
		 */
166
		on_shmem_exit(ProcFreeAllSemaphores, 0);
167

B
Bruce Momjian 已提交
168
		/*
169
		 * Pre-create the semaphores.
170
		 */
171
		for (i = 0; i < semMapEntries; i++)
172
		{
B
Bruce Momjian 已提交
173
			IpcSemaphoreId semId;
174 175 176 177 178

			semId = IpcSemaphoreCreate(PROC_NSEMS_PER_SET,
									   IPCProtection,
									   1,
									   false);
179
			ProcGlobal->procSemMap[i].procSemId = semId;
180
		}
181 182 183 184 185 186 187 188 189
	}
}

/* ------------------------
 * InitProc -- create a per-process data structure for this process
 * used by the lock manager on semaphore queues.
 * ------------------------
 */
void
190
InitProcess(void)
191
{
192 193 194 195 196 197 198 199 200 201 202
	SHMEM_OFFSET	myOffset;

	/*
	 * ProcGlobal should be set by a previous call to InitProcGlobal
	 * (if we are a backend, we inherit this by fork() from the postmaster).
	 */
	if (ProcGlobal == NULL)
		elog(STOP, "InitProcess: Proc Header uninitialized");

	if (MyProc != NULL)
		elog(ERROR, "InitProcess: you already exist");
203

204 205 206 207 208 209 210 211 212 213
	/*
	 * ProcStructLock protects the freelist of PROC entries and the map
	 * of free semaphores.  Note that when we acquire it here, we do not
	 * have a PROC entry and so the ownership of the spinlock is not
	 * recorded anywhere; even if it was, until we register ProcKill as
	 * an on_shmem_exit callback, there is no exit hook that will cause
	 * owned spinlocks to be released.  Upshot: during the first part of
	 * this routine, be careful to release the lock manually before any
	 * elog(), else you'll have a stuck spinlock to add to your woes.
	 */
214 215
	SpinAcquire(ProcStructLock);

216
	/* try to get a proc struct from the free list first */
217 218 219
	myOffset = ProcGlobal->freeProcs;

	if (myOffset != INVALID_OFFSET)
220
	{
221 222 223 224 225 226
		MyProc = (PROC *) MAKE_PTR(myOffset);
		ProcGlobal->freeProcs = MyProc->links.next;
	}
	else
	{
		/*
227 228 229 230
		 * 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).
231
		 */
232
		MyProc = (PROC *) ShmemAlloc(sizeof(PROC));
233
		if (!MyProc)
234
		{
235 236
			SpinRelease(ProcStructLock);
			elog(FATAL, "cannot create new proc: out of memory");
237 238
		}
	}
239

240
	/*
241
	 * Initialize all fields of MyProc.
242
	 */
243
	SHMQueueElemInit(&(MyProc->links));
244 245
	MyProc->sem.semId = -1;		/* no wait-semaphore acquired yet */
	MyProc->sem.semNum = -1;
246
	MyProc->errType = STATUS_OK;
247
	MyProc->xid = InvalidTransactionId;
248
	MyProc->xmin = InvalidTransactionId;
249
	MyProc->logRec.xrecoff = 0;
250 251
	MyProc->waitLock = NULL;
	MyProc->waitHolder = NULL;
252 253
	MyProc->pid = MyProcPid;
	MyProc->databaseId = MyDatabaseId;
254
	SHMQueueInit(&(MyProc->procHolders));
255 256 257 258 259 260 261
	/*
	 * 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.
	 */
	MemSet(MyProc->sLocks, 0, sizeof(MyProc->sLocks));
	MyProc->sLocks[ProcStructLock] = 1;
262

263
	/*
264 265 266
	 * Arrange to clean up at backend exit.  Once we do this, owned
	 * spinlocks will be released on exit, and so we can be a lot less
	 * tense about errors.
267
	 */
268
	on_shmem_exit(ProcKill, 0);
269

270
	/*
271 272
	 * Set up a wait-semaphore for the proc.  (We rely on ProcKill to clean
	 * up if this fails.)
273 274 275
	 */
	if (IsUnderPostmaster)
		ProcGetNewSemIdAndNum(&MyProc->sem.semId, &MyProc->sem.semNum);
276 277 278 279 280 281 282 283 284

	/* Done with freelist and sem map */
	SpinRelease(ProcStructLock);

	/*
	 * We might be reusing a semaphore that belongs to a dead backend.
	 * So be careful and reinitialize its value here.
	 */
	if (MyProc->sem.semId >= 0)
285 286
		ZeroProcSemaphore(MyProc);

287
	/*
B
Bruce Momjian 已提交
288 289
	 * Now that we have a PROC, we could try to acquire locks, so
	 * initialize the deadlock checker.
290 291
	 */
	InitDeadLockChecking();
292 293 294 295 296 297 298 299
}

/*
 * Initialize the proc's wait-semaphore to count zero.
 */
static void
ZeroProcSemaphore(PROC *proc)
{
B
Bruce Momjian 已提交
300
	union semun semun;
301 302 303 304 305 306 307 308

	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);
	}
309 310
}

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

327 328 329 330
	waitingForLock = false;

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

		MemSet(&timeval, 0, sizeof(struct itimerval));
		setitimer(ITIMER_REAL, &timeval, &dummy);
H
Hiroshi Inoue 已提交
337
	}
338 339
#else
	/* BeOS doesn't have setitimer, but has set_alarm */
B
Bruce Momjian 已提交
340 341
	set_alarm(B_INFINITE_TIMEOUT, B_PERIODIC_ALARM);
#endif	 /* __BEOS__ */
342 343 344 345 346

	/* 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 已提交
347 348
	UnlockLockTable();

349 350 351
	/*
	 * Reset the proc wait semaphore to zero.  This is necessary in the
	 * scenario where someone else granted us the lock we wanted before we
B
Bruce Momjian 已提交
352 353 354 355 356
	 * 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.
357 358
	 */
	ZeroProcSemaphore(MyProc);
359 360

	/*
B
Bruce Momjian 已提交
361 362
	 * Return true even if we were kicked off the lock before we were able
	 * to remove ourselves.
363 364
	 */
	return true;
H
Hiroshi Inoue 已提交
365
}
366

367

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


/*
 * ProcKill() -- Destroy the per-proc data structure for
395
 *		this process. Release any of its held spin locks.
396 397
 */
static void
398
ProcKill(void)
399
{
400
	Assert(MyProc != NULL);
401

402 403
	/* Release any spinlocks I am holding */
	ProcReleaseSpins(MyProc);
404

405 406
	/* Get off any wait queue I might be on */
	LockWaitCancel();
407

408
	/* Remove from the standard lock table */
409
	LockReleaseAll(DEFAULT_LOCKMETHOD, MyProc, true, InvalidTransactionId);
410

411 412
#ifdef USER_LOCKS
	/* Remove from the user lock table */
413
	LockReleaseAll(USER_LOCKMETHOD, MyProc, true, InvalidTransactionId);
414
#endif
415

416 417
	SpinAcquire(ProcStructLock);

418 419 420
	/* Free up my wait semaphore, if I got one */
	if (MyProc->sem.semId >= 0)
		ProcFreeSem(MyProc->sem.semId, MyProc->sem.semNum);
421 422 423 424 425

	/* Add PROC struct to freelist so space can be recycled in future */
	MyProc->links.next = ProcGlobal->freeProcs;
	ProcGlobal->freeProcs = MAKE_OFFSET(MyProc);

426
	/* PROC struct isn't mine anymore; stop tracking spinlocks with it! */
427
	MyProc = NULL;
428 429

	SpinRelease(ProcStructLock);
430 431
}

432

433 434
/*
 * ProcQueue package: routines for putting processes to sleep
435
 *		and  waking them up
436 437 438 439 440 441 442 443
 */

/*
 * 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
 */
444
#ifdef NOT_USED
445
PROC_QUEUE *
446 447
ProcQueueAlloc(char *name)
{
448 449
	bool		found;
	PROC_QUEUE *queue = (PROC_QUEUE *)
B
Bruce Momjian 已提交
450
	ShmemInitStruct(name, sizeof(PROC_QUEUE), &found);
451 452

	if (!queue)
453
		return NULL;
454 455
	if (!found)
		ProcQueueInit(queue);
456
	return queue;
457
}
458

459
#endif
460 461 462 463 464

/*
 * ProcQueueInit -- initialize a shared memory process queue
 */
void
465
ProcQueueInit(PROC_QUEUE *queue)
466
{
467 468
	SHMQueueInit(&(queue->links));
	queue->size = 0;
469 470 471 472 473 474
}


/*
 * ProcSleep -- put a process to sleep
 *
475 476
 * Caller must have set MyProc->heldLocks to reflect locks already held
 * on the lockable object by this process (under all XIDs).
477 478 479
 *
 * Locktable's spinlock must be held at entry, and will be held
 * at exit.
480
 *
481
 * Result: STATUS_OK if we acquired the lock, STATUS_ERROR if not (deadlock).
482
 *
483
 * ASSUME: that no one will fiddle with the queue until after
484
 *		we release the spin lock.
485 486
 *
 * NOTES: The process queue is now a priority queue for locking.
487 488 489
 *
 * P() on the semaphore should put us to sleep.  The process
 * semaphore is normally zero, so when we try to acquire it, we sleep.
490 491
 */
int
492
ProcSleep(LOCKMETHODTABLE *lockMethodTable,
493 494 495
		  LOCKMODE lockmode,
		  LOCK *lock,
		  HOLDER *holder)
496
{
497
	LOCKMETHODCTL *lockctl = lockMethodTable->ctl;
V
Vadim B. Mikheev 已提交
498
	SPINLOCK	spinlock = lockctl->masterLock;
499 500
	PROC_QUEUE *waitQueue = &(lock->waitProcs);
	int			myHeldLocks = MyProc->heldLocks;
501
	bool		early_deadlock = false;
502 503
	PROC	   *proc;
	int			i;
504
#ifndef __BEOS__
B
Bruce Momjian 已提交
505 506
	struct itimerval timeval,
				dummy;
507
#else
B
Bruce Momjian 已提交
508
	bigtime_t	time_interval;
509
#endif
510

511
	/*
512 513 514 515 516 517
	 * 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
518 519 520
	 * me to before that waiter anyway; but it's relatively cheap to
	 * detect such a conflict immediately, and avoid delaying till
	 * deadlock timeout.
521
	 *
522 523
	 * 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
524 525
	 * that waiter.  If not, then just grant myself the requested lock
	 * immediately.  This is the same as the test for immediate grant in
526 527
	 * LockAcquire, except we are only considering the part of the wait
	 * queue before my insertion point.
528 529
	 */
	if (myHeldLocks != 0)
V
Vadim B. Mikheev 已提交
530
	{
531 532
		int			aheadRequests = 0;

533 534
		proc = (PROC *) MAKE_PTR(waitQueue->links.next);
		for (i = 0; i < waitQueue->size; i++)
V
Vadim B. Mikheev 已提交
535
		{
536 537
			/* Must he wait for me? */
			if (lockctl->conflictTab[proc->waitLockMode] & myHeldLocks)
V
Vadim B. Mikheev 已提交
538
			{
539 540 541
				/* Must I wait for him ? */
				if (lockctl->conflictTab[lockmode] & proc->heldLocks)
				{
542 543 544 545 546 547 548 549
					/*
					 * Yes, so we have a deadlock.  Easiest way to clean up
					 * correctly is to call RemoveFromWaitQueue(), but we
					 * can't do that until we are *on* the wait queue.
					 * So, set a flag to check below, and break out of loop.
					 */
					early_deadlock = true;
					break;
550
				}
551 552 553 554 555 556 557 558
				/* I must go before this waiter.  Check special case. */
				if ((lockctl->conflictTab[lockmode] & aheadRequests) == 0 &&
					LockCheckConflicts(lockMethodTable,
									   lockmode,
									   lock,
									   holder,
									   MyProc,
									   NULL) == STATUS_OK)
559
				{
560 561 562
					/* Skip the wait and just grant myself the lock. */
					GrantLock(lock, holder, lockmode);
					return STATUS_OK;
563 564
				}
				/* Break out of loop to put myself before him */
V
Vadim B. Mikheev 已提交
565
				break;
566
			}
567 568
			/* Nope, so advance to next waiter */
			aheadRequests |= (1 << proc->waitLockMode);
569
			proc = (PROC *) MAKE_PTR(proc->links.next);
V
Vadim B. Mikheev 已提交
570
		}
B
Bruce Momjian 已提交
571

572 573 574 575
		/*
		 * If we fall out of loop normally, proc points to waitQueue head,
		 * so we will insert at tail of queue as desired.
		 */
576 577 578 579 580
	}
	else
	{
		/* I hold no locks, so I can't push in front of anyone. */
		proc = (PROC *) &(waitQueue->links);
V
Vadim B. Mikheev 已提交
581
	}
582

583 584 585
	/*
	 * Insert self into queue, ahead of the given proc (or at tail of
	 * queue).
586
	 */
587
	SHMQueueInsertBefore(&(proc->links), &(MyProc->links));
B
Bruce Momjian 已提交
588
	waitQueue->size++;
589

590
	lock->waitMask |= (1 << lockmode);
591

592 593 594 595 596
	/* Set up wait information in PROC object, too */
	MyProc->waitLock = lock;
	MyProc->waitHolder = holder;
	MyProc->waitLockMode = lockmode;

597 598 599 600 601 602 603 604 605 606 607 608 609
	MyProc->errType = STATUS_OK; /* initialize result for success */

	/*
	 * If we detected deadlock, give up without waiting.  This must agree
	 * with HandleDeadLock's recovery code, except that we shouldn't release
	 * the semaphore since we haven't tried to lock it yet.
	 */
	if (early_deadlock)
	{
		RemoveFromWaitQueue(MyProc);
		MyProc->errType = STATUS_ERROR;
		return STATUS_ERROR;
	}
610

611 612 613
	/* mark that we are waiting for a lock */
	waitingForLock = true;

614
	/*
615 616
	 * Release the locktable's spin lock.
	 *
617 618 619 620
	 * 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.
621
	 */
622 623
	SpinRelease(spinlock);

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

648
	/*
649
	 * If someone wakes us between SpinRelease and IpcSemaphoreLock,
650 651 652 653 654
	 * 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...
655 656 657 658 659 660
	 *
	 * 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).
661
	 */
662
	IpcSemaphoreLock(MyProc->sem.semId, MyProc->sem.semNum, true);
663

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

676 677 678 679 680
	/*
	 * Now there is nothing for LockWaitCancel to do.
	 */
	waitingForLock = false;

681
	/*
682 683
	 * Re-acquire the locktable's spin lock.
	 *
684 685
	 * We could accept a cancel/die interrupt here.  That's OK because the
	 * lock is now registered as being held by this process.
686 687 688
	 */
	SpinAcquire(spinlock);

689 690 691 692
	/*
	 * We don't have to do anything else, because the awaker did all the
	 * necessary update of the lock table and MyProc.
	 */
693
	return MyProc->errType;
694 695 696 697 698 699
}


/*
 * ProcWakeup -- wake up a process by releasing its private semaphore.
 *
700
 *	 Also remove the process from the wait queue and set its links invalid.
701
 *	 RETURN: the next process in the wait queue.
702 703 704 705
 *
 * XXX: presently, this code is only used for the "success" case, and only
 * works correctly for that case.  To clean up in failure case, would need
 * to twiddle the lock's request counts too --- see RemoveFromWaitQueue.
706
 */
B
Bruce Momjian 已提交
707
PROC *
708
ProcWakeup(PROC *proc, int errType)
709
{
710
	PROC	   *retProc;
711 712 713

	/* assume that spinlock has been acquired */

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

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

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

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

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

	return retProc;
735 736 737 738
}

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

751
	Assert(queue_size >= 0);
752

753 754
	if (queue_size == 0)
		return;
755

756
	proc = (PROC *) MAKE_PTR(waitQueue->links.next);
757

758 759
	while (queue_size-- > 0)
	{
B
Bruce Momjian 已提交
760
		LOCKMODE	lockmode = proc->waitLockMode;
M
 
Marc G. Fournier 已提交
761 762

		/*
763 764
		 * Waken if (a) doesn't conflict with requests of earlier waiters,
		 * and (b) doesn't conflict with already-held locks.
M
 
Marc G. Fournier 已提交
765
		 */
766
		if ((lockctl->conflictTab[lockmode] & aheadRequests) == 0 &&
767 768 769 770 771 772
			LockCheckConflicts(lockMethodTable,
							   lockmode,
							   lock,
							   proc->waitHolder,
							   proc,
							   NULL) == STATUS_OK)
M
 
Marc G. Fournier 已提交
773
		{
774 775 776
			/* OK to waken */
			GrantLock(lock, proc->waitHolder, lockmode);
			proc = ProcWakeup(proc, STATUS_OK);
B
Bruce Momjian 已提交
777

778
			/*
B
Bruce Momjian 已提交
779 780 781
			 * 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.
782
			 */
M
 
Marc G. Fournier 已提交
783
		}
784
		else
785
		{
B
Bruce Momjian 已提交
786 787 788 789 790

			/*
			 * Cannot wake this guy. Remember his request for later
			 * checks.
			 */
791
			aheadRequests |= (1 << lockmode);
792
			proc = (PROC *) MAKE_PTR(proc->links.next);
793
		}
M
 
Marc G. Fournier 已提交
794
	}
795 796

	Assert(waitQueue->size >= 0);
797 798 799
}

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

812
	/*
B
Bruce Momjian 已提交
813 814 815 816 817
	 * 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.
818
	 */
819 820
	LockLockTable();

821
	/*
822 823 824
	 * 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.
825 826
	 * Before we are awoken the process releasing the lock grants it to us
	 * so we know that we don't have to wait anymore.
827
	 *
828
	 * We check by looking to see if we've been unlinked from the wait queue.
829 830 831 832
	 * 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.
	 *
833 834 835 836 837
	 */
	if (MyProc->links.prev == INVALID_OFFSET ||
		MyProc->links.next == INVALID_OFFSET)
	{
		UnlockLockTable();
838
		errno = save_errno;
839 840 841
		return;
	}

842
#ifdef LOCK_DEBUG
B
Bruce Momjian 已提交
843 844
	if (Debug_deadlocks)
		DumpAllLocks();
845 846
#endif

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

855
	/*
856 857 858
	 * Oops.  We have a deadlock.
	 *
	 * Get this process out of wait state.
859
	 */
860 861
	RemoveFromWaitQueue(MyProc);

862 863 864
	/*
	 * Set MyProc->errType to STATUS_ERROR so that ProcSleep will report
	 * an error after we return from this signal handler.
865 866
	 */
	MyProc->errType = STATUS_ERROR;
867

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

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

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

	if (!proc)
		proc = MyProc;

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

909 910 911 912 913 914 915 916 917 918 919 920 921 922 923 924 925 926 927 928 929 930 931 932 933 934 935 936 937 938 939 940 941 942 943 944 945 946 947 948 949 950 951
/*
 * ProcWaitForSignal - wait for a signal from another backend.
 *
 * This can share the semaphore normally used for waiting for locks,
 * since a backend could never be waiting for a lock and a signal at
 * the same time.  As with locks, it's OK if the signal arrives just
 * before we actually reach the waiting state.
 */
void
ProcWaitForSignal(void)
{
	waitingForSignal = true;
	IpcSemaphoreLock(MyProc->sem.semId, MyProc->sem.semNum, true);
	waitingForSignal = false;
}

/*
 * ProcCancelWaitForSignal - clean up an aborted wait for signal
 *
 * We need this in case the signal arrived after we aborted waiting,
 * or if it arrived but we never reached ProcWaitForSignal() at all.
 * Caller should call this after resetting the signal request status.
 */
void
ProcCancelWaitForSignal(void)
{
	ZeroProcSemaphore(MyProc);
	waitingForSignal = false;
}

/*
 * ProcSendSignal - send a signal to a backend identified by BackendId
 */
void
ProcSendSignal(BackendId procId)
{
	PROC   *proc = BackendIdGetProc(procId);

	if (proc != NULL)
		IpcSemaphoreUnlock(proc->sem.semId, proc->sem.semNum);
}


952
/*****************************************************************************
953
 *
954 955 956
 *****************************************************************************/

/*
957
 * ProcGetNewSemIdAndNum -
958
 *	  scan the free semaphore bitmap and allocate a single semaphore from
959
 *	  a semaphore set.
960 961
 */
static void
962
ProcGetNewSemIdAndNum(IpcSemaphoreId *semId, int *semNum)
963
{
964
	int			i;
965 966
	int			semMapEntries = ProcGlobal->semMapEntries;
	SEM_MAP_ENTRY  *procSemMap = ProcGlobal->procSemMap;
967
	int32		fullmask = (1 << PROC_NSEMS_PER_SET) - 1;
968

969 970 971 972
	/*
	 * we hold ProcStructLock when entering this routine. We scan through
	 * the bitmap to look for a free semaphore.
	 */
973

974
	for (i = 0; i < semMapEntries; i++)
975
	{
976 977
		int			mask = 1;
		int			j;
978

979
		if (procSemMap[i].freeSemMap == fullmask)
980
			continue;			/* this set is fully allocated */
981
		if (procSemMap[i].procSemId < 0)
982
			continue;			/* this set hasn't been initialized */
983 984 985

		for (j = 0; j < PROC_NSEMS_PER_SET; j++)
		{
986
			if ((procSemMap[i].freeSemMap & mask) == 0)
987
			{
988
				/* A free semaphore found. Mark it as allocated. */
989
				procSemMap[i].freeSemMap |= mask;
990

991
				*semId = procSemMap[i].procSemId;
992 993 994 995 996
				*semNum = j;
				return;
			}
			mask <<= 1;
		}
997 998
	}

999 1000 1001 1002 1003 1004 1005
	/*
	 * If we reach here, all the semaphores are in use.  This is one of the
	 * possible places to detect "too many backends", so give the standard
	 * error message.  (Whether we detect it here or in sinval.c depends on
	 * whether MaxBackends is a multiple of PROC_NSEMS_PER_SET.)
	 */
	elog(FATAL, "Sorry, too many clients already");
1006 1007 1008 1009
}

/*
 * ProcFreeSem -
1010
 *	  free up our semaphore in the semaphore set.
1011 1012
 */
static void
1013
ProcFreeSem(IpcSemaphoreId semId, int semNum)
1014
{
1015
	int32		mask;
1016
	int			i;
1017
	int			semMapEntries = ProcGlobal->semMapEntries;
1018

1019
	mask = ~(1 << semNum);
1020

1021
	for (i = 0; i < semMapEntries; i++)
1022
	{
1023
		if (ProcGlobal->procSemMap[i].procSemId == semId)
1024
		{
1025
			ProcGlobal->procSemMap[i].freeSemMap &= mask;
1026 1027 1028 1029
			return;
		}
	}
	fprintf(stderr, "ProcFreeSem: no ProcGlobal entry for semId %d\n", semId);
1030 1031 1032 1033
}

/*
 * ProcFreeAllSemaphores -
1034 1035 1036
 *	  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.
1037
 */
1038
static void
1039
ProcFreeAllSemaphores(void)
1040
{
1041
	int			i;
1042

1043
	for (i = 0; i < ProcGlobal->semMapEntries; i++)
1044
	{
1045 1046
		if (ProcGlobal->procSemMap[i].procSemId >= 0)
			IpcSemaphoreKill(ProcGlobal->procSemMap[i].procSemId);
1047
	}
1048
}