From ac3ba1dac3dc8114a2bbc2297d578436c93b2a2b Mon Sep 17 00:00:00 2001 From: Miral Gadani Date: Thu, 1 Sep 2022 15:11:44 -0400 Subject: [PATCH 01/18] roachtest: Return vm createoptions to allow computed values to be reported. Some runtime values are probabilistically set or otherwise have defaults which are not evident at the time we report issues. This change returns the vm options associated with a created cluster, from which we can extract the final computed values for such fields including the type of filesystem and whether encryption is enabled. Release justification: test-only change Release note: none --- pkg/cmd/roachtest/cluster.go | 62 +++++++-------- pkg/cmd/roachtest/cluster/BUILD.bazel | 1 - .../roachtest/cluster/cluster_interface.go | 3 +- pkg/cmd/roachtest/test_runner.go | 75 ++++++++++++------- pkg/cmd/roachtest/test_test.go | 5 +- .../roachtest/tests/multitenant_fairness.go | 2 +- 6 files changed, 84 insertions(+), 64 deletions(-) diff --git a/pkg/cmd/roachtest/cluster.go b/pkg/cmd/roachtest/cluster.go index c19e0d2ba707..339e0e813acd 100644 --- a/pkg/cmd/roachtest/cluster.go +++ b/pkg/cmd/roachtest/cluster.go @@ -838,9 +838,9 @@ func (f *clusterFactory) clusterMock(cfg clusterConfig) *clusterImpl { // NOTE: setTest() needs to be called before a test can use this cluster. func (f *clusterFactory) newCluster( ctx context.Context, cfg clusterConfig, setStatus func(string), teeOpt logger.TeeOptType, -) (*clusterImpl, error) { +) (*clusterImpl, *vm.CreateOpts, error) { if ctx.Err() != nil { - return nil, errors.Wrap(ctx.Err(), "newCluster") + return nil, nil, errors.Wrap(ctx.Err(), "newCluster") } if overrideFlagset != nil && overrideFlagset.Changed("nodes") { @@ -851,9 +851,9 @@ func (f *clusterFactory) newCluster( // For tests, use a mock cluster. c := f.clusterMock(cfg) if err := f.r.registerCluster(c); err != nil { - return nil, err + return nil, nil, err } - return c, nil + return c, nil, nil } if cfg.localCluster { @@ -876,7 +876,7 @@ func (f *clusterFactory) newCluster( // We must release the allocation because cluster creation is not possible at this point. cfg.alloc.Release() - return nil, err + return nil, nil, err } if cfg.spec.Cloud != spec.Local { providerOptsContainer.SetProviderOpts(cfg.spec.Cloud, providerOpts) @@ -930,18 +930,18 @@ func (f *clusterFactory) newCluster( err = roachprod.Create(ctx, l, cfg.username, cfg.spec.NodeCount, createVMOpts, providerOptsContainer) if err == nil { if err := f.r.registerCluster(c); err != nil { - return nil, err + return nil, nil, err } c.status("idle") l.Close() - return c, nil + return c, &createVMOpts, nil } if errors.HasType(err, (*roachprod.ClusterAlreadyExistsError)(nil)) { // If the cluster couldn't be created because it existed already, bail. // In reality when this is hit is when running with the `local` flag // or a destroy from the previous iteration failed. - return nil, err + return nil, nil, err } l.PrintfCtx(ctx, "cluster creation failed, cleaning up in case it was partially created: %s", err) @@ -953,7 +953,7 @@ func (f *clusterFactory) newCluster( if i >= maxAttempts { // Here we have to release the alloc, as we are giving up. cfg.alloc.Release() - return nil, err + return nil, nil, err } // Try again to create the cluster. } @@ -1129,13 +1129,13 @@ func (c *clusterImpl) Node(i int) option.NodeListOption { // FetchLogs downloads the logs from the cluster using `roachprod get`. // The logs will be placed in the test's artifacts dir. -func (c *clusterImpl) FetchLogs(ctx context.Context, t test.Test) error { +func (c *clusterImpl) FetchLogs(ctx context.Context, l *logger.Logger) error { if c.spec.NodeCount == 0 { // No nodes can happen during unit tests and implies nothing to do. return nil } - t.L().Printf("fetching logs\n") + l.Printf("fetching logs\n") c.status("fetching logs") // Don't hang forever if we can't fetch the logs. @@ -1146,14 +1146,14 @@ func (c *clusterImpl) FetchLogs(ctx context.Context, t test.Test) error { } if err := c.Get(ctx, c.l, "logs" /* src */, path /* dest */); err != nil { - t.L().Printf("failed to fetch logs: %v", err) + l.Printf("failed to fetch logs: %v", err) if ctx.Err() != nil { return errors.Wrap(err, "cluster.FetchLogs") } } if err := c.RunE(ctx, c.All(), "mkdir -p logs/redacted && ./cockroach debug merge-logs --redact logs/*.log > logs/redacted/combined.log"); err != nil { - t.L().Printf("failed to redact logs: %v", err) + l.Printf("failed to redact logs: %v", err) if ctx.Err() != nil { return err } @@ -1205,17 +1205,17 @@ func (c *clusterImpl) CopyRoachprodState(ctx context.Context) error { // the first available node. They can be visualized via: // // `COCKROACH_DEBUG_TS_IMPORT_FILE=tsdump.gob ./cockroach start-single-node --insecure --store=$(mktemp -d)` -func (c *clusterImpl) FetchTimeseriesData(ctx context.Context, t test.Test) error { +func (c *clusterImpl) FetchTimeseriesData(ctx context.Context, l *logger.Logger) error { return contextutil.RunWithTimeout(ctx, "fetch tsdata", 5*time.Minute, func(ctx context.Context) error { node := 1 for ; node <= c.spec.NodeCount; node++ { - db, err := c.ConnE(ctx, t.L(), node) + db, err := c.ConnE(ctx, l, node) if err == nil { err = db.Ping() db.Close() } if err != nil { - t.L().Printf("node %d not responding to SQL, trying next one", node) + l.Printf("node %d not responding to SQL, trying next one", node) continue } break @@ -1242,7 +1242,7 @@ func (c *clusterImpl) FetchTimeseriesData(ctx context.Context, t test.Test) erro ); err != nil { return errors.Wrap(err, "cluster.FetchTimeseriesData") } - db, err := c.ConnE(ctx, t.L(), node) + db, err := c.ConnE(ctx, l, node) if err != nil { return err } @@ -1278,13 +1278,13 @@ COCKROACH_DEBUG_TS_IMPORT_FILE=tsdump.gob cockroach start-single-node --insecure // FetchDebugZip downloads the debug zip from the cluster using `roachprod ssh`. // The logs will be placed in the test's artifacts dir. -func (c *clusterImpl) FetchDebugZip(ctx context.Context, t test.Test) error { +func (c *clusterImpl) FetchDebugZip(ctx context.Context, l *logger.Logger) error { if c.spec.NodeCount == 0 { // No nodes can happen during unit tests and implies nothing to do. return nil } - t.L().Printf("fetching debug zip\n") + l.Printf("fetching debug zip\n") c.status("fetching debug zip") // Don't hang forever if we can't fetch the debug zip. @@ -1305,7 +1305,7 @@ func (c *clusterImpl) FetchDebugZip(ctx context.Context, t test.Test) error { si := strconv.Itoa(i) cmd := []string{"./cockroach", "debug", "zip", "--exclude-files='*.log,*.txt,*.pprof'", "--url", "{pgurl:" + si + "}", zipName} if err := c.RunE(ctx, c.All(), cmd...); err != nil { - t.L().Printf("./cockroach debug zip failed: %v", err) + l.Printf("./cockroach debug zip failed: %v", err) if i < c.spec.NodeCount { continue } @@ -1380,14 +1380,14 @@ func (c *clusterImpl) FailOnReplicaDivergence(ctx context.Context, t *testImpl) // FetchDmesg grabs the dmesg logs if possible. This requires being able to run // `sudo dmesg` on the remote nodes. -func (c *clusterImpl) FetchDmesg(ctx context.Context, t test.Test) error { +func (c *clusterImpl) FetchDmesg(ctx context.Context, l *logger.Logger) error { if c.spec.NodeCount == 0 || c.IsLocal() { // No nodes can happen during unit tests and implies nothing to do. // Also, don't grab dmesg on local runs. return nil } - t.L().Printf("fetching dmesg\n") + l.Printf("fetching dmesg\n") c.status("fetching dmesg") // Don't hang forever. @@ -1411,7 +1411,7 @@ func (c *clusterImpl) FetchDmesg(ctx context.Context, t test.Test) error { if result.Err != nil { // Store `Run` errors to return later (after copying files from successful nodes). combinedDmesgError = errors.CombineErrors(combinedDmesgError, result.Err) - t.L().Printf("running dmesg failed on node %d: %v", result.Node, result.Err) + l.Printf("running dmesg failed on node %d: %v", result.Node, result.Err) } else { // Only run `Get` on successful nodes to avoid pseudo-failure on `Get` caused by an earlier failure on `Run`. successfulNodes = append(successfulNodes, int(result.Node)) @@ -1420,7 +1420,7 @@ func (c *clusterImpl) FetchDmesg(ctx context.Context, t test.Test) error { // Get dmesg files from successful nodes only. if err := c.Get(ctx, c.l, name /* src */, path /* dest */, successfulNodes); err != nil { - t.L().Printf("getting dmesg files failed: %v", err) + l.Printf("getting dmesg files failed: %v", err) return errors.Wrap(err, "cluster.FetchDmesg") } @@ -1431,14 +1431,14 @@ func (c *clusterImpl) FetchDmesg(ctx context.Context, t test.Test) error { // FetchJournalctl grabs the journalctl logs if possible. This requires being // able to run `sudo journalctl` on the remote nodes. -func (c *clusterImpl) FetchJournalctl(ctx context.Context, t test.Test) error { +func (c *clusterImpl) FetchJournalctl(ctx context.Context, l *logger.Logger) error { if c.spec.NodeCount == 0 || c.IsLocal() { // No nodes can happen during unit tests and implies nothing to do. // Also, don't grab journalctl on local runs. return nil } - t.L().Printf("fetching journalctl\n") + l.Printf("fetching journalctl\n") c.status("fetching journalctl") // Don't hang forever. @@ -1462,7 +1462,7 @@ func (c *clusterImpl) FetchJournalctl(ctx context.Context, t test.Test) error { if result.Err != nil { // Store `Run` errors to return later (after copying files from successful nodes). combinedJournalctlError = errors.CombineErrors(combinedJournalctlError, result.Err) - t.L().Printf("running journalctl failed on node %d: %v", result.Node, result.Err) + l.Printf("running journalctl failed on node %d: %v", result.Node, result.Err) } else { // Only run `Get` on successful nodes to avoid pseudo-failure on `Get` caused by an earlier failure on `Run`. successfulNodes = append(successfulNodes, int(result.Node)) @@ -1471,7 +1471,7 @@ func (c *clusterImpl) FetchJournalctl(ctx context.Context, t test.Test) error { // Get files from successful nodes only. if err := c.Get(ctx, c.l, name /* src */, path /* dest */, successfulNodes); err != nil { - t.L().Printf("getting files failed: %v", err) + l.Printf("getting files failed: %v", err) return errors.Wrap(err, "cluster.FetchJournalctl") } @@ -1481,7 +1481,7 @@ func (c *clusterImpl) FetchJournalctl(ctx context.Context, t test.Test) error { } // FetchCores fetches any core files on the cluster. -func (c *clusterImpl) FetchCores(ctx context.Context, t test.Test) error { +func (c *clusterImpl) FetchCores(ctx context.Context, l *logger.Logger) error { if c.spec.NodeCount == 0 || c.IsLocal() { // No nodes can happen during unit tests and implies nothing to do. // Also, don't grab dmesg on local runs. @@ -1493,11 +1493,11 @@ func (c *clusterImpl) FetchCores(ctx context.Context, t test.Test) error { // from having the cores, but we should push them straight into a temp // bucket on S3 instead. OTOH, the ROI of this may be low; I don't know // of a recent example where we've wanted the Core dumps. - t.L().Printf("skipped fetching cores\n") + l.Printf("skipped fetching cores\n") return nil } - t.L().Printf("fetching cores\n") + l.Printf("fetching cores\n") c.status("fetching cores") // Don't hang forever. The core files can be large, so we give a generous diff --git a/pkg/cmd/roachtest/cluster/BUILD.bazel b/pkg/cmd/roachtest/cluster/BUILD.bazel index 736dab20a95c..afd151fae747 100644 --- a/pkg/cmd/roachtest/cluster/BUILD.bazel +++ b/pkg/cmd/roachtest/cluster/BUILD.bazel @@ -13,7 +13,6 @@ go_library( deps = [ "//pkg/cmd/roachtest/option", "//pkg/cmd/roachtest/spec", - "//pkg/cmd/roachtest/test", "//pkg/roachprod/install", "//pkg/roachprod/logger", "//pkg/roachprod/prometheus", diff --git a/pkg/cmd/roachtest/cluster/cluster_interface.go b/pkg/cmd/roachtest/cluster/cluster_interface.go index cf016c39375a..eb3ecfe4af5c 100644 --- a/pkg/cmd/roachtest/cluster/cluster_interface.go +++ b/pkg/cmd/roachtest/cluster/cluster_interface.go @@ -17,7 +17,6 @@ import ( "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/option" "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/spec" - "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/test" "github.com/cockroachdb/cockroach/pkg/roachprod/install" "github.com/cockroachdb/cockroach/pkg/roachprod/logger" "github.com/cockroachdb/cockroach/pkg/roachprod/prometheus" @@ -130,7 +129,7 @@ type Cluster interface { ctx context.Context, l *logger.Logger, src, dest, branch string, node option.NodeListOption, ) error - FetchTimeseriesData(ctx context.Context, t test.Test) error + FetchTimeseriesData(ctx context.Context, l *logger.Logger) error RefetchCertsFromNode(ctx context.Context, node int) error StartGrafana(ctx context.Context, l *logger.Logger, promCfg *prometheus.Config) error diff --git a/pkg/cmd/roachtest/test_runner.go b/pkg/cmd/roachtest/test_runner.go index c43408db967a..fb7cd303c241 100644 --- a/pkg/cmd/roachtest/test_runner.go +++ b/pkg/cmd/roachtest/test_runner.go @@ -37,6 +37,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/internal/team" "github.com/cockroachdb/cockroach/pkg/roachprod/config" "github.com/cockroachdb/cockroach/pkg/roachprod/logger" + "github.com/cockroachdb/cockroach/pkg/roachprod/vm" "github.com/cockroachdb/cockroach/pkg/util/ctxgroup" "github.com/cockroachdb/cockroach/pkg/util/log" "github.com/cockroachdb/cockroach/pkg/util/quotapool" @@ -347,7 +348,7 @@ func defaultClusterAllocator( alloc *quotapool.IntAlloc, artifactsDir string, wStatus *workerStatus, - ) (*clusterImpl, error) { + ) (*clusterImpl, *vm.CreateOpts, error) { wStatus.SetStatus("creating cluster") defer wStatus.SetStatus("") @@ -357,7 +358,7 @@ func defaultClusterAllocator( logPath := filepath.Join(artifactsDir, runnerLogsDir, "cluster-create", existingClusterName+".log") clusterL, err := logger.RootLogger(logPath, lopt.tee) if err != nil { - return nil, err + return nil, nil, err } defer clusterL.Close() opt := attachOpt{ @@ -368,10 +369,10 @@ func defaultClusterAllocator( lopt.l.PrintfCtx(ctx, "Attaching to existing cluster %s for test %s", existingClusterName, t.Name) c, err := attachToExistingCluster(ctx, existingClusterName, clusterL, t.Cluster, opt, r.cr) if err == nil { - return c, nil + return c, nil, nil } if !errors.Is(err, errClusterNotFound) { - return nil, err + return nil, nil, err } // Fall through to create new cluster with name override. lopt.l.PrintfCtx( @@ -401,7 +402,7 @@ type clusterAllocatorFn func( alloc *quotapool.IntAlloc, artifactsDir string, wStatus *workerStatus, -) (*clusterImpl, error) +) (*clusterImpl, *vm.CreateOpts, error) // runWorker runs tests in a loop until work is exhausted. // @@ -561,13 +562,14 @@ func (r *testRunner) runWorker( } var clusterCreateErr error + var vmCreateOpts *vm.CreateOpts if !testToRun.canReuseCluster { // Create a new cluster if can't reuse or reuse attempt failed. // N.B. non-reusable cluster would have been destroyed above. wStatus.SetTest(nil /* test */, testToRun) wStatus.SetStatus("creating cluster") - c, clusterCreateErr = allocateCluster(ctx, testToRun.spec, testToRun.alloc, artifactsRootDir, wStatus) + c, vmCreateOpts, clusterCreateErr = allocateCluster(ctx, testToRun.spec, testToRun.alloc, artifactsRootDir, wStatus) if clusterCreateErr != nil { atomic.AddInt32(&r.numClusterErrs, 1) shout(ctx, l, stdout, "Unable to create (or reuse) cluster for test %s due to: %s.", @@ -623,7 +625,7 @@ func (r *testRunner) runWorker( // N.B. issue title is of the form "roachtest: ${t.spec.Name} failed" (see UnitTestFormatter). t.spec.Name = "cluster_creation" t.spec.Owner = registry.OwnerDevInf - r.maybePostGithubIssue(ctx, l, t, stdout, issueOutput) + r.maybePostGithubIssue(ctx, l, t, c, vmCreateOpts, stdout, issueOutput) // Restore test name and owner. t.spec.Name = oldName t.spec.Owner = oldOwner @@ -652,7 +654,7 @@ func (r *testRunner) runWorker( wStatus.SetTest(t, testToRun) wStatus.SetStatus("running test") - err = r.runTest(ctx, t, testToRun.runNum, testToRun.runCount, c, stdout, testL) + err = r.runTest(ctx, t, testToRun.runNum, testToRun.runCount, c, vmCreateOpts, stdout, testL) } } @@ -764,6 +766,7 @@ func (r *testRunner) runTest( runNum int, runCount int, c *clusterImpl, + vmCreateOpts *vm.CreateOpts, stdout io.Writer, l *logger.Logger, ) error { @@ -819,7 +822,7 @@ func (r *testRunner) runTest( shout(ctx, l, stdout, "--- FAIL: %s (%s)\n%s", runID, durationStr, output) - r.maybePostGithubIssue(ctx, l, t, stdout, output) + r.maybePostGithubIssue(ctx, l, t, c, vmCreateOpts, stdout, output) } else { shout(ctx, l, stdout, "--- PASS: %s (%s)", runID, durationStr) // If `##teamcity[testFailed ...]` is not present before `##teamCity[testFinished ...]`, @@ -1021,7 +1024,7 @@ func (r *testRunner) teardownTest( c.FailOnReplicaDivergence(ctx, t) if timedOut || t.Failed() { - r.collectClusterArtifacts(ctx, c, t) + r.collectClusterArtifacts(ctx, c, t.L()) } }) @@ -1061,7 +1064,13 @@ func (r *testRunner) shouldPostGithubIssue(t test.Test) bool { } func (r *testRunner) maybePostGithubIssue( - ctx context.Context, l *logger.Logger, t test.Test, stdout io.Writer, output string, + ctx context.Context, + l *logger.Logger, + t test.Test, + c *clusterImpl, + vmCreateOpts *vm.CreateOpts, + stdout io.Writer, + output string, ) { if !r.shouldPostGithubIssue(t) { return @@ -1104,6 +1113,15 @@ func (r *testRunner) maybePostGithubIssue( roachtestParam("ssd"): fmt.Sprintf("%d", spec.Cluster.SSDs), } + // these params can be probabilistically set if requested + if vmCreateOpts != nil { + clusterParams[roachtestParam("fs")] = vmCreateOpts.SSDOpts.FileSystem + } + + if c != nil { + clusterParams[roachtestParam("encrypted")] = fmt.Sprintf("%v", c.encAtRest) + } + req := issues.PostRequest{ MentionOnCreate: mention, ProjectColumnID: projColID, @@ -1134,7 +1152,10 @@ func (r *testRunner) maybePostGithubIssue( } // TODO(tbg): nothing in this method should have the `t`; they should have a `Logger` only. -func (r *testRunner) collectClusterArtifacts(ctx context.Context, c *clusterImpl, t test.Test) { +// Maybe done +func (r *testRunner) collectClusterArtifacts( + ctx context.Context, c *clusterImpl, l *logger.Logger, +) { // NB: fetch the logs even when we have a debug zip because // debug zip can't ever get the logs for down nodes. // We only save artifacts for failed tests in CI, so this @@ -1143,32 +1164,32 @@ func (r *testRunner) collectClusterArtifacts(ctx context.Context, c *clusterImpl // below has problems. For example, `debug zip` is known to // hang sometimes at the time of writing, see: // https://github.com/cockroachdb/cockroach/issues/39620 - t.L().PrintfCtx(ctx, "collecting cluster logs") + l.PrintfCtx(ctx, "collecting cluster logs") // Do this before collecting logs to make sure the file gets // downloaded below. if err := saveDiskUsageToLogsDir(ctx, c); err != nil { - t.L().Printf("failed to fetch disk uage summary: %s", err) + l.Printf("failed to fetch disk uage summary: %s", err) } - if err := c.FetchLogs(ctx, t); err != nil { - t.L().Printf("failed to download logs: %s", err) + if err := c.FetchLogs(ctx, l); err != nil { + l.Printf("failed to download logs: %s", err) } - if err := c.FetchDmesg(ctx, t); err != nil { - t.L().Printf("failed to fetch dmesg: %s", err) + if err := c.FetchDmesg(ctx, l); err != nil { + l.Printf("failed to fetch dmesg: %s", err) } - if err := c.FetchJournalctl(ctx, t); err != nil { - t.L().Printf("failed to fetch journalctl: %s", err) + if err := c.FetchJournalctl(ctx, l); err != nil { + l.Printf("failed to fetch journalctl: %s", err) } - if err := c.FetchCores(ctx, t); err != nil { - t.L().Printf("failed to fetch cores: %s", err) + if err := c.FetchCores(ctx, l); err != nil { + l.Printf("failed to fetch cores: %s", err) } if err := c.CopyRoachprodState(ctx); err != nil { - t.L().Printf("failed to copy roachprod state: %s", err) + l.Printf("failed to copy roachprod state: %s", err) } - if err := c.FetchTimeseriesData(ctx, t); err != nil { - t.L().Printf("failed to fetch timeseries data: %s", err) + if err := c.FetchTimeseriesData(ctx, l); err != nil { + l.Printf("failed to fetch timeseries data: %s", err) } - if err := c.FetchDebugZip(ctx, t); err != nil { - t.L().Printf("failed to collect zip: %s", err) + if err := c.FetchDebugZip(ctx, l); err != nil { + l.Printf("failed to collect zip: %s", err) } } diff --git a/pkg/cmd/roachtest/test_test.go b/pkg/cmd/roachtest/test_test.go index 6fbbe4da3f32..0ed208df8c98 100644 --- a/pkg/cmd/roachtest/test_test.go +++ b/pkg/cmd/roachtest/test_test.go @@ -27,6 +27,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/spec" "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/test" "github.com/cockroachdb/cockroach/pkg/roachprod/logger" + "github.com/cockroachdb/cockroach/pkg/roachprod/vm" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/util/quotapool" "github.com/cockroachdb/cockroach/pkg/util/stop" @@ -98,8 +99,8 @@ func alwaysFailingClusterAllocator( alloc *quotapool.IntAlloc, artifactsDir string, wStatus *workerStatus, -) (*clusterImpl, error) { - return nil, errors.New("cluster creation failed") +) (*clusterImpl, *vm.CreateOpts, error) { + return nil, nil, errors.New("cluster creation failed") } func TestRunnerRun(t *testing.T) { diff --git a/pkg/cmd/roachtest/tests/multitenant_fairness.go b/pkg/cmd/roachtest/tests/multitenant_fairness.go index a5fe0fca414b..f447b93399f8 100644 --- a/pkg/cmd/roachtest/tests/multitenant_fairness.go +++ b/pkg/cmd/roachtest/tests/multitenant_fairness.go @@ -310,7 +310,7 @@ func runMultiTenantFairness( c.Run(ctx, c.Node(1), cmd) // get cluster timeseries data into artifacts - err := c.FetchTimeseriesData(ctx, t) + err := c.FetchTimeseriesData(ctx, t.L()) require.NoError(t, err) } From c48197fa712e6011c452326957006c6030d27f99 Mon Sep 17 00:00:00 2001 From: Faizan Qazi Date: Tue, 13 Sep 2022 00:46:18 -0400 Subject: [PATCH 02/18] ci: fix validation for mixed version schema change corpus Informs: #86381 Previously, the command for validating the mixed version corpus was invalid. This patch fixes the validation command to address this, so the mixed version corpus is automatically uploaded. Release note: None --- .../cockroach/nightlies/sqllogic_corpus_nightly_impl.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/build/teamcity/cockroach/nightlies/sqllogic_corpus_nightly_impl.sh b/build/teamcity/cockroach/nightlies/sqllogic_corpus_nightly_impl.sh index fb8ae718d429..90c4412d8783 100755 --- a/build/teamcity/cockroach/nightlies/sqllogic_corpus_nightly_impl.sh +++ b/build/teamcity/cockroach/nightlies/sqllogic_corpus_nightly_impl.sh @@ -143,7 +143,7 @@ done # Any generated corpus should be validated on the current version first, which # indicates we can replay it on the same version. -$BAZEL_BIN/pkg/cmd/bazci/bazci_/bazci test --config=ci \ +$BAZEL_BIN/pkg/cmd/bazci/bazci_/bazci test -- --config=ci \ //pkg/sql/schemachanger/corpus:corpus_test \ --test_arg=--declarative-corpus=$ARTIFACTS_DIR/corpus-mixed \ --test_filter='^TestValidateCorpuses$' \ From da6c6539408b1703a4cd3b0bf0a93bb1ace491c4 Mon Sep 17 00:00:00 2001 From: Xin Hao Zhang Date: Mon, 19 Sep 2022 10:57:30 -0400 Subject: [PATCH 03/18] server: return all completed stmt diag requests Fixes #80104 Previously, we only return statement diagnostics requests that have not yet expired. Since we use the results of this request to populate completed statement diagnostics bundles in addition to outstanding requests, completed statement diag bundles would disappear from the UI after the request expired. This commit ensures that `StatementDiagnosticsRequests` returns all completed stmt diag requests so that we can display the complete history of completed bundles in the UI. Release note (bug fix): completed stmt diagnostics bundles now persist in the UI in stmt diag bundle pages --- pkg/server/statement_diagnostics_requests.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/pkg/server/statement_diagnostics_requests.go b/pkg/server/statement_diagnostics_requests.go index 101fec92a831..7424aeb1f817 100644 --- a/pkg/server/statement_diagnostics_requests.go +++ b/pkg/server/statement_diagnostics_requests.go @@ -124,7 +124,7 @@ func (s *statusServer) CancelStatementDiagnosticsReport( // StatementDiagnosticsRequests retrieves all statement diagnostics // requests in the `system.statement_diagnostics_requests` table that -// have not yet expired. +// have either completed or have not yet expired. func (s *statusServer) StatementDiagnosticsRequests( ctx context.Context, _ *serverpb.StatementDiagnosticsReportsRequest, ) (*serverpb.StatementDiagnosticsReportsResponse, error) { @@ -193,7 +193,7 @@ func (s *statusServer) StatementDiagnosticsRequests( if expiresAt, ok := row[6].(*tree.DTimestampTZ); ok { req.ExpiresAt = expiresAt.Time // Don't return already expired requests. - if req.ExpiresAt.Before(timeutil.Now()) { + if !completed && req.ExpiresAt.Before(timeutil.Now()) { continue } } From 7ffd2e3afb9af18dffcd878ae80edac6a221c743 Mon Sep 17 00:00:00 2001 From: Xin Hao Zhang Date: Mon, 19 Sep 2022 14:05:26 -0400 Subject: [PATCH 04/18] ui/cluster-ui: fix jobs page polling Fixes #68109 Previously, the jobs pages would not poll for new data until a re-render was triggered. This commit updates the jobs page polling to every 10s regardless of whether or not the rest of the page has changed on the overview page, and polls every 10s on the details page if the job is in a non-terminal state. Release note (bug fix): jobs pages refreshes data at an interval of 10s --- .../src/jobs/jobDetailsPage/jobDetails.tsx | 16 ++++++++-- .../cluster-ui/src/jobs/jobsPage/jobsPage.tsx | 29 +++++++++---------- .../cluster-ui/src/jobs/util/jobOptions.tsx | 3 ++ .../src/store/jobs/jobs.sagas.spec.ts | 28 +----------------- .../cluster-ui/src/store/jobs/jobs.sagas.ts | 21 ++------------ .../db-console/src/redux/apiReducers.ts | 4 +-- .../db-console/src/views/jobs/jobsPage.tsx | 4 ++- 7 files changed, 40 insertions(+), 65 deletions(-) diff --git a/pkg/ui/workspaces/cluster-ui/src/jobs/jobDetailsPage/jobDetails.tsx b/pkg/ui/workspaces/cluster-ui/src/jobs/jobDetailsPage/jobDetails.tsx index 63affd125be3..e74e295a3529 100644 --- a/pkg/ui/workspaces/cluster-ui/src/jobs/jobDetailsPage/jobDetails.tsx +++ b/pkg/ui/workspaces/cluster-ui/src/jobs/jobDetailsPage/jobDetails.tsx @@ -27,6 +27,7 @@ import { getMatchParamByName } from "src/util/query"; import { HighwaterTimestamp } from "src/jobs/util/highwaterTimestamp"; import { JobStatusCell } from "src/jobs/util/jobStatusCell"; +import { isTerminalState } from "../util/jobOptions"; import { commonStyles } from "src/common"; import summaryCardStyles from "src/summaryCard/summaryCard.module.scss"; @@ -52,11 +53,18 @@ export type JobDetailsProps = JobDetailsStateProps & RouteComponentProps; export class JobDetails extends React.Component { + refreshDataInterval: NodeJS.Timeout; + constructor(props: JobDetailsProps) { super(props); } private refresh(): void { + if (isTerminalState(this.props.job?.status)) { + clearInterval(this.refreshDataInterval); + return; + } + this.props.refreshJob( new cockroach.server.serverpb.JobRequest({ job_id: Long.fromString(getMatchParamByName(this.props.match, "id")), @@ -66,10 +74,14 @@ export class JobDetails extends React.Component { componentDidMount(): void { this.refresh(); + // Refresh every 10s. + this.refreshDataInterval = setInterval(() => this.refresh(), 10 * 1000); } - componentDidUpdate(): void { - this.refresh(); + componentWillUnmount(): void { + if (this.refreshDataInterval) { + clearInterval(this.refreshDataInterval); + } } prevPage = (): void => this.props.history.goBack(); diff --git a/pkg/ui/workspaces/cluster-ui/src/jobs/jobsPage/jobsPage.tsx b/pkg/ui/workspaces/cluster-ui/src/jobs/jobsPage/jobsPage.tsx index c373d4d1de0c..5c6a198ba2fb 100644 --- a/pkg/ui/workspaces/cluster-ui/src/jobs/jobsPage/jobsPage.tsx +++ b/pkg/ui/workspaces/cluster-ui/src/jobs/jobsPage/jobsPage.tsx @@ -56,6 +56,8 @@ export type JobsPageProps = JobsPageStateProps & RouteComponentProps; export class JobsPage extends React.Component { + refreshDataInterval: NodeJS.Timeout; + constructor(props: JobsPageProps) { super(props); @@ -94,29 +96,26 @@ export class JobsPage extends React.Component { } } - private refresh(props = this.props): void { + refresh(): void { const jobsRequest = new cockroach.server.serverpb.JobsRequest({ - status: props.status, - type: props.type, - limit: parseInt(props.show, 10), + status: this.props.status, + type: this.props.type, + limit: parseInt(this.props.show, 10), }); - props.onFilterChange - ? props.onFilterChange(jobsRequest) - : props.refreshJobs(jobsRequest); + this.props.onFilterChange + ? this.props.onFilterChange(jobsRequest) + : this.props.refreshJobs(jobsRequest); } componentDidMount(): void { + // Refresh every 10 seconds this.refresh(); + this.refreshDataInterval = setInterval(() => this.refresh(), 10 * 1000); } - componentDidUpdate(prevProps: JobsPageProps): void { - if ( - prevProps.status !== this.props.status || - prevProps.type !== this.props.type || - prevProps.show !== this.props.show - ) { - this.refresh(); - } + componentWillUnmount(): void { + if (!this.refreshDataInterval) return; + clearInterval(this.refreshDataInterval); } onStatusSelected = (item: string): void => { diff --git a/pkg/ui/workspaces/cluster-ui/src/jobs/util/jobOptions.tsx b/pkg/ui/workspaces/cluster-ui/src/jobs/util/jobOptions.tsx index bf391f99cbaa..6c895f92c9e0 100644 --- a/pkg/ui/workspaces/cluster-ui/src/jobs/util/jobOptions.tsx +++ b/pkg/ui/workspaces/cluster-ui/src/jobs/util/jobOptions.tsx @@ -73,6 +73,9 @@ export function isRetrying(status: string): boolean { export function isRunning(status: string): boolean { return [JOB_STATUS_RUNNING, JOB_STATUS_RETRY_RUNNING].includes(status); } +export function isTerminalState(status: string): boolean { + return [JOB_STATUS_SUCCEEDED, JOB_STATUS_FAILED].includes(status); +} export const statusOptions = [ { value: "", name: "All" }, diff --git a/pkg/ui/workspaces/cluster-ui/src/store/jobs/jobs.sagas.spec.ts b/pkg/ui/workspaces/cluster-ui/src/store/jobs/jobs.sagas.spec.ts index 833e9c977792..24013d3868b0 100644 --- a/pkg/ui/workspaces/cluster-ui/src/store/jobs/jobs.sagas.spec.ts +++ b/pkg/ui/workspaces/cluster-ui/src/store/jobs/jobs.sagas.spec.ts @@ -18,11 +18,7 @@ import * as matchers from "redux-saga-test-plan/matchers"; import { cockroach } from "@cockroachlabs/crdb-protobuf-client"; import { getJobs } from "src/api/jobsApi"; -import { - refreshJobsSaga, - requestJobsSaga, - receivedJobsSaga, -} from "./jobs.sagas"; +import { refreshJobsSaga, requestJobsSaga } from "./jobs.sagas"; import { actions, reducer, JobsState } from "./jobs.reducer"; import { allJobsFixture, @@ -83,26 +79,4 @@ describe("jobs sagas", () => { .run(); }); }); - - describe("receivedJobsSaga", () => { - it("sets valid status to false after specified period of time", () => { - const timeout = 500; - return expectSaga(receivedJobsSaga, timeout) - .delay(timeout) - .put(actions.invalidated()) - .withReducer(reducer, { - data: jobsResponse, - lastError: null, - valid: true, - inFlight: false, - }) - .hasFinalState({ - data: jobsResponse, - lastError: null, - valid: false, - inFlight: false, - }) - .run(1000); - }); - }); }); diff --git a/pkg/ui/workspaces/cluster-ui/src/store/jobs/jobs.sagas.ts b/pkg/ui/workspaces/cluster-ui/src/store/jobs/jobs.sagas.ts index e54d6997805a..7b661639caf3 100644 --- a/pkg/ui/workspaces/cluster-ui/src/store/jobs/jobs.sagas.ts +++ b/pkg/ui/workspaces/cluster-ui/src/store/jobs/jobs.sagas.ts @@ -8,14 +8,12 @@ // by the Apache License, Version 2.0, included in the file // licenses/APL.txt. -import { all, call, delay, put, takeLatest } from "redux-saga/effects"; +import { all, call, put, takeLatest } from "redux-saga/effects"; import { cockroach } from "@cockroachlabs/crdb-protobuf-client"; import { actions } from "./jobs.reducer"; import { getJobs, JobsRequest } from "src/api/jobsApi"; -import { CACHE_INVALIDATION_PERIOD, throttleWithReset } from "../utils"; import { PayloadAction } from "@reduxjs/toolkit"; -import { rootActions } from "../reducers"; export function* refreshJobsSaga(action: PayloadAction) { yield put(actions.request(action.payload)); @@ -30,29 +28,16 @@ export function* requestJobsSaga(action: PayloadAction): any { } } -export function* receivedJobsSaga(delayMs: number) { - yield delay(delayMs); - yield put(actions.invalidated()); -} - export function* updateFilteredJobsSaga(action: PayloadAction) { yield put(actions.invalidated()); const req = new cockroach.server.serverpb.JobsRequest(action.payload); yield put(actions.refresh(req)); } -export function* jobsSaga( - cacheInvalidationPeriod: number = CACHE_INVALIDATION_PERIOD, -) { +export function* jobsSaga() { yield all([ - throttleWithReset( - cacheInvalidationPeriod, - actions.refresh, - [actions.invalidated, rootActions.resetState], - refreshJobsSaga, - ), + takeLatest(actions.refresh, refreshJobsSaga), takeLatest(actions.request, requestJobsSaga), - takeLatest(actions.received, receivedJobsSaga, cacheInvalidationPeriod), takeLatest(actions.updateFilteredJobs, updateFilteredJobsSaga), ]); } diff --git a/pkg/ui/workspaces/db-console/src/redux/apiReducers.ts b/pkg/ui/workspaces/db-console/src/redux/apiReducers.ts index 0d07274a7011..67ac931bfa60 100644 --- a/pkg/ui/workspaces/db-console/src/redux/apiReducers.ts +++ b/pkg/ui/workspaces/db-console/src/redux/apiReducers.ts @@ -194,7 +194,7 @@ const jobsReducerObj = new KeyedCachedDataReducer( api.getJobs, "jobs", jobsRequestKey, - moment.duration(10, "s"), + null, moment.duration(1, "minute"), ); export const refreshJobs = jobsReducerObj.refresh; @@ -206,7 +206,7 @@ const jobReducerObj = new KeyedCachedDataReducer( api.getJob, "job", jobRequestKey, - moment.duration(10, "s"), + null, ); export const refreshJob = jobReducerObj.refresh; diff --git a/pkg/ui/workspaces/db-console/src/views/jobs/jobsPage.tsx b/pkg/ui/workspaces/db-console/src/views/jobs/jobsPage.tsx index 9d10ffcd54ad..b2c76a0a2573 100644 --- a/pkg/ui/workspaces/db-console/src/views/jobs/jobsPage.tsx +++ b/pkg/ui/workspaces/db-console/src/views/jobs/jobsPage.tsx @@ -77,7 +77,9 @@ const mapStateToProps = ( const key = jobsKey(status, type, parseInt(show, 10)); const jobsState = selectJobsState(state, key); const jobs = jobsState ? jobsState.data : null; - const jobsLoading = jobsState ? jobsState.inFlight : false; + const jobsLoading = jobsState + ? jobsState.inFlight && !jobsState.valid + : false; const jobsError = jobsState ? jobsState.lastError : null; return { sort, From bcba12c9feb75d1b59c0b02e0425738dfeab5460 Mon Sep 17 00:00:00 2001 From: Raphael 'kena' Poss Date: Mon, 19 Sep 2022 22:52:23 +0200 Subject: [PATCH 05/18] cli/sql: put a limit on history size Previously, there was no limit. Some users managed to make their history run into megabyte-size, despite de-duplication, which was causing slowness. This patch fixes it by adding a limit of 1000 entries. Sufficiently large to not be inconvenient, but sufficiently small that it prevents the history file from growing abnormally large. Release note (cli change): The interactive SQL shell now retains a maximum of 1000 entries. There was no limit previously. --- pkg/cli/clisqlshell/sql.go | 12 +++++++++++- 1 file changed, 11 insertions(+), 1 deletion(-) diff --git a/pkg/cli/clisqlshell/sql.go b/pkg/cli/clisqlshell/sql.go index 0b1ef0d1511e..bdabe478d687 100644 --- a/pkg/cli/clisqlshell/sql.go +++ b/pkg/cli/clisqlshell/sql.go @@ -2158,8 +2158,18 @@ func (c *cliState) configurePreShellDefaults( // memory when e.g. piping a large SQL script through the // command-line client. + // maxHistEntries is the maximum number of entries to + // preserve. Note that libedit de-duplicates entries under the + // hood. We expect that folk entering SQL in a shell will often + // reuse the same queries over time, so we don't expect this limit + // to ever be reached in practice, or to be an annoyance to + // anyone. We do prefer a limit however (as opposed to no limit at + // all), to prevent abnormal situation where a history runs into + // megabytes and starts slowing down the shell. + const maxHistEntries = 1000 + c.ins.SetCompleter(c) - if err := c.ins.UseHistory(-1 /*maxEntries*/, true /*dedup*/); err != nil { + if err := c.ins.UseHistory(maxHistEntries, true /*dedup*/); err != nil { fmt.Fprintf(c.iCtx.stderr, "warning: cannot enable history: %v\n ", err) } else { homeDir, err := envutil.HomeDir() From 7457a705c3698a6edfc2ec974539ab589f16304f Mon Sep 17 00:00:00 2001 From: Eric Harmeling Date: Mon, 19 Sep 2022 15:38:37 -0400 Subject: [PATCH 06/18] sql: version gate idx recommendations in insert-stmt-stats This commit version gates index recommendation insert in insert-stmt-stats. Fixes #88140. Release note: None --- .../sqlstats/persistedsqlstats/BUILD.bazel | 1 + pkg/sql/sqlstats/persistedsqlstats/flush.go | 54 +++++++++++-------- 2 files changed, 33 insertions(+), 22 deletions(-) diff --git a/pkg/sql/sqlstats/persistedsqlstats/BUILD.bazel b/pkg/sql/sqlstats/persistedsqlstats/BUILD.bazel index bd86dc8808ea..0af9efcc0afe 100644 --- a/pkg/sql/sqlstats/persistedsqlstats/BUILD.bazel +++ b/pkg/sql/sqlstats/persistedsqlstats/BUILD.bazel @@ -24,6 +24,7 @@ go_library( visibility = ["//visibility:public"], deps = [ "//pkg/base", + "//pkg/clusterversion", "//pkg/jobs", "//pkg/jobs/jobspb", "//pkg/kv", diff --git a/pkg/sql/sqlstats/persistedsqlstats/flush.go b/pkg/sql/sqlstats/persistedsqlstats/flush.go index 64d0045c4f53..38eccaa15236 100644 --- a/pkg/sql/sqlstats/persistedsqlstats/flush.go +++ b/pkg/sql/sqlstats/persistedsqlstats/flush.go @@ -12,8 +12,10 @@ package persistedsqlstats import ( "context" + "fmt" "time" + "github.com/cockroachdb/cockroach/pkg/clusterversion" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/roachpb" "github.com/cockroachdb/cockroach/pkg/security/username" @@ -472,13 +474,7 @@ func (s *PersistedSQLStats) insertStatementStats( serializedPlanHash []byte, stats *roachpb.CollectedStatementStatistics, ) (rowsAffected int, err error) { - insertStmt := ` -INSERT INTO system.statement_statistics -VALUES ($1, $2, $3, $4, $5, $6, $7, $8, $9, $10, $11) -ON CONFLICT (crdb_internal_aggregated_ts_app_name_fingerprint_id_node_id_plan_hash_transaction_fingerprint_id_shard_8, - aggregated_ts, fingerprint_id, transaction_fingerprint_id, app_name, plan_hash, node_id) -DO NOTHING -` + aggInterval := s.GetAggregationInterval() // Prepare data for insertion. @@ -495,13 +491,37 @@ DO NOTHING statistics := tree.NewDJSON(statisticsJSON) plan := tree.NewDJSON(sqlstatsutil.ExplainTreePlanNodeToJSON(&stats.Stats.SensitiveInfo.MostRecentPlanDescription)) - indexRecommendations := tree.NewDArray(types.String) - for _, recommendation := range stats.Stats.IndexRecommendations { - if err := indexRecommendations.Append(tree.NewDString(recommendation)); err != nil { - return 0, err + values := "$1 ,$2, $3, $4, $5, $6, $7, $8, $9, $10" + args := append(make([]interface{}, 0, 11), + aggregatedTs, // aggregated_ts + serializedFingerprintID, // fingerprint_id + serializedTransactionFingerprintID, // transaction_fingerprint_id + serializedPlanHash, // plan_hash + stats.Key.App, // app_name + s.cfg.SQLIDContainer.SQLInstanceID(), // node_id + aggInterval, // agg_interval + metadata, // metadata + statistics, // statistics + plan, // plan + ) + if s.cfg.Settings.Version.IsActive(ctx, clusterversion.AlterSystemStatementStatisticsAddIndexRecommendations) { + values = values + ", $11" + indexRecommendations := tree.NewDArray(types.String) + for _, recommendation := range stats.Stats.IndexRecommendations { + if err := indexRecommendations.Append(tree.NewDString(recommendation)); err != nil { + return 0, err + } } + args = append(args, indexRecommendations) } + insertStmt := fmt.Sprintf(` +INSERT INTO system.statement_statistics +VALUES (%s) +ON CONFLICT (crdb_internal_aggregated_ts_app_name_fingerprint_id_node_id_plan_hash_transaction_fingerprint_id_shard_8, + aggregated_ts, fingerprint_id, transaction_fingerprint_id, app_name, plan_hash, node_id) +DO NOTHING +`, values) rowsAffected, err = s.cfg.InternalExecutor.ExecEx( ctx, "insert-stmt-stats", @@ -510,17 +530,7 @@ DO NOTHING User: username.NodeUserName(), }, insertStmt, - aggregatedTs, // aggregated_ts - serializedFingerprintID, // fingerprint_id - serializedTransactionFingerprintID, // transaction_fingerprint_id - serializedPlanHash, // plan_hash - stats.Key.App, // app_name - s.cfg.SQLIDContainer.SQLInstanceID(), // node_id - aggInterval, // agg_interval - metadata, // metadata - statistics, // statistics - plan, // plan - indexRecommendations, // index_recommendations + args..., ) return rowsAffected, err From 9d2a3897b27e1f6c24aa6b01c8559b72d3ba7b55 Mon Sep 17 00:00:00 2001 From: Miral Gadani Date: Tue, 6 Sep 2022 17:49:31 -0400 Subject: [PATCH 07/18] roachtest: Refactor github posting to separate source file, add unit test. No previous coverage exists for posting issues to github. This change moves the relevant code into a separate source file with associated test. External functions dealing with actual issue posting to github, and loading the TEAMS.yaml are now injected to facilitate easier testing. Release justification: test-only change Release note: none --- pkg/cmd/roachtest/BUILD.bazel | 3 + pkg/cmd/roachtest/github.go | 143 ++++++++++++++++++++++ pkg/cmd/roachtest/github_test.go | 201 +++++++++++++++++++++++++++++++ pkg/cmd/roachtest/test_runner.go | 124 +++---------------- 4 files changed, 363 insertions(+), 108 deletions(-) create mode 100644 pkg/cmd/roachtest/github.go create mode 100644 pkg/cmd/roachtest/github_test.go diff --git a/pkg/cmd/roachtest/BUILD.bazel b/pkg/cmd/roachtest/BUILD.bazel index e4ea11c28f6a..490120f4a3cf 100644 --- a/pkg/cmd/roachtest/BUILD.bazel +++ b/pkg/cmd/roachtest/BUILD.bazel @@ -5,6 +5,7 @@ go_library( name = "roachtest_lib", srcs = [ "cluster.go", + "github.go", "main.go", "monitor.go", "slack.go", @@ -65,6 +66,7 @@ go_test( size = "small", srcs = [ "cluster_test.go", + "github_test.go", "main_test.go", "test_registry_test.go", "test_test.go", @@ -79,6 +81,7 @@ go_test( "//pkg/cmd/roachtest/test", "//pkg/internal/team", "//pkg/roachprod/logger", + "//pkg/roachprod/vm", "//pkg/testutils", "//pkg/util/quotapool", "//pkg/util/stop", diff --git a/pkg/cmd/roachtest/github.go b/pkg/cmd/roachtest/github.go new file mode 100644 index 000000000000..d412f79493c5 --- /dev/null +++ b/pkg/cmd/roachtest/github.go @@ -0,0 +1,143 @@ +// 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 main + +import ( + "context" + "fmt" + "os" + + "github.com/cockroachdb/cockroach/pkg/cmd/internal/issues" + "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/registry" + "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/test" + "github.com/cockroachdb/cockroach/pkg/internal/team" + "github.com/cockroachdb/cockroach/pkg/roachprod/logger" + "github.com/cockroachdb/cockroach/pkg/roachprod/vm" +) + +type githubIssues struct { + disable bool + l *logger.Logger + cluster *clusterImpl + vmCreateOpts *vm.CreateOpts + issuePoster func(ctx context.Context, formatter issues.IssueFormatter, req issues.PostRequest) error + teamLoader func() (team.Map, error) +} + +func newGithubIssues( + disable bool, c *clusterImpl, vmCreateOpts *vm.CreateOpts, l *logger.Logger, +) *githubIssues { + + return &githubIssues{ + disable: disable, + vmCreateOpts: vmCreateOpts, + cluster: c, + l: l, + issuePoster: issues.Post, + teamLoader: team.DefaultLoadTeams, + } +} + +func roachtestPrefix(p string) string { + return "ROACHTEST_" + p +} + +func (g *githubIssues) shouldPost(t test.Test) bool { + opts := issues.DefaultOptionsFromEnv() + return !g.disable && opts.CanPost() && + opts.IsReleaseBranch() && + t.Spec().(*registry.TestSpec).Run != nil && + // NB: check NodeCount > 0 to avoid posting issues from this pkg's unit tests. + t.Spec().(*registry.TestSpec).Cluster.NodeCount > 0 +} + +func (g *githubIssues) createPostRequest(t test.Test, message string) issues.PostRequest { + var mention []string + var projColID int + + teams, err := g.teamLoader() + if err != nil { + t.Fatalf("could not load teams: %v", err) + } + + if sl, ok := teams.GetAliasesForPurpose(ownerToAlias(t.Spec().(*registry.TestSpec).Owner), team.PurposeRoachtest); ok { + for _, alias := range sl { + mention = append(mention, "@"+string(alias)) + } + projColID = teams[sl[0]].TriageColumnID + } + + branch := os.Getenv("TC_BUILD_BRANCH") + if branch == "" { + branch = "" + } + + artifacts := fmt.Sprintf("/%s", t.Name()) + + // Issues posted from roachtest are identifiable as such and + // they are also release blockers (this label may be removed + // by a human upon closer investigation). + spec := t.Spec().(*registry.TestSpec) + labels := []string{"O-roachtest"} + if !spec.NonReleaseBlocker { + labels = append(labels, "release-blocker") + } + + clusterParams := map[string]string{ + roachtestPrefix("cloud"): spec.Cluster.Cloud, + roachtestPrefix("cpu"): fmt.Sprintf("%d", spec.Cluster.CPUs), + roachtestPrefix("ssd"): fmt.Sprintf("%d", spec.Cluster.SSDs), + } + + // These params can be probabilistically set, so we pass them here to + // show what their actual values are in the posted issue. + if g.vmCreateOpts != nil { + clusterParams[roachtestPrefix("fs")] = g.vmCreateOpts.SSDOpts.FileSystem + clusterParams[roachtestPrefix("localSSD")] = fmt.Sprintf("%v", g.vmCreateOpts.SSDOpts.UseLocalSSD) + } + + if g.cluster != nil { + clusterParams[roachtestPrefix("encrypted")] = fmt.Sprintf("%v", g.cluster.encAtRest) + } + + return issues.PostRequest{ + MentionOnCreate: mention, + ProjectColumnID: projColID, + PackageName: "roachtest", + TestName: t.Name(), + Message: message, + Artifacts: artifacts, + ExtraLabels: labels, + ExtraParams: clusterParams, + HelpCommand: func(renderer *issues.Renderer) { + issues.HelpCommandAsLink( + "roachtest README", + "https://github.com/cockroachdb/cockroach/blob/master/pkg/cmd/roachtest/README.md", + )(renderer) + issues.HelpCommandAsLink( + "How To Investigate (internal)", + "https://cockroachlabs.atlassian.net/l/c/SSSBr8c7", + )(renderer) + }, + } +} + +func (g *githubIssues) MaybePost(t test.Test, message string) error { + if !g.shouldPost(t) { + return nil + } + + return g.issuePoster( + context.Background(), + issues.UnitTestFormatter, + g.createPostRequest(t, message), + ) +} diff --git a/pkg/cmd/roachtest/github_test.go b/pkg/cmd/roachtest/github_test.go new file mode 100644 index 000000000000..d0188fc87ff8 --- /dev/null +++ b/pkg/cmd/roachtest/github_test.go @@ -0,0 +1,201 @@ +// 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 main + +import ( + "context" + "strings" + "testing" + + "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/cluster" + "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/registry" + "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/spec" + "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/test" + "github.com/cockroachdb/cockroach/pkg/internal/team" + "github.com/cockroachdb/cockroach/pkg/roachprod/vm" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" +) + +var ( + teamsYaml = `cockroachdb/unowned: + aliases: + cockroachdb/rfc-prs: other + triage_column_id: 0` + + validTeamsFn = func() (team.Map, error) { return loadYamlTeams(teamsYaml) } + invalidTeamsFn = func() (team.Map, error) { return loadYamlTeams("invalid yaml") } +) + +func loadYamlTeams(yaml string) (team.Map, error) { + return team.LoadTeams(strings.NewReader(yaml)) +} + +func prefixAll(params map[string]string) map[string]string { + updated := make(map[string]string) + + for k, v := range params { + updated[roachtestPrefix(k)] = v + } + + return updated +} + +func TestShouldPost(t *testing.T) { + testCases := []struct { + disableIssues bool + nodeCount int + envGithubAPIToken string + envTcBuildBranch string + expected bool + }{ + /* Cases 1 - 4 verify that issues are not posted if any of on the relevant criteria checks fail */ + // disable + {true, 1, "token", "master", false}, + // nodeCount + {false, 0, "token", "master", false}, + // apiToken + {false, 1, "", "master", false}, + // branch + {false, 1, "token", "", false}, + {false, 1, "token", "master", true}, + } + + reg, _ := makeTestRegistry(spec.GCE, "", "", false) + for _, c := range testCases { + t.Setenv("GITHUB_API_TOKEN", c.envGithubAPIToken) + t.Setenv("TC_BUILD_BRANCH", c.envTcBuildBranch) + + clusterSpec := reg.MakeClusterSpec(c.nodeCount) + testSpec := ®istry.TestSpec{ + Name: "githubPost", + Owner: OwnerUnitTest, + Cluster: clusterSpec, + // `shouldPost` explicitly checks to ensure that the run function is defined + Run: func(ctx context.Context, t test.Test, c cluster.Cluster) {}, + } + + ti := &testImpl{ + spec: testSpec, + l: nilLogger(), + } + + github := &githubIssues{ + disable: c.disableIssues, + } + + require.Equal(t, c.expected, github.shouldPost(ti)) + } +} + +func TestCreatePostRequest(t *testing.T) { + testCases := []struct { + nonReleaseBlocker bool + clusterCreationFailed bool + loadTeamsFailed bool + localSSD bool + expectedPost bool + expectedParams map[string]string + }{ + {true, false, false, false, true, + prefixAll(map[string]string{ + "cloud": "gce", + "encrypted": "false", + "fs": "ext4", + "ssd": "0", + "cpu": "4", + "localSSD": "false", + }), + }, + {true, false, false, true, true, + prefixAll(map[string]string{ + "cloud": "gce", + "encrypted": "false", + "fs": "ext4", + "ssd": "0", + "cpu": "4", + "localSSD": "true", + }), + }, + // Assert that release-blocker label exists when !nonReleaseBlocker + // Also ensure that in the event of a failed cluster creation, + // nil `vmOptions` and `clusterImpl` are not dereferenced + {false, true, false, false, true, + prefixAll(map[string]string{ + "cloud": "gce", + "ssd": "0", + "cpu": "4", + }), + }, + //Simulate failure loading TEAMS.yaml + {true, false, true, false, false, nil}, + } + + reg, _ := makeTestRegistry(spec.GCE, "", "", false) + + for _, c := range testCases { + clusterSpec := reg.MakeClusterSpec(1) + + testSpec := ®istry.TestSpec{ + Name: "githubPost", + Owner: OwnerUnitTest, + Cluster: clusterSpec, + NonReleaseBlocker: c.nonReleaseBlocker, + } + + ti := &testImpl{ + spec: testSpec, + l: nilLogger(), + } + + testClusterImpl := &clusterImpl{spec: clusterSpec} + vo := vm.DefaultCreateOpts() + vmOpts := &vo + + if c.clusterCreationFailed { + testClusterImpl = nil + vmOpts = nil + } else if !c.localSSD { + // The default is true set in `vm.DefaultCreateOpts` + vmOpts.SSDOpts.UseLocalSSD = false + } + + teamLoadFn := validTeamsFn + + if c.loadTeamsFailed { + teamLoadFn = invalidTeamsFn + } + + github := &githubIssues{ + vmCreateOpts: vmOpts, + cluster: testClusterImpl, + l: nilLogger(), + teamLoader: teamLoadFn, + } + + if c.loadTeamsFailed { + // Assert that if TEAMS.yaml cannot be loaded then function panics. + assert.Panics(t, func() { github.createPostRequest(ti, "message") }) + } else { + req := github.createPostRequest(ti, "message") + + if c.expectedParams != nil { + require.Equal(t, c.expectedParams, req.ExtraParams) + } + + require.True(t, contains(req.ExtraLabels, nil, "O-roachtest")) + + if !c.nonReleaseBlocker { + require.True(t, contains(req.ExtraLabels, nil, "release-blocker")) + } + } + } +} diff --git a/pkg/cmd/roachtest/test_runner.go b/pkg/cmd/roachtest/test_runner.go index fb7cd303c241..6bb2b33f280c 100644 --- a/pkg/cmd/roachtest/test_runner.go +++ b/pkg/cmd/roachtest/test_runner.go @@ -29,12 +29,10 @@ import ( "sync/atomic" "time" - "github.com/cockroachdb/cockroach/pkg/cmd/internal/issues" "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/option" "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/registry" "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/spec" "github.com/cockroachdb/cockroach/pkg/cmd/roachtest/test" - "github.com/cockroachdb/cockroach/pkg/internal/team" "github.com/cockroachdb/cockroach/pkg/roachprod/config" "github.com/cockroachdb/cockroach/pkg/roachprod/logger" "github.com/cockroachdb/cockroach/pkg/roachprod/vm" @@ -51,8 +49,10 @@ import ( "github.com/petermattis/goid" ) -var errTestsFailed = fmt.Errorf("some tests failed") -var errClusterProvisioningFailed = fmt.Errorf("some clusters could not be created") +var ( + errTestsFailed = fmt.Errorf("some tests failed") + errClusterProvisioningFailed = fmt.Errorf("some clusters could not be created") +) // testRunner runs tests. type testRunner struct { @@ -611,6 +611,8 @@ func (r *testRunner) runWorker( // Now run the test. l.PrintfCtx(ctx, "starting test: %s:%d", testToRun.spec.Name, testToRun.runNum) + github := newGithubIssues(r.config.disableIssue, c, vmCreateOpts, l) + if clusterCreateErr != nil { // N.B. cluster creation must have failed... // We don't want to prematurely abort the test suite since it's likely a transient issue. @@ -625,7 +627,11 @@ func (r *testRunner) runWorker( // N.B. issue title is of the form "roachtest: ${t.spec.Name} failed" (see UnitTestFormatter). t.spec.Name = "cluster_creation" t.spec.Owner = registry.OwnerDevInf - r.maybePostGithubIssue(ctx, l, t, c, vmCreateOpts, stdout, issueOutput) + + if err := github.MaybePost(t, issueOutput); err != nil { + shout(ctx, l, stdout, "failed to post issue: %s", err) + } + // Restore test name and owner. t.spec.Name = oldName t.spec.Owner = oldOwner @@ -654,7 +660,7 @@ func (r *testRunner) runWorker( wStatus.SetTest(t, testToRun) wStatus.SetStatus("running test") - err = r.runTest(ctx, t, testToRun.runNum, testToRun.runCount, c, vmCreateOpts, stdout, testL) + err = r.runTest(ctx, t, testToRun.runNum, testToRun.runCount, c, stdout, testL, github) } } @@ -766,9 +772,9 @@ func (r *testRunner) runTest( runNum int, runCount int, c *clusterImpl, - vmCreateOpts *vm.CreateOpts, stdout io.Writer, l *logger.Logger, + github *githubIssues, ) error { if t.Spec().(*registry.TestSpec).Skip != "" { return fmt.Errorf("can't run skipped test: %s: %s", t.Name(), t.Spec().(*registry.TestSpec).Skip) @@ -822,7 +828,9 @@ func (r *testRunner) runTest( shout(ctx, l, stdout, "--- FAIL: %s (%s)\n%s", runID, durationStr, output) - r.maybePostGithubIssue(ctx, l, t, c, vmCreateOpts, stdout, output) + if err := github.MaybePost(t, output); err != nil { + shout(ctx, l, stdout, "failed to post issue: %s", err) + } } else { shout(ctx, l, stdout, "--- PASS: %s (%s)", runID, durationStr) // If `##teamcity[testFailed ...]` is not present before `##teamCity[testFinished ...]`, @@ -1053,106 +1061,6 @@ func (r *testRunner) teardownTest( return nil } -func (r *testRunner) shouldPostGithubIssue(t test.Test) bool { - opts := issues.DefaultOptionsFromEnv() - return !r.config.disableIssue && - opts.CanPost() && - opts.IsReleaseBranch() && - t.Spec().(*registry.TestSpec).Run != nil && - // NB: check NodeCount > 0 to avoid posting issues from this pkg's unit tests. - t.Spec().(*registry.TestSpec).Cluster.NodeCount > 0 -} - -func (r *testRunner) maybePostGithubIssue( - ctx context.Context, - l *logger.Logger, - t test.Test, - c *clusterImpl, - vmCreateOpts *vm.CreateOpts, - stdout io.Writer, - output string, -) { - if !r.shouldPostGithubIssue(t) { - return - } - - teams, err := team.DefaultLoadTeams() - if err != nil { - t.Fatalf("could not load teams: %v", err) - } - - var mention []string - var projColID int - if sl, ok := teams.GetAliasesForPurpose(ownerToAlias(t.Spec().(*registry.TestSpec).Owner), team.PurposeRoachtest); ok { - for _, alias := range sl { - mention = append(mention, "@"+string(alias)) - } - projColID = teams[sl[0]].TriageColumnID - } - - branch := os.Getenv("TC_BUILD_BRANCH") - if branch == "" { - branch = "" - } - - artifacts := fmt.Sprintf("/%s", t.Name()) - - // Issues posted from roachtest are identifiable as such and - // they are also release blockers (this label may be removed - // by a human upon closer investigation). - spec := t.Spec().(*registry.TestSpec) - labels := []string{"O-roachtest"} - if !spec.NonReleaseBlocker { - labels = append(labels, "release-blocker") - } - - roachtestParam := func(s string) string { return "ROACHTEST_" + s } - clusterParams := map[string]string{ - roachtestParam("cloud"): spec.Cluster.Cloud, - roachtestParam("cpu"): fmt.Sprintf("%d", spec.Cluster.CPUs), - roachtestParam("ssd"): fmt.Sprintf("%d", spec.Cluster.SSDs), - } - - // these params can be probabilistically set if requested - if vmCreateOpts != nil { - clusterParams[roachtestParam("fs")] = vmCreateOpts.SSDOpts.FileSystem - } - - if c != nil { - clusterParams[roachtestParam("encrypted")] = fmt.Sprintf("%v", c.encAtRest) - } - - req := issues.PostRequest{ - MentionOnCreate: mention, - ProjectColumnID: projColID, - PackageName: "roachtest", - TestName: t.Name(), - Message: output, - Artifacts: artifacts, - ExtraLabels: labels, - ExtraParams: clusterParams, - HelpCommand: func(renderer *issues.Renderer) { - issues.HelpCommandAsLink( - "roachtest README", - "https://github.com/cockroachdb/cockroach/blob/master/pkg/cmd/roachtest/README.md", - )(renderer) - issues.HelpCommandAsLink( - "How To Investigate (internal)", - "https://cockroachlabs.atlassian.net/l/c/SSSBr8c7", - )(renderer) - }, - } - if err := issues.Post( - context.Background(), - issues.UnitTestFormatter, - req, - ); err != nil { - shout(ctx, l, stdout, "failed to post issue: %s", err) - } -} - -// TODO(tbg): nothing in this method should have the `t`; they should have a `Logger` only. -// Maybe done func (r *testRunner) collectClusterArtifacts( ctx context.Context, c *clusterImpl, l *logger.Logger, ) { From 22e7027bc64258a0cf16156d337fed7e950b718c Mon Sep 17 00:00:00 2001 From: Jane Xing Date: Wed, 24 Aug 2022 10:13:38 -0500 Subject: [PATCH 08/18] *: infuse `CollectionFactory` for storage and backup This is part of the migration of existing DDL statement with internal executor to `descs.CollectionFactory.TxnWithExecutor()`. DDL statements should only be run with an internal executor that's created via this function. Release justification: Low risk, high benefit changes to existing functionality Release note: none --- pkg/ccl/backupccl/backup_metadata_test.go | 7 +- pkg/ccl/backupccl/backup_test.go | 13 +- .../backupccl/restore_data_processor_test.go | 7 +- .../changefeedccl/sink_cloudstorage_test.go | 8 +- pkg/ccl/cliccl/debug_backup.go | 15 +- pkg/ccl/storageccl/BUILD.bazel | 1 + .../storageccl/external_sst_reader_test.go | 7 +- pkg/ccl/workloadccl/storage.go | 15 +- pkg/cloud/BUILD.bazel | 1 + pkg/cloud/amazon/s3_storage_test.go | 102 +++++++-- pkg/cloud/azure/azure_storage_test.go | 14 +- pkg/cloud/cloudtestutils/BUILD.bazel | 1 + .../cloudtestutils/cloud_test_helpers.go | 47 ++-- pkg/cloud/external_storage.go | 2 + pkg/cloud/externalconn/connection_storage.go | 2 +- pkg/cloud/gcp/gcs_storage_test.go | 207 +++++++++++------- pkg/cloud/httpsink/http_storage_test.go | 51 +++-- pkg/cloud/impl_registry.go | 7 +- pkg/cloud/nodelocal/nodelocal_storage_test.go | 10 +- pkg/cloud/nullsink/nullsink_storage_test.go | 9 +- pkg/cloud/userfile/BUILD.bazel | 1 + pkg/cloud/userfile/file_table_storage.go | 2 +- pkg/cloud/userfile/file_table_storage_test.go | 21 +- pkg/cloud/userfile/filetable/BUILD.bazel | 1 + .../filetable/file_table_read_writer.go | 6 +- .../filetable/filetabletest/BUILD.bazel | 1 + .../file_table_read_writer_test.go | 13 +- pkg/server/external_storage_builder.go | 9 +- pkg/server/server.go | 4 + pkg/server/server_sql.go | 3 + pkg/server/tenant.go | 4 + pkg/sql/importer/import_processor_test.go | 7 +- pkg/sql/importer/import_stmt_test.go | 9 +- 33 files changed, 444 insertions(+), 163 deletions(-) diff --git a/pkg/ccl/backupccl/backup_metadata_test.go b/pkg/ccl/backupccl/backup_metadata_test.go index 76d541f067d3..47febbea773a 100644 --- a/pkg/ccl/backupccl/backup_metadata_test.go +++ b/pkg/ccl/backupccl/backup_metadata_test.go @@ -25,6 +25,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/security/username" "github.com/cockroachdb/cockroach/pkg/sql" "github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/descs" "github.com/cockroachdb/cockroach/pkg/sql/stats" "github.com/cockroachdb/cockroach/pkg/testutils/testcluster" "github.com/cockroachdb/cockroach/pkg/util/hlc" @@ -86,7 +87,11 @@ func checkMetadata( tc.Servers[0].ClusterSettings(), blobs.TestEmptyBlobClientFactory, username.RootUserName(), - tc.Servers[0].InternalExecutor().(*sql.InternalExecutor), tc.Servers[0].DB(), nil) + tc.Servers[0].InternalExecutor().(*sql.InternalExecutor), + tc.Servers[0].CollectionFactory().(*descs.CollectionFactory), + tc.Servers[0].DB(), + nil, /* limiters */ + ) if err != nil { t.Fatal(err) } diff --git a/pkg/ccl/backupccl/backup_test.go b/pkg/ccl/backupccl/backup_test.go index af2de50b8c60..e02733b04c35 100644 --- a/pkg/ccl/backupccl/backup_test.go +++ b/pkg/ccl/backupccl/backup_test.go @@ -572,7 +572,11 @@ func TestBackupRestoreAppend(t *testing.T) { tc.Servers[0].ClusterSettings(), blobs.TestEmptyBlobClientFactory, username.RootUserName(), - tc.Servers[0].InternalExecutor().(*sql.InternalExecutor), tc.Servers[0].DB(), nil) + tc.Servers[0].InternalExecutor().(*sql.InternalExecutor), + tc.Servers[0].CollectionFactory().(*descs.CollectionFactory), + tc.Servers[0].DB(), + nil, /* limiters */ + ) require.NoError(t, err) defer store.Close() var files []string @@ -8021,7 +8025,12 @@ func TestReadBackupManifestMemoryMonitoring(t *testing.T) { base.ExternalIODirConfig{}, st, blobs.TestBlobServiceClient(dir), - username.RootUserName(), nil, nil, nil) + username.RootUserName(), + nil, /* ie */ + nil, /* cf */ + nil, /* kvDB */ + nil, /* limiters */ + ) require.NoError(t, err) m := mon.NewMonitor("test-monitor", mon.MemoryResource, nil, nil, 0, 0, st) diff --git a/pkg/ccl/backupccl/restore_data_processor_test.go b/pkg/ccl/backupccl/restore_data_processor_test.go index 6f61e5ced6c3..238339db3ec1 100644 --- a/pkg/ccl/backupccl/restore_data_processor_test.go +++ b/pkg/ccl/backupccl/restore_data_processor_test.go @@ -253,7 +253,12 @@ func runTestIngest(t *testing.T, init func(*cluster.Settings)) { DB: kvDB, ExternalStorage: func(ctx context.Context, dest cloudpb.ExternalStorage, opts ...cloud.ExternalStorageOption) (cloud.ExternalStorage, error) { return cloud.MakeExternalStorage(ctx, dest, base.ExternalIODirConfig{}, - s.ClusterSettings(), blobs.TestBlobServiceClient(s.ClusterSettings().ExternalIODir), nil, nil, nil, opts...) + s.ClusterSettings(), blobs.TestBlobServiceClient(s.ClusterSettings().ExternalIODir), + nil, /* ie */ + nil, /* cf */ + nil, /* kvDB */ + nil, /* limiters */ + opts...) }, Settings: s.ClusterSettings(), Codec: keys.SystemSQLCodec, diff --git a/pkg/ccl/changefeedccl/sink_cloudstorage_test.go b/pkg/ccl/changefeedccl/sink_cloudstorage_test.go index 4545d73c3730..a0e6e305f0af 100644 --- a/pkg/ccl/changefeedccl/sink_cloudstorage_test.go +++ b/pkg/ccl/changefeedccl/sink_cloudstorage_test.go @@ -169,7 +169,13 @@ func TestCloudStorageSink(t *testing.T) { externalStorageFromURI := func(ctx context.Context, uri string, user username.SQLUsername, opts ...cloud.ExternalStorageOption) (cloud.ExternalStorage, error) { return cloud.ExternalStorageFromURI(ctx, uri, base.ExternalIODirConfig{}, settings, - clientFactory, user, nil, nil, nil, opts...) + clientFactory, + user, + nil, /* ie */ + nil, /* cf */ + nil, /* kvDB */ + nil, /* limiters */ + opts...) } user := username.RootUserName() diff --git a/pkg/ccl/cliccl/debug_backup.go b/pkg/ccl/cliccl/debug_backup.go index 3057ec621010..ada7b714b5cc 100644 --- a/pkg/ccl/cliccl/debug_backup.go +++ b/pkg/ccl/cliccl/debug_backup.go @@ -291,7 +291,12 @@ func externalStorageFromURIFactory( defaultSettings := &cluster.Settings{} defaultSettings.SV.Init(ctx, nil /* opaque */) return cloud.ExternalStorageFromURI(ctx, uri, base.ExternalIODirConfig{}, - defaultSettings, newBlobFactory, user, nil /*Internal Executor*/, nil /*kvDB*/, nil, opts...) + defaultSettings, newBlobFactory, user, + nil, /* ie */ + nil, /* cf */ + nil, /* kvDB */ + nil, /* limiters */ + opts...) } func getManifestFromURI(ctx context.Context, path string) (backuppb.BackupManifest, error) { @@ -588,7 +593,13 @@ func makeIters( var err error clusterSettings := cluster.MakeClusterSettings() dirStorage[i], err = cloud.MakeExternalStorage(ctx, file.Dir, base.ExternalIODirConfig{}, - clusterSettings, newBlobFactory, nil /*internal executor*/, nil /*kvDB*/, nil) + clusterSettings, + newBlobFactory, + nil, /* ie */ + nil, /* cf */ + nil, /* kvDB */ + nil, /* limiters */ + ) if err != nil { return nil, nil, errors.Wrapf(err, "making external storage") } diff --git a/pkg/ccl/storageccl/BUILD.bazel b/pkg/ccl/storageccl/BUILD.bazel index d54954e9d01e..6ed03b91393f 100644 --- a/pkg/ccl/storageccl/BUILD.bazel +++ b/pkg/ccl/storageccl/BUILD.bazel @@ -46,6 +46,7 @@ go_test( "//pkg/security/username", "//pkg/server", "//pkg/sql", + "//pkg/sql/catalog/descs", "//pkg/storage", "//pkg/testutils", "//pkg/testutils/serverutils", diff --git a/pkg/ccl/storageccl/external_sst_reader_test.go b/pkg/ccl/storageccl/external_sst_reader_test.go index 23fd41c74246..9525ad84fd75 100644 --- a/pkg/ccl/storageccl/external_sst_reader_test.go +++ b/pkg/ccl/storageccl/external_sst_reader_test.go @@ -20,6 +20,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/keys" "github.com/cockroachdb/cockroach/pkg/security/username" "github.com/cockroachdb/cockroach/pkg/sql" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/descs" "github.com/cockroachdb/cockroach/pkg/storage" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/testutils/storageutils" @@ -122,7 +123,11 @@ func TestNewExternalSSTReader(t *testing.T) { clusterSettings, blobs.TestBlobServiceClient(tempDir), username.RootUserName(), - tc.Servers[0].InternalExecutor().(*sql.InternalExecutor), tc.Servers[0].DB(), nil) + tc.Servers[0].InternalExecutor().(*sql.InternalExecutor), + tc.Servers[0].CollectionFactory().(*descs.CollectionFactory), + tc.Servers[0].DB(), + nil, /* limiters */ + ) require.NoError(t, err) fileStores[i].Store = store diff --git a/pkg/ccl/workloadccl/storage.go b/pkg/ccl/workloadccl/storage.go index 4588cf64e479..6856a6e10f0d 100644 --- a/pkg/ccl/workloadccl/storage.go +++ b/pkg/ccl/workloadccl/storage.go @@ -35,9 +35,18 @@ func GetStorage(ctx context.Context, cfg FixtureConfig) (cloud.ExternalStorage, return nil, errors.AssertionFailedf("unsupported external storage provider; valid providers are gs, s3, and azure") } - s, err := cloud.ExternalStorageFromURI(ctx, cfg.ObjectPathToURI(), - base.ExternalIODirConfig{}, clustersettings.MakeClusterSettings(), - nil, username.SQLUsername{}, nil, nil, nil) + s, err := cloud.ExternalStorageFromURI( + ctx, + cfg.ObjectPathToURI(), + base.ExternalIODirConfig{}, + clustersettings.MakeClusterSettings(), + nil, /* blobClientFactory */ + username.SQLUsername{}, + nil, /* ie */ + nil, /* cf */ + nil, /* kvDB */ + nil, /* limiters */ + ) if err != nil { return nil, errors.Wrap(err, storageError) } diff --git a/pkg/cloud/BUILD.bazel b/pkg/cloud/BUILD.bazel index 130caa7f945e..d80a2d38922a 100644 --- a/pkg/cloud/BUILD.bazel +++ b/pkg/cloud/BUILD.bazel @@ -23,6 +23,7 @@ go_library( "//pkg/security/username", "//pkg/settings", "//pkg/settings/cluster", + "//pkg/sql/catalog/descs", "//pkg/sql/sqlutil", "//pkg/util/ctxgroup", "//pkg/util/ioctx", diff --git a/pkg/cloud/amazon/s3_storage_test.go b/pkg/cloud/amazon/s3_storage_test.go index 2569b338b3a9..ac983b9ef0ea 100644 --- a/pkg/cloud/amazon/s3_storage_test.go +++ b/pkg/cloud/amazon/s3_storage_test.go @@ -46,7 +46,12 @@ func makeS3Storage( // Setup a sink for the given args. clientFactory := blobs.TestBlobServiceClient(testSettings.ExternalIODir) s, err := cloud.MakeExternalStorage(ctx, conf, base.ExternalIODirConfig{}, testSettings, - clientFactory, nil, nil, nil) + clientFactory, + nil, /* ie */ + nil, /* cf */ + nil, /* kvDB */ + nil, /* limiters */ + ) if err != nil { return nil, err } @@ -76,7 +81,12 @@ func TestPutS3(t *testing.T) { t.Run("auth-empty-no-cred", func(t *testing.T) { _, err := cloud.ExternalStorageFromURI(ctx, fmt.Sprintf("s3://%s/%s", bucket, "backup-test-default"), base.ExternalIODirConfig{}, testSettings, - blobs.TestEmptyBlobClientFactory, user, nil, nil, nil) + blobs.TestEmptyBlobClientFactory, user, + nil, /* ie */ + nil, /* cf */ + nil, /* kvDB */ + nil, /* limiters */ + ) require.EqualError(t, err, fmt.Sprintf( `%s is set to '%s', but %s is not set`, cloud.AuthParam, @@ -100,14 +110,23 @@ func TestPutS3(t *testing.T) { "s3://%s/%s?%s=%s", bucket, "backup-test-default", cloud.AuthParam, cloud.AuthParamImplicit, - ), false, user, nil, nil, testSettings) + ), false, user, + nil, /* ie */ + nil, /* cf */ + nil, /* kvDB */ + testSettings) }) t.Run("auth-specified", func(t *testing.T) { uri := S3URI(bucket, "backup-test", &cloudpb.ExternalStorage_S3{AccessKey: creds.AccessKeyID, Secret: creds.SecretAccessKey, Region: "us-east-1"}, ) - cloudtestutils.CheckExportStore(t, uri, false, user, nil, nil, testSettings) - cloudtestutils.CheckListFiles(t, uri, user, nil, nil, testSettings) + cloudtestutils.CheckExportStore(t, uri, false, user, + nil, /* ie */ + nil, /* cf */ + nil, /* kvDB */ + testSettings, + ) + cloudtestutils.CheckListFiles(t, uri, user, nil, nil, nil, testSettings) }) // Tests that we can put an object with server side encryption specified. @@ -128,7 +147,14 @@ func TestPutS3(t *testing.T) { bucket, "backup-test-sse-256", cloud.AuthParam, cloud.AuthParamImplicit, AWSServerSideEncryptionMode, "AES256", - ), false, user, nil, nil, testSettings) + ), + false, + user, + nil, /* ie */ + nil, /* cf */ + nil, /* kvDB */ + testSettings, + ) v := os.Getenv("AWS_KMS_KEY_ARN") if v == "" { @@ -139,7 +165,13 @@ func TestPutS3(t *testing.T) { bucket, "backup-test-sse-kms", cloud.AuthParam, cloud.AuthParamImplicit, AWSServerSideEncryptionMode, "aws:kms", AWSServerSideEncryptionKMSID, v, - ), false, user, nil, nil, testSettings) + ), + false, + user, + nil, /* ie */ + nil, /* cf */ + nil, /* kvDB */ + testSettings) }) t.Run("server-side-encryption-invalid-params", func(t *testing.T) { @@ -208,16 +240,36 @@ func TestPutS3AssumeRole(t *testing.T) { uri := S3URI(bucket, "backup-test", &cloudpb.ExternalStorage_S3{Auth: cloud.AuthParamImplicit, RoleARN: roleArn, Region: "us-east-1"}, ) - cloudtestutils.CheckExportStore(t, uri, false, user, nil, nil, testSettings) - cloudtestutils.CheckListFiles(t, uri, user, nil, nil, testSettings) + cloudtestutils.CheckExportStore(t, uri, false, user, + nil, /* ie */ + nil, /* cf */ + nil, /* kvDB */ + testSettings, + ) + cloudtestutils.CheckListFiles(t, uri, user, + nil, /* ie */ + nil, /* cf */ + nil, /* kvDB */ + testSettings, + ) }) t.Run("auth-specified", func(t *testing.T) { uri := S3URI(bucket, "backup-test", &cloudpb.ExternalStorage_S3{Auth: cloud.AuthParamSpecified, RoleARN: roleArn, AccessKey: creds.AccessKeyID, Secret: creds.SecretAccessKey, Region: "us-east-1"}, ) - cloudtestutils.CheckExportStore(t, uri, false, user, nil, nil, testSettings) - cloudtestutils.CheckListFiles(t, uri, user, nil, nil, testSettings) + cloudtestutils.CheckExportStore(t, uri, false, user, + nil, /* ie */ + nil, /* cf */ + nil, /* kvDB */ + testSettings, + ) + cloudtestutils.CheckListFiles(t, uri, user, + nil, /* ie */ + nil, /* cf */ + nil, /* kvDB */ + testSettings, + ) }) t.Run("role-chaining", func(t *testing.T) { @@ -247,7 +299,12 @@ func TestPutS3AssumeRole(t *testing.T) { Region: "us-east-1", }, ) - cloudtestutils.CheckNoPermission(t, roleURI, user, nil, nil, testSettings) + cloudtestutils.CheckNoPermission(t, roleURI, user, + nil, /* ie */ + nil, /* cf */ + nil, /* kvDB */ + testSettings, + ) } // Finally, check that the chain of roles can be used to access the storage. @@ -262,7 +319,12 @@ func TestPutS3AssumeRole(t *testing.T) { }, ) - cloudtestutils.CheckExportStore(t, uri, false, user, nil, nil, testSettings) + cloudtestutils.CheckExportStore(t, uri, false, user, + nil, /* ie */ + nil, /* cf */ + nil, /* kvDB */ + testSettings, + ) }) } }) @@ -301,7 +363,12 @@ func TestPutS3Endpoint(t *testing.T) { testSettings := cluster.MakeTestingClusterSettings() - cloudtestutils.CheckExportStore(t, u.String(), false, user, nil, nil, testSettings) + cloudtestutils.CheckExportStore(t, u.String(), false, user, + nil, /* ie */ + nil, /* cf */ + nil, /* kvDB */ + testSettings, + ) } func TestS3DisallowCustomEndpoints(t *testing.T) { @@ -372,7 +439,12 @@ func TestS3BucketDoesNotExist(t *testing.T) { // Setup a sink for the given args. clientFactory := blobs.TestBlobServiceClient(testSettings.ExternalIODir) s, err := cloud.MakeExternalStorage(ctx, conf, base.ExternalIODirConfig{}, testSettings, - clientFactory, nil, nil, nil) + clientFactory, + nil, /* ie */ + nil, /* cf */ + nil, /* kvDB */ + nil, /* limiters */ + ) if err != nil { t.Fatal(err) } diff --git a/pkg/cloud/azure/azure_storage_test.go b/pkg/cloud/azure/azure_storage_test.go index 11182c729a6d..9517afa25e16 100644 --- a/pkg/cloud/azure/azure_storage_test.go +++ b/pkg/cloud/azure/azure_storage_test.go @@ -67,9 +67,17 @@ func TestAzure(t *testing.T) { } testSettings := cluster.MakeTestingClusterSettings() cloudtestutils.CheckExportStore(t, cfg.filePath("backup-test"), - false, username.RootUserName(), nil, nil, testSettings) - cloudtestutils.CheckListFiles( - t, cfg.filePath("listing-test"), username.RootUserName(), nil, nil, testSettings, + false, username.RootUserName(), + nil, /* ie */ + nil, /* cf */ + nil, /* kvDB */ + testSettings, + ) + cloudtestutils.CheckListFiles(t, cfg.filePath("listing-test"), username.RootUserName(), + nil, /* ie */ + nil, /* cf */ + nil, /* kvDB */ + testSettings, ) } diff --git a/pkg/cloud/cloudtestutils/BUILD.bazel b/pkg/cloud/cloudtestutils/BUILD.bazel index db24a23ab80b..d99cbebc2797 100644 --- a/pkg/cloud/cloudtestutils/BUILD.bazel +++ b/pkg/cloud/cloudtestutils/BUILD.bazel @@ -14,6 +14,7 @@ go_library( "//pkg/kv", "//pkg/security/username", "//pkg/settings/cluster", + "//pkg/sql/catalog/descs", "//pkg/sql/sqlutil", "//pkg/util/ioctx", "//pkg/util/randutil", diff --git a/pkg/cloud/cloudtestutils/cloud_test_helpers.go b/pkg/cloud/cloudtestutils/cloud_test_helpers.go index 09d48c6eb405..6a7028418569 100644 --- a/pkg/cloud/cloudtestutils/cloud_test_helpers.go +++ b/pkg/cloud/cloudtestutils/cloud_test_helpers.go @@ -32,6 +32,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/security/username" "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/descs" "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" "github.com/cockroachdb/cockroach/pkg/util/ioctx" "github.com/cockroachdb/cockroach/pkg/util/randutil" @@ -108,6 +109,7 @@ func storeFromURI( clientFactory blobs.BlobClientFactory, user username.SQLUsername, ie sqlutil.InternalExecutor, + cf *descs.CollectionFactory, kvDB *kv.DB, testSettings *cluster.Settings, ) cloud.ExternalStorage { @@ -117,7 +119,7 @@ func storeFromURI( } // Setup a sink for the given args. s, err := cloud.MakeExternalStorage(ctx, conf, base.ExternalIODirConfig{}, testSettings, - clientFactory, ie, kvDB, nil) + clientFactory, ie, cf, kvDB, nil) if err != nil { t.Fatal(err) } @@ -131,6 +133,7 @@ func CheckExportStore( skipSingleFile bool, user username.SQLUsername, ie sqlutil.InternalExecutor, + cf *descs.CollectionFactory, kvDB *kv.DB, testSettings *cluster.Settings, ) { @@ -144,7 +147,8 @@ func CheckExportStore( // Setup a sink for the given args. clientFactory := blobs.TestBlobServiceClient(testSettings.ExternalIODir) - s, err := cloud.MakeExternalStorage(ctx, conf, ioConf, testSettings, clientFactory, ie, kvDB, nil) + s, err := cloud.MakeExternalStorage(ctx, conf, ioConf, testSettings, clientFactory, + ie, cf, kvDB, nil) if err != nil { t.Fatal(err) } @@ -252,7 +256,7 @@ func CheckExportStore( t.Fatal(err) } singleFile := storeFromURI(ctx, t, appendPath(t, storeURI, testingFilename), clientFactory, - user, ie, kvDB, testSettings) + user, ie, cf, kvDB, testSettings) defer singleFile.Close() res, err := singleFile.ReadFile(ctx, "") @@ -273,7 +277,7 @@ func CheckExportStore( t.Run("write-single-file-by-uri", func(t *testing.T) { const testingFilename = "B" singleFile := storeFromURI(ctx, t, appendPath(t, storeURI, testingFilename), clientFactory, - user, ie, kvDB, testSettings) + user, ie, cf, kvDB, testSettings) defer singleFile.Close() if err := cloud.WriteFile(ctx, singleFile, "", bytes.NewReader([]byte("bbb"))); err != nil { @@ -304,7 +308,7 @@ func CheckExportStore( if err := cloud.WriteFile(ctx, s, testingFilename, bytes.NewReader([]byte("aaa"))); err != nil { t.Fatal(err) } - singleFile := storeFromURI(ctx, t, storeURI, clientFactory, user, ie, kvDB, testSettings) + singleFile := storeFromURI(ctx, t, storeURI, clientFactory, user, ie, cf, kvDB, testSettings) defer singleFile.Close() // Read a valid file. @@ -346,10 +350,11 @@ func CheckListFiles( storeURI string, user username.SQLUsername, ie sqlutil.InternalExecutor, + cf *descs.CollectionFactory, kvDB *kv.DB, testSettings *cluster.Settings, ) { - CheckListFilesCanonical(t, storeURI, "", user, ie, kvDB, testSettings) + CheckListFilesCanonical(t, storeURI, "", user, ie, cf, kvDB, testSettings) } // CheckListFilesCanonical is like CheckListFiles but takes a canonical prefix @@ -361,6 +366,7 @@ func CheckListFilesCanonical( canonical string, user username.SQLUsername, ie sqlutil.InternalExecutor, + cf *descs.CollectionFactory, kvDB *kv.DB, testSettings *cluster.Settings, ) { @@ -374,7 +380,7 @@ func CheckListFilesCanonical( clientFactory := blobs.TestBlobServiceClient(testSettings.ExternalIODir) for _, fileName := range fileNames { - file := storeFromURI(ctx, t, storeURI, clientFactory, user, ie, kvDB, testSettings) + file := storeFromURI(ctx, t, storeURI, clientFactory, user, ie, cf, kvDB, testSettings) if err := cloud.WriteFile(ctx, file, fileName, bytes.NewReader([]byte("bbb"))); err != nil { t.Fatal(err) } @@ -462,7 +468,7 @@ func CheckListFilesCanonical( }, } { t.Run(tc.name, func(t *testing.T) { - s := storeFromURI(ctx, t, tc.uri, clientFactory, user, ie, kvDB, testSettings) + s := storeFromURI(ctx, t, tc.uri, clientFactory, user, ie, cf, kvDB, testSettings) var actual []string require.NoError(t, s.List(ctx, tc.prefix, tc.delimiter, func(f string) error { actual = append(actual, f) @@ -475,7 +481,7 @@ func CheckListFilesCanonical( }) for _, fileName := range fileNames { - file := storeFromURI(ctx, t, storeURI, clientFactory, user, ie, kvDB, testSettings) + file := storeFromURI(ctx, t, storeURI, clientFactory, user, ie, cf, kvDB, testSettings) if err := file.Delete(ctx, fileName); err != nil { t.Fatal(err) } @@ -493,9 +499,13 @@ func uploadData( data := randutil.RandBytes(rnd, 16<<20) ctx := context.Background() - s, err := cloud.MakeExternalStorage( - ctx, dest, base.ExternalIODirConfig{}, testSettings, - nil, nil, nil, nil) + s, err := cloud.MakeExternalStorage(ctx, dest, base.ExternalIODirConfig{}, testSettings, + nil, /* blobClientFactory */ + nil, /* ie */ + nil, /* cf */ + nil, /* kvDB */ + nil, /* limiters */ + ) require.NoError(t, err) require.NoError(t, cloud.WriteFile(ctx, s, basename, bytes.NewReader(data))) return data, func() { @@ -534,9 +544,13 @@ func CheckAntagonisticRead( }() ctx := context.Background() - s, err := cloud.MakeExternalStorage( - ctx, conf, base.ExternalIODirConfig{}, testSettings, - nil, nil, nil, nil) + s, err := cloud.MakeExternalStorage(ctx, conf, base.ExternalIODirConfig{}, testSettings, + nil, /* blobClientFactory */ + nil, /* ie */ + nil, /* cf */ + nil, /* kvDB */ + nil, /* limiters */ + ) require.NoError(t, err) defer s.Close() @@ -555,6 +569,7 @@ func CheckNoPermission( storeURI string, user username.SQLUsername, ie sqlutil.InternalExecutor, + cf *descs.CollectionFactory, kvDB *kv.DB, testSettings *cluster.Settings, ) { @@ -567,7 +582,7 @@ func CheckNoPermission( } clientFactory := blobs.TestBlobServiceClient(testSettings.ExternalIODir) - s, err := cloud.MakeExternalStorage(ctx, conf, ioConf, testSettings, clientFactory, ie, kvDB, nil) + s, err := cloud.MakeExternalStorage(ctx, conf, ioConf, testSettings, clientFactory, ie, cf, kvDB, nil) if err != nil { t.Fatal(err) } diff --git a/pkg/cloud/external_storage.go b/pkg/cloud/external_storage.go index 48482ff45fcf..901118ce0a03 100644 --- a/pkg/cloud/external_storage.go +++ b/pkg/cloud/external_storage.go @@ -22,6 +22,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/security/username" "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/descs" "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" "github.com/cockroachdb/cockroach/pkg/util/ioctx" "github.com/cockroachdb/errors" @@ -153,6 +154,7 @@ type ExternalStorageContext struct { Settings *cluster.Settings BlobClientFactory blobs.BlobClientFactory InternalExecutor sqlutil.InternalExecutor + CollectionFactory *descs.CollectionFactory DB *kv.DB Options []ExternalStorageOption Limiters Limiters diff --git a/pkg/cloud/externalconn/connection_storage.go b/pkg/cloud/externalconn/connection_storage.go index 902563ed9ec6..e98078e6e2eb 100644 --- a/pkg/cloud/externalconn/connection_storage.go +++ b/pkg/cloud/externalconn/connection_storage.go @@ -92,7 +92,7 @@ func makeExternalConnectionStorage( uri.Path = path.Join(uri.Path, cfg.Path) return cloud.ExternalStorageFromURI(ctx, uri.String(), args.IOConf, args.Settings, args.BlobClientFactory, username.MakeSQLUsernameFromPreNormalizedString(cfg.User), - args.InternalExecutor, args.DB, args.Limiters, args.Options...) + args.InternalExecutor, args.CollectionFactory, args.DB, args.Limiters, args.Options...) default: return nil, errors.Newf("cannot connect to %T; unsupported resource for an ExternalStorage connection", d) } diff --git a/pkg/cloud/gcp/gcs_storage_test.go b/pkg/cloud/gcp/gcs_storage_test.go index e573687697fb..cb0e302c451d 100644 --- a/pkg/cloud/gcp/gcs_storage_test.go +++ b/pkg/cloud/gcp/gcs_storage_test.go @@ -60,18 +60,20 @@ func TestPutGoogleCloud(t *testing.T) { if specified { uri += fmt.Sprintf("&%s=%s", cloud.AuthParam, cloud.AuthParamSpecified) } - cloudtestutils.CheckExportStore(t, uri, false, user, nil, nil, testSettings) - cloudtestutils.CheckListFiles(t, - fmt.Sprintf("gs://%s/%s/%s?%s=%s&%s=%s", - bucket, - "backup-test-specified", - "listing-test", - cloud.AuthParam, - cloud.AuthParamSpecified, - CredentialsParam, - url.QueryEscape(encoded), - ), - username.RootUserName(), nil, nil, testSettings, + cloudtestutils.CheckExportStore(t, uri, false, user, nil, nil, nil, testSettings) + cloudtestutils.CheckListFiles(t, fmt.Sprintf("gs://%s/%s/%s?%s=%s&%s=%s", + bucket, + "backup-test-specified", + "listing-test", + cloud.AuthParam, + cloud.AuthParamSpecified, + CredentialsParam, + url.QueryEscape(encoded), + ), username.RootUserName(), + nil, /* ie */ + nil, /* cf */ + nil, /* kvDB */ + testSettings, ) }) t.Run("auth-implicit", func(t *testing.T) { @@ -80,16 +82,18 @@ func TestPutGoogleCloud(t *testing.T) { } cloudtestutils.CheckExportStore(t, fmt.Sprintf("gs://%s/%s?%s=%s", bucket, "backup-test-implicit", - cloud.AuthParam, cloud.AuthParamImplicit), false, user, nil, nil, testSettings) - cloudtestutils.CheckListFiles(t, - fmt.Sprintf("gs://%s/%s/%s?%s=%s", - bucket, - "backup-test-implicit", - "listing-test", - cloud.AuthParam, - cloud.AuthParamImplicit, - ), - username.RootUserName(), nil, nil, testSettings, + cloud.AuthParam, cloud.AuthParamImplicit), false, user, nil, nil, nil, testSettings) + cloudtestutils.CheckListFiles(t, fmt.Sprintf("gs://%s/%s/%s?%s=%s", + bucket, + "backup-test-implicit", + "listing-test", + cloud.AuthParam, + cloud.AuthParamImplicit, + ), username.RootUserName(), + nil, /* ie */ + nil, /* cf */ + nil, /* kvDB */ + testSettings, ) }) @@ -114,18 +118,20 @@ func TestPutGoogleCloud(t *testing.T) { token.AccessToken, ) uri += fmt.Sprintf("&%s=%s", cloud.AuthParam, cloud.AuthParamSpecified) - cloudtestutils.CheckExportStore(t, uri, false, user, nil, nil, testSettings) - cloudtestutils.CheckListFiles(t, - fmt.Sprintf("gs://%s/%s/%s?%s=%s&%s=%s", - bucket, - "backup-test-specified", - "listing-test", - cloud.AuthParam, - cloud.AuthParamSpecified, - BearerTokenParam, - token.AccessToken, - ), - username.RootUserName(), nil, nil, testSettings, + cloudtestutils.CheckExportStore(t, uri, false, user, nil, nil, nil, testSettings) + cloudtestutils.CheckListFiles(t, fmt.Sprintf("gs://%s/%s/%s?%s=%s&%s=%s", + bucket, + "backup-test-specified", + "listing-test", + cloud.AuthParam, + cloud.AuthParamSpecified, + BearerTokenParam, + token.AccessToken, + ), username.RootUserName(), + nil, /* ie */ + nil, /* cf */ + nil, /* kvDB */ + testSettings, ) }) } @@ -153,7 +159,12 @@ func TestGCSAssumeRole(t *testing.T) { // Verify that specified permissions with the credentials do not give us // access to the bucket. cloudtestutils.CheckNoPermission(t, fmt.Sprintf("gs://%s/%s?%s=%s", limitedBucket, "backup-test-assume-role", - CredentialsParam, url.QueryEscape(encoded)), user, nil, nil, testSettings) + CredentialsParam, url.QueryEscape(encoded)), user, + nil, /* ie */ + nil, /* cf */ + nil, /* kvDB */ + testSettings, + ) cloudtestutils.CheckExportStore(t, fmt.Sprintf("gs://%s/%s?%s=%s&%s=%s&%s=%s", limitedBucket, @@ -163,21 +174,22 @@ func TestGCSAssumeRole(t *testing.T) { AssumeRoleParam, assumedAccount, CredentialsParam, url.QueryEscape(encoded), - ), - false, user, nil, nil, testSettings) - cloudtestutils.CheckListFiles(t, - fmt.Sprintf("gs://%s/%s/%s?%s=%s&%s=%s&%s=%s", - limitedBucket, - "backup-test-assume-role", - "listing-test", - cloud.AuthParam, - cloud.AuthParamSpecified, - AssumeRoleParam, - assumedAccount, - CredentialsParam, - url.QueryEscape(encoded), - ), - username.RootUserName(), nil, nil, testSettings, + ), false, user, nil, nil, nil, testSettings) + cloudtestutils.CheckListFiles(t, fmt.Sprintf("gs://%s/%s/%s?%s=%s&%s=%s&%s=%s", + limitedBucket, + "backup-test-assume-role", + "listing-test", + cloud.AuthParam, + cloud.AuthParamSpecified, + AssumeRoleParam, + assumedAccount, + CredentialsParam, + url.QueryEscape(encoded), + ), username.RootUserName(), + nil, /* ie */ + nil, /* cf */ + nil, /* kvDB */ + testSettings, ) }) @@ -189,21 +201,33 @@ func TestGCSAssumeRole(t *testing.T) { // Verify that implicit permissions with the credentials do not give us // access to the bucket. cloudtestutils.CheckNoPermission(t, fmt.Sprintf("gs://%s/%s?%s=%s", limitedBucket, "backup-test-assume-role", - cloud.AuthParam, cloud.AuthParamImplicit), user, nil, nil, testSettings) + cloud.AuthParam, cloud.AuthParamImplicit), user, + nil, /* ie */ + nil, /* cf */ + nil, /* kvDB */ + testSettings, + ) cloudtestutils.CheckExportStore(t, fmt.Sprintf("gs://%s/%s?%s=%s&%s=%s", limitedBucket, "backup-test-assume-role", - cloud.AuthParam, cloud.AuthParamImplicit, AssumeRoleParam, assumedAccount), false, user, nil, nil, testSettings) - cloudtestutils.CheckListFiles(t, - fmt.Sprintf("gs://%s/%s/%s?%s=%s&%s=%s", - limitedBucket, - "backup-test-assume-role", - "listing-test", - cloud.AuthParam, - cloud.AuthParamImplicit, - AssumeRoleParam, - assumedAccount, - ), - username.RootUserName(), nil, nil, testSettings, + cloud.AuthParam, cloud.AuthParamImplicit, AssumeRoleParam, assumedAccount), false, user, + nil, /* ie */ + nil, /* cf */ + nil, /* kvDB */ + testSettings, + ) + cloudtestutils.CheckListFiles(t, fmt.Sprintf("gs://%s/%s/%s?%s=%s&%s=%s", + limitedBucket, + "backup-test-assume-role", + "listing-test", + cloud.AuthParam, + cloud.AuthParamImplicit, + AssumeRoleParam, + assumedAccount, + ), username.RootUserName(), + nil, /* ie */ + nil, /* cf */ + nil, /* kvDB */ + testSettings, ) }) @@ -243,7 +267,12 @@ func TestGCSAssumeRole(t *testing.T) { "listing-test", q.Encode(), ) - cloudtestutils.CheckNoPermission(t, roleURI, user, nil, nil, testSettings) + cloudtestutils.CheckNoPermission(t, roleURI, user, + nil, /* ie */ + nil, /* cf */ + nil, /* kvDB */ + testSettings, + ) } // Finally, check that the chain of roles can be used to access the storage. @@ -254,8 +283,18 @@ func TestGCSAssumeRole(t *testing.T) { "listing-test", q.Encode(), ) - cloudtestutils.CheckExportStore(t, uri, false, user, nil, nil, testSettings) - cloudtestutils.CheckListFiles(t, uri, user, nil, nil, testSettings) + cloudtestutils.CheckExportStore(t, uri, false, user, + nil, /* ie */ + nil, /* cf */ + nil, /* kvDB */ + testSettings, + ) + cloudtestutils.CheckListFiles(t, uri, user, + nil, /* ie */ + nil, /* cf */ + nil, /* kvDB */ + testSettings, + ) }) } }) @@ -297,9 +336,13 @@ func TestFileDoesNotExist(t *testing.T) { conf, err := cloud.ExternalStorageConfFromURI(gsFile, user) require.NoError(t, err) - s, err := cloud.MakeExternalStorage( - context.Background(), conf, base.ExternalIODirConfig{}, testSettings, - nil, nil, nil, nil) + s, err := cloud.MakeExternalStorage(context.Background(), conf, base.ExternalIODirConfig{}, testSettings, + nil, /* blobClientFactory */ + nil, /* ie */ + nil, /* cf */ + nil, /* kvDB */ + nil, /* limiters */ + ) require.NoError(t, err) _, err = s.ReadFile(context.Background(), "") require.Error(t, err, "") @@ -312,9 +355,13 @@ func TestFileDoesNotExist(t *testing.T) { conf, err := cloud.ExternalStorageConfFromURI(gsFile, user) require.NoError(t, err) - s, err := cloud.MakeExternalStorage( - context.Background(), conf, base.ExternalIODirConfig{}, testSettings, nil, - nil, nil, nil) + s, err := cloud.MakeExternalStorage(context.Background(), conf, base.ExternalIODirConfig{}, testSettings, + nil, /* blobClientFactory */ + nil, /* ie */ + nil, /* cf */ + nil, /* kvDB */ + nil, /* limiters */ + ) require.NoError(t, err) _, err = s.ReadFile(context.Background(), "") require.Error(t, err, "") @@ -345,9 +392,21 @@ func TestCompressedGCS(t *testing.T) { conf2, err := cloud.ExternalStorageConfFromURI(gsFile2, user) require.NoError(t, err) - s1, err := cloud.MakeExternalStorage(ctx, conf1, base.ExternalIODirConfig{}, testSettings, nil, nil, nil, nil) + s1, err := cloud.MakeExternalStorage(ctx, conf1, base.ExternalIODirConfig{}, testSettings, + nil, /* blobClientFactory */ + nil, /* ie */ + nil, /* cf */ + nil, /* kvDB */ + nil, /* limiters */ + ) require.NoError(t, err) - s2, err := cloud.MakeExternalStorage(ctx, conf2, base.ExternalIODirConfig{}, testSettings, nil, nil, nil, nil) + s2, err := cloud.MakeExternalStorage(ctx, conf2, base.ExternalIODirConfig{}, testSettings, + nil, /* blobClientFactory */ + nil, /* ie */ + nil, /* cf */ + nil, /* kvDB */ + nil, /* limiters */ + ) require.NoError(t, err) reader1, err := s1.ReadFile(context.Background(), "") diff --git a/pkg/cloud/httpsink/http_storage_test.go b/pkg/cloud/httpsink/http_storage_test.go index a90d836618af..6e8823be4b3c 100644 --- a/pkg/cloud/httpsink/http_storage_test.go +++ b/pkg/cloud/httpsink/http_storage_test.go @@ -121,7 +121,12 @@ func TestPutHttp(t *testing.T) { t.Run("singleHost", func(t *testing.T) { srv, files, cleanup := makeServer() defer cleanup() - cloudtestutils.CheckExportStore(t, srv.String(), false, user, nil, nil, testSettings) + cloudtestutils.CheckExportStore(t, srv.String(), false, user, + nil, /* ie */ + nil, /* cf */ + nil, /* kvDB */ + testSettings, + ) if expected, actual := 14, files(); expected != actual { t.Fatalf("expected %d files to be written to single http store, got %d", expected, actual) } @@ -138,7 +143,12 @@ func TestPutHttp(t *testing.T) { combined := *srv1 combined.Host = strings.Join([]string{srv1.Host, srv2.Host, srv3.Host}, ",") - cloudtestutils.CheckExportStore(t, combined.String(), true, user, nil, nil, testSettings) + cloudtestutils.CheckExportStore(t, combined.String(), true, user, + nil, /* ie */ + nil, /* cf */ + nil, /* kvDB */ + testSettings, + ) if expected, actual := 3, files1(); expected != actual { t.Fatalf("expected %d files written to http host 1, got %d", expected, actual) } @@ -161,8 +171,12 @@ func TestPutHttp(t *testing.T) { if err != nil { t.Fatal(err) } - s, err := cloud.MakeExternalStorage(ctx, conf, base.ExternalIODirConfig{}, - testSettings, blobs.TestEmptyBlobClientFactory, nil, nil, nil) + s, err := cloud.MakeExternalStorage(ctx, conf, base.ExternalIODirConfig{}, testSettings, blobs.TestEmptyBlobClientFactory, + nil, /* ie */ + nil, /* cf */ + nil, /* kvDB */ + nil, /* limiters */ + ) if err != nil { t.Fatal(err) } @@ -315,10 +329,12 @@ func TestCanDisableHttp(t *testing.T) { } testSettings := cluster.MakeTestingClusterSettings() - s, err := cloud.MakeExternalStorage( - context.Background(), - cloudpb.ExternalStorage{Provider: cloudpb.ExternalStorageProvider_http}, - conf, testSettings, blobs.TestEmptyBlobClientFactory, nil, nil, nil) + s, err := cloud.MakeExternalStorage(context.Background(), cloudpb.ExternalStorage{Provider: cloudpb.ExternalStorageProvider_http}, conf, testSettings, blobs.TestEmptyBlobClientFactory, + nil, /* ie */ + nil, /* cf */ + nil, /* kvDB */ + nil, /* limiters */ + ) require.Nil(t, s) require.Error(t, err) } @@ -336,10 +352,12 @@ func TestCanDisableOutbound(t *testing.T) { cloudpb.ExternalStorageProvider_gs, cloudpb.ExternalStorageProvider_nodelocal, } { - s, err := cloud.MakeExternalStorage( - context.Background(), - cloudpb.ExternalStorage{Provider: provider}, - conf, testSettings, blobs.TestEmptyBlobClientFactory, nil, nil, nil) + s, err := cloud.MakeExternalStorage(context.Background(), cloudpb.ExternalStorage{Provider: provider}, conf, testSettings, blobs.TestEmptyBlobClientFactory, + nil, /* ie */ + nil, /* cf */ + nil, /* kvDB */ + nil, /* limiters */ + ) require.Nil(t, s) require.Error(t, err) } @@ -368,9 +386,12 @@ func TestExternalStorageCanUseHTTPProxy(t *testing.T) { conf, err := cloud.ExternalStorageConfFromURI("http://my-server", username.RootUserName()) require.NoError(t, err) - s, err := cloud.MakeExternalStorage( - context.Background(), conf, base.ExternalIODirConfig{}, testSettings, nil, - nil, nil, nil) + s, err := cloud.MakeExternalStorage(context.Background(), conf, base.ExternalIODirConfig{}, testSettings, nil, + nil, /* ie */ + nil, /* cf */ + nil, /* kvDB */ + nil, /* limiters */ + ) require.NoError(t, err) stream, err := s.ReadFile(context.Background(), "file") require.NoError(t, err) diff --git a/pkg/cloud/impl_registry.go b/pkg/cloud/impl_registry.go index 1f63eb0def7d..065258c0bd85 100644 --- a/pkg/cloud/impl_registry.go +++ b/pkg/cloud/impl_registry.go @@ -25,6 +25,7 @@ import ( "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/catalog/descs" "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" "github.com/cockroachdb/cockroach/pkg/util/ioctx" "github.com/cockroachdb/cockroach/pkg/util/log" @@ -151,6 +152,7 @@ func ExternalStorageFromURI( blobClientFactory blobs.BlobClientFactory, user username.SQLUsername, ie sqlutil.InternalExecutor, + cf *descs.CollectionFactory, kvDB *kv.DB, limiters Limiters, opts ...ExternalStorageOption, @@ -159,7 +161,8 @@ func ExternalStorageFromURI( if err != nil { return nil, err } - return MakeExternalStorage(ctx, conf, externalConfig, settings, blobClientFactory, ie, kvDB, limiters, opts...) + return MakeExternalStorage(ctx, conf, externalConfig, settings, blobClientFactory, + ie, cf, kvDB, limiters, opts...) } // SanitizeExternalStorageURI returns the external storage URI with with some @@ -204,6 +207,7 @@ func MakeExternalStorage( settings *cluster.Settings, blobClientFactory blobs.BlobClientFactory, ie sqlutil.InternalExecutor, + cf *descs.CollectionFactory, kvDB *kv.DB, limiters Limiters, opts ...ExternalStorageOption, @@ -213,6 +217,7 @@ func MakeExternalStorage( Settings: settings, BlobClientFactory: blobClientFactory, InternalExecutor: ie, + CollectionFactory: cf, DB: kvDB, Options: opts, Limiters: limiters, diff --git a/pkg/cloud/nodelocal/nodelocal_storage_test.go b/pkg/cloud/nodelocal/nodelocal_storage_test.go index 254f96f6c37d..4792c601b8e9 100644 --- a/pkg/cloud/nodelocal/nodelocal_storage_test.go +++ b/pkg/cloud/nodelocal/nodelocal_storage_test.go @@ -30,7 +30,11 @@ func TestPutLocal(t *testing.T) { testSettings.ExternalIODir = p dest := MakeLocalStorageURI(p) - cloudtestutils.CheckExportStore(t, dest, false, username.RootUserName(), nil, nil, testSettings) - cloudtestutils.CheckListFiles(t, "nodelocal://0/listing-test/basepath", - username.RootUserName(), nil, nil, testSettings) + cloudtestutils.CheckExportStore(t, dest, false, username.RootUserName(), nil, nil, nil, testSettings) + cloudtestutils.CheckListFiles(t, "nodelocal://0/listing-test/basepath", username.RootUserName(), + nil, /* ie */ + nil, /* cf */ + nil, /* kvDB */ + testSettings, + ) } diff --git a/pkg/cloud/nullsink/nullsink_storage_test.go b/pkg/cloud/nullsink/nullsink_storage_test.go index a2a68c1d6a3e..04f898ab33d6 100644 --- a/pkg/cloud/nullsink/nullsink_storage_test.go +++ b/pkg/cloud/nullsink/nullsink_storage_test.go @@ -36,7 +36,14 @@ func TestNullSinkReadAndWrite(t *testing.T) { t.Fatal(err) } - s, err := cloud.MakeExternalStorage(ctx, conf, base.ExternalIODirConfig{}, nil, nil, nil, nil, nil) + s, err := cloud.MakeExternalStorage(ctx, conf, base.ExternalIODirConfig{}, + nil, /* Cluster Settings */ + nil, /* blobClientFactory */ + nil, /* ie */ + nil, /* cf */ + nil, /* kvDB */ + nil, /* limiters */ + ) if err != nil { t.Fatal(err) } diff --git a/pkg/cloud/userfile/BUILD.bazel b/pkg/cloud/userfile/BUILD.bazel index 05ca28916e74..da46209f30b1 100644 --- a/pkg/cloud/userfile/BUILD.bazel +++ b/pkg/cloud/userfile/BUILD.bazel @@ -46,6 +46,7 @@ go_test( "//pkg/security/username", "//pkg/server", "//pkg/settings/cluster", + "//pkg/sql/catalog/descs", "//pkg/sql/sem/tree", "//pkg/sql/sqlutil", "//pkg/sql/tests", diff --git a/pkg/cloud/userfile/file_table_storage.go b/pkg/cloud/userfile/file_table_storage.go index b7cf6c42c055..c30b9fe7c06a 100644 --- a/pkg/cloud/userfile/file_table_storage.go +++ b/pkg/cloud/userfile/file_table_storage.go @@ -122,7 +122,7 @@ func makeFileTableStorage( // cfg.User is already a normalized SQL username. user := username.MakeSQLUsernameFromPreNormalizedString(cfg.User) - executor := filetable.MakeInternalFileToTableExecutor(args.InternalExecutor, args.DB) + executor := filetable.MakeInternalFileToTableExecutor(args.InternalExecutor, args.CollectionFactory, args.DB) fileToTableSystem, err := filetable.NewFileToTableSystem(ctx, cfg.QualifiedTableName, executor, user) diff --git a/pkg/cloud/userfile/file_table_storage_test.go b/pkg/cloud/userfile/file_table_storage_test.go index a835ccb875df..940d009a61b0 100644 --- a/pkg/cloud/userfile/file_table_storage_test.go +++ b/pkg/cloud/userfile/file_table_storage_test.go @@ -23,6 +23,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/cloud/cloudtestutils" "github.com/cockroachdb/cockroach/pkg/security/username" "github.com/cockroachdb/cockroach/pkg/settings/cluster" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/descs" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" "github.com/cockroachdb/cockroach/pkg/sql/tests" @@ -48,19 +49,21 @@ func TestPutUserFileTable(t *testing.T) { dest := MakeUserFileStorageURI(qualifiedTableName, filename) ie := s.InternalExecutor().(sqlutil.InternalExecutor) - cloudtestutils.CheckExportStore(t, dest, false, username.RootUserName(), ie, kvDB, testSettings) + cf := s.CollectionFactory().(*descs.CollectionFactory) + cloudtestutils.CheckExportStore(t, dest, false, username.RootUserName(), ie, cf, kvDB, testSettings) cloudtestutils.CheckListFiles(t, "userfile://defaultdb.public.file_list_table/listing-test/basepath", - username.RootUserName(), ie, kvDB, testSettings) + username.RootUserName(), ie, cf, kvDB, testSettings) t.Run("empty-qualified-table-name", func(t *testing.T) { dest := MakeUserFileStorageURI("", filename) ie := s.InternalExecutor().(sqlutil.InternalExecutor) - cloudtestutils.CheckExportStore(t, dest, false, username.RootUserName(), ie, kvDB, testSettings) + cf := s.CollectionFactory().(*descs.CollectionFactory) + cloudtestutils.CheckExportStore(t, dest, false, username.RootUserName(), ie, cf, kvDB, testSettings) cloudtestutils.CheckListFilesCanonical(t, "userfile:///listing-test/basepath", "userfile://defaultdb.public.userfiles_root/listing-test/basepath", - username.RootUserName(), ie, kvDB, testSettings) + username.RootUserName(), ie, cf, kvDB, testSettings) }) t.Run("reject-normalized-basename", func(t *testing.T) { @@ -68,8 +71,7 @@ func TestPutUserFileTable(t *testing.T) { userfileURL := url.URL{Scheme: "userfile", Host: qualifiedTableName, Path: ""} store, err := cloud.ExternalStorageFromURI(ctx, userfileURL.String()+"/", - base.ExternalIODirConfig{}, cluster.NoSettings, blobs.TestEmptyBlobClientFactory, - username.RootUserName(), ie, kvDB, nil) + base.ExternalIODirConfig{}, cluster.NoSettings, blobs.TestEmptyBlobClientFactory, username.RootUserName(), ie, cf, kvDB, nil) require.NoError(t, err) defer store.Close() @@ -107,6 +109,7 @@ func TestUserScoping(t *testing.T) { dest := MakeUserFileStorageURI(qualifiedTableName, "") ie := s.InternalExecutor().(sqlutil.InternalExecutor) + cf := s.CollectionFactory().(*descs.CollectionFactory) // Create two users and grant them all privileges on defaultdb. user1 := username.MakeSQLUsernameFromPreNormalizedString("foo") @@ -116,13 +119,13 @@ func TestUserScoping(t *testing.T) { // Write file as user1. fileTableSystem1, err := cloud.ExternalStorageFromURI(ctx, dest, base.ExternalIODirConfig{}, - cluster.NoSettings, blobs.TestEmptyBlobClientFactory, user1, ie, kvDB, nil) + cluster.NoSettings, blobs.TestEmptyBlobClientFactory, user1, ie, cf, kvDB, nil) require.NoError(t, err) require.NoError(t, cloud.WriteFile(ctx, fileTableSystem1, filename, bytes.NewReader([]byte("aaa")))) // Attempt to read/write file as user2 and expect to fail. fileTableSystem2, err := cloud.ExternalStorageFromURI(ctx, dest, base.ExternalIODirConfig{}, - cluster.NoSettings, blobs.TestEmptyBlobClientFactory, user2, ie, kvDB, nil) + cluster.NoSettings, blobs.TestEmptyBlobClientFactory, user2, ie, cf, kvDB, nil) require.NoError(t, err) _, err = fileTableSystem2.ReadFile(ctx, filename) require.Error(t, err) @@ -130,7 +133,7 @@ func TestUserScoping(t *testing.T) { // Read file as root and expect to succeed. fileTableSystem3, err := cloud.ExternalStorageFromURI(ctx, dest, base.ExternalIODirConfig{}, - cluster.NoSettings, blobs.TestEmptyBlobClientFactory, username.RootUserName(), ie, kvDB, nil) + cluster.NoSettings, blobs.TestEmptyBlobClientFactory, username.RootUserName(), ie, cf, kvDB, nil) require.NoError(t, err) _, err = fileTableSystem3.ReadFile(ctx, filename) require.NoError(t, err) diff --git a/pkg/cloud/userfile/filetable/BUILD.bazel b/pkg/cloud/userfile/filetable/BUILD.bazel index 4e2f82feda0c..5ca59c1ae3d4 100644 --- a/pkg/cloud/userfile/filetable/BUILD.bazel +++ b/pkg/cloud/userfile/filetable/BUILD.bazel @@ -10,6 +10,7 @@ go_library( "//pkg/cloud", "//pkg/kv", "//pkg/security/username", + "//pkg/sql/catalog/descs", "//pkg/sql/parser", "//pkg/sql/sem/tree", "//pkg/sql/sessiondata", diff --git a/pkg/cloud/userfile/filetable/file_table_read_writer.go b/pkg/cloud/userfile/filetable/file_table_read_writer.go index 75b5bbfd1514..994163e315d6 100644 --- a/pkg/cloud/userfile/filetable/file_table_read_writer.go +++ b/pkg/cloud/userfile/filetable/file_table_read_writer.go @@ -22,6 +22,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/cloud" "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/security/username" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/descs" "github.com/cockroachdb/cockroach/pkg/sql/parser" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" @@ -60,6 +61,7 @@ type FileToTableSystemExecutor interface { // SQL connection to interact with the database. type InternalFileToTableExecutor struct { ie sqlutil.InternalExecutor + cf *descs.CollectionFactory db *kv.DB } @@ -68,9 +70,9 @@ var _ FileToTableSystemExecutor = &InternalFileToTableExecutor{} // MakeInternalFileToTableExecutor returns an instance of a // InternalFileToTableExecutor. func MakeInternalFileToTableExecutor( - ie sqlutil.InternalExecutor, db *kv.DB, + ie sqlutil.InternalExecutor, cf *descs.CollectionFactory, db *kv.DB, ) *InternalFileToTableExecutor { - return &InternalFileToTableExecutor{ie, db} + return &InternalFileToTableExecutor{ie, cf, db} } // Query implements the FileToTableSystemExecutor interface. diff --git a/pkg/cloud/userfile/filetable/filetabletest/BUILD.bazel b/pkg/cloud/userfile/filetable/filetabletest/BUILD.bazel index dfd5e05f2e18..e99f83fa5def 100644 --- a/pkg/cloud/userfile/filetable/filetabletest/BUILD.bazel +++ b/pkg/cloud/userfile/filetable/filetabletest/BUILD.bazel @@ -17,6 +17,7 @@ go_test( "//pkg/security/username", "//pkg/server", "//pkg/sql", + "//pkg/sql/catalog/descs", "//pkg/sql/tests", "//pkg/testutils", "//pkg/testutils/serverutils", diff --git a/pkg/cloud/userfile/filetable/filetabletest/file_table_read_writer_test.go b/pkg/cloud/userfile/filetable/filetabletest/file_table_read_writer_test.go index 8fab37c5489b..d5ad2777b2b6 100644 --- a/pkg/cloud/userfile/filetable/filetabletest/file_table_read_writer_test.go +++ b/pkg/cloud/userfile/filetable/filetabletest/file_table_read_writer_test.go @@ -22,6 +22,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/security/username" "github.com/cockroachdb/cockroach/pkg/sql" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/descs" "github.com/cockroachdb/cockroach/pkg/sql/tests" "github.com/cockroachdb/cockroach/pkg/testutils" "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" @@ -107,7 +108,7 @@ func TestListAndDeleteFiles(t *testing.T) { defer s.Stopper().Stop(ctx) executor := filetable.MakeInternalFileToTableExecutor(s.InternalExecutor().(*sql. - InternalExecutor), kvDB) + InternalExecutor), s.CollectionFactory().(*descs.CollectionFactory), kvDB) fileTableReadWriter, err := filetable.NewFileToTableSystem(ctx, qualifiedTableName, executor, username.RootUserName()) require.NoError(t, err) @@ -158,7 +159,7 @@ func TestReadWriteFile(t *testing.T) { defer s.Stopper().Stop(ctx) executor := filetable.MakeInternalFileToTableExecutor(s.InternalExecutor().(*sql. - InternalExecutor), kvDB) + InternalExecutor), s.CollectionFactory().(*descs.CollectionFactory), kvDB) fileTableReadWriter, err := filetable.NewFileToTableSystem(ctx, qualifiedTableName, executor, username.RootUserName()) require.NoError(t, err) @@ -341,7 +342,7 @@ func TestUserGrants(t *testing.T) { // Operate under non-admin user. executor := filetable.MakeInternalFileToTableExecutor(s.InternalExecutor().(*sql. - InternalExecutor), kvDB) + InternalExecutor), s.CollectionFactory().(*descs.CollectionFactory), kvDB) johnUser := username.MakeSQLUsernameFromPreNormalizedString("john") fileTableReadWriter, err := filetable.NewFileToTableSystem(ctx, qualifiedTableName, executor, johnUser) @@ -425,7 +426,7 @@ func TestDifferentUserDisallowed(t *testing.T) { // Operate under non-admin user john. executor := filetable.MakeInternalFileToTableExecutor(s.InternalExecutor().(*sql. - InternalExecutor), kvDB) + InternalExecutor), s.CollectionFactory().(*descs.CollectionFactory), kvDB) johnUser := username.MakeSQLUsernameFromPreNormalizedString("john") fileTableReadWriter, err := filetable.NewFileToTableSystem(ctx, qualifiedTableName, executor, johnUser) @@ -483,7 +484,7 @@ func TestDifferentRoleDisallowed(t *testing.T) { // Operate under non-admin user john. executor := filetable.MakeInternalFileToTableExecutor(s.InternalExecutor().(*sql. - InternalExecutor), kvDB) + InternalExecutor), s.CollectionFactory().(*descs.CollectionFactory), kvDB) johnUser := username.MakeSQLUsernameFromPreNormalizedString("john") fileTableReadWriter, err := filetable.NewFileToTableSystem(ctx, qualifiedTableName, executor, johnUser) @@ -518,7 +519,7 @@ func TestDatabaseScope(t *testing.T) { defer s.Stopper().Stop(ctx) executor := filetable.MakeInternalFileToTableExecutor(s.InternalExecutor().(*sql. - InternalExecutor), kvDB) + InternalExecutor), s.CollectionFactory().(*descs.CollectionFactory), kvDB) fileTableReadWriter, err := filetable.NewFileToTableSystem(ctx, qualifiedTableName, executor, username.RootUserName()) require.NoError(t, err) diff --git a/pkg/server/external_storage_builder.go b/pkg/server/external_storage_builder.go index 77780f680cf9..d2da4d4da08f 100644 --- a/pkg/server/external_storage_builder.go +++ b/pkg/server/external_storage_builder.go @@ -24,6 +24,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/security/username" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/descs" "github.com/cockroachdb/errors" ) @@ -38,6 +39,7 @@ type externalStorageBuilder struct { blobClientFactory blobs.BlobClientFactory initCalled bool ie *sql.InternalExecutor + cf *descs.CollectionFactory db *kv.DB limiters cloud.Limiters recorder multitenant.TenantSideExternalIORecorder @@ -51,6 +53,7 @@ func (e *externalStorageBuilder) init( nodeDialer *nodedialer.Dialer, testingKnobs base.TestingKnobs, ie *sql.InternalExecutor, + cf *descs.CollectionFactory, db *kv.DB, recorder multitenant.TenantSideExternalIORecorder, ) { @@ -66,6 +69,7 @@ func (e *externalStorageBuilder) init( e.blobClientFactory = blobClientFactory e.initCalled = true e.ie = ie + e.cf = cf e.db = db e.limiters = cloud.MakeLimiters(ctx, &settings.SV) e.recorder = recorder @@ -78,7 +82,7 @@ func (e *externalStorageBuilder) makeExternalStorage( return nil, errors.New("cannot create external storage before init") } return cloud.MakeExternalStorage(ctx, dest, e.conf, e.settings, e.blobClientFactory, e.ie, - e.db, e.limiters, append(e.defaultOptions(), opts...)...) + e.cf, e.db, e.limiters, append(e.defaultOptions(), opts...)...) } func (e *externalStorageBuilder) makeExternalStorageFromURI( @@ -87,7 +91,8 @@ func (e *externalStorageBuilder) makeExternalStorageFromURI( if !e.initCalled { return nil, errors.New("cannot create external storage before init") } - return cloud.ExternalStorageFromURI(ctx, uri, e.conf, e.settings, e.blobClientFactory, user, e.ie, e.db, e.limiters, append(e.defaultOptions(), opts...)...) + return cloud.ExternalStorageFromURI(ctx, uri, e.conf, e.settings, e.blobClientFactory, + user, e.ie, e.cf, e.db, e.limiters, append(e.defaultOptions(), opts...)...) } func (e *externalStorageBuilder) defaultOptions() []cloud.ExternalStorageOption { diff --git a/pkg/server/server.go b/pkg/server/server.go index a25b70ab119f..a45df05d2055 100644 --- a/pkg/server/server.go +++ b/pkg/server/server.go @@ -64,6 +64,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/spanconfig/spanconfigkvsubscriber" "github.com/cockroachdb/cockroach/pkg/spanconfig/spanconfigptsreader" "github.com/cockroachdb/cockroach/pkg/sql" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/descs" _ "github.com/cockroachdb/cockroach/pkg/sql/catalog/schematelemetry" // register schedules declared outside of pkg/sql "github.com/cockroachdb/cockroach/pkg/sql/catalog/systemschema" "github.com/cockroachdb/cockroach/pkg/sql/flowinfra" @@ -487,6 +488,7 @@ func NewServer(cfg Config, stopper *stop.Stopper) (*Server, error) { // InternalExecutor uses this one instance. internalExecutor := &sql.InternalExecutor{} jobRegistry := &jobs.Registry{} // ditto + collectionFactory := &descs.CollectionFactory{} // Create an ExternalStorageBuilder. This is only usable after Start() where // we initialize all the configuration params. @@ -836,6 +838,7 @@ func NewServer(cfg Config, stopper *stop.Stopper) (*Server, error) { closedSessionCache: closedSessionCache, flowScheduler: flowScheduler, circularInternalExecutor: internalExecutor, + collectionFactory: collectionFactory, internalExecutorFactory: nil, // will be initialized in server.newSQLServer. circularJobRegistry: jobRegistry, jobAdoptionStopFile: jobAdoptionStopFile, @@ -1083,6 +1086,7 @@ func (s *Server) PreStart(ctx context.Context) error { s.nodeDialer, s.cfg.TestingKnobs, &fileTableInternalExecutor, + s.sqlServer.execCfg.CollectionFactory, s.db, nil, /* TenantExternalIORecorder */ ) diff --git a/pkg/server/server_sql.go b/pkg/server/server_sql.go index f8b48364579e..560777c008f0 100644 --- a/pkg/server/server_sql.go +++ b/pkg/server/server_sql.go @@ -306,6 +306,8 @@ type sqlServerArgs struct { // TODO(tbg): make this less hacky. circularInternalExecutor *sql.InternalExecutor // empty initially + collectionFactory *descs.CollectionFactory + // internalExecutorFactory is to initialize an internal executor. internalExecutorFactory sqlutil.InternalExecutorFactory @@ -988,6 +990,7 @@ func newSQLServer(ctx context.Context, cfg sqlServerArgs) (*SQLServer, error) { cfg.registry.AddMetricStruct(m) } *cfg.circularInternalExecutor = sql.MakeInternalExecutor(pgServer.SQLServer, internalMemMetrics, ieFactoryMonitor) + *cfg.collectionFactory = *collectionFactory cfg.internalExecutorFactory = ieFactory execCfg.InternalExecutor = cfg.circularInternalExecutor stmtDiagnosticsRegistry := stmtdiagnostics.NewRegistry( diff --git a/pkg/server/tenant.go b/pkg/server/tenant.go index adb9802b286f..e83e31883b9d 100644 --- a/pkg/server/tenant.go +++ b/pkg/server/tenant.go @@ -42,6 +42,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/server/systemconfigwatcher" "github.com/cockroachdb/cockroach/pkg/settings/cluster" "github.com/cockroachdb/cockroach/pkg/sql" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/descs" "github.com/cockroachdb/cockroach/pkg/sql/flowinfra" "github.com/cockroachdb/cockroach/pkg/sql/optionalnodeliveness" "github.com/cockroachdb/cockroach/pkg/sql/sessiondatapb" @@ -500,6 +501,7 @@ func makeTenantSQLServerArgs( circularInternalExecutor := &sql.InternalExecutor{} circularJobRegistry := &jobs.Registry{} + collectionFactory := &descs.CollectionFactory{} // Initialize the protectedts subsystem in multi-tenant clusters. var protectedTSProvider protectedts.Provider @@ -539,6 +541,7 @@ func makeTenantSQLServerArgs( nodeDialer, baseCfg.TestingKnobs, circularInternalExecutor, + collectionFactory, db, costController, ) @@ -607,6 +610,7 @@ func makeTenantSQLServerArgs( sessionRegistry: sessionRegistry, flowScheduler: flowScheduler, circularInternalExecutor: circularInternalExecutor, + collectionFactory: collectionFactory, circularJobRegistry: circularJobRegistry, protectedtsProvider: protectedTSProvider, rangeFeedFactory: rangeFeedFactory, diff --git a/pkg/sql/importer/import_processor_test.go b/pkg/sql/importer/import_processor_test.go index a12967e259b8..41ac6aac9511 100644 --- a/pkg/sql/importer/import_processor_test.go +++ b/pkg/sql/importer/import_processor_test.go @@ -876,7 +876,12 @@ func externalStorageFactory( return nil, err } return cloud.MakeExternalStorage(ctx, dest, base.ExternalIODirConfig{}, - nil, blobs.TestBlobServiceClient(workdir), nil, nil, nil) + nil, blobs.TestBlobServiceClient(workdir), + nil, /* ie */ + nil, /* cf */ + nil, /* kvDB */ + nil, /* limiters */ + ) } // Helper to create and initialize testSpec. diff --git a/pkg/sql/importer/import_stmt_test.go b/pkg/sql/importer/import_stmt_test.go index 013c27596508..e9345f9f0d52 100644 --- a/pkg/sql/importer/import_stmt_test.go +++ b/pkg/sql/importer/import_stmt_test.go @@ -2700,8 +2700,9 @@ func TestImportObjectLevelRBAC(t *testing.T) { writeToUserfile := func(filename, data string) { // Write to userfile storage now that testuser has CREATE privileges. ie := tc.Server(0).InternalExecutor().(*sql.InternalExecutor) + cf := tc.Server(0).CollectionFactory().(*descs.CollectionFactory) fileTableSystem1, err := cloud.ExternalStorageFromURI(ctx, dest, base.ExternalIODirConfig{}, - cluster.NoSettings, blobs.TestEmptyBlobClientFactory, username.TestUserName(), ie, tc.Server(0).DB(), nil) + cluster.NoSettings, blobs.TestEmptyBlobClientFactory, username.TestUserName(), ie, cf, tc.Server(0).DB(), nil) require.NoError(t, err) require.NoError(t, cloud.WriteFile(ctx, fileTableSystem1, filename, bytes.NewReader([]byte(data)))) } @@ -5848,7 +5849,11 @@ func TestImportPgDumpIgnoredStmts(t *testing.T) { tc.Server(0).ClusterSettings(), blobs.TestEmptyBlobClientFactory, username.RootUserName(), - tc.Server(0).InternalExecutor().(*sql.InternalExecutor), tc.Server(0).DB(), nil) + tc.Server(0).InternalExecutor().(*sql.InternalExecutor), + tc.Server(0).CollectionFactory().(*descs.CollectionFactory), + tc.Server(0).DB(), + nil, + ) require.NoError(t, err) defer store.Close() From 3c44cadf4b72802ea61df2b128ab01f1baee453c Mon Sep 17 00:00:00 2001 From: Jane Xing Date: Wed, 24 Aug 2022 10:17:13 -0500 Subject: [PATCH 09/18] filetable: use `CollectionFactory.TxnWithExecutor()` for DDL statements fixes #76764 Release justification: Low risk, high benefit changes to existing functionality Release note: none --- .../backup-restore/file_table_read_write | 43 ++++++++++++++++ .../filetable/file_table_read_writer.go | 49 ++++++++++--------- 2 files changed, 69 insertions(+), 23 deletions(-) create mode 100644 pkg/ccl/backupccl/testdata/backup-restore/file_table_read_write diff --git a/pkg/ccl/backupccl/testdata/backup-restore/file_table_read_write b/pkg/ccl/backupccl/testdata/backup-restore/file_table_read_write new file mode 100644 index 000000000000..328263642493 --- /dev/null +++ b/pkg/ccl/backupccl/testdata/backup-restore/file_table_read_write @@ -0,0 +1,43 @@ +subtest backup_file_table + +new-server name=s1 +---- + +exec-sql +CREATE DATABASE to_backup; +---- + +exec-sql +CREATE DATABASE backups; +---- + +exec-sql +BACKUP DATABASE to_backup INTO 'userfile://backups.public.userfiles_$user/data'; +---- + +query-sql +SELECT * FROM backups.crdb_internal.invalid_objects; +---- + +exec-sql +USE backups; +---- + +query-sql +SELECT * FROM pg_catalog.pg_tables where schemaname='public'; +---- +public userfiles_$user_upload_files root true false false false +public userfiles_$user_upload_payload root true false false false + +query-sql +SELECT conname FROM pg_catalog.pg_constraint con +INNER JOIN pg_catalog.pg_class rel ON rel.oid = con.conrelid +INNER JOIN pg_catalog.pg_namespace nsp +ON nsp.oid = connamespace +WHERE rel.relname='userfiles_$user_upload_payload' +ORDER BY conname; +---- +file_id_fk +userfiles_$user_upload_payload_pkey + +subtest end diff --git a/pkg/cloud/userfile/filetable/file_table_read_writer.go b/pkg/cloud/userfile/filetable/file_table_read_writer.go index 994163e315d6..182c6720112f 100644 --- a/pkg/cloud/userfile/filetable/file_table_read_writer.go +++ b/pkg/cloud/userfile/filetable/file_table_read_writer.go @@ -245,25 +245,27 @@ func NewFileToTableSystem( if err != nil { return nil, err } - if err := e.db.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error { + if err := e.cf.TxnWithExecutor(ctx, e.db, nil /* SessionData */, func( + ctx context.Context, txn *kv.Txn, descriptors *descs.Collection, ie sqlutil.InternalExecutor, + ) error { // TODO(adityamaru): Handle scenario where the user has already created // tables with the same names not via the FileToTableSystem // object. Not sure if we want to error out or work around it. - tablesExist, err := f.checkIfFileAndPayloadTableExist(ctx, txn, e.ie) + tablesExist, err := f.checkIfFileAndPayloadTableExist(ctx, txn, ie) if err != nil { return err } if !tablesExist { - if err := f.createFileAndPayloadTables(ctx, txn, e.ie); err != nil { + if err := f.createFileAndPayloadTables(ctx, txn, ie); err != nil { return err } - if err := f.grantCurrentUserTablePrivileges(ctx, txn, e.ie); err != nil { + if err := f.grantCurrentUserTablePrivileges(ctx, txn, ie); err != nil { return err } - if err := f.revokeOtherUserTablePrivileges(ctx, txn, e.ie); err != nil { + if err := f.revokeOtherUserTablePrivileges(ctx, txn, ie); err != nil { return err } } @@ -364,26 +366,27 @@ func DestroyUserFileSystem(ctx context.Context, f *FileToTableSystem) error { return err } - if err := e.db.Txn(ctx, - func(ctx context.Context, txn *kv.Txn) error { - dropPayloadTableQuery := fmt.Sprintf(`DROP TABLE %s`, f.GetFQPayloadTableName()) - _, err := e.ie.ExecEx(ctx, "drop-payload-table", txn, - sessiondata.InternalExecutorOverride{User: f.username}, - dropPayloadTableQuery) - if err != nil { - return errors.Wrap(err, "failed to drop payload table") - } + if err := e.cf.TxnWithExecutor(ctx, e.db, nil, func( + ctx context.Context, txn *kv.Txn, descriptors *descs.Collection, ie sqlutil.InternalExecutor, + ) error { + dropPayloadTableQuery := fmt.Sprintf(`DROP TABLE %s`, f.GetFQPayloadTableName()) + _, err := ie.ExecEx(ctx, "drop-payload-table", txn, + sessiondata.InternalExecutorOverride{User: f.username}, + dropPayloadTableQuery) + if err != nil { + return errors.Wrap(err, "failed to drop payload table") + } - dropFileTableQuery := fmt.Sprintf(`DROP TABLE %s CASCADE`, f.GetFQFileTableName()) - _, err = e.ie.ExecEx(ctx, "drop-file-table", txn, - sessiondata.InternalExecutorOverride{User: f.username}, - dropFileTableQuery) - if err != nil { - return errors.Wrap(err, "failed to drop file table") - } + dropFileTableQuery := fmt.Sprintf(`DROP TABLE %s CASCADE`, f.GetFQFileTableName()) + _, err = ie.ExecEx(ctx, "drop-file-table", txn, + sessiondata.InternalExecutorOverride{User: f.username}, + dropFileTableQuery) + if err != nil { + return errors.Wrap(err, "failed to drop file table") + } - return nil - }); err != nil { + return nil + }); err != nil { return err } From 5a4beb4a679c740ec874d5dd3d856ec4f5474ca2 Mon Sep 17 00:00:00 2001 From: Jane Xing Date: Wed, 24 Aug 2022 10:18:39 -0500 Subject: [PATCH 10/18] sql: migreate authorization_test to `CollectionFactory.TxnWithExecutor()` for DDLs This is part of the project to migrate existing DDL statements running with an internal executor to `descs.CollectionFactory()`. DDLs are only allowed to run with internal executor inited via this function. Release justification: Low risk, high benefit changes to existing functionality Release note: None --- pkg/sql/authorization_test.go | 83 +++++++++++++++++++---------------- 1 file changed, 46 insertions(+), 37 deletions(-) diff --git a/pkg/sql/authorization_test.go b/pkg/sql/authorization_test.go index cae7b84ab9d7..c628a982382f 100644 --- a/pkg/sql/authorization_test.go +++ b/pkg/sql/authorization_test.go @@ -15,8 +15,10 @@ import ( "testing" "github.com/cockroachdb/cockroach/pkg/base" + "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/security/username" "github.com/cockroachdb/cockroach/pkg/server" + "github.com/cockroachdb/cockroach/pkg/sql/catalog/descs" "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" @@ -30,7 +32,7 @@ func TestCheckAnyPrivilegeForNodeUser(t *testing.T) { defer log.Scope(t).Close(t) ctx := context.Background() - s, _, kv := serverutils.StartServer(t, base.TestServerArgs{}) + s, _, _ := serverutils.StartServer(t, base.TestServerArgs{}) defer s.Stopper().Stop(ctx) @@ -38,45 +40,52 @@ func TestCheckAnyPrivilegeForNodeUser(t *testing.T) { require.NotNil(t, ts.InternalExecutor()) - ie := ts.InternalExecutor().(sqlutil.InternalExecutor) + cf := ts.CollectionFactory().(*descs.CollectionFactory) - txn := kv.NewTxn(ctx, "get-all-databases") - row, err := ie.QueryRowEx( - ctx, "get-all-databases", txn, sessiondata.NodeUserSessionDataOverride, - "SELECT count(1) FROM crdb_internal.databases", - ) - require.NoError(t, err) - // 3 databases (system, defaultdb, postgres). - require.Equal(t, row.String(), "(3)") + if err := cf.TxnWithExecutor(ctx, s.DB(), nil, func( + ctx context.Context, txn *kv.Txn, descriptors *descs.Collection, ie sqlutil.InternalExecutor, + ) error { + row, err := ie.QueryRowEx( + ctx, "get-all-databases", txn, sessiondata.NodeUserSessionDataOverride, + "SELECT count(1) FROM crdb_internal.databases", + ) + require.NoError(t, err) + // 3 databases (system, defaultdb, postgres). + require.Equal(t, row.String(), "(3)") - _, err = ie.ExecEx(ctx, "create-database1", txn, sessiondata.InternalExecutorOverride{User: username.RootUserName()}, - "CREATE DATABASE test1") - require.NoError(t, err) + _, err = ie.ExecEx(ctx, "create-database1", txn, sessiondata.InternalExecutorOverride{User: username.RootUserName()}, + "CREATE DATABASE test1") + require.NoError(t, err) - _, err = ie.ExecEx(ctx, "create-database2", txn, sessiondata.InternalExecutorOverride{User: username.RootUserName()}, - "CREATE DATABASE test2") - require.NoError(t, err) + _, err = ie.ExecEx(ctx, "create-database2", txn, sessiondata.InternalExecutorOverride{User: username.RootUserName()}, + "CREATE DATABASE test2") + require.NoError(t, err) - // Revoke CONNECT on all non-system databases and ensure that when querying - // with node, we can still see all the databases. - _, err = ie.ExecEx(ctx, "revoke-privileges", txn, sessiondata.InternalExecutorOverride{User: username.RootUserName()}, - "REVOKE CONNECT ON DATABASE test1 FROM public") - require.NoError(t, err) - _, err = ie.ExecEx(ctx, "revoke-privileges", txn, sessiondata.InternalExecutorOverride{User: username.RootUserName()}, - "REVOKE CONNECT ON DATABASE test2 FROM public") - require.NoError(t, err) - _, err = ie.ExecEx(ctx, "revoke-privileges", txn, sessiondata.InternalExecutorOverride{User: username.RootUserName()}, - "REVOKE CONNECT ON DATABASE defaultdb FROM public") - require.NoError(t, err) - _, err = ie.ExecEx(ctx, "revoke-privileges", txn, sessiondata.InternalExecutorOverride{User: username.RootUserName()}, - "REVOKE CONNECT ON DATABASE postgres FROM public") - require.NoError(t, err) + // Revoke CONNECT on all non-system databases and ensure that when querying + // with node, we can still see all the databases. + _, err = ie.ExecEx(ctx, "revoke-privileges", txn, sessiondata.InternalExecutorOverride{User: username.RootUserName()}, + "REVOKE CONNECT ON DATABASE test1 FROM public") + require.NoError(t, err) + _, err = ie.ExecEx(ctx, "revoke-privileges", txn, sessiondata.InternalExecutorOverride{User: username.RootUserName()}, + "REVOKE CONNECT ON DATABASE test2 FROM public") + require.NoError(t, err) + _, err = ie.ExecEx(ctx, "revoke-privileges", txn, sessiondata.InternalExecutorOverride{User: username.RootUserName()}, + "REVOKE CONNECT ON DATABASE defaultdb FROM public") + require.NoError(t, err) + _, err = ie.ExecEx(ctx, "revoke-privileges", txn, sessiondata.InternalExecutorOverride{User: username.RootUserName()}, + "REVOKE CONNECT ON DATABASE postgres FROM public") + require.NoError(t, err) + + row, err = ie.QueryRowEx( + ctx, "get-all-databases", txn, sessiondata.NodeUserSessionDataOverride, + "SELECT count(1) FROM crdb_internal.databases", + ) + require.NoError(t, err) + // 3 databases (system, defaultdb, postgres, test1, test2). + require.Equal(t, row.String(), "(5)") + return nil + }); err != nil { + t.Fatal(err) + } - row, err = ie.QueryRowEx( - ctx, "get-all-databases", txn, sessiondata.NodeUserSessionDataOverride, - "SELECT count(1) FROM crdb_internal.databases", - ) - require.NoError(t, err) - // 3 databases (system, defaultdb, postgres, test1, test2). - require.Equal(t, row.String(), "(5)") } From cc0bbc8621f0153090dd572d1f16718fd467fcaa Mon Sep 17 00:00:00 2001 From: Jane Xing Date: Wed, 24 Aug 2022 10:21:00 -0500 Subject: [PATCH 11/18] sql: migrate create_external_connection to `planner.WithInternalExecutor()` for DDLs This commit is to migrate the existing DDLs to using `planner.WithInternalExecutor()`. DDLs with internal executors are only allowed if the latter is bounded with txn-realated metadata. Release justification: Low risk, high benefit changes to existing functionality Release note: none --- pkg/sql/create_external_connection.go | 36 +++++++++++++++------------ 1 file changed, 20 insertions(+), 16 deletions(-) diff --git a/pkg/sql/create_external_connection.go b/pkg/sql/create_external_connection.go index 56056a841399..16538b2ec257 100644 --- a/pkg/sql/create_external_connection.go +++ b/pkg/sql/create_external_connection.go @@ -16,12 +16,14 @@ import ( "github.com/cockroachdb/cockroach/pkg/cloud/externalconn" "github.com/cockroachdb/cockroach/pkg/clusterversion" + "github.com/cockroachdb/cockroach/pkg/kv" "github.com/cockroachdb/cockroach/pkg/security/username" "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/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sessiondata" + "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" "github.com/cockroachdb/cockroach/pkg/sql/syntheticprivilege" "github.com/cockroachdb/errors" ) @@ -112,22 +114,24 @@ func (p *planner) createExternalConnection( ex.SetConnectionType(exConn.ConnectionType()) ex.SetOwner(p.User()) - // Create the External Connection and persist it in the - // `system.external_connections` table. - if err := ex.Create(params.ctx, params.ExecCfg().InternalExecutor, params.p.User(), p.Txn()); err != nil { - return errors.Wrap(err, "failed to create external connection") - } - - // Grant user `ALL` on the newly created External Connection. - grantStatement := fmt.Sprintf(`GRANT ALL ON EXTERNAL CONNECTION "%s" TO %s`, - name, p.User().SQLIdentifier()) - _, err = params.ExecCfg().InternalExecutor.ExecEx(params.ctx, - "grant-on-create-external-connection", p.Txn(), - sessiondata.InternalExecutorOverride{User: username.NodeUserName()}, grantStatement) - if err != nil { - return errors.Wrap(err, "failed to grant on newly created External Connection") - } - return nil + return p.WithInternalExecutor(params.ctx, func(ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor) error { + // Create the External Connection and persist it in the + // `system.external_connections` table. + if err := ex.Create(params.ctx, ie, p.User(), txn); err != nil { + return errors.Wrap(err, "failed to create external connection") + } + + // Grant user `ALL` on the newly created External Connection. + grantStatement := fmt.Sprintf(`GRANT ALL ON EXTERNAL CONNECTION "%s" TO %s`, + name, p.User().SQLIdentifier()) + _, err = ie.ExecEx(params.ctx, + "grant-on-create-external-connection", txn, + sessiondata.InternalExecutorOverride{User: username.NodeUserName()}, grantStatement) + if err != nil { + return errors.Wrap(err, "failed to grant on newly created External Connection") + } + return nil + }) } func (c *createExternalConectionNode) Next(_ runParams) (bool, error) { return false, nil } From f2eb031052ddc857fa0442900e429058c57baf94 Mon Sep 17 00:00:00 2001 From: Jane Xing Date: Wed, 24 Aug 2022 10:24:30 -0500 Subject: [PATCH 12/18] backupccl: migrate job restoration to `CollectionFactory.TxnWithExecutor()` for DDLs This commit is part of the project to migrate DDLs running with internal executor with the correct interface. DDLs are only allowed to run with internal executor that is bound to txn-related metadata. Release justification: Low risk, high benefit changes to existing functionality Release note: none --- pkg/ccl/backupccl/restore_job.go | 5 ++--- pkg/ccl/backupccl/restore_planning.go | 6 ++++-- 2 files changed, 6 insertions(+), 5 deletions(-) diff --git a/pkg/ccl/backupccl/restore_job.go b/pkg/ccl/backupccl/restore_job.go index 1f2a8d54f6fe..c4c475860dbf 100644 --- a/pkg/ccl/backupccl/restore_job.go +++ b/pkg/ccl/backupccl/restore_job.go @@ -2214,8 +2214,8 @@ func (r *restoreResumer) OnFailOrCancel( logJobCompletion(ctx, restoreJobEventType, r.job.ID(), false, jobErr) execCfg := execCtx.(sql.JobExecContext).ExecCfg() - if err := sql.DescsTxn(ctx, execCfg, func( - ctx context.Context, txn *kv.Txn, descsCol *descs.Collection, + if err := execCfg.CollectionFactory.TxnWithExecutor(ctx, execCfg.DB, p.SessionData(), func( + ctx context.Context, txn *kv.Txn, descsCol *descs.Collection, ie sqlutil.InternalExecutor, ) error { for _, tenant := range details.Tenants { tenant.State = descpb.TenantInfo_DROP @@ -2233,7 +2233,6 @@ func (r *restoreResumer) OnFailOrCancel( if details.DescriptorCoverage == tree.AllDescriptors { // We've dropped defaultdb and postgres in the planning phase, we must // recreate them now if the full cluster restore failed. - ie := p.ExecCfg().InternalExecutor _, err := ie.Exec(ctx, "recreate-defaultdb", txn, "CREATE DATABASE IF NOT EXISTS defaultdb") if err != nil { return err diff --git a/pkg/ccl/backupccl/restore_planning.go b/pkg/ccl/backupccl/restore_planning.go index 3972e7af05e1..f8c1bb5e18bf 100644 --- a/pkg/ccl/backupccl/restore_planning.go +++ b/pkg/ccl/backupccl/restore_planning.go @@ -58,6 +58,7 @@ import ( "github.com/cockroachdb/cockroach/pkg/sql/sem/catconstants" "github.com/cockroachdb/cockroach/pkg/sql/sem/tree" "github.com/cockroachdb/cockroach/pkg/sql/sqlerrors" + "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" "github.com/cockroachdb/cockroach/pkg/sql/syntheticprivilege" "github.com/cockroachdb/cockroach/pkg/util/hlc" "github.com/cockroachdb/cockroach/pkg/util/log" @@ -705,8 +706,9 @@ func getDatabaseIDAndDesc( // as regular databases, we drop them before restoring them again in the // restore. func dropDefaultUserDBs(ctx context.Context, execCfg *sql.ExecutorConfig) error { - return sql.DescsTxn(ctx, execCfg, func(ctx context.Context, txn *kv.Txn, col *descs.Collection) error { - ie := execCfg.InternalExecutor + return execCfg.CollectionFactory.TxnWithExecutor(ctx, execCfg.DB, nil /* session data */, func( + ctx context.Context, txn *kv.Txn, _ *descs.Collection, ie sqlutil.InternalExecutor, + ) error { _, err := ie.Exec(ctx, "drop-defaultdb", txn, "DROP DATABASE IF EXISTS defaultdb") if err != nil { return err From e8a4d50bf715892c36582c1f8aa673f031b1d310 Mon Sep 17 00:00:00 2001 From: Jane Xing Date: Wed, 24 Aug 2022 10:30:05 -0500 Subject: [PATCH 13/18] backupccl: refactor `cleanupTempSystemTables()` We stripped `txn` from the parameter list in `cleanupTempSystemTables()`. It was run with not-nil txn by mistake, which is a mis-usage of running internal executor with DDLs. Release justification: bug fix Release note: none --- pkg/ccl/backupccl/restore_job.go | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/pkg/ccl/backupccl/restore_job.go b/pkg/ccl/backupccl/restore_job.go index c4c475860dbf..670079ea77cb 100644 --- a/pkg/ccl/backupccl/restore_job.go +++ b/pkg/ccl/backupccl/restore_job.go @@ -1669,7 +1669,7 @@ func (r *restoreResumer) doResume(ctx context.Context, execCtx interface{}) erro // Reload the details as we may have updated the job. details = r.job.Details().(jobspb.RestoreDetails) - if err := r.cleanupTempSystemTables(ctx, nil /* txn */); err != nil { + if err := r.cleanupTempSystemTables(ctx); err != nil { return err } } else if isSystemUserRestore(details) { @@ -1678,7 +1678,7 @@ func (r *restoreResumer) doResume(ctx context.Context, execCtx interface{}) erro } details = r.job.Details().(jobspb.RestoreDetails) - if err := r.cleanupTempSystemTables(ctx, nil /* txn */); err != nil { + if err := r.cleanupTempSystemTables(ctx); err != nil { return err } } @@ -2251,7 +2251,7 @@ func (r *restoreResumer) OnFailOrCancel( if details.DescriptorCoverage == tree.AllDescriptors { // The temporary system table descriptors should already have been dropped // in `dropDescriptors` but we still need to drop the temporary system db. - if err := execCfg.DB.Txn(ctx, r.cleanupTempSystemTables); err != nil { + if err := r.cleanupTempSystemTables(ctx); err != nil { return err } } @@ -2800,12 +2800,12 @@ func (r *restoreResumer) restoreSystemTables( return nil } -func (r *restoreResumer) cleanupTempSystemTables(ctx context.Context, txn *kv.Txn) error { +func (r *restoreResumer) cleanupTempSystemTables(ctx context.Context) error { executor := r.execCfg.InternalExecutor // Check if the temp system database has already been dropped. This can happen // if the restore job fails after the system database has cleaned up. checkIfDatabaseExists := "SELECT database_name FROM [SHOW DATABASES] WHERE database_name=$1" - if row, err := executor.QueryRow(ctx, "checking-for-temp-system-db" /* opName */, txn, checkIfDatabaseExists, restoreTempSystemDB); err != nil { + if row, err := executor.QueryRow(ctx, "checking-for-temp-system-db" /* opName */, nil /* txn */, checkIfDatabaseExists, restoreTempSystemDB); err != nil { return errors.Wrap(err, "checking for temporary system db") } else if row == nil { // Temporary system DB might already have been dropped by the restore job. @@ -2815,11 +2815,11 @@ func (r *restoreResumer) cleanupTempSystemTables(ctx context.Context, txn *kv.Tx // After restoring the system tables, drop the temporary database holding the // system tables. gcTTLQuery := fmt.Sprintf("ALTER DATABASE %s CONFIGURE ZONE USING gc.ttlseconds=1", restoreTempSystemDB) - if _, err := executor.Exec(ctx, "altering-gc-ttl-temp-system" /* opName */, txn, gcTTLQuery); err != nil { + if _, err := executor.Exec(ctx, "altering-gc-ttl-temp-system" /* opName */, nil /* txn */, gcTTLQuery); err != nil { log.Errorf(ctx, "failed to update the GC TTL of %q: %+v", restoreTempSystemDB, err) } dropTableQuery := fmt.Sprintf("DROP DATABASE %s CASCADE", restoreTempSystemDB) - if _, err := executor.Exec(ctx, "drop-temp-system-db" /* opName */, txn, dropTableQuery); err != nil { + if _, err := executor.Exec(ctx, "drop-temp-system-db" /* opName */, nil /* txn */, dropTableQuery); err != nil { return errors.Wrap(err, "dropping temporary system db") } return nil From b458abeebae7726d3ff808a17bd5e4f1846b4154 Mon Sep 17 00:00:00 2001 From: Jane Xing Date: Thu, 1 Sep 2022 15:57:20 -0500 Subject: [PATCH 14/18] sql: migrate `deleteTempTables` to use `planner.WithInternalExecutor()` This is another DDL statement executed via an internal executor mal-inited. Change it to use the right interface. Release justification: Release note: none. --- pkg/sql/discard.go | 28 +++++++++++++++------------- pkg/sql/temporary_schema.go | 2 -- pkg/sql/temporary_schema_test.go | 1 - 3 files changed, 15 insertions(+), 16 deletions(-) diff --git a/pkg/sql/discard.go b/pkg/sql/discard.go index 501642b62ed0..27085f728735 100644 --- a/pkg/sql/discard.go +++ b/pkg/sql/discard.go @@ -13,10 +13,12 @@ package sql import ( "context" + "github.com/cockroachdb/cockroach/pkg/kv" "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/sessiondata" + "github.com/cockroachdb/cockroach/pkg/sql/sqlutil" "github.com/cockroachdb/errors" ) @@ -84,20 +86,20 @@ func (n *discardNode) startExec(params runParams) error { } func deleteTempTables(ctx context.Context, p *planner) error { - codec := p.execCfg.Codec - descCol := p.Descriptors() - allDbDescs, err := descCol.GetAllDatabaseDescriptors(ctx, p.Txn()) - if err != nil { - return err - } - ie := p.execCfg.InternalExecutor - - for _, dbDesc := range allDbDescs { - schemaName := p.TemporarySchemaName() - err = cleanupSchemaObjects(ctx, p.execCfg.Settings, p.Txn(), descCol, codec, ie, dbDesc, schemaName) + return p.WithInternalExecutor(ctx, func(ctx context.Context, txn *kv.Txn, ie sqlutil.InternalExecutor) error { + codec := p.execCfg.Codec + descCol := p.Descriptors() + allDbDescs, err := descCol.GetAllDatabaseDescriptors(ctx, p.Txn()) if err != nil { return err } - } - return nil + for _, dbDesc := range allDbDescs { + schemaName := p.TemporarySchemaName() + err = cleanupSchemaObjects(ctx, p.Txn(), descCol, codec, ie, dbDesc, schemaName) + if err != nil { + return err + } + } + return nil + }) } diff --git a/pkg/sql/temporary_schema.go b/pkg/sql/temporary_schema.go index 6596b373c5bc..f0e3704e801c 100644 --- a/pkg/sql/temporary_schema.go +++ b/pkg/sql/temporary_schema.go @@ -193,7 +193,6 @@ func cleanupSessionTempObjects( for _, dbDesc := range allDbDescs { if err := cleanupSchemaObjects( ctx, - settings, txn, descsCol, codec, @@ -225,7 +224,6 @@ func cleanupSessionTempObjects( // API or avoid it entirely. func cleanupSchemaObjects( ctx context.Context, - settings *cluster.Settings, txn *kv.Txn, descsCol *descs.Collection, codec keys.SQLCodec, diff --git a/pkg/sql/temporary_schema_test.go b/pkg/sql/temporary_schema_test.go index 6dcc5ff9e1cd..fa45359fc465 100644 --- a/pkg/sql/temporary_schema_test.go +++ b/pkg/sql/temporary_schema_test.go @@ -111,7 +111,6 @@ INSERT INTO perm_table VALUES (DEFAULT, 1); } return cleanupSchemaObjects( ctx, - execCfg.Settings, txn, descsCol, execCfg.Codec, From f362a8f642764d6b4459c27b464079c36bb508c0 Mon Sep 17 00:00:00 2001 From: Jane Xing Date: Thu, 1 Sep 2022 17:57:08 -0500 Subject: [PATCH 15/18] sql: require txn-related metadata if running DDLs with internal executor with txn When using internal executor to run DDL statements under a not-nil outer txn, we require txn-related metadata (such as descriptor collections) to be passed to the internal executor from the outer caller too. This commit is to add a gate for this use case. Release justification: bug fix Release note: none --- pkg/sql/internal.go | 32 +++++++++++++++++++++++++++++++- 1 file changed, 31 insertions(+), 1 deletion(-) diff --git a/pkg/sql/internal.go b/pkg/sql/internal.go index 8ed85ce214b8..0c4e05ad3af6 100644 --- a/pkg/sql/internal.go +++ b/pkg/sql/internal.go @@ -616,7 +616,7 @@ func (ie *InternalExecutor) queryInternalBuffered( txn *kv.Txn, sessionDataOverride sessiondata.InternalExecutorOverride, stmt string, - // Non-zero limit specifies the limit on the number of rows returned. +// Non-zero limit specifies the limit on the number of rows returned. limit int, qargs ...interface{}, ) ([]tree.Datums, colinfo.ResultColumns, error) { @@ -831,6 +831,10 @@ func (ie *InternalExecutor) execInternal( stmt string, qargs ...interface{}, ) (r *rowsIterator, retErr error) { + if err := ie.checkIfTxnIsConsistent(txn); err != nil { + return nil, err + } + ctx = logtags.AddTag(ctx, "intExec", opName) var sd *sessiondata.SessionData @@ -896,6 +900,9 @@ func (ie *InternalExecutor) execInternal( timeReceived := timeutil.Now() parseStart := timeReceived parsed, err := parser.ParseOne(stmt) + if err := ie.checkIfStmtIsAllowed(parsed.AST, txn); err != nil { + return nil, err + } if err != nil { return nil, err } @@ -1075,6 +1082,29 @@ func (ie *InternalExecutor) commitTxn(ctx context.Context) error { return ex.commitSQLTransactionInternal(ctx) } +// checkIfStmtIsAllowed returns an error if the internal executor is not bound +// with the outer-txn-related info but is used to run DDL statements within an +// outer txn. +// TODO (janexing): this will be deprecate soon since it's not a good idea +// to have `extraTxnState` to store the info from a outer txn. +func (ie *InternalExecutor) checkIfStmtIsAllowed(stmt tree.Statement, txn *kv.Txn) error { + if tree.CanModifySchema(stmt) && txn != nil && ie.extraTxnState == nil { + return errors.New("DDL statement is disallowed if internal " + + "executor is not bound with txn metadata") + } + return nil +} + +// checkIfTxnIsConsistent returns true if the given txn is not nil and is not +// the same txn that is used to construct the internal executor. +func (ie *InternalExecutor) checkIfTxnIsConsistent(txn *kv.Txn) error { + if txn != nil && ie.extraTxnState != nil && ie.extraTxnState.txn != txn { + return errors.New("txn is inconsistent with the one when " + + "constructing the internal executor") + } + return nil +} + // internalClientComm is an implementation of ClientComm used by the // InternalExecutor. Result rows are buffered in memory. type internalClientComm struct { From ebd6076d396bb2eb190a49393c5b038a569cc9b9 Mon Sep 17 00:00:00 2001 From: Jane Xing Date: Mon, 19 Sep 2022 14:34:11 -0500 Subject: [PATCH 16/18] sql: add a space to a line to fix the lint It was causing the lint in CI to fail. Release note: None --- pkg/sql/internal.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pkg/sql/internal.go b/pkg/sql/internal.go index 0c4e05ad3af6..7b076cd04ca0 100644 --- a/pkg/sql/internal.go +++ b/pkg/sql/internal.go @@ -616,7 +616,7 @@ func (ie *InternalExecutor) queryInternalBuffered( txn *kv.Txn, sessionDataOverride sessiondata.InternalExecutorOverride, stmt string, -// Non-zero limit specifies the limit on the number of rows returned. + // Non-zero limit specifies the limit on the number of rows returned. limit int, qargs ...interface{}, ) ([]tree.Datums, colinfo.ResultColumns, error) { From 69fc8d3243c161051bf77a74e007abf50817456a Mon Sep 17 00:00:00 2001 From: Nathan VanBenschoten Date: Mon, 19 Sep 2022 18:59:57 -0400 Subject: [PATCH 17/18] kv: hoist log.ExpensiveLogEnabled outside loop in appendRefreshSpans This commit hoists the call to `log.ExpensiveLogEnabled` outside of the per-span loop in `txnSpanRefresher.appendRefreshSpans`. The function is too expensive to call once per span. Release justification: low risk change that may close perf gap. Release note: None. --- pkg/kv/kvclient/kvcoord/txn_interceptor_span_refresher.go | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/pkg/kv/kvclient/kvcoord/txn_interceptor_span_refresher.go b/pkg/kv/kvclient/kvcoord/txn_interceptor_span_refresher.go index 2bbd81c6c89a..d1a9e1fbe207 100644 --- a/pkg/kv/kvclient/kvcoord/txn_interceptor_span_refresher.go +++ b/pkg/kv/kvclient/kvcoord/txn_interceptor_span_refresher.go @@ -566,8 +566,9 @@ func (sr *txnSpanRefresher) tryRefreshTxnSpans( func (sr *txnSpanRefresher) appendRefreshSpans( ctx context.Context, ba roachpb.BatchRequest, br *roachpb.BatchResponse, ) error { + expLogEnabled := log.ExpensiveLogEnabled(ctx, 3) return ba.RefreshSpanIterate(br, func(span roachpb.Span) { - if log.ExpensiveLogEnabled(ctx, 3) { + if expLogEnabled { log.VEventf(ctx, 3, "recording span to refresh: %s", span.String()) } sr.refreshFootprint.insert(span) From fc976c1f4f69dc920ee2591696d94a4dd9809174 Mon Sep 17 00:00:00 2001 From: Marylia Gutierrez Date: Mon, 19 Sep 2022 16:16:53 -0400 Subject: [PATCH 18/18] api: increase timeout of sql api Previously, some request using the sql-over-http api were hitting a timeout, with the default value of 5s. This commit increases to 300s (5min) on the calls made from SQL Activity and Insights pages. Creates constant for long timeout and larger result size. Fixes #88094 Release note: None --- .../cluster-ui/src/api/clusterLocksApi.ts | 7 ++++- .../cluster-ui/src/api/indexActionsApi.ts | 7 ++++- .../cluster-ui/src/api/insightsApi.ts | 29 +++++++++++++------ .../cluster-ui/src/api/schemaInsightsApi.ts | 2 ++ .../workspaces/cluster-ui/src/api/sqlApi.ts | 2 ++ 5 files changed, 36 insertions(+), 11 deletions(-) diff --git a/pkg/ui/workspaces/cluster-ui/src/api/clusterLocksApi.ts b/pkg/ui/workspaces/cluster-ui/src/api/clusterLocksApi.ts index f179d85683a7..4ba504095873 100644 --- a/pkg/ui/workspaces/cluster-ui/src/api/clusterLocksApi.ts +++ b/pkg/ui/workspaces/cluster-ui/src/api/clusterLocksApi.ts @@ -9,7 +9,11 @@ // licenses/APL.txt. import moment from "moment"; -import { executeInternalSql, SqlExecutionRequest } from "./sqlApi"; +import { + executeInternalSql, + LONG_TIMEOUT, + SqlExecutionRequest, +} from "./sqlApi"; export type ClusterLockState = { databaseName?: string; @@ -65,6 +69,7 @@ WHERE }, ], execute: true, + timeout: LONG_TIMEOUT, }; return executeInternalSql(request).then(result => { diff --git a/pkg/ui/workspaces/cluster-ui/src/api/indexActionsApi.ts b/pkg/ui/workspaces/cluster-ui/src/api/indexActionsApi.ts index 585e7f5ae0ca..39e09bb1d776 100644 --- a/pkg/ui/workspaces/cluster-ui/src/api/indexActionsApi.ts +++ b/pkg/ui/workspaces/cluster-ui/src/api/indexActionsApi.ts @@ -8,7 +8,11 @@ // by the Apache License, Version 2.0, included in the file // licenses/APL.txt. -import { executeInternalSql, SqlExecutionRequest } from "./sqlApi"; +import { + executeInternalSql, + LONG_TIMEOUT, + SqlExecutionRequest, +} from "./sqlApi"; type IndexAction = { status: "SUCCESS" | "FAILED"; @@ -32,6 +36,7 @@ export function executeIndexRecAction( statements: statements, database: databaseName, execute: true, + timeout: LONG_TIMEOUT, }; return executeInternalSql(request) .then(result => { diff --git a/pkg/ui/workspaces/cluster-ui/src/api/insightsApi.ts b/pkg/ui/workspaces/cluster-ui/src/api/insightsApi.ts index 0922cc36856b..67cf23d6c969 100644 --- a/pkg/ui/workspaces/cluster-ui/src/api/insightsApi.ts +++ b/pkg/ui/workspaces/cluster-ui/src/api/insightsApi.ts @@ -13,6 +13,8 @@ import { SqlExecutionRequest, SqlExecutionResponse, INTERNAL_SQL_API_APP, + LONG_TIMEOUT, + LARGE_RESULT_SIZE, } from "./sqlApi"; import { InsightExecEnum, @@ -187,7 +189,8 @@ export function getTransactionInsightEventState(): Promise( txnContentionRequest, @@ -204,7 +207,8 @@ export function getTransactionInsightEventState(): Promise( txnFingerprintRequest, @@ -224,7 +228,8 @@ export function getTransactionInsightEventState(): Promise( fingerprintStmtsRequest, @@ -381,7 +386,8 @@ export function getTransactionInsightEventDetailsState( }, ], execute: true, - max_result_size: 50000, // 50 kib + max_result_size: LARGE_RESULT_SIZE, + timeout: LONG_TIMEOUT, }; return executeInternalSql( txnContentionDetailsRequest, @@ -398,7 +404,8 @@ export function getTransactionInsightEventDetailsState( }, ], execute: true, - max_result_size: 50000, // 50 kib + max_result_size: LARGE_RESULT_SIZE, + timeout: LONG_TIMEOUT, }; return executeInternalSql( waitingTxnFingerprintRequest, @@ -412,7 +419,8 @@ export function getTransactionInsightEventDetailsState( }, ], execute: true, - max_result_size: 50000, // 50 kib + max_result_size: LARGE_RESULT_SIZE, + timeout: LONG_TIMEOUT, }; return executeInternalSql( waitingFingerprintStmtsRequest, @@ -427,7 +435,8 @@ export function getTransactionInsightEventDetailsState( }, ], execute: true, - max_result_size: 50000, // 50 kib + max_result_size: LARGE_RESULT_SIZE, + timeout: LONG_TIMEOUT, }; return executeInternalSql( blockingTxnFingerprintRequest, @@ -442,7 +451,8 @@ export function getTransactionInsightEventDetailsState( }, ], execute: true, - max_result_size: 50000, // 50 kib + max_result_size: LARGE_RESULT_SIZE, + timeout: LONG_TIMEOUT, }; return executeInternalSql( blockingFingerprintStmtsRequest, @@ -624,7 +634,8 @@ export function getStatementInsightsApi(): Promise { }, ], execute: true, - max_result_size: 50000, // 50 kib + max_result_size: LARGE_RESULT_SIZE, + timeout: LONG_TIMEOUT, }; return executeInternalSql(request).then( result => { diff --git a/pkg/ui/workspaces/cluster-ui/src/api/schemaInsightsApi.ts b/pkg/ui/workspaces/cluster-ui/src/api/schemaInsightsApi.ts index c3c03a435eb5..b47f7f454ecb 100644 --- a/pkg/ui/workspaces/cluster-ui/src/api/schemaInsightsApi.ts +++ b/pkg/ui/workspaces/cluster-ui/src/api/schemaInsightsApi.ts @@ -12,6 +12,7 @@ import { SqlExecutionRequest, SqlTxnResult, executeInternalSql, + LONG_TIMEOUT, } from "./sqlApi"; import { InsightRecommendation, @@ -181,6 +182,7 @@ export function getSchemaInsights(): Promise { sql: insightQuery.query, })), execute: true, + timeout: LONG_TIMEOUT, }; return executeInternalSql(request).then(result => { const results: InsightRecommendation[] = []; diff --git a/pkg/ui/workspaces/cluster-ui/src/api/sqlApi.ts b/pkg/ui/workspaces/cluster-ui/src/api/sqlApi.ts index c733c810d2e6..88774147ed3c 100644 --- a/pkg/ui/workspaces/cluster-ui/src/api/sqlApi.ts +++ b/pkg/ui/workspaces/cluster-ui/src/api/sqlApi.ts @@ -78,6 +78,8 @@ export function executeSql( } export const INTERNAL_SQL_API_APP = "$ internal-console"; +export const LONG_TIMEOUT = "300s"; +export const LARGE_RESULT_SIZE = 50000; // 50 kib /** * executeInternalSql executes the provided SQL statements with