Statistics
| Branch: | Tag: | Revision:

root / lib / jqueue.py @ 4008c8ed

History | View | Annotate | Download (49 kB)

# Date Author Comment
4008c8ed 09/10/2010 02:23 pm Michael Hanselmann

jqueue: Resume jobs from “waitlock” status

After an unclean restart of ganeti-masterd, jobs in the “waitlock” status can
be safely restarted. They hadn't modified the cluster yet.

Signed-off-by: Michael Hanselmann <>
Reviewed-by: René Nussbaumer <>

de9d02c7 09/10/2010 02:23 pm Michael Hanselmann

jqueue: Move queue inspection into separate function

This makes the init function a lot smaller while not changing
functionality.

Signed-off-by: Michael Hanselmann <>
Reviewed-by: René Nussbaumer <>

747f6113 09/10/2010 02:23 pm Michael Hanselmann

jqueue: Don't update file in MarkUnfinishedOps

This reduced the number of updates to the job files. It's used in two places
while processing a job and the file is updated just afterwards.

Signed-off-by: Michael Hanselmann <>
Reviewed-by: Iustin Pop <>

6760e4ed 09/07/2010 12:44 pm Michael Hanselmann

jqueue: Use separate function for encoding errors

Comes with unittest.

Signed-off-by: Michael Hanselmann <>
Reviewed-by: Iustin Pop <>

daba67c7 08/24/2010 05:27 pm Michael Hanselmann

workerpool: Allow setting task name

With this patch, the task name is added to the thread name and will show up in
logs. Log messages from jobs will look like “pid=578/JobQueue14/Job13 mcpu:289
DEBUG LU locks acquired/cluster/BGL/shared”.

Signed-off-by: Michael Hanselmann <>...

9bdab621 08/19/2010 06:30 pm Michael Hanselmann

jqueue: Remove lock status field

With the job queue changes for Ganeti 2.2, watched and queried jobs are
loaded directly from disk, rendering the in-memory “lock_status” field
useless. Writing it to disk would be possible, but has a huge cost at
runtime (when tested, processing 1'000 opcodes involved 4'000 additional...

963a068b 08/18/2010 02:21 pm Michael Hanselmann

jqueue: Mark opcodes following failed ones as failed, too

When an opcode fails, the job queue would leave following opcodes as “queued”,
which can be quite confusing. With this patch, they're all marked as failed and
assertions are added to check this.

Signed-off-by: Michael Hanselmann <>...

6ea72e43 08/18/2010 02:21 pm Michael Hanselmann

jqueue: Work around race condition between job processing and archival

This is a simplified version of a patch I sent earlier to make sure the job
file is only written once with a finalized status.

Signed-off-by: Michael Hanselmann <>
Reviewed-by: Iustin Pop <>

dc1e2262 08/17/2010 04:53 pm Michael Hanselmann

jqueue: More checks for cancelling queued job

We can also check when the lock status is updated. This will
improve job cancelling.

Signed-off-by: Michael Hanselmann <>
Reviewed-by: Iustin Pop <>

e35344b4 08/17/2010 04:25 pm Michael Hanselmann

jqueue: Add more debug output

Signed-off-by: Michael Hanselmann <>
Reviewed-by: Iustin Pop <>

aa9f8167 07/30/2010 04:43 am Iustin Pop

Fix a few job archival issues

This patch fixes two issues with job archival. First, the
LoadJobFromDisk can return 'None' for no-such-job, and we shouldn't add
None to the job list; we can't anyway, as this raises an exception:

node1# gnt-job archive foo...
599ee321 07/30/2010 12:52 am Iustin Pop

Change handling of non-Ganeti errors in jqueue

Currently, if a job execution raises a Ganeti-specific error (i.e.
subclass of GenericError), then we encode it as (error class, [error
args]). This matches the RAPI documentation.

However, if we get a non-Ganeti error, then we encode it as simply...

b2e8a4d9 07/29/2010 04:05 pm Michael Hanselmann

workerpool: Change signature of AddTask function to not use *args

By changing it to a normal parameter, which must be a sequence, we can
start using keyword parameters.

Before this patch all arguments to “AddTask(self, *args)” were passed as
arguments to the worker's “RunTask” method. Priorities, which should be...

7f93570a 07/16/2010 04:56 pm Iustin Pop

Implement lock names for debugging purposes

This patch adds lock names to SharedLocks and LockSets, that can be used
later for displaying the actual locks being held/used in places where we
only have the lock, and not the entire context of the locking operation....

989a8bee 07/15/2010 05:37 pm Michael Hanselmann

jqueue: Factorize code waiting for job changes

By splitting the _WaitForJobChangesHelper class into multiple smaller
classes, we gain in several places:

- Simpler code, less interaction between functions and variables
- Easy to unittest (close to 100% coverage)...

2034c70d 07/12/2010 05:27 pm Michael Hanselmann

jqueue: Setup inotify before checking for any job changes

Since the code waiting for job changes was modified to use inotify,
a race condition between checking for changes the first time and
setting up inotify occurs. If the job is modified after the check...

a744b676 07/09/2010 04:37 pm Manuel Franceschini

Introduce lib/netutils.py

This patch moves network utility functions to a dedicated module.

Signed-off-by: Manuel Franceschini <>
Reviewed-by: Iustin Pop <>

271daef8 07/06/2010 07:05 pm Iustin Pop

Fix opcode transition from WAITLOCK to RUNNING

With the recent changes in the job queue, an old bug surfaced: we never
serialized the status change when in NotifyStart, thus a crash of the
master would have left the job queue oblivious to the fact that the job...

ebb80afa 06/28/2010 02:04 pm Guido Trotter

jqueue: remove the _big_jqueue_lock module global

By using ssynchronized in the new way, we can remove the module-global
_big_jqueue_lock and revert back to an internal _lock inside the jqueue.

Signed-off-by: Guido Trotter <>
Reviewed-by: Iustin Pop <>

3c0d60d0 06/28/2010 02:04 pm Guido Trotter

Share the jqueue lock on job-local changes

We can share the jqueue lock when we do per-job updates. These only
conflict with updates/checks on the same job from another thread (eg.
CancelJob, ArchiveJob, which keep the lock unshared, since they are less
frequent)....

9bf5e01f 06/28/2010 02:04 pm Guido Trotter

_OpExecCallbacks abstract _AppendFeedback

Move some code to a decorated function rather than explicitely
acquiring/releasing the lock in AppendFeedback.

Signed-off-by: Guido Trotter <>
Reviewed-by: Iustin Pop <>

99bd4f0a 06/28/2010 02:04 pm Guido Trotter

jqueue: convert to a SharedLock()

Remove the jqueue _lock member and convert to a _big_jqueue_lock
sharedlock. This allows smooth transition from the old single lock to a
more granular approach.

Signed-off-by: Guido Trotter <>
Reviewed-by: Iustin Pop <>

39ed3a98 06/28/2010 02:04 pm Guido Trotter

MarkUnfinishedOps: update job file on disk

Every time we call MarkUnfinishedOps we do it in a try/finally block
that updates the job file. With this patch we move the try/finally
inside. CancelJobUnlocked is removed, because it just becomes a wrapper
over MarkUnfinishedOps with two constant values....

a1bfdeb1 06/28/2010 02:04 pm Guido Trotter

Remove spurious empty line

Signed-off-by: Guido Trotter <>
Reviewed-by: Iustin Pop <>

41593f6b 06/23/2010 01:32 pm Guido Trotter

Remove job object condition

We don't need it anymore, since nobody waits on it.

Signed-off-by: Guido Trotter <>
Reviewed-by: Iustin Pop <>

6c2549d6 06/23/2010 01:32 pm Guido Trotter

Parallelize WaitForJobChanges

As for QueryJobs we rely on file updates rather than condition
notification to acquire job changes. In order to do that we use the
pyinotify module to watch files. This might make the client a bit slower
(pending planned improvements, such as subscription-based...

b3855790 06/23/2010 01:32 pm Guido Trotter

Update the job file on feedback

This is needed to convert waitforjobchanges to use inotify and the
on-disk version and decouple it from the job queue lock. No replication
to remote nodes is done, to keep the operation fast.

Signed-off-by: Guido Trotter <>...

9f7b4967 06/23/2010 01:32 pm Guido Trotter

Don't lock on QueryJobs, by using the disk version

We move from querying the in-memory version to loading all jobs from the
disk. Since the jobs are written/deleted on disk in an atomic manner, we
don't need to lock at all. Also, since we're just looking at the...

0f9c08dc 06/23/2010 01:32 pm Guido Trotter

Add JobQueue.SafeLoadJobFromDisk

This will be used to read a job file without having to deal with
exceptions from _LoadJobFromDisk.

Signed-off-by: Guido Trotter <>
Reviewed-by: Iustin Pop <>

3d6c5566 06/23/2010 01:32 pm Guido Trotter

jqueue._LoadJobFromDisk: remove safety archival

Currently _LoadJobFromDisk archives job files it finds corrupted. Since
we want to use it to load files without holding locks, this could cause
a conflict: we just move the feature to _LoadJobUnlocked which is always...

7beb1e53 06/17/2010 08:25 pm Guido Trotter

jqueue.AddManyJobs: use AddManyTasks

Rather than adding the jobs to the worker pool one at a time, we add
them all together, which is slightly faster, and ensures they don't get
started while we loop.

Signed-off-by: Guido Trotter <>
Reviewed-by: Michael Hanselmann <>

4c36bdf5 06/17/2010 01:00 pm Guido Trotter

jqueue: make replication on job update optional

Sometimes it's useful to write to the local filesystem, but immediate
replication to all master candidates is not needed.

The _WriteAndReplicateFileUnlocked function gets renamed to
_UpdateJobQueueFile, as calling "write and replicate, but don't...

6a290889 06/17/2010 12:53 pm Guido Trotter

s/queue._GetJobInfoUnlocked/job.GetInfo/

The job queue currently has a static _GetJobInfoUnlocked method.
Changing it to be a normal method of _QueuedJob, which makes more sense.

Signed-off-by: Guido Trotter <>
Reviewed-by: Michael Hanselmann <>

162c8636 06/17/2010 12:53 pm Guido Trotter

Abstract loading job file from disk

Move the work from _LoadJobUnlocked to _LoadJobFileFromDisk, which can
then be used in other contexts as well. Also, if we fail to deserialize
the job, archive it as well (before we archived it only if we failed to
create the related object, but kept it there if deserialization failed....

d8e0dc17 06/15/2010 12:49 pm Guido Trotter

jqueue: simplify removal from _nodes

Somewhere we do try/del/except and somewhere just pop. Using pop
everywhere saves lines of code.

Signed-off-by: Guido Trotter <>
Reviewed-by: Iustin Pop <>

b5b8309d 06/15/2010 12:49 pm Guido Trotter

ListVisibleFiles: do not sort output

Among all users, turns out just one may need the output to be sorted.
All the others can cope without.

Signed-off-by: Guido Trotter <>
Reviewed-by: Iustin Pop <>

20571a26 06/11/2010 07:06 pm Guido Trotter

Cache a few bits of status in jqueue

Currently each time we submit a job we check the job queue size, and the
drained file. With this change we keep these pieces of information in
memory and don't read them from the filesystem each time.

Significant changes include:...

c910bccb 06/11/2010 05:06 pm Guido Trotter

Fix a TODO in _QueuedJob

Rather than raising Exception use GenericError and explain a bit better
what happened.

Signed-off-by: Guido Trotter <>
Reviewed-by: Michael Hanselmann <>

69b03fd7 06/11/2010 05:06 pm Guido Trotter

Remove unused parameter from function

This also removes the relevant pylint disable.
No point in keeping unused parameters around: if/when we need them it's
easy to add it back.

Signed-off-by: Guido Trotter <>
Reviewed-by: Michael Hanselmann <>

85a1c57d 06/11/2010 05:06 pm Guido Trotter

Optimize _GetJobIDsUnlocked

Currently we sort the list of job queue files twice (once in
utils.ListVisibleFiles with sort and then later with NiceSort). We apply
the _RE_JOB_FILE regular expression twice (once in _ListJobFiles and
once in _ExtractJobID). This simplifies the code a little, and a couple...

a71f9c7d 06/11/2010 05:06 pm Guido Trotter

jqueue: Rename _queue_lock to _queue_filelock

The name clarifies the difference between this and the internal lock.
Also explain a bit better what it is.

Signed-off-by: Guido Trotter <>
Reviewed-by: Michael Hanselmann <>

6abf7f2c 06/11/2010 12:23 pm Guido Trotter

Remove the job queue drain rpc call

This call was introduced but never used. In two years.
Since it's just creating/removing a file it can also be in simpler ways,
without a special rpc call, if/when we need it again. In the meantime,
let's give it to history....

b9b5abcb 06/01/2010 11:47 am Iustin Pop

Add a new opcode timestamp field

Since the current start_timestamp opcode attribute refers to the inital
start time, before locks are acquired, it's not useful to determine the
actual execution order of two opcodes/jobs competing for the same lock.

This patch adds a new field, exec_timestamp, that is updated when the...

0411c011 03/08/2010 03:52 pm Iustin Pop

Switch more code to PathJoin

This should remove most of the remaining constructs which can be
replaced by PathJoin.

Signed-off-by: Iustin Pop <>
Reviewed-by: Michael Hanselmann <>

c4feafe8 03/08/2010 03:48 pm Iustin Pop

Switch from os.path.join to utils.PathJoin

This passes a full burnin with lots of instances, and should be safe as
we mostly to join a known root (various constants) to a run-time
variable.

Signed-off-by: Iustin Pop <>
Reviewed-by: Michael Hanselmann <>

d2c8afb1 01/13/2010 05:47 pm Michael Hanselmann

jqueue: Don't return negative number for unchecked jobs when archiving

When the queue was empty, the calculation for unchecked jobs while
archiving would return -1. ``last_touched`` is set to 0, the job ID list
(``all_job_ids``) is empty. Calculating ``len(all_job_ids) -...

9fa2e150 01/13/2010 03:03 pm Michael Hanselmann

Improve logging for workerpool tasks by providing repr

Before it would log something like “starting task
(<ganeti.http.client._HttpClientPendingRequest object at 0x2aaaad176790>,)”,
which isn't really useful for debugging. Now it'll log “[…]
<ganeti.http.client._HttpClientPendingRequest...

02fc74da 01/13/2010 03:03 pm Michael Hanselmann

workerpool: Simplify log messages

Signed-off-by: Michael Hanselmann <>
Reviewed-by: Iustin Pop <>

89e2b4d2 01/13/2010 03:02 pm Michael Hanselmann

workerpool: Make worker ID alphanumeric

Having a proper name instead of just a number makes debugging
easier.

Signed-off-by: Michael Hanselmann <>
Reviewed-by: Iustin Pop <>

2d54e29c 01/04/2010 11:17 am Iustin Pop

Further pylint disables, mostly for Unused args

Many of our functions have to follow a given API, and thus we have to
keep a given signature, but pylint doesn't understand this. Therefore,
we silence this warning.

The patch does a few other cleanups.

Signed-off-by: Iustin Pop <>...

45e0d704 01/04/2010 11:16 am Iustin Pop

jqueue/_CheckRpcResult: log the whole operation

Currently only the rpc call, but not its description (which also shows
the argument) is logged. We change this to log failmsg too, and this
also silences a warning.

Signed-off-by: Iustin Pop <>...

7e950d31 01/04/2010 11:16 am Iustin Pop

Convert to static methods (where appropriate)

Many methods are simple pure functions, and not depending on the object
state. We convert these to staticmethods.

Signed-off-by: Iustin Pop <>
Reviewed-by: Olivier Tharan <>

7260cfbe 01/04/2010 11:15 am Iustin Pop

Add targeted pylint disables

This patch should have only:

- pylint disables
- docstring changes
- whitespace changes

Signed-off-by: Iustin Pop <>
Reviewed-by: Olivier Tharan <>

1f864b60 11/25/2009 04:30 pm Iustin Pop

Remove quotes from CommaJoin and convert to it

This patch removes the quotes from CommaJoin and converts most of the
callers (that I could find) to it. Since CommaJoin does str(i) for i in
param, we can remove these, thus simplifying slightly a few calls....

adfa97e3 11/06/2009 04:08 pm Guido Trotter

Processor: support a unique execution id

When the processor is executing a job, it can export the execution id to
its callers. This is not supported for Queries, as they're not executed
in a job.

Signed-off-by: Guido Trotter <>
Reviewed-by: Iustin Pop <>

6c881c52 11/06/2009 03:48 pm Iustin Pop

Fix pylint 'E' (error) codes

This patch adds some silences and tweaks the code slightly so that
“pylint --rcfile pylintrc -e ganeti” doesn't give any errors.

The biggest change is in jqueue.py, the move of _RequireOpenQueue out of
the JobQueue class. Since that is actually a function and not a method...

6bcb1446 11/03/2009 12:51 pm Michael Hanselmann

jqueue: Convert to utils.Retry

Signed-off-by: Michael Hanselmann <>
Reviewed-by: Iustin Pop <>

69b99987 10/12/2009 01:48 pm Michael Hanselmann

Code and docstring style fixes

Found using pylint and epydoc.

Signed-off-by: Michael Hanselmann <>
Reviewed-by: Guido Trotter <>

3cebe102 09/17/2009 06:53 pm Michael Hanselmann

Remove RpcResult.RemoteFailMsg completely

Signed-off-by: Michael Hanselmann <>
Reviewed-by: Iustin Pop <>

d25c1d6a 09/15/2009 02:19 pm Michael Hanselmann

jqueue: Remove unused run_op_index attribute

Signed-off-by: Michael Hanselmann <>
Reviewed-by: Iustin Pop <>

1d2dcdfd 09/15/2009 02:19 pm Michael Hanselmann

Export new lock_status field to gnt-job

Signed-off-by: Michael Hanselmann <>
Reviewed-by: Iustin Pop <>

ef2df7d3 09/15/2009 02:19 pm Michael Hanselmann

Keep lock status with every job

This can be useful for debugging locking problems.

Signed-off-by: Michael Hanselmann <>
Reviewed-by: Iustin Pop <>

031a3e57 09/15/2009 02:19 pm Michael Hanselmann

Move OpCode processor callbacks into separate class

There are two major arguments for this:
- There will be more callbacks (e.g. for lock debugging) and extending the
parameter list is a lot of work.
- In the jqueue module this allows us to keep per-job or per-opcode variables in...

4d5fe81b 09/14/2009 01:31 pm Michael Hanselmann

Merge commit 'origin/next' into branch-2.1

  • commit 'origin/next':
    Fix gnt-node modify online help
    Fix gnt-job info entry in gnt-job(8)
    locking: Don't swallow exceptions
    Add check for duplicate MACs in instance add
    scripts/gnt-node: fix a help string...
009e73d0 09/07/2009 04:38 pm Iustin Pop

Optimise multi-job submit

Currently, on multi-job submits we simply iterate over the
single-job-submit function. This means we grab a new serial, write and
replicate (and wait for the remote nodes to ack) the serial file, and
only then create the job file; this is repeated N times, once for each...

13998ef2 09/03/2009 06:43 pm Michael Hanselmann

Use ReadFile/WriteFile in more places

This survived QA, burnin and unittests.

Signed-off-by: Michael Hanselmann <>
Reviewed-by: Luca Bigliardi <>

bcb66fca 08/27/2009 06:21 pm Iustin Pop

Encode the actual exception raised by LU execution

Currently, the actual exception raised during an LU execution (one of
OpPrereqError, OpExecError, HooksError, etc.) is lost because the
jqueue.py code simply sets that to a str(err), and the code in cli.py...

17cb7017 08/03/2009 12:25 pm Michael Hanselmann

Merge branch 'next' into branch-2.1

  • next:
    Add “gnt-job watch” command
    jqueue: Fix error when WaitForJobChange gets invalid ID
    jqueue: Update message for cancelling running job
6e237482 08/03/2009 12:22 pm Michael Hanselmann

jqueue: Fix error when WaitForJobChange gets invalid ID

When JobQueue.WaitForJobChange gets an invalid or no longer existing job ID it
tries to return job_info and log_entries, both of which aren't defined yet.

Signed-off-by: Michael Hanselmann <>...

a9e97393 08/03/2009 12:20 pm Michael Hanselmann

jqueue: Update message for cancelling running job

Signed-off-by: Michael Hanselmann <>
Reviewed-by: Iustin Pop <>

25f9901f 07/19/2009 07:47 pm Iustin Pop

Merge commit 'origin/next' into branch-2.1

Conflicts:
lib/cli.py: trivial extra empty line

34327f51 07/19/2009 07:36 pm Iustin Pop

job queue: fix loss of finalized opcode result

Currently, unclean master daemon shutdown overwrites all of a job's
opcode status and result with error/None. This is incorrect, since the
any already finished opcode(s) should have their status and result
preserved, and only not-yet-processed opcodes should be marked as...

56d8ff91 07/19/2009 02:53 pm Iustin Pop

Add a luxi call for multi-job submit

As a workaround for the job submit timeouts that we have, this patch
adds a new luxi call for multi-job submit; the advantage is that all the
jobs are added in the queue and only after the workers can start
processing them....

f6424741 07/19/2009 02:34 pm Iustin Pop

job queue: fix interrupted job processing

If a job with more than one opcodes is being processed, and the master
daemon crashes between two opcodes, we have the first N opcodes marked
successful, and the rest marked as queued. This means that the overall...

ed21712b 07/19/2009 02:33 pm Iustin Pop

Fix an error path in job queue worker's RunTask

In case the job fails, we try to set the job's run_op_idx to -1.
However, this is a wrong variable, which wasn't detected until the
slots addition. The correct variable is run_op_index.

Signed-off-by: Iustin Pop <>...

66d895a8 07/17/2009 06:36 pm Iustin Pop

Add slots on objects in jqueue

Adding slots to _QueuedOpCode decreases memory usage (of these objects)
by roughly four times. It is a lesser change for _QueuedJobs.

Signed-off-by: Iustin Pop <>
Reviewed-by: Michael Hanselmann <>

2f7140ba 07/08/2009 12:28 pm Guido Trotter

Merge branch 'next' into branch-2.1

  • next:
    Create a new --no-voting option for masterfailover
    ganeti-masterd: allow non-interactive --no-voting
    Fix pylint warnings
    Add custom pylintrc
    bootstrap: Don't leak file descriptor when generating SSL certificate...
5bbd3f7f 07/07/2009 03:51 pm Michael Hanselmann

Fix some typos

Signed-off-by: Michael Hanselmann <>
Reviewed-by: Iustin Pop <>

c8457ce7 06/15/2009 08:08 pm Iustin Pop

Convert the jobqueue rpc to new style result

This patch converts the job queue rpc calls to the new style result.
It's done in a single patch as there are helper function (in both jqueue
and backend) that are used by multiple rpcs and need synchronized
change....

2971c913 05/21/2009 08:10 pm Iustin Pop

Add a luxi call for multi-job submit

As a workaround for the job submit timeouts that we have, this patch
adds a new luxi call for multi-job submit; the advantage is that all the
jobs are added in the queue and only after the workers can start
processing them....

0f6be82a 02/12/2009 08:13 pm Iustin Pop

job queue: log the opcode error too

Currently we only log "Error in opcode ...", but we don't log the error itself.
This is not good for debugging.

Reviewed-by: ultrotter

df0fb067 01/28/2009 04:46 pm Iustin Pop

Fix some issues related to job cancelling

This patch fixes two issues with the cancel mechanism:
- cancelled jobs show as such, and not in error state (we mark them as
OP_STATUS_CANCELED and not OP_STATUS_ERROR)
- queued jobs which are cancelled don't raise errors in the master (we...

5278185a 01/27/2009 05:41 pm Iustin Pop

Fix single-job archiving (gnt-job archive)

This is a simply typo from the conversion to multi-job archiving.

Reviewed-by: imsnah

d21d09d6 01/20/2009 07:19 pm Iustin Pop

Update the logging output of job processing

(this is related to the master daemon log)

Currently it's not possible to follow (in the non-debug runs) the
logical execution thread of jobs. This is due to the fact that we don't
log the thread name (so we lose the association of log messages to jobs)...

25e7b43f 01/15/2009 12:00 pm Iustin Pop

Some docstring updates

This patch rewraps some comments to shorter lengths, changes
double-quotes to single-quotes inside triple-quoted docstrings for
better editor handling.

It also fixes some epydoc errors, namely invalid crossreferences (after
method rename), documentation for inexistent (removed) parameters, etc....

dd875d32 12/18/2008 06:39 pm Michael Hanselmann

Job queue: Allow more than one file rename per RPC call

Reviewed-by: ultrotter

d7fd1f28 12/18/2008 06:38 pm Michael Hanselmann

ganeti.jqueue: Group job archivals to reduce number of RPC calls

Reducing the actual number of RPC calls will come in another patch.

Reviewed-by: ultrotter

f8ad5591 12/18/2008 06:38 pm Michael Hanselmann

Prevent RPC timeout on auto-archiving jobs

With a large job queue, auto-archiving jobs can take a very long time,
causing timeouts on the luxi RPC layer. With this change, auto-
archive returns after half of the RPC timeout has passed. The user
will see how many jobs are left unchecked....

78d12585 12/18/2008 06:38 pm Michael Hanselmann

jqueue: When auto-archiving jobs, calculate job status only once

This is done by passing the job object to _ArchiveJobUnlocked instead
of only the job ID. Also return whether job was actually archived.

Reviewed-by: ultrotter

58b22b6e 12/18/2008 06:23 pm Michael Hanselmann

Use subdirectories for job queue archive

As it turned out, having many files in a single directory can be
very painful. With this patch, only 10'000 files are stored in a
directory for the job queue archive. With 10'000 directries, this
allows for up to 100 million jobs be archived without having large...

f87b405e 12/17/2008 03:18 pm Michael Hanselmann

Add job queue size limit

A job queue with too many jobs can increase memory usage and/or make
the master daemon slow. The current limit is just an arbitrary number.
A "soft" limit for automatic job archival is prepared.

Reviewed-by: iustinp

9728ae5d 12/14/2008 02:02 pm Iustin Pop

cleanup: exceptions should derive from Exception

Reviewed-by: amishchenko

c41eea6e 12/11/2008 07:13 pm Iustin Pop

Fix epydoc format warnings

This patch should fix all outstanding epydoc parsing errors; as such, we
switch epydoc into verbose mode so that any new errors will be visible.

Reviewed-by: imsnah

59303563 12/02/2008 07:05 am Iustin Pop

Restrict job propagation to master candidates only

This patch restricts the job propagation to master candidates only, by
not registering non-candidates in the job queue node lists.

Note that we do intentionally purge the job queue if a node is toggled
to non-master status....

b7cb9024 11/28/2008 12:29 pm Michael Hanselmann

jqueue: Always print message for 100% when inspecting queue

Reviewed-by: iustinp

fbf0262f 11/28/2008 12:28 pm Michael Hanselmann

jqueue: Allow jobs waiting for locks to be canceled

- Add new "canceling" status
- Notify clients when job is canceled
- Give a return value from CancelJob
- Handle it in the client library

Reviewed-by: iustinp

33987705 11/27/2008 05:13 am Iustin Pop

jqueue: fix a bug in an error path

Dictionaries raise KeyError, and not ValueError when invalid keys are
passes to del.

Reviewed-by: imsnah

711b5124 11/26/2008 06:16 pm Michael Hanselmann

jqueue: Log progress and load jobs one by one

By logging more information, a user can see how far it is in inspecting
the queue. This can be useful with a large number of jobs. Also, instead
of loading all jobs in one go, load only the list of job IDs and then...

16714921 11/26/2008 06:15 pm Michael Hanselmann

jqueue: Shutdown workerpool in case of a problem

Reviewed-by: ultrotter

a3811745 11/12/2008 02:51 pm Michael Hanselmann

jqueue: Always use rpc.RpcRunner

"from ganeti.rpc import RpcRunner" does not conform to the style guide.

Reviewed-by: iustinp