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/async_indexes/migration_helpers.rb54
-rw-r--r--lib/gitlab/database/background_migration/batched_job.rb2
-rw-r--r--lib/gitlab/database/background_migration/batched_migration.rb3
-rw-r--r--lib/gitlab/database/background_migration/health_status.rb5
-rw-r--r--lib/gitlab/database/background_migration/health_status/indicators/patroni_apdex.rb90
-rw-r--r--lib/gitlab/database/batch_count.rb2
-rw-r--r--lib/gitlab/database/load_balancing/action_cable_callbacks.rb4
-rw-r--r--lib/gitlab/database/load_balancing/connection_proxy.rb1
-rw-r--r--lib/gitlab/database/load_balancing/sidekiq_server_middleware.rb21
-rw-r--r--lib/gitlab/database/lock_writes_manager.rb10
-rw-r--r--lib/gitlab/database/migration_helpers.rb65
-rw-r--r--lib/gitlab/database/migration_helpers/convert_to_bigint.rb4
-rw-r--r--lib/gitlab/database/migration_helpers/loose_foreign_key_helpers.rb14
-rw-r--r--lib/gitlab/database/migration_helpers/v2.rb18
-rw-r--r--lib/gitlab/database/migration_helpers/wraparound_vacuum_helpers.rb90
-rw-r--r--lib/gitlab/database/migrations/pg_backend_pid.rb38
-rw-r--r--lib/gitlab/database/partitioning/convert_table_to_first_list_partition.rb70
-rw-r--r--lib/gitlab/database/partitioning_migration_helpers/backfill_partitioned_table.rb37
-rw-r--r--lib/gitlab/database/partitioning_migration_helpers/bulk_copy.rb42
-rw-r--r--lib/gitlab/database/partitioning_migration_helpers/table_management_helpers.rb32
-rw-r--r--lib/gitlab/database/postgres_foreign_key.rb2
-rw-r--r--lib/gitlab/database/query_analyzers/prevent_cross_database_modification.rb22
-rw-r--r--lib/gitlab/database/schema_helpers.rb6
-rw-r--r--lib/gitlab/database/schema_validation/adapters/column_database_adapter.rb43
-rw-r--r--lib/gitlab/database/schema_validation/adapters/column_structure_sql_adapter.rb114
-rw-r--r--lib/gitlab/database/schema_validation/database.rb42
-rw-r--r--lib/gitlab/database/schema_validation/inconsistency.rb61
-rw-r--r--lib/gitlab/database/schema_validation/pg_types.rb73
-rw-r--r--lib/gitlab/database/schema_validation/schema_inconsistency.rb15
-rw-r--r--lib/gitlab/database/schema_validation/schema_objects/base.rb4
-rw-r--r--lib/gitlab/database/schema_validation/schema_objects/column.rb23
-rw-r--r--lib/gitlab/database/schema_validation/schema_objects/table.rb40
-rw-r--r--lib/gitlab/database/schema_validation/structure_sql.rb24
-rw-r--r--lib/gitlab/database/schema_validation/track_inconsistency.rb77
-rw-r--r--lib/gitlab/database/schema_validation/validators/base_validator.rb13
-rw-r--r--lib/gitlab/database/schema_validation/validators/different_definition_indexes.rb4
-rw-r--r--lib/gitlab/database/schema_validation/validators/different_definition_tables.rb50
-rw-r--r--lib/gitlab/database/schema_validation/validators/different_definition_triggers.rb4
-rw-r--r--lib/gitlab/database/schema_validation/validators/extra_indexes.rb8
-rw-r--r--lib/gitlab/database/schema_validation/validators/extra_table_columns.rb32
-rw-r--r--lib/gitlab/database/schema_validation/validators/extra_tables.rb21
-rw-r--r--lib/gitlab/database/schema_validation/validators/extra_triggers.rb8
-rw-r--r--lib/gitlab/database/schema_validation/validators/missing_indexes.rb8
-rw-r--r--lib/gitlab/database/schema_validation/validators/missing_table_columns.rb32
-rw-r--r--lib/gitlab/database/schema_validation/validators/missing_tables.rb21
-rw-r--r--lib/gitlab/database/schema_validation/validators/missing_triggers.rb8
-rw-r--r--lib/gitlab/database/tables_locker.rb13
47 files changed, 1175 insertions, 195 deletions
diff --git a/lib/gitlab/database/async_indexes/migration_helpers.rb b/lib/gitlab/database/async_indexes/migration_helpers.rb
index f459c43e0ee..d7128a20a0b 100644
--- a/lib/gitlab/database/async_indexes/migration_helpers.rb
+++ b/lib/gitlab/database/async_indexes/migration_helpers.rb
@@ -77,6 +77,35 @@ module Gitlab
async_index
end
+ def prepare_async_index_from_sql(definition)
+ Gitlab::Database::QueryAnalyzers::RestrictAllowedSchemas.require_ddl_mode!
+
+ return unless async_index_creation_available?
+
+ table_name, index_name = extract_table_and_index_names_from_concurrent_index!(definition)
+
+ if index_name_exists?(table_name, index_name)
+ Gitlab::AppLogger.warn(
+ message: 'Index not prepared because it already exists',
+ table_name: table_name,
+ index_name: index_name)
+
+ return
+ end
+
+ async_index = Gitlab::Database::AsyncIndexes::PostgresAsyncIndex.find_or_create_by!(name: index_name) do |rec|
+ rec.table_name = table_name
+ rec.definition = definition
+ end
+
+ Gitlab::AppLogger.info(
+ message: 'Prepared index for async creation',
+ table_name: async_index.table_name,
+ index_name: async_index.name)
+
+ async_index
+ end
+
# Prepares an index for asynchronous destruction.
#
# Stores the index information in the postgres_async_indexes table to be removed later. The
@@ -110,7 +139,30 @@ module Gitlab
end
def async_index_creation_available?
- connection.table_exists?(:postgres_async_indexes)
+ table_exists?(:postgres_async_indexes)
+ end
+
+ private
+
+ delegate :table_exists?, to: :connection, private: true
+
+ def extract_table_and_index_names_from_concurrent_index!(definition)
+ statement = index_statement_from!(definition)
+
+ raise 'Index statement not found!' unless statement
+ raise 'Index must be created concurrently!' unless statement.concurrent
+ raise 'Table does not exist!' unless table_exists?(statement.relation.relname)
+
+ [statement.relation.relname, statement.idxname]
+ end
+
+ # This raises `PgQuery::ParseError` if the given statement
+ # is syntactically incorrect, therefore, validates that the
+ # index definition is correct.
+ def index_statement_from!(definition)
+ parsed_query = PgQuery.parse(definition)
+
+ parsed_query.tree.stmts[0].stmt.index_stmt
end
end
end
diff --git a/lib/gitlab/database/background_migration/batched_job.rb b/lib/gitlab/database/background_migration/batched_job.rb
index 5147ea92291..523ab2a9f27 100644
--- a/lib/gitlab/database/background_migration/batched_job.rb
+++ b/lib/gitlab/database/background_migration/batched_job.rb
@@ -130,8 +130,6 @@ module Gitlab
end
def can_reduce_sub_batch_size?
- return false unless Feature.enabled?(:reduce_sub_batch_size_on_timeouts)
-
still_retryable? && within_batch_size_boundaries?
end
diff --git a/lib/gitlab/database/background_migration/batched_migration.rb b/lib/gitlab/database/background_migration/batched_migration.rb
index 429dc79e170..a883996a5c5 100644
--- a/lib/gitlab/database/background_migration/batched_migration.rb
+++ b/lib/gitlab/database/background_migration/batched_migration.rb
@@ -25,6 +25,7 @@ module Gitlab
scope :queue_order, -> { order(id: :asc) }
scope :queued, -> { with_statuses(:active, :paused) }
+ scope :finalizing, -> { with_status(:finalizing) }
scope :ordered_by_created_at_desc, -> { order(created_at: :desc) }
# on_hold_until is a temporary runtime status which puts execution "on hold"
@@ -219,7 +220,7 @@ module Gitlab
end
def health_context
- HealthStatus::Context.new(connection, [table_name])
+ HealthStatus::Context.new(connection, [table_name], gitlab_schema.to_sym)
end
def hold!(until_time: 10.minutes.from_now)
diff --git a/lib/gitlab/database/background_migration/health_status.rb b/lib/gitlab/database/background_migration/health_status.rb
index 506d2996ad5..c66f30ffecc 100644
--- a/lib/gitlab/database/background_migration/health_status.rb
+++ b/lib/gitlab/database/background_migration/health_status.rb
@@ -6,11 +6,12 @@ module Gitlab
module HealthStatus
DEFAULT_INIDICATORS = [
Indicators::AutovacuumActiveOnTable,
- Indicators::WriteAheadLog
+ Indicators::WriteAheadLog,
+ Indicators::PatroniApdex
].freeze
# Rather than passing along the migration, we use a more explicitly defined context
- Context = Struct.new(:connection, :tables)
+ Context = Struct.new(:connection, :tables, :gitlab_schema)
def self.evaluate(migration, indicators = DEFAULT_INIDICATORS)
indicators.map do |indicator|
diff --git a/lib/gitlab/database/background_migration/health_status/indicators/patroni_apdex.rb b/lib/gitlab/database/background_migration/health_status/indicators/patroni_apdex.rb
new file mode 100644
index 00000000000..0dd6dd5c2a4
--- /dev/null
+++ b/lib/gitlab/database/background_migration/health_status/indicators/patroni_apdex.rb
@@ -0,0 +1,90 @@
+# frozen_string_literal: true
+
+module Gitlab
+ module Database
+ module BackgroundMigration
+ module HealthStatus
+ module Indicators
+ class PatroniApdex
+ include Gitlab::Utils::StrongMemoize
+
+ def initialize(context)
+ @context = context
+ end
+
+ def evaluate
+ return Signals::NotAvailable.new(self.class, reason: 'indicator disabled') unless enabled?
+
+ connection_error_message = fetch_connection_error_message
+ return unknown_signal(connection_error_message) if connection_error_message.present?
+
+ apdex_sli = fetch_sli(apdex_sli_query)
+ return unknown_signal('Patroni service apdex can not be calculated') unless apdex_sli.present?
+
+ if apdex_sli.to_f > apdex_slo.to_f
+ Signals::Normal.new(self.class, reason: 'Patroni service apdex is above SLO')
+ else
+ Signals::Stop.new(self.class, reason: 'Patroni service apdex is below SLO')
+ end
+ end
+
+ private
+
+ attr_reader :context
+
+ def enabled?
+ Feature.enabled?(:batched_migrations_health_status_patroni_apdex, type: :ops)
+ end
+
+ def unknown_signal(reason)
+ Signals::Unknown.new(self.class, reason: reason)
+ end
+
+ def fetch_connection_error_message
+ return 'Patroni Apdex Settings not configured' unless database_apdex_settings.present?
+ return 'Prometheus client is not ready' unless client.ready?
+ return 'Apdex SLI query is not configured' unless apdex_sli_query
+ return 'Apdex SLO is not configured' unless apdex_slo
+ end
+
+ def client
+ @client ||= Gitlab::PrometheusClient.new(
+ database_apdex_settings[:prometheus_api_url],
+ allow_local_requests: true,
+ verify: true
+ )
+ end
+
+ def database_apdex_settings
+ @database_apdex_settings ||= Gitlab::CurrentSettings.database_apdex_settings&.with_indifferent_access
+ end
+
+ def apdex_sli_query
+ {
+ gitlab_main: database_apdex_settings[:apdex_sli_query][:main],
+ gitlab_ci: database_apdex_settings[:apdex_sli_query][:ci]
+ }.fetch(context.gitlab_schema.to_sym)
+ end
+ strong_memoize_attr :apdex_sli_query
+
+ def apdex_slo
+ {
+ gitlab_main: database_apdex_settings[:apdex_slo][:main],
+ gitlab_ci: database_apdex_settings[:apdex_slo][:ci]
+ }.fetch(context.gitlab_schema.to_sym)
+ end
+ strong_memoize_attr :apdex_slo
+
+ def fetch_sli(query)
+ response = client.query(query)
+ metric = response&.first || {}
+ value = metric.fetch('value', [])
+
+ Array.wrap(value).second
+ end
+ end
+ end
+ end
+ end
+ end
+end
diff --git a/lib/gitlab/database/batch_count.rb b/lib/gitlab/database/batch_count.rb
index 7a064fb4005..7249cb3e73b 100644
--- a/lib/gitlab/database/batch_count.rb
+++ b/lib/gitlab/database/batch_count.rb
@@ -27,7 +27,7 @@
# batch_sum(User, :sign_in_count)
# batch_sum(Issue.group(:state_id), :weight))
# batch_average(Ci::Pipeline, :duration)
-# batch_average(MergeTrain.group(:status), :duration)
+# batch_average(MergeTrains::Car.group(:status), :duration)
module Gitlab
module Database
module BatchCount
diff --git a/lib/gitlab/database/load_balancing/action_cable_callbacks.rb b/lib/gitlab/database/load_balancing/action_cable_callbacks.rb
index 7164976ff73..fab691117ad 100644
--- a/lib/gitlab/database/load_balancing/action_cable_callbacks.rb
+++ b/lib/gitlab/database/load_balancing/action_cable_callbacks.rb
@@ -6,14 +6,10 @@ module Gitlab
module ActionCableCallbacks
def self.install
::ActionCable::Server::Worker.set_callback :work, :around, &wrapper
- ::ActionCable::Channel::Base.set_callback :subscribe, :around, &wrapper
- ::ActionCable::Channel::Base.set_callback :unsubscribe, :around, &wrapper
end
def self.wrapper
lambda do |_, inner|
- ::Gitlab::Database::LoadBalancing::Session.current.use_primary!
-
inner.call
ensure
::Gitlab::Database::LoadBalancing.release_hosts
diff --git a/lib/gitlab/database/load_balancing/connection_proxy.rb b/lib/gitlab/database/load_balancing/connection_proxy.rb
index 622e310ead3..0d39b47dbba 100644
--- a/lib/gitlab/database/load_balancing/connection_proxy.rb
+++ b/lib/gitlab/database/load_balancing/connection_proxy.rb
@@ -32,6 +32,7 @@ module Gitlab
select_one
select_rows
quote_column_name
+ schema_cache
).freeze
# hosts - The hosts to use for load balancing.
diff --git a/lib/gitlab/database/load_balancing/sidekiq_server_middleware.rb b/lib/gitlab/database/load_balancing/sidekiq_server_middleware.rb
index f7b8d2514ba..95e21c40795 100644
--- a/lib/gitlab/database/load_balancing/sidekiq_server_middleware.rb
+++ b/lib/gitlab/database/load_balancing/sidekiq_server_middleware.rb
@@ -6,7 +6,7 @@ module Gitlab
class SidekiqServerMiddleware
JobReplicaNotUpToDate = Class.new(::Gitlab::SidekiqMiddleware::RetryError)
- MINIMUM_DELAY_INTERVAL_SECONDS = 0.8
+ REPLICA_WAIT_SLEEP_SECONDS = 0.5
def call(worker, job, _queue)
worker_class = worker.class
@@ -18,7 +18,7 @@ module Gitlab
::Gitlab::Database::LoadBalancing::Session.current.use_primary!
elsif strategy == :retry
raise JobReplicaNotUpToDate, "Sidekiq job #{worker_class} JID-#{job['jid']} couldn't use the replica."\
- " Replica was not up to date."
+ " Replica was not up to date."
else
# this means we selected an up-to-date replica, but there is nothing to do in this case.
end
@@ -49,7 +49,10 @@ module Gitlab
# Happy case: we can read from a replica.
return replica_strategy(worker_class, job) if databases_in_sync?(wal_locations)
- sleep_if_needed(job)
+ 3.times do
+ sleep REPLICA_WAIT_SLEEP_SECONDS
+ break if databases_in_sync?(wal_locations)
+ end
if databases_in_sync?(wal_locations)
replica_strategy(worker_class, job)
@@ -62,12 +65,6 @@ module Gitlab
end
end
- def sleep_if_needed(job)
- remaining_delay = MINIMUM_DELAY_INTERVAL_SECONDS - (Time.current.to_f - job['created_at'].to_f)
-
- sleep remaining_delay if remaining_delay > 0 && remaining_delay < MINIMUM_DELAY_INTERVAL_SECONDS
- end
-
def get_wal_locations(job)
job['dedup_wal_locations'] || job['wal_locations']
end
@@ -79,7 +76,7 @@ module Gitlab
end
def can_retry?(worker_class, job)
- worker_class.get_data_consistency == :delayed && not_yet_retried?(job)
+ worker_class.get_data_consistency == :delayed && not_yet_requeued?(job)
end
def replica_strategy(worker_class, job)
@@ -87,10 +84,10 @@ module Gitlab
end
def retried_before?(worker_class, job)
- worker_class.get_data_consistency == :delayed && !not_yet_retried?(job)
+ worker_class.get_data_consistency == :delayed && !not_yet_requeued?(job)
end
- def not_yet_retried?(job)
+ def not_yet_requeued?(job)
# if `retry_count` is `nil` it indicates that this job was never retried
# the `0` indicates that this is a first retry
job['retry_count'].nil?
diff --git a/lib/gitlab/database/lock_writes_manager.rb b/lib/gitlab/database/lock_writes_manager.rb
index e8f7b51955d..7e429387ae6 100644
--- a/lib/gitlab/database/lock_writes_manager.rb
+++ b/lib/gitlab/database/lock_writes_manager.rb
@@ -38,7 +38,7 @@ module Gitlab
def lock_writes
if table_locked_for_writes?
logger&.info "Skipping lock_writes, because #{table_name} is already locked for writes"
- return
+ return result_hash(action: 'skipped')
end
logger&.info "Database: '#{database_name}', Table: '#{table_name}': Lock Writes".color(:yellow)
@@ -50,6 +50,8 @@ module Gitlab
SQL
execute_sql_statement(sql_statement)
+
+ result_hash(action: 'locked')
end
def unlock_writes
@@ -59,6 +61,8 @@ module Gitlab
SQL
execute_sql_statement(sql_statement)
+
+ result_hash(action: 'unlocked')
end
private
@@ -113,6 +117,10 @@ module Gitlab
def write_trigger_name
"gitlab_schema_write_trigger_for_#{table_name_without_schema}"
end
+
+ def result_hash(action:)
+ { action: action, database: database_name, table: table_name, dry_run: dry_run }
+ end
end
end
end
diff --git a/lib/gitlab/database/migration_helpers.rb b/lib/gitlab/database/migration_helpers.rb
index 3a342abe65d..291f483e6e4 100644
--- a/lib/gitlab/database/migration_helpers.rb
+++ b/lib/gitlab/database/migration_helpers.rb
@@ -15,6 +15,7 @@ module Gitlab
include RenameTableHelpers
include AsyncIndexes::MigrationHelpers
include AsyncConstraints::MigrationHelpers
+ include WraparoundVacuumHelpers
def define_batchable_model(table_name, connection: self.connection)
super(table_name, connection: connection)
@@ -79,63 +80,6 @@ module Gitlab
end
end
- # @deprecated Use `create_table` in V2 instead
- #
- # Creates a new table, optionally allowing the caller to add check constraints to the table.
- # Aside from that addition, this method should behave identically to Rails' `create_table` method.
- #
- # Example:
- #
- # create_table_with_constraints :some_table do |t|
- # t.integer :thing, null: false
- # t.text :other_thing
- #
- # t.check_constraint :thing_is_not_null, 'thing IS NOT NULL'
- # t.text_limit :other_thing, 255
- # end
- #
- # See Rails' `create_table` for more info on the available arguments.
- def create_table_with_constraints(table_name, **options, &block)
- helper_context = self
-
- with_lock_retries do
- check_constraints = []
-
- create_table(table_name, **options) do |t|
- t.define_singleton_method(:check_constraint) do |name, definition|
- helper_context.send(:validate_check_constraint_name!, name) # rubocop:disable GitlabSecurity/PublicSend
-
- check_constraints << { name: name, definition: definition }
- end
-
- t.define_singleton_method(:text_limit) do |column_name, limit, name: nil|
- # rubocop:disable GitlabSecurity/PublicSend
- name = helper_context.send(:text_limit_name, table_name, column_name, name: name)
- helper_context.send(:validate_check_constraint_name!, name)
- # rubocop:enable GitlabSecurity/PublicSend
-
- column_name = helper_context.quote_column_name(column_name)
- definition = "char_length(#{column_name}) <= #{limit}"
-
- check_constraints << { name: name, definition: definition }
- end
-
- t.instance_eval(&block) unless block.nil?
- end
-
- next if check_constraints.empty?
-
- constraint_clauses = check_constraints.map do |constraint|
- "ADD CONSTRAINT #{quote_table_name(constraint[:name])} CHECK (#{constraint[:definition]})"
- end
-
- execute(<<~SQL)
- ALTER TABLE #{quote_table_name(table_name)}
- #{constraint_clauses.join(",\n")}
- SQL
- end
- end
-
# Creates a new index, concurrently
#
# Example:
@@ -373,6 +317,13 @@ module Gitlab
end
end
+ # Since we may be migrating in one go from a previous version without
+ # `constrained_table_name` then we may see that this column exists
+ # (as above) but the schema cache is still outdated for the model.
+ unless Gitlab::Database::PostgresForeignKey.column_names.include?('constrained_table_name')
+ Gitlab::Database::PostgresForeignKey.reset_column_information
+ end
+
fks = Gitlab::Database::PostgresForeignKey.by_constrained_table_name_or_identifier(source)
fks = fks.by_referenced_table_name(target) if target
diff --git a/lib/gitlab/database/migration_helpers/convert_to_bigint.rb b/lib/gitlab/database/migration_helpers/convert_to_bigint.rb
index cf5640deb3d..63928d7dc09 100644
--- a/lib/gitlab/database/migration_helpers/convert_to_bigint.rb
+++ b/lib/gitlab/database/migration_helpers/convert_to_bigint.rb
@@ -11,7 +11,9 @@ module Gitlab
#
# Once we are done with the PK conversions we can remove this.
def com_or_dev_or_test_but_not_jh?
- !Gitlab.jh? && (Gitlab.com? || Gitlab.dev_or_test_env?)
+ return true if Gitlab.dev_or_test_env?
+
+ Gitlab.com? && !Gitlab.jh?
end
end
end
diff --git a/lib/gitlab/database/migration_helpers/loose_foreign_key_helpers.rb b/lib/gitlab/database/migration_helpers/loose_foreign_key_helpers.rb
index 30601bffd7a..2221aea9f46 100644
--- a/lib/gitlab/database/migration_helpers/loose_foreign_key_helpers.rb
+++ b/lib/gitlab/database/migration_helpers/loose_foreign_key_helpers.rb
@@ -9,11 +9,11 @@ module Gitlab
DELETED_RECORDS_INSERT_FUNCTION_NAME = 'insert_into_loose_foreign_keys_deleted_records'
def track_record_deletions(table)
- execute(<<~SQL)
- CREATE TRIGGER #{record_deletion_trigger_name(table)}
- AFTER DELETE ON #{table} REFERENCING OLD TABLE AS old_table
- FOR EACH STATEMENT
- EXECUTE FUNCTION #{DELETED_RECORDS_INSERT_FUNCTION_NAME}();
+ execute(<<~SQL.squish)
+ CREATE TRIGGER #{record_deletion_trigger_name(table)}
+ AFTER DELETE ON #{table} REFERENCING OLD TABLE AS old_table
+ FOR EACH STATEMENT
+ EXECUTE FUNCTION #{DELETED_RECORDS_INSERT_FUNCTION_NAME}();
SQL
end
@@ -21,6 +21,10 @@ module Gitlab
drop_trigger(table, record_deletion_trigger_name(table))
end
+ def has_loose_foreign_key?(table)
+ trigger_exists?(table, record_deletion_trigger_name(table))
+ end
+
private
def record_deletion_trigger_name(table)
diff --git a/lib/gitlab/database/migration_helpers/v2.rb b/lib/gitlab/database/migration_helpers/v2.rb
index b5b8b58681c..ef48d601eb9 100644
--- a/lib/gitlab/database/migration_helpers/v2.rb
+++ b/lib/gitlab/database/migration_helpers/v2.rb
@@ -5,24 +5,6 @@ module Gitlab
module MigrationHelpers
module V2
include Gitlab::Database::MigrationHelpers
-
- # Superseded by `create_table` override below
- def create_table_with_constraints(*_)
- raise <<~EOM
- #create_table_with_constraints is not supported anymore - use #create_table instead, for example:
-
- create_table :db_guides do |t|
- t.bigint :stars, default: 0, null: false
- t.text :title, limit: 128
- t.text :notes, limit: 1024
-
- t.check_constraint 'stars > 1000', name: 'so_many_stars'
- end
-
- See https://docs.gitlab.com/ee/development/database/strings_and_the_text_data_type.html
- EOM
- end
-
# Creates a new table, optionally allowing the caller to add text limit constraints to the table.
# This method only extends Rails' `create_table` method
#
diff --git a/lib/gitlab/database/migration_helpers/wraparound_vacuum_helpers.rb b/lib/gitlab/database/migration_helpers/wraparound_vacuum_helpers.rb
new file mode 100644
index 00000000000..01ff3dcbfb8
--- /dev/null
+++ b/lib/gitlab/database/migration_helpers/wraparound_vacuum_helpers.rb
@@ -0,0 +1,90 @@
+# frozen_string_literal: true
+
+module Gitlab
+ module Database
+ module MigrationHelpers
+ module WraparoundVacuumHelpers
+ class WraparoundCheck
+ WraparoundError = Class.new(StandardError)
+
+ def initialize(table_name, migration:)
+ @migration = migration
+ @table_name = table_name
+
+ validate_table_existence!
+ end
+
+ def execute
+ return if disabled?
+ return unless wraparound_vacuum.present?
+
+ log "Autovacuum with wraparound prevention mode is running on `#{table_name}`", title: true
+ log "This process prevents the migration from acquiring the necessary locks"
+ log "Query: `#{wraparound_vacuum[:query]}`"
+ log "Current duration: #{wraparound_vacuum[:duration].inspect}"
+ log "Process id: #{wraparound_vacuum[:pid]}"
+ log "You can wait until it completes or if absolutely necessary interrupt it using: " \
+ "`select pg_cancel_backend(#{wraparound_vacuum[:pid]});`"
+ log "Be aware that a new process will kick in immediately, so multiple interruptions " \
+ "might be required to time it right with the locks retry mechanism"
+ end
+
+ private
+
+ attr_reader :table_name
+
+ delegate :say, :connection, to: :@migration
+
+ def wraparound_vacuum
+ @wraparound_vacuum ||= transform_wraparound_vacuum
+ end
+
+ def transform_wraparound_vacuum
+ result = raw_wraparound_vacuum
+ values = Array.wrap(result.cast_values.first)
+
+ result.columns.zip(values).to_h.with_indifferent_access.compact
+ end
+
+ def raw_wraparound_vacuum
+ connection.select_all(<<~SQL.squish)
+ SELECT pid, state, age(clock_timestamp(), query_start) as duration, query
+ FROM pg_stat_activity
+ WHERE query ILIKE '%VACUUM%' || #{quoted_table_name} || '%(to prevent wraparound)'
+ AND backend_type = 'autovacuum worker'
+ LIMIT 1
+ SQL
+ end
+
+ def validate_table_existence!
+ return if connection.table_exists?(table_name)
+
+ raise WraparoundError, "Table #{table_name} does not exist"
+ end
+
+ def quoted_table_name
+ connection.quote(table_name)
+ end
+
+ def disabled?
+ return true unless wraparound_check_allowed?
+
+ Gitlab::Utils.to_boolean(ENV['GITLAB_MIGRATIONS_DISABLE_WRAPAROUND_CHECK'])
+ end
+
+ def wraparound_check_allowed?
+ Gitlab.com? || Gitlab.dev_or_test_env?
+ end
+
+ def log(text, title: false)
+ say text, !title
+ end
+ end
+
+ def check_if_wraparound_in_progress(table_name)
+ WraparoundCheck.new(table_name, migration: self).execute
+ end
+ end
+ end
+ end
+end
diff --git a/lib/gitlab/database/migrations/pg_backend_pid.rb b/lib/gitlab/database/migrations/pg_backend_pid.rb
new file mode 100644
index 00000000000..0c15aae9395
--- /dev/null
+++ b/lib/gitlab/database/migrations/pg_backend_pid.rb
@@ -0,0 +1,38 @@
+# frozen_string_literal: true
+
+module Gitlab
+ module Database
+ module Migrations
+ module PgBackendPid
+ module MigratorPgBackendPid
+ extend ::Gitlab::Utils::Override
+
+ override :with_advisory_lock_connection
+ def with_advisory_lock_connection
+ super do |conn|
+ Gitlab::Database::Migrations::PgBackendPid.say(conn)
+
+ yield(conn)
+
+ Gitlab::Database::Migrations::PgBackendPid.say(conn)
+ end
+ end
+ end
+
+ def self.patch!
+ ActiveRecord::Migrator.prepend(MigratorPgBackendPid)
+ end
+
+ def self.say(conn)
+ pg_backend_pid = conn.select_value('SELECT pg_backend_pid()')
+ db_name = Gitlab::Database.db_config_name(conn)
+
+ # rubocop:disable Rails/Output
+ puts "#{db_name}: == [advisory_lock_connection] " \
+ "object_id: #{conn.object_id}, pg_backend_pid: #{pg_backend_pid}"
+ # rubocop:enable Rails/Output
+ end
+ end
+ end
+ end
+end
diff --git a/lib/gitlab/database/partitioning/convert_table_to_first_list_partition.rb b/lib/gitlab/database/partitioning/convert_table_to_first_list_partition.rb
index 58447481e60..afca2368126 100644
--- a/lib/gitlab/database/partitioning/convert_table_to_first_list_partition.rb
+++ b/lib/gitlab/database/partitioning/convert_table_to_first_list_partition.rb
@@ -8,6 +8,8 @@ module Gitlab
SQL_STATEMENT_SEPARATOR = ";\n\n"
+ PARTITIONING_CONSTRAINT_NAME = 'partitioning_constraint'
+
attr_reader :partitioning_column, :table_name, :parent_table_name, :zero_partition_value
def initialize(
@@ -23,10 +25,10 @@ module Gitlab
@lock_tables = Array.wrap(lock_tables)
end
- def prepare_for_partitioning
+ def prepare_for_partitioning(async: false)
assert_existing_constraints_partitionable
- add_partitioning_check_constraint
+ add_partitioning_check_constraint(async: async)
end
def revert_preparation_for_partitioning
@@ -36,6 +38,7 @@ module Gitlab
def partition
assert_existing_constraints_partitionable
assert_partitioning_constraint_present
+
create_parent_table
attach_foreign_keys_to_parent
@@ -45,7 +48,9 @@ module Gitlab
}
migration_context.with_lock_retries(**lock_args) do
- migration_context.execute(sql_to_convert_table)
+ redefine_loose_foreign_key_triggers do
+ migration_context.execute(sql_to_convert_table)
+ end
end
end
@@ -118,16 +123,17 @@ module Gitlab
constraints_on_column = Gitlab::Database::PostgresConstraint
.by_table_identifier(table_identifier)
.check_constraints
- .valid
.including_column(partitioning_column)
- constraints_on_column.to_a.find do |constraint|
- constraint.definition == "CHECK ((#{partitioning_column} = #{zero_partition_value}))"
+ check_body = "CHECK ((#{partitioning_column} = #{zero_partition_value}))"
+
+ constraints_on_column.find do |constraint|
+ constraint.definition.start_with?(check_body)
end
end
def assert_partitioning_constraint_present
- return if partitioning_constraint
+ return if partitioning_constraint&.constraint_valid?
raise UnableToPartition, <<~MSG
Table #{table_name} is not ready for partitioning.
@@ -135,14 +141,43 @@ module Gitlab
MSG
end
- def add_partitioning_check_constraint
- return if partitioning_constraint.present?
+ def add_partitioning_check_constraint(async: false)
+ return validate_partitioning_constraint_synchronously if partitioning_constraint.present?
check_body = "#{partitioning_column} = #{connection.quote(zero_partition_value)}"
# Any constraint name would work. The constraint is found based on its definition before partitioning
- migration_context.add_check_constraint(table_name, check_body, 'partitioning_constraint')
+ migration_context.add_check_constraint(
+ table_name, check_body, PARTITIONING_CONSTRAINT_NAME,
+ validate: !async
+ )
+
+ if async
+ migration_context.prepare_async_check_constraint_validation(
+ table_name, name: PARTITIONING_CONSTRAINT_NAME
+ )
+ end
+
+ return if partitioning_constraint.present?
- raise UnableToPartition, 'Error adding partitioning constraint' unless partitioning_constraint.present?
+ raise UnableToPartition, <<~MSG
+ Error adding partitioning constraint `#{PARTITIONING_CONSTRAINT_NAME}` for `#{table_name}`
+ MSG
+ end
+
+ def validate_partitioning_constraint_synchronously
+ if partitioning_constraint.constraint_valid?
+ return Gitlab::AppLogger.info <<~MSG
+ Nothing to do, the partitioning constraint exists and is valid for `#{table_name}`
+ MSG
+ end
+
+ # Async validations are executed only on .com, we need to validate synchronously for self-managed
+ migration_context.validate_check_constraint(table_name, partitioning_constraint.name)
+ return if partitioning_constraint.constraint_valid?
+
+ raise UnableToPartition, <<~MSG
+ Error validating partitioning constraint `#{partitioning_constraint.name}` for `#{table_name}`
+ MSG
end
def create_parent_table
@@ -262,6 +297,19 @@ module Gitlab
iterations + aggressive_iterations
end
+
+ def redefine_loose_foreign_key_triggers
+ if migration_context.has_loose_foreign_key?(table_name)
+ migration_context.untrack_record_deletions(table_name)
+
+ yield if block_given?
+
+ migration_context.track_record_deletions(parent_table_name)
+ migration_context.track_record_deletions(table_name)
+ elsif block_given?
+ yield
+ end
+ end
end
end
end
diff --git a/lib/gitlab/database/partitioning_migration_helpers/backfill_partitioned_table.rb b/lib/gitlab/database/partitioning_migration_helpers/backfill_partitioned_table.rb
index dcf457b9d63..e87707953ae 100644
--- a/lib/gitlab/database/partitioning_migration_helpers/backfill_partitioned_table.rb
+++ b/lib/gitlab/database/partitioning_migration_helpers/backfill_partitioned_table.rb
@@ -21,7 +21,7 @@ module Gitlab
return
end
- bulk_copy = BulkCopy.new(source_table, partitioned_table, source_column, connection: connection)
+ bulk_copy = Gitlab::Database::PartitioningMigrationHelpers::BulkCopy.new(source_table, partitioned_table, source_column, connection: connection)
parent_batch_relation = relation_scoped_to_range(source_table, source_column, start_id, stop_id)
parent_batch_relation.each_batch(of: SUB_BATCH_SIZE) do |sub_batch|
@@ -56,41 +56,6 @@ module Gitlab
def mark_jobs_as_succeeded(*arguments)
BackgroundMigrationJob.mark_all_as_succeeded(self.class.name, arguments)
end
-
- # Helper class to copy data between two tables via upserts
- class BulkCopy
- DELIMITER = ', '
-
- attr_reader :source_table, :destination_table, :source_column, :connection
-
- def initialize(source_table, destination_table, source_column, connection:)
- @source_table = source_table
- @destination_table = destination_table
- @source_column = source_column
- @connection = connection
- end
-
- def copy_between(start_id, stop_id)
- connection.execute(<<~SQL)
- INSERT INTO #{destination_table} (#{column_listing})
- SELECT #{column_listing}
- FROM #{source_table}
- WHERE #{source_column} BETWEEN #{start_id} AND #{stop_id}
- FOR UPDATE
- ON CONFLICT (#{conflict_targets}) DO NOTHING
- SQL
- end
-
- private
-
- def column_listing
- @column_listing ||= connection.columns(source_table).map(&:name).join(DELIMITER)
- end
-
- def conflict_targets
- connection.primary_key(destination_table).join(DELIMITER)
- end
- end
end
end
end
diff --git a/lib/gitlab/database/partitioning_migration_helpers/bulk_copy.rb b/lib/gitlab/database/partitioning_migration_helpers/bulk_copy.rb
new file mode 100644
index 00000000000..b8f5a2e3ad4
--- /dev/null
+++ b/lib/gitlab/database/partitioning_migration_helpers/bulk_copy.rb
@@ -0,0 +1,42 @@
+# frozen_string_literal: true
+
+module Gitlab
+ module Database
+ module PartitioningMigrationHelpers
+ # Helper class to copy data between two tables via upserts
+ class BulkCopy
+ DELIMITER = ', '
+
+ attr_reader :source_table, :destination_table, :source_column, :connection
+
+ def initialize(source_table, destination_table, source_column, connection:)
+ @source_table = source_table
+ @destination_table = destination_table
+ @source_column = source_column
+ @connection = connection
+ end
+
+ def copy_between(start_id, stop_id)
+ connection.execute(<<~SQL)
+ INSERT INTO #{destination_table} (#{column_listing})
+ SELECT #{column_listing}
+ FROM #{source_table}
+ WHERE #{source_column} BETWEEN #{start_id} AND #{stop_id}
+ FOR UPDATE
+ ON CONFLICT (#{conflict_targets}) DO NOTHING
+ SQL
+ end
+
+ private
+
+ def column_listing
+ @column_listing ||= connection.columns(source_table).map(&:name).join(DELIMITER)
+ end
+
+ def conflict_targets
+ connection.primary_keys(destination_table).join(DELIMITER)
+ end
+ end
+ end
+ end
+end
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 f9790bf53b9..e3cf1298df6 100644
--- a/lib/gitlab/database/partitioning_migration_helpers/table_management_helpers.rb
+++ b/lib/gitlab/database/partitioning_migration_helpers/table_management_helpers.rb
@@ -6,13 +6,16 @@ module Gitlab
module TableManagementHelpers
include ::Gitlab::Database::SchemaHelpers
include ::Gitlab::Database::MigrationHelpers
+ include ::Gitlab::Database::MigrationHelpers::LooseForeignKeyHelpers
ALLOWED_TABLES = %w[audit_events web_hook_logs].freeze
ERROR_SCOPE = 'table partitioning'
MIGRATION_CLASS_NAME = "::#{module_parent_name}::BackfillPartitionedTable"
+ MIGRATION = "BackfillPartitionedTable"
BATCH_INTERVAL = 2.minutes.freeze
BATCH_SIZE = 50_000
+ SUB_BATCH_SIZE = 2_500
JobArguments = Struct.new(:start_id, :stop_id, :source_table_name, :partitioned_table_name, :source_column) do
def self.from_array(arguments)
@@ -107,7 +110,16 @@ module Gitlab
partitioned_table_name = make_partitioned_table_name(table_name)
primary_key = connection.primary_key(table_name)
- enqueue_background_migration(table_name, partitioned_table_name, primary_key)
+
+ queue_batched_background_migration(
+ MIGRATION,
+ table_name,
+ primary_key,
+ partitioned_table_name,
+ batch_size: BATCH_SIZE,
+ sub_batch_size: SUB_BATCH_SIZE,
+ job_interval: BATCH_INTERVAL
+ )
end
# Cleanup a previously enqueued background migration to copy data into a partitioned table. This will not
@@ -149,7 +161,7 @@ module Gitlab
# 2. Inline copy any missed rows from the original table to the partitioned table
#
# **NOTE** Migrations using this method cannot be scheduled in the same release as the migration that
- # schedules the background migration using the `enqueue_background_migration` helper, or else the
+ # schedules the background migration using the `enqueue_partitioning_data_migration` helper, or else the
# background migration jobs will be force-executed.
#
# Example:
@@ -251,7 +263,7 @@ module Gitlab
create_sync_trigger(source_table_name, trigger_name, function_name)
end
- def prepare_constraint_for_list_partitioning(table_name:, partitioning_column:, parent_table_name:, initial_partitioning_value:)
+ def prepare_constraint_for_list_partitioning(table_name:, partitioning_column:, parent_table_name:, initial_partitioning_value:, async: false)
validate_not_in_transaction!(:prepare_constraint_for_list_partitioning)
Gitlab::Database::Partitioning::ConvertTableToFirstListPartition
@@ -260,7 +272,7 @@ module Gitlab
parent_table_name: parent_table_name,
partitioning_column: partitioning_column,
zero_partition_value: initial_partitioning_value
- ).prepare_for_partitioning
+ ).prepare_for_partitioning(async: async)
end
def revert_preparing_constraint_for_list_partitioning(table_name:, partitioning_column:, parent_table_name:, initial_partitioning_value:)
@@ -444,18 +456,6 @@ module Gitlab
create_trigger(table_name, trigger_name, function_name, fires: 'AFTER INSERT OR UPDATE OR DELETE')
end
- def enqueue_background_migration(source_table_name, partitioned_table_name, source_column)
- source_model = define_batchable_model(source_table_name)
-
- queue_background_migration_jobs_by_range_at_intervals(
- source_model,
- MIGRATION_CLASS_NAME,
- BATCH_INTERVAL,
- batch_size: BATCH_SIZE,
- other_job_arguments: [source_table_name.to_s, partitioned_table_name, source_column],
- track_jobs: true)
- end
-
def cleanup_migration_jobs(table_name)
::Gitlab::Database::BackgroundMigrationJob.for_partitioning_migration(MIGRATION_CLASS_NAME, table_name).delete_all
end
diff --git a/lib/gitlab/database/postgres_foreign_key.rb b/lib/gitlab/database/postgres_foreign_key.rb
index 28044b42f44..bb3e1d45f15 100644
--- a/lib/gitlab/database/postgres_foreign_key.rb
+++ b/lib/gitlab/database/postgres_foreign_key.rb
@@ -5,6 +5,8 @@ module Gitlab
class PostgresForeignKey < SharedModel
self.primary_key = :oid
+ has_many :child_foreign_keys, class_name: 'Gitlab::Database::PostgresForeignKey', foreign_key: 'parent_oid'
+
# These values come from the possible confdeltype / confupdtype values in pg_constraint
ACTION_TYPES = {
restrict: 'r',
diff --git a/lib/gitlab/database/query_analyzers/prevent_cross_database_modification.rb b/lib/gitlab/database/query_analyzers/prevent_cross_database_modification.rb
index 713e1f772e3..50a3ad0d8ad 100644
--- a/lib/gitlab/database/query_analyzers/prevent_cross_database_modification.rb
+++ b/lib/gitlab/database/query_analyzers/prevent_cross_database_modification.rb
@@ -22,12 +22,27 @@ module Gitlab
self.with_suppressed(false, &blk)
end
+ # This method will temporary ignore the given tables in a current transaction
+ # This is meant to disable `PreventCrossDB` check for some well known failures
+ def self.temporary_ignore_tables_in_transaction(tables, url:, &blk)
+ return yield unless context&.dig(:ignored_tables)
+
+ begin
+ prev_ignored_tables = context[:ignored_tables]
+ context[:ignored_tables] = prev_ignored_tables + tables
+ yield
+ ensure
+ context[:ignored_tables] = prev_ignored_tables
+ end
+ end
+
def self.begin!
super
context.merge!({
transaction_depth_by_db: Hash.new { |h, k| h[k] = 0 },
- modified_tables_by_db: Hash.new { |h, k| h[k] = Set.new }
+ modified_tables_by_db: Hash.new { |h, k| h[k] = Set.new },
+ ignored_tables: []
})
end
@@ -57,7 +72,7 @@ module Gitlab
if context[:transaction_depth_by_db][database] == 0
context[:modified_tables_by_db][database].clear
- # Attempt to troubleshoot https://gitlab.com/gitlab-org/gitlab/-/issues/351531
+ # Attempt to troubleshoot https://gitlab.com/gitlab-org/gitlab/-/issues/351531
::CrossDatabaseModification::TransactionStackTrackRecord.log_gitlab_transactions_stack(action: :end_of_transaction)
elsif context[:transaction_depth_by_db][database] < 0
context[:transaction_depth_by_db][database] = 0
@@ -79,6 +94,9 @@ module Gitlab
# https://gitlab.com/gitlab-org/gitlab/-/issues/343394
tables -= %w[plans gitlab_subscriptions]
+ # Ignore some tables
+ tables -= context[:ignored_tables].to_a
+
return if tables.empty?
# All migrations will write to schema_migrations in the same transaction.
diff --git a/lib/gitlab/database/schema_helpers.rb b/lib/gitlab/database/schema_helpers.rb
index d81ff4ff1ae..3ae696a71d8 100644
--- a/lib/gitlab/database/schema_helpers.rb
+++ b/lib/gitlab/database/schema_helpers.rb
@@ -31,8 +31,8 @@ module Gitlab
end
def trigger_exists?(table_name, name)
- connection.select_value(<<~SQL)
- SELECT 1
+ result = connection.select_value(<<~SQL.squish)
+ SELECT true
FROM pg_catalog.pg_trigger trgr
INNER JOIN pg_catalog.pg_class rel
ON trgr.tgrelid = rel.oid
@@ -42,6 +42,8 @@ module Gitlab
AND rel.relname = #{connection.quote(table_name)}
AND trgr.tgname = #{connection.quote(name)}
SQL
+
+ !!result
end
def drop_function(name, if_exists: true)
diff --git a/lib/gitlab/database/schema_validation/adapters/column_database_adapter.rb b/lib/gitlab/database/schema_validation/adapters/column_database_adapter.rb
new file mode 100644
index 00000000000..10603b3dbad
--- /dev/null
+++ b/lib/gitlab/database/schema_validation/adapters/column_database_adapter.rb
@@ -0,0 +1,43 @@
+# frozen_string_literal: true
+
+module Gitlab
+ module Database
+ module SchemaValidation
+ module Adapters
+ class ColumnDatabaseAdapter
+ def initialize(query_result)
+ @query_result = query_result
+ end
+
+ def name
+ @name ||= query_result['column_name']
+ end
+
+ def table_name
+ query_result['table_name']
+ end
+
+ def data_type
+ query_result['data_type']
+ end
+
+ def default
+ return unless query_result['column_default']
+
+ return if name == 'id' || query_result['column_default'].include?('nextval')
+
+ "DEFAULT #{query_result['column_default']}"
+ end
+
+ def nullable
+ 'NOT NULL' if query_result['not_null']
+ end
+
+ private
+
+ attr_reader :query_result
+ end
+ end
+ end
+ end
+end
diff --git a/lib/gitlab/database/schema_validation/adapters/column_structure_sql_adapter.rb b/lib/gitlab/database/schema_validation/adapters/column_structure_sql_adapter.rb
new file mode 100644
index 00000000000..30a13b5dff1
--- /dev/null
+++ b/lib/gitlab/database/schema_validation/adapters/column_structure_sql_adapter.rb
@@ -0,0 +1,114 @@
+# frozen_string_literal: true
+
+module Gitlab
+ module Database
+ module SchemaValidation
+ module Adapters
+ UndefinedPGType = Class.new(StandardError)
+
+ class ColumnStructureSqlAdapter
+ NOT_NULL_CONSTR = :CONSTR_NOTNULL
+ DEFAULT_CONSTR = :CONSTR_DEFAULT
+
+ MAPPINGS = {
+ 't' => 'true',
+ 'f' => 'false'
+ }.freeze
+
+ attr_reader :table_name
+
+ def initialize(table_name, pg_query_stmt)
+ @table_name = table_name
+ @pg_query_stmt = pg_query_stmt
+ end
+
+ def name
+ @name ||= pg_query_stmt.colname
+ end
+
+ def data_type
+ type(pg_query_stmt.type_name)
+ end
+
+ def default
+ return if name == 'id'
+
+ value = parse_node(constraints.find { |node| node.constraint.contype == DEFAULT_CONSTR })
+
+ return unless value
+
+ "DEFAULT #{value}"
+ end
+
+ def nullable
+ 'NOT NULL' if constraints.any? { |node| node.constraint.contype == NOT_NULL_CONSTR }
+ end
+
+ private
+
+ attr_reader :pg_query_stmt
+
+ def constraints
+ @constraints ||= pg_query_stmt.constraints
+ end
+
+ # Returns the node type
+ #
+ # pg_type:: type alias, used internally by postgres, +int4+, +int8+, +bool+, +varchar+
+ # type:: type name, like +integer+, +bigint+, +boolean+, +character varying+.
+ # array_ext:: adds the +[]+ extension for array types.
+ # precision_ext:: adds the precision, if have any, like +(255)+, +(6)+.
+ #
+ # @info +timestamp+ and +timestamptz+ have a particular case when precision is defined.
+ # In this case, the order of the statement needs to be re-arranged from
+ # timestamp without time zone(6) to timestamp(6) without a time zone.
+ def type(node)
+ pg_type = parse_node(node.names.last)
+ type = PgTypes::TYPES.fetch(pg_type).dup
+ array_ext = '[]' if node.array_bounds.any?
+ precision_ext = "(#{node.typmods.map { |typmod| parse_node(typmod) }.join(',')})" if node.typmods.any?
+
+ if %w[timestamp timestamptz].include?(pg_type)
+ type.gsub!('timestamp', ['timestamp', precision_ext].compact.join(''))
+ precision_ext = nil
+ end
+
+ [type, precision_ext, array_ext].compact.join('')
+ rescue KeyError => exception
+ raise UndefinedPGType, exception.message
+ end
+
+ # Parses PGQuery nodes recursively
+ #
+ # :constraint:: nodes that groups column default info
+ # :func_cal:: nodes that stores functions, like +now()+
+ # :a_const:: nodes that stores constant values, like +t+, +f+, +0.0.0.0+, +255+, +1.0+
+ # :type_cast:: nodes that stores casting values, like +'name'::text+, +'0.0.0.0'::inet+
+ # else:: extract node values in the last iteration of the recursion, like +int4+, +1.0+, +now+, +255+
+ #
+ # @note boolean types types are mapped from +t+, +f+ to +true+, +false+
+ def parse_node(node)
+ return unless node
+
+ case node.node
+ when :constraint
+ parse_node(node.constraint.raw_expr)
+ when :func_call
+ "#{parse_node(node.func_call.funcname.first)}()"
+ when :a_const
+ parse_node(node.a_const.val)
+ when :type_cast
+ value = parse_node(node.type_cast.arg)
+ type = type(node.type_cast.type_name)
+ separator = MAPPINGS.key?(value) ? '' : "::#{type}"
+
+ [MAPPINGS.fetch(value, "'#{value}'"), separator].compact.join('')
+ else
+ node.to_h[node.node].values.last
+ end
+ end
+ end
+ end
+ end
+ end
+end
diff --git a/lib/gitlab/database/schema_validation/database.rb b/lib/gitlab/database/schema_validation/database.rb
index 07bd02e58e1..9ff4a843e6d 100644
--- a/lib/gitlab/database/schema_validation/database.rb
+++ b/lib/gitlab/database/schema_validation/database.rb
@@ -18,6 +18,10 @@ module Gitlab
trigger_map[trigger_name]
end
+ def fetch_table_by_name(table_name)
+ table_map[table_name]
+ end
+
def index_exists?(index_name)
index_map[index_name].present?
end
@@ -26,6 +30,10 @@ module Gitlab
trigger_map[trigger_name].present?
end
+ def table_exists?(table_name)
+ fetch_table_by_name(table_name).present?
+ end
+
def indexes
index_map.values
end
@@ -34,6 +42,10 @@ module Gitlab
trigger_map.values
end
+ def tables
+ table_map.values
+ end
+
private
attr_reader :connection
@@ -56,6 +68,14 @@ module Gitlab
end
end
+ def table_map
+ @table_map ||= fetch_tables.transform_values! do |stmt|
+ columns = stmt.map { |column| SchemaObjects::Column.new(Adapters::ColumnDatabaseAdapter.new(column)) }
+
+ SchemaObjects::Table.new(stmt.first['table_name'], columns)
+ end
+ end
+
def fetch_indexes
sql = <<~SQL
SELECT indexname, indexdef
@@ -78,6 +98,28 @@ module Gitlab
connection.select_rows(sql, nil, schemas).to_h
end
+
+ def fetch_tables
+ sql = <<~SQL
+ SELECT
+ table_information.relname AS table_name,
+ col_information.attname AS column_name,
+ col_information.attnotnull AS not_null,
+ format_type(col_information.atttypid, col_information.atttypmod) AS data_type,
+ pg_get_expr(col_default_information.adbin, col_default_information.adrelid) AS column_default
+ FROM pg_attribute AS col_information
+ JOIN pg_class AS table_information ON col_information.attrelid = table_information.oid
+ JOIN pg_namespace AS schema_information ON table_information.relnamespace = schema_information.oid
+ LEFT JOIN pg_attrdef AS col_default_information ON col_information.attrelid = col_default_information.adrelid
+ AND col_information.attnum = col_default_information.adnum
+ WHERE NOT col_information.attisdropped
+ AND col_information.attnum > 0
+ AND table_information.relkind IN ('r', 'p')
+ AND schema_information.nspname IN ($1, $2)
+ SQL
+
+ connection.exec_query(sql, nil, schemas).group_by { |row| row['table_name'] }
+ end
end
end
end
diff --git a/lib/gitlab/database/schema_validation/inconsistency.rb b/lib/gitlab/database/schema_validation/inconsistency.rb
new file mode 100644
index 00000000000..c834a6bd693
--- /dev/null
+++ b/lib/gitlab/database/schema_validation/inconsistency.rb
@@ -0,0 +1,61 @@
+# frozen_string_literal: true
+
+module Gitlab
+ module Database
+ module SchemaValidation
+ class Inconsistency
+ def initialize(validator_class, structure_sql_object, database_object)
+ @validator_class = validator_class
+ @structure_sql_object = structure_sql_object
+ @database_object = database_object
+ end
+
+ def error_message
+ format(validator_class::ERROR_MESSAGE, object_name)
+ end
+
+ def type
+ validator_class.name.demodulize.underscore
+ end
+
+ def table_name
+ structure_sql_object&.table_name || database_object&.table_name
+ end
+
+ def object_name
+ structure_sql_object&.name || database_object&.name
+ end
+
+ def diff
+ Diffy::Diff.new(structure_sql_statement, database_statement)
+ end
+
+ def inspect
+ <<~MSG
+ #{'-' * 54}
+ #{error_message}
+ Diff:
+ #{diff.to_s(:color)}
+ #{'-' * 54}
+ MSG
+ end
+
+ private
+
+ attr_reader :validator_class, :structure_sql_object, :database_object
+
+ def structure_sql_statement
+ return unless structure_sql_object
+
+ "#{structure_sql_object.statement}\n"
+ end
+
+ def database_statement
+ return unless database_object
+
+ "#{database_object.statement}\n"
+ end
+ end
+ end
+ end
+end
diff --git a/lib/gitlab/database/schema_validation/pg_types.rb b/lib/gitlab/database/schema_validation/pg_types.rb
new file mode 100644
index 00000000000..0a1999d056e
--- /dev/null
+++ b/lib/gitlab/database/schema_validation/pg_types.rb
@@ -0,0 +1,73 @@
+# frozen_string_literal: true
+
+module Gitlab
+ module Database
+ module SchemaValidation
+ class PgTypes
+ TYPES = {
+ 'bool' => 'boolean',
+ 'bytea' => 'bytea',
+ 'char' => '"char"',
+ 'int8' => 'bigint',
+ 'int2' => 'smallint',
+ 'int4' => 'integer',
+ 'regproc' => 'regproc',
+ 'text' => 'text',
+ 'oid' => 'oid',
+ 'tid' => 'tid',
+ 'xid' => 'xid',
+ 'cid' => 'cid',
+ 'json' => 'json',
+ 'xml' => 'xml',
+ 'pg_node_tree' => 'pg_node_tree',
+ 'pg_ndistinct' => 'pg_ndistinct',
+ 'pg_dependencies' => 'pg_dependencies',
+ 'pg_mcv_list' => 'pg_mcv_list',
+ 'xid8' => 'xid8',
+ 'path' => 'path',
+ 'polygon' => 'polygon',
+ 'float4' => 'real',
+ 'float8' => 'double precision',
+ 'circle' => 'circle',
+ 'money' => 'money',
+ 'macaddr' => 'macaddr',
+ 'inet' => 'inet',
+ 'cidr' => 'cidr',
+ 'macaddr8' => 'macaddr8',
+ 'aclitem' => 'aclitem',
+ 'bpchar' => 'character',
+ 'varchar' => 'character varying',
+ 'date' => 'date',
+ 'time' => 'time without time zone',
+ 'timestamp' => 'timestamp without time zone',
+ 'timestamptz' => 'timestamp with time zone',
+ 'interval' => 'interval',
+ 'timetz' => 'time with time zone',
+ 'bit' => 'bit',
+ 'varbit' => 'bit varying',
+ 'numeric' => 'numeric',
+ 'refcursor' => 'refcursor',
+ 'regprocedure' => 'regprocedure',
+ 'regoper' => 'regoper',
+ 'regoperator' => 'regoperator',
+ 'regclass' => 'regclass',
+ 'regcollation' => 'regcollation',
+ 'regtype' => 'regtype',
+ 'regrole' => 'regrole',
+ 'regnamespace' => 'regnamespace',
+ 'uuid' => 'uuid',
+ 'pg_lsn' => 'pg_lsn',
+ 'tsvector' => 'tsvector',
+ 'gtsvector' => 'gtsvector',
+ 'tsquery' => 'tsquery',
+ 'regconfig' => 'regconfig',
+ 'regdictionary' => 'regdictionary',
+ 'jsonb' => 'jsonb',
+ 'jsonpath' => 'jsonpath',
+ 'txid_snapshot' => 'txid_snapshot',
+ 'pg_snapshot' => 'pg_snapshot'
+ }.freeze
+ end
+ end
+ end
+end
diff --git a/lib/gitlab/database/schema_validation/schema_inconsistency.rb b/lib/gitlab/database/schema_validation/schema_inconsistency.rb
new file mode 100644
index 00000000000..6f50603e784
--- /dev/null
+++ b/lib/gitlab/database/schema_validation/schema_inconsistency.rb
@@ -0,0 +1,15 @@
+# frozen_string_literal: true
+
+module Gitlab
+ module Database
+ module SchemaValidation
+ class SchemaInconsistency < ApplicationRecord
+ self.table_name = :schema_inconsistencies
+
+ belongs_to :issue
+
+ validates :object_name, :valitador_name, :table_name, presence: true
+ end
+ end
+ end
+end
diff --git a/lib/gitlab/database/schema_validation/schema_objects/base.rb b/lib/gitlab/database/schema_validation/schema_objects/base.rb
index b0c8eb087dd..43d30dc54ae 100644
--- a/lib/gitlab/database/schema_validation/schema_objects/base.rb
+++ b/lib/gitlab/database/schema_validation/schema_objects/base.rb
@@ -13,6 +13,10 @@ module Gitlab
raise NoMethodError, "subclasses of #{self.class.name} must implement #{__method__}"
end
+ def table_name
+ parsed_stmt.relation.relname
+ end
+
def statement
@statement ||= PgQuery.deparse_stmt(parsed_stmt)
end
diff --git a/lib/gitlab/database/schema_validation/schema_objects/column.rb b/lib/gitlab/database/schema_validation/schema_objects/column.rb
new file mode 100644
index 00000000000..38ad8e309a3
--- /dev/null
+++ b/lib/gitlab/database/schema_validation/schema_objects/column.rb
@@ -0,0 +1,23 @@
+# frozen_string_literal: true
+
+module Gitlab
+ module Database
+ module SchemaValidation
+ module SchemaObjects
+ class Column
+ def initialize(adapter)
+ @adapter = adapter
+ end
+
+ attr_reader :adapter
+
+ delegate :name, :table_name, to: :adapter
+
+ def statement
+ [name, adapter.data_type, adapter.default, adapter.nullable].compact.join(' ')
+ end
+ end
+ end
+ end
+ end
+end
diff --git a/lib/gitlab/database/schema_validation/schema_objects/table.rb b/lib/gitlab/database/schema_validation/schema_objects/table.rb
new file mode 100644
index 00000000000..6f573e7027f
--- /dev/null
+++ b/lib/gitlab/database/schema_validation/schema_objects/table.rb
@@ -0,0 +1,40 @@
+# frozen_string_literal: true
+
+module Gitlab
+ module Database
+ module SchemaValidation
+ module SchemaObjects
+ class Table
+ def initialize(name, columns)
+ @name = name
+ @columns = columns
+ end
+
+ attr_reader :name, :columns
+
+ def table_name
+ name
+ end
+
+ def statement
+ format('CREATE TABLE %s (%s)', name, columns_statement)
+ end
+
+ def fetch_column_by_name(column_name)
+ columns.find { |column| column.name == column_name }
+ end
+
+ def column_exists?(column_name)
+ fetch_column_by_name(column_name).present?
+ end
+
+ private
+
+ def columns_statement
+ columns.map(&:statement).join(', ')
+ end
+ end
+ end
+ end
+ end
+end
diff --git a/lib/gitlab/database/schema_validation/structure_sql.rb b/lib/gitlab/database/schema_validation/structure_sql.rb
index cb62af8d8b8..e93c33aedcd 100644
--- a/lib/gitlab/database/schema_validation/structure_sql.rb
+++ b/lib/gitlab/database/schema_validation/structure_sql.rb
@@ -19,6 +19,14 @@ module Gitlab
triggers.find { |trigger| trigger.name == trigger_name }.present?
end
+ def fetch_table_by_name(table_name)
+ tables.find { |table| table.name == table_name }
+ end
+
+ def table_exists?(table_name)
+ fetch_table_by_name(table_name).present?
+ end
+
def indexes
@indexes ||= map_with_default_schema(index_statements, SchemaObjects::Index)
end
@@ -27,6 +35,18 @@ module Gitlab
@triggers ||= map_with_default_schema(trigger_statements, SchemaObjects::Trigger)
end
+ def tables
+ @tables ||= table_statements.map do |stmt|
+ table_name = stmt.relation.relname
+
+ columns = stmt.table_elts.select { |n| n.node == :column_def }.map do |column|
+ SchemaObjects::Column.new(Adapters::ColumnStructureSqlAdapter.new(table_name, column.column_def))
+ end
+
+ SchemaObjects::Table.new(table_name, columns)
+ end
+ end
+
private
attr_reader :structure_file_path, :schema_name
@@ -39,6 +59,10 @@ module Gitlab
statements.filter_map { |s| s.stmt.create_trig_stmt }
end
+ def table_statements
+ statements.filter_map { |s| s.stmt.create_stmt }
+ end
+
def statements
@statements ||= parsed_structure_file.tree.stmts
end
diff --git a/lib/gitlab/database/schema_validation/track_inconsistency.rb b/lib/gitlab/database/schema_validation/track_inconsistency.rb
new file mode 100644
index 00000000000..c7e946be647
--- /dev/null
+++ b/lib/gitlab/database/schema_validation/track_inconsistency.rb
@@ -0,0 +1,77 @@
+# frozen_string_literal: true
+
+module Gitlab
+ module Database
+ module SchemaValidation
+ class TrackInconsistency
+ def initialize(inconsistency, project, user)
+ @inconsistency = inconsistency
+ @project = project
+ @user = user
+ end
+
+ def execute
+ return unless Gitlab.com?
+ return if inconsistency_record.present?
+
+ result = ::Issues::CreateService.new(container: project, current_user: user, params: params,
+ spam_params: nil).execute
+
+ track_inconsistency(result[:issue]) if result.success?
+ end
+
+ private
+
+ attr_reader :inconsistency, :project, :user
+
+ def track_inconsistency(issue)
+ schema_inconsistency_model.create(
+ issue: issue,
+ object_name: inconsistency.object_name,
+ table_name: inconsistency.table_name,
+ valitador_name: inconsistency.type
+ )
+ end
+
+ def params
+ {
+ title: issue_title,
+ description: issue_description,
+ confidential: true,
+ issue_type: 'issue',
+ labels: %w[database database-inconsistency-report]
+ }
+ end
+
+ def issue_title
+ "New schema inconsistency: #{inconsistency.object_name}"
+ end
+
+ def issue_description
+ <<~MSG
+ We have detected a new schema inconsistency.
+
+ Table_name: #{inconsistency.table_name}
+ Object_name: #{inconsistency.object_name}
+ Validator_name: #{inconsistency.type}
+ Error_message: #{inconsistency.error_message}
+
+ For more information, please contact the database team.
+ MSG
+ end
+
+ def schema_inconsistency_model
+ Gitlab::Database::SchemaValidation::SchemaInconsistency
+ end
+
+ def inconsistency_record
+ schema_inconsistency_model.find_by(
+ object_name: inconsistency.object_name,
+ table_name: inconsistency.table_name,
+ valitador_name: inconsistency.type
+ )
+ end
+ end
+ end
+ end
+end
diff --git a/lib/gitlab/database/schema_validation/validators/base_validator.rb b/lib/gitlab/database/schema_validation/validators/base_validator.rb
index 14995b5f378..58e0bf5292b 100644
--- a/lib/gitlab/database/schema_validation/validators/base_validator.rb
+++ b/lib/gitlab/database/schema_validation/validators/base_validator.rb
@@ -5,7 +5,7 @@ module Gitlab
module SchemaValidation
module Validators
class BaseValidator
- Inconsistency = Struct.new(:type, :object_name, :statement)
+ ERROR_MESSAGE = 'A schema inconsistency has been found'
def initialize(structure_sql, database)
@structure_sql = structure_sql
@@ -14,10 +14,15 @@ module Gitlab
def self.all_validators
[
+ ExtraTables,
+ ExtraTableColumns,
ExtraIndexes,
ExtraTriggers,
+ MissingTables,
+ MissingTableColumns,
MissingIndexes,
MissingTriggers,
+ DifferentDefinitionTables,
DifferentDefinitionIndexes,
DifferentDefinitionTriggers
]
@@ -31,10 +36,8 @@ module Gitlab
attr_reader :structure_sql, :database
- def build_inconsistency(validator_class, schema_object)
- inconsistency_type = validator_class.name.demodulize.underscore
-
- Inconsistency.new(inconsistency_type, schema_object.name, schema_object.statement)
+ def build_inconsistency(validator_class, structure_sql_object, database_object)
+ Inconsistency.new(validator_class, structure_sql_object, database_object)
end
end
end
diff --git a/lib/gitlab/database/schema_validation/validators/different_definition_indexes.rb b/lib/gitlab/database/schema_validation/validators/different_definition_indexes.rb
index d54b62ac1e7..ba12b3cdc61 100644
--- a/lib/gitlab/database/schema_validation/validators/different_definition_indexes.rb
+++ b/lib/gitlab/database/schema_validation/validators/different_definition_indexes.rb
@@ -5,6 +5,8 @@ module Gitlab
module SchemaValidation
module Validators
class DifferentDefinitionIndexes < BaseValidator
+ ERROR_MESSAGE = "The %s index has a different statement between structure.sql and database"
+
def execute
structure_sql.indexes.filter_map do |structure_sql_index|
database_index = database.fetch_index_by_name(structure_sql_index.name)
@@ -12,7 +14,7 @@ module Gitlab
next if database_index.nil?
next if database_index.statement == structure_sql_index.statement
- build_inconsistency(self.class, structure_sql_index)
+ build_inconsistency(self.class, structure_sql_index, database_index)
end
end
end
diff --git a/lib/gitlab/database/schema_validation/validators/different_definition_tables.rb b/lib/gitlab/database/schema_validation/validators/different_definition_tables.rb
new file mode 100644
index 00000000000..9fbddbd3fcd
--- /dev/null
+++ b/lib/gitlab/database/schema_validation/validators/different_definition_tables.rb
@@ -0,0 +1,50 @@
+# frozen_string_literal: true
+
+module Gitlab
+ module Database
+ module SchemaValidation
+ module Validators
+ class DifferentDefinitionTables < BaseValidator
+ ERROR_MESSAGE = "The table %s has a different column statement between structure.sql and database"
+
+ def execute
+ structure_sql.tables.filter_map do |structure_sql_table|
+ table_name = structure_sql_table.name
+ database_table = database.fetch_table_by_name(table_name)
+
+ next unless database_table
+
+ db_diffs, structure_diffs = column_diffs(database_table, structure_sql_table.columns)
+
+ if db_diffs.any?
+ build_inconsistency(self.class,
+ SchemaObjects::Table.new(table_name, db_diffs),
+ SchemaObjects::Table.new(table_name, structure_diffs))
+ end
+ end
+ end
+
+ private
+
+ def column_diffs(db_table, columns)
+ db_diffs = []
+ structure_diffs = []
+
+ columns.each do |column|
+ db_column = db_table.fetch_column_by_name(column.name)
+
+ next unless db_column
+
+ next if db_column.statement == column.statement
+
+ db_diffs << db_column
+ structure_diffs << column
+ end
+
+ [db_diffs, structure_diffs]
+ end
+ end
+ end
+ end
+ end
+end
diff --git a/lib/gitlab/database/schema_validation/validators/different_definition_triggers.rb b/lib/gitlab/database/schema_validation/validators/different_definition_triggers.rb
index efb87a70ca8..79ffe9a6a98 100644
--- a/lib/gitlab/database/schema_validation/validators/different_definition_triggers.rb
+++ b/lib/gitlab/database/schema_validation/validators/different_definition_triggers.rb
@@ -5,6 +5,8 @@ module Gitlab
module SchemaValidation
module Validators
class DifferentDefinitionTriggers < BaseValidator
+ ERROR_MESSAGE = "The %s trigger has a different statement between structure.sql and database"
+
def execute
structure_sql.triggers.filter_map do |structure_sql_trigger|
database_trigger = database.fetch_trigger_by_name(structure_sql_trigger.name)
@@ -12,7 +14,7 @@ module Gitlab
next if database_trigger.nil?
next if database_trigger.statement == structure_sql_trigger.statement
- build_inconsistency(self.class, structure_sql_trigger)
+ build_inconsistency(self.class, structure_sql_trigger, nil)
end
end
end
diff --git a/lib/gitlab/database/schema_validation/validators/extra_indexes.rb b/lib/gitlab/database/schema_validation/validators/extra_indexes.rb
index 28384dd7cee..c8d3749894b 100644
--- a/lib/gitlab/database/schema_validation/validators/extra_indexes.rb
+++ b/lib/gitlab/database/schema_validation/validators/extra_indexes.rb
@@ -5,11 +5,13 @@ module Gitlab
module SchemaValidation
module Validators
class ExtraIndexes < BaseValidator
+ ERROR_MESSAGE = "The index %s is present in the database, but not in the structure.sql file"
+
def execute
- database.indexes.filter_map do |index|
- next if structure_sql.index_exists?(index.name)
+ database.indexes.filter_map do |database_index|
+ next if structure_sql.index_exists?(database_index.name)
- build_inconsistency(self.class, index)
+ build_inconsistency(self.class, nil, database_index)
end
end
end
diff --git a/lib/gitlab/database/schema_validation/validators/extra_table_columns.rb b/lib/gitlab/database/schema_validation/validators/extra_table_columns.rb
new file mode 100644
index 00000000000..823b01cf808
--- /dev/null
+++ b/lib/gitlab/database/schema_validation/validators/extra_table_columns.rb
@@ -0,0 +1,32 @@
+# frozen_string_literal: true
+
+module Gitlab
+ module Database
+ module SchemaValidation
+ module Validators
+ class ExtraTableColumns < BaseValidator
+ ERROR_MESSAGE = "The table %s has columns present in the database, but not in the structure.sql file"
+
+ def execute
+ database.tables.filter_map do |database_table|
+ table_name = database_table.name
+ structure_sql_table = structure_sql.fetch_table_by_name(table_name)
+
+ next unless structure_sql_table
+
+ inconsistencies = database_table.columns.filter_map do |database_table_column|
+ next if structure_sql_table.column_exists?(database_table_column.name)
+
+ database_table_column
+ end
+
+ if inconsistencies.any?
+ build_inconsistency(self.class, nil, SchemaObjects::Table.new(table_name, inconsistencies))
+ end
+ end
+ end
+ end
+ end
+ end
+ end
+end
diff --git a/lib/gitlab/database/schema_validation/validators/extra_tables.rb b/lib/gitlab/database/schema_validation/validators/extra_tables.rb
new file mode 100644
index 00000000000..99e98eb8f67
--- /dev/null
+++ b/lib/gitlab/database/schema_validation/validators/extra_tables.rb
@@ -0,0 +1,21 @@
+# frozen_string_literal: true
+
+module Gitlab
+ module Database
+ module SchemaValidation
+ module Validators
+ class ExtraTables < BaseValidator
+ ERROR_MESSAGE = "The table %s is present in the database, but not in the structure.sql file"
+
+ def execute
+ database.tables.filter_map do |database_table|
+ next if structure_sql.table_exists?(database_table.name)
+
+ build_inconsistency(self.class, nil, database_table)
+ end
+ end
+ end
+ end
+ end
+ end
+end
diff --git a/lib/gitlab/database/schema_validation/validators/extra_triggers.rb b/lib/gitlab/database/schema_validation/validators/extra_triggers.rb
index f03bb49526c..37dcbc53e2e 100644
--- a/lib/gitlab/database/schema_validation/validators/extra_triggers.rb
+++ b/lib/gitlab/database/schema_validation/validators/extra_triggers.rb
@@ -5,11 +5,13 @@ module Gitlab
module SchemaValidation
module Validators
class ExtraTriggers < BaseValidator
+ ERROR_MESSAGE = "The trigger %s is present in the database, but not in the structure.sql file"
+
def execute
- database.triggers.filter_map do |trigger|
- next if structure_sql.trigger_exists?(trigger.name)
+ database.triggers.filter_map do |database_trigger|
+ next if structure_sql.trigger_exists?(database_trigger.name)
- build_inconsistency(self.class, trigger)
+ build_inconsistency(self.class, nil, database_trigger)
end
end
end
diff --git a/lib/gitlab/database/schema_validation/validators/missing_indexes.rb b/lib/gitlab/database/schema_validation/validators/missing_indexes.rb
index ac0ea0152ba..7f81aaccf0f 100644
--- a/lib/gitlab/database/schema_validation/validators/missing_indexes.rb
+++ b/lib/gitlab/database/schema_validation/validators/missing_indexes.rb
@@ -5,11 +5,13 @@ module Gitlab
module SchemaValidation
module Validators
class MissingIndexes < BaseValidator
+ ERROR_MESSAGE = "The index %s is missing from the database"
+
def execute
- structure_sql.indexes.filter_map do |index|
- next if database.index_exists?(index.name)
+ structure_sql.indexes.filter_map do |structure_sql_index|
+ next if database.index_exists?(structure_sql_index.name)
- build_inconsistency(self.class, index)
+ build_inconsistency(self.class, structure_sql_index, nil)
end
end
end
diff --git a/lib/gitlab/database/schema_validation/validators/missing_table_columns.rb b/lib/gitlab/database/schema_validation/validators/missing_table_columns.rb
new file mode 100644
index 00000000000..b49d53823ee
--- /dev/null
+++ b/lib/gitlab/database/schema_validation/validators/missing_table_columns.rb
@@ -0,0 +1,32 @@
+# frozen_string_literal: true
+
+module Gitlab
+ module Database
+ module SchemaValidation
+ module Validators
+ class MissingTableColumns < BaseValidator
+ ERROR_MESSAGE = "The table %s has columns missing from the database"
+
+ def execute
+ structure_sql.tables.filter_map do |structure_sql_table|
+ table_name = structure_sql_table.name
+ database_table = database.fetch_table_by_name(table_name)
+
+ next unless database_table
+
+ inconsistencies = structure_sql_table.columns.filter_map do |structure_table_column|
+ next if database_table.column_exists?(structure_table_column.name)
+
+ structure_table_column
+ end
+
+ if inconsistencies.any?
+ build_inconsistency(self.class, nil, SchemaObjects::Table.new(table_name, inconsistencies))
+ end
+ end
+ end
+ end
+ end
+ end
+ end
+end
diff --git a/lib/gitlab/database/schema_validation/validators/missing_tables.rb b/lib/gitlab/database/schema_validation/validators/missing_tables.rb
new file mode 100644
index 00000000000..f1c9383487d
--- /dev/null
+++ b/lib/gitlab/database/schema_validation/validators/missing_tables.rb
@@ -0,0 +1,21 @@
+# frozen_string_literal: true
+
+module Gitlab
+ module Database
+ module SchemaValidation
+ module Validators
+ class MissingTables < BaseValidator
+ ERROR_MESSAGE = "The table %s is missing from the database"
+
+ def execute
+ structure_sql.tables.filter_map do |structure_sql_table|
+ next if database.table_exists?(structure_sql_table.name)
+
+ build_inconsistency(self.class, structure_sql_table, nil)
+ end
+ end
+ end
+ end
+ end
+ end
+end
diff --git a/lib/gitlab/database/schema_validation/validators/missing_triggers.rb b/lib/gitlab/database/schema_validation/validators/missing_triggers.rb
index c7137c68c1c..36236463bbf 100644
--- a/lib/gitlab/database/schema_validation/validators/missing_triggers.rb
+++ b/lib/gitlab/database/schema_validation/validators/missing_triggers.rb
@@ -5,11 +5,13 @@ module Gitlab
module SchemaValidation
module Validators
class MissingTriggers < BaseValidator
+ ERROR_MESSAGE = "The trigger %s is missing from the database"
+
def execute
- structure_sql.triggers.filter_map do |index|
- next if database.trigger_exists?(index.name)
+ structure_sql.triggers.filter_map do |structure_sql_trigger|
+ next if database.trigger_exists?(structure_sql_trigger.name)
- build_inconsistency(self.class, index)
+ build_inconsistency(self.class, structure_sql_trigger, nil)
end
end
end
diff --git a/lib/gitlab/database/tables_locker.rb b/lib/gitlab/database/tables_locker.rb
index 42a2c5c02f7..1b6ab3fb24b 100644
--- a/lib/gitlab/database/tables_locker.rb
+++ b/lib/gitlab/database/tables_locker.rb
@@ -8,6 +8,7 @@ module Gitlab
def initialize(logger: nil, dry_run: false)
@logger = logger
@dry_run = dry_run
+ @result = []
end
def unlock_writes
@@ -19,6 +20,8 @@ module Gitlab
unlock_writes_on_table(table_name, connection, database_name)
end
end
+
+ @result
end
# It locks the tables on the database where they don't belong. Also it unlocks the tables
@@ -38,25 +41,27 @@ module Gitlab
end
end
end
+
+ @result
end
private
# Unlocks the writes on the table and its partitions
def unlock_writes_on_table(table_name, connection, database_name)
- lock_writes_manager(table_name, connection, database_name).unlock_writes
+ @result << lock_writes_manager(table_name, connection, database_name).unlock_writes
table_attached_partitions(table_name, connection) do |postgres_partition|
- lock_writes_manager(postgres_partition.identifier, connection, database_name).unlock_writes
+ @result << lock_writes_manager(postgres_partition.identifier, connection, database_name).unlock_writes
end
end
# It locks the writes on the table and its partitions
def lock_writes_on_table(table_name, connection, database_name)
- lock_writes_manager(table_name, connection, database_name).lock_writes
+ @result << lock_writes_manager(table_name, connection, database_name).lock_writes
table_attached_partitions(table_name, connection) do |postgres_partition|
- lock_writes_manager(postgres_partition.identifier, connection, database_name).lock_writes
+ @result << lock_writes_manager(postgres_partition.identifier, connection, database_name).lock_writes
end
end