Welcome to mirror list, hosted at ThFree Co, Russian Federation.

gitlab.com/gitlab-org/gitlab-foss.git - Unnamed repository; edit this file 'description' to name the repository.
summaryrefslogtreecommitdiff
diff options
context:
space:
mode:
Diffstat (limited to 'lib/gitlab/database')
-rw-r--r--lib/gitlab/database/as_with_materialized.rb36
-rw-r--r--lib/gitlab/database/background_migration/batch_metrics.rb33
-rw-r--r--lib/gitlab/database/background_migration/batched_migration.rb20
-rw-r--r--lib/gitlab/database/background_migration/batched_migration_runner.rb (renamed from lib/gitlab/database/background_migration/scheduler.rb)38
-rw-r--r--lib/gitlab/database/background_migration/batched_migration_wrapper.rb73
-rw-r--r--lib/gitlab/database/batch_count.rb39
-rw-r--r--lib/gitlab/database/bulk_update.rb2
-rw-r--r--lib/gitlab/database/count/reltuples_count_strategy.rb6
-rw-r--r--lib/gitlab/database/loose_index_scan_distinct_count.rb102
-rw-r--r--lib/gitlab/database/migration_helpers.rb160
-rw-r--r--lib/gitlab/database/migrations/background_migration_helpers.rb21
-rw-r--r--lib/gitlab/database/partitioning_migration_helpers/foreign_key_helpers.rb74
-rw-r--r--lib/gitlab/database/partitioning_migration_helpers/table_management_helpers.rb32
-rw-r--r--lib/gitlab/database/pg_class.rb23
-rw-r--r--lib/gitlab/database/postgres_hll/batch_distinct_counter.rb31
-rw-r--r--lib/gitlab/database/similarity_score.rb2
-rw-r--r--lib/gitlab/database/unidirectional_copy_trigger.rb97
17 files changed, 659 insertions, 130 deletions
diff --git a/lib/gitlab/database/as_with_materialized.rb b/lib/gitlab/database/as_with_materialized.rb
new file mode 100644
index 00000000000..7c45f416638
--- /dev/null
+++ b/lib/gitlab/database/as_with_materialized.rb
@@ -0,0 +1,36 @@
+# frozen_string_literal: true
+
+module Gitlab
+ module Database
+ # This class is a special Arel node which allows optionally define the `MATERIALIZED` keyword for CTE and Recursive CTE queries.
+ class AsWithMaterialized < Arel::Nodes::Binary
+ extend Gitlab::Utils::StrongMemoize
+
+ MATERIALIZED = Arel.sql(' MATERIALIZED')
+ EMPTY_STRING = Arel.sql('')
+ attr_reader :expr
+
+ def initialize(left, right, materialized: true)
+ @expr = if materialized && self.class.materialized_supported?
+ MATERIALIZED
+ else
+ EMPTY_STRING
+ end
+
+ super(left, right)
+ end
+
+ # Note: to be deleted after the minimum PG version is set to 12.0
+ def self.materialized_supported?
+ strong_memoize(:materialized_supported) do
+ Gitlab::Database.version.match?(/^1[2-9]\./) # version 12.x and above
+ end
+ end
+
+ # Note: to be deleted after the minimum PG version is set to 12.0
+ def self.materialized_if_supported
+ materialized_supported? ? 'MATERIALIZED' : ''
+ end
+ end
+ end
+end
diff --git a/lib/gitlab/database/background_migration/batch_metrics.rb b/lib/gitlab/database/background_migration/batch_metrics.rb
new file mode 100644
index 00000000000..3e6d7ac3c9f
--- /dev/null
+++ b/lib/gitlab/database/background_migration/batch_metrics.rb
@@ -0,0 +1,33 @@
+# frozen_string_literal: true
+
+module Gitlab
+ module Database
+ module BackgroundMigration
+ class BatchMetrics
+ attr_reader :timings
+
+ def initialize
+ @timings = {}
+ end
+
+ def time_operation(label)
+ start_time = monotonic_time
+
+ yield
+
+ timings_for_label(label) << monotonic_time - start_time
+ end
+
+ private
+
+ def timings_for_label(label)
+ timings[label] ||= []
+ end
+
+ def monotonic_time
+ Gitlab::Metrics::System.monotonic_time
+ end
+ end
+ end
+ end
+end
diff --git a/lib/gitlab/database/background_migration/batched_migration.rb b/lib/gitlab/database/background_migration/batched_migration.rb
index 0c9add9b355..4aa33ed7946 100644
--- a/lib/gitlab/database/background_migration/batched_migration.rb
+++ b/lib/gitlab/database/background_migration/batched_migration.rb
@@ -5,7 +5,7 @@ module Gitlab
module BackgroundMigration
class BatchedMigration < ActiveRecord::Base # rubocop:disable Rails/ApplicationRecord
JOB_CLASS_MODULE = 'Gitlab::BackgroundMigration'
- BATCH_CLASS_MODULE = "#{JOB_CLASS_MODULE}::BatchingStrategies".freeze
+ BATCH_CLASS_MODULE = "#{JOB_CLASS_MODULE}::BatchingStrategies"
self.table_name = :batched_background_migrations
@@ -23,8 +23,15 @@ module Gitlab
finished: 3
}
- def interval_elapsed?
- last_job.nil? || last_job.created_at <= Time.current - interval
+ def self.active_migration
+ active.queue_order.first
+ end
+
+ def interval_elapsed?(variance: 0)
+ return true unless last_job
+
+ interval_with_variance = interval - variance
+ last_job.created_at <= Time.current - interval_with_variance
end
def create_batched_job!(min, max)
@@ -50,6 +57,13 @@ module Gitlab
def batch_class_name=(class_name)
write_attribute(:batch_class_name, class_name.demodulize)
end
+
+ def prometheus_labels
+ @prometheus_labels ||= {
+ migration_id: id,
+ migration_identifier: "%s/%s.%s" % [job_class_name, table_name, column_name]
+ }
+ end
end
end
end
diff --git a/lib/gitlab/database/background_migration/scheduler.rb b/lib/gitlab/database/background_migration/batched_migration_runner.rb
index 5f8a5ec06a5..cf8b61f5feb 100644
--- a/lib/gitlab/database/background_migration/scheduler.rb
+++ b/lib/gitlab/database/background_migration/batched_migration_runner.rb
@@ -3,12 +3,22 @@
module Gitlab
module Database
module BackgroundMigration
- class Scheduler
- def perform(migration_wrapper: BatchedMigrationWrapper.new)
- active_migration = BatchedMigration.active.queue_order.first
-
- return unless active_migration&.interval_elapsed?
+ class BatchedMigrationRunner
+ def initialize(migration_wrapper = BatchedMigrationWrapper.new)
+ @migration_wrapper = migration_wrapper
+ end
+ # Runs the next batched_job for a batched_background_migration.
+ #
+ # The batch bounds of the next job are calculated at runtime, based on the migration
+ # configuration and the bounds of the most recently created batched_job. Updating the
+ # migration configuration will cause future jobs to use the updated batch sizes.
+ #
+ # The job instance will automatically receive a set of arguments based on the migration
+ # configuration. For more details, see the BatchedMigrationWrapper class.
+ #
+ # Note that this method is primarily intended to called by a scheduled worker.
+ def run_migration_job(active_migration)
if next_batched_job = create_next_batched_job!(active_migration)
migration_wrapper.perform(next_batched_job)
else
@@ -16,8 +26,26 @@ module Gitlab
end
end
+ # Runs all remaining batched_jobs for a batched_background_migration.
+ #
+ # This method is intended to be used in a test/dev environment to execute the background
+ # migration inline. It should NOT be used in a real environment for any non-trivial migrations.
+ def run_entire_migration(migration)
+ unless Rails.env.development? || Rails.env.test?
+ raise 'this method is not intended for use in real environments'
+ end
+
+ while migration.active?
+ run_migration_job(migration)
+
+ migration.reload_last_job
+ end
+ end
+
private
+ attr_reader :migration_wrapper
+
def create_next_batched_job!(active_migration)
next_batch_range = find_next_batch_range(active_migration)
diff --git a/lib/gitlab/database/background_migration/batched_migration_wrapper.rb b/lib/gitlab/database/background_migration/batched_migration_wrapper.rb
index 299bd992197..c276f8ce75b 100644
--- a/lib/gitlab/database/background_migration/batched_migration_wrapper.rb
+++ b/lib/gitlab/database/background_migration/batched_migration_wrapper.rb
@@ -4,6 +4,15 @@ module Gitlab
module Database
module BackgroundMigration
class BatchedMigrationWrapper
+ extend Gitlab::Utils::StrongMemoize
+
+ # Wraps the execution of a batched_background_migration.
+ #
+ # Updates the job's tracking records with the status of the migration
+ # when starting and finishing execution, and optionally saves batch_metrics
+ # the migration provides, if any are given.
+ #
+ # The job's batch_metrics are serialized to JSON for storage.
def perform(batch_tracking_record)
start_tracking_execution(batch_tracking_record)
@@ -16,6 +25,7 @@ module Gitlab
raise e
ensure
finish_tracking_execution(batch_tracking_record)
+ track_prometheus_metrics(batch_tracking_record)
end
private
@@ -34,12 +44,75 @@ module Gitlab
tracking_record.migration_column_name,
tracking_record.sub_batch_size,
*tracking_record.migration_job_arguments)
+
+ if job_instance.respond_to?(:batch_metrics)
+ tracking_record.metrics = job_instance.batch_metrics
+ end
end
def finish_tracking_execution(tracking_record)
tracking_record.finished_at = Time.current
tracking_record.save!
end
+
+ def track_prometheus_metrics(tracking_record)
+ migration = tracking_record.batched_migration
+ base_labels = migration.prometheus_labels
+
+ metric_for(:gauge_batch_size).set(base_labels, tracking_record.batch_size)
+ metric_for(:gauge_sub_batch_size).set(base_labels, tracking_record.sub_batch_size)
+ metric_for(:counter_updated_tuples).increment(base_labels, tracking_record.batch_size)
+
+ # Time efficiency: Ratio of duration to interval (ideal: less than, but close to 1)
+ efficiency = (tracking_record.finished_at - tracking_record.started_at).to_i / migration.interval.to_f
+ metric_for(:histogram_time_efficiency).observe(base_labels, efficiency)
+
+ if metrics = tracking_record.metrics
+ metrics['timings']&.each do |key, timings|
+ summary = metric_for(:histogram_timings)
+ labels = base_labels.merge(operation: key)
+
+ timings.each do |timing|
+ summary.observe(labels, timing)
+ end
+ end
+ end
+ end
+
+ def metric_for(name)
+ self.class.metrics[name]
+ end
+
+ def self.metrics
+ strong_memoize(:metrics) do
+ {
+ gauge_batch_size: Gitlab::Metrics.gauge(
+ :batched_migration_job_batch_size,
+ 'Batch size for a batched migration job'
+ ),
+ gauge_sub_batch_size: Gitlab::Metrics.gauge(
+ :batched_migration_job_sub_batch_size,
+ 'Sub-batch size for a batched migration job'
+ ),
+ counter_updated_tuples: Gitlab::Metrics.counter(
+ :batched_migration_job_updated_tuples_total,
+ 'Number of tuples updated by batched migration job'
+ ),
+ histogram_timings: Gitlab::Metrics.histogram(
+ :batched_migration_job_duration_seconds,
+ 'Timings for a batched migration job',
+ {},
+ [0.1, 0.25, 0.5, 1, 5].freeze
+ ),
+ histogram_time_efficiency: Gitlab::Metrics.histogram(
+ :batched_migration_job_time_efficiency,
+ 'Ratio of job duration to interval',
+ {},
+ [0.5, 0.9, 1, 1.5, 2].freeze
+ )
+ }
+ end
+ end
end
end
end
diff --git a/lib/gitlab/database/batch_count.rb b/lib/gitlab/database/batch_count.rb
index 5a506da0d05..9002d39e1ee 100644
--- a/lib/gitlab/database/batch_count.rb
+++ b/lib/gitlab/database/batch_count.rb
@@ -88,11 +88,16 @@ module Gitlab
batch_start = start
while batch_start < finish
- batch_end = [batch_start + batch_size, finish].min
- batch_relation = build_relation_batch(batch_start, batch_end, mode)
-
begin
- results = merge_results(results, batch_relation.send(@operation, *@operation_args)) # rubocop:disable GitlabSecurity/PublicSend
+ batch_end = [batch_start + batch_size, finish].min
+ batch_relation = build_relation_batch(batch_start, batch_end, mode)
+
+ op_args = @operation_args
+ if @operation == :count && @operation_args.blank? && use_loose_index_scan_for_distinct_values?(mode)
+ op_args = [Gitlab::Database::LooseIndexScanDistinctCount::COLUMN_ALIAS]
+ end
+
+ results = merge_results(results, batch_relation.send(@operation, *op_args)) # rubocop:disable GitlabSecurity/PublicSend
batch_start = batch_end
rescue ActiveRecord::QueryCanceled => error
# retry with a safe batch size & warmer cache
@@ -102,6 +107,18 @@ module Gitlab
log_canceled_batch_fetch(batch_start, mode, batch_relation.to_sql, error)
return FALLBACK
end
+ rescue Gitlab::Database::LooseIndexScanDistinctCount::ColumnConfigurationError => error
+ Gitlab::AppJsonLogger
+ .error(
+ event: 'batch_count',
+ relation: @relation.table_name,
+ operation: @operation,
+ operation_args: @operation_args,
+ mode: mode,
+ message: "LooseIndexScanDistinctCount column error: #{error.message}"
+ )
+
+ return FALLBACK
end
sleep(SLEEP_TIME_IN_SECONDS)
@@ -123,7 +140,11 @@ module Gitlab
private
def build_relation_batch(start, finish, mode)
- @relation.select(@column).public_send(mode).where(between_condition(start, finish)) # rubocop:disable GitlabSecurity/PublicSend
+ if use_loose_index_scan_for_distinct_values?(mode)
+ Gitlab::Database::LooseIndexScanDistinctCount.new(@relation, @column).build_query(from: start, to: finish)
+ else
+ @relation.select(@column).public_send(mode).where(between_condition(start, finish)) # rubocop:disable GitlabSecurity/PublicSend
+ end
end
def batch_size_for_mode_and_operation(mode, operation)
@@ -165,6 +186,14 @@ module Gitlab
message: "Query has been canceled with message: #{error.message}"
)
end
+
+ def use_loose_index_scan_for_distinct_values?(mode)
+ Feature.enabled?(:loose_index_scan_for_distinct_values) && not_group_by_query? && mode == :distinct
+ end
+
+ def not_group_by_query?
+ !@relation.is_a?(ActiveRecord::Relation) || @relation.group_values.blank?
+ end
end
end
end
diff --git a/lib/gitlab/database/bulk_update.rb b/lib/gitlab/database/bulk_update.rb
index 1403d561890..b1f9da30585 100644
--- a/lib/gitlab/database/bulk_update.rb
+++ b/lib/gitlab/database/bulk_update.rb
@@ -130,7 +130,7 @@ module Gitlab
def sql
<<~SQL
- WITH cte(#{list_of(cte_columns)}) AS (VALUES #{list_of(values)})
+ WITH cte(#{list_of(cte_columns)}) AS #{Gitlab::Database::AsWithMaterialized.materialized_if_supported} (VALUES #{list_of(values)})
UPDATE #{table_name} SET #{list_of(updates)} FROM cte WHERE cte_id = id
SQL
end
diff --git a/lib/gitlab/database/count/reltuples_count_strategy.rb b/lib/gitlab/database/count/reltuples_count_strategy.rb
index 89190320cf9..a7bfafe2815 100644
--- a/lib/gitlab/database/count/reltuples_count_strategy.rb
+++ b/lib/gitlab/database/count/reltuples_count_strategy.rb
@@ -3,10 +3,6 @@
module Gitlab
module Database
module Count
- class PgClass < ActiveRecord::Base
- self.table_name = 'pg_class'
- end
-
# This strategy counts based on PostgreSQL's statistics in pg_stat_user_tables.
#
# Specifically, it relies on the column reltuples in said table. An additional
@@ -74,7 +70,7 @@ module Gitlab
def get_statistics(table_names, check_statistics: true)
time = 6.hours.ago
- query = PgClass.joins("LEFT JOIN pg_stat_user_tables ON pg_stat_user_tables.relid = pg_class.oid")
+ query = ::Gitlab::Database::PgClass.joins("LEFT JOIN pg_stat_user_tables ON pg_stat_user_tables.relid = pg_class.oid")
.where(relname: table_names)
.where('schemaname = current_schema()')
.select('pg_class.relname AS table_name, reltuples::bigint AS estimate')
diff --git a/lib/gitlab/database/loose_index_scan_distinct_count.rb b/lib/gitlab/database/loose_index_scan_distinct_count.rb
new file mode 100644
index 00000000000..884f4d47ff8
--- /dev/null
+++ b/lib/gitlab/database/loose_index_scan_distinct_count.rb
@@ -0,0 +1,102 @@
+# frozen_string_literal: true
+
+module Gitlab
+ module Database
+ # This class builds efficient batched distinct query by using loose index scan.
+ # Consider the following example:
+ # > Issue.distinct(:project_id).where(project_id: (1...100)).count
+ #
+ # Note: there is an index on project_id
+ #
+ # This query will read each element in the index matching the project_id filter.
+ # If for a project_id has 100_000 issues, all 100_000 elements will be read.
+ #
+ # A loose index scan will read only one entry from the index for each project_id to reduce the number of disk reads.
+ #
+ # Usage:
+ #
+ # Gitlab::Database::LooseIndexScanDisctinctCount.new(Issue, :project_id).count(from: 1, to: 100)
+ #
+ # The query will return the number of distinct projects_ids between 1 and 100
+ #
+ # Getting the Arel query:
+ #
+ # Gitlab::Database::LooseIndexScanDisctinctCount.new(Issue, :project_id).build_query(from: 1, to: 100)
+ class LooseIndexScanDistinctCount
+ COLUMN_ALIAS = 'distinct_count_column'
+
+ ColumnConfigurationError = Class.new(StandardError)
+
+ def initialize(scope, column)
+ if scope.is_a?(ActiveRecord::Relation)
+ @scope = scope
+ @model = scope.model
+ else
+ @scope = scope.where({})
+ @model = scope
+ end
+
+ @column = transform_column(column)
+ end
+
+ def count(from:, to:)
+ build_query(from: from, to: to).count(COLUMN_ALIAS)
+ end
+
+ def build_query(from:, to:) # rubocop:disable Metrics/AbcSize
+ cte = Gitlab::SQL::RecursiveCTE.new(:counter_cte, union_args: { remove_order: false })
+ table = model.arel_table
+
+ cte << @scope
+ .dup
+ .select(column.as(COLUMN_ALIAS))
+ .where(column.gteq(from))
+ .where(column.lt(to))
+ .order(column)
+ .limit(1)
+
+ inner_query = @scope
+ .dup
+ .where(column.gt(cte.table[COLUMN_ALIAS]))
+ .where(column.lt(to))
+ .select(column.as(COLUMN_ALIAS))
+ .order(column)
+ .limit(1)
+
+ cte << cte.table
+ .project(Arel::Nodes::Grouping.new(Arel.sql(inner_query.to_sql)).as(COLUMN_ALIAS))
+ .where(cte.table[COLUMN_ALIAS].lt(to))
+
+ model
+ .with
+ .recursive(cte.to_arel)
+ .from(cte.alias_to(table))
+ .unscope(where: :source_type)
+ .unscope(where: model.inheritance_column) # Remove STI query, not needed here
+ end
+
+ private
+
+ attr_reader :column, :model
+
+ # Transforms the column so it can be used in Arel expressions
+ #
+ # 'table.column' => 'table.column'
+ # 'column' => 'table_name.column'
+ # :column => 'table_name.column'
+ # Arel::Attributes::Attribute => name of the column
+ def transform_column(column)
+ if column.is_a?(String) || column.is_a?(Symbol)
+ column_as_string = column.to_s
+ column_as_string = "#{model.table_name}.#{column_as_string}" unless column_as_string.include?('.')
+
+ Arel.sql(column_as_string)
+ elsif column.is_a?(Arel::Attributes::Attribute)
+ column
+ else
+ raise ColumnConfigurationError.new("Cannot transform the column: #{column.inspect}, please provide the column name as string")
+ end
+ end
+ end
+ end
+end
diff --git a/lib/gitlab/database/migration_helpers.rb b/lib/gitlab/database/migration_helpers.rb
index 31e733050e1..d06a73da8ac 100644
--- a/lib/gitlab/database/migration_helpers.rb
+++ b/lib/gitlab/database/migration_helpers.rb
@@ -4,6 +4,7 @@ module Gitlab
module Database
module MigrationHelpers
include Migrations::BackgroundMigrationHelpers
+ include DynamicModelHelpers
# https://www.postgresql.org/docs/current/sql-syntax-lexical.html#SQL-SYNTAX-IDENTIFIERS
MAX_IDENTIFIER_NAME_LENGTH = 63
@@ -576,17 +577,7 @@ module Gitlab
# old_column - The name of the old column.
# new_column - The name of the new column.
def install_rename_triggers(table, old_column, new_column)
- trigger_name = rename_trigger_name(table, old_column, new_column)
- quoted_table = quote_table_name(table)
- quoted_old = quote_column_name(old_column)
- quoted_new = quote_column_name(new_column)
-
- install_rename_triggers_for_postgresql(
- trigger_name,
- quoted_table,
- quoted_old,
- quoted_new
- )
+ install_rename_triggers_for_postgresql(table, old_column, new_column)
end
# Changes the type of a column concurrently.
@@ -927,19 +918,67 @@ module Gitlab
# This is crucial for Primary Key conversions, because setting a column
# as the PK converts even check constraints to NOT NULL constraints
# and forces an inline re-verification of the whole table.
- # - It backfills the new column with the values of the existing primary key
- # by scheduling background jobs.
- # - It tracks the scheduled background jobs through the use of
- # Gitlab::Database::BackgroundMigrationJob
+ # - It sets up a trigger to keep the two columns in sync.
+ #
+ # Note: this helper is intended to be used in a regular (pre-deployment) migration.
+ #
+ # This helper is part 1 of a multi-step migration process:
+ # 1. initialize_conversion_of_integer_to_bigint to create the new column and database triggers
+ # 2. backfill_conversion_of_integer_to_bigint to copy historic data using background migrations
+ # 3. remaining steps TBD, see #288005
+ #
+ # table - The name of the database table containing the column
+ # column - The name of the column that we want to convert to bigint.
+ # primary_key - The name of the primary key column (most often :id)
+ def initialize_conversion_of_integer_to_bigint(table, column, primary_key: :id)
+ unless table_exists?(table)
+ raise "Table #{table} does not exist"
+ end
+
+ unless column_exists?(table, primary_key)
+ raise "Column #{primary_key} does not exist on #{table}"
+ end
+
+ unless column_exists?(table, column)
+ raise "Column #{column} does not exist on #{table}"
+ end
+
+ check_trigger_permissions!(table)
+
+ old_column = column_for(table, column)
+ tmp_column = "#{column}_convert_to_bigint"
+
+ with_lock_retries do
+ if (column.to_s == primary_key.to_s) || !old_column.null
+ # If the column to be converted is either a PK or is defined as NOT NULL,
+ # set it to `NOT NULL DEFAULT 0` and we'll copy paste the correct values bellow
+ # That way, we skip the expensive validation step required to add
+ # a NOT NULL constraint at the end of the process
+ add_column(table, tmp_column, :bigint, default: old_column.default || 0, null: false)
+ else
+ add_column(table, tmp_column, :bigint, default: old_column.default)
+ end
+
+ install_rename_triggers(table, column, tmp_column)
+ end
+ end
+
+ # Backfills the new column used in the conversion of an integer column to bigint using background migrations.
+ #
+ # - This helper should be called from a post-deployment migration.
+ # - In order for this helper to work properly, the new column must be first initialized with
+ # the `initialize_conversion_of_integer_to_bigint` helper.
+ # - It tracks the scheduled background jobs through Gitlab::Database::BackgroundMigration::BatchedMigration,
# which allows a more thorough check that all jobs succeeded in the
# cleanup migration and is way faster for very large tables.
- # - It sets up a trigger to keep the two columns in sync
- # - It does not schedule a cleanup job: we have to do that with followup
- # post deployment migrations in the next release.
#
- # This needs to be done manually by using the
- # `cleanup_initialize_conversion_of_integer_to_bigint`
- # (not yet implemented - check #288005)
+ # Note: this helper is intended to be used in a post-deployment migration, to ensure any new code is
+ # deployed (including background job changes) before we begin processing the background migration.
+ #
+ # This helper is part 2 of a multi-step migration process:
+ # 1. initialize_conversion_of_integer_to_bigint to create the new column and database triggers
+ # 2. backfill_conversion_of_integer_to_bigint to copy historic data using background migrations
+ # 3. remaining steps TBD, see #288005
#
# table - The name of the database table containing the column
# column - The name of the column that we want to convert to bigint.
@@ -960,7 +999,7 @@ module Gitlab
# and set the batch_size to 50_000 which will require
# ~50s = (50000 / 200) * (0.1 + 0.1) to complete and leaves breathing space
# between the scheduled jobs
- def initialize_conversion_of_integer_to_bigint(
+ def backfill_conversion_of_integer_to_bigint(
table,
column,
primary_key: :id,
@@ -969,10 +1008,6 @@ module Gitlab
interval: 2.minutes
)
- if transaction_open?
- raise 'initialize_conversion_of_integer_to_bigint can not be run inside a transaction'
- end
-
unless table_exists?(table)
raise "Table #{table} does not exist"
end
@@ -985,87 +1020,42 @@ module Gitlab
raise "Column #{column} does not exist on #{table}"
end
- check_trigger_permissions!(table)
-
- old_column = column_for(table, column)
tmp_column = "#{column}_convert_to_bigint"
- with_lock_retries do
- if (column.to_s == primary_key.to_s) || !old_column.null
- # If the column to be converted is either a PK or is defined as NOT NULL,
- # set it to `NOT NULL DEFAULT 0` and we'll copy paste the correct values bellow
- # That way, we skip the expensive validation step required to add
- # a NOT NULL constraint at the end of the process
- add_column(table, tmp_column, :bigint, default: old_column.default || 0, null: false)
- else
- add_column(table, tmp_column, :bigint, default: old_column.default)
- end
-
- install_rename_triggers(table, column, tmp_column)
- end
-
- source_model = Class.new(ActiveRecord::Base) do
- include EachBatch
-
- self.table_name = table
- self.inheritance_column = :_type_disabled
+ unless column_exists?(table, tmp_column)
+ raise 'The temporary column does not exist, initialize it with `initialize_conversion_of_integer_to_bigint`'
end
- queue_background_migration_jobs_by_range_at_intervals(
- source_model,
+ batched_migration = queue_batched_background_migration(
'CopyColumnUsingBackgroundMigrationJob',
- interval,
+ table,
+ primary_key,
+ column,
+ tmp_column,
+ job_interval: interval,
batch_size: batch_size,
- other_job_arguments: [table, primary_key, sub_batch_size, column, tmp_column],
- track_jobs: true,
- primary_column_name: primary_key
- )
+ sub_batch_size: sub_batch_size)
if perform_background_migration_inline?
# To ensure the schema is up to date immediately we perform the
# migration inline in dev / test environments.
- Gitlab::BackgroundMigration.steal('CopyColumnUsingBackgroundMigrationJob')
+ Gitlab::Database::BackgroundMigration::BatchedMigrationRunner.new.run_entire_migration(batched_migration)
end
end
# Performs a concurrent column rename when using PostgreSQL.
- def install_rename_triggers_for_postgresql(trigger, table, old, new)
- execute <<-EOF.strip_heredoc
- CREATE OR REPLACE FUNCTION #{trigger}()
- RETURNS trigger AS
- $BODY$
- BEGIN
- NEW.#{new} := NEW.#{old};
- RETURN NEW;
- END;
- $BODY$
- LANGUAGE 'plpgsql'
- VOLATILE
- EOF
-
- execute <<-EOF.strip_heredoc
- DROP TRIGGER IF EXISTS #{trigger}
- ON #{table}
- EOF
-
- execute <<-EOF.strip_heredoc
- CREATE TRIGGER #{trigger}
- BEFORE INSERT OR UPDATE
- ON #{table}
- FOR EACH ROW
- EXECUTE FUNCTION #{trigger}()
- EOF
+ def install_rename_triggers_for_postgresql(table, old, new, trigger_name: nil)
+ Gitlab::Database::UnidirectionalCopyTrigger.on_table(table).create(old, new, trigger_name: trigger_name)
end
# Removes the triggers used for renaming a PostgreSQL column concurrently.
def remove_rename_triggers_for_postgresql(table, trigger)
- execute("DROP TRIGGER IF EXISTS #{trigger} ON #{table}")
- execute("DROP FUNCTION IF EXISTS #{trigger}()")
+ Gitlab::Database::UnidirectionalCopyTrigger.on_table(table).drop(trigger)
end
# Returns the (base) name to use for triggers when renaming columns.
def rename_trigger_name(table, old, new)
- 'trigger_' + Digest::SHA256.hexdigest("#{table}_#{old}_#{new}").first(12)
+ Gitlab::Database::UnidirectionalCopyTrigger.on_table(table).name(old, new)
end
# Returns an Array containing the indexes for the given column
diff --git a/lib/gitlab/database/migrations/background_migration_helpers.rb b/lib/gitlab/database/migrations/background_migration_helpers.rb
index e8cbea72887..8d5ea652bfc 100644
--- a/lib/gitlab/database/migrations/background_migration_helpers.rb
+++ b/lib/gitlab/database/migrations/background_migration_helpers.rb
@@ -190,7 +190,7 @@ module Gitlab
migration_status = batch_max_value.nil? ? :finished : :active
batch_max_value ||= batch_min_value
- Gitlab::Database::BackgroundMigration::BatchedMigration.create!(
+ migration = Gitlab::Database::BackgroundMigration::BatchedMigration.create!(
job_class_name: job_class_name,
table_name: batch_table_name,
column_name: batch_column_name,
@@ -202,6 +202,17 @@ module Gitlab
sub_batch_size: sub_batch_size,
job_arguments: job_arguments,
status: migration_status)
+
+ # This guard is necessary since #total_tuple_count was only introduced schema-wise,
+ # after this migration helper had been used for the first time.
+ return migration unless migration.respond_to?(:total_tuple_count)
+
+ # We keep track of the estimated number of tuples to reason later
+ # about the overall progress of a migration.
+ migration.total_tuple_count = Gitlab::Database::PgClass.for_table(batch_table_name)&.cardinality_estimate
+ migration.save!
+
+ migration
end
def perform_background_migration_inline?
@@ -236,6 +247,14 @@ module Gitlab
Gitlab::ApplicationContext.with_context(caller_id: self.class.to_s, &block)
end
+ def delete_queued_jobs(class_name)
+ Gitlab::BackgroundMigration.steal(class_name) do |job|
+ job.delete
+
+ false
+ end
+ end
+
private
def track_in_database(class_name, arguments)
diff --git a/lib/gitlab/database/partitioning_migration_helpers/foreign_key_helpers.rb b/lib/gitlab/database/partitioning_migration_helpers/foreign_key_helpers.rb
index 2def3a4d3a9..4402c42b136 100644
--- a/lib/gitlab/database/partitioning_migration_helpers/foreign_key_helpers.rb
+++ b/lib/gitlab/database/partitioning_migration_helpers/foreign_key_helpers.rb
@@ -6,6 +6,80 @@ module Gitlab
module ForeignKeyHelpers
include ::Gitlab::Database::SchemaHelpers
+ # Adds a foreign key with only minimal locking on the tables involved.
+ #
+ # In concept it works similarly to add_concurrent_foreign_key, but we have
+ # to add a special helper for partitioned tables for the following reasons:
+ # - add_concurrent_foreign_key sets the constraint to `NOT VALID`
+ # before validating it
+ # - Setting an FK to NOT VALID is not supported currently in Postgres (up to PG13)
+ # - Also, PostgreSQL will currently ignore NOT VALID constraints on partitions
+ # when adding a valid FK to the partitioned table, so they have to
+ # also be validated before we can add the final FK.
+ # Solution:
+ # - Add the foreign key first to each partition by using
+ # add_concurrent_foreign_key and validating it
+ # - Once all partitions have a foreign key, add it also to the partitioned
+ # table (there will be no need for a validation at that level)
+ # For those reasons, this method does not include an option to delay the
+ # validation, we have to force validate: true.
+ #
+ # source - The source (partitioned) table containing the foreign key.
+ # target - The target table the key points to.
+ # column - The name of the column to create the foreign key on.
+ # on_delete - The action to perform when associated data is removed,
+ # defaults to "CASCADE".
+ # name - The name of the foreign key.
+ #
+ def add_concurrent_partitioned_foreign_key(source, target, column:, on_delete: :cascade, name: nil)
+ partition_options = {
+ column: column,
+ on_delete: on_delete,
+
+ # We'll use the same FK name for all partitions and match it to
+ # the name used for the partitioned table to follow the convention
+ # used by PostgreSQL when adding FKs to new partitions
+ name: name.presence || concurrent_partitioned_foreign_key_name(source, column),
+
+ # Force the FK validation to true for partitions (and the partitioned table)
+ validate: true
+ }
+
+ if foreign_key_exists?(source, target, **partition_options)
+ warning_message = "Foreign key not created because it exists already " \
+ "(this may be due to an aborted migration or similar): " \
+ "source: #{source}, target: #{target}, column: #{partition_options[:column]}, "\
+ "name: #{partition_options[:name]}, on_delete: #{partition_options[:on_delete]}"
+
+ Gitlab::AppLogger.warn warning_message
+
+ return
+ end
+
+ partitioned_table = find_partitioned_table(source)
+
+ partitioned_table.postgres_partitions.order(:name).each do |partition|
+ add_concurrent_foreign_key(partition.identifier, target, **partition_options)
+ end
+
+ with_lock_retries do
+ add_foreign_key(source, target, **partition_options)
+ end
+ end
+
+ # Returns the name for a concurrent partitioned foreign key.
+ #
+ # Similar to concurrent_foreign_key_name (Gitlab::Database::MigrationHelpers)
+ # we just keep a separate method in case we want a different behavior
+ # for partitioned tables
+ #
+ def concurrent_partitioned_foreign_key_name(table, column, prefix: 'fk_rails_')
+ identifier = "#{table}_#{column}_fk"
+ hashed_identifier = Digest::SHA256.hexdigest(identifier).first(10)
+
+ "#{prefix}#{hashed_identifier}"
+ end
+
# Creates a "foreign key" that references a partitioned table. Because foreign keys referencing partitioned
# tables are not supported in PG11, this does not create a true database foreign key, but instead implements the
# same functionality at the database level by using triggers.
diff --git a/lib/gitlab/database/partitioning_migration_helpers/table_management_helpers.rb b/lib/gitlab/database/partitioning_migration_helpers/table_management_helpers.rb
index 1c289391e21..9ccbdc9930e 100644
--- a/lib/gitlab/database/partitioning_migration_helpers/table_management_helpers.rb
+++ b/lib/gitlab/database/partitioning_migration_helpers/table_management_helpers.rb
@@ -223,6 +223,28 @@ module Gitlab
replace_table(table_name, archived_table_name, partitioned_table_name, primary_key_name)
end
+ def drop_nonpartitioned_archive_table(table_name)
+ assert_table_is_allowed(table_name)
+
+ archived_table_name = make_archived_table_name(table_name)
+
+ with_lock_retries do
+ drop_sync_trigger(table_name)
+ end
+
+ drop_table(archived_table_name)
+ end
+
+ def create_trigger_to_sync_tables(source_table_name, partitioned_table_name, unique_key)
+ function_name = make_sync_function_name(source_table_name)
+ trigger_name = make_sync_trigger_name(source_table_name)
+
+ create_sync_function(function_name, partitioned_table_name, unique_key)
+ create_comment('FUNCTION', function_name, "Partitioning migration: table sync for #{source_table_name} table")
+
+ create_sync_trigger(source_table_name, trigger_name, function_name)
+ end
+
private
def assert_table_is_allowed(table_name)
@@ -316,16 +338,6 @@ module Gitlab
create_range_partition(partition_name, table_name, lower_bound, upper_bound)
end
- def create_trigger_to_sync_tables(source_table_name, partitioned_table_name, unique_key)
- function_name = make_sync_function_name(source_table_name)
- trigger_name = make_sync_trigger_name(source_table_name)
-
- create_sync_function(function_name, partitioned_table_name, unique_key)
- create_comment('FUNCTION', function_name, "Partitioning migration: table sync for #{source_table_name} table")
-
- create_sync_trigger(source_table_name, trigger_name, function_name)
- end
-
def drop_sync_trigger(source_table_name)
trigger_name = make_sync_trigger_name(source_table_name)
drop_trigger(source_table_name, trigger_name)
diff --git a/lib/gitlab/database/pg_class.rb b/lib/gitlab/database/pg_class.rb
new file mode 100644
index 00000000000..0ce9eebc14c
--- /dev/null
+++ b/lib/gitlab/database/pg_class.rb
@@ -0,0 +1,23 @@
+# frozen_string_literal: true
+
+module Gitlab
+ module Database
+ class PgClass < ActiveRecord::Base
+ self.table_name = 'pg_class'
+
+ def self.for_table(relname)
+ joins("LEFT JOIN pg_stat_user_tables ON pg_stat_user_tables.relid = pg_class.oid")
+ .where('schemaname = current_schema()')
+ .find_by(relname: relname)
+ end
+
+ def cardinality_estimate
+ tuples = reltuples.to_i
+
+ return if tuples < 1
+
+ tuples
+ end
+ end
+ end
+end
diff --git a/lib/gitlab/database/postgres_hll/batch_distinct_counter.rb b/lib/gitlab/database/postgres_hll/batch_distinct_counter.rb
index 62dfaeeaae3..e8b49c7f62c 100644
--- a/lib/gitlab/database/postgres_hll/batch_distinct_counter.rb
+++ b/lib/gitlab/database/postgres_hll/batch_distinct_counter.rb
@@ -41,19 +41,6 @@ module Gitlab
BUCKET_ID_MASK = (Buckets::TOTAL_BUCKETS - ZERO_OFFSET).to_s(2)
BIT_31_MASK = "B'0#{'1' * 31}'"
BIT_32_NORMALIZED_BUCKET_ID_MASK = "B'#{'0' * (32 - BUCKET_ID_MASK.size)}#{BUCKET_ID_MASK}'"
- # @example source_query
- # SELECT CAST(('X' || md5(CAST(%{column} as text))) as bit(32)) attr_hash_32_bits
- # FROM %{relation}
- # WHERE %{pkey} >= %{batch_start}
- # AND %{pkey} < %{batch_end}
- # AND %{column} IS NOT NULL
- BUCKETED_DATA_SQL = <<~SQL
- WITH hashed_attributes AS (%{source_query})
- SELECT (attr_hash_32_bits & #{BIT_32_NORMALIZED_BUCKET_ID_MASK})::int AS bucket_num,
- (31 - floor(log(2, min((attr_hash_32_bits & #{BIT_31_MASK})::int))))::int as bucket_hash
- FROM hashed_attributes
- GROUP BY 1
- SQL
WRONG_CONFIGURATION_ERROR = Class.new(ActiveRecord::StatementInvalid)
@@ -103,7 +90,7 @@ module Gitlab
def hll_buckets_for_batch(start, finish)
@relation
.connection
- .execute(BUCKETED_DATA_SQL % { source_query: source_query(start, finish) })
+ .execute(bucketed_data_sql % { source_query: source_query(start, finish) })
.map(&:values)
.to_h
end
@@ -139,6 +126,22 @@ module Gitlab
def actual_finish(finish)
finish || @relation.unscope(:group, :having).maximum(@relation.primary_key) || 0
end
+
+ # @example source_query
+ # SELECT CAST(('X' || md5(CAST(%{column} as text))) as bit(32)) attr_hash_32_bits
+ # FROM %{relation}
+ # WHERE %{pkey} >= %{batch_start}
+ # AND %{pkey} < %{batch_end}
+ # AND %{column} IS NOT NULL
+ def bucketed_data_sql
+ <<~SQL
+ WITH hashed_attributes AS #{Gitlab::Database::AsWithMaterialized.materialized_if_supported} (%{source_query})
+ SELECT (attr_hash_32_bits & #{BIT_32_NORMALIZED_BUCKET_ID_MASK})::int AS bucket_num,
+ (31 - floor(log(2, min((attr_hash_32_bits & #{BIT_31_MASK})::int))))::int as bucket_hash
+ FROM hashed_attributes
+ GROUP BY 1
+ SQL
+ end
end
end
end
diff --git a/lib/gitlab/database/similarity_score.rb b/lib/gitlab/database/similarity_score.rb
index 40845c0d5e0..20bf6fa4d30 100644
--- a/lib/gitlab/database/similarity_score.rb
+++ b/lib/gitlab/database/similarity_score.rb
@@ -10,7 +10,7 @@ module Gitlab
# Adds a "magic" comment in the generated SQL expression in order to be able to tell if we're sorting by similarity.
# Example: /* gitlab/database/similarity_score */ SIMILARITY(COALESCE...
- SIMILARITY_FUNCTION_CALL_WITH_ANNOTATION = "/* #{DISPLAY_NAME} */ SIMILARITY".freeze
+ SIMILARITY_FUNCTION_CALL_WITH_ANNOTATION = "/* #{DISPLAY_NAME} */ SIMILARITY"
# This method returns an Arel expression that can be used in an ActiveRecord query to order the resultset by similarity.
#
diff --git a/lib/gitlab/database/unidirectional_copy_trigger.rb b/lib/gitlab/database/unidirectional_copy_trigger.rb
new file mode 100644
index 00000000000..029c894a5ff
--- /dev/null
+++ b/lib/gitlab/database/unidirectional_copy_trigger.rb
@@ -0,0 +1,97 @@
+# frozen_string_literal: true
+
+module Gitlab
+ module Database
+ class UnidirectionalCopyTrigger
+ def self.on_table(table_name, connection: ActiveRecord::Base.connection)
+ new(table_name, connection)
+ end
+
+ def name(from_column_names, to_column_names)
+ from_column_names, to_column_names = check_column_names!(from_column_names, to_column_names)
+
+ unchecked_name(from_column_names, to_column_names)
+ end
+
+ def create(from_column_names, to_column_names, trigger_name: nil)
+ from_column_names, to_column_names = check_column_names!(from_column_names, to_column_names)
+ trigger_name ||= unchecked_name(from_column_names, to_column_names)
+
+ assignment_clauses = assignment_clauses_for_columns(from_column_names, to_column_names)
+
+ connection.execute(<<~SQL)
+ CREATE OR REPLACE FUNCTION #{trigger_name}()
+ RETURNS trigger AS
+ $BODY$
+ BEGIN
+ #{assignment_clauses};
+ RETURN NEW;
+ END;
+ $BODY$
+ LANGUAGE 'plpgsql'
+ VOLATILE
+ SQL
+
+ connection.execute(<<~SQL)
+ DROP TRIGGER IF EXISTS #{trigger_name}
+ ON #{quoted_table_name}
+ SQL
+
+ connection.execute(<<~SQL)
+ CREATE TRIGGER #{trigger_name}
+ BEFORE INSERT OR UPDATE
+ ON #{quoted_table_name}
+ FOR EACH ROW
+ EXECUTE FUNCTION #{trigger_name}()
+ SQL
+ end
+
+ def drop(trigger_name)
+ connection.execute("DROP TRIGGER IF EXISTS #{trigger_name} ON #{quoted_table_name}")
+ connection.execute("DROP FUNCTION IF EXISTS #{trigger_name}()")
+ end
+
+ private
+
+ attr_reader :table_name, :connection
+
+ def initialize(table_name, connection)
+ @table_name = table_name
+ @connection = connection
+ end
+
+ def quoted_table_name
+ @quoted_table_name ||= connection.quote_table_name(table_name)
+ end
+
+ def check_column_names!(from_column_names, to_column_names)
+ from_column_names = Array.wrap(from_column_names)
+ to_column_names = Array.wrap(to_column_names)
+
+ unless from_column_names.size == to_column_names.size
+ raise ArgumentError, 'number of source and destination columns must match'
+ end
+
+ [from_column_names, to_column_names]
+ end
+
+ def unchecked_name(from_column_names, to_column_names)
+ joined_column_names = from_column_names.zip(to_column_names).flatten.join('_')
+ 'trigger_' + Digest::SHA256.hexdigest("#{table_name}_#{joined_column_names}").first(12)
+ end
+
+ def assignment_clauses_for_columns(from_column_names, to_column_names)
+ combined_column_names = to_column_names.zip(from_column_names)
+
+ assignment_clauses = combined_column_names.map do |(new_name, old_name)|
+ new_name = connection.quote_column_name(new_name)
+ old_name = connection.quote_column_name(old_name)
+
+ "NEW.#{new_name} := NEW.#{old_name}"
+ end
+
+ assignment_clauses.join(";\n ")
+ end
+ end
+ end
+end