Skip to content

Commit 7ca9db8

Browse files
craig[bot]michae2dtxinhaozcthumuluru-crdb
committed
144918: sql, opt: add UpdateSwap and DeleteSwap to exec factory r=DrewKimball a=michae2 This set of commits adds UpdateSwap and DeleteSwap constructors to the exec factory. The next PR will actually call these constructor functions during execbuild. See individual commits for details. Informs: #144503 Release note: None 148537: jobs: remove deprecated custom per-job auth r=asg0451 a=dt This was deprecated a couple releases ago in favor of using more standard approaches of either a) telling users to grant or revoke membership in a role that has ownership of a job to manage access to that job or b) features built on top of jobs can have their own control statements that perform their own auth checks before modifying or creating jobs if they so choose. With this having been deprecated for a couple of major releases now with the public docs suggesting using role membership instead, it can now be deleted to simplify the auth checks the jobs system needs to perform, paving the way for replacing the complex logic in the vtable for SHOW JOBS with a simple view instead. Release note (ops change): Non-admin users no longer have access to changefeed jobs they do not own and which are not owned by a role of which they are a member, regardless of whether they have the changefeed privilege on the table or tables those jobs may be watching. Epic: CRDB-48791. 148604: roachtest: update wal-failover/among-stores/with-progress to use process monitor r=xinhaoz a=xinhaoz Epic: none 148609: rpc: remove DRPC dependency from `Peer` and `Connection` generics r=cthumuluru-crdb a=cthumuluru-crdb Although `Peer` and `Connection` are generic and intended to support both gRPC and DRPC connections, the current implementation has a hardcoded dependency on DRPC. As a result, a DRPC connection is always dialed, regardless of the intended type. This PR removes the direct dependency on DRPC, allowing the appropriate connection type (gRPC or DRPC) to be used based on the generic parameter. Epic: CRDB-48923 Fixes: none Release note: none Co-authored-by: Michael Erickson <[email protected]> Co-authored-by: David Taylor <[email protected]> Co-authored-by: Xin Hao Zhang <[email protected]> Co-authored-by: Chandra Thumuluru <[email protected]>
5 parents fafb5e5 + c2447a6 + 1a490cb + f594ee2 + 5be50d0 commit 7ca9db8

31 files changed

+601
-522
lines changed

pkg/ccl/changefeedccl/alter_changefeed_stmt.go

Lines changed: 2 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -113,11 +113,8 @@ func alterChangefeedPlanHook(
113113
if err != nil {
114114
return err
115115
}
116-
getLegacyPayload := func(ctx context.Context) (*jobspb.Payload, error) {
117-
return &jobPayload, nil
118-
}
119-
err = jobsauth.AuthorizeAllowLegacyAuth(
120-
ctx, p, jobID, getLegacyPayload, jobPayload.UsernameProto.Decode(), jobPayload.Type(), jobsauth.ControlAccess, globalPrivileges,
116+
err = jobsauth.Authorize(
117+
ctx, p, jobID, jobPayload.UsernameProto.Decode(), jobsauth.ControlAccess, globalPrivileges,
121118
)
122119
if err != nil {
123120
return err

pkg/ccl/changefeedccl/alter_changefeed_test.go

Lines changed: 19 additions & 6 deletions
Original file line numberDiff line numberDiff line change
@@ -82,6 +82,8 @@ func TestAlterChangefeedAddTargetPrivileges(t *testing.T) {
8282
`CREATE TABLE table_b (id int, type type_a)`,
8383
`CREATE TABLE table_c (id int, type type_a)`,
8484
`CREATE USER feedCreator`,
85+
`CREATE ROLE feedowner`,
86+
`GRANT feedowner TO feedCreator`,
8587
`GRANT SELECT ON table_a TO feedCreator`,
8688
`GRANT CHANGEFEED ON table_a TO feedCreator`,
8789
`CREATE EXTERNAL CONNECTION "first" AS 'kafka://nope'`,
@@ -123,22 +125,24 @@ func TestAlterChangefeedAddTargetPrivileges(t *testing.T) {
123125
row.Scan(&jobID)
124126
userDB.Exec(t, `PAUSE JOB $1`, jobID)
125127
waitForJobState(userDB, t, catpb.JobID(jobID), `paused`)
128+
userDB.Exec(t, `ALTER JOB $1 OWNER TO feedowner`, jobID)
126129
})
127130

128131
// user1 is missing the CHANGEFEED privilege on table_b and table_c.
129132
withUser(t, "user1", func(userDB *sqlutils.SQLRunner) {
130133
userDB.ExpectErr(t,
131-
"user user1 requires the CHANGEFEED privilege on all target tables to be able to run an enterprise changefeed",
134+
"user user1 does not have privileges for job",
132135
fmt.Sprintf("ALTER CHANGEFEED %d ADD table_b, table_c set sink='external://second'", jobID),
133136
)
134137
})
135138
rootDB.Exec(t, `GRANT CHANGEFEED ON table_b TO user1`)
136139
withUser(t, "user1", func(userDB *sqlutils.SQLRunner) {
137140
userDB.ExpectErr(t,
138-
"user user1 requires the CHANGEFEED privilege on all target tables to be able to run an enterprise changefeed",
141+
"user user1 does not have privileges for job",
139142
fmt.Sprintf("ALTER CHANGEFEED %d ADD table_b, table_c set sink='external://second'", jobID),
140143
)
141144
})
145+
rootDB.Exec(t, `GRANT feedowner TO user1`)
142146
rootDB.Exec(t, `GRANT CHANGEFEED ON table_c TO user1`)
143147
withUser(t, "user1", func(userDB *sqlutils.SQLRunner) {
144148
userDB.Exec(t,
@@ -175,19 +179,27 @@ func TestAlterChangefeedAddTargetPrivileges(t *testing.T) {
175179
row.Scan(&jobID)
176180
userDB.Exec(t, `PAUSE JOB $1`, jobID)
177181
waitForJobState(userDB, t, catpb.JobID(jobID), `paused`)
182+
userDB.Exec(t, `ALTER JOB $1 OWNER TO feedowner`, jobID)
178183
})
179184

180185
// user2 is missing the SELECT privilege on table_b and table_c.
181186
withUser(t, "user2", func(userDB *sqlutils.SQLRunner) {
182187
userDB.ExpectErr(t,
183-
"pq: user user2 with CONTROLCHANGEFEED role option requires the SELECT privilege on all target tables to be able to run an enterprise changefeed",
188+
"pq: user user2 does not have privileges for job",
184189
fmt.Sprintf("ALTER CHANGEFEED %d ADD table_b, table_c set sink='kafka://bar'", jobID),
185190
)
186191
})
187192
rootDB.Exec(t, `GRANT SELECT ON table_b TO user2`)
188193
withUser(t, "user2", func(userDB *sqlutils.SQLRunner) {
189194
userDB.ExpectErr(t,
190-
"pq: user user2 with CONTROLCHANGEFEED role option requires the SELECT privilege on all target tables to be able to run an enterprise changefeed",
195+
"pq: user user2 does not have privileges for job",
196+
fmt.Sprintf("ALTER CHANGEFEED %d ADD table_b, table_c set sink='kafka://bar'", jobID),
197+
)
198+
})
199+
rootDB.Exec(t, `GRANT feedowner TO user2`)
200+
withUser(t, "user2", func(userDB *sqlutils.SQLRunner) {
201+
userDB.ExpectErr(t,
202+
"requires the SELECT privilege on all target tables",
191203
fmt.Sprintf("ALTER CHANGEFEED %d ADD table_b, table_c set sink='kafka://bar'", jobID),
192204
)
193205
})
@@ -1859,6 +1871,7 @@ func TestAlterChangefeedAccessControl(t *testing.T) {
18591871
})
18601872
rootDB.Exec(t, "PAUSE job $1", currentFeed.JobID())
18611873
waitForJobState(rootDB, t, currentFeed.JobID(), `paused`)
1874+
rootDB.Exec(t, "ALTER JOB $1 OWNER TO feedowner", currentFeed.JobID())
18621875

18631876
// Verify who can modify the existing changefeed.
18641877
asUser(t, f, `userWithAllGrants`, func(userDB *sqlutils.SQLRunner) {
@@ -1872,10 +1885,10 @@ func TestAlterChangefeedAccessControl(t *testing.T) {
18721885
userDB.ExpectErr(t, "pq: user jobcontroller requires the CHANGEFEED privilege on all target tables to be able to run an enterprise changefeed", fmt.Sprintf(`ALTER CHANGEFEED %d DROP table_b`, currentFeed.JobID()))
18731886
})
18741887
asUser(t, f, `userWithSomeGrants`, func(userDB *sqlutils.SQLRunner) {
1875-
userDB.ExpectErr(t, "pq: user userwithsomegrants does not have CHANGEFEED privilege on relation table_b", fmt.Sprintf(`ALTER CHANGEFEED %d ADD table_b`, currentFeed.JobID()))
1888+
userDB.ExpectErr(t, "does not have privileges for job", fmt.Sprintf(`ALTER CHANGEFEED %d ADD table_b`, currentFeed.JobID()))
18761889
})
18771890
asUser(t, f, `regularUser`, func(userDB *sqlutils.SQLRunner) {
1878-
userDB.ExpectErr(t, "pq: user regularuser does not have CHANGEFEED privilege on relation (table_a|table_b)", fmt.Sprintf(`ALTER CHANGEFEED %d ADD table_b`, currentFeed.JobID()))
1891+
userDB.ExpectErr(t, "does not have privileges for job", fmt.Sprintf(`ALTER CHANGEFEED %d ADD table_b`, currentFeed.JobID()))
18791892
})
18801893
closeCf()
18811894

pkg/ccl/changefeedccl/authorization.go

Lines changed: 0 additions & 44 deletions
Original file line numberDiff line numberDiff line change
@@ -11,8 +11,6 @@ import (
1111

1212
"github.com/cockroachdb/cockroach/pkg/ccl/changefeedccl/changefeedbase"
1313
"github.com/cockroachdb/cockroach/pkg/cloud/externalconn"
14-
"github.com/cockroachdb/cockroach/pkg/jobs/jobsauth"
15-
"github.com/cockroachdb/cockroach/pkg/jobs/jobspb"
1614
"github.com/cockroachdb/cockroach/pkg/sql"
1715
"github.com/cockroachdb/cockroach/pkg/sql/catalog"
1816
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode"
@@ -135,45 +133,3 @@ func authorizeUserToCreateChangefeed(
135133

136134
return nil
137135
}
138-
139-
// AuthorizeChangefeedJobAccess determines if a user has access to the changefeed job denoted
140-
// by the supplied jobID and payload.
141-
func AuthorizeChangefeedJobAccess(
142-
ctx context.Context,
143-
a jobsauth.AuthorizationAccessor,
144-
jobID jobspb.JobID,
145-
getLegacyPayload func(ctx context.Context) (*jobspb.Payload, error),
146-
) error {
147-
payload, err := getLegacyPayload(ctx)
148-
if err != nil {
149-
return err
150-
}
151-
specs, ok := payload.UnwrapDetails().(jobspb.ChangefeedDetails)
152-
if !ok {
153-
return errors.Newf("could not unwrap details from the payload of job %d", jobID)
154-
}
155-
156-
if len(specs.TargetSpecifications) == 0 {
157-
return pgerror.Newf(pgcode.InsufficientPrivilege, "job contains no tables on which the user has %s privilege", privilege.CHANGEFEED)
158-
}
159-
160-
for _, spec := range specs.TargetSpecifications {
161-
err := a.CheckPrivilegeForTableID(ctx, spec.TableID, privilege.CHANGEFEED)
162-
if err != nil {
163-
// When performing SHOW JOBS or SHOW CHANGEFEED JOBS, there may be old changefeed
164-
// records that reference tables which have been dropped or are being
165-
// dropped. In this case, we would prefer to skip the permissions check on
166-
// the dropped descriptor.
167-
if pgerror.GetPGCode(err) == pgcode.UndefinedTable || errors.Is(err, catalog.ErrDescriptorDropped) {
168-
continue
169-
}
170-
171-
return err
172-
}
173-
}
174-
return nil
175-
}
176-
177-
func init() {
178-
jobsauth.RegisterAuthorizer(jobspb.TypeChangefeed, AuthorizeChangefeedJobAccess)
179-
}

pkg/ccl/changefeedccl/changefeed_test.go

Lines changed: 3 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -3749,6 +3749,7 @@ func TestChangefeedJobControl(t *testing.T) {
37493749
asUser(t, f, `adminUser`, func(userDB *sqlutils.SQLRunner) {
37503750
userDB.Exec(t, "PAUSE job $1", currentFeed.JobID())
37513751
waitForJobState(userDB, t, currentFeed.JobID(), "paused")
3752+
userDB.Exec(t, "ALTER JOB $1 OWNER TO feedowner", currentFeed.JobID())
37523753
})
37533754
asUser(t, f, `userWithAllGrants`, func(userDB *sqlutils.SQLRunner) {
37543755
userDB.Exec(t, "RESUME job $1", currentFeed.JobID())
@@ -3759,10 +3760,10 @@ func TestChangefeedJobControl(t *testing.T) {
37593760
waitForJobState(userDB, t, currentFeed.JobID(), "running")
37603761
})
37613762
asUser(t, f, `userWithSomeGrants`, func(userDB *sqlutils.SQLRunner) {
3762-
userDB.ExpectErr(t, "user userwithsomegrants does not have CHANGEFEED privilege on relation table_b", "PAUSE job $1", currentFeed.JobID())
3763+
userDB.ExpectErr(t, "does not have privileges for job", "PAUSE job $1", currentFeed.JobID())
37633764
})
37643765
asUser(t, f, `regularUser`, func(userDB *sqlutils.SQLRunner) {
3765-
userDB.ExpectErr(t, "user regularuser does not have CHANGEFEED privilege on relation (table_a|table_b)", "PAUSE job $1", currentFeed.JobID())
3766+
userDB.ExpectErr(t, "does not have privileges for job", "PAUSE job $1", currentFeed.JobID())
37663767
})
37673768
closeCf()
37683769

pkg/ccl/changefeedccl/helpers_test.go

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -1736,6 +1736,7 @@ func ChangefeedJobPermissionsTestSetup(t *testing.T, s TestServer) {
17361736

17371737
`CREATE USER adminUser`,
17381738
`GRANT ADMIN TO adminUser`,
1739+
`CREATE ROLE feedowner`,
17391740

17401741
`CREATE USER otherAdminUser`,
17411742
`GRANT ADMIN TO otherAdminUser`,
@@ -1747,6 +1748,7 @@ func ChangefeedJobPermissionsTestSetup(t *testing.T, s TestServer) {
17471748
`CREATE USER jobController with CONTROLJOB`,
17481749

17491750
`CREATE USER userWithAllGrants`,
1751+
`GRANT feedowner TO userWithAllGrants`,
17501752
`GRANT CHANGEFEED ON table_a TO userWithAllGrants`,
17511753
`GRANT CHANGEFEED ON table_b TO userWithAllGrants`,
17521754

pkg/ccl/changefeedccl/show_changefeed_jobs_test.go

Lines changed: 1 addition & 12 deletions
Original file line numberDiff line numberDiff line change
@@ -568,7 +568,6 @@ func TestShowChangefeedJobsAuthorization(t *testing.T) {
568568
require.NoError(t, err)
569569
jobID = successfulFeed.(cdctest.EnterpriseTestFeed).JobID()
570570
}
571-
rootDB := sqlutils.MakeSQLRunner(s.DB)
572571

573572
// Create a changefeed and assert who can see it.
574573
asUser(t, f, `feedCreator`, func(userDB *sqlutils.SQLRunner) {
@@ -579,7 +578,7 @@ func TestShowChangefeedJobsAuthorization(t *testing.T) {
579578
userDB.CheckQueryResults(t, `SELECT job_id FROM [SHOW CHANGEFEED JOBS]`, [][]string{{expectedJobIDStr}})
580579
})
581580
asUser(t, f, `userWithAllGrants`, func(userDB *sqlutils.SQLRunner) {
582-
userDB.CheckQueryResults(t, `SELECT job_id FROM [SHOW CHANGEFEED JOBS]`, [][]string{{expectedJobIDStr}})
581+
userDB.CheckQueryResults(t, `SELECT job_id FROM [SHOW CHANGEFEED JOBS]`, [][]string{})
583582
})
584583
asUser(t, f, `userWithSomeGrants`, func(userDB *sqlutils.SQLRunner) {
585584
userDB.CheckQueryResults(t, `SELECT job_id FROM [SHOW CHANGEFEED JOBS]`, [][]string{})
@@ -590,16 +589,6 @@ func TestShowChangefeedJobsAuthorization(t *testing.T) {
590589
asUser(t, f, `regularUser`, func(userDB *sqlutils.SQLRunner) {
591590
userDB.CheckQueryResults(t, `SELECT job_id FROM [SHOW CHANGEFEED JOBS]`, [][]string{})
592591
})
593-
594-
// Assert behavior when one of the tables is dropped.
595-
rootDB.Exec(t, "DROP TABLE table_b")
596-
// Having CHANGEFEED on only table_a is now sufficient.
597-
asUser(t, f, `userWithSomeGrants`, func(userDB *sqlutils.SQLRunner) {
598-
userDB.CheckQueryResults(t, `SELECT job_id FROM [SHOW CHANGEFEED JOBS]`, [][]string{{expectedJobIDStr}})
599-
})
600-
asUser(t, f, `regularUser`, func(userDB *sqlutils.SQLRunner) {
601-
userDB.CheckQueryResults(t, `SELECT job_id FROM [SHOW CHANGEFEED JOBS]`, [][]string{})
602-
})
603592
}
604593

605594
// Only enterprise sinks create jobs.

pkg/cmd/roachtest/tests/disk_stall.go

Lines changed: 11 additions & 24 deletions
Original file line numberDiff line numberDiff line change
@@ -11,17 +11,18 @@ import (
1111
"math/rand"
1212
"strconv"
1313
"strings"
14-
"sync"
1514
"time"
1615

1716
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/cluster"
1817
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/clusterstats"
1918
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/option"
2019
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/registry"
2120
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/roachtestutil"
21+
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/roachtestutil/task"
2222
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/spec"
2323
"github.com/cockroachdb/cockroach/pkg/cmd/roachtest/test"
2424
"github.com/cockroachdb/cockroach/pkg/roachprod/install"
25+
"github.com/cockroachdb/cockroach/pkg/roachprod/logger"
2526
"github.com/cockroachdb/cockroach/pkg/roachprod/prometheus"
2627
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
2728
"github.com/stretchr/testify/require"
@@ -435,6 +436,7 @@ func registerDiskStalledWALFailoverWithProgress(r registry.Registry) {
435436
SkipPostValidations: registry.PostValidationNoDeadNodes,
436437
EncryptionSupport: registry.EncryptionMetamorphic,
437438
Leases: registry.MetamorphicLeases,
439+
Monitor: true,
438440
Run: func(ctx context.Context, t test.Test, c cluster.Cluster) {
439441
runDiskStalledWALFailoverWithProgress(ctx, t, c)
440442
},
@@ -484,7 +486,7 @@ func runDiskStalledWALFailoverWithProgress(ctx context.Context, t test.Test, c c
484486

485487
t.Status("starting oscillating workload and disk stall pattern")
486488
testStartedAt := timeutil.Now()
487-
m := c.NewMonitor(ctx, c.CRDBNodes())
489+
g := t.NewGroup(task.WithContext(ctx))
488490

489491
// Setup stats collector.
490492
promCfg := &prometheus.Config{}
@@ -511,7 +513,6 @@ func runDiskStalledWALFailoverWithProgress(ctx context.Context, t test.Test, c c
511513
for timeutil.Since(testStartedAt) < testDuration {
512514
if t.Failed() {
513515
t.Fatalf("test failed, stopping further iterations")
514-
return
515516
}
516517

517518
workloadWaitDur := operationWaitBase + time.Duration(rand.Int63n(int64(waitJitterMax)))
@@ -521,11 +522,7 @@ func runDiskStalledWALFailoverWithProgress(ctx context.Context, t test.Test, c c
521522
workloadStarted := make(chan struct{})
522523
workloadFinished := make(chan struct{})
523524

524-
var wg sync.WaitGroup
525-
wg.Add(1)
526-
m.Go(func(ctx context.Context) error {
527-
defer wg.Done()
528-
525+
g.Go(func(ctx context.Context, _ *logger.Logger) error {
529526
select {
530527
case <-ctx.Done():
531528
t.Fatalf("context done before workload started: %s", ctx.Err())
@@ -540,14 +537,12 @@ func runDiskStalledWALFailoverWithProgress(ctx context.Context, t test.Test, c c
540537
return nil
541538
}
542539
return nil
543-
})
540+
}, task.Name("workload-run"))
544541

545542
// Collecting QPS samples while the workload is running and verify
546543
// that the throughput is within errorTolerance of the mean.
547544
var samples []float64
548-
wg.Add(1)
549-
m.Go(func(ctx context.Context) error {
550-
defer wg.Done()
545+
g.Go(func(ctx context.Context, _ *logger.Logger) error {
551546

552547
// Wait for workload to start.
553548
select {
@@ -602,17 +597,15 @@ func runDiskStalledWALFailoverWithProgress(ctx context.Context, t test.Test, c c
602597

603598
t.Status(fmt.Sprintf("workload finished, %d samples collected", len(samples)))
604599
return nil
605-
})
600+
}, task.Name("qps-sampling"))
606601

607602
// Every 4th iteration, we'll skip the disk stall phase.
608603
if iteration%4 != 0 {
609604
// Calculate next stall phase with jitter.
610605
diskStallWaitDur := operationWaitBase + time.Duration(rand.Int63n(int64(waitJitterMax)))
611606
t.Status("next stall phase in ", diskStallWaitDur)
612607

613-
wg.Add(1)
614-
m.Go(func(ctx context.Context) error {
615-
defer wg.Done()
608+
g.Go(func(ctx context.Context, _ *logger.Logger) error {
616609
select {
617610
case <-ctx.Done():
618611
t.Fatalf("context done before stall started: %s", ctx.Err())
@@ -650,13 +643,13 @@ func runDiskStalledWALFailoverWithProgress(ctx context.Context, t test.Test, c c
650643
}
651644

652645
return nil
653-
})
646+
}, task.Name("disk-stall-phase"))
654647
} else {
655648
t.Status("skipping disk stall phase for this iteration")
656649
}
657650

658651
// Wait for all goroutines to complete.
659-
wg.Wait()
652+
g.Wait()
660653

661654
// Validate throughput samples are within tolerance.
662655
meanThroughput := roachtestutil.GetMeanOverLastN(len(samples), samples)
@@ -698,12 +691,6 @@ func runDiskStalledWALFailoverWithProgress(ctx context.Context, t test.Test, c c
698691
if durInFailover < 10*time.Minute {
699692
t.Errorf("expected s1 to spend at least 10m writing to secondary, but spent %s", durInFailover)
700693
}
701-
702-
// Wait for the workload to finish (if it hasn't already).
703-
m.Wait()
704-
705-
// Shut down the nodes, allowing any devices to be unmounted during cleanup.
706-
c.Stop(ctx, t.L(), option.DefaultStopOpts(), c.CRDBNodes())
707694
}
708695

709696
func getProcessStartMonotonic(

pkg/crosscluster/producer/replication_manager.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -442,7 +442,7 @@ func (r *replicationStreamManagerImpl) AuthorizeViaJob(
442442
}
443443

444444
if err := jobsauth.Authorize(
445-
ctx, planHook, jobspb.JobID(streamID), planHook.User(), jobspb.TypeReplicationStreamProducer, jobsauth.ControlAccess, globalPrivileges,
445+
ctx, planHook, jobspb.JobID(streamID), planHook.User(), jobsauth.ControlAccess, globalPrivileges,
446446
); err != nil {
447447
return err
448448
}

pkg/jobs/jobsauth/BUILD.bazel

Lines changed: 0 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -13,7 +13,6 @@ go_library(
1313
"//pkg/sql/pgwire/pgerror",
1414
"//pkg/sql/privilege",
1515
"//pkg/sql/roleoption",
16-
"@com_github_cockroachdb_errors//:errors",
1716
],
1817
)
1918

0 commit comments

Comments
 (0)