History | View | Annotate | Download (57.1 kB)
jqueue: Use timeout when acquiring locks
As already noted in the design document, an opcode's priority isincreased when the lock(s) can't be acquired within a certain amount oftime, except at the highest priority, where in such a case a blockingacquire is used....
jqueue: Introduce per-opcode context object
This is better to group per-opcode data.
Signed-off-by: Michael Hanselmann <hansmi@google.com>Reviewed-by: René Nussbaumer <rn@google.com>
jqueue: Rename current_op to better reflect what it actually is
Signed-off-by: Michael Hanselmann <hansmi@google.com>Reviewed-by: Iustin Pop <iustin@google.com>
jqueue: Separate function for in-memory variables
jqueue: Change model from per-job to per-opcode processing
In order to support priorities, the processing of jobs needs to bechanged. Instead of processing jobs as a whole, the code is changed toprocess one opcode at a time and then return to the queue. See the...
jqueue: Use priority for worker pool
A small helper function is added to make this easier. Priorities are notyet used in all necessary places.
jqueue: Add missing docstring to _QueuedJob.Cancel
This was forgotten in commit 099b2870b.
jqueue: Move CancelJob logic to separate function
Moving the internals of this function will allow it to be used fromunittests in the future. Splitting this into a pure, side-effect freefunction and an impure one makes the pure function easily testable....
Merge branch 'devel-2.2'
(no conflicts, took LGTM from original commit)
Signed-off-by: Iustin Pop <iustin@google.com>...
jqueue: Ensure only accepted priorities are allowed for submitting jobs
Quoting the design document: “Submitted opcodes can have one of the prioritieslisted below. Other priorities are reserved for internal use”. Submitting jobsat priority -20 should not be allowed....
Add support for job priority to opcodes and job queue objects
This allows clients to submit opcodes with a priority. Except for beingtracked by the job queue, it is not yet used by any code.
Unittests for jqueue._QueuedOpCode and jqueue._QueuedJob are provided for...
Remove mcpu's ReportLocks callback
This is no longer needed with the new lock monitor. One callback is kept tocheck for cancelled jobs.
Revert "jqueue: Resume jobs from “waitlock” status"
This reverts commit 4008c8edae31a3971fa8c4b200238afc8005d3d4.
While it worked in my initial tests, I've now found cases where this doesn'twork properly as it is. More work is needed and will be done as part of the...
jqueue: Resume jobs from “waitlock” status
After an unclean restart of ganeti-masterd, jobs in the “waitlock” status canbe safely restarted. They hadn't modified the cluster yet.
jqueue: Move queue inspection into separate function
This makes the init function a lot smaller while not changingfunctionality.
jqueue: Don't update file in MarkUnfinishedOps
This reduced the number of updates to the job files. It's used in two placeswhile processing a job and the file is updated just afterwards.
Move job queue to new ganeti.runtime
Signed-off-by: René Nussbaumer <rn@google.com>Reviewed-by: Michael Hanselmann <hansmi@google.com>
jqueue: Use separate function for encoding errors
Comes with unittest.
hansmi helped me with merging the conflict. Thanks
Conflicts: lib/workerpool.py
Signed-off-by: René Nussbaumer <rn@google.com>Reviewed-by: Iustin Pop <iustin@google.com>
workerpool: Allow setting task name
With this patch, the task name is added to the thread name and will show up inlogs. Log messages from jobs will look like “pid=578/JobQueue14/Job13 mcpu:289DEBUG LU locks acquired/cluster/BGL/shared”.
Signed-off-by: Michael Hanselmann <hansmi@google.com>...
jqueue: Remove lock status field
With the job queue changes for Ganeti 2.2, watched and queried jobs areloaded directly from disk, rendering the in-memory “lock_status” fielduseless. Writing it to disk would be possible, but has a huge cost atruntime (when tested, processing 1'000 opcodes involved 4'000 additional...
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 andassertions are added to check this.
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 jobfile is only written once with a finalized status.
Support for resolving hostnames to IPv6 addresses
This patch enables IPv6 name resolution by using socket.getaddrinfoinstead of socket.gethostbyname_ex.
It renames the HostInfo class to Hostname and unifies its use throughoutthe code. This is achieved by using static calls where no object is...
jqueue: More checks for cancelling queued job
We can also check when the lock status is updated. This willimprove job cancelling.
jqueue: Add more debug output
Fix a few job archival issues
This patch fixes two issues with job archival. First, theLoadJobFromDisk can return 'None' for no-such-job, and we shouldn't addNone to the job list; we can't anyway, as this raises an exception:
node1# gnt-job archive foo...
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, [errorargs]). This matches the RAPI documentation.
However, if we get a non-Ganeti error, then we encode it as simply...
workerpool: Change signature of AddTask function to not use *args
By changing it to a normal parameter, which must be a sequence, we canstart using keyword parameters.
Before this patch all arguments to “AddTask(self, *args)” were passed asarguments to the worker's “RunTask” method. Priorities, which should be...
Implement lock names for debugging purposes
This patch adds lock names to SharedLocks and LockSets, that can be usedlater for displaying the actual locks being held/used in places where weonly have the lock, and not the entire context of the locking operation....
jqueue: Factorize code waiting for job changes
By splitting the _WaitForJobChangesHelper class into multiple smallerclasses, we gain in several places:
- Simpler code, less interaction between functions and variables- Easy to unittest (close to 100% coverage)...
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 andsetting up inotify occurs. If the job is modified after the check...
Introduce lib/netutils.py
This patch moves network utility functions to a dedicated module.
Signed-off-by: Manuel Franceschini <livewire@google.com>Reviewed-by: Iustin Pop <iustin@google.com>
Fix opcode transition from WAITLOCK to RUNNING
With the recent changes in the job queue, an old bug surfaced: we neverserialized the status change when in NotifyStart, thus a crash of themaster would have left the job queue oblivious to the fact that the job...
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 <ultrotter@google.com>Reviewed-by: Iustin Pop <iustin@google.com>
Share the jqueue lock on job-local changes
We can share the jqueue lock when we do per-job updates. These onlyconflict with updates/checks on the same job from another thread (eg.CancelJob, ArchiveJob, which keep the lock unshared, since they are lessfrequent)....
_OpExecCallbacks abstract _AppendFeedback
Move some code to a decorated function rather than explicitelyacquiring/releasing the lock in AppendFeedback.
jqueue: convert to a SharedLock()
Remove the jqueue _lock member and convert to a _big_jqueue_locksharedlock. This allows smooth transition from the old single lock to amore granular approach.
MarkUnfinishedOps: update job file on disk
Every time we call MarkUnfinishedOps we do it in a try/finally blockthat updates the job file. With this patch we move the try/finallyinside. CancelJobUnlocked is removed, because it just becomes a wrapperover MarkUnfinishedOps with two constant values....
Remove spurious empty line
Remove job object condition
We don't need it anymore, since nobody waits on it.
Parallelize WaitForJobChanges
As for QueryJobs we rely on file updates rather than conditionnotification to acquire job changes. In order to do that we use thepyinotify module to watch files. This might make the client a bit slower(pending planned improvements, such as subscription-based...
Update the job file on feedback
This is needed to convert waitforjobchanges to use inotify and theon-disk version and decouple it from the job queue lock. No replicationto remote nodes is done, to keep the operation fast.
Signed-off-by: Guido Trotter <ultrotter@google.com>...
Don't lock on QueryJobs, by using the disk version
We move from querying the in-memory version to loading all jobs from thedisk. Since the jobs are written/deleted on disk in an atomic manner, wedon't need to lock at all. Also, since we're just looking at the...
Add JobQueue.SafeLoadJobFromDisk
This will be used to read a job file without having to deal withexceptions from _LoadJobFromDisk.
jqueue._LoadJobFromDisk: remove safety archival
Currently _LoadJobFromDisk archives job files it finds corrupted. Sincewe want to use it to load files without holding locks, this could causea conflict: we just move the feature to _LoadJobUnlocked which is always...
jqueue.AddManyJobs: use AddManyTasks
Rather than adding the jobs to the worker pool one at a time, we addthem all together, which is slightly faster, and ensures they don't getstarted while we loop.
Signed-off-by: Guido Trotter <ultrotter@google.com>Reviewed-by: Michael Hanselmann <hansmi@google.com>
jqueue: make replication on job update optional
Sometimes it's useful to write to the local filesystem, but immediatereplication to all master candidates is not needed.
The _WriteAndReplicateFileUnlocked function gets renamed to_UpdateJobQueueFile, as calling "write and replicate, but don't...
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.
Abstract loading job file from disk
Move the work from _LoadJobUnlocked to _LoadJobFileFromDisk, which canthen be used in other contexts as well. Also, if we fail to deserializethe job, archive it as well (before we archived it only if we failed tocreate the related object, but kept it there if deserialization failed....
jqueue: simplify removal from _nodes
Somewhere we do try/del/except and somewhere just pop. Using popeverywhere saves lines of code.
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.
Cache a few bits of status in jqueue
Currently each time we submit a job we check the job queue size, and thedrained file. With this change we keep these pieces of information inmemory and don't read them from the filesystem each time.
Significant changes include:...
Fix a TODO in _QueuedJob
Rather than raising Exception use GenericError and explain a bit betterwhat happened.
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'seasy to add it back.
Optimize _GetJobIDsUnlocked
Currently we sort the list of job queue files twice (once inutils.ListVisibleFiles with sort and then later with NiceSort). We applythe _RE_JOB_FILE regular expression twice (once in _ListJobFiles andonce in _ExtractJobID). This simplifies the code a little, and a couple...
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.
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....
Add a new opcode timestamp field
Since the current start_timestamp opcode attribute refers to the initalstart time, before locks are acquired, it's not useful to determine theactual execution order of two opcodes/jobs competing for the same lock.
This patch adds a new field, exec_timestamp, that is updated when the...
Switch more code to PathJoin
This should remove most of the remaining constructs which can bereplaced by PathJoin.
Signed-off-by: Iustin Pop <iustin@google.com>Reviewed-by: Michael Hanselmann <hansmi@google.com>
Switch from os.path.join to utils.PathJoin
This passes a full burnin with lots of instances, and should be safe aswe mostly to join a known root (various constants) to a run-timevariable.
jqueue: Don't return negative number for unchecked jobs when archiving
When the queue was empty, the calculation for unchecked jobs whilearchiving would return -1. ``last_touched`` is set to 0, the job ID list(``all_job_ids``) is empty. Calculating ``len(all_job_ids) -...
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...
workerpool: Simplify log messages
workerpool: Make worker ID alphanumeric
Having a proper name instead of just a number makes debuggingeasier.
Further pylint disables, mostly for Unused args
Many of our functions have to follow a given API, and thus we have tokeep a given signature, but pylint doesn't understand this. Therefore,we silence this warning.
The patch does a few other cleanups.
jqueue/_CheckRpcResult: log the whole operation
Currently only the rpc call, but not its description (which also showsthe argument) is logged. We change this to log failmsg too, and thisalso silences a warning.
Convert to static methods (where appropriate)
Many methods are simple pure functions, and not depending on the objectstate. We convert these to staticmethods.
Signed-off-by: Iustin Pop <iustin@google.com>Reviewed-by: Olivier Tharan <olive@google.com>
Add targeted pylint disables
This patch should have only:
- pylint disables- docstring changes- whitespace changes
Remove quotes from CommaJoin and convert to it
This patch removes the quotes from CommaJoin and converts most of thecallers (that I could find) to it. Since CommaJoin does str(i) for i inparam, we can remove these, thus simplifying slightly a few calls....
Processor: support a unique execution id
When the processor is executing a job, it can export the execution id toits callers. This is not supported for Queries, as they're not executedin a job.
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 ofthe JobQueue class. Since that is actually a function and not a method...
jqueue: Convert to utils.Retry
Code and docstring style fixes
Found using pylint and epydoc.
Signed-off-by: Michael Hanselmann <hansmi@google.com>Reviewed-by: Guido Trotter <ultrotter@google.com>
Remove RpcResult.RemoteFailMsg completely
jqueue: Remove unused run_op_index attribute
Export new lock_status field to gnt-job
Keep lock status with every job
This can be useful for debugging locking problems.
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...
Merge commit 'origin/next' into branch-2.1
Optimise multi-job submit
Currently, on multi-job submits we simply iterate over thesingle-job-submit function. This means we grab a new serial, write andreplicate (and wait for the remote nodes to ack) the serial file, andonly then create the job file; this is repeated N times, once for each...
Use ReadFile/WriteFile in more places
This survived QA, burnin and unittests.
Signed-off-by: Michael Hanselmann <hansmi@google.com>Reviewed-by: Luca Bigliardi <shammash@google.com>
Encode the actual exception raised by LU execution
Currently, the actual exception raised during an LU execution (one ofOpPrereqError, OpExecError, HooksError, etc.) is lost because thejqueue.py code simply sets that to a str(err), and the code in cli.py...
Merge branch 'next' into branch-2.1
jqueue: Fix error when WaitForJobChange gets invalid ID
When JobQueue.WaitForJobChange gets an invalid or no longer existing job ID ittries to return job_info and log_entries, both of which aren't defined yet.
jqueue: Update message for cancelling running job
Conflicts: lib/cli.py: trivial extra empty line
job queue: fix loss of finalized opcode result
Currently, unclean master daemon shutdown overwrites all of a job'sopcode status and result with error/None. This is incorrect, since theany already finished opcode(s) should have their status and resultpreserved, and only not-yet-processed opcodes should be marked as...
Add a luxi call for multi-job submit
As a workaround for the job submit timeouts that we have, this patchadds a new luxi call for multi-job submit; the advantage is that all thejobs are added in the queue and only after the workers can startprocessing them....
job queue: fix interrupted job processing
If a job with more than one opcodes is being processed, and the masterdaemon crashes between two opcodes, we have the first N opcodes markedsuccessful, and the rest marked as queued. This means that the overall...
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 theslots addition. The correct variable is run_op_index.
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.
Fix some typos
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 jqueueand backend) that are used by multiple rpcs and need synchronizedchange....