You cannot select more than 25 topics Topics must start with a letter or number, can include dashes ('-') and can be up to 35 characters long.

842 lines
32 KiB
Python

import json
import os
from datetime import datetime, timedelta, timezone
Allows enqueueing by the cli (#1466) * Allows enqueueing by the cli #372 * schedule support * `_` to `-` * fix flake8 * echo job-id * Some improvements - Description as in python jobs - return result - quiet mode - allows `--boolean` and `--integer` - raises errors if not used correctly * added tests * add schedule tests * add retry test * use click exceptions * add error test * add job_func test * change messages https://github.com/rq/rq/pull/1466#discussion_r640211128 https://github.com/rq/rq/pull/1466#discussion_r640210850 * Use different format for arguments View https://github.com/rq/rq/pull/1466#discussion_r650510889 * Add file support Usage: @filename * ast.literal_eval support with `#` instead of `:` * func -> function Makes error messages more readable * click Error * print function string * add docs * increase seconds in test * Update `parse_function_arg` Add `ParsingMode` enum (https://github.com/rq/rq/pull/1466#discussion_r656676114) Change error messages (https://github.com/rq/rq/pull/1466#discussion_r656676800, https://github.com/rq/rq/pull/1466#discussion_r656677082) * `#` to `%` `#` is the letter for a comment in bash * Add some tests (https://github.com/rq/rq/pull/1466#discussion_r656674539, https://github.com/rq/rq/pull/1466#discussion_r656676543) * Add some tests * docs: Add some examples * catch all literal_eval exceptions There are some edge cases with other exceptions * remove job_func (https://github.com/rq/rq/pull/1466#pullrequestreview-690110118) * edit docs https://github.com/rq/rq/pull/1466#pullrequestreview-695758691 * format examples * format examples `queue.enqueue(path.to.func, args=['abc'])` to `queue.enqueue(path.to.func, 'abc')` https://github.com/rq/rq/pull/1466#discussion_r673615464 * add examples https://github.com/rq/rq/pull/1466#discussion_r673658933 * add doc test https://github.com/rq/rq/pull/1466#discussion_r673659124 * Update index.md * Update test_cli.py * Update test_cli.py * Add version info Co-authored-by: rpkak <rpkak@users.noreply.github.com>
3 years ago
from time import sleep
from uuid import uuid4
Job scheduling (#1163) * First RQScheduler prototype * WIP job scheduling * Fixed Python 2.7 tests * Added ScheduledJobRegistry.get_scheduled_time(job) * WIP on scheduler's threading mechanism * Fixed test errors * Changed scheduler.acquire_locks() to instance method * Added scheduler.prepare_registries() * Somewhat working implementation of RQ scheduler * Only call stop_scheduler if there's a scheduler present * Use OSError rather than ProcessLookupError for PyPy compatibility * Added `auto_start` argument to scheduler.acquire_locks() * Make RQScheduler play better with timezone * Fixed test error * Added --with-scheduler flag to rq worker CLI * Fix tests on Python 2.x * More Python 2 fixes * Only call `scheduler.start` if worker is run in non burst mode * Fixed an issue where running worker with scheduler would fail sometimes * Make `worker.stop_scheduler()` more resilient to errors * worker.dequeue_job_and_maintain_ttl() should also periodically run maintenance tasks * Scheduler can now work with worker in both burst and non burst mode * Fixed scheduler logging message * Always log scheduler errors when running * Improve scheduler error logging message * Removed testing code * Scheduler should periodically try to acquire locks for other queues it doesn't have * Added tests for scheduler.should_reacquire_locks * Added queue.enqueue_in() * Fixes queue.enqueue_in() in Python 2.7 * First stab at documenting job scheduling * Remove unused methods * Remove Python 2.6 logging compatibility code * Remove more unused imports * Added convenience methods to access job registries from queue * Added test for worker.run_maintenance_tasks() * Simplify worker.queue_names() and worker.queue_keys() * Updated changelog to mention RQ's new job scheduling mechanism.
5 years ago
import pytest
from click.testing import CliRunner
RQ v1.0! (#1059) * Added FailedJobRegistry. * Added job.failure_ttl. * queue.enqueue() now supports failure_ttl * Added registry.get_queue(). * FailedJobRegistry.add() now assigns DEFAULT_FAILURE_TTL. * StartedJobRegistry.cleanup() now moves expired jobs to FailedJobRegistry. * Failed jobs are now added to FailedJobRegistry. * Added FailedJobRegistry.requeue() * Document the new `FailedJobRegistry` and changes in custom exception handler behavior. * Added worker.disable_default_exception_handler. * Document --disable-default-exception-handler option. * Deleted worker.failed_queue. * Deleted "move_to_failed_queue" exception handler. * StartedJobRegistry should no longer move jobs to FailedQueue. * Deleted requeue_job * Fixed test error. * Make requeue cli command work with FailedJobRegistry * Added .pytest_cache to gitignore. * Custom exception handlers are no longer run in reverse * Restored requeue_job function * Removed get_failed_queue * Deleted FailedQueue * Updated changelog. * Document `failure_ttl` * Updated docs. * Remove job.status * Fixed typo in test_registry.py * Replaced _pipeline() with pipeline() * FailedJobRegistry no longer fails on redis-py>=3 * Fixes test_clean_registries * Worker names are now randomized * Added a note about random worker names in CHANGES.md * Worker will now stop working when encountering an unhandled exception. * Worker should reraise SystemExit on cold shutdowns * Added anchor.js to docs * Support for Sentry-SDK (#1045) * Updated RQ to support sentry-sdk * Document Sentry integration * Install sentry-sdk before running tests * Improved rq info CLI command to be more efficient when displaying lar… (#1046) * Improved rq info CLI command to be more efficient when displaying large number of workers * Fixed an rq info --by-queue bug * Fixed worker.total_working_time bug (#1047) * queue.enqueue() no longer accepts `timeout` argument (#1055) * Clean worker registry (#1056) * queue.enqueue() no longer accepts `timeout` argument * Added clean_worker_registry() * Show worker hostname and PID on cli (#1058) * Show worker hostname and PID on cli * Improve test coverage * Remove Redis version check when SSL is used * Bump version to 1.0 * Removed pytest_cache/README.md * Changed worker logging to use exc_info=True * Removed unused queue.dequeue() * Fixed typo in CHANGES.md * setup_loghandlers() should always call logger.setLevel() if specified
6 years ago
from redis import Redis
from rq import Queue
from rq.cli import main
from rq.cli.helpers import CliConfig, parse_function_arg, parse_schedule, read_config_file
Worker pool (#1874) * First stab at implementating worker pool * Use process.is_alive() to check whether a process is still live * Handle shutdown signal * Check worker loop done * First working version of `WorkerPool`. * Added test for check_workers() * Added test for pool.start() * Better shutdown process * Comment out test_start() to see if it fixes CI * Make tests pass * Make CI pass * Comment out some tests * Comment out more tests * Re-enable a test * Re-enable another test * Uncomment check_workers test * Added run_worker test * Minor modification to dead worker detection * More test cases * Better process name for workers * Added back pool.stop_workers() when signal is received * Cleaned up cli.py * WIP on worker-pool command * Fix test * Test that worker pool ignores consecutive shutdown signals * Added test for worker-pool CLI command. * Added timeout to CI jobs * Fix worker pool test * Comment out test_scheduler.py * Fixed worker-pool in burst mode * Increase test coverage * Exclude tests directory from coverage.py * Improve test coverage * Renamed `Pool(num_workers=2) to `Pool(size=2)` * Revert "Renamed `Pool(num_workers=2) to `Pool(size=2)`" This reverts commit a1306f89ad0d8686c6bde447bff75e2f71f0733b. * Renamed Pool to WorkerPool * Added a new TestCase that doesn't use LocalStack * Added job_class, worker_class and serializer arguments to WorkerPool * Use parse_connection() in WorkerPool.__init__ * Added CLI arguments for worker-pool * Minor WorkerPool and test fixes * Fixed failing CLI test * Document WorkerPool
2 years ago
from rq.job import Job, JobStatus
Job scheduling (#1163) * First RQScheduler prototype * WIP job scheduling * Fixed Python 2.7 tests * Added ScheduledJobRegistry.get_scheduled_time(job) * WIP on scheduler's threading mechanism * Fixed test errors * Changed scheduler.acquire_locks() to instance method * Added scheduler.prepare_registries() * Somewhat working implementation of RQ scheduler * Only call stop_scheduler if there's a scheduler present * Use OSError rather than ProcessLookupError for PyPy compatibility * Added `auto_start` argument to scheduler.acquire_locks() * Make RQScheduler play better with timezone * Fixed test error * Added --with-scheduler flag to rq worker CLI * Fix tests on Python 2.x * More Python 2 fixes * Only call `scheduler.start` if worker is run in non burst mode * Fixed an issue where running worker with scheduler would fail sometimes * Make `worker.stop_scheduler()` more resilient to errors * worker.dequeue_job_and_maintain_ttl() should also periodically run maintenance tasks * Scheduler can now work with worker in both burst and non burst mode * Fixed scheduler logging message * Always log scheduler errors when running * Improve scheduler error logging message * Removed testing code * Scheduler should periodically try to acquire locks for other queues it doesn't have * Added tests for scheduler.should_reacquire_locks * Added queue.enqueue_in() * Fixes queue.enqueue_in() in Python 2.7 * First stab at documenting job scheduling * Remove unused methods * Remove Python 2.6 logging compatibility code * Remove more unused imports * Added convenience methods to access job registries from queue * Added test for worker.run_maintenance_tasks() * Simplify worker.queue_names() and worker.queue_keys() * Updated changelog to mention RQ's new job scheduling mechanism.
5 years ago
from rq.registry import FailedJobRegistry, ScheduledJobRegistry
from rq.scheduler import RQScheduler
from rq.serializers import JSONSerializer
from rq.timeouts import UnixSignalDeathPenalty
RQ v1.0! (#1059) * Added FailedJobRegistry. * Added job.failure_ttl. * queue.enqueue() now supports failure_ttl * Added registry.get_queue(). * FailedJobRegistry.add() now assigns DEFAULT_FAILURE_TTL. * StartedJobRegistry.cleanup() now moves expired jobs to FailedJobRegistry. * Failed jobs are now added to FailedJobRegistry. * Added FailedJobRegistry.requeue() * Document the new `FailedJobRegistry` and changes in custom exception handler behavior. * Added worker.disable_default_exception_handler. * Document --disable-default-exception-handler option. * Deleted worker.failed_queue. * Deleted "move_to_failed_queue" exception handler. * StartedJobRegistry should no longer move jobs to FailedQueue. * Deleted requeue_job * Fixed test error. * Make requeue cli command work with FailedJobRegistry * Added .pytest_cache to gitignore. * Custom exception handlers are no longer run in reverse * Restored requeue_job function * Removed get_failed_queue * Deleted FailedQueue * Updated changelog. * Document `failure_ttl` * Updated docs. * Remove job.status * Fixed typo in test_registry.py * Replaced _pipeline() with pipeline() * FailedJobRegistry no longer fails on redis-py>=3 * Fixes test_clean_registries * Worker names are now randomized * Added a note about random worker names in CHANGES.md * Worker will now stop working when encountering an unhandled exception. * Worker should reraise SystemExit on cold shutdowns * Added anchor.js to docs * Support for Sentry-SDK (#1045) * Updated RQ to support sentry-sdk * Document Sentry integration * Install sentry-sdk before running tests * Improved rq info CLI command to be more efficient when displaying lar… (#1046) * Improved rq info CLI command to be more efficient when displaying large number of workers * Fixed an rq info --by-queue bug * Fixed worker.total_working_time bug (#1047) * queue.enqueue() no longer accepts `timeout` argument (#1055) * Clean worker registry (#1056) * queue.enqueue() no longer accepts `timeout` argument * Added clean_worker_registry() * Show worker hostname and PID on cli (#1058) * Show worker hostname and PID on cli * Improve test coverage * Remove Redis version check when SSL is used * Bump version to 1.0 * Removed pytest_cache/README.md * Changed worker logging to use exc_info=True * Removed unused queue.dequeue() * Fixed typo in CHANGES.md * setup_loghandlers() should always call logger.setLevel() if specified
6 years ago
from rq.worker import Worker, WorkerStatus
from tests import RQTestCase
RQ v1.0! (#1059) * Added FailedJobRegistry. * Added job.failure_ttl. * queue.enqueue() now supports failure_ttl * Added registry.get_queue(). * FailedJobRegistry.add() now assigns DEFAULT_FAILURE_TTL. * StartedJobRegistry.cleanup() now moves expired jobs to FailedJobRegistry. * Failed jobs are now added to FailedJobRegistry. * Added FailedJobRegistry.requeue() * Document the new `FailedJobRegistry` and changes in custom exception handler behavior. * Added worker.disable_default_exception_handler. * Document --disable-default-exception-handler option. * Deleted worker.failed_queue. * Deleted "move_to_failed_queue" exception handler. * StartedJobRegistry should no longer move jobs to FailedQueue. * Deleted requeue_job * Fixed test error. * Make requeue cli command work with FailedJobRegistry * Added .pytest_cache to gitignore. * Custom exception handlers are no longer run in reverse * Restored requeue_job function * Removed get_failed_queue * Deleted FailedQueue * Updated changelog. * Document `failure_ttl` * Updated docs. * Remove job.status * Fixed typo in test_registry.py * Replaced _pipeline() with pipeline() * FailedJobRegistry no longer fails on redis-py>=3 * Fixes test_clean_registries * Worker names are now randomized * Added a note about random worker names in CHANGES.md * Worker will now stop working when encountering an unhandled exception. * Worker should reraise SystemExit on cold shutdowns * Added anchor.js to docs * Support for Sentry-SDK (#1045) * Updated RQ to support sentry-sdk * Document Sentry integration * Install sentry-sdk before running tests * Improved rq info CLI command to be more efficient when displaying lar… (#1046) * Improved rq info CLI command to be more efficient when displaying large number of workers * Fixed an rq info --by-queue bug * Fixed worker.total_working_time bug (#1047) * queue.enqueue() no longer accepts `timeout` argument (#1055) * Clean worker registry (#1056) * queue.enqueue() no longer accepts `timeout` argument * Added clean_worker_registry() * Show worker hostname and PID on cli (#1058) * Show worker hostname and PID on cli * Improve test coverage * Remove Redis version check when SSL is used * Bump version to 1.0 * Removed pytest_cache/README.md * Changed worker logging to use exc_info=True * Removed unused queue.dequeue() * Fixed typo in CHANGES.md * setup_loghandlers() should always call logger.setLevel() if specified
6 years ago
from tests.fixtures import div_by_zero, say_hello
Worker pool (#1874) * First stab at implementating worker pool * Use process.is_alive() to check whether a process is still live * Handle shutdown signal * Check worker loop done * First working version of `WorkerPool`. * Added test for check_workers() * Added test for pool.start() * Better shutdown process * Comment out test_start() to see if it fixes CI * Make tests pass * Make CI pass * Comment out some tests * Comment out more tests * Re-enable a test * Re-enable another test * Uncomment check_workers test * Added run_worker test * Minor modification to dead worker detection * More test cases * Better process name for workers * Added back pool.stop_workers() when signal is received * Cleaned up cli.py * WIP on worker-pool command * Fix test * Test that worker pool ignores consecutive shutdown signals * Added test for worker-pool CLI command. * Added timeout to CI jobs * Fix worker pool test * Comment out test_scheduler.py * Fixed worker-pool in burst mode * Increase test coverage * Exclude tests directory from coverage.py * Improve test coverage * Renamed `Pool(num_workers=2) to `Pool(size=2)` * Revert "Renamed `Pool(num_workers=2) to `Pool(size=2)`" This reverts commit a1306f89ad0d8686c6bde447bff75e2f71f0733b. * Renamed Pool to WorkerPool * Added a new TestCase that doesn't use LocalStack * Added job_class, worker_class and serializer arguments to WorkerPool * Use parse_connection() in WorkerPool.__init__ * Added CLI arguments for worker-pool * Minor WorkerPool and test fixes * Fixed failing CLI test * Document WorkerPool
2 years ago
class CLITestCase(RQTestCase):
def setUp(self):
super().setUp()
db_num = self.testconn.connection_pool.connection_kwargs['db']
self.redis_url = 'redis://127.0.0.1:6379/%d' % db_num
self.connection = Redis.from_url(self.redis_url)
def assert_normal_execution(self, result):
if result.exit_code == 0:
return True
else:
print("Non normal execution")
print("Exit Code: {}".format(result.exit_code))
print("Output: {}".format(result.output))
print("Exception: {}".format(result.exception))
self.assertEqual(result.exit_code, 0)
Worker pool (#1874) * First stab at implementating worker pool * Use process.is_alive() to check whether a process is still live * Handle shutdown signal * Check worker loop done * First working version of `WorkerPool`. * Added test for check_workers() * Added test for pool.start() * Better shutdown process * Comment out test_start() to see if it fixes CI * Make tests pass * Make CI pass * Comment out some tests * Comment out more tests * Re-enable a test * Re-enable another test * Uncomment check_workers test * Added run_worker test * Minor modification to dead worker detection * More test cases * Better process name for workers * Added back pool.stop_workers() when signal is received * Cleaned up cli.py * WIP on worker-pool command * Fix test * Test that worker pool ignores consecutive shutdown signals * Added test for worker-pool CLI command. * Added timeout to CI jobs * Fix worker pool test * Comment out test_scheduler.py * Fixed worker-pool in burst mode * Increase test coverage * Exclude tests directory from coverage.py * Improve test coverage * Renamed `Pool(num_workers=2) to `Pool(size=2)` * Revert "Renamed `Pool(num_workers=2) to `Pool(size=2)`" This reverts commit a1306f89ad0d8686c6bde447bff75e2f71f0733b. * Renamed Pool to WorkerPool * Added a new TestCase that doesn't use LocalStack * Added job_class, worker_class and serializer arguments to WorkerPool * Use parse_connection() in WorkerPool.__init__ * Added CLI arguments for worker-pool * Minor WorkerPool and test fixes * Fixed failing CLI test * Document WorkerPool
2 years ago
class TestRQCli(CLITestCase):
@pytest.fixture(autouse=True)
def set_tmpdir(self, tmpdir):
self.tmpdir = tmpdir
def assert_normal_execution(self, result):
if result.exit_code == 0:
return True
else:
print("Non normal execution")
print("Exit Code: {}".format(result.exit_code))
print("Output: {}".format(result.output))
print("Exception: {}".format(result.exception))
self.assertEqual(result.exit_code, 0)
"""Test rq_cli script"""
Worker pool (#1874) * First stab at implementating worker pool * Use process.is_alive() to check whether a process is still live * Handle shutdown signal * Check worker loop done * First working version of `WorkerPool`. * Added test for check_workers() * Added test for pool.start() * Better shutdown process * Comment out test_start() to see if it fixes CI * Make tests pass * Make CI pass * Comment out some tests * Comment out more tests * Re-enable a test * Re-enable another test * Uncomment check_workers test * Added run_worker test * Minor modification to dead worker detection * More test cases * Better process name for workers * Added back pool.stop_workers() when signal is received * Cleaned up cli.py * WIP on worker-pool command * Fix test * Test that worker pool ignores consecutive shutdown signals * Added test for worker-pool CLI command. * Added timeout to CI jobs * Fix worker pool test * Comment out test_scheduler.py * Fixed worker-pool in burst mode * Increase test coverage * Exclude tests directory from coverage.py * Improve test coverage * Renamed `Pool(num_workers=2) to `Pool(size=2)` * Revert "Renamed `Pool(num_workers=2) to `Pool(size=2)`" This reverts commit a1306f89ad0d8686c6bde447bff75e2f71f0733b. * Renamed Pool to WorkerPool * Added a new TestCase that doesn't use LocalStack * Added job_class, worker_class and serializer arguments to WorkerPool * Use parse_connection() in WorkerPool.__init__ * Added CLI arguments for worker-pool * Minor WorkerPool and test fixes * Fixed failing CLI test * Document WorkerPool
2 years ago
def setUp(self):
super().setUp()
job = Job.create(func=div_by_zero, args=(1, 2, 3))
job.origin = 'fake'
job.save()
def test_config_file(self):
RQ v1.0! (#1059) * Added FailedJobRegistry. * Added job.failure_ttl. * queue.enqueue() now supports failure_ttl * Added registry.get_queue(). * FailedJobRegistry.add() now assigns DEFAULT_FAILURE_TTL. * StartedJobRegistry.cleanup() now moves expired jobs to FailedJobRegistry. * Failed jobs are now added to FailedJobRegistry. * Added FailedJobRegistry.requeue() * Document the new `FailedJobRegistry` and changes in custom exception handler behavior. * Added worker.disable_default_exception_handler. * Document --disable-default-exception-handler option. * Deleted worker.failed_queue. * Deleted "move_to_failed_queue" exception handler. * StartedJobRegistry should no longer move jobs to FailedQueue. * Deleted requeue_job * Fixed test error. * Make requeue cli command work with FailedJobRegistry * Added .pytest_cache to gitignore. * Custom exception handlers are no longer run in reverse * Restored requeue_job function * Removed get_failed_queue * Deleted FailedQueue * Updated changelog. * Document `failure_ttl` * Updated docs. * Remove job.status * Fixed typo in test_registry.py * Replaced _pipeline() with pipeline() * FailedJobRegistry no longer fails on redis-py>=3 * Fixes test_clean_registries * Worker names are now randomized * Added a note about random worker names in CHANGES.md * Worker will now stop working when encountering an unhandled exception. * Worker should reraise SystemExit on cold shutdowns * Added anchor.js to docs * Support for Sentry-SDK (#1045) * Updated RQ to support sentry-sdk * Document Sentry integration * Install sentry-sdk before running tests * Improved rq info CLI command to be more efficient when displaying lar… (#1046) * Improved rq info CLI command to be more efficient when displaying large number of workers * Fixed an rq info --by-queue bug * Fixed worker.total_working_time bug (#1047) * queue.enqueue() no longer accepts `timeout` argument (#1055) * Clean worker registry (#1056) * queue.enqueue() no longer accepts `timeout` argument * Added clean_worker_registry() * Show worker hostname and PID on cli (#1058) * Show worker hostname and PID on cli * Improve test coverage * Remove Redis version check when SSL is used * Bump version to 1.0 * Removed pytest_cache/README.md * Changed worker logging to use exc_info=True * Removed unused queue.dequeue() * Fixed typo in CHANGES.md * setup_loghandlers() should always call logger.setLevel() if specified
6 years ago
settings = read_config_file('tests.config_files.dummy')
self.assertIn('REDIS_HOST', settings)
self.assertEqual(settings['REDIS_HOST'], 'testhost.example.com')
def test_config_file_option(self):
""""""
RQ v1.0! (#1059) * Added FailedJobRegistry. * Added job.failure_ttl. * queue.enqueue() now supports failure_ttl * Added registry.get_queue(). * FailedJobRegistry.add() now assigns DEFAULT_FAILURE_TTL. * StartedJobRegistry.cleanup() now moves expired jobs to FailedJobRegistry. * Failed jobs are now added to FailedJobRegistry. * Added FailedJobRegistry.requeue() * Document the new `FailedJobRegistry` and changes in custom exception handler behavior. * Added worker.disable_default_exception_handler. * Document --disable-default-exception-handler option. * Deleted worker.failed_queue. * Deleted "move_to_failed_queue" exception handler. * StartedJobRegistry should no longer move jobs to FailedQueue. * Deleted requeue_job * Fixed test error. * Make requeue cli command work with FailedJobRegistry * Added .pytest_cache to gitignore. * Custom exception handlers are no longer run in reverse * Restored requeue_job function * Removed get_failed_queue * Deleted FailedQueue * Updated changelog. * Document `failure_ttl` * Updated docs. * Remove job.status * Fixed typo in test_registry.py * Replaced _pipeline() with pipeline() * FailedJobRegistry no longer fails on redis-py>=3 * Fixes test_clean_registries * Worker names are now randomized * Added a note about random worker names in CHANGES.md * Worker will now stop working when encountering an unhandled exception. * Worker should reraise SystemExit on cold shutdowns * Added anchor.js to docs * Support for Sentry-SDK (#1045) * Updated RQ to support sentry-sdk * Document Sentry integration * Install sentry-sdk before running tests * Improved rq info CLI command to be more efficient when displaying lar… (#1046) * Improved rq info CLI command to be more efficient when displaying large number of workers * Fixed an rq info --by-queue bug * Fixed worker.total_working_time bug (#1047) * queue.enqueue() no longer accepts `timeout` argument (#1055) * Clean worker registry (#1056) * queue.enqueue() no longer accepts `timeout` argument * Added clean_worker_registry() * Show worker hostname and PID on cli (#1058) * Show worker hostname and PID on cli * Improve test coverage * Remove Redis version check when SSL is used * Bump version to 1.0 * Removed pytest_cache/README.md * Changed worker logging to use exc_info=True * Removed unused queue.dequeue() * Fixed typo in CHANGES.md * setup_loghandlers() should always call logger.setLevel() if specified
6 years ago
cli_config = CliConfig(config='tests.config_files.dummy')
self.assertEqual(
cli_config.connection.connection_pool.connection_kwargs['host'],
'testhost.example.com',
)
runner = CliRunner()
result = runner.invoke(main, ['info', '--config', cli_config.config])
self.assertEqual(result.exit_code, 1)
def test_config_file_default_options(self):
""""""
RQ v1.0! (#1059) * Added FailedJobRegistry. * Added job.failure_ttl. * queue.enqueue() now supports failure_ttl * Added registry.get_queue(). * FailedJobRegistry.add() now assigns DEFAULT_FAILURE_TTL. * StartedJobRegistry.cleanup() now moves expired jobs to FailedJobRegistry. * Failed jobs are now added to FailedJobRegistry. * Added FailedJobRegistry.requeue() * Document the new `FailedJobRegistry` and changes in custom exception handler behavior. * Added worker.disable_default_exception_handler. * Document --disable-default-exception-handler option. * Deleted worker.failed_queue. * Deleted "move_to_failed_queue" exception handler. * StartedJobRegistry should no longer move jobs to FailedQueue. * Deleted requeue_job * Fixed test error. * Make requeue cli command work with FailedJobRegistry * Added .pytest_cache to gitignore. * Custom exception handlers are no longer run in reverse * Restored requeue_job function * Removed get_failed_queue * Deleted FailedQueue * Updated changelog. * Document `failure_ttl` * Updated docs. * Remove job.status * Fixed typo in test_registry.py * Replaced _pipeline() with pipeline() * FailedJobRegistry no longer fails on redis-py>=3 * Fixes test_clean_registries * Worker names are now randomized * Added a note about random worker names in CHANGES.md * Worker will now stop working when encountering an unhandled exception. * Worker should reraise SystemExit on cold shutdowns * Added anchor.js to docs * Support for Sentry-SDK (#1045) * Updated RQ to support sentry-sdk * Document Sentry integration * Install sentry-sdk before running tests * Improved rq info CLI command to be more efficient when displaying lar… (#1046) * Improved rq info CLI command to be more efficient when displaying large number of workers * Fixed an rq info --by-queue bug * Fixed worker.total_working_time bug (#1047) * queue.enqueue() no longer accepts `timeout` argument (#1055) * Clean worker registry (#1056) * queue.enqueue() no longer accepts `timeout` argument * Added clean_worker_registry() * Show worker hostname and PID on cli (#1058) * Show worker hostname and PID on cli * Improve test coverage * Remove Redis version check when SSL is used * Bump version to 1.0 * Removed pytest_cache/README.md * Changed worker logging to use exc_info=True * Removed unused queue.dequeue() * Fixed typo in CHANGES.md * setup_loghandlers() should always call logger.setLevel() if specified
6 years ago
cli_config = CliConfig(config='tests.config_files.dummy')
self.assertEqual(
cli_config.connection.connection_pool.connection_kwargs['host'],
'testhost.example.com',
)
Worker pool (#1874) * First stab at implementating worker pool * Use process.is_alive() to check whether a process is still live * Handle shutdown signal * Check worker loop done * First working version of `WorkerPool`. * Added test for check_workers() * Added test for pool.start() * Better shutdown process * Comment out test_start() to see if it fixes CI * Make tests pass * Make CI pass * Comment out some tests * Comment out more tests * Re-enable a test * Re-enable another test * Uncomment check_workers test * Added run_worker test * Minor modification to dead worker detection * More test cases * Better process name for workers * Added back pool.stop_workers() when signal is received * Cleaned up cli.py * WIP on worker-pool command * Fix test * Test that worker pool ignores consecutive shutdown signals * Added test for worker-pool CLI command. * Added timeout to CI jobs * Fix worker pool test * Comment out test_scheduler.py * Fixed worker-pool in burst mode * Increase test coverage * Exclude tests directory from coverage.py * Improve test coverage * Renamed `Pool(num_workers=2) to `Pool(size=2)` * Revert "Renamed `Pool(num_workers=2) to `Pool(size=2)`" This reverts commit a1306f89ad0d8686c6bde447bff75e2f71f0733b. * Renamed Pool to WorkerPool * Added a new TestCase that doesn't use LocalStack * Added job_class, worker_class and serializer arguments to WorkerPool * Use parse_connection() in WorkerPool.__init__ * Added CLI arguments for worker-pool * Minor WorkerPool and test fixes * Fixed failing CLI test * Document WorkerPool
2 years ago
self.assertEqual(cli_config.connection.connection_pool.connection_kwargs['port'], 6379)
self.assertEqual(cli_config.connection.connection_pool.connection_kwargs['db'], 0)
self.assertEqual(cli_config.connection.connection_pool.connection_kwargs['password'], None)
def test_config_file_default_options_override(self):
""""""
RQ v1.0! (#1059) * Added FailedJobRegistry. * Added job.failure_ttl. * queue.enqueue() now supports failure_ttl * Added registry.get_queue(). * FailedJobRegistry.add() now assigns DEFAULT_FAILURE_TTL. * StartedJobRegistry.cleanup() now moves expired jobs to FailedJobRegistry. * Failed jobs are now added to FailedJobRegistry. * Added FailedJobRegistry.requeue() * Document the new `FailedJobRegistry` and changes in custom exception handler behavior. * Added worker.disable_default_exception_handler. * Document --disable-default-exception-handler option. * Deleted worker.failed_queue. * Deleted "move_to_failed_queue" exception handler. * StartedJobRegistry should no longer move jobs to FailedQueue. * Deleted requeue_job * Fixed test error. * Make requeue cli command work with FailedJobRegistry * Added .pytest_cache to gitignore. * Custom exception handlers are no longer run in reverse * Restored requeue_job function * Removed get_failed_queue * Deleted FailedQueue * Updated changelog. * Document `failure_ttl` * Updated docs. * Remove job.status * Fixed typo in test_registry.py * Replaced _pipeline() with pipeline() * FailedJobRegistry no longer fails on redis-py>=3 * Fixes test_clean_registries * Worker names are now randomized * Added a note about random worker names in CHANGES.md * Worker will now stop working when encountering an unhandled exception. * Worker should reraise SystemExit on cold shutdowns * Added anchor.js to docs * Support for Sentry-SDK (#1045) * Updated RQ to support sentry-sdk * Document Sentry integration * Install sentry-sdk before running tests * Improved rq info CLI command to be more efficient when displaying lar… (#1046) * Improved rq info CLI command to be more efficient when displaying large number of workers * Fixed an rq info --by-queue bug * Fixed worker.total_working_time bug (#1047) * queue.enqueue() no longer accepts `timeout` argument (#1055) * Clean worker registry (#1056) * queue.enqueue() no longer accepts `timeout` argument * Added clean_worker_registry() * Show worker hostname and PID on cli (#1058) * Show worker hostname and PID on cli * Improve test coverage * Remove Redis version check when SSL is used * Bump version to 1.0 * Removed pytest_cache/README.md * Changed worker logging to use exc_info=True * Removed unused queue.dequeue() * Fixed typo in CHANGES.md * setup_loghandlers() should always call logger.setLevel() if specified
6 years ago
cli_config = CliConfig(config='tests.config_files.dummy_override')
self.assertEqual(
cli_config.connection.connection_pool.connection_kwargs['host'],
'testhost.example.com',
)
Worker pool (#1874) * First stab at implementating worker pool * Use process.is_alive() to check whether a process is still live * Handle shutdown signal * Check worker loop done * First working version of `WorkerPool`. * Added test for check_workers() * Added test for pool.start() * Better shutdown process * Comment out test_start() to see if it fixes CI * Make tests pass * Make CI pass * Comment out some tests * Comment out more tests * Re-enable a test * Re-enable another test * Uncomment check_workers test * Added run_worker test * Minor modification to dead worker detection * More test cases * Better process name for workers * Added back pool.stop_workers() when signal is received * Cleaned up cli.py * WIP on worker-pool command * Fix test * Test that worker pool ignores consecutive shutdown signals * Added test for worker-pool CLI command. * Added timeout to CI jobs * Fix worker pool test * Comment out test_scheduler.py * Fixed worker-pool in burst mode * Increase test coverage * Exclude tests directory from coverage.py * Improve test coverage * Renamed `Pool(num_workers=2) to `Pool(size=2)` * Revert "Renamed `Pool(num_workers=2) to `Pool(size=2)`" This reverts commit a1306f89ad0d8686c6bde447bff75e2f71f0733b. * Renamed Pool to WorkerPool * Added a new TestCase that doesn't use LocalStack * Added job_class, worker_class and serializer arguments to WorkerPool * Use parse_connection() in WorkerPool.__init__ * Added CLI arguments for worker-pool * Minor WorkerPool and test fixes * Fixed failing CLI test * Document WorkerPool
2 years ago
self.assertEqual(cli_config.connection.connection_pool.connection_kwargs['port'], 6378)
self.assertEqual(cli_config.connection.connection_pool.connection_kwargs['db'], 2)
self.assertEqual(cli_config.connection.connection_pool.connection_kwargs['password'], '123')
def test_config_env_vars(self):
os.environ['REDIS_HOST'] = "testhost.example.com"
cli_config = CliConfig()
self.assertEqual(
cli_config.connection.connection_pool.connection_kwargs['host'],
'testhost.example.com',
)
def test_death_penalty_class(self):
cli_config = CliConfig()
Worker pool (#1874) * First stab at implementating worker pool * Use process.is_alive() to check whether a process is still live * Handle shutdown signal * Check worker loop done * First working version of `WorkerPool`. * Added test for check_workers() * Added test for pool.start() * Better shutdown process * Comment out test_start() to see if it fixes CI * Make tests pass * Make CI pass * Comment out some tests * Comment out more tests * Re-enable a test * Re-enable another test * Uncomment check_workers test * Added run_worker test * Minor modification to dead worker detection * More test cases * Better process name for workers * Added back pool.stop_workers() when signal is received * Cleaned up cli.py * WIP on worker-pool command * Fix test * Test that worker pool ignores consecutive shutdown signals * Added test for worker-pool CLI command. * Added timeout to CI jobs * Fix worker pool test * Comment out test_scheduler.py * Fixed worker-pool in burst mode * Increase test coverage * Exclude tests directory from coverage.py * Improve test coverage * Renamed `Pool(num_workers=2) to `Pool(size=2)` * Revert "Renamed `Pool(num_workers=2) to `Pool(size=2)`" This reverts commit a1306f89ad0d8686c6bde447bff75e2f71f0733b. * Renamed Pool to WorkerPool * Added a new TestCase that doesn't use LocalStack * Added job_class, worker_class and serializer arguments to WorkerPool * Use parse_connection() in WorkerPool.__init__ * Added CLI arguments for worker-pool * Minor WorkerPool and test fixes * Fixed failing CLI test * Document WorkerPool
2 years ago
self.assertEqual(UnixSignalDeathPenalty, cli_config.death_penalty_class)
cli_config = CliConfig(death_penalty_class='rq.job.Job')
Worker pool (#1874) * First stab at implementating worker pool * Use process.is_alive() to check whether a process is still live * Handle shutdown signal * Check worker loop done * First working version of `WorkerPool`. * Added test for check_workers() * Added test for pool.start() * Better shutdown process * Comment out test_start() to see if it fixes CI * Make tests pass * Make CI pass * Comment out some tests * Comment out more tests * Re-enable a test * Re-enable another test * Uncomment check_workers test * Added run_worker test * Minor modification to dead worker detection * More test cases * Better process name for workers * Added back pool.stop_workers() when signal is received * Cleaned up cli.py * WIP on worker-pool command * Fix test * Test that worker pool ignores consecutive shutdown signals * Added test for worker-pool CLI command. * Added timeout to CI jobs * Fix worker pool test * Comment out test_scheduler.py * Fixed worker-pool in burst mode * Increase test coverage * Exclude tests directory from coverage.py * Improve test coverage * Renamed `Pool(num_workers=2) to `Pool(size=2)` * Revert "Renamed `Pool(num_workers=2) to `Pool(size=2)`" This reverts commit a1306f89ad0d8686c6bde447bff75e2f71f0733b. * Renamed Pool to WorkerPool * Added a new TestCase that doesn't use LocalStack * Added job_class, worker_class and serializer arguments to WorkerPool * Use parse_connection() in WorkerPool.__init__ * Added CLI arguments for worker-pool * Minor WorkerPool and test fixes * Fixed failing CLI test * Document WorkerPool
2 years ago
self.assertEqual(Job, cli_config.death_penalty_class)
Worker pool (#1874) * First stab at implementating worker pool * Use process.is_alive() to check whether a process is still live * Handle shutdown signal * Check worker loop done * First working version of `WorkerPool`. * Added test for check_workers() * Added test for pool.start() * Better shutdown process * Comment out test_start() to see if it fixes CI * Make tests pass * Make CI pass * Comment out some tests * Comment out more tests * Re-enable a test * Re-enable another test * Uncomment check_workers test * Added run_worker test * Minor modification to dead worker detection * More test cases * Better process name for workers * Added back pool.stop_workers() when signal is received * Cleaned up cli.py * WIP on worker-pool command * Fix test * Test that worker pool ignores consecutive shutdown signals * Added test for worker-pool CLI command. * Added timeout to CI jobs * Fix worker pool test * Comment out test_scheduler.py * Fixed worker-pool in burst mode * Increase test coverage * Exclude tests directory from coverage.py * Improve test coverage * Renamed `Pool(num_workers=2) to `Pool(size=2)` * Revert "Renamed `Pool(num_workers=2) to `Pool(size=2)`" This reverts commit a1306f89ad0d8686c6bde447bff75e2f71f0733b. * Renamed Pool to WorkerPool * Added a new TestCase that doesn't use LocalStack * Added job_class, worker_class and serializer arguments to WorkerPool * Use parse_connection() in WorkerPool.__init__ * Added CLI arguments for worker-pool * Minor WorkerPool and test fixes * Fixed failing CLI test * Document WorkerPool
2 years ago
with self.assertRaises(ValueError):
CliConfig(death_penalty_class='rq.abcd')
def test_empty_nothing(self):
"""rq empty -u <url>"""
runner = CliRunner()
result = runner.invoke(main, ['empty', '-u', self.redis_url])
self.assert_normal_execution(result)
self.assertEqual(result.output.strip(), 'Nothing to do')
def test_requeue(self):
"""rq requeue -u <url> --all"""
RQ v1.0! (#1059) * Added FailedJobRegistry. * Added job.failure_ttl. * queue.enqueue() now supports failure_ttl * Added registry.get_queue(). * FailedJobRegistry.add() now assigns DEFAULT_FAILURE_TTL. * StartedJobRegistry.cleanup() now moves expired jobs to FailedJobRegistry. * Failed jobs are now added to FailedJobRegistry. * Added FailedJobRegistry.requeue() * Document the new `FailedJobRegistry` and changes in custom exception handler behavior. * Added worker.disable_default_exception_handler. * Document --disable-default-exception-handler option. * Deleted worker.failed_queue. * Deleted "move_to_failed_queue" exception handler. * StartedJobRegistry should no longer move jobs to FailedQueue. * Deleted requeue_job * Fixed test error. * Make requeue cli command work with FailedJobRegistry * Added .pytest_cache to gitignore. * Custom exception handlers are no longer run in reverse * Restored requeue_job function * Removed get_failed_queue * Deleted FailedQueue * Updated changelog. * Document `failure_ttl` * Updated docs. * Remove job.status * Fixed typo in test_registry.py * Replaced _pipeline() with pipeline() * FailedJobRegistry no longer fails on redis-py>=3 * Fixes test_clean_registries * Worker names are now randomized * Added a note about random worker names in CHANGES.md * Worker will now stop working when encountering an unhandled exception. * Worker should reraise SystemExit on cold shutdowns * Added anchor.js to docs * Support for Sentry-SDK (#1045) * Updated RQ to support sentry-sdk * Document Sentry integration * Install sentry-sdk before running tests * Improved rq info CLI command to be more efficient when displaying lar… (#1046) * Improved rq info CLI command to be more efficient when displaying large number of workers * Fixed an rq info --by-queue bug * Fixed worker.total_working_time bug (#1047) * queue.enqueue() no longer accepts `timeout` argument (#1055) * Clean worker registry (#1056) * queue.enqueue() no longer accepts `timeout` argument * Added clean_worker_registry() * Show worker hostname and PID on cli (#1058) * Show worker hostname and PID on cli * Improve test coverage * Remove Redis version check when SSL is used * Bump version to 1.0 * Removed pytest_cache/README.md * Changed worker logging to use exc_info=True * Removed unused queue.dequeue() * Fixed typo in CHANGES.md * setup_loghandlers() should always call logger.setLevel() if specified
6 years ago
connection = Redis.from_url(self.redis_url)
queue = Queue('requeue', connection=connection)
registry = queue.failed_job_registry
runner = CliRunner()
RQ v1.0! (#1059) * Added FailedJobRegistry. * Added job.failure_ttl. * queue.enqueue() now supports failure_ttl * Added registry.get_queue(). * FailedJobRegistry.add() now assigns DEFAULT_FAILURE_TTL. * StartedJobRegistry.cleanup() now moves expired jobs to FailedJobRegistry. * Failed jobs are now added to FailedJobRegistry. * Added FailedJobRegistry.requeue() * Document the new `FailedJobRegistry` and changes in custom exception handler behavior. * Added worker.disable_default_exception_handler. * Document --disable-default-exception-handler option. * Deleted worker.failed_queue. * Deleted "move_to_failed_queue" exception handler. * StartedJobRegistry should no longer move jobs to FailedQueue. * Deleted requeue_job * Fixed test error. * Make requeue cli command work with FailedJobRegistry * Added .pytest_cache to gitignore. * Custom exception handlers are no longer run in reverse * Restored requeue_job function * Removed get_failed_queue * Deleted FailedQueue * Updated changelog. * Document `failure_ttl` * Updated docs. * Remove job.status * Fixed typo in test_registry.py * Replaced _pipeline() with pipeline() * FailedJobRegistry no longer fails on redis-py>=3 * Fixes test_clean_registries * Worker names are now randomized * Added a note about random worker names in CHANGES.md * Worker will now stop working when encountering an unhandled exception. * Worker should reraise SystemExit on cold shutdowns * Added anchor.js to docs * Support for Sentry-SDK (#1045) * Updated RQ to support sentry-sdk * Document Sentry integration * Install sentry-sdk before running tests * Improved rq info CLI command to be more efficient when displaying lar… (#1046) * Improved rq info CLI command to be more efficient when displaying large number of workers * Fixed an rq info --by-queue bug * Fixed worker.total_working_time bug (#1047) * queue.enqueue() no longer accepts `timeout` argument (#1055) * Clean worker registry (#1056) * queue.enqueue() no longer accepts `timeout` argument * Added clean_worker_registry() * Show worker hostname and PID on cli (#1058) * Show worker hostname and PID on cli * Improve test coverage * Remove Redis version check when SSL is used * Bump version to 1.0 * Removed pytest_cache/README.md * Changed worker logging to use exc_info=True * Removed unused queue.dequeue() * Fixed typo in CHANGES.md * setup_loghandlers() should always call logger.setLevel() if specified
6 years ago
job = queue.enqueue(div_by_zero)
job2 = queue.enqueue(div_by_zero)
job3 = queue.enqueue(div_by_zero)
worker = Worker([queue])
worker.work(burst=True)
self.assertIn(job, registry)
self.assertIn(job2, registry)
self.assertIn(job3, registry)
Worker pool (#1874) * First stab at implementating worker pool * Use process.is_alive() to check whether a process is still live * Handle shutdown signal * Check worker loop done * First working version of `WorkerPool`. * Added test for check_workers() * Added test for pool.start() * Better shutdown process * Comment out test_start() to see if it fixes CI * Make tests pass * Make CI pass * Comment out some tests * Comment out more tests * Re-enable a test * Re-enable another test * Uncomment check_workers test * Added run_worker test * Minor modification to dead worker detection * More test cases * Better process name for workers * Added back pool.stop_workers() when signal is received * Cleaned up cli.py * WIP on worker-pool command * Fix test * Test that worker pool ignores consecutive shutdown signals * Added test for worker-pool CLI command. * Added timeout to CI jobs * Fix worker pool test * Comment out test_scheduler.py * Fixed worker-pool in burst mode * Increase test coverage * Exclude tests directory from coverage.py * Improve test coverage * Renamed `Pool(num_workers=2) to `Pool(size=2)` * Revert "Renamed `Pool(num_workers=2) to `Pool(size=2)`" This reverts commit a1306f89ad0d8686c6bde447bff75e2f71f0733b. * Renamed Pool to WorkerPool * Added a new TestCase that doesn't use LocalStack * Added job_class, worker_class and serializer arguments to WorkerPool * Use parse_connection() in WorkerPool.__init__ * Added CLI arguments for worker-pool * Minor WorkerPool and test fixes * Fixed failing CLI test * Document WorkerPool
2 years ago
result = runner.invoke(main, ['requeue', '-u', self.redis_url, '--queue', 'requeue', job.id])
self.assert_normal_execution(result)
RQ v1.0! (#1059) * Added FailedJobRegistry. * Added job.failure_ttl. * queue.enqueue() now supports failure_ttl * Added registry.get_queue(). * FailedJobRegistry.add() now assigns DEFAULT_FAILURE_TTL. * StartedJobRegistry.cleanup() now moves expired jobs to FailedJobRegistry. * Failed jobs are now added to FailedJobRegistry. * Added FailedJobRegistry.requeue() * Document the new `FailedJobRegistry` and changes in custom exception handler behavior. * Added worker.disable_default_exception_handler. * Document --disable-default-exception-handler option. * Deleted worker.failed_queue. * Deleted "move_to_failed_queue" exception handler. * StartedJobRegistry should no longer move jobs to FailedQueue. * Deleted requeue_job * Fixed test error. * Make requeue cli command work with FailedJobRegistry * Added .pytest_cache to gitignore. * Custom exception handlers are no longer run in reverse * Restored requeue_job function * Removed get_failed_queue * Deleted FailedQueue * Updated changelog. * Document `failure_ttl` * Updated docs. * Remove job.status * Fixed typo in test_registry.py * Replaced _pipeline() with pipeline() * FailedJobRegistry no longer fails on redis-py>=3 * Fixes test_clean_registries * Worker names are now randomized * Added a note about random worker names in CHANGES.md * Worker will now stop working when encountering an unhandled exception. * Worker should reraise SystemExit on cold shutdowns * Added anchor.js to docs * Support for Sentry-SDK (#1045) * Updated RQ to support sentry-sdk * Document Sentry integration * Install sentry-sdk before running tests * Improved rq info CLI command to be more efficient when displaying lar… (#1046) * Improved rq info CLI command to be more efficient when displaying large number of workers * Fixed an rq info --by-queue bug * Fixed worker.total_working_time bug (#1047) * queue.enqueue() no longer accepts `timeout` argument (#1055) * Clean worker registry (#1056) * queue.enqueue() no longer accepts `timeout` argument * Added clean_worker_registry() * Show worker hostname and PID on cli (#1058) * Show worker hostname and PID on cli * Improve test coverage * Remove Redis version check when SSL is used * Bump version to 1.0 * Removed pytest_cache/README.md * Changed worker logging to use exc_info=True * Removed unused queue.dequeue() * Fixed typo in CHANGES.md * setup_loghandlers() should always call logger.setLevel() if specified
6 years ago
# Only the first specified job is requeued
self.assertNotIn(job, registry)
self.assertIn(job2, registry)
self.assertIn(job3, registry)
Worker pool (#1874) * First stab at implementating worker pool * Use process.is_alive() to check whether a process is still live * Handle shutdown signal * Check worker loop done * First working version of `WorkerPool`. * Added test for check_workers() * Added test for pool.start() * Better shutdown process * Comment out test_start() to see if it fixes CI * Make tests pass * Make CI pass * Comment out some tests * Comment out more tests * Re-enable a test * Re-enable another test * Uncomment check_workers test * Added run_worker test * Minor modification to dead worker detection * More test cases * Better process name for workers * Added back pool.stop_workers() when signal is received * Cleaned up cli.py * WIP on worker-pool command * Fix test * Test that worker pool ignores consecutive shutdown signals * Added test for worker-pool CLI command. * Added timeout to CI jobs * Fix worker pool test * Comment out test_scheduler.py * Fixed worker-pool in burst mode * Increase test coverage * Exclude tests directory from coverage.py * Improve test coverage * Renamed `Pool(num_workers=2) to `Pool(size=2)` * Revert "Renamed `Pool(num_workers=2) to `Pool(size=2)`" This reverts commit a1306f89ad0d8686c6bde447bff75e2f71f0733b. * Renamed Pool to WorkerPool * Added a new TestCase that doesn't use LocalStack * Added job_class, worker_class and serializer arguments to WorkerPool * Use parse_connection() in WorkerPool.__init__ * Added CLI arguments for worker-pool * Minor WorkerPool and test fixes * Fixed failing CLI test * Document WorkerPool
2 years ago
result = runner.invoke(main, ['requeue', '-u', self.redis_url, '--queue', 'requeue', '--all'])
self.assert_normal_execution(result)
RQ v1.0! (#1059) * Added FailedJobRegistry. * Added job.failure_ttl. * queue.enqueue() now supports failure_ttl * Added registry.get_queue(). * FailedJobRegistry.add() now assigns DEFAULT_FAILURE_TTL. * StartedJobRegistry.cleanup() now moves expired jobs to FailedJobRegistry. * Failed jobs are now added to FailedJobRegistry. * Added FailedJobRegistry.requeue() * Document the new `FailedJobRegistry` and changes in custom exception handler behavior. * Added worker.disable_default_exception_handler. * Document --disable-default-exception-handler option. * Deleted worker.failed_queue. * Deleted "move_to_failed_queue" exception handler. * StartedJobRegistry should no longer move jobs to FailedQueue. * Deleted requeue_job * Fixed test error. * Make requeue cli command work with FailedJobRegistry * Added .pytest_cache to gitignore. * Custom exception handlers are no longer run in reverse * Restored requeue_job function * Removed get_failed_queue * Deleted FailedQueue * Updated changelog. * Document `failure_ttl` * Updated docs. * Remove job.status * Fixed typo in test_registry.py * Replaced _pipeline() with pipeline() * FailedJobRegistry no longer fails on redis-py>=3 * Fixes test_clean_registries * Worker names are now randomized * Added a note about random worker names in CHANGES.md * Worker will now stop working when encountering an unhandled exception. * Worker should reraise SystemExit on cold shutdowns * Added anchor.js to docs * Support for Sentry-SDK (#1045) * Updated RQ to support sentry-sdk * Document Sentry integration * Install sentry-sdk before running tests * Improved rq info CLI command to be more efficient when displaying lar… (#1046) * Improved rq info CLI command to be more efficient when displaying large number of workers * Fixed an rq info --by-queue bug * Fixed worker.total_working_time bug (#1047) * queue.enqueue() no longer accepts `timeout` argument (#1055) * Clean worker registry (#1056) * queue.enqueue() no longer accepts `timeout` argument * Added clean_worker_registry() * Show worker hostname and PID on cli (#1058) * Show worker hostname and PID on cli * Improve test coverage * Remove Redis version check when SSL is used * Bump version to 1.0 * Removed pytest_cache/README.md * Changed worker logging to use exc_info=True * Removed unused queue.dequeue() * Fixed typo in CHANGES.md * setup_loghandlers() should always call logger.setLevel() if specified
6 years ago
# With --all flag, all failed jobs are requeued
self.assertNotIn(job2, registry)
self.assertNotIn(job3, registry)
def test_requeue_with_serializer(self):
"""rq requeue -u <url> -S <serializer> --all"""
connection = Redis.from_url(self.redis_url)
queue = Queue('requeue', connection=connection, serializer=JSONSerializer)
registry = queue.failed_job_registry
runner = CliRunner()
job = queue.enqueue(div_by_zero)
job2 = queue.enqueue(div_by_zero)
job3 = queue.enqueue(div_by_zero)
worker = Worker([queue], serializer=JSONSerializer)
worker.work(burst=True)
self.assertIn(job, registry)
self.assertIn(job2, registry)
self.assertIn(job3, registry)
result = runner.invoke(
Worker pool (#1874) * First stab at implementating worker pool * Use process.is_alive() to check whether a process is still live * Handle shutdown signal * Check worker loop done * First working version of `WorkerPool`. * Added test for check_workers() * Added test for pool.start() * Better shutdown process * Comment out test_start() to see if it fixes CI * Make tests pass * Make CI pass * Comment out some tests * Comment out more tests * Re-enable a test * Re-enable another test * Uncomment check_workers test * Added run_worker test * Minor modification to dead worker detection * More test cases * Better process name for workers * Added back pool.stop_workers() when signal is received * Cleaned up cli.py * WIP on worker-pool command * Fix test * Test that worker pool ignores consecutive shutdown signals * Added test for worker-pool CLI command. * Added timeout to CI jobs * Fix worker pool test * Comment out test_scheduler.py * Fixed worker-pool in burst mode * Increase test coverage * Exclude tests directory from coverage.py * Improve test coverage * Renamed `Pool(num_workers=2) to `Pool(size=2)` * Revert "Renamed `Pool(num_workers=2) to `Pool(size=2)`" This reverts commit a1306f89ad0d8686c6bde447bff75e2f71f0733b. * Renamed Pool to WorkerPool * Added a new TestCase that doesn't use LocalStack * Added job_class, worker_class and serializer arguments to WorkerPool * Use parse_connection() in WorkerPool.__init__ * Added CLI arguments for worker-pool * Minor WorkerPool and test fixes * Fixed failing CLI test * Document WorkerPool
2 years ago
main, ['requeue', '-u', self.redis_url, '--queue', 'requeue', '-S', 'rq.serializers.JSONSerializer', job.id]
)
self.assert_normal_execution(result)
# Only the first specified job is requeued
self.assertNotIn(job, registry)
self.assertIn(job2, registry)
self.assertIn(job3, registry)
result = runner.invoke(
main,
Worker pool (#1874) * First stab at implementating worker pool * Use process.is_alive() to check whether a process is still live * Handle shutdown signal * Check worker loop done * First working version of `WorkerPool`. * Added test for check_workers() * Added test for pool.start() * Better shutdown process * Comment out test_start() to see if it fixes CI * Make tests pass * Make CI pass * Comment out some tests * Comment out more tests * Re-enable a test * Re-enable another test * Uncomment check_workers test * Added run_worker test * Minor modification to dead worker detection * More test cases * Better process name for workers * Added back pool.stop_workers() when signal is received * Cleaned up cli.py * WIP on worker-pool command * Fix test * Test that worker pool ignores consecutive shutdown signals * Added test for worker-pool CLI command. * Added timeout to CI jobs * Fix worker pool test * Comment out test_scheduler.py * Fixed worker-pool in burst mode * Increase test coverage * Exclude tests directory from coverage.py * Improve test coverage * Renamed `Pool(num_workers=2) to `Pool(size=2)` * Revert "Renamed `Pool(num_workers=2) to `Pool(size=2)`" This reverts commit a1306f89ad0d8686c6bde447bff75e2f71f0733b. * Renamed Pool to WorkerPool * Added a new TestCase that doesn't use LocalStack * Added job_class, worker_class and serializer arguments to WorkerPool * Use parse_connection() in WorkerPool.__init__ * Added CLI arguments for worker-pool * Minor WorkerPool and test fixes * Fixed failing CLI test * Document WorkerPool
2 years ago
['requeue', '-u', self.redis_url, '--queue', 'requeue', '-S', 'rq.serializers.JSONSerializer', '--all'],
)
self.assert_normal_execution(result)
# With --all flag, all failed jobs are requeued
self.assertNotIn(job2, registry)
self.assertNotIn(job3, registry)
def test_info(self):
"""rq info -u <url>"""
runner = CliRunner()
RQ v1.0! (#1059) * Added FailedJobRegistry. * Added job.failure_ttl. * queue.enqueue() now supports failure_ttl * Added registry.get_queue(). * FailedJobRegistry.add() now assigns DEFAULT_FAILURE_TTL. * StartedJobRegistry.cleanup() now moves expired jobs to FailedJobRegistry. * Failed jobs are now added to FailedJobRegistry. * Added FailedJobRegistry.requeue() * Document the new `FailedJobRegistry` and changes in custom exception handler behavior. * Added worker.disable_default_exception_handler. * Document --disable-default-exception-handler option. * Deleted worker.failed_queue. * Deleted "move_to_failed_queue" exception handler. * StartedJobRegistry should no longer move jobs to FailedQueue. * Deleted requeue_job * Fixed test error. * Make requeue cli command work with FailedJobRegistry * Added .pytest_cache to gitignore. * Custom exception handlers are no longer run in reverse * Restored requeue_job function * Removed get_failed_queue * Deleted FailedQueue * Updated changelog. * Document `failure_ttl` * Updated docs. * Remove job.status * Fixed typo in test_registry.py * Replaced _pipeline() with pipeline() * FailedJobRegistry no longer fails on redis-py>=3 * Fixes test_clean_registries * Worker names are now randomized * Added a note about random worker names in CHANGES.md * Worker will now stop working when encountering an unhandled exception. * Worker should reraise SystemExit on cold shutdowns * Added anchor.js to docs * Support for Sentry-SDK (#1045) * Updated RQ to support sentry-sdk * Document Sentry integration * Install sentry-sdk before running tests * Improved rq info CLI command to be more efficient when displaying lar… (#1046) * Improved rq info CLI command to be more efficient when displaying large number of workers * Fixed an rq info --by-queue bug * Fixed worker.total_working_time bug (#1047) * queue.enqueue() no longer accepts `timeout` argument (#1055) * Clean worker registry (#1056) * queue.enqueue() no longer accepts `timeout` argument * Added clean_worker_registry() * Show worker hostname and PID on cli (#1058) * Show worker hostname and PID on cli * Improve test coverage * Remove Redis version check when SSL is used * Bump version to 1.0 * Removed pytest_cache/README.md * Changed worker logging to use exc_info=True * Removed unused queue.dequeue() * Fixed typo in CHANGES.md * setup_loghandlers() should always call logger.setLevel() if specified
6 years ago
result = runner.invoke(main, ['info', '-u', self.redis_url])
self.assert_normal_execution(result)
self.assertIn('0 queues, 0 jobs total', result.output)
queue = Queue(connection=self.connection)
queue.enqueue(say_hello)
10 years ago
result = runner.invoke(main, ['info', '-u', self.redis_url])
self.assert_normal_execution(result)
self.assertIn('1 queues, 1 jobs total', result.output)
def test_info_only_queues(self):
"""rq info -u <url> --only-queues (-Q)"""
runner = CliRunner()
result = runner.invoke(main, ['info', '-u', self.redis_url, '--only-queues'])
self.assert_normal_execution(result)
RQ v1.0! (#1059) * Added FailedJobRegistry. * Added job.failure_ttl. * queue.enqueue() now supports failure_ttl * Added registry.get_queue(). * FailedJobRegistry.add() now assigns DEFAULT_FAILURE_TTL. * StartedJobRegistry.cleanup() now moves expired jobs to FailedJobRegistry. * Failed jobs are now added to FailedJobRegistry. * Added FailedJobRegistry.requeue() * Document the new `FailedJobRegistry` and changes in custom exception handler behavior. * Added worker.disable_default_exception_handler. * Document --disable-default-exception-handler option. * Deleted worker.failed_queue. * Deleted "move_to_failed_queue" exception handler. * StartedJobRegistry should no longer move jobs to FailedQueue. * Deleted requeue_job * Fixed test error. * Make requeue cli command work with FailedJobRegistry * Added .pytest_cache to gitignore. * Custom exception handlers are no longer run in reverse * Restored requeue_job function * Removed get_failed_queue * Deleted FailedQueue * Updated changelog. * Document `failure_ttl` * Updated docs. * Remove job.status * Fixed typo in test_registry.py * Replaced _pipeline() with pipeline() * FailedJobRegistry no longer fails on redis-py>=3 * Fixes test_clean_registries * Worker names are now randomized * Added a note about random worker names in CHANGES.md * Worker will now stop working when encountering an unhandled exception. * Worker should reraise SystemExit on cold shutdowns * Added anchor.js to docs * Support for Sentry-SDK (#1045) * Updated RQ to support sentry-sdk * Document Sentry integration * Install sentry-sdk before running tests * Improved rq info CLI command to be more efficient when displaying lar… (#1046) * Improved rq info CLI command to be more efficient when displaying large number of workers * Fixed an rq info --by-queue bug * Fixed worker.total_working_time bug (#1047) * queue.enqueue() no longer accepts `timeout` argument (#1055) * Clean worker registry (#1056) * queue.enqueue() no longer accepts `timeout` argument * Added clean_worker_registry() * Show worker hostname and PID on cli (#1058) * Show worker hostname and PID on cli * Improve test coverage * Remove Redis version check when SSL is used * Bump version to 1.0 * Removed pytest_cache/README.md * Changed worker logging to use exc_info=True * Removed unused queue.dequeue() * Fixed typo in CHANGES.md * setup_loghandlers() should always call logger.setLevel() if specified
6 years ago
self.assertIn('0 queues, 0 jobs total', result.output)
queue = Queue(connection=self.connection)
queue.enqueue(say_hello)
result = runner.invoke(main, ['info', '-u', self.redis_url])
self.assert_normal_execution(result)
self.assertIn('1 queues, 1 jobs total', result.output)
def test_info_only_workers(self):
"""rq info -u <url> --only-workers (-W)"""
runner = CliRunner()
result = runner.invoke(main, ['info', '-u', self.redis_url, '--only-workers'])
self.assert_normal_execution(result)
RQ v1.0! (#1059) * Added FailedJobRegistry. * Added job.failure_ttl. * queue.enqueue() now supports failure_ttl * Added registry.get_queue(). * FailedJobRegistry.add() now assigns DEFAULT_FAILURE_TTL. * StartedJobRegistry.cleanup() now moves expired jobs to FailedJobRegistry. * Failed jobs are now added to FailedJobRegistry. * Added FailedJobRegistry.requeue() * Document the new `FailedJobRegistry` and changes in custom exception handler behavior. * Added worker.disable_default_exception_handler. * Document --disable-default-exception-handler option. * Deleted worker.failed_queue. * Deleted "move_to_failed_queue" exception handler. * StartedJobRegistry should no longer move jobs to FailedQueue. * Deleted requeue_job * Fixed test error. * Make requeue cli command work with FailedJobRegistry * Added .pytest_cache to gitignore. * Custom exception handlers are no longer run in reverse * Restored requeue_job function * Removed get_failed_queue * Deleted FailedQueue * Updated changelog. * Document `failure_ttl` * Updated docs. * Remove job.status * Fixed typo in test_registry.py * Replaced _pipeline() with pipeline() * FailedJobRegistry no longer fails on redis-py>=3 * Fixes test_clean_registries * Worker names are now randomized * Added a note about random worker names in CHANGES.md * Worker will now stop working when encountering an unhandled exception. * Worker should reraise SystemExit on cold shutdowns * Added anchor.js to docs * Support for Sentry-SDK (#1045) * Updated RQ to support sentry-sdk * Document Sentry integration * Install sentry-sdk before running tests * Improved rq info CLI command to be more efficient when displaying lar… (#1046) * Improved rq info CLI command to be more efficient when displaying large number of workers * Fixed an rq info --by-queue bug * Fixed worker.total_working_time bug (#1047) * queue.enqueue() no longer accepts `timeout` argument (#1055) * Clean worker registry (#1056) * queue.enqueue() no longer accepts `timeout` argument * Added clean_worker_registry() * Show worker hostname and PID on cli (#1058) * Show worker hostname and PID on cli * Improve test coverage * Remove Redis version check when SSL is used * Bump version to 1.0 * Removed pytest_cache/README.md * Changed worker logging to use exc_info=True * Removed unused queue.dequeue() * Fixed typo in CHANGES.md * setup_loghandlers() should always call logger.setLevel() if specified
6 years ago
self.assertIn('0 workers, 0 queue', result.output)
Worker pool (#1874) * First stab at implementating worker pool * Use process.is_alive() to check whether a process is still live * Handle shutdown signal * Check worker loop done * First working version of `WorkerPool`. * Added test for check_workers() * Added test for pool.start() * Better shutdown process * Comment out test_start() to see if it fixes CI * Make tests pass * Make CI pass * Comment out some tests * Comment out more tests * Re-enable a test * Re-enable another test * Uncomment check_workers test * Added run_worker test * Minor modification to dead worker detection * More test cases * Better process name for workers * Added back pool.stop_workers() when signal is received * Cleaned up cli.py * WIP on worker-pool command * Fix test * Test that worker pool ignores consecutive shutdown signals * Added test for worker-pool CLI command. * Added timeout to CI jobs * Fix worker pool test * Comment out test_scheduler.py * Fixed worker-pool in burst mode * Increase test coverage * Exclude tests directory from coverage.py * Improve test coverage * Renamed `Pool(num_workers=2) to `Pool(size=2)` * Revert "Renamed `Pool(num_workers=2) to `Pool(size=2)`" This reverts commit a1306f89ad0d8686c6bde447bff75e2f71f0733b. * Renamed Pool to WorkerPool * Added a new TestCase that doesn't use LocalStack * Added job_class, worker_class and serializer arguments to WorkerPool * Use parse_connection() in WorkerPool.__init__ * Added CLI arguments for worker-pool * Minor WorkerPool and test fixes * Fixed failing CLI test * Document WorkerPool
2 years ago
result = runner.invoke(main, ['info', '--by-queue', '-u', self.redis_url, '--only-workers'])
self.assert_normal_execution(result)
self.assertIn('0 workers, 0 queue', result.output)
worker = Worker(['default'], connection=self.connection)
worker.register_birth()
result = runner.invoke(main, ['info', '-u', self.redis_url, '--only-workers'])
self.assert_normal_execution(result)
self.assertIn('1 workers, 0 queues', result.output)
worker.register_death()
RQ v1.0! (#1059) * Added FailedJobRegistry. * Added job.failure_ttl. * queue.enqueue() now supports failure_ttl * Added registry.get_queue(). * FailedJobRegistry.add() now assigns DEFAULT_FAILURE_TTL. * StartedJobRegistry.cleanup() now moves expired jobs to FailedJobRegistry. * Failed jobs are now added to FailedJobRegistry. * Added FailedJobRegistry.requeue() * Document the new `FailedJobRegistry` and changes in custom exception handler behavior. * Added worker.disable_default_exception_handler. * Document --disable-default-exception-handler option. * Deleted worker.failed_queue. * Deleted "move_to_failed_queue" exception handler. * StartedJobRegistry should no longer move jobs to FailedQueue. * Deleted requeue_job * Fixed test error. * Make requeue cli command work with FailedJobRegistry * Added .pytest_cache to gitignore. * Custom exception handlers are no longer run in reverse * Restored requeue_job function * Removed get_failed_queue * Deleted FailedQueue * Updated changelog. * Document `failure_ttl` * Updated docs. * Remove job.status * Fixed typo in test_registry.py * Replaced _pipeline() with pipeline() * FailedJobRegistry no longer fails on redis-py>=3 * Fixes test_clean_registries * Worker names are now randomized * Added a note about random worker names in CHANGES.md * Worker will now stop working when encountering an unhandled exception. * Worker should reraise SystemExit on cold shutdowns * Added anchor.js to docs * Support for Sentry-SDK (#1045) * Updated RQ to support sentry-sdk * Document Sentry integration * Install sentry-sdk before running tests * Improved rq info CLI command to be more efficient when displaying lar… (#1046) * Improved rq info CLI command to be more efficient when displaying large number of workers * Fixed an rq info --by-queue bug * Fixed worker.total_working_time bug (#1047) * queue.enqueue() no longer accepts `timeout` argument (#1055) * Clean worker registry (#1056) * queue.enqueue() no longer accepts `timeout` argument * Added clean_worker_registry() * Show worker hostname and PID on cli (#1058) * Show worker hostname and PID on cli * Improve test coverage * Remove Redis version check when SSL is used * Bump version to 1.0 * Removed pytest_cache/README.md * Changed worker logging to use exc_info=True * Removed unused queue.dequeue() * Fixed typo in CHANGES.md * setup_loghandlers() should always call logger.setLevel() if specified
6 years ago
queue = Queue(connection=self.connection)
queue.enqueue(say_hello)
result = runner.invoke(main, ['info', '-u', self.redis_url, '--only-workers'])
self.assert_normal_execution(result)
self.assertIn('0 workers, 1 queues', result.output)
RQ v1.0! (#1059) * Added FailedJobRegistry. * Added job.failure_ttl. * queue.enqueue() now supports failure_ttl * Added registry.get_queue(). * FailedJobRegistry.add() now assigns DEFAULT_FAILURE_TTL. * StartedJobRegistry.cleanup() now moves expired jobs to FailedJobRegistry. * Failed jobs are now added to FailedJobRegistry. * Added FailedJobRegistry.requeue() * Document the new `FailedJobRegistry` and changes in custom exception handler behavior. * Added worker.disable_default_exception_handler. * Document --disable-default-exception-handler option. * Deleted worker.failed_queue. * Deleted "move_to_failed_queue" exception handler. * StartedJobRegistry should no longer move jobs to FailedQueue. * Deleted requeue_job * Fixed test error. * Make requeue cli command work with FailedJobRegistry * Added .pytest_cache to gitignore. * Custom exception handlers are no longer run in reverse * Restored requeue_job function * Removed get_failed_queue * Deleted FailedQueue * Updated changelog. * Document `failure_ttl` * Updated docs. * Remove job.status * Fixed typo in test_registry.py * Replaced _pipeline() with pipeline() * FailedJobRegistry no longer fails on redis-py>=3 * Fixes test_clean_registries * Worker names are now randomized * Added a note about random worker names in CHANGES.md * Worker will now stop working when encountering an unhandled exception. * Worker should reraise SystemExit on cold shutdowns * Added anchor.js to docs * Support for Sentry-SDK (#1045) * Updated RQ to support sentry-sdk * Document Sentry integration * Install sentry-sdk before running tests * Improved rq info CLI command to be more efficient when displaying lar… (#1046) * Improved rq info CLI command to be more efficient when displaying large number of workers * Fixed an rq info --by-queue bug * Fixed worker.total_working_time bug (#1047) * queue.enqueue() no longer accepts `timeout` argument (#1055) * Clean worker registry (#1056) * queue.enqueue() no longer accepts `timeout` argument * Added clean_worker_registry() * Show worker hostname and PID on cli (#1058) * Show worker hostname and PID on cli * Improve test coverage * Remove Redis version check when SSL is used * Bump version to 1.0 * Removed pytest_cache/README.md * Changed worker logging to use exc_info=True * Removed unused queue.dequeue() * Fixed typo in CHANGES.md * setup_loghandlers() should always call logger.setLevel() if specified
6 years ago
foo_queue = Queue(name='foo', connection=self.connection)
foo_queue.enqueue(say_hello)
bar_queue = Queue(name='bar', connection=self.connection)
bar_queue.enqueue(say_hello)
worker_1 = Worker([foo_queue, bar_queue], connection=self.connection)
worker_1.register_birth()
RQ v1.0! (#1059) * Added FailedJobRegistry. * Added job.failure_ttl. * queue.enqueue() now supports failure_ttl * Added registry.get_queue(). * FailedJobRegistry.add() now assigns DEFAULT_FAILURE_TTL. * StartedJobRegistry.cleanup() now moves expired jobs to FailedJobRegistry. * Failed jobs are now added to FailedJobRegistry. * Added FailedJobRegistry.requeue() * Document the new `FailedJobRegistry` and changes in custom exception handler behavior. * Added worker.disable_default_exception_handler. * Document --disable-default-exception-handler option. * Deleted worker.failed_queue. * Deleted "move_to_failed_queue" exception handler. * StartedJobRegistry should no longer move jobs to FailedQueue. * Deleted requeue_job * Fixed test error. * Make requeue cli command work with FailedJobRegistry * Added .pytest_cache to gitignore. * Custom exception handlers are no longer run in reverse * Restored requeue_job function * Removed get_failed_queue * Deleted FailedQueue * Updated changelog. * Document `failure_ttl` * Updated docs. * Remove job.status * Fixed typo in test_registry.py * Replaced _pipeline() with pipeline() * FailedJobRegistry no longer fails on redis-py>=3 * Fixes test_clean_registries * Worker names are now randomized * Added a note about random worker names in CHANGES.md * Worker will now stop working when encountering an unhandled exception. * Worker should reraise SystemExit on cold shutdowns * Added anchor.js to docs * Support for Sentry-SDK (#1045) * Updated RQ to support sentry-sdk * Document Sentry integration * Install sentry-sdk before running tests * Improved rq info CLI command to be more efficient when displaying lar… (#1046) * Improved rq info CLI command to be more efficient when displaying large number of workers * Fixed an rq info --by-queue bug * Fixed worker.total_working_time bug (#1047) * queue.enqueue() no longer accepts `timeout` argument (#1055) * Clean worker registry (#1056) * queue.enqueue() no longer accepts `timeout` argument * Added clean_worker_registry() * Show worker hostname and PID on cli (#1058) * Show worker hostname and PID on cli * Improve test coverage * Remove Redis version check when SSL is used * Bump version to 1.0 * Removed pytest_cache/README.md * Changed worker logging to use exc_info=True * Removed unused queue.dequeue() * Fixed typo in CHANGES.md * setup_loghandlers() should always call logger.setLevel() if specified
6 years ago
worker_2 = Worker([foo_queue, bar_queue], connection=self.connection)
worker_2.register_birth()
worker_2.set_state(WorkerStatus.BUSY)
Worker pool (#1874) * First stab at implementating worker pool * Use process.is_alive() to check whether a process is still live * Handle shutdown signal * Check worker loop done * First working version of `WorkerPool`. * Added test for check_workers() * Added test for pool.start() * Better shutdown process * Comment out test_start() to see if it fixes CI * Make tests pass * Make CI pass * Comment out some tests * Comment out more tests * Re-enable a test * Re-enable another test * Uncomment check_workers test * Added run_worker test * Minor modification to dead worker detection * More test cases * Better process name for workers * Added back pool.stop_workers() when signal is received * Cleaned up cli.py * WIP on worker-pool command * Fix test * Test that worker pool ignores consecutive shutdown signals * Added test for worker-pool CLI command. * Added timeout to CI jobs * Fix worker pool test * Comment out test_scheduler.py * Fixed worker-pool in burst mode * Increase test coverage * Exclude tests directory from coverage.py * Improve test coverage * Renamed `Pool(num_workers=2) to `Pool(size=2)` * Revert "Renamed `Pool(num_workers=2) to `Pool(size=2)`" This reverts commit a1306f89ad0d8686c6bde447bff75e2f71f0733b. * Renamed Pool to WorkerPool * Added a new TestCase that doesn't use LocalStack * Added job_class, worker_class and serializer arguments to WorkerPool * Use parse_connection() in WorkerPool.__init__ * Added CLI arguments for worker-pool * Minor WorkerPool and test fixes * Fixed failing CLI test * Document WorkerPool
2 years ago
result = runner.invoke(main, ['info', 'foo', 'bar', '-u', self.redis_url, '--only-workers'])
RQ v1.0! (#1059) * Added FailedJobRegistry. * Added job.failure_ttl. * queue.enqueue() now supports failure_ttl * Added registry.get_queue(). * FailedJobRegistry.add() now assigns DEFAULT_FAILURE_TTL. * StartedJobRegistry.cleanup() now moves expired jobs to FailedJobRegistry. * Failed jobs are now added to FailedJobRegistry. * Added FailedJobRegistry.requeue() * Document the new `FailedJobRegistry` and changes in custom exception handler behavior. * Added worker.disable_default_exception_handler. * Document --disable-default-exception-handler option. * Deleted worker.failed_queue. * Deleted "move_to_failed_queue" exception handler. * StartedJobRegistry should no longer move jobs to FailedQueue. * Deleted requeue_job * Fixed test error. * Make requeue cli command work with FailedJobRegistry * Added .pytest_cache to gitignore. * Custom exception handlers are no longer run in reverse * Restored requeue_job function * Removed get_failed_queue * Deleted FailedQueue * Updated changelog. * Document `failure_ttl` * Updated docs. * Remove job.status * Fixed typo in test_registry.py * Replaced _pipeline() with pipeline() * FailedJobRegistry no longer fails on redis-py>=3 * Fixes test_clean_registries * Worker names are now randomized * Added a note about random worker names in CHANGES.md * Worker will now stop working when encountering an unhandled exception. * Worker should reraise SystemExit on cold shutdowns * Added anchor.js to docs * Support for Sentry-SDK (#1045) * Updated RQ to support sentry-sdk * Document Sentry integration * Install sentry-sdk before running tests * Improved rq info CLI command to be more efficient when displaying lar… (#1046) * Improved rq info CLI command to be more efficient when displaying large number of workers * Fixed an rq info --by-queue bug * Fixed worker.total_working_time bug (#1047) * queue.enqueue() no longer accepts `timeout` argument (#1055) * Clean worker registry (#1056) * queue.enqueue() no longer accepts `timeout` argument * Added clean_worker_registry() * Show worker hostname and PID on cli (#1058) * Show worker hostname and PID on cli * Improve test coverage * Remove Redis version check when SSL is used * Bump version to 1.0 * Removed pytest_cache/README.md * Changed worker logging to use exc_info=True * Removed unused queue.dequeue() * Fixed typo in CHANGES.md * setup_loghandlers() should always call logger.setLevel() if specified
6 years ago
self.assert_normal_execution(result)
self.assertIn('2 workers, 2 queues', result.output)
Worker pool (#1874) * First stab at implementating worker pool * Use process.is_alive() to check whether a process is still live * Handle shutdown signal * Check worker loop done * First working version of `WorkerPool`. * Added test for check_workers() * Added test for pool.start() * Better shutdown process * Comment out test_start() to see if it fixes CI * Make tests pass * Make CI pass * Comment out some tests * Comment out more tests * Re-enable a test * Re-enable another test * Uncomment check_workers test * Added run_worker test * Minor modification to dead worker detection * More test cases * Better process name for workers * Added back pool.stop_workers() when signal is received * Cleaned up cli.py * WIP on worker-pool command * Fix test * Test that worker pool ignores consecutive shutdown signals * Added test for worker-pool CLI command. * Added timeout to CI jobs * Fix worker pool test * Comment out test_scheduler.py * Fixed worker-pool in burst mode * Increase test coverage * Exclude tests directory from coverage.py * Improve test coverage * Renamed `Pool(num_workers=2) to `Pool(size=2)` * Revert "Renamed `Pool(num_workers=2) to `Pool(size=2)`" This reverts commit a1306f89ad0d8686c6bde447bff75e2f71f0733b. * Renamed Pool to WorkerPool * Added a new TestCase that doesn't use LocalStack * Added job_class, worker_class and serializer arguments to WorkerPool * Use parse_connection() in WorkerPool.__init__ * Added CLI arguments for worker-pool * Minor WorkerPool and test fixes * Fixed failing CLI test * Document WorkerPool
2 years ago
result = runner.invoke(main, ['info', 'foo', 'bar', '--by-queue', '-u', self.redis_url, '--only-workers'])
RQ v1.0! (#1059) * Added FailedJobRegistry. * Added job.failure_ttl. * queue.enqueue() now supports failure_ttl * Added registry.get_queue(). * FailedJobRegistry.add() now assigns DEFAULT_FAILURE_TTL. * StartedJobRegistry.cleanup() now moves expired jobs to FailedJobRegistry. * Failed jobs are now added to FailedJobRegistry. * Added FailedJobRegistry.requeue() * Document the new `FailedJobRegistry` and changes in custom exception handler behavior. * Added worker.disable_default_exception_handler. * Document --disable-default-exception-handler option. * Deleted worker.failed_queue. * Deleted "move_to_failed_queue" exception handler. * StartedJobRegistry should no longer move jobs to FailedQueue. * Deleted requeue_job * Fixed test error. * Make requeue cli command work with FailedJobRegistry * Added .pytest_cache to gitignore. * Custom exception handlers are no longer run in reverse * Restored requeue_job function * Removed get_failed_queue * Deleted FailedQueue * Updated changelog. * Document `failure_ttl` * Updated docs. * Remove job.status * Fixed typo in test_registry.py * Replaced _pipeline() with pipeline() * FailedJobRegistry no longer fails on redis-py>=3 * Fixes test_clean_registries * Worker names are now randomized * Added a note about random worker names in CHANGES.md * Worker will now stop working when encountering an unhandled exception. * Worker should reraise SystemExit on cold shutdowns * Added anchor.js to docs * Support for Sentry-SDK (#1045) * Updated RQ to support sentry-sdk * Document Sentry integration * Install sentry-sdk before running tests * Improved rq info CLI command to be more efficient when displaying lar… (#1046) * Improved rq info CLI command to be more efficient when displaying large number of workers * Fixed an rq info --by-queue bug * Fixed worker.total_working_time bug (#1047) * queue.enqueue() no longer accepts `timeout` argument (#1055) * Clean worker registry (#1056) * queue.enqueue() no longer accepts `timeout` argument * Added clean_worker_registry() * Show worker hostname and PID on cli (#1058) * Show worker hostname and PID on cli * Improve test coverage * Remove Redis version check when SSL is used * Bump version to 1.0 * Removed pytest_cache/README.md * Changed worker logging to use exc_info=True * Removed unused queue.dequeue() * Fixed typo in CHANGES.md * setup_loghandlers() should always call logger.setLevel() if specified
6 years ago
self.assert_normal_execution(result)
# Ensure both queues' workers are shown
self.assertIn('foo:', result.output)
self.assertIn('bar:', result.output)
self.assertIn('2 workers, 2 queues', result.output)
def test_worker(self):
"""rq worker -u <url> -b"""
runner = CliRunner()
result = runner.invoke(main, ['worker', '-u', self.redis_url, '-b'])
self.assert_normal_execution(result)
def test_worker_pid(self):
"""rq worker -u <url> /tmp/.."""
pid = self.tmpdir.join('rq.pid')
runner = CliRunner()
result = runner.invoke(main, ['worker', '-u', self.redis_url, '-b', '--pid', str(pid)])
self.assertTrue(len(pid.read()) > 0)
self.assert_normal_execution(result)
Job scheduling (#1163) * First RQScheduler prototype * WIP job scheduling * Fixed Python 2.7 tests * Added ScheduledJobRegistry.get_scheduled_time(job) * WIP on scheduler's threading mechanism * Fixed test errors * Changed scheduler.acquire_locks() to instance method * Added scheduler.prepare_registries() * Somewhat working implementation of RQ scheduler * Only call stop_scheduler if there's a scheduler present * Use OSError rather than ProcessLookupError for PyPy compatibility * Added `auto_start` argument to scheduler.acquire_locks() * Make RQScheduler play better with timezone * Fixed test error * Added --with-scheduler flag to rq worker CLI * Fix tests on Python 2.x * More Python 2 fixes * Only call `scheduler.start` if worker is run in non burst mode * Fixed an issue where running worker with scheduler would fail sometimes * Make `worker.stop_scheduler()` more resilient to errors * worker.dequeue_job_and_maintain_ttl() should also periodically run maintenance tasks * Scheduler can now work with worker in both burst and non burst mode * Fixed scheduler logging message * Always log scheduler errors when running * Improve scheduler error logging message * Removed testing code * Scheduler should periodically try to acquire locks for other queues it doesn't have * Added tests for scheduler.should_reacquire_locks * Added queue.enqueue_in() * Fixes queue.enqueue_in() in Python 2.7 * First stab at documenting job scheduling * Remove unused methods * Remove Python 2.6 logging compatibility code * Remove more unused imports * Added convenience methods to access job registries from queue * Added test for worker.run_maintenance_tasks() * Simplify worker.queue_names() and worker.queue_keys() * Updated changelog to mention RQ's new job scheduling mechanism.
5 years ago
def test_worker_with_scheduler(self):
"""rq worker -u <url> --with-scheduler"""
queue = Queue(connection=self.connection)
queue.enqueue_at(datetime(2019, 1, 1, tzinfo=timezone.utc), say_hello)
Job scheduling (#1163) * First RQScheduler prototype * WIP job scheduling * Fixed Python 2.7 tests * Added ScheduledJobRegistry.get_scheduled_time(job) * WIP on scheduler's threading mechanism * Fixed test errors * Changed scheduler.acquire_locks() to instance method * Added scheduler.prepare_registries() * Somewhat working implementation of RQ scheduler * Only call stop_scheduler if there's a scheduler present * Use OSError rather than ProcessLookupError for PyPy compatibility * Added `auto_start` argument to scheduler.acquire_locks() * Make RQScheduler play better with timezone * Fixed test error * Added --with-scheduler flag to rq worker CLI * Fix tests on Python 2.x * More Python 2 fixes * Only call `scheduler.start` if worker is run in non burst mode * Fixed an issue where running worker with scheduler would fail sometimes * Make `worker.stop_scheduler()` more resilient to errors * worker.dequeue_job_and_maintain_ttl() should also periodically run maintenance tasks * Scheduler can now work with worker in both burst and non burst mode * Fixed scheduler logging message * Always log scheduler errors when running * Improve scheduler error logging message * Removed testing code * Scheduler should periodically try to acquire locks for other queues it doesn't have * Added tests for scheduler.should_reacquire_locks * Added queue.enqueue_in() * Fixes queue.enqueue_in() in Python 2.7 * First stab at documenting job scheduling * Remove unused methods * Remove Python 2.6 logging compatibility code * Remove more unused imports * Added convenience methods to access job registries from queue * Added test for worker.run_maintenance_tasks() * Simplify worker.queue_names() and worker.queue_keys() * Updated changelog to mention RQ's new job scheduling mechanism.
5 years ago
registry = ScheduledJobRegistry(queue=queue)
runner = CliRunner()
result = runner.invoke(main, ['worker', '-u', self.redis_url, '-b'])
self.assert_normal_execution(result)
self.assertEqual(len(registry), 1) # 1 job still scheduled
result = runner.invoke(main, ['worker', '-u', self.redis_url, '-b', '--with-scheduler'])
self.assert_normal_execution(result)
self.assertEqual(len(registry), 0) # Job has been enqueued
def test_worker_logging_options(self):
"""--quiet and --verbose logging options are supported"""
runner = CliRunner()
args = ['worker', '-u', self.redis_url, '-b']
result = runner.invoke(main, args + ['--verbose'])
self.assert_normal_execution(result)
result = runner.invoke(main, args + ['--quiet'])
self.assert_normal_execution(result)
# --quiet and --verbose are mutually exclusive
result = runner.invoke(main, args + ['--quiet', '--verbose'])
self.assertNotEqual(result.exit_code, 0)
Job scheduling (#1163) * First RQScheduler prototype * WIP job scheduling * Fixed Python 2.7 tests * Added ScheduledJobRegistry.get_scheduled_time(job) * WIP on scheduler's threading mechanism * Fixed test errors * Changed scheduler.acquire_locks() to instance method * Added scheduler.prepare_registries() * Somewhat working implementation of RQ scheduler * Only call stop_scheduler if there's a scheduler present * Use OSError rather than ProcessLookupError for PyPy compatibility * Added `auto_start` argument to scheduler.acquire_locks() * Make RQScheduler play better with timezone * Fixed test error * Added --with-scheduler flag to rq worker CLI * Fix tests on Python 2.x * More Python 2 fixes * Only call `scheduler.start` if worker is run in non burst mode * Fixed an issue where running worker with scheduler would fail sometimes * Make `worker.stop_scheduler()` more resilient to errors * worker.dequeue_job_and_maintain_ttl() should also periodically run maintenance tasks * Scheduler can now work with worker in both burst and non burst mode * Fixed scheduler logging message * Always log scheduler errors when running * Improve scheduler error logging message * Removed testing code * Scheduler should periodically try to acquire locks for other queues it doesn't have * Added tests for scheduler.should_reacquire_locks * Added queue.enqueue_in() * Fixes queue.enqueue_in() in Python 2.7 * First stab at documenting job scheduling * Remove unused methods * Remove Python 2.6 logging compatibility code * Remove more unused imports * Added convenience methods to access job registries from queue * Added test for worker.run_maintenance_tasks() * Simplify worker.queue_names() and worker.queue_keys() * Updated changelog to mention RQ's new job scheduling mechanism.
5 years ago
def test_worker_dequeue_strategy(self):
"""--quiet and --verbose logging options are supported"""
runner = CliRunner()
args = ['worker', '-u', self.redis_url, '-b', '--dequeue-strategy', 'random']
result = runner.invoke(main, args)
self.assert_normal_execution(result)
args = ['worker', '-u', self.redis_url, '-b', '--dequeue-strategy', 'round_robin']
result = runner.invoke(main, args)
self.assert_normal_execution(result)
args = ['worker', '-u', self.redis_url, '-b', '--dequeue-strategy', 'wrong']
result = runner.invoke(main, args)
self.assertEqual(result.exit_code, 1)
def test_exception_handlers(self):
"""rq worker -u <url> -b --exception-handler <handler>"""
RQ v1.0! (#1059) * Added FailedJobRegistry. * Added job.failure_ttl. * queue.enqueue() now supports failure_ttl * Added registry.get_queue(). * FailedJobRegistry.add() now assigns DEFAULT_FAILURE_TTL. * StartedJobRegistry.cleanup() now moves expired jobs to FailedJobRegistry. * Failed jobs are now added to FailedJobRegistry. * Added FailedJobRegistry.requeue() * Document the new `FailedJobRegistry` and changes in custom exception handler behavior. * Added worker.disable_default_exception_handler. * Document --disable-default-exception-handler option. * Deleted worker.failed_queue. * Deleted "move_to_failed_queue" exception handler. * StartedJobRegistry should no longer move jobs to FailedQueue. * Deleted requeue_job * Fixed test error. * Make requeue cli command work with FailedJobRegistry * Added .pytest_cache to gitignore. * Custom exception handlers are no longer run in reverse * Restored requeue_job function * Removed get_failed_queue * Deleted FailedQueue * Updated changelog. * Document `failure_ttl` * Updated docs. * Remove job.status * Fixed typo in test_registry.py * Replaced _pipeline() with pipeline() * FailedJobRegistry no longer fails on redis-py>=3 * Fixes test_clean_registries * Worker names are now randomized * Added a note about random worker names in CHANGES.md * Worker will now stop working when encountering an unhandled exception. * Worker should reraise SystemExit on cold shutdowns * Added anchor.js to docs * Support for Sentry-SDK (#1045) * Updated RQ to support sentry-sdk * Document Sentry integration * Install sentry-sdk before running tests * Improved rq info CLI command to be more efficient when displaying lar… (#1046) * Improved rq info CLI command to be more efficient when displaying large number of workers * Fixed an rq info --by-queue bug * Fixed worker.total_working_time bug (#1047) * queue.enqueue() no longer accepts `timeout` argument (#1055) * Clean worker registry (#1056) * queue.enqueue() no longer accepts `timeout` argument * Added clean_worker_registry() * Show worker hostname and PID on cli (#1058) * Show worker hostname and PID on cli * Improve test coverage * Remove Redis version check when SSL is used * Bump version to 1.0 * Removed pytest_cache/README.md * Changed worker logging to use exc_info=True * Removed unused queue.dequeue() * Fixed typo in CHANGES.md * setup_loghandlers() should always call logger.setLevel() if specified
6 years ago
connection = Redis.from_url(self.redis_url)
q = Queue('default', connection=connection)
runner = CliRunner()
RQ v1.0! (#1059) * Added FailedJobRegistry. * Added job.failure_ttl. * queue.enqueue() now supports failure_ttl * Added registry.get_queue(). * FailedJobRegistry.add() now assigns DEFAULT_FAILURE_TTL. * StartedJobRegistry.cleanup() now moves expired jobs to FailedJobRegistry. * Failed jobs are now added to FailedJobRegistry. * Added FailedJobRegistry.requeue() * Document the new `FailedJobRegistry` and changes in custom exception handler behavior. * Added worker.disable_default_exception_handler. * Document --disable-default-exception-handler option. * Deleted worker.failed_queue. * Deleted "move_to_failed_queue" exception handler. * StartedJobRegistry should no longer move jobs to FailedQueue. * Deleted requeue_job * Fixed test error. * Make requeue cli command work with FailedJobRegistry * Added .pytest_cache to gitignore. * Custom exception handlers are no longer run in reverse * Restored requeue_job function * Removed get_failed_queue * Deleted FailedQueue * Updated changelog. * Document `failure_ttl` * Updated docs. * Remove job.status * Fixed typo in test_registry.py * Replaced _pipeline() with pipeline() * FailedJobRegistry no longer fails on redis-py>=3 * Fixes test_clean_registries * Worker names are now randomized * Added a note about random worker names in CHANGES.md * Worker will now stop working when encountering an unhandled exception. * Worker should reraise SystemExit on cold shutdowns * Added anchor.js to docs * Support for Sentry-SDK (#1045) * Updated RQ to support sentry-sdk * Document Sentry integration * Install sentry-sdk before running tests * Improved rq info CLI command to be more efficient when displaying lar… (#1046) * Improved rq info CLI command to be more efficient when displaying large number of workers * Fixed an rq info --by-queue bug * Fixed worker.total_working_time bug (#1047) * queue.enqueue() no longer accepts `timeout` argument (#1055) * Clean worker registry (#1056) * queue.enqueue() no longer accepts `timeout` argument * Added clean_worker_registry() * Show worker hostname and PID on cli (#1058) * Show worker hostname and PID on cli * Improve test coverage * Remove Redis version check when SSL is used * Bump version to 1.0 * Removed pytest_cache/README.md * Changed worker logging to use exc_info=True * Removed unused queue.dequeue() * Fixed typo in CHANGES.md * setup_loghandlers() should always call logger.setLevel() if specified
6 years ago
# If exception handler is not given, no custom exception handler is run
job = q.enqueue(div_by_zero)
runner.invoke(main, ['worker', '-u', self.redis_url, '-b'])
RQ v1.0! (#1059) * Added FailedJobRegistry. * Added job.failure_ttl. * queue.enqueue() now supports failure_ttl * Added registry.get_queue(). * FailedJobRegistry.add() now assigns DEFAULT_FAILURE_TTL. * StartedJobRegistry.cleanup() now moves expired jobs to FailedJobRegistry. * Failed jobs are now added to FailedJobRegistry. * Added FailedJobRegistry.requeue() * Document the new `FailedJobRegistry` and changes in custom exception handler behavior. * Added worker.disable_default_exception_handler. * Document --disable-default-exception-handler option. * Deleted worker.failed_queue. * Deleted "move_to_failed_queue" exception handler. * StartedJobRegistry should no longer move jobs to FailedQueue. * Deleted requeue_job * Fixed test error. * Make requeue cli command work with FailedJobRegistry * Added .pytest_cache to gitignore. * Custom exception handlers are no longer run in reverse * Restored requeue_job function * Removed get_failed_queue * Deleted FailedQueue * Updated changelog. * Document `failure_ttl` * Updated docs. * Remove job.status * Fixed typo in test_registry.py * Replaced _pipeline() with pipeline() * FailedJobRegistry no longer fails on redis-py>=3 * Fixes test_clean_registries * Worker names are now randomized * Added a note about random worker names in CHANGES.md * Worker will now stop working when encountering an unhandled exception. * Worker should reraise SystemExit on cold shutdowns * Added anchor.js to docs * Support for Sentry-SDK (#1045) * Updated RQ to support sentry-sdk * Document Sentry integration * Install sentry-sdk before running tests * Improved rq info CLI command to be more efficient when displaying lar… (#1046) * Improved rq info CLI command to be more efficient when displaying large number of workers * Fixed an rq info --by-queue bug * Fixed worker.total_working_time bug (#1047) * queue.enqueue() no longer accepts `timeout` argument (#1055) * Clean worker registry (#1056) * queue.enqueue() no longer accepts `timeout` argument * Added clean_worker_registry() * Show worker hostname and PID on cli (#1058) * Show worker hostname and PID on cli * Improve test coverage * Remove Redis version check when SSL is used * Bump version to 1.0 * Removed pytest_cache/README.md * Changed worker logging to use exc_info=True * Removed unused queue.dequeue() * Fixed typo in CHANGES.md * setup_loghandlers() should always call logger.setLevel() if specified
6 years ago
registry = FailedJobRegistry(queue=q)
self.assertTrue(job in registry)
# If disable-default-exception-handler is given, job is not moved to FailedJobRegistry
job = q.enqueue(div_by_zero)
Worker pool (#1874) * First stab at implementating worker pool * Use process.is_alive() to check whether a process is still live * Handle shutdown signal * Check worker loop done * First working version of `WorkerPool`. * Added test for check_workers() * Added test for pool.start() * Better shutdown process * Comment out test_start() to see if it fixes CI * Make tests pass * Make CI pass * Comment out some tests * Comment out more tests * Re-enable a test * Re-enable another test * Uncomment check_workers test * Added run_worker test * Minor modification to dead worker detection * More test cases * Better process name for workers * Added back pool.stop_workers() when signal is received * Cleaned up cli.py * WIP on worker-pool command * Fix test * Test that worker pool ignores consecutive shutdown signals * Added test for worker-pool CLI command. * Added timeout to CI jobs * Fix worker pool test * Comment out test_scheduler.py * Fixed worker-pool in burst mode * Increase test coverage * Exclude tests directory from coverage.py * Improve test coverage * Renamed `Pool(num_workers=2) to `Pool(size=2)` * Revert "Renamed `Pool(num_workers=2) to `Pool(size=2)`" This reverts commit a1306f89ad0d8686c6bde447bff75e2f71f0733b. * Renamed Pool to WorkerPool * Added a new TestCase that doesn't use LocalStack * Added job_class, worker_class and serializer arguments to WorkerPool * Use parse_connection() in WorkerPool.__init__ * Added CLI arguments for worker-pool * Minor WorkerPool and test fixes * Fixed failing CLI test * Document WorkerPool
2 years ago
runner.invoke(main, ['worker', '-u', self.redis_url, '-b', '--disable-default-exception-handler'])
RQ v1.0! (#1059) * Added FailedJobRegistry. * Added job.failure_ttl. * queue.enqueue() now supports failure_ttl * Added registry.get_queue(). * FailedJobRegistry.add() now assigns DEFAULT_FAILURE_TTL. * StartedJobRegistry.cleanup() now moves expired jobs to FailedJobRegistry. * Failed jobs are now added to FailedJobRegistry. * Added FailedJobRegistry.requeue() * Document the new `FailedJobRegistry` and changes in custom exception handler behavior. * Added worker.disable_default_exception_handler. * Document --disable-default-exception-handler option. * Deleted worker.failed_queue. * Deleted "move_to_failed_queue" exception handler. * StartedJobRegistry should no longer move jobs to FailedQueue. * Deleted requeue_job * Fixed test error. * Make requeue cli command work with FailedJobRegistry * Added .pytest_cache to gitignore. * Custom exception handlers are no longer run in reverse * Restored requeue_job function * Removed get_failed_queue * Deleted FailedQueue * Updated changelog. * Document `failure_ttl` * Updated docs. * Remove job.status * Fixed typo in test_registry.py * Replaced _pipeline() with pipeline() * FailedJobRegistry no longer fails on redis-py>=3 * Fixes test_clean_registries * Worker names are now randomized * Added a note about random worker names in CHANGES.md * Worker will now stop working when encountering an unhandled exception. * Worker should reraise SystemExit on cold shutdowns * Added anchor.js to docs * Support for Sentry-SDK (#1045) * Updated RQ to support sentry-sdk * Document Sentry integration * Install sentry-sdk before running tests * Improved rq info CLI command to be more efficient when displaying lar… (#1046) * Improved rq info CLI command to be more efficient when displaying large number of workers * Fixed an rq info --by-queue bug * Fixed worker.total_working_time bug (#1047) * queue.enqueue() no longer accepts `timeout` argument (#1055) * Clean worker registry (#1056) * queue.enqueue() no longer accepts `timeout` argument * Added clean_worker_registry() * Show worker hostname and PID on cli (#1058) * Show worker hostname and PID on cli * Improve test coverage * Remove Redis version check when SSL is used * Bump version to 1.0 * Removed pytest_cache/README.md * Changed worker logging to use exc_info=True * Removed unused queue.dequeue() * Fixed typo in CHANGES.md * setup_loghandlers() should always call logger.setLevel() if specified
6 years ago
registry = FailedJobRegistry(queue=q)
self.assertFalse(job in registry)
# Both default and custom exception handler is run
job = q.enqueue(div_by_zero)
Worker pool (#1874) * First stab at implementating worker pool * Use process.is_alive() to check whether a process is still live * Handle shutdown signal * Check worker loop done * First working version of `WorkerPool`. * Added test for check_workers() * Added test for pool.start() * Better shutdown process * Comment out test_start() to see if it fixes CI * Make tests pass * Make CI pass * Comment out some tests * Comment out more tests * Re-enable a test * Re-enable another test * Uncomment check_workers test * Added run_worker test * Minor modification to dead worker detection * More test cases * Better process name for workers * Added back pool.stop_workers() when signal is received * Cleaned up cli.py * WIP on worker-pool command * Fix test * Test that worker pool ignores consecutive shutdown signals * Added test for worker-pool CLI command. * Added timeout to CI jobs * Fix worker pool test * Comment out test_scheduler.py * Fixed worker-pool in burst mode * Increase test coverage * Exclude tests directory from coverage.py * Improve test coverage * Renamed `Pool(num_workers=2) to `Pool(size=2)` * Revert "Renamed `Pool(num_workers=2) to `Pool(size=2)`" This reverts commit a1306f89ad0d8686c6bde447bff75e2f71f0733b. * Renamed Pool to WorkerPool * Added a new TestCase that doesn't use LocalStack * Added job_class, worker_class and serializer arguments to WorkerPool * Use parse_connection() in WorkerPool.__init__ * Added CLI arguments for worker-pool * Minor WorkerPool and test fixes * Fixed failing CLI test * Document WorkerPool
2 years ago
runner.invoke(main, ['worker', '-u', self.redis_url, '-b', '--exception-handler', 'tests.fixtures.add_meta'])
RQ v1.0! (#1059) * Added FailedJobRegistry. * Added job.failure_ttl. * queue.enqueue() now supports failure_ttl * Added registry.get_queue(). * FailedJobRegistry.add() now assigns DEFAULT_FAILURE_TTL. * StartedJobRegistry.cleanup() now moves expired jobs to FailedJobRegistry. * Failed jobs are now added to FailedJobRegistry. * Added FailedJobRegistry.requeue() * Document the new `FailedJobRegistry` and changes in custom exception handler behavior. * Added worker.disable_default_exception_handler. * Document --disable-default-exception-handler option. * Deleted worker.failed_queue. * Deleted "move_to_failed_queue" exception handler. * StartedJobRegistry should no longer move jobs to FailedQueue. * Deleted requeue_job * Fixed test error. * Make requeue cli command work with FailedJobRegistry * Added .pytest_cache to gitignore. * Custom exception handlers are no longer run in reverse * Restored requeue_job function * Removed get_failed_queue * Deleted FailedQueue * Updated changelog. * Document `failure_ttl` * Updated docs. * Remove job.status * Fixed typo in test_registry.py * Replaced _pipeline() with pipeline() * FailedJobRegistry no longer fails on redis-py>=3 * Fixes test_clean_registries * Worker names are now randomized * Added a note about random worker names in CHANGES.md * Worker will now stop working when encountering an unhandled exception. * Worker should reraise SystemExit on cold shutdowns * Added anchor.js to docs * Support for Sentry-SDK (#1045) * Updated RQ to support sentry-sdk * Document Sentry integration * Install sentry-sdk before running tests * Improved rq info CLI command to be more efficient when displaying lar… (#1046) * Improved rq info CLI command to be more efficient when displaying large number of workers * Fixed an rq info --by-queue bug * Fixed worker.total_working_time bug (#1047) * queue.enqueue() no longer accepts `timeout` argument (#1055) * Clean worker registry (#1056) * queue.enqueue() no longer accepts `timeout` argument * Added clean_worker_registry() * Show worker hostname and PID on cli (#1058) * Show worker hostname and PID on cli * Improve test coverage * Remove Redis version check when SSL is used * Bump version to 1.0 * Removed pytest_cache/README.md * Changed worker logging to use exc_info=True * Removed unused queue.dequeue() * Fixed typo in CHANGES.md * setup_loghandlers() should always call logger.setLevel() if specified
6 years ago
registry = FailedJobRegistry(queue=q)
self.assertTrue(job in registry)
job.refresh()
self.assertEqual(job.meta, {'foo': 1})
RQ v1.0! (#1059) * Added FailedJobRegistry. * Added job.failure_ttl. * queue.enqueue() now supports failure_ttl * Added registry.get_queue(). * FailedJobRegistry.add() now assigns DEFAULT_FAILURE_TTL. * StartedJobRegistry.cleanup() now moves expired jobs to FailedJobRegistry. * Failed jobs are now added to FailedJobRegistry. * Added FailedJobRegistry.requeue() * Document the new `FailedJobRegistry` and changes in custom exception handler behavior. * Added worker.disable_default_exception_handler. * Document --disable-default-exception-handler option. * Deleted worker.failed_queue. * Deleted "move_to_failed_queue" exception handler. * StartedJobRegistry should no longer move jobs to FailedQueue. * Deleted requeue_job * Fixed test error. * Make requeue cli command work with FailedJobRegistry * Added .pytest_cache to gitignore. * Custom exception handlers are no longer run in reverse * Restored requeue_job function * Removed get_failed_queue * Deleted FailedQueue * Updated changelog. * Document `failure_ttl` * Updated docs. * Remove job.status * Fixed typo in test_registry.py * Replaced _pipeline() with pipeline() * FailedJobRegistry no longer fails on redis-py>=3 * Fixes test_clean_registries * Worker names are now randomized * Added a note about random worker names in CHANGES.md * Worker will now stop working when encountering an unhandled exception. * Worker should reraise SystemExit on cold shutdowns * Added anchor.js to docs * Support for Sentry-SDK (#1045) * Updated RQ to support sentry-sdk * Document Sentry integration * Install sentry-sdk before running tests * Improved rq info CLI command to be more efficient when displaying lar… (#1046) * Improved rq info CLI command to be more efficient when displaying large number of workers * Fixed an rq info --by-queue bug * Fixed worker.total_working_time bug (#1047) * queue.enqueue() no longer accepts `timeout` argument (#1055) * Clean worker registry (#1056) * queue.enqueue() no longer accepts `timeout` argument * Added clean_worker_registry() * Show worker hostname and PID on cli (#1058) * Show worker hostname and PID on cli * Improve test coverage * Remove Redis version check when SSL is used * Bump version to 1.0 * Removed pytest_cache/README.md * Changed worker logging to use exc_info=True * Removed unused queue.dequeue() * Fixed typo in CHANGES.md * setup_loghandlers() should always call logger.setLevel() if specified
6 years ago
# Only custom exception handler is run
job = q.enqueue(div_by_zero)
Worker pool (#1874) * First stab at implementating worker pool * Use process.is_alive() to check whether a process is still live * Handle shutdown signal * Check worker loop done * First working version of `WorkerPool`. * Added test for check_workers() * Added test for pool.start() * Better shutdown process * Comment out test_start() to see if it fixes CI * Make tests pass * Make CI pass * Comment out some tests * Comment out more tests * Re-enable a test * Re-enable another test * Uncomment check_workers test * Added run_worker test * Minor modification to dead worker detection * More test cases * Better process name for workers * Added back pool.stop_workers() when signal is received * Cleaned up cli.py * WIP on worker-pool command * Fix test * Test that worker pool ignores consecutive shutdown signals * Added test for worker-pool CLI command. * Added timeout to CI jobs * Fix worker pool test * Comment out test_scheduler.py * Fixed worker-pool in burst mode * Increase test coverage * Exclude tests directory from coverage.py * Improve test coverage * Renamed `Pool(num_workers=2) to `Pool(size=2)` * Revert "Renamed `Pool(num_workers=2) to `Pool(size=2)`" This reverts commit a1306f89ad0d8686c6bde447bff75e2f71f0733b. * Renamed Pool to WorkerPool * Added a new TestCase that doesn't use LocalStack * Added job_class, worker_class and serializer arguments to WorkerPool * Use parse_connection() in WorkerPool.__init__ * Added CLI arguments for worker-pool * Minor WorkerPool and test fixes * Fixed failing CLI test * Document WorkerPool
2 years ago
runner.invoke(
main,
[
'worker',
'-u',
self.redis_url,
'-b',
'--exception-handler',
'tests.fixtures.add_meta',
'--disable-default-exception-handler',
],
)
RQ v1.0! (#1059) * Added FailedJobRegistry. * Added job.failure_ttl. * queue.enqueue() now supports failure_ttl * Added registry.get_queue(). * FailedJobRegistry.add() now assigns DEFAULT_FAILURE_TTL. * StartedJobRegistry.cleanup() now moves expired jobs to FailedJobRegistry. * Failed jobs are now added to FailedJobRegistry. * Added FailedJobRegistry.requeue() * Document the new `FailedJobRegistry` and changes in custom exception handler behavior. * Added worker.disable_default_exception_handler. * Document --disable-default-exception-handler option. * Deleted worker.failed_queue. * Deleted "move_to_failed_queue" exception handler. * StartedJobRegistry should no longer move jobs to FailedQueue. * Deleted requeue_job * Fixed test error. * Make requeue cli command work with FailedJobRegistry * Added .pytest_cache to gitignore. * Custom exception handlers are no longer run in reverse * Restored requeue_job function * Removed get_failed_queue * Deleted FailedQueue * Updated changelog. * Document `failure_ttl` * Updated docs. * Remove job.status * Fixed typo in test_registry.py * Replaced _pipeline() with pipeline() * FailedJobRegistry no longer fails on redis-py>=3 * Fixes test_clean_registries * Worker names are now randomized * Added a note about random worker names in CHANGES.md * Worker will now stop working when encountering an unhandled exception. * Worker should reraise SystemExit on cold shutdowns * Added anchor.js to docs * Support for Sentry-SDK (#1045) * Updated RQ to support sentry-sdk * Document Sentry integration * Install sentry-sdk before running tests * Improved rq info CLI command to be more efficient when displaying lar… (#1046) * Improved rq info CLI command to be more efficient when displaying large number of workers * Fixed an rq info --by-queue bug * Fixed worker.total_working_time bug (#1047) * queue.enqueue() no longer accepts `timeout` argument (#1055) * Clean worker registry (#1056) * queue.enqueue() no longer accepts `timeout` argument * Added clean_worker_registry() * Show worker hostname and PID on cli (#1058) * Show worker hostname and PID on cli * Improve test coverage * Remove Redis version check when SSL is used * Bump version to 1.0 * Removed pytest_cache/README.md * Changed worker logging to use exc_info=True * Removed unused queue.dequeue() * Fixed typo in CHANGES.md * setup_loghandlers() should always call logger.setLevel() if specified
6 years ago
registry = FailedJobRegistry(queue=q)
self.assertFalse(job in registry)
job.refresh()
self.assertEqual(job.meta, {'foo': 1})
def test_suspend_and_resume(self):
"""rq suspend -u <url>
Worker pool (#1874) * First stab at implementating worker pool * Use process.is_alive() to check whether a process is still live * Handle shutdown signal * Check worker loop done * First working version of `WorkerPool`. * Added test for check_workers() * Added test for pool.start() * Better shutdown process * Comment out test_start() to see if it fixes CI * Make tests pass * Make CI pass * Comment out some tests * Comment out more tests * Re-enable a test * Re-enable another test * Uncomment check_workers test * Added run_worker test * Minor modification to dead worker detection * More test cases * Better process name for workers * Added back pool.stop_workers() when signal is received * Cleaned up cli.py * WIP on worker-pool command * Fix test * Test that worker pool ignores consecutive shutdown signals * Added test for worker-pool CLI command. * Added timeout to CI jobs * Fix worker pool test * Comment out test_scheduler.py * Fixed worker-pool in burst mode * Increase test coverage * Exclude tests directory from coverage.py * Improve test coverage * Renamed `Pool(num_workers=2) to `Pool(size=2)` * Revert "Renamed `Pool(num_workers=2) to `Pool(size=2)`" This reverts commit a1306f89ad0d8686c6bde447bff75e2f71f0733b. * Renamed Pool to WorkerPool * Added a new TestCase that doesn't use LocalStack * Added job_class, worker_class and serializer arguments to WorkerPool * Use parse_connection() in WorkerPool.__init__ * Added CLI arguments for worker-pool * Minor WorkerPool and test fixes * Fixed failing CLI test * Document WorkerPool
2 years ago
rq worker -u <url> -b
rq resume -u <url>
"""
runner = CliRunner()
result = runner.invoke(main, ['suspend', '-u', self.redis_url])
self.assert_normal_execution(result)
result = runner.invoke(main, ['worker', '-u', self.redis_url, '-b'])
self.assertEqual(result.exit_code, 1)
Worker pool (#1874) * First stab at implementating worker pool * Use process.is_alive() to check whether a process is still live * Handle shutdown signal * Check worker loop done * First working version of `WorkerPool`. * Added test for check_workers() * Added test for pool.start() * Better shutdown process * Comment out test_start() to see if it fixes CI * Make tests pass * Make CI pass * Comment out some tests * Comment out more tests * Re-enable a test * Re-enable another test * Uncomment check_workers test * Added run_worker test * Minor modification to dead worker detection * More test cases * Better process name for workers * Added back pool.stop_workers() when signal is received * Cleaned up cli.py * WIP on worker-pool command * Fix test * Test that worker pool ignores consecutive shutdown signals * Added test for worker-pool CLI command. * Added timeout to CI jobs * Fix worker pool test * Comment out test_scheduler.py * Fixed worker-pool in burst mode * Increase test coverage * Exclude tests directory from coverage.py * Improve test coverage * Renamed `Pool(num_workers=2) to `Pool(size=2)` * Revert "Renamed `Pool(num_workers=2) to `Pool(size=2)`" This reverts commit a1306f89ad0d8686c6bde447bff75e2f71f0733b. * Renamed Pool to WorkerPool * Added a new TestCase that doesn't use LocalStack * Added job_class, worker_class and serializer arguments to WorkerPool * Use parse_connection() in WorkerPool.__init__ * Added CLI arguments for worker-pool * Minor WorkerPool and test fixes * Fixed failing CLI test * Document WorkerPool
2 years ago
self.assertEqual(result.output.strip(), 'RQ is currently suspended, to resume job execution run "rq resume"')
result = runner.invoke(main, ['resume', '-u', self.redis_url])
self.assert_normal_execution(result)
def test_suspend_with_ttl(self):
Worker pool (#1874) * First stab at implementating worker pool * Use process.is_alive() to check whether a process is still live * Handle shutdown signal * Check worker loop done * First working version of `WorkerPool`. * Added test for check_workers() * Added test for pool.start() * Better shutdown process * Comment out test_start() to see if it fixes CI * Make tests pass * Make CI pass * Comment out some tests * Comment out more tests * Re-enable a test * Re-enable another test * Uncomment check_workers test * Added run_worker test * Minor modification to dead worker detection * More test cases * Better process name for workers * Added back pool.stop_workers() when signal is received * Cleaned up cli.py * WIP on worker-pool command * Fix test * Test that worker pool ignores consecutive shutdown signals * Added test for worker-pool CLI command. * Added timeout to CI jobs * Fix worker pool test * Comment out test_scheduler.py * Fixed worker-pool in burst mode * Increase test coverage * Exclude tests directory from coverage.py * Improve test coverage * Renamed `Pool(num_workers=2) to `Pool(size=2)` * Revert "Renamed `Pool(num_workers=2) to `Pool(size=2)`" This reverts commit a1306f89ad0d8686c6bde447bff75e2f71f0733b. * Renamed Pool to WorkerPool * Added a new TestCase that doesn't use LocalStack * Added job_class, worker_class and serializer arguments to WorkerPool * Use parse_connection() in WorkerPool.__init__ * Added CLI arguments for worker-pool * Minor WorkerPool and test fixes * Fixed failing CLI test * Document WorkerPool
2 years ago
"""rq suspend -u <url> --duration=2"""
runner = CliRunner()
result = runner.invoke(main, ['suspend', '-u', self.redis_url, '--duration', 1])
self.assert_normal_execution(result)
def test_suspend_with_invalid_ttl(self):
Worker pool (#1874) * First stab at implementating worker pool * Use process.is_alive() to check whether a process is still live * Handle shutdown signal * Check worker loop done * First working version of `WorkerPool`. * Added test for check_workers() * Added test for pool.start() * Better shutdown process * Comment out test_start() to see if it fixes CI * Make tests pass * Make CI pass * Comment out some tests * Comment out more tests * Re-enable a test * Re-enable another test * Uncomment check_workers test * Added run_worker test * Minor modification to dead worker detection * More test cases * Better process name for workers * Added back pool.stop_workers() when signal is received * Cleaned up cli.py * WIP on worker-pool command * Fix test * Test that worker pool ignores consecutive shutdown signals * Added test for worker-pool CLI command. * Added timeout to CI jobs * Fix worker pool test * Comment out test_scheduler.py * Fixed worker-pool in burst mode * Increase test coverage * Exclude tests directory from coverage.py * Improve test coverage * Renamed `Pool(num_workers=2) to `Pool(size=2)` * Revert "Renamed `Pool(num_workers=2) to `Pool(size=2)`" This reverts commit a1306f89ad0d8686c6bde447bff75e2f71f0733b. * Renamed Pool to WorkerPool * Added a new TestCase that doesn't use LocalStack * Added job_class, worker_class and serializer arguments to WorkerPool * Use parse_connection() in WorkerPool.__init__ * Added CLI arguments for worker-pool * Minor WorkerPool and test fixes * Fixed failing CLI test * Document WorkerPool
2 years ago
"""rq suspend -u <url> --duration=0"""
runner = CliRunner()
result = runner.invoke(main, ['suspend', '-u', self.redis_url, '--duration', 0])
self.assertEqual(result.exit_code, 1)
self.assertIn("Duration must be an integer greater than 1", result.output)
def test_serializer(self):
"""rq worker -u <url> --serializer <serializer>"""
connection = Redis.from_url(self.redis_url)
q = Queue('default', connection=connection, serializer=JSONSerializer)
runner = CliRunner()
job = q.enqueue(say_hello)
Worker pool (#1874) * First stab at implementating worker pool * Use process.is_alive() to check whether a process is still live * Handle shutdown signal * Check worker loop done * First working version of `WorkerPool`. * Added test for check_workers() * Added test for pool.start() * Better shutdown process * Comment out test_start() to see if it fixes CI * Make tests pass * Make CI pass * Comment out some tests * Comment out more tests * Re-enable a test * Re-enable another test * Uncomment check_workers test * Added run_worker test * Minor modification to dead worker detection * More test cases * Better process name for workers * Added back pool.stop_workers() when signal is received * Cleaned up cli.py * WIP on worker-pool command * Fix test * Test that worker pool ignores consecutive shutdown signals * Added test for worker-pool CLI command. * Added timeout to CI jobs * Fix worker pool test * Comment out test_scheduler.py * Fixed worker-pool in burst mode * Increase test coverage * Exclude tests directory from coverage.py * Improve test coverage * Renamed `Pool(num_workers=2) to `Pool(size=2)` * Revert "Renamed `Pool(num_workers=2) to `Pool(size=2)`" This reverts commit a1306f89ad0d8686c6bde447bff75e2f71f0733b. * Renamed Pool to WorkerPool * Added a new TestCase that doesn't use LocalStack * Added job_class, worker_class and serializer arguments to WorkerPool * Use parse_connection() in WorkerPool.__init__ * Added CLI arguments for worker-pool * Minor WorkerPool and test fixes * Fixed failing CLI test * Document WorkerPool
2 years ago
runner.invoke(main, ['worker', '-u', self.redis_url, '--serializer rq.serializer.JSONSerializer'])
self.assertIn(job.id, q.job_ids)
Allows enqueueing by the cli (#1466) * Allows enqueueing by the cli #372 * schedule support * `_` to `-` * fix flake8 * echo job-id * Some improvements - Description as in python jobs - return result - quiet mode - allows `--boolean` and `--integer` - raises errors if not used correctly * added tests * add schedule tests * add retry test * use click exceptions * add error test * add job_func test * change messages https://github.com/rq/rq/pull/1466#discussion_r640211128 https://github.com/rq/rq/pull/1466#discussion_r640210850 * Use different format for arguments View https://github.com/rq/rq/pull/1466#discussion_r650510889 * Add file support Usage: @filename * ast.literal_eval support with `#` instead of `:` * func -> function Makes error messages more readable * click Error * print function string * add docs * increase seconds in test * Update `parse_function_arg` Add `ParsingMode` enum (https://github.com/rq/rq/pull/1466#discussion_r656676114) Change error messages (https://github.com/rq/rq/pull/1466#discussion_r656676800, https://github.com/rq/rq/pull/1466#discussion_r656677082) * `#` to `%` `#` is the letter for a comment in bash * Add some tests (https://github.com/rq/rq/pull/1466#discussion_r656674539, https://github.com/rq/rq/pull/1466#discussion_r656676543) * Add some tests * docs: Add some examples * catch all literal_eval exceptions There are some edge cases with other exceptions * remove job_func (https://github.com/rq/rq/pull/1466#pullrequestreview-690110118) * edit docs https://github.com/rq/rq/pull/1466#pullrequestreview-695758691 * format examples * format examples `queue.enqueue(path.to.func, args=['abc'])` to `queue.enqueue(path.to.func, 'abc')` https://github.com/rq/rq/pull/1466#discussion_r673615464 * add examples https://github.com/rq/rq/pull/1466#discussion_r673658933 * add doc test https://github.com/rq/rq/pull/1466#discussion_r673659124 * Update index.md * Update test_cli.py * Update test_cli.py * Add version info Co-authored-by: rpkak <rpkak@users.noreply.github.com>
3 years ago
def test_cli_enqueue(self):
"""rq enqueue -u <url> tests.fixtures.say_hello"""
queue = Queue(connection=self.connection)
self.assertTrue(queue.is_empty())
runner = CliRunner()
result = runner.invoke(main, ['enqueue', '-u', self.redis_url, 'tests.fixtures.say_hello'])
self.assert_normal_execution(result)
prefix = 'Enqueued tests.fixtures.say_hello() with job-id \''
suffix = '\'.\n'
self.assertTrue(result.output.startswith(prefix))
self.assertTrue(result.output.endswith(suffix))
Allows enqueueing by the cli (#1466) * Allows enqueueing by the cli #372 * schedule support * `_` to `-` * fix flake8 * echo job-id * Some improvements - Description as in python jobs - return result - quiet mode - allows `--boolean` and `--integer` - raises errors if not used correctly * added tests * add schedule tests * add retry test * use click exceptions * add error test * add job_func test * change messages https://github.com/rq/rq/pull/1466#discussion_r640211128 https://github.com/rq/rq/pull/1466#discussion_r640210850 * Use different format for arguments View https://github.com/rq/rq/pull/1466#discussion_r650510889 * Add file support Usage: @filename * ast.literal_eval support with `#` instead of `:` * func -> function Makes error messages more readable * click Error * print function string * add docs * increase seconds in test * Update `parse_function_arg` Add `ParsingMode` enum (https://github.com/rq/rq/pull/1466#discussion_r656676114) Change error messages (https://github.com/rq/rq/pull/1466#discussion_r656676800, https://github.com/rq/rq/pull/1466#discussion_r656677082) * `#` to `%` `#` is the letter for a comment in bash * Add some tests (https://github.com/rq/rq/pull/1466#discussion_r656674539, https://github.com/rq/rq/pull/1466#discussion_r656676543) * Add some tests * docs: Add some examples * catch all literal_eval exceptions There are some edge cases with other exceptions * remove job_func (https://github.com/rq/rq/pull/1466#pullrequestreview-690110118) * edit docs https://github.com/rq/rq/pull/1466#pullrequestreview-695758691 * format examples * format examples `queue.enqueue(path.to.func, args=['abc'])` to `queue.enqueue(path.to.func, 'abc')` https://github.com/rq/rq/pull/1466#discussion_r673615464 * add examples https://github.com/rq/rq/pull/1466#discussion_r673658933 * add doc test https://github.com/rq/rq/pull/1466#discussion_r673659124 * Update index.md * Update test_cli.py * Update test_cli.py * Add version info Co-authored-by: rpkak <rpkak@users.noreply.github.com>
3 years ago
Worker pool (#1874) * First stab at implementating worker pool * Use process.is_alive() to check whether a process is still live * Handle shutdown signal * Check worker loop done * First working version of `WorkerPool`. * Added test for check_workers() * Added test for pool.start() * Better shutdown process * Comment out test_start() to see if it fixes CI * Make tests pass * Make CI pass * Comment out some tests * Comment out more tests * Re-enable a test * Re-enable another test * Uncomment check_workers test * Added run_worker test * Minor modification to dead worker detection * More test cases * Better process name for workers * Added back pool.stop_workers() when signal is received * Cleaned up cli.py * WIP on worker-pool command * Fix test * Test that worker pool ignores consecutive shutdown signals * Added test for worker-pool CLI command. * Added timeout to CI jobs * Fix worker pool test * Comment out test_scheduler.py * Fixed worker-pool in burst mode * Increase test coverage * Exclude tests directory from coverage.py * Improve test coverage * Renamed `Pool(num_workers=2) to `Pool(size=2)` * Revert "Renamed `Pool(num_workers=2) to `Pool(size=2)`" This reverts commit a1306f89ad0d8686c6bde447bff75e2f71f0733b. * Renamed Pool to WorkerPool * Added a new TestCase that doesn't use LocalStack * Added job_class, worker_class and serializer arguments to WorkerPool * Use parse_connection() in WorkerPool.__init__ * Added CLI arguments for worker-pool * Minor WorkerPool and test fixes * Fixed failing CLI test * Document WorkerPool
2 years ago
job_id = result.output[len(prefix) : -len(suffix)]
Allows enqueueing by the cli (#1466) * Allows enqueueing by the cli #372 * schedule support * `_` to `-` * fix flake8 * echo job-id * Some improvements - Description as in python jobs - return result - quiet mode - allows `--boolean` and `--integer` - raises errors if not used correctly * added tests * add schedule tests * add retry test * use click exceptions * add error test * add job_func test * change messages https://github.com/rq/rq/pull/1466#discussion_r640211128 https://github.com/rq/rq/pull/1466#discussion_r640210850 * Use different format for arguments View https://github.com/rq/rq/pull/1466#discussion_r650510889 * Add file support Usage: @filename * ast.literal_eval support with `#` instead of `:` * func -> function Makes error messages more readable * click Error * print function string * add docs * increase seconds in test * Update `parse_function_arg` Add `ParsingMode` enum (https://github.com/rq/rq/pull/1466#discussion_r656676114) Change error messages (https://github.com/rq/rq/pull/1466#discussion_r656676800, https://github.com/rq/rq/pull/1466#discussion_r656677082) * `#` to `%` `#` is the letter for a comment in bash * Add some tests (https://github.com/rq/rq/pull/1466#discussion_r656674539, https://github.com/rq/rq/pull/1466#discussion_r656676543) * Add some tests * docs: Add some examples * catch all literal_eval exceptions There are some edge cases with other exceptions * remove job_func (https://github.com/rq/rq/pull/1466#pullrequestreview-690110118) * edit docs https://github.com/rq/rq/pull/1466#pullrequestreview-695758691 * format examples * format examples `queue.enqueue(path.to.func, args=['abc'])` to `queue.enqueue(path.to.func, 'abc')` https://github.com/rq/rq/pull/1466#discussion_r673615464 * add examples https://github.com/rq/rq/pull/1466#discussion_r673658933 * add doc test https://github.com/rq/rq/pull/1466#discussion_r673659124 * Update index.md * Update test_cli.py * Update test_cli.py * Add version info Co-authored-by: rpkak <rpkak@users.noreply.github.com>
3 years ago
queue_key = 'rq:queue:default'
self.assertEqual(self.connection.llen(queue_key), 1)
self.assertEqual(self.connection.lrange(queue_key, 0, -1)[0].decode('ascii'), job_id)
worker = Worker(queue)
worker.work(True)
self.assertEqual(Job(job_id).result, 'Hi there, Stranger!')
def test_cli_enqueue_with_serializer(self):
"""rq enqueue -u <url> -S rq.serializers.JSONSerializer tests.fixtures.say_hello"""
queue = Queue(connection=self.connection, serializer=JSONSerializer)
self.assertTrue(queue.is_empty())
runner = CliRunner()
Worker pool (#1874) * First stab at implementating worker pool * Use process.is_alive() to check whether a process is still live * Handle shutdown signal * Check worker loop done * First working version of `WorkerPool`. * Added test for check_workers() * Added test for pool.start() * Better shutdown process * Comment out test_start() to see if it fixes CI * Make tests pass * Make CI pass * Comment out some tests * Comment out more tests * Re-enable a test * Re-enable another test * Uncomment check_workers test * Added run_worker test * Minor modification to dead worker detection * More test cases * Better process name for workers * Added back pool.stop_workers() when signal is received * Cleaned up cli.py * WIP on worker-pool command * Fix test * Test that worker pool ignores consecutive shutdown signals * Added test for worker-pool CLI command. * Added timeout to CI jobs * Fix worker pool test * Comment out test_scheduler.py * Fixed worker-pool in burst mode * Increase test coverage * Exclude tests directory from coverage.py * Improve test coverage * Renamed `Pool(num_workers=2) to `Pool(size=2)` * Revert "Renamed `Pool(num_workers=2) to `Pool(size=2)`" This reverts commit a1306f89ad0d8686c6bde447bff75e2f71f0733b. * Renamed Pool to WorkerPool * Added a new TestCase that doesn't use LocalStack * Added job_class, worker_class and serializer arguments to WorkerPool * Use parse_connection() in WorkerPool.__init__ * Added CLI arguments for worker-pool * Minor WorkerPool and test fixes * Fixed failing CLI test * Document WorkerPool
2 years ago
result = runner.invoke(
main, ['enqueue', '-u', self.redis_url, '-S', 'rq.serializers.JSONSerializer', 'tests.fixtures.say_hello']
)
self.assert_normal_execution(result)
prefix = 'Enqueued tests.fixtures.say_hello() with job-id \''
suffix = '\'.\n'
self.assertTrue(result.output.startswith(prefix))
self.assertTrue(result.output.endswith(suffix))
Worker pool (#1874) * First stab at implementating worker pool * Use process.is_alive() to check whether a process is still live * Handle shutdown signal * Check worker loop done * First working version of `WorkerPool`. * Added test for check_workers() * Added test for pool.start() * Better shutdown process * Comment out test_start() to see if it fixes CI * Make tests pass * Make CI pass * Comment out some tests * Comment out more tests * Re-enable a test * Re-enable another test * Uncomment check_workers test * Added run_worker test * Minor modification to dead worker detection * More test cases * Better process name for workers * Added back pool.stop_workers() when signal is received * Cleaned up cli.py * WIP on worker-pool command * Fix test * Test that worker pool ignores consecutive shutdown signals * Added test for worker-pool CLI command. * Added timeout to CI jobs * Fix worker pool test * Comment out test_scheduler.py * Fixed worker-pool in burst mode * Increase test coverage * Exclude tests directory from coverage.py * Improve test coverage * Renamed `Pool(num_workers=2) to `Pool(size=2)` * Revert "Renamed `Pool(num_workers=2) to `Pool(size=2)`" This reverts commit a1306f89ad0d8686c6bde447bff75e2f71f0733b. * Renamed Pool to WorkerPool * Added a new TestCase that doesn't use LocalStack * Added job_class, worker_class and serializer arguments to WorkerPool * Use parse_connection() in WorkerPool.__init__ * Added CLI arguments for worker-pool * Minor WorkerPool and test fixes * Fixed failing CLI test * Document WorkerPool
2 years ago
job_id = result.output[len(prefix) : -len(suffix)]
queue_key = 'rq:queue:default'
self.assertEqual(self.connection.llen(queue_key), 1)
self.assertEqual(self.connection.lrange(queue_key, 0, -1)[0].decode('ascii'), job_id)
worker = Worker(queue, serializer=JSONSerializer)
worker.work(True)
self.assertEqual(Job(job_id, serializer=JSONSerializer).result, 'Hi there, Stranger!')
Allows enqueueing by the cli (#1466) * Allows enqueueing by the cli #372 * schedule support * `_` to `-` * fix flake8 * echo job-id * Some improvements - Description as in python jobs - return result - quiet mode - allows `--boolean` and `--integer` - raises errors if not used correctly * added tests * add schedule tests * add retry test * use click exceptions * add error test * add job_func test * change messages https://github.com/rq/rq/pull/1466#discussion_r640211128 https://github.com/rq/rq/pull/1466#discussion_r640210850 * Use different format for arguments View https://github.com/rq/rq/pull/1466#discussion_r650510889 * Add file support Usage: @filename * ast.literal_eval support with `#` instead of `:` * func -> function Makes error messages more readable * click Error * print function string * add docs * increase seconds in test * Update `parse_function_arg` Add `ParsingMode` enum (https://github.com/rq/rq/pull/1466#discussion_r656676114) Change error messages (https://github.com/rq/rq/pull/1466#discussion_r656676800, https://github.com/rq/rq/pull/1466#discussion_r656677082) * `#` to `%` `#` is the letter for a comment in bash * Add some tests (https://github.com/rq/rq/pull/1466#discussion_r656674539, https://github.com/rq/rq/pull/1466#discussion_r656676543) * Add some tests * docs: Add some examples * catch all literal_eval exceptions There are some edge cases with other exceptions * remove job_func (https://github.com/rq/rq/pull/1466#pullrequestreview-690110118) * edit docs https://github.com/rq/rq/pull/1466#pullrequestreview-695758691 * format examples * format examples `queue.enqueue(path.to.func, args=['abc'])` to `queue.enqueue(path.to.func, 'abc')` https://github.com/rq/rq/pull/1466#discussion_r673615464 * add examples https://github.com/rq/rq/pull/1466#discussion_r673658933 * add doc test https://github.com/rq/rq/pull/1466#discussion_r673659124 * Update index.md * Update test_cli.py * Update test_cli.py * Add version info Co-authored-by: rpkak <rpkak@users.noreply.github.com>
3 years ago
def test_cli_enqueue_args(self):
"""rq enqueue -u <url> tests.fixtures.echo hello ':[1, {"key": "value"}]' json:=["abc"] nojson=def"""
queue = Queue(connection=self.connection)
self.assertTrue(queue.is_empty())
runner = CliRunner()
Worker pool (#1874) * First stab at implementating worker pool * Use process.is_alive() to check whether a process is still live * Handle shutdown signal * Check worker loop done * First working version of `WorkerPool`. * Added test for check_workers() * Added test for pool.start() * Better shutdown process * Comment out test_start() to see if it fixes CI * Make tests pass * Make CI pass * Comment out some tests * Comment out more tests * Re-enable a test * Re-enable another test * Uncomment check_workers test * Added run_worker test * Minor modification to dead worker detection * More test cases * Better process name for workers * Added back pool.stop_workers() when signal is received * Cleaned up cli.py * WIP on worker-pool command * Fix test * Test that worker pool ignores consecutive shutdown signals * Added test for worker-pool CLI command. * Added timeout to CI jobs * Fix worker pool test * Comment out test_scheduler.py * Fixed worker-pool in burst mode * Increase test coverage * Exclude tests directory from coverage.py * Improve test coverage * Renamed `Pool(num_workers=2) to `Pool(size=2)` * Revert "Renamed `Pool(num_workers=2) to `Pool(size=2)`" This reverts commit a1306f89ad0d8686c6bde447bff75e2f71f0733b. * Renamed Pool to WorkerPool * Added a new TestCase that doesn't use LocalStack * Added job_class, worker_class and serializer arguments to WorkerPool * Use parse_connection() in WorkerPool.__init__ * Added CLI arguments for worker-pool * Minor WorkerPool and test fixes * Fixed failing CLI test * Document WorkerPool
2 years ago
result = runner.invoke(
main,
[
'enqueue',
'-u',
self.redis_url,
'tests.fixtures.echo',
'hello',
':[1, {"key": "value"}]',
':@tests/test.json',
'%1, 2',
'json:=[3.0, true]',
'nojson=abc',
'file=@tests/test.json',
],
)
Allows enqueueing by the cli (#1466) * Allows enqueueing by the cli #372 * schedule support * `_` to `-` * fix flake8 * echo job-id * Some improvements - Description as in python jobs - return result - quiet mode - allows `--boolean` and `--integer` - raises errors if not used correctly * added tests * add schedule tests * add retry test * use click exceptions * add error test * add job_func test * change messages https://github.com/rq/rq/pull/1466#discussion_r640211128 https://github.com/rq/rq/pull/1466#discussion_r640210850 * Use different format for arguments View https://github.com/rq/rq/pull/1466#discussion_r650510889 * Add file support Usage: @filename * ast.literal_eval support with `#` instead of `:` * func -> function Makes error messages more readable * click Error * print function string * add docs * increase seconds in test * Update `parse_function_arg` Add `ParsingMode` enum (https://github.com/rq/rq/pull/1466#discussion_r656676114) Change error messages (https://github.com/rq/rq/pull/1466#discussion_r656676800, https://github.com/rq/rq/pull/1466#discussion_r656677082) * `#` to `%` `#` is the letter for a comment in bash * Add some tests (https://github.com/rq/rq/pull/1466#discussion_r656674539, https://github.com/rq/rq/pull/1466#discussion_r656676543) * Add some tests * docs: Add some examples * catch all literal_eval exceptions There are some edge cases with other exceptions * remove job_func (https://github.com/rq/rq/pull/1466#pullrequestreview-690110118) * edit docs https://github.com/rq/rq/pull/1466#pullrequestreview-695758691 * format examples * format examples `queue.enqueue(path.to.func, args=['abc'])` to `queue.enqueue(path.to.func, 'abc')` https://github.com/rq/rq/pull/1466#discussion_r673615464 * add examples https://github.com/rq/rq/pull/1466#discussion_r673658933 * add doc test https://github.com/rq/rq/pull/1466#discussion_r673659124 * Update index.md * Update test_cli.py * Update test_cli.py * Add version info Co-authored-by: rpkak <rpkak@users.noreply.github.com>
3 years ago
self.assert_normal_execution(result)
job_id = self.connection.lrange('rq:queue:default', 0, -1)[0].decode('ascii')
worker = Worker(queue)
worker.work(True)
args, kwargs = Job(job_id).result
self.assertEqual(args, ('hello', [1, {'key': 'value'}], {"test": True}, (1, 2)))
self.assertEqual(kwargs, {'json': [3.0, True], 'nojson': 'abc', 'file': '{\n "test": true\n}\n'})
def test_cli_enqueue_schedule_in(self):
"""rq enqueue -u <url> tests.fixtures.say_hello --schedule-in 1s"""
queue = Queue(connection=self.connection)
registry = ScheduledJobRegistry(queue=queue)
worker = Worker(queue)
scheduler = RQScheduler(queue, self.connection)
self.assertTrue(len(queue) == 0)
self.assertTrue(len(registry) == 0)
runner = CliRunner()
Worker pool (#1874) * First stab at implementating worker pool * Use process.is_alive() to check whether a process is still live * Handle shutdown signal * Check worker loop done * First working version of `WorkerPool`. * Added test for check_workers() * Added test for pool.start() * Better shutdown process * Comment out test_start() to see if it fixes CI * Make tests pass * Make CI pass * Comment out some tests * Comment out more tests * Re-enable a test * Re-enable another test * Uncomment check_workers test * Added run_worker test * Minor modification to dead worker detection * More test cases * Better process name for workers * Added back pool.stop_workers() when signal is received * Cleaned up cli.py * WIP on worker-pool command * Fix test * Test that worker pool ignores consecutive shutdown signals * Added test for worker-pool CLI command. * Added timeout to CI jobs * Fix worker pool test * Comment out test_scheduler.py * Fixed worker-pool in burst mode * Increase test coverage * Exclude tests directory from coverage.py * Improve test coverage * Renamed `Pool(num_workers=2) to `Pool(size=2)` * Revert "Renamed `Pool(num_workers=2) to `Pool(size=2)`" This reverts commit a1306f89ad0d8686c6bde447bff75e2f71f0733b. * Renamed Pool to WorkerPool * Added a new TestCase that doesn't use LocalStack * Added job_class, worker_class and serializer arguments to WorkerPool * Use parse_connection() in WorkerPool.__init__ * Added CLI arguments for worker-pool * Minor WorkerPool and test fixes * Fixed failing CLI test * Document WorkerPool
2 years ago
result = runner.invoke(
main, ['enqueue', '-u', self.redis_url, 'tests.fixtures.say_hello', '--schedule-in', '10s']
)
Allows enqueueing by the cli (#1466) * Allows enqueueing by the cli #372 * schedule support * `_` to `-` * fix flake8 * echo job-id * Some improvements - Description as in python jobs - return result - quiet mode - allows `--boolean` and `--integer` - raises errors if not used correctly * added tests * add schedule tests * add retry test * use click exceptions * add error test * add job_func test * change messages https://github.com/rq/rq/pull/1466#discussion_r640211128 https://github.com/rq/rq/pull/1466#discussion_r640210850 * Use different format for arguments View https://github.com/rq/rq/pull/1466#discussion_r650510889 * Add file support Usage: @filename * ast.literal_eval support with `#` instead of `:` * func -> function Makes error messages more readable * click Error * print function string * add docs * increase seconds in test * Update `parse_function_arg` Add `ParsingMode` enum (https://github.com/rq/rq/pull/1466#discussion_r656676114) Change error messages (https://github.com/rq/rq/pull/1466#discussion_r656676800, https://github.com/rq/rq/pull/1466#discussion_r656677082) * `#` to `%` `#` is the letter for a comment in bash * Add some tests (https://github.com/rq/rq/pull/1466#discussion_r656674539, https://github.com/rq/rq/pull/1466#discussion_r656676543) * Add some tests * docs: Add some examples * catch all literal_eval exceptions There are some edge cases with other exceptions * remove job_func (https://github.com/rq/rq/pull/1466#pullrequestreview-690110118) * edit docs https://github.com/rq/rq/pull/1466#pullrequestreview-695758691 * format examples * format examples `queue.enqueue(path.to.func, args=['abc'])` to `queue.enqueue(path.to.func, 'abc')` https://github.com/rq/rq/pull/1466#discussion_r673615464 * add examples https://github.com/rq/rq/pull/1466#discussion_r673658933 * add doc test https://github.com/rq/rq/pull/1466#discussion_r673659124 * Update index.md * Update test_cli.py * Update test_cli.py * Add version info Co-authored-by: rpkak <rpkak@users.noreply.github.com>
3 years ago
self.assert_normal_execution(result)
scheduler.acquire_locks()
scheduler.enqueue_scheduled_jobs()
self.assertTrue(len(queue) == 0)
self.assertTrue(len(registry) == 1)
self.assertFalse(worker.work(True))
sleep(11)
scheduler.enqueue_scheduled_jobs()
self.assertTrue(len(queue) == 1)
self.assertTrue(len(registry) == 0)
self.assertTrue(worker.work(True))
def test_cli_enqueue_schedule_at(self):
"""
rq enqueue -u <url> tests.fixtures.say_hello --schedule-at 2021-01-01T00:00:00
rq enqueue -u <url> tests.fixtures.say_hello --schedule-at 2100-01-01T00:00:00
"""
queue = Queue(connection=self.connection)
registry = ScheduledJobRegistry(queue=queue)
worker = Worker(queue)
scheduler = RQScheduler(queue, self.connection)
self.assertTrue(len(queue) == 0)
self.assertTrue(len(registry) == 0)
runner = CliRunner()
Worker pool (#1874) * First stab at implementating worker pool * Use process.is_alive() to check whether a process is still live * Handle shutdown signal * Check worker loop done * First working version of `WorkerPool`. * Added test for check_workers() * Added test for pool.start() * Better shutdown process * Comment out test_start() to see if it fixes CI * Make tests pass * Make CI pass * Comment out some tests * Comment out more tests * Re-enable a test * Re-enable another test * Uncomment check_workers test * Added run_worker test * Minor modification to dead worker detection * More test cases * Better process name for workers * Added back pool.stop_workers() when signal is received * Cleaned up cli.py * WIP on worker-pool command * Fix test * Test that worker pool ignores consecutive shutdown signals * Added test for worker-pool CLI command. * Added timeout to CI jobs * Fix worker pool test * Comment out test_scheduler.py * Fixed worker-pool in burst mode * Increase test coverage * Exclude tests directory from coverage.py * Improve test coverage * Renamed `Pool(num_workers=2) to `Pool(size=2)` * Revert "Renamed `Pool(num_workers=2) to `Pool(size=2)`" This reverts commit a1306f89ad0d8686c6bde447bff75e2f71f0733b. * Renamed Pool to WorkerPool * Added a new TestCase that doesn't use LocalStack * Added job_class, worker_class and serializer arguments to WorkerPool * Use parse_connection() in WorkerPool.__init__ * Added CLI arguments for worker-pool * Minor WorkerPool and test fixes * Fixed failing CLI test * Document WorkerPool
2 years ago
result = runner.invoke(
main, ['enqueue', '-u', self.redis_url, 'tests.fixtures.say_hello', '--schedule-at', '2021-01-01T00:00:00']
)
Allows enqueueing by the cli (#1466) * Allows enqueueing by the cli #372 * schedule support * `_` to `-` * fix flake8 * echo job-id * Some improvements - Description as in python jobs - return result - quiet mode - allows `--boolean` and `--integer` - raises errors if not used correctly * added tests * add schedule tests * add retry test * use click exceptions * add error test * add job_func test * change messages https://github.com/rq/rq/pull/1466#discussion_r640211128 https://github.com/rq/rq/pull/1466#discussion_r640210850 * Use different format for arguments View https://github.com/rq/rq/pull/1466#discussion_r650510889 * Add file support Usage: @filename * ast.literal_eval support with `#` instead of `:` * func -> function Makes error messages more readable * click Error * print function string * add docs * increase seconds in test * Update `parse_function_arg` Add `ParsingMode` enum (https://github.com/rq/rq/pull/1466#discussion_r656676114) Change error messages (https://github.com/rq/rq/pull/1466#discussion_r656676800, https://github.com/rq/rq/pull/1466#discussion_r656677082) * `#` to `%` `#` is the letter for a comment in bash * Add some tests (https://github.com/rq/rq/pull/1466#discussion_r656674539, https://github.com/rq/rq/pull/1466#discussion_r656676543) * Add some tests * docs: Add some examples * catch all literal_eval exceptions There are some edge cases with other exceptions * remove job_func (https://github.com/rq/rq/pull/1466#pullrequestreview-690110118) * edit docs https://github.com/rq/rq/pull/1466#pullrequestreview-695758691 * format examples * format examples `queue.enqueue(path.to.func, args=['abc'])` to `queue.enqueue(path.to.func, 'abc')` https://github.com/rq/rq/pull/1466#discussion_r673615464 * add examples https://github.com/rq/rq/pull/1466#discussion_r673658933 * add doc test https://github.com/rq/rq/pull/1466#discussion_r673659124 * Update index.md * Update test_cli.py * Update test_cli.py * Add version info Co-authored-by: rpkak <rpkak@users.noreply.github.com>
3 years ago
self.assert_normal_execution(result)
scheduler.acquire_locks()
self.assertTrue(len(queue) == 0)
self.assertTrue(len(registry) == 1)
scheduler.enqueue_scheduled_jobs()
self.assertTrue(len(queue) == 1)
self.assertTrue(len(registry) == 0)
self.assertTrue(worker.work(True))
self.assertTrue(len(queue) == 0)
self.assertTrue(len(registry) == 0)
Worker pool (#1874) * First stab at implementating worker pool * Use process.is_alive() to check whether a process is still live * Handle shutdown signal * Check worker loop done * First working version of `WorkerPool`. * Added test for check_workers() * Added test for pool.start() * Better shutdown process * Comment out test_start() to see if it fixes CI * Make tests pass * Make CI pass * Comment out some tests * Comment out more tests * Re-enable a test * Re-enable another test * Uncomment check_workers test * Added run_worker test * Minor modification to dead worker detection * More test cases * Better process name for workers * Added back pool.stop_workers() when signal is received * Cleaned up cli.py * WIP on worker-pool command * Fix test * Test that worker pool ignores consecutive shutdown signals * Added test for worker-pool CLI command. * Added timeout to CI jobs * Fix worker pool test * Comment out test_scheduler.py * Fixed worker-pool in burst mode * Increase test coverage * Exclude tests directory from coverage.py * Improve test coverage * Renamed `Pool(num_workers=2) to `Pool(size=2)` * Revert "Renamed `Pool(num_workers=2) to `Pool(size=2)`" This reverts commit a1306f89ad0d8686c6bde447bff75e2f71f0733b. * Renamed Pool to WorkerPool * Added a new TestCase that doesn't use LocalStack * Added job_class, worker_class and serializer arguments to WorkerPool * Use parse_connection() in WorkerPool.__init__ * Added CLI arguments for worker-pool * Minor WorkerPool and test fixes * Fixed failing CLI test * Document WorkerPool
2 years ago
result = runner.invoke(
main, ['enqueue', '-u', self.redis_url, 'tests.fixtures.say_hello', '--schedule-at', '2100-01-01T00:00:00']
)
Allows enqueueing by the cli (#1466) * Allows enqueueing by the cli #372 * schedule support * `_` to `-` * fix flake8 * echo job-id * Some improvements - Description as in python jobs - return result - quiet mode - allows `--boolean` and `--integer` - raises errors if not used correctly * added tests * add schedule tests * add retry test * use click exceptions * add error test * add job_func test * change messages https://github.com/rq/rq/pull/1466#discussion_r640211128 https://github.com/rq/rq/pull/1466#discussion_r640210850 * Use different format for arguments View https://github.com/rq/rq/pull/1466#discussion_r650510889 * Add file support Usage: @filename * ast.literal_eval support with `#` instead of `:` * func -> function Makes error messages more readable * click Error * print function string * add docs * increase seconds in test * Update `parse_function_arg` Add `ParsingMode` enum (https://github.com/rq/rq/pull/1466#discussion_r656676114) Change error messages (https://github.com/rq/rq/pull/1466#discussion_r656676800, https://github.com/rq/rq/pull/1466#discussion_r656677082) * `#` to `%` `#` is the letter for a comment in bash * Add some tests (https://github.com/rq/rq/pull/1466#discussion_r656674539, https://github.com/rq/rq/pull/1466#discussion_r656676543) * Add some tests * docs: Add some examples * catch all literal_eval exceptions There are some edge cases with other exceptions * remove job_func (https://github.com/rq/rq/pull/1466#pullrequestreview-690110118) * edit docs https://github.com/rq/rq/pull/1466#pullrequestreview-695758691 * format examples * format examples `queue.enqueue(path.to.func, args=['abc'])` to `queue.enqueue(path.to.func, 'abc')` https://github.com/rq/rq/pull/1466#discussion_r673615464 * add examples https://github.com/rq/rq/pull/1466#discussion_r673658933 * add doc test https://github.com/rq/rq/pull/1466#discussion_r673659124 * Update index.md * Update test_cli.py * Update test_cli.py * Add version info Co-authored-by: rpkak <rpkak@users.noreply.github.com>
3 years ago
self.assert_normal_execution(result)
self.assertTrue(len(queue) == 0)
self.assertTrue(len(registry) == 1)
scheduler.enqueue_scheduled_jobs()
self.assertTrue(len(queue) == 0)
self.assertTrue(len(registry) == 1)
self.assertFalse(worker.work(True))
def test_cli_enqueue_retry(self):
"""rq enqueue -u <url> tests.fixtures.say_hello --retry-max 3 --retry-interval 10 --retry-interval 20
--retry-interval 40"""
queue = Queue(connection=self.connection)
self.assertTrue(queue.is_empty())
runner = CliRunner()
Worker pool (#1874) * First stab at implementating worker pool * Use process.is_alive() to check whether a process is still live * Handle shutdown signal * Check worker loop done * First working version of `WorkerPool`. * Added test for check_workers() * Added test for pool.start() * Better shutdown process * Comment out test_start() to see if it fixes CI * Make tests pass * Make CI pass * Comment out some tests * Comment out more tests * Re-enable a test * Re-enable another test * Uncomment check_workers test * Added run_worker test * Minor modification to dead worker detection * More test cases * Better process name for workers * Added back pool.stop_workers() when signal is received * Cleaned up cli.py * WIP on worker-pool command * Fix test * Test that worker pool ignores consecutive shutdown signals * Added test for worker-pool CLI command. * Added timeout to CI jobs * Fix worker pool test * Comment out test_scheduler.py * Fixed worker-pool in burst mode * Increase test coverage * Exclude tests directory from coverage.py * Improve test coverage * Renamed `Pool(num_workers=2) to `Pool(size=2)` * Revert "Renamed `Pool(num_workers=2) to `Pool(size=2)`" This reverts commit a1306f89ad0d8686c6bde447bff75e2f71f0733b. * Renamed Pool to WorkerPool * Added a new TestCase that doesn't use LocalStack * Added job_class, worker_class and serializer arguments to WorkerPool * Use parse_connection() in WorkerPool.__init__ * Added CLI arguments for worker-pool * Minor WorkerPool and test fixes * Fixed failing CLI test * Document WorkerPool
2 years ago
result = runner.invoke(
main,
[
'enqueue',
'-u',
self.redis_url,
'tests.fixtures.say_hello',
'--retry-max',
'3',
'--retry-interval',
'10',
'--retry-interval',
'20',
'--retry-interval',
'40',
],
)
Allows enqueueing by the cli (#1466) * Allows enqueueing by the cli #372 * schedule support * `_` to `-` * fix flake8 * echo job-id * Some improvements - Description as in python jobs - return result - quiet mode - allows `--boolean` and `--integer` - raises errors if not used correctly * added tests * add schedule tests * add retry test * use click exceptions * add error test * add job_func test * change messages https://github.com/rq/rq/pull/1466#discussion_r640211128 https://github.com/rq/rq/pull/1466#discussion_r640210850 * Use different format for arguments View https://github.com/rq/rq/pull/1466#discussion_r650510889 * Add file support Usage: @filename * ast.literal_eval support with `#` instead of `:` * func -> function Makes error messages more readable * click Error * print function string * add docs * increase seconds in test * Update `parse_function_arg` Add `ParsingMode` enum (https://github.com/rq/rq/pull/1466#discussion_r656676114) Change error messages (https://github.com/rq/rq/pull/1466#discussion_r656676800, https://github.com/rq/rq/pull/1466#discussion_r656677082) * `#` to `%` `#` is the letter for a comment in bash * Add some tests (https://github.com/rq/rq/pull/1466#discussion_r656674539, https://github.com/rq/rq/pull/1466#discussion_r656676543) * Add some tests * docs: Add some examples * catch all literal_eval exceptions There are some edge cases with other exceptions * remove job_func (https://github.com/rq/rq/pull/1466#pullrequestreview-690110118) * edit docs https://github.com/rq/rq/pull/1466#pullrequestreview-695758691 * format examples * format examples `queue.enqueue(path.to.func, args=['abc'])` to `queue.enqueue(path.to.func, 'abc')` https://github.com/rq/rq/pull/1466#discussion_r673615464 * add examples https://github.com/rq/rq/pull/1466#discussion_r673658933 * add doc test https://github.com/rq/rq/pull/1466#discussion_r673659124 * Update index.md * Update test_cli.py * Update test_cli.py * Add version info Co-authored-by: rpkak <rpkak@users.noreply.github.com>
3 years ago
self.assert_normal_execution(result)
Worker pool (#1874) * First stab at implementating worker pool * Use process.is_alive() to check whether a process is still live * Handle shutdown signal * Check worker loop done * First working version of `WorkerPool`. * Added test for check_workers() * Added test for pool.start() * Better shutdown process * Comment out test_start() to see if it fixes CI * Make tests pass * Make CI pass * Comment out some tests * Comment out more tests * Re-enable a test * Re-enable another test * Uncomment check_workers test * Added run_worker test * Minor modification to dead worker detection * More test cases * Better process name for workers * Added back pool.stop_workers() when signal is received * Cleaned up cli.py * WIP on worker-pool command * Fix test * Test that worker pool ignores consecutive shutdown signals * Added test for worker-pool CLI command. * Added timeout to CI jobs * Fix worker pool test * Comment out test_scheduler.py * Fixed worker-pool in burst mode * Increase test coverage * Exclude tests directory from coverage.py * Improve test coverage * Renamed `Pool(num_workers=2) to `Pool(size=2)` * Revert "Renamed `Pool(num_workers=2) to `Pool(size=2)`" This reverts commit a1306f89ad0d8686c6bde447bff75e2f71f0733b. * Renamed Pool to WorkerPool * Added a new TestCase that doesn't use LocalStack * Added job_class, worker_class and serializer arguments to WorkerPool * Use parse_connection() in WorkerPool.__init__ * Added CLI arguments for worker-pool * Minor WorkerPool and test fixes * Fixed failing CLI test * Document WorkerPool
2 years ago
job = Job.fetch(
self.connection.lrange('rq:queue:default', 0, -1)[0].decode('ascii'), connection=self.connection
)
Allows enqueueing by the cli (#1466) * Allows enqueueing by the cli #372 * schedule support * `_` to `-` * fix flake8 * echo job-id * Some improvements - Description as in python jobs - return result - quiet mode - allows `--boolean` and `--integer` - raises errors if not used correctly * added tests * add schedule tests * add retry test * use click exceptions * add error test * add job_func test * change messages https://github.com/rq/rq/pull/1466#discussion_r640211128 https://github.com/rq/rq/pull/1466#discussion_r640210850 * Use different format for arguments View https://github.com/rq/rq/pull/1466#discussion_r650510889 * Add file support Usage: @filename * ast.literal_eval support with `#` instead of `:` * func -> function Makes error messages more readable * click Error * print function string * add docs * increase seconds in test * Update `parse_function_arg` Add `ParsingMode` enum (https://github.com/rq/rq/pull/1466#discussion_r656676114) Change error messages (https://github.com/rq/rq/pull/1466#discussion_r656676800, https://github.com/rq/rq/pull/1466#discussion_r656677082) * `#` to `%` `#` is the letter for a comment in bash * Add some tests (https://github.com/rq/rq/pull/1466#discussion_r656674539, https://github.com/rq/rq/pull/1466#discussion_r656676543) * Add some tests * docs: Add some examples * catch all literal_eval exceptions There are some edge cases with other exceptions * remove job_func (https://github.com/rq/rq/pull/1466#pullrequestreview-690110118) * edit docs https://github.com/rq/rq/pull/1466#pullrequestreview-695758691 * format examples * format examples `queue.enqueue(path.to.func, args=['abc'])` to `queue.enqueue(path.to.func, 'abc')` https://github.com/rq/rq/pull/1466#discussion_r673615464 * add examples https://github.com/rq/rq/pull/1466#discussion_r673658933 * add doc test https://github.com/rq/rq/pull/1466#discussion_r673659124 * Update index.md * Update test_cli.py * Update test_cli.py * Add version info Co-authored-by: rpkak <rpkak@users.noreply.github.com>
3 years ago
self.assertEqual(job.retries_left, 3)
self.assertEqual(job.retry_intervals, [10, 20, 40])
def test_cli_enqueue_errors(self):
"""
rq enqueue -u <url> tests.fixtures.echo :invalid_json
rq enqueue -u <url> tests.fixtures.echo %invalid_eval_statement
rq enqueue -u <url> tests.fixtures.echo key=value key=value
rq enqueue -u <url> tests.fixtures.echo --schedule-in 1s --schedule-at 2000-01-01T00:00:00
rq enqueue -u <url> tests.fixtures.echo @not_existing_file
"""
runner = CliRunner()
result = runner.invoke(main, ['enqueue', '-u', self.redis_url, 'tests.fixtures.echo', ':invalid_json'])
self.assertNotEqual(result.exit_code, 0)
self.assertIn('Unable to parse 1. non keyword argument as JSON.', result.output)
Worker pool (#1874) * First stab at implementating worker pool * Use process.is_alive() to check whether a process is still live * Handle shutdown signal * Check worker loop done * First working version of `WorkerPool`. * Added test for check_workers() * Added test for pool.start() * Better shutdown process * Comment out test_start() to see if it fixes CI * Make tests pass * Make CI pass * Comment out some tests * Comment out more tests * Re-enable a test * Re-enable another test * Uncomment check_workers test * Added run_worker test * Minor modification to dead worker detection * More test cases * Better process name for workers * Added back pool.stop_workers() when signal is received * Cleaned up cli.py * WIP on worker-pool command * Fix test * Test that worker pool ignores consecutive shutdown signals * Added test for worker-pool CLI command. * Added timeout to CI jobs * Fix worker pool test * Comment out test_scheduler.py * Fixed worker-pool in burst mode * Increase test coverage * Exclude tests directory from coverage.py * Improve test coverage * Renamed `Pool(num_workers=2) to `Pool(size=2)` * Revert "Renamed `Pool(num_workers=2) to `Pool(size=2)`" This reverts commit a1306f89ad0d8686c6bde447bff75e2f71f0733b. * Renamed Pool to WorkerPool * Added a new TestCase that doesn't use LocalStack * Added job_class, worker_class and serializer arguments to WorkerPool * Use parse_connection() in WorkerPool.__init__ * Added CLI arguments for worker-pool * Minor WorkerPool and test fixes * Fixed failing CLI test * Document WorkerPool
2 years ago
result = runner.invoke(
main, ['enqueue', '-u', self.redis_url, 'tests.fixtures.echo', '%invalid_eval_statement']
)
Allows enqueueing by the cli (#1466) * Allows enqueueing by the cli #372 * schedule support * `_` to `-` * fix flake8 * echo job-id * Some improvements - Description as in python jobs - return result - quiet mode - allows `--boolean` and `--integer` - raises errors if not used correctly * added tests * add schedule tests * add retry test * use click exceptions * add error test * add job_func test * change messages https://github.com/rq/rq/pull/1466#discussion_r640211128 https://github.com/rq/rq/pull/1466#discussion_r640210850 * Use different format for arguments View https://github.com/rq/rq/pull/1466#discussion_r650510889 * Add file support Usage: @filename * ast.literal_eval support with `#` instead of `:` * func -> function Makes error messages more readable * click Error * print function string * add docs * increase seconds in test * Update `parse_function_arg` Add `ParsingMode` enum (https://github.com/rq/rq/pull/1466#discussion_r656676114) Change error messages (https://github.com/rq/rq/pull/1466#discussion_r656676800, https://github.com/rq/rq/pull/1466#discussion_r656677082) * `#` to `%` `#` is the letter for a comment in bash * Add some tests (https://github.com/rq/rq/pull/1466#discussion_r656674539, https://github.com/rq/rq/pull/1466#discussion_r656676543) * Add some tests * docs: Add some examples * catch all literal_eval exceptions There are some edge cases with other exceptions * remove job_func (https://github.com/rq/rq/pull/1466#pullrequestreview-690110118) * edit docs https://github.com/rq/rq/pull/1466#pullrequestreview-695758691 * format examples * format examples `queue.enqueue(path.to.func, args=['abc'])` to `queue.enqueue(path.to.func, 'abc')` https://github.com/rq/rq/pull/1466#discussion_r673615464 * add examples https://github.com/rq/rq/pull/1466#discussion_r673658933 * add doc test https://github.com/rq/rq/pull/1466#discussion_r673659124 * Update index.md * Update test_cli.py * Update test_cli.py * Add version info Co-authored-by: rpkak <rpkak@users.noreply.github.com>
3 years ago
self.assertNotEqual(result.exit_code, 0)
self.assertIn('Unable to eval 1. non keyword argument as Python object.', result.output)
result = runner.invoke(main, ['enqueue', '-u', self.redis_url, 'tests.fixtures.echo', 'key=value', 'key=value'])
self.assertNotEqual(result.exit_code, 0)
self.assertIn('You can\'t specify multiple values for the same keyword.', result.output)
Worker pool (#1874) * First stab at implementating worker pool * Use process.is_alive() to check whether a process is still live * Handle shutdown signal * Check worker loop done * First working version of `WorkerPool`. * Added test for check_workers() * Added test for pool.start() * Better shutdown process * Comment out test_start() to see if it fixes CI * Make tests pass * Make CI pass * Comment out some tests * Comment out more tests * Re-enable a test * Re-enable another test * Uncomment check_workers test * Added run_worker test * Minor modification to dead worker detection * More test cases * Better process name for workers * Added back pool.stop_workers() when signal is received * Cleaned up cli.py * WIP on worker-pool command * Fix test * Test that worker pool ignores consecutive shutdown signals * Added test for worker-pool CLI command. * Added timeout to CI jobs * Fix worker pool test * Comment out test_scheduler.py * Fixed worker-pool in burst mode * Increase test coverage * Exclude tests directory from coverage.py * Improve test coverage * Renamed `Pool(num_workers=2) to `Pool(size=2)` * Revert "Renamed `Pool(num_workers=2) to `Pool(size=2)`" This reverts commit a1306f89ad0d8686c6bde447bff75e2f71f0733b. * Renamed Pool to WorkerPool * Added a new TestCase that doesn't use LocalStack * Added job_class, worker_class and serializer arguments to WorkerPool * Use parse_connection() in WorkerPool.__init__ * Added CLI arguments for worker-pool * Minor WorkerPool and test fixes * Fixed failing CLI test * Document WorkerPool
2 years ago
result = runner.invoke(
main,
[
'enqueue',
'-u',
self.redis_url,
'tests.fixtures.echo',
'--schedule-in',
'1s',
'--schedule-at',
'2000-01-01T00:00:00',
],
)
Allows enqueueing by the cli (#1466) * Allows enqueueing by the cli #372 * schedule support * `_` to `-` * fix flake8 * echo job-id * Some improvements - Description as in python jobs - return result - quiet mode - allows `--boolean` and `--integer` - raises errors if not used correctly * added tests * add schedule tests * add retry test * use click exceptions * add error test * add job_func test * change messages https://github.com/rq/rq/pull/1466#discussion_r640211128 https://github.com/rq/rq/pull/1466#discussion_r640210850 * Use different format for arguments View https://github.com/rq/rq/pull/1466#discussion_r650510889 * Add file support Usage: @filename * ast.literal_eval support with `#` instead of `:` * func -> function Makes error messages more readable * click Error * print function string * add docs * increase seconds in test * Update `parse_function_arg` Add `ParsingMode` enum (https://github.com/rq/rq/pull/1466#discussion_r656676114) Change error messages (https://github.com/rq/rq/pull/1466#discussion_r656676800, https://github.com/rq/rq/pull/1466#discussion_r656677082) * `#` to `%` `#` is the letter for a comment in bash * Add some tests (https://github.com/rq/rq/pull/1466#discussion_r656674539, https://github.com/rq/rq/pull/1466#discussion_r656676543) * Add some tests * docs: Add some examples * catch all literal_eval exceptions There are some edge cases with other exceptions * remove job_func (https://github.com/rq/rq/pull/1466#pullrequestreview-690110118) * edit docs https://github.com/rq/rq/pull/1466#pullrequestreview-695758691 * format examples * format examples `queue.enqueue(path.to.func, args=['abc'])` to `queue.enqueue(path.to.func, 'abc')` https://github.com/rq/rq/pull/1466#discussion_r673615464 * add examples https://github.com/rq/rq/pull/1466#discussion_r673658933 * add doc test https://github.com/rq/rq/pull/1466#discussion_r673659124 * Update index.md * Update test_cli.py * Update test_cli.py * Add version info Co-authored-by: rpkak <rpkak@users.noreply.github.com>
3 years ago
self.assertNotEqual(result.exit_code, 0)
self.assertIn('You can\'t specify both --schedule-in and --schedule-at', result.output)
result = runner.invoke(main, ['enqueue', '-u', self.redis_url, 'tests.fixtures.echo', '@not_existing_file'])
self.assertNotEqual(result.exit_code, 0)
self.assertIn('Not found', result.output)
def test_parse_schedule(self):
"""executes the rq.cli.helpers.parse_schedule function"""
self.assertEqual(parse_schedule(None, '2000-01-23T23:45:01'), datetime(2000, 1, 23, 23, 45, 1))
start = datetime.now(timezone.utc) + timedelta(minutes=5)
middle = parse_schedule('5m', None)
end = datetime.now(timezone.utc) + timedelta(minutes=5)
self.assertGreater(middle, start)
self.assertLess(middle, end)
def test_parse_function_arg(self):
"""executes the rq.cli.helpers.parse_function_arg function"""
self.assertEqual(parse_function_arg('abc', 0), (None, 'abc'))
self.assertEqual(parse_function_arg(':{"json": true}', 1), (None, {'json': True}))
self.assertEqual(parse_function_arg('%1, 2', 2), (None, (1, 2)))
self.assertEqual(parse_function_arg('key=value', 3), ('key', 'value'))
self.assertEqual(parse_function_arg('jsonkey:=["json", "value"]', 4), ('jsonkey', ['json', 'value']))
self.assertEqual(parse_function_arg('evalkey%=1.2', 5), ('evalkey', 1.2))
self.assertEqual(parse_function_arg(':@tests/test.json', 6), (None, {'test': True}))
self.assertEqual(parse_function_arg('@tests/test.json', 7), (None, '{\n "test": true\n}\n'))
def test_cli_enqueue_doc_test(self):
"""tests the examples of the documentation"""
runner = CliRunner()
id = str(uuid4())
result = runner.invoke(main, ['enqueue', '-u', self.redis_url, '--job-id', id, 'tests.fixtures.echo', 'abc'])
self.assert_normal_execution(result)
job = Job.fetch(id)
self.assertEqual((job.args, job.kwargs), (['abc'], {}))
id = str(uuid4())
Worker pool (#1874) * First stab at implementating worker pool * Use process.is_alive() to check whether a process is still live * Handle shutdown signal * Check worker loop done * First working version of `WorkerPool`. * Added test for check_workers() * Added test for pool.start() * Better shutdown process * Comment out test_start() to see if it fixes CI * Make tests pass * Make CI pass * Comment out some tests * Comment out more tests * Re-enable a test * Re-enable another test * Uncomment check_workers test * Added run_worker test * Minor modification to dead worker detection * More test cases * Better process name for workers * Added back pool.stop_workers() when signal is received * Cleaned up cli.py * WIP on worker-pool command * Fix test * Test that worker pool ignores consecutive shutdown signals * Added test for worker-pool CLI command. * Added timeout to CI jobs * Fix worker pool test * Comment out test_scheduler.py * Fixed worker-pool in burst mode * Increase test coverage * Exclude tests directory from coverage.py * Improve test coverage * Renamed `Pool(num_workers=2) to `Pool(size=2)` * Revert "Renamed `Pool(num_workers=2) to `Pool(size=2)`" This reverts commit a1306f89ad0d8686c6bde447bff75e2f71f0733b. * Renamed Pool to WorkerPool * Added a new TestCase that doesn't use LocalStack * Added job_class, worker_class and serializer arguments to WorkerPool * Use parse_connection() in WorkerPool.__init__ * Added CLI arguments for worker-pool * Minor WorkerPool and test fixes * Fixed failing CLI test * Document WorkerPool
2 years ago
result = runner.invoke(
main, ['enqueue', '-u', self.redis_url, '--job-id', id, 'tests.fixtures.echo', 'abc=def']
)
Allows enqueueing by the cli (#1466) * Allows enqueueing by the cli #372 * schedule support * `_` to `-` * fix flake8 * echo job-id * Some improvements - Description as in python jobs - return result - quiet mode - allows `--boolean` and `--integer` - raises errors if not used correctly * added tests * add schedule tests * add retry test * use click exceptions * add error test * add job_func test * change messages https://github.com/rq/rq/pull/1466#discussion_r640211128 https://github.com/rq/rq/pull/1466#discussion_r640210850 * Use different format for arguments View https://github.com/rq/rq/pull/1466#discussion_r650510889 * Add file support Usage: @filename * ast.literal_eval support with `#` instead of `:` * func -> function Makes error messages more readable * click Error * print function string * add docs * increase seconds in test * Update `parse_function_arg` Add `ParsingMode` enum (https://github.com/rq/rq/pull/1466#discussion_r656676114) Change error messages (https://github.com/rq/rq/pull/1466#discussion_r656676800, https://github.com/rq/rq/pull/1466#discussion_r656677082) * `#` to `%` `#` is the letter for a comment in bash * Add some tests (https://github.com/rq/rq/pull/1466#discussion_r656674539, https://github.com/rq/rq/pull/1466#discussion_r656676543) * Add some tests * docs: Add some examples * catch all literal_eval exceptions There are some edge cases with other exceptions * remove job_func (https://github.com/rq/rq/pull/1466#pullrequestreview-690110118) * edit docs https://github.com/rq/rq/pull/1466#pullrequestreview-695758691 * format examples * format examples `queue.enqueue(path.to.func, args=['abc'])` to `queue.enqueue(path.to.func, 'abc')` https://github.com/rq/rq/pull/1466#discussion_r673615464 * add examples https://github.com/rq/rq/pull/1466#discussion_r673658933 * add doc test https://github.com/rq/rq/pull/1466#discussion_r673659124 * Update index.md * Update test_cli.py * Update test_cli.py * Add version info Co-authored-by: rpkak <rpkak@users.noreply.github.com>
3 years ago
self.assert_normal_execution(result)
job = Job.fetch(id)
self.assertEqual((job.args, job.kwargs), ([], {'abc': 'def'}))
id = str(uuid4())
Worker pool (#1874) * First stab at implementating worker pool * Use process.is_alive() to check whether a process is still live * Handle shutdown signal * Check worker loop done * First working version of `WorkerPool`. * Added test for check_workers() * Added test for pool.start() * Better shutdown process * Comment out test_start() to see if it fixes CI * Make tests pass * Make CI pass * Comment out some tests * Comment out more tests * Re-enable a test * Re-enable another test * Uncomment check_workers test * Added run_worker test * Minor modification to dead worker detection * More test cases * Better process name for workers * Added back pool.stop_workers() when signal is received * Cleaned up cli.py * WIP on worker-pool command * Fix test * Test that worker pool ignores consecutive shutdown signals * Added test for worker-pool CLI command. * Added timeout to CI jobs * Fix worker pool test * Comment out test_scheduler.py * Fixed worker-pool in burst mode * Increase test coverage * Exclude tests directory from coverage.py * Improve test coverage * Renamed `Pool(num_workers=2) to `Pool(size=2)` * Revert "Renamed `Pool(num_workers=2) to `Pool(size=2)`" This reverts commit a1306f89ad0d8686c6bde447bff75e2f71f0733b. * Renamed Pool to WorkerPool * Added a new TestCase that doesn't use LocalStack * Added job_class, worker_class and serializer arguments to WorkerPool * Use parse_connection() in WorkerPool.__init__ * Added CLI arguments for worker-pool * Minor WorkerPool and test fixes * Fixed failing CLI test * Document WorkerPool
2 years ago
result = runner.invoke(
main, ['enqueue', '-u', self.redis_url, '--job-id', id, 'tests.fixtures.echo', ':{"json": "abc"}']
)
Allows enqueueing by the cli (#1466) * Allows enqueueing by the cli #372 * schedule support * `_` to `-` * fix flake8 * echo job-id * Some improvements - Description as in python jobs - return result - quiet mode - allows `--boolean` and `--integer` - raises errors if not used correctly * added tests * add schedule tests * add retry test * use click exceptions * add error test * add job_func test * change messages https://github.com/rq/rq/pull/1466#discussion_r640211128 https://github.com/rq/rq/pull/1466#discussion_r640210850 * Use different format for arguments View https://github.com/rq/rq/pull/1466#discussion_r650510889 * Add file support Usage: @filename * ast.literal_eval support with `#` instead of `:` * func -> function Makes error messages more readable * click Error * print function string * add docs * increase seconds in test * Update `parse_function_arg` Add `ParsingMode` enum (https://github.com/rq/rq/pull/1466#discussion_r656676114) Change error messages (https://github.com/rq/rq/pull/1466#discussion_r656676800, https://github.com/rq/rq/pull/1466#discussion_r656677082) * `#` to `%` `#` is the letter for a comment in bash * Add some tests (https://github.com/rq/rq/pull/1466#discussion_r656674539, https://github.com/rq/rq/pull/1466#discussion_r656676543) * Add some tests * docs: Add some examples * catch all literal_eval exceptions There are some edge cases with other exceptions * remove job_func (https://github.com/rq/rq/pull/1466#pullrequestreview-690110118) * edit docs https://github.com/rq/rq/pull/1466#pullrequestreview-695758691 * format examples * format examples `queue.enqueue(path.to.func, args=['abc'])` to `queue.enqueue(path.to.func, 'abc')` https://github.com/rq/rq/pull/1466#discussion_r673615464 * add examples https://github.com/rq/rq/pull/1466#discussion_r673658933 * add doc test https://github.com/rq/rq/pull/1466#discussion_r673659124 * Update index.md * Update test_cli.py * Update test_cli.py * Add version info Co-authored-by: rpkak <rpkak@users.noreply.github.com>
3 years ago
self.assert_normal_execution(result)
job = Job.fetch(id)
self.assertEqual((job.args, job.kwargs), ([{'json': 'abc'}], {}))
id = str(uuid4())
Worker pool (#1874) * First stab at implementating worker pool * Use process.is_alive() to check whether a process is still live * Handle shutdown signal * Check worker loop done * First working version of `WorkerPool`. * Added test for check_workers() * Added test for pool.start() * Better shutdown process * Comment out test_start() to see if it fixes CI * Make tests pass * Make CI pass * Comment out some tests * Comment out more tests * Re-enable a test * Re-enable another test * Uncomment check_workers test * Added run_worker test * Minor modification to dead worker detection * More test cases * Better process name for workers * Added back pool.stop_workers() when signal is received * Cleaned up cli.py * WIP on worker-pool command * Fix test * Test that worker pool ignores consecutive shutdown signals * Added test for worker-pool CLI command. * Added timeout to CI jobs * Fix worker pool test * Comment out test_scheduler.py * Fixed worker-pool in burst mode * Increase test coverage * Exclude tests directory from coverage.py * Improve test coverage * Renamed `Pool(num_workers=2) to `Pool(size=2)` * Revert "Renamed `Pool(num_workers=2) to `Pool(size=2)`" This reverts commit a1306f89ad0d8686c6bde447bff75e2f71f0733b. * Renamed Pool to WorkerPool * Added a new TestCase that doesn't use LocalStack * Added job_class, worker_class and serializer arguments to WorkerPool * Use parse_connection() in WorkerPool.__init__ * Added CLI arguments for worker-pool * Minor WorkerPool and test fixes * Fixed failing CLI test * Document WorkerPool
2 years ago
result = runner.invoke(
main, ['enqueue', '-u', self.redis_url, '--job-id', id, 'tests.fixtures.echo', 'key:={"json": "abc"}']
)
Allows enqueueing by the cli (#1466) * Allows enqueueing by the cli #372 * schedule support * `_` to `-` * fix flake8 * echo job-id * Some improvements - Description as in python jobs - return result - quiet mode - allows `--boolean` and `--integer` - raises errors if not used correctly * added tests * add schedule tests * add retry test * use click exceptions * add error test * add job_func test * change messages https://github.com/rq/rq/pull/1466#discussion_r640211128 https://github.com/rq/rq/pull/1466#discussion_r640210850 * Use different format for arguments View https://github.com/rq/rq/pull/1466#discussion_r650510889 * Add file support Usage: @filename * ast.literal_eval support with `#` instead of `:` * func -> function Makes error messages more readable * click Error * print function string * add docs * increase seconds in test * Update `parse_function_arg` Add `ParsingMode` enum (https://github.com/rq/rq/pull/1466#discussion_r656676114) Change error messages (https://github.com/rq/rq/pull/1466#discussion_r656676800, https://github.com/rq/rq/pull/1466#discussion_r656677082) * `#` to `%` `#` is the letter for a comment in bash * Add some tests (https://github.com/rq/rq/pull/1466#discussion_r656674539, https://github.com/rq/rq/pull/1466#discussion_r656676543) * Add some tests * docs: Add some examples * catch all literal_eval exceptions There are some edge cases with other exceptions * remove job_func (https://github.com/rq/rq/pull/1466#pullrequestreview-690110118) * edit docs https://github.com/rq/rq/pull/1466#pullrequestreview-695758691 * format examples * format examples `queue.enqueue(path.to.func, args=['abc'])` to `queue.enqueue(path.to.func, 'abc')` https://github.com/rq/rq/pull/1466#discussion_r673615464 * add examples https://github.com/rq/rq/pull/1466#discussion_r673658933 * add doc test https://github.com/rq/rq/pull/1466#discussion_r673659124 * Update index.md * Update test_cli.py * Update test_cli.py * Add version info Co-authored-by: rpkak <rpkak@users.noreply.github.com>
3 years ago
self.assert_normal_execution(result)
job = Job.fetch(id)
self.assertEqual((job.args, job.kwargs), ([], {'key': {'json': 'abc'}}))
id = str(uuid4())
result = runner.invoke(main, ['enqueue', '-u', self.redis_url, '--job-id', id, 'tests.fixtures.echo', '%1, 2'])
self.assert_normal_execution(result)
job = Job.fetch(id)
self.assertEqual((job.args, job.kwargs), ([(1, 2)], {}))
id = str(uuid4())
result = runner.invoke(main, ['enqueue', '-u', self.redis_url, '--job-id', id, 'tests.fixtures.echo', '%None'])
self.assert_normal_execution(result)
job = Job.fetch(id)
self.assertEqual((job.args, job.kwargs), ([None], {}))
id = str(uuid4())
result = runner.invoke(main, ['enqueue', '-u', self.redis_url, '--job-id', id, 'tests.fixtures.echo', '%True'])
self.assert_normal_execution(result)
job = Job.fetch(id)
self.assertEqual((job.args, job.kwargs), ([True], {}))
id = str(uuid4())
Worker pool (#1874) * First stab at implementating worker pool * Use process.is_alive() to check whether a process is still live * Handle shutdown signal * Check worker loop done * First working version of `WorkerPool`. * Added test for check_workers() * Added test for pool.start() * Better shutdown process * Comment out test_start() to see if it fixes CI * Make tests pass * Make CI pass * Comment out some tests * Comment out more tests * Re-enable a test * Re-enable another test * Uncomment check_workers test * Added run_worker test * Minor modification to dead worker detection * More test cases * Better process name for workers * Added back pool.stop_workers() when signal is received * Cleaned up cli.py * WIP on worker-pool command * Fix test * Test that worker pool ignores consecutive shutdown signals * Added test for worker-pool CLI command. * Added timeout to CI jobs * Fix worker pool test * Comment out test_scheduler.py * Fixed worker-pool in burst mode * Increase test coverage * Exclude tests directory from coverage.py * Improve test coverage * Renamed `Pool(num_workers=2) to `Pool(size=2)` * Revert "Renamed `Pool(num_workers=2) to `Pool(size=2)`" This reverts commit a1306f89ad0d8686c6bde447bff75e2f71f0733b. * Renamed Pool to WorkerPool * Added a new TestCase that doesn't use LocalStack * Added job_class, worker_class and serializer arguments to WorkerPool * Use parse_connection() in WorkerPool.__init__ * Added CLI arguments for worker-pool * Minor WorkerPool and test fixes * Fixed failing CLI test * Document WorkerPool
2 years ago
result = runner.invoke(
main, ['enqueue', '-u', self.redis_url, '--job-id', id, 'tests.fixtures.echo', 'key%=(1, 2)']
)
Allows enqueueing by the cli (#1466) * Allows enqueueing by the cli #372 * schedule support * `_` to `-` * fix flake8 * echo job-id * Some improvements - Description as in python jobs - return result - quiet mode - allows `--boolean` and `--integer` - raises errors if not used correctly * added tests * add schedule tests * add retry test * use click exceptions * add error test * add job_func test * change messages https://github.com/rq/rq/pull/1466#discussion_r640211128 https://github.com/rq/rq/pull/1466#discussion_r640210850 * Use different format for arguments View https://github.com/rq/rq/pull/1466#discussion_r650510889 * Add file support Usage: @filename * ast.literal_eval support with `#` instead of `:` * func -> function Makes error messages more readable * click Error * print function string * add docs * increase seconds in test * Update `parse_function_arg` Add `ParsingMode` enum (https://github.com/rq/rq/pull/1466#discussion_r656676114) Change error messages (https://github.com/rq/rq/pull/1466#discussion_r656676800, https://github.com/rq/rq/pull/1466#discussion_r656677082) * `#` to `%` `#` is the letter for a comment in bash * Add some tests (https://github.com/rq/rq/pull/1466#discussion_r656674539, https://github.com/rq/rq/pull/1466#discussion_r656676543) * Add some tests * docs: Add some examples * catch all literal_eval exceptions There are some edge cases with other exceptions * remove job_func (https://github.com/rq/rq/pull/1466#pullrequestreview-690110118) * edit docs https://github.com/rq/rq/pull/1466#pullrequestreview-695758691 * format examples * format examples `queue.enqueue(path.to.func, args=['abc'])` to `queue.enqueue(path.to.func, 'abc')` https://github.com/rq/rq/pull/1466#discussion_r673615464 * add examples https://github.com/rq/rq/pull/1466#discussion_r673658933 * add doc test https://github.com/rq/rq/pull/1466#discussion_r673659124 * Update index.md * Update test_cli.py * Update test_cli.py * Add version info Co-authored-by: rpkak <rpkak@users.noreply.github.com>
3 years ago
self.assert_normal_execution(result)
job = Job.fetch(id)
self.assertEqual((job.args, job.kwargs), ([], {'key': (1, 2)}))
id = str(uuid4())
Worker pool (#1874) * First stab at implementating worker pool * Use process.is_alive() to check whether a process is still live * Handle shutdown signal * Check worker loop done * First working version of `WorkerPool`. * Added test for check_workers() * Added test for pool.start() * Better shutdown process * Comment out test_start() to see if it fixes CI * Make tests pass * Make CI pass * Comment out some tests * Comment out more tests * Re-enable a test * Re-enable another test * Uncomment check_workers test * Added run_worker test * Minor modification to dead worker detection * More test cases * Better process name for workers * Added back pool.stop_workers() when signal is received * Cleaned up cli.py * WIP on worker-pool command * Fix test * Test that worker pool ignores consecutive shutdown signals * Added test for worker-pool CLI command. * Added timeout to CI jobs * Fix worker pool test * Comment out test_scheduler.py * Fixed worker-pool in burst mode * Increase test coverage * Exclude tests directory from coverage.py * Improve test coverage * Renamed `Pool(num_workers=2) to `Pool(size=2)` * Revert "Renamed `Pool(num_workers=2) to `Pool(size=2)`" This reverts commit a1306f89ad0d8686c6bde447bff75e2f71f0733b. * Renamed Pool to WorkerPool * Added a new TestCase that doesn't use LocalStack * Added job_class, worker_class and serializer arguments to WorkerPool * Use parse_connection() in WorkerPool.__init__ * Added CLI arguments for worker-pool * Minor WorkerPool and test fixes * Fixed failing CLI test * Document WorkerPool
2 years ago
result = runner.invoke(
main, ['enqueue', '-u', self.redis_url, '--job-id', id, 'tests.fixtures.echo', 'key%={"foo": True}']
)
Allows enqueueing by the cli (#1466) * Allows enqueueing by the cli #372 * schedule support * `_` to `-` * fix flake8 * echo job-id * Some improvements - Description as in python jobs - return result - quiet mode - allows `--boolean` and `--integer` - raises errors if not used correctly * added tests * add schedule tests * add retry test * use click exceptions * add error test * add job_func test * change messages https://github.com/rq/rq/pull/1466#discussion_r640211128 https://github.com/rq/rq/pull/1466#discussion_r640210850 * Use different format for arguments View https://github.com/rq/rq/pull/1466#discussion_r650510889 * Add file support Usage: @filename * ast.literal_eval support with `#` instead of `:` * func -> function Makes error messages more readable * click Error * print function string * add docs * increase seconds in test * Update `parse_function_arg` Add `ParsingMode` enum (https://github.com/rq/rq/pull/1466#discussion_r656676114) Change error messages (https://github.com/rq/rq/pull/1466#discussion_r656676800, https://github.com/rq/rq/pull/1466#discussion_r656677082) * `#` to `%` `#` is the letter for a comment in bash * Add some tests (https://github.com/rq/rq/pull/1466#discussion_r656674539, https://github.com/rq/rq/pull/1466#discussion_r656676543) * Add some tests * docs: Add some examples * catch all literal_eval exceptions There are some edge cases with other exceptions * remove job_func (https://github.com/rq/rq/pull/1466#pullrequestreview-690110118) * edit docs https://github.com/rq/rq/pull/1466#pullrequestreview-695758691 * format examples * format examples `queue.enqueue(path.to.func, args=['abc'])` to `queue.enqueue(path.to.func, 'abc')` https://github.com/rq/rq/pull/1466#discussion_r673615464 * add examples https://github.com/rq/rq/pull/1466#discussion_r673658933 * add doc test https://github.com/rq/rq/pull/1466#discussion_r673659124 * Update index.md * Update test_cli.py * Update test_cli.py * Add version info Co-authored-by: rpkak <rpkak@users.noreply.github.com>
3 years ago
self.assert_normal_execution(result)
job = Job.fetch(id)
self.assertEqual((job.args, job.kwargs), ([], {'key': {"foo": True}}))
id = str(uuid4())
Worker pool (#1874) * First stab at implementating worker pool * Use process.is_alive() to check whether a process is still live * Handle shutdown signal * Check worker loop done * First working version of `WorkerPool`. * Added test for check_workers() * Added test for pool.start() * Better shutdown process * Comment out test_start() to see if it fixes CI * Make tests pass * Make CI pass * Comment out some tests * Comment out more tests * Re-enable a test * Re-enable another test * Uncomment check_workers test * Added run_worker test * Minor modification to dead worker detection * More test cases * Better process name for workers * Added back pool.stop_workers() when signal is received * Cleaned up cli.py * WIP on worker-pool command * Fix test * Test that worker pool ignores consecutive shutdown signals * Added test for worker-pool CLI command. * Added timeout to CI jobs * Fix worker pool test * Comment out test_scheduler.py * Fixed worker-pool in burst mode * Increase test coverage * Exclude tests directory from coverage.py * Improve test coverage * Renamed `Pool(num_workers=2) to `Pool(size=2)` * Revert "Renamed `Pool(num_workers=2) to `Pool(size=2)`" This reverts commit a1306f89ad0d8686c6bde447bff75e2f71f0733b. * Renamed Pool to WorkerPool * Added a new TestCase that doesn't use LocalStack * Added job_class, worker_class and serializer arguments to WorkerPool * Use parse_connection() in WorkerPool.__init__ * Added CLI arguments for worker-pool * Minor WorkerPool and test fixes * Fixed failing CLI test * Document WorkerPool
2 years ago
result = runner.invoke(
main, ['enqueue', '-u', self.redis_url, '--job-id', id, 'tests.fixtures.echo', '@tests/test.json']
)
Allows enqueueing by the cli (#1466) * Allows enqueueing by the cli #372 * schedule support * `_` to `-` * fix flake8 * echo job-id * Some improvements - Description as in python jobs - return result - quiet mode - allows `--boolean` and `--integer` - raises errors if not used correctly * added tests * add schedule tests * add retry test * use click exceptions * add error test * add job_func test * change messages https://github.com/rq/rq/pull/1466#discussion_r640211128 https://github.com/rq/rq/pull/1466#discussion_r640210850 * Use different format for arguments View https://github.com/rq/rq/pull/1466#discussion_r650510889 * Add file support Usage: @filename * ast.literal_eval support with `#` instead of `:` * func -> function Makes error messages more readable * click Error * print function string * add docs * increase seconds in test * Update `parse_function_arg` Add `ParsingMode` enum (https://github.com/rq/rq/pull/1466#discussion_r656676114) Change error messages (https://github.com/rq/rq/pull/1466#discussion_r656676800, https://github.com/rq/rq/pull/1466#discussion_r656677082) * `#` to `%` `#` is the letter for a comment in bash * Add some tests (https://github.com/rq/rq/pull/1466#discussion_r656674539, https://github.com/rq/rq/pull/1466#discussion_r656676543) * Add some tests * docs: Add some examples * catch all literal_eval exceptions There are some edge cases with other exceptions * remove job_func (https://github.com/rq/rq/pull/1466#pullrequestreview-690110118) * edit docs https://github.com/rq/rq/pull/1466#pullrequestreview-695758691 * format examples * format examples `queue.enqueue(path.to.func, args=['abc'])` to `queue.enqueue(path.to.func, 'abc')` https://github.com/rq/rq/pull/1466#discussion_r673615464 * add examples https://github.com/rq/rq/pull/1466#discussion_r673658933 * add doc test https://github.com/rq/rq/pull/1466#discussion_r673659124 * Update index.md * Update test_cli.py * Update test_cli.py * Add version info Co-authored-by: rpkak <rpkak@users.noreply.github.com>
3 years ago
self.assert_normal_execution(result)
job = Job.fetch(id)
self.assertEqual((job.args, job.kwargs), ([open('tests/test.json', 'r').read()], {}))
id = str(uuid4())
Worker pool (#1874) * First stab at implementating worker pool * Use process.is_alive() to check whether a process is still live * Handle shutdown signal * Check worker loop done * First working version of `WorkerPool`. * Added test for check_workers() * Added test for pool.start() * Better shutdown process * Comment out test_start() to see if it fixes CI * Make tests pass * Make CI pass * Comment out some tests * Comment out more tests * Re-enable a test * Re-enable another test * Uncomment check_workers test * Added run_worker test * Minor modification to dead worker detection * More test cases * Better process name for workers * Added back pool.stop_workers() when signal is received * Cleaned up cli.py * WIP on worker-pool command * Fix test * Test that worker pool ignores consecutive shutdown signals * Added test for worker-pool CLI command. * Added timeout to CI jobs * Fix worker pool test * Comment out test_scheduler.py * Fixed worker-pool in burst mode * Increase test coverage * Exclude tests directory from coverage.py * Improve test coverage * Renamed `Pool(num_workers=2) to `Pool(size=2)` * Revert "Renamed `Pool(num_workers=2) to `Pool(size=2)`" This reverts commit a1306f89ad0d8686c6bde447bff75e2f71f0733b. * Renamed Pool to WorkerPool * Added a new TestCase that doesn't use LocalStack * Added job_class, worker_class and serializer arguments to WorkerPool * Use parse_connection() in WorkerPool.__init__ * Added CLI arguments for worker-pool * Minor WorkerPool and test fixes * Fixed failing CLI test * Document WorkerPool
2 years ago
result = runner.invoke(
main, ['enqueue', '-u', self.redis_url, '--job-id', id, 'tests.fixtures.echo', 'key=@tests/test.json']
)
Allows enqueueing by the cli (#1466) * Allows enqueueing by the cli #372 * schedule support * `_` to `-` * fix flake8 * echo job-id * Some improvements - Description as in python jobs - return result - quiet mode - allows `--boolean` and `--integer` - raises errors if not used correctly * added tests * add schedule tests * add retry test * use click exceptions * add error test * add job_func test * change messages https://github.com/rq/rq/pull/1466#discussion_r640211128 https://github.com/rq/rq/pull/1466#discussion_r640210850 * Use different format for arguments View https://github.com/rq/rq/pull/1466#discussion_r650510889 * Add file support Usage: @filename * ast.literal_eval support with `#` instead of `:` * func -> function Makes error messages more readable * click Error * print function string * add docs * increase seconds in test * Update `parse_function_arg` Add `ParsingMode` enum (https://github.com/rq/rq/pull/1466#discussion_r656676114) Change error messages (https://github.com/rq/rq/pull/1466#discussion_r656676800, https://github.com/rq/rq/pull/1466#discussion_r656677082) * `#` to `%` `#` is the letter for a comment in bash * Add some tests (https://github.com/rq/rq/pull/1466#discussion_r656674539, https://github.com/rq/rq/pull/1466#discussion_r656676543) * Add some tests * docs: Add some examples * catch all literal_eval exceptions There are some edge cases with other exceptions * remove job_func (https://github.com/rq/rq/pull/1466#pullrequestreview-690110118) * edit docs https://github.com/rq/rq/pull/1466#pullrequestreview-695758691 * format examples * format examples `queue.enqueue(path.to.func, args=['abc'])` to `queue.enqueue(path.to.func, 'abc')` https://github.com/rq/rq/pull/1466#discussion_r673615464 * add examples https://github.com/rq/rq/pull/1466#discussion_r673658933 * add doc test https://github.com/rq/rq/pull/1466#discussion_r673659124 * Update index.md * Update test_cli.py * Update test_cli.py * Add version info Co-authored-by: rpkak <rpkak@users.noreply.github.com>
3 years ago
self.assert_normal_execution(result)
job = Job.fetch(id)
self.assertEqual((job.args, job.kwargs), ([], {'key': open('tests/test.json', 'r').read()}))
id = str(uuid4())
Worker pool (#1874) * First stab at implementating worker pool * Use process.is_alive() to check whether a process is still live * Handle shutdown signal * Check worker loop done * First working version of `WorkerPool`. * Added test for check_workers() * Added test for pool.start() * Better shutdown process * Comment out test_start() to see if it fixes CI * Make tests pass * Make CI pass * Comment out some tests * Comment out more tests * Re-enable a test * Re-enable another test * Uncomment check_workers test * Added run_worker test * Minor modification to dead worker detection * More test cases * Better process name for workers * Added back pool.stop_workers() when signal is received * Cleaned up cli.py * WIP on worker-pool command * Fix test * Test that worker pool ignores consecutive shutdown signals * Added test for worker-pool CLI command. * Added timeout to CI jobs * Fix worker pool test * Comment out test_scheduler.py * Fixed worker-pool in burst mode * Increase test coverage * Exclude tests directory from coverage.py * Improve test coverage * Renamed `Pool(num_workers=2) to `Pool(size=2)` * Revert "Renamed `Pool(num_workers=2) to `Pool(size=2)`" This reverts commit a1306f89ad0d8686c6bde447bff75e2f71f0733b. * Renamed Pool to WorkerPool * Added a new TestCase that doesn't use LocalStack * Added job_class, worker_class and serializer arguments to WorkerPool * Use parse_connection() in WorkerPool.__init__ * Added CLI arguments for worker-pool * Minor WorkerPool and test fixes * Fixed failing CLI test * Document WorkerPool
2 years ago
result = runner.invoke(
main, ['enqueue', '-u', self.redis_url, '--job-id', id, 'tests.fixtures.echo', ':@tests/test.json']
)
Allows enqueueing by the cli (#1466) * Allows enqueueing by the cli #372 * schedule support * `_` to `-` * fix flake8 * echo job-id * Some improvements - Description as in python jobs - return result - quiet mode - allows `--boolean` and `--integer` - raises errors if not used correctly * added tests * add schedule tests * add retry test * use click exceptions * add error test * add job_func test * change messages https://github.com/rq/rq/pull/1466#discussion_r640211128 https://github.com/rq/rq/pull/1466#discussion_r640210850 * Use different format for arguments View https://github.com/rq/rq/pull/1466#discussion_r650510889 * Add file support Usage: @filename * ast.literal_eval support with `#` instead of `:` * func -> function Makes error messages more readable * click Error * print function string * add docs * increase seconds in test * Update `parse_function_arg` Add `ParsingMode` enum (https://github.com/rq/rq/pull/1466#discussion_r656676114) Change error messages (https://github.com/rq/rq/pull/1466#discussion_r656676800, https://github.com/rq/rq/pull/1466#discussion_r656677082) * `#` to `%` `#` is the letter for a comment in bash * Add some tests (https://github.com/rq/rq/pull/1466#discussion_r656674539, https://github.com/rq/rq/pull/1466#discussion_r656676543) * Add some tests * docs: Add some examples * catch all literal_eval exceptions There are some edge cases with other exceptions * remove job_func (https://github.com/rq/rq/pull/1466#pullrequestreview-690110118) * edit docs https://github.com/rq/rq/pull/1466#pullrequestreview-695758691 * format examples * format examples `queue.enqueue(path.to.func, args=['abc'])` to `queue.enqueue(path.to.func, 'abc')` https://github.com/rq/rq/pull/1466#discussion_r673615464 * add examples https://github.com/rq/rq/pull/1466#discussion_r673658933 * add doc test https://github.com/rq/rq/pull/1466#discussion_r673659124 * Update index.md * Update test_cli.py * Update test_cli.py * Add version info Co-authored-by: rpkak <rpkak@users.noreply.github.com>
3 years ago
self.assert_normal_execution(result)
job = Job.fetch(id)
self.assertEqual((job.args, job.kwargs), ([json.loads(open('tests/test.json', 'r').read())], {}))
id = str(uuid4())
Worker pool (#1874) * First stab at implementating worker pool * Use process.is_alive() to check whether a process is still live * Handle shutdown signal * Check worker loop done * First working version of `WorkerPool`. * Added test for check_workers() * Added test for pool.start() * Better shutdown process * Comment out test_start() to see if it fixes CI * Make tests pass * Make CI pass * Comment out some tests * Comment out more tests * Re-enable a test * Re-enable another test * Uncomment check_workers test * Added run_worker test * Minor modification to dead worker detection * More test cases * Better process name for workers * Added back pool.stop_workers() when signal is received * Cleaned up cli.py * WIP on worker-pool command * Fix test * Test that worker pool ignores consecutive shutdown signals * Added test for worker-pool CLI command. * Added timeout to CI jobs * Fix worker pool test * Comment out test_scheduler.py * Fixed worker-pool in burst mode * Increase test coverage * Exclude tests directory from coverage.py * Improve test coverage * Renamed `Pool(num_workers=2) to `Pool(size=2)` * Revert "Renamed `Pool(num_workers=2) to `Pool(size=2)`" This reverts commit a1306f89ad0d8686c6bde447bff75e2f71f0733b. * Renamed Pool to WorkerPool * Added a new TestCase that doesn't use LocalStack * Added job_class, worker_class and serializer arguments to WorkerPool * Use parse_connection() in WorkerPool.__init__ * Added CLI arguments for worker-pool * Minor WorkerPool and test fixes * Fixed failing CLI test * Document WorkerPool
2 years ago
result = runner.invoke(
main, ['enqueue', '-u', self.redis_url, '--job-id', id, 'tests.fixtures.echo', 'key:=@tests/test.json']
)
Allows enqueueing by the cli (#1466) * Allows enqueueing by the cli #372 * schedule support * `_` to `-` * fix flake8 * echo job-id * Some improvements - Description as in python jobs - return result - quiet mode - allows `--boolean` and `--integer` - raises errors if not used correctly * added tests * add schedule tests * add retry test * use click exceptions * add error test * add job_func test * change messages https://github.com/rq/rq/pull/1466#discussion_r640211128 https://github.com/rq/rq/pull/1466#discussion_r640210850 * Use different format for arguments View https://github.com/rq/rq/pull/1466#discussion_r650510889 * Add file support Usage: @filename * ast.literal_eval support with `#` instead of `:` * func -> function Makes error messages more readable * click Error * print function string * add docs * increase seconds in test * Update `parse_function_arg` Add `ParsingMode` enum (https://github.com/rq/rq/pull/1466#discussion_r656676114) Change error messages (https://github.com/rq/rq/pull/1466#discussion_r656676800, https://github.com/rq/rq/pull/1466#discussion_r656677082) * `#` to `%` `#` is the letter for a comment in bash * Add some tests (https://github.com/rq/rq/pull/1466#discussion_r656674539, https://github.com/rq/rq/pull/1466#discussion_r656676543) * Add some tests * docs: Add some examples * catch all literal_eval exceptions There are some edge cases with other exceptions * remove job_func (https://github.com/rq/rq/pull/1466#pullrequestreview-690110118) * edit docs https://github.com/rq/rq/pull/1466#pullrequestreview-695758691 * format examples * format examples `queue.enqueue(path.to.func, args=['abc'])` to `queue.enqueue(path.to.func, 'abc')` https://github.com/rq/rq/pull/1466#discussion_r673615464 * add examples https://github.com/rq/rq/pull/1466#discussion_r673658933 * add doc test https://github.com/rq/rq/pull/1466#discussion_r673659124 * Update index.md * Update test_cli.py * Update test_cli.py * Add version info Co-authored-by: rpkak <rpkak@users.noreply.github.com>
3 years ago
self.assert_normal_execution(result)
job = Job.fetch(id)
self.assertEqual((job.args, job.kwargs), ([], {'key': json.loads(open('tests/test.json', 'r').read())}))
Worker pool (#1874) * First stab at implementating worker pool * Use process.is_alive() to check whether a process is still live * Handle shutdown signal * Check worker loop done * First working version of `WorkerPool`. * Added test for check_workers() * Added test for pool.start() * Better shutdown process * Comment out test_start() to see if it fixes CI * Make tests pass * Make CI pass * Comment out some tests * Comment out more tests * Re-enable a test * Re-enable another test * Uncomment check_workers test * Added run_worker test * Minor modification to dead worker detection * More test cases * Better process name for workers * Added back pool.stop_workers() when signal is received * Cleaned up cli.py * WIP on worker-pool command * Fix test * Test that worker pool ignores consecutive shutdown signals * Added test for worker-pool CLI command. * Added timeout to CI jobs * Fix worker pool test * Comment out test_scheduler.py * Fixed worker-pool in burst mode * Increase test coverage * Exclude tests directory from coverage.py * Improve test coverage * Renamed `Pool(num_workers=2) to `Pool(size=2)` * Revert "Renamed `Pool(num_workers=2) to `Pool(size=2)`" This reverts commit a1306f89ad0d8686c6bde447bff75e2f71f0733b. * Renamed Pool to WorkerPool * Added a new TestCase that doesn't use LocalStack * Added job_class, worker_class and serializer arguments to WorkerPool * Use parse_connection() in WorkerPool.__init__ * Added CLI arguments for worker-pool * Minor WorkerPool and test fixes * Fixed failing CLI test * Document WorkerPool
2 years ago
class WorkerPoolCLITestCase(CLITestCase):
def test_worker_pool_burst_and_num_workers(self):
"""rq worker-pool -u <url> -b -n 3"""
runner = CliRunner()
result = runner.invoke(main, ['worker-pool', '-u', self.redis_url, '-b', '-n', '3'])
self.assert_normal_execution(result)
def test_serializer_and_queue_argument(self):
"""rq worker-pool foo bar -u <url> -b"""
queue = Queue('foo', connection=self.connection, serializer=JSONSerializer)
job = queue.enqueue(say_hello, 'Hello')
queue = Queue('bar', connection=self.connection, serializer=JSONSerializer)
job_2 = queue.enqueue(say_hello, 'Hello')
runner = CliRunner()
runner.invoke(
Worker pool (#1874) * First stab at implementating worker pool * Use process.is_alive() to check whether a process is still live * Handle shutdown signal * Check worker loop done * First working version of `WorkerPool`. * Added test for check_workers() * Added test for pool.start() * Better shutdown process * Comment out test_start() to see if it fixes CI * Make tests pass * Make CI pass * Comment out some tests * Comment out more tests * Re-enable a test * Re-enable another test * Uncomment check_workers test * Added run_worker test * Minor modification to dead worker detection * More test cases * Better process name for workers * Added back pool.stop_workers() when signal is received * Cleaned up cli.py * WIP on worker-pool command * Fix test * Test that worker pool ignores consecutive shutdown signals * Added test for worker-pool CLI command. * Added timeout to CI jobs * Fix worker pool test * Comment out test_scheduler.py * Fixed worker-pool in burst mode * Increase test coverage * Exclude tests directory from coverage.py * Improve test coverage * Renamed `Pool(num_workers=2) to `Pool(size=2)` * Revert "Renamed `Pool(num_workers=2) to `Pool(size=2)`" This reverts commit a1306f89ad0d8686c6bde447bff75e2f71f0733b. * Renamed Pool to WorkerPool * Added a new TestCase that doesn't use LocalStack * Added job_class, worker_class and serializer arguments to WorkerPool * Use parse_connection() in WorkerPool.__init__ * Added CLI arguments for worker-pool * Minor WorkerPool and test fixes * Fixed failing CLI test * Document WorkerPool
2 years ago
main,
['worker-pool', 'foo', 'bar', '-u', self.redis_url, '-b', '--serializer', 'rq.serializers.JSONSerializer'],
)
self.assertEqual(job.get_status(refresh=True), JobStatus.FINISHED)
self.assertEqual(job_2.get_status(refresh=True), JobStatus.FINISHED)
def test_worker_class_argument(self):
"""rq worker-pool -u <url> -b --worker-class rq.Worker"""
runner = CliRunner()
result = runner.invoke(main, ['worker-pool', '-u', self.redis_url, '-b', '--worker-class', 'rq.Worker'])
self.assert_normal_execution(result)
result = runner.invoke(
main, ['worker-pool', '-u', self.redis_url, '-b', '--worker-class', 'rq.worker.SimpleWorker']
)
self.assert_normal_execution(result)
# This one fails because the worker class doesn't exist
result = runner.invoke(
main, ['worker-pool', '-u', self.redis_url, '-b', '--worker-class', 'rq.worker.NonExistantWorker']
)
self.assertNotEqual(result.exit_code, 0)
def test_job_class_argument(self):
"""rq worker-pool -u <url> -b --job-class rq.job.Job"""
runner = CliRunner()
result = runner.invoke(main, ['worker-pool', '-u', self.redis_url, '-b', '--job-class', 'rq.job.Job'])
self.assert_normal_execution(result)
# This one fails because Job class doesn't exist
result = runner.invoke(
main, ['worker-pool', '-u', self.redis_url, '-b', '--job-class', 'rq.job.NonExistantJob']
)
self.assertNotEqual(result.exit_code, 0)