Skip to content

Commit

Permalink
Merge #87875 #88098
Browse files Browse the repository at this point in the history
87875: ccl/backupccl: add application name to backup/restore telemetry r=rhu713 a=rhu713

Add application name as a field in RecoveryEvent. Record the session application name for every invoked backup, restore, and backup schedule creation.

88098: sql: add missing obj_description case r=rafiss a=knz

Needed for #88061.

Found bug with the following query:

```sql
COMMENT ON SCHEMA public IS 'hello';
SELECT obj_description(oid, 'pg_namespace')
FROM pg_namespace WHERE nspname = 'public';
```

Release note (sql change): The PostgreSQL compatibility function `obj_description` now supports retrieving comments on schemas.

Co-authored-by: Rui Hu <[email protected]>
Co-authored-by: Raphael 'kena' Poss <[email protected]>
  • Loading branch information
3 people committed Sep 20, 2022
3 parents 9c0778f + 598c170 + 3e22bb1 commit 0efe4f5
Show file tree
Hide file tree
Showing 16 changed files with 217 additions and 79 deletions.
145 changes: 73 additions & 72 deletions docs/generated/eventlog.md

Large diffs are not rendered by default.

1 change: 1 addition & 0 deletions pkg/ccl/backupccl/backup_planning.go
Original file line number Diff line number Diff line change
Expand Up @@ -704,6 +704,7 @@ func backupPlanHook(
EncryptionOptions: &encryptionParams,
AsOfInterval: asOfInterval,
Detached: detached,
ApplicationName: p.SessionData().ApplicationName,
}
if backupStmt.CreatedByInfo != nil && backupStmt.CreatedByInfo.Name == jobs.CreatedByScheduledJobs {
initialDetails.ScheduleID = backupStmt.CreatedByInfo.ID
Expand Down
3 changes: 3 additions & 0 deletions pkg/ccl/backupccl/backup_telemetry.go
Original file line number Diff line number Diff line change
Expand Up @@ -167,6 +167,7 @@ func createBackupRecoveryEvent(
JobID: uint64(jobID),
AsOfInterval: initialDetails.AsOfInterval,
Options: options,
ApplicationName: initialDetails.ApplicationName,
}

event.DestinationAuthTypes = make([]string, 0, len(authTypes))
Expand Down Expand Up @@ -318,6 +319,7 @@ func logRestoreTelemetry(
descsByTablePattern map[tree.TablePattern]catalog.Descriptor,
restoreDBs []catalog.DatabaseDescriptor,
debugPauseOn string,
applicationName string,
) {
var requestedTargets []descpb.Descriptor
for _, desc := range descsByTablePattern {
Expand Down Expand Up @@ -417,6 +419,7 @@ func logRestoreTelemetry(
DebugPauseOn: debugPauseOn,
JobID: uint64(jobID),
Options: options,
ApplicationName: applicationName,
}

event.DestinationAuthTypes = make([]string, 0, len(authTypes))
Expand Down
3 changes: 3 additions & 0 deletions pkg/ccl/backupccl/backup_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -9926,6 +9926,7 @@ func TestBackupRestoreTelemetryEvents(t *testing.T) {
}
}

sqlDB.Exec(t, `SET application_name = 'backup_test'`)
sqlDB.Exec(t, `CREATE DATABASE r1`)
sqlDB.Exec(t, `CREATE TABLE r1.foo (id INT)`)
sqlDB.Exec(t, `CREATE DATABASE r2`)
Expand All @@ -9950,6 +9951,7 @@ func TestBackupRestoreTelemetryEvents(t *testing.T) {
IsLocalityAware: true,
AsOfInterval: -1 * time.Millisecond.Nanoseconds(),
WithRevisionHistory: true,
ApplicationName: "backup_test",
}

// Also verify that there's a telemetry event corresponding to the completion
Expand Down Expand Up @@ -9984,6 +9986,7 @@ func TestBackupRestoreTelemetryEvents(t *testing.T) {
IsLocalityAware: true,
AsOfInterval: 0,
Options: []string{telemetryOptionIntoDB, telemetryOptionSkipMissingFK},
ApplicationName: "backup_test",
}

// Also verify that there's a telemetry event corresponding to the completion
Expand Down
2 changes: 2 additions & 0 deletions pkg/ccl/backupccl/create_scheduled_backup_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1311,6 +1311,7 @@ WITH SCHEDULE OPTIONS on_execution_failure = 'pause', ignore_existing_backups, f
beforeBackup := th.env.Now()
firstRun := th.env.Now().Add(time.Minute).Round(time.Microsecond)

th.sqlDB.Exec(t, `SET application_name = 'backup_test'`)
schedules, err := th.createBackupSchedule(t, query, loc, firstRun)
if err != nil {
t.Fatal(err)
Expand All @@ -1335,6 +1336,7 @@ WITH SCHEDULE OPTIONS on_execution_failure = 'pause', ignore_existing_backups, f
OnPreviousRunning: "WAIT",
IgnoreExistingBackup: true,
CustomFirstRunTime: firstRun.UnixNano(),
ApplicationName: "backup_test",
}

requireRecoveryEvent(t, beforeBackup.UnixNano(), createdScheduleEventType, expectedCreateSchedule)
Expand Down
7 changes: 4 additions & 3 deletions pkg/ccl/backupccl/restore_planning.go
Original file line number Diff line number Diff line change
Expand Up @@ -1867,7 +1867,7 @@ func doRestorePlan(
}
resultsCh <- tree.Datums{tree.NewDInt(tree.DInt(jobID))}
collectRestoreTelemetry(ctx, jobID, restoreDetails, intoDB, newDBName, subdir, restoreStmt,
descsByTablePattern, restoreDBs, asOfInterval, debugPauseOn)
descsByTablePattern, restoreDBs, asOfInterval, debugPauseOn, p.SessionData().ApplicationName)
return nil
}

Expand Down Expand Up @@ -1901,7 +1901,7 @@ func doRestorePlan(
return err
}
collectRestoreTelemetry(ctx, sj.ID(), restoreDetails, intoDB, newDBName, subdir, restoreStmt,
descsByTablePattern, restoreDBs, asOfInterval, debugPauseOn)
descsByTablePattern, restoreDBs, asOfInterval, debugPauseOn, p.SessionData().ApplicationName)
if err := sj.Start(ctx); err != nil {
return err
}
Expand All @@ -1923,6 +1923,7 @@ func collectRestoreTelemetry(
restoreDBs []catalog.DatabaseDescriptor,
asOfInterval int64,
debugPauseOn string,
applicationName string,
) {
telemetry.Count("restore.total.started")
if restoreStmt.DescriptorCoverage == tree.AllDescriptors {
Expand All @@ -1935,7 +1936,7 @@ func collectRestoreTelemetry(
}

logRestoreTelemetry(ctx, jobID, details, intoDB, newDBName, subdir, asOfInterval, restoreStmt.Options,
descsByTablePattern, restoreDBs, debugPauseOn)
descsByTablePattern, restoreDBs, debugPauseOn, applicationName)
}

func filteredUserCreatedDescriptors(
Expand Down
4 changes: 4 additions & 0 deletions pkg/jobs/jobspb/jobs.proto
Original file line number Diff line number Diff line change
Expand Up @@ -260,6 +260,10 @@ message BackupDetails {
// timestamp and the timestamp resolved by the AS OF SYSTEM TIME expression.
// The interval is expressed in nanoseconds.
int64 as_of_interval = 22;

// ApplicationName is the application name in the session where the backup was
// invoked.
string application_name = 23;
}

message BackupProgress {
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -537,6 +537,7 @@ go_test(
"comment_on_constraint_test.go",
"comment_on_database_test.go",
"comment_on_index_test.go",
"comment_on_schema_test.go",
"comment_on_table_test.go",
"conn_executor_internal_test.go",
"conn_executor_savepoints_test.go",
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/comment_on_column_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -181,7 +181,7 @@ func TestCommentOnColumnWhenDropColumn(t *testing.T) {
t.Fatal(err)
}

t.Fatal("comment remain")
t.Fatal("comment remaining in system.comments despite drop")
}
}

Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/comment_on_database_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -104,6 +104,6 @@ func TestCommentOnDatabaseWhenDrop(t *testing.T) {
t.Fatal(err)
}

t.Fatal("dropped comment remain comment")
t.Fatal("comment remaining in system.comments despite drop")
}
}
109 changes: 109 additions & 0 deletions pkg/sql/comment_on_schema_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,109 @@
// 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_test

import (
"context"
gosql "database/sql"
"testing"

"github.com/cockroachdb/cockroach/pkg/sql/tests"
"github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/errors"
)

func TestCommentOnSchema(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)

params, _ := tests.CreateTestServerParams()
s, db, _ := serverutils.StartServer(t, params)
defer s.Stopper().Stop(context.Background())

if _, err := db.Exec(`
CREATE SCHEMA d;
`); err != nil {
t.Fatal(err)
}

testCases := []struct {
exec string
query string
expect gosql.NullString
}{
{
`COMMENT ON SCHEMA d IS 'foo'`,
`SELECT obj_description(oid, 'pg_namespace') FROM pg_namespace WHERE nspname = 'd'`,
gosql.NullString{String: `foo`, Valid: true},
},
{
`ALTER SCHEMA d RENAME TO d2`,
`SELECT obj_description(oid, 'pg_namespace') FROM pg_namespace WHERE nspname = 'd2'`,
gosql.NullString{String: `foo`, Valid: true},
},
{
`COMMENT ON SCHEMA d2 IS NULL`,
`SELECT obj_description(oid, 'pg_namespace') FROM pg_namespace WHERE nspname = 'd2'`,
gosql.NullString{Valid: false},
},
}

for _, tc := range testCases {
if _, err := db.Exec(tc.exec); err != nil {
t.Fatal(err)
}

row := db.QueryRow(tc.query)
var comment gosql.NullString
if err := row.Scan(&comment); err != nil {
t.Fatal(err)
}
if tc.expect != comment {
t.Fatalf("expected comment %v, got %v", tc.expect, comment)
}
}
}

func TestCommentOnSchemaWhenDrop(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)

params, _ := tests.CreateTestServerParams()
s, db, _ := serverutils.StartServer(t, params)
defer s.Stopper().Stop(context.Background())

if _, err := db.Exec(`
CREATE SCHEMA d;
`); err != nil {
t.Fatal(err)
}

if _, err := db.Exec(`COMMENT ON SCHEMA d IS 'foo'`); err != nil {
t.Fatal(err)
}

if _, err := db.Exec(`DROP SCHEMA d`); err != nil {
t.Fatal(err)
}

row := db.QueryRow(`SELECT comment FROM system.comments LIMIT 1`)
var comment string
err := row.Scan(&comment)
if !errors.Is(err, gosql.ErrNoRows) {
if err != nil {
t.Fatal(err)
}

t.Fatal("comment remaining in system.comments despite drop")
}
}
2 changes: 1 addition & 1 deletion pkg/sql/comment_on_table_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -108,6 +108,6 @@ func TestCommentOnTableWhenDrop(t *testing.T) {
t.Fatal(err)
}

t.Fatal("dropped comment remain comment")
t.Fatal("comment remaining in system.comments despite drop")
}
}
2 changes: 2 additions & 0 deletions pkg/sql/sem/builtins/pg_builtins.go
Original file line number Diff line number Diff line change
Expand Up @@ -2176,6 +2176,8 @@ func getCatalogOidForComments(catalogName string) (id int, ok bool) {
return catconstants.PgCatalogDescriptionTableID, true
case "pg_constraint":
return catconstants.PgCatalogConstraintTableID, true
case "pg_namespace":
return catconstants.PgCatalogNamespaceTableID, true
default:
// We currently only support comments on pg_class objects
// (columns, tables) in this context.
Expand Down
1 change: 0 additions & 1 deletion pkg/util/log/eventpb/events.proto
Original file line number Diff line number Diff line change
Expand Up @@ -46,7 +46,6 @@ message CommonSQLEventDetails {
// The application name for the session where the event was emitted.
// This is included in the event to ease filtering of logging output
// by application.
// Application names starting with a dollar sign (`$`) are not considered sensitive.
string application_name = 4 [(gogoproto.jsontag) = ",omitempty", (gogoproto.moretags) = "redact:\"nonsensitive\""];

// The mapping of SQL placeholders to their values, for prepared statements.
Expand Down
10 changes: 10 additions & 0 deletions pkg/util/log/eventpb/json_encode_generated.go

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

2 changes: 2 additions & 0 deletions pkg/util/log/eventpb/telemetry.proto
Original file line number Diff line number Diff line change
Expand Up @@ -341,6 +341,8 @@ message RecoveryEvent {
// created even if a backup is already present in its destination.
bool ignore_existing_backup = 29 [(gogoproto.jsontag) = ",omitempty"];

// The application name for the session where recovery event was created.
string application_name = 30 [(gogoproto.jsontag) = ",omitempty", (gogoproto.moretags) = "redact:\"nonsensitive\""];
}

// SchemaDescriptor is an event for schema telemetry, whose purpose is
Expand Down

0 comments on commit 0efe4f5

Please sign in to comment.