Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
…achdb#126487 cockroachdb#126498 cockroachdb#126524 cockroachdb#126533 cockroachdb#126536

126352: cli: add fallback query support for debug zip r=xinhaoz a=dhartunian

Previously, when SQL queries for dumping tables to debug zip would fail, we would have no follow-up. Engineers can now define "fallback" queries for tables in debug zip in order to make a second attempt with a simpler query. Often we want to run a more complex query to gather more debug data but these queries can fail when the cluster is experiencing problems. This change gives us a chance to define a simpler approach that can be attempted when necessary.

In order to define a fallback, there are two new optional fields in the `TableRegistryConfig` struct for redacted and unredacted queries respectively.

Debug zip output will still include the failed attempts at the original query along with the error message file as before. If a fallback query is defined, that query will produce its own output (and error) file with an additional `.fallback` suffix added to the base table name to identify it.

Resolves: cockroachdb#123964
Epic: CRDB-35278

Release note: None

126354: ui: alter role events render correctly r=xinhaoz a=dhartunian

Previously, ALTER ROLE events without role options would render with an "undefined" option in the event log on the DB Console. This change amends the rendering logic to correctly render events without any options.

Resolves cockroachdb#124871
Epic: None

Release note (bug fix,ui change): ALTER ROLE events in the DB Console event log now render correctly when the event does not contain any role options.

126486: kvserver/rangefeed: remove lockedRangefeedStream r=nvanbenschoten a=wenyihu6

**kvserver: wrap kvpb.RangeFeedEventSink in Stream**

Previously, we declared the same interface signature twice: once in
kvpb.RangeFeedEventSink and again in rangefeed.Stream. This patch embeds
kvpb.RangeFeedEventSink inside rangefeed.Stream, making rangefeed.Stream a
superset of kvpb.RangeFeedEventSink. This approach makes sense, as each
rangefeed server stream should be a rangefeed event sink, capable of making
thread-safe rangefeed event sends.

Epic: none
Release note: none

---

**kvserver/rangefeed: remove lockedRangefeedStream**

Previously, we created separate locked rangefeed streams for each individual
rangefeed stream to ensure Send can be called concurrently as the underlying
grpc stream is not thread safe. However, since the introduction of the mux
rangefeed support, we already have a dedicated lock for the underlying mux
stream, making the Send method on each rangefeed stream thread safe already.
This patch removes the redundant locks from each individual rangefeed stream.

Epic: none
Release note: none

126487: kvserver/rangefeed: remove non-mux rangefeed metrics r=nvanbenschoten a=wenyihu6

Previously, we removed non-mux rangefeed code in
cockroachdb#125610. However, that patch forgot
to remove non-mux rangefeed metrics. This patch removes these metrics as they
are no longer needed.

Epic: none
Release note: none

126498: status: fix TestTenantStatusAPI test r=xinhaoz a=dhartunian

Previously, this test would use a single connection, cancel it, and then use the connection to verify the cancellation.

The test is adjusted here to use two separate sessions, one to cancel for testing, and another to observe the cancellation.

Resolves: cockroachdb#125404
Epic: None

Release note: None

126524: sql: unskip Insights test r=dhartunian a=dhartunian

This test has been flaky for a while because of the async tagging of the TransactionID to the insight that somtimes takes too long to complete. This change removes that check and unskips the test so that we can catch regressions for this feature. In the future we may want to write a separate test to verify the async transactionID tagging separately.

Resolves: cockroachdb#125771
Resolves: cockroachdb#121986

Epic: None
Release note: None

126533: kv: hook Raft StoreLiveness into storeliveness package r=nvanbenschoten a=nvanbenschoten

Fixes cockroachdb#125242.

This commit adds a `replicaRLockedStoreLiveness` adapter type to hook the raft store liveness into the storeliveness package.

This is currently unused.

Release note: None

126536: roachpb: add Leader lease type definition r=nvanbenschoten a=nvanbenschoten

Fixes cockroachdb#125225.

This commit adds a new `Term` field to the Lease struct. This field defines the term of the raft leader that a leader lease is associated with. The lease is valid for as long as the raft leader has a guarantee from store liveness that it remains the leader under this term. The lease is invalid if the raft leader loses leadership (i.e. changes its term).

The field is not yet used.

Release note: None

Co-authored-by: David Hartunian <[email protected]>
Co-authored-by: Wenyi Hu <[email protected]>
Co-authored-by: Nathan VanBenschoten <[email protected]>
  • Loading branch information
4 people committed Jul 2, 2024
9 parents fa8149f + 535bef4 + 84cf77d + 2df6a9c + 9d963ac + 7d4d517 + 0c15a91 + 393d1ee + 41c2a01 commit 786cb46
Show file tree
Hide file tree
Showing 30 changed files with 812 additions and 108 deletions.
2 changes: 0 additions & 2 deletions docs/generated/metrics/metrics.html
Original file line number Diff line number Diff line change
Expand Up @@ -629,8 +629,6 @@
<tr><td>STORAGE</td><td>rpc.method.writebatch.recv</td><td>Number of WriteBatch requests processed</td><td>RPCs</td><td>COUNTER</td><td>COUNT</td><td>AVG</td><td>NON_NEGATIVE_DERIVATIVE</td></tr>
<tr><td>STORAGE</td><td>rpc.streams.mux_rangefeed.active</td><td>Number of currently running MuxRangeFeed streams</td><td>Streams</td><td>GAUGE</td><td>COUNT</td><td>AVG</td><td>NONE</td></tr>
<tr><td>STORAGE</td><td>rpc.streams.mux_rangefeed.recv</td><td>Total number of MuxRangeFeed streams</td><td>Streams</td><td>COUNTER</td><td>COUNT</td><td>AVG</td><td>NON_NEGATIVE_DERIVATIVE</td></tr>
<tr><td>STORAGE</td><td>rpc.streams.rangefeed.active</td><td>Number of currently running RangeFeed streams</td><td>Streams</td><td>GAUGE</td><td>COUNT</td><td>AVG</td><td>NONE</td></tr>
<tr><td>STORAGE</td><td>rpc.streams.rangefeed.recv</td><td>Total number of RangeFeed streams</td><td>Streams</td><td>COUNTER</td><td>COUNT</td><td>AVG</td><td>NON_NEGATIVE_DERIVATIVE</td></tr>
<tr><td>STORAGE</td><td>spanconfig.kvsubscriber.oldest_protected_record_nanos</td><td>Difference between the current time and the oldest protected timestamp (sudden drops indicate a record being released; an ever increasing number indicates that the oldest record is around and preventing GC if &gt; configured GC TTL)</td><td>Nanoseconds</td><td>GAUGE</td><td>NANOSECONDS</td><td>AVG</td><td>NONE</td></tr>
<tr><td>STORAGE</td><td>spanconfig.kvsubscriber.protected_record_count</td><td>Number of protected timestamp records, as seen by KV</td><td>Records</td><td>GAUGE</td><td>COUNT</td><td>AVG</td><td>NONE</td></tr>
<tr><td>STORAGE</td><td>spanconfig.kvsubscriber.update_behind_nanos</td><td>Difference between the current time and when the KVSubscriber received its last update (an ever increasing number indicates that we&#39;re no longer receiving updates)</td><td>Nanoseconds</td><td>GAUGE</td><td>NANOSECONDS</td><td>AVG</td><td>NONE</td></tr>
Expand Down
16 changes: 12 additions & 4 deletions pkg/ccl/serverccl/statusccl/tenant_status_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1102,8 +1102,16 @@ func selectClusterSessionIDs(t *testing.T, conn *sqlutils.SQLRunner) []string {

func testTenantStatusCancelSession(t *testing.T, helper serverccl.TenantTestHelper) {
// Open a SQL session on tenant SQL pod 0.
sqlPod0 := helper.TestCluster().TenantConn(0)
sqlPod0.Exec(t, "SELECT 1")
ctx := context.Background()
// Open two different SQL sessions on tenant SQL pod 0.
sqlPod0 := helper.TestCluster().TenantDB(0)
sqlPod0SessionToCancel, err := sqlPod0.Conn(ctx)
require.NoError(t, err)
sqlPod0SessionForIntrospection, err := sqlPod0.Conn(ctx)
require.NoError(t, err)
_, err = sqlPod0SessionToCancel.ExecContext(ctx, "SELECT 1")
require.NoError(t, err)
introspectionRunner := sqlutils.MakeSQLRunner(sqlPod0SessionForIntrospection)

// See the session over HTTP on tenant SQL pod 1.
httpPod1 := helper.TestCluster().TenantAdminHTTPClient(t, 1)
Expand All @@ -1122,7 +1130,7 @@ func testTenantStatusCancelSession(t *testing.T, helper serverccl.TenantTestHelp
// See the session over SQL on tenant SQL pod 0.
sessionID := hex.EncodeToString(session.ID)
require.Eventually(t, func() bool {
return strings.Contains(strings.Join(selectClusterSessionIDs(t, sqlPod0), ","), sessionID)
return strings.Contains(strings.Join(selectClusterSessionIDs(t, introspectionRunner), ","), sessionID)
}, 5*time.Second, 100*time.Millisecond)

// Cancel the session over HTTP from tenant SQL pod 1.
Expand All @@ -1134,7 +1142,7 @@ func testTenantStatusCancelSession(t *testing.T, helper serverccl.TenantTestHelp
// No longer see the session over SQL from tenant SQL pod 0.
// (The SQL client maintains an internal connection pool and automatically reconnects.)
require.Eventually(t, func() bool {
return !strings.Contains(strings.Join(selectClusterSessionIDs(t, sqlPod0), ","), sessionID)
return !strings.Contains(strings.Join(selectClusterSessionIDs(t, introspectionRunner), ","), sessionID)
}, 5*time.Second, 100*time.Millisecond)

// Attempt to cancel the session again over HTTP from tenant SQL pod 1, so that we can see the error message.
Expand Down
275 changes: 275 additions & 0 deletions pkg/cli/testdata/zip/testzip_fallback

Large diffs are not rendered by default.

19 changes: 17 additions & 2 deletions pkg/cli/zip.go
Original file line number Diff line number Diff line change
Expand Up @@ -490,14 +490,18 @@ INNER JOIN latestprogress ON j.id = latestprogress.job_id;`,
// An error is returned by this function if it is unable to write to
// the output file or some other unrecoverable error is encountered.
func (zc *debugZipContext) dumpTableDataForZip(
zr *zipReporter, conn clisqlclient.Conn, base, table, query string,
zr *zipReporter, conn clisqlclient.Conn, base, table string, tableQuery TableQuery,
) error {
ctx := context.Background()
baseName := base + "/" + sanitizeFilename(table)

s := zr.start("retrieving SQL data for %s", table)
const maxRetries = 5
suffix := ""

query := tableQuery.query
fallback := tableQuery.fallback != ""

for numRetries := 1; numRetries <= maxRetries; numRetries++ {
name := baseName + suffix + "." + zc.clusterPrinter.sqlOutputFilenameExtension
s.progress("writing output: %s", name)
Expand Down Expand Up @@ -545,7 +549,18 @@ func (zc *debugZipContext) dumpTableDataForZip(
break
}
if pgcode.MakeCode(pgErr.Code) != pgcode.SerializationFailure {
// A non-retry error. We've printed the error, and
// A non-retry error. If we have a fallback, try with that.
if fallback {
fallback = false

query = tableQuery.fallback
numRetries = 1 // Reset counter since this is a different query.
baseName = baseName + ".fallback"
s = zr.start("retrieving SQL data for %s (fallback)", table)

continue
}
// A non-retry error, no fallback. We've printed the error, and
// there's nothing to retry. Stop here.
break
}
Expand Down
46 changes: 39 additions & 7 deletions pkg/cli/zip_table_registry.go
Original file line number Diff line number Diff line change
Expand Up @@ -58,6 +58,15 @@ type TableRegistryConfig struct {
// customQueryRedacted should NOT be a `SELECT * FROM table` type query, as
// this could leak newly added sensitive columns into the output.
customQueryRedacted string

// customQueryUnredactedFallback is an alternative query that will be
// attempted if `customQueryUnredacted` does not return within the
// timeout or fails. If empty it will be ignored.
customQueryUnredactedFallback string
// customQueryRedactedFallback is an alternative query that will be
// attempted if `customQueryRedacted` does not return within the
// timeout or fails. If empty it will be ignored.
customQueryRedactedFallback string
}

// DebugZipTableRegistry is a registry of `crdb_internal` and `system` tables
Expand All @@ -74,26 +83,35 @@ type TableRegistryConfig struct {
// may be a way to avoid having to completely omit entire columns.
type DebugZipTableRegistry map[string]TableRegistryConfig

// TableQuery holds two sql query strings together that are used to
// dump tables when generating a debug zip. `query` is the primary
// query to run, and `fallback` is one to try if the primary fails or
// times out.
type TableQuery struct {
query string
fallback string
}

// QueryForTable produces the appropriate query for `debug zip` for the given
// table to use, taking redaction into account. If the provided tableName does
// not exist in the registry, or no redacted config exists in the registry for
// the tableName, an error is returned.
func (r DebugZipTableRegistry) QueryForTable(tableName string, redact bool) (string, error) {
func (r DebugZipTableRegistry) QueryForTable(tableName string, redact bool) (TableQuery, error) {
tableConfig, ok := r[tableName]
if !ok {
return "", errors.Newf("no entry found in table registry for: %s", tableName)
return TableQuery{}, errors.Newf("no entry found in table registry for: %s", tableName)
}
if !redact {
if tableConfig.customQueryUnredacted != "" {
return tableConfig.customQueryUnredacted, nil
return TableQuery{tableConfig.customQueryUnredacted, tableConfig.customQueryUnredactedFallback}, nil
}
return fmt.Sprintf("TABLE %s", tableName), nil
return TableQuery{fmt.Sprintf("TABLE %s", tableName), ""}, nil
}
if tableConfig.customQueryRedacted != "" {
return tableConfig.customQueryRedacted, nil
return TableQuery{tableConfig.customQueryRedacted, tableConfig.customQueryRedactedFallback}, nil
}
if len(tableConfig.nonSensitiveCols) == 0 {
return "", errors.Newf("requested redacted query for table %s, but no non-sensitive columns defined", tableName)
return TableQuery{}, errors.Newf("requested redacted query for table %s, but no non-sensitive columns defined", tableName)
}
var colsString strings.Builder
for i, colName := range tableConfig.nonSensitiveCols {
Expand All @@ -103,7 +121,7 @@ func (r DebugZipTableRegistry) QueryForTable(tableName string, redact bool) (str
colsString.WriteString(", ")
}
}
return fmt.Sprintf("SELECT %s FROM %s", colsString.String(), tableName), nil
return TableQuery{fmt.Sprintf("SELECT %s FROM %s", colsString.String(), tableName), ""}, nil
}

// GetTables returns all the table names within the registry. Useful for
Expand Down Expand Up @@ -611,6 +629,20 @@ WHERE ss.transaction_fingerprint_id != '\x0000000000000000' AND s.fingerprint_id
GROUP BY collection_ts, contention_duration, waiting_txn_id, waiting_txn_fingerprint_id, blocking_txn_id,
blocking_txn_fingerprint_id, waiting_stmt_fingerprint_id, contending_pretty_key, s.metadata ->> 'query',
index_name, table_name, database_name
`,
customQueryUnredactedFallback: `
SELECT collection_ts,
contention_duration,
waiting_txn_id,
waiting_txn_fingerprint_id,
waiting_stmt_fingerprint_id,
blocking_txn_id,
blocking_txn_fingerprint_id,
contending_pretty_key,
index_name,
table_name,
database_name
FROM crdb_internal.transaction_contention_events
`,
// `contending_key` column contains the contended key, which may
// contain sensitive row-level data. So, we will only fetch if the
Expand Down
30 changes: 23 additions & 7 deletions pkg/cli/zip_table_registry_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -42,6 +42,11 @@ func TestQueryForTable(t *testing.T) {
nonSensitiveCols: NonSensitiveColumns{"x", "crdb_internal.pretty_key(y, 0) as y", "z"},
customQueryUnredacted: "SELECT x, crdb_internal.pretty_key(y, 0) as y, z FROM table_with_non_sensitive_cols_and_custom_unredacted_query",
},
"table_with_non_sensitive_cols_and_custom_unredacted_query_with_fallback": {
nonSensitiveCols: NonSensitiveColumns{"x", "crdb_internal.pretty_key(y, 0) as y", "z"},
customQueryUnredacted: "SELECT x, crdb_internal.pretty_key(y, 0) as y, z FROM table_with_non_sensitive_cols_and_custom_unredacted_query_with_fallback",
customQueryUnredactedFallback: "SELECT x FROM table_with_non_sensitive_cols_and_custom_unredacted_query_with_fallback",
},
}

t.Run("errors if no table config present in registry", func(t *testing.T) {
Expand All @@ -53,31 +58,31 @@ func TestQueryForTable(t *testing.T) {

t.Run("produces `TABLE` query when unredacted with no custom query", func(t *testing.T) {
table := "table_with_sensitive_cols"
expected := "TABLE table_with_sensitive_cols"
expected := TableQuery{query: "TABLE table_with_sensitive_cols"}
actual, err := reg.QueryForTable(table, false /* redact */)
assert.NoError(t, err)
assert.Equal(t, expected, actual)
})

t.Run("produces custom query when unredacted and custom query supplied", func(t *testing.T) {
table := "table_with_custom_queries"
expected := "SELECT * FROM table_with_custom_queries"
expected := TableQuery{query: "SELECT * FROM table_with_custom_queries"}
actual, err := reg.QueryForTable(table, false /* redact */)
assert.NoError(t, err)
assert.Equal(t, expected, actual)
})

t.Run("produces query with only non-sensitive columns when redacted and no custom query", func(t *testing.T) {
table := "table_with_sensitive_cols"
expected := `SELECT x, y, z FROM table_with_sensitive_cols`
expected := TableQuery{query: `SELECT x, y, z FROM table_with_sensitive_cols`}
actual, err := reg.QueryForTable(table, true /* redact */)
assert.NoError(t, err)
assert.Equal(t, expected, actual)
})

t.Run("produces custom when redacted and custom query supplied", func(t *testing.T) {
table := "table_with_custom_queries"
expected := "SELECT a, b, c FROM table_with_custom_queries"
expected := TableQuery{query: "SELECT a, b, c FROM table_with_custom_queries"}
actual, err := reg.QueryForTable(table, true /* redact */)
assert.NoError(t, err)
assert.Equal(t, expected, actual)
Expand All @@ -93,7 +98,7 @@ func TestQueryForTable(t *testing.T) {

t.Run("produces query when a combination of nonSensitiveCols and customQueryUnredacted is supplied", func(t *testing.T) {
table := "table_with_non_sensitive_cols_and_custom_unredacted_query"
expected := "SELECT x, crdb_internal.pretty_key(y, 0) as y, z FROM table_with_non_sensitive_cols_and_custom_unredacted_query"
expected := TableQuery{query: "SELECT x, crdb_internal.pretty_key(y, 0) as y, z FROM table_with_non_sensitive_cols_and_custom_unredacted_query"}

t.Run("with redact flag", func(t *testing.T) {
actual, err := reg.QueryForTable(table, true /* redact */)
Expand All @@ -107,6 +112,17 @@ func TestQueryForTable(t *testing.T) {
assert.Equal(t, expected, actual)
})
})

t.Run("with fallback query", func(t *testing.T) {
table := "table_with_non_sensitive_cols_and_custom_unredacted_query_with_fallback"
expected := TableQuery{
query: "SELECT x, crdb_internal.pretty_key(y, 0) as y, z FROM table_with_non_sensitive_cols_and_custom_unredacted_query_with_fallback",
fallback: "SELECT x FROM table_with_non_sensitive_cols_and_custom_unredacted_query_with_fallback",
}
actual, err := reg.QueryForTable(table, false /* redact */)
assert.NoError(t, err)
assert.Equal(t, expected, actual)
})
}

func TestNoForbiddenSystemTablesInDebugZip(t *testing.T) {
Expand All @@ -125,8 +141,8 @@ func TestNoForbiddenSystemTablesInDebugZip(t *testing.T) {
"system.transaction_activity",
}
for _, forbiddenTable := range forbiddenSysTables {
query, err := zipSystemTables.QueryForTable(forbiddenTable, false /* redact */)
assert.Equal(t, "", query)
tableQuery, err := zipSystemTables.QueryForTable(forbiddenTable, false /* redact */)
assert.Equal(t, "", tableQuery.query)
assert.Error(t, err)
assert.Equal(t, fmt.Sprintf("no entry found in table registry for: %s", forbiddenTable), err.Error())
}
Expand Down
Loading

0 comments on commit 786cb46

Please sign in to comment.