Skip to content

Commit

Permalink
sql: improve handling of contention on non-SQL keys
Browse files Browse the repository at this point in the history
Previously, if we encountered contention on a non-SQL key, it would
result in an error being logged every time
`crdb_internal.transaction_contention_events` is populated. Properly
decoding non-SQL keys doesn't seem trivial (plus it wouldn't easily map
to columns in the virtual table), so this commit instead adjusts the
logic in `getContentionEventInfo` to do a best-effort decoding, with
erroneous cases being handled by including a hint into the `tableName`
string. This should remove concerningly-looking errors from the logs and
possibly provide better observability too.

Release note: None
  • Loading branch information
yuzefovich committed Jun 28, 2024
1 parent 9c91542 commit 7974916
Show file tree
Hide file tree
Showing 2 changed files with 40 additions and 38 deletions.
66 changes: 34 additions & 32 deletions pkg/sql/crdb_internal.go
Original file line number Diff line number Diff line change
Expand Up @@ -7713,15 +7713,7 @@ CREATE TABLE crdb_internal.transaction_contention_events (

waitingStmtId := tree.NewDString(hex.EncodeToString(resp.Events[i].WaitingStmtID.GetBytes()))

// getContentionEventInfo needs to handle both the time and type of
// possible descriptors. It just logs the error and uses empty string
// for the values if an error occurs.
// https://github.com/cockroachdb/cockroach/issues/101826
schemaName, dbName, tableName, indexName, err := getContentionEventInfo(ctx, p, resp.Events[i])
if err != nil {
log.Errorf(ctx, "getContentionEventInfo failed to decode key: %v", err)
}

dbName, schemaName, tableName, indexName := getContentionEventInfo(ctx, p, resp.Events[i])
row = append(row[:0],
collectionTs, // collection_ts
tree.NewDUuid(tree.DUuid{UUID: resp.Events[i].BlockingEvent.TxnMeta.ID}), // blocking_txn_id
Expand Down Expand Up @@ -8581,54 +8573,64 @@ func populateStmtInsights(
return
}

// getContentionEventInfo performs a best-effort decoding of the key on which
// the contention occurred into the corresponding db, schema, table, and index
// names. If the key doesn't belong to the SQL data, then returned strings will
// contain a hint about that.
// TODO(#101826): we should teach this function to properly handle non-SQL keys.
func getContentionEventInfo(
ctx context.Context, p *planner, contentionEvent contentionpb.ExtendedContentionEvent,
) (schemaName, dbName, tableName, indexName string, err error) {

_, tableID, err := p.ExecCfg().Codec.DecodeTablePrefix(contentionEvent.BlockingEvent.Key)
) (dbName, schemaName, tableName, indexName string) {
// Strip the tenant prefix right away if present.
key, err := p.ExecCfg().Codec.StripTenantPrefix(contentionEvent.BlockingEvent.Key)
if err != nil {
return "", "", "", "", err
// We really don't want to return errors, so we'll include the error
// details as the table name.
tableName = err.Error()
return "", "", tableName, ""
}
_, _, indexID, err := p.ExecCfg().Codec.DecodeIndexPrefix(contentionEvent.BlockingEvent.Key)
if keys.TableDataMin.Compare(key) > 0 || keys.TableDataMax.Compare(key) < 0 {
// Non-SQL keys are handled separately.
tableName = fmt.Sprintf("%q", key)
return "", "", tableName, ""
}
_, tableID, indexID, err := keys.DecodeTableIDIndexID(key)
if err != nil {
return "", "", "", "", err
// We really don't want to return errors, so we'll include the error
// details in the table name.
tableName = err.Error()
return "", "", tableName, ""
}

desc := p.Descriptors()
var tableDesc catalog.TableDescriptor
tableDesc, err = desc.ByIDWithLeased(p.txn).WithoutNonPublic().Get().Table(ctx, descpb.ID(tableID))
if err != nil {
return "", "", fmt.Sprintf("[dropped table id: %d]", tableID), "[dropped index]", nil //nolint:returnerrcheck
return "", "", fmt.Sprintf("[dropped table id: %d]", tableID), "[dropped index]" //nolint:returnerrcheck
}

var idxName string
idxDesc, err := catalog.MustFindIndexByID(tableDesc, descpb.IndexID(indexID))
if err != nil {
idxName = fmt.Sprintf("[dropped index id: %d]", indexID)
}
if idxDesc != nil {
idxName = idxDesc.GetName()
indexName = fmt.Sprintf("[dropped index id: %d]", indexID)
} else if idxDesc != nil {
indexName = idxDesc.GetName()
}

var databaseName string
dbDesc, err := desc.ByIDWithLeased(p.txn).WithoutNonPublic().Get().Database(ctx, tableDesc.GetParentID())
if err != nil {
databaseName = "[dropped database]"
}
if dbDesc != nil {
databaseName = dbDesc.GetName()
dbName = "[dropped database]"
} else if dbDesc != nil {
dbName = dbDesc.GetName()
}

var schName string
schemaDesc, err := desc.ByIDWithLeased(p.txn).WithoutNonPublic().Get().Schema(ctx, tableDesc.GetParentSchemaID())
if err != nil {
schName = "[dropped schema]"
}
if schemaDesc != nil {
schName = schemaDesc.GetName()
schemaName = "[dropped schema]"
} else if schemaDesc != nil {
schemaName = schemaDesc.GetName()
}

return schName, databaseName, tableDesc.GetName(), idxName, nil
return dbName, schemaName, tableDesc.GetName(), indexName
}

var crdbInternalNodeMemoryMonitors = virtualSchemaTable{
Expand Down
12 changes: 6 additions & 6 deletions pkg/sql/crdb_internal_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -958,8 +958,7 @@ func TestTxnContentionEventsTableWithRangeDescriptor(t *testing.T) {
ContentionType: contentionpb.ContentionType_LOCK_WAIT,
})

// Contention flush can take some time to flush
// the events
// Contention flush can take some time to flush the events.
testutils.SucceedsSoon(t, func() error {
row := sqlDB.QueryRow(`SELECT
database_name,
Expand All @@ -974,10 +973,11 @@ func TestTxnContentionEventsTableWithRangeDescriptor(t *testing.T) {
if err != nil {
return err
}
require.Equal(t, "", db)
require.Equal(t, "", schema)
require.Equal(t, "", table)
require.Equal(t, "", index)
if db != "" || schema != "" || table != rangeKeyEscaped || index != "" {
return errors.Newf(
"unexpected row: db=%s, schema=%s, table=%s, index=%s", db, schema, table, index,
)
}
return nil
})
}
Expand Down

0 comments on commit 7974916

Please sign in to comment.