Skip to content

Commit

Permalink
opt: don't apply disallow_full_table_scans to internal executors
Browse files Browse the repository at this point in the history
We have various checks for disallow_full_table_scans:
1. in connExecutor we fail the statement with an error if it uses a full
   scan
2. in execbuild we fail the statement with an error if there is an index
   hint that causes a full scan
3. in the coster we add hugeCost to full scans, to try and avoid them

We don't want disallow_full_table_scans to apply to anything done by
internal executors. (1) was already checking whether this connExecutor
was internal. (2) and (3) were not, which led to #137404. #137681 fixed
this by setting disallow_full_table_scans in `NewInternalSessionData`,
but we only use `NewInternalSessionData` for some uses of the internal
executor.

This commit explicitly checks `SessionData().Internal` in (2) and (3) to
match (1), so that we don't get any of the disallow_full_table_scans
behavior for any use of the internal executor, including populating
virtual tables.

This commit also adds `SessionData().Internal` to the memo staleness
check.

This behavior can be observed with the following diff, which adds an
index hint to the `crdb_internal.system_jobs` virtual table SQL:

```
diff --git a/pkg/sql/crdb_internal.go b/pkg/sql/crdb_internal.go
index 80a5342b56a..99ccb8f1611 100644
--- a/pkg/sql/crdb_internal.go
+++ b/pkg/sql/crdb_internal.go
@@ -975,7 +975,7 @@ SELECT
 DISTINCT(id), status, created, payload.value AS payload, progress.value AS progress,
 created_by_type, created_by_id, claim_session_id, claim_instance_id, num_runs, last_run, job_type
 FROM
-system.jobs AS j
+system.jobs@jobs_status_created_idx AS j
 LEFT JOIN system.job_info AS progress ON j.id = progress.job_id AND progress.info_key = 'legacy_progress'
 INNER JOIN system.job_info AS payload ON j.id = payload.job_id AND payload.info_key = 'legacy_payload'
 `
```

With this diff, the following SQL hits (2) and fails with an error, even
though it uses the internal executor and thus should not error:

```
SET disallow_full_table_scans = on;
SELECT * FROM crdb_internal.jobs;
```

With this commit we no longer fail, matching the behavior without the
index hint.

Informs: #137404
Informs: #123783

Release note: None
  • Loading branch information
michae2 committed Dec 28, 2024
1 parent 686a8fa commit 912b025
Show file tree
Hide file tree
Showing 6 changed files with 79 additions and 1 deletion.
1 change: 1 addition & 0 deletions pkg/sql/opt/exec/execbuilder/relational.go
Original file line number Diff line number Diff line change
Expand Up @@ -604,6 +604,7 @@ func (b *Builder) scanParams(
err = pgerror.Newf(pgcode.WrongObjectType,
"index \"%s\" cannot be used for this query", idx.Name())
if b.evalCtx.SessionData().DisallowFullTableScans &&
!b.evalCtx.SessionData().Internal &&
(b.flags.IsSet(exec.PlanFlagContainsLargeFullTableScan) ||
b.flags.IsSet(exec.PlanFlagContainsLargeFullIndexScan)) {
// TODO(#123783): this code might need an adjustment for virtual
Expand Down
3 changes: 3 additions & 0 deletions pkg/sql/opt/memo/memo.go
Original file line number Diff line number Diff line change
Expand Up @@ -199,6 +199,7 @@ type Memo struct {
pushLimitIntoProjectFilteredScan bool
unsafeAllowTriggersModifyingCascades bool
legacyVarcharTyping bool
internal bool

// txnIsoLevel is the isolation level under which the plan was created. This
// affects the planning of some locking operations, so it must be included in
Expand Down Expand Up @@ -289,6 +290,7 @@ func (m *Memo) Init(ctx context.Context, evalCtx *eval.Context) {
pushLimitIntoProjectFilteredScan: evalCtx.SessionData().OptimizerPushLimitIntoProjectFilteredScan,
unsafeAllowTriggersModifyingCascades: evalCtx.SessionData().UnsafeAllowTriggersModifyingCascades,
legacyVarcharTyping: evalCtx.SessionData().LegacyVarcharTyping,
internal: evalCtx.SessionData().Internal,
txnIsoLevel: evalCtx.TxnIsoLevel,
}
m.metadata.Init()
Expand Down Expand Up @@ -457,6 +459,7 @@ func (m *Memo) IsStale(
m.pushLimitIntoProjectFilteredScan != evalCtx.SessionData().OptimizerPushLimitIntoProjectFilteredScan ||
m.unsafeAllowTriggersModifyingCascades != evalCtx.SessionData().UnsafeAllowTriggersModifyingCascades ||
m.legacyVarcharTyping != evalCtx.SessionData().LegacyVarcharTyping ||
m.internal != evalCtx.SessionData().Internal ||
m.txnIsoLevel != evalCtx.TxnIsoLevel {
return true, nil
}
Expand Down
5 changes: 5 additions & 0 deletions pkg/sql/opt/memo/memo_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -532,6 +532,11 @@ func TestMemoIsStale(t *testing.T) {
evalCtx.SessionData().LegacyVarcharTyping = false
notStale()

evalCtx.SessionData().Internal = true
stale()
evalCtx.SessionData().Internal = false
notStale()

// User no longer has access to view.
catalog.View(tree.NewTableNameWithSchema("t", catconstants.PublicSchemaName, "abcview")).Revoked = true
_, err = o.Memo().IsStale(ctx, &evalCtx, catalog)
Expand Down
3 changes: 2 additions & 1 deletion pkg/sql/opt/xform/coster.go
Original file line number Diff line number Diff line change
Expand Up @@ -774,7 +774,8 @@ func (c *coster) computeScanCost(scan *memo.ScanExpr, required *physical.Require

stats := scan.Relational().Statistics()
rowCount := stats.RowCount
if isUnfiltered && c.evalCtx != nil && c.evalCtx.SessionData().DisallowFullTableScans {
if isUnfiltered && c.evalCtx != nil && c.evalCtx.SessionData().DisallowFullTableScans &&
!c.evalCtx.SessionData().Internal {
if !scan.IsVirtualTable(c.mem.Metadata()) {
// Don't apply the huge cost to full scans of virtual tables since
// we don't reject them anyway. In other words, we would only
Expand Down
3 changes: 3 additions & 0 deletions pkg/sql/sem/catconstants/constants.go
Original file line number Diff line number Diff line change
Expand Up @@ -223,6 +223,9 @@ const (
CrdbInternalFullyQualifiedNamesViewID
CrdbInternalStoreLivenessSupportFrom
CrdbInternalStoreLivenessSupportFor
// CrdbInternalTestID is reserved for tests that need to inject virtual tables
// into crdb_internal.
CrdbInternalTestID
InformationSchemaID
InformationSchemaAdministrableRoleAuthorizationsID
InformationSchemaApplicableRolesID
Expand Down
65 changes: 65 additions & 0 deletions pkg/sql/virtual_schema_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -26,10 +26,18 @@ import (
"strings"
"testing"

"github.com/cockroachdb/cockroach/pkg/base"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/sql/catalog"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/schemadesc"
"github.com/cockroachdb/cockroach/pkg/sql/sem/catconstants"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/sessiondata"
"github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
"github.com/cockroachdb/cockroach/pkg/testutils/sqlutils"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/errors"
"github.com/stretchr/testify/require"
)

Expand Down Expand Up @@ -119,3 +127,60 @@ func rewriteSchema(schemaName string, tableNames []string) {
}
})
}

// TestVirtualTablesIgnoreDisallowFullTableScans checks that we ignore
// disallow_full_table_scans for virtual tables, even when the virtual table is
// populated using an internal executor query that itself does a full scan.
func TestVirtualTablesIgnoreDisallowFullTableScans(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)

// Inject a virtual table into crdb_internal which is populated using an
// internal executor query that does a full scan.
crdbInternalFooTable := virtualSchemaTable{
schema: `
CREATE TABLE crdb_internal.foo (
status STRING NOT NULL,
created TIMESTAMP NOT NULL
)
`,
comment: "foo",
indexes: []virtualIndex{},
populate: func(ctx context.Context, p *planner, db catalog.DatabaseDescriptor, addRow func(...tree.Datum) error) (retErr error) {
it, err := p.InternalSQLTxn().QueryIteratorEx(
ctx, "foo", p.Txn(), sessiondata.NodeUserSessionDataOverride,
// Use a full-table scan query with an index hint to hit the
// disallow_full_table_scans check in execbuilder.
"SELECT status, created FROM system.jobs@jobs_status_created_idx",
)
if err != nil {
return err
}
defer func() {
if err := it.Close(); err != nil {
retErr = errors.CombineErrors(retErr, err)
}
}()
for {
hasNext, err := it.Next(ctx)
if !hasNext || err != nil {
return err
}
currentRow := it.Cur()
if err := addRow(currentRow...); err != nil {
return err
}
}
},
}
crdbInternal.tableDefs[catconstants.CrdbInternalTestID] = crdbInternalFooTable

ctx := context.Background()

srv, conn, _ := serverutils.StartServer(t, base.TestServerArgs{})
defer srv.Stopper().Stop(ctx)

sqlDB := sqlutils.MakeSQLRunner(conn)
sqlDB.Exec(t, "SET disallow_full_table_scans = on")
sqlDB.CheckQueryResults(t, "SELECT * FROM crdb_internal.foo WHERE status = 'banana'", [][]string{})
}

0 comments on commit 912b025

Please sign in to comment.