Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
100618: builtins: remove WaitPolicy error from fingerprinting builtin r=dt a=adityamaru

During development the wait policy for ExportRequests sent during fingerprinting was set to error. This meant that if the ExportRequest encountered an intent it would immediately return and error out. This is fine if we added a retry loop similar to how backup processor requeues the request to be sent at a later time by when the intent is resolved, but as is this is incorrect. We change the WaitPolicy to block so that the ExportRequest blocks until the intent is resolved. If in the future we want to make progress while another ExportRequest is stuck resolving intents we can rework this logic to look similar to our backup strategy.

Release note: None
Epic: none

100703: sql: add new persistedV22_2 views r=maryliag a=maryliag

Note to reviewers: this is a forward-porting from #100673

Fixes #100501

Adds {statement|transaction}_statistics_persisted_v22_2 like they were added on #96454. Check the version of the cluster before deciding which view to use. This is required for mixed version cluster with 22.2 and 23.1 versions.

Release note: None

100734: backupccl: gate writing slim manifests in backup via cluster setting r=rhu713 a=rhu713

Feature gate the writing of slim manifests at the end of backup jobs with
the cluster setting `backup.write_metadata_with_external_ssts.enabled`.

Release note: None

100916: roachtest: fix c2c/shutdown/dest bug r=dt,adityamaru a=msbutler

Previously, the c2c/shutdown/dest tests were incorrectly shutting down nodes on the source cluster. This patch fixes this.

Fixes #100907

Release note: None

101057: sqlproxyccl: skip TestDirectoryConnect r=adityamaru a=knz

Informs #76839.

Release note: None
Epic: None

Co-authored-by: adityamaru <[email protected]>
Co-authored-by: maryliag <[email protected]>
Co-authored-by: Rui Hu <[email protected]>
Co-authored-by: Michael Butler <[email protected]>
Co-authored-by: Raphael 'kena' Poss <[email protected]>
  • Loading branch information
6 people committed Apr 10, 2023
6 parents 12548c5 + 6505b38 + 3dabaaf + f7b3ea6 + 6dfd113 + d87761f commit bd2eaeb
Show file tree
Hide file tree
Showing 21 changed files with 1,232 additions and 1,089 deletions.
8 changes: 5 additions & 3 deletions pkg/ccl/backupccl/backup_job.go
Original file line number Diff line number Diff line change
Expand Up @@ -358,9 +358,11 @@ func backup(
// TODO(adityamaru,rhu713): Once backup/restore switches from writing and
// reading backup manifests to `metadata.sst` we can stop writing the slim
// manifest.
if err := backupinfo.WriteMetadataWithExternalSSTs(ctx, defaultStore, encryption,
&kmsEnv, backupManifest); err != nil {
return roachpb.RowCount{}, err
if backupinfo.WriteMetadataWithExternalSSTsEnabled.Get(&settings.SV) {
if err := backupinfo.WriteMetadataWithExternalSSTs(ctx, defaultStore, encryption,
&kmsEnv, backupManifest); err != nil {
return roachpb.RowCount{}, err
}
}

statsTable := getTableStatsForBackup(ctx, statsCache, backupManifest.Descriptors)
Expand Down
3 changes: 3 additions & 0 deletions pkg/ccl/backupccl/backup_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -8312,6 +8312,7 @@ func TestIncorrectAccessOfFilesInBackupMetadata(t *testing.T) {

_, sqlDB, rawDir, cleanupFn := backupRestoreTestSetup(t, singleNode, 1, InitManualReplication)
defer cleanupFn()
sqlDB.Exec(t, `SET CLUSTER SETTING backup.write_metadata_with_external_ssts.enabled=true`)
sqlDB.Exec(t, `CREATE DATABASE r1`)
sqlDB.Exec(t, `CREATE TABLE r1.foo ( id INT PRIMARY KEY)`)
sqlDB.Exec(t, `INSERT INTO r1.foo VALUES (1)`)
Expand Down Expand Up @@ -8357,6 +8358,7 @@ func TestRestoringAcrossVersions(t *testing.T) {
tc, sqlDB, rawDir, cleanupFn := backupRestoreTestSetup(t, singleNode, 1, InitManualReplication)
defer cleanupFn()

sqlDB.Exec(t, `SET CLUSTER SETTING backup.write_metadata_with_external_ssts.enabled=true`)
sqlDB.Exec(t, `CREATE DATABASE r1`)

sqlDB.Exec(t, `BACKUP DATABASE r1 TO 'nodelocal://1/cross_version'`)
Expand Down Expand Up @@ -8440,6 +8442,7 @@ func TestManifestBitFlip(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)
_, sqlDB, rawDir, cleanupFn := backupRestoreTestSetup(t, singleNode, 1, InitManualReplication)
sqlDB.Exec(t, `SET CLUSTER SETTING backup.write_metadata_with_external_ssts.enabled=true`)
defer cleanupFn()
sqlDB.Exec(t, `CREATE DATABASE r1; CREATE DATABASE r2; CREATE DATABASE r3;`)
const checksumError = "checksum mismatch"
Expand Down
11 changes: 11 additions & 0 deletions pkg/ccl/backupccl/backupinfo/manifest_handling.go
Original file line number Diff line number Diff line change
Expand Up @@ -97,6 +97,17 @@ var WriteMetadataSST = settings.RegisterBoolSetting(
util.ConstantWithMetamorphicTestBool("write-metadata-sst", false),
)

// WriteMetadataWithExternalSSTsEnabled controls if we write a `BACKUP_METADATA`
// file along with external SSTs containing lists of `BackupManifest_Files` and
// descriptors. This new format of metadata is written in addition to the
// `BACKUP_MANIFEST` file, and is expected to be its replacement in the future.
var WriteMetadataWithExternalSSTsEnabled = settings.RegisterBoolSetting(
settings.TenantWritable,
"backup.write_metadata_with_external_ssts.enabled",
"write BACKUP metadata along with supporting SST files",
util.ConstantWithMetamorphicTestBool("backup.write_metadata_with_external_ssts.enabled", true),
)

// IsGZipped detects whether the given bytes represent GZipped data. This check
// is used rather than a standard implementation such as http.DetectContentType
// since some zipped data may be mis-identified by that method. We've seen
Expand Down
172 changes: 87 additions & 85 deletions pkg/ccl/logictestccl/testdata/logic_test/crdb_internal_tenant
Original file line number Diff line number Diff line change
Expand Up @@ -22,91 +22,93 @@ DROP DATABASE crdb_internal
query TTTTIT
SHOW TABLES FROM crdb_internal
----
crdb_internal active_range_feeds table admin NULL NULL
crdb_internal backward_dependencies table admin NULL NULL
crdb_internal builtin_functions table admin NULL NULL
crdb_internal cluster_contended_indexes view admin NULL NULL
crdb_internal cluster_contended_keys view admin NULL NULL
crdb_internal cluster_contended_tables view admin NULL NULL
crdb_internal cluster_contention_events table admin NULL NULL
crdb_internal cluster_database_privileges table admin NULL NULL
crdb_internal cluster_distsql_flows table admin NULL NULL
crdb_internal cluster_execution_insights table admin NULL NULL
crdb_internal cluster_inflight_traces table admin NULL NULL
crdb_internal cluster_locks table admin NULL NULL
crdb_internal cluster_queries table admin NULL NULL
crdb_internal cluster_sessions table admin NULL NULL
crdb_internal cluster_settings table admin NULL NULL
crdb_internal cluster_statement_statistics table admin NULL NULL
crdb_internal cluster_transaction_statistics table admin NULL NULL
crdb_internal cluster_transactions table admin NULL NULL
crdb_internal cluster_txn_execution_insights table admin NULL NULL
crdb_internal create_function_statements table admin NULL NULL
crdb_internal create_schema_statements table admin NULL NULL
crdb_internal create_statements table admin NULL NULL
crdb_internal create_type_statements table admin NULL NULL
crdb_internal cross_db_references table admin NULL NULL
crdb_internal databases table admin NULL NULL
crdb_internal default_privileges table admin NULL NULL
crdb_internal feature_usage table admin NULL NULL
crdb_internal forward_dependencies table admin NULL NULL
crdb_internal gossip_alerts table admin NULL NULL
crdb_internal gossip_liveness table admin NULL NULL
crdb_internal gossip_network table admin NULL NULL
crdb_internal gossip_nodes table admin NULL NULL
crdb_internal index_columns table admin NULL NULL
crdb_internal index_spans table admin NULL NULL
crdb_internal index_usage_statistics table admin NULL NULL
crdb_internal invalid_objects table admin NULL NULL
crdb_internal jobs table admin NULL NULL
crdb_internal kv_catalog_comments table admin NULL NULL
crdb_internal kv_catalog_descriptor table admin NULL NULL
crdb_internal kv_catalog_namespace table admin NULL NULL
crdb_internal kv_catalog_zones table admin NULL NULL
crdb_internal kv_dropped_relations view admin NULL NULL
crdb_internal kv_node_liveness table admin NULL NULL
crdb_internal kv_node_status table admin NULL NULL
crdb_internal kv_store_status table admin NULL NULL
crdb_internal leases table admin NULL NULL
crdb_internal lost_descriptors_with_data table admin NULL NULL
crdb_internal node_build_info table admin NULL NULL
crdb_internal node_contention_events table admin NULL NULL
crdb_internal node_distsql_flows table admin NULL NULL
crdb_internal node_execution_insights table admin NULL NULL
crdb_internal node_inflight_trace_spans table admin NULL NULL
crdb_internal node_memory_monitors table admin NULL NULL
crdb_internal node_metrics table admin NULL NULL
crdb_internal node_queries table admin NULL NULL
crdb_internal node_runtime_info table admin NULL NULL
crdb_internal node_sessions table admin NULL NULL
crdb_internal node_statement_statistics table admin NULL NULL
crdb_internal node_tenant_capabilities_cache table admin NULL NULL
crdb_internal node_transaction_statistics table admin NULL NULL
crdb_internal node_transactions table admin NULL NULL
crdb_internal node_txn_execution_insights table admin NULL NULL
crdb_internal node_txn_stats table admin NULL NULL
crdb_internal partitions table admin NULL NULL
crdb_internal pg_catalog_table_is_implemented table admin NULL NULL
crdb_internal ranges view admin NULL NULL
crdb_internal ranges_no_leases table admin NULL NULL
crdb_internal regions table admin NULL NULL
crdb_internal schema_changes table admin NULL NULL
crdb_internal session_trace table admin NULL NULL
crdb_internal session_variables table admin NULL NULL
crdb_internal statement_statistics view admin NULL NULL
crdb_internal statement_statistics_persisted view admin NULL NULL
crdb_internal super_regions table admin NULL NULL
crdb_internal system_jobs table admin NULL NULL
crdb_internal table_columns table admin NULL NULL
crdb_internal table_indexes table admin NULL NULL
crdb_internal table_row_statistics table admin NULL NULL
crdb_internal table_spans table admin NULL NULL
crdb_internal tables table admin NULL NULL
crdb_internal tenant_usage_details view admin NULL NULL
crdb_internal transaction_contention_events table admin NULL NULL
crdb_internal transaction_statistics view admin NULL NULL
crdb_internal transaction_statistics_persisted view admin NULL NULL
crdb_internal zones table admin NULL NULL
crdb_internal active_range_feeds table admin NULL NULL
crdb_internal backward_dependencies table admin NULL NULL
crdb_internal builtin_functions table admin NULL NULL
crdb_internal cluster_contended_indexes view admin NULL NULL
crdb_internal cluster_contended_keys view admin NULL NULL
crdb_internal cluster_contended_tables view admin NULL NULL
crdb_internal cluster_contention_events table admin NULL NULL
crdb_internal cluster_database_privileges table admin NULL NULL
crdb_internal cluster_distsql_flows table admin NULL NULL
crdb_internal cluster_execution_insights table admin NULL NULL
crdb_internal cluster_inflight_traces table admin NULL NULL
crdb_internal cluster_locks table admin NULL NULL
crdb_internal cluster_queries table admin NULL NULL
crdb_internal cluster_sessions table admin NULL NULL
crdb_internal cluster_settings table admin NULL NULL
crdb_internal cluster_statement_statistics table admin NULL NULL
crdb_internal cluster_transaction_statistics table admin NULL NULL
crdb_internal cluster_transactions table admin NULL NULL
crdb_internal cluster_txn_execution_insights table admin NULL NULL
crdb_internal create_function_statements table admin NULL NULL
crdb_internal create_schema_statements table admin NULL NULL
crdb_internal create_statements table admin NULL NULL
crdb_internal create_type_statements table admin NULL NULL
crdb_internal cross_db_references table admin NULL NULL
crdb_internal databases table admin NULL NULL
crdb_internal default_privileges table admin NULL NULL
crdb_internal feature_usage table admin NULL NULL
crdb_internal forward_dependencies table admin NULL NULL
crdb_internal gossip_alerts table admin NULL NULL
crdb_internal gossip_liveness table admin NULL NULL
crdb_internal gossip_network table admin NULL NULL
crdb_internal gossip_nodes table admin NULL NULL
crdb_internal index_columns table admin NULL NULL
crdb_internal index_spans table admin NULL NULL
crdb_internal index_usage_statistics table admin NULL NULL
crdb_internal invalid_objects table admin NULL NULL
crdb_internal jobs table admin NULL NULL
crdb_internal kv_catalog_comments table admin NULL NULL
crdb_internal kv_catalog_descriptor table admin NULL NULL
crdb_internal kv_catalog_namespace table admin NULL NULL
crdb_internal kv_catalog_zones table admin NULL NULL
crdb_internal kv_dropped_relations view admin NULL NULL
crdb_internal kv_node_liveness table admin NULL NULL
crdb_internal kv_node_status table admin NULL NULL
crdb_internal kv_store_status table admin NULL NULL
crdb_internal leases table admin NULL NULL
crdb_internal lost_descriptors_with_data table admin NULL NULL
crdb_internal node_build_info table admin NULL NULL
crdb_internal node_contention_events table admin NULL NULL
crdb_internal node_distsql_flows table admin NULL NULL
crdb_internal node_execution_insights table admin NULL NULL
crdb_internal node_inflight_trace_spans table admin NULL NULL
crdb_internal node_memory_monitors table admin NULL NULL
crdb_internal node_metrics table admin NULL NULL
crdb_internal node_queries table admin NULL NULL
crdb_internal node_runtime_info table admin NULL NULL
crdb_internal node_sessions table admin NULL NULL
crdb_internal node_statement_statistics table admin NULL NULL
crdb_internal node_tenant_capabilities_cache table admin NULL NULL
crdb_internal node_transaction_statistics table admin NULL NULL
crdb_internal node_transactions table admin NULL NULL
crdb_internal node_txn_execution_insights table admin NULL NULL
crdb_internal node_txn_stats table admin NULL NULL
crdb_internal partitions table admin NULL NULL
crdb_internal pg_catalog_table_is_implemented table admin NULL NULL
crdb_internal ranges view admin NULL NULL
crdb_internal ranges_no_leases table admin NULL NULL
crdb_internal regions table admin NULL NULL
crdb_internal schema_changes table admin NULL NULL
crdb_internal session_trace table admin NULL NULL
crdb_internal session_variables table admin NULL NULL
crdb_internal statement_statistics view admin NULL NULL
crdb_internal statement_statistics_persisted view admin NULL NULL
crdb_internal statement_statistics_persisted_v22_2 view admin NULL NULL
crdb_internal super_regions table admin NULL NULL
crdb_internal system_jobs table admin NULL NULL
crdb_internal table_columns table admin NULL NULL
crdb_internal table_indexes table admin NULL NULL
crdb_internal table_row_statistics table admin NULL NULL
crdb_internal table_spans table admin NULL NULL
crdb_internal tables table admin NULL NULL
crdb_internal tenant_usage_details view admin NULL NULL
crdb_internal transaction_contention_events table admin NULL NULL
crdb_internal transaction_statistics view admin NULL NULL
crdb_internal transaction_statistics_persisted view admin NULL NULL
crdb_internal transaction_statistics_persisted_v22_2 view admin NULL NULL
crdb_internal zones table admin NULL NULL

statement ok
CREATE DATABASE testdb; CREATE TABLE testdb.foo(x INT)
Expand Down
2 changes: 2 additions & 0 deletions pkg/ccl/sqlproxyccl/proxy_handler_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -970,6 +970,8 @@ func TestDenylistUpdate(t *testing.T) {

func TestDirectoryConnect(t *testing.T) {
defer leaktest.AfterTest(t)()

skip.WithIssue(t, 76839, "flaky test")
// TODO(jaylim-crl): This is a potential port reuse issue, so skip this
// under stress. See linked GitHub issue.
skip.UnderStress(t, "https://github.com/cockroachdb/cockroach/issues/76839")
Expand Down
2 changes: 2 additions & 0 deletions pkg/cli/zip_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -97,9 +97,11 @@ table_name NOT IN (
'tables',
'cluster_statement_statistics',
'statement_statistics_persisted',
'statement_statistics_persisted_v22_2',
'cluster_transaction_statistics',
'statement_statistics',
'transaction_statistics_persisted',
'transaction_statistics_persisted_v22_2',
'transaction_statistics',
'tenant_usage_details',
'pg_catalog_table_is_implemented'
Expand Down
9 changes: 9 additions & 0 deletions pkg/cmd/roachtest/tests/cluster_to_cluster.go
Original file line number Diff line number Diff line change
Expand Up @@ -846,6 +846,15 @@ func (rsp *replResilienceSpec) getTargetAndWatcherNodes(ctx context.Context) {
return info.db.QueryRowContext(ctx,
`SELECT coordinator_id FROM crdb_internal.jobs WHERE job_id = $1`, jobID).Scan(&coordinatorNode)
}, time.Minute)
if !rsp.onSrc {
// From the destination cluster's perspective, node ids range from 1 to
// num_dest_nodes, but from roachprod's perspective they range from
// num_source_nodes+1 to num_crdb_roachprod nodes. We need to adjust for
// this to shut down the right node. Example: if the coordinator node on the
// dest cluster is 1, and there are 4 src cluster nodes, then
// shut down roachprod node 5.
coordinatorNode += rsp.spec.srcNodes
}

var targetNode int

Expand Down
Loading

0 comments on commit bd2eaeb

Please sign in to comment.