diff --git a/pkg/sql/opt/exec/execbuilder/relational.go b/pkg/sql/opt/exec/execbuilder/relational.go
index 04dd2da467cf..f24ed0814217 100644
--- a/pkg/sql/opt/exec/execbuilder/relational.go
+++ b/pkg/sql/opt/exec/execbuilder/relational.go
@@ -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
diff --git a/pkg/sql/opt/memo/memo.go b/pkg/sql/opt/memo/memo.go
index 6ac9d767a24e..7c60f193a8d0 100644
--- a/pkg/sql/opt/memo/memo.go
+++ b/pkg/sql/opt/memo/memo.go
@@ -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
@@ -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()
@@ -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
 	}
diff --git a/pkg/sql/opt/memo/memo_test.go b/pkg/sql/opt/memo/memo_test.go
index c83fdca3ad1e..c5383992b48d 100644
--- a/pkg/sql/opt/memo/memo_test.go
+++ b/pkg/sql/opt/memo/memo_test.go
@@ -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)
diff --git a/pkg/sql/opt/xform/coster.go b/pkg/sql/opt/xform/coster.go
index ff7609c89ae2..e09daa76717e 100644
--- a/pkg/sql/opt/xform/coster.go
+++ b/pkg/sql/opt/xform/coster.go
@@ -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
diff --git a/pkg/sql/sem/catconstants/constants.go b/pkg/sql/sem/catconstants/constants.go
index 20b4c0b829d7..264e496be899 100644
--- a/pkg/sql/sem/catconstants/constants.go
+++ b/pkg/sql/sem/catconstants/constants.go
@@ -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
diff --git a/pkg/sql/virtual_schema_test.go b/pkg/sql/virtual_schema_test.go
index 0bdba6713930..04e83d4fc67a 100644
--- a/pkg/sql/virtual_schema_test.go
+++ b/pkg/sql/virtual_schema_test.go
@@ -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"
 )
 
@@ -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{})
+}