diff --git a/README.md b/README.md index f048ce58..d03ecf10 100644 --- a/README.md +++ b/README.md @@ -27,6 +27,7 @@ Solid Queue can be used with SQL databases such as MySQL, PostgreSQL, or SQLite, - [Performance considerations](#performance-considerations) - [Failed jobs and retries](#failed-jobs-and-retries) - [Error reporting on jobs](#error-reporting-on-jobs) +- [Batch jobs](#batch-jobs) - [Puma plugin](#puma-plugin) - [Jobs and transactional integrity](#jobs-and-transactional-integrity) - [Recurring tasks](#recurring-tasks) @@ -584,6 +585,66 @@ class ApplicationMailer < ActionMailer::Base Rails.error.report(exception) raise exception end +``` + +## Batch jobs + +SolidQueue offers support for batching jobs. This allows you to track progress of a set of jobs, +and optionally trigger callbacks based on their status. It supports the following: + +- Relating jobs to a batch, to track their status +- Three available callbacks to fire: + - `on_finish`: Fired when all jobs have finished, including retries. Fires even when some jobs have failed. + - `on_success`: Fired when all jobs have succeeded, including retries. Will not fire if any jobs have failed, but will fire if jobs have been discarded using `discard_on` + - `on_failure`: Fired when all jobs have finished, including retries. Will only fire if one or more jobs have failed. +- If a job is part of a batch, it can enqueue more jobs for that batch using `batch#enqueue` +- Attaching arbitrary metadata to a batch + +```rb +class SleepyJob < ApplicationJob + def perform(seconds_to_sleep) + Rails.logger.info "Feeling #{seconds_to_sleep} seconds sleepy..." + sleep seconds_to_sleep + end +end + +class BatchFinishJob < ApplicationJob + def perform(batch) # batch is always the default first argument + Rails.logger.info "Good job finishing all jobs" + end +end + +class BatchSuccessJob < ApplicationJob + def perform(batch) # batch is always the default first argument + Rails.logger.info "Good job finishing all jobs, and all of them worked!" + end +end + +class BatchFailureJob < ApplicationJob + def perform(batch) # batch is always the default first argument + Rails.logger.info "At least one job failed, sorry!" + end +end + +SolidQueue::Batch.enqueue( + on_finish: BatchFinishJob, + on_success: BatchSuccessJob, + on_failure: BatchFailureJob, + user_id: 123 +) do + 5.times.map { |i| SleepyJob.perform_later(i) } +end +``` + +### Batch options + +In the case of an empty batch, a `SolidQueue::Batch::EmptyJob` is enqueued. + +By default, this jobs run on the `default` queue. You can specify an alternative queue for it in an initializer: + +```rb +Rails.application.config.after_initialize do # or to_prepare + SolidQueue::Batch.maintenance_queue_name = "my_batch_queue" end ``` diff --git a/app/jobs/solid_queue/batch/empty_job.rb b/app/jobs/solid_queue/batch/empty_job.rb new file mode 100644 index 00000000..d29e1ad0 --- /dev/null +++ b/app/jobs/solid_queue/batch/empty_job.rb @@ -0,0 +1,12 @@ +# frozen_string_literal: true + +module SolidQueue + class Batch + class EmptyJob < (defined?(ApplicationJob) ? ApplicationJob : ActiveJob::Base) + def perform + # This job does nothing - it just exists to trigger batch completion + # The batch completion will be handled by the normal job_finished! flow + end + end + end +end diff --git a/app/models/solid_queue/batch.rb b/app/models/solid_queue/batch.rb new file mode 100644 index 00000000..6292f328 --- /dev/null +++ b/app/models/solid_queue/batch.rb @@ -0,0 +1,157 @@ +# frozen_string_literal: true + +module SolidQueue + class Batch < Record + include Trackable + + has_many :jobs + has_many :batch_executions, class_name: "SolidQueue::BatchExecution", dependent: :destroy + + serialize :on_finish, coder: JSON + serialize :on_success, coder: JSON + serialize :on_failure, coder: JSON + serialize :metadata, coder: JSON + + after_initialize :set_active_job_batch_id + after_commit :start_batch, on: :create, unless: -> { ActiveRecord.respond_to?(:after_all_transactions_commit) } + + mattr_accessor :maintenance_queue_name + self.maintenance_queue_name = "default" + + def enqueue(&block) + raise "You cannot enqueue a batch that is already finished" if finished? + + transaction do + save! if new_record? + + Batch.wrap_in_batch_context(id) do + block&.call(self) + end + + if ActiveRecord.respond_to?(:after_all_transactions_commit) + ActiveRecord.after_all_transactions_commit do + start_batch + end + end + end + end + + def on_success=(value) + super(serialize_callback(value)) + end + + def on_failure=(value) + super(serialize_callback(value)) + end + + def on_finish=(value) + super(serialize_callback(value)) + end + + def metadata + (super || {}).with_indifferent_access + end + + def check_completion! + return if finished? || !ready? + return if batch_executions.limit(1).exists? + + rows = Batch + .where(id: id) + .unfinished + .empty_executions + .update_all(finished_at: Time.current) + + return if rows.zero? + + with_lock do + failed = jobs.joins(:failed_execution).count + finished_attributes = {} + if failed > 0 + finished_attributes[:failed_at] = Time.current + finished_attributes[:failed_jobs] = failed + end + finished_attributes[:completed_jobs] = total_jobs - failed + + update!(finished_attributes) + execute_callbacks + end + end + + private + + def set_active_job_batch_id + self.active_job_batch_id ||= SecureRandom.uuid + end + + def as_active_job(active_job_klass) + active_job_klass.is_a?(ActiveJob::Base) ? active_job_klass : active_job_klass.new + end + + def serialize_callback(value) + return value if value.blank? + active_job = as_active_job(value) + # We can pick up batch ids from context, but callbacks should never be considered a part of the batch + active_job.batch_id = nil + active_job.serialize + end + + def perform_completion_job(job_field, attrs) + active_job = ActiveJob::Base.deserialize(send(job_field)) + active_job.send(:deserialize_arguments_if_needed) + active_job.arguments = [ self ] + Array.wrap(active_job.arguments) + SolidQueue::Job.enqueue_all([ active_job ]) + + active_job.provider_job_id = Job.find_by(active_job_id: active_job.job_id).id + attrs[job_field] = active_job.serialize + end + + def execute_callbacks + if failed_at? + perform_completion_job(:on_failure, {}) if on_failure.present? + else + perform_completion_job(:on_success, {}) if on_success.present? + end + + perform_completion_job(:on_finish, {}) if on_finish.present? + end + + def enqueue_empty_job + Batch.wrap_in_batch_context(id) do + EmptyJob.set(queue: self.class.maintenance_queue_name || "default").perform_later + end + end + + def start_batch + enqueue_empty_job if reload.total_jobs == 0 + update!(enqueued_at: Time.current) + end + + class << self + def enqueue(on_success: nil, on_failure: nil, on_finish: nil, **metadata, &block) + new.tap do |batch| + batch.assign_attributes( + on_success: on_success, + on_failure: on_failure, + on_finish: on_finish, + metadata: metadata + ) + + batch.enqueue(&block) + end + end + + def current_batch_id + ActiveSupport::IsolatedExecutionState[:current_batch_id] + end + + def wrap_in_batch_context(batch_id) + previous_batch_id = current_batch_id.presence || nil + ActiveSupport::IsolatedExecutionState[:current_batch_id] = batch_id + yield + ensure + ActiveSupport::IsolatedExecutionState[:current_batch_id] = previous_batch_id + end + end + end +end diff --git a/app/models/solid_queue/batch/trackable.rb b/app/models/solid_queue/batch/trackable.rb new file mode 100644 index 00000000..61dca290 --- /dev/null +++ b/app/models/solid_queue/batch/trackable.rb @@ -0,0 +1,68 @@ +# frozen_string_literal: true + +module SolidQueue + class Batch + module Trackable + extend ActiveSupport::Concern + + included do + scope :finished, -> { where.not(finished_at: nil) } + scope :succeeded, -> { finished.where(failed_at: nil) } + scope :unfinished, -> { where(finished_at: nil) } + scope :failed, -> { where.not(failed_at: nil) } + scope :empty_executions, -> { + where(<<~SQL) + NOT EXISTS ( + SELECT 1 FROM solid_queue_batch_executions + WHERE solid_queue_batch_executions.batch_id = solid_queue_batches.id + LIMIT 1 + ) + SQL + } + end + + def status + if finished? + failed? ? "failed" : "completed" + elsif enqueued_at.present? + "processing" + else + "pending" + end + end + + def failed? + failed_at.present? + end + + def succeeded? + finished? && !failed? + end + + def finished? + finished_at.present? + end + + def ready? + enqueued_at.present? + end + + def completed_jobs + finished? ? self[:completed_jobs] : total_jobs - batch_executions.count + end + + def failed_jobs + finished? ? self[:failed_jobs] : jobs.joins(:failed_execution).count + end + + def pending_jobs + finished? ? 0 : batch_executions.count + end + + def progress_percentage + return 0 if total_jobs == 0 + ((completed_jobs + failed_jobs) * 100.0 / total_jobs).round(2) + end + end + end +end diff --git a/app/models/solid_queue/batch_execution.rb b/app/models/solid_queue/batch_execution.rb new file mode 100644 index 00000000..f0f5acea --- /dev/null +++ b/app/models/solid_queue/batch_execution.rb @@ -0,0 +1,32 @@ +# frozen_string_literal: true + +module SolidQueue + class BatchExecution < Record + belongs_to :job, optional: true + belongs_to :batch + + after_commit :check_completion, on: :destroy + + private + def check_completion + batch = Batch.find_by(id: batch_id) + batch.check_completion! if batch.present? + end + + class << self + def create_all_from_jobs(jobs) + batch_jobs = jobs.select { |job| job.batch_id.present? } + return if batch_jobs.empty? + + batch_jobs.group_by(&:batch_id).each do |batch_id, jobs| + BatchExecution.insert_all!(jobs.map { |job| + { batch_id:, job_id: job.respond_to?(:provider_job_id) ? job.provider_job_id : job.id } + }) + + total = jobs.size + SolidQueue::Batch.where(id: batch_id).update_all([ "total_jobs = total_jobs + ?", total ]) + end + end + end + end +end diff --git a/app/models/solid_queue/execution/batchable.rb b/app/models/solid_queue/execution/batchable.rb new file mode 100644 index 00000000..fe9aa6ad --- /dev/null +++ b/app/models/solid_queue/execution/batchable.rb @@ -0,0 +1,23 @@ +# frozen_string_literal: true + +module SolidQueue + class Execution + module Batchable + extend ActiveSupport::Concern + + included do + after_create :update_batch_progress, if: -> { job.batch_id? } + end + + private + def update_batch_progress + if is_a?(FailedExecution) + # FailedExecutions are only created when the job is done retrying + job.batch_execution&.destroy! + end + rescue => e + Rails.logger.error "[SolidQueue] Failed to notify batch #{job.batch_id} about job #{job.id} failure: #{e.message}" + end + end + end +end diff --git a/app/models/solid_queue/failed_execution.rb b/app/models/solid_queue/failed_execution.rb index 8bcdc92f..50066bcc 100644 --- a/app/models/solid_queue/failed_execution.rb +++ b/app/models/solid_queue/failed_execution.rb @@ -2,7 +2,7 @@ module SolidQueue class FailedExecution < Execution - include Dispatching + include Dispatching, Batchable serialize :error, coder: JSON diff --git a/app/models/solid_queue/job.rb b/app/models/solid_queue/job.rb index 75eaf627..6cb59e12 100644 --- a/app/models/solid_queue/job.rb +++ b/app/models/solid_queue/job.rb @@ -4,7 +4,7 @@ module SolidQueue class Job < Record class EnqueueError < StandardError; end - include Executable, Clearable, Recurrable + include Executable, Clearable, Recurrable, Batchable serialize :arguments, coder: JSON @@ -62,7 +62,8 @@ def attributes_from_active_job(active_job) scheduled_at: active_job.scheduled_at, class_name: active_job.class.name, arguments: active_job.serialize, - concurrency_key: active_job.concurrency_key + concurrency_key: active_job.concurrency_key, + batch_id: active_job.batch_id } end end diff --git a/app/models/solid_queue/job/batchable.rb b/app/models/solid_queue/job/batchable.rb new file mode 100644 index 00000000..5ab1bae4 --- /dev/null +++ b/app/models/solid_queue/job/batchable.rb @@ -0,0 +1,37 @@ +# frozen_string_literal: true + +module SolidQueue + class Job + module Batchable + extend ActiveSupport::Concern + + included do + belongs_to :batch, optional: true + has_one :batch_execution, foreign_key: :job_id, dependent: :destroy + + after_create :create_batch_execution, if: :batch_id? + after_update :update_batch_progress, if: :batch_id? + end + + class_methods do + def batch_all(jobs) + BatchExecution.create_all_from_jobs(jobs) + end + end + + private + def create_batch_execution + BatchExecution.create_all_from_jobs([ self ]) + end + + def update_batch_progress + return unless saved_change_to_finished_at? && finished_at.present? + return unless batch_id.present? + + batch_execution&.destroy! + rescue => e + Rails.logger.error "[SolidQueue] Failed to update batch #{batch_id} progress for job #{id}: #{e.message}" + end + end + end +end diff --git a/app/models/solid_queue/job/executable.rb b/app/models/solid_queue/job/executable.rb index b0a4cb93..32b070d9 100644 --- a/app/models/solid_queue/job/executable.rb +++ b/app/models/solid_queue/job/executable.rb @@ -19,7 +19,9 @@ module Executable class_methods do def prepare_all_for_execution(jobs) due, not_yet_due = jobs.partition(&:due?) - dispatch_all(due) + schedule_all(not_yet_due) + (dispatch_all(due) + schedule_all(not_yet_due)).tap do |jobs| + batch_all(jobs.select { |job| job.batch_id.present? }) + end end def dispatch_all(jobs) @@ -77,7 +79,7 @@ def dispatch_bypassing_concurrency_limits def finished! if SolidQueue.preserve_finished_jobs? - touch(:finished_at) + update!(finished_at: Time.current) else destroy! end diff --git a/lib/active_job/batch_id.rb b/lib/active_job/batch_id.rb new file mode 100644 index 00000000..1bd03bfc --- /dev/null +++ b/lib/active_job/batch_id.rb @@ -0,0 +1,37 @@ +# frozen_string_literal: true + +# Inspired by active_job/core.rb docs +# https://github.com/rails/rails/blob/1c2529b9a6ba5a1eff58be0d0373d7d9d401015b/activejob/lib/active_job/core.rb#L136 +module ActiveJob + module BatchId + extend ActiveSupport::Concern + + included do + attr_accessor :batch_id + end + + def initialize(*arguments, **kwargs) + super + self.batch_id = SolidQueue::Batch.current_batch_id if solid_queue_job? + end + + def serialize + super.merge("batch_id" => batch_id) + end + + def deserialize(job_data) + super + self.batch_id = job_data["batch_id"] + end + + def batch + @batch ||= SolidQueue::Batch.find_by(id: batch_id) + end + + private + + def solid_queue_job? + self.class.queue_adapter_name == "solid_queue" + end + end +end diff --git a/lib/generators/solid_queue/install/templates/db/queue_schema.rb b/lib/generators/solid_queue/install/templates/db/queue_schema.rb index 85194b6a..25f87688 100644 --- a/lib/generators/solid_queue/install/templates/db/queue_schema.rb +++ b/lib/generators/solid_queue/install/templates/db/queue_schema.rb @@ -37,7 +37,9 @@ t.string "concurrency_key" t.datetime "created_at", null: false t.datetime "updated_at", null: false + t.bigint "batch_id" t.index [ "active_job_id" ], name: "index_solid_queue_jobs_on_active_job_id" + t.index [ "batch_id" ], name: "index_solid_queue_jobs_on_batch_id" t.index [ "class_name" ], name: "index_solid_queue_jobs_on_class_name" t.index [ "finished_at" ], name: "index_solid_queue_jobs_on_finished_at" t.index [ "queue_name", "finished_at" ], name: "index_solid_queue_jobs_for_filtering" @@ -120,6 +122,31 @@ t.index [ "key" ], name: "index_solid_queue_semaphores_on_key", unique: true end + create_table "solid_queue_batches", force: :cascade do |t| + t.string "active_job_batch_id" + t.text "on_finish" + t.text "on_success" + t.text "on_failure" + t.text "metadata" + t.integer "total_jobs", default: 0, null: false + t.integer "completed_jobs", default: 0, null: false + t.integer "failed_jobs", default: 0, null: false + t.datetime "enqueued_at" + t.datetime "finished_at" + t.datetime "failed_at" + t.datetime "created_at", null: false + t.datetime "updated_at", null: false + t.index ["active_job_batch_id"], name: "index_solid_queue_batches_on_active_job_batch_id", unique: true + end + + create_table "solid_queue_batch_executions", force: :cascade do |t| + t.bigint "job_id", null: false + t.bigint "batch_id", null: false + t.datetime "created_at", null: false + t.index [ "job_id" ], name: "index_solid_queue_batch_executions_on_job_id", unique: true + t.index ["batch_id"], name: "index_solid_queue_batch_executions_on_batch_id" + end + add_foreign_key "solid_queue_blocked_executions", "solid_queue_jobs", column: "job_id", on_delete: :cascade add_foreign_key "solid_queue_claimed_executions", "solid_queue_jobs", column: "job_id", on_delete: :cascade add_foreign_key "solid_queue_failed_executions", "solid_queue_jobs", column: "job_id", on_delete: :cascade diff --git a/lib/solid_queue.rb b/lib/solid_queue.rb index e0d51c8c..58b459fd 100644 --- a/lib/solid_queue.rb +++ b/lib/solid_queue.rb @@ -5,6 +5,7 @@ require "active_job" require "active_job/queue_adapters" +require "active_job/batch_id" require "active_support" require "active_support/core_ext/numeric/time" diff --git a/lib/solid_queue/engine.rb b/lib/solid_queue/engine.rb index d10997c7..8daffe0e 100644 --- a/lib/solid_queue/engine.rb +++ b/lib/solid_queue/engine.rb @@ -35,6 +35,7 @@ class Engine < ::Rails::Engine initializer "solid_queue.active_job.extensions" do ActiveSupport.on_load :active_job do include ActiveJob::ConcurrencyControls + include ActiveJob::BatchId end end end diff --git a/test/dummy/app/jobs/batch_completion_job.rb b/test/dummy/app/jobs/batch_completion_job.rb new file mode 100644 index 00000000..0fb17284 --- /dev/null +++ b/test/dummy/app/jobs/batch_completion_job.rb @@ -0,0 +1,7 @@ +class BatchCompletionJob < ApplicationJob + queue_as :background + + def perform(batch) + Rails.logger.info "#{batch.jobs.size} jobs completed!" + end +end diff --git a/test/dummy/app/jobs/sleepy_job.rb b/test/dummy/app/jobs/sleepy_job.rb new file mode 100644 index 00000000..dd105cdc --- /dev/null +++ b/test/dummy/app/jobs/sleepy_job.rb @@ -0,0 +1,10 @@ +class SleepyJob < ApplicationJob + queue_as :background + + retry_on Exception, wait: 30.seconds, attempts: 5 + + def perform(seconds_to_sleep) + Rails.logger.info "Feeling #{seconds_to_sleep} seconds sleepy..." + sleep seconds_to_sleep + end +end diff --git a/test/dummy/db/queue_schema.rb b/test/dummy/db/queue_schema.rb index 697c2e92..050f1220 100644 --- a/test/dummy/db/queue_schema.rb +++ b/test/dummy/db/queue_schema.rb @@ -49,7 +49,9 @@ t.string "concurrency_key" t.datetime "created_at", null: false t.datetime "updated_at", null: false + t.bigint "batch_id" t.index ["active_job_id"], name: "index_solid_queue_jobs_on_active_job_id" + t.index ["batch_id"], name: "index_solid_queue_jobs_on_batch_id" t.index ["class_name"], name: "index_solid_queue_jobs_on_class_name" t.index ["finished_at"], name: "index_solid_queue_jobs_on_finished_at" t.index ["queue_name", "finished_at"], name: "index_solid_queue_jobs_for_filtering" @@ -132,6 +134,31 @@ t.index ["key"], name: "index_solid_queue_semaphores_on_key", unique: true end + create_table "solid_queue_batches", charset: "utf8mb4", collation: "utf8mb4_0900_ai_ci", force: :cascade do |t| + t.string "active_job_batch_id" + t.text "on_finish" + t.text "on_success" + t.text "on_failure" + t.text "metadata" + t.integer "total_jobs", default: 0, null: false + t.integer "completed_jobs", default: 0, null: false + t.integer "failed_jobs", default: 0, null: false + t.datetime "enqueued_at" + t.datetime "finished_at" + t.datetime "failed_at" + t.datetime "created_at", null: false + t.datetime "updated_at", null: false + t.index ["active_job_batch_id"], name: "index_solid_queue_batches_on_active_job_batch_id", unique: true + end + + create_table "solid_queue_batch_executions", charset: "utf8mb4", collation: "utf8mb4_0900_ai_ci", force: :cascade do |t| + t.bigint "job_id", null: false + t.bigint "batch_id", null: false + t.datetime "created_at", null: false + t.index ["job_id"], name: "index_solid_queue_batch_executions_on_job_id", unique: true + t.index ["batch_id"], name: "index_solid_queue_batch_executions_on_batch_id" + end + add_foreign_key "solid_queue_blocked_executions", "solid_queue_jobs", column: "job_id", on_delete: :cascade add_foreign_key "solid_queue_claimed_executions", "solid_queue_jobs", column: "job_id", on_delete: :cascade add_foreign_key "solid_queue_failed_executions", "solid_queue_jobs", column: "job_id", on_delete: :cascade diff --git a/test/integration/batch_lifecycle_test.rb b/test/integration/batch_lifecycle_test.rb new file mode 100644 index 00000000..90d06690 --- /dev/null +++ b/test/integration/batch_lifecycle_test.rb @@ -0,0 +1,335 @@ +# frozen_string_literal: true + +require "test_helper" + +class BatchLifecycleTest < ActiveSupport::TestCase + FailingJobError = Class.new(RuntimeError) + + setup do + @_on_thread_error = SolidQueue.on_thread_error + SolidQueue.on_thread_error = silent_on_thread_error_for([ FailingJobError ], @_on_thread_error) + @worker = SolidQueue::Worker.new(queues: "background", threads: 3) + @dispatcher = SolidQueue::Dispatcher.new(batch_size: 10, polling_interval: 0.2) + SolidQueue::Batch.maintenance_queue_name = "background" + end + + teardown do + SolidQueue.on_thread_error = @_on_thread_error + @worker.stop + @dispatcher.stop + + JobBuffer.clear + + SolidQueue::Job.destroy_all + SolidQueue::Batch.destroy_all + + ApplicationJob.enqueue_after_transaction_commit = false if defined?(ApplicationJob.enqueue_after_transaction_commit) + SolidQueue.preserve_finished_jobs = true + SolidQueue::Batch.maintenance_queue_name = nil + end + + class BatchOnSuccessJob < ApplicationJob + queue_as :background + + def perform(batch, custom_message = "") + JobBuffer.add "#{custom_message}: #{batch.completed_jobs} jobs succeeded!" + end + end + + class BatchOnFailureJob < ApplicationJob + queue_as :background + + def perform(batch, custom_message = "") + JobBuffer.add "#{custom_message}: #{batch.failed_jobs} jobs failed!" + end + end + + class FailingJob < ApplicationJob + queue_as :background + + retry_on FailingJobError, attempts: 3, wait: 0.1.seconds + + def perform + raise FailingJobError, "Failing job" + end + end + + class DiscardingJob < ApplicationJob + queue_as :background + + discard_on FailingJobError + + def perform + raise FailingJobError, "Failing job" + end + end + + class AddsMoreJobsJob < ApplicationJob + queue_as :background + + def perform + batch.enqueue do + AddToBufferJob.perform_later "added from inside 1" + AddToBufferJob.perform_later "added from inside 2" + SolidQueue::Batch.enqueue do + AddToBufferJob.perform_later "added from inside 3" + end + end + end + end + + test "empty batches fire callbacks" do + SolidQueue::Batch.enqueue(on_success: BatchOnSuccessJob.new("3")) do + SolidQueue::Batch.enqueue(on_success: BatchOnSuccessJob.new("2")) do + SolidQueue::Batch.enqueue(on_success: BatchOnSuccessJob.new("1")) { } + SolidQueue::Batch.enqueue(on_success: BatchOnSuccessJob.new("1.1")) { } + end + end + + @dispatcher.start + @worker.start + + wait_for_batches_to_finish_for(2.seconds) + wait_for_jobs_to_finish_for(1.second) + + expected_values = [ "1: 1 jobs succeeded!", "1.1: 1 jobs succeeded!", "2: 1 jobs succeeded!", "3: 1 jobs succeeded!" ] + assert_equal expected_values.sort, JobBuffer.values.sort + assert_equal 4, SolidQueue::Batch.finished.count + end + + test "all jobs are run, including jobs enqueued inside of other jobs" do + batch2 = nil + job1 = job2 = job3 = nil + batch1 = SolidQueue::Batch.enqueue do + job1 = AddToBufferJob.perform_later "hey" + batch2 = SolidQueue::Batch.enqueue do + job2 = AddToBufferJob.perform_later "ho" + job3 = AddsMoreJobsJob.perform_later + end + end + + @dispatcher.start + @worker.start + + wait_for_batches_to_finish_for(2.seconds) + + assert_equal [ "added from inside 1", "added from inside 2", "added from inside 3", "hey", "ho" ], JobBuffer.values.sort + assert_equal 3, SolidQueue::Batch.finished.count + assert_finished_in_order(job!(job3), batch2.reload) + assert_finished_in_order(job!(job2), batch2) + assert_finished_in_order(job!(job1), batch1.reload) + end + + test "when self.enqueue_after_transaction_commit = true" do + skip if Rails::VERSION::MAJOR == 7 && Rails::VERSION::MINOR == 1 + + ApplicationJob.enqueue_after_transaction_commit = true + batch1 = batch2 = batch3 = nil + job1 = job2 = job3 = nil + JobResult.transaction do + JobResult.create!(queue_name: "default", status: "") + + batch1 = SolidQueue::Batch.enqueue do + job1 = AddToBufferJob.perform_later "hey" + JobResult.transaction(requires_new: true) do + JobResult.create!(queue_name: "default", status: "") + batch2 = SolidQueue::Batch.enqueue do + job2 = AddToBufferJob.perform_later "ho" + batch3 = SolidQueue::Batch.enqueue do + job3 = AddToBufferJob.perform_later "let's go" + end + end + end + end + end + + @dispatcher.start + @worker.start + + wait_for_batches_to_finish_for(5.seconds) + + jobs = batch_jobs(batch1, batch2, batch3) + assert_equal [ "hey", "ho", "let's go" ], JobBuffer.values.sort + assert_equal 3, SolidQueue::Batch.finished.count + assert_equal 3, jobs.finished.count + assert_equal 3, jobs.count + assert_finished_in_order(job!(job3), batch3.reload) + assert_finished_in_order(job!(job2), batch2.reload) + assert_finished_in_order(job!(job1), batch1.reload) + end + + test "failed jobs fire properly" do + batch2 = nil + batch1 = SolidQueue::Batch.enqueue(on_failure: BatchOnFailureJob.new("0")) do + FailingJob.perform_later + batch2 = SolidQueue::Batch.enqueue(on_failure: BatchOnFailureJob.new("1")) do + FailingJob.perform_later + end + end + + @dispatcher.start + @worker.start + + wait_for_batches_to_finish_for(5.seconds) + wait_for_jobs_to_finish_for(5.second) + + job_batch1 = SolidQueue::Batch.find_by(id: batch1.id) + job_batch2 = SolidQueue::Batch.find_by(id: batch2.id) + + assert_equal 2, SolidQueue::Batch.count + assert_equal 2, SolidQueue::Batch.finished.count + + assert_equal 3, job_batch1.total_jobs # 1 original + 2 retries + assert_equal 1, job_batch1.failed_jobs # Final failure + assert_equal 2, job_batch1.completed_jobs # 2 retries marked as "finished" + assert_equal 0, job_batch1.pending_jobs + + assert_equal 3, job_batch2.total_jobs # 1 original + 2 retries + assert_equal 1, job_batch2.failed_jobs # Final failure + assert_equal 2, job_batch2.completed_jobs # 2 retries marked as "finished" + assert_equal 0, job_batch2.pending_jobs + + assert_equal [ true, true ].sort, SolidQueue::Batch.all.map(&:failed?) + assert_equal [ "0: 1 jobs failed!", "1: 1 jobs failed!" ], JobBuffer.values.sort + end + + test "executes the same with perform_all_later as it does a normal enqueue" do + batch2 = nil + batch1 = SolidQueue::Batch.enqueue do + ActiveJob.perform_all_later([ FailingJob.new, FailingJob.new ]) + batch2 = SolidQueue::Batch.enqueue do + ActiveJob.perform_all_later([ AddToBufferJob.new("ok"), AddToBufferJob.new("ok2") ]) + end + end + + @dispatcher.start + @worker.start + + wait_for_batches_to_finish_for(5.seconds) + wait_for_jobs_to_finish_for(5.second) + + assert_equal 6, batch1.reload.jobs.count + assert_equal 6, batch1.total_jobs + assert_equal 2, SolidQueue::Batch.finished.count + assert_equal true, batch1.failed? + assert_equal 2, batch2.reload.jobs.count + assert_equal 2, batch2.total_jobs + assert_equal true, batch2.succeeded? + end + + test "discarded jobs fire properly" do + batch2 = nil + batch1 = SolidQueue::Batch.enqueue(on_success: BatchOnSuccessJob.new("0")) do + DiscardingJob.perform_later + batch2 = SolidQueue::Batch.enqueue(on_success: BatchOnSuccessJob.new("1")) do + DiscardingJob.perform_later + end + end + + @dispatcher.start + @worker.start + + wait_for_batches_to_finish_for(5.seconds) + wait_for_jobs_to_finish_for(5.second) + + job_batch1 = SolidQueue::Batch.find_by(id: batch1.id) + job_batch2 = SolidQueue::Batch.find_by(id: batch2.id) + + assert_equal 2, SolidQueue::Batch.count + assert_equal 2, SolidQueue::Batch.finished.count + + assert_equal 1, job_batch1.total_jobs + assert_equal 0, job_batch1.failed_jobs + assert_equal 1, job_batch1.completed_jobs + assert_equal 0, job_batch1.pending_jobs + + assert_equal 1, job_batch2.total_jobs + assert_equal 0, job_batch2.failed_jobs + assert_equal 1, job_batch2.completed_jobs + assert_equal 0, job_batch2.pending_jobs + + assert_equal [ true, true ].sort, SolidQueue::Batch.all.map(&:succeeded?) + assert_equal [ "0: 1 jobs succeeded!", "1: 1 jobs succeeded!" ], JobBuffer.values.sort + end + + test "preserve_finished_jobs = false" do + SolidQueue.preserve_finished_jobs = false + batch1 = SolidQueue::Batch.enqueue do + AddToBufferJob.perform_later "hey" + end + + assert_equal false, batch1.reload.finished? + assert_equal 1, batch1.jobs.count + assert_equal 0, batch1.jobs.finished.count + + @dispatcher.start + @worker.start + + wait_for_batches_to_finish_for(5.seconds) + wait_for_jobs_to_finish_for(5.seconds) + + assert_equal true, batch1.reload.finished? + assert_equal 0, SolidQueue::Job.count + end + + test "batch interface" do + batch = SolidQueue::Batch.enqueue( + on_finish: OnFinishJob, + on_success: OnSuccessJob, + on_failure: OnFailureJob, + source: "test", priority: "high", user_id: 123 + ) do + AddToBufferJob.perform_later "hey" + end + + @dispatcher.start + @worker.start + + wait_for_batches_to_finish_for(2.seconds) + wait_for_jobs_to_finish_for(1.second) + + assert_equal [ "Hi finish #{batch.id}!", "Hi success #{batch.id}!", "hey" ].sort, JobBuffer.values.sort + assert_equal 1, batch.reload.completed_jobs + assert_equal 0, batch.failed_jobs + assert_equal 0, batch.pending_jobs + assert_equal 1, batch.total_jobs + end + + class OnFinishJob < ApplicationJob + queue_as :background + + def perform(batch) + JobBuffer.add "Hi finish #{batch.id}!" + end + end + + class OnSuccessJob < ApplicationJob + queue_as :background + + def perform(batch) + JobBuffer.add "Hi success #{batch.id}!" + end + end + + class OnFailureJob < ApplicationJob + queue_as :background + + def perform(batch) + JobBuffer.add "Hi failure #{batch.id}!" + end + end + + def assert_finished_in_order(*finishables) + finishables.each_cons(2) do |finished1, finished2| + assert_equal finished1.finished_at < finished2.finished_at, true + end + end + + def job!(active_job) + SolidQueue::Job.find_by!(active_job_id: active_job.job_id) + end + + def batch_jobs(*batches) + SolidQueue::Job.where(batch_id: batches.map(&:id)) + end +end diff --git a/test/models/solid_queue/batch_test.rb b/test/models/solid_queue/batch_test.rb new file mode 100644 index 00000000..d5e09a58 --- /dev/null +++ b/test/models/solid_queue/batch_test.rb @@ -0,0 +1,81 @@ +require "test_helper" + +class SolidQueue::BatchTest < ActiveSupport::TestCase + self.use_transactional_tests = false + + teardown do + SolidQueue::Job.destroy_all + SolidQueue::Batch.destroy_all + end + + class BatchWithArgumentsJob < ApplicationJob + def perform(batch, arg1, arg2) + Rails.logger.info "Hi #{batch.batch_id}, #{arg1}, #{arg2}!" + end + end + + class NiceJob < ApplicationJob + retry_on Exception, wait: 1.second + + def perform(arg) + Rails.logger.info "Hi #{arg}!" + end + end + + test "batch will be completed on success" do + batch = SolidQueue::Batch.enqueue(on_finish: BatchCompletionJob) { } + job_batch = SolidQueue::Batch.find_by(id: batch.id) + assert_not_nil job_batch.on_finish + assert_equal BatchCompletionJob.name, job_batch.on_finish["job_class"] + end + + test "batch will be completed on finish" do + batch = SolidQueue::Batch.enqueue(on_success: BatchCompletionJob) { } + job_batch = SolidQueue::Batch.find_by(id: batch.id) + assert_not_nil job_batch.on_success + assert_equal BatchCompletionJob.name, job_batch.on_success["job_class"] + end + + test "sets the batch_id on jobs created inside of the enqueue block" do + batch = SolidQueue::Batch.enqueue(on_finish: BatchCompletionJob) do + NiceJob.perform_later("world") + NiceJob.perform_later("people") + end + + assert_equal 2, SolidQueue::Job.count + assert_equal [ batch.id ] * 2, SolidQueue::Job.last(2).map(&:batch_id) + end + + test "batch id is present inside the block" do + assert_nil SolidQueue::Batch.current_batch_id + SolidQueue::Batch.enqueue(on_finish: BatchCompletionJob) do + assert_not_nil SolidQueue::Batch.current_batch_id + end + assert_nil SolidQueue::Batch.current_batch_id + end + + test "allow arguments and options for callbacks" do + SolidQueue::Batch.enqueue( + on_finish: BatchWithArgumentsJob.new(1, 2).set(queue: :batch), + ) do + NiceJob.perform_later("world") + end + + assert_not_nil SolidQueue::Batch.last.on_finish["arguments"] + assert_equal SolidQueue::Batch.last.on_finish["arguments"], [ 1, 2 ] + assert_equal SolidQueue::Batch.last.on_finish["queue_name"], "batch" + end + + test "creates batch with metadata" do + SolidQueue::Batch.enqueue( + source: "test", priority: "high", user_id: 123 + ) do + NiceJob.perform_later("world") + end + + assert_not_nil SolidQueue::Batch.last.metadata + assert_equal SolidQueue::Batch.last.metadata["source"], "test" + assert_equal SolidQueue::Batch.last.metadata["priority"], "high" + assert_equal SolidQueue::Batch.last.metadata["user_id"], 123 + end +end diff --git a/test/test_helpers/jobs_test_helper.rb b/test/test_helpers/jobs_test_helper.rb index 8b71e7f6..314f531a 100644 --- a/test/test_helpers/jobs_test_helper.rb +++ b/test/test_helpers/jobs_test_helper.rb @@ -17,6 +17,14 @@ def wait_for_jobs_to_be_released_for(timeout = 1.second) end end + def wait_for_batches_to_finish_for(timeout = 1.second) + wait_while_with_timeout(timeout) do + skip_active_record_query_cache do + SolidQueue::Batch.where(finished_at: nil).any? + end + end + end + def assert_unfinished_jobs(*jobs) skip_active_record_query_cache do assert_equal jobs.map(&:job_id).sort, SolidQueue::Job.where(finished_at: nil).map(&:active_job_id).sort