proc.c 25.7 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
B
Bruce Momjian 已提交
11
 *	  $Header: /cvsroot/pgsql/src/backend/storage/lmgr/proc.c,v 1.75 2000/06/15 04:10:07 momjian 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 32 33
 *
 * 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):
 *
 * ProcReleaseLocks -- frees the locks associated with this process,
 * ProcKill -- destroys the shared memory state (and locks)
34
 *		associated with the process.
35 36
 *
 * 5/15/91 -- removed the buffer pool based lock chain in favor
37 38 39 40 41 42
 *		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.
43 44
 *
 * 4/7/95 -- instead of allocating a set of 1 semaphore per process, we
45 46 47 48
 *		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
49
 *
B
Bruce Momjian 已提交
50
 * $Header: /cvsroot/pgsql/src/backend/storage/lmgr/proc.c,v 1.75 2000/06/15 04:10:07 momjian Exp $
51 52 53
 */
#include <sys/time.h>
#include <unistd.h>
54
#include <signal.h>
55
#include <sys/types.h>
M
Marc G. Fournier 已提交
56

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

M
Marc G. Fournier 已提交
62
#include "postgres.h"
63
#include "miscadmin.h"
64 65


66
/* In Ultrix and QNX, sem.h must be included after ipc.h */
67
#include <sys/sem.h>
B
Bruce Momjian 已提交
68

69 70
#include "storage/proc.h"

71
void		HandleDeadLock(SIGNAL_ARGS);
72
static void ProcFreeAllSemaphores(void);
73
static bool GetOffWaitqueue(PROC *);
74

75
int DeadlockTimeout = 1000;
M
 
Marc G. Fournier 已提交
76

77 78 79 80 81 82 83
/* --------------------
 * 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
 * --------------------
 */
84
SPINLOCK	ProcStructLock;
85 86 87

static PROC_HDR *ProcGlobal = NULL;

88
PROC	   *MyProc = NULL;
89

90
static void ProcKill(int exitStatus, int pid);
91
static void ProcGetNewSemKeyAndNum(IPCKey *key, int *semNum);
92
static void ProcFreeSem(IpcSemaphoreKey semKey, int semNum);
93

V
Vadim B. Mikheev 已提交
94 95
static char *DeadLockMessage = "Deadlock detected -- See the lock(l) manual page for a possible cause.";

96 97
/*
 * InitProcGlobal -
98
 *	  initializes the global process table. We put it here so that
99
 *	  the postmaster can do this initialization. (ProcFreeAllSemaphores needs
100 101 102
 *	  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.)
103 104 105 106 107 108 109 110 111 112 113
 *
 *	  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.
114 115
 */
void
116
InitProcGlobal(IPCKey key, int maxBackends)
117
{
118
	bool		found = false;
119

120 121 122
	/* attach to the free list */
	ProcGlobal = (PROC_HDR *)
		ShmemInitStruct("Proc Header", (unsigned) sizeof(PROC_HDR), &found);
123

124 125
	/* --------------------
	 * We're the first - initialize.
126 127
	 * XXX if found should ever be true, it is a sign of impending doom ...
	 * ought to complain if so?
128 129 130
	 * --------------------
	 */
	if (!found)
131
	{
132
		int			i;
133

134 135 136 137
		ProcGlobal->freeProcs = INVALID_OFFSET;
		ProcGlobal->currKey = IPCGetProcessSemaphoreInitKey(key);
		for (i = 0; i < MAX_PROC_SEMS / PROC_NSEMS_PER_SET; i++)
			ProcGlobal->freeSemMap[i] = 0;
138

B
Bruce Momjian 已提交
139 140 141
		/*
		 * Arrange to delete semas on exit --- set this up now so that we
		 * will clean up if pre-allocation fails...
142 143 144
		 */
		on_shmem_exit(ProcFreeAllSemaphores, NULL);

B
Bruce Momjian 已提交
145 146
		/*
		 * Pre-create the semaphores for the first maxBackends processes,
147 148 149
		 * unless we are running as a standalone backend.
		 */
		if (key != PrivateIPCKey)
150
		{
151
			for (i = 0;
B
Bruce Momjian 已提交
152
				 i < (maxBackends + PROC_NSEMS_PER_SET - 1) / PROC_NSEMS_PER_SET;
153 154 155 156 157 158 159 160 161
				 i++)
			{
				IPCKey		semKey = ProcGlobal->currKey + i;
				int			semId;

				semId = IpcSemaphoreCreate(semKey,
										   PROC_NSEMS_PER_SET,
										   IPCProtection,
										   IpcSemaphoreDefaultStartValue,
162 163 164
										   0);
				if (semId < 0)
					elog(FATAL, "InitProcGlobal: IpcSemaphoreCreate failed");
165 166 167
				/* mark this sema set allocated */
				ProcGlobal->freeSemMap[i] = (1 << PROC_NSEMS_PER_SET);
			}
168
		}
169 170 171 172 173 174 175 176 177 178 179
	}
}

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

	SpinAcquire(ProcStructLock);

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

	if (MyProc != NULL)
196
	{
197
		SpinRelease(ProcStructLock);
198
		elog(ERROR, "ProcInit: you already exist");
199
		return;
200
	}
201 202 203 204 205 206

	/* try to get a proc from the free list first */

	myOffset = ProcGlobal->freeProcs;

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

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

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

		/* this cannot be initialized until after the buffer pool */
		SHMQueueInit(&(MyProc->lockQueue));
230
	}
231

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


	if (IsUnderPostmaster)
	{
243 244 245 246
		IPCKey		semKey;
		int			semNum;
		int			semId;
		union semun semun;
247 248 249

		ProcGetNewSemKeyAndNum(&semKey, &semNum);

B
Bruce Momjian 已提交
250 251 252 253 254
		/*
		 * Note: because of the pre-allocation done in InitProcGlobal,
		 * this call should always attach to an existing semaphore. It
		 * will (try to) create a new group of semaphores only if the
		 * postmaster tries to start more backends than it said it would.
255
		 */
256 257 258 259
		semId = IpcSemaphoreCreate(semKey,
								   PROC_NSEMS_PER_SET,
								   IPCProtection,
								   IpcSemaphoreDefaultStartValue,
260
								   0);
261 262 263 264 265 266 267 268 269 270 271 272 273 274 275 276 277 278 279 280 281 282

		/*
		 * we might be reusing a semaphore that belongs to a dead backend.
		 * So be careful and reinitialize its value here.
		 */
		semun.val = IpcSemaphoreDefaultStartValue;
		semctl(semId, semNum, SETVAL, semun);

		IpcSemaphoreLock(semId, semNum, IpcExclusiveLock);
		MyProc->sem.semId = semId;
		MyProc->sem.semNum = semNum;
		MyProc->sem.semKey = semKey;
	}
	else
		MyProc->sem.semId = -1;

	/* ----------------------
	 * Release the lock.
	 * ----------------------
	 */
	SpinRelease(ProcStructLock);

B
Bruce Momjian 已提交
283
	MyProc->pid = MyProcPid;
284
	MyProc->databaseId = MyDatabaseId;
285
	MyProc->xid = InvalidTransactionId;
286
	MyProc->xmin = InvalidTransactionId;
287 288 289 290 291 292

	/* ----------------
	 * Start keeping spin lock stats from here on.	Any botch before
	 * this initialization is forever botched
	 * ----------------
	 */
B
Bruce Momjian 已提交
293
	MemSet(MyProc->sLocks, 0, MAX_SPINS * sizeof(*MyProc->sLocks));
294 295

	/* -------------------------
296
	 * Install ourselves in the shmem index table.	The name to
297 298 299 300 301 302
	 * 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);
B
Bruce Momjian 已提交
303
	if ((!ShmemPIDLookup(MyProcPid, &location)) || (location != MAKE_OFFSET(MyProc)))
304
		elog(STOP, "InitProc: ShmemPID table broken");
305 306 307 308

	MyProc->errType = NO_ERROR;
	SHMQueueElemInit(&(MyProc->links));

309
	on_shmem_exit(ProcKill, (caddr_t) MyProcPid);
310 311
}

H
Hiroshi Inoue 已提交
312 313 314 315
/* -----------------------
 * get off the wait queue
 * -----------------------
 */
316
static bool
H
Hiroshi Inoue 已提交
317 318
GetOffWaitqueue(PROC *proc)
{
319 320
	bool		getoffed = false;

H
Hiroshi Inoue 已提交
321 322 323
	LockLockTable();
	if (proc->links.next != INVALID_OFFSET)
	{
324 325
		int			lockmode = proc->token;

H
Hiroshi Inoue 已提交
326 327 328 329 330 331 332 333 334 335 336
		Assert(proc->waitLock->waitProcs.size > 0);
		SHMQueueDelete(&(proc->links));
		--proc->waitLock->waitProcs.size;
		Assert(proc->waitLock->nHolding > 0);
		Assert(proc->waitLock->nHolding > proc->waitLock->nActive);
		--proc->waitLock->nHolding;
		Assert(proc->waitLock->holders[lockmode] > 0);
		--proc->waitLock->holders[lockmode];
		if (proc->waitLock->activeHolders[lockmode] ==
			proc->waitLock->holders[lockmode])
			proc->waitLock->waitMask &= ~(1 << lockmode);
337
		getoffed = true;
H
Hiroshi Inoue 已提交
338 339 340 341
	}
	SHMQueueElemInit(&(proc->links));
	UnlockLockTable();

342
	return getoffed;
H
Hiroshi Inoue 已提交
343
}
344

345 346 347 348 349 350 351
/*
 * ProcReleaseLocks() -- release all locks associated with this process
 *
 */
void
ProcReleaseLocks()
{
352 353 354
	if (!MyProc)
		return;
	LockReleaseAll(1, &MyProc->lockQueue);
H
Hiroshi Inoue 已提交
355
	GetOffWaitqueue(MyProc);
356 357 358 359
}

/*
 * ProcRemove -
360 361 362 363 364
 *	  used by the postmaster to clean up the global tables. This also frees
 *	  up the semaphore used for the lmgr of the process. (We have to do
 *	  this is the postmaster instead of doing a IpcSemaphoreKill on exiting
 *	  the process because the semaphore set is shared among backends and
 *	  we don't want to remove other's semaphores on exit.)
365 366 367 368
 */
bool
ProcRemove(int pid)
{
369 370
	SHMEM_OFFSET location;
	PROC	   *proc;
371 372 373 374 375

	location = INVALID_OFFSET;

	location = ShmemPIDDestroy(pid);
	if (location == INVALID_OFFSET)
376
		return FALSE;
377 378 379 380 381 382 383 384 385 386 387
	proc = (PROC *) MAKE_PTR(location);

	SpinAcquire(ProcStructLock);

	ProcFreeSem(proc->sem.semKey, proc->sem.semNum);

	proc->links.next = ProcGlobal->freeProcs;
	ProcGlobal->freeProcs = MAKE_OFFSET(proc);

	SpinRelease(ProcStructLock);

388
	return TRUE;
389 390 391 392
}

/*
 * ProcKill() -- Destroy the per-proc data structure for
393
 *		this process. Release any of its held spin locks.
394 395 396 397
 */
static void
ProcKill(int exitStatus, int pid)
{
398 399
	PROC	   *proc;
	SHMEM_OFFSET location;
400 401 402 403 404 405 406 407 408 409

	/* --------------------
	 * If this is a FATAL exit the postmaster will have to kill all the
	 * existing backends and reinitialize shared memory.  So all we don't
	 * need to do anything here.
	 * --------------------
	 */
	if (exitStatus != 0)
		return;

B
Bruce Momjian 已提交
410
	ShmemPIDLookup(MyProcPid, &location);
411 412 413 414 415
	if (location == INVALID_OFFSET)
		return;

	proc = (PROC *) MAKE_PTR(location);

416 417 418
	Assert(proc == MyProc || pid != MyProcPid);

	MyProc = NULL;
419 420 421 422 423 424

	/* ---------------
	 * Assume one lock table.
	 * ---------------
	 */
	ProcReleaseSpins(proc);
M
 
Marc G. Fournier 已提交
425
	LockReleaseAll(DEFAULT_LOCKMETHOD, &proc->lockQueue);
426

427
#ifdef USER_LOCKS
428

M
 
Marc G. Fournier 已提交
429 430 431 432
	/*
	 * Assume we have a second lock table.
	 */
	LockReleaseAll(USER_LOCKMETHOD, &proc->lockQueue);
433 434
#endif

435 436 437 438
	/* ----------------
	 * get off the wait queue
	 * ----------------
	 */
H
Hiroshi Inoue 已提交
439
	GetOffWaitqueue(proc);
440 441

	return;
442 443 444 445
}

/*
 * ProcQueue package: routines for putting processes to sleep
446
 *		and  waking them up
447 448 449 450 451 452 453 454
 */

/*
 * 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
 */
455
#ifdef NOT_USED
456
PROC_QUEUE *
457 458
ProcQueueAlloc(char *name)
{
459 460
	bool		found;
	PROC_QUEUE *queue = (PROC_QUEUE *)
461 462 463
	ShmemInitStruct(name, (unsigned) sizeof(PROC_QUEUE), &found);

	if (!queue)
464
		return NULL;
465 466
	if (!found)
		ProcQueueInit(queue);
467
	return queue;
468
}
469

470
#endif
471 472 473 474 475

/*
 * ProcQueueInit -- initialize a shared memory process queue
 */
void
476
ProcQueueInit(PROC_QUEUE *queue)
477
{
478 479
	SHMQueueInit(&(queue->links));
	queue->size = 0;
480 481 482
}


483 484 485 486
/*
 *	Handling cancel request while waiting for lock
 *
 */
487 488 489
static bool lockWaiting = false;
void
SetWaitingForLock(bool waiting)
490
{
491 492
	if (waiting == lockWaiting)
		return;
493
	lockWaiting = waiting;
494 495
	if (lockWaiting)
	{
496 497 498 499 500 501
		/* The lock was already released ? */
		if (MyProc->links.next == INVALID_OFFSET)
		{
			lockWaiting = false;
			return;
		}
502
		if (QueryCancel)		/* cancel request pending */
503 504 505 506 507 508 509 510
		{
			if (GetOffWaitqueue(MyProc))
			{
				lockWaiting = false;
				elog(ERROR, "Query cancel requested while waiting lock");
			}
		}
	}
511
}
512 513
void
LockWaitCancel(void)
514
{
515 516
	struct itimerval timeval,
				dummy;
517

518 519
	if (!lockWaiting)
		return;
520 521 522 523 524 525 526
	lockWaiting = false;
	/* Deadlock timer off */
	MemSet(&timeval, 0, sizeof(struct itimerval));
	setitimer(ITIMER_REAL, &timeval, &dummy);
	if (GetOffWaitqueue(MyProc))
		elog(ERROR, "Query cancel requested while waiting lock");
}
527 528 529 530 531 532 533 534 535

/*
 * ProcSleep -- put a process to sleep
 *
 * P() on the semaphore should put us to sleep.  The process
 * semaphore is cleared by default, so the first time we try
 * to acquire it, we sleep.
 *
 * ASSUME: that no one will fiddle with the queue until after
536
 *		we release the spin lock.
537 538 539 540
 *
 * NOTES: The process queue is now a priority queue for locking.
 */
int
541
ProcSleep(PROC_QUEUE *waitQueue,/* lock->waitProcs */
542
		  LOCKMETHODCTL *lockctl,
543
		  int token,			/* lockmode */
V
Vadim B. Mikheev 已提交
544
		  LOCK *lock)
545
{
546
	int			i;
V
Vadim B. Mikheev 已提交
547
	SPINLOCK	spinlock = lockctl->masterLock;
548
	PROC	   *proc;
V
Vadim B. Mikheev 已提交
549 550 551 552 553
	int			myMask = (1 << token);
	int			waitMask = lock->waitMask;
	int			aheadHolders[MAX_LOCKMODES];
	bool		selfConflict = (lockctl->conflictTab[token] & myMask),
				prevSame = false;
B
Bruce Momjian 已提交
554 555 556
	bool		deadlock_checked = false;
	struct itimerval timeval,
				dummy;
557

V
Vadim B. Mikheev 已提交
558 559 560
	MyProc->token = token;
	MyProc->waitLock = lock;

B
Bruce Momjian 已提交
561
	proc = (PROC *) MAKE_PTR(waitQueue->links.prev);
562

V
Vadim B. Mikheev 已提交
563 564 565
	/* if we don't conflict with any waiter - be first in queue */
	if (!(lockctl->conflictTab[token] & waitMask))
		goto ins;
566

V
Vadim B. Mikheev 已提交
567 568 569
	for (i = 1; i < MAX_LOCKMODES; i++)
		aheadHolders[i] = lock->activeHolders[i];
	(aheadHolders[token])++;
570

V
Vadim B. Mikheev 已提交
571 572 573 574 575 576 577 578 579 580 581 582 583 584 585 586 587 588 589 590
	for (i = 0; i < waitQueue->size; i++)
	{
		/* am I waiting for him ? */
		if (lockctl->conflictTab[token] & proc->holdLock)
		{
			/* is he waiting for me ? */
			if (lockctl->conflictTab[proc->token] & MyProc->holdLock)
			{
				MyProc->errType = STATUS_ERROR;
				elog(NOTICE, DeadLockMessage);
				goto rt;
			}
			/* being waiting for him - go past */
		}
		/* if he waits for me */
		else if (lockctl->conflictTab[proc->token] & MyProc->holdLock)
			break;
		/* if conflicting locks requested */
		else if (lockctl->conflictTab[proc->token] & myMask)
		{
B
Bruce Momjian 已提交
591

V
Vadim B. Mikheev 已提交
592
			/*
B
Bruce Momjian 已提交
593 594
			 * If I request non self-conflicting lock and there are others
			 * requesting the same lock just before me - stay here.
V
Vadim B. Mikheev 已提交
595 596 597 598
			 */
			if (!selfConflict && prevSame)
				break;
		}
B
Bruce Momjian 已提交
599

V
Vadim B. Mikheev 已提交
600
		/*
B
Bruce Momjian 已提交
601 602
		 * Last attempt to don't move any more: if we don't conflict with
		 * rest waiters in queue.
V
Vadim B. Mikheev 已提交
603 604 605
		 */
		else if (!(lockctl->conflictTab[token] & waitMask))
			break;
606

V
Vadim B. Mikheev 已提交
607 608 609
		prevSame = (proc->token == token);
		(aheadHolders[proc->token])++;
		if (aheadHolders[proc->token] == lock->holders[proc->token])
B
Bruce Momjian 已提交
610
			waitMask &= ~(1 << proc->token);
V
Vadim B. Mikheev 已提交
611 612
		proc = (PROC *) MAKE_PTR(proc->links.prev);
	}
613

V
Vadim B. Mikheev 已提交
614
ins:;
615 616 617 618 619 620
	/* -------------------
	 * assume that these two operations are atomic (because
	 * of the spinlock).
	 * -------------------
	 */
	SHMQueueInsertTL(&(proc->links), &(MyProc->links));
B
Bruce Momjian 已提交
621
	waitQueue->size++;
622

V
Vadim B. Mikheev 已提交
623
	lock->waitMask |= myMask;
624 625 626
	SpinRelease(spinlock);

	/* --------------
B
Bruce Momjian 已提交
627
	 * We set this so we can wake up periodically and check for a deadlock.
B
Bruce Momjian 已提交
628 629
	 * If a deadlock is detected, the handler releases the processes
	 * semaphore and aborts the current transaction.
B
Bruce Momjian 已提交
630 631 632
	 *
	 * Need to zero out struct to set the interval and the micro seconds fields
	 * to 0.
633 634
	 * --------------
	 */
B
Bruce Momjian 已提交
635
	MemSet(&timeval, 0, sizeof(struct itimerval));
636 637
	timeval.it_value.tv_sec = DeadlockTimeout / 1000;
	timeval.it_value.tv_usec = (DeadlockTimeout % 1000) * 1000;
638

639
	SetWaitingForLock(true);
B
Bruce Momjian 已提交
640 641
	do
	{
642
		MyProc->errType = NO_ERROR;		/* reset flag after deadlock check */
643

B
Bruce Momjian 已提交
644 645 646 647 648
		if (!deadlock_checked)
			if (setitimer(ITIMER_REAL, &timeval, &dummy))
				elog(FATAL, "ProcSleep: Unable to set timer for process wakeup");
		deadlock_checked = true;

B
Bruce Momjian 已提交
649 650 651 652 653 654
		/* --------------
		 * if someone wakes us between SpinRelease and IpcSemaphoreLock,
		 * IpcSemaphoreLock will not block.  The wakeup is "saved" by
		 * the semaphore implementation.
		 * --------------
		 */
M
 
Marc G. Fournier 已提交
655 656
		IpcSemaphoreLock(MyProc->sem.semId, MyProc->sem.semNum,
						 IpcExclusiveLock);
657 658
	} while (MyProc->errType == STATUS_NOT_FOUND);		/* sleep after deadlock
														 * check */
659
	lockWaiting = false;
660

B
Bruce Momjian 已提交
661 662 663 664 665
	/* ---------------
	 * We were awoken before a timeout - now disable the timer
	 * ---------------
	 */
	timeval.it_value.tv_sec = 0;
666
	timeval.it_value.tv_usec = 0;
B
Bruce Momjian 已提交
667 668 669
	if (setitimer(ITIMER_REAL, &timeval, &dummy))
		elog(FATAL, "ProcSleep: Unable to diable timer for process wakeup");

670 671 672 673 674 675 676
	/* ----------------
	 * We were assumed to be in a critical section when we went
	 * to sleep.
	 * ----------------
	 */
	SpinAcquire(spinlock);

V
Vadim B. Mikheev 已提交
677 678
rt:;

679
#ifdef LOCK_DEBUG
M
 
Marc G. Fournier 已提交
680
	/* Just to get meaningful debug messages from DumpLocks() */
681
	MyProc->waitLock = (LOCK *) NULL;
M
 
Marc G. Fournier 已提交
682 683
#endif

684
	return MyProc->errType;
685 686 687 688 689 690
}


/*
 * ProcWakeup -- wake up a process by releasing its private semaphore.
 *
691 692
 *	 remove the process from the wait queue and set its links invalid.
 *	 RETURN: the next process in the wait queue.
693
 */
B
Bruce Momjian 已提交
694
PROC *
695
ProcWakeup(PROC *proc, int errType)
696
{
697
	PROC	   *retProc;
698 699 700 701 702

	/* assume that spinlock has been acquired */

	if (proc->links.prev == INVALID_OFFSET ||
		proc->links.next == INVALID_OFFSET)
703
		return (PROC *) NULL;
704 705 706 707 708 709 710 711 712 713 714 715

	retProc = (PROC *) MAKE_PTR(proc->links.prev);

	/* you have to update waitLock->waitProcs.size yourself */
	SHMQueueDelete(&(proc->links));
	SHMQueueElemInit(&(proc->links));

	proc->errType = errType;

	IpcSemaphoreUnlock(proc->sem.semId, proc->sem.semNum, IpcExclusiveLock);

	return retProc;
716 717 718 719
}

/*
 * ProcLockWakeup -- routine for waking up processes when a lock is
720
 *		released.
721 722
 */
int
723
ProcLockWakeup(PROC_QUEUE *queue, LOCKMETHOD lockmethod, LOCK *lock)
724
{
725
	PROC	   *proc;
V
Vadim B. Mikheev 已提交
726 727
	int			count = 0;
	int			last_locktype = 0;
M
 
Marc G. Fournier 已提交
728 729 730
	int			queue_size = queue->size;

	Assert(queue->size >= 0);
731 732

	if (!queue->size)
733
		return STATUS_NOT_FOUND;
734 735

	proc = (PROC *) MAKE_PTR(queue->links.prev);
M
 
Marc G. Fournier 已提交
736 737
	while ((queue_size--) && (proc))
	{
738

M
 
Marc G. Fournier 已提交
739
		/*
740 741
		 * This proc will conflict as the previous one did, don't even
		 * try.
M
 
Marc G. Fournier 已提交
742 743 744 745 746
		 */
		if (proc->token == last_locktype)
			continue;

		/*
V
Vadim B. Mikheev 已提交
747
		 * Does this proc conflict with locks held by others ?
M
 
Marc G. Fournier 已提交
748 749
		 */
		if (LockResolveConflicts(lockmethod,
750
								 lock,
751
								 proc->token,
M
 
Marc G. Fournier 已提交
752 753 754
								 proc->xid,
								 (XIDLookupEnt *) NULL) != STATUS_OK)
		{
V
Vadim B. Mikheev 已提交
755 756
			if (count != 0)
				break;
M
 
Marc G. Fournier 已提交
757 758 759
			last_locktype = proc->token;
			continue;
		}
760 761 762 763 764 765 766

		/*
		 * there was a waiting process, grant it the lock before waking it
		 * up.	This will prevent another process from seizing the lock
		 * between the time we release the lock master (spinlock) and the
		 * time that the awoken process begins executing again.
		 */
767
		GrantLock(lock, proc->token);
768 769 770

		/*
		 * ProcWakeup removes proc from the lock waiting process queue and
771
		 * returns the next proc in chain.
772 773 774
		 */

		count++;
M
 
Marc G. Fournier 已提交
775 776
		queue->size--;
		proc = ProcWakeup(proc, NO_ERROR);
777
	}
778

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

781
	if (count)
782
		return STATUS_OK;
783 784
	else
	{
785
		/* Something is still blocking us.	May have deadlocked. */
786 787 788 789 790
#ifdef LOCK_DEBUG
		if (lock->tag.lockmethod == USER_LOCKMETHOD ? Trace_userlocks : Trace_locks)
		{
			elog(DEBUG, "ProcLockWakeup: lock(%lx) can't wake up any process", MAKE_OFFSET(lock));
			if (Debug_deadlocks)
M
 
Marc G. Fournier 已提交
791
			DumpAllLocks();
792
		}
M
 
Marc G. Fournier 已提交
793
#endif
794
		return STATUS_NOT_FOUND;
M
 
Marc G. Fournier 已提交
795
	}
796 797 798
}

void
799
ProcAddLock(SHM_QUEUE *elem)
800
{
801
	SHMQueueInsertTL(&MyProc->lockQueue, elem);
802 803 804
}

/* --------------------
805
 * We only get to this routine if we got SIGALRM after DeadlockTimeout
B
Bruce Momjian 已提交
806 807
 * while waiting for a lock to be released by some other process.  If we have
 * a real deadlock, we must also indicate that I'm no longer waiting
808
 * on a lock so that other processes don't try to wake me up and screw
809 810 811
 * up my semaphore.
 * --------------------
 */
812 813
void
HandleDeadLock(SIGNAL_ARGS)
814
{
B
Bruce Momjian 已提交
815
	LOCK	   *mywaitlock;
816 817 818 819 820 821 822 823 824 825 826 827 828 829 830 831 832 833 834 835 836 837 838 839 840 841 842 843 844 845 846 847 848 849 850 851 852 853

	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.
	 *
	 * Damn these names are LONG! -mer
	 * ---------------------
	 */
	if (IpcSemaphoreGetCount(MyProc->sem.semId, MyProc->sem.semNum) ==
		IpcSemaphoreDefaultStartValue)
	{
		UnlockLockTable();
		return;
	}

	/*
	 * you would think this would be unnecessary, but...
	 *
	 * this also means we've been removed already.  in some ports (e.g.,
	 * sparc and aix) the semop(2) implementation is such that we can
	 * actually end up in this handler after someone has removed us from
	 * the queue and bopped the semaphore *but the test above fails to
	 * detect the semaphore update* (presumably something weird having to
	 * do with the order in which the semaphore wakeup signal and SIGALRM
	 * get handled).
	 */
	if (MyProc->links.prev == INVALID_OFFSET ||
		MyProc->links.next == INVALID_OFFSET)
	{
		UnlockLockTable();
		return;
	}

854 855 856
#ifdef LOCK_DEBUG
    if (Debug_deadlocks)
        DumpAllLocks();
857 858
#endif

B
Bruce Momjian 已提交
859 860
	MyProc->errType = STATUS_NOT_FOUND;
	if (!DeadLockCheck(MyProc, MyProc->waitLock))
B
Bruce Momjian 已提交
861 862 863 864 865 866 867
	{
		UnlockLockTable();
		return;
	}

	mywaitlock = MyProc->waitLock;

868 869 870 871
	/* ------------------------
	 * Get this process off the lock's wait queue
	 * ------------------------
	 */
B
Bruce Momjian 已提交
872
	Assert(mywaitlock->waitProcs.size > 0);
873
	lockWaiting = false;
B
Bruce Momjian 已提交
874
	--mywaitlock->waitProcs.size;
875 876 877 878 879 880 881 882
	SHMQueueDelete(&(MyProc->links));
	SHMQueueElemInit(&(MyProc->links));

	/* ------------------
	 * Unlock my semaphore so that the count is right for next time.
	 * I was awoken by a signal, not by someone unlocking my semaphore.
	 * ------------------
	 */
M
 
Marc G. Fournier 已提交
883 884
	IpcSemaphoreUnlock(MyProc->sem.semId, MyProc->sem.semNum,
					   IpcExclusiveLock);
885 886 887 888 889 890 891 892 893 894 895 896 897 898 899

	/* -------------
	 * Set MyProc->errType to STATUS_ERROR so that we abort after
	 * returning from this handler.
	 * -------------
	 */
	MyProc->errType = STATUS_ERROR;

	/*
	 * if this doesn't follow the IpcSemaphoreUnlock then we get lock
	 * table corruption ("LockReplace: xid table corrupted") due to race
	 * conditions.	i don't claim to understand this...
	 */
	UnlockLockTable();

V
Vadim B. Mikheev 已提交
900
	elog(NOTICE, DeadLockMessage);
901
	return;
902 903 904
}

void
905
ProcReleaseSpins(PROC *proc)
906
{
907
	int			i;
908 909 910 911 912 913 914

	if (!proc)
		proc = MyProc;

	if (!proc)
		return;
	for (i = 0; i < (int) MAX_SPINS; i++)
915
	{
916
		if (proc->sLocks[i])
917
		{
918 919
			Assert(proc->sLocks[i] == 1);
			SpinRelease(i);
920 921
		}
	}
H
 
Hiroshi Inoue 已提交
922
	AbortBufferIO();
923 924 925
}

/*****************************************************************************
926
 *
927 928 929 930
 *****************************************************************************/

/*
 * ProcGetNewSemKeyAndNum -
931 932 933 934
 *	  scan the free semaphore bitmap and allocate a single semaphore from
 *	  a semaphore set. (If the semaphore set doesn't exist yet,
 *	  IpcSemaphoreCreate will create it. Otherwise, we use the existing
 *	  semaphore set.)
935 936
 */
static void
937
ProcGetNewSemKeyAndNum(IPCKey *key, int *semNum)
938
{
939 940
	int			i;
	int32	   *freeSemMap = ProcGlobal->freeSemMap;
B
Bruce Momjian 已提交
941
	int32		fullmask = (1 << (PROC_NSEMS_PER_SET + 1)) - 1;
942

943 944 945 946
	/*
	 * we hold ProcStructLock when entering this routine. We scan through
	 * the bitmap to look for a free semaphore.
	 */
947

948 949
	for (i = 0; i < MAX_PROC_SEMS / PROC_NSEMS_PER_SET; i++)
	{
950 951
		int			mask = 1;
		int			j;
952 953

		if (freeSemMap[i] == fullmask)
954
			continue;			/* this set is fully allocated */
955 956 957 958 959 960 961

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

				/*
B
Bruce Momjian 已提交
962 963
				 * a free semaphore found. Mark it as allocated. Also set
				 * the bit indicating whole set is allocated.
964
				 */
965
				freeSemMap[i] |= mask + (1 << PROC_NSEMS_PER_SET);
966 967 968 969 970 971 972

				*key = ProcGlobal->currKey + i;
				*semNum = j;
				return;
			}
			mask <<= 1;
		}
973 974
	}

975
	/* if we reach here, all the semaphores are in use. */
976
	elog(ERROR, "InitProc: cannot allocate a free semaphore");
977 978 979 980
}

/*
 * ProcFreeSem -
981
 *	  free up our semaphore in the semaphore set.
982 983 984 985
 */
static void
ProcFreeSem(IpcSemaphoreKey semKey, int semNum)
{
986 987 988
	int			mask;
	int			i;
	int32	   *freeSemMap = ProcGlobal->freeSemMap;
989

990 991 992
	i = semKey - ProcGlobal->currKey;
	mask = ~(1 << semNum);
	freeSemMap[i] &= mask;
993

B
Bruce Momjian 已提交
994 995 996 997
	/*
	 * Formerly we'd release a semaphore set if it was now completely
	 * unused, but now we keep the semaphores to ensure we won't run out
	 * when starting new backends --- cf. InitProcGlobal.  Note that the
998 999 1000
	 * PROC_NSEMS_PER_SET+1'st bit of the freeSemMap entry remains set to
	 * indicate it is still allocated; ProcFreeAllSemaphores() needs that.
	 */
1001 1002 1003 1004
}

/*
 * ProcFreeAllSemaphores -
1005 1006 1007
 *	  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.
1008
 */
1009
static void
1010 1011
ProcFreeAllSemaphores()
{
1012 1013
	int			i;
	int32	   *freeSemMap = ProcGlobal->freeSemMap;
1014

1015 1016 1017 1018 1019
	for (i = 0; i < MAX_PROC_SEMS / PROC_NSEMS_PER_SET; i++)
	{
		if (freeSemMap[i] != 0)
			IpcSemaphoreKill(ProcGlobal->currKey + i);
	}
1020
}