From ceb5981e964cd94566c960fbbb4409299f11839d Mon Sep 17 00:00:00 2001 From: Gerardo Torres Date: Thu, 28 Apr 2022 11:36:31 -0400 Subject: [PATCH 1/9] server, sql: surface session txnCount, txn fingerprints, active time Partially addresses #74257. Previously, the status server did not provide session details such as total number of transactions executed, transaction fingerprint IDs, and total active time. This change adds the aforementioned session details to the `serverpb.Session` struct. To track recently executed transaction fingerprint IDs, a FIFO cache `TxnFingerprintIDCache` is introduced with its corresponding cluster setting `TxnFingerprintIDBufferCapacity` to control the capacity. The default capacity is set at 100 fingerprints. The total number of transactions executed is filled using the existing `txnCounter` from the `extraTxnState` in `connExecutor`. The total active time is calculated by introducing a `timeutil.StopWatch` to the connection executor, which is started and stopped when a transaction is started and finished respectively. Release note (api change): the `serverpb.Session` struct now has three new fields: number of transactions executed, transaction fingerprint IDs, and total active time. --- docs/generated/http/full.md | 6 + .../settings/settings-for-tenants.txt | 1 + docs/generated/settings/settings.html | 1 + docs/generated/swagger/spec.json | 29 ++++ .../testdata/logic_test/crdb_internal_tenant | 8 +- pkg/server/serverpb/status.proto | 16 ++ pkg/sql/BUILD.bazel | 2 + pkg/sql/conn_executor.go | 23 +++ pkg/sql/conn_executor_exec.go | 11 +- pkg/sql/conn_executor_test.go | 89 ++++++++++ pkg/sql/crdb_internal.go | 3 + pkg/sql/delegate/show_sessions.go | 2 +- .../testdata/logic_test/crdb_internal | 8 +- .../testdata/logic_test/create_statements | 4 + pkg/sql/testdata/txn_fingerprint_id_cache | 77 +++++++++ pkg/sql/txn_fingerprint_id_cache.go | 130 ++++++++++++++ pkg/sql/txn_fingerprint_id_cache_test.go | 159 ++++++++++++++++++ .../src/sessions/sessionsPage.fixture.ts | 8 + pkg/util/timeutil/stopwatch.go | 8 + 19 files changed, 575 insertions(+), 10 deletions(-) create mode 100644 pkg/sql/testdata/txn_fingerprint_id_cache create mode 100644 pkg/sql/txn_fingerprint_id_cache.go create mode 100644 pkg/sql/txn_fingerprint_id_cache_test.go diff --git a/docs/generated/http/full.md b/docs/generated/http/full.md index 4dfbb6168096..d2f740544ac0 100644 --- a/docs/generated/http/full.md +++ b/docs/generated/http/full.md @@ -2111,6 +2111,9 @@ Session represents one SQL session. | last_active_query_no_constants | [string](#cockroach.server.serverpb.ListSessionsResponse-string) | | The SQL statement fingerprint of the last query executed on this session, compatible with StatementStatisticsKey. | [reserved](#support-status) | | status | [Session.Status](#cockroach.server.serverpb.ListSessionsResponse-cockroach.server.serverpb.Session.Status) | | The session's status. | [reserved](#support-status) | | end | [google.protobuf.Timestamp](#cockroach.server.serverpb.ListSessionsResponse-google.protobuf.Timestamp) | | Timestamp of session's end. | [reserved](#support-status) | +| num_txns_executed | [int32](#cockroach.server.serverpb.ListSessionsResponse-int32) | | Count of the number of transactions that have been opened on this session. This count includes transactions that are in progress. | [reserved](#support-status) | +| txn_fingerprint_ids | [uint64](#cockroach.server.serverpb.ListSessionsResponse-uint64) | repeated | List of transaction fingerprint IDs in this session. | [reserved](#support-status) | +| total_active_time | [google.protobuf.Duration](#cockroach.server.serverpb.ListSessionsResponse-google.protobuf.Duration) | | The session's total active time. | [reserved](#support-status) | @@ -2247,6 +2250,9 @@ Session represents one SQL session. | last_active_query_no_constants | [string](#cockroach.server.serverpb.ListSessionsResponse-string) | | The SQL statement fingerprint of the last query executed on this session, compatible with StatementStatisticsKey. | [reserved](#support-status) | | status | [Session.Status](#cockroach.server.serverpb.ListSessionsResponse-cockroach.server.serverpb.Session.Status) | | The session's status. | [reserved](#support-status) | | end | [google.protobuf.Timestamp](#cockroach.server.serverpb.ListSessionsResponse-google.protobuf.Timestamp) | | Timestamp of session's end. | [reserved](#support-status) | +| num_txns_executed | [int32](#cockroach.server.serverpb.ListSessionsResponse-int32) | | Count of the number of transactions that have been opened on this session. This count includes transactions that are in progress. | [reserved](#support-status) | +| txn_fingerprint_ids | [uint64](#cockroach.server.serverpb.ListSessionsResponse-uint64) | repeated | List of transaction fingerprint IDs in this session. | [reserved](#support-status) | +| total_active_time | [google.protobuf.Duration](#cockroach.server.serverpb.ListSessionsResponse-google.protobuf.Duration) | | The session's total active time. | [reserved](#support-status) | diff --git a/docs/generated/settings/settings-for-tenants.txt b/docs/generated/settings/settings-for-tenants.txt index dad2d8c8f11a..2646980521c3 100644 --- a/docs/generated/settings/settings-for-tenants.txt +++ b/docs/generated/settings/settings-for-tenants.txt @@ -273,6 +273,7 @@ sql.ttl.default_delete_rate_limit integer 0 default delete rate limit for all TT sql.ttl.default_range_concurrency integer 1 default amount of ranges to process at once during a TTL delete sql.ttl.default_select_batch_size integer 500 default amount of rows to select in a single query during a TTL job sql.ttl.job.enabled boolean true whether the TTL job is enabled +sql.txn_fingerprint_id_cache.capacity integer 100 the maximum number of txn fingerprint IDs stored timeseries.storage.enabled boolean true if set, periodic timeseries data is stored within the cluster; disabling is not recommended unless you are storing the data elsewhere timeseries.storage.resolution_10s.ttl duration 240h0m0s the maximum age of time series data stored at the 10 second resolution. Data older than this is subject to rollup and deletion. timeseries.storage.resolution_30m.ttl duration 2160h0m0s the maximum age of time series data stored at the 30 minute resolution. Data older than this is subject to deletion. diff --git a/docs/generated/settings/settings.html b/docs/generated/settings/settings.html index bd41c08e3c9b..937184072562 100644 --- a/docs/generated/settings/settings.html +++ b/docs/generated/settings/settings.html @@ -204,6 +204,7 @@ sql.ttl.default_range_concurrencyinteger1default amount of ranges to process at once during a TTL delete sql.ttl.default_select_batch_sizeinteger500default amount of rows to select in a single query during a TTL job sql.ttl.job.enabledbooleantruewhether the TTL job is enabled +sql.txn_fingerprint_id_cache.capacityinteger100the maximum number of txn fingerprint IDs stored timeseries.storage.enabledbooleantrueif set, periodic timeseries data is stored within the cluster; disabling is not recommended unless you are storing the data elsewhere timeseries.storage.resolution_10s.ttlduration240h0m0sthe maximum age of time series data stored at the 10 second resolution. Data older than this is subject to rollup and deletion. timeseries.storage.resolution_30m.ttlduration2160h0m0sthe maximum age of time series data stored at the 30 minute resolution. Data older than this is subject to deletion. diff --git a/docs/generated/swagger/spec.json b/docs/generated/swagger/spec.json index 84da400c3c76..a5eea69af292 100644 --- a/docs/generated/swagger/spec.json +++ b/docs/generated/swagger/spec.json @@ -773,6 +773,12 @@ }, "x-go-package": "github.com/cockroachdb/cockroach/pkg/server/serverpb" }, + "Duration": { + "description": "A Duration represents the elapsed time between two instants\nas an int64 nanosecond count. The representation limits the\nlargest representable duration to approximately 290 years.", + "type": "integer", + "format": "int64", + "x-go-package": "time" + }, "EventsResponse": { "description": "EventsResponse contains a set of event log entries. This is always limited\nto the latest N entries (N is enforced in the associated endpoint).", "type": "object", @@ -1225,6 +1231,12 @@ "node_id": { "$ref": "#/definitions/NodeID" }, + "num_txns_executed": { + "description": "Count of the number of transactions that have been opened on this session.\nThis count includes transactions that are in progress.", + "type": "integer", + "format": "int32", + "x-go-name": "NumTxnsExecuted" + }, "start": { "description": "Timestamp of session's start.", "type": "string", @@ -1234,6 +1246,17 @@ "status": { "$ref": "#/definitions/Session_Status" }, + "total_active_time": { + "$ref": "#/definitions/Duration" + }, + "txn_fingerprint_ids": { + "description": "List of transaction fingerprint IDs in this session.", + "type": "array", + "items": { + "$ref": "#/definitions/TransactionFingerprintID" + }, + "x-go-name": "TxnFingerprintIDs" + }, "username": { "description": "Username of the user for this session.", "type": "string", @@ -1498,6 +1521,12 @@ }, "x-go-package": "github.com/cockroachdb/cockroach/pkg/util/hlc" }, + "TransactionFingerprintID": { + "description": "TransactionFingerprintID is the hashed string constructed using the\nindividual statement fingerprint IDs that comprise the transaction.", + "type": "integer", + "format": "uint64", + "x-go-package": "github.com/cockroachdb/cockroach/pkg/roachpb" + }, "TxnInfo": { "type": "object", "title": "TxnInfo represents an in flight user transaction on some Session.", diff --git a/pkg/ccl/logictestccl/testdata/logic_test/crdb_internal_tenant b/pkg/ccl/logictestccl/testdata/logic_test/crdb_internal_tenant index dfe0b113e969..ec7309b07d59 100644 --- a/pkg/ccl/logictestccl/testdata/logic_test/crdb_internal_tenant +++ b/pkg/ccl/logictestccl/testdata/logic_test/crdb_internal_tenant @@ -245,15 +245,15 @@ SELECT * FROM crdb_internal.cluster_transactions WHERE node_id < 0 ---- id node_id session_id start txn_string application_name num_stmts num_retries num_auto_retries last_auto_retry_reason -query ITTTTTTTTTTTTT colnames +query ITTTTTTTTTTTTTT colnames SELECT * FROM crdb_internal.node_sessions WHERE node_id < 0 ---- -node_id session_id user_name client_address application_name active_queries last_active_query session_start oldest_query_start kv_txn alloc_bytes max_alloc_bytes status session_end +node_id session_id user_name client_address application_name active_queries last_active_query num_txns_executed session_start oldest_query_start kv_txn alloc_bytes max_alloc_bytes status session_end -query ITTTTTTTTTTTTT colnames +query ITTTTTTTTTTTTTT colnames SELECT * FROM crdb_internal.cluster_sessions WHERE node_id < 0 ---- -node_id session_id user_name client_address application_name active_queries last_active_query session_start oldest_query_start kv_txn alloc_bytes max_alloc_bytes status session_end +node_id session_id user_name client_address application_name active_queries last_active_query num_txns_executed session_start oldest_query_start kv_txn alloc_bytes max_alloc_bytes status session_end query IIITTTI colnames SELECT * FROM crdb_internal.node_contention_events WHERE table_id < 0 diff --git a/pkg/server/serverpb/status.proto b/pkg/server/serverpb/status.proto index 7b91df975dd6..9388bbc3f10d 100644 --- a/pkg/server/serverpb/status.proto +++ b/pkg/server/serverpb/status.proto @@ -949,9 +949,11 @@ message Session { // Information about the txn in progress on this session. Nil if the // session doesn't currently have a transaction. TxnInfo active_txn = 12; + // The SQL statement fingerprint of the last query executed on this session, // compatible with StatementStatisticsKey. string last_active_query_no_constants = 13; + // Enum for sessions status. enum Status { ACTIVE = 0; @@ -960,9 +962,23 @@ message Session { } // The session's status. Status status = 14; + // Timestamp of session's end. google.protobuf.Timestamp end = 15 [ (gogoproto.nullable) = true, (gogoproto.stdtime) = true ]; + + // Count of the number of transactions that have been opened on this session. + // This count includes transactions that are in progress. + int32 num_txns_executed = 16; + + // List of transaction fingerprint IDs in this session. + repeated uint64 txn_fingerprint_ids = 17 [(gogoproto.customname) = "TxnFingerprintIDs", + (gogoproto.customtype) = "github.com/cockroachdb/cockroach/pkg/roachpb.TransactionFingerprintID", + (gogoproto.nullable) = false]; + + // The session's total active time. + google.protobuf.Duration total_active_time = 18 [(gogoproto.nullable) = false, + (gogoproto.stdduration) = true]; } // An error wrapper object for ListSessionsResponse. diff --git a/pkg/sql/BUILD.bazel b/pkg/sql/BUILD.bazel index dd593a2f2336..3eff0529147e 100644 --- a/pkg/sql/BUILD.bazel +++ b/pkg/sql/BUILD.bazel @@ -226,6 +226,7 @@ go_library( "testutils.go", "topk.go", "truncate.go", + "txn_fingerprint_id_cache.go", "txn_state.go", "type_change.go", "unary.go", @@ -585,6 +586,7 @@ go_test( "temporary_schema_test.go", "tenant_test.go", "trace_test.go", + "txn_fingerprint_id_cache_test.go", "txn_restart_test.go", "txn_state_test.go", "type_change_test.go", diff --git a/pkg/sql/conn_executor.go b/pkg/sql/conn_executor.go index 637289efc691..744d10bee10c 100644 --- a/pkg/sql/conn_executor.go +++ b/pkg/sql/conn_executor.go @@ -884,6 +884,8 @@ func (s *Server) newConnExecutor( stmtDiagnosticsRecorder: s.cfg.StmtDiagnosticsRecorder, indexUsageStats: s.indexUsageStats, txnIDCacheWriter: s.txnIDCache, + totalActiveTimeStopWatch: timeutil.NewStopWatch(), + txnFingerprintIDCache: NewTxnFingerprintIDCache(s.cfg.Settings, sessionRootMon), } ex.state.txnAbortCount = ex.metrics.EngineMetrics.TxnAbortCount @@ -1489,6 +1491,14 @@ type connExecutor struct { // txnIDCacheWriter is used to write txnidcache.ResolvedTxnID to the // Transaction ID Cache. txnIDCacheWriter txnidcache.Writer + + // txnFingerprintIDCache is used to track the most recent + // txnFingerprintIDs executed in this session. + txnFingerprintIDCache *TxnFingerprintIDCache + + // totalActiveTimeStopWatch tracks the total active time of the session. + // This is defined as the time spent executing transactions and statements. + totalActiveTimeStopWatch *timeutil.StopWatch } // ctxHolder contains a connection's context and, while session tracing is @@ -2865,6 +2875,10 @@ func (ex *connExecutor) txnStateTransitionsApplyWrapper( // Start of the transaction, so no statements were executed earlier. // Bump the txn counter for logging. ex.extraTxnState.txnCounter++ + + // Session is considered active when executing a transaction. + ex.totalActiveTimeStopWatch.Start() + if !ex.server.cfg.Codec.ForSystemTenant() { // Update the leased descriptor collection with the current sqlliveness.Session. // This is required in the multi-tenant environment to update the transaction @@ -3120,6 +3134,12 @@ func (ex *connExecutor) serialize() serverpb.Session { remoteStr = sd.RemoteAddr.String() } + txnFingerprintIDs := ex.txnFingerprintIDCache.GetAllTxnFingerprintIDs() + sessionActiveTime := ex.totalActiveTimeStopWatch.Elapsed() + if startedAt, started := ex.totalActiveTimeStopWatch.LastStartedAt(); started { + sessionActiveTime = time.Duration(sessionActiveTime.Nanoseconds() + timeutil.Since(startedAt).Nanoseconds()) + } + return serverpb.Session{ Username: sd.SessionUser().Normalized(), ClientAddress: remoteStr, @@ -3127,12 +3147,15 @@ func (ex *connExecutor) serialize() serverpb.Session { Start: ex.phaseTimes.GetSessionPhaseTime(sessionphase.SessionInit).UTC(), ActiveQueries: activeQueries, ActiveTxn: activeTxnInfo, + NumTxnsExecuted: int32(ex.extraTxnState.txnCounter), + TxnFingerprintIDs: txnFingerprintIDs, LastActiveQuery: lastActiveQuery, ID: ex.sessionID.GetBytes(), AllocBytes: ex.mon.AllocBytes(), MaxAllocBytes: ex.mon.MaximumBytes(), LastActiveQueryNoConstants: lastActiveQueryNoConstants, Status: status, + TotalActiveTime: sessionActiveTime, } } diff --git a/pkg/sql/conn_executor_exec.go b/pkg/sql/conn_executor_exec.go index be845b32310f..2d5f1b9d1271 100644 --- a/pkg/sql/conn_executor_exec.go +++ b/pkg/sql/conn_executor_exec.go @@ -2110,6 +2110,14 @@ func (ex *connExecutor) onTxnFinish(ctx context.Context, ev txnEvent) { ex.phaseTimes.SetSessionPhaseTime(sessionphase.SessionEndExecTransaction, timeutil.Now()) transactionFingerprintID := roachpb.TransactionFingerprintID(ex.extraTxnState.transactionStatementsHash.Sum()) + + err := ex.txnFingerprintIDCache.Add(transactionFingerprintID) + if err != nil { + if log.V(1) { + log.Warningf(ctx, "failed to enqueue transactionFingerprintID = %d: %s", transactionFingerprintID, err) + } + } + if !implicit { ex.statsCollector.EndExplicitTransaction( ctx, @@ -2123,7 +2131,7 @@ func (ex *connExecutor) onTxnFinish(ctx context.Context, ev txnEvent) { transactionFingerprintID, ) } - err := ex.recordTransactionFinish(ctx, transactionFingerprintID, ev, implicit, txnStart) + err = ex.recordTransactionFinish(ctx, transactionFingerprintID, ev, implicit, txnStart) if err != nil { if log.V(1) { log.Warningf(ctx, "failed to record transaction stats: %s", err) @@ -2219,6 +2227,7 @@ func (ex *connExecutor) recordTransactionFinish( txnEnd := timeutil.Now() txnTime := txnEnd.Sub(txnStart) + ex.totalActiveTimeStopWatch.Stop() if ex.executorType != executorTypeInternal { ex.metrics.EngineMetrics.SQLTxnsOpen.Dec(1) } diff --git a/pkg/sql/conn_executor_test.go b/pkg/sql/conn_executor_test.go index 568417f825a6..c7363de4ac55 100644 --- a/pkg/sql/conn_executor_test.go +++ b/pkg/sql/conn_executor_test.go @@ -29,6 +29,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/security/username" + "github.com/cockroachdb/cockroach/pkg/server/serverpb" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" @@ -1623,6 +1624,94 @@ func TestEmptyTxnIsBeingCorrectlyCounted(t *testing.T) { "after executing empty transactions, but it was not") } +//TestSessionTotalActiveTime tests that a session's total active time is +//correctly being recorded as transactions are executed. +func TestSessionTotalActiveTime(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + ctx := context.Background() + params := base.TestServerArgs{} + s, mainDB, _ := serverutils.StartServer(t, params) + defer s.Stopper().Stop(ctx) + + _, err := mainDB.Exec(fmt.Sprintf("CREATE USER %s", username.TestUser)) + if err != nil { + t.Fatal(err) + } + + pgURL, cleanupDB := sqlutils.PGUrl( + t, s.ServingSQLAddr(), "TestSessionTotalActiveTime", url.User(username.TestUser)) + defer cleanupDB() + rawSQL, err := gosql.Open("postgres", pgURL.String()) + if err != nil { + t.Fatal(err) + } + + defer func() { + err := rawSQL.Close() + if err != nil { + t.Fatal(err) + } + }() + + getSessionWithTestUser := func() *serverpb.Session { + sessions := s.SQLServer().(*sql.Server).GetExecutorConfig().SessionRegistry.SerializeAll() + for _, s := range sessions { + if s.Username == username.TestUser { + return &s + } + } + t.Fatalf("expected session with username %s", username.TestUser) + return nil + } + + sqlDB := sqlutils.MakeSQLRunner(rawSQL) + sqlDB.Exec(t, "SELECT 1") + session := getSessionWithTestUser() + activeTimeNanos := session.TotalActiveTime.Nanoseconds() + + // We will execute different types of transactions. + // After each execution, verify the total active time has increased, but is no + // longer increasing after the transaction has completed. + testCases := []struct { + Query string + // SessionActiveAfterExecution signifies that the active time should still be active after this query. + SessionActiveAfterExecution bool + }{ + {"SELECT 1", false}, + // Test explicit transaction. + {"BEGIN", true}, + {"SELECT 1", true}, + {"SELECT 1, 2", true}, + {"COMMIT", false}, + {"BEGIN", true}, + {"SELECT crdb_internal.force_retry('1s')", true}, + {"COMMIT", false}, + } + + for _, tc := range testCases { + sqlDB.Exec(t, tc.Query) + if tc.Query == "crdb_internal.force_retry('1s'" { + continue + } + // Check that the total active time has increased. + session = getSessionWithTestUser() + require.Greater(t, session.TotalActiveTime.Nanoseconds(), activeTimeNanos) + + activeTimeNanos = session.TotalActiveTime.Nanoseconds() + session = getSessionWithTestUser() + + if tc.SessionActiveAfterExecution { + require.Greater(t, session.TotalActiveTime.Nanoseconds(), activeTimeNanos) + } else { + require.Equal(t, activeTimeNanos, session.TotalActiveTime.Nanoseconds()) + } + + activeTimeNanos = session.TotalActiveTime.Nanoseconds() + } +} + // dynamicRequestFilter exposes a filter method which is a // kvserverbase.ReplicaRequestFilter but can be set dynamically. type dynamicRequestFilter struct { diff --git a/pkg/sql/crdb_internal.go b/pkg/sql/crdb_internal.go index d1284372d05e..dcf3c615cc13 100644 --- a/pkg/sql/crdb_internal.go +++ b/pkg/sql/crdb_internal.go @@ -1910,6 +1910,7 @@ CREATE TABLE crdb_internal.%s ( application_name STRING, -- the name of the application as per SET application_name active_queries STRING, -- the currently running queries as SQL last_active_query STRING, -- the query that finished last on this session as SQL + num_txns_executed INT, -- the number of transactions that were executed so far on this session session_start TIMESTAMP, -- the time when the session was opened oldest_query_start TIMESTAMP, -- the time when the oldest query in the session was started kv_txn STRING, -- the ID of the current KV transaction @@ -2011,6 +2012,7 @@ func populateSessionsTable( tree.NewDString(session.ApplicationName), tree.NewDString(activeQueries.String()), tree.NewDString(session.LastActiveQuery), + tree.NewDInt(tree.DInt(session.NumTxnsExecuted)), startTSDatum, oldestStartDatum, kvTxnIDDatum, @@ -2036,6 +2038,7 @@ func populateSessionsTable( tree.DNull, // application name tree.NewDString("-- "+rpcErr.Message), // active queries tree.DNull, // last active query + tree.DNull, // num txns executed tree.DNull, // session start tree.DNull, // oldest_query_start tree.DNull, // kv_txn diff --git a/pkg/sql/delegate/show_sessions.go b/pkg/sql/delegate/show_sessions.go index f4adcc97f7f8..8c8ea6ed8895 100644 --- a/pkg/sql/delegate/show_sessions.go +++ b/pkg/sql/delegate/show_sessions.go @@ -16,7 +16,7 @@ import ( ) func (d *delegator) delegateShowSessions(n *tree.ShowSessions) (tree.Statement, error) { - const query = `SELECT node_id, session_id, status, user_name, client_address, application_name, active_queries, last_active_query, session_start, oldest_query_start FROM crdb_internal.` + const query = `SELECT node_id, session_id, status, user_name, client_address, application_name, active_queries, last_active_query, session_start, oldest_query_start, num_txns_executed FROM crdb_internal.` table := `node_sessions` if n.Cluster { table = `cluster_sessions` diff --git a/pkg/sql/logictest/testdata/logic_test/crdb_internal b/pkg/sql/logictest/testdata/logic_test/crdb_internal index e270b9fb337d..3b9302392ab4 100644 --- a/pkg/sql/logictest/testdata/logic_test/crdb_internal +++ b/pkg/sql/logictest/testdata/logic_test/crdb_internal @@ -365,15 +365,15 @@ SELECT * FROM crdb_internal.cluster_transactions WHERE node_id < 0 ---- id node_id session_id start txn_string application_name num_stmts num_retries num_auto_retries last_auto_retry_reason -query ITTTTTTTTTTTTT colnames +query ITTTTTTTTTTTTTT colnames SELECT * FROM crdb_internal.node_sessions WHERE node_id < 0 ---- -node_id session_id user_name client_address application_name active_queries last_active_query session_start oldest_query_start kv_txn alloc_bytes max_alloc_bytes status session_end +node_id session_id user_name client_address application_name active_queries last_active_query num_txns_executed session_start oldest_query_start kv_txn alloc_bytes max_alloc_bytes status session_end -query ITTTTTTTTTTTTT colnames +query ITTTTTTTTTTTTTT colnames SELECT * FROM crdb_internal.cluster_sessions WHERE node_id < 0 ---- -node_id session_id user_name client_address application_name active_queries last_active_query session_start oldest_query_start kv_txn alloc_bytes max_alloc_bytes status session_end +node_id session_id user_name client_address application_name active_queries last_active_query num_txns_executed session_start oldest_query_start kv_txn alloc_bytes max_alloc_bytes status session_end query IIITTTI colnames SELECT * FROM crdb_internal.node_contention_events WHERE table_id < 0 diff --git a/pkg/sql/logictest/testdata/logic_test/create_statements b/pkg/sql/logictest/testdata/logic_test/create_statements index dd8da9a6ae65..bd8caa0ec624 100644 --- a/pkg/sql/logictest/testdata/logic_test/create_statements +++ b/pkg/sql/logictest/testdata/logic_test/create_statements @@ -326,6 +326,7 @@ CREATE TABLE crdb_internal.cluster_sessions ( application_name STRING NULL, active_queries STRING NULL, last_active_query STRING NULL, + num_txns_executed INT8 NULL, session_start TIMESTAMP NULL, oldest_query_start TIMESTAMP NULL, kv_txn STRING NULL, @@ -341,6 +342,7 @@ CREATE TABLE crdb_internal.cluster_sessions ( application_name STRING NULL, active_queries STRING NULL, last_active_query STRING NULL, + num_txns_executed INT8 NULL, session_start TIMESTAMP NULL, oldest_query_start TIMESTAMP NULL, kv_txn STRING NULL, @@ -964,6 +966,7 @@ CREATE TABLE crdb_internal.node_sessions ( application_name STRING NULL, active_queries STRING NULL, last_active_query STRING NULL, + num_txns_executed INT8 NULL, session_start TIMESTAMP NULL, oldest_query_start TIMESTAMP NULL, kv_txn STRING NULL, @@ -979,6 +982,7 @@ CREATE TABLE crdb_internal.node_sessions ( application_name STRING NULL, active_queries STRING NULL, last_active_query STRING NULL, + num_txns_executed INT8 NULL, session_start TIMESTAMP NULL, oldest_query_start TIMESTAMP NULL, kv_txn STRING NULL, diff --git a/pkg/sql/testdata/txn_fingerprint_id_cache b/pkg/sql/testdata/txn_fingerprint_id_cache new file mode 100644 index 000000000000..533be5d3e1bb --- /dev/null +++ b/pkg/sql/testdata/txn_fingerprint_id_cache @@ -0,0 +1,77 @@ +# Initialize TxnFingerprintIDCache +init capacity=10 +---- +size: 0 + +# Add four TxnFingerprintIDs: 5, 6, 7, 8 +enqueue id=05 +---- +size: 1 + +enqueue id=06 +---- +size: 2 + +enqueue id=07 +---- +size: 3 + +enqueue id=08 +---- +size: 4 + +# There should be 4 valid TxnFingerprintIDs +show +---- +[8 7 6 5] + +enqueue id=09 +---- +size: 5 + +enqueue id=10 +---- +size: 6 + +show +---- +[10 9 8 7 6 5] + +# Decrease the TxnFingerprintIDCacheCapacity cluster setting to below current size. +override capacity=3 +---- +TxnFingerprintIDCacheCapacity: 3 + +# Enqueue another id to +enqueue id=11 +---- +size: 3 + +# The cache should have the most recent 3 insertions. +show +---- +[11 10 9] + +# Check that retrieving IDs also properly truncates the cache when the capacity has +# been changed. +# Increase capacity back up to 5, insert some values, then decrease capacity to 2 and +# retrieve all ids. +override capacity=5 +---- +TxnFingerprintIDCacheCapacity: 5 + +enqueue id=12 +---- +size: 4 + +enqueue id=13 +---- +size: 5 + +override capacity=2 +---- +TxnFingerprintIDCacheCapacity: 2 + +show +---- +[13 12] diff --git a/pkg/sql/txn_fingerprint_id_cache.go b/pkg/sql/txn_fingerprint_id_cache.go new file mode 100644 index 000000000000..5f6c154743c7 --- /dev/null +++ b/pkg/sql/txn_fingerprint_id_cache.go @@ -0,0 +1,130 @@ +// Copyright 2022 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package sql + +import ( + "context" + + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/settings" + "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/util/cache" + "github.com/cockroachdb/cockroach/pkg/util/mon" + "github.com/cockroachdb/cockroach/pkg/util/syncutil" +) + +// TxnFingerprintIDCacheCapacity is the cluster setting that controls the +// capacity of the txn fingerprint ID cache. The cache will be resized +// on the next insert or get operation. +var TxnFingerprintIDCacheCapacity = settings.RegisterIntSetting( + settings.TenantWritable, + "sql.txn_fingerprint_id_cache.capacity", + "the maximum number of txn fingerprint IDs stored", + 100, + settings.NonNegativeInt, +).WithPublic() + +// TxnFingerprintIDCache is a thread-safe cache tracking transaction +// fingerprint IDs at the session level. +type TxnFingerprintIDCache struct { + st *cluster.Settings + + mu struct { + syncutil.RWMutex + acc *mon.BoundAccount + cache *cache.UnorderedCache + } + + mon *mon.BytesMonitor +} + +// NewTxnFingerprintIDCache returns a new TxnFingerprintIDCache. +func NewTxnFingerprintIDCache( + st *cluster.Settings, parentMon *mon.BytesMonitor, +) *TxnFingerprintIDCache { + b := &TxnFingerprintIDCache{st: st} + + b.mu.cache = cache.NewUnorderedCache(cache.Config{ + Policy: cache.CacheFIFO, + ShouldEvict: func(size int, _, _ interface{}) bool { + // Note that because the cache evicts as many elements as possible + // when adding an element, the cache will appropriately truncate + // when the capacity cluster setting is changed on the addition + // of an entry. + capacity := TxnFingerprintIDCacheCapacity.Get(&st.SV) + return int64(size) > capacity + }, + OnEvictedEntry: func(entry *cache.Entry) { + b.mu.acc.Shrink(context.Background(), 1) + }, + }) + + monitor := mon.NewMonitorInheritWithLimit("txn-fingerprint-id-cache", 0 /* limit */, parentMon) + b.mon = monitor + b.mon.Start(context.Background(), parentMon, mon.BoundAccount{}) + + return b +} + +// Add adds a TxnFingerprintID to the cache, truncating the cache to the cache's capacity +// if necessary. +func (b *TxnFingerprintIDCache) Add(value roachpb.TransactionFingerprintID) error { + b.mu.Lock() + defer b.mu.Unlock() + + if err := b.mu.acc.Grow(context.Background(), 1); err != nil { + return err + } + + b.mu.cache.Add(value, value) + + return nil +} + +// GetAllTxnFingerprintIDs returns a slice of all TxnFingerprintIDs in the cache. +// The cache may be truncated if the capacity was updated to a smaller size. +func (b *TxnFingerprintIDCache) GetAllTxnFingerprintIDs() []roachpb.TransactionFingerprintID { + b.mu.Lock() + defer b.mu.Unlock() + + size := int64(b.mu.cache.Len()) + capacity := TxnFingerprintIDCacheCapacity.Get(&b.st.SV) + if size > capacity { + size = capacity + } + + txnFingerprintIDs := make([]roachpb.TransactionFingerprintID, 0, size) + txnFingerprintIDsRemoved := make([]roachpb.TransactionFingerprintID, 0) + + b.mu.cache.Do(func(entry *cache.Entry) { + id := entry.Value.(roachpb.TransactionFingerprintID) + + if int64(len(txnFingerprintIDs)) == size { + txnFingerprintIDsRemoved = append(txnFingerprintIDsRemoved, id) + return + } + + txnFingerprintIDs = append(txnFingerprintIDs, id) + }) + + for _, id := range txnFingerprintIDsRemoved { + b.mu.cache.Del(id) + } + + return txnFingerprintIDs +} + +func (b *TxnFingerprintIDCache) size() int { + b.mu.RLock() + defer b.mu.RUnlock() + + return b.mu.cache.Len() +} diff --git a/pkg/sql/txn_fingerprint_id_cache_test.go b/pkg/sql/txn_fingerprint_id_cache_test.go new file mode 100644 index 000000000000..97de2eef0541 --- /dev/null +++ b/pkg/sql/txn_fingerprint_id_cache_test.go @@ -0,0 +1,159 @@ +// Copyright 2022 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package sql + +import ( + "context" + "fmt" + "math" + "sort" + "strconv" + "testing" + + "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/server/serverpb" + "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" + "github.com/cockroachdb/cockroach/pkg/sql/tests" + "github.com/cockroachdb/cockroach/pkg/testutils" + "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" + "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" + "github.com/cockroachdb/cockroach/pkg/util/leaktest" + "github.com/cockroachdb/cockroach/pkg/util/log" + "github.com/cockroachdb/cockroach/pkg/util/mon" + "github.com/cockroachdb/cockroach/pkg/util/uuid" + "github.com/cockroachdb/datadriven" + "github.com/stretchr/testify/require" +) + +func TestTxnFingerprintIDCacheDataDriven(t *testing.T) { + defer leaktest.AfterTest(t)() + var txnFingerprintIDCache *TxnFingerprintIDCache + + datadriven.Walk(t, testutils.TestDataPath(t, "txn_fingerprint_id_cache"), func(t *testing.T, path string) { + datadriven.RunTest(t, path, func(t *testing.T, d *datadriven.TestData) string { + ctx := context.Background() + switch d.Cmd { + case "init": + var capacity int + d.ScanArgs(t, "capacity", &capacity) + + st := &cluster.Settings{} + monitor := mon.NewUnlimitedMonitor( + ctx, + "test", + mon.MemoryResource, + nil, /* currCount */ + nil, /* maxHist */ + math.MaxInt64, + st, + ) + txnFingerprintIDCache = NewTxnFingerprintIDCache(st, monitor) + + TxnFingerprintIDCacheCapacity.Override(ctx, &st.SV, int64(capacity)) + + return fmt.Sprintf("size: %d", txnFingerprintIDCache.size()) + + case "override": + var capacity int + d.ScanArgs(t, "capacity", &capacity) + TxnFingerprintIDCacheCapacity.Override(ctx, &txnFingerprintIDCache.st.SV, int64(capacity)) + capacityClusterSetting := TxnFingerprintIDCacheCapacity.Get(&txnFingerprintIDCache.st.SV) + return fmt.Sprintf("TxnFingerprintIDCacheCapacity: %d", capacityClusterSetting) + + case "enqueue": + var idStr string + d.ScanArgs(t, "id", &idStr) + + id, err := strconv.ParseUint(idStr, 10, 64) + require.NoError(t, err) + txnFingerprintID := roachpb.TransactionFingerprintID(id) + + err = txnFingerprintIDCache.Add(txnFingerprintID) + require.NoError(t, err) + + return fmt.Sprintf("size: %d", txnFingerprintIDCache.size()) + + case "show": + return printTxnFingerprintIDCache(txnFingerprintIDCache) + + default: + } + return "" + + }) + }) +} + +func printTxnFingerprintIDCache(txnFingerprintCache *TxnFingerprintIDCache) string { + txnFingerprintIDs := txnFingerprintCache.GetAllTxnFingerprintIDs() + + return fmt.Sprintf("%d", txnFingerprintIDs) +} + +func TestTxnFingerprintIDCache(t *testing.T) { + defer leaktest.AfterTest(t)() + defer log.Scope(t).Close(t) + + ctx := context.Background() + + txnFingerprintIDsRecorded := make([]roachpb.TransactionFingerprintID, 0) + appName := "testTxnFingerprintIDCache" + + params, _ := tests.CreateTestServerParams() + params.Knobs.SQLExecutor = &ExecutorTestingKnobs{ + BeforeTxnStatsRecorded: func( + sessionData *sessiondata.SessionData, + _ uuid.UUID, + txnFingerprintID roachpb.TransactionFingerprintID, + ) { + if !sessionData.Internal { + // Record every query we issue through our sql connection. + txnFingerprintIDsRecorded = append(txnFingerprintIDsRecorded, txnFingerprintID) + } + }, + } + + testServer, sqlConn, _ := serverutils.StartServer(t, params) + + defer func() { + require.NoError(t, sqlConn.Close()) + testServer.Stopper().Stop(ctx) + }() + + testConn := sqlutils.MakeSQLRunner(sqlConn) + + testConn.Exec(t, "SET application_name = $1", appName) + testConn.Exec(t, "CREATE TABLE test AS SELECT generate_series(1, 10)") + testConn.Exec(t, "SELECT * FROM test") + testConn.Exec(t, "BEGIN; SELECT 1; SELECT 1, 2, 3; COMMIT;") + + sessions := testServer.SQLServer().(*Server).GetExecutorConfig().SessionRegistry.SerializeAll() + + var session *serverpb.Session + for i, s := range sessions { + if s.ApplicationName == appName { + session = &sessions[i] + break + } + } + require.NotNil(t, session) + + sort.Slice(session.TxnFingerprintIDs, func(i, j int) bool { + return session.TxnFingerprintIDs[i] < session.TxnFingerprintIDs[j] + }) + + sort.Slice(txnFingerprintIDsRecorded, func(i, j int) bool { + return txnFingerprintIDsRecorded[i] < txnFingerprintIDsRecorded[j] + }) + + require.Equal(t, txnFingerprintIDsRecorded, session.TxnFingerprintIDs) +} diff --git a/pkg/ui/workspaces/cluster-ui/src/sessions/sessionsPage.fixture.ts b/pkg/ui/workspaces/cluster-ui/src/sessions/sessionsPage.fixture.ts index be4e732e323d..9e425ac6fa80 100644 --- a/pkg/ui/workspaces/cluster-ui/src/sessions/sessionsPage.fixture.ts +++ b/pkg/ui/workspaces/cluster-ui/src/sessions/sessionsPage.fixture.ts @@ -47,6 +47,8 @@ export const idleSession: SessionInfo = { alloc_bytes: Long.fromNumber(0), max_alloc_bytes: Long.fromNumber(10240), active_queries: [], + num_txns_executed: 1, + txn_fingerprint_ids: [], status: Status.IDLE, toJSON: () => ({}), }, @@ -85,6 +87,8 @@ export const idleTransactionSession: SessionInfo = { }, last_active_query_no_constants: "SHOW database", active_queries: [], + num_txns_executed: 1, + txn_fingerprint_ids: [], status: Status.IDLE, toJSON: () => ({}), }, @@ -138,6 +142,8 @@ export const activeSession: SessionInfo = { }, last_active_query_no_constants: "SHOW database", status: Status.ACTIVE, + num_txns_executed: 1, + txn_fingerprint_ids: [], toJSON: () => ({}), }, }; @@ -163,6 +169,8 @@ export const closedSession: SessionInfo = { nanos: 369989000, }, status: Status.CLOSED, + num_txns_executed: 1, + txn_fingerprint_ids: [], toJSON: () => ({}), }, }; diff --git a/pkg/util/timeutil/stopwatch.go b/pkg/util/timeutil/stopwatch.go index ed3bf152b517..04c03ca50450 100644 --- a/pkg/util/timeutil/stopwatch.go +++ b/pkg/util/timeutil/stopwatch.go @@ -81,6 +81,14 @@ func (w *StopWatch) Elapsed() time.Duration { return w.mu.elapsed } +// LastStartedAt returns the time the stopwatch was last started, and a bool +// indicating if the stopwatch is currently started. +func (w *StopWatch) LastStartedAt() (startedAt time.Time, started bool) { + w.mu.Lock() + defer w.mu.Unlock() + return w.mu.startedAt, w.mu.started +} + // TestTimeSource is a source of time that remembers when it was created (in // terms of the real time) and returns the time based on its creation time and // the number of "advances" it has had. It is used for testing only. From a743d82e34f1fb08f6886d8080d4ad413baee26f Mon Sep 17 00:00:00 2001 From: Aditya Maru Date: Wed, 8 Jun 2022 14:34:27 -0400 Subject: [PATCH 2/9] backupinfo: introduce a backupinfo package The backupinfo package contains logic related to interacting with information and metadata describing the backup. After this change we have `backupdest` depending on `backupinfo`. Release note: None --- pkg/BUILD.bazel | 2 + pkg/ccl/backupccl/BUILD.bazel | 7 +- pkg/ccl/backupccl/backup_job.go | 43 +- pkg/ccl/backupccl/backup_planning.go | 135 +---- pkg/ccl/backupccl/backup_test.go | 48 +- pkg/ccl/backupccl/backupbase/constants.go | 7 +- pkg/ccl/backupccl/backupdest/BUILD.bazel | 3 + .../backupdest/backup_destination.go | 210 ++++++- .../backupccl/backupdest/incrementals_test.go | 2 +- .../backupccl/backupencryption/encryption.go | 7 + pkg/ccl/backupccl/backupinfo/BUILD.bazel | 87 +++ .../{ => backupinfo}/backup_metadata.go | 21 +- .../{ => backupinfo}/backup_metadata_test.go | 35 +- pkg/ccl/backupccl/backupinfo/main_test.go | 33 ++ .../{ => backupinfo}/manifest_handling.go | 511 ++++++++---------- pkg/ccl/backupccl/backuputils/testutils.go | 10 + pkg/ccl/backupccl/restore_job.go | 39 +- pkg/ccl/backupccl/restore_planning.go | 7 +- pkg/ccl/backupccl/restore_span_covering.go | 3 +- pkg/ccl/backupccl/show.go | 37 +- pkg/ccl/backupccl/targets.go | 7 +- pkg/ccl/backupccl/utils_test.go | 3 +- pkg/ccl/cliccl/BUILD.bazel | 1 + pkg/ccl/cliccl/debug_backup.go | 9 +- 24 files changed, 723 insertions(+), 544 deletions(-) create mode 100644 pkg/ccl/backupccl/backupinfo/BUILD.bazel rename pkg/ccl/backupccl/{ => backupinfo}/backup_metadata.go (98%) rename pkg/ccl/backupccl/{ => backupinfo}/backup_metadata_test.go (87%) create mode 100644 pkg/ccl/backupccl/backupinfo/main_test.go rename pkg/ccl/backupccl/{ => backupinfo}/manifest_handling.go (75%) diff --git a/pkg/BUILD.bazel b/pkg/BUILD.bazel index bcb4007fd292..76e79c4215e4 100644 --- a/pkg/BUILD.bazel +++ b/pkg/BUILD.bazel @@ -10,6 +10,8 @@ ALL_TESTS = [ "//pkg/build/starlarkutil:starlarkutil_test", "//pkg/build/util:util_test", "//pkg/ccl/backupccl/backupdest:backupdest_test", + "//pkg/ccl/backupccl/backupinfo:backupinfo_disallowed_imports_test", + "//pkg/ccl/backupccl/backupinfo:backupinfo_test", "//pkg/ccl/backupccl/backupresolver:backupresolver_test", "//pkg/ccl/backupccl:backupccl_test", "//pkg/ccl/baseccl:baseccl_test", diff --git a/pkg/ccl/backupccl/BUILD.bazel b/pkg/ccl/backupccl/BUILD.bazel index 045e3dc38358..e3ca409b3d20 100644 --- a/pkg/ccl/backupccl/BUILD.bazel +++ b/pkg/ccl/backupccl/BUILD.bazel @@ -5,7 +5,6 @@ go_library( srcs = [ "alter_backup_planning.go", "backup_job.go", - "backup_metadata.go", "backup_planning.go", "backup_planning_tenant.go", "backup_processor.go", @@ -14,7 +13,6 @@ go_library( "create_scheduled_backup.go", "file_sst_sink.go", "key_rewriter.go", - "manifest_handling.go", "restoration_data.go", "restore_data_processor.go", "restore_job.go", @@ -37,6 +35,7 @@ go_library( "//pkg/ccl/backupccl/backupbase", "//pkg/ccl/backupccl/backupdest", "//pkg/ccl/backupccl/backupencryption", + "//pkg/ccl/backupccl/backupinfo", "//pkg/ccl/backupccl/backuppb", "//pkg/ccl/backupccl/backupresolver", "//pkg/ccl/backupccl/backuputils", @@ -112,10 +111,8 @@ go_library( "//pkg/util/admission/admissionpb", "//pkg/util/contextutil", "//pkg/util/ctxgroup", - "//pkg/util/encoding", "//pkg/util/hlc", "//pkg/util/interval", - "//pkg/util/ioctx", "//pkg/util/json", "//pkg/util/log", "//pkg/util/log/eventpb", @@ -144,7 +141,6 @@ go_test( "alter_backup_test.go", "backup_cloud_test.go", "backup_intents_test.go", - "backup_metadata_test.go", "backup_planning_test.go", "backup_rand_test.go", "backup_tenant_test.go", @@ -178,6 +174,7 @@ go_test( "//pkg/ccl/backupccl/backupbase", "//pkg/ccl/backupccl/backupdest", "//pkg/ccl/backupccl/backupencryption", + "//pkg/ccl/backupccl/backupinfo", "//pkg/ccl/backupccl/backuppb", "//pkg/ccl/kvccl", "//pkg/ccl/multiregionccl", diff --git a/pkg/ccl/backupccl/backup_job.go b/pkg/ccl/backupccl/backup_job.go index 103cf6a1457a..c343ef3797fe 100644 --- a/pkg/ccl/backupccl/backup_job.go +++ b/pkg/ccl/backupccl/backup_job.go @@ -20,6 +20,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/ccl/backupccl/backupbase" "github.com/cockroachdb/cockroach/pkg/ccl/backupccl/backupdest" "github.com/cockroachdb/cockroach/pkg/ccl/backupccl/backupencryption" + "github.com/cockroachdb/cockroach/pkg/ccl/backupccl/backupinfo" "github.com/cockroachdb/cockroach/pkg/ccl/backupccl/backuppb" "github.com/cockroachdb/cockroach/pkg/ccl/backupccl/backuputils" "github.com/cockroachdb/cockroach/pkg/ccl/utilccl" @@ -243,7 +244,7 @@ func backup( RevisionStartTime: backupManifest.RevisionStartTime, }) - err := writeBackupManifestCheckpoint( + err := backupinfo.WriteBackupManifestCheckpoint( ctx, defaultURI, encryption, backupManifest, execCtx.ExecCfg(), execCtx.User(), ) if err != nil { @@ -290,8 +291,8 @@ func backup( // Set a unique filename for each partition backup descriptor. The ID // ensures uniqueness, and the kv string appended to the end is for // readability. - filename := fmt.Sprintf("%s_%d_%s", - backupPartitionDescriptorPrefix, nextPartitionedDescFilenameID, sanitizeLocalityKV(kv)) + filename := fmt.Sprintf("%s_%d_%s", backupPartitionDescriptorPrefix, + nextPartitionedDescFilenameID, backupinfo.SanitizeLocalityKV(kv)) nextPartitionedDescFilenameID++ backupManifest.PartitionDescriptorFilenames = append(backupManifest.PartitionDescriptorFilenames, filename) desc := backuppb.BackupPartitionDescriptor{ @@ -306,7 +307,7 @@ func backup( return err } defer store.Close() - return writeBackupPartitionDescriptor(ctx, store, filename, encryption, &desc) + return backupinfo.WriteBackupPartitionDescriptor(ctx, store, filename, encryption, &desc) }(); err != nil { return roachpb.RowCount{}, err } @@ -314,7 +315,8 @@ func backup( } resumerSpan.RecordStructured(&types.StringValue{Value: "writing backup manifest"}) - if err := writeBackupManifest(ctx, settings, defaultStore, backupbase.BackupManifestName, encryption, backupManifest); err != nil { + if err := backupinfo.WriteBackupManifest(ctx, settings, defaultStore, backupbase.BackupManifestName, + encryption, backupManifest); err != nil { return roachpb.RowCount{}, err } var tableStatistics []*stats.TableStatisticProto @@ -344,12 +346,12 @@ func backup( } resumerSpan.RecordStructured(&types.StringValue{Value: "writing backup table statistics"}) - if err := writeTableStatistics(ctx, defaultStore, backupStatisticsFileName, encryption, &statsTable); err != nil { + if err := backupinfo.WriteTableStatistics(ctx, defaultStore, encryption, &statsTable); err != nil { return roachpb.RowCount{}, err } - if writeMetadataSST.Get(&settings.SV) { - if err := writeBackupMetadataSST(ctx, defaultStore, encryption, backupManifest, tableStatistics); err != nil { + if backupinfo.WriteMetadataSST.Get(&settings.SV) { + if err := backupinfo.WriteBackupMetadataSST(ctx, defaultStore, encryption, backupManifest, tableStatistics); err != nil { err = errors.Wrap(err, "writing forward-compat metadata sst") if !build.IsRelease() { return roachpb.RowCount{}, err @@ -448,7 +450,7 @@ func (b *backupResumer) Resume(ctx context.Context, execCtx interface{}) error { } } - if err := writeBackupManifestCheckpoint( + if err := backupinfo.WriteBackupManifestCheckpoint( ctx, details.URI, details.EncryptionOptions, backupManifest, p.ExecCfg(), p.User(), ); err != nil { return err @@ -730,20 +732,21 @@ func (b *backupResumer) readManifestOnResume( // they could be using either the new or the old foreign key // representations. We should just preserve whatever representation the // table descriptors were using and leave them alone. - desc, memSize, err := readBackupCheckpointManifest(ctx, mem, defaultStore, backupManifestCheckpointName, - details.EncryptionOptions) + desc, memSize, err := backupinfo.ReadBackupCheckpointManifest(ctx, mem, defaultStore, + backupinfo.BackupManifestCheckpointName, details.EncryptionOptions) if err != nil { if !errors.Is(err, cloud.ErrFileDoesNotExist) { return nil, 0, errors.Wrapf(err, "reading backup checkpoint") } // Try reading temp checkpoint. - tmpCheckpoint := tempCheckpointFileNameForJob(b.job.ID()) - desc, memSize, err = readBackupCheckpointManifest(ctx, mem, defaultStore, tmpCheckpoint, details.EncryptionOptions) + tmpCheckpoint := backupinfo.TempCheckpointFileNameForJob(b.job.ID()) + desc, memSize, err = backupinfo.ReadBackupCheckpointManifest(ctx, mem, defaultStore, + tmpCheckpoint, details.EncryptionOptions) if err != nil { return nil, 0, err } // "Rename" temp checkpoint. - if err := writeBackupManifestCheckpoint( + if err := backupinfo.WriteBackupManifestCheckpoint( ctx, details.URI, details.EncryptionOptions, &desc, cfg, user, ); err != nil { mem.Shrink(ctx, memSize) @@ -753,8 +756,8 @@ func (b *backupResumer) readManifestOnResume( if err := defaultStore.Delete(ctx, tmpCheckpoint); err != nil { log.Errorf(ctx, "error removing temporary checkpoint %s", tmpCheckpoint) } - if err := defaultStore.Delete(ctx, backupProgressDirectory+"/"+tmpCheckpoint); err != nil { - log.Errorf(ctx, "error removing temporary checkpoint %s", backupProgressDirectory+"/"+tmpCheckpoint) + if err := defaultStore.Delete(ctx, backupinfo.BackupProgressDirectory+"/"+tmpCheckpoint); err != nil { + log.Errorf(ctx, "error removing temporary checkpoint %s", backupinfo.BackupProgressDirectory+"/"+tmpCheckpoint) } } @@ -850,18 +853,18 @@ func (b *backupResumer) deleteCheckpoint( defer exportStore.Close() // We first attempt to delete from base directory to account for older // backups, and then from the progress directory. - err = exportStore.Delete(ctx, backupManifestCheckpointName) + err = exportStore.Delete(ctx, backupinfo.BackupManifestCheckpointName) if err != nil { log.Warningf(ctx, "unable to delete checkpointed backup descriptor file in base directory: %+v", err) } - err = exportStore.Delete(ctx, backupManifestCheckpointName+backupManifestChecksumSuffix) + err = exportStore.Delete(ctx, backupinfo.BackupManifestCheckpointName+backupinfo.BackupManifestChecksumSuffix) if err != nil { log.Warningf(ctx, "unable to delete checkpoint checksum file in base directory: %+v", err) } // Delete will not delete a nonempty directory, so we have to go through // all files and delete each file one by one. - return exportStore.List(ctx, backupProgressDirectory, "", func(p string) error { - return exportStore.Delete(ctx, backupProgressDirectory+p) + return exportStore.List(ctx, backupinfo.BackupProgressDirectory, "", func(p string) error { + return exportStore.Delete(ctx, backupinfo.BackupProgressDirectory+p) }) }(); err != nil { log.Warningf(ctx, "unable to delete checkpointed backup descriptor file in progress directory: %+v", err) diff --git a/pkg/ccl/backupccl/backup_planning.go b/pkg/ccl/backupccl/backup_planning.go index ff6bc7947b21..2c8e668f6329 100644 --- a/pkg/ccl/backupccl/backup_planning.go +++ b/pkg/ccl/backupccl/backup_planning.go @@ -9,7 +9,6 @@ package backupccl import ( - "bytes" "context" "fmt" "reflect" @@ -22,12 +21,11 @@ import ( "github.com/cockroachdb/cockroach/pkg/ccl/backupccl/backupbase" "github.com/cockroachdb/cockroach/pkg/ccl/backupccl/backupdest" "github.com/cockroachdb/cockroach/pkg/ccl/backupccl/backupencryption" + "github.com/cockroachdb/cockroach/pkg/ccl/backupccl/backupinfo" "github.com/cockroachdb/cockroach/pkg/ccl/backupccl/backuppb" "github.com/cockroachdb/cockroach/pkg/ccl/backupccl/backupresolver" - "github.com/cockroachdb/cockroach/pkg/ccl/storageccl" "github.com/cockroachdb/cockroach/pkg/ccl/utilccl" "github.com/cockroachdb/cockroach/pkg/cloud" - "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/featureflag" "github.com/cockroachdb/cockroach/pkg/jobs" "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" @@ -55,7 +53,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/interval" "github.com/cockroachdb/cockroach/pkg/util/log" - "github.com/cockroachdb/cockroach/pkg/util/protoutil" "github.com/cockroachdb/cockroach/pkg/util/tracing" "github.com/cockroachdb/errors" pbtypes "github.com/gogo/protobuf/types" @@ -63,8 +60,6 @@ import ( const ( backupOptRevisionHistory = "revision_history" - backupOptEncPassphrase = "encryption_passphrase" - backupOptEncKMS = "kms" backupOptWithPrivileges = "privileges" backupOptAsJSON = "as_json" backupOptWithDebugIDs = "debug_ids" @@ -72,6 +67,9 @@ const ( backupOptDebugMetadataSST = "debug_dump_metadata_sst" backupOptEncDir = "encryption_info_dir" backupOptCheckFiles = "check_files" + // backupPartitionDescriptorPrefix is the file name prefix for serialized + // BackupPartitionDescriptor protos. + backupPartitionDescriptorPrefix = "BACKUP_PART" ) type tableAndIndex struct { @@ -786,120 +784,6 @@ func getScheduledBackupExecutionArgsFromSchedule( return sj, args, nil } -// writeBackupManifestCheckpoint writes a new BACKUP-CHECKPOINT MANIFEST and -// CHECKSUM file. If it is a pure v22.1 cluster or later, it will write a -// timestamped BACKUP-CHECKPOINT to the /progress directory. If it is a mixed -// cluster version, it will write a non timestamped BACKUP-CHECKPOINT to the -// base directory in order to not break backup jobs that resume on a v21.2 node. -func writeBackupManifestCheckpoint( - ctx context.Context, - storageURI string, - encryption *jobspb.BackupEncryptionOptions, - desc *backuppb.BackupManifest, - execCfg *sql.ExecutorConfig, - user username.SQLUsername, -) error { - var span *tracing.Span - ctx, span = tracing.ChildSpan(ctx, "write-backup-manifest-checkpoint") - defer span.Finish() - - defaultStore, err := execCfg.DistSQLSrv.ExternalStorageFromURI(ctx, storageURI, user) - if err != nil { - return err - } - defer defaultStore.Close() - - sort.Sort(BackupFileDescriptors(desc.Files)) - - descBuf, err := protoutil.Marshal(desc) - if err != nil { - return err - } - - descBuf, err = compressData(descBuf) - if err != nil { - return errors.Wrap(err, "compressing backup manifest") - } - - if encryption != nil { - encryptionKey, err := backupencryption.GetEncryptionKey(ctx, encryption, execCfg.Settings, defaultStore.ExternalIOConf()) - if err != nil { - return err - } - descBuf, err = storageccl.EncryptFile(descBuf, encryptionKey) - if err != nil { - return err - } - } - - // If the cluster is still running on a mixed version, we want to write - // to the base directory instead of the progress directory. That way if - // an old node resumes a backup, it doesn't have to start over. - if !execCfg.Settings.Version.IsActive(ctx, clusterversion.BackupDoesNotOverwriteLatestAndCheckpoint) { - // We want to overwrite the latest checkpoint in the base directory, - // just write to the non versioned BACKUP-CHECKPOINT file. - err = cloud.WriteFile(ctx, defaultStore, backupManifestCheckpointName, bytes.NewReader(descBuf)) - if err != nil { - return err - } - - checksum, err := getChecksum(descBuf) - if err != nil { - return err - } - - return cloud.WriteFile(ctx, defaultStore, backupManifestCheckpointName+backupManifestChecksumSuffix, bytes.NewReader(checksum)) - } - - // We timestamp the checkpoint files in order to enforce write once backups. - // When the job goes to read these timestamped files, it will List - // the checkpoints and pick the file whose name is lexicographically - // sorted to the top. This will be the last checkpoint we write, for - // details refer to newTimestampedCheckpointFileName. - filename := newTimestampedCheckpointFileName() - - // HTTP storage does not support listing and so we cannot rely on the - // above-mentioned List method to return us the latest checkpoint file. - // Instead, we will write a checkpoint once with a well-known filename, - // and teach the job to always reach for that filename in the face of - // a resume. We may lose progress, but this is a cost we are willing - // to pay to uphold write-once semantics. - if defaultStore.Conf().Provider == roachpb.ExternalStorageProvider_http { - // TODO (darryl): We should do this only for file not found or directory - // does not exist errors. As of right now we only specifically wrap - // ReadFile errors for file not found so this is not possible yet. - if r, err := defaultStore.ReadFile(ctx, backupProgressDirectory+"/"+backupManifestCheckpointName); err != nil { - // Since we did not find the checkpoint file this is the first time - // we are going to write a checkpoint, so write it with the well - // known filename. - filename = backupManifestCheckpointName - } else { - err = r.Close(ctx) - if err != nil { - return err - } - } - } - - err = cloud.WriteFile(ctx, defaultStore, backupProgressDirectory+"/"+filename, bytes.NewReader(descBuf)) - if err != nil { - return errors.Wrap(err, "calculating checksum") - } - - // Write the checksum file after we've successfully wrote the checkpoint. - checksum, err := getChecksum(descBuf) - if err != nil { - return errors.Wrap(err, "calculating checksum") - } - - err = cloud.WriteFile(ctx, defaultStore, backupProgressDirectory+"/"+filename+backupManifestChecksumSuffix, bytes.NewReader(checksum)) - if err != nil { - return err - } - - return nil -} - // planSchedulePTSChaining populates backupDetails with information relevant to // the chaining of protected timestamp records between scheduled backups. // Depending on whether backupStmt is a full or incremental backup, we populate @@ -1210,7 +1094,7 @@ func getBackupDetailAndManifest( } defer defaultStore.Close() - if err := checkForPreviousBackup(ctx, defaultStore, defaultURI); err != nil { + if err := backupinfo.CheckForPreviousBackup(ctx, defaultStore, defaultURI); err != nil { return jobspb.BackupDetails{}, backuppb.BackupManifest{}, err } @@ -1219,7 +1103,7 @@ func getBackupDetailAndManifest( mem := execCfg.RootMemoryMonitor.MakeBoundAccount() defer mem.Close(ctx) - prevBackups, encryptionOptions, memSize, err := fetchPreviousBackups(ctx, &mem, user, + prevBackups, encryptionOptions, memSize, err := backupinfo.FetchPreviousBackups(ctx, &mem, user, makeCloudStorage, prevs, *initialDetails.EncryptionOptions, kmsEnv) if err != nil { @@ -1357,6 +1241,9 @@ func getTenantInfo( return spans, tenants, nil } +// TODO(adityamaru): We need to move this method into manifest_handling.go but +// the method needs to be decomposed to decouple it from other planning related +// operations. func createBackupManifest( ctx context.Context, execCfg *sql.ExecutorConfig, @@ -1400,7 +1287,7 @@ func createBackupManifest( tables = append(tables, desc) // TODO (anzo): look into the tradeoffs of having all objects in the array to be in the same file, // vs having each object in a separate file, or somewhere in between. - statsFiles[desc.GetID()] = backupStatisticsFileName + statsFiles[desc.GetID()] = backupinfo.BackupStatisticsFileName } } @@ -1489,7 +1376,7 @@ func createBackupManifest( CompleteDbs: jobDetails.ResolvedCompleteDbs, Spans: spans, IntroducedSpans: newSpans, - FormatVersion: BackupFormatDescriptorTrackingVersion, + FormatVersion: backupinfo.BackupFormatDescriptorTrackingVersion, BuildInfo: build.GetInfo(), ClusterVersion: execCfg.Settings.Version.ActiveVersion(ctx).Version, ClusterID: execCfg.LogicalClusterID(), diff --git a/pkg/ccl/backupccl/backup_test.go b/pkg/ccl/backupccl/backup_test.go index 661d8b65099d..a9488b952d0b 100644 --- a/pkg/ccl/backupccl/backup_test.go +++ b/pkg/ccl/backupccl/backup_test.go @@ -39,6 +39,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/ccl/backupccl/backupbase" "github.com/cockroachdb/cockroach/pkg/ccl/backupccl/backupdest" "github.com/cockroachdb/cockroach/pkg/ccl/backupccl/backupencryption" + "github.com/cockroachdb/cockroach/pkg/ccl/backupccl/backupinfo" "github.com/cockroachdb/cockroach/pkg/ccl/backupccl/backuppb" _ "github.com/cockroachdb/cockroach/pkg/ccl/kvccl" _ "github.com/cockroachdb/cockroach/pkg/ccl/multiregionccl" @@ -140,7 +141,7 @@ func TestBackupRestoreStatementResult(t *testing.T) { if err != nil { t.Fatal(err) } - require.True(t, isGZipped(backupManifestBytes)) + require.True(t, backupinfo.IsGZipped(backupManifestBytes)) }) sqlDB.Exec(t, "CREATE DATABASE data2") @@ -297,7 +298,7 @@ func TestBackupRestorePartitioned(t *testing.T) { if err != nil { t.Fatal(err) } - require.True(t, isGZipped(backupPartitionBytes)) + require.True(t, backupinfo.IsGZipped(backupPartitionBytes)) } } } @@ -1535,7 +1536,7 @@ func TestBackupRestoreResume(t *testing.T) { testName string checkpointDirectory string }{ - {testName: "backup-progress-directory", checkpointDirectory: "/" + backupProgressDirectory}, + {testName: "backup-progress-directory", checkpointDirectory: "/" + backupinfo.BackupProgressDirectory}, {testName: "backup-base-directory", checkpointDirectory: ""}, } { item := item @@ -1561,7 +1562,7 @@ func TestBackupRestoreResume(t *testing.T) { if err := os.MkdirAll(backupDir+item.checkpointDirectory, 0755); err != nil { t.Fatal(err) } - checkpointFile := backupDir + item.checkpointDirectory + "/" + backupManifestCheckpointName + checkpointFile := backupDir + item.checkpointDirectory + "/" + backupinfo.BackupManifestCheckpointName if err := ioutil.WriteFile(checkpointFile, mockManifest, 0644); err != nil { t.Fatal(err) } @@ -1581,8 +1582,8 @@ func TestBackupRestoreResume(t *testing.T) { if err != nil { t.Fatal(err) } - if isGZipped(backupManifestBytes) { - backupManifestBytes, err = decompressData(ctx, nil, backupManifestBytes) + if backupinfo.IsGZipped(backupManifestBytes) { + backupManifestBytes, err = backupinfo.DecompressData(ctx, nil, backupManifestBytes) require.NoError(t, err) } var backupManifest backuppb.BackupManifest @@ -3918,8 +3919,8 @@ func TestBackupRestoreChecksum(t *testing.T) { if err != nil { t.Fatalf("%+v", err) } - if isGZipped(backupManifestBytes) { - backupManifestBytes, err = decompressData(context.Background(), nil, backupManifestBytes) + if backupinfo.IsGZipped(backupManifestBytes) { + backupManifestBytes, err = backupinfo.DecompressData(context.Background(), nil, backupManifestBytes) require.NoError(t, err) } if err := protoutil.Unmarshal(backupManifestBytes, &backupManifest); err != nil { @@ -5774,8 +5775,7 @@ func TestBackupRestoreCorruptedStatsIgnored(t *testing.T) { statsTable := backuppb.StatsTable{ Statistics: []*stats.TableStatisticProto{{TableID: descpb.ID(tableID + 1), Name: "notbank"}}, } - require.NoError(t, writeTableStatistics(ctx, store, backupStatisticsFileName, - nil /* encryption */, &statsTable)) + require.NoError(t, backupinfo.WriteTableStatistics(ctx, store, nil /* encryption */, &statsTable)) sqlDB.Exec(t, `CREATE DATABASE "data 2"`) sqlDB.Exec(t, fmt.Sprintf(`RESTORE data.bank FROM "%s" WITH skip_missing_foreign_keys, into_db = "%s"`, @@ -5783,7 +5783,7 @@ func TestBackupRestoreCorruptedStatsIgnored(t *testing.T) { // Delete the stats file to ensure a restore can succeed even if statistics do // not exist. - require.NoError(t, store.Delete(ctx, backupStatisticsFileName)) + require.NoError(t, store.Delete(ctx, backupinfo.BackupStatisticsFileName)) sqlDB.Exec(t, `CREATE DATABASE "data 3"`) sqlDB.Exec(t, fmt.Sprintf(`RESTORE data.bank FROM "%s" WITH skip_missing_foreign_keys, into_db = "%s"`, dest, "data 3")) @@ -8018,8 +8018,8 @@ func TestReadBackupManifestMemoryMonitoring(t *testing.T) { for i := 0; i < magic; i++ { desc.Files = append(desc.Files, backuppb.BackupManifest_File{Path: fmt.Sprintf("%d-file-%d", i, i)}) } - require.NoError(t, writeBackupManifest(ctx, st, storage, "testmanifest", encOpts, desc)) - _, sz, err := readBackupManifest(ctx, &mem, storage, "testmanifest", encOpts) + require.NoError(t, backupinfo.WriteBackupManifest(ctx, st, storage, "testmanifest", encOpts, desc)) + _, sz, err := backupinfo.ReadBackupManifest(ctx, &mem, storage, "testmanifest", encOpts) require.NoError(t, err) mem.Shrink(ctx, sz) mem.Close(ctx) @@ -8041,7 +8041,7 @@ func TestManifestTooNew(t *testing.T) { manifestPath := filepath.Join(rawDir, "too_new", backupbase.BackupManifestName) manifestData, err := ioutil.ReadFile(manifestPath) require.NoError(t, err) - manifestData, err = decompressData(context.Background(), nil, manifestData) + manifestData, err = backupinfo.DecompressData(context.Background(), nil, manifestData) require.NoError(t, err) var backupManifest backuppb.BackupManifest require.NoError(t, protoutil.Unmarshal(manifestData, &backupManifest)) @@ -8052,9 +8052,9 @@ func TestManifestTooNew(t *testing.T) { require.NoError(t, err) require.NoError(t, ioutil.WriteFile(manifestPath, manifestData, 0644 /* perm */)) // Also write the checksum file to match the new manifest. - checksum, err := getChecksum(manifestData) + checksum, err := backupinfo.GetChecksum(manifestData) require.NoError(t, err) - require.NoError(t, ioutil.WriteFile(manifestPath+backupManifestChecksumSuffix, checksum, 0644 /* perm */)) + require.NoError(t, ioutil.WriteFile(manifestPath+backupinfo.BackupManifestChecksumSuffix, checksum, 0644 /* perm */)) // Verify we reject it. sqlDB.ExpectErr(t, "backup from version 99.1 is newer than current version", `RESTORE DATABASE r1 FROM 'nodelocal://0/too_new'`) @@ -8065,9 +8065,9 @@ func TestManifestTooNew(t *testing.T) { require.NoError(t, err) require.NoError(t, ioutil.WriteFile(manifestPath, manifestData, 0644 /* perm */)) // Also write the checksum file to match the new manifest. - checksum, err = getChecksum(manifestData) + checksum, err = backupinfo.GetChecksum(manifestData) require.NoError(t, err) - require.NoError(t, ioutil.WriteFile(manifestPath+backupManifestChecksumSuffix, checksum, 0644 /* perm */)) + require.NoError(t, ioutil.WriteFile(manifestPath+backupinfo.BackupManifestChecksumSuffix, checksum, 0644 /* perm */)) // Prove we can restore again. sqlDB.Exec(t, `RESTORE DATABASE r1 FROM 'nodelocal://0/too_new'`) @@ -8079,9 +8079,9 @@ func TestManifestTooNew(t *testing.T) { require.NoError(t, err) require.NoError(t, ioutil.WriteFile(manifestPath, manifestData, 0644 /* perm */)) // Also write the checksum file to match the new manifest. - checksum, err = getChecksum(manifestData) + checksum, err = backupinfo.GetChecksum(manifestData) require.NoError(t, err) - require.NoError(t, ioutil.WriteFile(manifestPath+backupManifestChecksumSuffix, checksum, 0644 /* perm */)) + require.NoError(t, ioutil.WriteFile(manifestPath+backupinfo.BackupManifestChecksumSuffix, checksum, 0644 /* perm */)) // Prove we can restore again. sqlDB.Exec(t, `RESTORE DATABASE r1 FROM 'nodelocal://0/too_new'`) sqlDB.Exec(t, `DROP DATABASE r1`) @@ -9787,8 +9787,8 @@ func TestBackupNoOverwriteCheckpoint(t *testing.T) { require.NoError(t, store.List(ctx, latestFilePath+"/progress/", "", func(f string) error { // Don't double count checkpoints as there will be the manifest and // the checksum. - if !strings.HasSuffix(f, backupManifestChecksumSuffix) { - if strings.HasPrefix(f, backupManifestCheckpointName) { + if !strings.HasSuffix(f, backupinfo.BackupManifestChecksumSuffix) { + if strings.HasPrefix(f, backupinfo.BackupManifestCheckpointName) { actualNumCheckpointsWritten++ } } @@ -9814,7 +9814,7 @@ func TestBackupTimestampedCheckpointsAreLexicographical(t *testing.T) { numCheckpoints := 5 for i := 0; i < numCheckpoints; i++ { - checkpoints = append(checkpoints, newTimestampedCheckpointFileName()) + checkpoints = append(checkpoints, backupinfo.NewTimestampedCheckpointFileName()) // Occasionally, we call newTimestampedCheckpointFileName() in succession // too fast and the timestamp is the same. So wait for a moment to // avoid that. @@ -9914,7 +9914,7 @@ func TestBackupTimestampedCheckpointsAreLexicographical(t *testing.T) { require.NoError(t, err) for _, checkpoint := range checkpoints { var desc []byte - require.NoError(t, cloud.WriteFile(ctx, store, backupProgressDirectory+"/"+checkpoint, bytes.NewReader(desc))) + require.NoError(t, cloud.WriteFile(ctx, store, backupinfo.BackupProgressDirectory+"/"+checkpoint, bytes.NewReader(desc))) } require.NoError(t, err) var actual string diff --git a/pkg/ccl/backupccl/backupbase/constants.go b/pkg/ccl/backupccl/backupbase/constants.go index ff5bf49e43a4..ff1890d47c75 100644 --- a/pkg/ccl/backupccl/backupbase/constants.go +++ b/pkg/ccl/backupccl/backupbase/constants.go @@ -8,13 +8,8 @@ package backupbase +// TODO(adityamaru): Move constants to relevant backupccl packages. const ( - // IncludeManifest is a named const that can be passed to FindPriorBackups. - IncludeManifest = true - - // OmitManifest is a named const that can be passed to FindPriorBackups. - OmitManifest = false - // LatestFileName is the name of a file in the collection which contains the // path of the most recently taken full backup in the backup collection. LatestFileName = "LATEST" diff --git a/pkg/ccl/backupccl/backupdest/BUILD.bazel b/pkg/ccl/backupccl/backupdest/BUILD.bazel index 1653cb072fc1..484394f20385 100644 --- a/pkg/ccl/backupccl/backupdest/BUILD.bazel +++ b/pkg/ccl/backupccl/backupdest/BUILD.bazel @@ -10,6 +10,8 @@ go_library( visibility = ["//visibility:public"], deps = [ "//pkg/ccl/backupccl/backupbase", + "//pkg/ccl/backupccl/backupinfo", + "//pkg/ccl/backupccl/backuppb", "//pkg/ccl/backupccl/backuputils", "//pkg/cloud", "//pkg/clusterversion", @@ -24,6 +26,7 @@ go_library( "//pkg/util/encoding", "//pkg/util/hlc", "//pkg/util/ioctx", + "//pkg/util/mon", "//pkg/util/timeutil", "@com_github_cockroachdb_errors//:errors", ], diff --git a/pkg/ccl/backupccl/backupdest/backup_destination.go b/pkg/ccl/backupccl/backupdest/backup_destination.go index b3f27a4d40d9..cefb4f6050b3 100644 --- a/pkg/ccl/backupccl/backupdest/backup_destination.go +++ b/pkg/ccl/backupccl/backupdest/backup_destination.go @@ -13,10 +13,13 @@ import ( "encoding/hex" "fmt" "net/url" + "path" "regexp" "strings" "github.com/cockroachdb/cockroach/pkg/ccl/backupccl/backupbase" + "github.com/cockroachdb/cockroach/pkg/ccl/backupccl/backupinfo" + "github.com/cockroachdb/cockroach/pkg/ccl/backupccl/backuppb" "github.com/cockroachdb/cockroach/pkg/ccl/backupccl/backuputils" "github.com/cockroachdb/cockroach/pkg/cloud" "github.com/cockroachdb/cockroach/pkg/clusterversion" @@ -31,6 +34,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/util/encoding" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/ioctx" + "github.com/cockroachdb/cockroach/pkg/util/mon" "github.com/cockroachdb/cockroach/pkg/util/timeutil" "github.com/cockroachdb/errors" ) @@ -42,6 +46,10 @@ const ( // DefaultLocalityValue is the default locality tag used in a locality aware // backup/restore when an explicit COCKROACH_LOCALITY is not specified. DefaultLocalityValue = "default" + // includeManifest is a named const that can be passed to FindPriorBackups. + includeManifest = true + // OmitManifest is a named const that can be passed to FindPriorBackups. + OmitManifest = false ) // On some cloud storage platforms (i.e. GS, S3), backups in a base bucket may @@ -205,7 +213,7 @@ func ResolveDest( } defer incrementalStore.Close() - priors, err := FindPriorBackups(ctx, incrementalStore, backupbase.OmitManifest) + priors, err := FindPriorBackups(ctx, incrementalStore, OmitManifest) if err != nil { return "", "", "", nil, nil, errors.Wrap(err, "adjusting backup destination to append new layer to existing backup") } @@ -466,3 +474,203 @@ func ListFullBackupsInCollection( } return backupPaths, nil } + +// ResolveBackupManifests resolves the URIs that point to the incremental layers +// (each of which can be partitioned) of backups into the actual backup +// manifests and metadata required to RESTORE. If only one layer is explicitly +// provided, it is inspected to see if it contains "appended" layers internally +// that are then expanded into the result layers returned, similar to if those +// layers had been specified in `from` explicitly. +func ResolveBackupManifests( + ctx context.Context, + mem *mon.BoundAccount, + baseStores []cloud.ExternalStorage, + mkStore cloud.ExternalStorageFromURIFactory, + fullyResolvedBaseDirectory []string, + fullyResolvedIncrementalsDirectory []string, + endTime hlc.Timestamp, + encryption *jobspb.BackupEncryptionOptions, + user username.SQLUsername, +) ( + defaultURIs []string, + // mainBackupManifests contains the manifest located at each defaultURI in the backup chain. + mainBackupManifests []backuppb.BackupManifest, + localityInfo []jobspb.RestoreDetails_BackupLocalityInfo, + reservedMemSize int64, + _ error, +) { + var ownedMemSize int64 + defer func() { + if ownedMemSize != 0 { + mem.Shrink(ctx, ownedMemSize) + } + }() + baseManifest, memSize, err := backupinfo.ReadBackupManifestFromStore(ctx, mem, baseStores[0], encryption) + if err != nil { + return nil, nil, nil, 0, err + } + ownedMemSize += memSize + + incStores := make([]cloud.ExternalStorage, len(fullyResolvedIncrementalsDirectory)) + for i := range fullyResolvedIncrementalsDirectory { + store, err := mkStore(ctx, fullyResolvedIncrementalsDirectory[i], user) + if err != nil { + return nil, nil, nil, 0, errors.Wrapf(err, "failed to open backup storage location") + } + defer store.Close() + incStores[i] = store + } + + var prev []string + if len(incStores) > 0 { + prev, err = FindPriorBackups(ctx, incStores[0], includeManifest) + if err != nil { + return nil, nil, nil, 0, err + } + } + numLayers := len(prev) + 1 + + defaultURIs = make([]string, numLayers) + mainBackupManifests = make([]backuppb.BackupManifest, numLayers) + localityInfo = make([]jobspb.RestoreDetails_BackupLocalityInfo, numLayers) + + // Setup the full backup layer explicitly. + defaultURIs[0] = fullyResolvedBaseDirectory[0] + mainBackupManifests[0] = baseManifest + localityInfo[0], err = backupinfo.GetLocalityInfo( + ctx, baseStores, fullyResolvedBaseDirectory, baseManifest, encryption, "", /* prefix */ + ) + if err != nil { + return nil, nil, nil, 0, err + } + + // If we discovered additional layers, handle them too. + if numLayers > 1 { + numPartitions := len(fullyResolvedIncrementalsDirectory) + // We need the parsed base URI (/) for each partition to calculate the + // URI to each layer in that partition below. + baseURIs := make([]*url.URL, numPartitions) + for i := range fullyResolvedIncrementalsDirectory { + baseURIs[i], err = url.Parse(fullyResolvedIncrementalsDirectory[i]) + if err != nil { + return nil, nil, nil, 0, err + } + } + + // For each layer, we need to load the default manifest then calculate the URI and the + // locality info for each partition. + for i := range prev { + defaultManifestForLayer, memSize, err := backupinfo.ReadBackupManifest(ctx, mem, incStores[0], prev[i], encryption) + if err != nil { + return nil, nil, nil, 0, err + } + ownedMemSize += memSize + mainBackupManifests[i+1] = defaultManifestForLayer + + // prev[i] is the path to the manifest file itself for layer i -- the + // dirname piece of that path is the subdirectory in each of the + // partitions in which we'll also expect to find a partition manifest. + // Recall full inc URI is // + incSubDir := path.Dir(prev[i]) + partitionURIs := make([]string, numPartitions) + for j := range baseURIs { + u := *baseURIs[j] // NB: makes a copy to avoid mutating the baseURI. + u.Path = backuputils.JoinURLPath(u.Path, incSubDir) + partitionURIs[j] = u.String() + } + defaultURIs[i+1] = partitionURIs[0] + localityInfo[i+1], err = backupinfo.GetLocalityInfo(ctx, incStores, partitionURIs, defaultManifestForLayer, encryption, incSubDir) + if err != nil { + return nil, nil, nil, 0, err + } + } + } + + totalMemSize := ownedMemSize + ownedMemSize = 0 + + validatedDefaultURIs, validatedMainBackupManifests, validatedLocalityInfo, err := backupinfo.ValidateEndTimeAndTruncate( + defaultURIs, mainBackupManifests, localityInfo, endTime) + + if err != nil { + return nil, nil, nil, 0, err + } + return validatedDefaultURIs, validatedMainBackupManifests, validatedLocalityInfo, totalMemSize, nil +} + +// DeprecatedResolveBackupManifestsExplicitIncrementals reads the +// BACKUP_MANIFEST files from the incremental backup locations that have been +// explicitly provided by the user in `from`. The method uses the manifest file +// to return the defaultURI, backup manifest, and locality info for each +// incremental layer. +func DeprecatedResolveBackupManifestsExplicitIncrementals( + ctx context.Context, + mem *mon.BoundAccount, + mkStore cloud.ExternalStorageFromURIFactory, + from [][]string, + endTime hlc.Timestamp, + encryption *jobspb.BackupEncryptionOptions, + user username.SQLUsername, +) ( + defaultURIs []string, + // mainBackupManifests contains the manifest located at each defaultURI in the backup chain. + mainBackupManifests []backuppb.BackupManifest, + localityInfo []jobspb.RestoreDetails_BackupLocalityInfo, + reservedMemSize int64, + _ error, +) { + // If explicit incremental backups were are passed, we simply load them one + // by one as specified and return the results. + var ownedMemSize int64 + defer func() { + if ownedMemSize != 0 { + mem.Shrink(ctx, ownedMemSize) + } + }() + + defaultURIs = make([]string, len(from)) + localityInfo = make([]jobspb.RestoreDetails_BackupLocalityInfo, len(from)) + mainBackupManifests = make([]backuppb.BackupManifest, len(from)) + + var err error + for i, uris := range from { + // The first URI in the list must contain the main BACKUP manifest. + defaultURIs[i] = uris[0] + + stores := make([]cloud.ExternalStorage, len(uris)) + for j := range uris { + stores[j], err = mkStore(ctx, uris[j], user) + if err != nil { + return nil, nil, nil, 0, errors.Wrapf(err, "export configuration") + } + defer stores[j].Close() + } + + var memSize int64 + mainBackupManifests[i], memSize, err = backupinfo.ReadBackupManifestFromStore(ctx, mem, stores[0], encryption) + if err != nil { + return nil, nil, nil, 0, err + } + ownedMemSize += memSize + + if len(uris) > 1 { + localityInfo[i], err = backupinfo.GetLocalityInfo( + ctx, stores, uris, mainBackupManifests[i], encryption, "", /* prefix */ + ) + if err != nil { + return nil, nil, nil, 0, err + } + } + } + + totalMemSize := ownedMemSize + ownedMemSize = 0 + + validatedDefaultURIs, validatedMainBackupManifests, validatedLocalityInfo, err := + backupinfo.ValidateEndTimeAndTruncate(defaultURIs, mainBackupManifests, localityInfo, endTime) + + if err != nil { + return nil, nil, nil, 0, err + } + return validatedDefaultURIs, validatedMainBackupManifests, validatedLocalityInfo, totalMemSize, nil +} diff --git a/pkg/ccl/backupccl/backupdest/incrementals_test.go b/pkg/ccl/backupccl/backupdest/incrementals_test.go index 775c75eec6b3..6c9a1f00987d 100644 --- a/pkg/ccl/backupccl/backupdest/incrementals_test.go +++ b/pkg/ccl/backupccl/backupdest/incrementals_test.go @@ -6,7 +6,7 @@ // // https://github.com/cockroachdb/cockroach/blob/master/licenses/CCL.txt -package backupdest +package backupdest_test import ( "testing" diff --git a/pkg/ccl/backupccl/backupencryption/encryption.go b/pkg/ccl/backupccl/backupencryption/encryption.go index 86d2973c23ef..d08e2c434293 100644 --- a/pkg/ccl/backupccl/backupencryption/encryption.go +++ b/pkg/ccl/backupccl/backupencryption/encryption.go @@ -32,6 +32,13 @@ const ( // backupEncryptionInfoFile is the file name used to store the serialized // EncryptionInfo proto while the backup is in progress. backupEncryptionInfoFile = "ENCRYPTION-INFO" + + // BackupOptEncKMS is the option name in a BACKUP statement to specify a KMS + // URI for encryption. + BackupOptEncKMS = "kms" + // BackupOptEncPassphrase is the option name in a BACKUP statement to specify + // a passphrase for encryption. + BackupOptEncPassphrase = "encryption_passphrase" ) // ErrEncryptionInfoRead is a special error returned when the ENCRYPTION-INFO diff --git a/pkg/ccl/backupccl/backupinfo/BUILD.bazel b/pkg/ccl/backupccl/backupinfo/BUILD.bazel new file mode 100644 index 000000000000..ddde5478b66d --- /dev/null +++ b/pkg/ccl/backupccl/backupinfo/BUILD.bazel @@ -0,0 +1,87 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") +load("//pkg/testutils/buildutil:buildutil.bzl", "disallowed_imports_test") + +go_library( + name = "backupinfo", + srcs = [ + "backup_metadata.go", + "manifest_handling.go", + ], + importpath = "github.com/cockroachdb/cockroach/pkg/ccl/backupccl/backupinfo", + visibility = ["//visibility:public"], + deps = [ + "//pkg/ccl/backupccl/backupbase", + "//pkg/ccl/backupccl/backupencryption", + "//pkg/ccl/backupccl/backuppb", + "//pkg/ccl/backupccl/backuputils", + "//pkg/ccl/storageccl", + "//pkg/cloud", + "//pkg/clusterversion", + "//pkg/jobs/jobspb", + "//pkg/keys", + "//pkg/roachpb", + "//pkg/security/username", + "//pkg/settings", + "//pkg/settings/cluster", + "//pkg/sql", + "//pkg/sql/catalog", + "//pkg/sql/catalog/descbuilder", + "//pkg/sql/catalog/descpb", + "//pkg/sql/pgwire/pgcode", + "//pkg/sql/pgwire/pgerror", + "//pkg/sql/protoreflect", + "//pkg/sql/sem/tree", + "//pkg/sql/stats", + "//pkg/storage", + "//pkg/util/ctxgroup", + "//pkg/util/encoding", + "//pkg/util/hlc", + "//pkg/util/ioctx", + "//pkg/util/json", + "//pkg/util/mon", + "//pkg/util/protoutil", + "//pkg/util/syncutil", + "//pkg/util/timeutil", + "@com_github_cockroachdb_errors//:errors", + ], +) + +go_test( + name = "backupinfo_test", + srcs = [ + "backup_metadata_test.go", + "main_test.go", + ], + embed = [":backupinfo"], + deps = [ + "//pkg/base", + "//pkg/blobs", + "//pkg/ccl/backupccl/backupbase", + "//pkg/ccl/backupccl/backuppb", + "//pkg/ccl/backupccl/backuputils", + "//pkg/ccl/utilccl", + "//pkg/cloud", + "//pkg/roachpb", + "//pkg/security/securityassets", + "//pkg/security/securitytest", + "//pkg/security/username", + "//pkg/server", + "//pkg/sql", + "//pkg/sql/catalog/descpb", + "//pkg/sql/stats", + "//pkg/testutils/serverutils", + "//pkg/testutils/testcluster", + "//pkg/util/hlc", + "//pkg/util/ioctx", + "//pkg/util/leaktest", + "//pkg/util/log", + "//pkg/util/protoutil", + "//pkg/util/randutil", + "@com_github_stretchr_testify//require", + ], +) + +disallowed_imports_test( + "backupinfo", + ["//pkg/ccl/backupccl/backupdest"], +) diff --git a/pkg/ccl/backupccl/backup_metadata.go b/pkg/ccl/backupccl/backupinfo/backup_metadata.go similarity index 98% rename from pkg/ccl/backupccl/backup_metadata.go rename to pkg/ccl/backupccl/backupinfo/backup_metadata.go index b03ab6e96c83..ac074f8e023c 100644 --- a/pkg/ccl/backupccl/backup_metadata.go +++ b/pkg/ccl/backupccl/backupinfo/backup_metadata.go @@ -6,7 +6,7 @@ // // https://github.com/cockroachdb/cockroach/blob/master/licenses/CCL.txt -package backupccl +package backupinfo import ( "bytes" @@ -35,8 +35,11 @@ import ( ) const ( - metadataSSTName = "metadata.sst" - fileInfoPath = "fileinfo.sst" + // MetadataSSTName is the name of the SST file containing the backup metadata. + MetadataSSTName = "metadata.sst" + // FileInfoPath is the name of the SST file containing the + // BackupManifest_Files of the backup. + FileInfoPath = "fileinfo.sst" sstBackupKey = "backup" sstDescsPrefix = "desc/" sstFilesPrefix = "file/" @@ -46,7 +49,10 @@ const ( sstTenantsPrefix = "tenant/" ) -func writeBackupMetadataSST( +// WriteBackupMetadataSST is responsible for constructing and writing the +// `metadata.sst` to dest. This file contains the metadata corresponding to this +// backup. +func WriteBackupMetadataSST( ctx context.Context, dest cloud.ExternalStorage, enc *jobspb.BackupEncryptionOptions, @@ -62,7 +68,7 @@ func writeBackupMetadataSST( } }() - w, err := makeWriter(ctx, dest, metadataSSTName, enc) + w, err := makeWriter(ctx, dest, MetadataSSTName, enc) if err != nil { return err } @@ -125,7 +131,7 @@ func constructMetadataSST( return err } - if err := writeFilesToMetadata(ctx, sst, m, dest, enc, fileInfoPath); err != nil { + if err := writeFilesToMetadata(ctx, sst, m, dest, enc, FileInfoPath); err != nil { return err } @@ -783,7 +789,8 @@ type BackupMetadata struct { filename string } -func newBackupMetadata( +// NewBackupMetadata returns a new BackupMetadata instance. +func NewBackupMetadata( ctx context.Context, exportStore cloud.ExternalStorage, sstFileName string, diff --git a/pkg/ccl/backupccl/backup_metadata_test.go b/pkg/ccl/backupccl/backupinfo/backup_metadata_test.go similarity index 87% rename from pkg/ccl/backupccl/backup_metadata_test.go rename to pkg/ccl/backupccl/backupinfo/backup_metadata_test.go index b86715328244..c3d86ee7b2f1 100644 --- a/pkg/ccl/backupccl/backup_metadata_test.go +++ b/pkg/ccl/backupccl/backupinfo/backup_metadata_test.go @@ -6,7 +6,7 @@ // // https://github.com/cockroachdb/cockroach/blob/master/licenses/CCL.txt -package backupccl +package backupinfo_test import ( "context" @@ -16,7 +16,9 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/blobs" "github.com/cockroachdb/cockroach/pkg/ccl/backupccl/backupbase" + "github.com/cockroachdb/cockroach/pkg/ccl/backupccl/backupinfo" "github.com/cockroachdb/cockroach/pkg/ccl/backupccl/backuppb" + "github.com/cockroachdb/cockroach/pkg/ccl/backupccl/backuputils" "github.com/cockroachdb/cockroach/pkg/cloud" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/security/username" @@ -39,7 +41,8 @@ func TestMetadataSST(t *testing.T) { ctx := context.Background() const numAccounts = 1 userfile := "userfile:///0" - tc, sqlDB, _, cleanupFn := backupRestoreTestSetup(t, singleNode, numAccounts, InitManualReplication) + tc, sqlDB, _, cleanupFn := backuputils.BackupRestoreTestSetup(t, backuputils.SingleNode, numAccounts, + backuputils.InitManualReplication) defer cleanupFn() // Check that backup metadata is correct on full cluster backup. @@ -86,7 +89,7 @@ func checkMetadata( t.Fatal(err) } - bm, err := newBackupMetadata(ctx, store, metadataSSTName, nil) + bm, err := backupinfo.NewBackupMetadata(ctx, store, backupinfo.MetadataSSTName, nil) if err != nil { t.Fatal(err) } @@ -110,7 +113,7 @@ func checkMetadata( checkStats(ctx, t, store, m, bm) } -func checkManifest(t *testing.T, m *backuppb.BackupManifest, bm *BackupMetadata) { +func checkManifest(t *testing.T, m *backuppb.BackupManifest, bm *backupinfo.BackupMetadata) { expectedManifest := *m expectedManifest.Descriptors = nil expectedManifest.DescriptorChanges = nil @@ -124,7 +127,7 @@ func checkManifest(t *testing.T, m *backuppb.BackupManifest, bm *BackupMetadata) } func checkDescriptors( - ctx context.Context, t *testing.T, m *backuppb.BackupManifest, bm *BackupMetadata, + ctx context.Context, t *testing.T, m *backuppb.BackupManifest, bm *backupinfo.BackupMetadata, ) { var metaDescs []descpb.Descriptor var desc descpb.Descriptor @@ -143,7 +146,7 @@ func checkDescriptors( } func checkDescriptorChanges( - ctx context.Context, t *testing.T, m *backuppb.BackupManifest, bm *BackupMetadata, + ctx context.Context, t *testing.T, m *backuppb.BackupManifest, bm *backupinfo.BackupMetadata, ) { var metaRevs []backuppb.BackupManifest_DescriptorRevision var rev backuppb.BackupManifest_DescriptorRevision @@ -165,7 +168,9 @@ func checkDescriptorChanges( require.Equal(t, m.DescriptorChanges, metaRevs) } -func checkFiles(ctx context.Context, t *testing.T, m *backuppb.BackupManifest, bm *BackupMetadata) { +func checkFiles( + ctx context.Context, t *testing.T, m *backuppb.BackupManifest, bm *backupinfo.BackupMetadata, +) { var metaFiles []backuppb.BackupManifest_File var file backuppb.BackupManifest_File it := bm.FileIter(ctx) @@ -181,7 +186,9 @@ func checkFiles(ctx context.Context, t *testing.T, m *backuppb.BackupManifest, b require.Equal(t, m.Files, metaFiles) } -func checkSpans(ctx context.Context, t *testing.T, m *backuppb.BackupManifest, bm *BackupMetadata) { +func checkSpans( + ctx context.Context, t *testing.T, m *backuppb.BackupManifest, bm *backupinfo.BackupMetadata, +) { var metaSpans []roachpb.Span var span roachpb.Span it := bm.SpanIter(ctx) @@ -198,7 +205,7 @@ func checkSpans(ctx context.Context, t *testing.T, m *backuppb.BackupManifest, b } func checkIntroducedSpans( - ctx context.Context, t *testing.T, m *backuppb.BackupManifest, bm *BackupMetadata, + ctx context.Context, t *testing.T, m *backuppb.BackupManifest, bm *backupinfo.BackupMetadata, ) { var metaSpans []roachpb.Span var span roachpb.Span @@ -215,7 +222,7 @@ func checkIntroducedSpans( } func checkTenants( - ctx context.Context, t *testing.T, m *backuppb.BackupManifest, bm *BackupMetadata, + ctx context.Context, t *testing.T, m *backuppb.BackupManifest, bm *backupinfo.BackupMetadata, ) { var metaTenants []descpb.TenantInfoWithUsage var tenant descpb.TenantInfoWithUsage @@ -237,9 +244,9 @@ func checkStats( t *testing.T, store cloud.ExternalStorage, m *backuppb.BackupManifest, - bm *BackupMetadata, + bm *backupinfo.BackupMetadata, ) { - expectedStats, err := getStatisticsFromBackup(ctx, store, nil, *m) + expectedStats, err := backupinfo.GetStatisticsFromBackup(ctx, store, nil, *m) if err != nil { t.Fatal(err) } @@ -271,8 +278,8 @@ func testingReadBackupManifest( if err != nil { return nil, err } - if isGZipped(bytes) { - descBytes, err := decompressData(ctx, nil, bytes) + if backupinfo.IsGZipped(bytes) { + descBytes, err := backupinfo.DecompressData(ctx, nil, bytes) if err != nil { return nil, err } diff --git a/pkg/ccl/backupccl/backupinfo/main_test.go b/pkg/ccl/backupccl/backupinfo/main_test.go new file mode 100644 index 000000000000..2ed06175c9c9 --- /dev/null +++ b/pkg/ccl/backupccl/backupinfo/main_test.go @@ -0,0 +1,33 @@ +// Copyright 2022 The Cockroach Authors. +// +// Licensed as a CockroachDB Enterprise file under the Cockroach Community +// License (the "License"); you may not use this file except in compliance with +// the License. You may obtain a copy of the License at +// +// https://github.com/cockroachdb/cockroach/blob/master/licenses/CCL.txt + +package backupinfo + +import ( + "os" + "testing" + + "github.com/cockroachdb/cockroach/pkg/ccl/utilccl" + "github.com/cockroachdb/cockroach/pkg/security/securityassets" + "github.com/cockroachdb/cockroach/pkg/security/securitytest" + "github.com/cockroachdb/cockroach/pkg/server" + "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" + "github.com/cockroachdb/cockroach/pkg/testutils/testcluster" + "github.com/cockroachdb/cockroach/pkg/util/randutil" +) + +func TestMain(m *testing.M) { + defer utilccl.TestingEnableEnterprise()() + securityassets.SetLoader(securitytest.EmbeddedAssets) + randutil.SeedForTests() + serverutils.InitTestServerFactory(server.TestServerFactory) + serverutils.InitTestClusterFactory(testcluster.TestClusterFactory) + os.Exit(m.Run()) +} + +//go:generate ../../../util/leaktest/add-leaktest.sh *_test.go diff --git a/pkg/ccl/backupccl/manifest_handling.go b/pkg/ccl/backupccl/backupinfo/manifest_handling.go similarity index 75% rename from pkg/ccl/backupccl/manifest_handling.go rename to pkg/ccl/backupccl/backupinfo/manifest_handling.go index 8847b917104d..0562bac3ea3a 100644 --- a/pkg/ccl/backupccl/manifest_handling.go +++ b/pkg/ccl/backupccl/backupinfo/manifest_handling.go @@ -6,7 +6,7 @@ // // https://github.com/cockroachdb/cockroach/blob/master/licenses/CCL.txt -package backupccl +package backupinfo import ( "bytes" @@ -15,29 +15,30 @@ import ( "crypto/sha256" "encoding/hex" "fmt" - "net/url" "path" "sort" "strings" "github.com/cockroachdb/cockroach/pkg/ccl/backupccl/backupbase" - "github.com/cockroachdb/cockroach/pkg/ccl/backupccl/backupdest" "github.com/cockroachdb/cockroach/pkg/ccl/backupccl/backupencryption" "github.com/cockroachdb/cockroach/pkg/ccl/backupccl/backuppb" "github.com/cockroachdb/cockroach/pkg/ccl/backupccl/backuputils" "github.com/cockroachdb/cockroach/pkg/ccl/storageccl" "github.com/cockroachdb/cockroach/pkg/cloud" + "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/security/username" "github.com/cockroachdb/cockroach/pkg/settings" "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/sql" "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descbuilder" descpb "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/sql/stats" "github.com/cockroachdb/cockroach/pkg/util/ctxgroup" "github.com/cockroachdb/cockroach/pkg/util/encoding" "github.com/cockroachdb/cockroach/pkg/util/hlc" @@ -51,40 +52,38 @@ import ( // Files that may appear in a backup directory. const ( - // backupManifestChecksumSuffix indicates where the checksum for the manifest + // BackupManifestChecksumSuffix indicates where the checksum for the manifest // is stored if present. It can be found in the name of the backup manifest + // this suffix. - backupManifestChecksumSuffix = "-CHECKSUM" + BackupManifestChecksumSuffix = "-CHECKSUM" - // backupPartitionDescriptorPrefix is the file name prefix for serialized - // BackupPartitionDescriptor protos. - backupPartitionDescriptorPrefix = "BACKUP_PART" - // backupManifestCheckpointName is the file name used to store the serialized + // BackupManifestCheckpointName is the file name used to store the serialized // BackupManifest proto while the backup is in progress. - backupManifestCheckpointName = "BACKUP-CHECKPOINT" - // backupStatisticsFileName is the file name used to store the serialized + BackupManifestCheckpointName = "BACKUP-CHECKPOINT" + + // BackupStatisticsFileName is the file name used to store the serialized // table statistics for the tables being backed up. - backupStatisticsFileName = "BACKUP-STATISTICS" -) + BackupStatisticsFileName = "BACKUP-STATISTICS" -const ( // BackupFormatDescriptorTrackingVersion added tracking of complete DBs. BackupFormatDescriptorTrackingVersion uint32 = 1 - // backupProgressDirectory is the directory where all 22.1 and beyond + // BackupProgressDirectory is the directory where all 22.1 and beyond // CHECKPOINT files will be stored as we no longer want to overwrite // them. - backupProgressDirectory = "progress" + BackupProgressDirectory = "progress" ) -var writeMetadataSST = settings.RegisterBoolSetting( +// WriteMetadataSST controls if we write the experimental new format BACKUP +// metadata file. +var WriteMetadataSST = settings.RegisterBoolSetting( settings.TenantWritable, "kv.bulkio.write_metadata_sst.enabled", "write experimental new format BACKUP metadata file", true, ) -// isGZipped detects whether the given bytes represent GZipped data. This check +// 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 // gzipped data incorrectly identified as "application/vnd.ms-fontobject". The @@ -93,7 +92,7 @@ var writeMetadataSST = settings.RegisterBoolSetting( // // This method is only used to detect if protobufs are GZipped, and there are no // conflicts between the starting bytes of a protobuf and these magic bytes. -func isGZipped(dat []byte) bool { +func IsGZipped(dat []byte) bool { gzipPrefix := []byte("\x1F\x8B\x08") return bytes.HasPrefix(dat, gzipPrefix) } @@ -138,10 +137,10 @@ func ReadBackupManifestFromStore( exportStore cloud.ExternalStorage, encryption *jobspb.BackupEncryptionOptions, ) (backuppb.BackupManifest, int64, error) { - backupManifest, memSize, err := readBackupManifest(ctx, mem, exportStore, backupbase.BackupManifestName, + backupManifest, memSize, err := ReadBackupManifest(ctx, mem, exportStore, backupbase.BackupManifestName, encryption) if err != nil { - oldManifest, newMemSize, newErr := readBackupManifest(ctx, mem, exportStore, backupbase.BackupOldManifestName, + oldManifest, newMemSize, newErr := ReadBackupManifest(ctx, mem, exportStore, backupbase.BackupOldManifestName, encryption) if newErr != nil { return backuppb.BackupManifest{}, 0, err @@ -169,9 +168,8 @@ func compressData(descBuf []byte) ([]byte, error) { return gzipBuf.Bytes(), nil } -// decompressData decompresses gzip data buffer and -// returns decompressed bytes. -func decompressData(ctx context.Context, mem *mon.BoundAccount, descBytes []byte) ([]byte, error) { +// DecompressData decompresses gzip data buffer and returns decompressed bytes. +func DecompressData(ctx context.Context, mem *mon.BoundAccount, descBytes []byte) ([]byte, error) { r, err := gzip.NewReader(bytes.NewBuffer(descBytes)) if err != nil { return nil, err @@ -180,9 +178,9 @@ func decompressData(ctx context.Context, mem *mon.BoundAccount, descBytes []byte return mon.ReadAll(ctx, ioctx.ReaderAdapter(r), mem) } -// readBackupCheckpointManifest reads and unmarshals a BACKUP-CHECKPOINT +// ReadBackupCheckpointManifest reads and unmarshals a BACKUP-CHECKPOINT // manifest from filename in the provided export store. -func readBackupCheckpointManifest( +func ReadBackupCheckpointManifest( ctx context.Context, mem *mon.BoundAccount, exportStore cloud.ExternalStorage, @@ -197,7 +195,7 @@ func readBackupCheckpointManifest( // Look for a checksum, if one is not found it could be an older backup, // but we want to continue anyway. - checksumFile, err := readLatestCheckpointFile(ctx, exportStore, filename+backupManifestChecksumSuffix) + checksumFile, err := readLatestCheckpointFile(ctx, exportStore, filename+BackupManifestChecksumSuffix) if err != nil { if !errors.Is(err, cloud.ErrFileDoesNotExist) { return backuppb.BackupManifest{}, 0, err @@ -209,9 +207,9 @@ func readBackupCheckpointManifest( return readManifest(ctx, mem, exportStore, encryption, checkpointFile, checksumFile) } -// readBackupManifest reads and unmarshals a BackupManifest from filename in the +// ReadBackupManifest reads and unmarshals a BackupManifest from filename in the // provided export store. -func readBackupManifest( +func ReadBackupManifest( ctx context.Context, mem *mon.BoundAccount, exportStore cloud.ExternalStorage, @@ -226,7 +224,7 @@ func readBackupManifest( // Look for a checksum, if one is not found it could be an older backup, // but we want to continue anyway. - checksumFile, err := exportStore.ReadFile(ctx, filename+backupManifestChecksumSuffix) + checksumFile, err := exportStore.ReadFile(ctx, filename+BackupManifestChecksumSuffix) if err != nil { if !errors.Is(err, cloud.ErrFileDoesNotExist) { return backuppb.BackupManifest{}, 0, err @@ -265,7 +263,7 @@ func readManifest( if err != nil { return backuppb.BackupManifest{}, 0, errors.Wrap(err, "reading checksum file") } - checksum, err := getChecksum(descBytes) + checksum, err := GetChecksum(descBytes) if err != nil { return backuppb.BackupManifest{}, 0, errors.Wrap(err, "calculating checksum of manifest") } @@ -290,8 +288,8 @@ func readManifest( descBytes = plaintextBytes } - if isGZipped(descBytes) { - decompressedBytes, err := decompressData(ctx, mem, descBytes) + if IsGZipped(descBytes) { + decompressedBytes, err := DecompressData(ctx, mem, descBytes) if err != nil { return backuppb.BackupManifest{}, 0, errors.Wrap( err, "decompressing backup manifest") @@ -314,7 +312,7 @@ func readManifest( if encryption == nil && storageccl.AppearsEncrypted(descBytes) { return backuppb.BackupManifest{}, 0, errors.Wrapf( err, "file appears encrypted -- try specifying one of \"%s\" or \"%s\"", - backupOptEncPassphrase, backupOptEncKMS) + backupencryption.BackupOptEncPassphrase, backupencryption.BackupOptEncKMS) } return backuppb.BackupManifest{}, 0, err } @@ -374,8 +372,8 @@ func readBackupPartitionDescriptor( descBytes = plaintextData } - if isGZipped(descBytes) { - decompressedData, err := decompressData(ctx, mem, descBytes) + if IsGZipped(descBytes) { + decompressedData, err := DecompressData(ctx, mem, descBytes) if err != nil { return backuppb.BackupPartitionDescriptor{}, 0, errors.Wrap( err, "decompressing backup partition descriptor") @@ -434,7 +432,39 @@ func readTableStatistics( return &tableStats, err } -func writeBackupManifest( +// GetStatisticsFromBackup retrieves Statistics from backup manifest, +// either through the Statistics field or from the files. +func GetStatisticsFromBackup( + ctx context.Context, + exportStore cloud.ExternalStorage, + encryption *jobspb.BackupEncryptionOptions, + backup backuppb.BackupManifest, +) ([]*stats.TableStatisticProto, error) { + // This part deals with pre-20.2 stats format where backup statistics + // are stored as a field in backup manifests instead of in their + // individual files. + if backup.DeprecatedStatistics != nil { + return backup.DeprecatedStatistics, nil + } + tableStatistics := make([]*stats.TableStatisticProto, 0, len(backup.StatisticsFilenames)) + uniqueFileNames := make(map[string]struct{}) + for _, fname := range backup.StatisticsFilenames { + if _, exists := uniqueFileNames[fname]; !exists { + uniqueFileNames[fname] = struct{}{} + myStatsTable, err := readTableStatistics(ctx, exportStore, fname, encryption) + if err != nil { + return tableStatistics, err + } + tableStatistics = append(tableStatistics, myStatsTable.Statistics...) + } + } + + return tableStatistics, nil +} + +// WriteBackupManifest compresses and writes the passed in BackupManifest `desc` +// to `exportStore`. +func WriteBackupManifest( ctx context.Context, settings *cluster.Settings, exportStore cloud.ExternalStorage, @@ -470,20 +500,20 @@ func writeBackupManifest( } // Write the checksum file after we've successfully wrote the manifest. - checksum, err := getChecksum(descBuf) + checksum, err := GetChecksum(descBuf) if err != nil { return errors.Wrap(err, "calculating checksum") } - if err := cloud.WriteFile(ctx, exportStore, filename+backupManifestChecksumSuffix, bytes.NewReader(checksum)); err != nil { + if err := cloud.WriteFile(ctx, exportStore, filename+BackupManifestChecksumSuffix, bytes.NewReader(checksum)); err != nil { return errors.Wrap(err, "writing manifest checksum") } return nil } -// getChecksum returns a 32 bit keyed-checksum for the given data. -func getChecksum(data []byte) ([]byte, error) { +// GetChecksum returns a 32 bit keyed-checksum for the given data. +func GetChecksum(data []byte) ([]byte, error) { const checksumSizeBytes = 4 hash := sha256.New() if _, err := hash.Write(data); err != nil { @@ -493,10 +523,10 @@ func getChecksum(data []byte) ([]byte, error) { return hash.Sum(nil)[:checksumSizeBytes], nil } -// writeBackupPartitionDescriptor writes metadata (containing a locality KV and +// WriteBackupPartitionDescriptor writes metadata (containing a locality KV and // partial file listing) for a partitioned BACKUP to one of the stores in the // backup. -func writeBackupPartitionDescriptor( +func WriteBackupPartitionDescriptor( ctx context.Context, exportStore cloud.ExternalStorage, filename string, @@ -526,13 +556,12 @@ func writeBackupPartitionDescriptor( return cloud.WriteFile(ctx, exportStore, filename, bytes.NewReader(descBuf)) } -// writeTableStatistics writes a StatsTable object to a file of the filename +// WriteTableStatistics writes a StatsTable object to a file of the filename // to the specified exportStore. It will be encrypted according to the encryption // option given. -func writeTableStatistics( +func WriteTableStatistics( ctx context.Context, exportStore cloud.ExternalStorage, - filename string, encryption *jobspb.BackupEncryptionOptions, stats *backuppb.StatsTable, ) error { @@ -551,10 +580,15 @@ func writeTableStatistics( return err } } - return cloud.WriteFile(ctx, exportStore, filename, bytes.NewReader(statsBuf)) + return cloud.WriteFile(ctx, exportStore, BackupStatisticsFileName, bytes.NewReader(statsBuf)) } -func loadBackupManifests( +// LoadBackupManifests reads and returns the BackupManifests at the +// ExternalStorage locations in `uris`. +// +// The caller is responsible for shrinking `mem` by the returned size once they +// are done with the returned manifests. +func LoadBackupManifests( ctx context.Context, mem *mon.BoundAccount, uris []string, @@ -587,12 +621,14 @@ func loadBackupManifests( return backupManifests, memSize, nil } -var errLocalityDescriptor = errors.New(`Locality Descriptor not found`) +// ErrLocalityDescriptor is the sentinel error that is thrown when a locality +// descriptor is not found. +var ErrLocalityDescriptor = errors.New(`Locality Descriptor not found`) -// getLocalityInfo takes a list of stores and their URIs, along with the main +// GetLocalityInfo takes a list of stores and their URIs, along with the main // backup manifest searches each for the locality pieces listed in the the // main manifest, returning the mapping. -func getLocalityInfo( +func GetLocalityInfo( ctx context.Context, stores []cloud.ExternalStorage, uris []string, @@ -640,217 +676,24 @@ func getLocalityInfo( } } if !found { - return info, errors.Mark(errors.Newf("expected manifest %s not found in backup locations", filename), errLocalityDescriptor) + return info, errors.Mark(errors.Newf("expected manifest %s not found in backup locations", filename), ErrLocalityDescriptor) } } info.URIsByOriginalLocalityKV = urisByOrigLocality return info, nil } -func resolveBackupManifestsExplicitIncrementals( - ctx context.Context, - mem *mon.BoundAccount, - mkStore cloud.ExternalStorageFromURIFactory, - from [][]string, - endTime hlc.Timestamp, - encryption *jobspb.BackupEncryptionOptions, - user username.SQLUsername, -) ( - defaultURIs []string, - // mainBackupManifests contains the manifest located at each defaultURI in the backup chain. - mainBackupManifests []backuppb.BackupManifest, - localityInfo []jobspb.RestoreDetails_BackupLocalityInfo, - reservedMemSize int64, - _ error, -) { - // If explicit incremental backups were are passed, we simply load them one - // by one as specified and return the results. - var ownedMemSize int64 - defer func() { - if ownedMemSize != 0 { - mem.Shrink(ctx, ownedMemSize) - } - }() - - defaultURIs = make([]string, len(from)) - localityInfo = make([]jobspb.RestoreDetails_BackupLocalityInfo, len(from)) - mainBackupManifests = make([]backuppb.BackupManifest, len(from)) - - var err error - for i, uris := range from { - // The first URI in the list must contain the main BACKUP manifest. - defaultURIs[i] = uris[0] - - stores := make([]cloud.ExternalStorage, len(uris)) - for j := range uris { - stores[j], err = mkStore(ctx, uris[j], user) - if err != nil { - return nil, nil, nil, 0, errors.Wrapf(err, "export configuration") - } - defer stores[j].Close() - } - - var memSize int64 - mainBackupManifests[i], memSize, err = ReadBackupManifestFromStore(ctx, mem, stores[0], encryption) - if err != nil { - return nil, nil, nil, 0, err - } - ownedMemSize += memSize - - if len(uris) > 1 { - localityInfo[i], err = getLocalityInfo( - ctx, stores, uris, mainBackupManifests[i], encryption, "", /* prefix */ - ) - if err != nil { - return nil, nil, nil, 0, err - } - } - } - - totalMemSize := ownedMemSize - ownedMemSize = 0 - - validatedDefaultURIs, validatedMainBackupManifests, validatedLocalityInfo, err := validateEndTimeAndTruncate( - defaultURIs, mainBackupManifests, localityInfo, endTime) - - if err != nil { - return nil, nil, nil, 0, err - } - return validatedDefaultURIs, validatedMainBackupManifests, validatedLocalityInfo, totalMemSize, nil -} - -// resolveBackupManifests resolves the URIs that point to the incremental layers -// (each of which can be partitioned) of backups into the actual backup -// manifests and metadata required to RESTORE. If only one layer is explicitly -// provided, it is inspected to see if it contains "appended" layers internally -// that are then expanded into the result layers returned, similar to if those -// layers had been specified in `from` explicitly. -func resolveBackupManifests( - ctx context.Context, - mem *mon.BoundAccount, - baseStores []cloud.ExternalStorage, - mkStore cloud.ExternalStorageFromURIFactory, - fullyResolvedBaseDirectory []string, - fullyResolvedIncrementalsDirectory []string, - endTime hlc.Timestamp, - encryption *jobspb.BackupEncryptionOptions, - user username.SQLUsername, -) ( - defaultURIs []string, - // mainBackupManifests contains the manifest located at each defaultURI in the backup chain. - mainBackupManifests []backuppb.BackupManifest, - localityInfo []jobspb.RestoreDetails_BackupLocalityInfo, - reservedMemSize int64, - _ error, -) { - var ownedMemSize int64 - defer func() { - if ownedMemSize != 0 { - mem.Shrink(ctx, ownedMemSize) - } - }() - baseManifest, memSize, err := ReadBackupManifestFromStore(ctx, mem, baseStores[0], encryption) - if err != nil { - return nil, nil, nil, 0, err - } - ownedMemSize += memSize - - incStores := make([]cloud.ExternalStorage, len(fullyResolvedIncrementalsDirectory)) - for i := range fullyResolvedIncrementalsDirectory { - store, err := mkStore(ctx, fullyResolvedIncrementalsDirectory[i], user) - if err != nil { - return nil, nil, nil, 0, errors.Wrapf(err, "failed to open backup storage location") - } - defer store.Close() - incStores[i] = store - } - - var prev []string - if len(incStores) > 0 { - prev, err = backupdest.FindPriorBackups(ctx, incStores[0], backupbase.IncludeManifest) - if err != nil { - return nil, nil, nil, 0, err - } - } - numLayers := len(prev) + 1 - - defaultURIs = make([]string, numLayers) - mainBackupManifests = make([]backuppb.BackupManifest, numLayers) - localityInfo = make([]jobspb.RestoreDetails_BackupLocalityInfo, numLayers) - - // Setup the full backup layer explicitly. - defaultURIs[0] = fullyResolvedBaseDirectory[0] - mainBackupManifests[0] = baseManifest - localityInfo[0], err = getLocalityInfo( - ctx, baseStores, fullyResolvedBaseDirectory, baseManifest, encryption, "", /* prefix */ - ) - if err != nil { - return nil, nil, nil, 0, err - } - - // If we discovered additional layers, handle them too. - if numLayers > 1 { - numPartitions := len(fullyResolvedIncrementalsDirectory) - // We need the parsed base URI (/) for each partition to calculate the - // URI to each layer in that partition below. - baseURIs := make([]*url.URL, numPartitions) - for i := range fullyResolvedIncrementalsDirectory { - baseURIs[i], err = url.Parse(fullyResolvedIncrementalsDirectory[i]) - if err != nil { - return nil, nil, nil, 0, err - } - } - - // For each layer, we need to load the default manifest then calculate the URI and the - // locality info for each partition. - for i := range prev { - defaultManifestForLayer, memSize, err := readBackupManifest(ctx, mem, incStores[0], prev[i], encryption) - if err != nil { - return nil, nil, nil, 0, err - } - ownedMemSize += memSize - mainBackupManifests[i+1] = defaultManifestForLayer - - // prev[i] is the path to the manifest file itself for layer i -- the - // dirname piece of that path is the subdirectory in each of the - // partitions in which we'll also expect to find a partition manifest. - // Recall full inc URI is // - incSubDir := path.Dir(prev[i]) - partitionURIs := make([]string, numPartitions) - for j := range baseURIs { - u := *baseURIs[j] // NB: makes a copy to avoid mutating the baseURI. - u.Path = backuputils.JoinURLPath(u.Path, incSubDir) - partitionURIs[j] = u.String() - } - defaultURIs[i+1] = partitionURIs[0] - localityInfo[i+1], err = getLocalityInfo(ctx, incStores, partitionURIs, defaultManifestForLayer, encryption, incSubDir) - if err != nil { - return nil, nil, nil, 0, err - } - } - } - - totalMemSize := ownedMemSize - ownedMemSize = 0 - - validatedDefaultURIs, validatedMainBackupManifests, validatedLocalityInfo, err := validateEndTimeAndTruncate( - defaultURIs, mainBackupManifests, localityInfo, endTime) - - if err != nil { - return nil, nil, nil, 0, err - } - return validatedDefaultURIs, validatedMainBackupManifests, validatedLocalityInfo, totalMemSize, nil -} - -func validateEndTimeAndTruncate( +// ValidateEndTimeAndTruncate checks that the requested target time, if +// specified, is valid for the list of incremental backups resolved, truncating +// the results to the backup that contains the target time. +// The method also performs additional sanity checks to ensure the backups cover +// the requested time. +func ValidateEndTimeAndTruncate( defaultURIs []string, mainBackupManifests []backuppb.BackupManifest, localityInfo []jobspb.RestoreDetails_BackupLocalityInfo, endTime hlc.Timestamp, ) ([]string, []backuppb.BackupManifest, []jobspb.RestoreDetails_BackupLocalityInfo, error) { - // Check that the requested target time, if specified, is valid for the list - // of incremental backups resolved, truncating the results to the backup that - // contains the target time. if endTime.IsEmpty() { return defaultURIs, mainBackupManifests, localityInfo, nil } @@ -863,16 +706,15 @@ func validateEndTimeAndTruncate( // Ensure that the backup actually has revision history. if !endTime.Equal(b.EndTime) { if b.MVCCFilter != backuppb.MVCCFilter_All { - const errPrefix = "invalid RESTORE timestamp: restoring to arbitrary time requires that BACKUP for requested time be created with '%s' option." + const errPrefix = "invalid RESTORE timestamp: restoring to arbitrary time requires that BACKUP for requested time be created with 'revision_history' option." if i == 0 { return nil, nil, nil, errors.Errorf( - errPrefix+" nearest backup time is %s", backupOptRevisionHistory, + errPrefix+" nearest backup time is %s", timeutil.Unix(0, b.EndTime.WallTime).UTC(), ) } return nil, nil, nil, errors.Errorf( errPrefix+" nearest BACKUP times are %s or %s", - backupOptRevisionHistory, timeutil.Unix(0, mainBackupManifests[i-1].EndTime.WallTime).UTC(), timeutil.Unix(0, b.EndTime.WallTime).UTC(), ) @@ -898,9 +740,9 @@ func validateEndTimeAndTruncate( ) } -// TODO(anzoteh96): benchmark the performance of different search algorithms, -// e.g. linear search, binary search, reverse linear search. -func getBackupIndexAtTime( +// GetBackupIndexAtTime returns the index of the latest backup in +// `backupManifests` with a StartTime >= asOf. +func GetBackupIndexAtTime( backupManifests []backuppb.BackupManifest, asOf hlc.Timestamp, ) (int, error) { if len(backupManifests) == 0 { @@ -919,7 +761,9 @@ func getBackupIndexAtTime( return backupManifestIndex, nil } -func loadSQLDescsFromBackupsAtTime( +// LoadSQLDescsFromBackupsAtTime returns the Descriptors found in the last +// (latest) backup with a StartTime >= asOf. +func LoadSQLDescsFromBackupsAtTime( backupManifests []backuppb.BackupManifest, asOf hlc.Timestamp, ) ([]catalog.Descriptor, backuppb.BackupManifest) { lastBackupManifest := backupManifests[len(backupManifests)-1] @@ -989,9 +833,9 @@ func loadSQLDescsFromBackupsAtTime( return allDescs, lastBackupManifest } -// sanitizeLocalityKV returns a sanitized version of the input string where all +// SanitizeLocalityKV returns a sanitized version of the input string where all // characters that are not alphanumeric or -, =, or _ are replaced with _. -func sanitizeLocalityKV(kv string) string { +func SanitizeLocalityKV(kv string) string { sanitizedKV := make([]byte, len(kv)) for i := 0; i < len(kv); i++ { if (kv[i] >= 'a' && kv[i] <= 'z') || @@ -1005,11 +849,11 @@ func sanitizeLocalityKV(kv string) string { return string(sanitizedKV) } -// checkForPreviousBackup ensures that the target location does not already +// CheckForPreviousBackup ensures that the target location does not already // contain a BACKUP or checkpoint, locking out accidental concurrent operations // on that location. Note that the checkpoint file should be written as soon as // the job actually starts. -func checkForPreviousBackup( +func CheckForPreviousBackup( ctx context.Context, exportStore cloud.ExternalStorage, defaultURI string, ) error { redactedURI := backuputils.RedactURIForErrorMessage(defaultURI) @@ -1027,26 +871,137 @@ func checkForPreviousBackup( redactedURI, backupbase.BackupManifestName) } - r, err = readLatestCheckpointFile(ctx, exportStore, backupManifestCheckpointName) + r, err = readLatestCheckpointFile(ctx, exportStore, BackupManifestCheckpointName) if err == nil { r.Close(ctx) return pgerror.Newf(pgcode.FileAlreadyExists, "%s already contains a %s file (is another operation already in progress?)", - redactedURI, backupManifestCheckpointName) + redactedURI, BackupManifestCheckpointName) } if !errors.Is(err, cloud.ErrFileDoesNotExist) { return errors.Wrapf(err, "%s returned an unexpected error when checking for the existence of %s file", - redactedURI, backupManifestCheckpointName) + redactedURI, BackupManifestCheckpointName) } return nil } -// tempCheckpointFileNameForJob returns temporary filename for backup manifest checkpoint. -func tempCheckpointFileNameForJob(jobID jobspb.JobID) string { - return fmt.Sprintf("%s-%d", backupManifestCheckpointName, jobID) +// TempCheckpointFileNameForJob returns temporary filename for backup manifest checkpoint. +func TempCheckpointFileNameForJob(jobID jobspb.JobID) string { + return fmt.Sprintf("%s-%d", BackupManifestCheckpointName, jobID) +} + +// WriteBackupManifestCheckpoint writes a new BACKUP-CHECKPOINT MANIFEST and +// CHECKSUM file. If it is a pure v22.1 cluster or later, it will write a +// timestamped BACKUP-CHECKPOINT to the /progress directory. If it is a mixed +// cluster version, it will write a non timestamped BACKUP-CHECKPOINT to the +// base directory in order to not break backup jobs that resume on a v21.2 node. +func WriteBackupManifestCheckpoint( + ctx context.Context, + storageURI string, + encryption *jobspb.BackupEncryptionOptions, + desc *backuppb.BackupManifest, + execCfg *sql.ExecutorConfig, + user username.SQLUsername, +) error { + defaultStore, err := execCfg.DistSQLSrv.ExternalStorageFromURI(ctx, storageURI, user) + if err != nil { + return err + } + defer defaultStore.Close() + + sort.Sort(BackupFileDescriptors(desc.Files)) + + descBuf, err := protoutil.Marshal(desc) + if err != nil { + return err + } + + descBuf, err = compressData(descBuf) + if err != nil { + return errors.Wrap(err, "compressing backup manifest") + } + + if encryption != nil { + encryptionKey, err := backupencryption.GetEncryptionKey(ctx, encryption, execCfg.Settings, defaultStore.ExternalIOConf()) + if err != nil { + return err + } + descBuf, err = storageccl.EncryptFile(descBuf, encryptionKey) + if err != nil { + return err + } + } + + // If the cluster is still running on a mixed version, we want to write + // to the base directory instead of the progress directory. That way if + // an old node resumes a backup, it doesn't have to start over. + if !execCfg.Settings.Version.IsActive(ctx, clusterversion.BackupDoesNotOverwriteLatestAndCheckpoint) { + // We want to overwrite the latest checkpoint in the base directory, + // just write to the non versioned BACKUP-CHECKPOINT file. + err = cloud.WriteFile(ctx, defaultStore, BackupManifestCheckpointName, bytes.NewReader(descBuf)) + if err != nil { + return err + } + + checksum, err := GetChecksum(descBuf) + if err != nil { + return err + } + + return cloud.WriteFile(ctx, defaultStore, BackupManifestCheckpointName+ + BackupManifestChecksumSuffix, bytes.NewReader(checksum)) + } + + // We timestamp the checkpoint files in order to enforce write once backups. + // When the job goes to read these timestamped files, it will List + // the checkpoints and pick the file whose name is lexicographically + // sorted to the top. This will be the last checkpoint we write, for + // details refer to newTimestampedCheckpointFileName. + filename := NewTimestampedCheckpointFileName() + + // HTTP storage does not support listing and so we cannot rely on the + // above-mentioned List method to return us the latest checkpoint file. + // Instead, we will write a checkpoint once with a well-known filename, + // and teach the job to always reach for that filename in the face of + // a resume. We may lose progress, but this is a cost we are willing + // to pay to uphold write-once semantics. + if defaultStore.Conf().Provider == roachpb.ExternalStorageProvider_http { + // TODO (darryl): We should do this only for file not found or directory + // does not exist errors. As of right now we only specifically wrap + // ReadFile errors for file not found so this is not possible yet. + if r, err := defaultStore.ReadFile(ctx, BackupProgressDirectory+"/"+BackupManifestCheckpointName); err != nil { + // Since we did not find the checkpoint file this is the first time + // we are going to write a checkpoint, so write it with the well + // known filename. + filename = BackupManifestCheckpointName + } else { + err = r.Close(ctx) + if err != nil { + return err + } + } + } + + err = cloud.WriteFile(ctx, defaultStore, BackupProgressDirectory+"/"+filename, bytes.NewReader(descBuf)) + if err != nil { + return errors.Wrap(err, "calculating checksum") + } + + // Write the checksum file after we've successfully wrote the checkpoint. + checksum, err := GetChecksum(descBuf) + if err != nil { + return errors.Wrap(err, "calculating checksum") + } + + err = cloud.WriteFile(ctx, defaultStore, BackupProgressDirectory+"/"+filename+BackupManifestChecksumSuffix, bytes.NewReader(checksum)) + if err != nil { + return err + } + + return nil } // readLatestCheckpointFile returns an ioctx.ReaderCloserCtx of the latest @@ -1065,12 +1020,12 @@ func readLatestCheckpointFile( // We name files such that the most recent checkpoint will always // be at the top, so just grab the first filename. - err = exportStore.List(ctx, backupProgressDirectory, "", func(p string) error { + err = exportStore.List(ctx, BackupProgressDirectory, "", func(p string) error { // The first file returned by List could be either the checkpoint or // checksum file, but we are only concerned with the timestamped prefix. // We resolve if it is a checkpoint or checksum file separately below. p = strings.TrimPrefix(p, "/") - checkpoint = strings.TrimSuffix(p, backupManifestChecksumSuffix) + checkpoint = strings.TrimSuffix(p, BackupManifestChecksumSuffix) checkpointFound = true // We only want the first checkpoint so return an error that it is // done listing. @@ -1081,7 +1036,7 @@ func readLatestCheckpointFile( // directly. This can still fail if it is a mixed cluster and the // checkpoint was written in the base directory. if errors.Is(err, cloud.ErrListingUnsupported) { - r, err = exportStore.ReadFile(ctx, backupProgressDirectory+"/"+filename) + r, err = exportStore.ReadFile(ctx, BackupProgressDirectory+"/"+filename) // If we found the checkpoint in progress, then don't bother reading // from base, just return the reader. if err == nil { @@ -1092,10 +1047,10 @@ func readLatestCheckpointFile( } if checkpointFound { - if strings.HasSuffix(filename, backupManifestChecksumSuffix) { - return exportStore.ReadFile(ctx, backupProgressDirectory+"/"+checkpoint+backupManifestChecksumSuffix) + if strings.HasSuffix(filename, BackupManifestChecksumSuffix) { + return exportStore.ReadFile(ctx, BackupProgressDirectory+"/"+checkpoint+BackupManifestChecksumSuffix) } - return exportStore.ReadFile(ctx, backupProgressDirectory+"/"+checkpoint) + return exportStore.ReadFile(ctx, BackupProgressDirectory+"/"+checkpoint) } // If the checkpoint wasn't found in the progress directory, then try @@ -1109,22 +1064,22 @@ func readLatestCheckpointFile( } -// newTimestampedCheckpointFileName returns a string of a new checkpoint filename +// NewTimestampedCheckpointFileName returns a string of a new checkpoint filename // with a suffixed version. It returns it in the format of BACKUP-CHECKPOINT- // where version is a hex encoded one's complement of the timestamp. // This means that as long as the supplied timestamp is correct, the filenames // will adhere to a lexicographical/utf-8 ordering such that the most // recent file is at the top. -func newTimestampedCheckpointFileName() string { +func NewTimestampedCheckpointFileName() string { var buffer []byte buffer = encoding.EncodeStringDescending(buffer, timeutil.Now().String()) - return fmt.Sprintf("%s-%s", backupManifestCheckpointName, hex.EncodeToString(buffer)) + return fmt.Sprintf("%s-%s", BackupManifestCheckpointName, hex.EncodeToString(buffer)) } // FetchPreviousBackups takes a list of URIs of previous backups and returns // their manifest as well as the encryption options of the first backup in the // chain. -func fetchPreviousBackups( +func FetchPreviousBackups( ctx context.Context, mem *mon.BoundAccount, user username.SQLUsername, diff --git a/pkg/ccl/backupccl/backuputils/testutils.go b/pkg/ccl/backupccl/backuputils/testutils.go index 60bce41cbd88..e52908d31862 100644 --- a/pkg/ccl/backupccl/backuputils/testutils.go +++ b/pkg/ccl/backupccl/backuputils/testutils.go @@ -24,6 +24,8 @@ import ( ) const ( + // SingleNode is the size of a single node test cluster. + SingleNode = 1 // MultiNode is the size of a multi node test cluster. MultiNode = 3 ) @@ -100,3 +102,11 @@ func backupRestoreTestSetupWithParams( return tc, sqlDB, dir, cleanupFn } + +// BackupRestoreTestSetup creates and returns a pre-populated testing +// environment that can be used in backup and restore tests. +func BackupRestoreTestSetup( + t testing.TB, clusterSize int, numAccounts int, init func(*testcluster.TestCluster), +) (tc *testcluster.TestCluster, sqlDB *sqlutils.SQLRunner, tempDir string, cleanup func()) { + return backupRestoreTestSetupWithParams(t, clusterSize, numAccounts, init, base.TestClusterArgs{}) +} diff --git a/pkg/ccl/backupccl/restore_job.go b/pkg/ccl/backupccl/restore_job.go index f76461985da1..75edd655d690 100644 --- a/pkg/ccl/backupccl/restore_job.go +++ b/pkg/ccl/backupccl/restore_job.go @@ -17,6 +17,7 @@ import ( "time" "github.com/cockroachdb/cockroach/pkg/build" + "github.com/cockroachdb/cockroach/pkg/ccl/backupccl/backupinfo" "github.com/cockroachdb/cockroach/pkg/ccl/backupccl/backuppb" "github.com/cockroachdb/cockroach/pkg/cloud" "github.com/cockroachdb/cockroach/pkg/clusterversion" @@ -412,13 +413,13 @@ func loadBackupSQLDescs( details jobspb.RestoreDetails, encryption *jobspb.BackupEncryptionOptions, ) ([]backuppb.BackupManifest, backuppb.BackupManifest, []catalog.Descriptor, int64, error) { - backupManifests, sz, err := loadBackupManifests(ctx, mem, details.URIs, + backupManifests, sz, err := backupinfo.LoadBackupManifests(ctx, mem, details.URIs, p.User(), p.ExecCfg().DistSQLSrv.ExternalStorageFromURI, encryption) if err != nil { return nil, backuppb.BackupManifest{}, nil, 0, err } - allDescs, latestBackupManifest := loadSQLDescsFromBackupsAtTime(backupManifests, details.EndTime) + allDescs, latestBackupManifest := backupinfo.LoadSQLDescsFromBackupsAtTime(backupManifests, details.EndTime) for _, m := range details.DatabaseModifiers { for _, typ := range m.ExtraTypeDescs { @@ -478,36 +479,6 @@ type restoreResumer struct { } } -// getStatisticsFromBackup retrieves Statistics from backup manifest, -// either through the Statistics field or from the files. -func getStatisticsFromBackup( - ctx context.Context, - exportStore cloud.ExternalStorage, - encryption *jobspb.BackupEncryptionOptions, - backup backuppb.BackupManifest, -) ([]*stats.TableStatisticProto, error) { - // This part deals with pre-20.2 stats format where backup statistics - // are stored as a field in backup manifests instead of in their - // individual files. - if backup.DeprecatedStatistics != nil { - return backup.DeprecatedStatistics, nil - } - tableStatistics := make([]*stats.TableStatisticProto, 0, len(backup.StatisticsFilenames)) - uniqueFileNames := make(map[string]struct{}) - for _, fname := range backup.StatisticsFilenames { - if _, exists := uniqueFileNames[fname]; !exists { - uniqueFileNames[fname] = struct{}{} - myStatsTable, err := readTableStatistics(ctx, exportStore, fname, encryption) - if err != nil { - return tableStatistics, err - } - tableStatistics = append(tableStatistics, myStatsTable.Statistics...) - } - } - - return tableStatistics, nil -} - // remapRelevantStatistics changes the table ID references in the stats // from those they had in the backed up database to what they should be // in the restored database. @@ -1291,7 +1262,7 @@ func (r *restoreResumer) doResume(ctx context.Context, execCtx interface{}) erro } } - lastBackupIndex, err := getBackupIndexAtTime(backupManifests, details.EndTime) + lastBackupIndex, err := backupinfo.GetBackupIndexAtTime(backupManifests, details.EndTime) if err != nil { return err } @@ -1350,7 +1321,7 @@ func (r *restoreResumer) doResume(ctx context.Context, execCtx interface{}) erro } } var remappedStats []*stats.TableStatisticProto - backupStats, err := getStatisticsFromBackup(ctx, defaultStore, details.Encryption, + backupStats, err := backupinfo.GetStatisticsFromBackup(ctx, defaultStore, details.Encryption, latestBackupManifest) if err == nil { remappedStats = remapRelevantStatistics(ctx, backupStats, details.DescriptorRewrites, diff --git a/pkg/ccl/backupccl/restore_planning.go b/pkg/ccl/backupccl/restore_planning.go index 4a637e837e40..c717646a0f24 100644 --- a/pkg/ccl/backupccl/restore_planning.go +++ b/pkg/ccl/backupccl/restore_planning.go @@ -1483,15 +1483,16 @@ func doRestorePlan( if len(from) <= 1 { // Incremental layers are not specified explicitly. They will be searched for automatically. // This could be either INTO-syntax, OR TO-syntax. - defaultURIs, mainBackupManifests, localityInfo, memReserved, err = resolveBackupManifests( + defaultURIs, mainBackupManifests, localityInfo, memReserved, err = backupdest.ResolveBackupManifests( ctx, &mem, baseStores, mkStore, fullyResolvedBaseDirectory, fullyResolvedIncrementalsDirectory, endTime, encryption, p.User(), ) } else { // Incremental layers are specified explicitly. // This implies the old, deprecated TO-syntax. - defaultURIs, mainBackupManifests, localityInfo, memReserved, err = resolveBackupManifestsExplicitIncrementals( - ctx, &mem, mkStore, from, endTime, encryption, p.User()) + defaultURIs, mainBackupManifests, localityInfo, memReserved, err = + backupdest.DeprecatedResolveBackupManifestsExplicitIncrementals(ctx, &mem, mkStore, from, + endTime, encryption, p.User()) } if err != nil { diff --git a/pkg/ccl/backupccl/restore_span_covering.go b/pkg/ccl/backupccl/restore_span_covering.go index 5f259c056dcc..dfe7f6407841 100644 --- a/pkg/ccl/backupccl/restore_span_covering.go +++ b/pkg/ccl/backupccl/restore_span_covering.go @@ -11,6 +11,7 @@ package backupccl import ( "sort" + "github.com/cockroachdb/cockroach/pkg/ccl/backupccl/backupinfo" "github.com/cockroachdb/cockroach/pkg/ccl/backupccl/backuppb" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/sql/execinfrapb" @@ -61,7 +62,7 @@ func makeSimpleImportSpans( } for i := range backups { - sort.Sort(BackupFileDescriptors(backups[i].Files)) + sort.Sort(backupinfo.BackupFileDescriptors(backups[i].Files)) } var cover []execinfrapb.RestoreSpanEntry diff --git a/pkg/ccl/backupccl/show.go b/pkg/ccl/backupccl/show.go index a7bb5ad60af6..394be2cf50ea 100644 --- a/pkg/ccl/backupccl/show.go +++ b/pkg/ccl/backupccl/show.go @@ -18,6 +18,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/ccl/backupccl/backupbase" "github.com/cockroachdb/cockroach/pkg/ccl/backupccl/backupdest" "github.com/cockroachdb/cockroach/pkg/ccl/backupccl/backupencryption" + "github.com/cockroachdb/cockroach/pkg/ccl/backupccl/backupinfo" "github.com/cockroachdb/cockroach/pkg/ccl/backupccl/backuppb" "github.com/cockroachdb/cockroach/pkg/ccl/backupccl/backuputils" "github.com/cockroachdb/cockroach/pkg/ccl/storageccl" @@ -163,7 +164,7 @@ func (m metadataSSTInfoReader) showBackup( user username.SQLUsername, resultsCh chan<- tree.Datums, ) error { - filename := metadataSSTName + filename := backupinfo.MetadataSSTName push := func(_, readable string, value json.JSON) error { val := tree.DNull if value != nil { @@ -182,7 +183,7 @@ func (m metadataSSTInfoReader) showBackup( return errors.Wrapf(err, "creating external store") } defer store.Close() - if err := DebugDumpMetadataSST(ctx, store, filename, info.enc, push); err != nil { + if err := backupinfo.DebugDumpMetadataSST(ctx, store, filename, info.enc, push); err != nil { return err } } @@ -216,15 +217,15 @@ func showBackupPlanHook( } expected := map[string]sql.KVStringOptValidate{ - backupOptEncPassphrase: sql.KVStringOptRequireValue, - backupOptEncKMS: sql.KVStringOptRequireValue, - backupOptWithPrivileges: sql.KVStringOptRequireNoValue, - backupOptAsJSON: sql.KVStringOptRequireNoValue, - backupOptWithDebugIDs: sql.KVStringOptRequireNoValue, - backupOptIncStorage: sql.KVStringOptRequireValue, - backupOptDebugMetadataSST: sql.KVStringOptRequireNoValue, - backupOptEncDir: sql.KVStringOptRequireValue, - backupOptCheckFiles: sql.KVStringOptRequireNoValue, + backupencryption.BackupOptEncPassphrase: sql.KVStringOptRequireValue, + backupencryption.BackupOptEncKMS: sql.KVStringOptRequireValue, + backupOptWithPrivileges: sql.KVStringOptRequireNoValue, + backupOptAsJSON: sql.KVStringOptRequireNoValue, + backupOptWithDebugIDs: sql.KVStringOptRequireNoValue, + backupOptIncStorage: sql.KVStringOptRequireValue, + backupOptDebugMetadataSST: sql.KVStringOptRequireNoValue, + backupOptEncDir: sql.KVStringOptRequireValue, + backupOptCheckFiles: sql.KVStringOptRequireNoValue, } optsFn, err := p.TypeAsStringOpts(ctx, backup.Options, expected) if err != nil { @@ -334,7 +335,7 @@ func showBackupPlanHook( var encryption *jobspb.BackupEncryptionOptions showEncErr := `If you are running SHOW BACKUP exclusively on an incremental backup, you must pass the 'encryption_info_dir' parameter that points to the directory of your full backup` - if passphrase, ok := opts[backupOptEncPassphrase]; ok { + if passphrase, ok := opts[backupencryption.BackupOptEncPassphrase]; ok { opts, err := backupencryption.ReadEncryptionOptions(ctx, encStore) if errors.Is(err, backupencryption.ErrEncryptionInfoRead) { return errors.WithHint(err, showEncErr) @@ -347,7 +348,7 @@ you must pass the 'encryption_info_dir' parameter that points to the directory o Mode: jobspb.EncryptionMode_Passphrase, Key: encryptionKey, } - } else if kms, ok := opts[backupOptEncKMS]; ok { + } else if kms, ok := opts[backupencryption.BackupOptEncKMS]; ok { opts, err := backupencryption.ReadEncryptionOptions(ctx, encStore) if errors.Is(err, backupencryption.ErrEncryptionInfoRead) { return errors.WithHint(err, showEncErr) @@ -418,14 +419,14 @@ you must pass the 'encryption_info_dir' parameter that points to the directory o mkStore := p.ExecCfg().DistSQLSrv.ExternalStorageFromURI info.defaultURIs, info.manifests, info.localityInfo, memReserved, - err = resolveBackupManifests( + err = backupdest.ResolveBackupManifests( ctx, &mem, baseStores, mkStore, fullyResolvedDest, fullyResolvedIncrementalsDirectory, hlc.Timestamp{}, encryption, p.User()) defer func() { mem.Shrink(ctx, memReserved) }() if err != nil { - if errors.Is(err, errLocalityDescriptor) && subdir == "" { + if errors.Is(err, backupinfo.ErrLocalityDescriptor) && subdir == "" { p.BufferClientNotice(ctx, pgnotice.Newf("`SHOW BACKUP` using the old syntax ("+ "without the `IN` keyword) on a locality aware backup does not display or validate"+ @@ -518,9 +519,9 @@ func checkBackupFiles( // metadata files ( prefixed with `backupPartitionDescriptorPrefix`) , as // they're validated in resolveBackupManifests. for _, metaFile := range []string{ - fileInfoPath, - metadataSSTName, - backupbase.BackupManifestName + backupManifestChecksumSuffix} { + backupinfo.FileInfoPath, + backupinfo.MetadataSSTName, + backupbase.BackupManifestName + backupinfo.BackupManifestChecksumSuffix} { if _, err := defaultStore.Size(ctx, metaFile); err != nil { return nil, errors.Wrapf(err, "Error checking metadata file %s/%s", info.defaultURIs[layer], metaFile) diff --git a/pkg/ccl/backupccl/targets.go b/pkg/ccl/backupccl/targets.go index c7bb1744dea6..917eba421858 100644 --- a/pkg/ccl/backupccl/targets.go +++ b/pkg/ccl/backupccl/targets.go @@ -15,6 +15,7 @@ import ( "sort" "strings" + "github.com/cockroachdb/cockroach/pkg/ccl/backupccl/backupinfo" "github.com/cockroachdb/cockroach/pkg/ccl/backupccl/backuppb" "github.com/cockroachdb/cockroach/pkg/ccl/backupccl/backupresolver" "github.com/cockroachdb/cockroach/pkg/keys" @@ -340,7 +341,7 @@ func selectTargets( asOf hlc.Timestamp, restoreSystemUsers bool, ) ([]catalog.Descriptor, []catalog.DatabaseDescriptor, []descpb.TenantInfoWithUsage, error) { - allDescs, lastBackupManifest := loadSQLDescsFromBackupsAtTime(backupManifests, asOf) + allDescs, lastBackupManifest := backupinfo.LoadSQLDescsFromBackupsAtTime(backupManifests, asOf) if descriptorCoverage == tree.AllDescriptors { return fullClusterTargetsRestore(allDescs, lastBackupManifest) @@ -388,7 +389,7 @@ func selectTargets( return nil, nil, nil, errors.Errorf("no tables or databases matched the given targets: %s", tree.ErrString(&targets)) } - if lastBackupManifest.FormatVersion >= BackupFormatDescriptorTrackingVersion { + if lastBackupManifest.FormatVersion >= backupinfo.BackupFormatDescriptorTrackingVersion { if err := matched.CheckExpansions(lastBackupManifest.CompleteDbs); err != nil { return nil, nil, nil, err } @@ -448,7 +449,7 @@ func MakeBackupTableEntry( backupManifests = backupManifests[:ind+1] } - allDescs, _ := loadSQLDescsFromBackupsAtTime(backupManifests, endTime) + allDescs, _ := backupinfo.LoadSQLDescsFromBackupsAtTime(backupManifests, endTime) resolver, err := backupresolver.NewDescriptorResolver(allDescs) if err != nil { return BackupTableEntry{}, errors.Wrapf(err, "creating a new resolver for all descriptors") diff --git a/pkg/ccl/backupccl/utils_test.go b/pkg/ccl/backupccl/utils_test.go index 9284c15bfa7f..be4a00fc84f4 100644 --- a/pkg/ccl/backupccl/utils_test.go +++ b/pkg/ccl/backupccl/utils_test.go @@ -27,6 +27,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/base" "github.com/cockroachdb/cockroach/pkg/ccl/backupccl/backupbase" + "github.com/cockroachdb/cockroach/pkg/ccl/backupccl/backupinfo" "github.com/cockroachdb/cockroach/pkg/ccl/backupccl/backuppb" "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/kv" @@ -382,7 +383,7 @@ func getSpansFromManifest(ctx context.Context, t *testing.T, backupPath string) backupManifestBytes, err := ioutil.ReadFile(backupPath + "/" + backupbase.BackupManifestName) require.NoError(t, err) var backupManifest backuppb.BackupManifest - decompressedBytes, err := decompressData(ctx, nil, backupManifestBytes) + decompressedBytes, err := backupinfo.DecompressData(ctx, nil, backupManifestBytes) require.NoError(t, err) require.NoError(t, protoutil.Unmarshal(decompressedBytes, &backupManifest)) spans := make([]roachpb.Span, 0, len(backupManifest.Files)) diff --git a/pkg/ccl/cliccl/BUILD.bazel b/pkg/ccl/cliccl/BUILD.bazel index d5d060bf2ebc..7246cb6b70ba 100644 --- a/pkg/ccl/cliccl/BUILD.bazel +++ b/pkg/ccl/cliccl/BUILD.bazel @@ -17,6 +17,7 @@ go_library( "//pkg/ccl/backupccl", "//pkg/ccl/backupccl/backupbase", "//pkg/ccl/backupccl/backupdest", + "//pkg/ccl/backupccl/backupinfo", "//pkg/ccl/backupccl/backuppb", "//pkg/ccl/backupccl/backuputils", "//pkg/ccl/baseccl", diff --git a/pkg/ccl/cliccl/debug_backup.go b/pkg/ccl/cliccl/debug_backup.go index 64f68a8ace98..b8efe3d15fe8 100644 --- a/pkg/ccl/cliccl/debug_backup.go +++ b/pkg/ccl/cliccl/debug_backup.go @@ -28,6 +28,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/ccl/backupccl" "github.com/cockroachdb/cockroach/pkg/ccl/backupccl/backupbase" "github.com/cockroachdb/cockroach/pkg/ccl/backupccl/backupdest" + "github.com/cockroachdb/cockroach/pkg/ccl/backupccl/backupinfo" "github.com/cockroachdb/cockroach/pkg/ccl/backupccl/backuppb" "github.com/cockroachdb/cockroach/pkg/ccl/backupccl/backuputils" "github.com/cockroachdb/cockroach/pkg/ccl/storageccl" @@ -301,7 +302,7 @@ func getManifestFromURI(ctx context.Context, path string) (backuppb.BackupManife // upgraded from the old FK representation, or even older formats). If more // fields are added to the output, the table descriptors may need to be // upgraded. - backupManifest, _, err := backupccl.ReadBackupManifestFromURI(ctx, nil /* mem */, path, username.RootUserName(), + backupManifest, _, err := backupinfo.ReadBackupManifestFromURI(ctx, nil /* mem */, path, username.RootUserName(), externalStorageFromURIFactory, nil) if err != nil { return backuppb.BackupManifest{}, err @@ -404,7 +405,7 @@ func runListIncrementalCmd(cmd *cobra.Command, args []string) error { } defer baseStore.Close() - oldIncPaths, err := backupdest.FindPriorBackups(ctx, baseStore, backupbase.OmitManifest) + oldIncPaths, err := backupdest.FindPriorBackups(ctx, baseStore, backupdest.OmitManifest) if err != nil { return err } @@ -421,7 +422,7 @@ func runListIncrementalCmd(cmd *cobra.Command, args []string) error { } defer incStore.Close() - newIncPaths, err := backupdest.FindPriorBackups(ctx, incStore, backupbase.OmitManifest) + newIncPaths, err := backupdest.FindPriorBackups(ctx, incStore, backupdest.OmitManifest) if err != nil { return err } @@ -439,7 +440,7 @@ func runListIncrementalCmd(cmd *cobra.Command, args []string) error { return errors.Wrapf(err, "connect to external storage") } defer stores[i].Close() - manifest, _, err := backupccl.ReadBackupManifestFromStore(ctx, nil /* mem */, stores[i], nil) + manifest, _, err := backupinfo.ReadBackupManifestFromStore(ctx, nil /* mem */, stores[i], nil) if err != nil { return err } From e4d7f257f4c01a308d9fa0fdb9b692365bea0fc4 Mon Sep 17 00:00:00 2001 From: Marylia Gutierrez Date: Fri, 10 Jun 2022 18:28:34 -0400 Subject: [PATCH 3/9] ui: remove option 10/30 min from SQL Activity page Previously we had the options for 10 and 30min on SQL Activity pages, which created some confusion, since we would always show the last 1h info. This commit remove those 2 options. If the user select any of those options on the Metrics page, it will get updated to 1h on the SQL Activity pages. Fixes #82914 Release note (ui change): Removal of the 10 and 30min options on the SQL Activity page. --- .../src/statementDetails/statementDetails.tsx | 13 ++++++++ .../src/statementsPage/statementsPage.tsx | 11 +++++++ .../timeScaleDropdown/timeScaleDropdown.tsx | 21 ++++++++++-- .../cluster-ui/src/timeScaleDropdown/utils.ts | 32 ++++++++++++------- .../transactionDetails/transactionDetails.tsx | 11 +++++++ .../src/transactionsPage/transactionsPage.tsx | 11 +++++++ .../containers/raftMessages/index.tsx | 4 +-- 7 files changed, 87 insertions(+), 16 deletions(-) diff --git a/pkg/ui/workspaces/cluster-ui/src/statementDetails/statementDetails.tsx b/pkg/ui/workspaces/cluster-ui/src/statementDetails/statementDetails.tsx index 428ea5024c42..a5c2ecb64c1f 100644 --- a/pkg/ui/workspaces/cluster-ui/src/statementDetails/statementDetails.tsx +++ b/pkg/ui/workspaces/cluster-ui/src/statementDetails/statementDetails.tsx @@ -52,7 +52,9 @@ import { NodeSummaryStats } from "../nodes"; import { UIConfigState } from "../store"; import { StatementDetailsRequest } from "src/api/statementsApi"; import { + getValidOption, TimeScale, + timeScale1hMinOptions, TimeScaleDropdown, timeScaleToString, toRoundedDateRange, @@ -216,6 +218,14 @@ export class StatementDetails extends React.Component< currentTab: searchParams.get("tab") || "overview", }; this.activateDiagnosticsRef = React.createRef(); + + // In case the user selected a option not available on this page, + // force a selection of a valid option. This is necessary for the case + // where the value 10/30 min is selected on the Metrics page. + const ts = getValidOption(this.props.timeScale, timeScale1hMinOptions); + if (ts !== this.props.timeScale) { + this.props.onTimeScaleChange(ts); + } } static defaultProps: Partial = { @@ -431,6 +441,7 @@ export class StatementDetails extends React.Component< @@ -559,6 +570,7 @@ export class StatementDetails extends React.Component< @@ -704,6 +716,7 @@ export class StatementDetails extends React.Component< diff --git a/pkg/ui/workspaces/cluster-ui/src/statementsPage/statementsPage.tsx b/pkg/ui/workspaces/cluster-ui/src/statementsPage/statementsPage.tsx index 7d7633f29a73..cd4450fd07b2 100644 --- a/pkg/ui/workspaces/cluster-ui/src/statementsPage/statementsPage.tsx +++ b/pkg/ui/workspaces/cluster-ui/src/statementsPage/statementsPage.tsx @@ -76,6 +76,8 @@ import { TimeScale, toDateRange, timeScaleToString, + timeScale1hMinOptions, + getValidOption, } from "../timeScaleDropdown"; import { commonStyles } from "../common"; @@ -186,6 +188,14 @@ export class StatementsPage extends React.Component< const stateFromHistory = this.getStateFromHistory(); this.state = merge(defaultState, stateFromHistory); this.activateDiagnosticsRef = React.createRef(); + + // In case the user selected a option not available on this page, + // force a selection of a valid option. This is necessary for the case + // where the value 10/30 min is selected on the Metrics page. + const ts = getValidOption(this.props.timeScale, timeScale1hMinOptions); + if (ts !== this.props.timeScale) { + this.changeTimeScale(ts); + } } static defaultProps: Partial = { @@ -655,6 +665,7 @@ export class StatementsPage extends React.Component< diff --git a/pkg/ui/workspaces/cluster-ui/src/timeScaleDropdown/timeScaleDropdown.tsx b/pkg/ui/workspaces/cluster-ui/src/timeScaleDropdown/timeScaleDropdown.tsx index 06a275687d28..a5c0adb74b3b 100644 --- a/pkg/ui/workspaces/cluster-ui/src/timeScaleDropdown/timeScaleDropdown.tsx +++ b/pkg/ui/workspaces/cluster-ui/src/timeScaleDropdown/timeScaleDropdown.tsx @@ -12,10 +12,10 @@ import React, { useMemo } from "react"; import moment from "moment"; import classNames from "classnames/bind"; import { - TimeScale, - TimeWindow, ArrowDirection, + TimeScale, TimeScaleOptions, + TimeWindow, } from "./timeScaleTypes"; import TimeFrameControls from "./timeFrameControls"; import RangeSelect, { @@ -263,3 +263,20 @@ export const TimeScaleDropdown: React.FC = ({ ); }; + +// getValidOption check if the option selected is valid. If is valid returns +// the selected option, otherwise returns the first valid option. +export const getValidOption = ( + currentScale: TimeScale, + options: TimeScaleOptions, +): TimeScale => { + if (!(currentScale.key in options)) { + const firstValidKey = Object.keys(options)[0]; + return { + ...options[firstValidKey], + key: firstValidKey, + fixedWindowEnd: false, + }; + } + return currentScale; +}; diff --git a/pkg/ui/workspaces/cluster-ui/src/timeScaleDropdown/utils.ts b/pkg/ui/workspaces/cluster-ui/src/timeScaleDropdown/utils.ts index 11eda48412b1..845d05fd04ab 100644 --- a/pkg/ui/workspaces/cluster-ui/src/timeScaleDropdown/utils.ts +++ b/pkg/ui/workspaces/cluster-ui/src/timeScaleDropdown/utils.ts @@ -14,20 +14,10 @@ import { dateFormat, timeFormat } from "./timeScaleDropdown"; import React from "react"; /** - * defaultTimeScaleOptions is a preconfigured set of time scales that can be + * timeScale1hMinOptions is a preconfigured set of time scales with 1h minimum that can be * selected by the user. */ -export const defaultTimeScaleOptions: TimeScaleOptions = { - "Past 10 Minutes": { - windowSize: moment.duration(10, "minutes"), - windowValid: moment.duration(10, "seconds"), - sampleSize: moment.duration(10, "seconds"), - }, - "Past 30 Minutes": { - windowSize: moment.duration(30, "minutes"), - windowValid: moment.duration(30, "seconds"), - sampleSize: moment.duration(30, "seconds"), - }, +export const timeScale1hMinOptions: TimeScaleOptions = { "Past 1 Hour": { windowSize: moment.duration(1, "hour"), windowValid: moment.duration(1, "minute"), @@ -75,6 +65,24 @@ export const defaultTimeScaleOptions: TimeScaleOptions = { }, }; +/** + * defaultTimeScaleOptions is a preconfigured set of time scales that can be + * selected by the user. + */ +export const defaultTimeScaleOptions: TimeScaleOptions = { + "Past 10 Minutes": { + windowSize: moment.duration(10, "minutes"), + windowValid: moment.duration(10, "seconds"), + sampleSize: moment.duration(10, "seconds"), + }, + "Past 30 Minutes": { + windowSize: moment.duration(30, "minutes"), + windowValid: moment.duration(30, "seconds"), + sampleSize: moment.duration(30, "seconds"), + }, + ...timeScale1hMinOptions, +}; + export const defaultTimeScaleSelected: TimeScale = { ...defaultTimeScaleOptions["Past 1 Hour"], key: "Past 1 Hour", diff --git a/pkg/ui/workspaces/cluster-ui/src/transactionDetails/transactionDetails.tsx b/pkg/ui/workspaces/cluster-ui/src/transactionDetails/transactionDetails.tsx index bd14ea111a1f..dd580e3cc07f 100644 --- a/pkg/ui/workspaces/cluster-ui/src/transactionDetails/transactionDetails.tsx +++ b/pkg/ui/workspaces/cluster-ui/src/transactionDetails/transactionDetails.tsx @@ -60,7 +60,9 @@ import { Transaction } from "src/transactionsTable"; import Long from "long"; import { StatementsRequest } from "../api"; import { + getValidOption, TimeScale, + timeScale1hMinOptions, TimeScaleDropdown, timeScaleToString, toDateRange, @@ -134,6 +136,14 @@ export class TransactionDetails extends React.Component< }, latestTransactionText: "", }; + + // In case the user selected a option not available on this page, + // force a selection of a valid option. This is necessary for the case + // where the value 10/30 min is selected on the Metrics page. + const ts = getValidOption(this.props.timeScale, timeScale1hMinOptions); + if (ts !== this.props.timeScale) { + this.props.onTimeScaleChange(ts); + } } static defaultProps: Partial = { @@ -253,6 +263,7 @@ export class TransactionDetails extends React.Component< diff --git a/pkg/ui/workspaces/cluster-ui/src/transactionsPage/transactionsPage.tsx b/pkg/ui/workspaces/cluster-ui/src/transactionsPage/transactionsPage.tsx index b1d03b8df15e..3f75a47bcedd 100644 --- a/pkg/ui/workspaces/cluster-ui/src/transactionsPage/transactionsPage.tsx +++ b/pkg/ui/workspaces/cluster-ui/src/transactionsPage/transactionsPage.tsx @@ -65,6 +65,8 @@ import { TimeScale, toDateRange, timeScaleToString, + timeScale1hMinOptions, + getValidOption, } from "../timeScaleDropdown"; import { InlineAlert } from "@cockroachlabs/ui-components"; import { TransactionViewType } from "./transactionsPageTypes"; @@ -135,6 +137,14 @@ export class TransactionsPage extends React.Component< }; const stateFromHistory = this.getStateFromHistory(); this.state = merge(this.state, stateFromHistory); + + // In case the user selected a option not available on this page, + // force a selection of a valid option. This is necessary for the case + // where the value 10/30 min is selected on the Metrics page. + const ts = getValidOption(this.props.timeScale, timeScale1hMinOptions); + if (ts !== this.props.timeScale) { + this.changeTimeScale(ts); + } } getStateFromHistory = (): Partial => { @@ -409,6 +419,7 @@ export class TransactionsPage extends React.Component< diff --git a/pkg/ui/workspaces/db-console/src/views/devtools/containers/raftMessages/index.tsx b/pkg/ui/workspaces/db-console/src/views/devtools/containers/raftMessages/index.tsx index 190f122cf36a..58cac83382ab 100644 --- a/pkg/ui/workspaces/db-console/src/views/devtools/containers/raftMessages/index.tsx +++ b/pkg/ui/workspaces/db-console/src/views/devtools/containers/raftMessages/index.tsx @@ -23,6 +23,7 @@ import { } from "src/redux/hover"; import { NodesSummary, nodesSummarySelector } from "src/redux/nodes"; import { AdminUIState } from "src/redux/state"; +import { setGlobalTimeScaleAction } from "src/redux/statements"; import { nodeIDAttr } from "src/util/constants"; import { GraphDashboardProps, @@ -42,7 +43,6 @@ import { TimeWindow, TimeScale, setMetricsFixedWindow, - setTimeScale, } from "src/redux/timeScale"; interface NodeGraphsOwnProps { @@ -203,7 +203,7 @@ const mapDispatchToProps = { hoverOn: hoverOnAction, hoverOff: hoverOffAction, setMetricsFixedWindow: setMetricsFixedWindow, - setTimeScale, + setTimeScale: setGlobalTimeScaleAction, }; export default withRouter( From 566dff233d94aa18e2f9e53cdf888adf9ee22d53 Mon Sep 17 00:00:00 2001 From: Erik Grinaker Date: Fri, 11 Feb 2022 16:10:52 +0000 Subject: [PATCH 4/9] kvserver: emit MVCC range tombstones over rangefeeds This patch adds MVCC range tombstone support in rangefeeds. Whenever an MVCC range tombstone is written, a new `MVCCDeleteRangeOp` logical op is recorded and emitted across the rangefeed as a `RangeFeedDeleteRange` event. MVCC range tombstones will only be written when the `MVCCRangeTombstones` version gate has been enabled. Changefeeds will emit an error for these events. We do not expect to see these in online spans with changefeeds, since they are initially only planned for use with schema GC and import rollbacks. The rangefeed client library has been extended with support for these events, but no existing callers handle them for the same reason as changefeeds. Initial scans do not emit regular tombstones, and thus not range tombstones either, but catchup scans will emit them if encountered. This patch has rudimentary testing of MVCC range tombstones in rangefeeds. A later patch will add a data-driven test harness for rangefeeds with more exhaustive tests. Release note: None --- .../changefeedccl/kvfeed/physical_kv_feed.go | 15 ++ .../streamproducer/event_stream.go | 1 + pkg/kv/kvclient/rangefeed/BUILD.bazel | 1 + pkg/kv/kvclient/rangefeed/config.go | 19 +++ pkg/kv/kvclient/rangefeed/rangefeed.go | 6 + .../rangefeed/rangefeed_external_test.go | 157 ++++++++++++++++++ pkg/kv/kvserver/rangefeed/catchup_scan.go | 86 ++++++++-- .../rangefeed/catchup_scan_bench_test.go | 2 +- .../kvserver/rangefeed/catchup_scan_test.go | 5 + pkg/kv/kvserver/rangefeed/processor.go | 23 +++ pkg/kv/kvserver/rangefeed/registry.go | 15 ++ .../kvserver/rangefeed/resolved_timestamp.go | 4 + pkg/kv/kvserver/rangefeed/task_test.go | 7 +- pkg/kv/kvserver/replica_rangefeed.go | 6 +- pkg/roachpb/api.go | 3 + pkg/roachpb/api.proto | 17 +- pkg/roachpb/data.go | 5 + pkg/storage/enginepb/mvcc3.proto | 10 ++ pkg/storage/mvcc.go | 21 ++- pkg/storage/mvcc_key.go | 26 ++- pkg/storage/mvcc_key_test.go | 40 ++++- pkg/storage/mvcc_logical_ops.go | 13 ++ 22 files changed, 450 insertions(+), 32 deletions(-) diff --git a/pkg/ccl/changefeedccl/kvfeed/physical_kv_feed.go b/pkg/ccl/changefeedccl/kvfeed/physical_kv_feed.go index 0b8018894d61..142a9f700a88 100644 --- a/pkg/ccl/changefeedccl/kvfeed/physical_kv_feed.go +++ b/pkg/ccl/changefeedccl/kvfeed/physical_kv_feed.go @@ -123,6 +123,21 @@ func (p *rangefeed) addEventsToBuffer(ctx context.Context) error { // expect SST ingestion into spans with active changefeeds. return errors.Errorf("unexpected SST ingestion: %v", t) + case *roachpb.RangeFeedDeleteRange: + // For now, we just error on MVCC range tombstones. These are currently + // only expected to be used by schema GC and IMPORT INTO, and such spans + // should not have active changefeeds across them. + // + // TODO(erikgrinaker): Write an end-to-end test which verifies that an + // IMPORT INTO which gets rolled back using MVCC range tombstones will + // not be visible to a changefeed, neither when it was started before + // the import or when resuming from a timestamp before the import. The + // table decriptor should be marked as offline during the import, and + // catchup scans should detect that this happened and prevent reading + // anything in that timespan. See: + // https://github.com/cockroachdb/cockroach/issues/70433 + return errors.Errorf("unexpected MVCC range deletion: %v", t) + default: return errors.Errorf("unexpected RangeFeedEvent variant %v", t) } diff --git a/pkg/ccl/streamingccl/streamproducer/event_stream.go b/pkg/ccl/streamingccl/streamproducer/event_stream.go index 5c8929b848cf..fda74cf22210 100644 --- a/pkg/ccl/streamingccl/streamproducer/event_stream.go +++ b/pkg/ccl/streamingccl/streamproducer/event_stream.go @@ -472,6 +472,7 @@ func (s *eventStream) streamLoop(ctx context.Context, frontier *span.Frontier) e return err } default: + // TODO(erikgrinaker): Handle DeleteRange events (MVCC range tombstones). return errors.AssertionFailedf("unexpected event") } } diff --git a/pkg/kv/kvclient/rangefeed/BUILD.bazel b/pkg/kv/kvclient/rangefeed/BUILD.bazel index 1e4cffb02cda..888d3cd3f275 100644 --- a/pkg/kv/kvclient/rangefeed/BUILD.bazel +++ b/pkg/kv/kvclient/rangefeed/BUILD.bazel @@ -73,6 +73,7 @@ go_test( "//pkg/spanconfig", "//pkg/spanconfig/spanconfigptsreader", "//pkg/sql/catalog/desctestutils", + "//pkg/storage", "//pkg/testutils", "//pkg/testutils/serverutils", "//pkg/testutils/sqlutils", diff --git a/pkg/kv/kvclient/rangefeed/config.go b/pkg/kv/kvclient/rangefeed/config.go index 56a0e235730a..d7f0101a8ec1 100644 --- a/pkg/kv/kvclient/rangefeed/config.go +++ b/pkg/kv/kvclient/rangefeed/config.go @@ -41,6 +41,7 @@ type config struct { onCheckpoint OnCheckpoint onFrontierAdvance OnFrontierAdvance onSSTable OnSSTable + onDeleteRange OnDeleteRange extraPProfLabels []string } @@ -178,6 +179,24 @@ func WithOnSSTable(f OnSSTable) Option { }) } +// OnDeleteRange is called when an MVCC range tombstone is written (e.g. when +// DeleteRange is called with UseExperimentalRangeTombstone, but not when the +// range is deleted using point tombstones). If this callback is not provided, +// an error is emitted when these are encountered. +// +// MVCC range tombstones are currently experimental, and requires the +// MVCCRangeTombstones version gate. They are only expected during certain +// operations like schema GC and IMPORT INTO (i.e. not across live tables). +type OnDeleteRange func(ctx context.Context, value *roachpb.RangeFeedDeleteRange) + +// WithOnDeleteRange sets up a callback that's invoked whenever an MVCC range +// deletion tombstone is written. +func WithOnDeleteRange(f OnDeleteRange) Option { + return optionFunc(func(c *config) { + c.onDeleteRange = f + }) +} + // OnFrontierAdvance is called when the rangefeed frontier is advanced with the // new frontier timestamp. type OnFrontierAdvance func(ctx context.Context, timestamp hlc.Timestamp) diff --git a/pkg/kv/kvclient/rangefeed/rangefeed.go b/pkg/kv/kvclient/rangefeed/rangefeed.go index 950bc5610389..f039e0569f47 100644 --- a/pkg/kv/kvclient/rangefeed/rangefeed.go +++ b/pkg/kv/kvclient/rangefeed/rangefeed.go @@ -357,6 +357,12 @@ func (f *RangeFeed) processEvents( "received unexpected rangefeed SST event with no OnSSTable handler") } f.onSSTable(ctx, ev.SST) + case ev.DeleteRange != nil: + if f.onDeleteRange == nil { + return errors.AssertionFailedf( + "received unexpected rangefeed DeleteRange event with no OnDeleteRange handler: %s", ev) + } + f.onDeleteRange(ctx, ev.DeleteRange) case ev.Error != nil: // Intentionally do nothing, we'll get an error returned from the // call to RangeFeed. diff --git a/pkg/kv/kvclient/rangefeed/rangefeed_external_test.go b/pkg/kv/kvclient/rangefeed/rangefeed_external_test.go index e1fc934bf71f..e9c9673b6082 100644 --- a/pkg/kv/kvclient/rangefeed/rangefeed_external_test.go +++ b/pkg/kv/kvclient/rangefeed/rangefeed_external_test.go @@ -24,6 +24,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/spanconfig" "github.com/cockroachdb/cockroach/pkg/spanconfig/spanconfigptsreader" + "github.com/cockroachdb/cockroach/pkg/storage" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/testutils/sstutil" "github.com/cockroachdb/cockroach/pkg/testutils/testcluster" @@ -615,6 +616,162 @@ func TestWithOnSSTableCatchesUpIfNotSet(t *testing.T) { require.Equal(t, expectKVs, seenKVs) } +// TestWithOnDeleteRange tests that the rangefeed emits MVCC range tombstones. +// +// TODO(erikgrinaker): These kinds of tests should really use a data-driven test +// harness, for more exhaustive testing. But it'll do for now. +func TestWithOnDeleteRange(t *testing.T) { + defer leaktest.AfterTest(t)() + + ctx := context.Background() + tc := testcluster.StartTestCluster(t, 1, base.TestClusterArgs{}) + defer tc.Stopper().Stop(ctx) + srv := tc.Server(0) + db := srv.DB() + + _, _, err := tc.SplitRange(roachpb.Key("a")) + require.NoError(t, err) + require.NoError(t, tc.WaitForFullReplication()) + + _, err = tc.ServerConn(0).Exec("SET CLUSTER SETTING kv.rangefeed.enabled = true") + require.NoError(t, err) + f, err := rangefeed.NewFactory(srv.Stopper(), db, srv.ClusterSettings(), nil) + require.NoError(t, err) + + // We lay down a few MVCC range tombstones and points. The first range + // tombstone should not be visible, because initial scans do not emit + // tombstones, nor should the points covered by it. The second range tombstone + // should be visible, because catchup scans do emit tombstones. The range + // tombstone should be ordered after the initial point, but before the foo + // catchup point, and the previous values should respect the range tombstones. + require.NoError(t, db.Put(ctx, "covered", "covered")) + require.NoError(t, db.Put(ctx, "foo", "covered")) + require.NoError(t, db.ExperimentalDelRangeUsingTombstone(ctx, "a", "z")) + require.NoError(t, db.Put(ctx, "foo", "initial")) + rangeFeedTS := db.Clock().Now() + require.NoError(t, db.Put(ctx, "covered", "catchup")) + require.NoError(t, db.ExperimentalDelRangeUsingTombstone(ctx, "a", "z")) + require.NoError(t, db.Put(ctx, "foo", "catchup")) + + // We start the rangefeed over a narrower span than the DeleteRanges (c-g), + // to ensure the DeleteRange event is truncated to the registration span. + var checkpointOnce sync.Once + checkpointC := make(chan struct{}) + deleteRangeC := make(chan *roachpb.RangeFeedDeleteRange) + rowC := make(chan *roachpb.RangeFeedValue) + + spans := []roachpb.Span{{Key: roachpb.Key("c"), EndKey: roachpb.Key("g")}} + r, err := f.RangeFeed(ctx, "test", spans, rangeFeedTS, + func(ctx context.Context, e *roachpb.RangeFeedValue) { + select { + case rowC <- e: + case <-ctx.Done(): + } + }, + rangefeed.WithDiff(true), + rangefeed.WithInitialScan(nil), + rangefeed.WithOnCheckpoint(func(ctx context.Context, checkpoint *roachpb.RangeFeedCheckpoint) { + checkpointOnce.Do(func() { + close(checkpointC) + }) + }), + rangefeed.WithOnDeleteRange(func(ctx context.Context, e *roachpb.RangeFeedDeleteRange) { + select { + case deleteRangeC <- e: + case <-ctx.Done(): + } + }), + ) + require.NoError(t, err) + defer r.Close() + + // Wait for initial scan. We should see the foo=initial point, but not the + // range tombstone nor the covered points. + select { + case e := <-rowC: + require.Equal(t, roachpb.Key("foo"), e.Key) + value, err := e.Value.GetBytes() + require.NoError(t, err) + require.Equal(t, "initial", string(value)) + prevValue, err := e.PrevValue.GetBytes() + require.NoError(t, err) + require.Equal(t, "initial", string(prevValue)) // initial scans supply current as prev + case <-time.After(3 * time.Second): + require.Fail(t, "timed out waiting for initial scan event") + } + + // Wait for catchup scan. We should see the second range tombstone, truncated + // to the rangefeed bounds (c-g), and it should be ordered before the points + // covered=catchup and foo=catchup. both points should have a tombstone as the + // previous value. + select { + case e := <-deleteRangeC: + require.Equal(t, roachpb.Span{Key: roachpb.Key("c"), EndKey: roachpb.Key("g")}, e.Span) + require.NotEmpty(t, e.Timestamp) + case <-time.After(3 * time.Second): + require.Fail(t, "timed out waiting for DeleteRange event") + } + + select { + case e := <-rowC: + require.Equal(t, roachpb.Key("covered"), e.Key) + value, err := e.Value.GetBytes() + require.NoError(t, err) + require.Equal(t, "catchup", string(value)) + prevValue, err := storage.DecodeMVCCValue(e.PrevValue.RawBytes) + require.NoError(t, err) + require.True(t, prevValue.IsTombstone()) + case <-time.After(3 * time.Second): + require.Fail(t, "timed out waiting for foo=catchup event") + } + + select { + case e := <-rowC: + require.Equal(t, roachpb.Key("foo"), e.Key) + value, err := e.Value.GetBytes() + require.NoError(t, err) + require.Equal(t, "catchup", string(value)) + prevValue, err := storage.DecodeMVCCValue(e.PrevValue.RawBytes) + require.NoError(t, err) + require.True(t, prevValue.IsTombstone()) + case <-time.After(3 * time.Second): + require.Fail(t, "timed out waiting for foo=catchup event") + } + + // Wait for checkpoint after catchup scan. + select { + case <-checkpointC: + case <-time.After(3 * time.Second): + require.Fail(t, "timed out waiting for checkpoint") + } + + // Send another DeleteRange, and wait for the rangefeed event. This should + // be truncated to the rangefeed bounds (c-g). + require.NoError(t, db.ExperimentalDelRangeUsingTombstone(ctx, "a", "z")) + select { + case e := <-deleteRangeC: + require.Equal(t, roachpb.Span{Key: roachpb.Key("c"), EndKey: roachpb.Key("g")}, e.Span) + require.NotEmpty(t, e.Timestamp) + case <-time.After(3 * time.Second): + require.Fail(t, "timed out waiting for DeleteRange event") + } + + // A final point write should be emitted with a tombstone as the previous value. + require.NoError(t, db.Put(ctx, "foo", "final")) + select { + case e := <-rowC: + require.Equal(t, roachpb.Key("foo"), e.Key) + value, err := e.Value.GetBytes() + require.NoError(t, err) + require.Equal(t, "final", string(value)) + prevValue, err := storage.DecodeMVCCValue(e.PrevValue.RawBytes) + require.NoError(t, err) + require.True(t, prevValue.IsTombstone()) + case <-time.After(3 * time.Second): + require.Fail(t, "timed out waiting for foo=final event") + } +} + // TestUnrecoverableErrors verifies that unrecoverable internal errors are surfaced // to callers. func TestUnrecoverableErrors(t *testing.T) { diff --git a/pkg/kv/kvserver/rangefeed/catchup_scan.go b/pkg/kv/kvserver/rangefeed/catchup_scan.go index 622e3361bb95..f64258f941ab 100644 --- a/pkg/kv/kvserver/rangefeed/catchup_scan.go +++ b/pkg/kv/kvserver/rangefeed/catchup_scan.go @@ -65,6 +65,8 @@ func NewCatchUpIterator( return &CatchUpIterator{ simpleCatchupIter: storage.NewMVCCIncrementalIterator(reader, storage.MVCCIncrementalIterOptions{ + KeyTypes: storage.IterKeyTypePointsAndRanges, + StartKey: span.Key, EndKey: span.EndKey, StartTime: startTime, EndTime: hlc.MaxTimestamp, @@ -96,6 +98,15 @@ type outputEventFn func(e *roachpb.RangeFeedEvent) error // CatchUpScan iterates over all changes in the configured key/time span, and // emits them as RangeFeedEvents via outputFn in chronological order. +// +// MVCC range tombstones are emitted at their start key, in chronological order. +// Because the start key itself is not timestamped, these will be ordered before +// all of the timestamped point keys that they overlap. For more details, see +// MVCC range key info on storage.SimpleMVCCIterator. +// +// For example, with MVCC range tombstones [a-f)@5 and [a-f)@3 overlapping point +// keys a@6, a@4, and b@2, the emitted order is [a-f)@3,[a-f)@5,a@4,a@6,b@2 because +// the start key "a" is ordered before all of the timestamped point keys. func (i *CatchUpIterator) CatchUpScan(outputFn outputEventFn, withDiff bool) error { var a bufalloc.ByteAllocator // MVCCIterator will encounter historical values for each key in @@ -103,18 +114,7 @@ func (i *CatchUpIterator) CatchUpScan(outputFn outputEventFn, withDiff bool) err // events for the same key until a different key is encountered, then output // the encountered values in reverse. This also allows us to buffer events // as we fill in previous values. - var lastKey roachpb.Key reorderBuf := make([]roachpb.RangeFeedEvent, 0, 5) - addPrevToLastEvent := func(val []byte) { - if l := len(reorderBuf); l > 0 { - if reorderBuf[l-1].Val.PrevValue.IsPresent() { - panic("RangeFeedValue.PrevVal unexpectedly set") - } - // TODO(sumeer): find out if it is deliberate that we are not populating - // PrevValue.Timestamp. - reorderBuf[l-1].Val.PrevValue.RawBytes = val - } - } outputEvents := func() error { for i := len(reorderBuf) - 1; i >= 0; i-- { @@ -130,7 +130,9 @@ func (i *CatchUpIterator) CatchUpScan(outputFn outputEventFn, withDiff bool) err // Iterate though all keys using Next. We want to publish all committed // versions of each key that are after the registration's startTS, so we // can't use NextKey. + var lastKey roachpb.Key var meta enginepb.MVCCMetadata + var rangeKeysStart roachpb.Key i.SeekGE(storage.MVCCKey{Key: i.span.Key}) for { if ok, err := i.Valid(); err != nil { @@ -139,6 +141,45 @@ func (i *CatchUpIterator) CatchUpScan(outputFn outputEventFn, withDiff bool) err break } + hasPoint, hasRange := i.HasPointAndRange() + + // Emit any new MVCC range tombstones when their start key is encountered. + // Range keys can currently only be MVCC range tombstones. + // + // TODO(erikgrinaker): Find a faster/better way to detect range key changes + // that doesn't involve constant comparisons. Pebble probably already knows, + // we just need a way to ask it. + if hasRange { + if rangeBounds := i.RangeBounds(); !rangeBounds.Key.Equal(rangeKeysStart) { + rangeKeysStart = append(rangeKeysStart[:0], rangeBounds.Key...) + + // Emit events for these MVCC range tombstones, in chronological order. + rangeKeys := i.RangeKeys() + for i := len(rangeKeys) - 1; i >= 0; i-- { + var span roachpb.Span + a, span.Key = a.Copy(rangeBounds.Key, 0) + a, span.EndKey = a.Copy(rangeBounds.EndKey, 0) + err := outputFn(&roachpb.RangeFeedEvent{ + DeleteRange: &roachpb.RangeFeedDeleteRange{ + Span: span, + Timestamp: rangeKeys[i].RangeKey.Timestamp, + }, + }) + if err != nil { + return err + } + } + } + } + + // If there's no point key here (i.e. we found a bare range key above), then + // step onto the next key. This may be a point key version at the same key + // as the range key's start bound, or a later point/range key. + if !hasPoint { + i.Next() + continue + } + unsafeKey := i.UnsafeKey() unsafeValRaw := i.UnsafeValue() if !unsafeKey.IsValue() { @@ -217,9 +258,26 @@ func (i *CatchUpIterator) CatchUpScan(outputFn outputEventFn, withDiff bool) err var val []byte a, val = a.Copy(unsafeVal, 0) if withDiff { - // Update the last version with its - // previous value (this version). - addPrevToLastEvent(val) + // Update the last version with its previous value (this version). + if l := len(reorderBuf) - 1; l >= 0 { + if reorderBuf[l].Val.PrevValue.IsPresent() { + return errors.AssertionFailedf("unexpected previous value %s for key %s", + reorderBuf[l].Val.PrevValue, key) + } + // If an MVCC range tombstone exists between this value and the next + // one, we don't emit the value after all -- it should be a tombstone. + // + // TODO(erikgrinaker): We can't save range keys when we detect changes + // to rangeKeysStart above, because NextIgnoringTime() could reveal + // additional MVCC range tombstones below StartTime that cover this + // point. We need to find a more performant way to handle this. + if !hasRange || !storage.HasRangeKeyBetween( + i.RangeKeys(), reorderBuf[l].Val.Value.Timestamp, ts) { + // TODO(sumeer): find out if it is deliberate that we are not populating + // PrevValue.Timestamp. + reorderBuf[l].Val.PrevValue.RawBytes = val + } + } } if !ignore { diff --git a/pkg/kv/kvserver/rangefeed/catchup_scan_bench_test.go b/pkg/kv/kvserver/rangefeed/catchup_scan_bench_test.go index 568fcb2edf8a..15eddbe29298 100644 --- a/pkg/kv/kvserver/rangefeed/catchup_scan_bench_test.go +++ b/pkg/kv/kvserver/rangefeed/catchup_scan_bench_test.go @@ -175,7 +175,7 @@ func setupMVCCPebble(b testing.TB, dir string, lBaseMaxBytes int64, readOnly boo opts.FS = vfs.Default opts.LBaseMaxBytes = lBaseMaxBytes opts.ReadOnly = readOnly - opts.FormatMajorVersion = pebble.FormatBlockPropertyCollector + opts.FormatMajorVersion = pebble.FormatRangeKeys peb, err := storage.NewPebble( context.Background(), storage.PebbleConfig{ diff --git a/pkg/kv/kvserver/rangefeed/catchup_scan_test.go b/pkg/kv/kvserver/rangefeed/catchup_scan_test.go index 9d9c78e0ac13..bf0b7ff6689a 100644 --- a/pkg/kv/kvserver/rangefeed/catchup_scan_test.go +++ b/pkg/kv/kvserver/rangefeed/catchup_scan_test.go @@ -26,6 +26,11 @@ import ( "github.com/stretchr/testify/require" ) +// TODO(erikgrinaker): This should be migrated to a data-driven test harness for +// end-to-end rangefeed testing, with more exhaustive test cases. See: +// https://github.com/cockroachdb/cockroach/issues/82715 +// +// For now, see rangefeed_external_test.go for rudimentary range key tests. func TestCatchupScan(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) diff --git a/pkg/kv/kvserver/rangefeed/processor.go b/pkg/kv/kvserver/rangefeed/processor.go index 1f2e8d55a1e0..3bd0b58bf7c7 100644 --- a/pkg/kv/kvserver/rangefeed/processor.go +++ b/pkg/kv/kvserver/rangefeed/processor.go @@ -700,6 +700,10 @@ func (p *Processor) consumeLogicalOps( // Publish the new value directly. p.publishValue(ctx, t.Key, t.Timestamp, t.Value, t.PrevValue, allocation) + case *enginepb.MVCCDeleteRangeOp: + // Publish the range deletion directly. + p.publishDeleteRange(ctx, t.StartKey, t.EndKey, t.Timestamp, allocation) + case *enginepb.MVCCWriteIntentOp: // No updates to publish. @@ -777,6 +781,25 @@ func (p *Processor) publishValue( p.reg.PublishToOverlapping(ctx, roachpb.Span{Key: key}, &event, allocation) } +func (p *Processor) publishDeleteRange( + ctx context.Context, + startKey, endKey roachpb.Key, + timestamp hlc.Timestamp, + allocation *SharedBudgetAllocation, +) { + span := roachpb.Span{Key: startKey, EndKey: endKey} + if !p.Span.ContainsKeyRange(roachpb.RKey(startKey), roachpb.RKey(endKey)) { + log.Fatalf(ctx, "span %s not in Processor's key range %v", span, p.Span) + } + + var event roachpb.RangeFeedEvent + event.MustSetValue(&roachpb.RangeFeedDeleteRange{ + Span: span, + Timestamp: timestamp, + }) + p.reg.PublishToOverlapping(ctx, span, &event, allocation) +} + func (p *Processor) publishSSTable( ctx context.Context, sst []byte, diff --git a/pkg/kv/kvserver/rangefeed/registry.go b/pkg/kv/kvserver/rangefeed/registry.go index 09a7c2005b85..a945160e95f6 100644 --- a/pkg/kv/kvserver/rangefeed/registry.go +++ b/pkg/kv/kvserver/rangefeed/registry.go @@ -198,6 +198,13 @@ func (r *registration) validateEvent(event *roachpb.RangeFeedEvent) { if t.WriteTS.IsEmpty() { panic(fmt.Sprintf("unexpected empty RangeFeedSSTable.Timestamp: %v", t)) } + case *roachpb.RangeFeedDeleteRange: + if len(t.Span.Key) == 0 || len(t.Span.EndKey) == 0 { + panic(fmt.Sprintf("unexpected empty key in RangeFeedDeleteRange.Span: %v", t)) + } + if t.Timestamp.IsEmpty() { + panic(fmt.Sprintf("unexpected empty RangeFeedDeleteRange.Timestamp: %v", t)) + } default: panic(fmt.Sprintf("unexpected RangeFeedEvent variant: %v", t)) } @@ -244,6 +251,12 @@ func (r *registration) maybeStripEvent(event *roachpb.RangeFeedEvent) *roachpb.R t = copyOnWrite().(*roachpb.RangeFeedCheckpoint) t.Span = r.span } + case *roachpb.RangeFeedDeleteRange: + // Truncate the range tombstone to the registration bounds. + if i := t.Span.Intersect(r.span); !i.Equal(t.Span) { + t = copyOnWrite().(*roachpb.RangeFeedDeleteRange) + t.Span = i.Clone() + } case *roachpb.RangeFeedSSTable: // SSTs are always sent in their entirety, it is up to the caller to // filter out irrelevant entries. @@ -440,6 +453,8 @@ func (reg *registry) PublishToOverlapping( minTS = t.Value.Timestamp case *roachpb.RangeFeedSSTable: minTS = t.WriteTS + case *roachpb.RangeFeedDeleteRange: + minTS = t.Timestamp case *roachpb.RangeFeedCheckpoint: // Always publish checkpoint notifications, regardless of a registration's // starting timestamp. diff --git a/pkg/kv/kvserver/rangefeed/resolved_timestamp.go b/pkg/kv/kvserver/rangefeed/resolved_timestamp.go index 0119b735bc20..4b7e6dd6f743 100644 --- a/pkg/kv/kvserver/rangefeed/resolved_timestamp.go +++ b/pkg/kv/kvserver/rangefeed/resolved_timestamp.go @@ -142,6 +142,10 @@ func (rts *resolvedTimestamp) consumeLogicalOp(op enginepb.MVCCLogicalOp) bool { rts.assertOpAboveRTS(op, t.Timestamp) return false + case *enginepb.MVCCDeleteRangeOp: + rts.assertOpAboveRTS(op, t.Timestamp) + return false + case *enginepb.MVCCWriteIntentOp: rts.assertOpAboveRTS(op, t.Timestamp) return rts.intentQ.IncRef(t.TxnID, t.TxnKey, t.TxnMinTimestamp, t.Timestamp) diff --git a/pkg/kv/kvserver/rangefeed/task_test.go b/pkg/kv/kvserver/rangefeed/task_test.go index f3d2b7c1d773..e236c10ebec4 100644 --- a/pkg/kv/kvserver/rangefeed/task_test.go +++ b/pkg/kv/kvserver/rangefeed/task_test.go @@ -196,17 +196,18 @@ func (s *testIterator) curKV() storage.MVCCKeyValue { // HasPointAndRange implements SimpleMVCCIterator. func (s *testIterator) HasPointAndRange() (bool, bool) { - panic("not implemented") + ok, err := s.Valid() + return ok && err == nil, false } // RangeBounds implements SimpleMVCCIterator. func (s *testIterator) RangeBounds() roachpb.Span { - panic("not implemented") + return roachpb.Span{} } // RangeTombstones implements SimpleMVCCIterator. func (s *testIterator) RangeKeys() []storage.MVCCRangeKeyValue { - panic("not implemented") + return []storage.MVCCRangeKeyValue{} } func TestInitResolvedTSScan(t *testing.T) { diff --git a/pkg/kv/kvserver/replica_rangefeed.go b/pkg/kv/kvserver/replica_rangefeed.go index 88e4013878d5..c3b62db37ef2 100644 --- a/pkg/kv/kvserver/replica_rangefeed.go +++ b/pkg/kv/kvserver/replica_rangefeed.go @@ -513,7 +513,8 @@ func (r *Replica) populatePrevValsInLogicalOpLogRaftMuLocked( case *enginepb.MVCCWriteIntentOp, *enginepb.MVCCUpdateIntentOp, *enginepb.MVCCAbortIntentOp, - *enginepb.MVCCAbortTxnOp: + *enginepb.MVCCAbortTxnOp, + *enginepb.MVCCDeleteRangeOp: // Nothing to do. continue default: @@ -587,7 +588,8 @@ func (r *Replica) handleLogicalOpLogRaftMuLocked( case *enginepb.MVCCWriteIntentOp, *enginepb.MVCCUpdateIntentOp, *enginepb.MVCCAbortIntentOp, - *enginepb.MVCCAbortTxnOp: + *enginepb.MVCCAbortTxnOp, + *enginepb.MVCCDeleteRangeOp: // Nothing to do. continue default: diff --git a/pkg/roachpb/api.go b/pkg/roachpb/api.go index 5eba5ae37a20..d9cb171fe036 100644 --- a/pkg/roachpb/api.go +++ b/pkg/roachpb/api.go @@ -1519,6 +1519,9 @@ func (e *RangeFeedEvent) ShallowCopy() *RangeFeedEvent { case *RangeFeedSSTable: cpySST := *t cpy.MustSetValue(&cpySST) + case *RangeFeedDeleteRange: + cpyDelRange := *t + cpy.MustSetValue(&cpyDelRange) case *RangeFeedError: cpyErr := *t cpy.MustSetValue(&cpyErr) diff --git a/pkg/roachpb/api.proto b/pkg/roachpb/api.proto index 10e2768da420..9620f4c9833e 100644 --- a/pkg/roachpb/api.proto +++ b/pkg/roachpb/api.proto @@ -2783,15 +2783,24 @@ message RangeFeedSSTable { util.hlc.Timestamp write_ts = 3 [(gogoproto.nullable) = false, (gogoproto.customname) = "WriteTS"]; } +// RangeFeedDeleteRange is a variant of RangeFeedEvent that represents a +// deletion of the specified key range at the given timestamp using an MVCC +// range tombstone. +message RangeFeedDeleteRange { + Span span = 1 [(gogoproto.nullable) = false]; + util.hlc.Timestamp timestamp = 2 [(gogoproto.nullable) = false]; +} + // RangeFeedEvent is a union of all event types that may be returned on a // RangeFeed response stream. message RangeFeedEvent { option (gogoproto.onlyone) = true; - RangeFeedValue val = 1; - RangeFeedCheckpoint checkpoint = 2; - RangeFeedError error = 3; - RangeFeedSSTable sst = 4 [(gogoproto.customname) = "SST"]; + RangeFeedValue val = 1; + RangeFeedCheckpoint checkpoint = 2; + RangeFeedError error = 3; + RangeFeedSSTable sst = 4 [(gogoproto.customname) = "SST"]; + RangeFeedDeleteRange delete_range = 5; } diff --git a/pkg/roachpb/data.go b/pkg/roachpb/data.go index 1814d1ee63c0..f0dc9643ff0a 100644 --- a/pkg/roachpb/data.go +++ b/pkg/roachpb/data.go @@ -2088,6 +2088,11 @@ func (ls LockStateInfo) String() string { return redact.StringWithoutMarkers(ls) } +// Clone returns a copy of the span. +func (s Span) Clone() Span { + return Span{Key: s.Key.Clone(), EndKey: s.EndKey.Clone()} +} + // EqualValue is Equal. // // TODO(tbg): remove this passthrough. diff --git a/pkg/storage/enginepb/mvcc3.proto b/pkg/storage/enginepb/mvcc3.proto index ec05dad30d65..f34a47a9c85d 100644 --- a/pkg/storage/enginepb/mvcc3.proto +++ b/pkg/storage/enginepb/mvcc3.proto @@ -343,6 +343,15 @@ message MVCCAbortTxnOp { (gogoproto.nullable) = false]; } +// MVCCDeleteRangeOp corresponds to a range deletion using an MVCC range +// tombstone. +message MVCCDeleteRangeOp { + bytes start_key = 1; + bytes end_key = 2; + util.hlc.Timestamp timestamp = 3 [(gogoproto.nullable) = false]; +} + + // MVCCLogicalOp is a union of all logical MVCC operation types. message MVCCLogicalOp { option (gogoproto.onlyone) = true; @@ -353,4 +362,5 @@ message MVCCLogicalOp { MVCCCommitIntentOp commit_intent = 4; MVCCAbortIntentOp abort_intent = 5; MVCCAbortTxnOp abort_txn = 6; + MVCCDeleteRangeOp delete_range = 7; } diff --git a/pkg/storage/mvcc.go b/pkg/storage/mvcc.go index deeeac6facec..0ef3dca7c5e3 100644 --- a/pkg/storage/mvcc.go +++ b/pkg/storage/mvcc.go @@ -973,7 +973,7 @@ func mvccGetMetadata( // metadata), or the point version's timestamp if it was a tombstone. if hasRange { rangeKeys := iter.RangeKeys() - if rkv, ok := firstRangeKeyAbove(rangeKeys, unsafeKey.Timestamp); ok { + if rkv, ok := FirstRangeKeyAbove(rangeKeys, unsafeKey.Timestamp); ok { meta.Deleted = true meta.Timestamp = rangeKeys[0].RangeKey.Timestamp.ToLegacyTimestamp() keyLastSeen := rkv.RangeKey.Timestamp @@ -2735,8 +2735,19 @@ func ExperimentalMVCCDeleteRangeUsingTombstone( } } - // Write the tombstone. - return rw.ExperimentalPutMVCCRangeKey(rangeKey, value) + if err := rw.ExperimentalPutMVCCRangeKey(rangeKey, value); err != nil { + return err + } + + // Record the logical operation, for rangefeed emission. + rw.LogLogicalOp(MVCCDeleteRangeOpType, MVCCLogicalOpDetails{ + Safe: true, + Key: rangeKey.StartKey, + EndKey: rangeKey.EndKey, + Timestamp: rangeKey.Timestamp, + }) + + return nil } func recordIteratorStats(traceSpan *tracing.Span, iteratorStats IteratorStats) { @@ -3798,7 +3809,7 @@ func mvccResolveWriteIntent( // synthesize a point tombstone at the lowest range tombstone covering it. // This is where the point key ceases to exist, contributing to GCBytesAge. if len(unsafeNextValueRaw) > 0 { - if rk, found := firstRangeKeyAbove(iter.RangeKeys(), unsafeNextKey.Timestamp); found { + if rk, found := FirstRangeKeyAbove(iter.RangeKeys(), unsafeNextKey.Timestamp); found { unsafeNextKey.Timestamp = rk.RangeKey.Timestamp unsafeNextValueRaw = []byte{} } @@ -4485,7 +4496,7 @@ func ComputeStatsForRangeWithVisitors( // only take them into account for versioned values. var nextRangeTombstone hlc.Timestamp if isValue { - if rkv, ok := firstRangeKeyAbove(rangeKeys, unsafeKey.Timestamp); ok { + if rkv, ok := FirstRangeKeyAbove(rangeKeys, unsafeKey.Timestamp); ok { nextRangeTombstone = rkv.RangeKey.Timestamp } } diff --git a/pkg/storage/mvcc_key.go b/pkg/storage/mvcc_key.go index 8bf9b39abb9f..dbbbbfa868c3 100644 --- a/pkg/storage/mvcc_key.go +++ b/pkg/storage/mvcc_key.go @@ -17,6 +17,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/storage/enginepb" + "github.com/cockroachdb/cockroach/pkg/util" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/errors" ) @@ -438,11 +439,14 @@ func (k MVCCRangeKey) Validate() (err error) { } } -// firstRangeKeyAbove does a binary search for the first range key at or above +// FirstRangeKeyAbove does a binary search for the first range key at or above // the given timestamp. It assumes the range keys are ordered in descending // timestamp order, as returned by SimpleMVCCIterator.RangeKeys(). Returns false // if no matching range key was found. -func firstRangeKeyAbove(rangeKeys []MVCCRangeKeyValue, ts hlc.Timestamp) (MVCCRangeKeyValue, bool) { +// +// TODO(erikgrinaker): Consider using a new type for []MVCCRangeKeyValue as +// returned by SimpleMVCCIterator.RangeKeys(), and add this as a method. +func FirstRangeKeyAbove(rangeKeys []MVCCRangeKeyValue, ts hlc.Timestamp) (MVCCRangeKeyValue, bool) { // This is kind of odd due to sort.Search() semantics: we do a binary search // for the first range tombstone that's below the timestamp, then return the // previous range tombstone if any. @@ -453,3 +457,21 @@ func firstRangeKeyAbove(rangeKeys []MVCCRangeKeyValue, ts hlc.Timestamp) (MVCCRa } return MVCCRangeKeyValue{}, false } + +// HasRangeKeyBetween checks whether an MVCC range key exists between the two +// given timestamps (in order). It assumes the range keys are ordered in +// descending timestamp order, as returned by SimpleMVCCIterator.RangeKeys(). +func HasRangeKeyBetween(rangeKeys []MVCCRangeKeyValue, upper, lower hlc.Timestamp) bool { + if len(rangeKeys) == 0 { + return false + } + if util.RaceEnabled && upper.Less(lower) { + panic(errors.AssertionFailedf("HasRangeKeyBetween given upper %s <= lower %s", upper, lower)) + } + if rkv, ok := FirstRangeKeyAbove(rangeKeys, lower); ok { + // Consider equal timestamps to be "between". This shouldn't really happen, + // since MVCC enforces point and range keys can't have the same timestamp. + return rkv.RangeKey.Timestamp.LessEq(upper) + } + return false +} diff --git a/pkg/storage/mvcc_key_test.go b/pkg/storage/mvcc_key_test.go index f99b1b5f1fa2..8bac4ddbf4b8 100644 --- a/pkg/storage/mvcc_key_test.go +++ b/pkg/storage/mvcc_key_test.go @@ -22,6 +22,7 @@ import ( "testing/quick" "github.com/cockroachdb/cockroach/pkg/roachpb" + "github.com/cockroachdb/cockroach/pkg/util" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/leaktest" "github.com/cockroachdb/cockroach/pkg/util/log" @@ -523,7 +524,7 @@ func TestFirstRangeKeyAbove(t *testing.T) { } for _, tc := range testcases { t.Run(fmt.Sprintf("%d", tc.ts), func(t *testing.T) { - rkv, ok := firstRangeKeyAbove(rangeKVs, hlc.Timestamp{WallTime: tc.ts}) + rkv, ok := FirstRangeKeyAbove(rangeKVs, hlc.Timestamp{WallTime: tc.ts}) if tc.expect == 0 { require.False(t, ok) require.Empty(t, rkv) @@ -535,6 +536,43 @@ func TestFirstRangeKeyAbove(t *testing.T) { } } +func TestHasRangeKeyBetween(t *testing.T) { + defer leaktest.AfterTest(t)() + + rangeKVs := []MVCCRangeKeyValue{ + rangeKV("a", "f", 5, MVCCValue{}), + rangeKV("a", "f", 1, MVCCValue{}), + } + + testcases := []struct { + upper, lower int + expect bool + }{ + {0, 0, false}, + {0, 1, false}, // wrong order + {1, 0, true}, + {1, 1, true}, + {0, 2, false}, // wrong order + {4, 6, false}, // wrong order + {6, 4, true}, + {5, 5, true}, + {4, 4, false}, + {6, 6, false}, + {4, 2, false}, + {0, 9, false}, // wrong order + {9, 0, true}, + } + for _, tc := range testcases { + t.Run(fmt.Sprintf("%d,%d", tc.upper, tc.lower), func(t *testing.T) { + if util.RaceEnabled && tc.upper < tc.lower { + require.Panics(t, func() { HasRangeKeyBetween(rangeKVs, wallTS(tc.upper), wallTS(tc.lower)) }) + } else { + require.Equal(t, tc.expect, HasRangeKeyBetween(rangeKVs, wallTS(tc.upper), wallTS(tc.lower))) + } + }) + } +} + func pointKey(key string, ts int) MVCCKey { return MVCCKey{Key: roachpb.Key(key), Timestamp: wallTS(ts)} } diff --git a/pkg/storage/mvcc_logical_ops.go b/pkg/storage/mvcc_logical_ops.go index 7872fb8d0610..6ce5d27480f2 100644 --- a/pkg/storage/mvcc_logical_ops.go +++ b/pkg/storage/mvcc_logical_ops.go @@ -45,6 +45,8 @@ const ( MVCCCommitIntentOpType // MVCCAbortIntentOpType corresponds to the MVCCAbortIntentOp variant. MVCCAbortIntentOpType + // MVCCDeleteRangeOpType corresponds to the MVCCDeleteRangeOp variant. + MVCCDeleteRangeOpType ) // MVCCLogicalOpDetails contains details about the occurrence of an MVCC logical @@ -52,6 +54,7 @@ const ( type MVCCLogicalOpDetails struct { Txn enginepb.TxnMeta Key roachpb.Key + EndKey roachpb.Key // only set for MVCCDeleteRangeOpType Timestamp hlc.Timestamp // Safe indicates that the values in this struct will never be invalidated @@ -142,6 +145,16 @@ func (ol *OpLoggerBatch) logLogicalOp(op MVCCLogicalOpType, details MVCCLogicalO ol.recordOp(&enginepb.MVCCAbortIntentOp{ TxnID: details.Txn.ID, }) + case MVCCDeleteRangeOpType: + if !details.Safe { + ol.opsAlloc, details.Key = ol.opsAlloc.Copy(details.Key, 0) + ol.opsAlloc, details.EndKey = ol.opsAlloc.Copy(details.EndKey, 0) + } + ol.recordOp(&enginepb.MVCCDeleteRangeOp{ + StartKey: details.Key, + EndKey: details.EndKey, + Timestamp: details.Timestamp, + }) default: panic(fmt.Sprintf("unexpected op type %v", op)) } From c9267e23fe7f2643a50d536ee91d5c3603a46cb9 Mon Sep 17 00:00:00 2001 From: Eric Harmeling Date: Mon, 27 Jun 2022 12:35:51 -0400 Subject: [PATCH 5/9] sql: rename anonymizedStmt in sqlstats pkg to stmtNoConstants This commit changes all instances of anonymizedStmt to stmtNoConstants in the sqlstats package. The commit does not change any files outside the sqlstats package. Fixes #80725. Release note: None --- .../sqlstats/ssmemstorage/ss_mem_iterator.go | 2 +- .../sqlstats/ssmemstorage/ss_mem_storage.go | 42 +++++++++---------- .../sqlstats/ssmemstorage/ss_mem_writer.go | 6 +-- pkg/sql/sqlstats/ssmemstorage/utils.go | 2 +- 4 files changed, 26 insertions(+), 26 deletions(-) diff --git a/pkg/sql/sqlstats/ssmemstorage/ss_mem_iterator.go b/pkg/sql/sqlstats/ssmemstorage/ss_mem_iterator.go index 531e70cecf0c..8ad677314bb4 100644 --- a/pkg/sql/sqlstats/ssmemstorage/ss_mem_iterator.go +++ b/pkg/sql/sqlstats/ssmemstorage/ss_mem_iterator.go @@ -85,7 +85,7 @@ func (s *StmtStatsIterator) Next() bool { s.currentValue = &roachpb.CollectedStatementStatistics{ Key: roachpb.StatementStatisticsKey{ - Query: stmtKey.anonymizedStmt, + Query: stmtKey.stmtNoConstants, QuerySummary: querySummary, DistSQL: distSQLUsed, Vec: vectorized, diff --git a/pkg/sql/sqlstats/ssmemstorage/ss_mem_storage.go b/pkg/sql/sqlstats/ssmemstorage/ss_mem_storage.go index a0d09361a2d2..4befcc80aacd 100644 --- a/pkg/sql/sqlstats/ssmemstorage/ss_mem_storage.go +++ b/pkg/sql/sqlstats/ssmemstorage/ss_mem_storage.go @@ -48,21 +48,21 @@ type stmtKey struct { // sampledPlanKey is used by the Optimizer to determine if we should build a full EXPLAIN plan. type sampledPlanKey struct { - anonymizedStmt string - failed bool - implicitTxn bool - database string + stmtNoConstants string + failed bool + implicitTxn bool + database string } func (p sampledPlanKey) size() int64 { - return int64(unsafe.Sizeof(p)) + int64(len(p.anonymizedStmt)) + int64(len(p.database)) + return int64(unsafe.Sizeof(p)) + int64(len(p.stmtNoConstants)) + int64(len(p.database)) } func (s stmtKey) String() string { if s.failed { - return "!" + s.anonymizedStmt + return "!" + s.stmtNoConstants } - return s.anonymizedStmt + return s.stmtNoConstants } func (s stmtKey) size() int64 { @@ -259,10 +259,10 @@ func NewTempContainerFromExistingStmtStats( } key := stmtKey{ sampledPlanKey: sampledPlanKey{ - anonymizedStmt: statistics[i].Key.KeyData.Query, - failed: statistics[i].Key.KeyData.Failed, - implicitTxn: statistics[i].Key.KeyData.ImplicitTxn, - database: statistics[i].Key.KeyData.Database, + stmtNoConstants: statistics[i].Key.KeyData.Query, + failed: statistics[i].Key.KeyData.Failed, + implicitTxn: statistics[i].Key.KeyData.ImplicitTxn, + database: statistics[i].Key.KeyData.Database, }, planHash: statistics[i].Key.KeyData.PlanHash, transactionFingerprintID: statistics[i].Key.KeyData.TransactionFingerprintID, @@ -479,7 +479,7 @@ func (s *stmtStats) mergeStatsLocked(statistics *roachpb.CollectedStatementStati // stat object is returned or not, we always return the correct stmtFingerprintID // for the given stmt. func (s *Container) getStatsForStmt( - anonymizedStmt string, + stmtNoConstants string, implicitTxn bool, database string, failed bool, @@ -497,10 +497,10 @@ func (s *Container) getStatsForStmt( // that we use separate buckets for the different situations. key = stmtKey{ sampledPlanKey: sampledPlanKey{ - anonymizedStmt: anonymizedStmt, - failed: failed, - implicitTxn: implicitTxn, - database: database, + stmtNoConstants: stmtNoConstants, + failed: failed, + implicitTxn: implicitTxn, + database: database, }, planHash: planHash, transactionFingerprintID: transactionFingerprintID, @@ -671,10 +671,10 @@ func (s *Container) MergeApplicationStatementStats( } key := stmtKey{ sampledPlanKey: sampledPlanKey{ - anonymizedStmt: statistics.Key.Query, - failed: statistics.Key.Failed, - implicitTxn: statistics.Key.ImplicitTxn, - database: statistics.Key.Database, + stmtNoConstants: statistics.Key.Query, + failed: statistics.Key.Failed, + implicitTxn: statistics.Key.ImplicitTxn, + database: statistics.Key.Database, }, planHash: statistics.Key.PlanHash, transactionFingerprintID: statistics.Key.TransactionFingerprintID, @@ -961,6 +961,6 @@ type transactionCounts struct { func constructStatementFingerprintIDFromStmtKey(key stmtKey) roachpb.StmtFingerprintID { return roachpb.ConstructStatementFingerprintID( - key.anonymizedStmt, key.failed, key.implicitTxn, key.database, + key.stmtNoConstants, key.failed, key.implicitTxn, key.database, ) } diff --git a/pkg/sql/sqlstats/ssmemstorage/ss_mem_writer.go b/pkg/sql/sqlstats/ssmemstorage/ss_mem_writer.go index ed8095698559..5ef4569ffcad 100644 --- a/pkg/sql/sqlstats/ssmemstorage/ss_mem_writer.go +++ b/pkg/sql/sqlstats/ssmemstorage/ss_mem_writer.go @@ -190,9 +190,9 @@ func (s *Container) ShouldSaveLogicalPlanDesc( fingerprint string, implicitTxn bool, database string, ) bool { lastSampled := s.getLogicalPlanLastSampled(sampledPlanKey{ - anonymizedStmt: fingerprint, - implicitTxn: implicitTxn, - database: database, + stmtNoConstants: fingerprint, + implicitTxn: implicitTxn, + database: database, }) return s.shouldSaveLogicalPlanDescription(lastSampled) } diff --git a/pkg/sql/sqlstats/ssmemstorage/utils.go b/pkg/sql/sqlstats/ssmemstorage/utils.go index fa9bdc570e84..26a1094a2027 100644 --- a/pkg/sql/sqlstats/ssmemstorage/utils.go +++ b/pkg/sql/sqlstats/ssmemstorage/utils.go @@ -25,7 +25,7 @@ func (s stmtList) Swap(i, j int) { s[i], s[j] = s[j], s[i] } func (s stmtList) Less(i, j int) bool { - cmp := strings.Compare(s[i].anonymizedStmt, s[j].anonymizedStmt) + cmp := strings.Compare(s[i].stmtNoConstants, s[j].stmtNoConstants) if cmp == -1 { return true } From b44d6ec57a457b176df941c7601e403e4cadb477 Mon Sep 17 00:00:00 2001 From: Marylia Gutierrez Date: Mon, 27 Jun 2022 17:33:03 -0400 Subject: [PATCH 6/9] ui: update all dates to use same format Update all dates to use the same format. Fixes #81159 Release note: None --- .../activeStatementsTable/activeStatementsTable.tsx | 4 ++-- .../activeTransactionsTable/activeTransactionsTable.tsx | 4 ++-- .../src/databaseDetailsPage/databaseDetailsPage.tsx | 3 ++- .../cluster-ui/src/databaseTablePage/databaseTablePage.tsx | 5 ++--- pkg/ui/workspaces/cluster-ui/src/graphs/utils/domain.ts | 3 ++- .../cluster-ui/src/indexDetailsPage/indexDetailsPage.tsx | 4 ++-- .../src/statementDetails/activeStatementDetails.tsx | 5 ++--- .../src/statementDetails/diagnostics/diagnosticsView.tsx | 3 ++- pkg/ui/workspaces/cluster-ui/src/util/format.ts | 3 ++- pkg/ui/workspaces/db-console/src/util/format.ts | 3 ++- .../src/views/cluster/containers/nodeLogs/index.tsx | 5 ++++- .../src/views/cluster/containers/nodesOverview/index.tsx | 4 ++-- .../containers/nodeHistory/decommissionedNodeHistory.tsx | 3 ++- .../reports/containers/statementDiagnosticsHistory/index.tsx | 3 ++- 14 files changed, 30 insertions(+), 22 deletions(-) diff --git a/pkg/ui/workspaces/cluster-ui/src/activeExecutions/activeStatementsTable/activeStatementsTable.tsx b/pkg/ui/workspaces/cluster-ui/src/activeExecutions/activeStatementsTable/activeStatementsTable.tsx index cec95b80b5cf..075fda706327 100644 --- a/pkg/ui/workspaces/cluster-ui/src/activeExecutions/activeStatementsTable/activeStatementsTable.tsx +++ b/pkg/ui/workspaces/cluster-ui/src/activeExecutions/activeStatementsTable/activeStatementsTable.tsx @@ -25,6 +25,7 @@ import { ExecutionType, ExecutionsColumn, } from "../execTableCommon"; +import { DATE_FORMAT } from "../../util"; interface ActiveStatementsTable { data: ActiveStatement[]; @@ -73,8 +74,7 @@ export function makeActiveStatementsColumns(): ColumnDescriptor { name: "startTime", title: executionsTableTitles.startTime(execType), - cell: (item: ActiveStatement) => - item.start.format("MMM D, YYYY [at] H:mm"), + cell: (item: ActiveStatement) => item.start.format(DATE_FORMAT), sort: (item: ActiveStatement) => item.start.unix(), }, { diff --git a/pkg/ui/workspaces/cluster-ui/src/activeExecutions/activeTransactionsTable/activeTransactionsTable.tsx b/pkg/ui/workspaces/cluster-ui/src/activeExecutions/activeTransactionsTable/activeTransactionsTable.tsx index b80259a735c2..daf40041a247 100644 --- a/pkg/ui/workspaces/cluster-ui/src/activeExecutions/activeTransactionsTable/activeTransactionsTable.tsx +++ b/pkg/ui/workspaces/cluster-ui/src/activeExecutions/activeTransactionsTable/activeTransactionsTable.tsx @@ -25,6 +25,7 @@ import { ExecutionType, ExecutionsColumn, } from "../execTableCommon"; +import { DATE_FORMAT } from "../../util"; interface ActiveTransactionsTable { data: ActiveTransaction[]; @@ -75,8 +76,7 @@ export function makeActiveTransactionsColumns(): ColumnDescriptor - item.start.format("MMM D, YYYY [at] H:mm"), + cell: (item: ActiveTransaction) => item.start.format(DATE_FORMAT), sort: (item: ActiveTransaction) => item.start.unix(), }, { diff --git a/pkg/ui/workspaces/cluster-ui/src/databaseDetailsPage/databaseDetailsPage.tsx b/pkg/ui/workspaces/cluster-ui/src/databaseDetailsPage/databaseDetailsPage.tsx index 0f3330803c3f..d5b8a65e39c1 100644 --- a/pkg/ui/workspaces/cluster-ui/src/databaseDetailsPage/databaseDetailsPage.tsx +++ b/pkg/ui/workspaces/cluster-ui/src/databaseDetailsPage/databaseDetailsPage.tsx @@ -37,6 +37,7 @@ import { } from "src/transactionsPage/transactionsPageClasses"; import { Moment } from "moment"; import { Caution } from "@cockroachlabs/icons"; +import { DATE_FORMAT } from "src/util/format"; const cx = classNames.bind(styles); const sortableTableCx = classNames.bind(sortableTableStyles); @@ -379,7 +380,7 @@ export class DatabaseDetailsPage extends React.Component< cell: table => !table.details.statsLastUpdated ? "No table statistics found" - : table.details.statsLastUpdated.format("MMM DD, YYYY [at] H:mm"), + : table.details.statsLastUpdated.format(DATE_FORMAT), sort: table => table.details.statsLastUpdated, className: cx("database-table__col--table-stats"), name: "tableStatsUpdated", diff --git a/pkg/ui/workspaces/cluster-ui/src/databaseTablePage/databaseTablePage.tsx b/pkg/ui/workspaces/cluster-ui/src/databaseTablePage/databaseTablePage.tsx index 878e5e01f2e5..4251b0e48857 100644 --- a/pkg/ui/workspaces/cluster-ui/src/databaseTablePage/databaseTablePage.tsx +++ b/pkg/ui/workspaces/cluster-ui/src/databaseTablePage/databaseTablePage.tsx @@ -43,6 +43,7 @@ import classnames from "classnames/bind"; import booleanSettingStyles from "../settings/booleanSetting.module.scss"; import { CircleFilled } from "../icon"; import { performanceTuningRecipes } from "src/util/docs"; +import { DATE_FORMAT_24_UTC } from "src/util/format"; const cx = classNames.bind(styles); const booleanSettingCx = classnames.bind(booleanSettingStyles); @@ -241,9 +242,7 @@ export class DatabaseTablePage extends React.Component< if (lastReset.isSame(this.minDate)) { return "Last reset: Never"; } else { - return ( - "Last reset: " + lastReset.format("MMM DD, YYYY [at] H:mm [(UTC)]") - ); + return "Last reset: " + lastReset.format(DATE_FORMAT_24_UTC); } } diff --git a/pkg/ui/workspaces/cluster-ui/src/graphs/utils/domain.ts b/pkg/ui/workspaces/cluster-ui/src/graphs/utils/domain.ts index 578cc42b2b80..d361723b944a 100644 --- a/pkg/ui/workspaces/cluster-ui/src/graphs/utils/domain.ts +++ b/pkg/ui/workspaces/cluster-ui/src/graphs/utils/domain.ts @@ -15,6 +15,7 @@ import { BytesFitScale, ComputeByteScale, ComputeDurationScale, + DATE_WITH_SECONDS_FORMAT_24_UTC, DurationFitScale, } from "src/util/format"; @@ -264,7 +265,7 @@ const timeIncrements: number[] = timeIncrementDurations.map(inc => ); export function formatTimeStamp(timeMillis: number): string { - return moment.utc(timeMillis).format("HH:mm:ss on MMM Do, YYYY"); + return moment.utc(timeMillis).format(DATE_WITH_SECONDS_FORMAT_24_UTC); } function ComputeTimeAxisDomain(extent: Extent): AxisDomain { diff --git a/pkg/ui/workspaces/cluster-ui/src/indexDetailsPage/indexDetailsPage.tsx b/pkg/ui/workspaces/cluster-ui/src/indexDetailsPage/indexDetailsPage.tsx index c55c4cb9ad7f..d90674ee93ff 100644 --- a/pkg/ui/workspaces/cluster-ui/src/indexDetailsPage/indexDetailsPage.tsx +++ b/pkg/ui/workspaces/cluster-ui/src/indexDetailsPage/indexDetailsPage.tsx @@ -26,7 +26,7 @@ import { SummaryCard } from "../summaryCard"; import moment, { Moment } from "moment"; import { Heading } from "@cockroachlabs/ui-components"; import { Anchor } from "../anchor"; -import { performanceTuningRecipes } from "../util"; +import { DATE_FORMAT_24_UTC, performanceTuningRecipes } from "../util"; const cx = classNames.bind(styles); @@ -127,7 +127,7 @@ export class IndexDetailsPage extends React.Component< if (timestamp.isSame(minDate)) { return "Never"; } else { - return timestamp.format("MMM DD, YYYY [at] H:mm [(UTC)]"); + return timestamp.format(DATE_FORMAT_24_UTC); } } diff --git a/pkg/ui/workspaces/cluster-ui/src/statementDetails/activeStatementDetails.tsx b/pkg/ui/workspaces/cluster-ui/src/statementDetails/activeStatementDetails.tsx index bf8c1e3f1ad2..529ef391e078 100644 --- a/pkg/ui/workspaces/cluster-ui/src/statementDetails/activeStatementDetails.tsx +++ b/pkg/ui/workspaces/cluster-ui/src/statementDetails/activeStatementDetails.tsx @@ -29,6 +29,7 @@ import { StatusIcon } from "src/activeExecutions/statusIcon"; import styles from "./statementDetails.module.scss"; import { SqlBox } from "src/sql/box"; +import { DATE_FORMAT_24_UTC } from "../util"; const cx = classNames.bind(styles); const summaryCardStylesCx = classNames.bind(summaryCardStyles); @@ -96,9 +97,7 @@ export const ActiveStatementDetails: React.FC = ({
Start Time (UTC) - - {statement.start.format("MMM D, YYYY [at] H:mm (UTC)")} - + {statement.start.format(DATE_FORMAT_24_UTC)}
Elapsed Time diff --git a/pkg/ui/workspaces/cluster-ui/src/statementDetails/diagnostics/diagnosticsView.tsx b/pkg/ui/workspaces/cluster-ui/src/statementDetails/diagnostics/diagnosticsView.tsx index 8725e049da0f..0d0c0775ba8e 100644 --- a/pkg/ui/workspaces/cluster-ui/src/statementDetails/diagnostics/diagnosticsView.tsx +++ b/pkg/ui/workspaces/cluster-ui/src/statementDetails/diagnostics/diagnosticsView.tsx @@ -31,6 +31,7 @@ import { import { EmptyTable } from "src/empty"; import styles from "./diagnosticsView.module.scss"; import { getBasePath } from "../../api"; +import { DATE_FORMAT_24_UTC } from "../../util"; type IStatementDiagnosticsReport = cockroach.server.serverpb.IStatementDiagnosticsReport; @@ -127,7 +128,7 @@ export class DiagnosticsView extends React.Component< defaultSortOrder: "descend", render: (_text, record) => { const timestamp = record.requested_at.seconds.toNumber() * 1000; - return moment.utc(timestamp).format("LL[ at ]H:mm"); + return moment.utc(timestamp).format(DATE_FORMAT_24_UTC); }, }, { diff --git a/pkg/ui/workspaces/cluster-ui/src/util/format.ts b/pkg/ui/workspaces/cluster-ui/src/util/format.ts index 60ea3262f2ec..a1a2a7592783 100644 --- a/pkg/ui/workspaces/cluster-ui/src/util/format.ts +++ b/pkg/ui/workspaces/cluster-ui/src/util/format.ts @@ -174,7 +174,8 @@ export const DATE_FORMAT = "MMM DD, YYYY [at] H:mm"; /** * Alternate 24 hour UTC format */ -export const DATE_FORMAT_24_UTC = "MMM DD, YYYY [at] HH:mm UTC"; +export const DATE_FORMAT_24_UTC = "MMM DD, YYYY [at] H:mm UTC"; +export const DATE_WITH_SECONDS_FORMAT_24_UTC = "MMM DD, YYYY [at] H:mm:ss UTC"; export function RenderCount(yesCount: Long, totalCount: Long): string { if (longToInt(yesCount) == 0) { diff --git a/pkg/ui/workspaces/db-console/src/util/format.ts b/pkg/ui/workspaces/db-console/src/util/format.ts index 4fab828335db..717d9f043d39 100644 --- a/pkg/ui/workspaces/db-console/src/util/format.ts +++ b/pkg/ui/workspaces/db-console/src/util/format.ts @@ -156,4 +156,5 @@ export const DATE_FORMAT = "MMM DD, YYYY [at] H:mm"; /** * Alternate 24 hour UTC format */ -export const DATE_FORMAT_24_UTC = "MMM DD, YYYY [at] HH:mm UTC"; +export const DATE_FORMAT_24_UTC = "MMM DD, YYYY [at] H:mm UTC"; +export const DATE_WITH_SECONDS_FORMAT_24_UTC = "MMM DD, YYYY [at] H:mm:ss UTC"; diff --git a/pkg/ui/workspaces/db-console/src/views/cluster/containers/nodeLogs/index.tsx b/pkg/ui/workspaces/db-console/src/views/cluster/containers/nodeLogs/index.tsx index 31b7f626d506..02f4f9d1b352 100644 --- a/pkg/ui/workspaces/db-console/src/views/cluster/containers/nodeLogs/index.tsx +++ b/pkg/ui/workspaces/db-console/src/views/cluster/containers/nodeLogs/index.tsx @@ -26,6 +26,7 @@ import { getDisplayName } from "src/redux/nodes"; import { Loading, SortedTable, util } from "@cockroachlabs/cluster-ui"; import { getMatchParamByName } from "src/util/query"; import "./logs.styl"; +import { DATE_WITH_SECONDS_FORMAT_24_UTC } from "src/util/format"; type LogEntries = protos.cockroach.util.log.IEntry; @@ -55,7 +56,9 @@ export class Logs extends React.Component { title: "Time", name: "time", cell: (logEntry: LogEntries) => - util.LongToMoment(logEntry.time).format("YYYY-MM-DD HH:mm:ss"), + util + .LongToMoment(logEntry.time) + .format(DATE_WITH_SECONDS_FORMAT_24_UTC), }, { title: "Severity", diff --git a/pkg/ui/workspaces/db-console/src/views/cluster/containers/nodesOverview/index.tsx b/pkg/ui/workspaces/db-console/src/views/cluster/containers/nodesOverview/index.tsx index 2f5d905f74c9..d40f40b43d64 100644 --- a/pkg/ui/workspaces/db-console/src/views/cluster/containers/nodesOverview/index.tsx +++ b/pkg/ui/workspaces/db-console/src/views/cluster/containers/nodesOverview/index.tsx @@ -33,7 +33,7 @@ import { SortSetting, util, } from "@cockroachlabs/cluster-ui"; -import { Percentage } from "src/util/format"; +import { DATE_FORMAT_24_UTC, Percentage } from "src/util/format"; import { FixLong } from "src/util/fixLong"; import { getNodeLocalityTiers } from "src/util/localities"; import { LocalityTier } from "src/redux/localities"; @@ -440,7 +440,7 @@ class DecommissionedNodeList extends React.Component - record.decommissionedDate.format("LL[ at ]H:mm UTC"), + record.decommissionedDate.format(DATE_FORMAT_24_UTC), }, { key: "status", diff --git a/pkg/ui/workspaces/db-console/src/views/reports/containers/nodeHistory/decommissionedNodeHistory.tsx b/pkg/ui/workspaces/db-console/src/views/reports/containers/nodeHistory/decommissionedNodeHistory.tsx index 51e3d9090c92..e7538d47beff 100644 --- a/pkg/ui/workspaces/db-console/src/views/reports/containers/nodeHistory/decommissionedNodeHistory.tsx +++ b/pkg/ui/workspaces/db-console/src/views/reports/containers/nodeHistory/decommissionedNodeHistory.tsx @@ -31,6 +31,7 @@ import { util, } from "@cockroachlabs/cluster-ui"; import { createSelector } from "reselect"; +import { DATE_FORMAT_24_UTC } from "src/util/format"; const decommissionedNodesSortSetting = new LocalSetting< AdminUIState, @@ -88,7 +89,7 @@ export class DecommissionedNodeHistory extends React.Component { - return record.decommissionedDate.format("LL[ at ]H:mm UTC"); + return record.decommissionedDate.format(DATE_FORMAT_24_UTC); }, }, ]; diff --git a/pkg/ui/workspaces/db-console/src/views/reports/containers/statementDiagnosticsHistory/index.tsx b/pkg/ui/workspaces/db-console/src/views/reports/containers/statementDiagnosticsHistory/index.tsx index f6b664fb1222..110901e14a69 100644 --- a/pkg/ui/workspaces/db-console/src/views/reports/containers/statementDiagnosticsHistory/index.tsx +++ b/pkg/ui/workspaces/db-console/src/views/reports/containers/statementDiagnosticsHistory/index.tsx @@ -49,6 +49,7 @@ import { } from "@cockroachlabs/cluster-ui"; import { cancelStatementDiagnosticsReportAction } from "src/redux/statements"; import { trackCancelDiagnosticsBundleAction } from "src/redux/analyticsActions"; +import { DATE_FORMAT_24_UTC } from "src/util/format"; type StatementDiagnosticsHistoryViewProps = MapStateToProps & MapDispatchToProps; @@ -98,7 +99,7 @@ class StatementDiagnosticsHistoryView extends React.Component< cell: record => moment .utc(record.requested_at.seconds.toNumber() * 1000) - .format("LL[ at ]H:mm"), + .format(DATE_FORMAT_24_UTC), sort: record => moment(record.requested_at.seconds.toNumber() * 1000), }, { From b3571fe52936af915d89d9e6bbf8aa845dc68db6 Mon Sep 17 00:00:00 2001 From: Eric Harmeling Date: Mon, 27 Jun 2022 11:27:58 -0400 Subject: [PATCH 7/9] ui: improve tooltip UX with text updates Fixes #81374. Fixes #83256. Fixes #81248. Fixes #79018. Release note (ui change): Updated tooltips on the Statements and Transactions pages in the DB Console for improved UX. --- .../src/statsTableUtil/statsTableUtil.tsx | 44 +++++-------------- pkg/ui/workspaces/cluster-ui/src/util/docs.ts | 3 -- pkg/ui/workspaces/db-console/src/util/docs.ts | 3 -- 3 files changed, 12 insertions(+), 38 deletions(-) diff --git a/pkg/ui/workspaces/cluster-ui/src/statsTableUtil/statsTableUtil.tsx b/pkg/ui/workspaces/cluster-ui/src/statsTableUtil/statsTableUtil.tsx index a756925756c7..484444f74331 100644 --- a/pkg/ui/workspaces/cluster-ui/src/statsTableUtil/statsTableUtil.tsx +++ b/pkg/ui/workspaces/cluster-ui/src/statsTableUtil/statsTableUtil.tsx @@ -17,7 +17,6 @@ import { statementDiagnostics, statementsRetries, statementsSql, - statementsTimeInterval, readFromDisk, writtenToDisk, planningExecutionTime, @@ -343,11 +342,7 @@ export const statisticsTableTitles: StatisticTableTitleType = { content={ <>

- {`Cumulative number of executions of ${contentModifier} with this fingerprint${fingerprintModifier} within the last hour or specified `} - - time interval - - .  + {`Cumulative number of executions of ${contentModifier} with this fingerprint${fingerprintModifier} within the specified time interval.`}

{"The bar indicates the ratio of runtime success (gray) to "} @@ -412,11 +407,11 @@ export const statisticsTableTitles: StatisticTableTitleType = { read - {" and "} + {" from and "} written - {` to disk across all operators for ${contentModifier} with this fingerprint${fingerprintModifier} within the specified time interval.`} + {` to disk per execution for ${contentModifier} with this fingerprint${fingerprintModifier} within the specified time interval.`}

} @@ -453,11 +448,7 @@ export const statisticsTableTitles: StatisticTableTitleType = { read from disk - {` across all operators for ${contentModifier} with this fingerprint${fingerprintModifier} within the last hour or specified `} - - time interval - - .  + {` across all operators for ${contentModifier} with this fingerprint${fingerprintModifier} within the specified time interval.`}

The gray bar indicates the mean number of bytes read from disk. @@ -502,7 +493,7 @@ export const statisticsTableTitles: StatisticTableTitleType = { planning and execution time - {` of ${contentModifier} with this fingerprint${fingerprintModifier} within the last hour or specified time interval. `} + {` of ${contentModifier} with this fingerprint${fingerprintModifier} within the specified time interval. `}

The gray bar indicates the mean latency. The blue bar indicates @@ -543,15 +534,12 @@ export const statisticsTableTitles: StatisticTableTitleType = { in contention - {` with other ${contentModifier} within the last hour or specified `} - - time interval - - .  + {` with other ${contentModifier} within the specified time interval.`}

The gray bar indicates mean contention time. The blue bar - indicates one standard deviation from the mean. + indicates one standard deviation from the mean. This time does not + include the time it takes to stream results back to the client.

} @@ -584,11 +572,7 @@ export const statisticsTableTitles: StatisticTableTitleType = { content={ <>

- {`Maximum memory used by a ${contentModifier} with this fingerprint${fingerprintModifier} at any time during its execution within the last hour or specified `} - - time interval - - .  + {`Maximum memory used by a ${contentModifier} with this fingerprint${fingerprintModifier} at any time during its execution within the specified time interval.`}

The gray bar indicates the average max memory usage. The blue bar @@ -629,11 +613,7 @@ export const statisticsTableTitles: StatisticTableTitleType = { data transferred over the network - {` (e.g., between regions and nodes) for ${contentModifier} with this fingerprint${fingerprintModifier} within the last hour or specified `} - - time interval - - .  + {` (e.g., between regions and nodes) for ${contentModifier} with this fingerprint${fingerprintModifier} within the specified time interval.`}

If this value is 0, the statement was executed on a single node. @@ -677,7 +657,7 @@ export const statisticsTableTitles: StatisticTableTitleType = { retries - {` of ${contentModifier} with this fingerprint${fingerprintModifier} within the last hour or specified time interval.`} + {` (including internal and automatic retries) of ${contentModifier} with this fingerprint${fingerprintModifier} within the specified time interval.`}

} @@ -710,7 +690,7 @@ export const statisticsTableTitles: StatisticTableTitleType = {

% of runtime all {contentModifier} with this fingerprint {fingerprintModifier} represent, compared to the cumulative runtime - of all queries within the last hour or specified time interval. + of all queries within the specified time interval.

} > diff --git a/pkg/ui/workspaces/cluster-ui/src/util/docs.ts b/pkg/ui/workspaces/cluster-ui/src/util/docs.ts index 3ca175209b43..cfa4a5f01c3a 100644 --- a/pkg/ui/workspaces/cluster-ui/src/util/docs.ts +++ b/pkg/ui/workspaces/cluster-ui/src/util/docs.ts @@ -57,9 +57,6 @@ export const statementsSql = docsURL( export const statementsRetries = docsURL( "transactions.html#transaction-retries", ); -export const statementsTimeInterval = docsURL( - "ui-statements-page.html#time-interval", -); export const readFromDisk = docsURL( "architecture/life-of-a-distributed-transaction.html#reads-from-the-storage-layer", ); diff --git a/pkg/ui/workspaces/db-console/src/util/docs.ts b/pkg/ui/workspaces/db-console/src/util/docs.ts index 0b7fc8de8d10..21f037259b14 100644 --- a/pkg/ui/workspaces/db-console/src/util/docs.ts +++ b/pkg/ui/workspaces/db-console/src/util/docs.ts @@ -71,9 +71,6 @@ export const statementsRetries = docsURL( export const transactionRetryErrorReference = docsURL( "transaction-retry-error-reference.html", ); -export const statementsTimeInterval = docsURL( - "ui-statements-page.html#time-interval", -); export const capacityMetrics = docsURL( "ui-cluster-overview-page.html#capacity-metrics", ); From 421856bceba5dadf4d635a00ee5dad277d795f6f Mon Sep 17 00:00:00 2001 From: Jason Chan Date: Wed, 15 Jun 2022 07:34:36 -0700 Subject: [PATCH 8/9] sql/schemachanger: implement DROP OWNED BY Previously, we did not support the DROP OWNED BY statement (#55381). This commit adds partial support for DROP OWNED BY in the declarative schema changer. Followup work is needed to support the CASCADE modifier. Release note (sql change): Support `DROP OWNED BY`. --- .../testdata/logic_test/drop_owned_by | 626 ++++++++++++++ .../schemachanger/scbuild/builder_state.go | 14 + pkg/sql/schemachanger/scbuild/dependencies.go | 4 + .../scbuild/internal/scbuildstmt/BUILD.bazel | 3 + .../internal/scbuildstmt/dependencies.go | 8 + .../internal/scbuildstmt/drop_owned_by.go | 106 +++ .../scbuild/internal/scbuildstmt/process.go | 1 + .../scbuild/testdata/drop_owned_by | 798 ++++++++++++++++++ pkg/sql/schemachanger/scdeps/build_deps.go | 5 + .../scdeps/sctestdeps/test_deps.go | 12 + .../scexec/scmutationexec/BUILD.bazel | 1 + .../scexec/scmutationexec/drop.go | 14 + pkg/sql/schemachanger/scop/mutation.go | 7 + .../scop/mutation_visitor_generated.go | 6 + .../internal/opgen/opgen_user_privileges.go | 5 +- .../scplan/testdata/drop_owned_by | 753 +++++++++++++++++ pkg/sql/sem/tree/schema_feature_name.go | 2 +- pkg/sql/testdata/telemetry/drop_owned_by | 2 - 18 files changed, 2363 insertions(+), 4 deletions(-) create mode 100644 pkg/sql/logictest/testdata/logic_test/drop_owned_by create mode 100644 pkg/sql/schemachanger/scbuild/internal/scbuildstmt/drop_owned_by.go create mode 100644 pkg/sql/schemachanger/scbuild/testdata/drop_owned_by create mode 100644 pkg/sql/schemachanger/scplan/testdata/drop_owned_by diff --git a/pkg/sql/logictest/testdata/logic_test/drop_owned_by b/pkg/sql/logictest/testdata/logic_test/drop_owned_by new file mode 100644 index 000000000000..b490053ce535 --- /dev/null +++ b/pkg/sql/logictest/testdata/logic_test/drop_owned_by @@ -0,0 +1,626 @@ +# LogicTest: !local-legacy-schema-changer + +# Test dropping nothing. +statement ok +DROP OWNED BY testuser + +statement ok +CREATE USER testuser2 + +# DROP-OBJECTS: Test that DROP OWNED BY drops objects owned by the specified +# roles. +# +# In this test, testuser creates multiple objects and drops all of them in one +# go. Additionally, testuser2 owns a table that shouldn't be dropped by +# testuser's DROP OWNED BY. +subtest drop-objects + +user testuser2 + +statement ok +CREATE TABLE u() + +user root + +statement ok +GRANT CREATE ON DATABASE test TO testuser WITH GRANT OPTION + +user testuser + +statement ok +CREATE TABLE t(a INT) + +statement ok +CREATE VIEW v AS SELECT 1 + +statement ok +CREATE SEQUENCE seq + +statement ok +CREATE TYPE enum AS ENUM('a', 'b') + +query TTTTIT +SHOW TABLES FROM public +---- +public seq sequence testuser 0 NULL +public t table testuser 0 NULL +public u table testuser2 0 NULL +public v view testuser 0 NULL + +query TTTT +SHOW ENUMS +---- +public enum {a,b} testuser + +statement ok +DROP OWNED BY testuser + +query TTTTIT +SHOW TABLES FROM public +---- +public u table testuser2 0 NULL + +query error pgcode 42P01 relation "t" does not exist +SELECT * FROM t + +query error pgcode 42P01 relation "v" does not exist +SELECT * FROM v + +query TTTT +SHOW ENUMS +---- + +user testuser2 + +statement ok +DROP OWNED BY testuser2 + +query TTTTIT +SHOW TABLES FROM public +---- + +# DROP-BEHAVIOR-VIEW: Test RESTRICT/CASCADE. +# +# In this test, testuser2 creates a view dependent on a table owned by +# testuser. Under RESTRICT, testuser cannot DROP OWNED BY due to this +# dependency. Under CASCADE, testuser can DROP OWNED BY, which drops both +# testuser's table and testuser2's view. +subtest drop-behavior-view + +user testuser + +statement ok +CREATE TABLE t(a INT) + +statement ok +GRANT SELECT ON t TO testuser2 WITH GRANT OPTION + +user testuser2 + +statement ok +CREATE VIEW v AS SELECT a FROM t + +user testuser + +statement error pq: cannot drop desired object\(s\) because other objects depend on them +DROP OWNED BY testuser + +statement error pq: cannot drop desired object\(s\) because other objects depend on them +DROP OWNED BY testuser RESTRICT + +query TTTTIT +SHOW TABLES FROM public +---- +public t table testuser 0 NULL +public v view testuser2 0 NULL + +user root + +statement error unimplemented: DROP OWNED BY CASCADE is not yet supported +DROP OWNED BY testuser2 CASCADE + +statement ok +DROP OWNED BY testuser, testuser2 + +query TTTTIT +SHOW TABLES FROM public +---- + +# DROP-BEHAVIOR-TYPE-1: Test RESTRICT behavior by trying to drop a table +# dependent on a type owned by another role. +subtest drop-behavior-type-1 + +user root + +statement ok +GRANT CREATE ON DATABASE test TO testuser WITH GRANT OPTION + +user testuser + +statement ok +CREATE TYPE type AS ENUM ('hello') + +statement ok +GRANT USAGE ON TYPE type TO testuser2 + +user testuser2 + +statement ok +CREATE TABLE t(x type) + +user root + +statement error pq: cannot drop desired object\(s\) because other objects depend on them +DROP OWNED BY testuser + +statement ok +DROP OWNED BY testuser, testuser2 + +query TTTTIT +SHOW TABLES FROM public +---- + +query TTT +SHOW TYPES +---- + +# DROP-BEHAVIOR-TYPE-2: Test RESTRICT behavior by trying to drop a view +# dependent on a type owned by another role. +subtest drop-behavior-type-2 + +user root + +statement ok +GRANT CREATE ON DATABASE test TO testuser WITH GRANT OPTION + +user testuser + +statement ok +CREATE TYPE type AS ENUM ('hello') + +statement ok +GRANT USAGE ON TYPE type TO testuser2 + +user testuser2 + +statement ok +CREATE TABLE t(a int) + +statement ok +CREATE VIEW v AS SELECT a, 'hello'::type FROM t + +user root + +statement error pq: cannot drop desired object\(s\) because other objects depend on them +DROP OWNED BY testuser + +statement ok +DROP OWNED BY testuser, testuser2 + +query TTTTIT +SHOW TABLES FROM public +---- + +query TTT +SHOW TYPES +---- + +# DROP-SCHEMA: Test that schemas and the objects that they contain can all be +# dropped together by a single DROP OWNED BY (when they are all owned by the +# specified roles). +subtest drop-schema + +user root + +statement ok +GRANT ALL ON DATABASE test TO testuser WITH GRANT OPTION + +user testuser + +statement ok +CREATE SCHEMA s + +statement ok +CREATE TABLE s.t1() + +statement ok +CREATE TABLE s.t2() + +statement ok +DROP OWNED BY testuser + +statement error pq: target database or schema does not exist +SHOW TABLES FROM s + +user root + +# REVOKE-PRIVILEGES-DB: Test that DROP OWNED BY revokes privileges on the +# current database. +# +# The DROP OWNED BY from the previous subtest did not revoke testuser's +# privileges for the DATABASE. This is because a user should not revoke its own +# database privileges. However, the root user should be able to drop testuser's +# database privileges via DROP OWNED BY. +subtest revoke-privileges-db + +query TTTB +SHOW GRANTS ON DATABASE test +---- +test admin ALL true +test public CONNECT false +test root ALL true +test testuser ALL true + +user root + +statement ok +DROP OWNED BY testuser + +query TTTB +SHOW GRANTS ON DATABASE test +---- +test admin ALL true +test public CONNECT false +test root ALL true + +# REVOKE-PRIVILEGES-SCHEMA: Test that DROP OWNED BY revokes privileges on +# schemas in the current database. +# +# In this test, root creates a schema and grants privileges for the schema to +# testuser. When testuser issues a DROP OWNED BY, those privileges should be +# revoked. +subtest revoke-privileges-schema + +user root + +statement ok +CREATE SCHEMA s + +statement ok +GRANT CREATE ON SCHEMA s TO testuser WITH GRANT OPTION + +user testuser + +statement ok +CREATE TABLE s.t() + +statement ok +DROP OWNED BY testuser + +query TTTTB +SHOW GRANTS ON SCHEMA s +---- +test s admin ALL true +test s root ALL true + +query TTTTIT +SHOW TABLES FROM s +---- + +user root + +statement ok +DROP SCHEMA s + +# REVOKE-PRIVILEGES-TABLE: Test that DROP OWNED BY revokes privileges on +# objects in the database. +subtest revoke-privileges-table + +user root + +statement ok +CREATE TABLE t() + +statement ok +GRANT ALL ON t TO testuser WITH GRANT OPTION + +user testuser + +query TTTTTB +SHOW GRANTS ON t +---- +test public t admin ALL true +test public t root ALL true +test public t testuser ALL true + +statement ok +DROP OWNED BY testuser + +query TTTTTB +SHOW GRANTS ON t +---- +test public t admin ALL true +test public t root ALL true + +user root + +statement ok +DROP TABLE t + +# MUTIROLE: Test DROP OWNED BY with multiple roles. +subtest multirole + +statement ok +CREATE ROLE r1 + +statement ok +CREATE ROLE r2 + +statement ok +SET ROLE r1 + +statement ok +CREATE TABLE t1() + +statement ok +SET ROLE r2 + +statement ok +CREATE TABLE t2() + +statement ok +SET ROLE root + +query TTTTIT +SHOW TABLES FROM public +---- +public t1 table r1 0 NULL +public t2 table r2 0 NULL + +statement ok +DROP OWNED BY r1, r2 + +query TTTTIT +SHOW TABLES FROM public +---- + +# ROLES: Test that the current user is a member of all the specified roles. The +# admin role and the root user are off-limits. +subtest roles + +user testuser + +statement error pq: permission denied to drop objects +DROP OWNED BY testuser2 + +statement error pq: permission denied to drop objects +DROP OWNED BY testuser, testuser2 + +statement error pq: cannot drop objects owned by role "root" because they are required by the database system +DROP OWNED BY root + +statement error pq: cannot drop objects owned by role "admin" because they are required by the database system +DROP OWNED BY admin + +# KITCHEN-SINK: Test DROP OWNED BY when there are multiple databases/schemas. +# Only objects/privileges in the current database should be dropped. +subtest kitchen-sink + +user root + +statement ok +CREATE DATABASE d1 + +statement ok +CREATE DATABASE d2 + +statement ok +CREATE DATABASE d3 + +statement ok +CREATE DATABASE d4 + +statement ok +CREATE SCHEMA d1.s1 + +statement ok +CREATE SCHEMA d1.s2 + +statement ok +GRANT CREATE, DROP ON DATABASE d1 TO testuser WITH GRANT OPTION + +statement ok +GRANT ALL ON DATABASE d2 TO testuser WITH GRANT OPTION + +statement ok +GRANT CREATE ON DATABASE d3 TO testuser WITH GRANT OPTION + +statement ok +GRANT CREATE ON SCHEMA d1.s1 TO testuser WITH GRANT OPTION + +statement ok +CREATE TABLE d1.t1 (k STRING PRIMARY KEY, v STRING) + +statement ok +CREATE VIEW d1.v1 AS SELECT k,v FROM d1.t1 + +statement ok +CREATE TABLE d1.s1.t1 (a INT) + +statement ok +CREATE SCHEMA d2.s1 + +user testuser + +statement ok +CREATE SCHEMA d1.s3 + +statement ok +CREATE SCHEMA d1.s4 + +statement ok +CREATE TABLE d1.t2 (k STRING PRIMARY KEY, v STRING) + +statement ok +CREATE VIEW d1.v2 AS SELECT k,v FROM d1.t2 + +statement ok +CREATE TABLE d1.s1.t2 (a INT) + +statement ok +CREATE SCHEMA d2.s2 + +statement ok +CREATE TABLE d2.t1() + +user root + +query TTTTT +SHOW DATABASES +---- +d1 root NULL {} NULL +d2 root NULL {} NULL +d3 root NULL {} NULL +d4 root NULL {} NULL +defaultdb root NULL {} NULL +postgres root NULL {} NULL +system node NULL {} NULL +test root NULL {} NULL + +statement ok +SET DATABASE = d1 + +query TT +SHOW SCHEMAS FROM d1 +---- +crdb_internal NULL +information_schema NULL +pg_catalog NULL +pg_extension NULL +public admin +s1 root +s2 root +s3 testuser +s4 testuser + +query TTTTIT +SHOW TABLES FROM d1 +---- +public t1 table root 0 NULL +public t2 table testuser 0 NULL +public v1 view root 0 NULL +public v2 view testuser 0 NULL +s1 t1 table root 0 NULL +s1 t2 table testuser 0 NULL + +query TTTB +SHOW GRANTS ON DATABASE d1 +---- +d1 admin ALL true +d1 public CONNECT false +d1 root ALL true +d1 testuser CREATE true +d1 testuser DROP true + +query TTTTB +SHOW GRANTS ON SCHEMA d1.s1 +---- +d1 s1 admin ALL true +d1 s1 root ALL true +d1 s1 testuser CREATE true + +statement ok +DROP OWNED BY testuser + +query TT +SHOW SCHEMAS FROM d1 +---- +crdb_internal NULL +information_schema NULL +pg_catalog NULL +pg_extension NULL +public admin +s1 root +s2 root + +query TTTTIT +SHOW TABLES FROM d1 +---- +public t1 table root 0 NULL +public v1 view root 0 NULL +s1 t1 table root 0 NULL + +query TTTTB +SHOW GRANTS ON SCHEMA d1.s1 +---- +d1 s1 admin ALL true +d1 s1 root ALL true + +query TTTB +SHOW GRANTS ON DATABASE d1 +---- +d1 admin ALL true +d1 public CONNECT false +d1 root ALL true + +statement ok +SET DATABASE = d2 + +query TT +SHOW SCHEMAS FROM d2 +---- +crdb_internal NULL +information_schema NULL +pg_catalog NULL +pg_extension NULL +public admin +s1 root +s2 testuser + +query TTTTIT +SHOW TABLES FROM d2 +---- +public t1 table testuser 0 NULL + +query TTTB +SHOW GRANTS ON DATABASE d2 +---- +d2 admin ALL true +d2 public CONNECT false +d2 root ALL true +d2 testuser ALL true + +statement ok +DROP OWNED BY testuser + +query TT +SHOW SCHEMAS FROM d2 +---- +crdb_internal NULL +information_schema NULL +pg_catalog NULL +pg_extension NULL +public admin +s1 root + +query TTTTIT +SHOW TABLES FROM d2 +---- + +query TTTB +SHOW GRANTS ON DATABASE d2 +---- +d2 admin ALL true +d2 public CONNECT false +d2 root ALL true + +statement ok +SET DATABASE = d3 + +query TTTB +SHOW GRANTS ON DATABASE d3 +---- +d3 admin ALL true +d3 public CONNECT false +d3 root ALL true +d3 testuser CREATE true + +statement ok +DROP OWNED BY testuser + +query TTTB +SHOW GRANTS ON DATABASE d3 +---- +d3 admin ALL true +d3 public CONNECT false +d3 root ALL true diff --git a/pkg/sql/schemachanger/scbuild/builder_state.go b/pkg/sql/schemachanger/scbuild/builder_state.go index 8c116bab70c8..8b01519e9b19 100644 --- a/pkg/sql/schemachanger/scbuild/builder_state.go +++ b/pkg/sql/schemachanger/scbuild/builder_state.go @@ -15,6 +15,7 @@ import ( "strings" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/security/username" "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/catpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" @@ -156,6 +157,19 @@ func (b *builderState) checkPrivilege(id catid.DescID, priv privilege.Kind) { } } +// CurrentUserHasAdminOrIsMemberOf implements the scbuildstmt.PrivilegeChecker interface. +func (b *builderState) CurrentUserHasAdminOrIsMemberOf(role username.SQLUsername) bool { + if b.hasAdmin { + return true + } + memberships, err := b.auth.MemberOfWithAdminOption(b.ctx, role) + if err != nil { + panic(err) + } + _, ok := memberships[b.evalCtx.SessionData().User()] + return ok +} + var _ scbuildstmt.TableHelpers = (*builderState)(nil) // NextTableColumnID implements the scbuildstmt.TableHelpers interface. diff --git a/pkg/sql/schemachanger/scbuild/dependencies.go b/pkg/sql/schemachanger/scbuild/dependencies.go index 7dc21187c2e5..489ae9950008 100644 --- a/pkg/sql/schemachanger/scbuild/dependencies.go +++ b/pkg/sql/schemachanger/scbuild/dependencies.go @@ -166,6 +166,10 @@ type AuthorizationAccessor interface { CheckPrivilegeForUser( ctx context.Context, privilegeObject catalog.PrivilegeObject, privilege privilege.Kind, user username.SQLUsername, ) error + + // MemberOfWithAdminOption looks up all the roles 'member' belongs to (direct + // and indirect) and returns a map of "role" -> "isAdmin". + MemberOfWithAdminOption(ctx context.Context, member username.SQLUsername) (map[username.SQLUsername]bool, error) } // AstFormatter provides interfaces for formatting AST nodes. diff --git a/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/BUILD.bazel b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/BUILD.bazel index 397dca826e5a..1c6e2e1f1cc4 100644 --- a/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/BUILD.bazel +++ b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/BUILD.bazel @@ -9,6 +9,7 @@ go_library( "create_index.go", "dependencies.go", "drop_database.go", + "drop_owned_by.go", "drop_schema.go", "drop_sequence.go", "drop_table.go", @@ -20,6 +21,7 @@ go_library( importpath = "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scbuild/internal/scbuildstmt", visibility = ["//pkg/sql/schemachanger/scbuild:__subpackages__"], deps = [ + "//pkg/security/username", "//pkg/server/telemetry", "//pkg/settings/cluster", "//pkg/sql/catalog", @@ -29,6 +31,7 @@ go_library( "//pkg/sql/catalog/schemaexpr", "//pkg/sql/catalog/tabledesc", "//pkg/sql/catalog/typedesc", + "//pkg/sql/decodeusername", "//pkg/sql/parser", "//pkg/sql/pgwire/pgcode", "//pkg/sql/pgwire/pgerror", diff --git a/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/dependencies.go b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/dependencies.go index e4f136168f8a..095b38b16998 100644 --- a/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/dependencies.go +++ b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/dependencies.go @@ -13,6 +13,7 @@ package scbuildstmt import ( "context" + "github.com/cockroachdb/cockroach/pkg/security/username" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql/catalog/catpb" "github.com/cockroachdb/cockroach/pkg/sql/privilege" @@ -148,6 +149,9 @@ type Telemetry interface { // IncrementEnumCounter increments the selected enum telemetry counter. IncrementEnumCounter(counterType sqltelemetry.EnumTelemetryType) + + // IncrementDropOwnedByCounter increments the DROP OWNED BY telemetry counter. + IncrementDropOwnedByCounter() } // SchemaFeatureChecker checks if a schema change feature is allowed by the @@ -167,6 +171,10 @@ type PrivilegeChecker interface { // CheckPrivilege panics if the current user does not have the specified // privilege for the element. CheckPrivilege(e scpb.Element, privilege privilege.Kind) + + // CurrentUserHasAdminOrIsMemberOf returns true iff the current user is (1) + // an admin or (2) has membership in the specified role. + CurrentUserHasAdminOrIsMemberOf(member username.SQLUsername) bool } // TableHelpers has methods useful for creating new table elements. diff --git a/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/drop_owned_by.go b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/drop_owned_by.go new file mode 100644 index 000000000000..8e2971de0914 --- /dev/null +++ b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/drop_owned_by.go @@ -0,0 +1,106 @@ +// Copyright 2022 The Cockroach Authors. +// +// Use of this software is governed by the Business Source License +// included in the file licenses/BSL.txt. +// +// As of the Change Date specified in that file, in accordance with +// the Business Source License, use of this software will be governed +// by the Apache License, Version 2.0, included in the file +// licenses/APL.txt. + +package scbuildstmt + +import ( + "github.com/cockroachdb/cockroach/pkg/security/username" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + "github.com/cockroachdb/cockroach/pkg/sql/decodeusername" + "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode" + "github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror" + "github.com/cockroachdb/cockroach/pkg/sql/privilege" + "github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scpb" + "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" + "github.com/cockroachdb/cockroach/pkg/util/errorutil/unimplemented" +) + +// DropOwnedBy implements DROP OWNED BY. +func DropOwnedBy(b BuildCtx, n *tree.DropOwnedBy) { + normalizedRoles, err := decodeusername.FromRoleSpecList( + b.SessionData(), username.PurposeValidation, n.Roles, + ) + if err != nil { + panic(err) + } + for _, role := range normalizedRoles { + if role.IsAdminRole() || role.IsRootUser() || role.IsNodeUser() { + panic(pgerror.Newf(pgcode.DependentObjectsStillExist, + "cannot drop objects owned by role %q because they are required by the database system", role)) + } + if role != b.SessionData().User() && !b.CurrentUserHasAdminOrIsMemberOf(role) { + panic(pgerror.New(pgcode.InsufficientPrivilege, "permission denied to drop objects")) + } + } + + var objects []descpb.ID + var toCheckBackrefs []descpb.ID + + // Lookup all objects in the current database. + _, _, db := scpb.FindDatabase(b.ResolveDatabase(tree.Name(b.SessionData().Database), ResolveParams{ + IsExistenceOptional: false, + RequiredPrivilege: privilege.CONNECT, + })) + dbRefs := undroppedBackrefs(b, db.DatabaseID) + scpb.ForEachSchemaParent(dbRefs, func(_ scpb.Status, _ scpb.TargetStatus, sp *scpb.SchemaParent) { + schemaRefs := undroppedBackrefs(b, sp.SchemaID) + scpb.ForEachObjectParent(schemaRefs, func(_ scpb.Status, _ scpb.TargetStatus, op *scpb.ObjectParent) { + objects = append(objects, op.ObjectID) + }) + objects = append(objects, sp.SchemaID) + }) + + // Drop owned objects and revoke user privileges for the specified roles. + for _, id := range objects { + elts := b.QueryByID(id) + _, _, owner := scpb.FindOwner(elts) + for _, role := range normalizedRoles { + if owner.Owner == role.Normalized() { + if n.DropBehavior == tree.DropCascade { + panic(unimplemented.NewWithIssue(55908, "DROP OWNED BY CASCADE is not yet supported")) + } else { + if dropRestrictDescriptor(b, id) { + toCheckBackrefs = append(toCheckBackrefs, id) + } + } + break + } + scpb.ForEachUserPrivileges(elts, func(_ scpb.Status, _ scpb.TargetStatus, e *scpb.UserPrivileges) { + if e.UserName == role.Normalized() { + b.Drop(e) + } + }) + } + } + + // Revoke privileges for the database. The current user shouldn't revoke + // their own database privileges. + dbElts := b.QueryByID(db.DatabaseID) + scpb.ForEachUserPrivileges(dbElts, func(_ scpb.Status, _ scpb.TargetStatus, e *scpb.UserPrivileges) { + for _, role := range normalizedRoles { + if e.UserName == role.Normalized() && e.UserName != b.SessionData().User().Normalized() { + b.Drop(e) + break + } + } + }) + + b.IncrementSubWorkID() + b.IncrementDropOwnedByCounter() + + // Enforce RESTRICT semantics by checking for backreferences. + for _, id := range toCheckBackrefs { + backrefs := undroppedBackrefs(b, id) + if !backrefs.IsEmpty() { + panic(pgerror.New(pgcode.DependentObjectsStillExist, + "cannot drop desired object(s) because other objects depend on them")) + } + } +} diff --git a/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/process.go b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/process.go index 196410ebd1d6..5ca420776a44 100644 --- a/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/process.go +++ b/pkg/sql/schemachanger/scbuild/internal/scbuildstmt/process.go @@ -49,6 +49,7 @@ var supportedStatements = map[reflect.Type]supportedStatement{ reflect.TypeOf((*tree.AlterTable)(nil)): {AlterTable, true}, reflect.TypeOf((*tree.CreateIndex)(nil)): {CreateIndex, false}, reflect.TypeOf((*tree.DropDatabase)(nil)): {DropDatabase, true}, + reflect.TypeOf((*tree.DropOwnedBy)(nil)): {DropOwnedBy, true}, reflect.TypeOf((*tree.DropSchema)(nil)): {DropSchema, true}, reflect.TypeOf((*tree.DropSequence)(nil)): {DropSequence, true}, reflect.TypeOf((*tree.DropTable)(nil)): {DropTable, true}, diff --git a/pkg/sql/schemachanger/scbuild/testdata/drop_owned_by b/pkg/sql/schemachanger/scbuild/testdata/drop_owned_by new file mode 100644 index 000000000000..2478bde4ed15 --- /dev/null +++ b/pkg/sql/schemachanger/scbuild/testdata/drop_owned_by @@ -0,0 +1,798 @@ +setup +CREATE ROLE r; +CREATE TABLE tab(); +GRANT ALL ON tab TO r; +GRANT CREATE ON DATABASE defaultdb TO r WITH GRANT OPTION; +SET ROLE r; +CREATE SCHEMA s; +CREATE SEQUENCE public.sq; +CREATE SEQUENCE s.sq; +CREATE TABLE s.t (id INT PRIMARY KEY, name VARCHAR(256), val INT DEFAULT nextval('s.sq')); +CREATE TABLE public.t (id INT PRIMARY KEY, name VARCHAR(256), val INT DEFAULT nextval('public.sq')); +CREATE VIEW s.v1 AS (SELECT name FROM s.t); +CREATE TYPE s.typ AS ENUM('a'); +CREATE VIEW s.v2 AS (SELECT 'a'::s.typ::string AS k, name FROM s.v1); +SET ROLE root; +---- + +build +DROP OWNED BY r +---- +- [[AliasType:{DescID: 112}, ABSENT], PUBLIC] + details: + embeddedTypeT: + closedTypeIds: + - 111 + - 112 + type: + arrayContents: + family: EnumFamily + oid: 100111 + udtMetadata: + arrayTypeOid: 100112 + arrayElemType: EnumFamily + family: ArrayFamily + oid: 100112 + typeId: 112 +- [[Column:{DescID: 108, ColumnID: 1}, ABSENT], PUBLIC] + details: + columnId: 1 + pgAttributeNum: 1 + tableId: 108 +- [[Column:{DescID: 108, ColumnID: 2}, ABSENT], PUBLIC] + details: + columnId: 2 + pgAttributeNum: 2 + tableId: 108 +- [[Column:{DescID: 108, ColumnID: 3}, ABSENT], PUBLIC] + details: + columnId: 3 + pgAttributeNum: 3 + tableId: 108 +- [[Column:{DescID: 108, ColumnID: 4294967294}, ABSENT], PUBLIC] + details: + columnId: 4.294967294e+09 + isHidden: true + isSystemColumn: true + pgAttributeNum: 4.294967294e+09 + tableId: 108 +- [[Column:{DescID: 108, ColumnID: 4294967295}, ABSENT], PUBLIC] + details: + columnId: 4.294967295e+09 + isHidden: true + isSystemColumn: true + pgAttributeNum: 4.294967295e+09 + tableId: 108 +- [[Column:{DescID: 109, ColumnID: 1}, ABSENT], PUBLIC] + details: + columnId: 1 + pgAttributeNum: 1 + tableId: 109 +- [[Column:{DescID: 109, ColumnID: 2}, ABSENT], PUBLIC] + details: + columnId: 2 + pgAttributeNum: 2 + tableId: 109 +- [[Column:{DescID: 109, ColumnID: 3}, ABSENT], PUBLIC] + details: + columnId: 3 + pgAttributeNum: 3 + tableId: 109 +- [[Column:{DescID: 109, ColumnID: 4294967294}, ABSENT], PUBLIC] + details: + columnId: 4.294967294e+09 + isHidden: true + isSystemColumn: true + pgAttributeNum: 4.294967294e+09 + tableId: 109 +- [[Column:{DescID: 109, ColumnID: 4294967295}, ABSENT], PUBLIC] + details: + columnId: 4.294967295e+09 + isHidden: true + isSystemColumn: true + pgAttributeNum: 4.294967295e+09 + tableId: 109 +- [[Column:{DescID: 110, ColumnID: 1}, ABSENT], PUBLIC] + details: + columnId: 1 + pgAttributeNum: 1 + tableId: 110 +- [[Column:{DescID: 110, ColumnID: 4294967294}, ABSENT], PUBLIC] + details: + columnId: 4.294967294e+09 + isHidden: true + isSystemColumn: true + pgAttributeNum: 4.294967294e+09 + tableId: 110 +- [[Column:{DescID: 110, ColumnID: 4294967295}, ABSENT], PUBLIC] + details: + columnId: 4.294967295e+09 + isHidden: true + isSystemColumn: true + pgAttributeNum: 4.294967295e+09 + tableId: 110 +- [[Column:{DescID: 113, ColumnID: 1}, ABSENT], PUBLIC] + details: + columnId: 1 + pgAttributeNum: 1 + tableId: 113 +- [[Column:{DescID: 113, ColumnID: 2}, ABSENT], PUBLIC] + details: + columnId: 2 + pgAttributeNum: 2 + tableId: 113 +- [[Column:{DescID: 113, ColumnID: 4294967294}, ABSENT], PUBLIC] + details: + columnId: 4.294967294e+09 + isHidden: true + isSystemColumn: true + pgAttributeNum: 4.294967294e+09 + tableId: 113 +- [[Column:{DescID: 113, ColumnID: 4294967295}, ABSENT], PUBLIC] + details: + columnId: 4.294967295e+09 + isHidden: true + isSystemColumn: true + pgAttributeNum: 4.294967295e+09 + tableId: 113 +- [[ColumnDefaultExpression:{DescID: 108, ColumnID: 3}, ABSENT], PUBLIC] + details: + columnId: 3 + embeddedExpr: + expr: nextval(107:::REGCLASS) + usesSequenceIds: + - 107 + tableId: 108 +- [[ColumnDefaultExpression:{DescID: 109, ColumnID: 3}, ABSENT], PUBLIC] + details: + columnId: 3 + embeddedExpr: + expr: nextval(106:::REGCLASS) + usesSequenceIds: + - 106 + tableId: 109 +- [[ColumnFamily:{DescID: 108, ColumnFamilyID: 0, Name: primary}, ABSENT], PUBLIC] + details: + name: primary + tableId: 108 +- [[ColumnFamily:{DescID: 109, ColumnFamilyID: 0, Name: primary}, ABSENT], PUBLIC] + details: + name: primary + tableId: 109 +- [[ColumnName:{DescID: 108, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT], PUBLIC] + details: + columnId: 4.294967295e+09 + name: crdb_internal_mvcc_timestamp + tableId: 108 +- [[ColumnName:{DescID: 108, Name: id, ColumnID: 1}, ABSENT], PUBLIC] + details: + columnId: 1 + name: id + tableId: 108 +- [[ColumnName:{DescID: 108, Name: name, ColumnID: 2}, ABSENT], PUBLIC] + details: + columnId: 2 + name: name + tableId: 108 +- [[ColumnName:{DescID: 108, Name: tableoid, ColumnID: 4294967294}, ABSENT], PUBLIC] + details: + columnId: 4.294967294e+09 + name: tableoid + tableId: 108 +- [[ColumnName:{DescID: 108, Name: val, ColumnID: 3}, ABSENT], PUBLIC] + details: + columnId: 3 + name: val + tableId: 108 +- [[ColumnName:{DescID: 109, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT], PUBLIC] + details: + columnId: 4.294967295e+09 + name: crdb_internal_mvcc_timestamp + tableId: 109 +- [[ColumnName:{DescID: 109, Name: id, ColumnID: 1}, ABSENT], PUBLIC] + details: + columnId: 1 + name: id + tableId: 109 +- [[ColumnName:{DescID: 109, Name: name, ColumnID: 2}, ABSENT], PUBLIC] + details: + columnId: 2 + name: name + tableId: 109 +- [[ColumnName:{DescID: 109, Name: tableoid, ColumnID: 4294967294}, ABSENT], PUBLIC] + details: + columnId: 4.294967294e+09 + name: tableoid + tableId: 109 +- [[ColumnName:{DescID: 109, Name: val, ColumnID: 3}, ABSENT], PUBLIC] + details: + columnId: 3 + name: val + tableId: 109 +- [[ColumnName:{DescID: 110, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT], PUBLIC] + details: + columnId: 4.294967295e+09 + name: crdb_internal_mvcc_timestamp + tableId: 110 +- [[ColumnName:{DescID: 110, Name: name, ColumnID: 1}, ABSENT], PUBLIC] + details: + columnId: 1 + name: name + tableId: 110 +- [[ColumnName:{DescID: 110, Name: tableoid, ColumnID: 4294967294}, ABSENT], PUBLIC] + details: + columnId: 4.294967294e+09 + name: tableoid + tableId: 110 +- [[ColumnName:{DescID: 113, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT], PUBLIC] + details: + columnId: 4.294967295e+09 + name: crdb_internal_mvcc_timestamp + tableId: 113 +- [[ColumnName:{DescID: 113, Name: k, ColumnID: 1}, ABSENT], PUBLIC] + details: + columnId: 1 + name: k + tableId: 113 +- [[ColumnName:{DescID: 113, Name: name, ColumnID: 2}, ABSENT], PUBLIC] + details: + columnId: 2 + name: name + tableId: 113 +- [[ColumnName:{DescID: 113, Name: tableoid, ColumnID: 4294967294}, ABSENT], PUBLIC] + details: + columnId: 4.294967294e+09 + name: tableoid + tableId: 113 +- [[ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 1}, ABSENT], PUBLIC] + details: + columnId: 1 + embeddedTypeT: + type: + family: IntFamily + oid: 20 + width: 64 + isRelationBeingDropped: true + tableId: 108 +- [[ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 2}, ABSENT], PUBLIC] + details: + columnId: 2 + embeddedTypeT: + type: + family: StringFamily + oid: 1043 + visibleType: 7 + width: 256 + isNullable: true + isRelationBeingDropped: true + tableId: 108 +- [[ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 3}, ABSENT], PUBLIC] + details: + columnId: 3 + embeddedTypeT: + type: + family: IntFamily + oid: 20 + width: 64 + isNullable: true + isRelationBeingDropped: true + tableId: 108 +- [[ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT], PUBLIC] + details: + columnId: 4.294967294e+09 + embeddedTypeT: + type: + family: OidFamily + oid: 26 + isNullable: true + isRelationBeingDropped: true + tableId: 108 +- [[ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT], PUBLIC] + details: + columnId: 4.294967295e+09 + embeddedTypeT: + type: + family: DecimalFamily + oid: 1700 + isNullable: true + isRelationBeingDropped: true + tableId: 108 +- [[ColumnType:{DescID: 109, ColumnFamilyID: 0, ColumnID: 1}, ABSENT], PUBLIC] + details: + columnId: 1 + embeddedTypeT: + type: + family: IntFamily + oid: 20 + width: 64 + isRelationBeingDropped: true + tableId: 109 +- [[ColumnType:{DescID: 109, ColumnFamilyID: 0, ColumnID: 2}, ABSENT], PUBLIC] + details: + columnId: 2 + embeddedTypeT: + type: + family: StringFamily + oid: 1043 + visibleType: 7 + width: 256 + isNullable: true + isRelationBeingDropped: true + tableId: 109 +- [[ColumnType:{DescID: 109, ColumnFamilyID: 0, ColumnID: 3}, ABSENT], PUBLIC] + details: + columnId: 3 + embeddedTypeT: + type: + family: IntFamily + oid: 20 + width: 64 + isNullable: true + isRelationBeingDropped: true + tableId: 109 +- [[ColumnType:{DescID: 109, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT], PUBLIC] + details: + columnId: 4.294967294e+09 + embeddedTypeT: + type: + family: OidFamily + oid: 26 + isNullable: true + isRelationBeingDropped: true + tableId: 109 +- [[ColumnType:{DescID: 109, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT], PUBLIC] + details: + columnId: 4.294967295e+09 + embeddedTypeT: + type: + family: DecimalFamily + oid: 1700 + isNullable: true + isRelationBeingDropped: true + tableId: 109 +- [[ColumnType:{DescID: 110, ColumnFamilyID: 0, ColumnID: 1}, ABSENT], PUBLIC] + details: + columnId: 1 + embeddedTypeT: + type: + family: StringFamily + oid: 1043 + visibleType: 7 + width: 256 + isNullable: true + isRelationBeingDropped: true + tableId: 110 +- [[ColumnType:{DescID: 110, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT], PUBLIC] + details: + columnId: 4.294967294e+09 + embeddedTypeT: + type: + family: OidFamily + oid: 26 + isNullable: true + isRelationBeingDropped: true + tableId: 110 +- [[ColumnType:{DescID: 110, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT], PUBLIC] + details: + columnId: 4.294967295e+09 + embeddedTypeT: + type: + family: DecimalFamily + oid: 1700 + isNullable: true + isRelationBeingDropped: true + tableId: 110 +- [[ColumnType:{DescID: 113, ColumnFamilyID: 0, ColumnID: 1}, ABSENT], PUBLIC] + details: + columnId: 1 + embeddedTypeT: + type: + family: StringFamily + oid: 25 + isNullable: true + isRelationBeingDropped: true + tableId: 113 +- [[ColumnType:{DescID: 113, ColumnFamilyID: 0, ColumnID: 2}, ABSENT], PUBLIC] + details: + columnId: 2 + embeddedTypeT: + type: + family: StringFamily + oid: 1043 + visibleType: 7 + width: 256 + isNullable: true + isRelationBeingDropped: true + tableId: 113 +- [[ColumnType:{DescID: 113, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT], PUBLIC] + details: + columnId: 4.294967294e+09 + embeddedTypeT: + type: + family: OidFamily + oid: 26 + isNullable: true + isRelationBeingDropped: true + tableId: 113 +- [[ColumnType:{DescID: 113, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT], PUBLIC] + details: + columnId: 4.294967295e+09 + embeddedTypeT: + type: + family: DecimalFamily + oid: 1700 + isNullable: true + isRelationBeingDropped: true + tableId: 113 +- [[EnumType:{DescID: 111}, ABSENT], PUBLIC] + details: + arrayTypeId: 112 + typeId: 111 +- [[IndexColumn:{DescID: 108, ColumnID: 1, IndexID: 1}, ABSENT], PUBLIC] + details: + columnId: 1 + indexId: 1 + tableId: 108 +- [[IndexColumn:{DescID: 108, ColumnID: 2, IndexID: 1}, ABSENT], PUBLIC] + details: + columnId: 2 + indexId: 1 + kind: STORED + tableId: 108 +- [[IndexColumn:{DescID: 108, ColumnID: 3, IndexID: 1}, ABSENT], PUBLIC] + details: + columnId: 3 + indexId: 1 + kind: STORED + ordinalInKind: 1 + tableId: 108 +- [[IndexColumn:{DescID: 109, ColumnID: 1, IndexID: 1}, ABSENT], PUBLIC] + details: + columnId: 1 + indexId: 1 + tableId: 109 +- [[IndexColumn:{DescID: 109, ColumnID: 2, IndexID: 1}, ABSENT], PUBLIC] + details: + columnId: 2 + indexId: 1 + kind: STORED + tableId: 109 +- [[IndexColumn:{DescID: 109, ColumnID: 3, IndexID: 1}, ABSENT], PUBLIC] + details: + columnId: 3 + indexId: 1 + kind: STORED + ordinalInKind: 1 + tableId: 109 +- [[IndexName:{DescID: 108, Name: t_pkey, IndexID: 1}, ABSENT], PUBLIC] + details: + indexId: 1 + name: t_pkey + tableId: 108 +- [[IndexName:{DescID: 109, Name: t_pkey, IndexID: 1}, ABSENT], PUBLIC] + details: + indexId: 1 + name: t_pkey + tableId: 109 +- [[Namespace:{DescID: 105, Name: s, ReferencedDescID: 100}, ABSENT], PUBLIC] + details: + databaseId: 100 + descriptorId: 105 + name: s +- [[Namespace:{DescID: 106, Name: sq, ReferencedDescID: 100}, ABSENT], PUBLIC] + details: + databaseId: 100 + descriptorId: 106 + name: sq + schemaId: 101 +- [[Namespace:{DescID: 107, Name: sq, ReferencedDescID: 100}, ABSENT], PUBLIC] + details: + databaseId: 100 + descriptorId: 107 + name: sq + schemaId: 105 +- [[Namespace:{DescID: 108, Name: t, ReferencedDescID: 100}, ABSENT], PUBLIC] + details: + databaseId: 100 + descriptorId: 108 + name: t + schemaId: 105 +- [[Namespace:{DescID: 109, Name: t, ReferencedDescID: 100}, ABSENT], PUBLIC] + details: + databaseId: 100 + descriptorId: 109 + name: t + schemaId: 101 +- [[Namespace:{DescID: 110, Name: v1, ReferencedDescID: 100}, ABSENT], PUBLIC] + details: + databaseId: 100 + descriptorId: 110 + name: v1 + schemaId: 105 +- [[Namespace:{DescID: 111, Name: typ, ReferencedDescID: 100}, ABSENT], PUBLIC] + details: + databaseId: 100 + descriptorId: 111 + name: typ + schemaId: 105 +- [[Namespace:{DescID: 112, Name: _typ, ReferencedDescID: 100}, ABSENT], PUBLIC] + details: + databaseId: 100 + descriptorId: 112 + name: _typ + schemaId: 105 +- [[Namespace:{DescID: 113, Name: v2, ReferencedDescID: 100}, ABSENT], PUBLIC] + details: + databaseId: 100 + descriptorId: 113 + name: v2 + schemaId: 105 +- [[ObjectParent:{DescID: 106, ReferencedDescID: 101}, ABSENT], PUBLIC] + details: + objectId: 106 + parentSchemaId: 101 +- [[ObjectParent:{DescID: 107, ReferencedDescID: 105}, ABSENT], PUBLIC] + details: + objectId: 107 + parentSchemaId: 105 +- [[ObjectParent:{DescID: 108, ReferencedDescID: 105}, ABSENT], PUBLIC] + details: + objectId: 108 + parentSchemaId: 105 +- [[ObjectParent:{DescID: 109, ReferencedDescID: 101}, ABSENT], PUBLIC] + details: + objectId: 109 + parentSchemaId: 101 +- [[ObjectParent:{DescID: 110, ReferencedDescID: 105}, ABSENT], PUBLIC] + details: + objectId: 110 + parentSchemaId: 105 +- [[ObjectParent:{DescID: 111, ReferencedDescID: 105}, ABSENT], PUBLIC] + details: + objectId: 111 + parentSchemaId: 105 +- [[ObjectParent:{DescID: 112, ReferencedDescID: 105}, ABSENT], PUBLIC] + details: + objectId: 112 + parentSchemaId: 105 +- [[ObjectParent:{DescID: 113, ReferencedDescID: 105}, ABSENT], PUBLIC] + details: + objectId: 113 + parentSchemaId: 105 +- [[Owner:{DescID: 105}, ABSENT], PUBLIC] + details: + descriptorId: 105 + owner: r +- [[Owner:{DescID: 106}, ABSENT], PUBLIC] + details: + descriptorId: 106 + owner: r +- [[Owner:{DescID: 107}, ABSENT], PUBLIC] + details: + descriptorId: 107 + owner: r +- [[Owner:{DescID: 108}, ABSENT], PUBLIC] + details: + descriptorId: 108 + owner: r +- [[Owner:{DescID: 109}, ABSENT], PUBLIC] + details: + descriptorId: 109 + owner: r +- [[Owner:{DescID: 110}, ABSENT], PUBLIC] + details: + descriptorId: 110 + owner: r +- [[Owner:{DescID: 111}, ABSENT], PUBLIC] + details: + descriptorId: 111 + owner: r +- [[Owner:{DescID: 112}, ABSENT], PUBLIC] + details: + descriptorId: 112 + owner: r +- [[Owner:{DescID: 113}, ABSENT], PUBLIC] + details: + descriptorId: 113 + owner: r +- [[PrimaryIndex:{DescID: 108, IndexID: 1, ConstraintID: 1}, ABSENT], PUBLIC] + details: + embeddedIndex: + constraintId: 1 + indexId: 1 + isUnique: true + tableId: 108 +- [[PrimaryIndex:{DescID: 109, IndexID: 1, ConstraintID: 1}, ABSENT], PUBLIC] + details: + embeddedIndex: + constraintId: 1 + indexId: 1 + isUnique: true + tableId: 109 +- [[Schema:{DescID: 105}, ABSENT], PUBLIC] + details: + schemaId: 105 +- [[SchemaParent:{DescID: 105, ReferencedDescID: 100}, ABSENT], PUBLIC] + details: + parentDatabaseId: 100 + schemaId: 105 +- [[Sequence:{DescID: 106}, ABSENT], PUBLIC] + details: + sequenceId: 106 +- [[Sequence:{DescID: 107}, ABSENT], PUBLIC] + details: + sequenceId: 107 +- [[Table:{DescID: 108}, ABSENT], PUBLIC] + details: + tableId: 108 +- [[Table:{DescID: 109}, ABSENT], PUBLIC] + details: + tableId: 109 +- [[UserPrivileges:{DescID: 100, Name: r}, ABSENT], PUBLIC] + details: + descriptorId: 100 + privileges: 4 + userName: r +- [[UserPrivileges:{DescID: 104, Name: r}, ABSENT], PUBLIC] + details: + descriptorId: 104 + privileges: 2 + userName: r +- [[UserPrivileges:{DescID: 105, Name: admin}, ABSENT], PUBLIC] + details: + descriptorId: 105 + privileges: 2 + userName: admin +- [[UserPrivileges:{DescID: 105, Name: root}, ABSENT], PUBLIC] + details: + descriptorId: 105 + privileges: 2 + userName: root +- [[UserPrivileges:{DescID: 105, Name: r}, ABSENT], PUBLIC] + details: + descriptorId: 105 + privileges: 2 + userName: r +- [[UserPrivileges:{DescID: 106, Name: admin}, ABSENT], PUBLIC] + details: + descriptorId: 106 + privileges: 2 + userName: admin +- [[UserPrivileges:{DescID: 106, Name: root}, ABSENT], PUBLIC] + details: + descriptorId: 106 + privileges: 2 + userName: root +- [[UserPrivileges:{DescID: 106, Name: r}, ABSENT], PUBLIC] + details: + descriptorId: 106 + privileges: 2 + userName: r +- [[UserPrivileges:{DescID: 107, Name: admin}, ABSENT], PUBLIC] + details: + descriptorId: 107 + privileges: 2 + userName: admin +- [[UserPrivileges:{DescID: 107, Name: root}, ABSENT], PUBLIC] + details: + descriptorId: 107 + privileges: 2 + userName: root +- [[UserPrivileges:{DescID: 107, Name: r}, ABSENT], PUBLIC] + details: + descriptorId: 107 + privileges: 2 + userName: r +- [[UserPrivileges:{DescID: 108, Name: admin}, ABSENT], PUBLIC] + details: + descriptorId: 108 + privileges: 2 + userName: admin +- [[UserPrivileges:{DescID: 108, Name: root}, ABSENT], PUBLIC] + details: + descriptorId: 108 + privileges: 2 + userName: root +- [[UserPrivileges:{DescID: 108, Name: r}, ABSENT], PUBLIC] + details: + descriptorId: 108 + privileges: 2 + userName: r +- [[UserPrivileges:{DescID: 109, Name: admin}, ABSENT], PUBLIC] + details: + descriptorId: 109 + privileges: 2 + userName: admin +- [[UserPrivileges:{DescID: 109, Name: root}, ABSENT], PUBLIC] + details: + descriptorId: 109 + privileges: 2 + userName: root +- [[UserPrivileges:{DescID: 109, Name: r}, ABSENT], PUBLIC] + details: + descriptorId: 109 + privileges: 2 + userName: r +- [[UserPrivileges:{DescID: 110, Name: admin}, ABSENT], PUBLIC] + details: + descriptorId: 110 + privileges: 2 + userName: admin +- [[UserPrivileges:{DescID: 110, Name: root}, ABSENT], PUBLIC] + details: + descriptorId: 110 + privileges: 2 + userName: root +- [[UserPrivileges:{DescID: 110, Name: r}, ABSENT], PUBLIC] + details: + descriptorId: 110 + privileges: 2 + userName: r +- [[UserPrivileges:{DescID: 111, Name: admin}, ABSENT], PUBLIC] + details: + descriptorId: 111 + privileges: 2 + userName: admin +- [[UserPrivileges:{DescID: 111, Name: public}, ABSENT], PUBLIC] + details: + descriptorId: 111 + privileges: 512 + userName: public +- [[UserPrivileges:{DescID: 111, Name: root}, ABSENT], PUBLIC] + details: + descriptorId: 111 + privileges: 2 + userName: root +- [[UserPrivileges:{DescID: 111, Name: r}, ABSENT], PUBLIC] + details: + descriptorId: 111 + privileges: 2 + userName: r +- [[UserPrivileges:{DescID: 112, Name: admin}, ABSENT], PUBLIC] + details: + descriptorId: 112 + privileges: 2 + userName: admin +- [[UserPrivileges:{DescID: 112, Name: public}, ABSENT], PUBLIC] + details: + descriptorId: 112 + privileges: 512 + userName: public +- [[UserPrivileges:{DescID: 112, Name: root}, ABSENT], PUBLIC] + details: + descriptorId: 112 + privileges: 2 + userName: root +- [[UserPrivileges:{DescID: 112, Name: r}, ABSENT], PUBLIC] + details: + descriptorId: 112 + privileges: 2 + userName: r +- [[UserPrivileges:{DescID: 113, Name: admin}, ABSENT], PUBLIC] + details: + descriptorId: 113 + privileges: 2 + userName: admin +- [[UserPrivileges:{DescID: 113, Name: root}, ABSENT], PUBLIC] + details: + descriptorId: 113 + privileges: 2 + userName: root +- [[UserPrivileges:{DescID: 113, Name: r}, ABSENT], PUBLIC] + details: + descriptorId: 113 + privileges: 2 + userName: r +- [[View:{DescID: 110}, ABSENT], PUBLIC] + details: + usesRelationIds: + - 108 + viewId: 110 +- [[View:{DescID: 113}, ABSENT], PUBLIC] + details: + usesRelationIds: + - 110 + usesTypeIds: + - 111 + - 112 + viewId: 113 diff --git a/pkg/sql/schemachanger/scdeps/build_deps.go b/pkg/sql/schemachanger/scdeps/build_deps.go index a885f0347b0e..aec9ebf7f62e 100644 --- a/pkg/sql/schemachanger/scdeps/build_deps.go +++ b/pkg/sql/schemachanger/scdeps/build_deps.go @@ -375,6 +375,11 @@ func (d *buildDeps) IncrementEnumCounter(counterType sqltelemetry.EnumTelemetryT sqltelemetry.IncrementEnumCounter(counterType) } +// IncrementDropOwnedByCounter implements the scbuild.Dependencies interface. +func (d *buildDeps) IncrementDropOwnedByCounter() { + telemetry.Inc(sqltelemetry.CreateDropOwnedByCounter()) +} + func (d *buildDeps) DescriptorCommentCache() scbuild.CommentCache { return descmetadata.NewCommentCache(d.txn, d.internalExecutor) } diff --git a/pkg/sql/schemachanger/scdeps/sctestdeps/test_deps.go b/pkg/sql/schemachanger/scdeps/sctestdeps/test_deps.go index b5d6ded54aed..3856b1ef702b 100644 --- a/pkg/sql/schemachanger/scdeps/sctestdeps/test_deps.go +++ b/pkg/sql/schemachanger/scdeps/sctestdeps/test_deps.go @@ -127,6 +127,11 @@ func (s *TestState) IncrementEnumCounter(counterType sqltelemetry.EnumTelemetryT s.LogSideEffectf("increment telemetry for sql.udts.%s", counterType) } +// IncrementDropOwnedByCounter implements the scbuild.Dependencies interface. +func (s *TestState) IncrementDropOwnedByCounter() { + s.LogSideEffectf("increment telemetry for sql.drop_owned_by") +} + var _ scbuild.AuthorizationAccessor = (*TestState)(nil) // CheckPrivilege implements the scbuild.AuthorizationAccessor interface. @@ -158,6 +163,13 @@ func (s *TestState) CheckPrivilegeForUser( return nil } +// MemberOfWithAdminOption implements the scbuild.AuthorizationAccessor interface. +func (s *TestState) MemberOfWithAdminOption( + ctx context.Context, member username.SQLUsername, +) (map[username.SQLUsername]bool, error) { + return nil, nil +} + // IndexPartitioningCCLCallback implements the scbuild.Dependencies interface. func (s *TestState) IndexPartitioningCCLCallback() scbuild.CreatePartitioningCCLCallback { if ccl := scdeps.CreatePartitioningCCL; ccl != nil { diff --git a/pkg/sql/schemachanger/scexec/scmutationexec/BUILD.bazel b/pkg/sql/schemachanger/scexec/scmutationexec/BUILD.bazel index 737da7bbf3a1..8128c3994e4c 100644 --- a/pkg/sql/schemachanger/scexec/scmutationexec/BUILD.bazel +++ b/pkg/sql/schemachanger/scexec/scmutationexec/BUILD.bazel @@ -20,6 +20,7 @@ go_library( deps = [ "//pkg/jobs/jobspb", "//pkg/keys", + "//pkg/security/username", "//pkg/sql/catalog", "//pkg/sql/catalog/catpb", "//pkg/sql/catalog/colinfo", diff --git a/pkg/sql/schemachanger/scexec/scmutationexec/drop.go b/pkg/sql/schemachanger/scexec/scmutationexec/drop.go index 4156c24c4c59..053959a33d17 100644 --- a/pkg/sql/schemachanger/scexec/scmutationexec/drop.go +++ b/pkg/sql/schemachanger/scexec/scmutationexec/drop.go @@ -14,6 +14,7 @@ import ( "context" "github.com/cockroachdb/cockroach/pkg/keys" + "github.com/cockroachdb/cockroach/pkg/security/username" "github.com/cockroachdb/cockroach/pkg/sql/catalog" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" "github.com/cockroachdb/cockroach/pkg/sql/catalog/tabledesc" @@ -132,6 +133,19 @@ func (m *visitor) RemoveDatabaseRoleSettings( return m.s.DeleteDatabaseRoleSettings(ctx, op.DatabaseID) } +func (m *visitor) RemoveUserPrivileges(ctx context.Context, op scop.RemoveUserPrivileges) error { + desc, err := m.s.CheckOutDescriptor(ctx, op.DescID) + if err != nil { + return err + } + user, err := username.MakeSQLUsernameFromUserInput(op.User, username.PurposeValidation) + if err != nil { + return err + } + desc.GetPrivileges().RemoveUser(user) + return nil +} + func (m *visitor) DeleteSchedule(_ context.Context, op scop.DeleteSchedule) error { if op.ScheduleID != 0 { m.s.DeleteSchedule(op.ScheduleID) diff --git a/pkg/sql/schemachanger/scop/mutation.go b/pkg/sql/schemachanger/scop/mutation.go index e3888f7b8aa3..b5c79c9884af 100644 --- a/pkg/sql/schemachanger/scop/mutation.go +++ b/pkg/sql/schemachanger/scop/mutation.go @@ -530,6 +530,13 @@ type RemoveDatabaseRoleSettings struct { DatabaseID descpb.ID } +// RemoveUserPrivileges is used to revoke a user's privileges. +type RemoveUserPrivileges struct { + mutationOp + DescID descpb.ID + User string +} + // DeleteSchedule is used to delete a schedule ID from the database. type DeleteSchedule struct { mutationOp diff --git a/pkg/sql/schemachanger/scop/mutation_visitor_generated.go b/pkg/sql/schemachanger/scop/mutation_visitor_generated.go index 1807a0715561..4c569d7e6011 100644 --- a/pkg/sql/schemachanger/scop/mutation_visitor_generated.go +++ b/pkg/sql/schemachanger/scop/mutation_visitor_generated.go @@ -86,6 +86,7 @@ type MutationVisitor interface { UpsertConstraintComment(context.Context, UpsertConstraintComment) error RemoveConstraintComment(context.Context, RemoveConstraintComment) error RemoveDatabaseRoleSettings(context.Context, RemoveDatabaseRoleSettings) error + RemoveUserPrivileges(context.Context, RemoveUserPrivileges) error DeleteSchedule(context.Context, DeleteSchedule) error RefreshStats(context.Context, RefreshStats) error AddColumnToIndex(context.Context, AddColumnToIndex) error @@ -412,6 +413,11 @@ func (op RemoveDatabaseRoleSettings) Visit(ctx context.Context, v MutationVisito return v.RemoveDatabaseRoleSettings(ctx, op) } +// Visit is part of the MutationOp interface. +func (op RemoveUserPrivileges) Visit(ctx context.Context, v MutationVisitor) error { + return v.RemoveUserPrivileges(ctx, op) +} + // Visit is part of the MutationOp interface. func (op DeleteSchedule) Visit(ctx context.Context, v MutationVisitor) error { return v.DeleteSchedule(ctx, op) diff --git a/pkg/sql/schemachanger/scplan/internal/opgen/opgen_user_privileges.go b/pkg/sql/schemachanger/scplan/internal/opgen/opgen_user_privileges.go index f71f7ea9429c..ddbdcd9d6cd5 100644 --- a/pkg/sql/schemachanger/scplan/internal/opgen/opgen_user_privileges.go +++ b/pkg/sql/schemachanger/scplan/internal/opgen/opgen_user_privileges.go @@ -31,7 +31,10 @@ func init() { // TODO(postamar): remove revertibility constraint when possible revertible(false), emit(func(this *scpb.UserPrivileges) scop.Op { - return notImplemented(this) + return &scop.RemoveUserPrivileges{ + DescID: this.DescriptorID, + User: this.UserName, + } }), ), ), diff --git a/pkg/sql/schemachanger/scplan/testdata/drop_owned_by b/pkg/sql/schemachanger/scplan/testdata/drop_owned_by new file mode 100644 index 000000000000..d1097de4cc21 --- /dev/null +++ b/pkg/sql/schemachanger/scplan/testdata/drop_owned_by @@ -0,0 +1,753 @@ +setup +CREATE ROLE r; +CREATE TABLE tab(); +GRANT ALL ON tab TO r; +GRANT CREATE ON DATABASE defaultdb TO r WITH GRANT OPTION; +SET ROLE r; +CREATE SCHEMA s; +CREATE SEQUENCE public.sq; +CREATE SEQUENCE s.sq; +CREATE TABLE s.t (id INT PRIMARY KEY, name VARCHAR(256), val INT DEFAULT nextval('s.sq')); +CREATE TABLE public.t (id INT PRIMARY KEY, name VARCHAR(256), val INT DEFAULT nextval('public.sq')); +CREATE VIEW s.v1 AS (SELECT name FROM s.t); +CREATE TYPE s.typ AS ENUM('a'); +CREATE VIEW s.v2 AS (SELECT 'a'::s.typ::string AS k, name FROM s.v1); +---- + +ops +DROP OWNED BY r +---- +StatementPhase stage 1 of 1 with 34 MutationType ops + transitions: + [[UserPrivileges:{DescID: 100, Name: r}, ABSENT], PUBLIC] -> ABSENT + [[Namespace:{DescID: 105, Name: s, ReferencedDescID: 100}, ABSENT], PUBLIC] -> ABSENT + [[Owner:{DescID: 105}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 105, Name: admin}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 105, Name: r}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 105, Name: root}, ABSENT], PUBLIC] -> ABSENT + [[Schema:{DescID: 105}, ABSENT], PUBLIC] -> DROPPED + [[SchemaParent:{DescID: 105, ReferencedDescID: 100}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 104, Name: r}, ABSENT], PUBLIC] -> ABSENT + [[Namespace:{DescID: 106, Name: sq, ReferencedDescID: 100}, ABSENT], PUBLIC] -> ABSENT + [[Owner:{DescID: 106}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 106, Name: admin}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 106, Name: r}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 106, Name: root}, ABSENT], PUBLIC] -> ABSENT + [[Sequence:{DescID: 106}, ABSENT], PUBLIC] -> DROPPED + [[ObjectParent:{DescID: 106, ReferencedDescID: 101}, ABSENT], PUBLIC] -> ABSENT + [[Namespace:{DescID: 109, Name: t, ReferencedDescID: 100}, ABSENT], PUBLIC] -> ABSENT + [[Owner:{DescID: 109}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 109, Name: admin}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 109, Name: r}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 109, Name: root}, ABSENT], PUBLIC] -> ABSENT + [[Table:{DescID: 109}, ABSENT], PUBLIC] -> DROPPED + [[ObjectParent:{DescID: 109, ReferencedDescID: 101}, ABSENT], PUBLIC] -> ABSENT + [[ColumnFamily:{DescID: 109, ColumnFamilyID: 0, Name: primary}, ABSENT], PUBLIC] -> ABSENT + [[Column:{DescID: 109, ColumnID: 1}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 109, Name: id, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 109, ColumnFamilyID: 0, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT + [[Column:{DescID: 109, ColumnID: 2}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 109, Name: name, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 109, ColumnFamilyID: 0, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT + [[Column:{DescID: 109, ColumnID: 3}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 109, Name: val, ColumnID: 3}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 109, ColumnFamilyID: 0, ColumnID: 3}, ABSENT], PUBLIC] -> ABSENT + [[ColumnDefaultExpression:{DescID: 109, ColumnID: 3}, ABSENT], PUBLIC] -> ABSENT + [[Column:{DescID: 109, ColumnID: 4294967295}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 109, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 109, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT + [[Column:{DescID: 109, ColumnID: 4294967294}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 109, Name: tableoid, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 109, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT + [[IndexColumn:{DescID: 109, ColumnID: 1, IndexID: 1}, ABSENT], PUBLIC] -> ABSENT + [[IndexColumn:{DescID: 109, ColumnID: 2, IndexID: 1}, ABSENT], PUBLIC] -> ABSENT + [[IndexColumn:{DescID: 109, ColumnID: 3, IndexID: 1}, ABSENT], PUBLIC] -> ABSENT + [[PrimaryIndex:{DescID: 109, IndexID: 1, ConstraintID: 1}, ABSENT], PUBLIC] -> VALIDATED + [[IndexName:{DescID: 109, Name: t_pkey, IndexID: 1}, ABSENT], PUBLIC] -> ABSENT + [[Namespace:{DescID: 107, Name: sq, ReferencedDescID: 100}, ABSENT], PUBLIC] -> ABSENT + [[Owner:{DescID: 107}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 107, Name: admin}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 107, Name: r}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 107, Name: root}, ABSENT], PUBLIC] -> ABSENT + [[Sequence:{DescID: 107}, ABSENT], PUBLIC] -> DROPPED + [[ObjectParent:{DescID: 107, ReferencedDescID: 105}, ABSENT], PUBLIC] -> ABSENT + [[Namespace:{DescID: 108, Name: t, ReferencedDescID: 100}, ABSENT], PUBLIC] -> ABSENT + [[Owner:{DescID: 108}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 108, Name: admin}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 108, Name: r}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 108, Name: root}, ABSENT], PUBLIC] -> ABSENT + [[Table:{DescID: 108}, ABSENT], PUBLIC] -> DROPPED + [[ObjectParent:{DescID: 108, ReferencedDescID: 105}, ABSENT], PUBLIC] -> ABSENT + [[ColumnFamily:{DescID: 108, ColumnFamilyID: 0, Name: primary}, ABSENT], PUBLIC] -> ABSENT + [[Column:{DescID: 108, ColumnID: 1}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 108, Name: id, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT + [[Column:{DescID: 108, ColumnID: 2}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 108, Name: name, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT + [[Column:{DescID: 108, ColumnID: 3}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 108, Name: val, ColumnID: 3}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 3}, ABSENT], PUBLIC] -> ABSENT + [[ColumnDefaultExpression:{DescID: 108, ColumnID: 3}, ABSENT], PUBLIC] -> ABSENT + [[Column:{DescID: 108, ColumnID: 4294967295}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 108, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT + [[Column:{DescID: 108, ColumnID: 4294967294}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 108, Name: tableoid, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 108, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT + [[IndexColumn:{DescID: 108, ColumnID: 1, IndexID: 1}, ABSENT], PUBLIC] -> ABSENT + [[IndexColumn:{DescID: 108, ColumnID: 2, IndexID: 1}, ABSENT], PUBLIC] -> ABSENT + [[IndexColumn:{DescID: 108, ColumnID: 3, IndexID: 1}, ABSENT], PUBLIC] -> ABSENT + [[PrimaryIndex:{DescID: 108, IndexID: 1, ConstraintID: 1}, ABSENT], PUBLIC] -> VALIDATED + [[IndexName:{DescID: 108, Name: t_pkey, IndexID: 1}, ABSENT], PUBLIC] -> ABSENT + [[Namespace:{DescID: 110, Name: v1, ReferencedDescID: 100}, ABSENT], PUBLIC] -> ABSENT + [[Owner:{DescID: 110}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 110, Name: admin}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 110, Name: r}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 110, Name: root}, ABSENT], PUBLIC] -> ABSENT + [[View:{DescID: 110}, ABSENT], PUBLIC] -> DROPPED + [[ObjectParent:{DescID: 110, ReferencedDescID: 105}, ABSENT], PUBLIC] -> ABSENT + [[Column:{DescID: 110, ColumnID: 1}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 110, Name: name, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 110, ColumnFamilyID: 0, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT + [[Column:{DescID: 110, ColumnID: 4294967295}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 110, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 110, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT + [[Column:{DescID: 110, ColumnID: 4294967294}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 110, Name: tableoid, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 110, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT + [[Namespace:{DescID: 111, Name: typ, ReferencedDescID: 100}, ABSENT], PUBLIC] -> ABSENT + [[Owner:{DescID: 111}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 111, Name: admin}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 111, Name: public}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 111, Name: r}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 111, Name: root}, ABSENT], PUBLIC] -> ABSENT + [[EnumType:{DescID: 111}, ABSENT], PUBLIC] -> DROPPED + [[ObjectParent:{DescID: 111, ReferencedDescID: 105}, ABSENT], PUBLIC] -> ABSENT + [[Namespace:{DescID: 112, Name: _typ, ReferencedDescID: 100}, ABSENT], PUBLIC] -> ABSENT + [[Owner:{DescID: 112}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 112, Name: admin}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 112, Name: public}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 112, Name: r}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 112, Name: root}, ABSENT], PUBLIC] -> ABSENT + [[AliasType:{DescID: 112}, ABSENT], PUBLIC] -> DROPPED + [[ObjectParent:{DescID: 112, ReferencedDescID: 105}, ABSENT], PUBLIC] -> ABSENT + [[Namespace:{DescID: 113, Name: v2, ReferencedDescID: 100}, ABSENT], PUBLIC] -> ABSENT + [[Owner:{DescID: 113}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 113, Name: admin}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 113, Name: r}, ABSENT], PUBLIC] -> ABSENT + [[UserPrivileges:{DescID: 113, Name: root}, ABSENT], PUBLIC] -> ABSENT + [[View:{DescID: 113}, ABSENT], PUBLIC] -> DROPPED + [[ObjectParent:{DescID: 113, ReferencedDescID: 105}, ABSENT], PUBLIC] -> ABSENT + [[Column:{DescID: 113, ColumnID: 1}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 113, Name: k, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 113, ColumnFamilyID: 0, ColumnID: 1}, ABSENT], PUBLIC] -> ABSENT + [[Column:{DescID: 113, ColumnID: 2}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 113, Name: name, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 113, ColumnFamilyID: 0, ColumnID: 2}, ABSENT], PUBLIC] -> ABSENT + [[Column:{DescID: 113, ColumnID: 4294967295}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 113, Name: crdb_internal_mvcc_timestamp, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 113, ColumnFamilyID: 0, ColumnID: 4294967295}, ABSENT], PUBLIC] -> ABSENT + [[Column:{DescID: 113, ColumnID: 4294967294}, ABSENT], PUBLIC] -> WRITE_ONLY + [[ColumnName:{DescID: 113, Name: tableoid, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT + [[ColumnType:{DescID: 113, ColumnFamilyID: 0, ColumnID: 4294967294}, ABSENT], PUBLIC] -> ABSENT + ops: + *scop.RemoveUserPrivileges + DescID: 100 + User: r + *scop.RemoveUserPrivileges + DescID: 104 + User: r + *scop.MarkDescriptorAsDropped + DescID: 105 + *scop.RemoveSchemaParent + Parent: + ParentDatabaseID: 100 + SchemaID: 105 + *scop.MarkDescriptorAsDropped + DescID: 106 + *scop.RemoveAllTableComments + TableID: 106 + *scop.MarkDescriptorAsDropped + DescID: 109 + *scop.RemoveAllTableComments + TableID: 109 + *scop.RemoveColumnDefaultExpression + ColumnID: 3 + TableID: 109 + *scop.UpdateBackReferencesInSequences + BackReferencedColumnID: 3 + BackReferencedTableID: 109 + SequenceIDs: + - 106 + *scop.MarkDescriptorAsDropped + DescID: 107 + *scop.RemoveAllTableComments + TableID: 107 + *scop.MarkDescriptorAsDropped + DescID: 108 + *scop.RemoveAllTableComments + TableID: 108 + *scop.RemoveColumnDefaultExpression + ColumnID: 3 + TableID: 108 + *scop.UpdateBackReferencesInSequences + BackReferencedColumnID: 3 + BackReferencedTableID: 108 + SequenceIDs: + - 107 + *scop.MarkDescriptorAsDropped + DescID: 110 + *scop.RemoveViewBackReferencesInRelations + BackReferencedViewID: 110 + RelationIDs: + - 108 + *scop.RemoveAllTableComments + TableID: 110 + *scop.MarkDescriptorAsDropped + DescID: 111 + *scop.MarkDescriptorAsDropped + DescID: 112 + *scop.MarkDescriptorAsDropped + DescID: 113 + *scop.RemoveBackReferenceInTypes + BackReferencedDescID: 113 + TypeIDs: + - 111 + - 112 + *scop.RemoveViewBackReferencesInRelations + BackReferencedViewID: 113 + RelationIDs: + - 110 + *scop.RemoveAllTableComments + TableID: 113 + *scop.DrainDescriptorName + Namespace: + DatabaseID: 100 + DescriptorID: 105 + Name: s + *scop.DrainDescriptorName + Namespace: + DatabaseID: 100 + DescriptorID: 106 + Name: sq + SchemaID: 101 + *scop.DrainDescriptorName + Namespace: + DatabaseID: 100 + DescriptorID: 109 + Name: t + SchemaID: 101 + *scop.DrainDescriptorName + Namespace: + DatabaseID: 100 + DescriptorID: 107 + Name: sq + SchemaID: 105 + *scop.DrainDescriptorName + Namespace: + DatabaseID: 100 + DescriptorID: 108 + Name: t + SchemaID: 105 + *scop.DrainDescriptorName + Namespace: + DatabaseID: 100 + DescriptorID: 110 + Name: v1 + SchemaID: 105 + *scop.DrainDescriptorName + Namespace: + DatabaseID: 100 + DescriptorID: 111 + Name: typ + SchemaID: 105 + *scop.DrainDescriptorName + Namespace: + DatabaseID: 100 + DescriptorID: 112 + Name: _typ + SchemaID: 105 + *scop.DrainDescriptorName + Namespace: + DatabaseID: 100 + DescriptorID: 113 + Name: v2 + SchemaID: 105 +PreCommitPhase stage 1 of 1 with 12 MutationType ops + transitions: + ops: + *scop.SetJobStateOnDescriptor + DescriptorID: 100 + Initialize: true + *scop.SetJobStateOnDescriptor + DescriptorID: 104 + Initialize: true + *scop.SetJobStateOnDescriptor + DescriptorID: 105 + Initialize: true + *scop.SetJobStateOnDescriptor + DescriptorID: 106 + Initialize: true + *scop.SetJobStateOnDescriptor + DescriptorID: 107 + Initialize: true + *scop.SetJobStateOnDescriptor + DescriptorID: 108 + Initialize: true + *scop.SetJobStateOnDescriptor + DescriptorID: 109 + Initialize: true + *scop.SetJobStateOnDescriptor + DescriptorID: 110 + Initialize: true + *scop.SetJobStateOnDescriptor + DescriptorID: 111 + Initialize: true + *scop.SetJobStateOnDescriptor + DescriptorID: 112 + Initialize: true + *scop.SetJobStateOnDescriptor + DescriptorID: 113 + Initialize: true + *scop.CreateSchemaChangerJob + Authorization: + UserName: root + DescriptorIDs: + - 100 + - 104 + - 105 + - 106 + - 107 + - 108 + - 109 + - 110 + - 111 + - 112 + - 113 + JobID: 1 + NonCancelable: true + RunningStatus: PostCommitNonRevertiblePhase stage 1 of 1 with 35 MutationType ops + pending + Statements: + - statement: DROP OWNED BY r + redactedstatement: DROP OWNED BY r + statementtag: DROP OWNED BY +PostCommitNonRevertiblePhase stage 1 of 1 with 47 MutationType ops + transitions: + [[Schema:{DescID: 105}, ABSENT], DROPPED] -> ABSENT + [[Sequence:{DescID: 106}, ABSENT], DROPPED] -> ABSENT + [[Table:{DescID: 109}, ABSENT], DROPPED] -> ABSENT + [[Column:{DescID: 109, ColumnID: 1}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 109, ColumnID: 2}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 109, ColumnID: 3}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 109, ColumnID: 4294967295}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 109, ColumnID: 4294967294}, ABSENT], WRITE_ONLY] -> ABSENT + [[PrimaryIndex:{DescID: 109, IndexID: 1, ConstraintID: 1}, ABSENT], VALIDATED] -> ABSENT + [[Sequence:{DescID: 107}, ABSENT], DROPPED] -> ABSENT + [[Table:{DescID: 108}, ABSENT], DROPPED] -> ABSENT + [[Column:{DescID: 108, ColumnID: 1}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 108, ColumnID: 2}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 108, ColumnID: 3}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 108, ColumnID: 4294967295}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 108, ColumnID: 4294967294}, ABSENT], WRITE_ONLY] -> ABSENT + [[PrimaryIndex:{DescID: 108, IndexID: 1, ConstraintID: 1}, ABSENT], VALIDATED] -> ABSENT + [[View:{DescID: 110}, ABSENT], DROPPED] -> ABSENT + [[Column:{DescID: 110, ColumnID: 1}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 110, ColumnID: 4294967295}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 110, ColumnID: 4294967294}, ABSENT], WRITE_ONLY] -> ABSENT + [[EnumType:{DescID: 111}, ABSENT], DROPPED] -> ABSENT + [[AliasType:{DescID: 112}, ABSENT], DROPPED] -> ABSENT + [[View:{DescID: 113}, ABSENT], DROPPED] -> ABSENT + [[Column:{DescID: 113, ColumnID: 1}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 113, ColumnID: 2}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 113, ColumnID: 4294967295}, ABSENT], WRITE_ONLY] -> ABSENT + [[Column:{DescID: 113, ColumnID: 4294967294}, ABSENT], WRITE_ONLY] -> ABSENT + ops: + *scop.LogEvent + Element: + Schema: + schemaId: 105 + EventBase: + Authorization: + UserName: root + Statement: DROP OWNED BY r + StatementTag: DROP OWNED BY + TargetMetadata: + SourceElementID: 1 + SubWorkID: 1 + TargetStatus: 1 + *scop.DeleteDescriptor + DescriptorID: 105 + *scop.LogEvent + Element: + Sequence: + sequenceId: 106 + EventBase: + Authorization: + UserName: root + Statement: DROP OWNED BY r + StatementTag: DROP OWNED BY + TargetMetadata: + SourceElementID: 1 + SubWorkID: 1 + TargetStatus: 1 + *scop.CreateGcJobForTable + StatementForDropJob: + Statement: DROP OWNED BY r + TableID: 106 + *scop.LogEvent + Element: + Sequence: + sequenceId: 107 + EventBase: + Authorization: + UserName: root + Statement: DROP OWNED BY r + StatementTag: DROP OWNED BY + TargetMetadata: + SourceElementID: 1 + SubWorkID: 1 + TargetStatus: 1 + *scop.CreateGcJobForTable + StatementForDropJob: + Statement: DROP OWNED BY r + TableID: 107 + *scop.LogEvent + Element: + EnumType: + arrayTypeId: 112 + typeId: 111 + EventBase: + Authorization: + UserName: root + Statement: DROP OWNED BY r + StatementTag: DROP OWNED BY + TargetMetadata: + SourceElementID: 1 + SubWorkID: 1 + TargetStatus: 1 + *scop.DeleteDescriptor + DescriptorID: 111 + *scop.LogEvent + Element: + AliasType: + embeddedTypeT: + closedTypeIds: + - 111 + - 112 + type: + arrayContents: + family: EnumFamily + oid: 100111 + udtMetadata: + arrayTypeOid: 100112 + arrayElemType: EnumFamily + family: ArrayFamily + oid: 100112 + typeId: 112 + EventBase: + Authorization: + UserName: root + Statement: DROP OWNED BY r + StatementTag: DROP OWNED BY + TargetMetadata: + SourceElementID: 1 + SubWorkID: 1 + TargetStatus: 1 + *scop.DeleteDescriptor + DescriptorID: 112 + *scop.LogEvent + Element: + View: + usesRelationIds: + - 108 + viewId: 110 + EventBase: + Authorization: + UserName: root + Statement: DROP OWNED BY r + StatementTag: DROP OWNED BY + TargetMetadata: + SourceElementID: 1 + SubWorkID: 1 + TargetStatus: 1 + *scop.DeleteDescriptor + DescriptorID: 110 + *scop.MakeColumnAbsent + ColumnID: 1 + EventBase: + Authorization: + UserName: root + Statement: DROP OWNED BY r + StatementTag: DROP OWNED BY + TargetMetadata: + SourceElementID: 1 + SubWorkID: 1 + TableID: 110 + *scop.MakeColumnAbsent + ColumnID: 4294967295 + EventBase: + Authorization: + UserName: root + Statement: DROP OWNED BY r + StatementTag: DROP OWNED BY + TargetMetadata: + SourceElementID: 1 + SubWorkID: 1 + TableID: 110 + *scop.MakeColumnAbsent + ColumnID: 4294967294 + EventBase: + Authorization: + UserName: root + Statement: DROP OWNED BY r + StatementTag: DROP OWNED BY + TargetMetadata: + SourceElementID: 1 + SubWorkID: 1 + TableID: 110 + *scop.LogEvent + Element: + View: + usesRelationIds: + - 110 + usesTypeIds: + - 111 + - 112 + viewId: 113 + EventBase: + Authorization: + UserName: root + Statement: DROP OWNED BY r + StatementTag: DROP OWNED BY + TargetMetadata: + SourceElementID: 1 + SubWorkID: 1 + TargetStatus: 1 + *scop.DeleteDescriptor + DescriptorID: 113 + *scop.MakeColumnAbsent + ColumnID: 1 + EventBase: + Authorization: + UserName: root + Statement: DROP OWNED BY r + StatementTag: DROP OWNED BY + TargetMetadata: + SourceElementID: 1 + SubWorkID: 1 + TableID: 113 + *scop.MakeColumnAbsent + ColumnID: 2 + EventBase: + Authorization: + UserName: root + Statement: DROP OWNED BY r + StatementTag: DROP OWNED BY + TargetMetadata: + SourceElementID: 1 + SubWorkID: 1 + TableID: 113 + *scop.MakeColumnAbsent + ColumnID: 4294967295 + EventBase: + Authorization: + UserName: root + Statement: DROP OWNED BY r + StatementTag: DROP OWNED BY + TargetMetadata: + SourceElementID: 1 + SubWorkID: 1 + TableID: 113 + *scop.MakeColumnAbsent + ColumnID: 4294967294 + EventBase: + Authorization: + UserName: root + Statement: DROP OWNED BY r + StatementTag: DROP OWNED BY + TargetMetadata: + SourceElementID: 1 + SubWorkID: 1 + TableID: 113 + *scop.LogEvent + Element: + Table: + tableId: 109 + EventBase: + Authorization: + UserName: root + Statement: DROP OWNED BY r + StatementTag: DROP OWNED BY + TargetMetadata: + SourceElementID: 1 + SubWorkID: 1 + TargetStatus: 1 + *scop.CreateGcJobForTable + StatementForDropJob: + Statement: DROP OWNED BY r + TableID: 109 + *scop.MakeColumnAbsent + ColumnID: 1 + EventBase: + Authorization: + UserName: root + Statement: DROP OWNED BY r + StatementTag: DROP OWNED BY + TargetMetadata: + SourceElementID: 1 + SubWorkID: 1 + TableID: 109 + *scop.MakeColumnAbsent + ColumnID: 2 + EventBase: + Authorization: + UserName: root + Statement: DROP OWNED BY r + StatementTag: DROP OWNED BY + TargetMetadata: + SourceElementID: 1 + SubWorkID: 1 + TableID: 109 + *scop.MakeColumnAbsent + ColumnID: 3 + EventBase: + Authorization: + UserName: root + Statement: DROP OWNED BY r + StatementTag: DROP OWNED BY + TargetMetadata: + SourceElementID: 1 + SubWorkID: 1 + TableID: 109 + *scop.MakeColumnAbsent + ColumnID: 4294967295 + EventBase: + Authorization: + UserName: root + Statement: DROP OWNED BY r + StatementTag: DROP OWNED BY + TargetMetadata: + SourceElementID: 1 + SubWorkID: 1 + TableID: 109 + *scop.MakeColumnAbsent + ColumnID: 4294967294 + EventBase: + Authorization: + UserName: root + Statement: DROP OWNED BY r + StatementTag: DROP OWNED BY + TargetMetadata: + SourceElementID: 1 + SubWorkID: 1 + TableID: 109 + *scop.LogEvent + Element: + Table: + tableId: 108 + EventBase: + Authorization: + UserName: root + Statement: DROP OWNED BY r + StatementTag: DROP OWNED BY + TargetMetadata: + SourceElementID: 1 + SubWorkID: 1 + TargetStatus: 1 + *scop.CreateGcJobForTable + StatementForDropJob: + Statement: DROP OWNED BY r + TableID: 108 + *scop.MakeColumnAbsent + ColumnID: 1 + EventBase: + Authorization: + UserName: root + Statement: DROP OWNED BY r + StatementTag: DROP OWNED BY + TargetMetadata: + SourceElementID: 1 + SubWorkID: 1 + TableID: 108 + *scop.MakeColumnAbsent + ColumnID: 2 + EventBase: + Authorization: + UserName: root + Statement: DROP OWNED BY r + StatementTag: DROP OWNED BY + TargetMetadata: + SourceElementID: 1 + SubWorkID: 1 + TableID: 108 + *scop.MakeColumnAbsent + ColumnID: 3 + EventBase: + Authorization: + UserName: root + Statement: DROP OWNED BY r + StatementTag: DROP OWNED BY + TargetMetadata: + SourceElementID: 1 + SubWorkID: 1 + TableID: 108 + *scop.MakeColumnAbsent + ColumnID: 4294967295 + EventBase: + Authorization: + UserName: root + Statement: DROP OWNED BY r + StatementTag: DROP OWNED BY + TargetMetadata: + SourceElementID: 1 + SubWorkID: 1 + TableID: 108 + *scop.MakeColumnAbsent + ColumnID: 4294967294 + EventBase: + Authorization: + UserName: root + Statement: DROP OWNED BY r + StatementTag: DROP OWNED BY + TargetMetadata: + SourceElementID: 1 + SubWorkID: 1 + TableID: 108 + *scop.RemoveJobStateFromDescriptor + DescriptorID: 100 + JobID: 1 + *scop.RemoveJobStateFromDescriptor + DescriptorID: 104 + JobID: 1 + *scop.RemoveJobStateFromDescriptor + DescriptorID: 105 + JobID: 1 + *scop.RemoveJobStateFromDescriptor + DescriptorID: 106 + JobID: 1 + *scop.RemoveJobStateFromDescriptor + DescriptorID: 107 + JobID: 1 + *scop.RemoveJobStateFromDescriptor + DescriptorID: 108 + JobID: 1 + *scop.RemoveJobStateFromDescriptor + DescriptorID: 109 + JobID: 1 + *scop.RemoveJobStateFromDescriptor + DescriptorID: 110 + JobID: 1 + *scop.RemoveJobStateFromDescriptor + DescriptorID: 111 + JobID: 1 + *scop.RemoveJobStateFromDescriptor + DescriptorID: 112 + JobID: 1 + *scop.RemoveJobStateFromDescriptor + DescriptorID: 113 + JobID: 1 + *scop.UpdateSchemaChangerJob + IsNonCancelable: true + JobID: 1 diff --git a/pkg/sql/sem/tree/schema_feature_name.go b/pkg/sql/sem/tree/schema_feature_name.go index 769e71bd0e3c..8dfb9cdd5b74 100644 --- a/pkg/sql/sem/tree/schema_feature_name.go +++ b/pkg/sql/sem/tree/schema_feature_name.go @@ -24,7 +24,7 @@ func GetSchemaFeatureNameFromStmt(stmt Statement) SchemaFeatureName { switch stmt.(type) { case *CommentOnDatabase, *CommentOnSchema, *CommentOnTable, - *CommentOnColumn, *CommentOnIndex, *CommentOnConstraint: + *CommentOnColumn, *CommentOnIndex, *CommentOnConstraint, *DropOwnedBy: return SchemaFeatureName(statementTag) } // Only grab the first two words (i.e. ALTER TABLE, etc..). diff --git a/pkg/sql/testdata/telemetry/drop_owned_by b/pkg/sql/testdata/telemetry/drop_owned_by index 1da9d9586212..bddbb209f7f8 100644 --- a/pkg/sql/testdata/telemetry/drop_owned_by +++ b/pkg/sql/testdata/telemetry/drop_owned_by @@ -11,9 +11,7 @@ GRANT CREATE ON DATABASE defaultdb TO testuser; ALTER TABLE t OWNER TO testuser ---- -# TODO(angelaw): Remove unimplemented message after implementation. feature-usage DROP OWNED BY testuser ---- -error: pq: unimplemented: drop owned by is not yet implemented sql.drop_owned_by From 5d4c0d71bf9ac91ffece021b2d2ee9636ad65e6e Mon Sep 17 00:00:00 2001 From: Nathan VanBenschoten Date: Tue, 28 Jun 2022 10:27:40 -0700 Subject: [PATCH 9/9] kv: don't try to reject lease transfer when flushing proposal buffer Fixes #83498. Fixes #83402. Fixes #83308. This was fallout from #82758. This commit adds logic to `propBuf.maybeRejectUnsafeProposalLocked` to avoid trying to reject proposals based on the state of the raft group when the group is not provided (e.g. when flushing the buffer). We already had this logic for `RequestLease` (indirectly), but did not for `TransferLease`. --- pkg/kv/kvserver/replica_proposal_buf.go | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/pkg/kv/kvserver/replica_proposal_buf.go b/pkg/kv/kvserver/replica_proposal_buf.go index f1f35bb5877e..b93a79e17640 100644 --- a/pkg/kv/kvserver/replica_proposal_buf.go +++ b/pkg/kv/kvserver/replica_proposal_buf.go @@ -567,6 +567,13 @@ func (b *propBuf) FlushLockedWithRaftGroup( func (b *propBuf) maybeRejectUnsafeProposalLocked( ctx context.Context, raftGroup proposerRaft, p *ProposalData, ) (rejected bool) { + if raftGroup == nil { + // If we do not have a raft group, we won't try to propose this proposal. + // Instead, we will register the proposal so that it can be reproposed later + // with a raft group. Wait until that point to determine whether to reject + // the proposal or not. + return false + } switch { case p.Request.IsSingleRequestLeaseRequest(): // Handle an edge case about lease acquisitions: we don't want to forward @@ -706,9 +713,6 @@ func (b *propBuf) maybeRejectUnsafeProposalLocked( // leaderStatusRLocked returns the rangeLeaderInfo for the provided raft group, // or an empty rangeLeaderInfo if the raftGroup is nil. func (b *propBuf) leaderStatusRLocked(ctx context.Context, raftGroup proposerRaft) rangeLeaderInfo { - if raftGroup == nil { - return rangeLeaderInfo{} - } leaderInfo := b.p.leaderStatusRLocked(raftGroup) // Sanity check. if leaderInfo.leaderKnown && leaderInfo.leader == b.p.getReplicaID() &&